From f953ffffc5d4717eb36057c3b6d62c4b4918b01f Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 5 Mar 2025 00:34:10 +0800 Subject: [PATCH 01/76] refactor(query): refactor aggreagte spill code --- .../base/src/runtime/memory/mem_stat.rs | 40 +- .../src/runtime/memory/stat_buffer_global.rs | 4 +- .../runtime/memory/stat_buffer_mem_stat.rs | 4 +- .../expression/src/aggregate/payload_flush.rs | 16 - .../pipeline/core/src/processors/processor.rs | 20 + .../transforms/transform_accumulating.rs | 67 ++- .../src/interpreters/common/query_log.rs | 6 +- .../pipelines/builders/builder_aggregate.rs | 39 +- .../aggregator/transform_aggregate_partial.rs | 411 ++++++++++++++++-- .../v1/exchange/exchange_transform_shuffle.rs | 6 + src/query/service/src/spillers/spiller.rs | 47 ++ .../storages/system/src/query_log_table.rs | 4 +- 12 files changed, 563 insertions(+), 101 deletions(-) diff --git a/src/common/base/src/runtime/memory/mem_stat.rs b/src/common/base/src/runtime/memory/mem_stat.rs index 295ff6e4927ff..2ace8dcd0ca66 100644 --- a/src/common/base/src/runtime/memory/mem_stat.rs +++ b/src/common/base/src/runtime/memory/mem_stat.rs @@ -40,14 +40,14 @@ pub struct MemStat { name: Option, pub(crate) used: AtomicI64, - pub(crate) peek_used: AtomicI64, + pub(crate) peak_used: AtomicI64, /// The limit of max used memory for this tracker. /// /// Set to 0 to disable the limit. limit: AtomicI64, - parent_memory_stat: Vec>, + parent_memory_stat: Option>, } impl MemStat { @@ -56,17 +56,17 @@ impl MemStat { id: 0, name: None, used: AtomicI64::new(0), - peek_used: AtomicI64::new(0), + peak_used: AtomicI64::new(0), limit: AtomicI64::new(0), - parent_memory_stat: vec![], + parent_memory_stat: None, } } pub fn create(name: String) -> Arc { - MemStat::create_child(name, vec![]) + MemStat::create_child(name, None) } - pub fn create_child(name: String, parent_memory_stat: Vec>) -> Arc { + pub fn create_child(name: String, parent_memory_stat: Option>) -> Arc { let id = match GlobalSequence::next() { 0 => GlobalSequence::next(), id => id, @@ -76,16 +76,12 @@ impl MemStat { id, name: Some(name), used: AtomicI64::new(0), - peek_used: AtomicI64::new(0), + peak_used: AtomicI64::new(0), limit: AtomicI64::new(0), parent_memory_stat, }) } - pub fn get_parent_memory_stat(&self) -> Vec> { - self.parent_memory_stat.clone() - } - pub fn set_limit(&self, mut size: i64) { // It may cause the process unable to run if memory limit is too low. if size > 0 && size < MINIMUM_MEMORY_LIMIT { @@ -107,9 +103,9 @@ impl MemStat { let mut used = self.used.fetch_add(batch_memory_used, Ordering::Relaxed); used += batch_memory_used; - self.peek_used.fetch_max(used, Ordering::Relaxed); + self.peak_used.fetch_max(used, Ordering::Relaxed); - for (idx, parent_memory_stat) in self.parent_memory_stat.iter().enumerate() { + if let Some(parent_memory_stat) = self.parent_memory_stat.as_ref() { if let Err(cause) = parent_memory_stat .record_memory::(batch_memory_used, current_memory_alloc) { @@ -117,9 +113,7 @@ impl MemStat { // We only roll back the memory that alloc failed self.used.fetch_sub(current_memory_alloc, Ordering::Relaxed); - for index in 0..idx { - self.parent_memory_stat[index].rollback(current_memory_alloc); - } + parent_memory_stat.rollback(current_memory_alloc); } return Err(cause); @@ -142,8 +136,8 @@ impl MemStat { pub fn rollback(&self, memory_usage: i64) { self.used.fetch_sub(memory_usage, Ordering::Relaxed); - for parent_memory_stat in &self.parent_memory_stat { - parent_memory_stat.rollback(memory_usage) + if let Some(parent_memory_stat) = &self.parent_memory_stat { + parent_memory_stat.rollback(memory_usage); } } @@ -171,7 +165,7 @@ impl MemStat { #[inline] pub fn get_peek_memory_usage(&self) -> i64 { - self.peek_used.load(Ordering::Relaxed) + self.peak_used.load(Ordering::Relaxed) } } @@ -268,7 +262,7 @@ mod tests { fn test_multiple_level_mem_stat() -> Result<()> { let mem_stat = MemStat::create("TEST".to_string()); let child_mem_stat = - MemStat::create_child("TEST_CHILD".to_string(), vec![mem_stat.clone()]); + MemStat::create_child("TEST_CHILD".to_string(), Some(mem_stat.clone())); mem_stat.record_memory::(1, 1).unwrap(); mem_stat.record_memory::(2, 2).unwrap(); @@ -292,7 +286,7 @@ mod tests { let mem_stat = MemStat::create("TEST".to_string()); mem_stat.set_limit(MINIMUM_MEMORY_LIMIT * 2); let child_mem_stat = - MemStat::create_child("TEST_CHILD".to_string(), vec![mem_stat.clone()]); + MemStat::create_child("TEST_CHILD".to_string(), Some(mem_stat.clone())); child_mem_stat.set_limit(MINIMUM_MEMORY_LIMIT); mem_stat.record_memory::(1, 1).unwrap(); @@ -322,7 +316,7 @@ mod tests { let mem_stat = MemStat::create("TEST".to_string()); mem_stat.set_limit(MINIMUM_MEMORY_LIMIT); let child_mem_stat = - MemStat::create_child("TEST_CHILD".to_string(), vec![mem_stat.clone()]); + MemStat::create_child("TEST_CHILD".to_string(), Some(mem_stat.clone())); child_mem_stat.set_limit(MINIMUM_MEMORY_LIMIT * 2); assert!(child_mem_stat @@ -335,7 +329,7 @@ mod tests { let mem_stat = MemStat::create("TEST".to_string()); mem_stat.set_limit(MINIMUM_MEMORY_LIMIT * 2); let child_mem_stat = - MemStat::create_child("TEST_CHILD".to_string(), vec![mem_stat.clone()]); + MemStat::create_child("TEST_CHILD".to_string(), Some(mem_stat.clone())); child_mem_stat.set_limit(MINIMUM_MEMORY_LIMIT); assert!(child_mem_stat diff --git a/src/common/base/src/runtime/memory/stat_buffer_global.rs b/src/common/base/src/runtime/memory/stat_buffer_global.rs index 4eb20f411f296..cce85443b3054 100644 --- a/src/common/base/src/runtime/memory/stat_buffer_global.rs +++ b/src/common/base/src/runtime/memory/stat_buffer_global.rs @@ -90,7 +90,7 @@ impl GlobalStatBuffer { .used .fetch_add(memory_usage, Ordering::Relaxed); self.global_mem_stat - .peek_used + .peak_used .fetch_max(used + memory_usage, Ordering::Relaxed); return Ok(()); } @@ -126,7 +126,7 @@ impl GlobalStatBuffer { .used .fetch_add(memory_usage, Ordering::Relaxed); self.global_mem_stat - .peek_used + .peak_used .fetch_max(used + memory_usage, Ordering::Relaxed); return; } diff --git a/src/common/base/src/runtime/memory/stat_buffer_mem_stat.rs b/src/common/base/src/runtime/memory/stat_buffer_mem_stat.rs index 8c890598dd61b..71f035fb9d726 100644 --- a/src/common/base/src/runtime/memory/stat_buffer_mem_stat.rs +++ b/src/common/base/src/runtime/memory/stat_buffer_mem_stat.rs @@ -93,7 +93,7 @@ impl MemStatBuffer { if self.destroyed_thread_local_macro { let used = mem_stat.used.fetch_add(usage, Ordering::Relaxed); mem_stat - .peek_used + .peak_used .fetch_max(used + usage, Ordering::Relaxed); return Ok(()); } @@ -134,7 +134,7 @@ impl MemStatBuffer { if self.destroyed_thread_local_macro { let used = mem_stat.used.fetch_add(memory_usage, Ordering::Relaxed); mem_stat - .peek_used + .peak_used .fetch_max(used + memory_usage, Ordering::Relaxed); return; } diff --git a/src/query/expression/src/aggregate/payload_flush.rs b/src/query/expression/src/aggregate/payload_flush.rs index 4fe9f35830227..2b3161c252950 100644 --- a/src/query/expression/src/aggregate/payload_flush.rs +++ b/src/query/expression/src/aggregate/payload_flush.rs @@ -164,22 +164,6 @@ impl Payload { Ok(Some(DataBlock::new_from_columns(cols))) } - pub fn group_by_flush_all(&self) -> Result { - let mut state = PayloadFlushState::default(); - let mut blocks = vec![]; - - while self.flush(&mut state) { - let cols = state.take_group_columns(); - blocks.push(DataBlock::new_from_columns(cols)); - } - - if blocks.is_empty() { - return Ok(self.empty_block(None)); - } - - DataBlock::concat(&blocks) - } - pub fn flush(&self, state: &mut PayloadFlushState) -> bool { if state.flush_page >= self.pages.len() { return false; diff --git a/src/query/pipeline/core/src/processors/processor.rs b/src/query/pipeline/core/src/processors/processor.rs index ce70053b80ded..2d2fd18b7640e 100644 --- a/src/query/pipeline/core/src/processors/processor.rs +++ b/src/query/pipeline/core/src/processors/processor.rs @@ -80,6 +80,22 @@ pub trait Processor: Send { Err(ErrorCode::Unimplemented("Unimplemented async_process.")) } + fn prepare_spill_payload(&mut self) -> Result { + Err(ErrorCode::Unimplemented( + "Unimplemented prepare_spill_payload", + )) + } + + async fn flush_spill_payload(&mut self) -> Result { + Err(ErrorCode::Unimplemented( + "Unimplemented flush_spill_payload", + )) + } + + fn configure_peer_nodes(&mut self, nodes: &[String]) { + // do nothing by default + } + fn details_status(&self) -> Option { None } @@ -198,6 +214,10 @@ impl ProcessorPtr { .boxed() } + pub fn configure_peer_nodes(&self, nodes: &[String]) { + unsafe { (*self.inner.get()).configure_peer_nodes(nodes) } + } + /// # Safety pub unsafe fn details_status(&self) -> Option { (*self.inner.get()).details_status() diff --git a/src/query/pipeline/transforms/src/processors/transforms/transform_accumulating.rs b/src/query/pipeline/transforms/src/processors/transforms/transform_accumulating.rs index 1ffc9dcbcf2ea..976b5a5b905c2 100644 --- a/src/query/pipeline/transforms/src/processors/transforms/transform_accumulating.rs +++ b/src/query/pipeline/transforms/src/processors/transforms/transform_accumulating.rs @@ -18,6 +18,7 @@ use std::marker::PhantomData; use std::sync::Arc; use databend_common_base::runtime::drop_guard; +use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::BlockMetaInfo; use databend_common_expression::BlockMetaInfoDowncast; @@ -27,9 +28,12 @@ use databend_common_pipeline_core::processors::InputPort; use databend_common_pipeline_core::processors::OutputPort; use databend_common_pipeline_core::processors::Processor; +#[async_trait::async_trait] pub trait AccumulatingTransform: Send { const NAME: &'static str; + const SUPPORT_SPILL: bool = false; + fn transform(&mut self, data: DataBlock) -> Result>; fn on_finish(&mut self, _output: bool) -> Result> { @@ -37,6 +41,24 @@ pub trait AccumulatingTransform: Send { } fn interrupt(&self) {} + + fn configure_peer_nodes(&mut self, _nodes: &[String]) {} + + fn need_spill(&self) -> bool { + false + } + + fn prepare_spill_payload(&mut self) -> Result { + Err(ErrorCode::Unimplemented( + "Unimplemented prepare_spill_payload", + )) + } + + async fn flush_spill_payload(&mut self) -> Result { + Err(ErrorCode::Unimplemented( + "Unimplemented flush_spill_payload", + )) + } } pub struct AccumulatingTransformer { @@ -47,6 +69,9 @@ pub struct AccumulatingTransformer { called_on_finish: bool, input_data: Option, output_data: VecDeque, + + flush_spill_payload: bool, + prepare_spill_payload: bool, } impl AccumulatingTransformer { @@ -58,6 +83,8 @@ impl AccumulatingTransformer { input_data: None, output_data: VecDeque::with_capacity(1), called_on_finish: false, + flush_spill_payload: false, + prepare_spill_payload: false, }) } } @@ -93,6 +120,14 @@ impl Processor for AccumulatingTransformer Processor for AccumulatingTransformer Result<()> { + if self.prepare_spill_payload { + self.prepare_spill_payload = false; + self.flush_spill_payload = self.prepare_spill_payload()?; + return Ok(()); + } + if let Some(data_block) = self.input_data.take() { self.output_data.extend(self.inner.transform(data_block)?); + self.prepare_spill_payload = self.inner.need_spill(); return Ok(()); } @@ -140,8 +190,21 @@ impl Processor for AccumulatingTransformer Result<()> { + if self.flush_spill_payload { + self.flush_spill_payload = false; + self.prepare_spill_payload = self.flush_spill_payload().await?; + } + + Ok(()) + } + + fn prepare_spill_payload(&mut self) -> Result { + self.inner.prepare_spill_payload() + } + + async fn flush_spill_payload(&mut self) -> Result { + self.inner.flush_spill_payload().await } } diff --git a/src/query/service/src/interpreters/common/query_log.rs b/src/query/service/src/interpreters/common/query_log.rs index 2896858421360..2e9dd024b642b 100644 --- a/src/query/service/src/interpreters/common/query_log.rs +++ b/src/query/service/src/interpreters/common/query_log.rs @@ -225,7 +225,7 @@ impl InterpreterQueryLog { has_profiles: false, txn_state, txn_id, - peek_memory_usage: HashMap::new(), + peak_memory_usage: HashMap::new(), }) } @@ -337,7 +337,7 @@ impl InterpreterQueryLog { let txn_id = guard.txn_id().to_string(); drop(guard); - let peek_memory_usage = ctx.get_node_peek_memory_usage(); + let peak_memory_usage = ctx.get_node_peek_memory_usage(); Self::write_log(QueryLogElement { log_type, @@ -402,7 +402,7 @@ impl InterpreterQueryLog { has_profiles, txn_state, txn_id, - peek_memory_usage, + peak_memory_usage, }) } } diff --git a/src/query/service/src/pipelines/builders/builder_aggregate.rs b/src/query/service/src/pipelines/builders/builder_aggregate.rs index fde91de7ca754..cf7353b11a6a6 100644 --- a/src/query/service/src/pipelines/builders/builder_aggregate.rs +++ b/src/query/service/src/pipelines/builders/builder_aggregate.rs @@ -43,7 +43,6 @@ use crate::pipelines::processors::transforms::aggregator::AggregateInjector; use crate::pipelines::processors::transforms::aggregator::AggregatorParams; use crate::pipelines::processors::transforms::aggregator::FinalSingleStateAggregator; 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::TransformPartialAggregate; use crate::pipelines::PipelineBuilder; @@ -153,34 +152,38 @@ impl PipelineBuilder { }); } + let location_prefix = self.ctx.query_id_spill_prefix(); + let operator = DataOperator::instance().spill_operator(); self.main_pipeline.add_transform(|input, output| { Ok(ProcessorPtr::create(TransformPartialAggregate::try_create( self.ctx.clone(), input, output, + operator.clone(), params.clone(), partial_agg_config.clone(), + location_prefix.clone(), )?)) })?; // If cluster mode, spill write will be completed in exchange serialize, because we need scatter the block data first - if !self.is_exchange_neighbor { - let operator = DataOperator::instance().spill_operator(); - let location_prefix = self.ctx.query_id_spill_prefix(); - - self.main_pipeline.add_transform(|input, output| { - Ok(ProcessorPtr::create( - TransformAggregateSpillWriter::try_create( - self.ctx.clone(), - input, - output, - operator.clone(), - params.clone(), - location_prefix.clone(), - )?, - )) - })?; - } + // if !self.is_exchange_neighbor { + // let operator = DataOperator::instance().spill_operator(); + // let location_prefix = self.ctx.query_id_spill_prefix(); + // + // self.main_pipeline.add_transform(|input, output| { + // Ok(ProcessorPtr::create( + // TransformAggregateSpillWriter::try_create( + // self.ctx.clone(), + // input, + // output, + // operator.clone(), + // params.clone(), + // location_prefix.clone(), + // )?, + // )) + // })?; + // } self.exchange_injector = AggregateInjector::create(self.ctx.clone(), params.clone()); Ok(()) 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 404d963f8ba6e..69f7f7f29a00d 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 @@ -17,15 +17,22 @@ use std::time::Instant; use std::vec; use bumpalo::Bump; +use byteorder::BigEndian; +use byteorder::WriteBytesExt; use databend_common_base::base::convert_byte_size; use databend_common_base::base::convert_number_size; use databend_common_catalog::plan::AggIndexMeta; +use databend_common_catalog::table_context::TableContext; +use databend_common_exception::ErrorCode; use databend_common_exception::Result; +use databend_common_expression::arrow::write_column; use databend_common_expression::AggregateHashTable; use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::DataBlock; use databend_common_expression::HashTableConfig; use databend_common_expression::InputColumns; +use databend_common_expression::PartitionedPayload; +use databend_common_expression::Payload; use databend_common_expression::PayloadFlushState; use databend_common_expression::ProbeState; use databend_common_pipeline_core::processors::InputPort; @@ -34,11 +41,18 @@ use databend_common_pipeline_core::processors::Processor; use databend_common_pipeline_transforms::processors::AccumulatingTransform; use databend_common_pipeline_transforms::processors::AccumulatingTransformer; use databend_common_pipeline_transforms::MemorySettings; +use opendal::Operator; use crate::pipelines::memory_settings::MemorySettingsExt; use crate::pipelines::processors::transforms::aggregator::aggregate_meta::AggregateMeta; use crate::pipelines::processors::transforms::aggregator::AggregatorParams; +use crate::pipelines::processors::transforms::aggregator::BucketSpilledPayload; use crate::sessions::QueryContext; +use crate::spillers::SpillWriter; +use crate::spillers::Spiller; +use crate::spillers::SpillerConfig; +use crate::spillers::SpillerType; + #[allow(clippy::enum_variant_names)] enum HashTable { MovedOut, @@ -61,6 +75,10 @@ pub struct TransformPartialAggregate { processed_bytes: usize, processed_rows: usize, settings: MemorySettings, + configure_peer_nodes: Vec, + spilling_state: Option, + spiller: Arc, + spill_blocks: Vec, } impl TransformPartialAggregate { @@ -68,8 +86,10 @@ impl TransformPartialAggregate { ctx: Arc, input: Arc, output: Arc, + operator: Operator, params: Arc, config: HashTableConfig, + location_prefix: String, ) -> Result> { let hash_table = { let arena = Arc::new(Bump::new()); @@ -92,6 +112,15 @@ impl TransformPartialAggregate { } }; + let config = SpillerConfig { + spiller_type: SpillerType::Aggregation, + location_prefix, + disk_spill: None, + use_parquet: ctx.get_settings().get_spilling_file_format()?.is_parquet(), + }; + + let spiller = Spiller::create(ctx.clone(), operator, config.clone())?; + Ok(AccumulatingTransformer::create( input, output, @@ -104,6 +133,10 @@ impl TransformPartialAggregate { first_block_start: None, processed_bytes: 0, processed_rows: 0, + configure_peer_nodes: vec![], + spilling_state: None, + spiller: Arc::new(spiller), + spill_blocks: vec![], }, )) } @@ -184,45 +217,46 @@ impl TransformPartialAggregate { } } +#[async_trait::async_trait] impl AccumulatingTransform for TransformPartialAggregate { const NAME: &'static str = "TransformPartialAggregate"; fn transform(&mut self, block: DataBlock) -> Result> { self.execute_one_block(block)?; - if self.settings.check_spill() { - 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!() - } + // if self.settings.check_spill() { + // 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![]) } @@ -272,4 +306,315 @@ impl AccumulatingTransform for TransformPartialAggregate { } }) } + + fn configure_peer_nodes(&mut self, nodes: &[String]) { + self.configure_peer_nodes = nodes.to_vec(); + } + + fn need_spill(&self) -> bool { + self.settings.check_spill() + } + + fn prepare_spill_payload(&mut self) -> Result { + if self.spilling_state.is_none() { + let HashTable::AggregateHashTable(hashtable) = std::mem::take(&mut self.hash_table) + else { + return Ok(false); + }; + + if hashtable.len() == 0 { + return Ok(false); + } + + self.spilling_state = Some(HashtableSpillingState::create( + hashtable, + self.configure_peer_nodes.len(), + )); + } + + let Some(hashtable_spilling_state) = &mut self.spilling_state else { + return Ok(false); + }; + + if hashtable_spilling_state.finished { + return Ok(false); + } + + hashtable_spilling_state.last_prepare_payload = + hashtable_spilling_state.serialize_partition_payload()?; + Ok(true) + } + + async fn flush_spill_payload(&mut self) -> Result { + let hashtable_spilling_state = self.spilling_state.as_mut().unwrap(); + let max_partition = 1 << hashtable_spilling_state.ht.config.max_radix_bits; + + if hashtable_spilling_state.writer.is_none() { + let location = self.spiller.create_unique_location(); + hashtable_spilling_state.writer = + Some(self.spiller.create_aggregate_writer(location).await?); + } + + let writer = hashtable_spilling_state.writer.as_mut().unwrap(); + + if !hashtable_spilling_state.flush_data.is_empty() { + let mut flush_data = Vec::with_capacity(4 * 1024 * 1024); + std::mem::swap(&mut flush_data, &mut hashtable_spilling_state.flush_data); + writer.write(flush_data).await?; + hashtable_spilling_state.flush_data.clear(); + } + + if hashtable_spilling_state.last_prepare_payload { + if writer.write_bytes() > hashtable_spilling_state.last_flush_partition_offset { + // TODO: + self.spill_blocks.push(DataBlock::empty_with_meta( + AggregateMeta::create_bucket_spilled(BucketSpilledPayload { + bucket: hashtable_spilling_state.work_partition as isize, + location: "".to_string(), + data_range: Default::default(), + columns_layout: vec![], + max_partition_count: 0, + }), + )); + + hashtable_spilling_state.last_flush_partition_offset = writer.write_bytes(); + } + + hashtable_spilling_state.work_partition += 1; + + if hashtable_spilling_state.work_partition < max_partition { + return Ok(true); + } + + writer.complete().await?; + let location = self.spiller.create_unique_location(); + hashtable_spilling_state.writer = + Some(self.spiller.create_aggregate_writer(location).await?); + + hashtable_spilling_state.payload_idx = 0; + hashtable_spilling_state.work_partition = 0; + hashtable_spilling_state.scatter_work_bucket += 1; + + if hashtable_spilling_state.scatter_work_bucket + < hashtable_spilling_state.scatter_max_bucket + { + return Ok(true); + } + + hashtable_spilling_state.finished = true; + hashtable_spilling_state + .ht + .config + .update_current_max_radix_bits(); + + let config = hashtable_spilling_state + .ht + .config + .clone() + .with_initial_radix_bits(hashtable_spilling_state.ht.config.max_radix_bits); + + let aggrs = hashtable_spilling_state.ht.payload.aggrs.clone(); + let group_types = hashtable_spilling_state.ht.payload.group_types.clone(); + self.spilling_state = None; + self.hash_table = HashTable::AggregateHashTable(AggregateHashTable::new( + group_types, + aggrs, + config, + Arc::new(Bump::new()), + )); + + return Ok(false); + } + + Ok(true) + } +} + +pub struct HashtableSpillingState { + ht: AggregateHashTable, + payload_idx: usize, + work_partition: usize, + partition_state: PayloadFlushState, + + scatter_max_bucket: usize, + scatter_work_bucket: usize, + scatter_state: PayloadFlushState, + + serialize_state: PayloadFlushState, + + flush_data: Vec, + writer: Option, + finished: bool, + last_prepare_payload: bool, + + last_flush_partition_offset: usize, +} + +impl HashtableSpillingState { + pub fn create(ht: AggregateHashTable, scatter_max_bucket: usize) -> Self { + HashtableSpillingState { + ht, + payload_idx: 0, + work_partition: 0, + partition_state: PayloadFlushState::default(), + scatter_max_bucket, + scatter_work_bucket: 0, + scatter_state: PayloadFlushState::default(), + serialize_state: PayloadFlushState::default(), + flush_data: Vec::with_capacity(6 * 1024 * 1024), + writer: None, + finished: false, + last_prepare_payload: false, + last_flush_partition_offset: 0, + } + } + pub fn serialize_payload(&mut self, payload: Option) -> Result { + let payload = match payload.as_ref() { + Some(payload) => payload, + None => &self.ht.payload.payloads[self.work_partition], + }; + + if payload.len() == 0 { + return Ok(true); + } + + if let Some(data_block) = payload.aggregate_flush(&mut self.serialize_state)? { + if data_block.num_rows() == 0 { + return Ok(true); + } + + let columns = data_block.columns().to_vec(); + for column in columns.into_iter() { + let column = column.to_column(data_block.num_rows()); + + let offset = self.flush_data.len(); + self.flush_data + .write_u64::(0) + .map_err(|_| ErrorCode::Internal("Cannot serialize column"))?; + + write_column(&column, &mut self.flush_data) + .map_err(|_| ErrorCode::Internal("Cannot serialize column"))?; + + let len = self.flush_data.len(); + let mut buffer = &mut self.flush_data[offset..]; + buffer + .write_u64::((len - offset) as u64) + .map_err(|_| ErrorCode::Internal("Cannot serialize column"))?; + } + } + + Ok(self.flush_data.len() < 4 * 1024 * 1024) + } + + pub fn serialize_scatter_payload(&mut self, raw_payload: Option) -> Result { + if self.scatter_max_bucket <= 1 { + return self.serialize_payload(raw_payload); + } + + // using if-else to avoid mutable borrow occurs here + if let Some(payload) = raw_payload { + while payload.scatter(&mut self.scatter_state, self.scatter_max_bucket) { + let idx = self.scatter_work_bucket; + let rows = self.scatter_state.probe_state.partition_count[idx]; + + if rows == 0 { + continue; + } + + let sel = &self.scatter_state.probe_state.partition_entries[idx]; + + let mut scattered_payload = Payload::new( + payload.arena.clone(), + payload.group_types.clone(), + payload.aggrs.clone(), + payload.states_layout.clone(), + ); + + scattered_payload.copy_rows(sel, rows, &self.scatter_state.addresses); + + if !self.serialize_payload(Some(scattered_payload))? { + return Ok(false); + } + } + } else { + while self.ht.payload.payloads[self.work_partition] + .scatter(&mut self.scatter_state, self.scatter_max_bucket) + { + let idx = self.scatter_work_bucket; + let rows = self.scatter_state.probe_state.partition_count[idx]; + + if rows == 0 { + continue; + } + + let sel = &self.scatter_state.probe_state.partition_entries[idx]; + + let mut scattered_payload = Payload::new( + self.ht.payload.payloads[self.work_partition].arena.clone(), + self.ht.payload.payloads[self.work_partition] + .group_types + .clone(), + self.ht.payload.payloads[self.work_partition].aggrs.clone(), + self.ht.payload.payloads[self.work_partition] + .states_layout + .clone(), + ); + + scattered_payload.copy_rows(sel, rows, &self.scatter_state.addresses); + + if !self.serialize_payload(Some(scattered_payload))? { + return Ok(false); + } + } + } + + Ok(true) + } + + pub fn serialize_partition_payload(&mut self) -> Result { + let max_partitions = 1 << self.ht.config.max_radix_bits; + if self.ht.payload.partition_count() == max_partitions { + return self.serialize_scatter_payload(None); + } + + let partition_payload = PartitionedPayload::new( + self.ht.payload.group_types.clone(), + self.ht.payload.aggrs.clone(), + max_partitions as u64, + self.ht.payload.arenas.clone(), + ); + + for idx in self.payload_idx..self.ht.payload.payloads.len() { + while partition_payload + .gather_flush(&self.ht.payload.payloads[idx], &mut self.partition_state) + { + let rows = self.partition_state.probe_state.partition_count[self.work_partition]; + + if rows == 0 { + continue; + } + + let selector = + &self.partition_state.probe_state.partition_entries[self.work_partition]; + let addresses = &self.partition_state.addresses; + + let mut new_payload = Payload::new( + self.ht.payload.payloads[idx].arena.clone(), + self.ht.payload.payloads[idx].group_types.clone(), + self.ht.payload.payloads[idx].aggrs.clone(), + self.ht.payload.payloads[idx].states_layout.clone(), + ); + + new_payload.copy_rows(selector, rows, addresses); + + if !self.serialize_scatter_payload(Some(new_payload))? { + return Ok(false); + } + } + + self.payload_idx += 1; + } + + Ok(true) + } } diff --git a/src/query/service/src/servers/flight/v1/exchange/exchange_transform_shuffle.rs b/src/query/service/src/servers/flight/v1/exchange/exchange_transform_shuffle.rs index 9c3242147c8d3..f071b1fa5ac15 100644 --- a/src/query/service/src/servers/flight/v1/exchange/exchange_transform_shuffle.rs +++ b/src/query/service/src/servers/flight/v1/exchange/exchange_transform_shuffle.rs @@ -382,6 +382,12 @@ pub fn exchange_shuffle( params: &ShuffleExchangeParams, pipeline: &mut Pipeline, ) -> Result<()> { + if let Some(last_pipe) = pipeline.pipes.last() { + for item in &last_pipe.items { + item.processor.configure_peer_nodes(¶ms.destination_ids); + } + } + // append scatter transform pipeline.add_transform(|input, output| { Ok(ScatterTransform::create( diff --git a/src/query/service/src/spillers/spiller.rs b/src/query/service/src/spillers/spiller.rs index 5111d597c0346..c2cc13d07f96c 100644 --- a/src/query/service/src/spillers/spiller.rs +++ b/src/query/service/src/spillers/spiller.rs @@ -35,6 +35,7 @@ use databend_storages_common_cache::TempDir; use databend_storages_common_cache::TempPath; use opendal::Buffer; use opendal::Operator; +use opendal::Writer; use parking_lot::RwLock; use super::serialize::*; @@ -187,6 +188,17 @@ impl Spiller { format!("{}/{}", self.location_prefix, GlobalUniqName::unique()) } + pub async fn create_aggregate_writer(&self, location: String) -> Result { + let writer = self.operator.writer(&location).await?; + Ok(SpillWriter { + bytes: 0, + writer, + location, + ctx: self.ctx.clone(), + private_spilled_files: self.private_spilled_files.clone(), + }) + } + pub async fn spill_stream_aggregate_buffer( &self, location: Option, @@ -558,3 +570,38 @@ fn record_read_profile(location: &Location, start: &Instant, read_bytes: usize) } } } + +pub struct SpillWriter { + bytes: usize, + writer: Writer, + location: String, + ctx: Arc, + private_spilled_files: Arc>>, +} + +impl SpillWriter { + pub async fn write(&mut self, bytes: Vec) -> Result<()> { + self.bytes += bytes.len(); + Ok(self.writer.write(bytes).await?) + } + + pub fn write_bytes(&self) -> usize { + self.bytes + } + + pub async fn complete(&mut self) -> Result<()> { + self.writer.close().await?; + + self.ctx.add_spill_file( + Location::Remote(self.location.clone()), + Layout::Aggregate, + self.bytes, + ); + + self.private_spilled_files + .write() + .insert(Location::Remote(self.location.clone()), Layout::Aggregate); + + Ok(()) + } +} diff --git a/src/query/storages/system/src/query_log_table.rs b/src/query/storages/system/src/query_log_table.rs index c6c8e16946fdf..fc83a87d05cc4 100644 --- a/src/query/storages/system/src/query_log_table.rs +++ b/src/query/storages/system/src/query_log_table.rs @@ -179,7 +179,7 @@ pub struct QueryLogElement { // Transaction pub txn_state: String, pub txn_id: String, - pub peek_memory_usage: HashMap, + pub peak_memory_usage: HashMap, } impl SystemLogElement for QueryLogElement { @@ -575,7 +575,7 @@ impl SystemLogElement for QueryLogElement { columns.next().unwrap().push( Scalar::Variant( jsonb::Value::from(jsonb::Object::from_iter( - self.peek_memory_usage + self.peak_memory_usage .iter() .map(|(k, v)| (k.clone(), jsonb::Value::from(*v))), )) From d6e7d7c12539625e9fcdf0b0710051f952f7a0f2 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 5 Mar 2025 21:36:42 +0800 Subject: [PATCH 02/76] refactor(query): refactor aggreagte spill code --- .../new_transform_partition_bucket.rs | 8 +- .../serde/transform_spill_reader.rs | 36 +- .../aggregator/transform_aggregate_partial.rs | 318 +++++++++--------- src/query/service/src/spillers/spiller.rs | 4 + 4 files changed, 191 insertions(+), 175 deletions(-) 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 5c5cddc4258fd..6f82e38ef8d1e 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 @@ -598,7 +598,13 @@ pub fn build_partition_bucket( let operator = DataOperator::instance().spill_operator(); pipeline.add_transform(|input, output| { let operator = operator.clone(); - TransformAggregateSpillReader::create(input, output, operator, semaphore.clone()) + TransformAggregateSpillReader::create( + input, + output, + operator, + semaphore.clone(), + params.clone(), + ) })?; pipeline.add_transform(|input, output| { 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 fd03b09e2f3f7..dcd220b0dcc5b 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 @@ -18,6 +18,8 @@ use std::sync::Arc; use std::time::Duration; use std::time::Instant; +use byteorder::BigEndian; +use byteorder::ReadBytesExt; use databend_common_base::runtime::profile::Profile; use databend_common_base::runtime::profile::ProfileStatisticsName; use databend_common_exception::ErrorCode; @@ -37,6 +39,7 @@ use opendal::Operator; use tokio::sync::Semaphore; 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::SerializedPayload; @@ -48,6 +51,7 @@ pub struct TransformSpillReader { operator: Operator, semaphore: Arc, + params: Arc, deserialized_meta: Option, reading_meta: Option, deserializing_meta: Option, @@ -141,7 +145,7 @@ impl Processor for TransformSpillReader { debug_assert!(read_data.len() == 1); let data = read_data.pop_front().unwrap(); - self.deserialized_meta = Some(Box::new(Self::deserialize(payload, data))); + self.deserialized_meta = Some(Box::new(self.deserialize(payload, data))); } AggregateMeta::Partitioned { bucket, data } => { let mut new_data = Vec::with_capacity(data.len()); @@ -150,7 +154,7 @@ impl Processor for TransformSpillReader { if matches!(&meta, AggregateMeta::BucketSpilled(_)) { if let AggregateMeta::BucketSpilled(payload) = meta { let data = read_data.pop_front().unwrap(); - new_data.push(Self::deserialize(payload, data)); + new_data.push(self.deserialize(payload, data)); } continue; @@ -282,30 +286,44 @@ impl TransformSpillReader { output: Arc, operator: Operator, semaphore: Arc, + params: Arc, ) -> Result { Ok(ProcessorPtr::create(Box::new(TransformSpillReader { input, output, operator, semaphore, + params, deserialized_meta: None, reading_meta: None, deserializing_meta: None, }))) } - fn deserialize(payload: BucketSpilledPayload, data: Vec) -> AggregateMeta { - let mut begin = 0; - let mut columns = Vec::with_capacity(payload.columns_layout.len()); + fn deserialize(&self, payload: BucketSpilledPayload, data: Vec) -> AggregateMeta { + let columns = self.params.group_data_types.len() + self.params.aggregate_functions.len(); - for column_layout in payload.columns_layout { - columns.push(deserialize_column(&data[begin..begin + column_layout as usize]).unwrap()); - begin += column_layout as usize; + let mut blocks = vec![]; + let mut cursor = data.as_slice(); + + while !cursor.is_empty() { + let mut block_columns = Vec::with_capacity(columns); + + for _idx in 0..columns { + let column_size = cursor.read_u64::().unwrap(); + let (left, right) = cursor.split_at(column_size as usize); + block_columns.push(deserialize_column(left).unwrap()); + cursor = right; + } + + blocks.push(DataBlock::new_from_columns(block_columns)); } + let block = DataBlock::concat(&blocks).unwrap(); + AggregateMeta::Serialized(SerializedPayload { bucket: payload.bucket, - data_block: DataBlock::new_from_columns(columns), + data_block: block, max_partition_count: payload.max_partition_count, }) } 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 69f7f7f29a00d..e90afa5f922a9 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 @@ -215,6 +215,31 @@ impl TransformPartialAggregate { } } } + + fn reset_hashtable(&mut self) { + let hashtable_spilling_state = self.spilling_state.as_mut().unwrap(); + + hashtable_spilling_state + .ht + .config + .update_current_max_radix_bits(); + + let config = hashtable_spilling_state + .ht + .config + .clone() + .with_initial_radix_bits(hashtable_spilling_state.ht.config.max_radix_bits); + + let aggrs = hashtable_spilling_state.ht.payload.aggrs.clone(); + let group_types = hashtable_spilling_state.ht.payload.group_types.clone(); + self.spilling_state = None; + self.hash_table = HashTable::AggregateHashTable(AggregateHashTable::new( + group_types, + aggrs, + config, + Arc::new(Bump::new()), + )); + } } #[async_trait::async_trait] @@ -224,41 +249,7 @@ impl AccumulatingTransform for TransformPartialAggregate { fn transform(&mut self, block: DataBlock) -> Result> { self.execute_one_block(block)?; - // if self.settings.check_spill() { - // 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![]) + Ok(std::mem::take(&mut self.spill_blocks)) } fn on_finish(&mut self, output: bool) -> Result> { @@ -269,7 +260,8 @@ impl AccumulatingTransform for TransformPartialAggregate { }, HashTable::AggregateHashTable(hashtable) => { let partition_count = hashtable.payload.partition_count(); - let mut blocks = Vec::with_capacity(partition_count); + let mut blocks = std::mem::take(&mut self.spill_blocks); + blocks.reserve(partition_count); log::info!( "Aggregated {} to {} rows in {} sec(real: {}). ({} rows/sec, {}/sec, {})", @@ -317,111 +309,90 @@ impl AccumulatingTransform for TransformPartialAggregate { fn prepare_spill_payload(&mut self) -> Result { if self.spilling_state.is_none() { - let HashTable::AggregateHashTable(hashtable) = std::mem::take(&mut self.hash_table) - else { + let HashTable::AggregateHashTable(ht) = std::mem::take(&mut self.hash_table) else { return Ok(false); }; - if hashtable.len() == 0 { + if ht.len() == 0 { return Ok(false); } - self.spilling_state = Some(HashtableSpillingState::create( - hashtable, - self.configure_peer_nodes.len(), - )); + let max_bucket = self.configure_peer_nodes.len(); + self.spilling_state = Some(HashtableSpillingState::create(ht, max_bucket)); } - let Some(hashtable_spilling_state) = &mut self.spilling_state else { - return Ok(false); - }; + if let Some(spilling_state) = self.spilling_state.as_mut() { + // spill is finished. + if spilling_state.finished { + return Ok(false); + } - if hashtable_spilling_state.finished { - return Ok(false); + spilling_state.last_prepare_payload = spilling_state.serialize_partition_payload()?; + return Ok(true); } - hashtable_spilling_state.last_prepare_payload = - hashtable_spilling_state.serialize_partition_payload()?; - Ok(true) + Ok(false) } async fn flush_spill_payload(&mut self) -> Result { let hashtable_spilling_state = self.spilling_state.as_mut().unwrap(); + + let max_bucket = hashtable_spilling_state.max_bucket; let max_partition = 1 << hashtable_spilling_state.ht.config.max_radix_bits; - if hashtable_spilling_state.writer.is_none() { - let location = self.spiller.create_unique_location(); - hashtable_spilling_state.writer = - Some(self.spiller.create_aggregate_writer(location).await?); - } + if !hashtable_spilling_state.data_payload.is_empty() { + if hashtable_spilling_state.writer.is_none() { + let location = self.spiller.create_unique_location(); + hashtable_spilling_state.writer = + Some(self.spiller.create_aggregate_writer(location).await?); + } - let writer = hashtable_spilling_state.writer.as_mut().unwrap(); + let writer = hashtable_spilling_state.writer.as_mut().unwrap(); - if !hashtable_spilling_state.flush_data.is_empty() { let mut flush_data = Vec::with_capacity(4 * 1024 * 1024); - std::mem::swap(&mut flush_data, &mut hashtable_spilling_state.flush_data); + std::mem::swap(&mut flush_data, &mut hashtable_spilling_state.data_payload); writer.write(flush_data).await?; - hashtable_spilling_state.flush_data.clear(); } if hashtable_spilling_state.last_prepare_payload { - if writer.write_bytes() > hashtable_spilling_state.last_flush_partition_offset { - // TODO: - self.spill_blocks.push(DataBlock::empty_with_meta( - AggregateMeta::create_bucket_spilled(BucketSpilledPayload { - bucket: hashtable_spilling_state.work_partition as isize, - location: "".to_string(), - data_range: Default::default(), - columns_layout: vec![], - max_partition_count: 0, - }), - )); - - hashtable_spilling_state.last_flush_partition_offset = writer.write_bytes(); + if let Some(writer) = hashtable_spilling_state.writer.as_mut() { + let last_offset = hashtable_spilling_state.last_flush_partition_offset; + if writer.write_bytes() > last_offset { + self.spill_blocks.push(DataBlock::empty_with_meta( + AggregateMeta::create_bucket_spilled(BucketSpilledPayload { + bucket: hashtable_spilling_state.working_partition as isize, + location: writer.location(), + data_range: last_offset as u64..writer.write_bytes() as u64, + columns_layout: vec![], + max_partition_count: max_partition, + }), + )); + + hashtable_spilling_state.last_flush_partition_offset = writer.write_bytes(); + } } - hashtable_spilling_state.work_partition += 1; - - if hashtable_spilling_state.work_partition < max_partition { + hashtable_spilling_state.payload_idx = 0; + hashtable_spilling_state.working_partition += 1; + if hashtable_spilling_state.working_partition < max_partition { return Ok(true); } - writer.complete().await?; - let location = self.spiller.create_unique_location(); - hashtable_spilling_state.writer = - Some(self.spiller.create_aggregate_writer(location).await?); + if let Some(writer) = hashtable_spilling_state.writer.as_mut() { + writer.complete().await?; + hashtable_spilling_state.writer = None; + } hashtable_spilling_state.payload_idx = 0; - hashtable_spilling_state.work_partition = 0; - hashtable_spilling_state.scatter_work_bucket += 1; + hashtable_spilling_state.working_partition = 0; + hashtable_spilling_state.working_bucket += 1; - if hashtable_spilling_state.scatter_work_bucket - < hashtable_spilling_state.scatter_max_bucket - { + if hashtable_spilling_state.working_bucket < max_bucket { return Ok(true); } hashtable_spilling_state.finished = true; - hashtable_spilling_state - .ht - .config - .update_current_max_radix_bits(); - - let config = hashtable_spilling_state - .ht - .config - .clone() - .with_initial_radix_bits(hashtable_spilling_state.ht.config.max_radix_bits); - - let aggrs = hashtable_spilling_state.ht.payload.aggrs.clone(); - let group_types = hashtable_spilling_state.ht.payload.group_types.clone(); - self.spilling_state = None; - self.hash_table = HashTable::AggregateHashTable(AggregateHashTable::new( - group_types, - aggrs, - config, - Arc::new(Bump::new()), - )); + self.reset_hashtable(); return Ok(false); } @@ -433,19 +404,20 @@ impl AccumulatingTransform for TransformPartialAggregate { pub struct HashtableSpillingState { ht: AggregateHashTable, payload_idx: usize, - work_partition: usize, - partition_state: PayloadFlushState, + working_partition: usize, + partition_flush_state: PayloadFlushState, - scatter_max_bucket: usize, - scatter_work_bucket: usize, - scatter_state: PayloadFlushState, + max_bucket: usize, + working_bucket: usize, + bucket_flush_state: PayloadFlushState, - serialize_state: PayloadFlushState, + serialize_flush_state: PayloadFlushState, + + data_payload: Vec, - flush_data: Vec, - writer: Option, finished: bool, last_prepare_payload: bool, + writer: Option, last_flush_partition_offset: usize, } @@ -455,13 +427,13 @@ impl HashtableSpillingState { HashtableSpillingState { ht, payload_idx: 0, - work_partition: 0, - partition_state: PayloadFlushState::default(), - scatter_max_bucket, - scatter_work_bucket: 0, - scatter_state: PayloadFlushState::default(), - serialize_state: PayloadFlushState::default(), - flush_data: Vec::with_capacity(6 * 1024 * 1024), + working_partition: 0, + partition_flush_state: PayloadFlushState::default(), + max_bucket: scatter_max_bucket, + working_bucket: 0, + bucket_flush_state: PayloadFlushState::default(), + serialize_flush_state: PayloadFlushState::default(), + data_payload: Vec::with_capacity(6 * 1024 * 1024), writer: None, finished: false, last_prepare_payload: false, @@ -471,57 +443,64 @@ impl HashtableSpillingState { pub fn serialize_payload(&mut self, payload: Option) -> Result { let payload = match payload.as_ref() { Some(payload) => payload, - None => &self.ht.payload.payloads[self.work_partition], + None => &self.ht.payload.payloads[self.working_partition], }; if payload.len() == 0 { return Ok(true); } - if let Some(data_block) = payload.aggregate_flush(&mut self.serialize_state)? { + while let Some(data_block) = payload.aggregate_flush(&mut self.serialize_flush_state)? { if data_block.num_rows() == 0 { - return Ok(true); + // next batch rows + continue; } let columns = data_block.columns().to_vec(); for column in columns.into_iter() { let column = column.to_column(data_block.num_rows()); - let offset = self.flush_data.len(); - self.flush_data - .write_u64::(0) - .map_err(|_| ErrorCode::Internal("Cannot serialize column"))?; + let offset = self.data_payload.len(); - write_column(&column, &mut self.flush_data) - .map_err(|_| ErrorCode::Internal("Cannot serialize column"))?; + self.data_payload.write_u64::(0)?; + write_column(&column, &mut self.data_payload)?; - let len = self.flush_data.len(); - let mut buffer = &mut self.flush_data[offset..]; - buffer - .write_u64::((len - offset) as u64) - .map_err(|_| ErrorCode::Internal("Cannot serialize column"))?; + // rewrite column length + let len = self.data_payload.len(); + let mut buffer = &mut self.data_payload[offset..]; + buffer.write_u64::((len - offset - size_of::()) as u64)?; + } + + if self.data_payload.len() >= 4 * 1024 * 1024 { + // flush data if >= 4MB + return Ok(false); } } - Ok(self.flush_data.len() < 4 * 1024 * 1024) + self.serialize_flush_state.clear(); + Ok(true) } pub fn serialize_scatter_payload(&mut self, raw_payload: Option) -> Result { - if self.scatter_max_bucket <= 1 { + // If no need scatter + if self.max_bucket <= 1 { return self.serialize_payload(raw_payload); } // using if-else to avoid mutable borrow occurs here if let Some(payload) = raw_payload { - while payload.scatter(&mut self.scatter_state, self.scatter_max_bucket) { - let idx = self.scatter_work_bucket; - let rows = self.scatter_state.probe_state.partition_count[idx]; + while payload.scatter(&mut self.bucket_flush_state, self.max_bucket) { + let working_bucket = self.working_bucket; + let flush_state = &mut self.bucket_flush_state; + + let rows = flush_state.probe_state.partition_count[working_bucket]; if rows == 0 { + // next batch rows continue; } - let sel = &self.scatter_state.probe_state.partition_entries[idx]; + let sel = &flush_state.probe_state.partition_entries[working_bucket]; let mut scattered_payload = Payload::new( payload.arena.clone(), @@ -530,37 +509,36 @@ impl HashtableSpillingState { payload.states_layout.clone(), ); - scattered_payload.copy_rows(sel, rows, &self.scatter_state.addresses); + scattered_payload.copy_rows(sel, rows, &flush_state.addresses); if !self.serialize_payload(Some(scattered_payload))? { return Ok(false); } } } else { - while self.ht.payload.payloads[self.work_partition] - .scatter(&mut self.scatter_state, self.scatter_max_bucket) + while self.ht.payload.payloads[self.working_partition] + .scatter(&mut self.bucket_flush_state, self.max_bucket) { - let idx = self.scatter_work_bucket; - let rows = self.scatter_state.probe_state.partition_count[idx]; + let working_bucket = self.working_bucket; + let flush_state = &mut self.bucket_flush_state; + let rows = flush_state.probe_state.partition_count[working_bucket]; if rows == 0 { + // next batch rows continue; } - let sel = &self.scatter_state.probe_state.partition_entries[idx]; + let sel = &flush_state.probe_state.partition_entries[working_bucket]; + let working_payload = &self.ht.payload.payloads[self.working_partition]; let mut scattered_payload = Payload::new( - self.ht.payload.payloads[self.work_partition].arena.clone(), - self.ht.payload.payloads[self.work_partition] - .group_types - .clone(), - self.ht.payload.payloads[self.work_partition].aggrs.clone(), - self.ht.payload.payloads[self.work_partition] - .states_layout - .clone(), + working_payload.arena.clone(), + working_payload.group_types.clone(), + working_payload.aggrs.clone(), + working_payload.states_layout.clone(), ); - scattered_payload.copy_rows(sel, rows, &self.scatter_state.addresses); + scattered_payload.copy_rows(sel, rows, &flush_state.addresses); if !self.serialize_payload(Some(scattered_payload))? { return Ok(false); @@ -568,11 +546,14 @@ impl HashtableSpillingState { } } + self.bucket_flush_state.clear(); Ok(true) } pub fn serialize_partition_payload(&mut self) -> Result { let max_partitions = 1 << self.ht.config.max_radix_bits; + + // If no need repartition if self.ht.payload.partition_count() == max_partitions { return self.serialize_scatter_payload(None); } @@ -584,37 +565,44 @@ impl HashtableSpillingState { self.ht.payload.arenas.clone(), ); + // repartition and get current partition payload for idx in self.payload_idx..self.ht.payload.payloads.len() { - while partition_payload - .gather_flush(&self.ht.payload.payloads[idx], &mut self.partition_state) - { - let rows = self.partition_state.probe_state.partition_count[self.work_partition]; + while partition_payload.gather_flush( + &self.ht.payload.payloads[idx], + &mut self.partition_flush_state, + ) { + let working_partition = self.working_partition; + let flush_state = &mut self.partition_flush_state; + + let rows = flush_state.probe_state.partition_count[working_partition]; if rows == 0 { + // next batch rows continue; } - let selector = - &self.partition_state.probe_state.partition_entries[self.work_partition]; - let addresses = &self.partition_state.addresses; + let address = &flush_state.addresses; + let selector = &flush_state.probe_state.partition_entries[working_partition]; - let mut new_payload = Payload::new( + let mut working_partition_payload = Payload::new( self.ht.payload.payloads[idx].arena.clone(), self.ht.payload.payloads[idx].group_types.clone(), self.ht.payload.payloads[idx].aggrs.clone(), self.ht.payload.payloads[idx].states_layout.clone(), ); - new_payload.copy_rows(selector, rows, addresses); + working_partition_payload.copy_rows(selector, rows, address); - if !self.serialize_scatter_payload(Some(new_payload))? { + if !self.serialize_scatter_payload(Some(working_partition_payload))? { return Ok(false); } } self.payload_idx += 1; + self.partition_flush_state.clear(); } + self.partition_flush_state.clear(); Ok(true) } } diff --git a/src/query/service/src/spillers/spiller.rs b/src/query/service/src/spillers/spiller.rs index c2cc13d07f96c..8f71411eb67c3 100644 --- a/src/query/service/src/spillers/spiller.rs +++ b/src/query/service/src/spillers/spiller.rs @@ -585,6 +585,10 @@ impl SpillWriter { Ok(self.writer.write(bytes).await?) } + pub fn location(&self) -> String { + self.location.clone() + } + pub fn write_bytes(&self) -> usize { self.bytes } From 259fec80c8984297144a094a4fa6a96a95656d6b Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Fri, 7 Mar 2025 02:09:03 +0800 Subject: [PATCH 03/76] refactor(query): refactor aggreagte spill code --- .../src/aggregate/partitioned_payload.rs | 16 + src/query/pipeline/core/src/processors/mod.rs | 1 + .../core/src/processors/shuffle_processor.rs | 4 +- .../aggregator/aggregate_exchange_injector.rs | 513 +++++++----- .../transforms/aggregator/aggregate_meta.rs | 95 ++- .../processors/transforms/aggregator/mod.rs | 1 + .../new_transform_partition_bucket.rs | 49 +- .../serde/transform_aggregate_serializer.rs | 6 +- .../serde/transform_aggregate_spill_writer.rs | 440 +++++------ .../serde/transform_deserializer.rs | 181 +++-- ...transform_exchange_aggregate_serializer.rs | 570 +++++++------- .../serde/transform_spill_reader.rs | 4 - .../aggregator/transform_aggregate_final.rs | 4 +- .../aggregator/transform_aggregate_partial.rs | 67 +- .../partition/window_partition_exchange.rs | 3 +- ...window_partition_partial_top_n_exchange.rs | 3 +- .../flight/v1/exchange/exchange_injector.rs | 148 ++-- .../flight/v1/exchange/exchange_sink.rs | 14 +- .../flight/v1/exchange/exchange_source.rs | 5 +- .../flight/v1/exchange/exchange_transform.rs | 5 +- .../v1/exchange/exchange_transform_shuffle.rs | 745 +++++++++--------- .../v1/exchange/serde/exchange_serializer.rs | 5 +- .../flight/v1/scatter/flight_scatter_merge.rs | 37 + .../src/servers/flight/v1/scatter/mod.rs | 2 + 24 files changed, 1574 insertions(+), 1344 deletions(-) create mode 100644 src/query/service/src/servers/flight/v1/scatter/flight_scatter_merge.rs diff --git a/src/query/expression/src/aggregate/partitioned_payload.rs b/src/query/expression/src/aggregate/partitioned_payload.rs index 5b27d6939f330..69988dfb50741 100644 --- a/src/query/expression/src/aggregate/partitioned_payload.rs +++ b/src/query/expression/src/aggregate/partitioned_payload.rs @@ -16,6 +16,8 @@ use std::sync::Arc; use bumpalo::Bump; use itertools::Itertools; +use serde::Deserializer; +use serde::Serializer; use super::payload::Payload; use super::probe_state::ProbeState; @@ -50,6 +52,20 @@ pub struct PartitionedPayload { unsafe impl Send for PartitionedPayload {} unsafe impl Sync for PartitionedPayload {} +impl serde::Serialize for PartitionedPayload { + fn serialize(&self, serializer: S) -> Result + where S: Serializer { + todo!() + } +} + +impl<'de> serde::Deserialize<'de> for PartitionedPayload { + fn deserialize(deserializer: D) -> Result + where D: Deserializer<'de> { + todo!() + } +} + impl PartitionedPayload { pub fn new( group_types: Vec, diff --git a/src/query/pipeline/core/src/processors/mod.rs b/src/query/pipeline/core/src/processors/mod.rs index c3b0e1772a341..095a9d597be61 100644 --- a/src/query/pipeline/core/src/processors/mod.rs +++ b/src/query/pipeline/core/src/processors/mod.rs @@ -39,5 +39,6 @@ pub use resize_processor::create_resize_item; pub use resize_processor::ResizeProcessor; pub use shuffle_processor::Exchange; pub use shuffle_processor::MergePartitionProcessor; +pub use shuffle_processor::MultiwayStrategy; pub use shuffle_processor::PartitionProcessor; pub use shuffle_processor::ShuffleProcessor; diff --git a/src/query/pipeline/core/src/processors/shuffle_processor.rs b/src/query/pipeline/core/src/processors/shuffle_processor.rs index 2b57c3b3cc333..99d4c602b0f83 100644 --- a/src/query/pipeline/core/src/processors/shuffle_processor.rs +++ b/src/query/pipeline/core/src/processors/shuffle_processor.rs @@ -35,7 +35,7 @@ pub trait Exchange: Send + Sync + 'static { const SKIP_EMPTY_DATA_BLOCK: bool = false; const STRATEGY: MultiwayStrategy = MultiwayStrategy::Random; - fn partition(&self, data_block: DataBlock, n: usize) -> Result>; + fn partition(&self, data_block: DataBlock, n: usize) -> Result>; fn multiway_pick(&self, _partitions: &[Option]) -> Result { unimplemented!() @@ -254,7 +254,7 @@ impl Processor for PartitionProcessor { let partitioned = self.exchange.partition(block, self.outputs.len())?; - for (index, block) in partitioned.into_iter().enumerate() { + for (index, block) in partitioned.into_iter() { if block.is_empty() && block.get_meta().is_none() { continue; } 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 55688a4347259..a59995ad28400 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,9 +12,14 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::cmp::Ordering; +use std::collections::HashMap; use std::sync::Arc; +use arrow_ipc::writer::IpcWriteOptions; +use arrow_ipc::CompressionType; use bumpalo::Bump; +use databend_common_config::GlobalConfig; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::BlockMetaInfoDowncast; @@ -22,23 +27,18 @@ use databend_common_expression::DataBlock; use databend_common_expression::PartitionedPayload; use databend_common_expression::Payload; use databend_common_expression::PayloadFlushState; -use databend_common_pipeline_core::processors::ProcessorPtr; -use databend_common_pipeline_core::Pipeline; +use databend_common_pipeline_core::processors::Exchange; +use databend_common_pipeline_core::processors::MultiwayStrategy; use databend_common_settings::FlightCompression; -use databend_common_storage::DataOperator; +use itertools::Itertools; use crate::pipelines::processors::transforms::aggregator::aggregate_meta::AggregateMeta; -use crate::pipelines::processors::transforms::aggregator::serde::TransformExchangeAggregateSerializer; -use crate::pipelines::processors::transforms::aggregator::serde::TransformExchangeAsyncBarrier; use crate::pipelines::processors::transforms::aggregator::AggregatorParams; -use crate::pipelines::processors::transforms::aggregator::TransformAggregateDeserializer; -use crate::pipelines::processors::transforms::aggregator::TransformAggregateSerializer; -use crate::pipelines::processors::transforms::aggregator::TransformAggregateSpillWriter; +use crate::servers::flight::v1::exchange::serde::serialize_block; +use crate::servers::flight::v1::exchange::serde::ExchangeSerializeMeta; use crate::servers::flight::v1::exchange::DataExchange; use crate::servers::flight::v1::exchange::ExchangeInjector; use crate::servers::flight::v1::exchange::ExchangeSorting; -use crate::servers::flight::v1::exchange::MergeExchangeParams; -use crate::servers::flight::v1::exchange::ShuffleExchangeParams; use crate::servers::flight::v1::scatter::FlightScatter; use crate::sessions::QueryContext; @@ -63,11 +63,9 @@ impl ExchangeSorting for AggregateExchangeSorting { compute_block_number(v.bucket, v.max_partition_count) } AggregateMeta::AggregatePayload(v) => { - compute_block_number(v.bucket, v.max_partition_count) + compute_block_number(v.partition, v.max_partition_count) } - AggregateMeta::AggregateSpilling(_) - | AggregateMeta::Spilled(_) - | AggregateMeta::BucketSpilled(_) => Ok(-1), + AggregateMeta::BucketSpilled(_) => Ok(-1), }, }, } @@ -75,7 +73,20 @@ impl ExchangeSorting for AggregateExchangeSorting { } struct HashTableHashScatter { - buckets: usize, + bucket_lookup: HashMap, +} + +impl HashTableHashScatter { + pub fn create(nodes: &[String]) -> Arc> { + Arc::new(Box::new(HashTableHashScatter { + bucket_lookup: nodes + .iter() + .cloned() + .enumerate() + .map(|(l, r)| (r, l)) + .collect::>(), + })) + } } fn scatter_payload(mut payload: Payload, buckets: usize) -> Result> { @@ -112,94 +123,90 @@ fn scatter_payload(mut payload: Payload, buckets: usize) -> Result> Ok(buckets) } -fn scatter_partitioned_payload( - partitioned_payload: PartitionedPayload, - buckets: usize, -) -> Result> { - let mut buckets = Vec::with_capacity(buckets); - - let group_types = partitioned_payload.group_types.clone(); - let aggrs = partitioned_payload.aggrs.clone(); - let partition_count = partitioned_payload.partition_count() as u64; - let mut state = PayloadFlushState::default(); - - for _ in 0..buckets.capacity() { - buckets.push(PartitionedPayload::new( - group_types.clone(), - aggrs.clone(), - partition_count, - partitioned_payload.arenas.clone(), - )); - } - - let mut payloads = Vec::with_capacity(buckets.len()); - - for _ in 0..payloads.capacity() { - payloads.push(Payload::new( - Arc::new(Bump::new()), - group_types.clone(), - aggrs.clone(), - partitioned_payload.states_layout.clone(), - )); - } - - for mut payload in partitioned_payload.payloads.into_iter() { - // scatter each page of the payload. - while payload.scatter(&mut state, buckets.len()) { - // copy to the corresponding bucket. - for (idx, bucket) in payloads.iter_mut().enumerate() { - let count = state.probe_state.partition_count[idx]; - - if count > 0 { - let sel = &state.probe_state.partition_entries[idx]; - bucket.copy_rows(sel, count, &state.addresses); - } - } - } - state.clear(); - payload.state_move_out = true; - } - - for (idx, payload) in payloads.into_iter().enumerate() { - buckets[idx].combine_single(payload, &mut state, None); - } - - Ok(buckets) -} +// fn scatter_partitioned_payload( +// partitioned_payload: PartitionedPayload, +// buckets: usize, +// ) -> Result> { +// let mut buckets = Vec::with_capacity(buckets); +// +// let group_types = partitioned_payload.group_types.clone(); +// let aggrs = partitioned_payload.aggrs.clone(); +// let partition_count = partitioned_payload.partition_count() as u64; +// let mut state = PayloadFlushState::default(); +// +// for _ in 0..buckets.capacity() { +// buckets.push(PartitionedPayload::new( +// group_types.clone(), +// aggrs.clone(), +// partition_count, +// partitioned_payload.arenas.clone(), +// )); +// } +// +// let mut payloads = Vec::with_capacity(buckets.len()); +// +// for _ in 0..payloads.capacity() { +// payloads.push(Payload::new( +// Arc::new(Bump::new()), +// group_types.clone(), +// aggrs.clone(), +// partitioned_payload.states_layout.clone(), +// )); +// } +// +// for mut payload in partitioned_payload.payloads.into_iter() { +// // scatter each page of the payload. +// while payload.scatter(&mut state, buckets.len()) { +// // copy to the corresponding bucket. +// for (idx, bucket) in payloads.iter_mut().enumerate() { +// let count = state.probe_state.partition_count[idx]; +// +// if count > 0 { +// let sel = &state.probe_state.partition_entries[idx]; +// bucket.copy_rows(sel, count, &state.addresses); +// } +// } +// } +// state.clear(); +// payload.state_move_out = true; +// } +// +// for (idx, payload) in payloads.into_iter().enumerate() { +// buckets[idx].combine_single(payload, &mut state, None); +// } +// +// Ok(buckets) +// } 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) { - let mut blocks = Vec::with_capacity(self.buckets); - match block_meta { - AggregateMeta::Spilled(_) => unreachable!(), - AggregateMeta::BucketSpilled(_) => unreachable!(), - AggregateMeta::Serialized(_) => unreachable!(), - AggregateMeta::Partitioned { .. } => unreachable!(), - AggregateMeta::AggregateSpilling(payload) => { - for p in scatter_partitioned_payload(payload, self.buckets)? { - blocks.push(DataBlock::empty_with_meta( - AggregateMeta::create_agg_spilling(p), - )); - } - } - AggregateMeta::AggregatePayload(p) => { - for payload in scatter_payload(p.payload, self.buckets)? { - blocks.push(DataBlock::empty_with_meta( - AggregateMeta::create_agg_payload( - p.bucket, - payload, - p.max_partition_count, - ), - )); - } - } - }; - - return Ok(blocks); - } - } + // if let Some(block_meta) = data_block.take_meta() { + // if let Some(block_meta) = AggregateMeta::downcast_from(block_meta) { + // let mut blocks = Vec::with_capacity(self.bucket_lookup.len()); + // match block_meta { + // AggregateMeta::Spilled(_) => unreachable!(), + // AggregateMeta::Serialized(_) => unreachable!(), + // AggregateMeta::Partitioned { .. } => unreachable!(), + // AggregateMeta::AggregateSpilling(_) => unreachable!(), + // AggregateMeta::BucketSpilled(v) => { + // // v.destination_node + // } + // AggregateMeta::AggregatePayload(p) => { + // for payload in scatter_payload(p.payload, self.buckets)? { + // blocks.push(DataBlock::empty_with_meta( + // AggregateMeta::create_agg_payload( + // payload, + // p.partition, + // p.max_partition_count, + // ), + // )); + // } + // } + // }; + // + // return Ok(blocks); + // } + // } Err(ErrorCode::Internal( "Internal, HashTableHashScatter only recv AggregateMeta", @@ -234,9 +241,7 @@ impl ExchangeInjector for AggregateInjector { DataExchange::Merge(_) => unreachable!(), DataExchange::Broadcast(_) => unreachable!(), DataExchange::ShuffleDataExchange(exchange) => { - Ok(Arc::new(Box::new(HashTableHashScatter { - buckets: exchange.destination_ids.len(), - }))) + Ok(HashTableHashScatter::create(&exchange.destination_ids)) } } } @@ -245,89 +250,237 @@ impl ExchangeInjector for AggregateInjector { Some(Arc::new(AggregateExchangeSorting {})) } - fn apply_merge_serializer( - &self, - _: &MergeExchangeParams, - _compression: Option, - pipeline: &mut Pipeline, - ) -> Result<()> { - let params = self.aggregator_params.clone(); - - let operator = DataOperator::instance().spill_operator(); - let location_prefix = self.ctx.query_id_spill_prefix(); - - pipeline.add_transform(|input, output| { - Ok(ProcessorPtr::create( - TransformAggregateSpillWriter::try_create( - self.ctx.clone(), - input, - output, - operator.clone(), - params.clone(), - location_prefix.clone(), - )?, - )) - })?; - - pipeline.add_transform(|input, output| { - TransformAggregateSerializer::try_create(input, output, params.clone()) - }) - } + // fn apply_merge_serializer( + // &self, + // _: &MergeExchangeParams, + // _compression: Option, + // pipeline: &mut Pipeline, + // ) -> Result<()> { + // let params = self.aggregator_params.clone(); + // + // let operator = DataOperator::instance().spill_operator(); + // let location_prefix = self.ctx.query_id_spill_prefix(); + // + // pipeline.add_transform(|input, output| { + // Ok(ProcessorPtr::create( + // TransformAggregateSpillWriter::try_create( + // self.ctx.clone(), + // input, + // output, + // operator.clone(), + // params.clone(), + // location_prefix.clone(), + // )?, + // )) + // })?; + // + // pipeline.add_transform(|input, output| { + // TransformAggregateSerializer::try_create(input, output, params.clone()) + // }) + // } + // + // fn apply_shuffle_serializer( + // &self, + // shuffle_params: &ShuffleExchangeParams, + // compression: Option, + // pipeline: &mut Pipeline, + // ) -> Result<()> { + // // let params = self.aggregator_params.clone(); + // // let operator = DataOperator::instance().spill_operator(); + // // let location_prefix = self.ctx.query_id_spill_prefix(); + // // + // // let schema = shuffle_params.schema.clone(); + // // let local_id = &shuffle_params.executor_id; + // // let local_pos = shuffle_params + // // .destination_ids + // // .iter() + // // .position(|x| x == local_id) + // // .unwrap(); + // // + // // pipeline.add_transform(|input, output| { + // // Ok(ProcessorPtr::create( + // // TransformExchangeAggregateSerializer::try_create( + // // self.ctx.clone(), + // // input, + // // output, + // // operator.clone(), + // // location_prefix.clone(), + // // params.clone(), + // // compression, + // // schema.clone(), + // // local_pos, + // // )?, + // // )) + // // })?; + // // + // // pipeline.add_transform(TransformExchangeAsyncBarrier::try_create) + // Ok(()) + // } + // + // fn apply_merge_deserializer( + // &self, + // params: &MergeExchangeParams, + // pipeline: &mut Pipeline, + // ) -> Result<()> { + // pipeline.add_transform(|input, output| { + // TransformAggregateDeserializer::try_create(input, output, ¶ms.schema) + // }) + // } + // + // fn apply_shuffle_deserializer( + // &self, + // params: &ShuffleExchangeParams, + // pipeline: &mut Pipeline, + // ) -> Result<()> { + // pipeline.add_transform(|input, output| { + // TransformAggregateDeserializer::try_create(input, output, ¶ms.schema) + // }) + // } +} - fn apply_shuffle_serializer( - &self, - shuffle_params: &ShuffleExchangeParams, +pub struct FlightExchange { + local_id: String, + bucket_lookup: HashMap, + recv_bucket_lookup: Vec, + options: IpcWriteOptions, + shuffle_scatter: Arc>, +} + +impl FlightExchange { + pub fn create( + lookup: Vec, compression: Option, - pipeline: &mut Pipeline, - ) -> Result<()> { - let params = self.aggregator_params.clone(); - let operator = DataOperator::instance().spill_operator(); - let location_prefix = self.ctx.query_id_spill_prefix(); - - let schema = shuffle_params.schema.clone(); - let local_id = &shuffle_params.executor_id; - let local_pos = shuffle_params - .destination_ids + shuffle_scatter: Arc>, + ) -> Arc { + let compression = match compression { + None => None, + Some(compression) => match compression { + FlightCompression::Lz4 => Some(CompressionType::LZ4_FRAME), + FlightCompression::Zstd => Some(CompressionType::ZSTD), + }, + }; + + let bucket_lookup = lookup .iter() - .position(|x| x == local_id) - .unwrap(); - - pipeline.add_transform(|input, output| { - Ok(ProcessorPtr::create( - TransformExchangeAggregateSerializer::try_create( - self.ctx.clone(), - input, - output, - operator.clone(), - location_prefix.clone(), - params.clone(), - compression, - schema.clone(), - local_pos, - )?, - )) - })?; - - pipeline.add_transform(TransformExchangeAsyncBarrier::try_create) + .cloned() + .enumerate() + .map(|(x, y)| (y, x)) + .collect::>(); + + Arc::new(FlightExchange { + local_id: GlobalConfig::instance().query.node_id.clone(), + bucket_lookup, + recv_bucket_lookup: lookup, + options: IpcWriteOptions::default() + .try_with_compression(compression) + .unwrap(), + shuffle_scatter, + }) } +} - fn apply_merge_deserializer( - &self, - params: &MergeExchangeParams, - pipeline: &mut Pipeline, - ) -> Result<()> { - pipeline.add_transform(|input, output| { - TransformAggregateDeserializer::try_create(input, output, ¶ms.schema) - }) +impl FlightExchange { + fn default_partition(&self, data_block: DataBlock) -> Result> { + let data_blocks = self.shuffle_scatter.execute(data_block)?; + + let mut blocks = Vec::with_capacity(data_blocks.len()); + for (idx, data_block) in data_blocks.into_iter().enumerate() { + if data_block.is_empty() || self.recv_bucket_lookup[idx] == self.local_id { + blocks.push((idx, data_block)); + continue; + } + + let data_block = serialize_block(0, data_block, &self.options)?; + blocks.push((idx, data_block)); + } + + Ok(blocks) } +} - fn apply_shuffle_deserializer( - &self, - params: &ShuffleExchangeParams, - pipeline: &mut Pipeline, - ) -> Result<()> { - pipeline.add_transform(|input, output| { - TransformAggregateDeserializer::try_create(input, output, ¶ms.schema) - }) +impl Exchange for FlightExchange { + const NAME: &'static str = "AggregateExchange"; + const STRATEGY: MultiwayStrategy = MultiwayStrategy::Custom; + + fn partition(&self, mut data_block: DataBlock, n: usize) -> Result> { + assert_eq!(self.bucket_lookup.len(), n); + + let Some(meta) = data_block.take_meta() else { + return self.default_partition(data_block); + }; + + let Some(meta) = AggregateMeta::downcast_from(meta) else { + return self.default_partition(data_block); + }; + + match meta { + AggregateMeta::Serialized(_) => unreachable!(), + AggregateMeta::Partitioned { .. } => unreachable!(), + AggregateMeta::BucketSpilled(v) => match self.bucket_lookup.get(&v.destination_node) { + None => unreachable!(), + Some(idx) => match v.destination_node == self.local_id { + true => Ok(vec![( + *idx, + DataBlock::empty_with_meta(AggregateMeta::create_bucket_spilled(v)), + )]), + false => { + let block = + DataBlock::empty_with_meta(AggregateMeta::create_bucket_spilled(v)); + Ok(vec![(*idx, serialize_block(-2, block, &self.options)?)]) + } + }, + }, + AggregateMeta::AggregatePayload(p) => { + let mut blocks = Vec::with_capacity(n); + for (idx, payload) in scatter_payload(p.payload, n)?.into_iter().enumerate() { + if self.recv_bucket_lookup[idx] == self.local_id { + blocks.push(( + idx, + DataBlock::empty_with_meta(AggregateMeta::create_agg_payload( + payload, + p.partition, + p.max_partition_count, + )), + )); + + continue; + } + + let data_block = payload.aggregate_flush_all()?; + let data_block = serialize_block(p.partition, data_block, &self.options)?; + blocks.push((idx, data_block)); + } + + Ok(blocks) + } + } + } + + fn multiway_pick(&self, data_blocks: &[Option]) -> Result { + let position = data_blocks.iter().position_min_by(|left, right| { + let Some(left_block) = left else { + return Ordering::Greater; + }; + let Some(left_meta) = left_block.get_meta() else { + return Ordering::Greater; + }; + let Some(left_meta) = ExchangeSerializeMeta::downcast_ref_from(left_meta) else { + return Ordering::Greater; + }; + + let Some(right_block) = right else { + return Ordering::Less; + }; + let Some(right_meta) = right_block.get_meta() else { + return Ordering::Less; + }; + let Some(right_meta) = ExchangeSerializeMeta::downcast_ref_from(right_meta) else { + return Ordering::Less; + }; + + left_meta.block_number.cmp(&right_meta.block_number) + }); + + position.ok_or_else(|| ErrorCode::Internal("Cannot multiway pick with all none")) } } 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 2ae3cc620b928..e5dd55ad127d8 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 @@ -31,6 +31,8 @@ use databend_common_expression::InputColumns; use databend_common_expression::PartitionedPayload; use databend_common_expression::Payload; use databend_common_expression::ProbeState; +use serde::Deserializer; +use serde::Serializer; pub struct SerializedPayload { pub bucket: isize, @@ -39,6 +41,20 @@ pub struct SerializedPayload { pub max_partition_count: usize, } +impl serde::Serialize for SerializedPayload { + fn serialize(&self, serializer: S) -> std::result::Result + where S: Serializer { + todo!() + } +} + +impl<'de> serde::Deserialize<'de> for SerializedPayload { + fn deserialize(deserializer: D) -> std::result::Result + where D: Deserializer<'de> { + todo!() + } +} + impl SerializedPayload { pub fn get_group_by_column(&self) -> &Column { let entry = self.data_block.columns().last().unwrap(); @@ -106,48 +122,58 @@ impl SerializedPayload { } } +#[derive(serde::Serialize, serde::Deserialize)] pub struct BucketSpilledPayload { pub bucket: isize, pub location: String, pub data_range: Range, - pub columns_layout: Vec, + pub destination_node: String, pub max_partition_count: usize, } pub struct AggregatePayload { - pub bucket: isize, + pub partition: isize, pub payload: Payload, // use for new agg_hashtable pub max_partition_count: usize, } +impl serde::Serialize for AggregatePayload { + fn serialize(&self, serializer: S) -> std::result::Result + where S: Serializer { + todo!() + } +} + +impl<'de> serde::Deserialize<'de> for AggregatePayload { + fn deserialize(deserializer: D) -> std::result::Result + where D: Deserializer<'de> { + todo!() + } +} + +#[derive(serde::Serialize, serde::Deserialize)] pub enum AggregateMeta { Serialized(SerializedPayload), AggregatePayload(AggregatePayload), - AggregateSpilling(PartitionedPayload), BucketSpilled(BucketSpilledPayload), - Spilled(Vec), Partitioned { bucket: isize, data: Vec }, } impl AggregateMeta { pub fn create_agg_payload( - bucket: isize, payload: Payload, + partition: isize, max_partition_count: usize, ) -> BlockMetaInfoPtr { Box::new(AggregateMeta::AggregatePayload(AggregatePayload { - bucket, payload, + partition, max_partition_count, })) } - pub fn create_agg_spilling(payload: PartitionedPayload) -> BlockMetaInfoPtr { - Box::new(AggregateMeta::AggregateSpilling(payload)) - } - pub fn create_serialized( bucket: isize, block: DataBlock, @@ -160,10 +186,6 @@ impl AggregateMeta { })) } - pub fn create_spilled(buckets_payload: Vec) -> BlockMetaInfoPtr { - Box::new(AggregateMeta::Spilled(buckets_payload)) - } - pub fn create_bucket_spilled(payload: BucketSpilledPayload) -> BlockMetaInfoPtr { Box::new(AggregateMeta::BucketSpilled(payload)) } @@ -173,19 +195,23 @@ impl 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> 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 serde::Serialize for AggregateMeta { +// fn serialize(&self, s: S) -> std::result::Result +// where +// S: serde::Serializer, +// { +// unreachable!("AggregateMeta does not support exchanging between multiple nodes") +// } +// } +// +// 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 { fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { @@ -196,24 +222,13 @@ impl Debug for AggregateMeta { AggregateMeta::Serialized { .. } => { f.debug_struct("AggregateMeta::Serialized").finish() } - AggregateMeta::Spilled(_) => f.debug_struct("Aggregate::Spilled").finish(), AggregateMeta::BucketSpilled(_) => f.debug_struct("Aggregate::BucketSpilled").finish(), AggregateMeta::AggregatePayload(_) => { f.debug_struct("AggregateMeta:AggregatePayload").finish() } - AggregateMeta::AggregateSpilling(_) => { - f.debug_struct("AggregateMeta:AggregateSpilling").finish() - } } } } -impl BlockMetaInfo for AggregateMeta { - fn typetag_deserialize(&self) { - unimplemented!("AggregateMeta does not support exchanging between multiple nodes") - } - - fn typetag_name(&self) -> &'static str { - unimplemented!("AggregateMeta does not support exchanging between multiple nodes") - } -} +#[typetag::serde(name = "AggregateMeta")] +impl BlockMetaInfo for AggregateMeta {} 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 bdd17a88364fc..574e571650178 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/mod.rs @@ -24,6 +24,7 @@ mod transform_single_key; mod udaf_script; pub use aggregate_exchange_injector::AggregateInjector; +pub use aggregate_exchange_injector::FlightExchange; pub use aggregate_meta::*; pub use aggregator_params::AggregatorParams; pub use new_transform_partition_bucket::build_partition_bucket; 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 6f82e38ef8d1e..39b46527a2200 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 @@ -211,13 +211,13 @@ impl NewTransformPartitionBucket { #[allow(unused_assignments)] fn add_bucket(&mut self, mut data_block: DataBlock) -> Result<(isize, usize)> { + eprintln!("recv bucket {:?} {:?}", data_block, serde_json::to_string(&data_block.get_meta())); let (mut bucket, mut partition_count) = (0, 0); let mut is_empty_block = false; if let Some(block_meta) = data_block.get_meta() { if let Some(block_meta) = AggregateMeta::downcast_ref_from(block_meta) { (bucket, partition_count) = match block_meta { AggregateMeta::Partitioned { .. } => unreachable!(), - AggregateMeta::AggregateSpilling(_) => unreachable!(), AggregateMeta::BucketSpilled(_) => { let meta = data_block.take_meta().unwrap(); @@ -245,39 +245,6 @@ impl NewTransformPartitionBucket { } unreachable!() } - AggregateMeta::Spilled(_) => { - let meta = data_block.take_meta().unwrap(); - - if let Some(AggregateMeta::Spilled(buckets_payload)) = - AggregateMeta::downcast_from(meta) - { - let partition_count = if !buckets_payload.is_empty() { - buckets_payload[0].max_partition_count - } else { - MAX_PARTITION_COUNT - }; - self.max_partition_count = - self.max_partition_count.max(partition_count); - - for bucket_payload in buckets_payload { - let bucket = bucket_payload.bucket; - 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]); - } - Entry::Occupied(mut v) => { - v.get_mut().push(data_block); - } - }; - } - - return Ok((SINGLE_LEVEL_BUCKET_NUM, partition_count)); - } - unreachable!() - } AggregateMeta::Serialized(payload) => { is_empty_block = payload.data_block.is_empty(); self.max_partition_count = @@ -290,7 +257,7 @@ impl NewTransformPartitionBucket { self.max_partition_count = self.max_partition_count.max(payload.max_partition_count); - (payload.bucket, payload.max_partition_count) + (payload.partition, payload.max_partition_count) } }; } else { @@ -309,9 +276,9 @@ impl NewTransformPartitionBucket { if self.all_inputs_init { if partition_count != self.max_partition_count { return Err(ErrorCode::Internal( - "Internal, the partition count does not equal the max partition count on TransformPartitionBucket. + "Internal, the partition count does not equal the max partition count on TransformPartitionBucket. ", - )); + )); } match self.buckets_blocks.entry(bucket) { Entry::Vacant(v) => { @@ -382,8 +349,8 @@ impl NewTransformPartitionBucket { for (bucket, payload) in partitioned_payload.payloads.into_iter().enumerate() { blocks.push(Some(DataBlock::empty_with_meta( AggregateMeta::create_agg_payload( - bucket as isize, payload, + bucket as isize, self.max_partition_count, ), ))); @@ -395,7 +362,7 @@ impl NewTransformPartitionBucket { fn partition_payload(&mut self, payload: AggregatePayload) -> Result>> { // already is max partition if payload.max_partition_count == self.max_partition_count { - let bucket = payload.bucket; + let bucket = payload.partition; let data_block = DataBlock::empty_with_meta(Box::new(AggregateMeta::AggregatePayload(payload))); match self.buckets_blocks.entry(bucket) { @@ -423,8 +390,8 @@ impl NewTransformPartitionBucket { for (bucket, payload) in partitioned_payload.payloads.into_iter().enumerate() { blocks.push(Some(DataBlock::empty_with_meta( AggregateMeta::create_agg_payload( - bucket as isize, payload, + bucket as isize, self.max_partition_count, ), ))); @@ -550,9 +517,7 @@ impl Processor for NewTransformPartitionBucket { if let Some(agg_block_meta) = block_meta { let data_blocks = match agg_block_meta { - AggregateMeta::Spilled(_) => unreachable!(), AggregateMeta::Partitioned { .. } => unreachable!(), - AggregateMeta::AggregateSpilling(_) => unreachable!(), AggregateMeta::BucketSpilled(_) => unreachable!(), AggregateMeta::Serialized(payload) => self.partition_block(payload)?, AggregateMeta::AggregatePayload(payload) => self.partition_payload(payload)?, 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 096485fa98fcc..c036e9c28b8c6 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 @@ -124,11 +124,9 @@ impl TransformAggregateSerializer { 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::Serialized(_) => unreachable!(), AggregateMeta::BucketSpilled(_) => unreachable!(), AggregateMeta::Partitioned { .. } => unreachable!(), - AggregateMeta::AggregateSpilling(_) => unreachable!(), AggregateMeta::AggregatePayload(p) => { self.input_data = Some(SerializeAggregateStream::create( &self.params, @@ -231,7 +229,7 @@ impl SerializeAggregateStream { self.nums += 1; Ok(Some(block.add_meta(Some( AggregateSerdeMeta::create_agg_payload( - p.bucket, + p.partition, p.max_partition_count, false, ), @@ -244,7 +242,7 @@ impl SerializeAggregateStream { let block = p.payload.empty_block(Some(1)); Ok(Some(block.add_meta(Some( AggregateSerdeMeta::create_agg_payload( - p.bucket, + p.partition, p.max_partition_count, true, ), 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 d37051de63f96..168b180818958 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 @@ -42,223 +42,223 @@ use crate::spillers::Spiller; use crate::spillers::SpillerConfig; use crate::spillers::SpillerType; -pub struct TransformAggregateSpillWriter { - ctx: Arc, - input: Arc, - output: Arc, - _params: Arc, - - spiller: Arc, - spilled_block: Option, - spilling_meta: Option, - spilling_future: Option>>, -} - -impl TransformAggregateSpillWriter { - pub fn try_create( - ctx: Arc, - input: Arc, - output: Arc, - operator: Operator, - params: Arc, - location_prefix: String, - ) -> Result> { - let config = SpillerConfig { - spiller_type: SpillerType::Aggregation, - location_prefix, - disk_spill: None, - use_parquet: ctx.get_settings().get_spilling_file_format()?.is_parquet(), - }; - - let spiller = Spiller::create(ctx.clone(), operator, config.clone())?; - Ok(Box::new(TransformAggregateSpillWriter { - ctx, - input, - output, - _params: params, - spiller: Arc::new(spiller), - spilled_block: None, - spilling_meta: None, - spilling_future: None, - })) - } -} - -#[async_trait::async_trait] -impl Processor for TransformAggregateSpillWriter { - fn name(&self) -> String { - String::from("TransformAggregateSpillWriter") - } - - 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); - } - - while 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::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::AggregateSpilling(payload) => { - self.spilling_future = Some(agg_spilling_aggregate_payload( - self.ctx.clone(), - self.spiller.clone(), - payload, - )?); - - return Ok(()); - } - _ => { - return Err(ErrorCode::Internal("")); - } - } - } - - 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_aggregate_payload( - ctx: Arc, - spiller: Arc, - partitioned_payload: PartitionedPayload, -) -> Result>> { - 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); - // Record how many rows are spilled. - let mut rows = 0; - let location = spiller.create_unique_location(); - for (bucket, payload) in partitioned_payload.payloads.into_iter().enumerate() { - if payload.len() == 0 { - continue; - } - - let data_block = payload.aggregate_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.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: partition_count, - }); - } - - Ok(Box::pin(async move { - let instant = Instant::now(); - if !write_data.is_empty() { - let (location, write_bytes) = spiller - .spill_stream_aggregate_buffer(Some(location), write_data) - .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() - ); - } - - Ok(DataBlock::empty_with_meta(AggregateMeta::create_spilled( - spilled_buckets_payloads, - ))) - })) -} +// pub struct TransformAggregateSpillWriter { +// ctx: Arc, +// input: Arc, +// output: Arc, +// _params: Arc, +// +// spiller: Arc, +// spilled_block: Option, +// spilling_meta: Option, +// spilling_future: Option>>, +// } +// +// impl TransformAggregateSpillWriter { +// pub fn try_create( +// ctx: Arc, +// input: Arc, +// output: Arc, +// operator: Operator, +// params: Arc, +// location_prefix: String, +// ) -> Result> { +// let config = SpillerConfig { +// spiller_type: SpillerType::Aggregation, +// location_prefix, +// disk_spill: None, +// use_parquet: ctx.get_settings().get_spilling_file_format()?.is_parquet(), +// }; +// +// let spiller = Spiller::create(ctx.clone(), operator, config.clone())?; +// Ok(Box::new(TransformAggregateSpillWriter { +// ctx, +// input, +// output, +// _params: params, +// spiller: Arc::new(spiller), +// spilled_block: None, +// spilling_meta: None, +// spilling_future: None, +// })) +// } +// } +// +// #[async_trait::async_trait] +// impl Processor for TransformAggregateSpillWriter { +// fn name(&self) -> String { +// String::from("TransformAggregateSpillWriter") +// } +// +// 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); +// } +// +// while 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::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::AggregateSpilling(payload) => { +// self.spilling_future = Some(agg_spilling_aggregate_payload( +// self.ctx.clone(), +// self.spiller.clone(), +// payload, +// )?); +// +// return Ok(()); +// } +// _ => { +// return Err(ErrorCode::Internal("")); +// } +// } +// } +// +// 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_aggregate_payload( +// ctx: Arc, +// spiller: Arc, +// partitioned_payload: PartitionedPayload, +// ) -> Result>> { +// 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); +// // Record how many rows are spilled. +// let mut rows = 0; +// let location = spiller.create_unique_location(); +// for (bucket, payload) in partitioned_payload.payloads.into_iter().enumerate() { +// if payload.len() == 0 { +// continue; +// } +// +// let data_block = payload.aggregate_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.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, +// max_partition_count: partition_count, +// destination_node: "".to_string(), +// }); +// } +// +// Ok(Box::pin(async move { +// let instant = Instant::now(); +// if !write_data.is_empty() { +// let (location, write_bytes) = spiller +// .spill_stream_aggregate_buffer(Some(location), write_data) +// .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() +// ); +// } +// +// 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 ed82428bbc7d2..bfb6645b43e58 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 @@ -69,100 +69,109 @@ impl TransformDeserializer { fn recv_data(&self, dict: Vec, fragment_data: FragmentData) -> Result { const ROW_HEADER_SIZE: usize = std::mem::size_of::(); - let meta = bincode_deserialize_from_slice(&fragment_data.get_meta()[ROW_HEADER_SIZE..]) - .map_err(|_| ErrorCode::BadBytes("block meta deserialize error when exchange"))?; + let meta_json = unsafe { + String::from_utf8_unchecked(fragment_data.get_meta()[ROW_HEADER_SIZE..].to_vec()) + }; + let meta = + serde_json::from_slice(&fragment_data.get_meta()[ROW_HEADER_SIZE..]).map_err(|_| { + ErrorCode::BadBytes(format!( + "block meta deserialize error when exchange {}", + unsafe { + String::from_utf8_unchecked( + fragment_data.get_meta()[ROW_HEADER_SIZE..].to_vec(), + ) + } + )) + })?; let mut row_count_meta = &fragment_data.get_meta()[..ROW_HEADER_SIZE]; let row_count: u32 = row_count_meta.read_scalar()?; if row_count == 0 { + eprintln!("recv block: null, meta: {}", meta_json); return Ok(DataBlock::new_with_meta(vec![], 0, meta)); } - let data_block = match &meta { - None => { - deserialize_block(dict, fragment_data, &self.schema, self.arrow_schema.clone())? - } - Some(meta) => match AggregateSerdeMeta::downcast_ref_from(meta) { - None => { - deserialize_block(dict, fragment_data, &self.schema, self.arrow_schema.clone())? - } - Some(meta) => { - return match meta.typ == BUCKET_TYPE { - true => { - let mut block = deserialize_block( - dict, - fragment_data, - &self.schema, - self.arrow_schema.clone(), - )?; - - if meta.is_empty { - block = block.slice(0..0); - } - - Ok(DataBlock::empty_with_meta( - AggregateMeta::create_serialized( - meta.bucket, - block, - meta.max_partition_count, - ), - )) - } - false => { - let data_schema = Arc::new(exchange_defines::spilled_schema()); - let arrow_schema = Arc::new(exchange_defines::spilled_arrow_schema()); - let data_block = deserialize_block( - dict, - fragment_data, - &data_schema, - arrow_schema.clone(), - )?; - - let columns = data_block - .columns() - .iter() - .map(|c| c.value.clone().into_column()) - .try_collect::>() - .unwrap(); - - let buckets = - NumberType::::try_downcast_column(&columns[0]).unwrap(); - let data_range_start = - NumberType::::try_downcast_column(&columns[1]).unwrap(); - let data_range_end = - NumberType::::try_downcast_column(&columns[2]).unwrap(); - let columns_layout = - ArrayType::::try_downcast_column(&columns[3]).unwrap(); - - let columns_layout_data = columns_layout.values.as_slice(); - - let mut buckets_payload = Vec::with_capacity(data_block.num_rows()); - for index in 0..data_block.num_rows() { - unsafe { - buckets_payload.push(BucketSpilledPayload { - bucket: *buckets.get_unchecked(index) as isize, - location: meta.location.clone().unwrap(), - data_range: *data_range_start.get_unchecked(index) - ..*data_range_end.get_unchecked(index), - columns_layout: columns_layout_data[columns_layout.offsets - [index] - as usize - ..columns_layout.offsets[index + 1] as usize] - .to_vec(), - max_partition_count: meta.max_partition_count, - }); - } - } - - Ok(DataBlock::empty_with_meta(AggregateMeta::create_spilled( - buckets_payload, - ))) - } - }; - } - }, - }; + let data_block = + deserialize_block(dict, fragment_data, &self.schema, self.arrow_schema.clone())?; + + eprintln!("recv block: {:?}, meta: {}", data_block, meta_json); + // let data_block = match &meta { + // None => { + // deserialize_block(dict, fragment_data, &self.schema, self.arrow_schema.clone())? + // } + // Some(meta) => match AggregateSerdeMeta::downcast_ref_from(meta) { + // None => { + // deserialize_block(dict, fragment_data, &self.schema, self.arrow_schema.clone())? + // } + // Some(meta) => { + // return match meta.typ == BUCKET_TYPE { + // true => { + // let mut block = deserialize_block( + // dict, + // fragment_data, + // &self.schema, + // self.arrow_schema.clone(), + // )?; + // + // if meta.is_empty { + // block = block.slice(0..0); + // } + // + // Ok(DataBlock::empty_with_meta( + // AggregateMeta::create_serialized( + // meta.bucket, + // block, + // meta.max_partition_count, + // ), + // )) + // } + // false => { + // let data_schema = Arc::new(exchange_defines::spilled_schema()); + // let arrow_schema = Arc::new(exchange_defines::spilled_arrow_schema()); + // let data_block = deserialize_block( + // dict, + // fragment_data, + // &data_schema, + // arrow_schema.clone(), + // )?; + // + // let columns = data_block + // .columns() + // .iter() + // .map(|c| c.value.clone().into_column()) + // .try_collect::>() + // .unwrap(); + // + // let buckets = + // NumberType::::try_downcast_column(&columns[0]).unwrap(); + // let data_range_start = + // NumberType::::try_downcast_column(&columns[1]).unwrap(); + // let data_range_end = + // NumberType::::try_downcast_column(&columns[2]).unwrap(); + // + // let mut buckets_payload = Vec::with_capacity(data_block.num_rows()); + // for index in 0..data_block.num_rows() { + // unsafe { + // buckets_payload.push(BucketSpilledPayload { + // bucket: *buckets.get_unchecked(index) as isize, + // location: meta.location.clone().unwrap(), + // data_range: *data_range_start.get_unchecked(index) + // ..*data_range_end.get_unchecked(index), + // destination_node: String::new(), + // max_partition_count: meta.max_partition_count, + // }); + // } + // } + // + // Ok(DataBlock::empty_with_meta(AggregateMeta::create_spilled( + // buckets_payload, + // ))) + // } + // }; + // } + // }, + // }; match data_block.num_columns() == 0 { true => Ok(DataBlock::new_with_meta(vec![], row_count as usize, meta)), 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 a034b86038acb..d455b38af9c22 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 @@ -1,288 +1,288 @@ -// Copyright 2021 Datafuse Labs +// // 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. // -// 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 +// use std::sync::Arc; +// use std::time::Instant; // -// http://www.apache.org/licenses/LICENSE-2.0 +// use arrow_ipc::writer::IpcWriteOptions; +// use arrow_ipc::CompressionType; +// 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::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::BlockMetaInfoDowncast; +// use databend_common_expression::DataBlock; +// use databend_common_expression::DataSchemaRef; +// use databend_common_expression::FromData; +// use databend_common_expression::PartitionedPayload; +// 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_settings::FlightCompression; +// use futures_util::future::BoxFuture; +// use log::info; +// use opendal::Operator; // -// 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 arrow_ipc::writer::IpcWriteOptions; -use arrow_ipc::CompressionType; -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::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::BlockMetaInfoDowncast; -use databend_common_expression::DataBlock; -use databend_common_expression::DataSchemaRef; -use databend_common_expression::FromData; -use databend_common_expression::PartitionedPayload; -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_settings::FlightCompression; -use futures_util::future::BoxFuture; -use log::info; -use opendal::Operator; - -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::exchange_defines; -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::servers::flight::v1::exchange::serde::serialize_block; -use crate::servers::flight::v1::exchange::ExchangeShuffleMeta; -use crate::sessions::QueryContext; -use crate::spillers::Spiller; -use crate::spillers::SpillerConfig; -use crate::spillers::SpillerType; - -pub struct TransformExchangeAggregateSerializer { - ctx: Arc, - local_pos: usize, - options: IpcWriteOptions, - - params: Arc, - spiller: Arc, -} - -impl TransformExchangeAggregateSerializer { - #[allow(clippy::too_many_arguments)] - pub fn try_create( - ctx: Arc, - input: Arc, - output: Arc, - - operator: Operator, - location_prefix: String, - params: Arc, - compression: Option, - _schema: DataSchemaRef, - local_pos: usize, - ) -> Result> { - let compression = match compression { - None => None, - Some(compression) => match compression { - FlightCompression::Lz4 => Some(CompressionType::LZ4_FRAME), - FlightCompression::Zstd => Some(CompressionType::ZSTD), - }, - }; - let config = SpillerConfig { - spiller_type: SpillerType::Aggregation, - location_prefix, - disk_spill: None, - use_parquet: ctx.get_settings().get_spilling_file_format()?.is_parquet(), - }; - - let spiller = Spiller::create(ctx.clone(), operator, config.clone())?; - Ok(BlockMetaTransformer::create( - input, - output, - TransformExchangeAggregateSerializer { - ctx, - params, - local_pos, - spiller: spiller.into(), - options: IpcWriteOptions::default() - .try_with_compression(compression) - .unwrap(), - }, - )) - } -} - -impl BlockMetaTransform for TransformExchangeAggregateSerializer { - const NAME: &'static str = "TransformExchangeAggregateSerializer"; - - 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::Serialized(_)) => unreachable!(), - Some(AggregateMeta::BucketSpilled(_)) => unreachable!(), - Some(AggregateMeta::Partitioned { .. }) => unreachable!(), - Some(AggregateMeta::AggregateSpilling(payload)) => { - serialized_blocks.push(FlightSerialized::Future( - match index == self.local_pos { - true => local_agg_spilling_aggregate_payload( - self.ctx.clone(), - self.spiller.clone(), - payload, - )?, - false => exchange_agg_spilling_aggregate_payload( - self.ctx.clone(), - self.spiller.clone(), - payload, - )?, - }, - )); - } - - Some(AggregateMeta::AggregatePayload(p)) => { - let (bucket, max_partition_count) = (p.bucket, p.max_partition_count); - - if index == self.local_pos { - serialized_blocks.push(FlightSerialized::DataBlock( - block.add_meta(Some(Box::new(AggregateMeta::AggregatePayload(p))))?, - )); - continue; - } - - let block_number = compute_block_number(bucket, max_partition_count)?; - let stream = SerializeAggregateStream::create( - &self.params, - SerializePayload::AggregatePayload(p), - ); - let mut stream_blocks = stream.into_iter().collect::>>()?; - debug_assert!(!stream_blocks.is_empty()); - let mut c = DataBlock::concat(&stream_blocks)?; - if let Some(meta) = stream_blocks[0].take_meta() { - c.replace_meta(meta); - } - let c = serialize_block(block_number, c, &self.options)?; - serialized_blocks.push(FlightSerialized::DataBlock(c)); - } - }; - } - - Ok(vec![DataBlock::empty_with_meta( - FlightSerializedMeta::create(serialized_blocks), - )]) - } -} - -fn exchange_agg_spilling_aggregate_payload( - ctx: Arc, - spiller: Arc, - partitioned_payload: PartitionedPayload, -) -> Result>> { - let partition_count = partitioned_payload.partition_count(); - let mut write_size = 0; - 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.aggregate_flush_all()?; - 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 (location, write_bytes) = spiller - .spill_stream_aggregate_buffer(None, write_data) - .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_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()) - })) -} +// 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::exchange_defines; +// 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::servers::flight::v1::exchange::serde::serialize_block; +// use crate::servers::flight::v1::exchange::ExchangeShuffleMeta; +// use crate::sessions::QueryContext; +// use crate::spillers::Spiller; +// use crate::spillers::SpillerConfig; +// use crate::spillers::SpillerType; +// +// pub struct TransformExchangeAggregateSerializer { +// ctx: Arc, +// local_pos: usize, +// options: IpcWriteOptions, +// +// params: Arc, +// spiller: Arc, +// } +// +// impl TransformExchangeAggregateSerializer { +// #[allow(clippy::too_many_arguments)] +// pub fn try_create( +// ctx: Arc, +// input: Arc, +// output: Arc, +// +// operator: Operator, +// location_prefix: String, +// params: Arc, +// compression: Option, +// _schema: DataSchemaRef, +// local_pos: usize, +// ) -> Result> { +// let compression = match compression { +// None => None, +// Some(compression) => match compression { +// FlightCompression::Lz4 => Some(CompressionType::LZ4_FRAME), +// FlightCompression::Zstd => Some(CompressionType::ZSTD), +// }, +// }; +// let config = SpillerConfig { +// spiller_type: SpillerType::Aggregation, +// location_prefix, +// disk_spill: None, +// use_parquet: ctx.get_settings().get_spilling_file_format()?.is_parquet(), +// }; +// +// let spiller = Spiller::create(ctx.clone(), operator, config.clone())?; +// Ok(BlockMetaTransformer::create( +// input, +// output, +// TransformExchangeAggregateSerializer { +// ctx, +// params, +// local_pos, +// spiller: spiller.into(), +// options: IpcWriteOptions::default() +// .try_with_compression(compression) +// .unwrap(), +// }, +// )) +// } +// } +// +// impl BlockMetaTransform for TransformExchangeAggregateSerializer { +// const NAME: &'static str = "TransformExchangeAggregateSerializer"; +// +// 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::Serialized(_)) => unreachable!(), +// Some(AggregateMeta::BucketSpilled(_)) => unreachable!(), +// Some(AggregateMeta::Partitioned { .. }) => unreachable!(), +// Some(AggregateMeta::AggregateSpilling(payload)) => { +// serialized_blocks.push(FlightSerialized::Future( +// match index == self.local_pos { +// true => local_agg_spilling_aggregate_payload( +// self.ctx.clone(), +// self.spiller.clone(), +// payload, +// )?, +// false => exchange_agg_spilling_aggregate_payload( +// self.ctx.clone(), +// self.spiller.clone(), +// payload, +// )?, +// }, +// )); +// } +// +// Some(AggregateMeta::AggregatePayload(p)) => { +// let (bucket, max_partition_count) = (p.partition, p.max_partition_count); +// +// if index == self.local_pos { +// serialized_blocks.push(FlightSerialized::DataBlock( +// block.add_meta(Some(Box::new(AggregateMeta::AggregatePayload(p))))?, +// )); +// continue; +// } +// +// let block_number = compute_block_number(bucket, max_partition_count)?; +// let stream = SerializeAggregateStream::create( +// &self.params, +// SerializePayload::AggregatePayload(p), +// ); +// let mut stream_blocks = stream.into_iter().collect::>>()?; +// debug_assert!(!stream_blocks.is_empty()); +// let mut c = DataBlock::concat(&stream_blocks)?; +// if let Some(meta) = stream_blocks[0].take_meta() { +// c.replace_meta(meta); +// } +// let c = serialize_block(block_number, c, &self.options)?; +// serialized_blocks.push(FlightSerialized::DataBlock(c)); +// } +// }; +// } +// +// Ok(vec![DataBlock::empty_with_meta( +// FlightSerializedMeta::create(serialized_blocks), +// )]) +// } +// } +// +// fn exchange_agg_spilling_aggregate_payload( +// ctx: Arc, +// spiller: Arc, +// partitioned_payload: PartitionedPayload, +// ) -> Result>> { +// let partition_count = partitioned_payload.partition_count(); +// let mut write_size = 0; +// 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.aggregate_flush_all()?; +// 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 (location, write_bytes) = spiller +// .spill_stream_aggregate_buffer(None, write_data) +// .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_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()) +// })) +// } 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 dcd220b0dcc5b..fbc3eb485bf4c 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 @@ -137,9 +137,7 @@ impl Processor for TransformSpillReader { fn process(&mut self) -> Result<()> { if let Some((meta, mut read_data)) = self.deserializing_meta.take() { match meta { - AggregateMeta::Spilled(_) => unreachable!(), AggregateMeta::AggregatePayload(_) => unreachable!(), - AggregateMeta::AggregateSpilling(_) => unreachable!(), AggregateMeta::Serialized(_) => unreachable!(), AggregateMeta::BucketSpilled(payload) => { debug_assert!(read_data.len() == 1); @@ -176,9 +174,7 @@ impl Processor for TransformSpillReader { async fn async_process(&mut self) -> Result<()> { if let Some(block_meta) = self.reading_meta.take() { match &block_meta { - AggregateMeta::Spilled(_) => unreachable!(), AggregateMeta::AggregatePayload(_) => unreachable!(), - AggregateMeta::AggregateSpilling(_) => unreachable!(), AggregateMeta::Serialized(_) => unreachable!(), AggregateMeta::BucketSpilled(payload) => { let _guard = self.semaphore.acquire().await; 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 048d7e6ed5a1c..e619947669ffd 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 @@ -83,11 +83,11 @@ impl TransformFinalAggregate { }, AggregateMeta::AggregatePayload(payload) => match agg_hashtable.as_mut() { Some(ht) => { - debug_assert!(bucket == payload.bucket); + debug_assert!(bucket == payload.partition); ht.combine_payload(&payload.payload, &mut self.flush_state)?; } None => { - debug_assert!(bucket == payload.bucket); + debug_assert!(bucket == payload.partition); let capacity = AggregateHashTable::get_capacity_for_count(payload.payload.len()); let mut hashtable = AggregateHashTable::new_with_capacity( 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 e90afa5f922a9..1a2a9adaf5769 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 @@ -23,6 +23,7 @@ use databend_common_base::base::convert_byte_size; use databend_common_base::base::convert_number_size; use databend_common_catalog::plan::AggIndexMeta; use databend_common_catalog::table_context::TableContext; +use databend_common_config::GlobalConfig; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::arrow::write_column; @@ -133,7 +134,7 @@ impl TransformPartialAggregate { first_block_start: None, processed_bytes: 0, processed_rows: 0, - configure_peer_nodes: vec![], + configure_peer_nodes: vec![GlobalConfig::instance().query.node_id.clone()], spilling_state: None, spiller: Arc::new(spiller), spill_blocks: vec![], @@ -282,12 +283,12 @@ impl AccumulatingTransform for TransformPartialAggregate { convert_byte_size(self.processed_bytes as f64), ); - for (bucket, payload) in hashtable.payload.payloads.into_iter().enumerate() { + for (partition, 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 as isize, partition_count, ), )); @@ -335,63 +336,64 @@ impl AccumulatingTransform for TransformPartialAggregate { } async fn flush_spill_payload(&mut self) -> Result { - let hashtable_spilling_state = self.spilling_state.as_mut().unwrap(); + let spilling_state = self.spilling_state.as_mut().unwrap(); - let max_bucket = hashtable_spilling_state.max_bucket; - let max_partition = 1 << hashtable_spilling_state.ht.config.max_radix_bits; + let max_bucket = spilling_state.max_bucket; + let max_partition = 1 << spilling_state.ht.config.max_radix_bits; - if !hashtable_spilling_state.data_payload.is_empty() { - if hashtable_spilling_state.writer.is_none() { + if !spilling_state.data_payload.is_empty() { + if spilling_state.writer.is_none() { let location = self.spiller.create_unique_location(); - hashtable_spilling_state.writer = - Some(self.spiller.create_aggregate_writer(location).await?); + spilling_state.writer = Some(self.spiller.create_aggregate_writer(location).await?); } - let writer = hashtable_spilling_state.writer.as_mut().unwrap(); + let writer = spilling_state.writer.as_mut().unwrap(); let mut flush_data = Vec::with_capacity(4 * 1024 * 1024); - std::mem::swap(&mut flush_data, &mut hashtable_spilling_state.data_payload); + std::mem::swap(&mut flush_data, &mut spilling_state.data_payload); writer.write(flush_data).await?; } - if hashtable_spilling_state.last_prepare_payload { - if let Some(writer) = hashtable_spilling_state.writer.as_mut() { - let last_offset = hashtable_spilling_state.last_flush_partition_offset; + if spilling_state.last_prepare_payload { + if let Some(writer) = spilling_state.writer.as_mut() { + let last_offset = spilling_state.last_flush_partition_offset; if writer.write_bytes() > last_offset { self.spill_blocks.push(DataBlock::empty_with_meta( AggregateMeta::create_bucket_spilled(BucketSpilledPayload { - bucket: hashtable_spilling_state.working_partition as isize, + bucket: spilling_state.working_partition as isize, location: writer.location(), data_range: last_offset as u64..writer.write_bytes() as u64, - columns_layout: vec![], + destination_node: self.configure_peer_nodes + [spilling_state.working_bucket] + .clone(), max_partition_count: max_partition, }), )); - hashtable_spilling_state.last_flush_partition_offset = writer.write_bytes(); + spilling_state.last_flush_partition_offset = writer.write_bytes(); } } - hashtable_spilling_state.payload_idx = 0; - hashtable_spilling_state.working_partition += 1; - if hashtable_spilling_state.working_partition < max_partition { + spilling_state.payload_idx = 0; + spilling_state.working_partition += 1; + if spilling_state.working_partition < max_partition { return Ok(true); } - if let Some(writer) = hashtable_spilling_state.writer.as_mut() { + if let Some(writer) = spilling_state.writer.as_mut() { writer.complete().await?; - hashtable_spilling_state.writer = None; + spilling_state.writer = None; } - hashtable_spilling_state.payload_idx = 0; - hashtable_spilling_state.working_partition = 0; - hashtable_spilling_state.working_bucket += 1; + spilling_state.payload_idx = 0; + spilling_state.working_bucket += 1; + spilling_state.working_partition = 0; - if hashtable_spilling_state.working_bucket < max_bucket { + if spilling_state.working_bucket < max_bucket { return Ok(true); } - hashtable_spilling_state.finished = true; + spilling_state.finished = true; self.reset_hashtable(); return Ok(false); @@ -584,11 +586,12 @@ impl HashtableSpillingState { let address = &flush_state.addresses; let selector = &flush_state.probe_state.partition_entries[working_partition]; + let working_payload = &self.ht.payload.payloads[idx]; let mut working_partition_payload = Payload::new( - self.ht.payload.payloads[idx].arena.clone(), - self.ht.payload.payloads[idx].group_types.clone(), - self.ht.payload.payloads[idx].aggrs.clone(), - self.ht.payload.payloads[idx].states_layout.clone(), + working_payload.arena.clone(), + working_payload.group_types.clone(), + working_payload.aggrs.clone(), + working_payload.states_layout.clone(), ); working_partition_payload.copy_rows(selector, rows, address); diff --git a/src/query/service/src/pipelines/processors/transforms/window/partition/window_partition_exchange.rs b/src/query/service/src/pipelines/processors/transforms/window/partition/window_partition_exchange.rs index bf6ea988acf65..3aa40cb5e6504 100644 --- a/src/query/service/src/pipelines/processors/transforms/window/partition/window_partition_exchange.rs +++ b/src/query/service/src/pipelines/processors/transforms/window/partition/window_partition_exchange.rs @@ -38,7 +38,7 @@ impl WindowPartitionExchange { impl Exchange for WindowPartitionExchange { const NAME: &'static str = "Window"; - fn partition(&self, data_block: DataBlock, n: usize) -> Result> { + fn partition(&self, data_block: DataBlock, n: usize) -> Result> { let num_rows = data_block.num_rows(); // Extract the columns used for hash computation. @@ -67,6 +67,7 @@ impl Exchange for WindowPartitionExchange { .into_iter() .map(WindowPartitionMeta::create) .map(DataBlock::empty_with_meta) + .enumerate() .collect()) } } diff --git a/src/query/service/src/pipelines/processors/transforms/window/partition/window_partition_partial_top_n_exchange.rs b/src/query/service/src/pipelines/processors/transforms/window/partition/window_partition_partial_top_n_exchange.rs index 283b2c78852c9..c5e5465cba942 100644 --- a/src/query/service/src/pipelines/processors/transforms/window/partition/window_partition_partial_top_n_exchange.rs +++ b/src/query/service/src/pipelines/processors/transforms/window/partition/window_partition_partial_top_n_exchange.rs @@ -69,7 +69,7 @@ impl Exchange for WindowPartitionTopNExchange { const NAME: &'static str = "WindowTopN"; const SKIP_EMPTY_DATA_BLOCK: bool = true; - fn partition(&self, block: DataBlock, n: usize) -> Result> { + fn partition(&self, block: DataBlock, n: usize) -> Result> { let partition_permutation = self.partition_permutation(&block); // Partition the data blocks to different processors. @@ -83,6 +83,7 @@ impl Exchange for WindowPartitionTopNExchange { .into_iter() .map(WindowPartitionMeta::create) .map(DataBlock::empty_with_meta) + .enumerate() .collect()) } } diff --git a/src/query/service/src/servers/flight/v1/exchange/exchange_injector.rs b/src/query/service/src/servers/flight/v1/exchange/exchange_injector.rs index 4aa65ba175a83..c6e7eacf23858 100644 --- a/src/query/service/src/servers/flight/v1/exchange/exchange_injector.rs +++ b/src/query/service/src/servers/flight/v1/exchange/exchange_injector.rs @@ -40,31 +40,31 @@ pub trait ExchangeInjector: Send + Sync + 'static { fn exchange_sorting(&self) -> Option>; - fn apply_merge_serializer( - &self, - params: &MergeExchangeParams, - compression: Option, - pipeline: &mut Pipeline, - ) -> Result<()>; - - fn apply_shuffle_serializer( - &self, - params: &ShuffleExchangeParams, - compression: Option, - pipeline: &mut Pipeline, - ) -> Result<()>; - - fn apply_merge_deserializer( - &self, - params: &MergeExchangeParams, - pipeline: &mut Pipeline, - ) -> Result<()>; - - fn apply_shuffle_deserializer( - &self, - params: &ShuffleExchangeParams, - pipeline: &mut Pipeline, - ) -> Result<()>; + // fn apply_merge_serializer( + // &self, + // params: &MergeExchangeParams, + // compression: Option, + // pipeline: &mut Pipeline, + // ) -> Result<()>; + // + // fn apply_shuffle_serializer( + // &self, + // params: &ShuffleExchangeParams, + // compression: Option, + // pipeline: &mut Pipeline, + // ) -> Result<()>; + // + // fn apply_merge_deserializer( + // &self, + // params: &MergeExchangeParams, + // pipeline: &mut Pipeline, + // ) -> Result<()>; + // + // fn apply_shuffle_deserializer( + // &self, + // params: &ShuffleExchangeParams, + // pipeline: &mut Pipeline, + // ) -> Result<()>; } pub struct DefaultExchangeInjector; @@ -107,53 +107,53 @@ impl ExchangeInjector for DefaultExchangeInjector { None } - fn apply_merge_serializer( - &self, - params: &MergeExchangeParams, - compression: Option, - pipeline: &mut Pipeline, - ) -> Result<()> { - pipeline.add_transform(|input, output| { - TransformExchangeSerializer::create(input, output, params, compression) - }) - } - - fn apply_shuffle_serializer( - &self, - params: &ShuffleExchangeParams, - compression: Option, - pipeline: &mut Pipeline, - ) -> Result<()> { - pipeline.add_transform(|input, output| { - TransformScatterExchangeSerializer::create(input, output, compression, params) - }) - } - - fn apply_merge_deserializer( - &self, - params: &MergeExchangeParams, - pipeline: &mut Pipeline, - ) -> Result<()> { - pipeline.add_transform(|input, output| { - Ok(TransformExchangeDeserializer::create( - input, - output, - ¶ms.schema, - )) - }) - } - - fn apply_shuffle_deserializer( - &self, - params: &ShuffleExchangeParams, - pipeline: &mut Pipeline, - ) -> Result<()> { - pipeline.add_transform(|input, output| { - Ok(TransformExchangeDeserializer::create( - input, - output, - ¶ms.schema, - )) - }) - } + // fn apply_merge_serializer( + // &self, + // params: &MergeExchangeParams, + // compression: Option, + // pipeline: &mut Pipeline, + // ) -> Result<()> { + // pipeline.add_transform(|input, output| { + // TransformExchangeSerializer::create(input, output, params, compression) + // }) + // } + // + // fn apply_shuffle_serializer( + // &self, + // params: &ShuffleExchangeParams, + // compression: Option, + // pipeline: &mut Pipeline, + // ) -> Result<()> { + // pipeline.add_transform(|input, output| { + // TransformScatterExchangeSerializer::create(input, output, compression, params) + // }) + // } + // + // fn apply_merge_deserializer( + // &self, + // params: &MergeExchangeParams, + // pipeline: &mut Pipeline, + // ) -> Result<()> { + // pipeline.add_transform(|input, output| { + // Ok(TransformExchangeDeserializer::create( + // input, + // output, + // ¶ms.schema, + // )) + // }) + // } + // + // fn apply_shuffle_deserializer( + // &self, + // params: &ShuffleExchangeParams, + // pipeline: &mut Pipeline, + // ) -> Result<()> { + // pipeline.add_transform(|input, output| { + // Ok(TransformExchangeDeserializer::create( + // input, + // output, + // ¶ms.schema, + // )) + // }) + // } } diff --git a/src/query/service/src/servers/flight/v1/exchange/exchange_sink.rs b/src/query/service/src/servers/flight/v1/exchange/exchange_sink.rs index 73ed08eb1c021..170f56c87aaf3 100644 --- a/src/query/service/src/servers/flight/v1/exchange/exchange_sink.rs +++ b/src/query/service/src/servers/flight/v1/exchange/exchange_sink.rs @@ -14,6 +14,7 @@ use std::sync::Arc; +use databend_common_config::GlobalConfig; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::BlockMetaInfoDowncast; @@ -30,6 +31,8 @@ use super::exchange_sorting::TransformExchangeSorting; use super::exchange_transform_shuffle::exchange_shuffle; use super::serde::ExchangeSerializeMeta; use crate::clusters::ClusterHelper; +use crate::pipelines::processors::transforms::aggregator::FlightExchange; +use crate::servers::flight::v1::scatter::MergeFlightScatter; use crate::sessions::QueryContext; use crate::sessions::TableContext; @@ -60,7 +63,16 @@ impl ExchangeSink { if !params.ignore_exchange { let settings = ctx.get_settings(); let compression = settings.get_query_flight_compression()?; - exchange_injector.apply_merge_serializer(params, compression, pipeline)?; + + let nodes = vec![GlobalConfig::instance().query.node_id.clone()]; + pipeline.exchange( + nodes.len(), + FlightExchange::create( + nodes, + compression, + Arc::new(Box::new(MergeFlightScatter)), + ), + ); } if !params.ignore_exchange && exchange_injector.exchange_sorting().is_some() { diff --git a/src/query/service/src/servers/flight/v1/exchange/exchange_source.rs b/src/query/service/src/servers/flight/v1/exchange/exchange_source.rs index acdfb66de123e..840d221daf36c 100644 --- a/src/query/service/src/servers/flight/v1/exchange/exchange_source.rs +++ b/src/query/service/src/servers/flight/v1/exchange/exchange_source.rs @@ -28,6 +28,7 @@ use super::exchange_params::ExchangeParams; use super::exchange_params::MergeExchangeParams; use super::exchange_source_reader::ExchangeSourceReader; use crate::clusters::ClusterHelper; +use crate::pipelines::processors::transforms::aggregator::TransformAggregateDeserializer; use crate::servers::flight::v1::exchange::ExchangeInjector; use crate::sessions::QueryContext; @@ -93,5 +94,7 @@ pub fn via_exchange_source( pipeline.try_resize(last_output_len)?; } - injector.apply_merge_deserializer(params, pipeline) + pipeline.add_transform(|input, output| { + TransformAggregateDeserializer::try_create(input, output, ¶ms.schema) + }) } diff --git a/src/query/service/src/servers/flight/v1/exchange/exchange_transform.rs b/src/query/service/src/servers/flight/v1/exchange/exchange_transform.rs index 47be1d1f473f8..4aff81888783d 100644 --- a/src/query/service/src/servers/flight/v1/exchange/exchange_transform.rs +++ b/src/query/service/src/servers/flight/v1/exchange/exchange_transform.rs @@ -27,6 +27,7 @@ use super::exchange_source::via_exchange_source; use super::exchange_source_reader::create_reader_item; use super::exchange_transform_shuffle::exchange_shuffle; use crate::clusters::ClusterHelper; +use crate::pipelines::processors::transforms::aggregator::TransformAggregateDeserializer; use crate::servers::flight::v1::exchange::ExchangeInjector; use crate::sessions::QueryContext; @@ -91,7 +92,9 @@ impl ExchangeTransform { pipeline.try_resize(max_threads)?; } - injector.apply_shuffle_deserializer(params, pipeline) + pipeline.add_transform(|input, output| { + TransformAggregateDeserializer::try_create(input, output, ¶ms.schema) + }) } } } diff --git a/src/query/service/src/servers/flight/v1/exchange/exchange_transform_shuffle.rs b/src/query/service/src/servers/flight/v1/exchange/exchange_transform_shuffle.rs index f071b1fa5ac15..4cfd4a89b83e8 100644 --- a/src/query/service/src/servers/flight/v1/exchange/exchange_transform_shuffle.rs +++ b/src/query/service/src/servers/flight/v1/exchange/exchange_transform_shuffle.rs @@ -41,6 +41,7 @@ use super::exchange_sorting::ExchangeSorting; use super::exchange_sorting::TransformExchangeSorting; use super::exchange_transform_scatter::ScatterTransform; use super::serde::ExchangeSerializeMeta; +use crate::pipelines::processors::transforms::aggregator::FlightExchange; use crate::sessions::QueryContext; pub struct ExchangeShuffleMeta { @@ -63,318 +64,318 @@ local_block_meta_serde!(ExchangeShuffleMeta); #[typetag::serde(name = "exchange_shuffle")] impl BlockMetaInfo for ExchangeShuffleMeta {} - -struct OutputsBuffer { - inner: Vec>, -} - -impl OutputsBuffer { - pub fn create(capacity: usize, outputs: usize) -> OutputsBuffer { - OutputsBuffer { - inner: vec![capacity; outputs] - .into_iter() - .map(VecDeque::with_capacity) - .collect::>(), - } - } - - pub fn is_all_empty(&self) -> bool { - self.inner.iter().all(|x| x.is_empty()) - } - - pub fn is_empty(&self, index: usize) -> bool { - self.inner[index].is_empty() - } - - pub fn is_full(&self) -> bool { - self.inner.iter().any(|x| x.len() == x.capacity()) - } - - pub fn clear(&mut self, index: usize) { - self.inner[index].clear(); - } - - pub fn pop(&mut self, index: usize) -> Option { - self.inner[index].pop_front() - } - - pub fn push_back(&mut self, index: usize, block: DataBlock) -> usize { - self.inner[index].push_back(block); - self.inner[index].len() - } -} - -#[derive(PartialEq)] -enum PortStatus { - Idle, - HasData, - NeedData, - Finished, -} - -struct PortWithStatus { - pub status: PortStatus, - pub port: Arc, -} - -struct ExchangeShuffleTransform { - initialized: bool, - - finished_inputs: usize, - finished_outputs: usize, - - waiting_outputs: Vec, - waiting_inputs: VecDeque, - - buffer: OutputsBuffer, - inputs: Vec>, - outputs: Vec>, -} - -impl Processor for ExchangeShuffleTransform { - fn name(&self) -> String { - String::from("ExchangeShuffleTransform") - } - - fn as_any(&mut self) -> &mut dyn Any { - self - } - - fn event_with_cause(&mut self, cause: EventCause) -> Result { - if let EventCause::Output(output_index) = &cause { - let output = &mut self.outputs[*output_index]; - - if output.port.is_finished() { - if output.status != PortStatus::Finished { - self.finished_outputs += 1; - output.status = PortStatus::Finished; - } - - self.buffer.clear(*output_index); - - self.wakeup_inputs(); - self.wakeup_outputs(); - } else if output.port.can_push() { - if !self.buffer.is_empty(*output_index) { - let data_block = self.buffer.pop(*output_index).unwrap(); - output.status = PortStatus::Idle; - output.port.push_data(Ok(data_block)); - - self.wakeup_inputs(); - self.wakeup_outputs(); - } else if output.status != PortStatus::NeedData { - output.status = PortStatus::NeedData; - self.waiting_outputs.push(*output_index); - } - } - } - - if !self.initialized && !self.waiting_outputs.is_empty() { - self.initialized = true; - for input in &self.inputs { - input.port.set_need_data(); - } - } - - if self.finished_outputs == self.outputs.len() { - for input in &self.inputs { - input.port.finish(); - } - - return Ok(Event::Finished); - } - - if let EventCause::Input(input_index) = &cause { - let input = &mut self.inputs[*input_index]; - - if input.port.is_finished() { - if input.status != PortStatus::Finished { - self.finished_inputs += 1; - input.status = PortStatus::Finished; - } - - self.wakeup_outputs(); - self.wakeup_inputs(); - } else if input.port.has_data() { - if !self.buffer.is_full() { - self.take_input_data_into_buffer(*input_index); - - self.wakeup_outputs(); - self.wakeup_inputs(); - } else if input.status != PortStatus::HasData { - input.status = PortStatus::HasData; - self.waiting_inputs.push_back(*input_index); - } - } - } - - if self.finished_outputs == self.outputs.len() { - for input in &self.inputs { - input.port.finish(); - } - - return Ok(Event::Finished); - } - - if self.finished_inputs == self.inputs.len() { - for (index, output) in self.outputs.iter_mut().enumerate() { - if self.buffer.is_empty(index) && output.status != PortStatus::Finished { - self.finished_outputs += 1; - output.status = PortStatus::Finished; - output.port.finish(); - } - } - - if self.buffer.is_all_empty() { - return Ok(Event::Finished); - } - } - - match self.waiting_outputs.is_empty() { - true => Ok(Event::NeedConsume), - false => Ok(Event::NeedData), - } - } - - fn details_status(&self) -> Option { - #[derive(Debug)] - #[allow(dead_code)] - struct Display { - queue_status: Vec<(usize, usize)>, - inputs: usize, - finished_inputs: usize, - outputs: usize, - finished_outputs: usize, - - waiting_outputs: Vec, - waiting_inputs: VecDeque, - } - - let mut queue_status = vec![]; - for (idx, queue) in self.buffer.inner.iter().enumerate() { - queue_status.push((idx, queue.len())); - } - - Some(format!("{:?}", Display { - queue_status, - inputs: self.inputs.len(), - outputs: self.outputs.len(), - finished_inputs: self.finished_inputs, - finished_outputs: self.finished_outputs, - waiting_inputs: self.waiting_inputs.clone(), - waiting_outputs: self.waiting_outputs.clone(), - })) - } -} - -impl ExchangeShuffleTransform { - fn wakeup_inputs(&mut self) { - while !self.waiting_inputs.is_empty() && !self.buffer.is_full() { - let input_index = self.waiting_inputs.pop_front().unwrap(); - - self.take_input_data_into_buffer(input_index); - } - } - - fn wakeup_outputs(&mut self) { - let mut new_waiting_output = Vec::with_capacity(self.waiting_outputs.len()); - - for waiting_output in &self.waiting_outputs { - let output = &mut self.outputs[*waiting_output]; - - if output.port.is_finished() { - if output.status != PortStatus::Finished { - self.finished_outputs += 1; - output.status = PortStatus::Finished; - } - - self.buffer.clear(*waiting_output); - continue; - } - - if self.buffer.is_empty(*waiting_output) { - new_waiting_output.push(*waiting_output); - continue; - } - - let data_block = self.buffer.pop(*waiting_output).unwrap(); - output.status = PortStatus::Idle; - output.port.push_data(Ok(data_block)); - } - - self.waiting_outputs = new_waiting_output; - } - - fn take_input_data_into_buffer(&mut self, input_index: usize) { - let input = &mut self.inputs[input_index]; - - input.status = PortStatus::Idle; - let mut data_block = input.port.pull_data().unwrap().unwrap(); - - if let Some(block_meta) = data_block.take_meta() { - if let Some(shuffle_meta) = ExchangeShuffleMeta::downcast_from(block_meta) { - for (index, block) in shuffle_meta.blocks.into_iter().enumerate() { - if (!block.is_empty() || block.get_meta().is_some()) - && self.outputs[index].status != PortStatus::Finished - { - self.buffer.push_back(index, block); - } - } - } - } - - if input.port.is_finished() { - if input.status != PortStatus::Finished { - self.finished_inputs += 1; - input.status = PortStatus::Finished; - } - - return; - } - - input.port.set_need_data(); - } -} - -impl ExchangeShuffleTransform { - pub fn create(inputs: usize, outputs: usize, buffer: usize) -> ExchangeShuffleTransform { - let mut inputs_port = Vec::with_capacity(inputs); - let mut outputs_port = Vec::with_capacity(outputs); - - for _index in 0..inputs { - inputs_port.push(PortWithStatus { - status: PortStatus::Idle, - port: InputPort::create(), - }); - } - - for _index in 0..outputs { - outputs_port.push(PortWithStatus { - status: PortStatus::Idle, - port: OutputPort::create(), - }); - } - - ExchangeShuffleTransform { - initialized: false, - finished_inputs: 0, - finished_outputs: 0, - inputs: inputs_port, - outputs: outputs_port, - buffer: OutputsBuffer::create(buffer, outputs), - waiting_inputs: VecDeque::with_capacity(inputs), - waiting_outputs: Vec::with_capacity(outputs), - } - } - - pub fn get_inputs(&self) -> Vec> { - self.inputs.iter().map(|x| x.port.clone()).collect() - } - - pub fn get_outputs(&self) -> Vec> { - self.outputs.iter().map(|x| x.port.clone()).collect() - } -} +// +// struct OutputsBuffer { +// inner: Vec>, +// } +// +// impl OutputsBuffer { +// pub fn create(capacity: usize, outputs: usize) -> OutputsBuffer { +// OutputsBuffer { +// inner: vec![capacity; outputs] +// .into_iter() +// .map(VecDeque::with_capacity) +// .collect::>(), +// } +// } +// +// pub fn is_all_empty(&self) -> bool { +// self.inner.iter().all(|x| x.is_empty()) +// } +// +// pub fn is_empty(&self, index: usize) -> bool { +// self.inner[index].is_empty() +// } +// +// pub fn is_full(&self) -> bool { +// self.inner.iter().any(|x| x.len() == x.capacity()) +// } +// +// pub fn clear(&mut self, index: usize) { +// self.inner[index].clear(); +// } +// +// pub fn pop(&mut self, index: usize) -> Option { +// self.inner[index].pop_front() +// } +// +// pub fn push_back(&mut self, index: usize, block: DataBlock) -> usize { +// self.inner[index].push_back(block); +// self.inner[index].len() +// } +// } +// +// #[derive(PartialEq)] +// enum PortStatus { +// Idle, +// HasData, +// NeedData, +// Finished, +// } +// +// struct PortWithStatus { +// pub status: PortStatus, +// pub port: Arc, +// } +// +// struct ExchangeShuffleTransform { +// initialized: bool, +// +// finished_inputs: usize, +// finished_outputs: usize, +// +// waiting_outputs: Vec, +// waiting_inputs: VecDeque, +// +// buffer: OutputsBuffer, +// inputs: Vec>, +// outputs: Vec>, +// } +// +// impl Processor for ExchangeShuffleTransform { +// fn name(&self) -> String { +// String::from("ExchangeShuffleTransform") +// } +// +// fn as_any(&mut self) -> &mut dyn Any { +// self +// } +// +// fn event_with_cause(&mut self, cause: EventCause) -> Result { +// if let EventCause::Output(output_index) = &cause { +// let output = &mut self.outputs[*output_index]; +// +// if output.port.is_finished() { +// if output.status != PortStatus::Finished { +// self.finished_outputs += 1; +// output.status = PortStatus::Finished; +// } +// +// self.buffer.clear(*output_index); +// +// self.wakeup_inputs(); +// self.wakeup_outputs(); +// } else if output.port.can_push() { +// if !self.buffer.is_empty(*output_index) { +// let data_block = self.buffer.pop(*output_index).unwrap(); +// output.status = PortStatus::Idle; +// output.port.push_data(Ok(data_block)); +// +// self.wakeup_inputs(); +// self.wakeup_outputs(); +// } else if output.status != PortStatus::NeedData { +// output.status = PortStatus::NeedData; +// self.waiting_outputs.push(*output_index); +// } +// } +// } +// +// if !self.initialized && !self.waiting_outputs.is_empty() { +// self.initialized = true; +// for input in &self.inputs { +// input.port.set_need_data(); +// } +// } +// +// if self.finished_outputs == self.outputs.len() { +// for input in &self.inputs { +// input.port.finish(); +// } +// +// return Ok(Event::Finished); +// } +// +// if let EventCause::Input(input_index) = &cause { +// let input = &mut self.inputs[*input_index]; +// +// if input.port.is_finished() { +// if input.status != PortStatus::Finished { +// self.finished_inputs += 1; +// input.status = PortStatus::Finished; +// } +// +// self.wakeup_outputs(); +// self.wakeup_inputs(); +// } else if input.port.has_data() { +// if !self.buffer.is_full() { +// self.take_input_data_into_buffer(*input_index); +// +// self.wakeup_outputs(); +// self.wakeup_inputs(); +// } else if input.status != PortStatus::HasData { +// input.status = PortStatus::HasData; +// self.waiting_inputs.push_back(*input_index); +// } +// } +// } +// +// if self.finished_outputs == self.outputs.len() { +// for input in &self.inputs { +// input.port.finish(); +// } +// +// return Ok(Event::Finished); +// } +// +// if self.finished_inputs == self.inputs.len() { +// for (index, output) in self.outputs.iter_mut().enumerate() { +// if self.buffer.is_empty(index) && output.status != PortStatus::Finished { +// self.finished_outputs += 1; +// output.status = PortStatus::Finished; +// output.port.finish(); +// } +// } +// +// if self.buffer.is_all_empty() { +// return Ok(Event::Finished); +// } +// } +// +// match self.waiting_outputs.is_empty() { +// true => Ok(Event::NeedConsume), +// false => Ok(Event::NeedData), +// } +// } +// +// fn details_status(&self) -> Option { +// #[derive(Debug)] +// #[allow(dead_code)] +// struct Display { +// queue_status: Vec<(usize, usize)>, +// inputs: usize, +// finished_inputs: usize, +// outputs: usize, +// finished_outputs: usize, +// +// waiting_outputs: Vec, +// waiting_inputs: VecDeque, +// } +// +// let mut queue_status = vec![]; +// for (idx, queue) in self.buffer.inner.iter().enumerate() { +// queue_status.push((idx, queue.len())); +// } +// +// Some(format!("{:?}", Display { +// queue_status, +// inputs: self.inputs.len(), +// outputs: self.outputs.len(), +// finished_inputs: self.finished_inputs, +// finished_outputs: self.finished_outputs, +// waiting_inputs: self.waiting_inputs.clone(), +// waiting_outputs: self.waiting_outputs.clone(), +// })) +// } +// } +// +// impl ExchangeShuffleTransform { +// fn wakeup_inputs(&mut self) { +// while !self.waiting_inputs.is_empty() && !self.buffer.is_full() { +// let input_index = self.waiting_inputs.pop_front().unwrap(); +// +// self.take_input_data_into_buffer(input_index); +// } +// } +// +// fn wakeup_outputs(&mut self) { +// let mut new_waiting_output = Vec::with_capacity(self.waiting_outputs.len()); +// +// for waiting_output in &self.waiting_outputs { +// let output = &mut self.outputs[*waiting_output]; +// +// if output.port.is_finished() { +// if output.status != PortStatus::Finished { +// self.finished_outputs += 1; +// output.status = PortStatus::Finished; +// } +// +// self.buffer.clear(*waiting_output); +// continue; +// } +// +// if self.buffer.is_empty(*waiting_output) { +// new_waiting_output.push(*waiting_output); +// continue; +// } +// +// let data_block = self.buffer.pop(*waiting_output).unwrap(); +// output.status = PortStatus::Idle; +// output.port.push_data(Ok(data_block)); +// } +// +// self.waiting_outputs = new_waiting_output; +// } +// +// fn take_input_data_into_buffer(&mut self, input_index: usize) { +// let input = &mut self.inputs[input_index]; +// +// input.status = PortStatus::Idle; +// let mut data_block = input.port.pull_data().unwrap().unwrap(); +// +// if let Some(block_meta) = data_block.take_meta() { +// if let Some(shuffle_meta) = ExchangeShuffleMeta::downcast_from(block_meta) { +// for (index, block) in shuffle_meta.blocks.into_iter().enumerate() { +// if (!block.is_empty() || block.get_meta().is_some()) +// && self.outputs[index].status != PortStatus::Finished +// { +// self.buffer.push_back(index, block); +// } +// } +// } +// } +// +// if input.port.is_finished() { +// if input.status != PortStatus::Finished { +// self.finished_inputs += 1; +// input.status = PortStatus::Finished; +// } +// +// return; +// } +// +// input.port.set_need_data(); +// } +// } + +// impl ExchangeShuffleTransform { +// pub fn create(inputs: usize, outputs: usize, buffer: usize) -> ExchangeShuffleTransform { +// let mut inputs_port = Vec::with_capacity(inputs); +// let mut outputs_port = Vec::with_capacity(outputs); +// +// for _index in 0..inputs { +// inputs_port.push(PortWithStatus { +// status: PortStatus::Idle, +// port: InputPort::create(), +// }); +// } +// +// for _index in 0..outputs { +// outputs_port.push(PortWithStatus { +// status: PortStatus::Idle, +// port: OutputPort::create(), +// }); +// } +// +// ExchangeShuffleTransform { +// initialized: false, +// finished_inputs: 0, +// finished_outputs: 0, +// inputs: inputs_port, +// outputs: outputs_port, +// buffer: OutputsBuffer::create(buffer, outputs), +// waiting_inputs: VecDeque::with_capacity(inputs), +// waiting_outputs: Vec::with_capacity(outputs), +// } +// } +// +// pub fn get_inputs(&self) -> Vec> { +// self.inputs.iter().map(|x| x.port.clone()).collect() +// } +// +// pub fn get_outputs(&self) -> Vec> { +// self.outputs.iter().map(|x| x.port.clone()).collect() +// } +// } // Scatter the data block and push it to the corresponding output port pub fn exchange_shuffle( @@ -388,44 +389,55 @@ pub fn exchange_shuffle( } } - // append scatter transform - pipeline.add_transform(|input, output| { - Ok(ScatterTransform::create( - input, - output, - params.shuffle_scatter.clone(), - )) - })?; - - let exchange_injector = ¶ms.exchange_injector; - let settings = ctx.get_settings(); let compression = settings.get_query_flight_compression()?; - exchange_injector.apply_shuffle_serializer(params, compression, pipeline)?; - let output_len = pipeline.output_len(); - if let Some(exchange_sorting) = &exchange_injector.exchange_sorting() { - let sorting = ShuffleExchangeSorting::create(exchange_sorting.clone()); - let transform = TransformExchangeSorting::create(output_len, sorting); - - let output = transform.get_output(); - let inputs = transform.get_inputs(); - pipeline.add_pipe(Pipe::create(output_len, 1, vec![PipeItem::create( - ProcessorPtr::create(Box::new(transform)), - inputs, - vec![output], - )])); - } - - let inputs_size = pipeline.output_len(); let outputs_size = params.destination_ids.len(); - let transform = ExchangeShuffleTransform::create(inputs_size, outputs_size, output_len); - - let inputs = transform.get_inputs(); - let outputs = transform.get_outputs(); - pipeline.add_pipe(Pipe::create(inputs_size, outputs_size, vec![ - PipeItem::create(ProcessorPtr::create(Box::new(transform)), inputs, outputs), - ])); + pipeline.exchange( + outputs_size, + FlightExchange::create( + params.destination_ids.clone(), + compression, + params.shuffle_scatter.clone(), + ), + ); + // append scatter transform + // pipeline.add_transform(|input, output| { + // Ok(ScatterTransform::create( + // input, + // output, + // params.shuffle_scatter.clone(), + // )) + // })?; + + // let exchange_injector = ¶ms.exchange_injector; + // + + // exchange_injector.apply_shuffle_serializer(params, compression, pipeline)?; + // + // let output_len = pipeline.output_len(); + // if let Some(exchange_sorting) = &exchange_injector.exchange_sorting() { + // let sorting = ShuffleExchangeSorting::create(exchange_sorting.clone()); + // let transform = TransformExchangeSorting::create(output_len, sorting); + // + // let output = transform.get_output(); + // let inputs = transform.get_inputs(); + // pipeline.add_pipe(Pipe::create(output_len, 1, vec![PipeItem::create( + // ProcessorPtr::create(Box::new(transform)), + // inputs, + // vec![output], + // )])); + // } + // + // let inputs_size = pipeline.output_len(); + // let outputs_size = params.destination_ids.len(); + // let transform = ExchangeShuffleTransform::create(inputs_size, outputs_size, output_len); + // + // let inputs = transform.get_inputs(); + // let outputs = transform.get_outputs(); + // pipeline.add_pipe(Pipe::create(inputs_size, outputs_size, vec![ + // PipeItem::create(ProcessorPtr::create(Box::new(transform)), inputs, outputs), + // ])); Ok(()) } @@ -442,25 +454,26 @@ impl ShuffleExchangeSorting { impl ExchangeSorting for ShuffleExchangeSorting { fn block_number(&self, data_block: &DataBlock) -> Result { - let block_meta = data_block.get_meta(); - let shuffle_meta = block_meta - .and_then(ExchangeShuffleMeta::downcast_ref_from) - .unwrap(); - - for block in &shuffle_meta.blocks { - if let Some(block_meta) = block.get_meta() { - if let Some(block_meta) = ExchangeSerializeMeta::downcast_ref_from(block_meta) { - return Ok(block_meta.block_number); - } - } - - if !block.is_empty() || block.get_meta().is_some() { - return self.inner.block_number(block); - } - } - - Err(ErrorCode::Internal( - "Internal, ShuffleExchangeSorting only recv ExchangeSerializeMeta.", - )) + unimplemented!() + // let block_meta = data_block.get_meta(); + // let shuffle_meta = block_meta + // .and_then(ExchangeShuffleMeta::downcast_ref_from) + // .unwrap(); + // + // for block in &shuffle_meta.blocks { + // if let Some(block_meta) = block.get_meta() { + // if let Some(block_meta) = ExchangeSerializeMeta::downcast_ref_from(block_meta) { + // return Ok(block_meta.block_number); + // } + // } + // + // if !block.is_empty() || block.get_meta().is_some() { + // return self.inner.block_number(block); + // } + // } + // + // Err(ErrorCode::Internal( + // "Internal, ShuffleExchangeSorting only recv ExchangeSerializeMeta.", + // )) } } diff --git a/src/query/service/src/servers/flight/v1/exchange/serde/exchange_serializer.rs b/src/query/service/src/servers/flight/v1/exchange/serde/exchange_serializer.rs index 2c659a2aa7a92..7e5e1b04a7881 100644 --- a/src/query/service/src/servers/flight/v1/exchange/serde/exchange_serializer.rs +++ b/src/query/service/src/servers/flight/v1/exchange/serde/exchange_serializer.rs @@ -190,8 +190,9 @@ pub fn serialize_block( let mut meta = vec![]; meta.write_scalar_own(data_block.num_rows() as u32)?; - bincode_serialize_into_buf(&mut meta, &data_block.get_meta()) - .map_err(|_| ErrorCode::BadBytes("block meta serialize error when exchange"))?; + serde_json::to_writer(&mut meta, &data_block.get_meta())?; + // bincode_serialize_into_buf(&mut meta, &data_block.get_meta()).map_err(|_| ErrorCode::BadBytes("block meta serialize error when exchange"))?; + // .map_err(|_| ErrorCode::BadBytes("block meta serialize error when exchange"))?; let (_, dict, values) = match data_block.is_empty() { true => batches_to_flight_data_with_options( diff --git a/src/query/service/src/servers/flight/v1/scatter/flight_scatter_merge.rs b/src/query/service/src/servers/flight/v1/scatter/flight_scatter_merge.rs new file mode 100644 index 0000000000000..02cebbfcbe938 --- /dev/null +++ b/src/query/service/src/servers/flight/v1/scatter/flight_scatter_merge.rs @@ -0,0 +1,37 @@ +// 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 databend_common_exception::Result; +use databend_common_expression::DataBlock; + +use crate::servers::flight::v1::scatter::flight_scatter::FlightScatter; + +pub struct MergeFlightScatter; + +// impl MergeFlightScatter { +// pub fn try_create(scattered_size: usize) -> Result { +// Ok(MergeFlightScatter { scattered_size }) +// } +// } + +impl FlightScatter for MergeFlightScatter { + fn execute(&self, data_block: DataBlock) -> Result> { + // let mut data_blocks = vec![]; + // for _ in 0..self.scattered_size { + // data_blocks.push(data_block.clone()); + // } + + Ok(vec![data_block]) + } +} diff --git a/src/query/service/src/servers/flight/v1/scatter/mod.rs b/src/query/service/src/servers/flight/v1/scatter/mod.rs index b5f5f900dab71..caaee700968e4 100644 --- a/src/query/service/src/servers/flight/v1/scatter/mod.rs +++ b/src/query/service/src/servers/flight/v1/scatter/mod.rs @@ -15,7 +15,9 @@ mod flight_scatter; mod flight_scatter_broadcast; mod flight_scatter_hash; +mod flight_scatter_merge; pub use flight_scatter::FlightScatter; pub use flight_scatter_broadcast::BroadcastFlightScatter; pub use flight_scatter_hash::HashFlightScatter; +pub use flight_scatter_merge::MergeFlightScatter; From 3e397cd383bc5058aca8f24b977c0648be2a035c Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 11 Mar 2025 16:48:43 +0800 Subject: [PATCH 04/76] refactor(query): refactor aggreagte spill code --- .../src/aggregate/partitioned_payload.rs | 10 +- .../pipeline/core/src/processors/processor.rs | 2 +- .../pipelines/builders/builder_aggregate.rs | 23 +- .../pipelines/builders/builder_exchange.rs | 1 - .../aggregator/aggregate_exchange_injector.rs | 314 ++------ .../transforms/aggregator/aggregate_meta.rs | 93 +-- .../processors/transforms/aggregator/mod.rs | 4 +- .../new_transform_partition_bucket.rs | 583 -------------- .../transforms/aggregator/serde/mod.rs | 6 - .../serde/transform_aggregate_serializer.rs | 5 +- .../serde/transform_aggregate_spill_writer.rs | 264 ------- .../serde/transform_deserializer.rs | 106 +-- ...transform_exchange_aggregate_serializer.rs | 288 ------- .../serde/transform_exchange_async_barrier.rs | 73 -- .../serde/transform_spill_reader.rs | 50 +- .../aggregator/transform_aggregate_final.rs | 73 +- .../aggregator/transform_aggregate_merge.rs | 17 + .../aggregator/transform_aggregate_partial.rs | 23 +- .../aggregator/transform_partition_bucket.rs | 738 ++++++++++++++++++ .../flight/v1/exchange/exchange_injector.rs | 7 - .../flight/v1/exchange/exchange_manager.rs | 16 +- .../flight/v1/exchange/exchange_sink.rs | 5 +- .../v1/exchange/exchange_sink_writer.rs | 7 +- .../flight/v1/exchange/exchange_source.rs | 2 - .../flight/v1/exchange/exchange_transform.rs | 4 +- .../v1/exchange/exchange_transform_scatter.rs | 53 -- .../v1/exchange/exchange_transform_shuffle.rs | 401 ---------- .../src/servers/flight/v1/exchange/mod.rs | 1 - .../v1/exchange/serde/exchange_serializer.rs | 4 - 29 files changed, 1002 insertions(+), 2171 deletions(-) delete mode 100644 src/query/service/src/pipelines/processors/transforms/aggregator/new_transform_partition_bucket.rs delete mode 100644 src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_aggregate_spill_writer.rs delete mode 100644 src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_exchange_aggregate_serializer.rs delete mode 100644 src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_exchange_async_barrier.rs create mode 100644 src/query/service/src/pipelines/processors/transforms/aggregator/transform_aggregate_merge.rs create mode 100644 src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_bucket.rs delete mode 100644 src/query/service/src/servers/flight/v1/exchange/exchange_transform_scatter.rs diff --git a/src/query/expression/src/aggregate/partitioned_payload.rs b/src/query/expression/src/aggregate/partitioned_payload.rs index 69988dfb50741..e999e435fd13c 100644 --- a/src/query/expression/src/aggregate/partitioned_payload.rs +++ b/src/query/expression/src/aggregate/partitioned_payload.rs @@ -53,16 +53,14 @@ unsafe impl Send for PartitionedPayload {} unsafe impl Sync for PartitionedPayload {} impl serde::Serialize for PartitionedPayload { - fn serialize(&self, serializer: S) -> Result - where S: Serializer { - todo!() + fn serialize(&self, _: S) -> Result { + unreachable!("PartitionedPayload must not be exchanged between multiple nodes.") } } impl<'de> serde::Deserialize<'de> for PartitionedPayload { - fn deserialize(deserializer: D) -> Result - where D: Deserializer<'de> { - todo!() + fn deserialize>(_: D) -> Result { + unreachable!("PartitionedPayload must not be exchanged between multiple nodes.") } } diff --git a/src/query/pipeline/core/src/processors/processor.rs b/src/query/pipeline/core/src/processors/processor.rs index 2d2fd18b7640e..d9e885a1b69ad 100644 --- a/src/query/pipeline/core/src/processors/processor.rs +++ b/src/query/pipeline/core/src/processors/processor.rs @@ -92,7 +92,7 @@ pub trait Processor: Send { )) } - fn configure_peer_nodes(&mut self, nodes: &[String]) { + fn configure_peer_nodes(&mut self, _nodes: &[String]) { // do nothing by default } diff --git a/src/query/service/src/pipelines/builders/builder_aggregate.rs b/src/query/service/src/pipelines/builders/builder_aggregate.rs index cf7353b11a6a6..5900c882cf550 100644 --- a/src/query/service/src/pipelines/builders/builder_aggregate.rs +++ b/src/query/service/src/pipelines/builders/builder_aggregate.rs @@ -166,26 +166,7 @@ impl PipelineBuilder { )?)) })?; - // If cluster mode, spill write will be completed in exchange serialize, because we need scatter the block data first - // if !self.is_exchange_neighbor { - // let operator = DataOperator::instance().spill_operator(); - // let location_prefix = self.ctx.query_id_spill_prefix(); - // - // self.main_pipeline.add_transform(|input, output| { - // Ok(ProcessorPtr::create( - // TransformAggregateSpillWriter::try_create( - // self.ctx.clone(), - // input, - // output, - // operator.clone(), - // params.clone(), - // location_prefix.clone(), - // )?, - // )) - // })?; - // } - - self.exchange_injector = AggregateInjector::create(self.ctx.clone(), params.clone()); + self.exchange_injector = AggregateInjector::create(); Ok(()) } @@ -222,7 +203,7 @@ impl PipelineBuilder { let input: &PhysicalPlan = &aggregate.input; if matches!(input, PhysicalPlan::ExchangeSource(_)) { - self.exchange_injector = AggregateInjector::create(self.ctx.clone(), params.clone()); + self.exchange_injector = AggregateInjector::create(); } self.build_pipeline(&aggregate.input)?; self.exchange_injector = old_inject; diff --git a/src/query/service/src/pipelines/builders/builder_exchange.rs b/src/query/service/src/pipelines/builders/builder_exchange.rs index 6c27b81ae366e..26af09a82e82f 100644 --- a/src/query/service/src/pipelines/builders/builder_exchange.rs +++ b/src/query/service/src/pipelines/builders/builder_exchange.rs @@ -24,7 +24,6 @@ impl PipelineBuilder { let mut build_res = exchange_manager.get_fragment_source( &exchange_source.query_id, exchange_source.source_fragment_id, - self.exchange_injector.clone(), )?; // add profile 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 a59995ad28400..ef389c7150d81 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 @@ -18,13 +18,12 @@ use std::sync::Arc; use arrow_ipc::writer::IpcWriteOptions; use arrow_ipc::CompressionType; -use bumpalo::Bump; +use databend_common_catalog::table_context::TableContext; use databend_common_config::GlobalConfig; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::DataBlock; -use databend_common_expression::PartitionedPayload; use databend_common_expression::Payload; use databend_common_expression::PayloadFlushState; use databend_common_pipeline_core::processors::Exchange; @@ -33,19 +32,36 @@ use databend_common_settings::FlightCompression; use itertools::Itertools; use crate::pipelines::processors::transforms::aggregator::aggregate_meta::AggregateMeta; -use crate::pipelines::processors::transforms::aggregator::AggregatorParams; use crate::servers::flight::v1::exchange::serde::serialize_block; use crate::servers::flight::v1::exchange::serde::ExchangeSerializeMeta; use crate::servers::flight::v1::exchange::DataExchange; use crate::servers::flight::v1::exchange::ExchangeInjector; use crate::servers::flight::v1::exchange::ExchangeSorting; +use crate::servers::flight::v1::scatter::BroadcastFlightScatter; use crate::servers::flight::v1::scatter::FlightScatter; +use crate::servers::flight::v1::scatter::HashFlightScatter; use crate::sessions::QueryContext; struct AggregateExchangeSorting {} pub fn compute_block_number(bucket: isize, max_partition_count: usize) -> Result { - Ok(max_partition_count as isize * 1000 + bucket) + match bucket.is_negative() { + true => Ok(((1_usize << 16) & (bucket.unsigned_abs() << 8) & max_partition_count) as isize), + false => Ok((((bucket as usize) << 8) & max_partition_count) as isize), + } +} + +pub fn restore_block_number(value: isize) -> (isize, usize) { + let mut value = value as usize; + let max_partition = value & 0xFF_usize; + value >>= 8; + let abs_partition = value & 0xFF_usize; + value >>= 8; + + match value & 1 { + 1 => (0 - abs_partition as isize, max_partition), + _ => (abs_partition as isize, max_partition), + } } impl ExchangeSorting for AggregateExchangeSorting { @@ -58,37 +74,21 @@ impl ExchangeSorting for AggregateExchangeSorting { serde_json::to_string(block_meta_info) ))), Some(meta_info) => match meta_info { - AggregateMeta::Partitioned { .. } => unreachable!(), + AggregateMeta::FinalPayload(_) => unreachable!(), + AggregateMeta::InFlightPayload(_) => unreachable!(), AggregateMeta::Serialized(v) => { compute_block_number(v.bucket, v.max_partition_count) } AggregateMeta::AggregatePayload(v) => { compute_block_number(v.partition, v.max_partition_count) } - AggregateMeta::BucketSpilled(_) => Ok(-1), + AggregateMeta::SpilledPayload(_) => Ok(-1), }, }, } } } -struct HashTableHashScatter { - bucket_lookup: HashMap, -} - -impl HashTableHashScatter { - pub fn create(nodes: &[String]) -> Arc> { - Arc::new(Box::new(HashTableHashScatter { - bucket_lookup: nodes - .iter() - .cloned() - .enumerate() - .map(|(l, r)| (r, l)) - .collect::>(), - })) - } -} - fn scatter_payload(mut payload: Payload, buckets: usize) -> Result> { let mut buckets = Vec::with_capacity(buckets); @@ -123,225 +123,51 @@ fn scatter_payload(mut payload: Payload, buckets: usize) -> Result> Ok(buckets) } -// fn scatter_partitioned_payload( -// partitioned_payload: PartitionedPayload, -// buckets: usize, -// ) -> Result> { -// let mut buckets = Vec::with_capacity(buckets); -// -// let group_types = partitioned_payload.group_types.clone(); -// let aggrs = partitioned_payload.aggrs.clone(); -// let partition_count = partitioned_payload.partition_count() as u64; -// let mut state = PayloadFlushState::default(); -// -// for _ in 0..buckets.capacity() { -// buckets.push(PartitionedPayload::new( -// group_types.clone(), -// aggrs.clone(), -// partition_count, -// partitioned_payload.arenas.clone(), -// )); -// } -// -// let mut payloads = Vec::with_capacity(buckets.len()); -// -// for _ in 0..payloads.capacity() { -// payloads.push(Payload::new( -// Arc::new(Bump::new()), -// group_types.clone(), -// aggrs.clone(), -// partitioned_payload.states_layout.clone(), -// )); -// } -// -// for mut payload in partitioned_payload.payloads.into_iter() { -// // scatter each page of the payload. -// while payload.scatter(&mut state, buckets.len()) { -// // copy to the corresponding bucket. -// for (idx, bucket) in payloads.iter_mut().enumerate() { -// let count = state.probe_state.partition_count[idx]; -// -// if count > 0 { -// let sel = &state.probe_state.partition_entries[idx]; -// bucket.copy_rows(sel, count, &state.addresses); -// } -// } -// } -// state.clear(); -// payload.state_move_out = true; -// } -// -// for (idx, payload) in payloads.into_iter().enumerate() { -// buckets[idx].combine_single(payload, &mut state, None); -// } -// -// Ok(buckets) -// } - -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) { - // let mut blocks = Vec::with_capacity(self.bucket_lookup.len()); - // match block_meta { - // AggregateMeta::Spilled(_) => unreachable!(), - // AggregateMeta::Serialized(_) => unreachable!(), - // AggregateMeta::Partitioned { .. } => unreachable!(), - // AggregateMeta::AggregateSpilling(_) => unreachable!(), - // AggregateMeta::BucketSpilled(v) => { - // // v.destination_node - // } - // AggregateMeta::AggregatePayload(p) => { - // for payload in scatter_payload(p.payload, self.buckets)? { - // blocks.push(DataBlock::empty_with_meta( - // AggregateMeta::create_agg_payload( - // payload, - // p.partition, - // p.max_partition_count, - // ), - // )); - // } - // } - // }; - // - // return Ok(blocks); - // } - // } - - Err(ErrorCode::Internal( - "Internal, HashTableHashScatter only recv AggregateMeta", - )) - } -} - -pub struct AggregateInjector { - ctx: Arc, - aggregator_params: Arc, -} +pub struct AggregateInjector; impl AggregateInjector { - pub fn create( - ctx: Arc, - params: Arc, - ) -> Arc { - Arc::new(AggregateInjector { - ctx, - aggregator_params: params, - }) + pub fn create() -> Arc { + Arc::new(AggregateInjector) } } impl ExchangeInjector for AggregateInjector { fn flight_scatter( &self, - _: &Arc, + ctx: &Arc, exchange: &DataExchange, ) -> Result>> { - match exchange { + Ok(Arc::new(match exchange { DataExchange::Merge(_) => unreachable!(), - DataExchange::Broadcast(_) => unreachable!(), + DataExchange::Broadcast(exchange) => Box::new(BroadcastFlightScatter::try_create( + exchange.destination_ids.len(), + )?), DataExchange::ShuffleDataExchange(exchange) => { - Ok(HashTableHashScatter::create(&exchange.destination_ids)) + let local_id = &ctx.get_cluster().local_id; + let local_pos = exchange + .destination_ids + .iter() + .position(|x| x == local_id) + .unwrap(); + HashFlightScatter::try_create( + ctx.get_function_context()?, + exchange.shuffle_keys.clone(), + exchange.destination_ids.len(), + local_pos, + )? } - } + })) } fn exchange_sorting(&self) -> Option> { Some(Arc::new(AggregateExchangeSorting {})) } - - // fn apply_merge_serializer( - // &self, - // _: &MergeExchangeParams, - // _compression: Option, - // pipeline: &mut Pipeline, - // ) -> Result<()> { - // let params = self.aggregator_params.clone(); - // - // let operator = DataOperator::instance().spill_operator(); - // let location_prefix = self.ctx.query_id_spill_prefix(); - // - // pipeline.add_transform(|input, output| { - // Ok(ProcessorPtr::create( - // TransformAggregateSpillWriter::try_create( - // self.ctx.clone(), - // input, - // output, - // operator.clone(), - // params.clone(), - // location_prefix.clone(), - // )?, - // )) - // })?; - // - // pipeline.add_transform(|input, output| { - // TransformAggregateSerializer::try_create(input, output, params.clone()) - // }) - // } - // - // fn apply_shuffle_serializer( - // &self, - // shuffle_params: &ShuffleExchangeParams, - // compression: Option, - // pipeline: &mut Pipeline, - // ) -> Result<()> { - // // let params = self.aggregator_params.clone(); - // // let operator = DataOperator::instance().spill_operator(); - // // let location_prefix = self.ctx.query_id_spill_prefix(); - // // - // // let schema = shuffle_params.schema.clone(); - // // let local_id = &shuffle_params.executor_id; - // // let local_pos = shuffle_params - // // .destination_ids - // // .iter() - // // .position(|x| x == local_id) - // // .unwrap(); - // // - // // pipeline.add_transform(|input, output| { - // // Ok(ProcessorPtr::create( - // // TransformExchangeAggregateSerializer::try_create( - // // self.ctx.clone(), - // // input, - // // output, - // // operator.clone(), - // // location_prefix.clone(), - // // params.clone(), - // // compression, - // // schema.clone(), - // // local_pos, - // // )?, - // // )) - // // })?; - // // - // // pipeline.add_transform(TransformExchangeAsyncBarrier::try_create) - // Ok(()) - // } - // - // fn apply_merge_deserializer( - // &self, - // params: &MergeExchangeParams, - // pipeline: &mut Pipeline, - // ) -> Result<()> { - // pipeline.add_transform(|input, output| { - // TransformAggregateDeserializer::try_create(input, output, ¶ms.schema) - // }) - // } - // - // fn apply_shuffle_deserializer( - // &self, - // params: &ShuffleExchangeParams, - // pipeline: &mut Pipeline, - // ) -> Result<()> { - // pipeline.add_transform(|input, output| { - // TransformAggregateDeserializer::try_create(input, output, ¶ms.schema) - // }) - // } } pub struct FlightExchange { local_id: String, bucket_lookup: HashMap, - recv_bucket_lookup: Vec, + rev_bucket_lookup: Vec, options: IpcWriteOptions, shuffle_scatter: Arc>, } @@ -370,7 +196,7 @@ impl FlightExchange { Arc::new(FlightExchange { local_id: GlobalConfig::instance().query.node_id.clone(), bucket_lookup, - recv_bucket_lookup: lookup, + rev_bucket_lookup: lookup, options: IpcWriteOptions::default() .try_with_compression(compression) .unwrap(), @@ -381,11 +207,16 @@ impl FlightExchange { impl FlightExchange { fn default_partition(&self, data_block: DataBlock) -> Result> { + if self.rev_bucket_lookup.is_empty() { + let data_block = serialize_block(0, data_block, &self.options)?; + return Ok(vec![(0, data_block)]); + } + let data_blocks = self.shuffle_scatter.execute(data_block)?; let mut blocks = Vec::with_capacity(data_blocks.len()); for (idx, data_block) in data_blocks.into_iter().enumerate() { - if data_block.is_empty() || self.recv_bucket_lookup[idx] == self.local_id { + if self.rev_bucket_lookup[idx] == self.local_id { blocks.push((idx, data_block)); continue; } @@ -403,8 +234,6 @@ impl Exchange for FlightExchange { const STRATEGY: MultiwayStrategy = MultiwayStrategy::Custom; fn partition(&self, mut data_block: DataBlock, n: usize) -> Result> { - assert_eq!(self.bucket_lookup.len(), n); - let Some(meta) = data_block.take_meta() else { return self.default_partition(data_block); }; @@ -413,27 +242,33 @@ impl Exchange for FlightExchange { return self.default_partition(data_block); }; + assert_eq!(self.bucket_lookup.len(), n); match meta { AggregateMeta::Serialized(_) => unreachable!(), - AggregateMeta::Partitioned { .. } => unreachable!(), - AggregateMeta::BucketSpilled(v) => match self.bucket_lookup.get(&v.destination_node) { + AggregateMeta::FinalPayload(_) => unreachable!(), + AggregateMeta::InFlightPayload(_) => unreachable!(), + AggregateMeta::SpilledPayload(v) => match self.bucket_lookup.get(&v.destination_node) { None => unreachable!(), Some(idx) => match v.destination_node == self.local_id { true => Ok(vec![( *idx, - DataBlock::empty_with_meta(AggregateMeta::create_bucket_spilled(v)), + DataBlock::empty_with_meta(AggregateMeta::create_spilled_payload(v)), )]), false => { + let block_number = compute_block_number(-1, v.max_partition_count)?; let block = - DataBlock::empty_with_meta(AggregateMeta::create_bucket_spilled(v)); - Ok(vec![(*idx, serialize_block(-2, block, &self.options)?)]) + DataBlock::empty_with_meta(AggregateMeta::create_spilled_payload(v)); + Ok(vec![( + *idx, + serialize_block(block_number, block, &self.options)?, + )]) } }, }, AggregateMeta::AggregatePayload(p) => { let mut blocks = Vec::with_capacity(n); for (idx, payload) in scatter_payload(p.payload, n)?.into_iter().enumerate() { - if self.recv_bucket_lookup[idx] == self.local_id { + if self.rev_bucket_lookup[idx] == self.local_id { blocks.push(( idx, DataBlock::empty_with_meta(AggregateMeta::create_agg_payload( @@ -447,7 +282,12 @@ impl Exchange for FlightExchange { } let data_block = payload.aggregate_flush_all()?; - let data_block = serialize_block(p.partition, data_block, &self.options)?; + let data_block = data_block.add_meta(Some( + AggregateMeta::create_in_flight_payload(p.partition, p.max_partition_count), + ))?; + + let block_number = compute_block_number(p.partition, p.max_partition_count)?; + let data_block = serialize_block(block_number, data_block, &self.options)?; blocks.push((idx, data_block)); } @@ -478,7 +318,19 @@ impl Exchange for FlightExchange { return Ordering::Less; }; - left_meta.block_number.cmp(&right_meta.block_number) + let (l_partition, l_max_partition) = restore_block_number(left_meta.block_number); + let (r_partition, r_max_partition) = restore_block_number(right_meta.block_number); + + // ORDER BY max_partition asc, partition asc + match l_max_partition.cmp(&r_max_partition) { + Ordering::Less => Ordering::Less, + Ordering::Greater => Ordering::Greater, + Ordering::Equal => match l_partition.cmp(&r_partition) { + Ordering::Less => Ordering::Less, + Ordering::Equal => Ordering::Equal, + Ordering::Greater => Ordering::Greater, + }, + } }); position.ok_or_else(|| ErrorCode::Internal("Cannot multiway pick with all none")) 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 e5dd55ad127d8..d44960f028861 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 @@ -19,6 +19,7 @@ use std::sync::Arc; use bumpalo::Bump; use databend_common_exception::Result; +use databend_common_expression::local_block_meta_serde; use databend_common_expression::types::DataType; use databend_common_expression::AggregateFunction; use databend_common_expression::AggregateHashTable; @@ -31,8 +32,6 @@ use databend_common_expression::InputColumns; use databend_common_expression::PartitionedPayload; use databend_common_expression::Payload; use databend_common_expression::ProbeState; -use serde::Deserializer; -use serde::Serializer; pub struct SerializedPayload { pub bucket: isize, @@ -41,20 +40,6 @@ pub struct SerializedPayload { pub max_partition_count: usize, } -impl serde::Serialize for SerializedPayload { - fn serialize(&self, serializer: S) -> std::result::Result - where S: Serializer { - todo!() - } -} - -impl<'de> serde::Deserialize<'de> for SerializedPayload { - fn deserialize(deserializer: D) -> std::result::Result - where D: Deserializer<'de> { - todo!() - } -} - impl SerializedPayload { pub fn get_group_by_column(&self) -> &Column { let entry = self.data_block.columns().last().unwrap(); @@ -122,9 +107,9 @@ impl SerializedPayload { } } -#[derive(serde::Serialize, serde::Deserialize)] -pub struct BucketSpilledPayload { - pub bucket: isize, +#[derive(serde::Serialize, serde::Deserialize, Clone, Debug)] +pub struct SpilledPayload { + pub partition: isize, pub location: String, pub data_range: Range, pub destination_node: String, @@ -138,27 +123,23 @@ pub struct AggregatePayload { pub max_partition_count: usize, } -impl serde::Serialize for AggregatePayload { - fn serialize(&self, serializer: S) -> std::result::Result - where S: Serializer { - todo!() - } +#[derive(serde::Serialize, serde::Deserialize)] +pub struct InFlightPayload { + pub partition: isize, + pub max_partition: usize, } -impl<'de> serde::Deserialize<'de> for AggregatePayload { - fn deserialize(deserializer: D) -> std::result::Result - where D: Deserializer<'de> { - todo!() - } +pub struct FinalPayload { + pub data: Vec<(AggregateMeta, DataBlock)>, } #[derive(serde::Serialize, serde::Deserialize)] pub enum AggregateMeta { Serialized(SerializedPayload), + SpilledPayload(SpilledPayload), AggregatePayload(AggregatePayload), - BucketSpilled(BucketSpilledPayload), - - Partitioned { bucket: isize, data: Vec }, + InFlightPayload(InFlightPayload), + FinalPayload(FinalPayload), } impl AggregateMeta { @@ -174,6 +155,13 @@ impl AggregateMeta { })) } + pub fn create_in_flight_payload(partition: isize, max_partition: usize) -> BlockMetaInfoPtr { + Box::new(AggregateMeta::InFlightPayload(InFlightPayload { + partition, + max_partition, + })) + } + pub fn create_serialized( bucket: isize, block: DataBlock, @@ -186,43 +174,28 @@ impl AggregateMeta { })) } - pub fn create_bucket_spilled(payload: BucketSpilledPayload) -> BlockMetaInfoPtr { - Box::new(AggregateMeta::BucketSpilled(payload)) + pub fn create_spilled_payload(payload: SpilledPayload) -> BlockMetaInfoPtr { + Box::new(AggregateMeta::SpilledPayload(payload)) } - pub fn create_partitioned(bucket: isize, data: Vec) -> BlockMetaInfoPtr { - Box::new(AggregateMeta::Partitioned { data, bucket }) + pub fn create_final(data: Vec<(AggregateMeta, DataBlock)>) -> BlockMetaInfoPtr { + Box::new(AggregateMeta::FinalPayload(FinalPayload { data })) } } -// impl serde::Serialize for AggregateMeta { -// fn serialize(&self, s: S) -> std::result::Result -// where -// S: serde::Serializer, -// { -// unreachable!("AggregateMeta does not support exchanging between multiple nodes") -// } -// } -// -// 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 { fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { match self { - AggregateMeta::Partitioned { .. } => { - f.debug_struct("AggregateMeta::Partitioned").finish() - } + AggregateMeta::FinalPayload(_) => f.debug_struct("AggregateMeta::Partitioned").finish(), AggregateMeta::Serialized { .. } => { f.debug_struct("AggregateMeta::Serialized").finish() } - AggregateMeta::BucketSpilled(_) => f.debug_struct("Aggregate::BucketSpilled").finish(), + AggregateMeta::SpilledPayload(_) => { + f.debug_struct("Aggregate::SpilledPayload").finish() + } + AggregateMeta::InFlightPayload(_) => { + f.debug_struct("Aggregate:InFlightPayload").finish() + } AggregateMeta::AggregatePayload(_) => { f.debug_struct("AggregateMeta:AggregatePayload").finish() } @@ -232,3 +205,7 @@ impl Debug for AggregateMeta { #[typetag::serde(name = "AggregateMeta")] impl BlockMetaInfo for AggregateMeta {} + +local_block_meta_serde!(FinalPayload); +local_block_meta_serde!(AggregatePayload); +local_block_meta_serde!(SerializedPayload); 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 574e571650178..d33411cc45da7 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/mod.rs @@ -15,11 +15,11 @@ mod aggregate_exchange_injector; mod aggregate_meta; mod aggregator_params; -mod new_transform_partition_bucket; mod serde; mod transform_aggregate_expand; mod transform_aggregate_final; mod transform_aggregate_partial; +mod transform_partition_bucket; mod transform_single_key; mod udaf_script; @@ -27,10 +27,10 @@ pub use aggregate_exchange_injector::AggregateInjector; pub use aggregate_exchange_injector::FlightExchange; 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_partition_bucket::build_partition_bucket; pub use transform_single_key::FinalSingleStateAggregator; pub use transform_single_key::PartialSingleStateAggregator; pub use udaf_script::*; 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 deleted file mode 100644 index 39b46527a2200..0000000000000 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/new_transform_partition_bucket.rs +++ /dev/null @@ -1,583 +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::sync::Arc; - -use bumpalo::Bump; -use databend_common_exception::ErrorCode; -use databend_common_exception::Result; -use databend_common_expression::BlockMetaInfoDowncast; -use databend_common_expression::DataBlock; -use databend_common_expression::PartitionedPayload; -use databend_common_expression::PayloadFlushState; -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; - -static SINGLE_LEVEL_BUCKET_NUM: isize = -1; -static MAX_PARTITION_COUNT: usize = 128; - -struct InputPortState { - port: Arc, - bucket: isize, - max_partition_count: usize, -} -pub struct NewTransformPartitionBucket { - output: Arc, - inputs: Vec, - params: Arc, - working_bucket: isize, - pushing_bucket: isize, - initialized_all_inputs: bool, - all_inputs_init: bool, - buckets_blocks: BTreeMap>, - flush_state: PayloadFlushState, - unpartitioned_blocks: Vec, - max_partition_count: usize, -} - -impl NewTransformPartitionBucket { - pub fn create(input_nums: usize, params: Arc) -> Result { - let mut inputs = Vec::with_capacity(input_nums); - - for _index in 0..input_nums { - inputs.push(InputPortState { - bucket: -1, - port: InputPort::create(), - max_partition_count: 0, - }); - } - - Ok(NewTransformPartitionBucket { - params, - inputs, - working_bucket: 0, - pushing_bucket: 0, - output: OutputPort::create(), - buckets_blocks: BTreeMap::new(), - unpartitioned_blocks: vec![], - flush_state: PayloadFlushState::default(), - initialized_all_inputs: false, - all_inputs_init: false, - max_partition_count: 0, - }) - } - - 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; - // in a cluster where partitions are only 8 and 128, - // we need to pull all data where the partition equals 8 until the partition changes to 128 or there is no data available. - if self.params.cluster_aggregator { - for index in 0..self.inputs.len() { - if self.inputs[index].port.is_finished() { - continue; - } - - // We pull all the data that are not the max_partition_count and all spill data - if self.inputs[index].max_partition_count == MAX_PARTITION_COUNT - && 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.inputs[index].max_partition_count, - ) = self.add_bucket(data_block)?; - - // we need pull all spill data in init, and data less than max partition - if self.inputs[index].bucket <= SINGLE_LEVEL_BUCKET_NUM - || self.inputs[index].max_partition_count < MAX_PARTITION_COUNT - { - self.inputs[index].port.set_need_data(); - self.initialized_all_inputs = false; - } - } - } else { - // in singleton, the partition is 8, 32, 128. - // We pull the first data to ensure the max partition, - // and then pull all data that is less than the max partition - let mut refresh_index = 0; - for index in 0..self.inputs.len() { - if self.inputs[index].port.is_finished() { - continue; - } - - // We pull all the data that are not the max_partition_count - if self.inputs[index].max_partition_count > 0 - && self.inputs[index].bucket > SINGLE_LEVEL_BUCKET_NUM - && self.inputs[index].max_partition_count == self.max_partition_count - { - 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()?; - - let before_max_partition_count = self.max_partition_count; - ( - self.inputs[index].bucket, - self.inputs[index].max_partition_count, - ) = self.add_bucket(data_block)?; - - // we need pull all spill data in init, and data less than max partition - if self.inputs[index].bucket <= SINGLE_LEVEL_BUCKET_NUM - || self.inputs[index].max_partition_count < self.max_partition_count - { - self.inputs[index].port.set_need_data(); - self.initialized_all_inputs = false; - } - - // max partition count change - if before_max_partition_count > 0 - && before_max_partition_count != self.max_partition_count - { - // set need data for inputs which is less than the max partition - for i in refresh_index..index { - if !self.inputs[i].port.is_finished() - && !self.inputs[i].port.has_data() - && self.inputs[i].max_partition_count != self.max_partition_count - { - self.inputs[i].port.set_need_data(); - self.initialized_all_inputs = false; - } - } - refresh_index = index; - } - } - } - - if self.initialized_all_inputs { - self.all_inputs_init = true; - } - - Ok(self.initialized_all_inputs) - } - - #[allow(unused_assignments)] - fn add_bucket(&mut self, mut data_block: DataBlock) -> Result<(isize, usize)> { - eprintln!("recv bucket {:?} {:?}", data_block, serde_json::to_string(&data_block.get_meta())); - let (mut bucket, mut partition_count) = (0, 0); - let mut is_empty_block = false; - if let Some(block_meta) = data_block.get_meta() { - if let Some(block_meta) = AggregateMeta::downcast_ref_from(block_meta) { - (bucket, partition_count) = match block_meta { - AggregateMeta::Partitioned { .. } => unreachable!(), - AggregateMeta::BucketSpilled(_) => { - let meta = data_block.take_meta().unwrap(); - - if let Some(AggregateMeta::BucketSpilled(payload)) = - AggregateMeta::downcast_from(meta) - { - let bucket = payload.bucket; - let partition_count = payload.max_partition_count; - self.max_partition_count = - self.max_partition_count.max(partition_count); - - let data_block = DataBlock::empty_with_meta( - AggregateMeta::create_bucket_spilled(payload), - ); - 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((SINGLE_LEVEL_BUCKET_NUM, partition_count)); - } - unreachable!() - } - AggregateMeta::Serialized(payload) => { - is_empty_block = payload.data_block.is_empty(); - self.max_partition_count = - self.max_partition_count.max(payload.max_partition_count); - - (payload.bucket, payload.max_partition_count) - } - AggregateMeta::AggregatePayload(payload) => { - is_empty_block = payload.payload.len() == 0; - self.max_partition_count = - self.max_partition_count.max(payload.max_partition_count); - - (payload.partition, payload.max_partition_count) - } - }; - } else { - return Err(ErrorCode::Internal(format!( - "Internal, TransformPartitionBucket only recv AggregateMeta, but got {:?}", - block_meta - ))); - } - } else { - return Err(ErrorCode::Internal( - "Internal, TransformPartitionBucket only recv DataBlock with meta.", - )); - } - - if !is_empty_block { - if self.all_inputs_init { - if partition_count != self.max_partition_count { - return Err(ErrorCode::Internal( - "Internal, the partition count does not equal the max partition count on TransformPartitionBucket. - ", - )); - } - match self.buckets_blocks.entry(bucket) { - Entry::Vacant(v) => { - v.insert(vec![data_block]); - } - Entry::Occupied(mut v) => { - v.get_mut().push(data_block); - } - }; - } else { - self.unpartitioned_blocks.push(data_block); - } - } - - Ok((bucket, partition_count)) - } - - fn try_push_data_block(&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 partition_block(&mut self, payload: SerializedPayload) -> Result>> { - // 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))); - 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(vec![]); - } - - // need repartition - let mut blocks = Vec::with_capacity(self.max_partition_count); - let p = payload.convert_to_partitioned_payload( - self.params.group_data_types.clone(), - self.params.aggregate_functions.clone(), - self.params.num_states(), - 0, - Arc::new(Bump::new()), - )?; - - let mut partitioned_payload = PartitionedPayload::new( - self.params.group_data_types.clone(), - self.params.aggregate_functions.clone(), - self.max_partition_count as u64, - p.arenas.clone(), - ); - partitioned_payload.combine(p, &mut self.flush_state); - - for (bucket, payload) in partitioned_payload.payloads.into_iter().enumerate() { - blocks.push(Some(DataBlock::empty_with_meta( - AggregateMeta::create_agg_payload( - payload, - bucket as isize, - self.max_partition_count, - ), - ))); - } - - Ok(blocks) - } - - fn partition_payload(&mut self, payload: AggregatePayload) -> Result>> { - // already is max partition - if payload.max_partition_count == self.max_partition_count { - let bucket = payload.partition; - 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]); - } - Entry::Occupied(mut v) => { - v.get_mut().push(data_block); - } - }; - return Ok(vec![]); - } - - // need repartition - let mut blocks = Vec::with_capacity(self.max_partition_count); - let mut partitioned_payload = PartitionedPayload::new( - self.params.group_data_types.clone(), - self.params.aggregate_functions.clone(), - self.max_partition_count as u64, - vec![payload.payload.arena.clone()], - ); - - partitioned_payload.combine_single(payload.payload, &mut self.flush_state, None); - - for (bucket, payload) in partitioned_payload.payloads.into_iter().enumerate() { - blocks.push(Some(DataBlock::empty_with_meta( - AggregateMeta::create_agg_payload( - payload, - bucket as isize, - self.max_partition_count, - ), - ))); - } - - Ok(blocks) - } - - 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)) - } -} - -#[async_trait::async_trait] -impl Processor for NewTransformPartitionBucket { - 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.unpartitioned_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)?; - - 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 - .unpartitioned_blocks - .pop() - .and_then(|mut block| block.take_meta()) - .and_then(AggregateMeta::downcast_from); - - if let Some(agg_block_meta) = block_meta { - let data_blocks = match agg_block_meta { - AggregateMeta::Partitioned { .. } => unreachable!(), - AggregateMeta::BucketSpilled(_) => unreachable!(), - AggregateMeta::Serialized(payload) => self.partition_block(payload)?, - AggregateMeta::AggregatePayload(payload) => self.partition_payload(payload)?, - }; - - 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( - 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().spill_operator(); - pipeline.add_transform(|input, output| { - let operator = operator.clone(); - TransformAggregateSpillReader::create( - input, - output, - operator, - semaphore.clone(), - params.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 76a55b10e85b3..a939193179d48 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 @@ -14,18 +14,12 @@ mod serde_meta; mod transform_aggregate_serializer; -mod transform_aggregate_spill_writer; mod transform_deserializer; -mod transform_exchange_aggregate_serializer; -mod transform_exchange_async_barrier; mod transform_spill_reader; pub use serde_meta::*; pub use transform_aggregate_serializer::*; -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_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 c036e9c28b8c6..e40af19054f0b 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 @@ -124,9 +124,10 @@ impl TransformAggregateSerializer { if let Some(block_meta) = data_block.take_meta() { if let Some(block_meta) = AggregateMeta::downcast_from(block_meta) { match block_meta { + AggregateMeta::SpilledPayload(_) => unreachable!(), AggregateMeta::Serialized(_) => unreachable!(), - AggregateMeta::BucketSpilled(_) => unreachable!(), - AggregateMeta::Partitioned { .. } => unreachable!(), + AggregateMeta::InFlightPayload(_) => unreachable!(), + AggregateMeta::FinalPayload(_) => unreachable!(), AggregateMeta::AggregatePayload(p) => { self.input_data = Some(SerializeAggregateStream::create( &self.params, 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 deleted file mode 100644 index 168b180818958..0000000000000 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_aggregate_spill_writer.rs +++ /dev/null @@ -1,264 +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::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_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::AggregateMeta; -use crate::pipelines::processors::transforms::aggregator::AggregatorParams; -use crate::pipelines::processors::transforms::aggregator::BucketSpilledPayload; -use crate::sessions::QueryContext; -use crate::spillers::Spiller; -use crate::spillers::SpillerConfig; -use crate::spillers::SpillerType; - -// pub struct TransformAggregateSpillWriter { -// ctx: Arc, -// input: Arc, -// output: Arc, -// _params: Arc, -// -// spiller: Arc, -// spilled_block: Option, -// spilling_meta: Option, -// spilling_future: Option>>, -// } -// -// impl TransformAggregateSpillWriter { -// pub fn try_create( -// ctx: Arc, -// input: Arc, -// output: Arc, -// operator: Operator, -// params: Arc, -// location_prefix: String, -// ) -> Result> { -// let config = SpillerConfig { -// spiller_type: SpillerType::Aggregation, -// location_prefix, -// disk_spill: None, -// use_parquet: ctx.get_settings().get_spilling_file_format()?.is_parquet(), -// }; -// -// let spiller = Spiller::create(ctx.clone(), operator, config.clone())?; -// Ok(Box::new(TransformAggregateSpillWriter { -// ctx, -// input, -// output, -// _params: params, -// spiller: Arc::new(spiller), -// spilled_block: None, -// spilling_meta: None, -// spilling_future: None, -// })) -// } -// } -// -// #[async_trait::async_trait] -// impl Processor for TransformAggregateSpillWriter { -// fn name(&self) -> String { -// String::from("TransformAggregateSpillWriter") -// } -// -// 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); -// } -// -// while 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::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::AggregateSpilling(payload) => { -// self.spilling_future = Some(agg_spilling_aggregate_payload( -// self.ctx.clone(), -// self.spiller.clone(), -// payload, -// )?); -// -// return Ok(()); -// } -// _ => { -// return Err(ErrorCode::Internal("")); -// } -// } -// } -// -// 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_aggregate_payload( -// ctx: Arc, -// spiller: Arc, -// partitioned_payload: PartitionedPayload, -// ) -> Result>> { -// 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); -// // Record how many rows are spilled. -// let mut rows = 0; -// let location = spiller.create_unique_location(); -// for (bucket, payload) in partitioned_payload.payloads.into_iter().enumerate() { -// if payload.len() == 0 { -// continue; -// } -// -// let data_block = payload.aggregate_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.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, -// max_partition_count: partition_count, -// destination_node: "".to_string(), -// }); -// } -// -// Ok(Box::pin(async move { -// let instant = Instant::now(); -// if !write_data.is_empty() { -// let (location, write_bytes) = spiller -// .spill_stream_aggregate_buffer(Some(location), write_data) -// .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() -// ); -// } -// -// 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 bfb6645b43e58..ed9eaab533161 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 @@ -17,14 +17,8 @@ use std::sync::Arc; use arrow_schema::Schema as ArrowSchema; use databend_common_exception::ErrorCode; use databend_common_exception::Result; -use databend_common_expression::types::ArrayType; -use databend_common_expression::types::NumberType; -use databend_common_expression::types::UInt64Type; -use databend_common_expression::types::ValueType; -use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::DataBlock; use databend_common_expression::DataSchemaRef; -use databend_common_io::prelude::bincode_deserialize_from_slice; use databend_common_io::prelude::BinaryRead; use databend_common_pipeline_core::processors::InputPort; use databend_common_pipeline_core::processors::OutputPort; @@ -33,11 +27,6 @@ use databend_common_pipeline_transforms::processors::BlockMetaTransform; use databend_common_pipeline_transforms::processors::BlockMetaTransformer; use databend_common_pipeline_transforms::processors::UnknownMode; -use crate::pipelines::processors::transforms::aggregator::exchange_defines; -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::servers::flight::v1::exchange::serde::deserialize_block; use crate::servers::flight::v1::exchange::serde::ExchangeDeserializeMeta; use crate::servers::flight::v1::packets::DataPacket; @@ -69,110 +58,19 @@ impl TransformDeserializer { fn recv_data(&self, dict: Vec, fragment_data: FragmentData) -> Result { const ROW_HEADER_SIZE: usize = std::mem::size_of::(); - let meta_json = unsafe { - String::from_utf8_unchecked(fragment_data.get_meta()[ROW_HEADER_SIZE..].to_vec()) - }; - let meta = - serde_json::from_slice(&fragment_data.get_meta()[ROW_HEADER_SIZE..]).map_err(|_| { - ErrorCode::BadBytes(format!( - "block meta deserialize error when exchange {}", - unsafe { - String::from_utf8_unchecked( - fragment_data.get_meta()[ROW_HEADER_SIZE..].to_vec(), - ) - } - )) - })?; + let meta = serde_json::from_slice(&fragment_data.get_meta()[ROW_HEADER_SIZE..]) + .map_err(|_| ErrorCode::BadBytes("block meta deserialize error when exchange"))?; let mut row_count_meta = &fragment_data.get_meta()[..ROW_HEADER_SIZE]; let row_count: u32 = row_count_meta.read_scalar()?; if row_count == 0 { - eprintln!("recv block: null, meta: {}", meta_json); return Ok(DataBlock::new_with_meta(vec![], 0, meta)); } let data_block = deserialize_block(dict, fragment_data, &self.schema, self.arrow_schema.clone())?; - eprintln!("recv block: {:?}, meta: {}", data_block, meta_json); - // let data_block = match &meta { - // None => { - // deserialize_block(dict, fragment_data, &self.schema, self.arrow_schema.clone())? - // } - // Some(meta) => match AggregateSerdeMeta::downcast_ref_from(meta) { - // None => { - // deserialize_block(dict, fragment_data, &self.schema, self.arrow_schema.clone())? - // } - // Some(meta) => { - // return match meta.typ == BUCKET_TYPE { - // true => { - // let mut block = deserialize_block( - // dict, - // fragment_data, - // &self.schema, - // self.arrow_schema.clone(), - // )?; - // - // if meta.is_empty { - // block = block.slice(0..0); - // } - // - // Ok(DataBlock::empty_with_meta( - // AggregateMeta::create_serialized( - // meta.bucket, - // block, - // meta.max_partition_count, - // ), - // )) - // } - // false => { - // let data_schema = Arc::new(exchange_defines::spilled_schema()); - // let arrow_schema = Arc::new(exchange_defines::spilled_arrow_schema()); - // let data_block = deserialize_block( - // dict, - // fragment_data, - // &data_schema, - // arrow_schema.clone(), - // )?; - // - // let columns = data_block - // .columns() - // .iter() - // .map(|c| c.value.clone().into_column()) - // .try_collect::>() - // .unwrap(); - // - // let buckets = - // NumberType::::try_downcast_column(&columns[0]).unwrap(); - // let data_range_start = - // NumberType::::try_downcast_column(&columns[1]).unwrap(); - // let data_range_end = - // NumberType::::try_downcast_column(&columns[2]).unwrap(); - // - // let mut buckets_payload = Vec::with_capacity(data_block.num_rows()); - // for index in 0..data_block.num_rows() { - // unsafe { - // buckets_payload.push(BucketSpilledPayload { - // bucket: *buckets.get_unchecked(index) as isize, - // location: meta.location.clone().unwrap(), - // data_range: *data_range_start.get_unchecked(index) - // ..*data_range_end.get_unchecked(index), - // destination_node: String::new(), - // max_partition_count: meta.max_partition_count, - // }); - // } - // } - // - // Ok(DataBlock::empty_with_meta(AggregateMeta::create_spilled( - // buckets_payload, - // ))) - // } - // }; - // } - // }, - // }; - match data_block.num_columns() == 0 { true => Ok(DataBlock::new_with_meta(vec![], row_count as usize, meta)), false => data_block.add_meta(meta), 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 deleted file mode 100644 index d455b38af9c22..0000000000000 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_exchange_aggregate_serializer.rs +++ /dev/null @@ -1,288 +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 arrow_ipc::writer::IpcWriteOptions; -// use arrow_ipc::CompressionType; -// 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::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::BlockMetaInfoDowncast; -// use databend_common_expression::DataBlock; -// use databend_common_expression::DataSchemaRef; -// use databend_common_expression::FromData; -// use databend_common_expression::PartitionedPayload; -// 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_settings::FlightCompression; -// use futures_util::future::BoxFuture; -// use log::info; -// use opendal::Operator; -// -// 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::exchange_defines; -// 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::servers::flight::v1::exchange::serde::serialize_block; -// use crate::servers::flight::v1::exchange::ExchangeShuffleMeta; -// use crate::sessions::QueryContext; -// use crate::spillers::Spiller; -// use crate::spillers::SpillerConfig; -// use crate::spillers::SpillerType; -// -// pub struct TransformExchangeAggregateSerializer { -// ctx: Arc, -// local_pos: usize, -// options: IpcWriteOptions, -// -// params: Arc, -// spiller: Arc, -// } -// -// impl TransformExchangeAggregateSerializer { -// #[allow(clippy::too_many_arguments)] -// pub fn try_create( -// ctx: Arc, -// input: Arc, -// output: Arc, -// -// operator: Operator, -// location_prefix: String, -// params: Arc, -// compression: Option, -// _schema: DataSchemaRef, -// local_pos: usize, -// ) -> Result> { -// let compression = match compression { -// None => None, -// Some(compression) => match compression { -// FlightCompression::Lz4 => Some(CompressionType::LZ4_FRAME), -// FlightCompression::Zstd => Some(CompressionType::ZSTD), -// }, -// }; -// let config = SpillerConfig { -// spiller_type: SpillerType::Aggregation, -// location_prefix, -// disk_spill: None, -// use_parquet: ctx.get_settings().get_spilling_file_format()?.is_parquet(), -// }; -// -// let spiller = Spiller::create(ctx.clone(), operator, config.clone())?; -// Ok(BlockMetaTransformer::create( -// input, -// output, -// TransformExchangeAggregateSerializer { -// ctx, -// params, -// local_pos, -// spiller: spiller.into(), -// options: IpcWriteOptions::default() -// .try_with_compression(compression) -// .unwrap(), -// }, -// )) -// } -// } -// -// impl BlockMetaTransform for TransformExchangeAggregateSerializer { -// const NAME: &'static str = "TransformExchangeAggregateSerializer"; -// -// 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::Serialized(_)) => unreachable!(), -// Some(AggregateMeta::BucketSpilled(_)) => unreachable!(), -// Some(AggregateMeta::Partitioned { .. }) => unreachable!(), -// Some(AggregateMeta::AggregateSpilling(payload)) => { -// serialized_blocks.push(FlightSerialized::Future( -// match index == self.local_pos { -// true => local_agg_spilling_aggregate_payload( -// self.ctx.clone(), -// self.spiller.clone(), -// payload, -// )?, -// false => exchange_agg_spilling_aggregate_payload( -// self.ctx.clone(), -// self.spiller.clone(), -// payload, -// )?, -// }, -// )); -// } -// -// Some(AggregateMeta::AggregatePayload(p)) => { -// let (bucket, max_partition_count) = (p.partition, p.max_partition_count); -// -// if index == self.local_pos { -// serialized_blocks.push(FlightSerialized::DataBlock( -// block.add_meta(Some(Box::new(AggregateMeta::AggregatePayload(p))))?, -// )); -// continue; -// } -// -// let block_number = compute_block_number(bucket, max_partition_count)?; -// let stream = SerializeAggregateStream::create( -// &self.params, -// SerializePayload::AggregatePayload(p), -// ); -// let mut stream_blocks = stream.into_iter().collect::>>()?; -// debug_assert!(!stream_blocks.is_empty()); -// let mut c = DataBlock::concat(&stream_blocks)?; -// if let Some(meta) = stream_blocks[0].take_meta() { -// c.replace_meta(meta); -// } -// let c = serialize_block(block_number, c, &self.options)?; -// serialized_blocks.push(FlightSerialized::DataBlock(c)); -// } -// }; -// } -// -// Ok(vec![DataBlock::empty_with_meta( -// FlightSerializedMeta::create(serialized_blocks), -// )]) -// } -// } -// -// fn exchange_agg_spilling_aggregate_payload( -// ctx: Arc, -// spiller: Arc, -// partitioned_payload: PartitionedPayload, -// ) -> Result>> { -// let partition_count = partitioned_payload.partition_count(); -// let mut write_size = 0; -// 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.aggregate_flush_all()?; -// 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 (location, write_bytes) = spiller -// .spill_stream_aggregate_buffer(None, write_data) -// .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_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()) -// })) -// } diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_exchange_async_barrier.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_exchange_async_barrier.rs deleted file mode 100644 index 1628bc9af5beb..0000000000000 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_exchange_async_barrier.rs +++ /dev/null @@ -1,73 +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 databend_common_exception::ErrorCode; -use databend_common_exception::Result; -use databend_common_expression::BlockMetaInfoDowncast; -use databend_common_expression::DataBlock; -use databend_common_pipeline_core::processors::InputPort; -use databend_common_pipeline_core::processors::OutputPort; -use databend_common_pipeline_core::processors::ProcessorPtr; -use databend_common_pipeline_transforms::processors::AsyncTransform; -use databend_common_pipeline_transforms::processors::AsyncTransformer; - -use crate::pipelines::processors::transforms::aggregator::FlightSerialized; -use crate::pipelines::processors::transforms::aggregator::FlightSerializedMeta; -use crate::servers::flight::v1::exchange::ExchangeShuffleMeta; - -pub struct TransformExchangeAsyncBarrier; - -impl TransformExchangeAsyncBarrier { - pub fn try_create(input: Arc, output: Arc) -> Result { - Ok(ProcessorPtr::create(AsyncTransformer::create( - input, - output, - TransformExchangeAsyncBarrier {}, - ))) - } -} - -#[async_trait::async_trait] -impl AsyncTransform for TransformExchangeAsyncBarrier { - const NAME: &'static str = "TransformExchangeAsyncBarrier"; - - async fn transform(&mut self, mut data: DataBlock) -> Result { - if let Some(meta) = data - .take_meta() - .and_then(FlightSerializedMeta::downcast_from) - { - let mut futures = Vec::with_capacity(meta.serialized_blocks.len()); - - for serialized_block in meta.serialized_blocks { - futures.push(databend_common_base::runtime::spawn(async move { - match serialized_block { - FlightSerialized::DataBlock(v) => Ok(v), - FlightSerialized::Future(f) => f.await, - } - })); - } - - return match futures::future::try_join_all(futures).await { - Err(_) => Err(ErrorCode::TokioError("Cannot join tokio job")), - Ok(spilled_data) => Ok(DataBlock::empty_with_meta(ExchangeShuffleMeta::create( - spilled_data.into_iter().collect::>>()?, - ))), - }; - } - - Err(ErrorCode::Internal("")) - } -} 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 fbc3eb485bf4c..e40d8dde3ade4 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 @@ -40,8 +40,8 @@ use tokio::sync::Semaphore; 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::SerializedPayload; +use crate::pipelines::processors::transforms::aggregator::SpilledPayload; type DeserializingMeta = (AggregateMeta, VecDeque>); @@ -101,17 +101,18 @@ impl Processor for TransformSpillReader { .get_meta() .and_then(AggregateMeta::downcast_ref_from) { - if matches!(block_meta, AggregateMeta::BucketSpilled(_)) { + if matches!(block_meta, AggregateMeta::SpilledPayload(_)) { self.input.set_not_need_data(); let block_meta = data_block.take_meta().unwrap(); self.reading_meta = AggregateMeta::downcast_from(block_meta); return Ok(Event::Async); } - if let AggregateMeta::Partitioned { data, .. } = block_meta { - if data + if let AggregateMeta::FinalPayload(payload) = block_meta { + if payload + .data .iter() - .any(|meta| matches!(meta, AggregateMeta::BucketSpilled(_))) + .any(|(meta, _)| matches!(meta, AggregateMeta::SpilledPayload(_))) { self.input.set_not_need_data(); let block_meta = data_block.take_meta().unwrap(); @@ -139,30 +140,30 @@ impl Processor for TransformSpillReader { match meta { AggregateMeta::AggregatePayload(_) => unreachable!(), AggregateMeta::Serialized(_) => unreachable!(), - AggregateMeta::BucketSpilled(payload) => { + AggregateMeta::InFlightPayload(_) => unreachable!(), + AggregateMeta::SpilledPayload(payload) => { debug_assert!(read_data.len() == 1); let data = read_data.pop_front().unwrap(); self.deserialized_meta = Some(Box::new(self.deserialize(payload, data))); } - AggregateMeta::Partitioned { bucket, data } => { - let mut new_data = Vec::with_capacity(data.len()); + AggregateMeta::FinalPayload(payload) => { + let mut new_data = Vec::with_capacity(payload.data.len()); - for meta in data { - if matches!(&meta, AggregateMeta::BucketSpilled(_)) { - if let AggregateMeta::BucketSpilled(payload) = meta { + for (meta, block) in payload.data { + if matches!(&meta, AggregateMeta::SpilledPayload(_)) { + if let AggregateMeta::SpilledPayload(payload) = meta { let data = read_data.pop_front().unwrap(); - new_data.push(self.deserialize(payload, data)); + new_data.push((self.deserialize(payload, data), block)); } continue; } - new_data.push(meta); + new_data.push((meta, block)); } - self.deserialized_meta = - Some(AggregateMeta::create_partitioned(bucket, new_data)); + self.deserialized_meta = Some(AggregateMeta::create_final(new_data)); } } } @@ -176,7 +177,8 @@ impl Processor for TransformSpillReader { match &block_meta { AggregateMeta::AggregatePayload(_) => unreachable!(), AggregateMeta::Serialized(_) => unreachable!(), - AggregateMeta::BucketSpilled(payload) => { + AggregateMeta::InFlightPayload(_) => unreachable!(), + AggregateMeta::SpilledPayload(payload) => { let _guard = self.semaphore.acquire().await; let instant = Instant::now(); let data = self @@ -194,15 +196,16 @@ impl Processor for TransformSpillReader { self.deserializing_meta = Some((block_meta, VecDeque::from(vec![data]))); } - AggregateMeta::Partitioned { data, .. } => { + AggregateMeta::FinalPayload(payload) => { // For log progress. let mut total_elapsed = Duration::default(); let log_interval = 100; let mut processed_count = 0; - let mut read_data = Vec::with_capacity(data.len()); - for meta in data { - if let AggregateMeta::BucketSpilled(payload) = meta { + let mut read_data = Vec::with_capacity(payload.data.len()); + for (meta, _block) in &payload.data { + if let AggregateMeta::SpilledPayload(payload) = meta { + eprintln!("recv spill payload {:?}", payload); let location = payload.location.clone(); let operator = self.operator.clone(); let data_range = payload.data_range.clone(); @@ -296,7 +299,7 @@ impl TransformSpillReader { }))) } - fn deserialize(&self, payload: BucketSpilledPayload, data: Vec) -> AggregateMeta { + fn deserialize(&self, payload: SpilledPayload, data: Vec) -> AggregateMeta { let columns = self.params.group_data_types.len() + self.params.aggregate_functions.len(); let mut blocks = vec![]; @@ -312,13 +315,14 @@ impl TransformSpillReader { cursor = right; } - blocks.push(DataBlock::new_from_columns(block_columns)); + let block1 = DataBlock::new_from_columns(block_columns); + blocks.push(block1); } let block = DataBlock::concat(&blocks).unwrap(); AggregateMeta::Serialized(SerializedPayload { - bucket: payload.bucket, + bucket: payload.partition, data_block: block, max_partition_count: payload.max_partition_count, }) 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 e619947669ffd..18a966fd2d802 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 @@ -19,7 +19,10 @@ 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::Payload; use databend_common_expression::PayloadFlushState; +use databend_common_expression::ProbeState; use databend_common_pipeline_core::processors::InputPort; use databend_common_pipeline_core::processors::OutputPort; use databend_common_pipeline_core::processors::Processor; @@ -51,14 +54,52 @@ impl TransformFinalAggregate { )) } + fn deserialize_flight(&mut self, data: DataBlock) -> Result { + let rows_num = data.num_rows(); + let group_len = self.params.group_data_types.len(); + + let mut state = ProbeState::default(); + + // create single partition hash table for deserialize + let capacity = AggregateHashTable::get_capacity_for_count(rows_num); + let config = HashTableConfig::default().with_initial_radix_bits(0); + let mut hashtable = AggregateHashTable::new_directly( + self.params.group_data_types.clone(), + self.params.aggregate_functions.clone(), + config, + capacity, + Arc::new(Bump::new()), + false, + ); + + let num_states = self.params.num_states(); + let states_index: Vec = (0..num_states).collect(); + let agg_states = InputColumns::new_block_proxy(&states_index, &data); + + let group_index: Vec = (num_states..(num_states + group_len)).collect(); + let group_columns = InputColumns::new_block_proxy(&group_index, &data); + + let _ = hashtable.add_groups( + &mut state, + group_columns, + &[(&[]).into()], + agg_states, + rows_num, + )?; + + hashtable.payload.mark_min_cardinality(); + assert_eq!(hashtable.payload.len(), 1); + Ok(hashtable.payload.payloads.pop().unwrap()) + } + 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 { + if let AggregateMeta::FinalPayload(payload) = meta { + for (bucket_data, block) in payload.data { match bucket_data { AggregateMeta::Serialized(payload) => match agg_hashtable.as_mut() { Some(ht) => { - debug_assert!(bucket == payload.bucket); + // debug_assert!(bucket == payload.bucket); let payload = payload.convert_to_partitioned_payload( self.params.group_data_types.clone(), @@ -70,7 +111,7 @@ impl TransformFinalAggregate { ht.combine_payloads(&payload, &mut self.flush_state)?; } None => { - debug_assert!(bucket == payload.bucket); + // debug_assert!(bucket == payload.bucket); agg_hashtable = Some(payload.convert_to_aggregate_table( self.params.group_data_types.clone(), self.params.aggregate_functions.clone(), @@ -81,13 +122,30 @@ impl TransformFinalAggregate { )?); } }, + AggregateMeta::InFlightPayload(_payload) => match agg_hashtable.as_mut() { + Some(ht) => { + let payload = self.deserialize_flight(block)?; + ht.combine_payload(&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(), + // self.params.num_states(), + // 0, + // Arc::new(Bump::new()), + // true, + // )?); + } + }, AggregateMeta::AggregatePayload(payload) => match agg_hashtable.as_mut() { Some(ht) => { - debug_assert!(bucket == payload.partition); + // debug_assert!(bucket == payload.partition); ht.combine_payload(&payload.payload, &mut self.flush_state)?; } None => { - debug_assert!(bucket == payload.partition); + // debug_assert!(bucket == payload.partition); let capacity = AggregateHashTable::get_capacity_for_count(payload.payload.len()); let mut hashtable = AggregateHashTable::new_with_capacity( @@ -101,7 +159,8 @@ impl TransformFinalAggregate { agg_hashtable = Some(hashtable); } }, - _ => unreachable!(), + AggregateMeta::FinalPayload(_) => unreachable!(), + AggregateMeta::SpilledPayload(_) => unreachable!(), } } } diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_aggregate_merge.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_aggregate_merge.rs new file mode 100644 index 0000000000000..2d7d48b7ff94e --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_aggregate_merge.rs @@ -0,0 +1,17 @@ +// 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. + +pub struct TransformAggregateMerge { + +} 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 1a2a9adaf5769..125ea366a23f9 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 @@ -24,7 +24,6 @@ use databend_common_base::base::convert_number_size; use databend_common_catalog::plan::AggIndexMeta; use databend_common_catalog::table_context::TableContext; use databend_common_config::GlobalConfig; -use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::arrow::write_column; use databend_common_expression::AggregateHashTable; @@ -47,7 +46,7 @@ use opendal::Operator; use crate::pipelines::memory_settings::MemorySettingsExt; use crate::pipelines::processors::transforms::aggregator::aggregate_meta::AggregateMeta; use crate::pipelines::processors::transforms::aggregator::AggregatorParams; -use crate::pipelines::processors::transforms::aggregator::BucketSpilledPayload; +use crate::pipelines::processors::transforms::aggregator::SpilledPayload; use crate::sessions::QueryContext; use crate::spillers::SpillWriter; use crate::spillers::Spiller; @@ -358,16 +357,17 @@ impl AccumulatingTransform for TransformPartialAggregate { if let Some(writer) = spilling_state.writer.as_mut() { let last_offset = spilling_state.last_flush_partition_offset; if writer.write_bytes() > last_offset { + let spilled_payload = SpilledPayload { + partition: spilling_state.working_partition as isize, + location: writer.location(), + data_range: last_offset as u64..writer.write_bytes() as u64, + destination_node: self.configure_peer_nodes[spilling_state.working_bucket] + .clone(), + max_partition_count: max_partition, + }; + self.spill_blocks.push(DataBlock::empty_with_meta( - AggregateMeta::create_bucket_spilled(BucketSpilledPayload { - bucket: spilling_state.working_partition as isize, - location: writer.location(), - data_range: last_offset as u64..writer.write_bytes() as u64, - destination_node: self.configure_peer_nodes - [spilling_state.working_bucket] - .clone(), - max_partition_count: max_partition, - }), + AggregateMeta::create_spilled_payload(spilled_payload), )); spilling_state.last_flush_partition_offset = writer.write_bytes(); @@ -383,6 +383,7 @@ impl AccumulatingTransform for TransformPartialAggregate { if let Some(writer) = spilling_state.writer.as_mut() { writer.complete().await?; spilling_state.writer = None; + spilling_state.last_flush_partition_offset = 0; } spilling_state.payload_idx = 0; 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 new file mode 100644 index 0000000000000..3896e70019304 --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_bucket.rs @@ -0,0 +1,738 @@ +// 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::hash_map::Entry; +use std::collections::BTreeMap; +use std::collections::HashMap; +use std::fmt::Debug; +use std::fmt::Formatter; +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::BlockMetaInfoDowncast; +use databend_common_expression::DataBlock; +use databend_common_expression::HashTableConfig; +use databend_common_expression::InputColumns; +use databend_common_expression::PartitionedPayload; +use databend_common_expression::Payload; +use databend_common_expression::PayloadFlushState; +use databend_common_expression::ProbeState; +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::AggregatorParams; + +static SINGLE_LEVEL_BUCKET_NUM: isize = -1; +static MAX_PARTITION_COUNT: usize = 128; + +struct InputPortState { + port: Arc, + partition: isize, + max_partition: usize, +} + +impl Debug for InputPortState { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + f.debug_struct("InputPortState") + .field("bucket", &self.partition) + .field("max_partition_count", &self.max_partition) + .finish() + } +} + +pub struct TransformPartitionDispatch { + output: Arc, + inputs: Vec, + max_partition: usize, + initialized_all_inputs: bool, + partitions: Partitions, +} + +impl TransformPartitionDispatch { + pub fn create(input_nums: usize, params: Arc) -> Result { + let mut inputs = Vec::with_capacity(input_nums); + + for _index in 0..input_nums { + inputs.push(InputPortState { + partition: -1, + port: InputPort::create(), + max_partition: 0, + }); + } + + let max_partition = match params.cluster_aggregator { + true => MAX_PARTITION_COUNT, + false => 0, + }; + + Ok(TransformPartitionDispatch { + inputs, + max_partition, + output: OutputPort::create(), + initialized_all_inputs: false, + partitions: Partitions::create_unaligned(params), + }) + } + + 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() + } + + // Align each input's max_partition to the maximum max_partition. + // If an input's max_partition is smaller than the maximum, continuously fetch its data until either the stream ends or its max_partition reaches/exceeds the maximum value. + fn initialize_all_inputs(&mut self) -> Result { + let mut initialized_all_inputs = true; + + for index in 0..self.inputs.len() { + if self.inputs[index].port.is_finished() { + self.inputs[index].partition = self.max_partition as isize; + continue; + } + + if self.inputs[index].max_partition > 0 + && self.inputs[index].partition > SINGLE_LEVEL_BUCKET_NUM + && self.inputs[index].max_partition == self.max_partition + { + continue; + } + + if !self.inputs[index].port.has_data() { + self.inputs[index].port.set_need_data(); + initialized_all_inputs = false; + continue; + } + + let before_max_partition_count = self.max_partition; + + self.add_block(index, self.inputs[index].port.pull_data().unwrap()?)?; + + // we need pull all spill data in init, and data less than max partition + if self.inputs[index].partition <= SINGLE_LEVEL_BUCKET_NUM + || self.inputs[index].max_partition < self.max_partition + { + self.inputs[index].port.set_need_data(); + initialized_all_inputs = false; + } + + // max partition count change + if before_max_partition_count > 0 && before_max_partition_count != self.max_partition { + // set need data for inputs which is less than the max partition + for i in 0..index { + if !self.inputs[i].port.is_finished() + && !self.inputs[i].port.has_data() + && self.inputs[i].max_partition != self.max_partition + { + self.inputs[i].port.set_need_data(); + initialized_all_inputs = false; + } + } + } + } + + Ok(initialized_all_inputs) + } + + fn add_block(&mut self, index: usize, data_block: DataBlock) -> Result<()> { + ( + self.inputs[index].partition, + self.inputs[index].max_partition, + ) = self.partitions.add_block(data_block)?; + + self.max_partition = std::cmp::max(self.max_partition, self.inputs[index].max_partition); + Ok(()) + } + + fn ready_partition(&mut self) -> Option { + let inputs_min_partition = self.working_partition()?; + let stroage_min_partition = self.partitions.min_partition()?; + + if stroage_min_partition >= inputs_min_partition { + return None; + } + + Some(stroage_min_partition) + } + + fn working_partition(&mut self) -> Option { + self.inputs.iter().map(|x| x.partition).min() + } +} + +#[async_trait::async_trait] +impl Processor for TransformPartitionDispatch { + fn name(&self) -> String { + String::from("TransformPartitionDispatch") + } + + 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(); + } + + return Ok(Event::Finished); + } + + // We pull the first unsplitted data block + if !self.initialized_all_inputs { + if self.initialize_all_inputs()? { + return Ok(Event::Sync); + } + + return Ok(Event::NeedData); + } + + if !self.output.can_push() { + for input_state in &self.inputs { + input_state.port.set_not_need_data(); + } + + return Ok(Event::NeedConsume); + } + + if let Some(ready_partition) = self.ready_partition() { + // TODO: read spill data + let ready_partition = self.partitions.take_partition(ready_partition); + self.output + .push_data(Ok(DataBlock::empty_with_meta(AggregateMeta::create_final( + ready_partition, + )))); + } + + let working_partition = self.working_partition().unwrap_or(0); + + let mut all_inputs_is_finished = true; + for index in 0..self.inputs.len() { + if self.inputs[index].port.is_finished() { + self.inputs[index].partition = self.max_partition as isize; + continue; + } + + all_inputs_is_finished = false; + + eprintln!( + "working partition: {}, input_{}:{}", + working_partition, index, self.inputs[index].partition + ); + + if self.inputs[index].partition > working_partition { + continue; + } + + if !self.inputs[index].port.has_data() { + self.inputs[index].port.set_need_data(); + continue; + } + + self.add_block(index, self.inputs[index].port.pull_data().unwrap()?)?; + + if self.inputs[index].partition <= working_partition { + self.inputs[index].port.set_need_data(); + } + } + + if let Some(ready_partition) = self.ready_partition() { + if self.output.can_push() { + let ready_partition = self.partitions.take_partition(ready_partition); + self.output + .push_data(Ok(DataBlock::empty_with_meta(AggregateMeta::create_final( + ready_partition, + )))); + } + + return Ok(Event::NeedConsume); + // TODO: read spill data + // TODO: try push + } + + if all_inputs_is_finished { + self.output.finish(); + return Ok(Event::Finished); + } + + Ok(Event::NeedData) + } + + fn process(&mut self) -> Result<()> { + if !self.initialized_all_inputs { + self.initialized_all_inputs = true; + return self.partitions.align(self.max_partition); + } + + Ok(()) + } +} + +pub fn build_partition_bucket( + pipeline: &mut Pipeline, + params: Arc, +) -> Result<()> { + let input_nums = pipeline.output_len(); + let transform = TransformPartitionDispatch::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().spill_operator(); + pipeline.add_transform(|input, output| { + let operator = operator.clone(); + TransformAggregateSpillReader::create( + input, + output, + operator, + semaphore.clone(), + params.clone(), + ) + })?; + + pipeline.add_transform(|input, output| { + Ok(ProcessorPtr::create(TransformFinalAggregate::try_create( + input, + output, + params.clone(), + )?)) + })?; + Ok(()) +} + +struct UnalignedPartitions { + params: Arc, + data: HashMap>, +} + +impl Debug for UnalignedPartitions { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + f.debug_struct("UnalignedPartitions") + .field("data", &self.data) + .finish() + } +} + +impl UnalignedPartitions { + pub fn create(params: Arc) -> UnalignedPartitions { + UnalignedPartitions { + params, + data: HashMap::new(), + } + } + + fn insert_data(&mut self, idx: usize, meta: AggregateMeta, block: DataBlock) { + match self.data.entry(idx) { + Entry::Vacant(v) => { + v.insert(vec![(meta, block)]); + } + Entry::Occupied(mut v) => { + v.get_mut().push((meta, block)); + } + } + } + + pub fn add_data(&mut self, meta: AggregateMeta, block: DataBlock) -> (isize, usize) { + match &meta { + AggregateMeta::Serialized(_) => unreachable!(), + AggregateMeta::FinalPayload(_) => unreachable!(), + AggregateMeta::SpilledPayload(payload) => { + let max_partition = payload.max_partition_count; + self.insert_data(max_partition, meta, block); + + (SINGLE_LEVEL_BUCKET_NUM, max_partition) + } + AggregateMeta::InFlightPayload(payload) => { + let partition = payload.partition; + let max_partition = payload.max_partition; + self.insert_data(max_partition, meta, block); + + (partition, max_partition) + } + AggregateMeta::AggregatePayload(payload) => { + let partition = payload.partition; + let max_partition = payload.max_partition_count; + self.insert_data(max_partition, meta, block); + + (partition, max_partition) + } + } + } + + fn deserialize_flight(&mut self, data: DataBlock) -> Result { + let rows_num = data.num_rows(); + let group_len = self.params.group_data_types.len(); + + let mut state = ProbeState::default(); + + // create single partition hash table for deserialize + let capacity = AggregateHashTable::get_capacity_for_count(rows_num); + let config = HashTableConfig::default().with_initial_radix_bits(0); + let mut hashtable = AggregateHashTable::new_directly( + self.params.group_data_types.clone(), + self.params.aggregate_functions.clone(), + config, + capacity, + Arc::new(Bump::new()), + false, + ); + + let num_states = self.params.num_states(); + let states_index: Vec = (0..num_states).collect(); + let agg_states = InputColumns::new_block_proxy(&states_index, &data); + + let group_index: Vec = (num_states..(num_states + group_len)).collect(); + let group_columns = InputColumns::new_block_proxy(&group_index, &data); + + let _ = hashtable.add_groups( + &mut state, + group_columns, + &[(&[]).into()], + agg_states, + rows_num, + )?; + + hashtable.payload.mark_min_cardinality(); + assert_eq!(hashtable.payload.len(), 1); + Ok(hashtable.payload.payloads.pop().unwrap()) + } + + fn partition_payload(&mut self, from: AggregatePayload, to: usize) -> Vec { + let mut partitioned = Vec::with_capacity(to); + let mut partitioned_payload = PartitionedPayload::new( + self.params.group_data_types.clone(), + self.params.aggregate_functions.clone(), + to as u64, + vec![from.payload.arena.clone()], + ); + + let mut flush_state = PayloadFlushState::default(); + partitioned_payload.combine_single(from.payload, &mut flush_state, None); + + for (partition, payload) in partitioned_payload.payloads.into_iter().enumerate() { + partitioned.push(AggregatePayload { + payload, + partition: partition as isize, + max_partition_count: to, + }); + } + + partitioned + } + + pub fn align(mut self, max_partitions: usize) -> Result { + let repartition_data = self + .data + .extract_if(|k, _| *k != max_partitions) + .collect::>(); + + let mut aligned_partitions = AlignedPartitions { + max_partition: max_partitions, + data: BTreeMap::new(), + }; + + for (_max_partition, data) in std::mem::take(&mut self.data) { + for (meta, block) in data { + aligned_partitions.add_data(meta, block); + } + } + + for (_, repartition_data) in repartition_data { + for (meta, block) in repartition_data { + match meta { + AggregateMeta::Serialized(_) => unreachable!(), + AggregateMeta::FinalPayload(_) => unreachable!(), + AggregateMeta::SpilledPayload(_) => unreachable!(), + AggregateMeta::InFlightPayload(payload) => { + let payload = AggregatePayload { + partition: payload.partition, + max_partition_count: payload.max_partition, + payload: self.deserialize_flight(block)?, + }; + + let partitioned = self.partition_payload(payload, max_partitions); + + for payload in partitioned { + aligned_partitions.add_data( + AggregateMeta::AggregatePayload(payload), + DataBlock::empty(), + ); + } + } + AggregateMeta::AggregatePayload(payload) => { + let partitioned = self.partition_payload(payload, max_partitions); + for payload in partitioned { + aligned_partitions.add_data( + AggregateMeta::AggregatePayload(payload), + DataBlock::empty(), + ); + } + } + } + } + } + + Ok(aligned_partitions) + } +} + +#[derive(Debug)] +struct AlignedPartitions { + max_partition: usize, + data: BTreeMap>, +} + +impl AlignedPartitions { + pub fn add_data(&mut self, meta: AggregateMeta, block: DataBlock) -> (isize, usize) { + let (partition, max_partition) = match &meta { + AggregateMeta::Serialized(_) => unreachable!(), + AggregateMeta::FinalPayload(_) => unreachable!(), + AggregateMeta::SpilledPayload(v) => (v.partition, v.max_partition_count), + AggregateMeta::AggregatePayload(v) => (v.partition, v.max_partition_count), + AggregateMeta::InFlightPayload(v) => (v.partition, v.max_partition), + }; + + assert_eq!(max_partition, self.max_partition); + match self.data.entry(partition) { + std::collections::btree_map::Entry::Vacant(v) => { + v.insert(vec![(meta, block)]); + } + std::collections::btree_map::Entry::Occupied(mut v) => { + v.get_mut().push((meta, block)); + } + } + + (partition, max_partition) + } +} + +#[derive(Debug)] +enum Partitions { + Aligned(AlignedPartitions), + Unaligned(UnalignedPartitions), +} + +impl Partitions { + pub fn create_unaligned(params: Arc) -> Partitions { + Partitions::Unaligned(UnalignedPartitions::create(params)) + } + + fn add_data(&mut self, meta: AggregateMeta, block: DataBlock) -> (isize, usize) { + match self { + Partitions::Aligned(v) => v.add_data(meta, block), + Partitions::Unaligned(v) => v.add_data(meta, block), + } + } + + pub fn add_block(&mut self, mut block: DataBlock) -> Result<(isize, usize)> { + let Some(meta) = block.take_meta() else { + return Err(ErrorCode::Internal( + "Internal, TransformPartitionBucket only recv DataBlock with meta.", + )); + }; + + let Some(meta) = AggregateMeta::downcast_from(meta) else { + return Err(ErrorCode::Internal( + "Internal, TransformPartitionBucket only recv AggregateMeta".to_string(), + )); + }; + + Ok(self.add_data(meta, block)) + } + + pub fn min_partition(&self) -> Option { + match self { + Partitions::Unaligned(_) => unreachable!(), + Partitions::Aligned(v) => v.data.keys().min().cloned(), + } + } + + pub fn take_partition(&mut self, partition: isize) -> Vec<(AggregateMeta, DataBlock)> { + match self { + Partitions::Unaligned(_) => unreachable!(), + Partitions::Aligned(v) => v.data.remove(&partition).unwrap_or_default(), + } + } + + pub fn align(&mut self, max_partitions: usize) -> Result<()> { + let mut partitions = match self { + Partitions::Aligned(_) => { + return Ok(()); + } + Partitions::Unaligned(v) => Self::create_unaligned(v.params.clone()), + }; + + std::mem::swap(self, &mut partitions); + + *self = match partitions { + Partitions::Aligned(_) => unreachable!(), + Partitions::Unaligned(v) => Partitions::Aligned(v.align(max_partitions)?), + }; + + Ok(()) + } +} + +#[cfg(test)] +mod tests { + use databend_common_expression::types::DataType; + use databend_common_expression::types::NumberDataType; + use databend_common_expression::DataBlock; + use databend_common_expression::DataField; + use databend_common_expression::DataSchemaRefExt; + use databend_common_functions::aggregates::AggregateFunctionFactory; + + use crate::pipelines::processors::transforms::aggregator::transform_partition_bucket::UnalignedPartitions; + use crate::pipelines::processors::transforms::aggregator::transform_partition_bucket::SINGLE_LEVEL_BUCKET_NUM; + use crate::pipelines::processors::transforms::aggregator::AggregateMeta; + use crate::pipelines::processors::transforms::aggregator::AggregatorParams; + use crate::pipelines::processors::transforms::aggregator::InFlightPayload; + use crate::pipelines::processors::transforms::aggregator::SpilledPayload; + + fn create_unaligned_partitions() -> UnalignedPartitions { + let schema = DataSchemaRefExt::create(vec![ + DataField::new("a", DataType::Number(NumberDataType::Int16)), + DataField::new("b", DataType::Number(NumberDataType::Float32)), + DataField::new("c", DataType::String), + ]); + + let aggregate_functions = vec![AggregateFunctionFactory::instance() + .get("count", vec![], vec![], vec![]) + .unwrap()]; + + let params = AggregatorParams::try_create( + schema, + vec![ + DataType::Number(NumberDataType::Int16), + DataType::Number(NumberDataType::Float32), + DataType::String, + ], + &[0, 1, 2], + &aggregate_functions, + &[], + true, + false, + 1024, + 1024, + ); + + UnalignedPartitions::create(params.unwrap()) + } + + #[test] + fn test_add_data_spilled_payload() { + let mut partitions = create_unaligned_partitions(); + let max_partition = 5; + let meta = AggregateMeta::SpilledPayload(SpilledPayload { + partition: 0, + location: "".to_string(), + data_range: Default::default(), + destination_node: "".to_string(), + max_partition_count: max_partition, + }); + + let result = partitions.add_data(meta, DataBlock::empty()); + + assert_eq!(result, (SINGLE_LEVEL_BUCKET_NUM, max_partition)); + assert_eq!(partitions.data.get(&max_partition).unwrap().len(), 1); + } + + #[test] + fn test_add_data_in_flight_payload() { + let mut partitions = create_unaligned_partitions(); + let partition = 2; + let max_partition = 8; + let meta = AggregateMeta::InFlightPayload(InFlightPayload { + partition, + max_partition, + }); + + let result = partitions.add_data(meta, DataBlock::empty()); + + assert_eq!(result, (partition, max_partition)); + assert_eq!(partitions.data.get(&max_partition).unwrap().len(), 1); + } + + #[test] + fn test_add_data_aggregate_payload() { + // let mut partitions = create_unaligned_partitions(); + // let partition = 3; + // let max_partition = 10; + // // Payload::new() + // let meta = AggregateMeta::AggregatePayload(AggregatePayload { + // partition, + // // payload: Payload {}, + // max_partition_count: max_partition, + // }); + // + // let result = partitions.add_data(meta, DataBlock::empty()); + // + // assert_eq!(result, (partition, max_partition)); + // assert_eq!(partitions.data.get(&max_partition).unwrap().len(), 1); + } + + // #[test] + // fn test_multiple_inserts_same_partition() { + // let mut container = YourContainerStruct::new(); + // let max_partition = 5; + // + // let meta1 = AggregateMeta::SpilledPayload(SpilledPayload { + // max_partition_count: max_partition, + // // ... + // }); + // container.add_data(meta1, DataBlock); + // + // let meta2 = AggregateMeta::SpilledPayload(SpilledPayload { + // max_partition_count: max_partition, + // // ... + // }); + // container.add_data(meta2, DataBlock); + // + // assert_eq!(container.data.get(&max_partition).unwrap().len(), 2); + // } +} diff --git a/src/query/service/src/servers/flight/v1/exchange/exchange_injector.rs b/src/query/service/src/servers/flight/v1/exchange/exchange_injector.rs index c6e7eacf23858..2355eb8b27428 100644 --- a/src/query/service/src/servers/flight/v1/exchange/exchange_injector.rs +++ b/src/query/service/src/servers/flight/v1/exchange/exchange_injector.rs @@ -16,16 +16,9 @@ use std::sync::Arc; use databend_common_catalog::table_context::TableContext; use databend_common_exception::Result; -use databend_common_pipeline_core::Pipeline; -use databend_common_settings::FlightCompression; -use super::exchange_params::MergeExchangeParams; -use crate::servers::flight::v1::exchange::serde::TransformExchangeDeserializer; -use crate::servers::flight::v1::exchange::serde::TransformExchangeSerializer; -use crate::servers::flight::v1::exchange::serde::TransformScatterExchangeSerializer; use crate::servers::flight::v1::exchange::DataExchange; use crate::servers::flight::v1::exchange::ExchangeSorting; -use crate::servers::flight::v1::exchange::ShuffleExchangeParams; use crate::servers::flight::v1::scatter::BroadcastFlightScatter; use crate::servers::flight::v1::scatter::FlightScatter; use crate::servers::flight::v1::scatter::HashFlightScatter; diff --git a/src/query/service/src/servers/flight/v1/exchange/exchange_manager.rs b/src/query/service/src/servers/flight/v1/exchange/exchange_manager.rs index d158ccf3c9b89..c427b1b20d1bb 100644 --- a/src/query/service/src/servers/flight/v1/exchange/exchange_manager.rs +++ b/src/query/service/src/servers/flight/v1/exchange/exchange_manager.rs @@ -63,7 +63,6 @@ use crate::servers::flight::v1::actions::init_query_fragments; use crate::servers::flight::v1::actions::INIT_QUERY_FRAGMENTS; use crate::servers::flight::v1::actions::START_PREPARED_QUERY; use crate::servers::flight::v1::exchange::DataExchange; -use crate::servers::flight::v1::exchange::DefaultExchangeInjector; use crate::servers::flight::v1::exchange::ExchangeInjector; use crate::servers::flight::v1::packets::Edge; use crate::servers::flight::v1::packets::QueryEnv; @@ -470,9 +469,7 @@ impl DataExchangeManager { None => Err(ErrorCode::Internal("Query not exists.")), Some(query_coordinator) => { assert!(query_coordinator.fragment_exchanges.is_empty()); - let injector = DefaultExchangeInjector::create(); - let mut build_res = - query_coordinator.subscribe_fragment(&ctx, fragment_id, injector)?; + let mut build_res = query_coordinator.subscribe_fragment(&ctx, fragment_id)?; let exchanges = std::mem::take(&mut query_coordinator.statistics_exchanges); let statistics_receiver = StatisticsReceiver::spawn_receiver(&ctx, exchanges)?; @@ -533,7 +530,6 @@ impl DataExchangeManager { &self, query_id: &str, fragment_id: usize, - injector: Arc, ) -> Result { let queries_coordinator_guard = self.queries_coordinator.lock(); let queries_coordinator = unsafe { &mut *queries_coordinator_guard.deref().get() }; @@ -548,7 +544,7 @@ impl DataExchangeManager { .query_ctx .clone(); - query_coordinator.subscribe_fragment(&query_ctx, fragment_id, injector) + query_coordinator.subscribe_fragment(&query_ctx, fragment_id) } } } @@ -735,7 +731,6 @@ impl QueryCoordinator { &mut self, ctx: &Arc, fragment_id: usize, - injector: Arc, ) -> Result { // Merge pipelines if exist locally pipeline if let Some(mut fragment_coordinator) = self.fragments_coordinator.remove(&fragment_id) { @@ -768,12 +763,7 @@ impl QueryCoordinator { // Add exchange data transform. - ExchangeTransform::via( - ctx, - &exchange_params, - &mut build_res.main_pipeline, - injector, - )?; + ExchangeTransform::via(ctx, &exchange_params, &mut build_res.main_pipeline)?; return Ok(build_res); } diff --git a/src/query/service/src/servers/flight/v1/exchange/exchange_sink.rs b/src/query/service/src/servers/flight/v1/exchange/exchange_sink.rs index 170f56c87aaf3..fafde8933cdcd 100644 --- a/src/query/service/src/servers/flight/v1/exchange/exchange_sink.rs +++ b/src/query/service/src/servers/flight/v1/exchange/exchange_sink.rs @@ -14,7 +14,6 @@ use std::sync::Arc; -use databend_common_config::GlobalConfig; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::BlockMetaInfoDowncast; @@ -64,9 +63,9 @@ impl ExchangeSink { let settings = ctx.get_settings(); let compression = settings.get_query_flight_compression()?; - let nodes = vec![GlobalConfig::instance().query.node_id.clone()]; + let nodes = vec![]; pipeline.exchange( - nodes.len(), + 1, FlightExchange::create( nodes, compression, diff --git a/src/query/service/src/servers/flight/v1/exchange/exchange_sink_writer.rs b/src/query/service/src/servers/flight/v1/exchange/exchange_sink_writer.rs index abebc2ba6a254..704359391bc0a 100644 --- a/src/query/service/src/servers/flight/v1/exchange/exchange_sink_writer.rs +++ b/src/query/service/src/servers/flight/v1/exchange/exchange_sink_writer.rs @@ -69,9 +69,10 @@ impl AsyncSink for ExchangeWriterSink { #[async_backtrace::framed] async fn consume(&mut self, mut data_block: DataBlock) -> Result { let serialize_meta = match data_block.take_meta() { - None => Err(ErrorCode::Internal( - "ExchangeWriterSink only recv ExchangeSerializeMeta.", - )), + None => Err(ErrorCode::Internal(format!( + "ExchangeWriterSink only recv ExchangeSerializeMeta. {:?}", + data_block + ))), Some(block_meta) => ExchangeSerializeMeta::downcast_from(block_meta).ok_or_else(|| { ErrorCode::Internal("ExchangeWriterSink only recv ExchangeSerializeMeta.") }), diff --git a/src/query/service/src/servers/flight/v1/exchange/exchange_source.rs b/src/query/service/src/servers/flight/v1/exchange/exchange_source.rs index 840d221daf36c..2d4bcdef3d32a 100644 --- a/src/query/service/src/servers/flight/v1/exchange/exchange_source.rs +++ b/src/query/service/src/servers/flight/v1/exchange/exchange_source.rs @@ -29,14 +29,12 @@ use super::exchange_params::MergeExchangeParams; use super::exchange_source_reader::ExchangeSourceReader; use crate::clusters::ClusterHelper; use crate::pipelines::processors::transforms::aggregator::TransformAggregateDeserializer; -use crate::servers::flight::v1::exchange::ExchangeInjector; use crate::sessions::QueryContext; /// Add Exchange Source to the pipeline. pub fn via_exchange_source( ctx: Arc, params: &MergeExchangeParams, - injector: Arc, pipeline: &mut Pipeline, ) -> Result<()> { // UpstreamTransform ---> DummyTransform ---> DummyTransform ---> DownstreamTransform diff --git a/src/query/service/src/servers/flight/v1/exchange/exchange_transform.rs b/src/query/service/src/servers/flight/v1/exchange/exchange_transform.rs index 4aff81888783d..1fbc47179b729 100644 --- a/src/query/service/src/servers/flight/v1/exchange/exchange_transform.rs +++ b/src/query/service/src/servers/flight/v1/exchange/exchange_transform.rs @@ -28,7 +28,6 @@ use super::exchange_source_reader::create_reader_item; use super::exchange_transform_shuffle::exchange_shuffle; use crate::clusters::ClusterHelper; use crate::pipelines::processors::transforms::aggregator::TransformAggregateDeserializer; -use crate::servers::flight::v1::exchange::ExchangeInjector; use crate::sessions::QueryContext; pub struct ExchangeTransform; @@ -38,11 +37,10 @@ impl ExchangeTransform { ctx: &Arc, params: &ExchangeParams, pipeline: &mut Pipeline, - injector: Arc, ) -> Result<()> { match params { ExchangeParams::MergeExchange(params) => { - via_exchange_source(ctx.clone(), params, injector, pipeline) + via_exchange_source(ctx.clone(), params, pipeline) } ExchangeParams::ShuffleExchange(params) => { exchange_shuffle(ctx, params, pipeline)?; diff --git a/src/query/service/src/servers/flight/v1/exchange/exchange_transform_scatter.rs b/src/query/service/src/servers/flight/v1/exchange/exchange_transform_scatter.rs deleted file mode 100644 index 0b69270eab8e3..0000000000000 --- a/src/query/service/src/servers/flight/v1/exchange/exchange_transform_scatter.rs +++ /dev/null @@ -1,53 +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 databend_common_expression::DataBlock; -use databend_common_pipeline_core::processors::InputPort; -use databend_common_pipeline_core::processors::OutputPort; -use databend_common_pipeline_core::processors::ProcessorPtr; -use databend_common_pipeline_transforms::processors::Transform; -use databend_common_pipeline_transforms::processors::Transformer; - -use super::exchange_transform_shuffle::ExchangeShuffleMeta; -use crate::servers::flight::v1::scatter::FlightScatter; - -pub struct ScatterTransform { - scatter: Arc>, -} - -impl ScatterTransform { - pub fn create( - input: Arc, - output: Arc, - scatter: Arc>, - ) -> ProcessorPtr { - ProcessorPtr::create(Transformer::create(input, output, ScatterTransform { - scatter, - })) - } -} - -impl Transform for ScatterTransform { - const NAME: &'static str = "ScatterTransform"; - - fn transform(&mut self, data: DataBlock) -> databend_common_exception::Result { - let blocks = self.scatter.execute(data)?; - - Ok(DataBlock::empty_with_meta(ExchangeShuffleMeta::create( - blocks, - ))) - } -} diff --git a/src/query/service/src/servers/flight/v1/exchange/exchange_transform_shuffle.rs b/src/query/service/src/servers/flight/v1/exchange/exchange_transform_shuffle.rs index 4cfd4a89b83e8..c83b5ad8e9913 100644 --- a/src/query/service/src/servers/flight/v1/exchange/exchange_transform_shuffle.rs +++ b/src/query/service/src/servers/flight/v1/exchange/exchange_transform_shuffle.rs @@ -12,35 +12,19 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::any::Any; -use std::collections::VecDeque; use std::fmt::Debug; use std::fmt::Formatter; use std::sync::Arc; use databend_common_catalog::table_context::TableContext; -use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::local_block_meta_serde; use databend_common_expression::BlockMetaInfo; -use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::BlockMetaInfoPtr; use databend_common_expression::DataBlock; -use databend_common_pipeline_core::processors::Event; -use databend_common_pipeline_core::processors::EventCause; -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 super::exchange_params::ShuffleExchangeParams; -use super::exchange_sorting::ExchangeSorting; -use super::exchange_sorting::TransformExchangeSorting; -use super::exchange_transform_scatter::ScatterTransform; -use super::serde::ExchangeSerializeMeta; use crate::pipelines::processors::transforms::aggregator::FlightExchange; use crate::sessions::QueryContext; @@ -64,318 +48,6 @@ local_block_meta_serde!(ExchangeShuffleMeta); #[typetag::serde(name = "exchange_shuffle")] impl BlockMetaInfo for ExchangeShuffleMeta {} -// -// struct OutputsBuffer { -// inner: Vec>, -// } -// -// impl OutputsBuffer { -// pub fn create(capacity: usize, outputs: usize) -> OutputsBuffer { -// OutputsBuffer { -// inner: vec![capacity; outputs] -// .into_iter() -// .map(VecDeque::with_capacity) -// .collect::>(), -// } -// } -// -// pub fn is_all_empty(&self) -> bool { -// self.inner.iter().all(|x| x.is_empty()) -// } -// -// pub fn is_empty(&self, index: usize) -> bool { -// self.inner[index].is_empty() -// } -// -// pub fn is_full(&self) -> bool { -// self.inner.iter().any(|x| x.len() == x.capacity()) -// } -// -// pub fn clear(&mut self, index: usize) { -// self.inner[index].clear(); -// } -// -// pub fn pop(&mut self, index: usize) -> Option { -// self.inner[index].pop_front() -// } -// -// pub fn push_back(&mut self, index: usize, block: DataBlock) -> usize { -// self.inner[index].push_back(block); -// self.inner[index].len() -// } -// } -// -// #[derive(PartialEq)] -// enum PortStatus { -// Idle, -// HasData, -// NeedData, -// Finished, -// } -// -// struct PortWithStatus { -// pub status: PortStatus, -// pub port: Arc, -// } -// -// struct ExchangeShuffleTransform { -// initialized: bool, -// -// finished_inputs: usize, -// finished_outputs: usize, -// -// waiting_outputs: Vec, -// waiting_inputs: VecDeque, -// -// buffer: OutputsBuffer, -// inputs: Vec>, -// outputs: Vec>, -// } -// -// impl Processor for ExchangeShuffleTransform { -// fn name(&self) -> String { -// String::from("ExchangeShuffleTransform") -// } -// -// fn as_any(&mut self) -> &mut dyn Any { -// self -// } -// -// fn event_with_cause(&mut self, cause: EventCause) -> Result { -// if let EventCause::Output(output_index) = &cause { -// let output = &mut self.outputs[*output_index]; -// -// if output.port.is_finished() { -// if output.status != PortStatus::Finished { -// self.finished_outputs += 1; -// output.status = PortStatus::Finished; -// } -// -// self.buffer.clear(*output_index); -// -// self.wakeup_inputs(); -// self.wakeup_outputs(); -// } else if output.port.can_push() { -// if !self.buffer.is_empty(*output_index) { -// let data_block = self.buffer.pop(*output_index).unwrap(); -// output.status = PortStatus::Idle; -// output.port.push_data(Ok(data_block)); -// -// self.wakeup_inputs(); -// self.wakeup_outputs(); -// } else if output.status != PortStatus::NeedData { -// output.status = PortStatus::NeedData; -// self.waiting_outputs.push(*output_index); -// } -// } -// } -// -// if !self.initialized && !self.waiting_outputs.is_empty() { -// self.initialized = true; -// for input in &self.inputs { -// input.port.set_need_data(); -// } -// } -// -// if self.finished_outputs == self.outputs.len() { -// for input in &self.inputs { -// input.port.finish(); -// } -// -// return Ok(Event::Finished); -// } -// -// if let EventCause::Input(input_index) = &cause { -// let input = &mut self.inputs[*input_index]; -// -// if input.port.is_finished() { -// if input.status != PortStatus::Finished { -// self.finished_inputs += 1; -// input.status = PortStatus::Finished; -// } -// -// self.wakeup_outputs(); -// self.wakeup_inputs(); -// } else if input.port.has_data() { -// if !self.buffer.is_full() { -// self.take_input_data_into_buffer(*input_index); -// -// self.wakeup_outputs(); -// self.wakeup_inputs(); -// } else if input.status != PortStatus::HasData { -// input.status = PortStatus::HasData; -// self.waiting_inputs.push_back(*input_index); -// } -// } -// } -// -// if self.finished_outputs == self.outputs.len() { -// for input in &self.inputs { -// input.port.finish(); -// } -// -// return Ok(Event::Finished); -// } -// -// if self.finished_inputs == self.inputs.len() { -// for (index, output) in self.outputs.iter_mut().enumerate() { -// if self.buffer.is_empty(index) && output.status != PortStatus::Finished { -// self.finished_outputs += 1; -// output.status = PortStatus::Finished; -// output.port.finish(); -// } -// } -// -// if self.buffer.is_all_empty() { -// return Ok(Event::Finished); -// } -// } -// -// match self.waiting_outputs.is_empty() { -// true => Ok(Event::NeedConsume), -// false => Ok(Event::NeedData), -// } -// } -// -// fn details_status(&self) -> Option { -// #[derive(Debug)] -// #[allow(dead_code)] -// struct Display { -// queue_status: Vec<(usize, usize)>, -// inputs: usize, -// finished_inputs: usize, -// outputs: usize, -// finished_outputs: usize, -// -// waiting_outputs: Vec, -// waiting_inputs: VecDeque, -// } -// -// let mut queue_status = vec![]; -// for (idx, queue) in self.buffer.inner.iter().enumerate() { -// queue_status.push((idx, queue.len())); -// } -// -// Some(format!("{:?}", Display { -// queue_status, -// inputs: self.inputs.len(), -// outputs: self.outputs.len(), -// finished_inputs: self.finished_inputs, -// finished_outputs: self.finished_outputs, -// waiting_inputs: self.waiting_inputs.clone(), -// waiting_outputs: self.waiting_outputs.clone(), -// })) -// } -// } -// -// impl ExchangeShuffleTransform { -// fn wakeup_inputs(&mut self) { -// while !self.waiting_inputs.is_empty() && !self.buffer.is_full() { -// let input_index = self.waiting_inputs.pop_front().unwrap(); -// -// self.take_input_data_into_buffer(input_index); -// } -// } -// -// fn wakeup_outputs(&mut self) { -// let mut new_waiting_output = Vec::with_capacity(self.waiting_outputs.len()); -// -// for waiting_output in &self.waiting_outputs { -// let output = &mut self.outputs[*waiting_output]; -// -// if output.port.is_finished() { -// if output.status != PortStatus::Finished { -// self.finished_outputs += 1; -// output.status = PortStatus::Finished; -// } -// -// self.buffer.clear(*waiting_output); -// continue; -// } -// -// if self.buffer.is_empty(*waiting_output) { -// new_waiting_output.push(*waiting_output); -// continue; -// } -// -// let data_block = self.buffer.pop(*waiting_output).unwrap(); -// output.status = PortStatus::Idle; -// output.port.push_data(Ok(data_block)); -// } -// -// self.waiting_outputs = new_waiting_output; -// } -// -// fn take_input_data_into_buffer(&mut self, input_index: usize) { -// let input = &mut self.inputs[input_index]; -// -// input.status = PortStatus::Idle; -// let mut data_block = input.port.pull_data().unwrap().unwrap(); -// -// if let Some(block_meta) = data_block.take_meta() { -// if let Some(shuffle_meta) = ExchangeShuffleMeta::downcast_from(block_meta) { -// for (index, block) in shuffle_meta.blocks.into_iter().enumerate() { -// if (!block.is_empty() || block.get_meta().is_some()) -// && self.outputs[index].status != PortStatus::Finished -// { -// self.buffer.push_back(index, block); -// } -// } -// } -// } -// -// if input.port.is_finished() { -// if input.status != PortStatus::Finished { -// self.finished_inputs += 1; -// input.status = PortStatus::Finished; -// } -// -// return; -// } -// -// input.port.set_need_data(); -// } -// } - -// impl ExchangeShuffleTransform { -// pub fn create(inputs: usize, outputs: usize, buffer: usize) -> ExchangeShuffleTransform { -// let mut inputs_port = Vec::with_capacity(inputs); -// let mut outputs_port = Vec::with_capacity(outputs); -// -// for _index in 0..inputs { -// inputs_port.push(PortWithStatus { -// status: PortStatus::Idle, -// port: InputPort::create(), -// }); -// } -// -// for _index in 0..outputs { -// outputs_port.push(PortWithStatus { -// status: PortStatus::Idle, -// port: OutputPort::create(), -// }); -// } -// -// ExchangeShuffleTransform { -// initialized: false, -// finished_inputs: 0, -// finished_outputs: 0, -// inputs: inputs_port, -// outputs: outputs_port, -// buffer: OutputsBuffer::create(buffer, outputs), -// waiting_inputs: VecDeque::with_capacity(inputs), -// waiting_outputs: Vec::with_capacity(outputs), -// } -// } -// -// pub fn get_inputs(&self) -> Vec> { -// self.inputs.iter().map(|x| x.port.clone()).collect() -// } -// -// pub fn get_outputs(&self) -> Vec> { -// self.outputs.iter().map(|x| x.port.clone()).collect() -// } -// } // Scatter the data block and push it to the corresponding output port pub fn exchange_shuffle( @@ -401,79 +73,6 @@ pub fn exchange_shuffle( params.shuffle_scatter.clone(), ), ); - // append scatter transform - // pipeline.add_transform(|input, output| { - // Ok(ScatterTransform::create( - // input, - // output, - // params.shuffle_scatter.clone(), - // )) - // })?; - - // let exchange_injector = ¶ms.exchange_injector; - // - - // exchange_injector.apply_shuffle_serializer(params, compression, pipeline)?; - // - // let output_len = pipeline.output_len(); - // if let Some(exchange_sorting) = &exchange_injector.exchange_sorting() { - // let sorting = ShuffleExchangeSorting::create(exchange_sorting.clone()); - // let transform = TransformExchangeSorting::create(output_len, sorting); - // - // let output = transform.get_output(); - // let inputs = transform.get_inputs(); - // pipeline.add_pipe(Pipe::create(output_len, 1, vec![PipeItem::create( - // ProcessorPtr::create(Box::new(transform)), - // inputs, - // vec![output], - // )])); - // } - // - // let inputs_size = pipeline.output_len(); - // let outputs_size = params.destination_ids.len(); - // let transform = ExchangeShuffleTransform::create(inputs_size, outputs_size, output_len); - // - // let inputs = transform.get_inputs(); - // let outputs = transform.get_outputs(); - // pipeline.add_pipe(Pipe::create(inputs_size, outputs_size, vec![ - // PipeItem::create(ProcessorPtr::create(Box::new(transform)), inputs, outputs), - // ])); Ok(()) } - -struct ShuffleExchangeSorting { - inner: Arc, -} - -impl ShuffleExchangeSorting { - pub fn create(inner: Arc) -> Arc { - Arc::new(ShuffleExchangeSorting { inner }) - } -} - -impl ExchangeSorting for ShuffleExchangeSorting { - fn block_number(&self, data_block: &DataBlock) -> Result { - unimplemented!() - // let block_meta = data_block.get_meta(); - // let shuffle_meta = block_meta - // .and_then(ExchangeShuffleMeta::downcast_ref_from) - // .unwrap(); - // - // for block in &shuffle_meta.blocks { - // if let Some(block_meta) = block.get_meta() { - // if let Some(block_meta) = ExchangeSerializeMeta::downcast_ref_from(block_meta) { - // return Ok(block_meta.block_number); - // } - // } - // - // if !block.is_empty() || block.get_meta().is_some() { - // return self.inner.block_number(block); - // } - // } - // - // Err(ErrorCode::Internal( - // "Internal, ShuffleExchangeSorting only recv ExchangeSerializeMeta.", - // )) - } -} diff --git a/src/query/service/src/servers/flight/v1/exchange/mod.rs b/src/query/service/src/servers/flight/v1/exchange/mod.rs index 194f2cbe1e3e5..631968023ec35 100644 --- a/src/query/service/src/servers/flight/v1/exchange/mod.rs +++ b/src/query/service/src/servers/flight/v1/exchange/mod.rs @@ -22,7 +22,6 @@ mod exchange_sorting; mod exchange_source; mod exchange_source_reader; mod exchange_transform; -mod exchange_transform_scatter; mod exchange_transform_shuffle; mod statistics_receiver; mod statistics_sender; diff --git a/src/query/service/src/servers/flight/v1/exchange/serde/exchange_serializer.rs b/src/query/service/src/servers/flight/v1/exchange/serde/exchange_serializer.rs index 7e5e1b04a7881..945d2f09403ee 100644 --- a/src/query/service/src/servers/flight/v1/exchange/serde/exchange_serializer.rs +++ b/src/query/service/src/servers/flight/v1/exchange/serde/exchange_serializer.rs @@ -29,13 +29,11 @@ use arrow_schema::Schema as ArrowSchema; use bytes::Bytes; use databend_common_base::runtime::profile::Profile; use databend_common_base::runtime::profile::ProfileStatisticsName; -use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::local_block_meta_serde; use databend_common_expression::BlockMetaInfo; use databend_common_expression::BlockMetaInfoPtr; use databend_common_expression::DataBlock; -use databend_common_io::prelude::bincode_serialize_into_buf; use databend_common_io::prelude::BinaryWrite; use databend_common_pipeline_core::processors::InputPort; use databend_common_pipeline_core::processors::OutputPort; @@ -191,8 +189,6 @@ pub fn serialize_block( let mut meta = vec![]; meta.write_scalar_own(data_block.num_rows() as u32)?; serde_json::to_writer(&mut meta, &data_block.get_meta())?; - // bincode_serialize_into_buf(&mut meta, &data_block.get_meta()).map_err(|_| ErrorCode::BadBytes("block meta serialize error when exchange"))?; - // .map_err(|_| ErrorCode::BadBytes("block meta serialize error when exchange"))?; let (_, dict, values) = match data_block.is_empty() { true => batches_to_flight_data_with_options( From e03ce663797a434ecbf3083831ff0befb95df61c Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Thu, 13 Mar 2025 13:18:10 +0800 Subject: [PATCH 05/76] refactor(query): refactor aggreagte spill code --- src/query/pipeline/core/src/pipeline.rs | 6 +- .../core/src/processors/shuffle_processor.rs | 41 ++-- .../aggregator/aggregate_exchange_injector.rs | 65 +++--- .../transforms/aggregator/aggregate_meta.rs | 8 +- .../serde/transform_aggregate_serializer.rs | 2 +- .../serde/transform_spill_reader.rs | 145 ++----------- .../aggregator/transform_aggregate_final.rs | 196 ++++++++---------- .../aggregator/transform_aggregate_merge.rs | 17 -- .../aggregator/transform_partition_bucket.rs | 146 +++++++++---- 9 files changed, 264 insertions(+), 362 deletions(-) delete mode 100644 src/query/service/src/pipelines/processors/transforms/aggregator/transform_aggregate_merge.rs diff --git a/src/query/pipeline/core/src/pipeline.rs b/src/query/pipeline/core/src/pipeline.rs index 8072b7b997b88..bf9b3bc46468b 100644 --- a/src/query/pipeline/core/src/pipeline.rs +++ b/src/query/pipeline/core/src/pipeline.rs @@ -460,7 +460,11 @@ impl Pipeline { let input = InputPort::create(); let outputs: Vec<_> = (0..n).map(|_| OutputPort::create()).collect(); items.push(PipeItem::create( - PartitionProcessor::create(input.clone(), outputs.clone(), exchange.clone()), + PartitionProcessor::create( + input.clone(), + outputs.clone(), + exchange.clone(), + ), vec![input], outputs, )); diff --git a/src/query/pipeline/core/src/processors/shuffle_processor.rs b/src/query/pipeline/core/src/processors/shuffle_processor.rs index 99d4c602b0f83..cd37e953484a0 100644 --- a/src/query/pipeline/core/src/processors/shuffle_processor.rs +++ b/src/query/pipeline/core/src/processors/shuffle_processor.rs @@ -13,6 +13,7 @@ // limitations under the License. use std::any::Any; +use std::cmp::Ordering; use std::sync::Arc; use databend_common_exception::Result; @@ -37,7 +38,7 @@ pub trait Exchange: Send + Sync + 'static { fn partition(&self, data_block: DataBlock, n: usize) -> Result>; - fn multiway_pick(&self, _partitions: &[Option]) -> Result { + fn sorting_function(_: &DataBlock, _: &DataBlock) -> Ordering { unimplemented!() } } @@ -255,10 +256,6 @@ impl Processor for PartitionProcessor { let partitioned = self.exchange.partition(block, self.outputs.len())?; for (index, block) in partitioned.into_iter() { - if block.is_empty() && block.get_meta().is_none() { - continue; - } - self.partitioned_data[index] = Some(block); } } @@ -289,6 +286,22 @@ impl MergePartitionProcessor { inputs_data, })) } + + fn multiway_pick(&self, data_blocks: &[Option]) -> Option { + let position = data_blocks + .iter() + .enumerate() + .filter_map(|(idx, x)| x.as_ref().map(|d| (idx, d))) + .min_by(|(left_idx, left_block), (right_idx, right_block)| { + match T::sorting_function(left_block, right_block) { + Ordering::Less => Ordering::Less, + Ordering::Greater => Ordering::Greater, + Ordering::Equal => left_idx.cmp(right_idx), + } + }); + + position.map(|(idx, _)| idx) + } } impl Processor for MergePartitionProcessor { @@ -345,20 +358,20 @@ impl Processor for MergePartitionProcessor { input.set_need_data(); } + if need_pick_block_to_push { + if let Some(pick_index) = self.multiway_pick(&self.inputs_data) { + if let Some(block) = self.inputs_data[pick_index].take() { + self.output.push_data(Ok(block)); + return Ok(Event::NeedConsume); + } + } + } + if all_inputs_finished { self.output.finish(); return Ok(Event::Finished); } - if need_pick_block_to_push { - let pick_index = self.exchange.multiway_pick(&self.inputs_data)?; - - if let Some(block) = self.inputs_data[pick_index].take() { - self.output.push_data(Ok(block)); - return Ok(Event::NeedConsume); - } - } - Ok(Event::NeedData) } } 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 ef389c7150d81..9d703ee0ff31f 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 @@ -29,7 +29,6 @@ use databend_common_expression::PayloadFlushState; use databend_common_pipeline_core::processors::Exchange; use databend_common_pipeline_core::processors::MultiwayStrategy; use databend_common_settings::FlightCompression; -use itertools::Itertools; use crate::pipelines::processors::transforms::aggregator::aggregate_meta::AggregateMeta; use crate::servers::flight::v1::exchange::serde::serialize_block; @@ -74,7 +73,7 @@ impl ExchangeSorting for AggregateExchangeSorting { serde_json::to_string(block_meta_info) ))), Some(meta_info) => match meta_info { - AggregateMeta::FinalPayload(_) => unreachable!(), + AggregateMeta::FinalPartition => unreachable!(), AggregateMeta::InFlightPayload(_) => unreachable!(), AggregateMeta::Serialized(v) => { compute_block_number(v.bucket, v.max_partition_count) @@ -245,7 +244,7 @@ impl Exchange for FlightExchange { assert_eq!(self.bucket_lookup.len(), n); match meta { AggregateMeta::Serialized(_) => unreachable!(), - AggregateMeta::FinalPayload(_) => unreachable!(), + AggregateMeta::FinalPartition => unreachable!(), AggregateMeta::InFlightPayload(_) => unreachable!(), AggregateMeta::SpilledPayload(v) => match self.bucket_lookup.get(&v.destination_node) { None => unreachable!(), @@ -296,43 +295,29 @@ impl Exchange for FlightExchange { } } - fn multiway_pick(&self, data_blocks: &[Option]) -> Result { - let position = data_blocks.iter().position_min_by(|left, right| { - let Some(left_block) = left else { - return Ordering::Greater; - }; - let Some(left_meta) = left_block.get_meta() else { - return Ordering::Greater; - }; - let Some(left_meta) = ExchangeSerializeMeta::downcast_ref_from(left_meta) else { - return Ordering::Greater; - }; - - let Some(right_block) = right else { - return Ordering::Less; - }; - let Some(right_meta) = right_block.get_meta() else { - return Ordering::Less; - }; - let Some(right_meta) = ExchangeSerializeMeta::downcast_ref_from(right_meta) else { - return Ordering::Less; - }; - - let (l_partition, l_max_partition) = restore_block_number(left_meta.block_number); - let (r_partition, r_max_partition) = restore_block_number(right_meta.block_number); - - // ORDER BY max_partition asc, partition asc - match l_max_partition.cmp(&r_max_partition) { - Ordering::Less => Ordering::Less, - Ordering::Greater => Ordering::Greater, - Ordering::Equal => match l_partition.cmp(&r_partition) { - Ordering::Less => Ordering::Less, - Ordering::Equal => Ordering::Equal, - Ordering::Greater => Ordering::Greater, - }, - } - }); + fn sorting_function(left_block: &DataBlock, right_block: &DataBlock) -> Ordering { + let Some(left_meta) = left_block.get_meta() else { + return Ordering::Equal; + }; + let Some(left_meta) = ExchangeSerializeMeta::downcast_ref_from(left_meta) else { + return Ordering::Equal; + }; - position.ok_or_else(|| ErrorCode::Internal("Cannot multiway pick with all none")) + let Some(right_meta) = right_block.get_meta() else { + return Ordering::Equal; + }; + let Some(right_meta) = ExchangeSerializeMeta::downcast_ref_from(right_meta) else { + return Ordering::Equal; + }; + + let (l_partition, l_max_partition) = restore_block_number(left_meta.block_number); + let (r_partition, r_max_partition) = restore_block_number(right_meta.block_number); + + // ORDER BY max_partition asc, partition asc + match l_max_partition.cmp(&r_max_partition) { + Ordering::Less => Ordering::Less, + Ordering::Greater => Ordering::Greater, + Ordering::Equal => l_partition.cmp(&r_partition), + } } } 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 d44960f028861..38d0d63ee2090 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 @@ -139,7 +139,7 @@ pub enum AggregateMeta { SpilledPayload(SpilledPayload), AggregatePayload(AggregatePayload), InFlightPayload(InFlightPayload), - FinalPayload(FinalPayload), + FinalPartition, } impl AggregateMeta { @@ -178,15 +178,15 @@ impl AggregateMeta { Box::new(AggregateMeta::SpilledPayload(payload)) } - pub fn create_final(data: Vec<(AggregateMeta, DataBlock)>) -> BlockMetaInfoPtr { - Box::new(AggregateMeta::FinalPayload(FinalPayload { data })) + pub fn create_final() -> BlockMetaInfoPtr { + Box::new(AggregateMeta::FinalPartition) } } impl Debug for AggregateMeta { fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { match self { - AggregateMeta::FinalPayload(_) => f.debug_struct("AggregateMeta::Partitioned").finish(), + AggregateMeta::FinalPartition => f.debug_struct("AggregateMeta::Partitioned").finish(), AggregateMeta::Serialized { .. } => { f.debug_struct("AggregateMeta::Serialized").finish() } 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 e40af19054f0b..32db416fa9284 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 @@ -127,7 +127,7 @@ impl TransformAggregateSerializer { AggregateMeta::SpilledPayload(_) => unreachable!(), AggregateMeta::Serialized(_) => unreachable!(), AggregateMeta::InFlightPayload(_) => unreachable!(), - AggregateMeta::FinalPayload(_) => unreachable!(), + AggregateMeta::FinalPartition => unreachable!(), AggregateMeta::AggregatePayload(p) => { self.input_data = Some(SerializeAggregateStream::create( &self.params, 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 e40d8dde3ade4..b7354f2f8a9e3 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 @@ -15,25 +15,19 @@ use std::any::Any; use std::collections::VecDeque; use std::sync::Arc; -use std::time::Duration; use std::time::Instant; use byteorder::BigEndian; use byteorder::ReadBytesExt; -use databend_common_base::runtime::profile::Profile; -use databend_common_base::runtime::profile::ProfileStatisticsName; -use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::arrow::deserialize_column; use databend_common_expression::BlockMetaInfoDowncast; -use databend_common_expression::BlockMetaInfoPtr; use databend_common_expression::DataBlock; 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 itertools::Itertools; use log::info; use opendal::Operator; use tokio::sync::Semaphore; @@ -52,7 +46,7 @@ pub struct TransformSpillReader { operator: Operator, semaphore: Arc, params: Arc, - deserialized_meta: Option, + output_data: Option, reading_meta: Option, deserializing_meta: Option, } @@ -78,9 +72,8 @@ impl Processor for TransformSpillReader { return Ok(Event::NeedConsume); } - if let Some(deserialized_meta) = self.deserialized_meta.take() { - self.output - .push_data(Ok(DataBlock::empty_with_meta(deserialized_meta))); + if let Some(output_data) = self.output_data.take() { + self.output.push_data(Ok(output_data)); return Ok(Event::NeedConsume); } @@ -107,19 +100,6 @@ impl Processor for TransformSpillReader { self.reading_meta = AggregateMeta::downcast_from(block_meta); return Ok(Event::Async); } - - if let AggregateMeta::FinalPayload(payload) = block_meta { - if payload - .data - .iter() - .any(|(meta, _)| matches!(meta, AggregateMeta::SpilledPayload(_))) - { - self.input.set_not_need_data(); - let block_meta = data_block.take_meta().unwrap(); - self.reading_meta = AggregateMeta::downcast_from(block_meta); - return Ok(Event::Async); - } - } } self.output.push_data(Ok(data_block)); @@ -138,33 +118,14 @@ impl Processor for TransformSpillReader { fn process(&mut self) -> Result<()> { if let Some((meta, mut read_data)) = self.deserializing_meta.take() { match meta { - AggregateMeta::AggregatePayload(_) => unreachable!(), - AggregateMeta::Serialized(_) => unreachable!(), - AggregateMeta::InFlightPayload(_) => unreachable!(), AggregateMeta::SpilledPayload(payload) => { debug_assert!(read_data.len() == 1); let data = read_data.pop_front().unwrap(); - - self.deserialized_meta = Some(Box::new(self.deserialize(payload, data))); - } - AggregateMeta::FinalPayload(payload) => { - let mut new_data = Vec::with_capacity(payload.data.len()); - - for (meta, block) in payload.data { - if matches!(&meta, AggregateMeta::SpilledPayload(_)) { - if let AggregateMeta::SpilledPayload(payload) = meta { - let data = read_data.pop_front().unwrap(); - new_data.push((self.deserialize(payload, data), block)); - } - - continue; - } - - new_data.push((meta, block)); - } - - self.deserialized_meta = Some(AggregateMeta::create_final(new_data)); + self.output_data = Some(DataBlock::empty_with_meta(Box::new( + self.deserialize(payload, data), + ))); } + _ => unreachable!(), } } @@ -175,9 +136,6 @@ impl Processor for TransformSpillReader { async fn async_process(&mut self) -> Result<()> { if let Some(block_meta) = self.reading_meta.take() { match &block_meta { - AggregateMeta::AggregatePayload(_) => unreachable!(), - AggregateMeta::Serialized(_) => unreachable!(), - AggregateMeta::InFlightPayload(_) => unreachable!(), AggregateMeta::SpilledPayload(payload) => { let _guard = self.semaphore.acquire().await; let instant = Instant::now(); @@ -188,90 +146,15 @@ impl Processor for TransformSpillReader { .await? .to_vec(); - info!( - "Read aggregate spill {} successfully, elapsed: {:?}", - &payload.location, - instant.elapsed() - ); + // info!( + // "Read aggregate spill {} successfully, elapsed: {:?}", + // &payload.location, + // instant.elapsed() + // ); self.deserializing_meta = Some((block_meta, VecDeque::from(vec![data]))); } - AggregateMeta::FinalPayload(payload) => { - // For log progress. - let mut total_elapsed = Duration::default(); - let log_interval = 100; - let mut processed_count = 0; - - let mut read_data = Vec::with_capacity(payload.data.len()); - for (meta, _block) in &payload.data { - if let AggregateMeta::SpilledPayload(payload) = meta { - eprintln!("recv spill payload {:?}", payload); - let location = payload.location.clone(); - let operator = self.operator.clone(); - let data_range = payload.data_range.clone(); - let semaphore = self.semaphore.clone(); - read_data.push(databend_common_base::runtime::spawn(async move { - let _guard = semaphore.acquire().await; - let instant = Instant::now(); - let data = operator - .read_with(&location) - .range(data_range) - .await? - .to_vec(); - - // perf - { - Profile::record_usize_profile( - ProfileStatisticsName::RemoteSpillReadCount, - 1, - ); - Profile::record_usize_profile( - ProfileStatisticsName::RemoteSpillReadBytes, - data.len(), - ); - Profile::record_usize_profile( - ProfileStatisticsName::RemoteSpillReadTime, - instant.elapsed().as_millis() as usize, - ); - } - - total_elapsed += instant.elapsed(); - processed_count += 1; - - // log the progress - if processed_count % log_interval == 0 { - info!( - "Read aggregate {}/{} spilled buckets, elapsed: {:?}", - processed_count, - data.len(), - total_elapsed - ); - } - - Ok(data) - })); - } - } - - match futures::future::try_join_all(read_data).await { - Err(_) => { - return Err(ErrorCode::TokioError("Cannot join tokio job")); - } - Ok(read_data) => { - let read_data: std::result::Result>, opendal::Error> = - read_data.into_iter().try_collect(); - - self.deserializing_meta = Some((block_meta, read_data?)); - } - }; - - if processed_count != 0 { - info!( - "Read {} aggregate spills successfully, total elapsed: {:?}", - processed_count, total_elapsed - ); - } - } + _ => unreachable!(), } } @@ -293,7 +176,7 @@ impl TransformSpillReader { operator, semaphore, params, - deserialized_meta: None, + output_data: None, reading_meta: None, deserializing_meta: None, }))) 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 18a966fd2d802..1da78cf3c5abc 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 @@ -15,8 +15,10 @@ 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::BlockMetaInfoDowncast; use databend_common_expression::DataBlock; use databend_common_expression::HashTableConfig; use databend_common_expression::InputColumns; @@ -26,8 +28,8 @@ use databend_common_expression::ProbeState; 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::AccumulatingTransform; +use databend_common_pipeline_transforms::AccumulatingTransformer; use crate::pipelines::processors::transforms::aggregator::aggregate_meta::AggregateMeta; use crate::pipelines::processors::transforms::aggregator::AggregatorParams; @@ -35,20 +37,102 @@ use crate::pipelines::processors::transforms::aggregator::AggregatorParams; pub struct TransformFinalAggregate { params: Arc, flush_state: PayloadFlushState, + hash_table: AggregateHashTable, +} + +impl AccumulatingTransform for TransformFinalAggregate { + const NAME: &'static str = "TransformFinalAggregate"; + + fn transform(&mut self, mut data: DataBlock) -> Result> { + let Some(meta) = data.take_meta() else { + return Err(ErrorCode::Internal("")); + }; + + let Some(aggregate_meta) = AggregateMeta::downcast_from(meta) else { + return Err(ErrorCode::Internal("")); + }; + + match aggregate_meta { + AggregateMeta::SpilledPayload(_) => unreachable!(), + AggregateMeta::Serialized(payload) => { + let payload = payload.convert_to_partitioned_payload( + self.params.group_data_types.clone(), + self.params.aggregate_functions.clone(), + self.params.num_states(), + 0, + Arc::new(Bump::new()), + )?; + + self.hash_table + .combine_payloads(&payload, &mut self.flush_state)?; + } + AggregateMeta::InFlightPayload(_payload) => { + let payload = self.deserialize_flight(data)?; + self.hash_table + .combine_payload(&payload, &mut self.flush_state)?; + } + AggregateMeta::AggregatePayload(payload) => { + self.hash_table + .combine_payload(&payload.payload, &mut self.flush_state)?; + } + AggregateMeta::FinalPartition => { + if self.hash_table.len() == 0 { + return Ok(vec![self.params.empty_result_block()]); + } + + let mut blocks = vec![]; + self.flush_state.clear(); + + loop { + if self.hash_table.merge_result(&mut self.flush_state)? { + let mut cols = self.flush_state.take_aggregate_results(); + cols.extend_from_slice(&self.flush_state.take_group_columns()); + blocks.push(DataBlock::new_from_columns(cols)); + } else { + break; + } + } + + if blocks.is_empty() { + blocks.push(self.params.empty_result_block()); + } + + let config = HashTableConfig::default().with_initial_radix_bits(0); + self.hash_table = AggregateHashTable::new( + self.params.group_data_types.clone(), + self.params.aggregate_functions.clone(), + config, + Arc::new(Bump::new()), + ); + return Ok(blocks); + } + } + + Ok(vec![]) + } } impl TransformFinalAggregate { pub fn try_create( input: Arc, output: Arc, - params: Arc, ) -> Result> { - Ok(BlockMetaTransformer::create( + let config = HashTableConfig::default().with_initial_radix_bits(0); + + let hash_table = AggregateHashTable::new( + params.group_data_types.clone(), + params.aggregate_functions.clone(), + config, + Arc::new(Bump::new()), + ); + + Ok(AccumulatingTransformer::create( input, output, TransformFinalAggregate { params, + hash_table, flush_state: PayloadFlushState::default(), }, )) @@ -91,108 +175,4 @@ impl TransformFinalAggregate { assert_eq!(hashtable.payload.len(), 1); Ok(hashtable.payload.payloads.pop().unwrap()) } - - fn transform_agg_hashtable(&mut self, meta: AggregateMeta) -> Result { - let mut agg_hashtable: Option = None; - if let AggregateMeta::FinalPayload(payload) = meta { - for (bucket_data, block) in payload.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(), - self.params.num_states(), - 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(), - self.params.num_states(), - 0, - Arc::new(Bump::new()), - true, - )?); - } - }, - AggregateMeta::InFlightPayload(_payload) => match agg_hashtable.as_mut() { - Some(ht) => { - let payload = self.deserialize_flight(block)?; - ht.combine_payload(&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(), - // self.params.num_states(), - // 0, - // Arc::new(Bump::new()), - // true, - // )?); - } - }, - AggregateMeta::AggregatePayload(payload) => match agg_hashtable.as_mut() { - Some(ht) => { - // debug_assert!(bucket == payload.partition); - ht.combine_payload(&payload.payload, &mut self.flush_state)?; - } - None => { - // debug_assert!(bucket == payload.partition); - 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); - } - }, - AggregateMeta::FinalPayload(_) => unreachable!(), - AggregateMeta::SpilledPayload(_) => 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 mut cols = self.flush_state.take_aggregate_results(); - cols.extend_from_slice(&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 TransformFinalAggregate { - const NAME: &'static str = "TransformFinalAggregate"; - - 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_merge.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_aggregate_merge.rs deleted file mode 100644 index 2d7d48b7ff94e..0000000000000 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_aggregate_merge.rs +++ /dev/null @@ -1,17 +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. - -pub struct TransformAggregateMerge { - -} 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 index 3896e70019304..6f1c0d25e9d14 100644 --- 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 @@ -16,6 +16,7 @@ use std::any::Any; use std::collections::hash_map::Entry; use std::collections::BTreeMap; use std::collections::HashMap; +use std::collections::VecDeque; use std::fmt::Debug; use std::fmt::Formatter; use std::sync::Arc; @@ -68,15 +69,21 @@ impl Debug for InputPortState { } pub struct TransformPartitionDispatch { - output: Arc, + outputs: Vec>, inputs: Vec, + outputs_data: Vec>, + output_index: usize, max_partition: usize, initialized_all_inputs: bool, partitions: Partitions, } impl TransformPartitionDispatch { - pub fn create(input_nums: usize, params: Arc) -> Result { + pub fn create( + input_nums: usize, + output_nums: usize, + params: Arc, + ) -> Result { let mut inputs = Vec::with_capacity(input_nums); for _index in 0..input_nums { @@ -87,6 +94,14 @@ impl TransformPartitionDispatch { }); } + let mut outputs = Vec::with_capacity(output_nums); + let mut outputs_data = Vec::with_capacity(output_nums); + + for _index in 0..output_nums { + outputs.push(OutputPort::create()); + outputs_data.push(VecDeque::new()); + } + let max_partition = match params.cluster_aggregator { true => MAX_PARTITION_COUNT, false => 0, @@ -94,8 +109,10 @@ impl TransformPartitionDispatch { Ok(TransformPartitionDispatch { inputs, + outputs, + outputs_data, max_partition, - output: OutputPort::create(), + output_index: 0, initialized_all_inputs: false, partitions: Partitions::create_unaligned(params), }) @@ -111,8 +128,8 @@ impl TransformPartitionDispatch { inputs } - pub fn get_output(&self) -> Arc { - self.output.clone() + pub fn get_outputs(&self) -> Vec> { + self.outputs.clone() } // Align each input's max_partition to the maximum max_partition. @@ -193,6 +210,28 @@ impl TransformPartitionDispatch { fn working_partition(&mut self) -> Option { self.inputs.iter().map(|x| x.partition).min() } + + fn fetch_ready_partition(&mut self) -> Result<()> { + if let Some(ready_partition) = self.ready_partition() { + // TODO: read spill data + let ready_partition = self.partitions.take_partition(ready_partition); + + for (meta, data_block) in ready_partition { + self.outputs_data[self.output_index] + .push_back(data_block.add_meta(Some(Box::new(meta)))?); + } + + self.outputs_data[self.output_index] + .push_back(DataBlock::empty_with_meta(AggregateMeta::create_final())); + + self.output_index += 1; + if self.output_index >= self.outputs_data.len() { + self.output_index = 0; + } + } + + Ok(()) + } } #[async_trait::async_trait] @@ -206,7 +245,16 @@ impl Processor for TransformPartitionDispatch { } fn event(&mut self) -> Result { - if self.output.is_finished() { + let mut all_output_finished = true; + + for output in &self.outputs { + if !output.is_finished() { + all_output_finished = false; + break; + } + } + + if all_output_finished { for input_state in &self.inputs { input_state.port.finish(); } @@ -223,7 +271,17 @@ impl Processor for TransformPartitionDispatch { return Ok(Event::NeedData); } - if !self.output.can_push() { + let mut output_can_push = false; + for (idx, output) in self.outputs.iter().enumerate() { + if output.can_push() { + output_can_push = true; + if let Some(block) = self.outputs_data[idx].pop_front() { + output.push_data(Ok(block)); + } + } + } + + if !output_can_push { for input_state in &self.inputs { input_state.port.set_not_need_data(); } @@ -231,14 +289,7 @@ impl Processor for TransformPartitionDispatch { return Ok(Event::NeedConsume); } - if let Some(ready_partition) = self.ready_partition() { - // TODO: read spill data - let ready_partition = self.partitions.take_partition(ready_partition); - self.output - .push_data(Ok(DataBlock::empty_with_meta(AggregateMeta::create_final( - ready_partition, - )))); - } + self.fetch_ready_partition()?; let working_partition = self.working_partition().unwrap_or(0); @@ -251,11 +302,6 @@ impl Processor for TransformPartitionDispatch { all_inputs_is_finished = false; - eprintln!( - "working partition: {}, input_{}:{}", - working_partition, index, self.inputs[index].partition - ); - if self.inputs[index].partition > working_partition { continue; } @@ -272,22 +318,27 @@ impl Processor for TransformPartitionDispatch { } } - if let Some(ready_partition) = self.ready_partition() { - if self.output.can_push() { - let ready_partition = self.partitions.take_partition(ready_partition); - self.output - .push_data(Ok(DataBlock::empty_with_meta(AggregateMeta::create_final( - ready_partition, - )))); + self.fetch_ready_partition()?; + + let mut has_data = false; + for (idx, output) in self.outputs.iter().enumerate() { + if self.outputs_data[idx].is_empty() { + continue; } - return Ok(Event::NeedConsume); - // TODO: read spill data - // TODO: try push + has_data = true; + if output.can_push() { + if let Some(block) = self.outputs_data[idx].pop_front() { + output.push_data(Ok(block)); + } + } } - if all_inputs_is_finished { - self.output.finish(); + if all_inputs_is_finished && !has_data { + for output in &self.outputs { + output.finish(); + } + return Ok(Event::Finished); } @@ -308,19 +359,22 @@ pub fn build_partition_bucket( pipeline: &mut Pipeline, params: Arc, ) -> Result<()> { - let input_nums = pipeline.output_len(); - let transform = TransformPartitionDispatch::create(input_nums, params.clone())?; + let transform = TransformPartitionDispatch::create( + pipeline.output_len(), + pipeline.output_len(), + params.clone(), + )?; - let output = transform.get_output(); let inputs_port = transform.get_inputs(); + let outputs_port = transform.get_outputs(); - 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)?; + pipeline.add_pipe(Pipe::create(inputs_port.len(), outputs_port.len(), vec![ + PipeItem::create( + ProcessorPtr::create(Box::new(transform)), + inputs_port, + outputs_port, + ), + ])); let semaphore = Arc::new(Semaphore::new(params.max_spill_io_requests)); let operator = DataOperator::instance().spill_operator(); @@ -380,7 +434,7 @@ impl UnalignedPartitions { pub fn add_data(&mut self, meta: AggregateMeta, block: DataBlock) -> (isize, usize) { match &meta { AggregateMeta::Serialized(_) => unreachable!(), - AggregateMeta::FinalPayload(_) => unreachable!(), + AggregateMeta::FinalPartition => unreachable!(), AggregateMeta::SpilledPayload(payload) => { let max_partition = payload.max_partition_count; self.insert_data(max_partition, meta, block); @@ -438,7 +492,7 @@ impl UnalignedPartitions { )?; hashtable.payload.mark_min_cardinality(); - assert_eq!(hashtable.payload.len(), 1); + assert_eq!(hashtable.payload.payloads.len(), 1); Ok(hashtable.payload.payloads.pop().unwrap()) } @@ -486,7 +540,7 @@ impl UnalignedPartitions { for (meta, block) in repartition_data { match meta { AggregateMeta::Serialized(_) => unreachable!(), - AggregateMeta::FinalPayload(_) => unreachable!(), + AggregateMeta::FinalPartition => unreachable!(), AggregateMeta::SpilledPayload(_) => unreachable!(), AggregateMeta::InFlightPayload(payload) => { let payload = AggregatePayload { @@ -531,7 +585,7 @@ impl AlignedPartitions { pub fn add_data(&mut self, meta: AggregateMeta, block: DataBlock) -> (isize, usize) { let (partition, max_partition) = match &meta { AggregateMeta::Serialized(_) => unreachable!(), - AggregateMeta::FinalPayload(_) => unreachable!(), + AggregateMeta::FinalPartition => unreachable!(), AggregateMeta::SpilledPayload(v) => (v.partition, v.max_partition_count), AggregateMeta::AggregatePayload(v) => (v.partition, v.max_partition_count), AggregateMeta::InFlightPayload(v) => (v.partition, v.max_partition), From be7c1da95162ff3cb0904bc197af69877bf50468 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Thu, 13 Mar 2025 13:48:25 +0800 Subject: [PATCH 06/76] refactor(query): refactor aggreagte spill code --- src/query/pipeline/core/src/pipeline.rs | 12 ++---------- .../core/src/processors/shuffle_processor.rs | 14 +++++--------- .../aggregator/serde/transform_spill_reader.rs | 3 --- 3 files changed, 7 insertions(+), 22 deletions(-) diff --git a/src/query/pipeline/core/src/pipeline.rs b/src/query/pipeline/core/src/pipeline.rs index bf9b3bc46468b..b5eae42644db0 100644 --- a/src/query/pipeline/core/src/pipeline.rs +++ b/src/query/pipeline/core/src/pipeline.rs @@ -460,11 +460,7 @@ impl Pipeline { let input = InputPort::create(); let outputs: Vec<_> = (0..n).map(|_| OutputPort::create()).collect(); items.push(PipeItem::create( - PartitionProcessor::create( - input.clone(), - outputs.clone(), - exchange.clone(), - ), + PartitionProcessor::create(input.clone(), outputs.clone(), exchange.clone()), vec![input], outputs, )); @@ -485,11 +481,7 @@ impl Pipeline { let output = OutputPort::create(); let inputs: Vec<_> = (0..input_len).map(|_| InputPort::create()).collect(); items.push(PipeItem::create( - MergePartitionProcessor::create( - inputs.clone(), - output.clone(), - exchange.clone(), - ), + MergePartitionProcessor::::create(inputs.clone(), output.clone()), inputs, vec![output], )); diff --git a/src/query/pipeline/core/src/processors/shuffle_processor.rs b/src/query/pipeline/core/src/processors/shuffle_processor.rs index cd37e953484a0..11c094f5569fa 100644 --- a/src/query/pipeline/core/src/processors/shuffle_processor.rs +++ b/src/query/pipeline/core/src/processors/shuffle_processor.rs @@ -14,6 +14,7 @@ use std::any::Any; use std::cmp::Ordering; +use std::marker::PhantomData; use std::sync::Arc; use databend_common_exception::Result; @@ -265,25 +266,20 @@ impl Processor for PartitionProcessor { } pub struct MergePartitionProcessor { - exchange: Arc, - output: Arc, inputs: Vec>, inputs_data: Vec>, + _phantom_data: PhantomData, } impl MergePartitionProcessor { - pub fn create( - inputs: Vec>, - output: Arc, - exchange: Arc, - ) -> ProcessorPtr { + pub fn create(inputs: Vec>, output: Arc) -> ProcessorPtr { let inputs_data = vec![None; inputs.len()]; - ProcessorPtr::create(Box::new(MergePartitionProcessor { + ProcessorPtr::create(Box::new(MergePartitionProcessor:: { output, inputs, - exchange, inputs_data, + _phantom_data: Default::default(), })) } 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 b7354f2f8a9e3..f0a0cda73b6ae 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 @@ -15,7 +15,6 @@ use std::any::Any; use std::collections::VecDeque; use std::sync::Arc; -use std::time::Instant; use byteorder::BigEndian; use byteorder::ReadBytesExt; @@ -28,7 +27,6 @@ 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 log::info; use opendal::Operator; use tokio::sync::Semaphore; @@ -138,7 +136,6 @@ impl Processor for TransformSpillReader { match &block_meta { AggregateMeta::SpilledPayload(payload) => { let _guard = self.semaphore.acquire().await; - let instant = Instant::now(); let data = self .operator .read_with(&payload.location) From b7cf0727cf72209456d273f331f72f5e59e783c0 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Thu, 13 Mar 2025 14:00:53 +0800 Subject: [PATCH 07/76] refactor(query): refactor aggreagte spill code --- .../transforms/aggregator/transform_partition_bucket.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) 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 index 6f1c0d25e9d14..75112b66d7499 100644 --- 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 @@ -198,13 +198,13 @@ impl TransformPartitionDispatch { fn ready_partition(&mut self) -> Option { let inputs_min_partition = self.working_partition()?; - let stroage_min_partition = self.partitions.min_partition()?; + let storage_min_partition = self.partitions.min_partition()?; - if stroage_min_partition >= inputs_min_partition { + if storage_min_partition >= inputs_min_partition { return None; } - Some(stroage_min_partition) + Some(storage_min_partition) } fn working_partition(&mut self) -> Option { From bf59696c87051150416d71534587dbd8377f4608 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Thu, 13 Mar 2025 17:37:25 +0800 Subject: [PATCH 08/76] refactor(query): refactor aggreagte spill code --- src/query/pipeline/core/src/lib.rs | 1 + .../core/src/processors/shuffle_processor.rs | 13 +- .../pipelines/builders/builder_aggregate.rs | 14 +- .../src/pipelines/pipeline_build_res.rs | 9 +- .../service/src/pipelines/pipeline_builder.rs | 8 +- .../aggregator/aggregate_exchange_injector.rs | 93 ++--------- .../processors/transforms/aggregator/mod.rs | 1 - .../aggregator/transform_partition_bucket.rs | 8 + .../flight/v1/exchange/exchange_injector.rs | 152 ------------------ .../flight/v1/exchange/exchange_manager.rs | 35 ++-- .../flight/v1/exchange/exchange_params.rs | 5 +- .../flight/v1/exchange/exchange_sink.rs | 72 ++------- .../flight/v1/exchange/exchange_sorting.rs | 140 ---------------- .../flight/v1/exchange/exchange_transform.rs | 2 +- .../v1/exchange/exchange_transform_shuffle.rs | 25 ++- .../src/servers/flight/v1/exchange/mod.rs | 5 - .../flight/v1/scatter/flight_scatter_hash.rs | 12 +- 17 files changed, 104 insertions(+), 491 deletions(-) delete mode 100644 src/query/service/src/servers/flight/v1/exchange/exchange_injector.rs delete mode 100644 src/query/service/src/servers/flight/v1/exchange/exchange_sorting.rs diff --git a/src/query/pipeline/core/src/lib.rs b/src/query/pipeline/core/src/lib.rs index d064965129771..395d7b78153a8 100644 --- a/src/query/pipeline/core/src/lib.rs +++ b/src/query/pipeline/core/src/lib.rs @@ -15,6 +15,7 @@ #![feature(once_cell_try)] #![feature(variant_count)] #![feature(associated_type_defaults)] +#![feature(adt_const_params)] #![allow(clippy::arc_with_non_send_sync)] #![allow(clippy::useless_asref)] diff --git a/src/query/pipeline/core/src/processors/shuffle_processor.rs b/src/query/pipeline/core/src/processors/shuffle_processor.rs index 11c094f5569fa..55b492064cb42 100644 --- a/src/query/pipeline/core/src/processors/shuffle_processor.rs +++ b/src/query/pipeline/core/src/processors/shuffle_processor.rs @@ -27,15 +27,18 @@ use crate::processors::OutputPort; use crate::processors::Processor; use crate::processors::ProcessorPtr; +#[derive(Eq, PartialEq)] pub enum MultiwayStrategy { Random, Custom, } +// std::marker::ConstParamTy!(MultiwayStrategy); + pub trait Exchange: Send + Sync + 'static { const NAME: &'static str; + const MULTIWAY_SORT: bool = false; const SKIP_EMPTY_DATA_BLOCK: bool = false; - const STRATEGY: MultiwayStrategy = MultiwayStrategy::Random; fn partition(&self, data_block: DataBlock, n: usize) -> Result>; @@ -323,7 +326,7 @@ impl Processor for MergePartitionProcessor { } let mut all_inputs_finished = true; - let mut need_pick_block_to_push = matches!(T::STRATEGY, MultiwayStrategy::Custom); + let mut need_pick_block_to_push = T::MULTIWAY_SORT; for (index, input) in self.inputs.iter().enumerate() { if input.is_finished() { @@ -333,13 +336,13 @@ impl Processor for MergePartitionProcessor { all_inputs_finished = false; if input.has_data() { - match T::STRATEGY { - MultiwayStrategy::Random => { + match T::MULTIWAY_SORT { + false => { if self.output.can_push() { self.output.push_data(Ok(input.pull_data().unwrap()?)); } } - MultiwayStrategy::Custom => { + true => { if self.inputs_data[index].is_none() { self.inputs_data[index] = Some(input.pull_data().unwrap()?); } diff --git a/src/query/service/src/pipelines/builders/builder_aggregate.rs b/src/query/service/src/pipelines/builders/builder_aggregate.rs index 5900c882cf550..996554cc691ef 100644 --- a/src/query/service/src/pipelines/builders/builder_aggregate.rs +++ b/src/query/service/src/pipelines/builders/builder_aggregate.rs @@ -39,7 +39,6 @@ use itertools::Itertools; use crate::pipelines::processors::transforms::aggregator::build_partition_bucket; use crate::pipelines::processors::transforms::aggregator::create_udaf_script_function; -use crate::pipelines::processors::transforms::aggregator::AggregateInjector; use crate::pipelines::processors::transforms::aggregator::AggregatorParams; use crate::pipelines::processors::transforms::aggregator::FinalSingleStateAggregator; use crate::pipelines::processors::transforms::aggregator::PartialSingleStateAggregator; @@ -166,7 +165,8 @@ impl PipelineBuilder { )?)) })?; - self.exchange_injector = AggregateInjector::create(); + self.enable_multiway_sort = true; + // self.exchange_injector = AggregateInjector::create(); Ok(()) } @@ -199,14 +199,12 @@ impl PipelineBuilder { return Ok(()); } - let old_inject = self.exchange_injector.clone(); - let input: &PhysicalPlan = &aggregate.input; - if matches!(input, PhysicalPlan::ExchangeSource(_)) { - self.exchange_injector = AggregateInjector::create(); - } + let old_value = self.enable_multiway_sort; + self.enable_multiway_sort |= matches!(input, PhysicalPlan::ExchangeSource(_)); + self.build_pipeline(&aggregate.input)?; - self.exchange_injector = old_inject; + self.enable_multiway_sort = old_value; build_partition_bucket(&mut self.main_pipeline, params.clone()) } diff --git a/src/query/service/src/pipelines/pipeline_build_res.rs b/src/query/service/src/pipelines/pipeline_build_res.rs index fd40f817e3cda..7d5819b4e6bad 100644 --- a/src/query/service/src/pipelines/pipeline_build_res.rs +++ b/src/query/service/src/pipelines/pipeline_build_res.rs @@ -24,8 +24,6 @@ use databend_common_pipeline_sources::OneBlockSource; use crate::interpreters::CreateTableInterpreter; use crate::pipelines::processors::transforms::HashJoinBuildState; -use crate::servers::flight::v1::exchange::DefaultExchangeInjector; -use crate::servers::flight::v1::exchange::ExchangeInjector; #[derive(Clone)] pub struct PipelineBuilderData { @@ -38,7 +36,8 @@ pub struct PipelineBuildResult { // Containing some sub queries pipelines, must be complete pipeline pub sources_pipelines: Vec, - pub exchange_injector: Arc, + pub enable_multiway_sort: bool, + // pub exchange_injector: Arc, /// for local fragment data sharing pub builder_data: PipelineBuilderData, pub r_cte_scan_interpreters: Vec, @@ -49,7 +48,7 @@ impl PipelineBuildResult { PipelineBuildResult { main_pipeline: Pipeline::create(), sources_pipelines: vec![], - exchange_injector: DefaultExchangeInjector::create(), + enable_multiway_sort: false, builder_data: PipelineBuilderData { input_join_state: None, input_probe_schema: None, @@ -72,7 +71,7 @@ impl PipelineBuildResult { Ok(PipelineBuildResult { main_pipeline, sources_pipelines: vec![], - exchange_injector: DefaultExchangeInjector::create(), + enable_multiway_sort: false, builder_data: PipelineBuilderData { input_join_state: None, input_probe_schema: None, diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs index e796d2cb4d705..c318fa9305ed5 100644 --- a/src/query/service/src/pipelines/pipeline_builder.rs +++ b/src/query/service/src/pipelines/pipeline_builder.rs @@ -33,8 +33,6 @@ use crate::interpreters::CreateTableInterpreter; use crate::pipelines::processors::transforms::HashJoinBuildState; use crate::pipelines::processors::HashJoinState; use crate::pipelines::PipelineBuildResult; -use crate::servers::flight::v1::exchange::DefaultExchangeInjector; -use crate::servers::flight::v1::exchange::ExchangeInjector; use crate::sessions::QueryContext; pub struct PipelineBuilder { @@ -49,7 +47,7 @@ pub struct PipelineBuilder { pub merge_into_probe_data_fields: Option>, pub join_state: Option>, - pub(crate) exchange_injector: Arc, + pub(crate) enable_multiway_sort: bool, pub hash_join_states: HashMap>, @@ -72,13 +70,13 @@ impl PipelineBuilder { settings, pipelines: vec![], main_pipeline: Pipeline::with_scopes(scopes), - exchange_injector: DefaultExchangeInjector::create(), merge_into_probe_data_fields: None, join_state: None, hash_join_states: HashMap::new(), r_cte_scan_interpreters: vec![], is_exchange_neighbor: false, contain_sink_processor: false, + enable_multiway_sort: false, } } @@ -105,12 +103,12 @@ impl PipelineBuilder { Ok(PipelineBuildResult { main_pipeline: self.main_pipeline, sources_pipelines: self.pipelines, - exchange_injector: self.exchange_injector, builder_data: PipelineBuilderData { input_join_state: self.join_state, input_probe_schema: self.merge_into_probe_data_fields, }, r_cte_scan_interpreters: self.r_cte_scan_interpreters, + enable_multiway_sort: self.enable_multiway_sort, }) } 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 9d703ee0ff31f..e27a7f0a1e4b2 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 @@ -18,30 +18,19 @@ use std::sync::Arc; use arrow_ipc::writer::IpcWriteOptions; use arrow_ipc::CompressionType; -use databend_common_catalog::table_context::TableContext; use databend_common_config::GlobalConfig; -use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::DataBlock; use databend_common_expression::Payload; use databend_common_expression::PayloadFlushState; use databend_common_pipeline_core::processors::Exchange; -use databend_common_pipeline_core::processors::MultiwayStrategy; use databend_common_settings::FlightCompression; use crate::pipelines::processors::transforms::aggregator::aggregate_meta::AggregateMeta; use crate::servers::flight::v1::exchange::serde::serialize_block; use crate::servers::flight::v1::exchange::serde::ExchangeSerializeMeta; -use crate::servers::flight::v1::exchange::DataExchange; -use crate::servers::flight::v1::exchange::ExchangeInjector; -use crate::servers::flight::v1::exchange::ExchangeSorting; -use crate::servers::flight::v1::scatter::BroadcastFlightScatter; use crate::servers::flight::v1::scatter::FlightScatter; -use crate::servers::flight::v1::scatter::HashFlightScatter; -use crate::sessions::QueryContext; - -struct AggregateExchangeSorting {} pub fn compute_block_number(bucket: isize, max_partition_count: usize) -> Result { match bucket.is_negative() { @@ -63,31 +52,6 @@ pub fn restore_block_number(value: isize) -> (isize, usize) { } } -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::FinalPartition => unreachable!(), - AggregateMeta::InFlightPayload(_) => unreachable!(), - AggregateMeta::Serialized(v) => { - compute_block_number(v.bucket, v.max_partition_count) - } - AggregateMeta::AggregatePayload(v) => { - compute_block_number(v.partition, v.max_partition_count) - } - AggregateMeta::SpilledPayload(_) => Ok(-1), - }, - }, - } - } -} - fn scatter_payload(mut payload: Payload, buckets: usize) -> Result> { let mut buckets = Vec::with_capacity(buckets); @@ -122,48 +86,7 @@ fn scatter_payload(mut payload: Payload, buckets: usize) -> Result> Ok(buckets) } -pub struct AggregateInjector; - -impl AggregateInjector { - pub fn create() -> Arc { - Arc::new(AggregateInjector) - } -} - -impl ExchangeInjector for AggregateInjector { - fn flight_scatter( - &self, - ctx: &Arc, - exchange: &DataExchange, - ) -> Result>> { - Ok(Arc::new(match exchange { - DataExchange::Merge(_) => unreachable!(), - DataExchange::Broadcast(exchange) => Box::new(BroadcastFlightScatter::try_create( - exchange.destination_ids.len(), - )?), - DataExchange::ShuffleDataExchange(exchange) => { - let local_id = &ctx.get_cluster().local_id; - let local_pos = exchange - .destination_ids - .iter() - .position(|x| x == local_id) - .unwrap(); - HashFlightScatter::try_create( - ctx.get_function_context()?, - exchange.shuffle_keys.clone(), - exchange.destination_ids.len(), - local_pos, - )? - } - })) - } - - fn exchange_sorting(&self) -> Option> { - Some(Arc::new(AggregateExchangeSorting {})) - } -} - -pub struct FlightExchange { +pub struct FlightExchange { local_id: String, bucket_lookup: HashMap, rev_bucket_lookup: Vec, @@ -171,12 +94,12 @@ pub struct FlightExchange { shuffle_scatter: Arc>, } -impl FlightExchange { +impl FlightExchange { pub fn create( lookup: Vec, compression: Option, shuffle_scatter: Arc>, - ) -> Arc { + ) -> Arc { let compression = match compression { None => None, Some(compression) => match compression { @@ -204,7 +127,7 @@ impl FlightExchange { } } -impl FlightExchange { +impl FlightExchange { fn default_partition(&self, data_block: DataBlock) -> Result> { if self.rev_bucket_lookup.is_empty() { let data_block = serialize_block(0, data_block, &self.options)?; @@ -228,9 +151,9 @@ impl FlightExchange { } } -impl Exchange for FlightExchange { +impl Exchange for FlightExchange { const NAME: &'static str = "AggregateExchange"; - const STRATEGY: MultiwayStrategy = MultiwayStrategy::Custom; + const MULTIWAY_SORT: bool = MULTIWAY_SORT; fn partition(&self, mut data_block: DataBlock, n: usize) -> Result> { let Some(meta) = data_block.take_meta() else { @@ -267,6 +190,10 @@ impl Exchange for FlightExchange { AggregateMeta::AggregatePayload(p) => { let mut blocks = Vec::with_capacity(n); for (idx, payload) in scatter_payload(p.payload, n)?.into_iter().enumerate() { + if payload.len() == 0 { + continue; + } + if self.rev_bucket_lookup[idx] == self.local_id { blocks.push(( idx, 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 d33411cc45da7..442b0e38ee0d9 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/mod.rs @@ -23,7 +23,6 @@ mod transform_partition_bucket; mod transform_single_key; mod udaf_script; -pub use aggregate_exchange_injector::AggregateInjector; pub use aggregate_exchange_injector::FlightExchange; pub use aggregate_meta::*; pub use aggregator_params::AggregatorParams; 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 index 75112b66d7499..7f999034434cd 100644 --- 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 @@ -543,6 +543,10 @@ impl UnalignedPartitions { AggregateMeta::FinalPartition => unreachable!(), AggregateMeta::SpilledPayload(_) => unreachable!(), AggregateMeta::InFlightPayload(payload) => { + if block.is_empty() { + continue; + } + let payload = AggregatePayload { partition: payload.partition, max_partition_count: payload.max_partition, @@ -559,6 +563,10 @@ impl UnalignedPartitions { } } AggregateMeta::AggregatePayload(payload) => { + if payload.payload.len() == 0 { + continue; + } + let partitioned = self.partition_payload(payload, max_partitions); for payload in partitioned { aligned_partitions.add_data( diff --git a/src/query/service/src/servers/flight/v1/exchange/exchange_injector.rs b/src/query/service/src/servers/flight/v1/exchange/exchange_injector.rs deleted file mode 100644 index 2355eb8b27428..0000000000000 --- a/src/query/service/src/servers/flight/v1/exchange/exchange_injector.rs +++ /dev/null @@ -1,152 +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 databend_common_catalog::table_context::TableContext; -use databend_common_exception::Result; - -use crate::servers::flight::v1::exchange::DataExchange; -use crate::servers::flight::v1::exchange::ExchangeSorting; -use crate::servers::flight::v1::scatter::BroadcastFlightScatter; -use crate::servers::flight::v1::scatter::FlightScatter; -use crate::servers::flight::v1::scatter::HashFlightScatter; -use crate::sessions::QueryContext; - -pub trait ExchangeInjector: Send + Sync + 'static { - fn flight_scatter( - &self, - ctx: &Arc, - exchange: &DataExchange, - ) -> Result>>; - - fn exchange_sorting(&self) -> Option>; - - // fn apply_merge_serializer( - // &self, - // params: &MergeExchangeParams, - // compression: Option, - // pipeline: &mut Pipeline, - // ) -> Result<()>; - // - // fn apply_shuffle_serializer( - // &self, - // params: &ShuffleExchangeParams, - // compression: Option, - // pipeline: &mut Pipeline, - // ) -> Result<()>; - // - // fn apply_merge_deserializer( - // &self, - // params: &MergeExchangeParams, - // pipeline: &mut Pipeline, - // ) -> Result<()>; - // - // fn apply_shuffle_deserializer( - // &self, - // params: &ShuffleExchangeParams, - // pipeline: &mut Pipeline, - // ) -> Result<()>; -} - -pub struct DefaultExchangeInjector; - -impl DefaultExchangeInjector { - pub fn create() -> Arc { - Arc::new(DefaultExchangeInjector {}) - } -} - -impl ExchangeInjector for DefaultExchangeInjector { - fn flight_scatter( - &self, - ctx: &Arc, - exchange: &DataExchange, - ) -> Result>> { - Ok(Arc::new(match exchange { - DataExchange::Merge(_) => unreachable!(), - DataExchange::Broadcast(exchange) => Box::new(BroadcastFlightScatter::try_create( - exchange.destination_ids.len(), - )?), - DataExchange::ShuffleDataExchange(exchange) => { - let local_id = &ctx.get_cluster().local_id; - let local_pos = exchange - .destination_ids - .iter() - .position(|x| x == local_id) - .unwrap(); - HashFlightScatter::try_create( - ctx.get_function_context()?, - exchange.shuffle_keys.clone(), - exchange.destination_ids.len(), - local_pos, - )? - } - })) - } - - fn exchange_sorting(&self) -> Option> { - None - } - - // fn apply_merge_serializer( - // &self, - // params: &MergeExchangeParams, - // compression: Option, - // pipeline: &mut Pipeline, - // ) -> Result<()> { - // pipeline.add_transform(|input, output| { - // TransformExchangeSerializer::create(input, output, params, compression) - // }) - // } - // - // fn apply_shuffle_serializer( - // &self, - // params: &ShuffleExchangeParams, - // compression: Option, - // pipeline: &mut Pipeline, - // ) -> Result<()> { - // pipeline.add_transform(|input, output| { - // TransformScatterExchangeSerializer::create(input, output, compression, params) - // }) - // } - // - // fn apply_merge_deserializer( - // &self, - // params: &MergeExchangeParams, - // pipeline: &mut Pipeline, - // ) -> Result<()> { - // pipeline.add_transform(|input, output| { - // Ok(TransformExchangeDeserializer::create( - // input, - // output, - // ¶ms.schema, - // )) - // }) - // } - // - // fn apply_shuffle_deserializer( - // &self, - // params: &ShuffleExchangeParams, - // pipeline: &mut Pipeline, - // ) -> Result<()> { - // pipeline.add_transform(|input, output| { - // Ok(TransformExchangeDeserializer::create( - // input, - // output, - // ¶ms.schema, - // )) - // }) - // } -} diff --git a/src/query/service/src/servers/flight/v1/exchange/exchange_manager.rs b/src/query/service/src/servers/flight/v1/exchange/exchange_manager.rs index c427b1b20d1bb..c3d399b7c1895 100644 --- a/src/query/service/src/servers/flight/v1/exchange/exchange_manager.rs +++ b/src/query/service/src/servers/flight/v1/exchange/exchange_manager.rs @@ -63,11 +63,12 @@ use crate::servers::flight::v1::actions::init_query_fragments; use crate::servers::flight::v1::actions::INIT_QUERY_FRAGMENTS; use crate::servers::flight::v1::actions::START_PREPARED_QUERY; use crate::servers::flight::v1::exchange::DataExchange; -use crate::servers::flight::v1::exchange::ExchangeInjector; use crate::servers::flight::v1::packets::Edge; use crate::servers::flight::v1::packets::QueryEnv; use crate::servers::flight::v1::packets::QueryFragment; use crate::servers::flight::v1::packets::QueryFragments; +use crate::servers::flight::v1::scatter::BroadcastFlightScatter; +use crate::servers::flight::v1::scatter::HashFlightScatter; use crate::servers::flight::FlightClient; use crate::servers::flight::FlightExchange; use crate::servers::flight::FlightReceiver; @@ -754,10 +755,8 @@ impl QueryCoordinator { fragment_coordinator .pipeline_build_res .as_ref() - .map(|x| x.exchange_injector.clone()) - .ok_or_else(|| { - ErrorCode::Internal("Pipeline build result is none, It's a bug") - })?, + .map(|x| x.enable_multiway_sort) + .unwrap_or(false), )?; let mut build_res = fragment_coordinator.pipeline_build_res.unwrap(); @@ -811,10 +810,8 @@ impl QueryCoordinator { coordinator .pipeline_build_res .as_ref() - .map(|x| x.exchange_injector.clone()) - .ok_or_else(|| { - ErrorCode::Internal("Pipeline build result is none, It's a bug") - })?, + .map(|x| x.enable_multiway_sort) + .unwrap_or(false), )?, ); } @@ -906,13 +903,13 @@ impl FragmentCoordinator { pub fn create_exchange_params( &self, info: &QueryInfo, - exchange_injector: Arc, + enable_multiway_sort: bool, ) -> Result { if let Some(data_exchange) = &self.data_exchange { return match data_exchange { DataExchange::Merge(exchange) => { Ok(ExchangeParams::MergeExchange(MergeExchangeParams { - exchange_injector: exchange_injector.clone(), + enable_multiway_sort, schema: self.physical_plan.output_schema()?, fragment_id: self.fragment_id, query_id: info.query_id.to_string(), @@ -923,26 +920,30 @@ impl FragmentCoordinator { } DataExchange::Broadcast(exchange) => { Ok(ExchangeParams::ShuffleExchange(ShuffleExchangeParams { - exchange_injector: exchange_injector.clone(), + enable_multiway_sort, schema: self.physical_plan.output_schema()?, fragment_id: self.fragment_id, query_id: info.query_id.to_string(), executor_id: info.current_executor.to_string(), destination_ids: exchange.destination_ids.to_owned(), - shuffle_scatter: exchange_injector - .flight_scatter(&info.query_ctx, data_exchange)?, + shuffle_scatter: Arc::new(Box::new(BroadcastFlightScatter::try_create( + exchange.destination_ids.len(), + )?)), })) } DataExchange::ShuffleDataExchange(exchange) => { Ok(ExchangeParams::ShuffleExchange(ShuffleExchangeParams { - exchange_injector: exchange_injector.clone(), + enable_multiway_sort, schema: self.physical_plan.output_schema()?, fragment_id: self.fragment_id, query_id: info.query_id.to_string(), executor_id: info.current_executor.to_string(), destination_ids: exchange.destination_ids.to_owned(), - shuffle_scatter: exchange_injector - .flight_scatter(&info.query_ctx, data_exchange)?, + shuffle_scatter: Arc::new(HashFlightScatter::try_create( + &info.query_ctx, + exchange.shuffle_keys.clone(), + &exchange.destination_ids, + )?), })) } }; diff --git a/src/query/service/src/servers/flight/v1/exchange/exchange_params.rs b/src/query/service/src/servers/flight/v1/exchange/exchange_params.rs index 799efe506affe..15607c0454f8d 100644 --- a/src/query/service/src/servers/flight/v1/exchange/exchange_params.rs +++ b/src/query/service/src/servers/flight/v1/exchange/exchange_params.rs @@ -16,7 +16,6 @@ use std::sync::Arc; use databend_common_expression::DataSchemaRef; -use crate::servers::flight::v1::exchange::ExchangeInjector; use crate::servers::flight::v1::scatter::FlightScatter; #[derive(Clone)] @@ -27,7 +26,7 @@ pub struct ShuffleExchangeParams { pub schema: DataSchemaRef, pub destination_ids: Vec, pub shuffle_scatter: Arc>, - pub exchange_injector: Arc, + pub enable_multiway_sort: bool, } #[derive(Clone)] @@ -37,8 +36,8 @@ pub struct MergeExchangeParams { pub destination_id: String, pub schema: DataSchemaRef, pub ignore_exchange: bool, + pub enable_multiway_sort: bool, pub allow_adjust_parallelism: bool, - pub exchange_injector: Arc, } pub enum ExchangeParams { diff --git a/src/query/service/src/servers/flight/v1/exchange/exchange_sink.rs b/src/query/service/src/servers/flight/v1/exchange/exchange_sink.rs index fafde8933cdcd..be5a45e51bf9f 100644 --- a/src/query/service/src/servers/flight/v1/exchange/exchange_sink.rs +++ b/src/query/service/src/servers/flight/v1/exchange/exchange_sink.rs @@ -16,19 +16,12 @@ 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_pipeline_core::processors::ProcessorPtr; use databend_common_pipeline_core::Pipe; -use databend_common_pipeline_core::PipeItem; use databend_common_pipeline_core::Pipeline; use super::exchange_params::ExchangeParams; use super::exchange_sink_writer::create_writer_item; -use super::exchange_sorting::ExchangeSorting; -use super::exchange_sorting::TransformExchangeSorting; use super::exchange_transform_shuffle::exchange_shuffle; -use super::serde::ExchangeSerializeMeta; use crate::clusters::ClusterHelper; use crate::pipelines::processors::transforms::aggregator::FlightExchange; use crate::servers::flight::v1::scatter::MergeFlightScatter; @@ -57,38 +50,31 @@ impl ExchangeSink { ))); } - let exchange_injector = ¶ms.exchange_injector; - if !params.ignore_exchange { let settings = ctx.get_settings(); let compression = settings.get_query_flight_compression()?; let nodes = vec![]; - pipeline.exchange( - 1, - FlightExchange::create( - nodes, - compression, - Arc::new(Box::new(MergeFlightScatter)), + match params.enable_multiway_sort { + true => pipeline.exchange( + 1, + FlightExchange::::create( + nodes, + compression, + Arc::new(Box::new(MergeFlightScatter)), + ), ), - ); - } - - if !params.ignore_exchange && exchange_injector.exchange_sorting().is_some() { - let output_len = pipeline.output_len(); - let sorting = SinkExchangeSorting::create(); - let transform = TransformExchangeSorting::create(output_len, sorting); - - let output = transform.get_output(); - let inputs = transform.get_inputs(); - pipeline.add_pipe(Pipe::create(output_len, 1, vec![PipeItem::create( - ProcessorPtr::create(Box::new(transform)), - inputs, - vec![output], - )])); + false => pipeline.exchange( + 1, + FlightExchange::::create( + nodes, + compression, + Arc::new(Box::new(MergeFlightScatter)), + ), + ), + }; } - pipeline.try_resize(1)?; assert_eq!(senders.len(), 1); pipeline.add_pipe(Pipe::create(1, 0, vec![create_writer_item( senders.remove(0), @@ -122,27 +108,3 @@ impl ExchangeSink { } } } - -struct SinkExchangeSorting; - -impl SinkExchangeSorting { - pub fn create() -> Arc { - Arc::new(SinkExchangeSorting {}) - } -} - -impl ExchangeSorting for SinkExchangeSorting { - fn block_number(&self, data_block: &DataBlock) -> Result { - let block_meta = data_block.get_meta(); - let shuffle_meta = block_meta - .and_then(ExchangeSerializeMeta::downcast_ref_from) - .ok_or_else(|| { - ErrorCode::Internal(format!( - "Failed to downcast ExchangeSerializeMeta from BlockMeta: {:?}", - block_meta - )) - })?; - - Ok(shuffle_meta.block_number) - } -} diff --git a/src/query/service/src/servers/flight/v1/exchange/exchange_sorting.rs b/src/query/service/src/servers/flight/v1/exchange/exchange_sorting.rs deleted file mode 100644 index 8cc931d64641a..0000000000000 --- a/src/query/service/src/servers/flight/v1/exchange/exchange_sorting.rs +++ /dev/null @@ -1,140 +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 databend_common_exception::Result; -use databend_common_expression::DataBlock; -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; - -pub trait ExchangeSorting: Send + Sync + 'static { - fn block_number(&self, data_block: &DataBlock) -> Result; -} - -// N input one output -pub struct TransformExchangeSorting { - inputs: Vec>, - output: Arc, - sorting: Arc, - - buffer_len: usize, - buffer: Vec>, -} - -impl TransformExchangeSorting { - pub fn create(inputs: usize, sorting: Arc) -> TransformExchangeSorting { - let output = OutputPort::create(); - let mut buffer = Vec::with_capacity(inputs); - let mut inputs_port = Vec::with_capacity(inputs); - - for _ in 0..inputs { - buffer.push(None); - inputs_port.push(InputPort::create()); - } - - TransformExchangeSorting { - output, - sorting, - buffer, - buffer_len: 0, - inputs: inputs_port, - } - } - - pub fn get_output(&self) -> Arc { - self.output.clone() - } - pub fn get_inputs(&self) -> Vec> { - self.inputs.clone() - } -} - -#[async_trait::async_trait] -impl Processor for TransformExchangeSorting { - fn name(&self) -> String { - String::from("TransformExchangeSorting") - } - - fn as_any(&mut self) -> &mut dyn Any { - self - } - - fn event(&mut self) -> Result { - if self.output.is_finished() { - for input in &self.inputs { - input.finish(); - } - - return Ok(Event::Finished); - } - - let mut unready_inputs = false; - let mut all_inputs_finished = true; - for (index, input) in self.inputs.iter().enumerate() { - if input.is_finished() { - continue; - } - - all_inputs_finished = false; - if self.buffer[index].is_none() { - if input.has_data() { - let data_block = input.pull_data().unwrap()?; - let block_number = self.sorting.block_number(&data_block)?; - self.buffer[index] = Some((block_number, data_block)); - self.buffer_len += 1; - input.set_need_data(); - continue; - } - - unready_inputs = true; - } - - input.set_need_data(); - } - - if !self.output.can_push() { - return Ok(Event::NeedConsume); - } - - if all_inputs_finished && self.buffer_len == 0 { - self.output.finish(); - return Ok(Event::Finished); - } - - if !unready_inputs { - let mut min_index = 0; - let mut min_value = isize::MAX; - for (index, buffer) in self.buffer.iter().enumerate() { - if let Some((block_number, _)) = buffer { - if *block_number < min_value { - min_index = index; - min_value = *block_number; - } - } - } - - if let Some((_, block)) = self.buffer[min_index].take() { - self.buffer_len -= 1; - self.output.push_data(Ok(block)); - return Ok(Event::NeedConsume); - } - } - - Ok(Event::NeedData) - } -} diff --git a/src/query/service/src/servers/flight/v1/exchange/exchange_transform.rs b/src/query/service/src/servers/flight/v1/exchange/exchange_transform.rs index 1fbc47179b729..089b28720ae6c 100644 --- a/src/query/service/src/servers/flight/v1/exchange/exchange_transform.rs +++ b/src/query/service/src/servers/flight/v1/exchange/exchange_transform.rs @@ -86,7 +86,7 @@ impl ExchangeTransform { let new_outputs = max_threads + nodes_source; pipeline.add_pipe(Pipe::create(len, new_outputs, items)); - if params.exchange_injector.exchange_sorting().is_none() { + if !params.enable_multiway_sort { pipeline.try_resize(max_threads)?; } diff --git a/src/query/service/src/servers/flight/v1/exchange/exchange_transform_shuffle.rs b/src/query/service/src/servers/flight/v1/exchange/exchange_transform_shuffle.rs index c83b5ad8e9913..5ed1ce361589f 100644 --- a/src/query/service/src/servers/flight/v1/exchange/exchange_transform_shuffle.rs +++ b/src/query/service/src/servers/flight/v1/exchange/exchange_transform_shuffle.rs @@ -64,15 +64,24 @@ pub fn exchange_shuffle( let settings = ctx.get_settings(); let compression = settings.get_query_flight_compression()?; - let outputs_size = params.destination_ids.len(); - pipeline.exchange( - outputs_size, - FlightExchange::create( - params.destination_ids.clone(), - compression, - params.shuffle_scatter.clone(), + match params.enable_multiway_sort { + true => pipeline.exchange( + params.destination_ids.len(), + FlightExchange::::create( + params.destination_ids.clone(), + compression, + params.shuffle_scatter.clone(), + ), ), - ); + false => pipeline.exchange( + params.destination_ids.len(), + FlightExchange::::create( + params.destination_ids.clone(), + compression, + params.shuffle_scatter.clone(), + ), + ), + }; Ok(()) } diff --git a/src/query/service/src/servers/flight/v1/exchange/mod.rs b/src/query/service/src/servers/flight/v1/exchange/mod.rs index 631968023ec35..ac51beb3bb7de 100644 --- a/src/query/service/src/servers/flight/v1/exchange/mod.rs +++ b/src/query/service/src/servers/flight/v1/exchange/mod.rs @@ -13,12 +13,10 @@ // limitations under the License. mod data_exchange; -mod exchange_injector; mod exchange_manager; mod exchange_params; mod exchange_sink; mod exchange_sink_writer; -mod exchange_sorting; mod exchange_source; mod exchange_source_reader; mod exchange_transform; @@ -32,10 +30,7 @@ pub use data_exchange::BroadcastExchange; pub use data_exchange::DataExchange; pub use data_exchange::MergeExchange; pub use data_exchange::ShuffleDataExchange; -pub use exchange_injector::DefaultExchangeInjector; -pub use exchange_injector::ExchangeInjector; pub use exchange_manager::DataExchangeManager; pub use exchange_params::MergeExchangeParams; pub use exchange_params::ShuffleExchangeParams; -pub use exchange_sorting::ExchangeSorting; pub use exchange_transform_shuffle::ExchangeShuffleMeta; diff --git a/src/query/service/src/servers/flight/v1/scatter/flight_scatter_hash.rs b/src/query/service/src/servers/flight/v1/scatter/flight_scatter_hash.rs index cc82de51e9dc1..dfc14be8249ae 100644 --- a/src/query/service/src/servers/flight/v1/scatter/flight_scatter_hash.rs +++ b/src/query/service/src/servers/flight/v1/scatter/flight_scatter_hash.rs @@ -15,6 +15,7 @@ use std::collections::hash_map::DefaultHasher; use std::hash::Hasher; +use databend_common_catalog::table_context::TableContext; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::type_check::check_function; @@ -37,6 +38,7 @@ use databend_common_expression::Value; use databend_common_functions::BUILTIN_FUNCTIONS; use crate::servers::flight::v1::scatter::flight_scatter::FlightScatter; +use crate::sessions::QueryContext; #[derive(Clone)] pub struct HashFlightScatter { @@ -47,11 +49,15 @@ pub struct HashFlightScatter { impl HashFlightScatter { pub fn try_create( - func_ctx: FunctionContext, + ctx: &QueryContext, hash_keys: Vec, - scatter_size: usize, - local_pos: usize, + destination_ids: &[String], ) -> Result> { + let local_id = &ctx.get_cluster().local_id; + let func_ctx = ctx.get_function_context()?; + let scatter_size = destination_ids.len(); + let local_pos = destination_ids.iter().position(|x| x == local_id).unwrap(); + if hash_keys.len() == 1 { return OneHashKeyFlightScatter::try_create( func_ctx, From b77c04131b66532585d632761e15804a87bc6dd3 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Thu, 13 Mar 2025 17:48:59 +0800 Subject: [PATCH 09/76] refactor(query): refactor aggreagte spill code --- .../transforms/aggregator/aggregate_exchange_injector.rs | 8 ++++++++ 1 file changed, 8 insertions(+) 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 e27a7f0a1e4b2..d290e8a210fa4 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 @@ -157,10 +157,18 @@ impl Exchange for FlightExchange { fn partition(&self, mut data_block: DataBlock, n: usize) -> Result> { let Some(meta) = data_block.take_meta() else { + if data_block.is_empty() { + return Ok(vec![]); + } + return self.default_partition(data_block); }; let Some(meta) = AggregateMeta::downcast_from(meta) else { + if data_block.is_empty() { + return Ok(vec![]); + } + return self.default_partition(data_block); }; From 78a019978f95bb223639f21d4e31efdb69fec244 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Thu, 13 Mar 2025 19:10:21 +0800 Subject: [PATCH 10/76] refactor(query): refactor aggreagte spill code --- .../transforms/aggregator/aggregate_exchange_injector.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) 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 d290e8a210fa4..37f254bb820b1 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 @@ -196,12 +196,12 @@ impl Exchange for FlightExchange { }, }, AggregateMeta::AggregatePayload(p) => { + if p.payload.len() == 0 { + return Ok(vec![]); + } + let mut blocks = Vec::with_capacity(n); for (idx, payload) in scatter_payload(p.payload, n)?.into_iter().enumerate() { - if payload.len() == 0 { - continue; - } - if self.rev_bucket_lookup[idx] == self.local_id { blocks.push(( idx, From 22dad6e42171a4badb4fdb4f699e2dee03c815b2 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sun, 16 Mar 2025 13:46:22 +0800 Subject: [PATCH 11/76] refactor(query): refactor aggreagte spill code --- .../core/src/processors/shuffle_processor.rs | 9 ++- .../aggregator/aggregate_exchange_injector.rs | 57 +++++++++++-------- .../aggregator/transform_partition_bucket.rs | 28 ++++++--- .../partition/window_partition_exchange.rs | 3 +- ...window_partition_partial_top_n_exchange.rs | 3 +- .../flight/v1/exchange/exchange_sink.rs | 40 +++++++------ 6 files changed, 80 insertions(+), 60 deletions(-) diff --git a/src/query/pipeline/core/src/processors/shuffle_processor.rs b/src/query/pipeline/core/src/processors/shuffle_processor.rs index 55b492064cb42..6dd9baba69e69 100644 --- a/src/query/pipeline/core/src/processors/shuffle_processor.rs +++ b/src/query/pipeline/core/src/processors/shuffle_processor.rs @@ -40,7 +40,7 @@ pub trait Exchange: Send + Sync + 'static { const MULTIWAY_SORT: bool = false; const SKIP_EMPTY_DATA_BLOCK: bool = false; - fn partition(&self, data_block: DataBlock, n: usize) -> Result>; + fn partition(&self, data_block: DataBlock, n: usize) -> Result>; fn sorting_function(_: &DataBlock, _: &DataBlock) -> Ordering { unimplemented!() @@ -259,7 +259,12 @@ impl Processor for PartitionProcessor { let partitioned = self.exchange.partition(block, self.outputs.len())?; - for (index, block) in partitioned.into_iter() { + if partitioned.is_empty() { + return Ok(()); + } + + assert_eq!(partitioned.len(), self.outputs.len()); + for (index, block) in partitioned.into_iter().enumerate() { self.partitioned_data[index] = Some(block); } } 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 37f254bb820b1..0c62741d35e64 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 @@ -128,10 +128,10 @@ impl FlightExchange { } impl FlightExchange { - fn default_partition(&self, data_block: DataBlock) -> Result> { + fn default_partition(&self, data_block: DataBlock) -> Result> { if self.rev_bucket_lookup.is_empty() { let data_block = serialize_block(0, data_block, &self.options)?; - return Ok(vec![(0, data_block)]); + return Ok(vec![data_block]); } let data_blocks = self.shuffle_scatter.execute(data_block)?; @@ -139,12 +139,11 @@ impl FlightExchange { let mut blocks = Vec::with_capacity(data_blocks.len()); for (idx, data_block) in data_blocks.into_iter().enumerate() { if self.rev_bucket_lookup[idx] == self.local_id { - blocks.push((idx, data_block)); + blocks.push(data_block); continue; } - let data_block = serialize_block(0, data_block, &self.options)?; - blocks.push((idx, data_block)); + blocks.push(serialize_block(0, data_block, &self.options)?); } Ok(blocks) @@ -155,7 +154,7 @@ impl Exchange for FlightExchange { const NAME: &'static str = "AggregateExchange"; const MULTIWAY_SORT: bool = MULTIWAY_SORT; - fn partition(&self, mut data_block: DataBlock, n: usize) -> Result> { + fn partition(&self, mut data_block: DataBlock, n: usize) -> Result> { let Some(meta) = data_block.take_meta() else { if data_block.is_empty() { return Ok(vec![]); @@ -179,21 +178,30 @@ impl Exchange for FlightExchange { AggregateMeta::InFlightPayload(_) => unreachable!(), AggregateMeta::SpilledPayload(v) => match self.bucket_lookup.get(&v.destination_node) { None => unreachable!(), - Some(idx) => match v.destination_node == self.local_id { - true => Ok(vec![( - *idx, - DataBlock::empty_with_meta(AggregateMeta::create_spilled_payload(v)), - )]), - false => { - let block_number = compute_block_number(-1, v.max_partition_count)?; - let block = - DataBlock::empty_with_meta(AggregateMeta::create_spilled_payload(v)); - Ok(vec![( - *idx, - serialize_block(block_number, block, &self.options)?, - )]) + Some(idx) => { + let block_num = compute_block_number(-1, v.max_partition_count)?; + + let mut blocks = Vec::with_capacity(n); + for _index in 0..n { + blocks.push(DataBlock::empty_with_meta(ExchangeSerializeMeta::create( + block_num, + Vec::with_capacity(0), + ))); } - }, + + blocks[*idx] = match v.destination_node == self.local_id { + true => { + DataBlock::empty_with_meta(AggregateMeta::create_spilled_payload(v)) + } + false => serialize_block( + block_num, + DataBlock::empty_with_meta(AggregateMeta::create_spilled_payload(v)), + &self.options, + )?, + }; + + Ok(blocks) + } }, AggregateMeta::AggregatePayload(p) => { if p.payload.len() == 0 { @@ -203,13 +211,12 @@ impl Exchange for FlightExchange { let mut blocks = Vec::with_capacity(n); for (idx, payload) in scatter_payload(p.payload, n)?.into_iter().enumerate() { if self.rev_bucket_lookup[idx] == self.local_id { - blocks.push(( - idx, - DataBlock::empty_with_meta(AggregateMeta::create_agg_payload( + blocks.push(DataBlock::empty_with_meta( + AggregateMeta::create_agg_payload( payload, p.partition, p.max_partition_count, - )), + ), )); continue; @@ -222,7 +229,7 @@ impl Exchange for FlightExchange { let block_number = compute_block_number(p.partition, p.max_partition_count)?; let data_block = serialize_block(block_number, data_block, &self.options)?; - blocks.push((idx, data_block)); + blocks.push(data_block); } Ok(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 index 7f999034434cd..92687a786e128 100644 --- 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 @@ -139,6 +139,7 @@ impl TransformPartitionDispatch { for index in 0..self.inputs.len() { if self.inputs[index].port.is_finished() { + self.inputs[index].max_partition = self.max_partition; self.inputs[index].partition = self.max_partition as isize; continue; } @@ -172,10 +173,18 @@ impl TransformPartitionDispatch { if before_max_partition_count > 0 && before_max_partition_count != self.max_partition { // set need data for inputs which is less than the max partition for i in 0..index { - if !self.inputs[i].port.is_finished() - && !self.inputs[i].port.has_data() - && self.inputs[i].max_partition != self.max_partition - { + if self.inputs[i].port.is_finished() { + self.inputs[index].max_partition = self.max_partition; + self.inputs[index].partition = self.max_partition as isize; + continue; + } + + // It will soon wake itself up + if self.inputs[i].port.has_data() { + continue; + } + + if self.inputs[i].max_partition != self.max_partition { self.inputs[i].port.set_need_data(); initialized_all_inputs = false; } @@ -208,12 +217,15 @@ impl TransformPartitionDispatch { } fn working_partition(&mut self) -> Option { - self.inputs.iter().map(|x| x.partition).min() + self.inputs + .iter() + .filter(|x| !x.port.is_finished()) + .map(|x| x.partition) + .min() } fn fetch_ready_partition(&mut self) -> Result<()> { if let Some(ready_partition) = self.ready_partition() { - // TODO: read spill data let ready_partition = self.partitions.take_partition(ready_partition); for (meta, data_block) in ready_partition { @@ -301,7 +313,6 @@ impl Processor for TransformPartitionDispatch { } all_inputs_is_finished = false; - if self.inputs[index].partition > working_partition { continue; } @@ -326,12 +337,13 @@ impl Processor for TransformPartitionDispatch { continue; } - has_data = true; if output.can_push() { if let Some(block) = self.outputs_data[idx].pop_front() { output.push_data(Ok(block)); } } + + has_data |= !self.outputs_data[idx].is_empty(); } if all_inputs_is_finished && !has_data { diff --git a/src/query/service/src/pipelines/processors/transforms/window/partition/window_partition_exchange.rs b/src/query/service/src/pipelines/processors/transforms/window/partition/window_partition_exchange.rs index 3aa40cb5e6504..bf6ea988acf65 100644 --- a/src/query/service/src/pipelines/processors/transforms/window/partition/window_partition_exchange.rs +++ b/src/query/service/src/pipelines/processors/transforms/window/partition/window_partition_exchange.rs @@ -38,7 +38,7 @@ impl WindowPartitionExchange { impl Exchange for WindowPartitionExchange { const NAME: &'static str = "Window"; - fn partition(&self, data_block: DataBlock, n: usize) -> Result> { + fn partition(&self, data_block: DataBlock, n: usize) -> Result> { let num_rows = data_block.num_rows(); // Extract the columns used for hash computation. @@ -67,7 +67,6 @@ impl Exchange for WindowPartitionExchange { .into_iter() .map(WindowPartitionMeta::create) .map(DataBlock::empty_with_meta) - .enumerate() .collect()) } } diff --git a/src/query/service/src/pipelines/processors/transforms/window/partition/window_partition_partial_top_n_exchange.rs b/src/query/service/src/pipelines/processors/transforms/window/partition/window_partition_partial_top_n_exchange.rs index c5e5465cba942..283b2c78852c9 100644 --- a/src/query/service/src/pipelines/processors/transforms/window/partition/window_partition_partial_top_n_exchange.rs +++ b/src/query/service/src/pipelines/processors/transforms/window/partition/window_partition_partial_top_n_exchange.rs @@ -69,7 +69,7 @@ impl Exchange for WindowPartitionTopNExchange { const NAME: &'static str = "WindowTopN"; const SKIP_EMPTY_DATA_BLOCK: bool = true; - fn partition(&self, block: DataBlock, n: usize) -> Result> { + fn partition(&self, block: DataBlock, n: usize) -> Result> { let partition_permutation = self.partition_permutation(&block); // Partition the data blocks to different processors. @@ -83,7 +83,6 @@ impl Exchange for WindowPartitionTopNExchange { .into_iter() .map(WindowPartitionMeta::create) .map(DataBlock::empty_with_meta) - .enumerate() .collect()) } } diff --git a/src/query/service/src/servers/flight/v1/exchange/exchange_sink.rs b/src/query/service/src/servers/flight/v1/exchange/exchange_sink.rs index be5a45e51bf9f..b7d9e4abfe164 100644 --- a/src/query/service/src/servers/flight/v1/exchange/exchange_sink.rs +++ b/src/query/service/src/servers/flight/v1/exchange/exchange_sink.rs @@ -50,30 +50,28 @@ impl ExchangeSink { ))); } - if !params.ignore_exchange { - let settings = ctx.get_settings(); - let compression = settings.get_query_flight_compression()?; + let settings = ctx.get_settings(); + let compression = settings.get_query_flight_compression()?; - let nodes = vec![]; - match params.enable_multiway_sort { - true => pipeline.exchange( - 1, - FlightExchange::::create( - nodes, - compression, - Arc::new(Box::new(MergeFlightScatter)), - ), + let nodes = vec![]; + match params.enable_multiway_sort { + true => pipeline.exchange( + 1, + FlightExchange::::create( + nodes, + compression, + Arc::new(Box::new(MergeFlightScatter)), ), - false => pipeline.exchange( - 1, - FlightExchange::::create( - nodes, - compression, - Arc::new(Box::new(MergeFlightScatter)), - ), + ), + false => pipeline.exchange( + 1, + FlightExchange::::create( + nodes, + compression, + Arc::new(Box::new(MergeFlightScatter)), ), - }; - } + ), + }; assert_eq!(senders.len(), 1); pipeline.add_pipe(Pipe::create(1, 0, vec![create_writer_item( From 417f64dcb40fa7e4aea4f3dfca1232a4300e29e9 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sun, 16 Mar 2025 15:43:39 +0800 Subject: [PATCH 12/76] refactor(query): refactor aggreagte spill code --- .../aggregator/transform_partition_bucket.rs | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) 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 index 92687a786e128..8512fa00bff3e 100644 --- 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 @@ -174,8 +174,8 @@ impl TransformPartitionDispatch { // set need data for inputs which is less than the max partition for i in 0..index { if self.inputs[i].port.is_finished() { - self.inputs[index].max_partition = self.max_partition; - self.inputs[index].partition = self.max_partition as isize; + self.inputs[i].max_partition = self.max_partition; + self.inputs[i].partition = self.max_partition as isize; continue; } @@ -217,15 +217,11 @@ impl TransformPartitionDispatch { } fn working_partition(&mut self) -> Option { - self.inputs - .iter() - .filter(|x| !x.port.is_finished()) - .map(|x| x.partition) - .min() + self.inputs.iter().map(|x| x.partition).min() } fn fetch_ready_partition(&mut self) -> Result<()> { - if let Some(ready_partition) = self.ready_partition() { + while let Some(ready_partition) = self.ready_partition() { let ready_partition = self.partitions.take_partition(ready_partition); for (meta, data_block) in ready_partition { From 4193c6c6b9c6a30d564b8f5a0f5b1a2998cd7d60 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sun, 16 Mar 2025 17:22:00 +0800 Subject: [PATCH 13/76] refactor(query): refactor aggreagte spill code --- .../aggregator/aggregate_exchange_injector.rs | 11 ++++------- .../aggregator/transform_partition_bucket.rs | 2 +- 2 files changed, 5 insertions(+), 8 deletions(-) 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 0c62741d35e64..d81d2301abf99 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 @@ -156,6 +156,7 @@ impl Exchange for FlightExchange { fn partition(&self, mut data_block: DataBlock, n: usize) -> Result> { let Some(meta) = data_block.take_meta() else { + // only exchange data if data_block.is_empty() { return Ok(vec![]); } @@ -163,16 +164,12 @@ impl Exchange for FlightExchange { return self.default_partition(data_block); }; - let Some(meta) = AggregateMeta::downcast_from(meta) else { - if data_block.is_empty() { - return Ok(vec![]); - } - - return self.default_partition(data_block); + let Some(_) = AggregateMeta::downcast_ref_from(&meta) else { + return self.default_partition(data_block.add_meta(Some(meta))?); }; assert_eq!(self.bucket_lookup.len(), n); - match meta { + match AggregateMeta::downcast_from(meta).unwrap() { AggregateMeta::Serialized(_) => unreachable!(), AggregateMeta::FinalPartition => unreachable!(), AggregateMeta::InFlightPayload(_) => unreachable!(), 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 index 8512fa00bff3e..07984ce355a88 100644 --- 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 @@ -170,7 +170,7 @@ impl TransformPartitionDispatch { } // max partition count change - if before_max_partition_count > 0 && before_max_partition_count != self.max_partition { + if before_max_partition_count != self.max_partition { // set need data for inputs which is less than the max partition for i in 0..index { if self.inputs[i].port.is_finished() { From 20ca2a91d5db56bdf4a0a222b3c13c3c5e070abb Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sun, 16 Mar 2025 19:40:00 +0800 Subject: [PATCH 14/76] refactor(query): refactor aggreagte spill code --- .../aggregator/aggregate_exchange_injector.rs | 48 ++++++++++--------- .../aggregator/transform_partition_bucket.rs | 24 ++++++---- 2 files changed, 39 insertions(+), 33 deletions(-) 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 d81d2301abf99..d8a485575eea7 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 @@ -168,38 +168,40 @@ impl Exchange for FlightExchange { return self.default_partition(data_block.add_meta(Some(meta))?); }; + assert!(MULTIWAY_SORT); assert_eq!(self.bucket_lookup.len(), n); match AggregateMeta::downcast_from(meta).unwrap() { AggregateMeta::Serialized(_) => unreachable!(), AggregateMeta::FinalPartition => unreachable!(), AggregateMeta::InFlightPayload(_) => unreachable!(), - AggregateMeta::SpilledPayload(v) => match self.bucket_lookup.get(&v.destination_node) { - None => unreachable!(), - Some(idx) => { - let block_num = compute_block_number(-1, v.max_partition_count)?; - - let mut blocks = Vec::with_capacity(n); - for _index in 0..n { - blocks.push(DataBlock::empty_with_meta(ExchangeSerializeMeta::create( + AggregateMeta::SpilledPayload(v) => { + let block_num = compute_block_number(-1, v.max_partition_count)?; + + let mut blocks = Vec::with_capacity(n); + for local in &self.rev_bucket_lookup { + blocks.push(match *local == self.local_id { + true => DataBlock::empty_with_meta( + AggregateMeta::create_in_flight_payload(-1, v.max_partition_count), + ), + false => DataBlock::empty_with_meta(ExchangeSerializeMeta::create( block_num, Vec::with_capacity(0), - ))); - } + )), + }); + } - blocks[*idx] = match v.destination_node == self.local_id { - true => { - DataBlock::empty_with_meta(AggregateMeta::create_spilled_payload(v)) - } - false => serialize_block( - block_num, - DataBlock::empty_with_meta(AggregateMeta::create_spilled_payload(v)), - &self.options, - )?, - }; + let index = *self.bucket_lookup.get(&v.destination_node).unwrap(); + blocks[index] = match v.destination_node == self.local_id { + true => DataBlock::empty_with_meta(AggregateMeta::create_spilled_payload(v)), + false => serialize_block( + block_num, + DataBlock::empty_with_meta(AggregateMeta::create_spilled_payload(v)), + &self.options, + )?, + }; - Ok(blocks) - } - }, + Ok(blocks) + } AggregateMeta::AggregatePayload(p) => { if p.payload.len() == 0 { return Ok(vec![]); 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 index 07984ce355a88..634fba00c01a7 100644 --- 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 @@ -599,21 +599,25 @@ struct AlignedPartitions { impl AlignedPartitions { pub fn add_data(&mut self, meta: AggregateMeta, block: DataBlock) -> (isize, usize) { - let (partition, max_partition) = match &meta { + let (is_empty, partition, max_partition) = match &meta { AggregateMeta::Serialized(_) => unreachable!(), AggregateMeta::FinalPartition => unreachable!(), - AggregateMeta::SpilledPayload(v) => (v.partition, v.max_partition_count), - AggregateMeta::AggregatePayload(v) => (v.partition, v.max_partition_count), - AggregateMeta::InFlightPayload(v) => (v.partition, v.max_partition), + AggregateMeta::SpilledPayload(v) => (false, v.partition, v.max_partition_count), + AggregateMeta::AggregatePayload(v) => { + (v.payload.len() == 0, v.partition, v.max_partition_count) + } + AggregateMeta::InFlightPayload(v) => (block.is_empty(), v.partition, v.max_partition), }; assert_eq!(max_partition, self.max_partition); - match self.data.entry(partition) { - std::collections::btree_map::Entry::Vacant(v) => { - v.insert(vec![(meta, block)]); - } - std::collections::btree_map::Entry::Occupied(mut v) => { - v.get_mut().push((meta, block)); + if !is_empty { + match self.data.entry(partition) { + std::collections::btree_map::Entry::Vacant(v) => { + v.insert(vec![(meta, block)]); + } + std::collections::btree_map::Entry::Occupied(mut v) => { + v.get_mut().push((meta, block)); + } } } From 472aa3713b275071bae77ee6390ecb7da7621da3 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 18 Mar 2025 00:48:32 +0800 Subject: [PATCH 15/76] refactor(query): refactor aggreagte spill code --- .../core/src/processors/shuffle_processor.rs | 35 +- .../aggregator/aggregate_exchange_injector.rs | 108 ++-- .../transforms/aggregator/aggregate_meta.rs | 39 +- .../serde/transform_aggregate_serializer.rs | 4 +- .../serde/transform_spill_reader.rs | 3 +- .../aggregator/transform_aggregate_final.rs | 2 +- .../aggregator/transform_aggregate_partial.rs | 19 +- .../aggregator/transform_partition_bucket.rs | 528 +++++++++--------- .../flight/v1/exchange/exchange_transform.rs | 41 +- .../v1/exchange/serde/exchange_serializer.rs | 129 +---- .../servers/flight/v1/exchange/serde/mod.rs | 2 - 11 files changed, 429 insertions(+), 481 deletions(-) diff --git a/src/query/pipeline/core/src/processors/shuffle_processor.rs b/src/query/pipeline/core/src/processors/shuffle_processor.rs index 6dd9baba69e69..4ecf295abf00f 100644 --- a/src/query/pipeline/core/src/processors/shuffle_processor.rs +++ b/src/query/pipeline/core/src/processors/shuffle_processor.rs @@ -45,6 +45,23 @@ pub trait Exchange: Send + Sync + 'static { fn sorting_function(_: &DataBlock, _: &DataBlock) -> Ordering { unimplemented!() } + + fn multiway_pick(data_blocks: &mut [Option]) -> Option { + let position = + data_blocks + .iter() + .enumerate() + .filter_map(|(idx, x)| x.as_ref().map(|d| (idx, d))) + .min_by(|(left_idx, left_block), (right_idx, right_block)| { + match Self::sorting_function(left_block, right_block) { + Ordering::Less => Ordering::Less, + Ordering::Greater => Ordering::Greater, + Ordering::Equal => left_idx.cmp(right_idx), + } + }); + + position.map(|(idx, _)| idx) + } } pub struct ShuffleProcessor { @@ -290,22 +307,6 @@ impl MergePartitionProcessor { _phantom_data: Default::default(), })) } - - fn multiway_pick(&self, data_blocks: &[Option]) -> Option { - let position = data_blocks - .iter() - .enumerate() - .filter_map(|(idx, x)| x.as_ref().map(|d| (idx, d))) - .min_by(|(left_idx, left_block), (right_idx, right_block)| { - match T::sorting_function(left_block, right_block) { - Ordering::Less => Ordering::Less, - Ordering::Greater => Ordering::Greater, - Ordering::Equal => left_idx.cmp(right_idx), - } - }); - - position.map(|(idx, _)| idx) - } } impl Processor for MergePartitionProcessor { @@ -363,7 +364,7 @@ impl Processor for MergePartitionProcessor { } if need_pick_block_to_push { - if let Some(pick_index) = self.multiway_pick(&self.inputs_data) { + if let Some(pick_index) = T::multiway_pick(&mut self.inputs_data) { if let Some(block) = self.inputs_data[pick_index].take() { self.output.push_data(Ok(block)); return Ok(Event::NeedConsume); 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 d8a485575eea7..6c9f06b5468ee 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 @@ -130,7 +130,7 @@ impl FlightExchange { impl FlightExchange { fn default_partition(&self, data_block: DataBlock) -> Result> { if self.rev_bucket_lookup.is_empty() { - let data_block = serialize_block(0, data_block, &self.options)?; + let data_block = serialize_block(0, 0, data_block, &self.options)?; return Ok(vec![data_block]); } @@ -143,7 +143,7 @@ impl FlightExchange { continue; } - blocks.push(serialize_block(0, data_block, &self.options)?); + blocks.push(serialize_block(0, 0, data_block, &self.options)?); } Ok(blocks) @@ -157,7 +157,7 @@ impl Exchange for FlightExchange { fn partition(&self, mut data_block: DataBlock, n: usize) -> Result> { let Some(meta) = data_block.take_meta() else { // only exchange data - if data_block.is_empty() { + if !MULTIWAY_SORT && data_block.is_empty() { return Ok(vec![]); } @@ -175,16 +175,17 @@ impl Exchange for FlightExchange { AggregateMeta::FinalPartition => unreachable!(), AggregateMeta::InFlightPayload(_) => unreachable!(), AggregateMeta::SpilledPayload(v) => { - let block_num = compute_block_number(-1, v.max_partition_count)?; + let block_num = compute_block_number(-1, v.max_partition)?; let mut blocks = Vec::with_capacity(n); for local in &self.rev_bucket_lookup { blocks.push(match *local == self.local_id { true => DataBlock::empty_with_meta( - AggregateMeta::create_in_flight_payload(-1, v.max_partition_count), + AggregateMeta::create_in_flight_payload(-1, v.max_partition), ), false => DataBlock::empty_with_meta(ExchangeSerializeMeta::create( block_num, + v.global_max_partition, Vec::with_capacity(0), )), }); @@ -195,6 +196,7 @@ impl Exchange for FlightExchange { true => DataBlock::empty_with_meta(AggregateMeta::create_spilled_payload(v)), false => serialize_block( block_num, + v.global_max_partition, DataBlock::empty_with_meta(AggregateMeta::create_spilled_payload(v)), &self.options, )?, @@ -203,9 +205,9 @@ impl Exchange for FlightExchange { Ok(blocks) } AggregateMeta::AggregatePayload(p) => { - if p.payload.len() == 0 { - return Ok(vec![]); - } + // if p.payload.len() == 0 { + // return Ok(vec![]); + // } let mut blocks = Vec::with_capacity(n); for (idx, payload) in scatter_payload(p.payload, n)?.into_iter().enumerate() { @@ -214,20 +216,29 @@ impl Exchange for FlightExchange { AggregateMeta::create_agg_payload( payload, p.partition, - p.max_partition_count, + p.max_partition, ), )); continue; } - let data_block = payload.aggregate_flush_all()?; + let data_block = match payload.len() == 0 { + true => DataBlock::empty(), + false => payload.aggregate_flush_all()?, + }; + let data_block = data_block.add_meta(Some( - AggregateMeta::create_in_flight_payload(p.partition, p.max_partition_count), + AggregateMeta::create_in_flight_payload(p.partition, p.max_partition), ))?; - let block_number = compute_block_number(p.partition, p.max_partition_count)?; - let data_block = serialize_block(block_number, data_block, &self.options)?; + let block_number = compute_block_number(p.partition, p.max_partition)?; + let data_block = serialize_block( + block_number, + p.global_max_partition, + data_block, + &self.options, + )?; blocks.push(data_block); } @@ -236,29 +247,60 @@ impl Exchange for FlightExchange { } } - fn sorting_function(left_block: &DataBlock, right_block: &DataBlock) -> Ordering { - let Some(left_meta) = left_block.get_meta() else { - return Ordering::Equal; - }; - let Some(left_meta) = ExchangeSerializeMeta::downcast_ref_from(left_meta) else { - return Ordering::Equal; - }; + fn multiway_pick(data_blocks: &mut [Option]) -> Option { + let mut global_max_partition = 0_usize; + let global_max_partition_ref = &mut global_max_partition; - let Some(right_meta) = right_block.get_meta() else { - return Ordering::Equal; - }; - let Some(right_meta) = ExchangeSerializeMeta::downcast_ref_from(right_meta) else { - return Ordering::Equal; - }; + let min_position = data_blocks + .iter() + .enumerate() + .filter_map(|(idx, x)| x.as_ref().map(|d| (idx, d))) + .min_by(move |(left_idx, left_block), (right_idx, right_block)| { + let Some(left_meta) = left_block.get_meta() else { + return Ordering::Equal; + }; + let Some(left_meta) = ExchangeSerializeMeta::downcast_ref_from(left_meta) else { + return Ordering::Equal; + }; - let (l_partition, l_max_partition) = restore_block_number(left_meta.block_number); - let (r_partition, r_max_partition) = restore_block_number(right_meta.block_number); + let Some(right_meta) = right_block.get_meta() else { + return Ordering::Equal; + }; + let Some(right_meta) = ExchangeSerializeMeta::downcast_ref_from(right_meta) else { + return Ordering::Equal; + }; + + let max_block_number = left_meta.max_block_number.max(right_meta.max_block_number); + *global_max_partition_ref = (*global_max_partition_ref).max(max_block_number); + let (l_partition, l_max_partition) = restore_block_number(left_meta.block_number); + let (r_partition, r_max_partition) = restore_block_number(right_meta.block_number); + + // ORDER BY max_partition asc, partition asc, idx asc + match l_max_partition.cmp(&r_max_partition) { + Ordering::Less => Ordering::Less, + Ordering::Greater => Ordering::Greater, + Ordering::Equal => match l_partition.cmp(&r_partition) { + Ordering::Less => Ordering::Less, + Ordering::Greater => Ordering::Greater, + Ordering::Equal => left_idx.cmp(right_idx), + }, + } + }) + .map(|(idx, _)| idx); - // ORDER BY max_partition asc, partition asc - match l_max_partition.cmp(&r_max_partition) { - Ordering::Less => Ordering::Less, - Ordering::Greater => Ordering::Greater, - Ordering::Equal => l_partition.cmp(&r_partition), + if let Some(min_pos) = min_position { + if global_max_partition == 0 { + return Some(min_pos); + } + + if let Some(mut block) = data_blocks[min_pos].take() { + let mut meta = + ExchangeSerializeMeta::downcast_from(block.take_meta().unwrap()).unwrap(); + meta.max_block_number = global_max_partition; + data_blocks[min_pos] = Some(block.add_meta(Some(Box::new(meta))).unwrap()); + } } + + min_position } } 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 38d0d63ee2090..7c55911af9889 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 @@ -37,7 +37,8 @@ pub struct SerializedPayload { pub bucket: isize, pub data_block: DataBlock, // use for new agg_hashtable - pub max_partition_count: usize, + pub max_partition: usize, + pub global_max_partition: usize, } impl SerializedPayload { @@ -113,20 +114,23 @@ pub struct SpilledPayload { pub location: String, pub data_range: Range, pub destination_node: String, - pub max_partition_count: usize, + pub max_partition: usize, + pub global_max_partition: usize, } pub struct AggregatePayload { pub partition: isize, pub payload: Payload, // use for new agg_hashtable - pub max_partition_count: usize, + pub max_partition: usize, + pub global_max_partition: usize, } #[derive(serde::Serialize, serde::Deserialize)] pub struct InFlightPayload { pub partition: isize, pub max_partition: usize, + pub global_max_partition: usize, } pub struct FinalPayload { @@ -146,12 +150,13 @@ impl AggregateMeta { pub fn create_agg_payload( payload: Payload, partition: isize, - max_partition_count: usize, + max_partition: usize, ) -> BlockMetaInfoPtr { Box::new(AggregateMeta::AggregatePayload(AggregatePayload { payload, partition, - max_partition_count, + max_partition, + global_max_partition: max_partition, })) } @@ -159,18 +164,20 @@ impl AggregateMeta { Box::new(AggregateMeta::InFlightPayload(InFlightPayload { partition, max_partition, + global_max_partition: max_partition, })) } pub fn create_serialized( bucket: isize, block: DataBlock, - max_partition_count: usize, + max_partition: usize, ) -> BlockMetaInfoPtr { Box::new(AggregateMeta::Serialized(SerializedPayload { bucket, data_block: block, - max_partition_count, + max_partition, + global_max_partition: max_partition, })) } @@ -181,6 +188,24 @@ impl AggregateMeta { pub fn create_final() -> BlockMetaInfoPtr { Box::new(AggregateMeta::FinalPartition) } + + pub fn set_global_max_partition(&mut self, global_max_partition: usize) { + match self { + AggregateMeta::Serialized(v) => { + v.global_max_partition = global_max_partition; + } + AggregateMeta::SpilledPayload(v) => { + v.global_max_partition = global_max_partition; + } + AggregateMeta::AggregatePayload(v) => { + v.global_max_partition = global_max_partition; + } + AggregateMeta::InFlightPayload(v) => { + v.global_max_partition = global_max_partition; + } + AggregateMeta::FinalPartition => unreachable!(), + } + } } impl Debug for AggregateMeta { 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 32db416fa9284..9ba343c199d8b 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 @@ -231,7 +231,7 @@ impl SerializeAggregateStream { Ok(Some(block.add_meta(Some( AggregateSerdeMeta::create_agg_payload( p.partition, - p.max_partition_count, + p.max_partition, false, ), ))?)) @@ -244,7 +244,7 @@ impl SerializeAggregateStream { Ok(Some(block.add_meta(Some( AggregateSerdeMeta::create_agg_payload( p.partition, - p.max_partition_count, + p.max_partition, true, ), ))?)) 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 f0a0cda73b6ae..4efa995e3ca56 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 @@ -204,7 +204,8 @@ impl TransformSpillReader { AggregateMeta::Serialized(SerializedPayload { bucket: payload.partition, data_block: block, - max_partition_count: payload.max_partition_count, + max_partition: payload.max_partition, + global_max_partition: payload.global_max_partition, }) } } 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 1da78cf3c5abc..f294aec338508 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 @@ -172,7 +172,7 @@ impl TransformFinalAggregate { )?; hashtable.payload.mark_min_cardinality(); - assert_eq!(hashtable.payload.len(), 1); + assert_eq!(hashtable.payload.payloads.len(), 1); Ok(hashtable.payload.payloads.pop().unwrap()) } } 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 e91ffe141e744..6ed460440b78e 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 @@ -283,15 +283,13 @@ impl AccumulatingTransform for TransformPartialAggregate { ); for (partition, payload) in hashtable.payload.payloads.into_iter().enumerate() { - if payload.len() != 0 { - blocks.push(DataBlock::empty_with_meta( - AggregateMeta::create_agg_payload( - payload, - partition as isize, - partition_count, - ), - )); - } + blocks.push(DataBlock::empty_with_meta( + AggregateMeta::create_agg_payload( + payload, + partition as isize, + partition_count, + ), + )); } blocks @@ -363,7 +361,8 @@ impl AccumulatingTransform for TransformPartialAggregate { data_range: last_offset as u64..writer.write_bytes() as u64, destination_node: self.configure_peer_nodes[spilling_state.working_bucket] .clone(), - max_partition_count: max_partition, + max_partition, + global_max_partition: max_partition, }; self.spill_blocks.push(DataBlock::empty_with_meta( 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 index 634fba00c01a7..bd21c721be2a3 100644 --- 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 @@ -13,6 +13,7 @@ // limitations under the License. use std::any::Any; +use std::cmp::Ordering; use std::collections::hash_map::Entry; use std::collections::BTreeMap; use std::collections::HashMap; @@ -34,6 +35,7 @@ use databend_common_expression::Payload; use databend_common_expression::PayloadFlushState; use databend_common_expression::ProbeState; use databend_common_pipeline_core::processors::Event; +use databend_common_pipeline_core::processors::Exchange; use databend_common_pipeline_core::processors::InputPort; use databend_common_pipeline_core::processors::OutputPort; use databend_common_pipeline_core::processors::Processor; @@ -51,49 +53,21 @@ use crate::pipelines::processors::transforms::aggregator::aggregate_meta::Aggreg use crate::pipelines::processors::transforms::aggregator::AggregatorParams; static SINGLE_LEVEL_BUCKET_NUM: isize = -1; -static MAX_PARTITION_COUNT: usize = 128; - -struct InputPortState { - port: Arc, - partition: isize, - max_partition: usize, -} - -impl Debug for InputPortState { - fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { - f.debug_struct("InputPortState") - .field("bucket", &self.partition) - .field("max_partition_count", &self.max_partition) - .finish() - } -} pub struct TransformPartitionDispatch { outputs: Vec>, - inputs: Vec, + input: Arc, outputs_data: Vec>, output_index: usize, + initialized_input: bool, + max_partition: usize, - initialized_all_inputs: bool, + working_partition: isize, partitions: Partitions, } impl TransformPartitionDispatch { - pub fn create( - input_nums: usize, - output_nums: usize, - params: Arc, - ) -> Result { - let mut inputs = Vec::with_capacity(input_nums); - - for _index in 0..input_nums { - inputs.push(InputPortState { - partition: -1, - port: InputPort::create(), - max_partition: 0, - }); - } - + pub fn create(output_nums: usize, params: Arc) -> Result { let mut outputs = Vec::with_capacity(output_nums); let mut outputs_data = Vec::with_capacity(output_nums); @@ -102,30 +76,20 @@ impl TransformPartitionDispatch { outputs_data.push(VecDeque::new()); } - let max_partition = match params.cluster_aggregator { - true => MAX_PARTITION_COUNT, - false => 0, - }; - Ok(TransformPartitionDispatch { - inputs, outputs, outputs_data, - max_partition, + input: InputPort::create(), output_index: 0, - initialized_all_inputs: false, + max_partition: 0, + initialized_input: false, partitions: Partitions::create_unaligned(params), + working_partition: 0, }) } - 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_input(&self) -> Arc { + self.input.clone() } pub fn get_outputs(&self) -> Vec> { @@ -134,95 +98,36 @@ impl TransformPartitionDispatch { // Align each input's max_partition to the maximum max_partition. // If an input's max_partition is smaller than the maximum, continuously fetch its data until either the stream ends or its max_partition reaches/exceeds the maximum value. - fn initialize_all_inputs(&mut self) -> Result { - let mut initialized_all_inputs = true; - - for index in 0..self.inputs.len() { - if self.inputs[index].port.is_finished() { - self.inputs[index].max_partition = self.max_partition; - self.inputs[index].partition = self.max_partition as isize; - continue; - } - - if self.inputs[index].max_partition > 0 - && self.inputs[index].partition > SINGLE_LEVEL_BUCKET_NUM - && self.inputs[index].max_partition == self.max_partition - { - continue; - } - - if !self.inputs[index].port.has_data() { - self.inputs[index].port.set_need_data(); - initialized_all_inputs = false; - continue; - } - - let before_max_partition_count = self.max_partition; - - self.add_block(index, self.inputs[index].port.pull_data().unwrap()?)?; - - // we need pull all spill data in init, and data less than max partition - if self.inputs[index].partition <= SINGLE_LEVEL_BUCKET_NUM - || self.inputs[index].max_partition < self.max_partition - { - self.inputs[index].port.set_need_data(); - initialized_all_inputs = false; - } - - // max partition count change - if before_max_partition_count != self.max_partition { - // set need data for inputs which is less than the max partition - for i in 0..index { - if self.inputs[i].port.is_finished() { - self.inputs[i].max_partition = self.max_partition; - self.inputs[i].partition = self.max_partition as isize; - continue; - } - - // It will soon wake itself up - if self.inputs[i].port.has_data() { - continue; - } - - if self.inputs[i].max_partition != self.max_partition { - self.inputs[i].port.set_need_data(); - initialized_all_inputs = false; - } - } - } + fn initialize_input(&mut self) -> Result { + if self.input.is_finished() { + return Ok(true); } - Ok(initialized_all_inputs) - } - - fn add_block(&mut self, index: usize, data_block: DataBlock) -> Result<()> { - ( - self.inputs[index].partition, - self.inputs[index].max_partition, - ) = self.partitions.add_block(data_block)?; + if !self.input.has_data() { + self.input.set_need_data(); + return Ok(false); + } - self.max_partition = std::cmp::max(self.max_partition, self.inputs[index].max_partition); - Ok(()) + let data_block = self.input.pull_data().unwrap()?; + let (partition, max_partition, global_max_partition) = + self.partitions.add_block(data_block)?; + self.max_partition = global_max_partition; + Ok(partition > SINGLE_LEVEL_BUCKET_NUM && max_partition == global_max_partition) } fn ready_partition(&mut self) -> Option { - let inputs_min_partition = self.working_partition()?; let storage_min_partition = self.partitions.min_partition()?; - if storage_min_partition >= inputs_min_partition { + if storage_min_partition >= self.working_partition { return None; } Some(storage_min_partition) } - fn working_partition(&mut self) -> Option { - self.inputs.iter().map(|x| x.partition).min() - } - fn fetch_ready_partition(&mut self) -> Result<()> { - while let Some(ready_partition) = self.ready_partition() { - let ready_partition = self.partitions.take_partition(ready_partition); + while let Some(ready_partition_id) = self.ready_partition() { + let ready_partition = self.partitions.take_partition(ready_partition_id); for (meta, data_block) in ready_partition { self.outputs_data[self.output_index] @@ -263,16 +168,13 @@ impl Processor for TransformPartitionDispatch { } if all_output_finished { - for input_state in &self.inputs { - input_state.port.finish(); - } - + self.input.finish(); return Ok(Event::Finished); } // We pull the first unsplitted data block - if !self.initialized_all_inputs { - if self.initialize_all_inputs()? { + if !self.initialized_input { + if self.initialize_input()? { return Ok(Event::Sync); } @@ -290,42 +192,26 @@ impl Processor for TransformPartitionDispatch { } if !output_can_push { - for input_state in &self.inputs { - input_state.port.set_not_need_data(); - } - + self.input.set_not_need_data(); return Ok(Event::NeedConsume); } - self.fetch_ready_partition()?; - - let working_partition = self.working_partition().unwrap_or(0); - - let mut all_inputs_is_finished = true; - for index in 0..self.inputs.len() { - if self.inputs[index].port.is_finished() { - self.inputs[index].partition = self.max_partition as isize; - continue; - } - - all_inputs_is_finished = false; - if self.inputs[index].partition > working_partition { - continue; - } - - if !self.inputs[index].port.has_data() { - self.inputs[index].port.set_need_data(); - continue; - } + if self.input.is_finished() { + self.working_partition = self.max_partition as isize; + self.fetch_ready_partition()?; + } - self.add_block(index, self.inputs[index].port.pull_data().unwrap()?)?; + if self.input.has_data() { + let data_block = self.input.pull_data().unwrap()?; + let (partition, _, _) = self.partitions.add_block(data_block)?; - if self.inputs[index].partition <= working_partition { - self.inputs[index].port.set_need_data(); + if partition != self.working_partition { + // ready partition + self.fetch_ready_partition()?; } } - self.fetch_ready_partition()?; + self.input.set_need_data(); let mut has_data = false; for (idx, output) in self.outputs.iter().enumerate() { @@ -342,7 +228,7 @@ impl Processor for TransformPartitionDispatch { has_data |= !self.outputs_data[idx].is_empty(); } - if all_inputs_is_finished && !has_data { + if self.input.is_finished() && !has_data { for output in &self.outputs { output.finish(); } @@ -354,8 +240,8 @@ impl Processor for TransformPartitionDispatch { } fn process(&mut self) -> Result<()> { - if !self.initialized_all_inputs { - self.initialized_all_inputs = true; + if !self.initialized_input { + self.initialized_input = true; return self.partitions.align(self.max_partition); } @@ -363,26 +249,112 @@ impl Processor for TransformPartitionDispatch { } } +struct ResortingPartition; + +impl ResortingPartition { + fn block_number(meta: &AggregateMeta) -> (isize, usize, usize) { + match meta { + AggregateMeta::Serialized(v) => (v.bucket, v.max_partition, v.global_max_partition), + AggregateMeta::SpilledPayload(v) => { + (v.partition, v.max_partition, v.global_max_partition) + } + AggregateMeta::AggregatePayload(v) => { + (v.partition, v.max_partition, v.global_max_partition) + } + AggregateMeta::InFlightPayload(v) => { + (v.partition, v.max_partition, v.global_max_partition) + } + AggregateMeta::FinalPartition => unreachable!(), + } + } +} + +impl Exchange for ResortingPartition { + const NAME: &'static str = "PartitionResorting"; + + fn partition(&self, data_block: DataBlock, n: usize) -> Result> { + debug_assert_eq!(n, 1); + Ok(vec![data_block]) + } + + fn multiway_pick(data_blocks: &mut [Option]) -> Option { + let mut global_max_partition = 0_usize; + let global_max_partition_ref = &mut global_max_partition; + + let min_position = data_blocks + .iter() + .enumerate() + .filter_map(|(idx, x)| x.as_ref().map(|d| (idx, d))) + .min_by(move |(left_idx, left_block), (right_idx, right_block)| { + let Some(left_meta) = left_block.get_meta() else { + return Ordering::Equal; + }; + let Some(left_meta) = AggregateMeta::downcast_ref_from(left_meta) else { + return Ordering::Equal; + }; + + let Some(right_meta) = right_block.get_meta() else { + return Ordering::Equal; + }; + let Some(right_meta) = AggregateMeta::downcast_ref_from(right_meta) else { + return Ordering::Equal; + }; + + let (l_partition, l_max_partition, l_global_max_partition) = + ResortingPartition::block_number(left_meta); + let (r_partition, r_max_partition, r_global_max_partition) = + ResortingPartition::block_number(right_meta); + let global_max_partition = l_global_max_partition.max(r_global_max_partition); + *global_max_partition_ref = (*global_max_partition_ref).max(global_max_partition); + + // ORDER BY max_partition asc, partition asc, idx asc + match l_max_partition.cmp(&r_max_partition) { + Ordering::Less => Ordering::Less, + Ordering::Greater => Ordering::Greater, + Ordering::Equal => match l_partition.cmp(&r_partition) { + Ordering::Less => Ordering::Less, + Ordering::Greater => Ordering::Greater, + Ordering::Equal => left_idx.cmp(right_idx), + }, + } + }) + .map(|(idx, _)| idx); + + if let Some(min_pos) = min_position { + if global_max_partition == 0 { + return Some(min_pos); + } + + if let Some(mut block) = data_blocks[min_pos].take() { + let mut meta = AggregateMeta::downcast_from(block.take_meta().unwrap()).unwrap(); + meta.set_global_max_partition(global_max_partition); + data_blocks[min_pos] = Some(block.add_meta(Some(Box::new(meta))).unwrap()); + } + } + + min_position + } +} + pub fn build_partition_bucket( pipeline: &mut Pipeline, params: Arc, ) -> Result<()> { - let transform = TransformPartitionDispatch::create( - pipeline.output_len(), - pipeline.output_len(), - params.clone(), - )?; + let output = pipeline.output_len(); - let inputs_port = transform.get_inputs(); + // 1. reorder partition + pipeline.exchange(1, Arc::new(ResortingPartition)); + + let transform = TransformPartitionDispatch::create(output, params.clone())?; + + let input_port = transform.get_input(); let outputs_port = transform.get_outputs(); - pipeline.add_pipe(Pipe::create(inputs_port.len(), outputs_port.len(), vec![ - PipeItem::create( - ProcessorPtr::create(Box::new(transform)), - inputs_port, - outputs_port, - ), - ])); + pipeline.add_pipe(Pipe::create(1, outputs_port.len(), vec![PipeItem::create( + ProcessorPtr::create(Box::new(transform)), + vec![input_port], + outputs_port, + )])); let semaphore = Arc::new(Semaphore::new(params.max_spill_io_requests)); let operator = DataOperator::instance().spill_operator(); @@ -439,29 +411,32 @@ impl UnalignedPartitions { } } - pub fn add_data(&mut self, meta: AggregateMeta, block: DataBlock) -> (isize, usize) { + pub fn add_data(&mut self, meta: AggregateMeta, block: DataBlock) -> (isize, usize, usize) { match &meta { AggregateMeta::Serialized(_) => unreachable!(), AggregateMeta::FinalPartition => unreachable!(), AggregateMeta::SpilledPayload(payload) => { - let max_partition = payload.max_partition_count; + let max_partition = payload.max_partition; + let global_max_partition = payload.global_max_partition; self.insert_data(max_partition, meta, block); - (SINGLE_LEVEL_BUCKET_NUM, max_partition) + (SINGLE_LEVEL_BUCKET_NUM, max_partition, global_max_partition) } AggregateMeta::InFlightPayload(payload) => { let partition = payload.partition; let max_partition = payload.max_partition; + let global_max_partition = payload.global_max_partition; self.insert_data(max_partition, meta, block); - (partition, max_partition) + (partition, max_partition, global_max_partition) } AggregateMeta::AggregatePayload(payload) => { let partition = payload.partition; - let max_partition = payload.max_partition_count; - self.insert_data(max_partition, meta, block); + let max_partition = payload.max_partition; + let global_max_partition = payload.global_max_partition; - (partition, max_partition) + self.insert_data(max_partition, meta, block); + (partition, max_partition, global_max_partition) } } } @@ -520,7 +495,8 @@ impl UnalignedPartitions { partitioned.push(AggregatePayload { payload, partition: partition as isize, - max_partition_count: to, + max_partition: to, + global_max_partition: 0, }); } @@ -557,8 +533,9 @@ impl UnalignedPartitions { let payload = AggregatePayload { partition: payload.partition, - max_partition_count: payload.max_partition, + max_partition: payload.max_partition, payload: self.deserialize_flight(block)?, + global_max_partition: 0, }; let partitioned = self.partition_payload(payload, max_partitions); @@ -598,30 +575,32 @@ struct AlignedPartitions { } impl AlignedPartitions { - pub fn add_data(&mut self, meta: AggregateMeta, block: DataBlock) -> (isize, usize) { - let (is_empty, partition, max_partition) = match &meta { + pub fn add_data(&mut self, meta: AggregateMeta, block: DataBlock) -> (isize, usize, usize) { + let (partition, max_partition, global_max_partition) = match &meta { AggregateMeta::Serialized(_) => unreachable!(), AggregateMeta::FinalPartition => unreachable!(), - AggregateMeta::SpilledPayload(v) => (false, v.partition, v.max_partition_count), + AggregateMeta::SpilledPayload(v) => { + (v.partition, v.max_partition, v.global_max_partition) + } AggregateMeta::AggregatePayload(v) => { - (v.payload.len() == 0, v.partition, v.max_partition_count) + (v.partition, v.max_partition, v.global_max_partition) + } + AggregateMeta::InFlightPayload(v) => { + (v.partition, v.max_partition, v.global_max_partition) } - AggregateMeta::InFlightPayload(v) => (block.is_empty(), v.partition, v.max_partition), }; assert_eq!(max_partition, self.max_partition); - if !is_empty { - match self.data.entry(partition) { - std::collections::btree_map::Entry::Vacant(v) => { - v.insert(vec![(meta, block)]); - } - std::collections::btree_map::Entry::Occupied(mut v) => { - v.get_mut().push((meta, block)); - } + match self.data.entry(partition) { + std::collections::btree_map::Entry::Vacant(v) => { + v.insert(vec![(meta, block)]); + } + std::collections::btree_map::Entry::Occupied(mut v) => { + v.get_mut().push((meta, block)); } } - (partition, max_partition) + (partition, max_partition, global_max_partition) } } @@ -636,14 +615,20 @@ impl Partitions { Partitions::Unaligned(UnalignedPartitions::create(params)) } - fn add_data(&mut self, meta: AggregateMeta, block: DataBlock) -> (isize, usize) { + // pub fn is_empty(&self) -> bool { + // match self { + // Partitions::Aligned(c) => {} + // } + // } + + fn add_data(&mut self, meta: AggregateMeta, block: DataBlock) -> (isize, usize, usize) { match self { Partitions::Aligned(v) => v.add_data(meta, block), Partitions::Unaligned(v) => v.add_data(meta, block), } } - pub fn add_block(&mut self, mut block: DataBlock) -> Result<(isize, usize)> { + pub fn add_block(&mut self, mut block: DataBlock) -> Result<(isize, usize, usize)> { let Some(meta) = block.take_meta() else { return Err(ErrorCode::Internal( "Internal, TransformPartitionBucket only recv DataBlock with meta.", @@ -694,83 +679,78 @@ impl Partitions { #[cfg(test)] mod tests { - use databend_common_expression::types::DataType; - use databend_common_expression::types::NumberDataType; - use databend_common_expression::DataBlock; - use databend_common_expression::DataField; - use databend_common_expression::DataSchemaRefExt; - use databend_common_functions::aggregates::AggregateFunctionFactory; - - use crate::pipelines::processors::transforms::aggregator::transform_partition_bucket::UnalignedPartitions; - use crate::pipelines::processors::transforms::aggregator::transform_partition_bucket::SINGLE_LEVEL_BUCKET_NUM; - use crate::pipelines::processors::transforms::aggregator::AggregateMeta; - use crate::pipelines::processors::transforms::aggregator::AggregatorParams; - use crate::pipelines::processors::transforms::aggregator::InFlightPayload; - use crate::pipelines::processors::transforms::aggregator::SpilledPayload; - - fn create_unaligned_partitions() -> UnalignedPartitions { - let schema = DataSchemaRefExt::create(vec![ - DataField::new("a", DataType::Number(NumberDataType::Int16)), - DataField::new("b", DataType::Number(NumberDataType::Float32)), - DataField::new("c", DataType::String), - ]); - - let aggregate_functions = vec![AggregateFunctionFactory::instance() - .get("count", vec![], vec![], vec![]) - .unwrap()]; - - let params = AggregatorParams::try_create( - schema, - vec![ - DataType::Number(NumberDataType::Int16), - DataType::Number(NumberDataType::Float32), - DataType::String, - ], - &[0, 1, 2], - &aggregate_functions, - &[], - true, - false, - 1024, - 1024, - ); - - UnalignedPartitions::create(params.unwrap()) - } - - #[test] - fn test_add_data_spilled_payload() { - let mut partitions = create_unaligned_partitions(); - let max_partition = 5; - let meta = AggregateMeta::SpilledPayload(SpilledPayload { - partition: 0, - location: "".to_string(), - data_range: Default::default(), - destination_node: "".to_string(), - max_partition_count: max_partition, - }); - - let result = partitions.add_data(meta, DataBlock::empty()); - - assert_eq!(result, (SINGLE_LEVEL_BUCKET_NUM, max_partition)); - assert_eq!(partitions.data.get(&max_partition).unwrap().len(), 1); - } + // use databend_common_expression::types::DataType; + // use databend_common_expression::types::NumberDataType; + // use databend_common_expression::DataField; + // use databend_common_expression::DataSchemaRefExt; + // use databend_common_functions::aggregates::AggregateFunctionFactory; + // + // use crate::pipelines::processors::transforms::aggregator::transform_partition_bucket::UnalignedPartitions; + // use crate::pipelines::processors::transforms::aggregator::AggregatorParams; + + // fn create_unaligned_partitions() -> UnalignedPartitions { + // let schema = DataSchemaRefExt::create(vec![ + // DataField::new("a", DataType::Number(NumberDataType::Int16)), + // DataField::new("b", DataType::Number(NumberDataType::Float32)), + // DataField::new("c", DataType::String), + // ]); + // + // let aggregate_functions = vec![AggregateFunctionFactory::instance() + // .get("count", vec![], vec![], vec![]) + // .unwrap()]; + // + // let params = AggregatorParams::try_create( + // schema, + // vec![ + // DataType::Number(NumberDataType::Int16), + // DataType::Number(NumberDataType::Float32), + // DataType::String, + // ], + // &[0, 1, 2], + // &aggregate_functions, + // &[], + // true, + // false, + // 1024, + // 1024, + // ); + // + // UnalignedPartitions::create(params.unwrap()) + // } - #[test] - fn test_add_data_in_flight_payload() { - let mut partitions = create_unaligned_partitions(); - let partition = 2; - let max_partition = 8; - let meta = AggregateMeta::InFlightPayload(InFlightPayload { - partition, - max_partition, - }); - - let result = partitions.add_data(meta, DataBlock::empty()); - - assert_eq!(result, (partition, max_partition)); - assert_eq!(partitions.data.get(&max_partition).unwrap().len(), 1); - } + // #[test] + // fn test_add_data_spilled_payload() { + // let mut partitions = create_unaligned_partitions(); + // let max_partition = 5; + // let meta = AggregateMeta::SpilledPayload(SpilledPayload { + // partition: 0, + // location: "".to_string(), + // data_range: Default::default(), + // destination_node: "".to_string(), + // max_partition: max_partition, + // }); + // + // let result = partitions.add_data(meta, DataBlock::empty(), -1); + // + // assert_eq!(result, (SINGLE_LEVEL_BUCKET_NUM, max_partition)); + // assert_eq!(partitions.data.get(&max_partition).unwrap().len(), 1); + // } + // + // #[test] + // fn test_add_data_in_flight_payload() { + // let mut partitions = create_unaligned_partitions(); + // let partition = 2; + // let max_partition = 8; + // let meta = AggregateMeta::InFlightPayload(InFlightPayload { + // partition, + // max_partition, + // }); + // + // let result = partitions.add_data(meta, DataBlock::empty(), -1); + // + // assert_eq!(result, (partition, max_partition)); + // assert_eq!(partitions.data.get(&max_partition).unwrap().len(), 1); + // } #[test] fn test_add_data_aggregate_payload() { diff --git a/src/query/service/src/servers/flight/v1/exchange/exchange_transform.rs b/src/query/service/src/servers/flight/v1/exchange/exchange_transform.rs index 089b28720ae6c..b0dcefdba431f 100644 --- a/src/query/service/src/servers/flight/v1/exchange/exchange_transform.rs +++ b/src/query/service/src/servers/flight/v1/exchange/exchange_transform.rs @@ -16,7 +16,6 @@ use std::sync::Arc; use databend_common_catalog::table_context::TableContext; use databend_common_exception::Result; -use databend_common_pipeline_core::processors::create_resize_item; use databend_common_pipeline_core::Pipe; use databend_common_pipeline_core::Pipeline; use databend_common_pipeline_transforms::processors::create_dummy_item; @@ -57,8 +56,7 @@ impl ExchangeTransform { let senders = flight_senders.into_iter(); for (destination_id, sender) in params.destination_ids.iter().zip(senders) { items.push(match destination_id == ¶ms.executor_id { - true if max_threads == 1 => create_dummy_item(), - true => create_resize_item(1, max_threads), + true => create_dummy_item(), false => create_writer_item( sender, false, @@ -69,26 +67,33 @@ impl ExchangeTransform { }); } - let mut nodes_source = 0; let receivers = exchange_manager.get_flight_receiver(&exchange_params)?; - for (destination_id, receiver) in receivers { - if destination_id != params.executor_id { - nodes_source += 1; - items.push(create_reader_item( - receiver, - &destination_id, - ¶ms.executor_id, - params.fragment_id, - )); + let nodes_source = receivers.len(); + let mut idx = 1; + let mut reorder = vec![0_usize; nodes_source]; + + for (index, (destination_id, receiver)) in receivers.into_iter().enumerate() { + if destination_id == params.executor_id { + reorder[0] = index; + continue; } + + reorder[idx] = index; + idx += 1; + items.push(create_reader_item( + receiver, + &destination_id, + ¶ms.executor_id, + params.fragment_id, + )); } - let new_outputs = max_threads + nodes_source; - pipeline.add_pipe(Pipe::create(len, new_outputs, items)); + pipeline.add_pipe(Pipe::create(len, nodes_source, items)); - if !params.enable_multiway_sort { - pipeline.try_resize(max_threads)?; - } + match params.enable_multiway_sort { + true => pipeline.reorder_inputs(reorder), + false => pipeline.try_resize(max_threads)?, + }; pipeline.add_transform(|input, output| { TransformAggregateDeserializer::try_create(input, output, ¶ms.schema) diff --git a/src/query/service/src/servers/flight/v1/exchange/serde/exchange_serializer.rs b/src/query/service/src/servers/flight/v1/exchange/serde/exchange_serializer.rs index 945d2f09403ee..78b47b44e97ad 100644 --- a/src/query/service/src/servers/flight/v1/exchange/serde/exchange_serializer.rs +++ b/src/query/service/src/servers/flight/v1/exchange/serde/exchange_serializer.rs @@ -23,44 +23,35 @@ use arrow_flight::SchemaAsIpc; use arrow_ipc::writer::DictionaryTracker; use arrow_ipc::writer::IpcDataGenerator; use arrow_ipc::writer::IpcWriteOptions; -use arrow_ipc::CompressionType; use arrow_schema::ArrowError; use arrow_schema::Schema as ArrowSchema; use bytes::Bytes; -use databend_common_base::runtime::profile::Profile; -use databend_common_base::runtime::profile::ProfileStatisticsName; use databend_common_exception::Result; use databend_common_expression::local_block_meta_serde; use databend_common_expression::BlockMetaInfo; use databend_common_expression::BlockMetaInfoPtr; use databend_common_expression::DataBlock; use databend_common_io::prelude::BinaryWrite; -use databend_common_pipeline_core::processors::InputPort; -use databend_common_pipeline_core::processors::OutputPort; -use databend_common_pipeline_core::processors::ProcessorPtr; -use databend_common_pipeline_transforms::processors::BlockMetaTransform; -use databend_common_pipeline_transforms::processors::BlockMetaTransformer; -use databend_common_pipeline_transforms::processors::Transform; -use databend_common_pipeline_transforms::processors::Transformer; -use databend_common_pipeline_transforms::processors::UnknownMode; -use databend_common_settings::FlightCompression; - -use crate::servers::flight::v1::exchange::ExchangeShuffleMeta; -use crate::servers::flight::v1::exchange::MergeExchangeParams; -use crate::servers::flight::v1::exchange::ShuffleExchangeParams; + use crate::servers::flight::v1::packets::DataPacket; use crate::servers::flight::v1::packets::FragmentData; pub struct ExchangeSerializeMeta { pub block_number: isize, + pub max_block_number: usize, pub packet: Vec, } impl ExchangeSerializeMeta { - pub fn create(block_number: isize, packet: Vec) -> BlockMetaInfoPtr { + pub fn create( + block_number: isize, + max_block_number: usize, + packet: Vec, + ) -> BlockMetaInfoPtr { Box::new(ExchangeSerializeMeta { packet, block_number, + max_block_number, }) } } @@ -76,112 +67,16 @@ local_block_meta_serde!(ExchangeSerializeMeta); #[typetag::serde(name = "exchange_serialize")] impl BlockMetaInfo for ExchangeSerializeMeta {} -pub struct TransformExchangeSerializer { - options: IpcWriteOptions, -} - -impl TransformExchangeSerializer { - pub fn create( - input: Arc, - output: Arc, - _params: &MergeExchangeParams, - compression: Option, - ) -> Result { - let compression = match compression { - None => None, - Some(compression) => match compression { - FlightCompression::Lz4 => Some(CompressionType::LZ4_FRAME), - FlightCompression::Zstd => Some(CompressionType::ZSTD), - }, - }; - - Ok(ProcessorPtr::create(Transformer::create( - input, - output, - TransformExchangeSerializer { - options: IpcWriteOptions::default().try_with_compression(compression)?, - }, - ))) - } -} - -impl Transform for TransformExchangeSerializer { - const NAME: &'static str = "ExchangeSerializerTransform"; - - fn transform(&mut self, data_block: DataBlock) -> Result { - Profile::record_usize_profile(ProfileStatisticsName::ExchangeRows, data_block.num_rows()); - serialize_block(0, data_block, &self.options) - } -} - -pub struct TransformScatterExchangeSerializer { - local_pos: usize, - options: IpcWriteOptions, -} - -impl TransformScatterExchangeSerializer { - pub fn create( - input: Arc, - output: Arc, - compression: Option, - params: &ShuffleExchangeParams, - ) -> Result { - let local_id = ¶ms.executor_id; - let compression = match compression { - None => None, - Some(compression) => match compression { - FlightCompression::Lz4 => Some(CompressionType::LZ4_FRAME), - FlightCompression::Zstd => Some(CompressionType::ZSTD), - }, - }; - - Ok(ProcessorPtr::create(BlockMetaTransformer::create( - input, - output, - TransformScatterExchangeSerializer { - options: IpcWriteOptions::default().try_with_compression(compression)?, - local_pos: params - .destination_ids - .iter() - .position(|x| x == local_id) - .unwrap(), - }, - ))) - } -} - -impl BlockMetaTransform for TransformScatterExchangeSerializer { - const UNKNOWN_MODE: UnknownMode = UnknownMode::Error; - const NAME: &'static str = "TransformScatterExchangeSerializer"; - - fn transform(&mut self, meta: ExchangeShuffleMeta) -> Result> { - let mut new_blocks = Vec::with_capacity(meta.blocks.len()); - for (index, block) in meta.blocks.into_iter().enumerate() { - if block.is_empty() { - new_blocks.push(block); - continue; - } - - new_blocks.push(match self.local_pos == index { - true => block, - false => serialize_block(0, block, &self.options)?, - }); - } - - Ok(vec![DataBlock::empty_with_meta( - ExchangeShuffleMeta::create(new_blocks), - )]) - } -} - pub fn serialize_block( block_num: isize, + max_block_num: usize, data_block: DataBlock, options: &IpcWriteOptions, ) -> Result { if data_block.is_empty() && data_block.get_meta().is_none() { return Ok(DataBlock::empty_with_meta(ExchangeSerializeMeta::create( block_num, + max_block_num, vec![], ))); } @@ -223,7 +118,9 @@ pub fn serialize_block( } Ok(DataBlock::empty_with_meta(ExchangeSerializeMeta::create( - block_num, packet, + block_num, + max_block_num, + packet, ))) } diff --git a/src/query/service/src/servers/flight/v1/exchange/serde/mod.rs b/src/query/service/src/servers/flight/v1/exchange/serde/mod.rs index 7349b2f46b0c1..ccf7abfb694b6 100644 --- a/src/query/service/src/servers/flight/v1/exchange/serde/mod.rs +++ b/src/query/service/src/servers/flight/v1/exchange/serde/mod.rs @@ -20,5 +20,3 @@ pub use exchange_deserializer::ExchangeDeserializeMeta; pub use exchange_deserializer::TransformExchangeDeserializer; pub use exchange_serializer::serialize_block; pub use exchange_serializer::ExchangeSerializeMeta; -pub use exchange_serializer::TransformExchangeSerializer; -pub use exchange_serializer::TransformScatterExchangeSerializer; From 2f62d3d2e1aa0796d137de42193f7030ad9555fb Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 18 Mar 2025 02:08:38 +0800 Subject: [PATCH 16/76] refactor(query): refactor aggreagte spill code --- .../aggregator/aggregate_exchange_injector.rs | 8 ++++---- .../aggregator/transform_aggregate_final.rs | 14 +++++++++----- 2 files changed, 13 insertions(+), 9 deletions(-) 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 6c9f06b5468ee..d67d1bddc228f 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 @@ -157,7 +157,7 @@ impl Exchange for FlightExchange { fn partition(&self, mut data_block: DataBlock, n: usize) -> Result> { let Some(meta) = data_block.take_meta() else { // only exchange data - if !MULTIWAY_SORT && data_block.is_empty() { + if data_block.is_empty() { return Ok(vec![]); } @@ -205,9 +205,9 @@ impl Exchange for FlightExchange { Ok(blocks) } AggregateMeta::AggregatePayload(p) => { - // if p.payload.len() == 0 { - // return Ok(vec![]); - // } + if p.payload.len() == 0 { + return Ok(vec![]); + } let mut blocks = Vec::with_capacity(n); for (idx, payload) in scatter_payload(p.payload, n)?.into_iter().enumerate() { 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 f294aec338508..49af84ed0add9 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 @@ -67,13 +67,17 @@ impl AccumulatingTransform for TransformFinalAggregate { .combine_payloads(&payload, &mut self.flush_state)?; } AggregateMeta::InFlightPayload(_payload) => { - let payload = self.deserialize_flight(data)?; - self.hash_table - .combine_payload(&payload, &mut self.flush_state)?; + if !data.is_empty() { + let payload = self.deserialize_flight(data)?; + self.hash_table + .combine_payload(&payload, &mut self.flush_state)?; + } } AggregateMeta::AggregatePayload(payload) => { - self.hash_table - .combine_payload(&payload.payload, &mut self.flush_state)?; + if payload.payload.len() != 0 { + self.hash_table + .combine_payload(&payload.payload, &mut self.flush_state)?; + } } AggregateMeta::FinalPartition => { if self.hash_table.len() == 0 { From 3a0652be82cba2c4f82e5ee9f464416f8bc60015 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 18 Mar 2025 02:50:43 +0800 Subject: [PATCH 17/76] refactor(query): refactor aggreagte spill code --- .../transforms/aggregator/transform_partition_bucket.rs | 2 ++ 1 file changed, 2 insertions(+) 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 index bd21c721be2a3..2ec2badc01136 100644 --- 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 @@ -178,6 +178,7 @@ impl Processor for TransformPartitionDispatch { return Ok(Event::Sync); } + self.input.set_need_data(); return Ok(Event::NeedData); } @@ -208,6 +209,7 @@ impl Processor for TransformPartitionDispatch { if partition != self.working_partition { // ready partition self.fetch_ready_partition()?; + self.working_partition = partition; } } From 9a56d229fc10fa76e705916666d08baf862df5d6 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 18 Mar 2025 05:35:22 +0800 Subject: [PATCH 18/76] refactor(query): refactor aggreagte spill code --- src/query/pipeline/core/src/pipeline.rs | 6 +- .../core/src/processors/shuffle_processor.rs | 16 ++- .../aggregator/aggregate_exchange_injector.rs | 132 +++++++++++++----- .../transforms/aggregator/aggregate_meta.rs | 10 ++ .../aggregator/transform_partition_bucket.rs | 121 ++++++++++------ 5 files changed, 195 insertions(+), 90 deletions(-) diff --git a/src/query/pipeline/core/src/pipeline.rs b/src/query/pipeline/core/src/pipeline.rs index b5eae42644db0..0ee7d92d044c4 100644 --- a/src/query/pipeline/core/src/pipeline.rs +++ b/src/query/pipeline/core/src/pipeline.rs @@ -481,7 +481,11 @@ impl Pipeline { let output = OutputPort::create(); let inputs: Vec<_> = (0..input_len).map(|_| InputPort::create()).collect(); items.push(PipeItem::create( - MergePartitionProcessor::::create(inputs.clone(), output.clone()), + MergePartitionProcessor::::create( + inputs.clone(), + output.clone(), + exchange.clone(), + ), inputs, vec![output], )); diff --git a/src/query/pipeline/core/src/processors/shuffle_processor.rs b/src/query/pipeline/core/src/processors/shuffle_processor.rs index 4ecf295abf00f..83511b61c85fe 100644 --- a/src/query/pipeline/core/src/processors/shuffle_processor.rs +++ b/src/query/pipeline/core/src/processors/shuffle_processor.rs @@ -14,7 +14,6 @@ use std::any::Any; use std::cmp::Ordering; -use std::marker::PhantomData; use std::sync::Arc; use databend_common_exception::Result; @@ -46,7 +45,7 @@ pub trait Exchange: Send + Sync + 'static { unimplemented!() } - fn multiway_pick(data_blocks: &mut [Option]) -> Option { + fn multiway_pick(&self, data_blocks: &mut [Option]) -> Option { let position = data_blocks .iter() @@ -294,17 +293,22 @@ pub struct MergePartitionProcessor { output: Arc, inputs: Vec>, inputs_data: Vec>, - _phantom_data: PhantomData, + exchange: Arc, + // _phantom_data: PhantomData, } impl MergePartitionProcessor { - pub fn create(inputs: Vec>, output: Arc) -> ProcessorPtr { + pub fn create( + inputs: Vec>, + output: Arc, + exchange: Arc, + ) -> ProcessorPtr { let inputs_data = vec![None; inputs.len()]; ProcessorPtr::create(Box::new(MergePartitionProcessor:: { output, inputs, inputs_data, - _phantom_data: Default::default(), + exchange, })) } } @@ -364,7 +368,7 @@ impl Processor for MergePartitionProcessor { } if need_pick_block_to_push { - if let Some(pick_index) = T::multiway_pick(&mut self.inputs_data) { + if let Some(pick_index) = self.exchange.multiway_pick(&mut self.inputs_data) { if let Some(block) = self.inputs_data[pick_index].take() { self.output.push_data(Ok(block)); return Ok(Event::NeedConsume); 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 d67d1bddc228f..759f2cb341b79 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 @@ -14,6 +14,8 @@ use std::cmp::Ordering; use std::collections::HashMap; +use std::sync::atomic::AtomicUsize; +use std::sync::atomic::Ordering as AtomicOrdering; use std::sync::Arc; use arrow_ipc::writer::IpcWriteOptions; @@ -92,6 +94,7 @@ pub struct FlightExchange { rev_bucket_lookup: Vec, options: IpcWriteOptions, shuffle_scatter: Arc>, + global_max_partition: AtomicUsize, } impl FlightExchange { @@ -123,6 +126,7 @@ impl FlightExchange { .try_with_compression(compression) .unwrap(), shuffle_scatter, + global_max_partition: AtomicUsize::new(0), }) } } @@ -148,6 +152,34 @@ impl FlightExchange { Ok(blocks) } + + fn get_global_max_partition(data_blocks: &[Option]) -> usize { + let mut global_max_partition = 0; + + for data_block in data_blocks { + let Some(data_block) = data_block else { + continue; + }; + + let Some(meta) = data_block.get_meta() else { + continue; + }; + + let meta_value = match ExchangeSerializeMeta::downcast_ref_from(meta) { + Some(meta) => meta.max_block_number, + None => match AggregateMeta::downcast_ref_from(meta) { + Some(meta) => meta.get_global_max_partition(), + None => { + continue; + } + }, + }; + + global_max_partition = global_max_partition.max(meta_value) + } + + global_max_partition + } } impl Exchange for FlightExchange { @@ -247,46 +279,54 @@ impl Exchange for FlightExchange { } } - fn multiway_pick(data_blocks: &mut [Option]) -> Option { - let mut global_max_partition = 0_usize; - let global_max_partition_ref = &mut global_max_partition; + fn sorting_function(left_block: &DataBlock, right_block: &DataBlock) -> Ordering { + let Some(left_meta) = left_block.get_meta() else { + return Ordering::Equal; + }; + let Some(left_meta) = ExchangeSerializeMeta::downcast_ref_from(left_meta) else { + return Ordering::Equal; + }; - let min_position = data_blocks - .iter() - .enumerate() - .filter_map(|(idx, x)| x.as_ref().map(|d| (idx, d))) - .min_by(move |(left_idx, left_block), (right_idx, right_block)| { - let Some(left_meta) = left_block.get_meta() else { - return Ordering::Equal; - }; - let Some(left_meta) = ExchangeSerializeMeta::downcast_ref_from(left_meta) else { - return Ordering::Equal; - }; + let Some(right_meta) = right_block.get_meta() else { + return Ordering::Equal; + }; + let Some(right_meta) = ExchangeSerializeMeta::downcast_ref_from(right_meta) else { + return Ordering::Equal; + }; - let Some(right_meta) = right_block.get_meta() else { - return Ordering::Equal; - }; - let Some(right_meta) = ExchangeSerializeMeta::downcast_ref_from(right_meta) else { - return Ordering::Equal; - }; + let (l_partition, l_max_partition) = restore_block_number(left_meta.block_number); + let (r_partition, r_max_partition) = restore_block_number(right_meta.block_number); - let max_block_number = left_meta.max_block_number.max(right_meta.max_block_number); - *global_max_partition_ref = (*global_max_partition_ref).max(max_block_number); - let (l_partition, l_max_partition) = restore_block_number(left_meta.block_number); - let (r_partition, r_max_partition) = restore_block_number(right_meta.block_number); + // ORDER BY max_partition asc, partition asc + match l_max_partition.cmp(&r_max_partition) { + Ordering::Less => Ordering::Less, + Ordering::Greater => Ordering::Greater, + Ordering::Equal => l_partition.cmp(&r_partition), + } + } - // ORDER BY max_partition asc, partition asc, idx asc - match l_max_partition.cmp(&r_max_partition) { - Ordering::Less => Ordering::Less, - Ordering::Greater => Ordering::Greater, - Ordering::Equal => match l_partition.cmp(&r_partition) { + fn multiway_pick(&self, data_blocks: &mut [Option]) -> Option { + let new_value = Self::get_global_max_partition(data_blocks); + let old_value = self + .global_max_partition + .fetch_max(new_value, AtomicOrdering::SeqCst); + + let global_max_partition = std::cmp::max(new_value, old_value); + + let min_position = + data_blocks + .iter() + .enumerate() + .filter_map(|(idx, x)| x.as_ref().map(|d| (idx, d))) + .min_by(move |(left_idx, left_block), (right_idx, right_block)| { + match FlightExchange::::sorting_function(left_block, right_block) + { Ordering::Less => Ordering::Less, Ordering::Greater => Ordering::Greater, Ordering::Equal => left_idx.cmp(right_idx), - }, - } - }) - .map(|(idx, _)| idx); + } + }) + .map(|(idx, _)| idx); if let Some(min_pos) = min_position { if global_max_partition == 0 { @@ -294,10 +334,28 @@ impl Exchange for FlightExchange { } if let Some(mut block) = data_blocks[min_pos].take() { - let mut meta = - ExchangeSerializeMeta::downcast_from(block.take_meta().unwrap()).unwrap(); - meta.max_block_number = global_max_partition; - data_blocks[min_pos] = Some(block.add_meta(Some(Box::new(meta))).unwrap()); + let meta = block.get_meta().unwrap(); + match ExchangeSerializeMeta::downcast_ref_from(meta) { + Some(_) => { + let mut meta = + ExchangeSerializeMeta::downcast_from(block.take_meta().unwrap()) + .unwrap(); + meta.max_block_number = global_max_partition; + data_blocks[min_pos] = Some(block.add_meta(Some(Box::new(meta))).unwrap()); + } + None => match AggregateMeta::downcast_ref_from(meta) { + Some(_) => { + let mut meta = + AggregateMeta::downcast_from(block.take_meta().unwrap()).unwrap(); + meta.set_global_max_partition(global_max_partition); + data_blocks[min_pos] = + Some(block.add_meta(Some(Box::new(meta))).unwrap()); + } + None => { + // do nothing + } + }, + } } } 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 7c55911af9889..7306d0e2d7f7c 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 @@ -189,6 +189,16 @@ impl AggregateMeta { Box::new(AggregateMeta::FinalPartition) } + pub fn get_global_max_partition(&self) -> usize { + match self { + AggregateMeta::Serialized(v) => v.global_max_partition, + AggregateMeta::SpilledPayload(v) => v.global_max_partition, + AggregateMeta::AggregatePayload(v) => v.global_max_partition, + AggregateMeta::InFlightPayload(v) => v.global_max_partition, + AggregateMeta::FinalPartition => unreachable!(), + } + } + pub fn set_global_max_partition(&mut self, global_max_partition: usize) { match self { AggregateMeta::Serialized(v) => { 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 index 2ec2badc01136..8d0150dffe342 100644 --- 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 @@ -20,6 +20,8 @@ use std::collections::HashMap; use std::collections::VecDeque; use std::fmt::Debug; use std::fmt::Formatter; +use std::sync::atomic::AtomicUsize; +use std::sync::atomic::Ordering as AtomicOrdering; use std::sync::Arc; use bumpalo::Bump; @@ -251,73 +253,95 @@ impl Processor for TransformPartitionDispatch { } } -struct ResortingPartition; +struct ResortingPartition { + global_max_partition: AtomicUsize, +} impl ResortingPartition { - fn block_number(meta: &AggregateMeta) -> (isize, usize, usize) { + fn block_number(meta: &AggregateMeta) -> (isize, usize) { match meta { - AggregateMeta::Serialized(v) => (v.bucket, v.max_partition, v.global_max_partition), - AggregateMeta::SpilledPayload(v) => { - (v.partition, v.max_partition, v.global_max_partition) - } - AggregateMeta::AggregatePayload(v) => { - (v.partition, v.max_partition, v.global_max_partition) - } - AggregateMeta::InFlightPayload(v) => { - (v.partition, v.max_partition, v.global_max_partition) - } + AggregateMeta::Serialized(v) => (v.bucket, v.max_partition), + AggregateMeta::SpilledPayload(v) => (v.partition, v.max_partition), + AggregateMeta::AggregatePayload(v) => (v.partition, v.max_partition), + AggregateMeta::InFlightPayload(v) => (v.partition, v.max_partition), AggregateMeta::FinalPartition => unreachable!(), } } + + fn get_global_max_partition(data_blocks: &[Option]) -> usize { + let mut global_max_partition = 0; + + for data_block in data_blocks { + let Some(data_block) = data_block else { + continue; + }; + + let Some(meta) = data_block.get_meta() else { + continue; + }; + let Some(meta) = AggregateMeta::downcast_ref_from(meta) else { + continue; + }; + + global_max_partition = global_max_partition.max(meta.get_global_max_partition()) + } + + global_max_partition + } } impl Exchange for ResortingPartition { const NAME: &'static str = "PartitionResorting"; + const MULTIWAY_SORT: bool = true; fn partition(&self, data_block: DataBlock, n: usize) -> Result> { debug_assert_eq!(n, 1); Ok(vec![data_block]) } - fn multiway_pick(data_blocks: &mut [Option]) -> Option { - let mut global_max_partition = 0_usize; - let global_max_partition_ref = &mut global_max_partition; + fn sorting_function(left_block: &DataBlock, right_block: &DataBlock) -> Ordering { + let Some(left_meta) = left_block.get_meta() else { + return Ordering::Equal; + }; + let Some(left_meta) = AggregateMeta::downcast_ref_from(left_meta) else { + return Ordering::Equal; + }; + + let Some(right_meta) = right_block.get_meta() else { + return Ordering::Equal; + }; + let Some(right_meta) = AggregateMeta::downcast_ref_from(right_meta) else { + return Ordering::Equal; + }; + + let (l_partition, l_max_partition) = ResortingPartition::block_number(left_meta); + let (r_partition, r_max_partition) = ResortingPartition::block_number(right_meta); + + // ORDER BY max_partition asc, partition asc, idx asc + match l_max_partition.cmp(&r_max_partition) { + Ordering::Less => Ordering::Less, + Ordering::Greater => Ordering::Greater, + Ordering::Equal => l_partition.cmp(&r_partition), + } + } + + fn multiway_pick(&self, data_blocks: &mut [Option]) -> Option { + let new_value = Self::get_global_max_partition(data_blocks); + let old_value = self + .global_max_partition + .fetch_max(new_value, AtomicOrdering::SeqCst); + + let global_max_partition = std::cmp::max(new_value, old_value); let min_position = data_blocks .iter() .enumerate() - .filter_map(|(idx, x)| x.as_ref().map(|d| (idx, d))) - .min_by(move |(left_idx, left_block), (right_idx, right_block)| { - let Some(left_meta) = left_block.get_meta() else { - return Ordering::Equal; - }; - let Some(left_meta) = AggregateMeta::downcast_ref_from(left_meta) else { - return Ordering::Equal; - }; - - let Some(right_meta) = right_block.get_meta() else { - return Ordering::Equal; - }; - let Some(right_meta) = AggregateMeta::downcast_ref_from(right_meta) else { - return Ordering::Equal; - }; - - let (l_partition, l_max_partition, l_global_max_partition) = - ResortingPartition::block_number(left_meta); - let (r_partition, r_max_partition, r_global_max_partition) = - ResortingPartition::block_number(right_meta); - let global_max_partition = l_global_max_partition.max(r_global_max_partition); - *global_max_partition_ref = (*global_max_partition_ref).max(global_max_partition); - - // ORDER BY max_partition asc, partition asc, idx asc - match l_max_partition.cmp(&r_max_partition) { + .filter_map(|(i, x)| x.as_ref().map(|x| (i, x))) + .min_by(|(left_idx, left_block), (right_idx, right_block)| { + match ResortingPartition::sorting_function(left_block, right_block) { Ordering::Less => Ordering::Less, Ordering::Greater => Ordering::Greater, - Ordering::Equal => match l_partition.cmp(&r_partition) { - Ordering::Less => Ordering::Less, - Ordering::Greater => Ordering::Greater, - Ordering::Equal => left_idx.cmp(right_idx), - }, + Ordering::Equal => left_idx.cmp(right_idx), } }) .map(|(idx, _)| idx); @@ -345,7 +369,12 @@ pub fn build_partition_bucket( let output = pipeline.output_len(); // 1. reorder partition - pipeline.exchange(1, Arc::new(ResortingPartition)); + pipeline.exchange( + 1, + Arc::new(ResortingPartition { + global_max_partition: AtomicUsize::new(0), + }), + ); let transform = TransformPartitionDispatch::create(output, params.clone())?; From 552bcda9b9d3bc05bb6fd693905eade5bd78bc84 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 19 Mar 2025 13:19:53 +0800 Subject: [PATCH 19/76] refactor(query): refactor aggreagte spill code --- .../aggregator/aggregate_exchange_injector.rs | 43 ++++++++++++++----- .../transforms/aggregator/aggregate_meta.rs | 20 +++++++++ .../aggregator/transform_partition_bucket.rs | 5 ++- 3 files changed, 56 insertions(+), 12 deletions(-) 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 759f2cb341b79..54bd0d373f96e 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 @@ -215,11 +215,15 @@ impl Exchange for FlightExchange { true => DataBlock::empty_with_meta( AggregateMeta::create_in_flight_payload(-1, v.max_partition), ), - false => DataBlock::empty_with_meta(ExchangeSerializeMeta::create( + false => serialize_block( block_num, v.global_max_partition, - Vec::with_capacity(0), - )), + DataBlock::empty_with_meta(AggregateMeta::create_in_flight_payload( + -1, + v.max_partition, + )), + &self.options, + )?, }); } @@ -283,19 +287,36 @@ impl Exchange for FlightExchange { let Some(left_meta) = left_block.get_meta() else { return Ordering::Equal; }; - let Some(left_meta) = ExchangeSerializeMeta::downcast_ref_from(left_meta) else { - return Ordering::Equal; - }; + + let (l_partition, l_max_partition) = + match ExchangeSerializeMeta::downcast_ref_from(left_meta) { + Some(left_meta) => restore_block_number(left_meta.block_number), + None => { + // to local + let Some(meta) = AggregateMeta::downcast_ref_from(left_meta) else { + return Ordering::Equal; + }; + + (meta.get_partition(), meta.get_max_partition()) + } + }; let Some(right_meta) = right_block.get_meta() else { return Ordering::Equal; }; - let Some(right_meta) = ExchangeSerializeMeta::downcast_ref_from(right_meta) else { - return Ordering::Equal; - }; - let (l_partition, l_max_partition) = restore_block_number(left_meta.block_number); - let (r_partition, r_max_partition) = restore_block_number(right_meta.block_number); + let (r_partition, r_max_partition) = + match ExchangeSerializeMeta::downcast_ref_from(right_meta) { + Some(meta) => restore_block_number(meta.block_number), + None => { + // to local + let Some(meta) = AggregateMeta::downcast_ref_from(right_meta) else { + return Ordering::Equal; + }; + + (meta.get_partition(), meta.get_max_partition()) + } + }; // ORDER BY max_partition asc, partition asc match l_max_partition.cmp(&r_max_partition) { 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 7306d0e2d7f7c..8b383aaf530a0 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 @@ -199,6 +199,26 @@ impl AggregateMeta { } } + pub fn get_partition(&self) -> isize { + match self { + AggregateMeta::Serialized(v) => v.bucket, + AggregateMeta::SpilledPayload(v) => v.partition, + AggregateMeta::AggregatePayload(v) => v.partition, + AggregateMeta::InFlightPayload(v) => v.partition, + AggregateMeta::FinalPartition => unreachable!(), + } + } + + pub fn get_max_partition(&self) -> usize { + match self { + AggregateMeta::Serialized(v) => v.max_partition, + AggregateMeta::SpilledPayload(v) => v.max_partition, + AggregateMeta::AggregatePayload(v) => v.max_partition, + AggregateMeta::InFlightPayload(v) => v.max_partition, + AggregateMeta::FinalPartition => unreachable!(), + } + } + pub fn set_global_max_partition(&mut self, global_max_partition: usize) { match self { AggregateMeta::Serialized(v) => { 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 index 8d0150dffe342..d2c182f6e3769 100644 --- 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 @@ -457,7 +457,10 @@ impl UnalignedPartitions { let partition = payload.partition; let max_partition = payload.max_partition; let global_max_partition = payload.global_max_partition; - self.insert_data(max_partition, meta, block); + + if !block.is_empty() { + self.insert_data(max_partition, meta, block); + } (partition, max_partition, global_max_partition) } From 7d46563ff6dbf7378c66cd806feff1380354680a Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 19 Mar 2025 17:16:14 +0800 Subject: [PATCH 20/76] refactor(query): refactor aggreagte spill code --- .../aggregator/aggregate_exchange_injector.rs | 70 ++++++++----------- 1 file changed, 30 insertions(+), 40 deletions(-) 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 54bd0d373f96e..9a69465ac95ea 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 @@ -90,16 +90,17 @@ fn scatter_payload(mut payload: Payload, buckets: usize) -> Result> pub struct FlightExchange { local_id: String, - bucket_lookup: HashMap, - rev_bucket_lookup: Vec, + node_list: Vec, + node_list_lookup: HashMap, + options: IpcWriteOptions, - shuffle_scatter: Arc>, global_max_partition: AtomicUsize, + shuffle_scatter: Arc>, } impl FlightExchange { pub fn create( - lookup: Vec, + node_list: Vec, compression: Option, shuffle_scatter: Arc>, ) -> Arc { @@ -111,7 +112,7 @@ impl FlightExchange { }, }; - let bucket_lookup = lookup + let node_list_lookup = node_list .iter() .cloned() .enumerate() @@ -120,8 +121,8 @@ impl FlightExchange { Arc::new(FlightExchange { local_id: GlobalConfig::instance().query.node_id.clone(), - bucket_lookup, - rev_bucket_lookup: lookup, + node_list, + node_list_lookup, options: IpcWriteOptions::default() .try_with_compression(compression) .unwrap(), @@ -133,7 +134,7 @@ impl FlightExchange { impl FlightExchange { fn default_partition(&self, data_block: DataBlock) -> Result> { - if self.rev_bucket_lookup.is_empty() { + if self.node_list.is_empty() { let data_block = serialize_block(0, 0, data_block, &self.options)?; return Ok(vec![data_block]); } @@ -142,7 +143,7 @@ impl FlightExchange { let mut blocks = Vec::with_capacity(data_blocks.len()); for (idx, data_block) in data_blocks.into_iter().enumerate() { - if self.rev_bucket_lookup[idx] == self.local_id { + if self.node_list[idx] == self.local_id { blocks.push(data_block); continue; } @@ -201,7 +202,7 @@ impl Exchange for FlightExchange { }; assert!(MULTIWAY_SORT); - assert_eq!(self.bucket_lookup.len(), n); + assert_eq!(self.node_list_lookup.len(), n); match AggregateMeta::downcast_from(meta).unwrap() { AggregateMeta::Serialized(_) => unreachable!(), AggregateMeta::FinalPartition => unreachable!(), @@ -210,16 +211,16 @@ impl Exchange for FlightExchange { let block_num = compute_block_number(-1, v.max_partition)?; let mut blocks = Vec::with_capacity(n); - for local in &self.rev_bucket_lookup { - blocks.push(match *local == self.local_id { + for node_id in &self.node_list { + blocks.push(match *node_id == self.local_id { true => DataBlock::empty_with_meta( - AggregateMeta::create_in_flight_payload(-1, v.max_partition), + AggregateMeta::create_in_flight_payload(v.partition, v.max_partition), ), false => serialize_block( block_num, v.global_max_partition, DataBlock::empty_with_meta(AggregateMeta::create_in_flight_payload( - -1, + v.partition, v.max_partition, )), &self.options, @@ -227,7 +228,7 @@ impl Exchange for FlightExchange { }); } - let index = *self.bucket_lookup.get(&v.destination_node).unwrap(); + let index = *self.node_list_lookup.get(&v.destination_node).unwrap(); blocks[index] = match v.destination_node == self.local_id { true => DataBlock::empty_with_meta(AggregateMeta::create_spilled_payload(v)), false => serialize_block( @@ -247,7 +248,7 @@ impl Exchange for FlightExchange { let mut blocks = Vec::with_capacity(n); for (idx, payload) in scatter_payload(p.payload, n)?.into_iter().enumerate() { - if self.rev_bucket_lookup[idx] == self.local_id { + if self.node_list[idx] == self.local_id { blocks.push(DataBlock::empty_with_meta( AggregateMeta::create_agg_payload( payload, @@ -290,9 +291,8 @@ impl Exchange for FlightExchange { let (l_partition, l_max_partition) = match ExchangeSerializeMeta::downcast_ref_from(left_meta) { - Some(left_meta) => restore_block_number(left_meta.block_number), + Some(meta) => restore_block_number(meta.block_number), None => { - // to local let Some(meta) = AggregateMeta::downcast_ref_from(left_meta) else { return Ordering::Equal; }; @@ -309,7 +309,6 @@ impl Exchange for FlightExchange { match ExchangeSerializeMeta::downcast_ref_from(right_meta) { Some(meta) => restore_block_number(meta.block_number), None => { - // to local let Some(meta) = AggregateMeta::downcast_ref_from(right_meta) else { return Ordering::Equal; }; @@ -355,27 +354,18 @@ impl Exchange for FlightExchange { } if let Some(mut block) = data_blocks[min_pos].take() { - let meta = block.get_meta().unwrap(); - match ExchangeSerializeMeta::downcast_ref_from(meta) { - Some(_) => { - let mut meta = - ExchangeSerializeMeta::downcast_from(block.take_meta().unwrap()) - .unwrap(); - meta.max_block_number = global_max_partition; - data_blocks[min_pos] = Some(block.add_meta(Some(Box::new(meta))).unwrap()); - } - None => match AggregateMeta::downcast_ref_from(meta) { - Some(_) => { - let mut meta = - AggregateMeta::downcast_from(block.take_meta().unwrap()).unwrap(); - meta.set_global_max_partition(global_max_partition); - data_blocks[min_pos] = - Some(block.add_meta(Some(Box::new(meta))).unwrap()); - } - None => { - // do nothing - } - }, + let Some(meta) = block.get_meta() else { + return Some(min_pos); + }; + + if ExchangeSerializeMeta::downcast_ref_from(meta).is_some() { + let mut meta = ExchangeSerializeMeta::downcast_from(block.take_meta()?)?; + meta.max_block_number = global_max_partition; + data_blocks[min_pos] = Some(block.add_meta(Some(Box::new(meta))).unwrap()); + } else if AggregateMeta::downcast_ref_from(meta).is_some() { + let mut meta = AggregateMeta::downcast_from(block.take_meta()?)?; + meta.set_global_max_partition(global_max_partition); + data_blocks[min_pos] = Some(block.add_meta(Some(Box::new(meta))).unwrap()); } } } From 5f1fbbcbce607f0cde0caf08083f1e289aeebab7 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Thu, 20 Mar 2025 02:13:48 +0800 Subject: [PATCH 21/76] refactor(query): refactor aggreagte spill code --- Cargo.lock | 18 +- src/query/pipeline/core/src/lib.rs | 1 + src/query/pipeline/core/src/pipeline.rs | 12 +- .../core/src/processors/shuffle_processor.rs | 41 ++- .../aggregator/aggregate_exchange_injector.rs | 189 ++++--------- .../transforms/aggregator/aggregate_meta.rs | 27 +- .../transforms/aggregator/serde/mod.rs | 4 - .../transforms/aggregator/serde/serde_meta.rs | 100 ------- .../serde/transform_aggregate_serializer.rs | 259 ------------------ .../serde/transform_deserializer.rs | 6 +- .../aggregator/transform_aggregate_partial.rs | 1 + .../aggregator/transform_partition_bucket.rs | 94 ++----- .../flight/v1/exchange/exchange_transform.rs | 30 +- .../v1/exchange/serde/exchange_serializer.rs | 30 +- 14 files changed, 211 insertions(+), 601 deletions(-) delete mode 100644 src/query/service/src/pipelines/processors/transforms/aggregator/serde/serde_meta.rs delete mode 100644 src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_aggregate_serializer.rs diff --git a/Cargo.lock b/Cargo.lock index 86dc06cc821a3..a3dc01d6b1277 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -5761,7 +5761,7 @@ checksum = "97369cbbc041bc366949bc74d34658d6cda5621039731c6310521892a3a20ae0" dependencies = [ "proc-macro2", "quote", - "syn 2.0.95", + "syn 2.0.100", ] [[package]] @@ -8621,7 +8621,7 @@ checksum = "1ec89e9337638ecdc08744df490b221a7399bf8d164eb52a665454e60e075ad6" dependencies = [ "proc-macro2", "quote", - "syn 2.0.95", + "syn 2.0.100", ] [[package]] @@ -12682,7 +12682,7 @@ dependencies = [ "proc-macro2", "quote", "rquickjs-core", - "syn 2.0.95", + "syn 2.0.100", ] [[package]] @@ -14246,7 +14246,7 @@ checksum = "c8af7666ab7b6390ab78131fb5b0fce11d6b7a6951602017c35fa82800708971" dependencies = [ "proc-macro2", "quote", - "syn 2.0.95", + "syn 2.0.100", ] [[package]] @@ -14615,7 +14615,7 @@ checksum = "7f7cf42b4507d8ea322120659672cf1b9dbb93f8f2d4ecfd6e51350ff5b17a1d" dependencies = [ "proc-macro2", "quote", - "syn 2.0.95", + "syn 2.0.100", ] [[package]] @@ -16800,7 +16800,7 @@ checksum = "2380878cad4ac9aac1e2435f3eb4020e8374b5f13c296cb75b4620ff8e229154" dependencies = [ "proc-macro2", "quote", - "syn 2.0.95", + "syn 2.0.100", "synstructure", ] @@ -16848,7 +16848,7 @@ checksum = "6352c01d0edd5db859a63e2605f4ea3183ddbd15e2c4a9e7d32184df75e4f154" dependencies = [ "proc-macro2", "quote", - "syn 2.0.95", + "syn 2.0.100", ] [[package]] @@ -16868,7 +16868,7 @@ checksum = "d71e5d6e06ab090c67b5e44993ec16b72dcbaabc526db883a360057678b48502" dependencies = [ "proc-macro2", "quote", - "syn 2.0.95", + "syn 2.0.100", "synstructure", ] @@ -16897,7 +16897,7 @@ checksum = "6eafa6dfb17584ea3e2bd6e76e0cc15ad7af12b09abdd1ca55961bed9b1063c6" dependencies = [ "proc-macro2", "quote", - "syn 2.0.95", + "syn 2.0.100", ] [[package]] diff --git a/src/query/pipeline/core/src/lib.rs b/src/query/pipeline/core/src/lib.rs index 395d7b78153a8..a8a59cadeb076 100644 --- a/src/query/pipeline/core/src/lib.rs +++ b/src/query/pipeline/core/src/lib.rs @@ -16,6 +16,7 @@ #![feature(variant_count)] #![feature(associated_type_defaults)] #![feature(adt_const_params)] +#![feature(let_chains)] #![allow(clippy::arc_with_non_send_sync)] #![allow(clippy::useless_asref)] diff --git a/src/query/pipeline/core/src/pipeline.rs b/src/query/pipeline/core/src/pipeline.rs index 0ee7d92d044c4..88807fbe8f1fa 100644 --- a/src/query/pipeline/core/src/pipeline.rs +++ b/src/query/pipeline/core/src/pipeline.rs @@ -20,6 +20,7 @@ use std::sync::atomic::Ordering; use std::sync::Arc; use std::time::Instant; +use databend_common_base::base::tokio::sync::Barrier; use databend_common_base::runtime::defer; use databend_common_base::runtime::drop_guard; use databend_common_exception::ErrorCode; @@ -456,11 +457,18 @@ impl Pipeline { let input_len = pipe.output_length; let mut items = Vec::with_capacity(input_len); - for _index in 0..input_len { + let barrier = Arc::new(Barrier::new(input_len)); + for index in 0..input_len { let input = InputPort::create(); let outputs: Vec<_> = (0..n).map(|_| OutputPort::create()).collect(); items.push(PipeItem::create( - PartitionProcessor::create(input.clone(), outputs.clone(), exchange.clone()), + PartitionProcessor::create( + input.clone(), + outputs.clone(), + exchange.clone(), + index, + barrier.clone(), + ), vec![input], outputs, )); diff --git a/src/query/pipeline/core/src/processors/shuffle_processor.rs b/src/query/pipeline/core/src/processors/shuffle_processor.rs index 83511b61c85fe..212f9bc238ac8 100644 --- a/src/query/pipeline/core/src/processors/shuffle_processor.rs +++ b/src/query/pipeline/core/src/processors/shuffle_processor.rs @@ -16,6 +16,7 @@ use std::any::Any; use std::cmp::Ordering; use std::sync::Arc; +use databend_common_base::base::tokio::sync::Barrier; use databend_common_exception::Result; use databend_common_expression::DataBlock; @@ -32,8 +33,6 @@ pub enum MultiwayStrategy { Custom, } -// std::marker::ConstParamTy!(MultiwayStrategy); - pub trait Exchange: Send + Sync + 'static { const NAME: &'static str; const MULTIWAY_SORT: bool = false; @@ -41,6 +40,10 @@ pub trait Exchange: Send + Sync + 'static { fn partition(&self, data_block: DataBlock, n: usize) -> Result>; + fn init_way(&self, _index: usize, _first_data: &DataBlock) -> Result<()> { + Ok(()) + } + fn sorting_function(_: &DataBlock, _: &DataBlock) -> Ordering { unimplemented!() } @@ -187,6 +190,10 @@ pub struct PartitionProcessor { exchange: Arc, input_data: Option, partitioned_data: Vec>, + + index: usize, + initialized: bool, + barrier: Arc, } impl PartitionProcessor { @@ -194,6 +201,8 @@ impl PartitionProcessor { input: Arc, outputs: Vec>, exchange: Arc, + index: usize, + barrier: Arc, ) -> ProcessorPtr { let partitioned_data = vec![None; outputs.len()]; ProcessorPtr::create(Box::new(PartitionProcessor { @@ -202,10 +211,14 @@ impl PartitionProcessor { exchange, partitioned_data, input_data: None, + initialized: false, + index, + barrier, })) } } +#[async_trait::async_trait] impl Processor for PartitionProcessor { fn name(&self) -> String { format!("ShufflePartition({})", T::NAME) @@ -250,9 +263,20 @@ impl Processor for PartitionProcessor { return Ok(Event::NeedConsume); } + if self.input_data.is_some() { + return match self.initialized { + true => Ok(Event::Sync), + false => Ok(Event::Async), + }; + } + if self.input.has_data() { self.input_data = Some(self.input.pull_data().unwrap()?); - return Ok(Event::Sync); + + return match self.initialized { + true => Ok(Event::Sync), + false => Ok(Event::Async), + }; } if self.input.is_finished() { @@ -287,6 +311,16 @@ impl Processor for PartitionProcessor { Ok(()) } + + async fn async_process(&mut self) -> Result<()> { + if let Some(data_block) = self.input_data.as_ref() { + self.initialized = true; + self.exchange.init_way(self.index, data_block)?; + self.barrier.wait().await; + } + + Ok(()) + } } pub struct MergePartitionProcessor { @@ -294,7 +328,6 @@ pub struct MergePartitionProcessor { inputs: Vec>, inputs_data: Vec>, exchange: Arc, - // _phantom_data: PhantomData, } impl MergePartitionProcessor { 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 9a69465ac95ea..8dfb579090bab 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 @@ -34,26 +34,6 @@ use crate::servers::flight::v1::exchange::serde::serialize_block; use crate::servers::flight::v1::exchange::serde::ExchangeSerializeMeta; use crate::servers::flight::v1::scatter::FlightScatter; -pub fn compute_block_number(bucket: isize, max_partition_count: usize) -> Result { - match bucket.is_negative() { - true => Ok(((1_usize << 16) & (bucket.unsigned_abs() << 8) & max_partition_count) as isize), - false => Ok((((bucket as usize) << 8) & max_partition_count) as isize), - } -} - -pub fn restore_block_number(value: isize) -> (isize, usize) { - let mut value = value as usize; - let max_partition = value & 0xFF_usize; - value >>= 8; - let abs_partition = value & 0xFF_usize; - value >>= 8; - - match value & 1 { - 1 => (0 - abs_partition as isize, max_partition), - _ => (abs_partition as isize, max_partition), - } -} - fn scatter_payload(mut payload: Payload, buckets: usize) -> Result> { let mut buckets = Vec::with_capacity(buckets); @@ -94,7 +74,7 @@ pub struct FlightExchange { node_list_lookup: HashMap, options: IpcWriteOptions, - global_max_partition: AtomicUsize, + global_max_partition: Arc, shuffle_scatter: Arc>, } @@ -127,7 +107,7 @@ impl FlightExchange { .try_with_compression(compression) .unwrap(), shuffle_scatter, - global_max_partition: AtomicUsize::new(0), + global_max_partition: Arc::new(AtomicUsize::new(0)), }) } } @@ -135,7 +115,7 @@ impl FlightExchange { impl FlightExchange { fn default_partition(&self, data_block: DataBlock) -> Result> { if self.node_list.is_empty() { - let data_block = serialize_block(0, 0, data_block, &self.options)?; + let data_block = serialize_block(0, 0, 0, data_block, &self.options)?; return Ok(vec![data_block]); } @@ -148,39 +128,11 @@ impl FlightExchange { continue; } - blocks.push(serialize_block(0, 0, data_block, &self.options)?); + blocks.push(serialize_block(0, 0, 0, data_block, &self.options)?); } Ok(blocks) } - - fn get_global_max_partition(data_blocks: &[Option]) -> usize { - let mut global_max_partition = 0; - - for data_block in data_blocks { - let Some(data_block) = data_block else { - continue; - }; - - let Some(meta) = data_block.get_meta() else { - continue; - }; - - let meta_value = match ExchangeSerializeMeta::downcast_ref_from(meta) { - Some(meta) => meta.max_block_number, - None => match AggregateMeta::downcast_ref_from(meta) { - Some(meta) => meta.get_global_max_partition(), - None => { - continue; - } - }, - }; - - global_max_partition = global_max_partition.max(meta_value) - } - - global_max_partition - } } impl Exchange for FlightExchange { @@ -208,36 +160,34 @@ impl Exchange for FlightExchange { AggregateMeta::FinalPartition => unreachable!(), AggregateMeta::InFlightPayload(_) => unreachable!(), AggregateMeta::SpilledPayload(v) => { - let block_num = compute_block_number(-1, v.max_partition)?; - let mut blocks = Vec::with_capacity(n); + let global_max_partition = self.global_max_partition.load(AtomicOrdering::SeqCst); for node_id in &self.node_list { - blocks.push(match *node_id == self.local_id { - true => DataBlock::empty_with_meta( - AggregateMeta::create_in_flight_payload(v.partition, v.max_partition), - ), - false => serialize_block( - block_num, - v.global_max_partition, + let mut node_data_block = match *node_id == v.destination_node { + true => DataBlock::empty_with_meta(AggregateMeta::create_spilled_payload( + v.clone(), + )), + false => { DataBlock::empty_with_meta(AggregateMeta::create_in_flight_payload( - v.partition, + v.get_sorting_partition(), v.max_partition, - )), + global_max_partition, + )) + } + }; + + if *node_id != self.local_id { + node_data_block = serialize_block( + v.get_sorting_partition(), + v.max_partition, + global_max_partition, + node_data_block, &self.options, - )?, - }); - } + )? + } - let index = *self.node_list_lookup.get(&v.destination_node).unwrap(); - blocks[index] = match v.destination_node == self.local_id { - true => DataBlock::empty_with_meta(AggregateMeta::create_spilled_payload(v)), - false => serialize_block( - block_num, - v.global_max_partition, - DataBlock::empty_with_meta(AggregateMeta::create_spilled_payload(v)), - &self.options, - )?, - }; + blocks.push(node_data_block); + } Ok(blocks) } @@ -247,6 +197,7 @@ impl Exchange for FlightExchange { } let mut blocks = Vec::with_capacity(n); + let global_max_partition = self.global_max_partition.load(AtomicOrdering::SeqCst); for (idx, payload) in scatter_payload(p.payload, n)?.into_iter().enumerate() { if self.node_list[idx] == self.local_id { blocks.push(DataBlock::empty_with_meta( @@ -254,6 +205,7 @@ impl Exchange for FlightExchange { payload, p.partition, p.max_partition, + global_max_partition, ), )); @@ -265,14 +217,17 @@ impl Exchange for FlightExchange { false => payload.aggregate_flush_all()?, }; - let data_block = data_block.add_meta(Some( - AggregateMeta::create_in_flight_payload(p.partition, p.max_partition), - ))?; + let data_block = + data_block.add_meta(Some(AggregateMeta::create_in_flight_payload( + p.partition, + p.max_partition, + global_max_partition, + )))?; - let block_number = compute_block_number(p.partition, p.max_partition)?; let data_block = serialize_block( - block_number, - p.global_max_partition, + p.partition, + p.max_partition, + global_max_partition, data_block, &self.options, )?; @@ -284,6 +239,20 @@ impl Exchange for FlightExchange { } } + fn init_way(&self, _index: usize, block: &DataBlock) -> Result<()> { + let max_partition = match block.get_meta() { + None => 0, + Some(meta) => match AggregateMeta::downcast_ref_from(meta) { + None => 0, + Some(v) => v.get_max_partition(), + }, + }; + + self.global_max_partition + .fetch_max(max_partition, std::sync::atomic::Ordering::SeqCst); + Ok(()) + } + fn sorting_function(left_block: &DataBlock, right_block: &DataBlock) -> Ordering { let Some(left_meta) = left_block.get_meta() else { return Ordering::Equal; @@ -291,13 +260,13 @@ impl Exchange for FlightExchange { let (l_partition, l_max_partition) = match ExchangeSerializeMeta::downcast_ref_from(left_meta) { - Some(meta) => restore_block_number(meta.block_number), + Some(meta) => (meta.partition, meta.max_partition), None => { let Some(meta) = AggregateMeta::downcast_ref_from(left_meta) else { return Ordering::Equal; }; - (meta.get_partition(), meta.get_max_partition()) + (meta.get_sorting_partition(), meta.get_max_partition()) } }; @@ -307,13 +276,13 @@ impl Exchange for FlightExchange { let (r_partition, r_max_partition) = match ExchangeSerializeMeta::downcast_ref_from(right_meta) { - Some(meta) => restore_block_number(meta.block_number), + Some(meta) => (meta.partition, meta.max_partition), None => { let Some(meta) = AggregateMeta::downcast_ref_from(right_meta) else { return Ordering::Equal; }; - (meta.get_partition(), meta.get_max_partition()) + (meta.get_sorting_partition(), meta.get_max_partition()) } }; @@ -324,52 +293,4 @@ impl Exchange for FlightExchange { Ordering::Equal => l_partition.cmp(&r_partition), } } - - fn multiway_pick(&self, data_blocks: &mut [Option]) -> Option { - let new_value = Self::get_global_max_partition(data_blocks); - let old_value = self - .global_max_partition - .fetch_max(new_value, AtomicOrdering::SeqCst); - - let global_max_partition = std::cmp::max(new_value, old_value); - - let min_position = - data_blocks - .iter() - .enumerate() - .filter_map(|(idx, x)| x.as_ref().map(|d| (idx, d))) - .min_by(move |(left_idx, left_block), (right_idx, right_block)| { - match FlightExchange::::sorting_function(left_block, right_block) - { - Ordering::Less => Ordering::Less, - Ordering::Greater => Ordering::Greater, - Ordering::Equal => left_idx.cmp(right_idx), - } - }) - .map(|(idx, _)| idx); - - if let Some(min_pos) = min_position { - if global_max_partition == 0 { - return Some(min_pos); - } - - if let Some(mut block) = data_blocks[min_pos].take() { - let Some(meta) = block.get_meta() else { - return Some(min_pos); - }; - - if ExchangeSerializeMeta::downcast_ref_from(meta).is_some() { - let mut meta = ExchangeSerializeMeta::downcast_from(block.take_meta()?)?; - meta.max_block_number = global_max_partition; - data_blocks[min_pos] = Some(block.add_meta(Some(Box::new(meta))).unwrap()); - } else if AggregateMeta::downcast_ref_from(meta).is_some() { - let mut meta = AggregateMeta::downcast_from(block.take_meta()?)?; - meta.set_global_max_partition(global_max_partition); - data_blocks[min_pos] = Some(block.add_meta(Some(Box::new(meta))).unwrap()); - } - } - } - - min_position - } } 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 8b383aaf530a0..f4c8681ed5e7d 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 @@ -118,6 +118,12 @@ pub struct SpilledPayload { pub global_max_partition: usize, } +impl SpilledPayload { + pub fn get_sorting_partition(&self) -> isize { + -(self.max_partition as isize - self.partition) + } +} + pub struct AggregatePayload { pub partition: isize, pub payload: Payload, @@ -151,20 +157,25 @@ impl AggregateMeta { payload: Payload, partition: isize, max_partition: usize, + global_max_partition: usize, ) -> BlockMetaInfoPtr { Box::new(AggregateMeta::AggregatePayload(AggregatePayload { payload, partition, max_partition, - global_max_partition: max_partition, + global_max_partition, })) } - pub fn create_in_flight_payload(partition: isize, max_partition: usize) -> BlockMetaInfoPtr { + pub fn create_in_flight_payload( + partition: isize, + max_partition: usize, + global_max_partition: usize, + ) -> BlockMetaInfoPtr { Box::new(AggregateMeta::InFlightPayload(InFlightPayload { partition, max_partition, - global_max_partition: max_partition, + global_max_partition, })) } @@ -209,6 +220,16 @@ impl AggregateMeta { } } + pub fn get_sorting_partition(&self) -> isize { + match self { + AggregateMeta::Serialized(v) => v.bucket, + AggregateMeta::AggregatePayload(v) => v.partition, + AggregateMeta::InFlightPayload(v) => v.partition, + AggregateMeta::SpilledPayload(v) => v.get_sorting_partition(), + AggregateMeta::FinalPartition => unreachable!(), + } + } + pub fn get_max_partition(&self) -> usize { match self { AggregateMeta::Serialized(v) => v.max_partition, 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 a939193179d48..d4b73500c0864 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 @@ -12,13 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. -mod serde_meta; -mod transform_aggregate_serializer; mod transform_deserializer; mod transform_spill_reader; -pub use serde_meta::*; -pub use transform_aggregate_serializer::*; pub use transform_deserializer::*; pub use transform_spill_reader::*; diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/serde_meta.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/serde_meta.rs deleted file mode 100644 index b83cf2c97c90e..0000000000000 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/serde_meta.rs +++ /dev/null @@ -1,100 +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::ops::Range; - -use databend_common_expression::BlockMetaInfo; -use databend_common_expression::BlockMetaInfoDowncast; -use databend_common_expression::BlockMetaInfoPtr; - -pub const BUCKET_TYPE: usize = 1; -pub const SPILLED_TYPE: usize = 2; - -// Cannot change to enum, because bincode cannot deserialize custom enum -#[derive(serde::Serialize, serde::Deserialize, Clone, Debug, PartialEq)] -pub struct AggregateSerdeMeta { - pub typ: usize, - pub bucket: isize, - pub location: Option, - pub data_range: Option>, - pub columns_layout: Vec, - // use for new agg hashtable - pub max_partition_count: usize, - pub is_empty: bool, -} - -impl AggregateSerdeMeta { - pub fn create_agg_payload( - bucket: isize, - max_partition_count: usize, - is_empty: bool, - ) -> BlockMetaInfoPtr { - Box::new(AggregateSerdeMeta { - typ: BUCKET_TYPE, - bucket, - location: None, - data_range: None, - columns_layout: vec![], - max_partition_count, - is_empty, - }) - } - - pub fn create_spilled( - bucket: isize, - location: String, - data_range: Range, - columns_layout: Vec, - is_empty: bool, - ) -> BlockMetaInfoPtr { - Box::new(AggregateSerdeMeta { - typ: SPILLED_TYPE, - bucket, - columns_layout, - location: Some(location), - data_range: Some(data_range), - max_partition_count: 0, - is_empty, - }) - } - - pub fn create_agg_spilled( - bucket: isize, - location: String, - data_range: Range, - columns_layout: Vec, - max_partition_count: usize, - ) -> BlockMetaInfoPtr { - Box::new(AggregateSerdeMeta { - typ: SPILLED_TYPE, - bucket, - columns_layout, - location: Some(location), - data_range: Some(data_range), - max_partition_count, - is_empty: false, - }) - } -} - -#[typetag::serde(name = "aggregate_serde")] -impl BlockMetaInfo for AggregateSerdeMeta { - fn equals(&self, info: &Box) -> bool { - AggregateSerdeMeta::downcast_ref_from(info).is_some_and(|other| self == other) - } - - fn clone_self(&self) -> Box { - Box::new(self.clone()) - } -} 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 deleted file mode 100644 index 9ba343c199d8b..0000000000000 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_aggregate_serializer.rs +++ /dev/null @@ -1,259 +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::fmt::Formatter; -use std::pin::Pin; -use std::sync::Arc; - -use databend_common_exception::Result; -use databend_common_expression::local_block_meta_serde; -use databend_common_expression::BlockMetaInfo; -use databend_common_expression::BlockMetaInfoDowncast; -use databend_common_expression::BlockMetaInfoPtr; -use databend_common_expression::DataBlock; -use databend_common_expression::PayloadFlushState; -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 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; -pub struct TransformAggregateSerializer { - params: Arc, - - input: Arc, - output: Arc, - output_data: Option, - input_data: Option, -} - -impl TransformAggregateSerializer { - pub fn try_create( - input: Arc, - output: Arc, - params: Arc, - ) -> Result { - Ok(ProcessorPtr::create(Box::new( - TransformAggregateSerializer { - input, - output, - params, - input_data: None, - output_data: None, - }, - ))) - } -} - -impl Processor for TransformAggregateSerializer { - fn name(&self) -> String { - String::from("TransformAggregateSerializer") - } - - 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 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) { - match block_meta { - AggregateMeta::SpilledPayload(_) => unreachable!(), - AggregateMeta::Serialized(_) => unreachable!(), - AggregateMeta::InFlightPayload(_) => unreachable!(), - AggregateMeta::FinalPartition => unreachable!(), - AggregateMeta::AggregatePayload(p) => { - self.input_data = Some(SerializeAggregateStream::create( - &self.params, - SerializePayload::AggregatePayload(p), - )); - return Ok(Event::Sync); - } - } - } - } - - unreachable!() - } -} - -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, -} - -impl FlightSerializedMeta { - pub fn create(blocks: Vec) -> BlockMetaInfoPtr { - Box::new(FlightSerializedMeta { - serialized_blocks: blocks, - }) - } -} - -impl std::fmt::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 {} - -pub struct SerializeAggregateStream { - _params: Arc, - pub payload: Pin>, - flush_state: PayloadFlushState, - end_iter: bool, - nums: usize, -} - -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, - nums: 0, - } - } -} - -impl Iterator for SerializeAggregateStream { - type Item = Result; - - fn next(&mut self) -> Option { - Result::transpose(self.next_impl()) - } -} - -impl SerializeAggregateStream { - fn next_impl(&mut self) -> Result> { - if self.end_iter { - return Ok(None); - } - - match self.payload.as_ref().get_ref() { - SerializePayload::AggregatePayload(p) => { - let block = p.payload.aggregate_flush(&mut self.flush_state)?; - - if block.is_none() { - self.end_iter = true; - } - - match block { - Some(block) => { - self.nums += 1; - Ok(Some(block.add_meta(Some( - AggregateSerdeMeta::create_agg_payload( - p.partition, - p.max_partition, - false, - ), - ))?)) - } - None => { - // always return at least one block - if self.nums == 0 { - self.nums += 1; - let block = p.payload.empty_block(Some(1)); - Ok(Some(block.add_meta(Some( - AggregateSerdeMeta::create_agg_payload( - p.partition, - p.max_partition, - true, - ), - ))?)) - } else { - Ok(None) - } - } - } - } - } - } -} 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 ed9eaab533161..ddd0bd38d4a0f 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 @@ -17,6 +17,7 @@ use std::sync::Arc; use arrow_schema::Schema as ArrowSchema; use databend_common_exception::ErrorCode; use databend_common_exception::Result; +use databend_common_expression::BlockMetaInfoPtr; use databend_common_expression::DataBlock; use databend_common_expression::DataSchemaRef; use databend_common_io::prelude::BinaryRead; @@ -58,8 +59,9 @@ impl TransformDeserializer { fn recv_data(&self, dict: Vec, fragment_data: FragmentData) -> Result { const ROW_HEADER_SIZE: usize = std::mem::size_of::(); - let meta = serde_json::from_slice(&fragment_data.get_meta()[ROW_HEADER_SIZE..]) - .map_err(|_| ErrorCode::BadBytes("block meta deserialize error when exchange"))?; + let meta: Option = + serde_json::from_slice(&fragment_data.get_meta()[ROW_HEADER_SIZE..]) + .map_err(|_| ErrorCode::BadBytes("block meta deserialize error when exchange"))?; let mut row_count_meta = &fragment_data.get_meta()[..ROW_HEADER_SIZE]; let row_count: u32 = row_count_meta.read_scalar()?; 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 6ed460440b78e..951d24b8ef95a 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 @@ -288,6 +288,7 @@ impl AccumulatingTransform for TransformPartialAggregate { payload, partition as isize, partition_count, + partition_count, ), )); } 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 index d2c182f6e3769..d6fa74baf51d5 100644 --- 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 @@ -259,44 +259,44 @@ struct ResortingPartition { impl ResortingPartition { fn block_number(meta: &AggregateMeta) -> (isize, usize) { - match meta { - AggregateMeta::Serialized(v) => (v.bucket, v.max_partition), - AggregateMeta::SpilledPayload(v) => (v.partition, v.max_partition), - AggregateMeta::AggregatePayload(v) => (v.partition, v.max_partition), - AggregateMeta::InFlightPayload(v) => (v.partition, v.max_partition), - AggregateMeta::FinalPartition => unreachable!(), - } + (meta.get_sorting_partition(), meta.get_max_partition()) } +} - fn get_global_max_partition(data_blocks: &[Option]) -> usize { - let mut global_max_partition = 0; +impl Exchange for ResortingPartition { + const NAME: &'static str = "PartitionResorting"; + const MULTIWAY_SORT: bool = true; - for data_block in data_blocks { - let Some(data_block) = data_block else { - continue; - }; + fn partition(&self, mut data_block: DataBlock, n: usize) -> Result> { + debug_assert_eq!(n, 1); - let Some(meta) = data_block.get_meta() else { - continue; - }; - let Some(meta) = AggregateMeta::downcast_ref_from(meta) else { - continue; - }; + let Some(meta) = data_block.take_meta() else { + return Ok(vec![data_block]); + }; - global_max_partition = global_max_partition.max(meta.get_global_max_partition()) - } + let Some(_) = AggregateMeta::downcast_ref_from(&meta) else { + return Ok(vec![data_block]); + }; + + let global_max_partition = self.global_max_partition.load(AtomicOrdering::SeqCst); + let mut meta = AggregateMeta::downcast_from(meta).unwrap(); + meta.set_global_max_partition(global_max_partition); - global_max_partition + Ok(vec![data_block.add_meta(Some(Box::new(meta)))?]) } -} -impl Exchange for ResortingPartition { - const NAME: &'static str = "PartitionResorting"; - const MULTIWAY_SORT: bool = true; + fn init_way(&self, _index: usize, first_data: &DataBlock) -> Result<()> { + let max_partition = match first_data.get_meta() { + None => 0, + Some(meta) => match AggregateMeta::downcast_ref_from(meta) { + None => 0, + Some(v) => v.get_global_max_partition(), + }, + }; - fn partition(&self, data_block: DataBlock, n: usize) -> Result> { - debug_assert_eq!(n, 1); - Ok(vec![data_block]) + self.global_max_partition + .fetch_max(max_partition, std::sync::atomic::Ordering::SeqCst); + Ok(()) } fn sorting_function(left_block: &DataBlock, right_block: &DataBlock) -> Ordering { @@ -324,42 +324,6 @@ impl Exchange for ResortingPartition { Ordering::Equal => l_partition.cmp(&r_partition), } } - - fn multiway_pick(&self, data_blocks: &mut [Option]) -> Option { - let new_value = Self::get_global_max_partition(data_blocks); - let old_value = self - .global_max_partition - .fetch_max(new_value, AtomicOrdering::SeqCst); - - let global_max_partition = std::cmp::max(new_value, old_value); - - let min_position = data_blocks - .iter() - .enumerate() - .filter_map(|(i, x)| x.as_ref().map(|x| (i, x))) - .min_by(|(left_idx, left_block), (right_idx, right_block)| { - match ResortingPartition::sorting_function(left_block, right_block) { - Ordering::Less => Ordering::Less, - Ordering::Greater => Ordering::Greater, - Ordering::Equal => left_idx.cmp(right_idx), - } - }) - .map(|(idx, _)| idx); - - if let Some(min_pos) = min_position { - if global_max_partition == 0 { - return Some(min_pos); - } - - if let Some(mut block) = data_blocks[min_pos].take() { - let mut meta = AggregateMeta::downcast_from(block.take_meta().unwrap()).unwrap(); - meta.set_global_max_partition(global_max_partition); - data_blocks[min_pos] = Some(block.add_meta(Some(Box::new(meta))).unwrap()); - } - } - - min_position - } } pub fn build_partition_bucket( diff --git a/src/query/service/src/servers/flight/v1/exchange/exchange_transform.rs b/src/query/service/src/servers/flight/v1/exchange/exchange_transform.rs index b0dcefdba431f..0afb02e227455 100644 --- a/src/query/service/src/servers/flight/v1/exchange/exchange_transform.rs +++ b/src/query/service/src/servers/flight/v1/exchange/exchange_transform.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::HashMap; use std::sync::Arc; use databend_common_catalog::table_context::TableContext; @@ -69,17 +70,28 @@ impl ExchangeTransform { let receivers = exchange_manager.get_flight_receiver(&exchange_params)?; let nodes_source = receivers.len(); - let mut idx = 1; - let mut reorder = vec![0_usize; nodes_source]; - for (index, (destination_id, receiver)) in receivers.into_iter().enumerate() { + let mut lookup = params + .destination_ids + .iter() + .cloned() + .enumerate() + .map(|(x, y)| (y, x)) + .collect::>(); + + let mut nodes = Vec::with_capacity(nodes_source); + let mut reorder = Vec::with_capacity(nodes_source); + nodes.push(params.executor_id.clone()); + reorder.push(lookup.remove(¶ms.executor_id).unwrap()); + + for (destination_id, receiver) in receivers { if destination_id == params.executor_id { - reorder[0] = index; continue; } - reorder[idx] = index; - idx += 1; + nodes.push(destination_id.clone()); + reorder.push(lookup.remove(&destination_id).unwrap()); + items.push(create_reader_item( receiver, &destination_id, @@ -96,7 +108,11 @@ impl ExchangeTransform { }; pipeline.add_transform(|input, output| { - TransformAggregateDeserializer::try_create(input, output, ¶ms.schema) + TransformAggregateDeserializer::try_create( + input.clone(), + output.clone(), + ¶ms.schema, + ) }) } } diff --git a/src/query/service/src/servers/flight/v1/exchange/serde/exchange_serializer.rs b/src/query/service/src/servers/flight/v1/exchange/serde/exchange_serializer.rs index b6b8086806d0b..2d58d9d9c707b 100644 --- a/src/query/service/src/servers/flight/v1/exchange/serde/exchange_serializer.rs +++ b/src/query/service/src/servers/flight/v1/exchange/serde/exchange_serializer.rs @@ -37,21 +37,24 @@ use crate::servers::flight::v1::packets::DataPacket; use crate::servers::flight::v1::packets::FragmentData; pub struct ExchangeSerializeMeta { - pub block_number: isize, - pub max_block_number: usize, + pub partition: isize, + pub max_partition: usize, + pub global_max_partition: usize, pub packet: Vec, } impl ExchangeSerializeMeta { pub fn create( - block_number: isize, - max_block_number: usize, + partition: isize, + max_partition: usize, + global_max_partition: usize, packet: Vec, ) -> BlockMetaInfoPtr { Box::new(ExchangeSerializeMeta { packet, - block_number, - max_block_number, + partition, + max_partition, + global_max_partition, }) } } @@ -68,15 +71,17 @@ local_block_meta_serde!(ExchangeSerializeMeta); impl BlockMetaInfo for ExchangeSerializeMeta {} pub fn serialize_block( - block_num: isize, - max_block_num: usize, + partition: isize, + max_partition: usize, + global_max_partition: usize, data_block: DataBlock, options: &IpcWriteOptions, ) -> Result { if data_block.is_empty() && data_block.get_meta().is_none() { return Ok(DataBlock::empty_with_meta(ExchangeSerializeMeta::create( - block_num, - max_block_num, + partition, + max_partition, + global_max_partition, vec![], ))); } @@ -118,8 +123,9 @@ pub fn serialize_block( } Ok(DataBlock::empty_with_meta(ExchangeSerializeMeta::create( - block_num, - max_block_num, + partition, + max_partition, + global_max_partition, packet, ))) } From d61d925899f9b7635e8e60006906d961b8d694fb Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Thu, 20 Mar 2025 12:25:46 +0800 Subject: [PATCH 22/76] refactor(query): refactor aggreagte spill code --- .../core/src/processors/shuffle_processor.rs | 15 +++++++++++---- .../aggregator/transform_partition_bucket.rs | 11 +++++------ 2 files changed, 16 insertions(+), 10 deletions(-) diff --git a/src/query/pipeline/core/src/processors/shuffle_processor.rs b/src/query/pipeline/core/src/processors/shuffle_processor.rs index 212f9bc238ac8..09515276ffe15 100644 --- a/src/query/pipeline/core/src/processors/shuffle_processor.rs +++ b/src/query/pipeline/core/src/processors/shuffle_processor.rs @@ -255,7 +255,11 @@ impl Processor for PartitionProcessor { if all_output_finished { self.input.finish(); - return Ok(Event::Finished); + + return match self.initialized { + true => Ok(Event::Finished), + false => Ok(Event::Async), + }; } if !all_data_pushed_output { @@ -284,7 +288,10 @@ impl Processor for PartitionProcessor { output.finish(); } - return Ok(Event::Finished); + return match self.initialized { + true => Ok(Event::Finished), + false => Ok(Event::Async), + }; } self.input.set_need_data(); @@ -313,12 +320,12 @@ impl Processor for PartitionProcessor { } async fn async_process(&mut self) -> Result<()> { + self.initialized = true; if let Some(data_block) = self.input_data.as_ref() { - self.initialized = true; self.exchange.init_way(self.index, data_block)?; - self.barrier.wait().await; } + self.barrier.wait().await; Ok(()) } } 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 index d6fa74baf51d5..4257f59d4e6fd 100644 --- 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 @@ -218,8 +218,13 @@ impl Processor for TransformPartitionDispatch { self.input.set_need_data(); let mut has_data = false; + let input_is_finished = self.input.is_finished(); for (idx, output) in self.outputs.iter().enumerate() { if self.outputs_data[idx].is_empty() { + if input_is_finished { + output.finish(); + } + continue; } @@ -613,12 +618,6 @@ impl Partitions { Partitions::Unaligned(UnalignedPartitions::create(params)) } - // pub fn is_empty(&self) -> bool { - // match self { - // Partitions::Aligned(c) => {} - // } - // } - fn add_data(&mut self, meta: AggregateMeta, block: DataBlock) -> (isize, usize, usize) { match self { Partitions::Aligned(v) => v.add_data(meta, block), From 120621193382c10fa116cc046fdf5ca257b8b721 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 24 Mar 2025 15:35:02 +0800 Subject: [PATCH 23/76] refactor(query): refactor aggreagte spill code --- .../transforms/aggregator/transform_aggregate_partial.rs | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) 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 951d24b8ef95a..1cdefb5af888f 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 @@ -512,6 +512,7 @@ impl HashtableSpillingState { payload.states_layout.clone(), ); + scattered_payload.state_move_out = true; scattered_payload.copy_rows(sel, rows, &flush_state.addresses); if !self.serialize_payload(Some(scattered_payload))? { @@ -541,6 +542,7 @@ impl HashtableSpillingState { working_payload.states_layout.clone(), ); + scattered_payload.state_move_out = true; scattered_payload.copy_rows(sel, rows, &flush_state.addresses); if !self.serialize_payload(Some(scattered_payload))? { @@ -561,13 +563,17 @@ impl HashtableSpillingState { return self.serialize_scatter_payload(None); } - let partition_payload = PartitionedPayload::new( + let mut partition_payload = PartitionedPayload::new( self.ht.payload.group_types.clone(), self.ht.payload.aggrs.clone(), max_partitions as u64, self.ht.payload.arenas.clone(), ); + for payload in &mut partition_payload.payloads { + payload.state_move_out = true; + } + // repartition and get current partition payload for idx in self.payload_idx..self.ht.payload.payloads.len() { while partition_payload.gather_flush( @@ -595,6 +601,7 @@ impl HashtableSpillingState { working_payload.states_layout.clone(), ); + working_partition_payload.state_move_out = true; working_partition_payload.copy_rows(selector, rows, address); if !self.serialize_scatter_payload(Some(working_partition_payload))? { From 0374a7f000a24a75397b2534ebbed3bed39f553a Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 24 Mar 2025 17:21:22 +0800 Subject: [PATCH 24/76] refactor(query): refactor aggreagte spill code --- .../service/src/pipelines/executor/processor_async_task.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/query/service/src/pipelines/executor/processor_async_task.rs b/src/query/service/src/pipelines/executor/processor_async_task.rs index b864e5dab8d60..75143362d36ee 100644 --- a/src/query/service/src/pipelines/executor/processor_async_task.rs +++ b/src/query/service/src/pipelines/executor/processor_async_task.rs @@ -139,7 +139,7 @@ impl ProcessorAsyncTask { let elapsed = start.elapsed(); let active_workers = queue_clone.active_workers(); match elapsed >= Duration::from_secs(200) - && active_workers == 0 + // && active_workers == 0 && !log_graph { false => { From ce6e7547bc38485644fada3960cab7219c538899 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Fri, 28 Mar 2025 16:53:58 +0800 Subject: [PATCH 25/76] refactor(query): refactor aggreagte spill code --- .../core/src/processors/shuffle_processor.rs | 4 +++- .../aggregator/transform_aggregate_final.rs | 16 ++++++++++++---- .../transforms/range_join/merge_join_state.rs | 7 +++++-- .../transforms/range_join/range_join_state.rs | 6 +++++- 4 files changed, 25 insertions(+), 8 deletions(-) diff --git a/src/query/pipeline/core/src/processors/shuffle_processor.rs b/src/query/pipeline/core/src/processors/shuffle_processor.rs index 09515276ffe15..6d61e0de180fd 100644 --- a/src/query/pipeline/core/src/processors/shuffle_processor.rs +++ b/src/query/pipeline/core/src/processors/shuffle_processor.rs @@ -242,7 +242,9 @@ impl Processor for PartitionProcessor { if output.can_push() { if let Some(block) = self.partitioned_data[index].take() { - output.push_data(Ok(block)); + if !block.is_empty() || block.get_meta().is_some() { + output.push_data(Ok(block)); + } continue; } 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 49af84ed0add9..936f7a061da76 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 @@ -38,6 +38,7 @@ pub struct TransformFinalAggregate { params: Arc, flush_state: PayloadFlushState, hash_table: AggregateHashTable, + has_output: bool, } impl AccumulatingTransform for TransformFinalAggregate { @@ -97,10 +98,6 @@ impl AccumulatingTransform for TransformFinalAggregate { } } - if blocks.is_empty() { - blocks.push(self.params.empty_result_block()); - } - let config = HashTableConfig::default().with_initial_radix_bits(0); self.hash_table = AggregateHashTable::new( self.params.group_data_types.clone(), @@ -108,12 +105,22 @@ impl AccumulatingTransform for TransformFinalAggregate { config, Arc::new(Bump::new()), ); + + self.has_output |= !blocks.is_empty(); return Ok(blocks); } } Ok(vec![]) } + + fn on_finish(&mut self, output: bool) -> Result> { + if output && !self.has_output { + return Ok(vec![self.params.empty_result_block()]); + } + + Ok(vec![]) + } } impl TransformFinalAggregate { @@ -138,6 +145,7 @@ impl TransformFinalAggregate { params, hash_table, flush_state: PayloadFlushState::default(), + has_output: false, }, )) } diff --git a/src/query/service/src/pipelines/processors/transforms/range_join/merge_join_state.rs b/src/query/service/src/pipelines/processors/transforms/range_join/merge_join_state.rs index 10e0f7cd547c0..b409acf8ad0b4 100644 --- a/src/query/service/src/pipelines/processors/transforms/range_join/merge_join_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/range_join/merge_join_state.rs @@ -79,11 +79,12 @@ impl RangeJoinState { let right_table = self.right_table.read(); while i < left_len { - if j == right_len { + eprintln!("range join {task_id} {i}, {j}, {left_len}, {right_len}"); + if j >= right_len { i += 1; j = 0; } - if i == left_len { + if i >= left_len { break; } let left_scalar = unsafe { left_join_key_col.index_unchecked(i) }; @@ -137,6 +138,8 @@ impl RangeJoinState { j += 1; } } + + eprintln!("range join end {task_id} {i}, {j}, {left_len}, {right_len}"); Ok(result_blocks) } diff --git a/src/query/service/src/pipelines/processors/transforms/range_join/range_join_state.rs b/src/query/service/src/pipelines/processors/transforms/range_join/range_join_state.rs index bdc79cd0df387..bd9e6e0d2611c 100644 --- a/src/query/service/src/pipelines/processors/transforms/range_join/range_join_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/range_join/range_join_state.rs @@ -133,9 +133,13 @@ impl RangeJoinState { pub fn task_id(&self) -> Option { let task_id = self.finished_tasks.fetch_add(1, atomic::Ordering::SeqCst); - if task_id >= self.tasks.read().len() as u64 { + let task_len = self.tasks.read().len() as u64; + + if task_id >= task_len { return None; } + + eprintln!("task id {}, task len {}", task_id, task_len); Some(task_id as usize) } From 6a53857dba715da7fbbfa01cbdea0d5fc6435106 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sat, 29 Mar 2025 12:22:24 +0800 Subject: [PATCH 26/76] refactor(query): refactor aggreagte spill code --- .../transforms/aggregator/transform_aggregate_partial.rs | 6 +----- .../processors/transforms/range_join/merge_join_state.rs | 2 -- .../processors/transforms/range_join/range_join_state.rs | 4 +--- 3 files changed, 2 insertions(+), 10 deletions(-) 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 1cdefb5af888f..a5d849eb62930 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 @@ -313,6 +313,7 @@ impl AccumulatingTransform for TransformPartialAggregate { }; if ht.len() == 0 { + self.hash_table = HashTable::AggregateHashTable(ht); return Ok(false); } @@ -321,11 +322,6 @@ impl AccumulatingTransform for TransformPartialAggregate { } if let Some(spilling_state) = self.spilling_state.as_mut() { - // spill is finished. - if spilling_state.finished { - return Ok(false); - } - spilling_state.last_prepare_payload = spilling_state.serialize_partition_payload()?; return Ok(true); } diff --git a/src/query/service/src/pipelines/processors/transforms/range_join/merge_join_state.rs b/src/query/service/src/pipelines/processors/transforms/range_join/merge_join_state.rs index b409acf8ad0b4..f6dd2af1508e9 100644 --- a/src/query/service/src/pipelines/processors/transforms/range_join/merge_join_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/range_join/merge_join_state.rs @@ -79,7 +79,6 @@ impl RangeJoinState { let right_table = self.right_table.read(); while i < left_len { - eprintln!("range join {task_id} {i}, {j}, {left_len}, {right_len}"); if j >= right_len { i += 1; j = 0; @@ -139,7 +138,6 @@ impl RangeJoinState { } } - eprintln!("range join end {task_id} {i}, {j}, {left_len}, {right_len}"); Ok(result_blocks) } diff --git a/src/query/service/src/pipelines/processors/transforms/range_join/range_join_state.rs b/src/query/service/src/pipelines/processors/transforms/range_join/range_join_state.rs index bd9e6e0d2611c..3d144a0d4c4f2 100644 --- a/src/query/service/src/pipelines/processors/transforms/range_join/range_join_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/range_join/range_join_state.rs @@ -133,13 +133,11 @@ impl RangeJoinState { pub fn task_id(&self) -> Option { let task_id = self.finished_tasks.fetch_add(1, atomic::Ordering::SeqCst); - let task_len = self.tasks.read().len() as u64; - if task_id >= task_len { + if task_id >= self.tasks.read().len() as u64 { return None; } - eprintln!("task id {}, task len {}", task_id, task_len); Some(task_id as usize) } From 31037d1d664dbd5b253640467d273c65313868bd Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sun, 30 Mar 2025 14:50:05 +0800 Subject: [PATCH 27/76] refactor(query): refactor aggreagte spill code --- src/common/base/src/runtime/memory/mem_stat.rs | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/common/base/src/runtime/memory/mem_stat.rs b/src/common/base/src/runtime/memory/mem_stat.rs index 2ace8dcd0ca66..d646d925750ce 100644 --- a/src/common/base/src/runtime/memory/mem_stat.rs +++ b/src/common/base/src/runtime/memory/mem_stat.rs @@ -112,8 +112,6 @@ impl MemStat { if NEED_ROLLBACK { // We only roll back the memory that alloc failed self.used.fetch_sub(current_memory_alloc, Ordering::Relaxed); - - parent_memory_stat.rollback(current_memory_alloc); } return Err(cause); From f687164c3780148ba14f8d00505ddcdaa81aca80 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 1 Apr 2025 14:59:21 +0800 Subject: [PATCH 28/76] refactor(query): refactor aggreagte spill code --- .../aggregator/transform_aggregate_final.rs | 24 +++++++++++-------- .../aggregator/transform_partition_bucket.rs | 23 +++++++++++++++--- .../transforms/range_join/merge_join_state.rs | 4 ++-- .../transforms/range_join/range_join_state.rs | 20 +++++++++++----- 4 files changed, 50 insertions(+), 21 deletions(-) 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 936f7a061da76..bbf2ca207eca9 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 @@ -56,6 +56,8 @@ impl AccumulatingTransform for TransformFinalAggregate { match aggregate_meta { AggregateMeta::SpilledPayload(_) => unreachable!(), AggregateMeta::Serialized(payload) => { + debug_assert_eq!(payload.max_partition, payload.global_max_partition); + let payload = payload.convert_to_partitioned_payload( self.params.group_data_types.clone(), self.params.aggregate_functions.clone(), @@ -67,7 +69,9 @@ impl AccumulatingTransform for TransformFinalAggregate { self.hash_table .combine_payloads(&payload, &mut self.flush_state)?; } - AggregateMeta::InFlightPayload(_payload) => { + AggregateMeta::InFlightPayload(payload) => { + debug_assert_eq!(payload.max_partition, payload.global_max_partition); + if !data.is_empty() { let payload = self.deserialize_flight(data)?; self.hash_table @@ -75,6 +79,8 @@ impl AccumulatingTransform for TransformFinalAggregate { } } AggregateMeta::AggregatePayload(payload) => { + debug_assert_eq!(payload.max_partition, payload.global_max_partition); + if payload.payload.len() != 0 { self.hash_table .combine_payload(&payload.payload, &mut self.flush_state)?; @@ -82,20 +88,16 @@ impl AccumulatingTransform for TransformFinalAggregate { } AggregateMeta::FinalPartition => { if self.hash_table.len() == 0 { - return Ok(vec![self.params.empty_result_block()]); + return Ok(vec![]); } let mut blocks = vec![]; self.flush_state.clear(); - loop { - if self.hash_table.merge_result(&mut self.flush_state)? { - let mut cols = self.flush_state.take_aggregate_results(); - cols.extend_from_slice(&self.flush_state.take_group_columns()); - blocks.push(DataBlock::new_from_columns(cols)); - } else { - break; - } + while self.hash_table.merge_result(&mut self.flush_state)? { + let mut cols = self.flush_state.take_aggregate_results(); + cols.extend_from_slice(&self.flush_state.take_group_columns()); + blocks.push(DataBlock::new_from_columns(cols)); } let config = HashTableConfig::default().with_initial_radix_bits(0); @@ -115,6 +117,8 @@ impl AccumulatingTransform for TransformFinalAggregate { } fn on_finish(&mut self, output: bool) -> Result> { + assert_eq!(self.hash_table.len(), 0); + if output && !self.has_output { return Ok(vec![self.params.empty_result_block()]); } 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 index 4257f59d4e6fd..fd7f1b081dad1 100644 --- 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 @@ -120,7 +120,7 @@ impl TransformPartitionDispatch { fn ready_partition(&mut self) -> Option { let storage_min_partition = self.partitions.min_partition()?; - if storage_min_partition >= self.working_partition { + if storage_min_partition > self.working_partition { return None; } @@ -220,7 +220,7 @@ impl Processor for TransformPartitionDispatch { let mut has_data = false; let input_is_finished = self.input.is_finished(); for (idx, output) in self.outputs.iter().enumerate() { - if self.outputs_data[idx].is_empty() { + if self.outputs_data[idx].is_empty() && self.partitions.is_empty() { if input_is_finished { output.finish(); } @@ -238,6 +238,8 @@ impl Processor for TransformPartitionDispatch { } if self.input.is_finished() && !has_data { + debug_assert!(self.partitions.is_empty()); + for output in &self.outputs { output.finish(); } @@ -258,7 +260,7 @@ impl Processor for TransformPartitionDispatch { } } -struct ResortingPartition { +pub struct ResortingPartition { global_max_partition: AtomicUsize, } @@ -400,6 +402,10 @@ impl UnalignedPartitions { } } + pub fn is_empty(&self) -> bool { + self.data.is_empty() + } + fn insert_data(&mut self, idx: usize, meta: AggregateMeta, block: DataBlock) { match self.data.entry(idx) { Entry::Vacant(v) => { @@ -578,6 +584,10 @@ struct AlignedPartitions { } impl AlignedPartitions { + pub fn is_empty(&self) -> bool { + self.data.is_empty() + } + pub fn add_data(&mut self, meta: AggregateMeta, block: DataBlock) -> (isize, usize, usize) { let (partition, max_partition, global_max_partition) = match &meta { AggregateMeta::Serialized(_) => unreachable!(), @@ -618,6 +628,13 @@ impl Partitions { Partitions::Unaligned(UnalignedPartitions::create(params)) } + pub fn is_empty(&self) -> bool { + match self { + Partitions::Aligned(v) => v.is_empty(), + Partitions::Unaligned(v) => v.is_empty(), + } + } + fn add_data(&mut self, meta: AggregateMeta, block: DataBlock) -> (isize, usize, usize) { match self { Partitions::Aligned(v) => v.add_data(meta, block), diff --git a/src/query/service/src/pipelines/processors/transforms/range_join/merge_join_state.rs b/src/query/service/src/pipelines/processors/transforms/range_join/merge_join_state.rs index f6dd2af1508e9..1e512b1a2c630 100644 --- a/src/query/service/src/pipelines/processors/transforms/range_join/merge_join_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/range_join/merge_join_state.rs @@ -79,11 +79,11 @@ impl RangeJoinState { let right_table = self.right_table.read(); while i < left_len { - if j >= right_len { + if j == right_len { i += 1; j = 0; } - if i >= left_len { + if i == left_len { break; } let left_scalar = unsafe { left_join_key_col.index_unchecked(i) }; diff --git a/src/query/service/src/pipelines/processors/transforms/range_join/range_join_state.rs b/src/query/service/src/pipelines/processors/transforms/range_join/range_join_state.rs index 3d144a0d4c4f2..c376b45706ebd 100644 --- a/src/query/service/src/pipelines/processors/transforms/range_join/range_join_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/range_join/range_join_state.rs @@ -92,16 +92,22 @@ impl RangeJoinState { } pub(crate) fn sink_right(&self, block: DataBlock) -> Result<()> { - // Sink block to right table - let mut right_table = self.right_table.write(); - right_table.push(block); + if !block.is_empty() { + // Sink block to right table + let mut right_table = self.right_table.write(); + right_table.push(block); + } + Ok(()) } pub(crate) fn sink_left(&self, block: DataBlock) -> Result<()> { - // Sink block to left table - let mut left_table = self.left_table.write(); - left_table.push(block); + if !block.is_empty() { + // Sink block to left table + let mut left_table = self.left_table.write(); + left_table.push(block); + } + Ok(()) } @@ -178,6 +184,7 @@ impl RangeJoinState { let left_table = self.left_table.read(); // Right table is bigger than left table let mut right_table = self.right_table.write(); + if !left_table.is_empty() && !right_table.is_empty() && left_table.len() * right_table.len() < max_threads @@ -274,6 +281,7 @@ impl RangeJoinState { right_offset = 0; left_offset += left_block.num_rows(); } + Ok(()) } } From 7e5963fc502d893d0acc8706ec648eb98446f3cd Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 1 Apr 2025 15:57:27 +0800 Subject: [PATCH 29/76] refactor(query): refactor aggreagte spill code --- .../aggregator/aggregate_exchange_injector.rs | 1 - .../transforms/aggregator/aggregate_meta.rs | 24 ---------- .../serde/transform_spill_reader.rs | 19 +++----- .../aggregator/transform_aggregate_final.rs | 14 ------ .../aggregator/transform_partition_bucket.rs | 44 +++++++------------ 5 files changed, 24 insertions(+), 78 deletions(-) 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 8dfb579090bab..eea38704d82d7 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 @@ -156,7 +156,6 @@ impl Exchange for FlightExchange { assert!(MULTIWAY_SORT); assert_eq!(self.node_list_lookup.len(), n); match AggregateMeta::downcast_from(meta).unwrap() { - AggregateMeta::Serialized(_) => unreachable!(), AggregateMeta::FinalPartition => unreachable!(), AggregateMeta::InFlightPayload(_) => unreachable!(), AggregateMeta::SpilledPayload(v) => { 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 f4c8681ed5e7d..0378769138f73 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 @@ -145,7 +145,6 @@ pub struct FinalPayload { #[derive(serde::Serialize, serde::Deserialize)] pub enum AggregateMeta { - Serialized(SerializedPayload), SpilledPayload(SpilledPayload), AggregatePayload(AggregatePayload), InFlightPayload(InFlightPayload), @@ -179,19 +178,6 @@ impl AggregateMeta { })) } - pub fn create_serialized( - bucket: isize, - block: DataBlock, - max_partition: usize, - ) -> BlockMetaInfoPtr { - Box::new(AggregateMeta::Serialized(SerializedPayload { - bucket, - data_block: block, - max_partition, - global_max_partition: max_partition, - })) - } - pub fn create_spilled_payload(payload: SpilledPayload) -> BlockMetaInfoPtr { Box::new(AggregateMeta::SpilledPayload(payload)) } @@ -202,7 +188,6 @@ impl AggregateMeta { pub fn get_global_max_partition(&self) -> usize { match self { - AggregateMeta::Serialized(v) => v.global_max_partition, AggregateMeta::SpilledPayload(v) => v.global_max_partition, AggregateMeta::AggregatePayload(v) => v.global_max_partition, AggregateMeta::InFlightPayload(v) => v.global_max_partition, @@ -212,7 +197,6 @@ impl AggregateMeta { pub fn get_partition(&self) -> isize { match self { - AggregateMeta::Serialized(v) => v.bucket, AggregateMeta::SpilledPayload(v) => v.partition, AggregateMeta::AggregatePayload(v) => v.partition, AggregateMeta::InFlightPayload(v) => v.partition, @@ -222,7 +206,6 @@ impl AggregateMeta { pub fn get_sorting_partition(&self) -> isize { match self { - AggregateMeta::Serialized(v) => v.bucket, AggregateMeta::AggregatePayload(v) => v.partition, AggregateMeta::InFlightPayload(v) => v.partition, AggregateMeta::SpilledPayload(v) => v.get_sorting_partition(), @@ -232,7 +215,6 @@ impl AggregateMeta { pub fn get_max_partition(&self) -> usize { match self { - AggregateMeta::Serialized(v) => v.max_partition, AggregateMeta::SpilledPayload(v) => v.max_partition, AggregateMeta::AggregatePayload(v) => v.max_partition, AggregateMeta::InFlightPayload(v) => v.max_partition, @@ -242,9 +224,6 @@ impl AggregateMeta { pub fn set_global_max_partition(&mut self, global_max_partition: usize) { match self { - AggregateMeta::Serialized(v) => { - v.global_max_partition = global_max_partition; - } AggregateMeta::SpilledPayload(v) => { v.global_max_partition = global_max_partition; } @@ -263,9 +242,6 @@ impl Debug for AggregateMeta { fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { match self { AggregateMeta::FinalPartition => f.debug_struct("AggregateMeta::Partitioned").finish(), - AggregateMeta::Serialized { .. } => { - f.debug_struct("AggregateMeta::Serialized").finish() - } AggregateMeta::SpilledPayload(_) => { f.debug_struct("Aggregate::SpilledPayload").finish() } 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 4efa995e3ca56..a359c27227f8b 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 @@ -119,9 +119,7 @@ impl Processor for TransformSpillReader { AggregateMeta::SpilledPayload(payload) => { debug_assert!(read_data.len() == 1); let data = read_data.pop_front().unwrap(); - self.output_data = Some(DataBlock::empty_with_meta(Box::new( - self.deserialize(payload, data), - ))); + self.output_data = Some(self.deserialize(payload, data)?); } _ => unreachable!(), } @@ -179,7 +177,7 @@ impl TransformSpillReader { }))) } - fn deserialize(&self, payload: SpilledPayload, data: Vec) -> AggregateMeta { + fn deserialize(&self, payload: SpilledPayload, data: Vec) -> Result { let columns = self.params.group_data_types.len() + self.params.aggregate_functions.len(); let mut blocks = vec![]; @@ -201,13 +199,10 @@ impl TransformSpillReader { let block = DataBlock::concat(&blocks).unwrap(); - AggregateMeta::Serialized(SerializedPayload { - bucket: payload.partition, - data_block: block, - max_partition: payload.max_partition, - global_max_partition: payload.global_max_partition, - }) + block.add_meta(Some(AggregateMeta::create_in_flight_payload( + payload.partition, + payload.max_partition, + payload.global_max_partition, + ))) } } - -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 bbf2ca207eca9..255f2683640d4 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 @@ -55,20 +55,6 @@ impl AccumulatingTransform for TransformFinalAggregate { match aggregate_meta { AggregateMeta::SpilledPayload(_) => unreachable!(), - AggregateMeta::Serialized(payload) => { - debug_assert_eq!(payload.max_partition, payload.global_max_partition); - - let payload = payload.convert_to_partitioned_payload( - self.params.group_data_types.clone(), - self.params.aggregate_functions.clone(), - self.params.num_states(), - 0, - Arc::new(Bump::new()), - )?; - - self.hash_table - .combine_payloads(&payload, &mut self.flush_state)?; - } AggregateMeta::InFlightPayload(payload) => { debug_assert_eq!(payload.max_partition, payload.global_max_partition); 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 index fd7f1b081dad1..8f182fcbe5ca6 100644 --- 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 @@ -49,8 +49,8 @@ use databend_common_storage::DataOperator; use tokio::sync::Semaphore; use super::AggregatePayload; -use super::TransformAggregateSpillReader; use super::TransformFinalAggregate; +use super::TransformSpillReader; use crate::pipelines::processors::transforms::aggregator::aggregate_meta::AggregateMeta; use crate::pipelines::processors::transforms::aggregator::AggregatorParams; @@ -201,7 +201,10 @@ impl Processor for TransformPartitionDispatch { if self.input.is_finished() { self.working_partition = self.max_partition as isize; - self.fetch_ready_partition()?; + // fetch all partition + while !self.partitions.is_empty() { + self.fetch_ready_partition()?; + } } if self.input.has_data() { @@ -217,37 +220,33 @@ impl Processor for TransformPartitionDispatch { self.input.set_need_data(); - let mut has_data = false; - let input_is_finished = self.input.is_finished(); + let mut all_output_finished = true; for (idx, output) in self.outputs.iter().enumerate() { + if output.is_finished() { + continue; + } + if self.outputs_data[idx].is_empty() && self.partitions.is_empty() { - if input_is_finished { + if self.input.is_finished() { output.finish(); } continue; } + all_output_finished = false; + if output.can_push() { if let Some(block) = self.outputs_data[idx].pop_front() { output.push_data(Ok(block)); } } - - has_data |= !self.outputs_data[idx].is_empty(); } - if self.input.is_finished() && !has_data { - debug_assert!(self.partitions.is_empty()); - - for output in &self.outputs { - output.finish(); - } - - return Ok(Event::Finished); + match all_output_finished { + true => Ok(Event::Finished), + false => Ok(Event::NeedData), } - - Ok(Event::NeedData) } fn process(&mut self) -> Result<()> { @@ -362,13 +361,7 @@ pub fn build_partition_bucket( let operator = DataOperator::instance().spill_operator(); pipeline.add_transform(|input, output| { let operator = operator.clone(); - TransformAggregateSpillReader::create( - input, - output, - operator, - semaphore.clone(), - params.clone(), - ) + TransformSpillReader::create(input, output, operator, semaphore.clone(), params.clone()) })?; pipeline.add_transform(|input, output| { @@ -419,7 +412,6 @@ impl UnalignedPartitions { pub fn add_data(&mut self, meta: AggregateMeta, block: DataBlock) -> (isize, usize, usize) { match &meta { - AggregateMeta::Serialized(_) => unreachable!(), AggregateMeta::FinalPartition => unreachable!(), AggregateMeta::SpilledPayload(payload) => { let max_partition = payload.max_partition; @@ -532,7 +524,6 @@ impl UnalignedPartitions { for (_, repartition_data) in repartition_data { for (meta, block) in repartition_data { match meta { - AggregateMeta::Serialized(_) => unreachable!(), AggregateMeta::FinalPartition => unreachable!(), AggregateMeta::SpilledPayload(_) => unreachable!(), AggregateMeta::InFlightPayload(payload) => { @@ -590,7 +581,6 @@ impl AlignedPartitions { pub fn add_data(&mut self, meta: AggregateMeta, block: DataBlock) -> (isize, usize, usize) { let (partition, max_partition, global_max_partition) = match &meta { - AggregateMeta::Serialized(_) => unreachable!(), AggregateMeta::FinalPartition => unreachable!(), AggregateMeta::SpilledPayload(v) => { (v.partition, v.max_partition, v.global_max_partition) From 55f522dbaddab1810008b97369b20a7820fa2a4a Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 1 Apr 2025 16:06:18 +0800 Subject: [PATCH 30/76] refactor(query): refactor aggreagte spill code --- .../transforms/aggregator/serde/transform_spill_reader.rs | 1 - 1 file changed, 1 deletion(-) 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 a359c27227f8b..300eb3edc6ff1 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 @@ -32,7 +32,6 @@ use tokio::sync::Semaphore; use crate::pipelines::processors::transforms::aggregator::AggregateMeta; use crate::pipelines::processors::transforms::aggregator::AggregatorParams; -use crate::pipelines::processors::transforms::aggregator::SerializedPayload; use crate::pipelines::processors::transforms::aggregator::SpilledPayload; type DeserializingMeta = (AggregateMeta, VecDeque>); From 2a7f2bce64a734ddb0d91b1d3f3ba9181234b3b4 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 1 Apr 2025 16:42:17 +0800 Subject: [PATCH 31/76] refactor(query): refactor aggreagte spill code --- .../transforms/aggregator/transform_aggregate_final.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 255f2683640d4..b5a7ba09e282f 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 @@ -103,7 +103,7 @@ impl AccumulatingTransform for TransformFinalAggregate { } fn on_finish(&mut self, output: bool) -> Result> { - assert_eq!(self.hash_table.len(), 0); + assert!(!output || self.hash_table.len() == 0); if output && !self.has_output { return Ok(vec![self.params.empty_result_block()]); From 7af425547dc822ef3d6188d0baed4984c43e221d Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 2 Apr 2025 12:07:21 +0800 Subject: [PATCH 32/76] refactor(query): refactor aggreagte spill code --- .../pipelines/builders/builder_aggregate.rs | 4 +- .../processors/transforms/aggregator/mod.rs | 2 +- .../aggregator/transform_partition_bucket.rs | 123 ++++++++---------- 3 files changed, 59 insertions(+), 70 deletions(-) diff --git a/src/query/service/src/pipelines/builders/builder_aggregate.rs b/src/query/service/src/pipelines/builders/builder_aggregate.rs index 996554cc691ef..40bc0a896c7ab 100644 --- a/src/query/service/src/pipelines/builders/builder_aggregate.rs +++ b/src/query/service/src/pipelines/builders/builder_aggregate.rs @@ -37,7 +37,7 @@ use databend_common_sql::IndexType; use databend_common_storage::DataOperator; use itertools::Itertools; -use crate::pipelines::processors::transforms::aggregator::build_partition_bucket; +use crate::pipelines::processors::transforms::aggregator::build_partition_dispatch; use crate::pipelines::processors::transforms::aggregator::create_udaf_script_function; use crate::pipelines::processors::transforms::aggregator::AggregatorParams; use crate::pipelines::processors::transforms::aggregator::FinalSingleStateAggregator; @@ -205,7 +205,7 @@ impl PipelineBuilder { self.build_pipeline(&aggregate.input)?; self.enable_multiway_sort = old_value; - build_partition_bucket(&mut self.main_pipeline, params.clone()) + build_partition_dispatch(&mut self.main_pipeline, params.clone()) } fn build_aggregator_params( 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 442b0e38ee0d9..1d7a7f94b47f0 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/mod.rs @@ -29,7 +29,7 @@ pub use aggregator_params::AggregatorParams; pub use transform_aggregate_expand::TransformExpandGroupingSets; pub use transform_aggregate_final::TransformFinalAggregate; pub use transform_aggregate_partial::TransformPartialAggregate; -pub use transform_partition_bucket::build_partition_bucket; +pub use transform_partition_bucket::build_partition_dispatch; pub use transform_single_key::FinalSingleStateAggregator; pub use transform_single_key::PartialSingleStateAggregator; pub use udaf_script::*; 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 index 8f182fcbe5ca6..0cd909d4fb5e0 100644 --- 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 @@ -62,6 +62,7 @@ pub struct TransformPartitionDispatch { outputs_data: Vec>, output_index: usize, initialized_input: bool, + finished: bool, max_partition: usize, working_partition: isize, @@ -87,6 +88,7 @@ impl TransformPartitionDispatch { initialized_input: false, partitions: Partitions::create_unaligned(params), working_partition: 0, + finished: false, }) } @@ -128,7 +130,7 @@ impl TransformPartitionDispatch { } fn fetch_ready_partition(&mut self) -> Result<()> { - while let Some(ready_partition_id) = self.ready_partition() { + if let Some(ready_partition_id) = self.ready_partition() { let ready_partition = self.partitions.take_partition(ready_partition_id); for (meta, data_block) in ready_partition { @@ -160,92 +162,79 @@ impl Processor for TransformPartitionDispatch { } fn event(&mut self) -> Result { - let mut all_output_finished = true; + loop { + let mut all_output_finished = true; + let mut all_data_pushed_output = true; + + for (idx, output) in self.outputs.iter().enumerate() { + if output.is_finished() { + self.outputs_data[idx].clear(); + continue; + } - for output in &self.outputs { - if !output.is_finished() { - all_output_finished = false; - break; - } - } + if self.finished && self.outputs_data[idx].is_empty() { + output.finish(); + continue; + } - if all_output_finished { - self.input.finish(); - return Ok(Event::Finished); - } + all_output_finished = false; - // We pull the first unsplitted data block - if !self.initialized_input { - if self.initialize_input()? { - return Ok(Event::Sync); - } + if output.can_push() { + if let Some(block) = self.outputs_data[idx].pop_front() { + if !block.is_empty() || block.get_meta().is_some() { + output.push_data(Ok(block)); + } - self.input.set_need_data(); - return Ok(Event::NeedData); - } + continue; + } + } - let mut output_can_push = false; - for (idx, output) in self.outputs.iter().enumerate() { - if output.can_push() { - output_can_push = true; - if let Some(block) = self.outputs_data[idx].pop_front() { - output.push_data(Ok(block)); + if !self.outputs_data[idx].is_empty() { + all_data_pushed_output = false; } } - } - - if !output_can_push { - self.input.set_not_need_data(); - return Ok(Event::NeedConsume); - } - if self.input.is_finished() { - self.working_partition = self.max_partition as isize; - // fetch all partition - while !self.partitions.is_empty() { - self.fetch_ready_partition()?; + if all_output_finished { + self.input.finish(); + return Ok(Event::Finished); } - } - if self.input.has_data() { - let data_block = self.input.pull_data().unwrap()?; - let (partition, _, _) = self.partitions.add_block(data_block)?; - - if partition != self.working_partition { - // ready partition - self.fetch_ready_partition()?; - self.working_partition = partition; + if !all_data_pushed_output { + self.input.set_not_need_data(); + return Ok(Event::NeedConsume); } - } - self.input.set_need_data(); + // We pull the first unaligned partition + if !self.initialized_input { + if self.initialize_input()? { + return Ok(Event::Sync); + } - let mut all_output_finished = true; - for (idx, output) in self.outputs.iter().enumerate() { - if output.is_finished() { - continue; + self.input.set_need_data(); + return Ok(Event::NeedData); } - if self.outputs_data[idx].is_empty() && self.partitions.is_empty() { - if self.input.is_finished() { - output.finish(); - } + if self.input.has_data() { + let data_block = self.input.pull_data().unwrap()?; + let (partition, _, _) = self.partitions.add_block(data_block)?; - continue; + if partition != self.working_partition { + self.fetch_ready_partition()?; + self.working_partition = partition; + continue; + } } - all_output_finished = false; + if self.input.is_finished() { + self.working_partition = self.max_partition as isize; - if output.can_push() { - if let Some(block) = self.outputs_data[idx].pop_front() { - output.push_data(Ok(block)); - } + self.fetch_ready_partition()?; + self.finished = self.partitions.is_empty(); + continue; } - } - match all_output_finished { - true => Ok(Event::Finished), - false => Ok(Event::NeedData), + self.input.set_need_data(); + return Ok(Event::NeedData); } } @@ -332,7 +321,7 @@ impl Exchange for ResortingPartition { } } -pub fn build_partition_bucket( +pub fn build_partition_dispatch( pipeline: &mut Pipeline, params: Arc, ) -> Result<()> { From 9728660743c34053790308883e2813ca226e1f17 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 2 Apr 2025 13:23:06 +0800 Subject: [PATCH 33/76] refactor(query): refactor aggreagte spill code --- .../aggregator/transform_aggregate_partial.rs | 14 +- .../aggregator/transform_partition_bucket.rs | 569 ++++++++++-------- 2 files changed, 334 insertions(+), 249 deletions(-) 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 a5d849eb62930..434a86d21f9a5 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 @@ -78,7 +78,7 @@ pub struct TransformPartialAggregate { configure_peer_nodes: Vec, spilling_state: Option, spiller: Arc, - spill_blocks: Vec, + output_blocks: Vec, } impl TransformPartialAggregate { @@ -136,7 +136,7 @@ impl TransformPartialAggregate { configure_peer_nodes: vec![GlobalConfig::instance().query.node_id.clone()], spilling_state: None, spiller: Arc::new(spiller), - spill_blocks: vec![], + output_blocks: vec![], }, )) } @@ -249,7 +249,7 @@ impl AccumulatingTransform for TransformPartialAggregate { fn transform(&mut self, block: DataBlock) -> Result> { self.execute_one_block(block)?; - Ok(std::mem::take(&mut self.spill_blocks)) + Ok(vec![]) } fn on_finish(&mut self, output: bool) -> Result> { @@ -260,8 +260,6 @@ impl AccumulatingTransform for TransformPartialAggregate { }, HashTable::AggregateHashTable(hashtable) => { let partition_count = hashtable.payload.partition_count(); - let mut blocks = std::mem::take(&mut self.spill_blocks); - blocks.reserve(partition_count); log::info!( "Aggregated {} to {} rows in {} sec(real: {}). ({} rows/sec, {}/sec, {})", @@ -283,7 +281,7 @@ impl AccumulatingTransform for TransformPartialAggregate { ); for (partition, payload) in hashtable.payload.payloads.into_iter().enumerate() { - blocks.push(DataBlock::empty_with_meta( + self.output_blocks.push(DataBlock::empty_with_meta( AggregateMeta::create_agg_payload( payload, partition as isize, @@ -293,7 +291,7 @@ impl AccumulatingTransform for TransformPartialAggregate { )); } - blocks + std::mem::take(&mut self.output_blocks) } }) } @@ -362,7 +360,7 @@ impl AccumulatingTransform for TransformPartialAggregate { global_max_partition: max_partition, }; - self.spill_blocks.push(DataBlock::empty_with_meta( + self.output_blocks.push(DataBlock::empty_with_meta( AggregateMeta::create_spilled_payload(spilled_payload), )); 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 index 0cd909d4fb5e0..3b3c5db6e37c7 100644 --- 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 @@ -14,12 +14,9 @@ use std::any::Any; use std::cmp::Ordering; -use std::collections::hash_map::Entry; use std::collections::BTreeMap; -use std::collections::HashMap; use std::collections::VecDeque; use std::fmt::Debug; -use std::fmt::Formatter; use std::sync::atomic::AtomicUsize; use std::sync::atomic::Ordering as AtomicOrdering; use std::sync::Arc; @@ -57,16 +54,18 @@ use crate::pipelines::processors::transforms::aggregator::AggregatorParams; static SINGLE_LEVEL_BUCKET_NUM: isize = -1; pub struct TransformPartitionDispatch { + params: Arc, outputs: Vec>, input: Arc, outputs_data: Vec>, output_index: usize, - initialized_input: bool, + // initialized_input: bool, finished: bool, + input_data: Option<(AggregateMeta, DataBlock)>, max_partition: usize, working_partition: isize, - partitions: Partitions, + partitions: AlignedPartitions, } impl TransformPartitionDispatch { @@ -80,15 +79,16 @@ impl TransformPartitionDispatch { } Ok(TransformPartitionDispatch { + params, outputs, outputs_data, input: InputPort::create(), output_index: 0, max_partition: 0, - initialized_input: false, - partitions: Partitions::create_unaligned(params), working_partition: 0, finished: false, + input_data: None, + partitions: AlignedPartitions::create(), }) } @@ -100,25 +100,6 @@ impl TransformPartitionDispatch { self.outputs.clone() } - // Align each input's max_partition to the maximum max_partition. - // If an input's max_partition is smaller than the maximum, continuously fetch its data until either the stream ends or its max_partition reaches/exceeds the maximum value. - fn initialize_input(&mut self) -> Result { - if self.input.is_finished() { - return Ok(true); - } - - if !self.input.has_data() { - self.input.set_need_data(); - return Ok(false); - } - - let data_block = self.input.pull_data().unwrap()?; - let (partition, max_partition, global_max_partition) = - self.partitions.add_block(data_block)?; - self.max_partition = global_max_partition; - Ok(partition > SINGLE_LEVEL_BUCKET_NUM && max_partition == global_max_partition) - } - fn ready_partition(&mut self) -> Option { let storage_min_partition = self.partitions.min_partition()?; @@ -149,6 +130,22 @@ impl TransformPartitionDispatch { Ok(()) } + + fn unpark_block(&self, mut data_block: DataBlock) -> Result<(AggregateMeta, DataBlock)> { + let Some(meta) = data_block.take_meta() else { + return Err(ErrorCode::Internal( + "Internal, TransformPartitionBucket only recv DataBlock with meta.", + )); + }; + + let Some(meta) = AggregateMeta::downcast_from(meta) else { + return Err(ErrorCode::Internal( + "Internal, TransformPartitionBucket only recv AggregateMeta".to_string(), + )); + }; + + Ok((meta, data_block)) + } } #[async_trait::async_trait] @@ -204,21 +201,20 @@ impl Processor for TransformPartitionDispatch { return Ok(Event::NeedConsume); } - // We pull the first unaligned partition - if !self.initialized_input { - if self.initialize_input()? { + if self.input.has_data() { + let data_block = self.input.pull_data().unwrap()?; + let (meta, data_block) = self.unpark_block(data_block)?; + + // need repartition + if meta.get_max_partition() != meta.get_global_max_partition() { + self.input_data = Some((meta, data_block)); return Ok(Event::Sync); } - self.input.set_need_data(); - return Ok(Event::NeedData); - } - - if self.input.has_data() { - let data_block = self.input.pull_data().unwrap()?; - let (partition, _, _) = self.partitions.add_block(data_block)?; + let partition = meta.get_sorting_partition(); + self.partitions.add_data(meta, data_block); - if partition != self.working_partition { + if partition != SINGLE_LEVEL_BUCKET_NUM && partition != self.working_partition { self.fetch_ready_partition()?; self.working_partition = partition; continue; @@ -239,9 +235,45 @@ impl Processor for TransformPartitionDispatch { } fn process(&mut self) -> Result<()> { - if !self.initialized_input { - self.initialized_input = true; - return self.partitions.align(self.max_partition); + let Some((meta, data_block)) = self.input_data.take() else { + return Ok(()); + }; + + match meta { + AggregateMeta::FinalPartition => unreachable!(), + AggregateMeta::SpilledPayload(_payload) => unreachable!(), + AggregateMeta::InFlightPayload(payload) => { + if data_block.is_empty() { + return Ok(()); + } + + let payload = AggregatePayload { + partition: payload.partition, + max_partition: payload.max_partition, + payload: self.deserialize_flight(data_block)?, + global_max_partition: payload.global_max_partition, + }; + + let repartition = payload.global_max_partition; + let partitioned = self.partition_payload(payload, repartition); + + for payload in partitioned { + self.partitions + .add_data(AggregateMeta::AggregatePayload(payload), DataBlock::empty()); + } + } + AggregateMeta::AggregatePayload(payload) => { + if payload.payload.len() == 0 { + return Ok(()); + } + + let repartition = payload.global_max_partition; + let partitioned = self.partition_payload(payload, repartition); + for payload in partitioned { + self.partitions + .add_data(AggregateMeta::AggregatePayload(payload), DataBlock::empty()); + } + } } Ok(()) @@ -363,74 +395,8 @@ pub fn build_partition_dispatch( Ok(()) } -struct UnalignedPartitions { - params: Arc, - data: HashMap>, -} - -impl Debug for UnalignedPartitions { - fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { - f.debug_struct("UnalignedPartitions") - .field("data", &self.data) - .finish() - } -} - -impl UnalignedPartitions { - pub fn create(params: Arc) -> UnalignedPartitions { - UnalignedPartitions { - params, - data: HashMap::new(), - } - } - - pub fn is_empty(&self) -> bool { - self.data.is_empty() - } - - fn insert_data(&mut self, idx: usize, meta: AggregateMeta, block: DataBlock) { - match self.data.entry(idx) { - Entry::Vacant(v) => { - v.insert(vec![(meta, block)]); - } - Entry::Occupied(mut v) => { - v.get_mut().push((meta, block)); - } - } - } - - pub fn add_data(&mut self, meta: AggregateMeta, block: DataBlock) -> (isize, usize, usize) { - match &meta { - AggregateMeta::FinalPartition => unreachable!(), - AggregateMeta::SpilledPayload(payload) => { - let max_partition = payload.max_partition; - let global_max_partition = payload.global_max_partition; - self.insert_data(max_partition, meta, block); - - (SINGLE_LEVEL_BUCKET_NUM, max_partition, global_max_partition) - } - AggregateMeta::InFlightPayload(payload) => { - let partition = payload.partition; - let max_partition = payload.max_partition; - let global_max_partition = payload.global_max_partition; - - if !block.is_empty() { - self.insert_data(max_partition, meta, block); - } - - (partition, max_partition, global_max_partition) - } - AggregateMeta::AggregatePayload(payload) => { - let partition = payload.partition; - let max_partition = payload.max_partition; - let global_max_partition = payload.global_max_partition; - - self.insert_data(max_partition, meta, block); - (partition, max_partition, global_max_partition) - } - } - } - +// repartition implementation +impl TransformPartitionDispatch { fn deserialize_flight(&mut self, data: DataBlock) -> Result { let rows_num = data.num_rows(); let group_len = self.params.group_data_types.len(); @@ -492,78 +458,214 @@ impl UnalignedPartitions { partitioned } - - pub fn align(mut self, max_partitions: usize) -> Result { - let repartition_data = self - .data - .extract_if(|k, _| *k != max_partitions) - .collect::>(); - - let mut aligned_partitions = AlignedPartitions { - max_partition: max_partitions, - data: BTreeMap::new(), - }; - - for (_max_partition, data) in std::mem::take(&mut self.data) { - for (meta, block) in data { - aligned_partitions.add_data(meta, block); - } - } - - for (_, repartition_data) in repartition_data { - for (meta, block) in repartition_data { - match meta { - AggregateMeta::FinalPartition => unreachable!(), - AggregateMeta::SpilledPayload(_) => unreachable!(), - AggregateMeta::InFlightPayload(payload) => { - if block.is_empty() { - continue; - } - - let payload = AggregatePayload { - partition: payload.partition, - max_partition: payload.max_partition, - payload: self.deserialize_flight(block)?, - global_max_partition: 0, - }; - - let partitioned = self.partition_payload(payload, max_partitions); - - for payload in partitioned { - aligned_partitions.add_data( - AggregateMeta::AggregatePayload(payload), - DataBlock::empty(), - ); - } - } - AggregateMeta::AggregatePayload(payload) => { - if payload.payload.len() == 0 { - continue; - } - - let partitioned = self.partition_payload(payload, max_partitions); - for payload in partitioned { - aligned_partitions.add_data( - AggregateMeta::AggregatePayload(payload), - DataBlock::empty(), - ); - } - } - } - } - } - - Ok(aligned_partitions) - } } +// struct UnalignedPartitions { +// params: Arc, +// data: HashMap>, +// } +// +// impl Debug for UnalignedPartitions { +// fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { +// f.debug_struct("UnalignedPartitions") +// .field("data", &self.data) +// .finish() +// } +// } +// +// impl UnalignedPartitions { +// pub fn create(params: Arc) -> UnalignedPartitions { +// UnalignedPartitions { +// params, +// data: HashMap::new(), +// } +// } +// +// pub fn is_empty(&self) -> bool { +// self.data.is_empty() +// } +// +// fn insert_data(&mut self, idx: usize, meta: AggregateMeta, block: DataBlock) { +// match self.data.entry(idx) { +// Entry::Vacant(v) => { +// v.insert(vec![(meta, block)]); +// } +// Entry::Occupied(mut v) => { +// v.get_mut().push((meta, block)); +// } +// } +// } +// +// pub fn add_data(&mut self, meta: AggregateMeta, block: DataBlock) -> (isize, usize, usize) { +// match &meta { +// AggregateMeta::FinalPartition => unreachable!(), +// AggregateMeta::SpilledPayload(payload) => { +// let max_partition = payload.max_partition; +// let global_max_partition = payload.global_max_partition; +// self.insert_data(max_partition, meta, block); +// +// (SINGLE_LEVEL_BUCKET_NUM, max_partition, global_max_partition) +// } +// AggregateMeta::InFlightPayload(payload) => { +// let partition = payload.partition; +// let max_partition = payload.max_partition; +// let global_max_partition = payload.global_max_partition; +// +// if !block.is_empty() { +// self.insert_data(max_partition, meta, block); +// } +// +// (partition, max_partition, global_max_partition) +// } +// AggregateMeta::AggregatePayload(payload) => { +// let partition = payload.partition; +// let max_partition = payload.max_partition; +// let global_max_partition = payload.global_max_partition; +// +// self.insert_data(max_partition, meta, block); +// (partition, max_partition, global_max_partition) +// } +// } +// } +// +// fn deserialize_flight(&mut self, data: DataBlock) -> Result { +// let rows_num = data.num_rows(); +// let group_len = self.params.group_data_types.len(); +// +// let mut state = ProbeState::default(); +// +// // create single partition hash table for deserialize +// let capacity = AggregateHashTable::get_capacity_for_count(rows_num); +// let config = HashTableConfig::default().with_initial_radix_bits(0); +// let mut hashtable = AggregateHashTable::new_directly( +// self.params.group_data_types.clone(), +// self.params.aggregate_functions.clone(), +// config, +// capacity, +// Arc::new(Bump::new()), +// false, +// ); +// +// let num_states = self.params.num_states(); +// let states_index: Vec = (0..num_states).collect(); +// let agg_states = InputColumns::new_block_proxy(&states_index, &data); +// +// let group_index: Vec = (num_states..(num_states + group_len)).collect(); +// let group_columns = InputColumns::new_block_proxy(&group_index, &data); +// +// let _ = hashtable.add_groups( +// &mut state, +// group_columns, +// &[(&[]).into()], +// agg_states, +// rows_num, +// )?; +// +// hashtable.payload.mark_min_cardinality(); +// assert_eq!(hashtable.payload.payloads.len(), 1); +// Ok(hashtable.payload.payloads.pop().unwrap()) +// } +// +// fn partition_payload(&mut self, from: AggregatePayload, to: usize) -> Vec { +// let mut partitioned = Vec::with_capacity(to); +// let mut partitioned_payload = PartitionedPayload::new( +// self.params.group_data_types.clone(), +// self.params.aggregate_functions.clone(), +// to as u64, +// vec![from.payload.arena.clone()], +// ); +// +// let mut flush_state = PayloadFlushState::default(); +// partitioned_payload.combine_single(from.payload, &mut flush_state, None); +// +// for (partition, payload) in partitioned_payload.payloads.into_iter().enumerate() { +// partitioned.push(AggregatePayload { +// payload, +// partition: partition as isize, +// max_partition: to, +// global_max_partition: 0, +// }); +// } +// +// partitioned +// } +// +// pub fn align(mut self, max_partitions: usize) -> Result { +// let repartition_data = self +// .data +// .extract_if(|k, _| *k != max_partitions) +// .collect::>(); +// +// let mut aligned_partitions = AlignedPartitions { +// max_partition: max_partitions, +// data: BTreeMap::new(), +// }; +// +// for (_max_partition, data) in std::mem::take(&mut self.data) { +// for (meta, block) in data { +// aligned_partitions.add_data(meta, block); +// } +// } +// +// for (_, repartition_data) in repartition_data { +// for (meta, block) in repartition_data { +// match meta { +// AggregateMeta::FinalPartition => unreachable!(), +// AggregateMeta::SpilledPayload(_) => unreachable!(), +// AggregateMeta::InFlightPayload(payload) => { +// if block.is_empty() { +// continue; +// } +// +// let payload = AggregatePayload { +// partition: payload.partition, +// max_partition: payload.max_partition, +// payload: self.deserialize_flight(block)?, +// global_max_partition: 0, +// }; +// +// let partitioned = self.partition_payload(payload, max_partitions); +// +// for payload in partitioned { +// aligned_partitions.add_data( +// AggregateMeta::AggregatePayload(payload), +// DataBlock::empty(), +// ); +// } +// } +// AggregateMeta::AggregatePayload(payload) => { +// if payload.payload.len() == 0 { +// continue; +// } +// +// let partitioned = self.partition_payload(payload, max_partitions); +// for payload in partitioned { +// aligned_partitions.add_data( +// AggregateMeta::AggregatePayload(payload), +// DataBlock::empty(), +// ); +// } +// } +// } +// } +// } +// +// Ok(aligned_partitions) +// } +// } + #[derive(Debug)] struct AlignedPartitions { - max_partition: usize, data: BTreeMap>, } impl AlignedPartitions { + pub fn create() -> AlignedPartitions { + AlignedPartitions { + data: BTreeMap::new(), + } + } + pub fn is_empty(&self) -> bool { self.data.is_empty() } @@ -582,7 +684,6 @@ impl AlignedPartitions { } }; - assert_eq!(max_partition, self.max_partition); match self.data.entry(partition) { std::collections::btree_map::Entry::Vacant(v) => { v.insert(vec![(meta, block)]); @@ -594,82 +695,68 @@ impl AlignedPartitions { (partition, max_partition, global_max_partition) } -} - -#[derive(Debug)] -enum Partitions { - Aligned(AlignedPartitions), - Unaligned(UnalignedPartitions), -} - -impl Partitions { - pub fn create_unaligned(params: Arc) -> Partitions { - Partitions::Unaligned(UnalignedPartitions::create(params)) - } - - pub fn is_empty(&self) -> bool { - match self { - Partitions::Aligned(v) => v.is_empty(), - Partitions::Unaligned(v) => v.is_empty(), - } - } - - fn add_data(&mut self, meta: AggregateMeta, block: DataBlock) -> (isize, usize, usize) { - match self { - Partitions::Aligned(v) => v.add_data(meta, block), - Partitions::Unaligned(v) => v.add_data(meta, block), - } - } - - pub fn add_block(&mut self, mut block: DataBlock) -> Result<(isize, usize, usize)> { - let Some(meta) = block.take_meta() else { - return Err(ErrorCode::Internal( - "Internal, TransformPartitionBucket only recv DataBlock with meta.", - )); - }; - - let Some(meta) = AggregateMeta::downcast_from(meta) else { - return Err(ErrorCode::Internal( - "Internal, TransformPartitionBucket only recv AggregateMeta".to_string(), - )); - }; - - Ok(self.add_data(meta, block)) - } pub fn min_partition(&self) -> Option { - match self { - Partitions::Unaligned(_) => unreachable!(), - Partitions::Aligned(v) => v.data.keys().min().cloned(), - } + self.data.keys().min().cloned() } pub fn take_partition(&mut self, partition: isize) -> Vec<(AggregateMeta, DataBlock)> { - match self { - Partitions::Unaligned(_) => unreachable!(), - Partitions::Aligned(v) => v.data.remove(&partition).unwrap_or_default(), - } - } - - pub fn align(&mut self, max_partitions: usize) -> Result<()> { - let mut partitions = match self { - Partitions::Aligned(_) => { - return Ok(()); - } - Partitions::Unaligned(v) => Self::create_unaligned(v.params.clone()), - }; - - std::mem::swap(self, &mut partitions); - - *self = match partitions { - Partitions::Aligned(_) => unreachable!(), - Partitions::Unaligned(v) => Partitions::Aligned(v.align(max_partitions)?), - }; - - Ok(()) + self.data.remove(&partition).unwrap_or_default() } } +// #[derive(Debug)] +// enum Partitions { +// Aligned(AlignedPartitions), +// Unaligned(UnalignedPartitions), +// } +// +// impl Partitions { +// pub fn create_unaligned(params: Arc) -> Partitions { +// Partitions::Unaligned(UnalignedPartitions::create(params)) +// } +// +// pub fn is_empty(&self) -> bool { +// match self { +// Partitions::Aligned(v) => v.is_empty(), +// Partitions::Unaligned(v) => v.is_empty(), +// } +// } +// +// fn add_data(&mut self, meta: AggregateMeta, block: DataBlock) -> (isize, usize, usize) { +// match self { +// Partitions::Aligned(v) => v.add_data(meta, block), +// Partitions::Unaligned(v) => v.add_data(meta, block), +// } +// } +// +// pub fn min_partition(&self) -> Option { +// match self { +// Partitions::Unaligned(_) => unreachable!(), +// Partitions::Aligned(v) => v.data.keys().min().cloned(), +// } +// } +// +// +// pub fn align(&mut self, max_partitions: usize) -> Result<()> { +// let mut partitions = match self { +// Partitions::Aligned(_) => { +// return Ok(()); +// } +// Partitions::Unaligned(v) => Self::create_unaligned(v.params.clone()), +// }; +// +// std::mem::swap(self, &mut partitions); +// +// *self = match partitions { +// Partitions::Aligned(_) => unreachable!(), +// Partitions::Unaligned(v) => Partitions::Aligned(v.align(max_partitions)?), +// }; +// +// Ok(()) +// } +// } + #[cfg(test)] mod tests { // use databend_common_expression::types::DataType; From b37b273aee057a3b3e0f4992b1417aa0caab0738 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 2 Apr 2025 13:39:58 +0800 Subject: [PATCH 34/76] refactor(query): refactor aggreagte spill code --- .../aggregator/transform_aggregate_partial.rs | 1 - .../aggregator/transform_partition_bucket.rs | 218 +----------------- 2 files changed, 4 insertions(+), 215 deletions(-) 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 434a86d21f9a5..691903e2554f9 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 @@ -248,7 +248,6 @@ impl AccumulatingTransform for TransformPartialAggregate { fn transform(&mut self, block: DataBlock) -> Result> { self.execute_one_block(block)?; - Ok(vec![]) } 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 index 3b3c5db6e37c7..6fd05aa904f92 100644 --- 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 @@ -59,7 +59,6 @@ pub struct TransformPartitionDispatch { input: Arc, outputs_data: Vec>, output_index: usize, - // initialized_input: bool, finished: bool, input_data: Option<(AggregateMeta, DataBlock)>, @@ -214,7 +213,7 @@ impl Processor for TransformPartitionDispatch { let partition = meta.get_sorting_partition(); self.partitions.add_data(meta, data_block); - if partition != SINGLE_LEVEL_BUCKET_NUM && partition != self.working_partition { + if partition > SINGLE_LEVEL_BUCKET_NUM && partition != self.working_partition { self.fetch_ready_partition()?; self.working_partition = partition; continue; @@ -460,200 +459,6 @@ impl TransformPartitionDispatch { } } -// struct UnalignedPartitions { -// params: Arc, -// data: HashMap>, -// } -// -// impl Debug for UnalignedPartitions { -// fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { -// f.debug_struct("UnalignedPartitions") -// .field("data", &self.data) -// .finish() -// } -// } -// -// impl UnalignedPartitions { -// pub fn create(params: Arc) -> UnalignedPartitions { -// UnalignedPartitions { -// params, -// data: HashMap::new(), -// } -// } -// -// pub fn is_empty(&self) -> bool { -// self.data.is_empty() -// } -// -// fn insert_data(&mut self, idx: usize, meta: AggregateMeta, block: DataBlock) { -// match self.data.entry(idx) { -// Entry::Vacant(v) => { -// v.insert(vec![(meta, block)]); -// } -// Entry::Occupied(mut v) => { -// v.get_mut().push((meta, block)); -// } -// } -// } -// -// pub fn add_data(&mut self, meta: AggregateMeta, block: DataBlock) -> (isize, usize, usize) { -// match &meta { -// AggregateMeta::FinalPartition => unreachable!(), -// AggregateMeta::SpilledPayload(payload) => { -// let max_partition = payload.max_partition; -// let global_max_partition = payload.global_max_partition; -// self.insert_data(max_partition, meta, block); -// -// (SINGLE_LEVEL_BUCKET_NUM, max_partition, global_max_partition) -// } -// AggregateMeta::InFlightPayload(payload) => { -// let partition = payload.partition; -// let max_partition = payload.max_partition; -// let global_max_partition = payload.global_max_partition; -// -// if !block.is_empty() { -// self.insert_data(max_partition, meta, block); -// } -// -// (partition, max_partition, global_max_partition) -// } -// AggregateMeta::AggregatePayload(payload) => { -// let partition = payload.partition; -// let max_partition = payload.max_partition; -// let global_max_partition = payload.global_max_partition; -// -// self.insert_data(max_partition, meta, block); -// (partition, max_partition, global_max_partition) -// } -// } -// } -// -// fn deserialize_flight(&mut self, data: DataBlock) -> Result { -// let rows_num = data.num_rows(); -// let group_len = self.params.group_data_types.len(); -// -// let mut state = ProbeState::default(); -// -// // create single partition hash table for deserialize -// let capacity = AggregateHashTable::get_capacity_for_count(rows_num); -// let config = HashTableConfig::default().with_initial_radix_bits(0); -// let mut hashtable = AggregateHashTable::new_directly( -// self.params.group_data_types.clone(), -// self.params.aggregate_functions.clone(), -// config, -// capacity, -// Arc::new(Bump::new()), -// false, -// ); -// -// let num_states = self.params.num_states(); -// let states_index: Vec = (0..num_states).collect(); -// let agg_states = InputColumns::new_block_proxy(&states_index, &data); -// -// let group_index: Vec = (num_states..(num_states + group_len)).collect(); -// let group_columns = InputColumns::new_block_proxy(&group_index, &data); -// -// let _ = hashtable.add_groups( -// &mut state, -// group_columns, -// &[(&[]).into()], -// agg_states, -// rows_num, -// )?; -// -// hashtable.payload.mark_min_cardinality(); -// assert_eq!(hashtable.payload.payloads.len(), 1); -// Ok(hashtable.payload.payloads.pop().unwrap()) -// } -// -// fn partition_payload(&mut self, from: AggregatePayload, to: usize) -> Vec { -// let mut partitioned = Vec::with_capacity(to); -// let mut partitioned_payload = PartitionedPayload::new( -// self.params.group_data_types.clone(), -// self.params.aggregate_functions.clone(), -// to as u64, -// vec![from.payload.arena.clone()], -// ); -// -// let mut flush_state = PayloadFlushState::default(); -// partitioned_payload.combine_single(from.payload, &mut flush_state, None); -// -// for (partition, payload) in partitioned_payload.payloads.into_iter().enumerate() { -// partitioned.push(AggregatePayload { -// payload, -// partition: partition as isize, -// max_partition: to, -// global_max_partition: 0, -// }); -// } -// -// partitioned -// } -// -// pub fn align(mut self, max_partitions: usize) -> Result { -// let repartition_data = self -// .data -// .extract_if(|k, _| *k != max_partitions) -// .collect::>(); -// -// let mut aligned_partitions = AlignedPartitions { -// max_partition: max_partitions, -// data: BTreeMap::new(), -// }; -// -// for (_max_partition, data) in std::mem::take(&mut self.data) { -// for (meta, block) in data { -// aligned_partitions.add_data(meta, block); -// } -// } -// -// for (_, repartition_data) in repartition_data { -// for (meta, block) in repartition_data { -// match meta { -// AggregateMeta::FinalPartition => unreachable!(), -// AggregateMeta::SpilledPayload(_) => unreachable!(), -// AggregateMeta::InFlightPayload(payload) => { -// if block.is_empty() { -// continue; -// } -// -// let payload = AggregatePayload { -// partition: payload.partition, -// max_partition: payload.max_partition, -// payload: self.deserialize_flight(block)?, -// global_max_partition: 0, -// }; -// -// let partitioned = self.partition_payload(payload, max_partitions); -// -// for payload in partitioned { -// aligned_partitions.add_data( -// AggregateMeta::AggregatePayload(payload), -// DataBlock::empty(), -// ); -// } -// } -// AggregateMeta::AggregatePayload(payload) => { -// if payload.payload.len() == 0 { -// continue; -// } -// -// let partitioned = self.partition_payload(payload, max_partitions); -// for payload in partitioned { -// aligned_partitions.add_data( -// AggregateMeta::AggregatePayload(payload), -// DataBlock::empty(), -// ); -// } -// } -// } -// } -// } -// -// Ok(aligned_partitions) -// } -// } - #[derive(Debug)] struct AlignedPartitions { data: BTreeMap>, @@ -670,30 +475,15 @@ impl AlignedPartitions { self.data.is_empty() } - pub fn add_data(&mut self, meta: AggregateMeta, block: DataBlock) -> (isize, usize, usize) { - let (partition, max_partition, global_max_partition) = match &meta { - AggregateMeta::FinalPartition => unreachable!(), - AggregateMeta::SpilledPayload(v) => { - (v.partition, v.max_partition, v.global_max_partition) - } - AggregateMeta::AggregatePayload(v) => { - (v.partition, v.max_partition, v.global_max_partition) - } - AggregateMeta::InFlightPayload(v) => { - (v.partition, v.max_partition, v.global_max_partition) - } - }; - - match self.data.entry(partition) { + pub fn add_data(&mut self, meta: AggregateMeta, block: DataBlock) { + match self.data.entry(meta.get_partition()) { std::collections::btree_map::Entry::Vacant(v) => { v.insert(vec![(meta, block)]); } std::collections::btree_map::Entry::Occupied(mut v) => { v.get_mut().push((meta, block)); } - } - - (partition, max_partition, global_max_partition) + }; } pub fn min_partition(&self) -> Option { From 5ae50c7e38d298b88c4c7f3a64cbba85fc6532a5 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 2 Apr 2025 15:27:35 +0800 Subject: [PATCH 35/76] refactor(query): refactor aggreagte spill code --- .../transforms/transform_accumulating.rs | 21 +- .../aggregator/transform_aggregate_partial.rs | 20 +- .../aggregator/transform_partition_bucket.rs | 185 +----------------- 3 files changed, 35 insertions(+), 191 deletions(-) diff --git a/src/query/pipeline/transforms/src/processors/transforms/transform_accumulating.rs b/src/query/pipeline/transforms/src/processors/transforms/transform_accumulating.rs index 976b5a5b905c2..dc46a6c6d759d 100644 --- a/src/query/pipeline/transforms/src/processors/transforms/transform_accumulating.rs +++ b/src/query/pipeline/transforms/src/processors/transforms/transform_accumulating.rs @@ -70,6 +70,7 @@ pub struct AccumulatingTransformer { input_data: Option, output_data: VecDeque, + has_spill: bool, flush_spill_payload: bool, prepare_spill_payload: bool, } @@ -83,6 +84,7 @@ impl AccumulatingTransformer { input_data: None, output_data: VecDeque::with_capacity(1), called_on_finish: false, + has_spill: false, flush_spill_payload: false, prepare_spill_payload: false, }) @@ -149,7 +151,15 @@ impl Processor for AccumulatingTransformer Ok(Event::Sync), + true => { + // To avoid downstream out-of-memory, once a spill occurs, all data must be spilled entirely. + if self.has_spill { + self.has_spill = false; + self.prepare_spill_payload = true; + } + + Ok(Event::Sync) + } false => { self.output.finish(); Ok(Event::Finished) @@ -165,10 +175,6 @@ impl Processor for AccumulatingTransformer Result<()> { if self.prepare_spill_payload { self.prepare_spill_payload = false; @@ -179,6 +185,7 @@ impl Processor for AccumulatingTransformer Processor for AccumulatingTransformer Result { self.inner.flush_spill_payload().await } + + fn configure_peer_nodes(&mut self, nodes: &[String]) { + self.inner.configure_peer_nodes(nodes) + } } pub trait BlockMetaAccumulatingTransform: Send + 'static { 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 691903e2554f9..553a1af91dc40 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 @@ -279,15 +279,17 @@ impl AccumulatingTransform for TransformPartialAggregate { convert_byte_size(self.processed_bytes as f64), ); - for (partition, payload) in hashtable.payload.payloads.into_iter().enumerate() { - self.output_blocks.push(DataBlock::empty_with_meta( - AggregateMeta::create_agg_payload( - payload, - partition as isize, - partition_count, - partition_count, - ), - )); + if hashtable.len() != 0 { + for (partition, payload) in hashtable.payload.payloads.into_iter().enumerate() { + self.output_blocks.push(DataBlock::empty_with_meta( + AggregateMeta::create_agg_payload( + payload, + partition as isize, + partition_count, + partition_count, + ), + )); + } } std::mem::take(&mut self.output_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 index 6fd05aa904f92..0183228c3c774 100644 --- 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 @@ -64,7 +64,7 @@ pub struct TransformPartitionDispatch { max_partition: usize, working_partition: isize, - partitions: AlignedPartitions, + partitions: Partitions, } impl TransformPartitionDispatch { @@ -87,7 +87,7 @@ impl TransformPartitionDispatch { working_partition: 0, finished: false, input_data: None, - partitions: AlignedPartitions::create(), + partitions: Partitions::create(), }) } @@ -177,11 +177,7 @@ impl Processor for TransformPartitionDispatch { if output.can_push() { if let Some(block) = self.outputs_data[idx].pop_front() { - if !block.is_empty() || block.get_meta().is_some() { - output.push_data(Ok(block)); - } - - continue; + output.push_data(Ok(block)); } } @@ -203,6 +199,7 @@ impl Processor for TransformPartitionDispatch { if self.input.has_data() { let data_block = self.input.pull_data().unwrap()?; let (meta, data_block) = self.unpark_block(data_block)?; + self.max_partition = meta.get_global_max_partition(); // need repartition if meta.get_max_partition() != meta.get_global_max_partition() { @@ -460,13 +457,13 @@ impl TransformPartitionDispatch { } #[derive(Debug)] -struct AlignedPartitions { +struct Partitions { data: BTreeMap>, } -impl AlignedPartitions { - pub fn create() -> AlignedPartitions { - AlignedPartitions { +impl Partitions { + pub fn create() -> Partitions { + Partitions { data: BTreeMap::new(), } } @@ -494,169 +491,3 @@ impl AlignedPartitions { self.data.remove(&partition).unwrap_or_default() } } - -// #[derive(Debug)] -// enum Partitions { -// Aligned(AlignedPartitions), -// Unaligned(UnalignedPartitions), -// } -// -// impl Partitions { -// pub fn create_unaligned(params: Arc) -> Partitions { -// Partitions::Unaligned(UnalignedPartitions::create(params)) -// } -// -// pub fn is_empty(&self) -> bool { -// match self { -// Partitions::Aligned(v) => v.is_empty(), -// Partitions::Unaligned(v) => v.is_empty(), -// } -// } -// -// fn add_data(&mut self, meta: AggregateMeta, block: DataBlock) -> (isize, usize, usize) { -// match self { -// Partitions::Aligned(v) => v.add_data(meta, block), -// Partitions::Unaligned(v) => v.add_data(meta, block), -// } -// } -// -// pub fn min_partition(&self) -> Option { -// match self { -// Partitions::Unaligned(_) => unreachable!(), -// Partitions::Aligned(v) => v.data.keys().min().cloned(), -// } -// } -// -// -// pub fn align(&mut self, max_partitions: usize) -> Result<()> { -// let mut partitions = match self { -// Partitions::Aligned(_) => { -// return Ok(()); -// } -// Partitions::Unaligned(v) => Self::create_unaligned(v.params.clone()), -// }; -// -// std::mem::swap(self, &mut partitions); -// -// *self = match partitions { -// Partitions::Aligned(_) => unreachable!(), -// Partitions::Unaligned(v) => Partitions::Aligned(v.align(max_partitions)?), -// }; -// -// Ok(()) -// } -// } - -#[cfg(test)] -mod tests { - // use databend_common_expression::types::DataType; - // use databend_common_expression::types::NumberDataType; - // use databend_common_expression::DataField; - // use databend_common_expression::DataSchemaRefExt; - // use databend_common_functions::aggregates::AggregateFunctionFactory; - // - // use crate::pipelines::processors::transforms::aggregator::transform_partition_bucket::UnalignedPartitions; - // use crate::pipelines::processors::transforms::aggregator::AggregatorParams; - - // fn create_unaligned_partitions() -> UnalignedPartitions { - // let schema = DataSchemaRefExt::create(vec![ - // DataField::new("a", DataType::Number(NumberDataType::Int16)), - // DataField::new("b", DataType::Number(NumberDataType::Float32)), - // DataField::new("c", DataType::String), - // ]); - // - // let aggregate_functions = vec![AggregateFunctionFactory::instance() - // .get("count", vec![], vec![], vec![]) - // .unwrap()]; - // - // let params = AggregatorParams::try_create( - // schema, - // vec![ - // DataType::Number(NumberDataType::Int16), - // DataType::Number(NumberDataType::Float32), - // DataType::String, - // ], - // &[0, 1, 2], - // &aggregate_functions, - // &[], - // true, - // false, - // 1024, - // 1024, - // ); - // - // UnalignedPartitions::create(params.unwrap()) - // } - - // #[test] - // fn test_add_data_spilled_payload() { - // let mut partitions = create_unaligned_partitions(); - // let max_partition = 5; - // let meta = AggregateMeta::SpilledPayload(SpilledPayload { - // partition: 0, - // location: "".to_string(), - // data_range: Default::default(), - // destination_node: "".to_string(), - // max_partition: max_partition, - // }); - // - // let result = partitions.add_data(meta, DataBlock::empty(), -1); - // - // assert_eq!(result, (SINGLE_LEVEL_BUCKET_NUM, max_partition)); - // assert_eq!(partitions.data.get(&max_partition).unwrap().len(), 1); - // } - // - // #[test] - // fn test_add_data_in_flight_payload() { - // let mut partitions = create_unaligned_partitions(); - // let partition = 2; - // let max_partition = 8; - // let meta = AggregateMeta::InFlightPayload(InFlightPayload { - // partition, - // max_partition, - // }); - // - // let result = partitions.add_data(meta, DataBlock::empty(), -1); - // - // assert_eq!(result, (partition, max_partition)); - // assert_eq!(partitions.data.get(&max_partition).unwrap().len(), 1); - // } - - #[test] - fn test_add_data_aggregate_payload() { - // let mut partitions = create_unaligned_partitions(); - // let partition = 3; - // let max_partition = 10; - // // Payload::new() - // let meta = AggregateMeta::AggregatePayload(AggregatePayload { - // partition, - // // payload: Payload {}, - // max_partition_count: max_partition, - // }); - // - // let result = partitions.add_data(meta, DataBlock::empty()); - // - // assert_eq!(result, (partition, max_partition)); - // assert_eq!(partitions.data.get(&max_partition).unwrap().len(), 1); - } - - // #[test] - // fn test_multiple_inserts_same_partition() { - // let mut container = YourContainerStruct::new(); - // let max_partition = 5; - // - // let meta1 = AggregateMeta::SpilledPayload(SpilledPayload { - // max_partition_count: max_partition, - // // ... - // }); - // container.add_data(meta1, DataBlock); - // - // let meta2 = AggregateMeta::SpilledPayload(SpilledPayload { - // max_partition_count: max_partition, - // // ... - // }); - // container.add_data(meta2, DataBlock); - // - // assert_eq!(container.data.get(&max_partition).unwrap().len(), 2); - // } -} From 62bf28bced0441b3f56c9fbcfabb339ea81d6ae4 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 2 Apr 2025 17:43:40 +0800 Subject: [PATCH 36/76] refactor(query): refactor aggreagte spill code --- .../pipeline/core/src/processors/shuffle_processor.rs | 7 +++++-- .../service/src/pipelines/builders/builder_aggregate.rs | 1 - 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/src/query/pipeline/core/src/processors/shuffle_processor.rs b/src/query/pipeline/core/src/processors/shuffle_processor.rs index 6d61e0de180fd..f9997e28736df 100644 --- a/src/query/pipeline/core/src/processors/shuffle_processor.rs +++ b/src/query/pipeline/core/src/processors/shuffle_processor.rs @@ -211,7 +211,7 @@ impl PartitionProcessor { exchange, partitioned_data, input_data: None, - initialized: false, + initialized: !T::MULTIWAY_SORT, index, barrier, })) @@ -357,7 +357,10 @@ impl MergePartitionProcessor { impl Processor for MergePartitionProcessor { fn name(&self) -> String { - format!("ShuffleMergePartition({})", T::NAME) + match T::MULTIWAY_SORT { + true => format!("ShuffleSortMergePartition({})", T::NAME), + false => format!("ShuffleMergePartition({})", T::NAME), + } } fn as_any(&mut self) -> &mut dyn Any { diff --git a/src/query/service/src/pipelines/builders/builder_aggregate.rs b/src/query/service/src/pipelines/builders/builder_aggregate.rs index 40bc0a896c7ab..ca63188da275d 100644 --- a/src/query/service/src/pipelines/builders/builder_aggregate.rs +++ b/src/query/service/src/pipelines/builders/builder_aggregate.rs @@ -166,7 +166,6 @@ impl PipelineBuilder { })?; self.enable_multiway_sort = true; - // self.exchange_injector = AggregateInjector::create(); Ok(()) } From cd8160a4c57df1afbb9ce9091f2e47692c71555b Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 2 Apr 2025 21:10:18 +0800 Subject: [PATCH 37/76] refactor(query): refactor aggreagte spill code --- .../transforms/aggregator/transform_partition_bucket.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 index 0183228c3c774..b75fed1c6b104 100644 --- 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 @@ -448,7 +448,7 @@ impl TransformPartitionDispatch { payload, partition: partition as isize, max_partition: to, - global_max_partition: 0, + global_max_partition: from.global_max_partition, }); } From 0da9b3b4128c726581514d764d7aa9b804443252 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Thu, 3 Apr 2025 19:09:33 +0800 Subject: [PATCH 38/76] refactor(query): refactor aggreagte spill code --- src/query/expression/src/aggregate/payload.rs | 36 +- .../pipelines/builders/builder_aggregate.rs | 4 +- .../aggregator/aggregate_exchange_injector.rs | 2 +- .../transforms/aggregator/aggregate_meta.rs | 22 +- .../processors/transforms/aggregator/mod.rs | 7 +- .../transforms/aggregator/serde/mod.rs | 4 +- .../aggregator/transform_aggregate_final.rs | 17 +- .../aggregator/transform_partition_align.rs | 269 ++++++++++ .../aggregator/transform_partition_bucket.rs | 496 ++---------------- .../transform_partition_dispatch.rs | 247 +++++++++ .../transform_partition_exchange.rs | 183 +++++++ .../transform_partition_resorting.rs | 110 ++++ ...ader.rs => transform_partition_restore.rs} | 19 +- 13 files changed, 924 insertions(+), 492 deletions(-) create mode 100644 src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_align.rs create mode 100644 src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_dispatch.rs create mode 100644 src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_exchange.rs create mode 100644 src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_resorting.rs rename src/query/service/src/pipelines/processors/transforms/aggregator/{serde/transform_spill_reader.rs => transform_partition_restore.rs} (91%) diff --git a/src/query/expression/src/aggregate/payload.rs b/src/query/expression/src/aggregate/payload.rs index 788f187ed9699..8e9d796da9833 100644 --- a/src/query/expression/src/aggregate/payload.rs +++ b/src/query/expression/src/aggregate/payload.rs @@ -385,7 +385,11 @@ impl Payload { ); } - pub fn scatter(&self, state: &mut PayloadFlushState, partition_count: usize) -> bool { + pub fn scatter_with_seed( + &self, + state: &mut PayloadFlushState, + partitions: usize, + ) -> bool { if state.flush_page >= self.pages.len() { return false; } @@ -397,23 +401,27 @@ impl Payload { state.flush_page += 1; state.flush_page_row = 0; state.row_count = 0; - return self.scatter(state, partition_count); + return self.scatter_with_seed::(state, partitions); } let end = (state.flush_page_row + BATCH_SIZE).min(page.rows); let rows = end - state.flush_page_row; state.row_count = rows; - state.probe_state.reset_partitions(partition_count); + state.probe_state.reset_partitions(partitions); + + let mods: StrengthReducedU64 = StrengthReducedU64::new(partitions as u64); - let mods: StrengthReducedU64 = StrengthReducedU64::new(partition_count as u64); for idx in 0..rows { state.addresses[idx] = self.data_ptr(page, idx + state.flush_page_row); - let hash = unsafe { read::(state.addresses[idx].add(self.hash_offset) as _) }; + let mut hash = unsafe { read::(state.addresses[idx].add(self.hash_offset) as _) }; - let partition_idx = (hash % mods) as usize; + if SEED != 0 { + hash = Self::combine_hash(hash, SEED); + } + let partition_idx = (hash % mods) as usize; let sel = &mut state.probe_state.partition_entries[partition_idx]; sel[state.probe_state.partition_count[partition_idx]] = idx; state.probe_state.partition_count[partition_idx] += 1; @@ -422,6 +430,10 @@ impl Payload { true } + pub fn scatter(&self, state: &mut PayloadFlushState, partitions: usize) -> bool { + self.scatter_with_seed::<0>(state, partitions) + } + pub fn empty_block(&self, fake_rows: Option) -> DataBlock { let fake_rows = fake_rows.unwrap_or(0); let columns = (0..self.aggrs.len()) @@ -434,6 +446,18 @@ impl Payload { .collect_vec(); DataBlock::new_from_columns(columns) } + + #[allow(unused_parens)] + fn combine_hash(hash: u64, seed: u64) -> u64 { + static KMUL: u64 = 0x9ddfea08eb382d69; + + let mut a = (seed ^ hash).wrapping_mul(KMUL); + a ^= (a >> 47); + + let mut b = (hash ^ a).wrapping_mul(KMUL); + b ^= (b >> 47); + b.wrapping_mul(KMUL) + } } impl Drop for Payload { diff --git a/src/query/service/src/pipelines/builders/builder_aggregate.rs b/src/query/service/src/pipelines/builders/builder_aggregate.rs index ca63188da275d..0879a258e7a60 100644 --- a/src/query/service/src/pipelines/builders/builder_aggregate.rs +++ b/src/query/service/src/pipelines/builders/builder_aggregate.rs @@ -37,7 +37,7 @@ use databend_common_sql::IndexType; use databend_common_storage::DataOperator; use itertools::Itertools; -use crate::pipelines::processors::transforms::aggregator::build_partition_dispatch; +use crate::pipelines::processors::transforms::aggregator::build_final_aggregate; use crate::pipelines::processors::transforms::aggregator::create_udaf_script_function; use crate::pipelines::processors::transforms::aggregator::AggregatorParams; use crate::pipelines::processors::transforms::aggregator::FinalSingleStateAggregator; @@ -204,7 +204,7 @@ impl PipelineBuilder { self.build_pipeline(&aggregate.input)?; self.enable_multiway_sort = old_value; - build_partition_dispatch(&mut self.main_pipeline, params.clone()) + build_final_aggregate(&mut self.main_pipeline, params.clone()) } fn build_aggregator_params( 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 eea38704d82d7..5c91c2621275a 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 @@ -156,7 +156,7 @@ impl Exchange for FlightExchange { assert!(MULTIWAY_SORT); assert_eq!(self.node_list_lookup.len(), n); match AggregateMeta::downcast_from(meta).unwrap() { - AggregateMeta::FinalPartition => unreachable!(), + AggregateMeta::FinalPartition(_) => unreachable!(), AggregateMeta::InFlightPayload(_) => unreachable!(), AggregateMeta::SpilledPayload(v) => { let mut blocks = Vec::with_capacity(n); 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 0378769138f73..9f588382d1b0a 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 @@ -140,7 +140,7 @@ pub struct InFlightPayload { } pub struct FinalPayload { - pub data: Vec<(AggregateMeta, DataBlock)>, + pub data: Arc>, } #[derive(serde::Serialize, serde::Deserialize)] @@ -148,7 +148,7 @@ pub enum AggregateMeta { SpilledPayload(SpilledPayload), AggregatePayload(AggregatePayload), InFlightPayload(InFlightPayload), - FinalPartition, + FinalPartition(Option>), } impl AggregateMeta { @@ -182,8 +182,8 @@ impl AggregateMeta { Box::new(AggregateMeta::SpilledPayload(payload)) } - pub fn create_final() -> BlockMetaInfoPtr { - Box::new(AggregateMeta::FinalPartition) + pub fn create_final(data: Option>) -> BlockMetaInfoPtr { + Box::new(AggregateMeta::FinalPartition(data)) } pub fn get_global_max_partition(&self) -> usize { @@ -191,7 +191,7 @@ impl AggregateMeta { AggregateMeta::SpilledPayload(v) => v.global_max_partition, AggregateMeta::AggregatePayload(v) => v.global_max_partition, AggregateMeta::InFlightPayload(v) => v.global_max_partition, - AggregateMeta::FinalPartition => unreachable!(), + AggregateMeta::FinalPartition(_) => unreachable!(), } } @@ -200,7 +200,7 @@ impl AggregateMeta { AggregateMeta::SpilledPayload(v) => v.partition, AggregateMeta::AggregatePayload(v) => v.partition, AggregateMeta::InFlightPayload(v) => v.partition, - AggregateMeta::FinalPartition => unreachable!(), + AggregateMeta::FinalPartition(_) => unreachable!(), } } @@ -209,7 +209,7 @@ impl AggregateMeta { AggregateMeta::AggregatePayload(v) => v.partition, AggregateMeta::InFlightPayload(v) => v.partition, AggregateMeta::SpilledPayload(v) => v.get_sorting_partition(), - AggregateMeta::FinalPartition => unreachable!(), + AggregateMeta::FinalPartition(_) => unreachable!(), } } @@ -218,7 +218,7 @@ impl AggregateMeta { AggregateMeta::SpilledPayload(v) => v.max_partition, AggregateMeta::AggregatePayload(v) => v.max_partition, AggregateMeta::InFlightPayload(v) => v.max_partition, - AggregateMeta::FinalPartition => unreachable!(), + AggregateMeta::FinalPartition(_) => unreachable!(), } } @@ -233,7 +233,7 @@ impl AggregateMeta { AggregateMeta::InFlightPayload(v) => { v.global_max_partition = global_max_partition; } - AggregateMeta::FinalPartition => unreachable!(), + AggregateMeta::FinalPartition(_) => unreachable!(), } } } @@ -241,7 +241,9 @@ impl AggregateMeta { impl Debug for AggregateMeta { fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { match self { - AggregateMeta::FinalPartition => f.debug_struct("AggregateMeta::Partitioned").finish(), + AggregateMeta::FinalPartition(_) => { + f.debug_struct("AggregateMeta::Partitioned").finish() + } AggregateMeta::SpilledPayload(_) => { f.debug_struct("Aggregate::SpilledPayload").finish() } 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 1d7a7f94b47f0..94f2d0ec7bba8 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/mod.rs @@ -19,7 +19,12 @@ mod serde; mod transform_aggregate_expand; mod transform_aggregate_final; mod transform_aggregate_partial; +mod transform_partition_align; mod transform_partition_bucket; +mod transform_partition_dispatch; +mod transform_partition_exchange; +mod transform_partition_resorting; +mod transform_partition_restore; mod transform_single_key; mod udaf_script; @@ -29,7 +34,7 @@ pub use aggregator_params::AggregatorParams; pub use transform_aggregate_expand::TransformExpandGroupingSets; pub use transform_aggregate_final::TransformFinalAggregate; pub use transform_aggregate_partial::TransformPartialAggregate; -pub use transform_partition_bucket::build_partition_dispatch; +pub use transform_partition_bucket::build_final_aggregate; pub use transform_single_key::FinalSingleStateAggregator; pub use transform_single_key::PartialSingleStateAggregator; pub use udaf_script::*; 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 d4b73500c0864..16152f57bef77 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 @@ -13,10 +13,10 @@ // limitations under the License. mod transform_deserializer; -mod transform_spill_reader; pub use transform_deserializer::*; -pub use transform_spill_reader::*; + +pub use crate::pipelines::processors::transforms::aggregator::transform_partition_restore::*; pub mod exchange_defines { use arrow_ipc::writer::IpcWriteOptions; 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 b5a7ba09e282f..d9bb08e165cff 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 @@ -39,6 +39,9 @@ pub struct TransformFinalAggregate { flush_state: PayloadFlushState, hash_table: AggregateHashTable, has_output: bool, + + totals_upstream: usize, + recv_final_payload: usize, } impl AccumulatingTransform for TransformFinalAggregate { @@ -60,6 +63,7 @@ impl AccumulatingTransform for TransformFinalAggregate { if !data.is_empty() { let payload = self.deserialize_flight(data)?; + self.hash_table .combine_payload(&payload, &mut self.flush_state)?; } @@ -72,11 +76,19 @@ impl AccumulatingTransform for TransformFinalAggregate { .combine_payload(&payload.payload, &mut self.flush_state)?; } } - AggregateMeta::FinalPartition => { + AggregateMeta::FinalPartition(_) => { + self.recv_final_payload += 1; + if self.hash_table.len() == 0 { return Ok(vec![]); } + // Due to the local shuffle, we will receive the same number of final partitions as the upstream. + // We must wait for all final partitions to arrive before we can flush out the results. + if self.recv_final_payload % self.totals_upstream != 0 { + return Ok(vec![]); + } + let mut blocks = vec![]; self.flush_state.clear(); @@ -115,6 +127,7 @@ impl AccumulatingTransform for TransformFinalAggregate { impl TransformFinalAggregate { pub fn try_create( + totals_upstream: usize, input: Arc, output: Arc, params: Arc, @@ -136,6 +149,8 @@ impl TransformFinalAggregate { hash_table, flush_state: PayloadFlushState::default(), has_output: false, + totals_upstream, + recv_final_payload: 0, }, )) } diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_align.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_align.rs new file mode 100644 index 0000000000000..0f8b0fc235eb3 --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_align.rs @@ -0,0 +1,269 @@ +// 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::collections::BTreeMap; +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::BlockMetaInfoDowncast; +use databend_common_expression::DataBlock; +use databend_common_expression::HashTableConfig; +use databend_common_expression::InputColumns; +use databend_common_expression::PartitionedPayload; +use databend_common_expression::Payload; +use databend_common_expression::PayloadFlushState; +use databend_common_expression::ProbeState; +use databend_common_pipeline_transforms::AccumulatingTransform; + +use crate::pipelines::processors::transforms::aggregator::transform_partition_bucket::SINGLE_LEVEL_BUCKET_NUM; +use crate::pipelines::processors::transforms::aggregator::AggregateMeta; +use crate::pipelines::processors::transforms::aggregator::AggregatePayload; +use crate::pipelines::processors::transforms::aggregator::AggregatorParams; + +pub struct TransformPartitionAlign { + params: Arc, + + max_partition: usize, + working_partition: isize, + partitions: Partitions, +} + +impl TransformPartitionAlign { + pub fn create(params: Arc) -> Result { + Ok(TransformPartitionAlign { + params, + max_partition: 0, + working_partition: 0, + partitions: Partitions::create(), + }) + } + + fn ready_partition(&mut self) -> Option { + let storage_min_partition = self.partitions.min_partition()?; + + if storage_min_partition > self.working_partition { + return None; + } + + Some(storage_min_partition) + } + + fn fetch_ready_partition(&mut self) -> Result> { + if let Some(ready_partition_id) = self.ready_partition() { + let ready_partition = self.partitions.take_partition(ready_partition_id); + + let mut ready_data = Vec::with_capacity(ready_partition.len()); + for (meta, data_block) in ready_partition { + ready_data.push(data_block.add_meta(Some(Box::new(meta)))?); + } + + ready_data.push(DataBlock::empty_with_meta(AggregateMeta::create_final( + None, + ))); + return Ok(ready_data); + } + + Ok(vec![]) + } + + fn unpark_block(&self, mut data_block: DataBlock) -> Result<(AggregateMeta, DataBlock)> { + let Some(meta) = data_block.take_meta() else { + return Err(ErrorCode::Internal( + "Internal, TransformPartitionBucket only recv DataBlock with meta.", + )); + }; + + let Some(meta) = AggregateMeta::downcast_from(meta) else { + return Err(ErrorCode::Internal( + "Internal, TransformPartitionBucket only recv AggregateMeta".to_string(), + )); + }; + + Ok((meta, data_block)) + } + + fn repartition(&mut self, meta: AggregateMeta, data_block: DataBlock) -> Result<()> { + match meta { + AggregateMeta::FinalPartition(_) => unreachable!(), + AggregateMeta::SpilledPayload(_payload) => unreachable!(), + AggregateMeta::InFlightPayload(payload) => { + if data_block.is_empty() { + return Ok(()); + } + + let payload = AggregatePayload { + partition: payload.partition, + max_partition: payload.max_partition, + payload: self.deserialize_flight(data_block)?, + global_max_partition: payload.global_max_partition, + }; + + let repartition = payload.global_max_partition; + let partitioned = self.partition_payload(payload, repartition); + + for payload in partitioned { + self.partitions + .add_data(AggregateMeta::AggregatePayload(payload), DataBlock::empty()); + } + } + AggregateMeta::AggregatePayload(payload) => { + if payload.payload.len() == 0 { + return Ok(()); + } + + let repartition = payload.global_max_partition; + let partitioned = self.partition_payload(payload, repartition); + for payload in partitioned { + self.partitions + .add_data(AggregateMeta::AggregatePayload(payload), DataBlock::empty()); + } + } + } + + Ok(()) + } + + fn deserialize_flight(&mut self, data: DataBlock) -> Result { + let rows_num = data.num_rows(); + let group_len = self.params.group_data_types.len(); + + let mut state = ProbeState::default(); + + // create single partition hash table for deserialize + let capacity = AggregateHashTable::get_capacity_for_count(rows_num); + let config = HashTableConfig::default().with_initial_radix_bits(0); + let mut hashtable = AggregateHashTable::new_directly( + self.params.group_data_types.clone(), + self.params.aggregate_functions.clone(), + config, + capacity, + Arc::new(Bump::new()), + false, + ); + + let num_states = self.params.num_states(); + let states_index: Vec = (0..num_states).collect(); + let agg_states = InputColumns::new_block_proxy(&states_index, &data); + + let group_index: Vec = (num_states..(num_states + group_len)).collect(); + let group_columns = InputColumns::new_block_proxy(&group_index, &data); + + let _ = hashtable.add_groups( + &mut state, + group_columns, + &[(&[]).into()], + agg_states, + rows_num, + )?; + + hashtable.payload.mark_min_cardinality(); + assert_eq!(hashtable.payload.payloads.len(), 1); + Ok(hashtable.payload.payloads.pop().unwrap()) + } + + fn partition_payload(&mut self, from: AggregatePayload, to: usize) -> Vec { + let mut partitioned = Vec::with_capacity(to); + let mut partitioned_payload = PartitionedPayload::new( + self.params.group_data_types.clone(), + self.params.aggregate_functions.clone(), + to as u64, + vec![from.payload.arena.clone()], + ); + + let mut flush_state = PayloadFlushState::default(); + partitioned_payload.combine_single(from.payload, &mut flush_state, None); + + for (partition, payload) in partitioned_payload.payloads.into_iter().enumerate() { + partitioned.push(AggregatePayload { + payload, + partition: partition as isize, + max_partition: to, + global_max_partition: from.global_max_partition, + }); + } + + partitioned + } +} + +impl AccumulatingTransform for TransformPartitionAlign { + const NAME: &'static str = "TransformPartitionAlign"; + + fn transform(&mut self, data_block: DataBlock) -> Result> { + let (meta, data_block) = self.unpark_block(data_block)?; + self.max_partition = meta.get_global_max_partition(); + + // need repartition + if meta.get_max_partition() != meta.get_global_max_partition() { + self.repartition(meta, data_block)?; + return Ok(vec![]); + } + + let partition = meta.get_sorting_partition(); + self.partitions.add_data(meta, data_block); + + if partition > SINGLE_LEVEL_BUCKET_NUM && partition != self.working_partition { + let ready_partition = self.fetch_ready_partition()?; + self.working_partition = partition; + return Ok(ready_partition); + } + + Ok(vec![]) + } + + fn on_finish(&mut self, _output: bool) -> Result> { + self.working_partition = self.max_partition as isize; + self.fetch_ready_partition() + } + + fn need_spill(&self) -> bool { + // TODO: spill if need + false + } +} + +#[derive(Debug)] +struct Partitions { + data: BTreeMap>, +} + +impl Partitions { + pub fn create() -> Partitions { + Partitions { + data: BTreeMap::new(), + } + } + + pub fn add_data(&mut self, meta: AggregateMeta, block: DataBlock) { + match self.data.entry(meta.get_partition()) { + std::collections::btree_map::Entry::Vacant(v) => { + v.insert(vec![(meta, block)]); + } + std::collections::btree_map::Entry::Occupied(mut v) => { + v.get_mut().push((meta, block)); + } + }; + } + + pub fn min_partition(&self) -> Option { + self.data.keys().min().cloned() + } + + pub fn take_partition(&mut self, partition: isize) -> Vec<(AggregateMeta, DataBlock)> { + self.data.remove(&partition).unwrap_or_default() + } +} 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 index b75fed1c6b104..3e79f57d9bc94 100644 --- 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 @@ -12,482 +12,70 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::any::Any; -use std::cmp::Ordering; -use std::collections::BTreeMap; -use std::collections::VecDeque; -use std::fmt::Debug; -use std::sync::atomic::AtomicUsize; -use std::sync::atomic::Ordering as AtomicOrdering; 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::BlockMetaInfoDowncast; -use databend_common_expression::DataBlock; -use databend_common_expression::HashTableConfig; -use databend_common_expression::InputColumns; -use databend_common_expression::PartitionedPayload; -use databend_common_expression::Payload; -use databend_common_expression::PayloadFlushState; -use databend_common_expression::ProbeState; -use databend_common_pipeline_core::processors::Event; -use databend_common_pipeline_core::processors::Exchange; -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_pipeline_transforms::AccumulatingTransformer; use databend_common_storage::DataOperator; -use tokio::sync::Semaphore; -use super::AggregatePayload; use super::TransformFinalAggregate; -use super::TransformSpillReader; -use crate::pipelines::processors::transforms::aggregator::aggregate_meta::AggregateMeta; +use super::TransformPartitionRestore; +use crate::pipelines::processors::transforms::aggregator::transform_partition_align::TransformPartitionAlign; +use crate::pipelines::processors::transforms::aggregator::transform_partition_dispatch::TransformPartitionDispatch; +use crate::pipelines::processors::transforms::aggregator::transform_partition_exchange::ExchangePartition; +use crate::pipelines::processors::transforms::aggregator::transform_partition_resorting::ResortingPartition; use crate::pipelines::processors::transforms::aggregator::AggregatorParams; -static SINGLE_LEVEL_BUCKET_NUM: isize = -1; +pub static SINGLE_LEVEL_BUCKET_NUM: isize = -1; -pub struct TransformPartitionDispatch { - params: Arc, - outputs: Vec>, - input: Arc, - outputs_data: Vec>, - output_index: usize, - finished: bool, - input_data: Option<(AggregateMeta, DataBlock)>, +pub fn build_final_aggregate(pipeline: &mut Pipeline, params: Arc) -> Result<()> { + let pipe_size = pipeline.output_len(); - max_partition: usize, - working_partition: isize, - partitions: Partitions, -} - -impl TransformPartitionDispatch { - pub fn create(output_nums: usize, params: Arc) -> Result { - let mut outputs = Vec::with_capacity(output_nums); - let mut outputs_data = Vec::with_capacity(output_nums); - - for _index in 0..output_nums { - outputs.push(OutputPort::create()); - outputs_data.push(VecDeque::new()); - } - - Ok(TransformPartitionDispatch { - params, - outputs, - outputs_data, - input: InputPort::create(), - output_index: 0, - max_partition: 0, - working_partition: 0, - finished: false, - input_data: None, - partitions: Partitions::create(), - }) - } - - pub fn get_input(&self) -> Arc { - self.input.clone() - } - - pub fn get_outputs(&self) -> Vec> { - self.outputs.clone() - } - - fn ready_partition(&mut self) -> Option { - let storage_min_partition = self.partitions.min_partition()?; - - if storage_min_partition > self.working_partition { - return None; - } - - Some(storage_min_partition) - } - - fn fetch_ready_partition(&mut self) -> Result<()> { - if let Some(ready_partition_id) = self.ready_partition() { - let ready_partition = self.partitions.take_partition(ready_partition_id); - - for (meta, data_block) in ready_partition { - self.outputs_data[self.output_index] - .push_back(data_block.add_meta(Some(Box::new(meta)))?); - } - - self.outputs_data[self.output_index] - .push_back(DataBlock::empty_with_meta(AggregateMeta::create_final())); - - self.output_index += 1; - if self.output_index >= self.outputs_data.len() { - self.output_index = 0; - } - } - - Ok(()) - } - - fn unpark_block(&self, mut data_block: DataBlock) -> Result<(AggregateMeta, DataBlock)> { - let Some(meta) = data_block.take_meta() else { - return Err(ErrorCode::Internal( - "Internal, TransformPartitionBucket only recv DataBlock with meta.", - )); - }; - - let Some(meta) = AggregateMeta::downcast_from(meta) else { - return Err(ErrorCode::Internal( - "Internal, TransformPartitionBucket only recv AggregateMeta".to_string(), - )); - }; - - Ok((meta, data_block)) - } -} - -#[async_trait::async_trait] -impl Processor for TransformPartitionDispatch { - fn name(&self) -> String { - String::from("TransformPartitionDispatch") - } - - fn as_any(&mut self) -> &mut dyn Any { - self - } - - fn event(&mut self) -> Result { - loop { - let mut all_output_finished = true; - let mut all_data_pushed_output = true; - - for (idx, output) in self.outputs.iter().enumerate() { - if output.is_finished() { - self.outputs_data[idx].clear(); - continue; - } - - if self.finished && self.outputs_data[idx].is_empty() { - output.finish(); - continue; - } - - all_output_finished = false; - - if output.can_push() { - if let Some(block) = self.outputs_data[idx].pop_front() { - output.push_data(Ok(block)); - } - } - - if !self.outputs_data[idx].is_empty() { - all_data_pushed_output = false; - } - } - - if all_output_finished { - self.input.finish(); - return Ok(Event::Finished); - } - - if !all_data_pushed_output { - self.input.set_not_need_data(); - return Ok(Event::NeedConsume); - } - - if self.input.has_data() { - let data_block = self.input.pull_data().unwrap()?; - let (meta, data_block) = self.unpark_block(data_block)?; - self.max_partition = meta.get_global_max_partition(); - - // need repartition - if meta.get_max_partition() != meta.get_global_max_partition() { - self.input_data = Some((meta, data_block)); - return Ok(Event::Sync); - } - - let partition = meta.get_sorting_partition(); - self.partitions.add_data(meta, data_block); - - if partition > SINGLE_LEVEL_BUCKET_NUM && partition != self.working_partition { - self.fetch_ready_partition()?; - self.working_partition = partition; - continue; - } - } - - if self.input.is_finished() { - self.working_partition = self.max_partition as isize; - - self.fetch_ready_partition()?; - self.finished = self.partitions.is_empty(); - continue; - } - - self.input.set_need_data(); - return Ok(Event::NeedData); - } - } - - fn process(&mut self) -> Result<()> { - let Some((meta, data_block)) = self.input_data.take() else { - return Ok(()); - }; - - match meta { - AggregateMeta::FinalPartition => unreachable!(), - AggregateMeta::SpilledPayload(_payload) => unreachable!(), - AggregateMeta::InFlightPayload(payload) => { - if data_block.is_empty() { - return Ok(()); - } - - let payload = AggregatePayload { - partition: payload.partition, - max_partition: payload.max_partition, - payload: self.deserialize_flight(data_block)?, - global_max_partition: payload.global_max_partition, - }; - - let repartition = payload.global_max_partition; - let partitioned = self.partition_payload(payload, repartition); - - for payload in partitioned { - self.partitions - .add_data(AggregateMeta::AggregatePayload(payload), DataBlock::empty()); - } - } - AggregateMeta::AggregatePayload(payload) => { - if payload.payload.len() == 0 { - return Ok(()); - } - - let repartition = payload.global_max_partition; - let partitioned = self.partition_payload(payload, repartition); - for payload in partitioned { - self.partitions - .add_data(AggregateMeta::AggregatePayload(payload), DataBlock::empty()); - } - } - } - - Ok(()) - } -} - -pub struct ResortingPartition { - global_max_partition: AtomicUsize, -} - -impl ResortingPartition { - fn block_number(meta: &AggregateMeta) -> (isize, usize) { - (meta.get_sorting_partition(), meta.get_max_partition()) - } -} - -impl Exchange for ResortingPartition { - const NAME: &'static str = "PartitionResorting"; - const MULTIWAY_SORT: bool = true; - - fn partition(&self, mut data_block: DataBlock, n: usize) -> Result> { - debug_assert_eq!(n, 1); - - let Some(meta) = data_block.take_meta() else { - return Ok(vec![data_block]); - }; - - let Some(_) = AggregateMeta::downcast_ref_from(&meta) else { - return Ok(vec![data_block]); - }; - - let global_max_partition = self.global_max_partition.load(AtomicOrdering::SeqCst); - let mut meta = AggregateMeta::downcast_from(meta).unwrap(); - meta.set_global_max_partition(global_max_partition); - - Ok(vec![data_block.add_meta(Some(Box::new(meta)))?]) - } - - fn init_way(&self, _index: usize, first_data: &DataBlock) -> Result<()> { - let max_partition = match first_data.get_meta() { - None => 0, - Some(meta) => match AggregateMeta::downcast_ref_from(meta) { - None => 0, - Some(v) => v.get_global_max_partition(), - }, - }; - - self.global_max_partition - .fetch_max(max_partition, std::sync::atomic::Ordering::SeqCst); - Ok(()) - } - - fn sorting_function(left_block: &DataBlock, right_block: &DataBlock) -> Ordering { - let Some(left_meta) = left_block.get_meta() else { - return Ordering::Equal; - }; - let Some(left_meta) = AggregateMeta::downcast_ref_from(left_meta) else { - return Ordering::Equal; - }; - - let Some(right_meta) = right_block.get_meta() else { - return Ordering::Equal; - }; - let Some(right_meta) = AggregateMeta::downcast_ref_from(right_meta) else { - return Ordering::Equal; - }; - - let (l_partition, l_max_partition) = ResortingPartition::block_number(left_meta); - let (r_partition, r_max_partition) = ResortingPartition::block_number(right_meta); + // 1. resorting partition + pipeline.exchange(1, Arc::new(ResortingPartition::create())); - // ORDER BY max_partition asc, partition asc, idx asc - match l_max_partition.cmp(&r_max_partition) { - Ordering::Less => Ordering::Less, - Ordering::Greater => Ordering::Greater, - Ordering::Equal => l_partition.cmp(&r_partition), - } - } -} - -pub fn build_partition_dispatch( - pipeline: &mut Pipeline, - params: Arc, -) -> Result<()> { - let output = pipeline.output_len(); - - // 1. reorder partition - pipeline.exchange( - 1, - Arc::new(ResortingPartition { - global_max_partition: AtomicUsize::new(0), - }), - ); - - let transform = TransformPartitionDispatch::create(output, params.clone())?; - - let input_port = transform.get_input(); - let outputs_port = transform.get_outputs(); - - pipeline.add_pipe(Pipe::create(1, outputs_port.len(), vec![PipeItem::create( - ProcessorPtr::create(Box::new(transform)), - vec![input_port], - outputs_port, - )])); + // 2. align partitions + pipeline.add_transform(|input, output| { + Ok(ProcessorPtr::create(AccumulatingTransformer::create( + input, + output, + TransformPartitionAlign::create(params.clone())?, + ))) + })?; - let semaphore = Arc::new(Semaphore::new(params.max_spill_io_requests)); + // 3. dispatch partition + let processor = TransformPartitionDispatch::create(pipe_size); + let inputs_port = processor.get_inputs(); + let outputs_port = processor.get_outputs(); + pipeline.add_pipe(Pipe::create(inputs_port.len(), outputs_port.len(), vec![ + PipeItem::create( + ProcessorPtr::create(Box::new(processor)), + inputs_port, + outputs_port, + ), + ])); + + // 4. restore partition let operator = DataOperator::instance().spill_operator(); pipeline.add_transform(|input, output| { - let operator = operator.clone(); - TransformSpillReader::create(input, output, operator, semaphore.clone(), params.clone()) + TransformPartitionRestore::create(input, output, operator.clone(), params.clone()) })?; + // 5. exchange local + let pipe_size = pipeline.output_len(); + pipeline.exchange(pipe_size, ExchangePartition::create(params.clone())); + + // 6. final aggregate pipeline.add_transform(|input, output| { Ok(ProcessorPtr::create(TransformFinalAggregate::try_create( - input, - output, + pipe_size, + input.clone(), + output.clone(), params.clone(), )?)) - })?; - Ok(()) -} - -// repartition implementation -impl TransformPartitionDispatch { - fn deserialize_flight(&mut self, data: DataBlock) -> Result { - let rows_num = data.num_rows(); - let group_len = self.params.group_data_types.len(); - - let mut state = ProbeState::default(); - - // create single partition hash table for deserialize - let capacity = AggregateHashTable::get_capacity_for_count(rows_num); - let config = HashTableConfig::default().with_initial_radix_bits(0); - let mut hashtable = AggregateHashTable::new_directly( - self.params.group_data_types.clone(), - self.params.aggregate_functions.clone(), - config, - capacity, - Arc::new(Bump::new()), - false, - ); - - let num_states = self.params.num_states(); - let states_index: Vec = (0..num_states).collect(); - let agg_states = InputColumns::new_block_proxy(&states_index, &data); - - let group_index: Vec = (num_states..(num_states + group_len)).collect(); - let group_columns = InputColumns::new_block_proxy(&group_index, &data); - - let _ = hashtable.add_groups( - &mut state, - group_columns, - &[(&[]).into()], - agg_states, - rows_num, - )?; - - hashtable.payload.mark_min_cardinality(); - assert_eq!(hashtable.payload.payloads.len(), 1); - Ok(hashtable.payload.payloads.pop().unwrap()) - } - - fn partition_payload(&mut self, from: AggregatePayload, to: usize) -> Vec { - let mut partitioned = Vec::with_capacity(to); - let mut partitioned_payload = PartitionedPayload::new( - self.params.group_data_types.clone(), - self.params.aggregate_functions.clone(), - to as u64, - vec![from.payload.arena.clone()], - ); - - let mut flush_state = PayloadFlushState::default(); - partitioned_payload.combine_single(from.payload, &mut flush_state, None); - - for (partition, payload) in partitioned_payload.payloads.into_iter().enumerate() { - partitioned.push(AggregatePayload { - payload, - partition: partition as isize, - max_partition: to, - global_max_partition: from.global_max_partition, - }); - } - - partitioned - } -} - -#[derive(Debug)] -struct Partitions { - data: BTreeMap>, -} - -impl Partitions { - pub fn create() -> Partitions { - Partitions { - data: BTreeMap::new(), - } - } - - pub fn is_empty(&self) -> bool { - self.data.is_empty() - } - - pub fn add_data(&mut self, meta: AggregateMeta, block: DataBlock) { - match self.data.entry(meta.get_partition()) { - std::collections::btree_map::Entry::Vacant(v) => { - v.insert(vec![(meta, block)]); - } - std::collections::btree_map::Entry::Occupied(mut v) => { - v.get_mut().push((meta, block)); - } - }; - } - - pub fn min_partition(&self) -> Option { - self.data.keys().min().cloned() - } - - pub fn take_partition(&mut self, partition: isize) -> Vec<(AggregateMeta, DataBlock)> { - self.data.remove(&partition).unwrap_or_default() - } + }) } diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_dispatch.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_dispatch.rs new file mode 100644 index 0000000000000..d95d89a85ed56 --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_dispatch.rs @@ -0,0 +1,247 @@ +// 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::VecDeque; +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_pipeline_core::processors::Event; +use databend_common_pipeline_core::processors::EventCause; +use databend_common_pipeline_core::processors::InputPort; +use databend_common_pipeline_core::processors::OutputPort; +use databend_common_pipeline_core::processors::Processor; + +use crate::pipelines::processors::transforms::aggregator::AggregateMeta; + +#[derive(PartialEq)] +enum PortStatus { + Idle, + NeedData, + Finished, +} + +struct PortWithStatus { + pub status: PortStatus, + pub port: Arc, +} + +pub struct TransformPartitionDispatch { + initialized: bool, + + finished_outputs: usize, + waiting_outputs: VecDeque, + waiting_outputs_2: VecDeque, + + sync_final_partition: bool, + sent_final_partition: Vec, + synchronized_final_partition: Vec, + + current_data: Option, + + input: Arc, + outputs: Vec>, +} + +impl TransformPartitionDispatch { + pub fn create(outputs: usize) -> TransformPartitionDispatch { + let mut outputs_port = Vec::with_capacity(outputs); + + for _index in 0..outputs { + outputs_port.push(PortWithStatus { + status: PortStatus::Idle, + port: OutputPort::create(), + }); + } + + TransformPartitionDispatch { + initialized: false, + finished_outputs: 0, + outputs: outputs_port, + input: InputPort::create(), + waiting_outputs: VecDeque::with_capacity(outputs), + waiting_outputs_2: VecDeque::with_capacity(outputs), + current_data: None, + sync_final_partition: false, + sent_final_partition: vec![false; outputs], + synchronized_final_partition: vec![false; outputs], + } + } + + pub fn get_inputs(&self) -> Vec> { + vec![self.input.clone()] + } + + pub fn get_outputs(&self) -> Vec> { + self.outputs.iter().map(|x| x.port.clone()).collect() + } + + fn unpark_block(mut data_block: DataBlock) -> Result<(AggregateMeta, DataBlock)> { + let Some(meta) = data_block.take_meta() else { + return Err(ErrorCode::Internal( + "Internal, TransformPartitionBucket only recv DataBlock with meta.", + )); + }; + + let Some(meta) = AggregateMeta::downcast_from(meta) else { + return Err(ErrorCode::Internal( + "Internal, TransformPartitionBucket only recv AggregateMeta".to_string(), + )); + }; + + Ok((meta, data_block)) + } +} + +impl Processor for TransformPartitionDispatch { + fn name(&self) -> String { + String::from("TransformPartitionDispatch") + } + + fn as_any(&mut self) -> &mut dyn Any { + self + } + + #[allow(clippy::collapsible_if)] + fn event_with_cause(&mut self, cause: EventCause) -> Result { + if let EventCause::Output(output_index) = &cause { + let output = &mut self.outputs[*output_index]; + + if output.port.is_finished() { + if output.status != PortStatus::Finished { + self.finished_outputs += 1; + output.status = PortStatus::Finished; + } + } else if output.port.can_push() { + if self.sync_final_partition { + if self.sent_final_partition[*output_index] { + self.waiting_outputs_2.push_back(*output_index); + self.synchronized_final_partition[*output_index] = true; + } else { + self.sent_final_partition[*output_index] = true; + output.port.push_data(Ok(DataBlock::empty_with_meta( + AggregateMeta::create_final(None), + ))); + } + } else if output.status != PortStatus::NeedData { + output.status = PortStatus::NeedData; + self.waiting_outputs.push_back(*output_index); + } + } + } + + if !self.initialized && !self.waiting_outputs.is_empty() { + self.initialized = true; + self.input.set_need_data(); + } + + if self.finished_outputs == self.outputs.len() { + self.input.finish(); + return Ok(Event::Finished); + } + + if let EventCause::Input(_) = &cause { + if !self.sync_final_partition && self.input.has_data() && self.current_data.is_none() { + let data_block = self.input.pull_data().unwrap()?; + let (meta, data_block) = Self::unpark_block(data_block)?; + + match meta { + AggregateMeta::FinalPartition(_) => { + self.sync_final_partition = true; + self.input.set_not_need_data(); + } + meta => { + self.input.set_need_data(); + self.current_data = Some(data_block.add_meta(Some(Box::new(meta)))?); + } + }; + } + } + + if self.sync_final_partition { + while let Some(output_index) = self.waiting_outputs.pop_front() { + if self.outputs[output_index].port.is_finished() { + self.synchronized_final_partition[output_index] = true; + + if self.outputs[output_index].status != PortStatus::Finished { + self.finished_outputs += 1; + self.outputs[output_index].status = PortStatus::Finished; + } + } + + self.outputs[output_index] + .port + .push_data(Ok(DataBlock::empty_with_meta(AggregateMeta::create_final( + None, + )))); + self.sent_final_partition[output_index] = true; + self.outputs[output_index].status = PortStatus::Idle; + } + + for (idx, synchronized) in self.synchronized_final_partition.iter().enumerate() { + if !synchronized && !self.outputs[idx].port.is_finished() { + return Ok(Event::NeedConsume); + } + } + + self.sync_final_partition = false; + self.sent_final_partition = vec![false; self.sent_final_partition.len()]; + self.synchronized_final_partition = vec![false; self.sent_final_partition.len()]; + std::mem::swap(&mut self.waiting_outputs, &mut self.waiting_outputs_2); + self.input.set_need_data(); + return Ok(Event::NeedData); + } + + while !self.waiting_outputs.is_empty() && self.current_data.is_some() { + let output_index = self.waiting_outputs.pop_front().unwrap(); + + // Port is finished when waiting. + if self.outputs[output_index].port.is_finished() { + if self.outputs[output_index].status != PortStatus::Finished { + self.finished_outputs += 1; + self.outputs[output_index].status = PortStatus::Finished; + } + + continue; + } + + if let Some(data_block) = self.current_data.take() { + self.outputs[output_index].port.push_data(Ok(data_block)); + self.outputs[output_index].status = PortStatus::Idle; + self.input.set_need_data(); + } + } + + if self.finished_outputs == self.outputs.len() { + self.input.finish(); + return Ok(Event::Finished); + } + + if self.input.is_finished() && self.current_data.is_none() { + for output in &self.outputs { + output.port.finish(); + } + + return Ok(Event::Finished); + } + + match self.waiting_outputs.is_empty() { + true => Ok(Event::NeedConsume), + false => Ok(Event::NeedData), + } + } +} diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_exchange.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_exchange.rs new file mode 100644 index 0000000000000..d4b236bcfe9c4 --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_exchange.rs @@ -0,0 +1,183 @@ +// 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::Result; +use databend_common_expression::AggregateHashTable; +use databend_common_expression::BlockMetaInfoDowncast; +use databend_common_expression::DataBlock; +use databend_common_expression::HashTableConfig; +use databend_common_expression::InputColumns; +use databend_common_expression::Payload; +use databend_common_expression::PayloadFlushState; +use databend_common_expression::ProbeState; +use databend_common_pipeline_core::processors::Exchange; + +use crate::pipelines::processors::transforms::aggregator::AggregateMeta; +use crate::pipelines::processors::transforms::aggregator::AggregatePayload; +use crate::pipelines::processors::transforms::aggregator::AggregatorParams; +use crate::pipelines::processors::transforms::aggregator::InFlightPayload; + +pub struct ExchangePartition { + params: Arc, +} + +impl ExchangePartition { + pub fn create(params: Arc) -> Arc { + Arc::new(ExchangePartition { params }) + } +} + +impl ExchangePartition { + fn partition_final_payload(n: usize) -> Result> { + Ok((0..n) + .map(|_| DataBlock::empty_with_meta(AggregateMeta::create_final(None))) + .collect()) + } + + fn partition_aggregate_payload( + mut payload: AggregatePayload, + n: usize, + ) -> Result> { + let mut repartition_payloads = Vec::with_capacity(n); + + let group_types = payload.payload.group_types.clone(); + let aggrs = payload.payload.aggrs.clone(); + let mut state = PayloadFlushState::default(); + + for _ in 0..repartition_payloads.capacity() { + repartition_payloads.push(Payload::new( + payload.payload.arena.clone(), + group_types.clone(), + aggrs.clone(), + payload.payload.states_layout.clone(), + )); + } + + // scatter each page of the payload. + while payload + .payload + .scatter_with_seed::<9263883436177860930>(&mut state, repartition_payloads.len()) + { + // copy to the corresponding bucket. + for (idx, bucket) in repartition_payloads.iter_mut().enumerate() { + let count = state.probe_state.partition_count[idx]; + + if count > 0 { + let sel = &state.probe_state.partition_entries[idx]; + bucket.copy_rows(sel, count, &state.addresses); + } + } + } + + payload.payload.state_move_out = true; + + let mut partitions = Vec::with_capacity(repartition_payloads.len()); + + for repartition_payload in repartition_payloads { + partitions.push(DataBlock::empty_with_meta( + AggregateMeta::create_agg_payload( + repartition_payload, + payload.partition, + payload.max_partition, + payload.global_max_partition, + ), + )); + } + + Ok(partitions) + } + + fn partition_flight_payload( + &self, + payload: InFlightPayload, + block: DataBlock, + n: usize, + ) -> Result> { + let rows_num = block.num_rows(); + let group_len = self.params.group_data_types.len(); + + let mut state = ProbeState::default(); + + // create single partition hash table for deserialize + let capacity = AggregateHashTable::get_capacity_for_count(rows_num); + let config = HashTableConfig::default().with_initial_radix_bits(0); + let mut hashtable = AggregateHashTable::new_directly( + self.params.group_data_types.clone(), + self.params.aggregate_functions.clone(), + config, + capacity, + Arc::new(Bump::new()), + false, + ); + + let num_states = self.params.num_states(); + let states_index: Vec = (0..num_states).collect(); + let agg_states = InputColumns::new_block_proxy(&states_index, &block); + + let group_index: Vec = (num_states..(num_states + group_len)).collect(); + let group_columns = InputColumns::new_block_proxy(&group_index, &block); + + let _ = hashtable.add_groups( + &mut state, + group_columns, + &[(&[]).into()], + agg_states, + rows_num, + )?; + + hashtable.payload.mark_min_cardinality(); + assert_eq!(hashtable.payload.payloads.len(), 1); + + Self::partition_aggregate_payload( + AggregatePayload { + partition: payload.partition, + payload: hashtable.payload.payloads.pop().unwrap(), + max_partition: payload.max_partition, + global_max_partition: payload.global_max_partition, + }, + n, + ) + } +} + +impl Exchange for ExchangePartition { + const NAME: &'static str = "PartitionResorting"; + const MULTIWAY_SORT: bool = false; + + fn partition(&self, mut data_block: DataBlock, n: usize) -> Result> { + let Some(meta) = data_block.take_meta() else { + return Ok(vec![data_block]); + }; + + let Some(meta) = AggregateMeta::downcast_from(meta) else { + return Ok(vec![data_block]); + }; + + match meta { + // already restore in upstream + AggregateMeta::SpilledPayload(_) => unreachable!(), + // broadcast final partition to downstream + AggregateMeta::FinalPartition(_) => Self::partition_final_payload(n), + AggregateMeta::AggregatePayload(payload) => { + Self::partition_aggregate_payload(payload, n) + } + AggregateMeta::InFlightPayload(payload) => { + self.partition_flight_payload(payload, data_block, n) + } + } + } +} diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_resorting.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_resorting.rs new file mode 100644 index 0000000000000..8e02b93f0e6a1 --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_resorting.rs @@ -0,0 +1,110 @@ +// 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::cmp::Ordering; +use std::sync::atomic::AtomicUsize; +use std::sync::atomic::Ordering as AtomicOrdering; + +use databend_common_expression::BlockMetaInfoDowncast; +use databend_common_expression::DataBlock; +use databend_common_pipeline_core::processors::Exchange; + +use crate::pipelines::processors::transforms::aggregator::AggregateMeta; + +pub struct ResortingPartition { + global_max_partition: AtomicUsize, +} + +impl ResortingPartition { + pub fn create() -> Self { + ResortingPartition { + global_max_partition: AtomicUsize::new(0), + } + } + + fn block_number(meta: &AggregateMeta) -> (isize, usize) { + (meta.get_sorting_partition(), meta.get_max_partition()) + } +} + +impl Exchange for ResortingPartition { + const NAME: &'static str = "PartitionResorting"; + const MULTIWAY_SORT: bool = true; + + fn partition( + &self, + mut data_block: DataBlock, + n: usize, + ) -> databend_common_exception::Result> { + debug_assert_eq!(n, 1); + + let Some(meta) = data_block.take_meta() else { + return Ok(vec![data_block]); + }; + + let Some(_) = AggregateMeta::downcast_ref_from(&meta) else { + return Ok(vec![data_block]); + }; + + let global_max_partition = self.global_max_partition.load(AtomicOrdering::SeqCst); + let mut meta = AggregateMeta::downcast_from(meta).unwrap(); + meta.set_global_max_partition(global_max_partition); + + Ok(vec![data_block.add_meta(Some(Box::new(meta)))?]) + } + + fn init_way( + &self, + _index: usize, + first_data: &DataBlock, + ) -> databend_common_exception::Result<()> { + let max_partition = match first_data.get_meta() { + None => 0, + Some(meta) => match AggregateMeta::downcast_ref_from(meta) { + None => 0, + Some(v) => v.get_global_max_partition(), + }, + }; + + self.global_max_partition + .fetch_max(max_partition, std::sync::atomic::Ordering::SeqCst); + Ok(()) + } + + fn sorting_function(left_block: &DataBlock, right_block: &DataBlock) -> Ordering { + let Some(left_meta) = left_block.get_meta() else { + return Ordering::Equal; + }; + let Some(left_meta) = AggregateMeta::downcast_ref_from(left_meta) else { + return Ordering::Equal; + }; + + let Some(right_meta) = right_block.get_meta() else { + return Ordering::Equal; + }; + let Some(right_meta) = AggregateMeta::downcast_ref_from(right_meta) else { + return Ordering::Equal; + }; + + let (l_partition, l_max_partition) = ResortingPartition::block_number(left_meta); + let (r_partition, r_max_partition) = ResortingPartition::block_number(right_meta); + + // ORDER BY max_partition asc, partition asc, idx asc + match l_max_partition.cmp(&r_max_partition) { + Ordering::Less => Ordering::Less, + Ordering::Greater => Ordering::Greater, + Ordering::Equal => l_partition.cmp(&r_partition), + } + } +} 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/transform_partition_restore.rs similarity index 91% rename from src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_spill_reader.rs rename to src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_restore.rs index 300eb3edc6ff1..d304efe27e464 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_spill_reader.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_restore.rs @@ -28,7 +28,6 @@ use databend_common_pipeline_core::processors::OutputPort; use databend_common_pipeline_core::processors::Processor; use databend_common_pipeline_core::processors::ProcessorPtr; use opendal::Operator; -use tokio::sync::Semaphore; use crate::pipelines::processors::transforms::aggregator::AggregateMeta; use crate::pipelines::processors::transforms::aggregator::AggregatorParams; @@ -36,12 +35,11 @@ use crate::pipelines::processors::transforms::aggregator::SpilledPayload; type DeserializingMeta = (AggregateMeta, VecDeque>); -pub struct TransformSpillReader { +pub struct TransformPartitionRestore { input: Arc, output: Arc, operator: Operator, - semaphore: Arc, params: Arc, output_data: Option, reading_meta: Option, @@ -49,7 +47,7 @@ pub struct TransformSpillReader { } #[async_trait::async_trait] -impl Processor for TransformSpillReader { +impl Processor for TransformPartitionRestore { fn name(&self) -> String { String::from("TransformSpillReader") } @@ -132,7 +130,6 @@ impl Processor for TransformSpillReader { if let Some(block_meta) = self.reading_meta.take() { match &block_meta { AggregateMeta::SpilledPayload(payload) => { - let _guard = self.semaphore.acquire().await; let data = self .operator .read_with(&payload.location) @@ -140,12 +137,6 @@ impl Processor for TransformSpillReader { .await? .to_vec(); - // info!( - // "Read aggregate spill {} successfully, elapsed: {:?}", - // &payload.location, - // instant.elapsed() - // ); - self.deserializing_meta = Some((block_meta, VecDeque::from(vec![data]))); } _ => unreachable!(), @@ -156,19 +147,17 @@ impl Processor for TransformSpillReader { } } -impl TransformSpillReader { +impl TransformPartitionRestore { pub fn create( input: Arc, output: Arc, operator: Operator, - semaphore: Arc, params: Arc, ) -> Result { - Ok(ProcessorPtr::create(Box::new(TransformSpillReader { + Ok(ProcessorPtr::create(Box::new(TransformPartitionRestore { input, output, operator, - semaphore, params, output_data: None, reading_meta: None, From 715a50d3811b2d1eec8197243e9a6b24a3b841d9 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Thu, 3 Apr 2025 21:10:08 +0800 Subject: [PATCH 39/76] refactor(query): refactor aggreagte spill code --- .../pipelines/builders/builder_aggregate.rs | 2 +- .../aggregator/transform_partition_align.rs | 27 ++++++++++++++++--- .../aggregator/transform_partition_bucket.rs | 9 +++++-- .../transform_partition_dispatch.rs | 21 +++++++++++++-- .../transform_partition_exchange.rs | 2 +- 5 files changed, 52 insertions(+), 9 deletions(-) diff --git a/src/query/service/src/pipelines/builders/builder_aggregate.rs b/src/query/service/src/pipelines/builders/builder_aggregate.rs index 0879a258e7a60..e2a81b426cfed 100644 --- a/src/query/service/src/pipelines/builders/builder_aggregate.rs +++ b/src/query/service/src/pipelines/builders/builder_aggregate.rs @@ -204,7 +204,7 @@ impl PipelineBuilder { self.build_pipeline(&aggregate.input)?; self.enable_multiway_sort = old_value; - build_final_aggregate(&mut self.main_pipeline, params.clone()) + build_final_aggregate(self.ctx.clone(), &mut self.main_pipeline, params.clone()) } fn build_aggregator_params( diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_align.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_align.rs index 0f8b0fc235eb3..17c2019715a8e 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_align.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_align.rs @@ -28,13 +28,17 @@ use databend_common_expression::Payload; use databend_common_expression::PayloadFlushState; use databend_common_expression::ProbeState; use databend_common_pipeline_transforms::AccumulatingTransform; +use databend_common_pipeline_transforms::MemorySettings; +use crate::pipelines::memory_settings::MemorySettingsExt; use crate::pipelines::processors::transforms::aggregator::transform_partition_bucket::SINGLE_LEVEL_BUCKET_NUM; use crate::pipelines::processors::transforms::aggregator::AggregateMeta; use crate::pipelines::processors::transforms::aggregator::AggregatePayload; use crate::pipelines::processors::transforms::aggregator::AggregatorParams; +use crate::sessions::QueryContext; pub struct TransformPartitionAlign { + settings: MemorySettings, params: Arc, max_partition: usize, @@ -43,9 +47,11 @@ pub struct TransformPartitionAlign { } impl TransformPartitionAlign { - pub fn create(params: Arc) -> Result { + pub fn create(ctx: Arc, params: Arc) -> Result { + let settings = MemorySettings::from_aggregate_settings(&ctx)?; Ok(TransformPartitionAlign { params, + settings, max_partition: 0, working_partition: 0, partitions: Partitions::create(), @@ -200,6 +206,7 @@ impl TransformPartitionAlign { } } +#[async_trait::async_trait] impl AccumulatingTransform for TransformPartitionAlign { const NAME: &'static str = "TransformPartitionAlign"; @@ -231,8 +238,16 @@ impl AccumulatingTransform for TransformPartitionAlign { } fn need_spill(&self) -> bool { - // TODO: spill if need - false + self.settings.check_spill() + } + + fn prepare_spill_payload(&mut self) -> Result { + // self.partitions.data.f + Ok(false) + } + + async fn flush_spill_payload(&mut self) -> Result { + Ok(false) } } @@ -249,6 +264,12 @@ impl Partitions { } pub fn add_data(&mut self, meta: AggregateMeta, block: DataBlock) { + if matches!(&meta, AggregateMeta::AggregatePayload(v) if v.payload.len() == 0) + || matches!(&meta, AggregateMeta::InFlightPayload(_) if block.is_empty()) + { + return; + } + match self.data.entry(meta.get_partition()) { std::collections::btree_map::Entry::Vacant(v) => { v.insert(vec![(meta, block)]); 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 index 3e79f57d9bc94..d3771745b5fd8 100644 --- 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 @@ -29,10 +29,15 @@ use crate::pipelines::processors::transforms::aggregator::transform_partition_di use crate::pipelines::processors::transforms::aggregator::transform_partition_exchange::ExchangePartition; use crate::pipelines::processors::transforms::aggregator::transform_partition_resorting::ResortingPartition; use crate::pipelines::processors::transforms::aggregator::AggregatorParams; +use crate::sessions::QueryContext; pub static SINGLE_LEVEL_BUCKET_NUM: isize = -1; -pub fn build_final_aggregate(pipeline: &mut Pipeline, params: Arc) -> Result<()> { +pub fn build_final_aggregate( + ctx: Arc, + pipeline: &mut Pipeline, + params: Arc, +) -> Result<()> { let pipe_size = pipeline.output_len(); // 1. resorting partition @@ -43,7 +48,7 @@ pub fn build_final_aggregate(pipeline: &mut Pipeline, params: Arc { + self.sync_final_partition = true; + self.input.set_not_need_data(); + continue; + } + meta => { + self.current_data = Some(data_block.add_meta(Some(Box::new(meta)))?); + } + }; + } + self.input.set_need_data(); - return Ok(Event::NeedData); + break; } while !self.waiting_outputs.is_empty() && self.current_data.is_some() { diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_exchange.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_exchange.rs index d4b236bcfe9c4..43d5631863911 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_exchange.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_exchange.rs @@ -155,7 +155,7 @@ impl ExchangePartition { } impl Exchange for ExchangePartition { - const NAME: &'static str = "PartitionResorting"; + const NAME: &'static str = "AggregatePartitionExchange"; const MULTIWAY_SORT: bool = false; fn partition(&self, mut data_block: DataBlock, n: usize) -> Result> { From 404af38876ce900f910962b1018a0f5a2b5c0ba3 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Thu, 3 Apr 2025 23:04:07 +0800 Subject: [PATCH 40/76] refactor(query): refactor aggreagte spill code --- .../executor/processor_async_task.rs | 2 +- .../src/pipelines/pipeline_build_res.rs | 1 - .../transform_partition_dispatch.rs | 29 +++++++++---------- 3 files changed, 15 insertions(+), 17 deletions(-) diff --git a/src/query/service/src/pipelines/executor/processor_async_task.rs b/src/query/service/src/pipelines/executor/processor_async_task.rs index 75143362d36ee..b864e5dab8d60 100644 --- a/src/query/service/src/pipelines/executor/processor_async_task.rs +++ b/src/query/service/src/pipelines/executor/processor_async_task.rs @@ -139,7 +139,7 @@ impl ProcessorAsyncTask { let elapsed = start.elapsed(); let active_workers = queue_clone.active_workers(); match elapsed >= Duration::from_secs(200) - // && active_workers == 0 + && active_workers == 0 && !log_graph { false => { diff --git a/src/query/service/src/pipelines/pipeline_build_res.rs b/src/query/service/src/pipelines/pipeline_build_res.rs index 7d5819b4e6bad..615caf8216153 100644 --- a/src/query/service/src/pipelines/pipeline_build_res.rs +++ b/src/query/service/src/pipelines/pipeline_build_res.rs @@ -37,7 +37,6 @@ pub struct PipelineBuildResult { pub sources_pipelines: Vec, pub enable_multiway_sort: bool, - // pub exchange_injector: Arc, /// for local fragment data sharing pub builder_data: PipelineBuilderData, pub r_cte_scan_interpreters: Vec, diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_dispatch.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_dispatch.rs index 9c27d8c61810e..e59af05a13d85 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_dispatch.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_dispatch.rs @@ -129,6 +129,7 @@ impl Processor for TransformPartitionDispatch { } else if output.port.can_push() { if self.sync_final_partition { if self.sent_final_partition[*output_index] { + output.status = PortStatus::Idle; self.waiting_outputs_2.push_back(*output_index); self.synchronized_final_partition[*output_index] = true; } else { @@ -154,22 +155,20 @@ impl Processor for TransformPartitionDispatch { return Ok(Event::Finished); } - if let EventCause::Input(_) = &cause { - if !self.sync_final_partition && self.input.has_data() && self.current_data.is_none() { - let data_block = self.input.pull_data().unwrap()?; - let (meta, data_block) = Self::unpark_block(data_block)?; + if !self.sync_final_partition && self.input.has_data() && self.current_data.is_none() { + let data_block = self.input.pull_data().unwrap()?; + let (meta, data_block) = Self::unpark_block(data_block)?; - match meta { - AggregateMeta::FinalPartition(_) => { - self.sync_final_partition = true; - self.input.set_not_need_data(); - } - meta => { - self.input.set_need_data(); - self.current_data = Some(data_block.add_meta(Some(Box::new(meta)))?); - } - }; - } + match meta { + AggregateMeta::FinalPartition(_) => { + self.sync_final_partition = true; + self.input.set_not_need_data(); + } + meta => { + self.input.set_need_data(); + self.current_data = Some(data_block.add_meta(Some(Box::new(meta)))?); + } + }; } while self.sync_final_partition { From 48237d6ecb9631ac07296afe1d4c878818503213 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sat, 5 Apr 2025 15:00:51 +0800 Subject: [PATCH 41/76] refactor(query): refactor aggreagte spill code --- .../aggregator/aggregate_exchange_injector.rs | 2 +- .../transforms/aggregator/aggregate_meta.rs | 20 ++-- .../aggregator/transform_aggregate_final.rs | 99 ++++++++++--------- .../aggregator/transform_partition_align.rs | 6 +- .../aggregator/transform_partition_bucket.rs | 5 +- .../transform_partition_dispatch.rs | 13 +-- .../transform_partition_exchange.rs | 37 ++++--- .../aggregator/transform_partition_restore.rs | 2 +- 8 files changed, 101 insertions(+), 83 deletions(-) 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 5c91c2621275a..eea38704d82d7 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 @@ -156,7 +156,7 @@ impl Exchange for FlightExchange { assert!(MULTIWAY_SORT); assert_eq!(self.node_list_lookup.len(), n); match AggregateMeta::downcast_from(meta).unwrap() { - AggregateMeta::FinalPartition(_) => unreachable!(), + AggregateMeta::FinalPartition => unreachable!(), AggregateMeta::InFlightPayload(_) => unreachable!(), AggregateMeta::SpilledPayload(v) => { let mut blocks = Vec::with_capacity(n); 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 9f588382d1b0a..7ebf7f86ebbeb 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 @@ -148,7 +148,7 @@ pub enum AggregateMeta { SpilledPayload(SpilledPayload), AggregatePayload(AggregatePayload), InFlightPayload(InFlightPayload), - FinalPartition(Option>), + FinalPartition, } impl AggregateMeta { @@ -182,8 +182,8 @@ impl AggregateMeta { Box::new(AggregateMeta::SpilledPayload(payload)) } - pub fn create_final(data: Option>) -> BlockMetaInfoPtr { - Box::new(AggregateMeta::FinalPartition(data)) + pub fn create_final() -> BlockMetaInfoPtr { + Box::new(AggregateMeta::FinalPartition) } pub fn get_global_max_partition(&self) -> usize { @@ -191,7 +191,7 @@ impl AggregateMeta { AggregateMeta::SpilledPayload(v) => v.global_max_partition, AggregateMeta::AggregatePayload(v) => v.global_max_partition, AggregateMeta::InFlightPayload(v) => v.global_max_partition, - AggregateMeta::FinalPartition(_) => unreachable!(), + AggregateMeta::FinalPartition => unreachable!(), } } @@ -200,7 +200,7 @@ impl AggregateMeta { AggregateMeta::SpilledPayload(v) => v.partition, AggregateMeta::AggregatePayload(v) => v.partition, AggregateMeta::InFlightPayload(v) => v.partition, - AggregateMeta::FinalPartition(_) => unreachable!(), + AggregateMeta::FinalPartition => unreachable!(), } } @@ -209,7 +209,7 @@ impl AggregateMeta { AggregateMeta::AggregatePayload(v) => v.partition, AggregateMeta::InFlightPayload(v) => v.partition, AggregateMeta::SpilledPayload(v) => v.get_sorting_partition(), - AggregateMeta::FinalPartition(_) => unreachable!(), + AggregateMeta::FinalPartition => unreachable!(), } } @@ -218,7 +218,7 @@ impl AggregateMeta { AggregateMeta::SpilledPayload(v) => v.max_partition, AggregateMeta::AggregatePayload(v) => v.max_partition, AggregateMeta::InFlightPayload(v) => v.max_partition, - AggregateMeta::FinalPartition(_) => unreachable!(), + AggregateMeta::FinalPartition => unreachable!(), } } @@ -233,7 +233,7 @@ impl AggregateMeta { AggregateMeta::InFlightPayload(v) => { v.global_max_partition = global_max_partition; } - AggregateMeta::FinalPartition(_) => unreachable!(), + AggregateMeta::FinalPartition => unreachable!(), } } } @@ -241,8 +241,8 @@ impl AggregateMeta { impl Debug for AggregateMeta { fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { match self { - AggregateMeta::FinalPartition(_) => { - f.debug_struct("AggregateMeta::Partitioned").finish() + AggregateMeta::FinalPartition => { + f.debug_struct("AggregateMeta::FinalPartition").finish() } AggregateMeta::SpilledPayload(_) => { f.debug_struct("Aggregate::SpilledPayload").finish() 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 d9bb08e165cff..7f121f3bc3932 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 @@ -40,8 +40,7 @@ pub struct TransformFinalAggregate { hash_table: AggregateHashTable, has_output: bool, - totals_upstream: usize, - recv_final_payload: usize, + working_partition: isize, } impl AccumulatingTransform for TransformFinalAggregate { @@ -49,18 +48,29 @@ impl AccumulatingTransform for TransformFinalAggregate { fn transform(&mut self, mut data: DataBlock) -> Result> { let Some(meta) = data.take_meta() else { - return Err(ErrorCode::Internal("")); + return Err(ErrorCode::Internal( + "Internal, TransformFinalAggregate only recv DataBlock with meta.", + )); }; let Some(aggregate_meta) = AggregateMeta::downcast_from(meta) else { - return Err(ErrorCode::Internal("")); + return Err(ErrorCode::Internal( + "Internal, TransformFinalAggregate only recv DataBlock with meta.", + )); }; + let mut flush_blocks = vec![]; + match aggregate_meta { AggregateMeta::SpilledPayload(_) => unreachable!(), + AggregateMeta::FinalPartition => {} AggregateMeta::InFlightPayload(payload) => { debug_assert_eq!(payload.max_partition, payload.global_max_partition); + if self.working_partition != payload.partition && self.hash_table.len() != 0 { + flush_blocks = self.flush_result_blocks()?; + } + if !data.is_empty() { let payload = self.deserialize_flight(data)?; @@ -71,63 +81,39 @@ impl AccumulatingTransform for TransformFinalAggregate { AggregateMeta::AggregatePayload(payload) => { debug_assert_eq!(payload.max_partition, payload.global_max_partition); + if self.working_partition != payload.partition && self.hash_table.len() != 0 { + flush_blocks = self.flush_result_blocks()?; + } + if payload.payload.len() != 0 { self.hash_table .combine_payload(&payload.payload, &mut self.flush_state)?; } } - AggregateMeta::FinalPartition(_) => { - self.recv_final_payload += 1; - - if self.hash_table.len() == 0 { - return Ok(vec![]); - } - - // Due to the local shuffle, we will receive the same number of final partitions as the upstream. - // We must wait for all final partitions to arrive before we can flush out the results. - if self.recv_final_payload % self.totals_upstream != 0 { - return Ok(vec![]); - } - - let mut blocks = vec![]; - self.flush_state.clear(); - - while self.hash_table.merge_result(&mut self.flush_state)? { - let mut cols = self.flush_state.take_aggregate_results(); - cols.extend_from_slice(&self.flush_state.take_group_columns()); - blocks.push(DataBlock::new_from_columns(cols)); - } - - let config = HashTableConfig::default().with_initial_radix_bits(0); - self.hash_table = AggregateHashTable::new( - self.params.group_data_types.clone(), - self.params.aggregate_functions.clone(), - config, - Arc::new(Bump::new()), - ); - - self.has_output |= !blocks.is_empty(); - return Ok(blocks); - } } - Ok(vec![]) + Ok(flush_blocks) } fn on_finish(&mut self, output: bool) -> Result> { - assert!(!output || self.hash_table.len() == 0); - - if output && !self.has_output { - return Ok(vec![self.params.empty_result_block()]); + if !output { + return Ok(vec![]); } - Ok(vec![]) + let flush_blocks = match self.hash_table.len() == 0 { + true => vec![], + false => self.flush_result_blocks()?, + }; + + match self.has_output { + true => Ok(flush_blocks), + false => Ok(vec![self.params.empty_result_block()]), + } } } impl TransformFinalAggregate { pub fn try_create( - totals_upstream: usize, input: Arc, output: Arc, params: Arc, @@ -149,8 +135,7 @@ impl TransformFinalAggregate { hash_table, flush_state: PayloadFlushState::default(), has_output: false, - totals_upstream, - recv_final_payload: 0, + working_partition: 0, }, )) } @@ -192,4 +177,26 @@ impl TransformFinalAggregate { assert_eq!(hashtable.payload.payloads.len(), 1); Ok(hashtable.payload.payloads.pop().unwrap()) } + + fn flush_result_blocks(&mut self) -> Result> { + let mut blocks = vec![]; + self.flush_state.clear(); + + while self.hash_table.merge_result(&mut self.flush_state)? { + let mut cols = self.flush_state.take_aggregate_results(); + cols.extend_from_slice(&self.flush_state.take_group_columns()); + blocks.push(DataBlock::new_from_columns(cols)); + } + + let config = HashTableConfig::default().with_initial_radix_bits(0); + self.hash_table = AggregateHashTable::new( + self.params.group_data_types.clone(), + self.params.aggregate_functions.clone(), + config, + Arc::new(Bump::new()), + ); + + self.has_output |= !blocks.is_empty(); + Ok(blocks) + } } diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_align.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_align.rs index 17c2019715a8e..c672a55623a3d 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_align.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_align.rs @@ -77,9 +77,7 @@ impl TransformPartitionAlign { ready_data.push(data_block.add_meta(Some(Box::new(meta)))?); } - ready_data.push(DataBlock::empty_with_meta(AggregateMeta::create_final( - None, - ))); + ready_data.push(DataBlock::empty_with_meta(AggregateMeta::create_final())); return Ok(ready_data); } @@ -104,7 +102,7 @@ impl TransformPartitionAlign { fn repartition(&mut self, meta: AggregateMeta, data_block: DataBlock) -> Result<()> { match meta { - AggregateMeta::FinalPartition(_) => unreachable!(), + AggregateMeta::FinalPartition => unreachable!(), AggregateMeta::SpilledPayload(_payload) => unreachable!(), AggregateMeta::InFlightPayload(payload) => { if data_block.is_empty() { 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 index d3771745b5fd8..81746b8e6fc64 100644 --- 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 @@ -14,6 +14,7 @@ use std::sync::Arc; +use databend_common_catalog::table_context::TableContext; use databend_common_exception::Result; use databend_common_pipeline_core::processors::ProcessorPtr; use databend_common_pipeline_core::Pipe; @@ -38,7 +39,8 @@ pub fn build_final_aggregate( pipeline: &mut Pipeline, params: Arc, ) -> Result<()> { - let pipe_size = pipeline.output_len(); + let settings = ctx.get_settings(); + let pipe_size = settings.get_max_threads()? as usize; // 1. resorting partition pipeline.exchange(1, Arc::new(ResortingPartition::create())); @@ -77,7 +79,6 @@ pub fn build_final_aggregate( // 6. final aggregate pipeline.add_transform(|input, output| { Ok(ProcessorPtr::create(TransformFinalAggregate::try_create( - pipe_size, input.clone(), output.clone(), params.clone(), diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_dispatch.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_dispatch.rs index e59af05a13d85..771441b960d00 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_dispatch.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_dispatch.rs @@ -118,6 +118,7 @@ impl Processor for TransformPartitionDispatch { #[allow(clippy::collapsible_if)] fn event_with_cause(&mut self, cause: EventCause) -> Result { + // eprintln!("TransformPartitionDispatch cause: {:?}, sync_final_partition:{:?}, sent_final_partition: {:?}, synchronized_final_partition: {:?}, waiting_outputs: {:?}, waiting_outputs_2: {:?}", cause, self.sync_final_partition, self.sent_final_partition, self.synchronized_final_partition, self.waiting_outputs, self.waiting_outputs_2); if let EventCause::Output(output_index) = &cause { let output = &mut self.outputs[*output_index]; @@ -135,7 +136,7 @@ impl Processor for TransformPartitionDispatch { } else { self.sent_final_partition[*output_index] = true; output.port.push_data(Ok(DataBlock::empty_with_meta( - AggregateMeta::create_final(None), + AggregateMeta::create_final(), ))); } } else if output.status != PortStatus::NeedData { @@ -160,7 +161,7 @@ impl Processor for TransformPartitionDispatch { let (meta, data_block) = Self::unpark_block(data_block)?; match meta { - AggregateMeta::FinalPartition(_) => { + AggregateMeta::FinalPartition => { self.sync_final_partition = true; self.input.set_not_need_data(); } @@ -184,9 +185,9 @@ impl Processor for TransformPartitionDispatch { self.outputs[output_index] .port - .push_data(Ok(DataBlock::empty_with_meta(AggregateMeta::create_final( - None, - )))); + .push_data(Ok( + DataBlock::empty_with_meta(AggregateMeta::create_final()), + )); self.sent_final_partition[output_index] = true; self.outputs[output_index].status = PortStatus::Idle; } @@ -207,7 +208,7 @@ impl Processor for TransformPartitionDispatch { let (meta, data_block) = Self::unpark_block(data_block)?; match meta { - AggregateMeta::FinalPartition(_) => { + AggregateMeta::FinalPartition => { self.sync_final_partition = true; self.input.set_not_need_data(); continue; diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_exchange.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_exchange.rs index 43d5631863911..87c732f08a152 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_exchange.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_exchange.rs @@ -15,6 +15,7 @@ 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::BlockMetaInfoDowncast; @@ -31,6 +32,8 @@ use crate::pipelines::processors::transforms::aggregator::AggregatePayload; use crate::pipelines::processors::transforms::aggregator::AggregatorParams; use crate::pipelines::processors::transforms::aggregator::InFlightPayload; +const HASH_SEED: u64 = 9263883436177860930; + pub struct ExchangePartition { params: Arc, } @@ -44,14 +47,15 @@ impl ExchangePartition { impl ExchangePartition { fn partition_final_payload(n: usize) -> Result> { Ok((0..n) - .map(|_| DataBlock::empty_with_meta(AggregateMeta::create_final(None))) + .map(|_| DataBlock::empty_with_meta(AggregateMeta::create_final())) .collect()) } - fn partition_aggregate_payload( - mut payload: AggregatePayload, - n: usize, - ) -> Result> { + fn partition_aggregate(mut payload: AggregatePayload, n: usize) -> Result> { + if payload.payload.len() == 0 { + return Ok(vec![]); + } + let mut repartition_payloads = Vec::with_capacity(n); let group_types = payload.payload.group_types.clone(); @@ -70,7 +74,7 @@ impl ExchangePartition { // scatter each page of the payload. while payload .payload - .scatter_with_seed::<9263883436177860930>(&mut state, repartition_payloads.len()) + .scatter_with_seed::(&mut state, repartition_payloads.len()) { // copy to the corresponding bucket. for (idx, bucket) in repartition_payloads.iter_mut().enumerate() { @@ -108,6 +112,11 @@ impl ExchangePartition { n: usize, ) -> Result> { let rows_num = block.num_rows(); + + if rows_num == 0 { + return Ok(vec![]); + } + let group_len = self.params.group_data_types.len(); let mut state = ProbeState::default(); @@ -142,7 +151,7 @@ impl ExchangePartition { hashtable.payload.mark_min_cardinality(); assert_eq!(hashtable.payload.payloads.len(), 1); - Self::partition_aggregate_payload( + Self::partition_aggregate( AggregatePayload { partition: payload.partition, payload: hashtable.payload.payloads.pop().unwrap(), @@ -160,21 +169,23 @@ impl Exchange for ExchangePartition { fn partition(&self, mut data_block: DataBlock, n: usize) -> Result> { let Some(meta) = data_block.take_meta() else { - return Ok(vec![data_block]); + return Err(ErrorCode::Internal( + "AggregatePartitionExchange only recv AggregateMeta", + )); }; let Some(meta) = AggregateMeta::downcast_from(meta) else { - return Ok(vec![data_block]); + return Err(ErrorCode::Internal( + "AggregatePartitionExchange only recv AggregateMeta", + )); }; match meta { // already restore in upstream AggregateMeta::SpilledPayload(_) => unreachable!(), // broadcast final partition to downstream - AggregateMeta::FinalPartition(_) => Self::partition_final_payload(n), - AggregateMeta::AggregatePayload(payload) => { - Self::partition_aggregate_payload(payload, n) - } + AggregateMeta::FinalPartition => Self::partition_final_payload(n), + AggregateMeta::AggregatePayload(payload) => Self::partition_aggregate(payload, n), AggregateMeta::InFlightPayload(payload) => { self.partition_flight_payload(payload, data_block, n) } diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_restore.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_restore.rs index d304efe27e464..6cc9cab78642a 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_restore.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_restore.rs @@ -49,7 +49,7 @@ pub struct TransformPartitionRestore { #[async_trait::async_trait] impl Processor for TransformPartitionRestore { fn name(&self) -> String { - String::from("TransformSpillReader") + String::from("TransformPartitionRestore") } fn as_any(&mut self) -> &mut dyn Any { From 00f9096574eb43d35af26d0435c906e0fb7e3066 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sat, 5 Apr 2025 15:42:36 +0800 Subject: [PATCH 42/76] refactor(query): refactor aggreagte spill code --- .../aggregator/transform_aggregate_final.rs | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) 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 7f121f3bc3932..111a21b58f172 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 @@ -67,8 +67,12 @@ impl AccumulatingTransform for TransformFinalAggregate { AggregateMeta::InFlightPayload(payload) => { debug_assert_eq!(payload.max_partition, payload.global_max_partition); - if self.working_partition != payload.partition && self.hash_table.len() != 0 { - flush_blocks = self.flush_result_blocks()?; + if self.working_partition != payload.partition { + if self.hash_table.len() != 0 { + flush_blocks = self.flush_result_blocks()?; + } + + self.working_partition = payload.partition; } if !data.is_empty() { @@ -81,8 +85,12 @@ impl AccumulatingTransform for TransformFinalAggregate { AggregateMeta::AggregatePayload(payload) => { debug_assert_eq!(payload.max_partition, payload.global_max_partition); - if self.working_partition != payload.partition && self.hash_table.len() != 0 { - flush_blocks = self.flush_result_blocks()?; + if self.working_partition != payload.partition { + if self.hash_table.len() != 0 { + flush_blocks = self.flush_result_blocks()?; + } + + self.working_partition = payload.partition; } if payload.payload.len() != 0 { From e39e64b69c0bc19e8de8487a58a6079644b77a33 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sat, 5 Apr 2025 16:16:33 +0800 Subject: [PATCH 43/76] refactor(query): refactor aggreagte spill code --- .../transforms/aggregator/transform_partition_align.rs | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_align.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_align.rs index c672a55623a3d..3cc52027f8c7d 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_align.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_align.rs @@ -218,10 +218,11 @@ impl AccumulatingTransform for TransformPartitionAlign { return Ok(vec![]); } - let partition = meta.get_sorting_partition(); + let partition = meta.get_partition(); + let sorting_partition = meta.get_sorting_partition(); self.partitions.add_data(meta, data_block); - if partition > SINGLE_LEVEL_BUCKET_NUM && partition != self.working_partition { + if sorting_partition > SINGLE_LEVEL_BUCKET_NUM && partition != self.working_partition { let ready_partition = self.fetch_ready_partition()?; self.working_partition = partition; return Ok(ready_partition); From a6b2dc2dea7bb35879e66d48a2c8f9015a2a9cf9 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sat, 5 Apr 2025 16:56:45 +0800 Subject: [PATCH 44/76] refactor(query): refactor aggreagte spill code --- .../aggregator/transform_partition_align.rs | 25 ++++++++++++++++--- .../transforms/range_join/range_join_state.rs | 4 +-- 2 files changed, 23 insertions(+), 6 deletions(-) diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_align.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_align.rs index 3cc52027f8c7d..55cd1608d8fe4 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_align.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_align.rs @@ -218,11 +218,10 @@ impl AccumulatingTransform for TransformPartitionAlign { return Ok(vec![]); } - let partition = meta.get_partition(); - let sorting_partition = meta.get_sorting_partition(); + let partition = meta.get_sorting_partition(); self.partitions.add_data(meta, data_block); - if sorting_partition > SINGLE_LEVEL_BUCKET_NUM && partition != self.working_partition { + if partition > SINGLE_LEVEL_BUCKET_NUM && partition != self.working_partition { let ready_partition = self.fetch_ready_partition()?; self.working_partition = partition; return Ok(ready_partition); @@ -232,8 +231,26 @@ impl AccumulatingTransform for TransformPartitionAlign { } fn on_finish(&mut self, _output: bool) -> Result> { + let remain_size = self + .partitions + .data + .iter() + .map(|(_, x)| x.len()) + .sum::(); + + let mut remain_partitions = Vec::with_capacity(remain_size + self.partitions.data.len()); self.working_partition = self.max_partition as isize; - self.fetch_ready_partition() + + loop { + let ready_partition = self.fetch_ready_partition()?; + + if !ready_partition.is_empty() { + remain_partitions.extend(ready_partition); + continue; + } + + return Ok(remain_partitions); + } } fn need_spill(&self) -> bool { diff --git a/src/query/service/src/pipelines/processors/transforms/range_join/range_join_state.rs b/src/query/service/src/pipelines/processors/transforms/range_join/range_join_state.rs index c376b45706ebd..83a2aa153414f 100644 --- a/src/query/service/src/pipelines/processors/transforms/range_join/range_join_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/range_join/range_join_state.rs @@ -92,7 +92,7 @@ impl RangeJoinState { } pub(crate) fn sink_right(&self, block: DataBlock) -> Result<()> { - if !block.is_empty() { + if !block.is_empty() || block.get_meta().is_some() { // Sink block to right table let mut right_table = self.right_table.write(); right_table.push(block); @@ -102,7 +102,7 @@ impl RangeJoinState { } pub(crate) fn sink_left(&self, block: DataBlock) -> Result<()> { - if !block.is_empty() { + if !block.is_empty() || block.get_meta().is_some() { // Sink block to left table let mut left_table = self.left_table.write(); left_table.push(block); From 8ae9c4c4109ba982068d5b2dcc2a0330681c3826 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sat, 5 Apr 2025 17:54:56 +0800 Subject: [PATCH 45/76] refactor(query): refactor aggreagte spill code --- .../transforms/aggregator/transform_partition_align.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_align.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_align.rs index 55cd1608d8fe4..272037dcc1df3 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_align.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_align.rs @@ -234,8 +234,8 @@ impl AccumulatingTransform for TransformPartitionAlign { let remain_size = self .partitions .data - .iter() - .map(|(_, x)| x.len()) + .values() + .map(|x| x.len()) .sum::(); let mut remain_partitions = Vec::with_capacity(remain_size + self.partitions.data.len()); From b6b4fd98667eca262f917bf956290402eb5acc61 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sat, 5 Apr 2025 21:49:32 +0800 Subject: [PATCH 46/76] refactor(query): refactor aggreagte spill code --- tests/sqllogictests/suites/query/cte/basic_r_cte.test | 6 ++---- .../suites/query/window_function/window_bound.test | 2 +- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/tests/sqllogictests/suites/query/cte/basic_r_cte.test b/tests/sqllogictests/suites/query/cte/basic_r_cte.test index 1d4ce93efcd9d..60b9d6bae97a0 100644 --- a/tests/sqllogictests/suites/query/cte/basic_r_cte.test +++ b/tests/sqllogictests/suites/query/cte/basic_r_cte.test @@ -254,7 +254,7 @@ select concat('城市',rn::varchar) city from t1 where rn<=5; statement ok insert into train -select concat('G',row_number()over()::varchar),c1.city,c2.city, n from city c1, city c2, (select 600 n union select 800 union select 1200 union select 1600) a ; +select concat('G',row_number()over()::varchar),c1_city,c2_city, n from (SELECT c1.city as c1_city,c2.city as c2_city, n FROM city c1, city c2, (select 600 n union select 800 union select 1200 union select 1600) a order by c1.city,c2.city, n); statement ok insert into passenger @@ -281,10 +281,8 @@ select from t0,(select 1 n union all select 2); ---- -261700 523200 210000 +224100 448000 210000 statement ok use default; -statement ok -drop database db; diff --git a/tests/sqllogictests/suites/query/window_function/window_bound.test b/tests/sqllogictests/suites/query/window_function/window_bound.test index b9f7b17571ed4..a12957584be0d 100644 --- a/tests/sqllogictests/suites/query/window_function/window_bound.test +++ b/tests/sqllogictests/suites/query/window_function/window_bound.test @@ -276,7 +276,7 @@ FROM range(100, 12000000, 467); (861,0,0) (849,1,1) -query II +query II rowsort SELECT DISTINCT lead((861, FALSE, FALSE), 9, (849, TRUE, TRUE)) OVER ( PARTITION BY 15560425903542832284, 965871850213131579 ORDER BY 13746504519650342222, 5897530378272856518 ASC NULLS FIRST) From b8d280d166f806776f0553262ea784838dce3a0c Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sat, 5 Apr 2025 22:41:59 +0800 Subject: [PATCH 47/76] refactor(query): refactor aggreagte spill code --- .../query/window_function/window_bound.test | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/tests/sqllogictests/suites/query/window_function/window_bound.test b/tests/sqllogictests/suites/query/window_function/window_bound.test index a12957584be0d..763e2bd51507a 100644 --- a/tests/sqllogictests/suites/query/window_function/window_bound.test +++ b/tests/sqllogictests/suites/query/window_function/window_bound.test @@ -267,23 +267,23 @@ SELECT a, DENSE_RANK() OVER (ORDER BY a ROWS BETWEEN UNBOUNDED PRECEDING AND UNB 6 4 7 5 -query I -SELECT DISTINCT lead((861, FALSE, FALSE), 9, (849, TRUE, TRUE)) OVER ( +query II +SELECT * FROM (SELECT DISTINCT lead((861, FALSE, FALSE), 9, (849, TRUE, TRUE)) OVER ( PARTITION BY 15560425903542832284, 965871850213131579 - ORDER BY 13746504519650342222, 5897530378272856518 ASC NULLS FIRST) -FROM range(100, 12000000, 467); + ORDER BY 13746504519650342222, 5897530378272856518 ASC NULLS FIRST) AS C +FROM range(100, 12000000, 467)) ORDER BY C.1; ---- -(861,0,0) (849,1,1) +(861,0,0) -query II rowsort -SELECT DISTINCT lead((861, FALSE, FALSE), 9, (849, TRUE, TRUE)) OVER ( +query II +SELECT * FROM (SELECT DISTINCT lead((861, FALSE, FALSE), 9, (849, TRUE, TRUE)) OVER ( PARTITION BY 15560425903542832284, 965871850213131579 - ORDER BY 13746504519650342222, 5897530378272856518 ASC NULLS FIRST) -FROM range(100, 120000000, 467); + ORDER BY 13746504519650342222, 5897530378272856518 ASC NULLS FIRST) AS C +FROM range(100, 120000000, 467)) ORDER BY C.1; ---- -(861,0,0) (849,1,1) +(861,0,0) statement ok DROP DATABASE test_window_bound; From dcbcd53eaacba4f0587606302835aa8a401d67d2 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sun, 6 Apr 2025 15:24:09 +0800 Subject: [PATCH 48/76] refactor(query): refactor aggreagte spill code --- .../aggregator/transform_aggregate_final.rs | 26 ++++++- .../aggregator/transform_partition_bucket.rs | 75 +++++++++++++++++++ 2 files changed, 100 insertions(+), 1 deletion(-) 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 111a21b58f172..2b3e920d7c713 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 @@ -41,6 +41,8 @@ pub struct TransformFinalAggregate { has_output: bool, working_partition: isize, + recv_final: usize, + init: bool, } impl AccumulatingTransform for TransformFinalAggregate { @@ -63,11 +65,21 @@ impl AccumulatingTransform for TransformFinalAggregate { match aggregate_meta { AggregateMeta::SpilledPayload(_) => unreachable!(), - AggregateMeta::FinalPartition => {} + AggregateMeta::FinalPartition => { + self.recv_final += 1; + } AggregateMeta::InFlightPayload(payload) => { + debug_assert!(payload.partition >= self.working_partition); debug_assert_eq!(payload.max_partition, payload.global_max_partition); if self.working_partition != payload.partition { + assert!( + !self.init || self.recv_final > 0, + "payload partition: {}, working partition: {}", + payload.partition, + self.working_partition + ); + self.recv_final = 0; if self.hash_table.len() != 0 { flush_blocks = self.flush_result_blocks()?; } @@ -75,6 +87,7 @@ impl AccumulatingTransform for TransformFinalAggregate { self.working_partition = payload.partition; } + self.init = true; if !data.is_empty() { let payload = self.deserialize_flight(data)?; @@ -83,9 +96,17 @@ impl AccumulatingTransform for TransformFinalAggregate { } } AggregateMeta::AggregatePayload(payload) => { + debug_assert!(payload.partition >= self.working_partition); debug_assert_eq!(payload.max_partition, payload.global_max_partition); if self.working_partition != payload.partition { + assert!( + !self.init || self.recv_final > 0, + "payload partition: {}, working partition: {}", + payload.partition, + self.working_partition + ); + self.recv_final = 0; if self.hash_table.len() != 0 { flush_blocks = self.flush_result_blocks()?; } @@ -93,6 +114,7 @@ impl AccumulatingTransform for TransformFinalAggregate { self.working_partition = payload.partition; } + self.init = true; if payload.payload.len() != 0 { self.hash_table .combine_payload(&payload.payload, &mut self.flush_state)?; @@ -144,6 +166,8 @@ impl TransformFinalAggregate { flush_state: PayloadFlushState::default(), has_output: false, working_partition: 0, + recv_final: 0, + init: false, }, )) } 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 index 81746b8e6fc64..bb3a2d7d132e0 100644 --- 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 @@ -16,13 +16,20 @@ use std::sync::Arc; use databend_common_catalog::table_context::TableContext; use databend_common_exception::Result; +use databend_common_expression::BlockMetaInfoDowncast; +use databend_common_expression::DataBlock; +use databend_common_pipeline_core::processors::InputPort; +use databend_common_pipeline_core::processors::OutputPort; 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_pipeline_transforms::AccumulatingTransformer; +use databend_common_pipeline_transforms::Transform; +use databend_common_pipeline_transforms::Transformer; use databend_common_storage::DataOperator; +use super::AggregateMeta; use super::TransformFinalAggregate; use super::TransformPartitionRestore; use crate::pipelines::processors::transforms::aggregator::transform_partition_align::TransformPartitionAlign; @@ -54,6 +61,10 @@ pub fn build_final_aggregate( ))) })?; + pipeline.add_transform(|input, output| { + CheckPartition::create(input, output, String::from("after align")) + })?; + // 3. dispatch partition let processor = TransformPartitionDispatch::create(pipe_size); let inputs_port = processor.get_inputs(); @@ -66,16 +77,28 @@ pub fn build_final_aggregate( ), ])); + pipeline.add_transform(|input, output| { + CheckPartition::create(input, output, String::from("after dispatch")) + })?; + // 4. restore partition let operator = DataOperator::instance().spill_operator(); pipeline.add_transform(|input, output| { TransformPartitionRestore::create(input, output, operator.clone(), params.clone()) })?; + pipeline.add_transform(|input, output| { + CheckPartition::create(input, output, String::from("after restore")) + })?; + // 5. exchange local let pipe_size = pipeline.output_len(); pipeline.exchange(pipe_size, ExchangePartition::create(params.clone())); + pipeline.add_transform(|input, output| { + CheckPartition::create(input, output, String::from("after exchange")) + })?; + // 6. final aggregate pipeline.add_transform(|input, output| { Ok(ProcessorPtr::create(TransformFinalAggregate::try_create( @@ -85,3 +108,55 @@ pub fn build_final_aggregate( )?)) }) } + +pub struct CheckPartition { + name: String, + cur_partition: Option, +} + +impl CheckPartition { + pub fn create( + input: Arc, + output: Arc, + name: String, + ) -> Result { + Ok(ProcessorPtr::create(Transformer::create( + input, + output, + CheckPartition { + name, + cur_partition: None, + }, + ))) + } +} + +impl Transform for CheckPartition { + const NAME: &'static str = "CheckPartition"; + + fn transform(&mut self, data: DataBlock) -> Result { + let Some(meta) = data.get_meta() else { + unreachable!(); + }; + + let Some(meta) = AggregateMeta::downcast_ref_from(meta) else { + unreachable!(); + }; + + if let AggregateMeta::FinalPartition = meta { + self.cur_partition = None; + return Ok(data); + } + + let partition = meta.get_partition(); + assert!( + self.cur_partition.is_none() || matches!(self.cur_partition, Some(v) if v == partition), + "{:?} assert failure partition({}) != current_partition({:?})", + self.name, + partition, + self.cur_partition + ); + self.cur_partition = Some(partition); + Ok(data) + } +} From 9dc9d1d26251374d55834ab6e0d5790a31b2b8cd Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sun, 6 Apr 2025 19:29:56 +0800 Subject: [PATCH 49/76] refactor(query): refactor aggreagte spill code --- .../core/src/processors/shuffle_processor.rs | 101 +++++++++++++++--- 1 file changed, 86 insertions(+), 15 deletions(-) diff --git a/src/query/pipeline/core/src/processors/shuffle_processor.rs b/src/query/pipeline/core/src/processors/shuffle_processor.rs index f9997e28736df..ff842729e45fe 100644 --- a/src/query/pipeline/core/src/processors/shuffle_processor.rs +++ b/src/query/pipeline/core/src/processors/shuffle_processor.rs @@ -14,6 +14,7 @@ use std::any::Any; use std::cmp::Ordering; +use std::collections::VecDeque; use std::sync::Arc; use databend_common_base::base::tokio::sync::Barrier; @@ -332,11 +333,23 @@ impl Processor for PartitionProcessor { } } +#[derive(Clone, PartialEq)] +enum PortStatus { + Idle, + HasData, + Finished, +} + pub struct MergePartitionProcessor { output: Arc, inputs: Vec>, inputs_data: Vec>, exchange: Arc, + + initialize: bool, + finished_inputs: usize, + waiting_inputs: VecDeque, + inputs_status: Vec, } impl MergePartitionProcessor { @@ -346,11 +359,18 @@ impl MergePartitionProcessor { exchange: Arc, ) -> ProcessorPtr { let inputs_data = vec![None; inputs.len()]; + let inputs_status = vec![PortStatus::Idle; inputs.len()]; + let waiting_inputs = VecDeque::with_capacity(inputs.len()); + ProcessorPtr::create(Box::new(MergePartitionProcessor:: { output, inputs, inputs_data, exchange, + inputs_status, + waiting_inputs, + initialize: false, + finished_inputs: 0, })) } } @@ -381,8 +401,7 @@ impl Processor for MergePartitionProcessor { } let mut all_inputs_finished = true; - let mut need_pick_block_to_push = T::MULTIWAY_SORT; - + let mut need_pick_block_to_push = true; for (index, input) in self.inputs.iter().enumerate() { if input.is_finished() { continue; @@ -390,19 +409,8 @@ impl Processor for MergePartitionProcessor { all_inputs_finished = false; - if input.has_data() { - match T::MULTIWAY_SORT { - false => { - if self.output.can_push() { - self.output.push_data(Ok(input.pull_data().unwrap()?)); - } - } - true => { - if self.inputs_data[index].is_none() { - self.inputs_data[index] = Some(input.pull_data().unwrap()?); - } - } - } + if input.has_data() && self.inputs_data[index].is_none() { + self.inputs_data[index] = Some(input.pull_data().unwrap()?); } if self.inputs_data[index].is_none() { @@ -428,4 +436,67 @@ impl Processor for MergePartitionProcessor { Ok(Event::NeedData) } + + fn event_with_cause(&mut self, cause: EventCause) -> Result { + if T::MULTIWAY_SORT { + return self.event(); + } + + if let EventCause::Output(_) = cause { + if self.output.is_finished() { + for input in &self.inputs { + input.finish(); + } + + return Ok(Event::Finished); + } + + if !self.output.can_push() { + return Ok(Event::NeedConsume); + } + + // if self.waiting_inputs.is_empty() { + // return Ok(Event::NeedData); + // } + } + + if !self.initialize && self.waiting_inputs.is_empty() { + self.initialize = true; + + for input in &self.inputs { + input.set_need_data(); + } + + return Ok(Event::NeedData); + } + + if let EventCause::Input(idx) = cause { + if self.inputs[idx].is_finished() && self.inputs_status[idx] != PortStatus::Finished { + self.finished_inputs += 1; + self.inputs_status[idx] = PortStatus::Finished; + } + + if self.inputs[idx].has_data() && self.inputs_status[idx] != PortStatus::HasData { + self.waiting_inputs.push_back(idx); + self.inputs_status[idx] = PortStatus::HasData; + } + } + + if self.finished_inputs == self.inputs.len() { + self.output.finish(); + return Ok(Event::Finished); + } + + while !self.waiting_inputs.is_empty() && self.output.can_push() { + let idx = self.waiting_inputs.pop_front().unwrap(); + self.output.push_data(self.inputs[idx].pull_data().unwrap()); + self.inputs_status[idx] = PortStatus::Idle; + self.inputs[idx].set_need_data(); + } + + match self.waiting_inputs.is_empty() { + true => Ok(Event::NeedData), + false => Ok(Event::NeedConsume), + } + } } From b90b6c11a423fdc6f30cc16582cd570a8e31a31c Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sun, 6 Apr 2025 20:22:46 +0800 Subject: [PATCH 50/76] refactor(query): refactor aggreagte spill code --- .../core/src/processors/shuffle_processor.rs | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/src/query/pipeline/core/src/processors/shuffle_processor.rs b/src/query/pipeline/core/src/processors/shuffle_processor.rs index ff842729e45fe..2195b89775f31 100644 --- a/src/query/pipeline/core/src/processors/shuffle_processor.rs +++ b/src/query/pipeline/core/src/processors/shuffle_processor.rs @@ -454,10 +454,6 @@ impl Processor for MergePartitionProcessor { if !self.output.can_push() { return Ok(Event::NeedConsume); } - - // if self.waiting_inputs.is_empty() { - // return Ok(Event::NeedData); - // } } if !self.initialize && self.waiting_inputs.is_empty() { @@ -491,6 +487,16 @@ impl Processor for MergePartitionProcessor { let idx = self.waiting_inputs.pop_front().unwrap(); self.output.push_data(self.inputs[idx].pull_data().unwrap()); self.inputs_status[idx] = PortStatus::Idle; + + if self.inputs[idx].is_finished() { + if self.inputs_status[idx] != PortStatus::Finished { + self.finished_inputs += 1; + self.inputs_status[idx] = PortStatus::Finished; + } + + continue; + } + self.inputs[idx].set_need_data(); } From 34d6770452981725e6460f2ea052708c2906c853 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 7 Apr 2025 09:38:35 +0800 Subject: [PATCH 51/76] refactor(query): refactor aggreagte spill code --- Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Cargo.toml b/Cargo.toml index a35c7f61d916e..4d2be41cf4802 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -589,7 +589,7 @@ debug = 1 lto = "thin" overflow-checks = false opt-level = "s" # defaults to be 3 -incremental = true +#incremental = true [profile.ci] inherits = "release" From b90f249b76d96c290613fc110ac192eb7e54ca02 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 8 Apr 2025 09:50:28 +0800 Subject: [PATCH 52/76] refactor(query): refactor aggreagte spill code --- Cargo.toml | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 15f5e2b4c2da7..3a63087e51845 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -585,11 +585,13 @@ result_large_err = "allow" # codegen-units = 1 [profile.release] -debug = 1 -lto = "thin" +debug = true +strip = "none" +lto = false overflow-checks = false -opt-level = "s" # defaults to be 3 -#incremental = true +opt-level = 0 # defaults to be 3 +codegen-units = 256 +incremental = false [profile.ci] inherits = "release" From 76eabfed677ff6957225e375212c4d7cf8ccebbf Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 8 Apr 2025 11:49:46 +0800 Subject: [PATCH 53/76] refactor(query): refactor aggreagte spill code --- .github/actions/build_linux/action.yml | 4 ++-- Cargo.toml | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/actions/build_linux/action.yml b/.github/actions/build_linux/action.yml index 66f4ed7950e7f..0ad9c83023c9d 100644 --- a/.github/actions/build_linux/action.yml +++ b/.github/actions/build_linux/action.yml @@ -98,8 +98,8 @@ runs: run: | objcopy --only-keep-debug ./target/${{ inputs.target }}/${{ env.BUILD_PROFILE }}/databend-query ./target/${{ inputs.target }}/${{ env.BUILD_PROFILE }}/databend-query.debug chmod 0644 ./target/${{ inputs.target }}/${{ env.BUILD_PROFILE }}/databend-query.debug - strip --strip-debug --remove-section=.comment --remove-section=.note ./target/${{ inputs.target }}/${{ env.BUILD_PROFILE }}/databend-query - pushd ./target/${{ inputs.target }}/${{ env.BUILD_PROFILE }} && objcopy --add-gnu-debuglink databend-query.debug databend-query && popd +# strip --strip-debug --remove-section=.comment --remove-section=.note ./target/${{ inputs.target }}/${{ env.BUILD_PROFILE }}/databend-query +# pushd ./target/${{ inputs.target }}/${{ env.BUILD_PROFILE }} && objcopy --add-gnu-debuglink databend-query.debug databend-query && popd # - name: Compress Binaries with UPX # if: env.BUILD_PROFILE == 'debug' diff --git a/Cargo.toml b/Cargo.toml index 3a63087e51845..96d5c911cd3b6 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -589,7 +589,7 @@ debug = true strip = "none" lto = false overflow-checks = false -opt-level = 0 # defaults to be 3 +opt-level = 0 # defaults to be 3 codegen-units = 256 incremental = false From 12ba91b5db3c13d3d92a0ff6daf6c6b82da269de Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 8 Apr 2025 12:27:32 +0800 Subject: [PATCH 54/76] refactor(query): refactor aggreagte spill code --- src/common/base/src/mem_allocator/default.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/common/base/src/mem_allocator/default.rs b/src/common/base/src/mem_allocator/default.rs index fca17168ce4d3..f3b0629d3f937 100644 --- a/src/common/base/src/mem_allocator/default.rs +++ b/src/common/base/src/mem_allocator/default.rs @@ -13,7 +13,7 @@ // limitations under the License. // Default allocator is jemalloc, you can change it to std: -#[cfg(feature = "jemalloc")] -pub type DefaultAllocator = crate::mem_allocator::JEAllocator; -#[cfg(not(feature = "jemalloc"))] +// #[cfg(feature = "jemalloc")] +// pub type DefaultAllocator = crate::mem_allocator::JEAllocator; +// #[cfg(not(feature = "jemalloc"))] pub type DefaultAllocator = crate::mem_allocator::StdAllocator; From 98b58d863cb8803304e6129e3831fa4ba78b5ac9 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 8 Apr 2025 16:04:38 +0800 Subject: [PATCH 55/76] refactor(query): refactor aggreagte spill code --- .github/actions/build_linux/action.yml | 4 ++-- Cargo.toml | 11 ++++------- src/common/base/src/mem_allocator/default.rs | 6 +++--- .../aggregator/transform_aggregate_partial.rs | 4 ++-- 4 files changed, 11 insertions(+), 14 deletions(-) diff --git a/.github/actions/build_linux/action.yml b/.github/actions/build_linux/action.yml index 0ad9c83023c9d..66f4ed7950e7f 100644 --- a/.github/actions/build_linux/action.yml +++ b/.github/actions/build_linux/action.yml @@ -98,8 +98,8 @@ runs: run: | objcopy --only-keep-debug ./target/${{ inputs.target }}/${{ env.BUILD_PROFILE }}/databend-query ./target/${{ inputs.target }}/${{ env.BUILD_PROFILE }}/databend-query.debug chmod 0644 ./target/${{ inputs.target }}/${{ env.BUILD_PROFILE }}/databend-query.debug -# strip --strip-debug --remove-section=.comment --remove-section=.note ./target/${{ inputs.target }}/${{ env.BUILD_PROFILE }}/databend-query -# pushd ./target/${{ inputs.target }}/${{ env.BUILD_PROFILE }} && objcopy --add-gnu-debuglink databend-query.debug databend-query && popd + strip --strip-debug --remove-section=.comment --remove-section=.note ./target/${{ inputs.target }}/${{ env.BUILD_PROFILE }}/databend-query + pushd ./target/${{ inputs.target }}/${{ env.BUILD_PROFILE }} && objcopy --add-gnu-debuglink databend-query.debug databend-query && popd # - name: Compress Binaries with UPX # if: env.BUILD_PROFILE == 'debug' diff --git a/Cargo.toml b/Cargo.toml index 96d5c911cd3b6..d217d582b50a9 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -585,13 +585,10 @@ result_large_err = "allow" # codegen-units = 1 [profile.release] -debug = true -strip = "none" -lto = false -overflow-checks = false -opt-level = 0 # defaults to be 3 -codegen-units = 256 -incremental = false +debug = 1 +lto = "thin" +opt-level = "s" # defaults to be 3 +#incremental = true [profile.ci] inherits = "release" diff --git a/src/common/base/src/mem_allocator/default.rs b/src/common/base/src/mem_allocator/default.rs index f3b0629d3f937..fca17168ce4d3 100644 --- a/src/common/base/src/mem_allocator/default.rs +++ b/src/common/base/src/mem_allocator/default.rs @@ -13,7 +13,7 @@ // limitations under the License. // Default allocator is jemalloc, you can change it to std: -// #[cfg(feature = "jemalloc")] -// pub type DefaultAllocator = crate::mem_allocator::JEAllocator; -// #[cfg(not(feature = "jemalloc"))] +#[cfg(feature = "jemalloc")] +pub type DefaultAllocator = crate::mem_allocator::JEAllocator; +#[cfg(not(feature = "jemalloc"))] pub type DefaultAllocator = crate::mem_allocator::StdAllocator; 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 553a1af91dc40..e92159a9c15d3 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 @@ -77,7 +77,7 @@ pub struct TransformPartialAggregate { settings: MemorySettings, configure_peer_nodes: Vec, spilling_state: Option, - spiller: Arc, + spiller: Spiller, output_blocks: Vec, } @@ -126,6 +126,7 @@ impl TransformPartialAggregate { output, TransformPartialAggregate { params, + spiller, hash_table, probe_state: ProbeState::default(), settings: MemorySettings::from_aggregate_settings(&ctx)?, @@ -135,7 +136,6 @@ impl TransformPartialAggregate { processed_rows: 0, configure_peer_nodes: vec![GlobalConfig::instance().query.node_id.clone()], spilling_state: None, - spiller: Arc::new(spiller), output_blocks: vec![], }, )) From b54d32a727a13881f8f4404c4de8368cac7fe4dc Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 8 Apr 2025 22:00:13 +0800 Subject: [PATCH 56/76] refactor(query): refactor aggreagte spill code --- .../src/aggregate/aggregate_hashtable.rs | 4 +-- src/query/expression/src/aggregate/mod.rs | 5 +-- .../src/aggregate/partitioned_payload.rs | 12 +++---- .../expression/src/aggregate/payload_row.rs | 8 ++--- .../expression/src/aggregate/probe_state.rs | 32 ++++++++++++++----- src/query/expression/src/lib.rs | 1 + .../hash_join/hash_join_probe_state.rs | 1 + src/query/sql/src/planner/plans/join.rs | 9 ++---- 8 files changed, 43 insertions(+), 29 deletions(-) diff --git a/src/query/expression/src/aggregate/aggregate_hashtable.rs b/src/query/expression/src/aggregate/aggregate_hashtable.rs index 3fae88f9f636b..0cf85106e336f 100644 --- a/src/query/expression/src/aggregate/aggregate_hashtable.rs +++ b/src/query/expression/src/aggregate/aggregate_hashtable.rs @@ -177,7 +177,7 @@ impl AggregateHashTable { row_count: usize, ) -> Result { state.row_count = row_count; - group_hash_columns(group_columns, &mut state.group_hashes); + group_hash_columns(group_columns, state.group_hashes.as_mut_slice()); let new_group_count = if self.direct_append { for idx in 0..row_count { @@ -337,7 +337,7 @@ impl AggregateHashTable { unsafe { row_match_columns( group_columns, - &state.addresses, + state.addresses.as_slice(), &mut state.group_compare_vector, &mut state.temp_vector, need_compare_count, diff --git a/src/query/expression/src/aggregate/mod.rs b/src/query/expression/src/aggregate/mod.rs index 6911a0efc3cf3..3f1c0696505dd 100644 --- a/src/query/expression/src/aggregate/mod.rs +++ b/src/query/expression/src/aggregate/mod.rs @@ -38,10 +38,11 @@ pub use payload::*; pub use payload_flush::*; pub use probe_state::*; -pub type SelectVector = [usize; BATCH_SIZE]; +pub type SelectVector = Box<[usize; BATCH_SIZE]>; pub fn new_sel() -> SelectVector { - [0; BATCH_SIZE] + let ptr = Box::into_raw(vec![0; BATCH_SIZE].into_boxed_slice()); + unsafe { Box::from_raw(ptr as *mut [usize; BATCH_SIZE]) } } // A batch size to probe, flush, repartition, etc. diff --git a/src/query/expression/src/aggregate/partitioned_payload.rs b/src/query/expression/src/aggregate/partitioned_payload.rs index e999e435fd13c..46b666020453a 100644 --- a/src/query/expression/src/aggregate/partitioned_payload.rs +++ b/src/query/expression/src/aggregate/partitioned_payload.rs @@ -130,9 +130,9 @@ impl PartitionedPayload { if self.payloads.len() == 1 { self.payloads[0].reserve_append_rows( &state.empty_vector, - &state.group_hashes, - &mut state.addresses, - &mut state.page_index, + state.group_hashes.as_slice(), + state.addresses.as_mut_slice(), + state.page_index.as_mut_slice(), new_group_rows, group_columns, ); @@ -157,9 +157,9 @@ impl PartitionedPayload { self.payloads[partition_index].reserve_append_rows( sel, - &state.group_hashes, - &mut state.addresses, - &mut state.page_index, + state.group_hashes.as_slice(), + state.addresses.as_mut_slice(), + state.page_index.as_mut_slice(), count, group_columns, ); diff --git a/src/query/expression/src/aggregate/payload_row.rs b/src/query/expression/src/aggregate/payload_row.rs index ce8b908e0b5ae..ee73c9142e30c 100644 --- a/src/query/expression/src/aggregate/payload_row.rs +++ b/src/query/expression/src/aggregate/payload_row.rs @@ -421,7 +421,7 @@ unsafe fn row_match_binary_column( } } - select_vector.clone_from_slice(temp_vector); + select_vector.clone_from_slice(temp_vector.as_slice()); *count = match_count; } @@ -502,7 +502,7 @@ unsafe fn row_match_string_column( } } - select_vector.clone_from_slice(temp_vector); + select_vector.clone_from_slice(temp_vector.as_slice()); *count = match_count; } @@ -567,7 +567,7 @@ unsafe fn row_match_column_type( } } - select_vector.clone_from_slice(temp_vector); + select_vector.clone_from_slice(temp_vector.as_slice()); *count = match_count; } @@ -604,6 +604,6 @@ unsafe fn row_match_generic_column( *no_match_count += 1; } } - select_vector.clone_from_slice(temp_vector); + select_vector.clone_from_slice(temp_vector.as_slice()); *count = match_count; } diff --git a/src/query/expression/src/aggregate/probe_state.rs b/src/query/expression/src/aggregate/probe_state.rs index 896c1ff46cca9..9925231933794 100644 --- a/src/query/expression/src/aggregate/probe_state.rs +++ b/src/query/expression/src/aggregate/probe_state.rs @@ -20,10 +20,10 @@ use crate::BATCH_SIZE; /// ProbeState is the state to probe HT /// It could be reuse during multiple probe process pub struct ProbeState { - pub group_hashes: [u64; BATCH_SIZE], - pub addresses: [*const u8; BATCH_SIZE], - pub page_index: [usize; BATCH_SIZE], - pub state_places: [StateAddr; BATCH_SIZE], + pub group_hashes: Box<[u64; BATCH_SIZE]>, + pub addresses: Box<[*const u8; BATCH_SIZE]>, + pub page_index: Box<[usize; BATCH_SIZE]>, + pub state_places: Box<[StateAddr; BATCH_SIZE]>, pub group_compare_vector: SelectVector, pub no_match_vector: SelectVector, pub empty_vector: SelectVector, @@ -34,13 +34,29 @@ pub struct ProbeState { pub partition_count: Vec, } +// https://github.com/rust-lang/rust/issues/53827#issuecomment-576450631 +macro_rules! box_array { + ($val:expr ; $len:expr) => {{ + // Use a generic function so that the pointer cast remains type-safe + fn vec_to_boxed_array(vec: Vec) -> Box<[T; $len]> { + let boxed_slice = vec.into_boxed_slice(); + + let ptr = ::std::boxed::Box::into_raw(boxed_slice) as *mut [T; $len]; + + unsafe { Box::from_raw(ptr) } + } + + vec_to_boxed_array(vec![$val; $len]) + }}; +} + impl Default for ProbeState { fn default() -> Self { Self { - group_hashes: [0_u64; BATCH_SIZE], - addresses: [std::ptr::null::(); BATCH_SIZE], - page_index: [0; BATCH_SIZE], - state_places: [StateAddr::new(0); BATCH_SIZE], + group_hashes: box_array!(0_u64; BATCH_SIZE), + addresses: box_array!(std::ptr::null::(); BATCH_SIZE), + page_index: box_array!(0; BATCH_SIZE), + state_places: box_array!(StateAddr::new(0); BATCH_SIZE), group_compare_vector: new_sel(), no_match_vector: new_sel(), empty_vector: new_sel(), diff --git a/src/query/expression/src/lib.rs b/src/query/expression/src/lib.rs index e402fe927d1a2..0de870bd811ff 100755 --- a/src/query/expression/src/lib.rs +++ b/src/query/expression/src/lib.rs @@ -43,6 +43,7 @@ #![feature(alloc_layout_extra)] #![feature(debug_closure_helpers)] #![feature(never_type)] +extern crate core; #[allow(dead_code)] mod block; 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 3cf2d0621770f..3feed32aadce9 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 @@ -541,6 +541,7 @@ impl HashJoinProbeState { } else { None }; + result_blocks.push(self.merge_eq_block( probe_block, build_block, diff --git a/src/query/sql/src/planner/plans/join.rs b/src/query/sql/src/planner/plans/join.rs index 1aaa2a001d8c6..9ca0d8f60cc06 100644 --- a/src/query/sql/src/planner/plans/join.rs +++ b/src/query/sql/src/planner/plans/join.rs @@ -610,11 +610,7 @@ impl Operator for Join { // Try to use broadcast join if !matches!( self.join_type, - JoinType::Right - | JoinType::Full - | JoinType::RightAnti - | JoinType::RightSemi - | JoinType::LeftMark + JoinType::Full | JoinType::RightAnti | JoinType::RightSemi | JoinType::LeftMark ) { let settings = ctx.get_settings(); let left_stat_info = rel_expr.derive_cardinality_child(0)?; @@ -686,8 +682,7 @@ impl Operator for Join { if !matches!( self.join_type, - JoinType::Right - | JoinType::Full + JoinType::Full | JoinType::RightAnti | JoinType::RightSemi | JoinType::LeftMark From 6a6bace4c696a3a73bab8a4301082f84e81d2f46 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 8 Apr 2025 22:51:37 +0800 Subject: [PATCH 57/76] refactor(query): refactor aggreagte spill code --- src/query/expression/src/aggregate/probe_state.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/query/expression/src/aggregate/probe_state.rs b/src/query/expression/src/aggregate/probe_state.rs index 9925231933794..5efe07a8b7547 100644 --- a/src/query/expression/src/aggregate/probe_state.rs +++ b/src/query/expression/src/aggregate/probe_state.rs @@ -80,8 +80,9 @@ impl ProbeState { pub fn reset_partitions(&mut self, partition_count: usize) { if self.partition_entries.len() < partition_count { - self.partition_entries.resize(partition_count, new_sel()); self.partition_count.resize(partition_count, 0); + self.partition_entries + .resize_with(partition_count, || new_sel()); } for i in 0..partition_count { From 1f809592ace7036061f678896a399f12970fafe6 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 8 Apr 2025 23:16:11 +0800 Subject: [PATCH 58/76] refactor(query): refactor aggreagte spill code --- src/query/expression/src/aggregate/probe_state.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/query/expression/src/aggregate/probe_state.rs b/src/query/expression/src/aggregate/probe_state.rs index 5efe07a8b7547..28ae43da7052c 100644 --- a/src/query/expression/src/aggregate/probe_state.rs +++ b/src/query/expression/src/aggregate/probe_state.rs @@ -81,8 +81,7 @@ impl ProbeState { pub fn reset_partitions(&mut self, partition_count: usize) { if self.partition_entries.len() < partition_count { self.partition_count.resize(partition_count, 0); - self.partition_entries - .resize_with(partition_count, || new_sel()); + self.partition_entries.resize_with(partition_count, new_sel); } for i in 0..partition_count { From 2d6bf9acd8668db392c500c26ff8ff760fba4611 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 9 Apr 2025 00:54:18 +0800 Subject: [PATCH 59/76] refactor(query): refactor aggreagte spill code --- src/query/expression/src/aggregate/mod.rs | 5 ++- .../expression/src/aggregate/probe_state.rs | 32 +++++-------------- 2 files changed, 10 insertions(+), 27 deletions(-) diff --git a/src/query/expression/src/aggregate/mod.rs b/src/query/expression/src/aggregate/mod.rs index 3f1c0696505dd..28a8a0a9eae0f 100644 --- a/src/query/expression/src/aggregate/mod.rs +++ b/src/query/expression/src/aggregate/mod.rs @@ -38,11 +38,10 @@ pub use payload::*; pub use payload_flush::*; pub use probe_state::*; -pub type SelectVector = Box<[usize; BATCH_SIZE]>; +pub type SelectVector = Vec; pub fn new_sel() -> SelectVector { - let ptr = Box::into_raw(vec![0; BATCH_SIZE].into_boxed_slice()); - unsafe { Box::from_raw(ptr as *mut [usize; BATCH_SIZE]) } + vec![0; BATCH_SIZE] } // A batch size to probe, flush, repartition, etc. diff --git a/src/query/expression/src/aggregate/probe_state.rs b/src/query/expression/src/aggregate/probe_state.rs index 28ae43da7052c..5b1cb702abb18 100644 --- a/src/query/expression/src/aggregate/probe_state.rs +++ b/src/query/expression/src/aggregate/probe_state.rs @@ -20,10 +20,10 @@ use crate::BATCH_SIZE; /// ProbeState is the state to probe HT /// It could be reuse during multiple probe process pub struct ProbeState { - pub group_hashes: Box<[u64; BATCH_SIZE]>, - pub addresses: Box<[*const u8; BATCH_SIZE]>, - pub page_index: Box<[usize; BATCH_SIZE]>, - pub state_places: Box<[StateAddr; BATCH_SIZE]>, + pub group_hashes: Vec, + pub addresses: Vec<*const u8>, + pub page_index: Vec, + pub state_places: Vec, pub group_compare_vector: SelectVector, pub no_match_vector: SelectVector, pub empty_vector: SelectVector, @@ -34,29 +34,13 @@ pub struct ProbeState { pub partition_count: Vec, } -// https://github.com/rust-lang/rust/issues/53827#issuecomment-576450631 -macro_rules! box_array { - ($val:expr ; $len:expr) => {{ - // Use a generic function so that the pointer cast remains type-safe - fn vec_to_boxed_array(vec: Vec) -> Box<[T; $len]> { - let boxed_slice = vec.into_boxed_slice(); - - let ptr = ::std::boxed::Box::into_raw(boxed_slice) as *mut [T; $len]; - - unsafe { Box::from_raw(ptr) } - } - - vec_to_boxed_array(vec![$val; $len]) - }}; -} - impl Default for ProbeState { fn default() -> Self { Self { - group_hashes: box_array!(0_u64; BATCH_SIZE), - addresses: box_array!(std::ptr::null::(); BATCH_SIZE), - page_index: box_array!(0; BATCH_SIZE), - state_places: box_array!(StateAddr::new(0); BATCH_SIZE), + group_hashes: vec![0_u64; BATCH_SIZE], + addresses: vec![std::ptr::null::(); BATCH_SIZE], + page_index: vec![0; BATCH_SIZE], + state_places: vec![StateAddr::new(0); BATCH_SIZE], group_compare_vector: new_sel(), no_match_vector: new_sel(), empty_vector: new_sel(), From b8fad43f2bbbbb752ff940f09316adf13fb1d3ba Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 9 Apr 2025 01:34:39 +0800 Subject: [PATCH 60/76] refactor(query): refactor aggreagte spill code --- src/query/sql/src/planner/plans/join.rs | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/src/query/sql/src/planner/plans/join.rs b/src/query/sql/src/planner/plans/join.rs index 9ca0d8f60cc06..1aaa2a001d8c6 100644 --- a/src/query/sql/src/planner/plans/join.rs +++ b/src/query/sql/src/planner/plans/join.rs @@ -610,7 +610,11 @@ impl Operator for Join { // Try to use broadcast join if !matches!( self.join_type, - JoinType::Full | JoinType::RightAnti | JoinType::RightSemi | JoinType::LeftMark + JoinType::Right + | JoinType::Full + | JoinType::RightAnti + | JoinType::RightSemi + | JoinType::LeftMark ) { let settings = ctx.get_settings(); let left_stat_info = rel_expr.derive_cardinality_child(0)?; @@ -682,7 +686,8 @@ impl Operator for Join { if !matches!( self.join_type, - JoinType::Full + JoinType::Right + | JoinType::Full | JoinType::RightAnti | JoinType::RightSemi | JoinType::LeftMark From bb7ca6986dc4b74bcfe5694757a85bd02a0225ee Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 9 Apr 2025 13:52:15 +0800 Subject: [PATCH 61/76] refactor(query): refactor aggreagte spill code --- .../core/src/processors/shuffle_processor.rs | 11 +++++++++-- .../aggregator/transform_partition_bucket.rs | 18 +++++++++++++++--- 2 files changed, 24 insertions(+), 5 deletions(-) diff --git a/src/query/pipeline/core/src/processors/shuffle_processor.rs b/src/query/pipeline/core/src/processors/shuffle_processor.rs index 2195b89775f31..4393f07a1d4a4 100644 --- a/src/query/pipeline/core/src/processors/shuffle_processor.rs +++ b/src/query/pipeline/core/src/processors/shuffle_processor.rs @@ -251,7 +251,7 @@ impl Processor for PartitionProcessor { } } - if self.partitioned_data[index].is_some() { + if !output.can_push() || self.partitioned_data[index].is_some() { all_data_pushed_output = false; } } @@ -349,6 +349,7 @@ pub struct MergePartitionProcessor { initialize: bool, finished_inputs: usize, waiting_inputs: VecDeque, + wakeup_inputs: VecDeque, inputs_status: Vec, } @@ -361,6 +362,7 @@ impl MergePartitionProcessor { let inputs_data = vec![None; inputs.len()]; let inputs_status = vec![PortStatus::Idle; inputs.len()]; let waiting_inputs = VecDeque::with_capacity(inputs.len()); + let wakeup_inputs = VecDeque::with_capacity(inputs.len()); ProcessorPtr::create(Box::new(MergePartitionProcessor:: { output, @@ -371,6 +373,7 @@ impl MergePartitionProcessor { waiting_inputs, initialize: false, finished_inputs: 0, + wakeup_inputs, })) } } @@ -454,6 +457,10 @@ impl Processor for MergePartitionProcessor { if !self.output.can_push() { return Ok(Event::NeedConsume); } + + while let Some(idx) = self.wakeup_inputs.pop_front() { + self.inputs[idx].set_need_data(); + } } if !self.initialize && self.waiting_inputs.is_empty() { @@ -497,7 +504,7 @@ impl Processor for MergePartitionProcessor { continue; } - self.inputs[idx].set_need_data(); + self.wakeup_inputs.push_back(idx); } match self.waiting_inputs.is_empty() { 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 index bb3a2d7d132e0..7ad0d7c79cd13 100644 --- 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 @@ -24,6 +24,7 @@ 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_pipeline_core::TransformPipeBuilder; use databend_common_pipeline_transforms::AccumulatingTransformer; use databend_common_pipeline_transforms::Transform; use databend_common_pipeline_transforms::Transformer; @@ -95,9 +96,20 @@ pub fn build_final_aggregate( let pipe_size = pipeline.output_len(); pipeline.exchange(pipe_size, ExchangePartition::create(params.clone())); - pipeline.add_transform(|input, output| { - CheckPartition::create(input, output, String::from("after exchange")) - })?; + let mut transform_builder = TransformPipeBuilder::create(); + for _index in 0..pipeline.output_len() { + let input = InputPort::create(); + let output = OutputPort::create(); + + let processor = CheckPartition::create( + input.clone(), + output.clone(), + format!("after exchange {}", _index), + )?; + transform_builder.add_transform(input, output, processor); + } + + pipeline.add_pipe(transform_builder.finalize()); // 6. final aggregate pipeline.add_transform(|input, output| { From 2be5e4a06ba3354e433d42e9c8815161f08ba78d Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 9 Apr 2025 15:38:23 +0800 Subject: [PATCH 62/76] refactor(query): refactor aggreagte spill code --- .../aggregator/transform_partition_bucket.rs | 79 ------------------- 1 file changed, 79 deletions(-) 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 index 7ad0d7c79cd13..372ea4ce5fa66 100644 --- 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 @@ -62,10 +62,6 @@ pub fn build_final_aggregate( ))) })?; - pipeline.add_transform(|input, output| { - CheckPartition::create(input, output, String::from("after align")) - })?; - // 3. dispatch partition let processor = TransformPartitionDispatch::create(pipe_size); let inputs_port = processor.get_inputs(); @@ -78,39 +74,16 @@ pub fn build_final_aggregate( ), ])); - pipeline.add_transform(|input, output| { - CheckPartition::create(input, output, String::from("after dispatch")) - })?; - // 4. restore partition let operator = DataOperator::instance().spill_operator(); pipeline.add_transform(|input, output| { TransformPartitionRestore::create(input, output, operator.clone(), params.clone()) })?; - pipeline.add_transform(|input, output| { - CheckPartition::create(input, output, String::from("after restore")) - })?; - // 5. exchange local let pipe_size = pipeline.output_len(); pipeline.exchange(pipe_size, ExchangePartition::create(params.clone())); - let mut transform_builder = TransformPipeBuilder::create(); - for _index in 0..pipeline.output_len() { - let input = InputPort::create(); - let output = OutputPort::create(); - - let processor = CheckPartition::create( - input.clone(), - output.clone(), - format!("after exchange {}", _index), - )?; - transform_builder.add_transform(input, output, processor); - } - - pipeline.add_pipe(transform_builder.finalize()); - // 6. final aggregate pipeline.add_transform(|input, output| { Ok(ProcessorPtr::create(TransformFinalAggregate::try_create( @@ -120,55 +93,3 @@ pub fn build_final_aggregate( )?)) }) } - -pub struct CheckPartition { - name: String, - cur_partition: Option, -} - -impl CheckPartition { - pub fn create( - input: Arc, - output: Arc, - name: String, - ) -> Result { - Ok(ProcessorPtr::create(Transformer::create( - input, - output, - CheckPartition { - name, - cur_partition: None, - }, - ))) - } -} - -impl Transform for CheckPartition { - const NAME: &'static str = "CheckPartition"; - - fn transform(&mut self, data: DataBlock) -> Result { - let Some(meta) = data.get_meta() else { - unreachable!(); - }; - - let Some(meta) = AggregateMeta::downcast_ref_from(meta) else { - unreachable!(); - }; - - if let AggregateMeta::FinalPartition = meta { - self.cur_partition = None; - return Ok(data); - } - - let partition = meta.get_partition(); - assert!( - self.cur_partition.is_none() || matches!(self.cur_partition, Some(v) if v == partition), - "{:?} assert failure partition({}) != current_partition({:?})", - self.name, - partition, - self.cur_partition - ); - self.cur_partition = Some(partition); - Ok(data) - } -} From 1551f0eef00affd0b506be01c5d978ca1d226c9e Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 9 Apr 2025 15:59:40 +0800 Subject: [PATCH 63/76] refactor(query): refactor aggreagte spill code --- .../transforms/aggregator/transform_partition_bucket.rs | 8 -------- .../aggregator/transform_partition_resorting.rs | 7 ++----- 2 files changed, 2 insertions(+), 13 deletions(-) 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 index 372ea4ce5fa66..81746b8e6fc64 100644 --- 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 @@ -16,21 +16,13 @@ use std::sync::Arc; use databend_common_catalog::table_context::TableContext; use databend_common_exception::Result; -use databend_common_expression::BlockMetaInfoDowncast; -use databend_common_expression::DataBlock; -use databend_common_pipeline_core::processors::InputPort; -use databend_common_pipeline_core::processors::OutputPort; 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_pipeline_core::TransformPipeBuilder; use databend_common_pipeline_transforms::AccumulatingTransformer; -use databend_common_pipeline_transforms::Transform; -use databend_common_pipeline_transforms::Transformer; use databend_common_storage::DataOperator; -use super::AggregateMeta; use super::TransformFinalAggregate; use super::TransformPartitionRestore; use crate::pipelines::processors::transforms::aggregator::transform_partition_align::TransformPartitionAlign; diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_resorting.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_resorting.rs index 8e02b93f0e6a1..00697bc99efe4 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_resorting.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_resorting.rs @@ -16,6 +16,7 @@ use std::cmp::Ordering; use std::sync::atomic::AtomicUsize; use std::sync::atomic::Ordering as AtomicOrdering; +use databend_common_exception::Result; use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::DataBlock; use databend_common_pipeline_core::processors::Exchange; @@ -42,11 +43,7 @@ impl Exchange for ResortingPartition { const NAME: &'static str = "PartitionResorting"; const MULTIWAY_SORT: bool = true; - fn partition( - &self, - mut data_block: DataBlock, - n: usize, - ) -> databend_common_exception::Result> { + fn partition(&self, mut data_block: DataBlock, n: usize) -> Result> { debug_assert_eq!(n, 1); let Some(meta) = data_block.take_meta() else { From c7bc7da5e5dcd3a4c1ddb12f5004fc5a64506ba4 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 9 Apr 2025 20:16:25 +0800 Subject: [PATCH 64/76] refactor(query): refactor aggreagte spill code --- src/query/pipeline/core/src/pipeline.rs | 24 +++-- src/query/pipeline/core/src/processors/mod.rs | 1 + .../core/src/processors/shuffle_processor.rs | 100 ++++++++++++++++++ .../transform_partition_dispatch.rs | 1 - 4 files changed, 118 insertions(+), 8 deletions(-) diff --git a/src/query/pipeline/core/src/pipeline.rs b/src/query/pipeline/core/src/pipeline.rs index 88807fbe8f1fa..801718d2e36e2 100644 --- a/src/query/pipeline/core/src/pipeline.rs +++ b/src/query/pipeline/core/src/pipeline.rs @@ -37,6 +37,7 @@ use crate::processors::DuplicateProcessor; use crate::processors::Exchange; use crate::processors::InputPort; use crate::processors::MergePartitionProcessor; +use crate::processors::OnePartitionProcessor; use crate::processors::OutputPort; use crate::processors::PartitionProcessor; use crate::processors::PlanScope; @@ -449,29 +450,38 @@ impl Pipeline { } pub fn exchange(&mut self, n: usize, exchange: Arc) { + debug_assert_ne!(n, 0); + if let Some(pipe) = self.pipes.last() { if pipe.output_length < 1 { return; } let input_len = pipe.output_length; + let barrier = Arc::new(Barrier::new(input_len)); let mut items = Vec::with_capacity(input_len); - let barrier = Arc::new(Barrier::new(input_len)); for index in 0..input_len { let input = InputPort::create(); - let outputs: Vec<_> = (0..n).map(|_| OutputPort::create()).collect(); - items.push(PipeItem::create( - PartitionProcessor::create( + let outputs = (0..n).map(|_| OutputPort::create()).collect::>(); + let partition_processor = match n { + 1 => OnePartitionProcessor::create( + input.clone(), + outputs[0].clone(), + exchange.clone(), + index, + barrier.clone(), + ), + _ => PartitionProcessor::create( input.clone(), outputs.clone(), exchange.clone(), index, barrier.clone(), ), - vec![input], - outputs, - )); + }; + + items.push(PipeItem::create(partition_processor, vec![input], outputs)); } // partition data block diff --git a/src/query/pipeline/core/src/processors/mod.rs b/src/query/pipeline/core/src/processors/mod.rs index 095a9d597be61..88baef3f1f5c0 100644 --- a/src/query/pipeline/core/src/processors/mod.rs +++ b/src/query/pipeline/core/src/processors/mod.rs @@ -40,5 +40,6 @@ pub use resize_processor::ResizeProcessor; pub use shuffle_processor::Exchange; pub use shuffle_processor::MergePartitionProcessor; pub use shuffle_processor::MultiwayStrategy; +pub use shuffle_processor::OnePartitionProcessor; pub use shuffle_processor::PartitionProcessor; pub use shuffle_processor::ShuffleProcessor; diff --git a/src/query/pipeline/core/src/processors/shuffle_processor.rs b/src/query/pipeline/core/src/processors/shuffle_processor.rs index 4393f07a1d4a4..6c8a77929facc 100644 --- a/src/query/pipeline/core/src/processors/shuffle_processor.rs +++ b/src/query/pipeline/core/src/processors/shuffle_processor.rs @@ -333,6 +333,106 @@ impl Processor for PartitionProcessor { } } +pub struct OnePartitionProcessor { + input: Arc, + output: Arc, + + exchange: Arc, + input_data: Option, + + index: usize, + initialized: bool, + barrier: Arc, +} + +impl OnePartitionProcessor { + pub fn create( + input: Arc, + outputs: Arc, + exchange: Arc, + index: usize, + barrier: Arc, + ) -> ProcessorPtr { + ProcessorPtr::create(Box::new(OnePartitionProcessor { + input, + output: outputs, + exchange, + input_data: None, + initialized: !T::MULTIWAY_SORT, + index, + barrier, + })) + } +} + +#[async_trait::async_trait] +impl Processor for OnePartitionProcessor { + fn name(&self) -> String { + format!("ShuffleOnePartition({})", T::NAME) + } + + fn as_any(&mut self) -> &mut dyn Any { + self + } + + fn event(&mut self) -> Result { + if self.output.is_finished() { + self.input.finish(); + + return match self.initialized { + true => Ok(Event::Finished), + false => Ok(Event::Async), + }; + } + + if !self.output.can_push() { + self.input.set_not_need_data(); + return Ok(Event::NeedConsume); + } + + if self.input_data.is_some() { + if !self.initialized { + return Ok(Event::Async); + } + + self.output.push_data(Ok(self.input_data.take().unwrap())); + return Ok(Event::NeedConsume); + } + + if self.input.has_data() { + if !self.initialized { + self.input_data = Some(self.input.pull_data().unwrap()?); + return Ok(Event::Async); + } + + self.output.push_data(self.input.pull_data().unwrap()); + return Ok(Event::NeedConsume); + } + + if self.input.is_finished() { + self.output.finish(); + + return match self.initialized { + true => Ok(Event::Finished), + false => Ok(Event::Async), + }; + } + + self.input.set_need_data(); + Ok(Event::NeedData) + } + + async fn async_process(&mut self) -> Result<()> { + self.initialized = true; + if let Some(data_block) = self.input_data.as_ref() { + self.exchange.init_way(self.index, data_block)?; + } + + self.barrier.wait().await; + Ok(()) + } +} + #[derive(Clone, PartialEq)] enum PortStatus { Idle, diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_dispatch.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_dispatch.rs index 771441b960d00..58149dbb1e5fe 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_dispatch.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_dispatch.rs @@ -118,7 +118,6 @@ impl Processor for TransformPartitionDispatch { #[allow(clippy::collapsible_if)] fn event_with_cause(&mut self, cause: EventCause) -> Result { - // eprintln!("TransformPartitionDispatch cause: {:?}, sync_final_partition:{:?}, sent_final_partition: {:?}, synchronized_final_partition: {:?}, waiting_outputs: {:?}, waiting_outputs_2: {:?}", cause, self.sync_final_partition, self.sent_final_partition, self.synchronized_final_partition, self.waiting_outputs, self.waiting_outputs_2); if let EventCause::Output(output_index) = &cause { let output = &mut self.outputs[*output_index]; From b06e8ef71c2be736c6bf182455b64aabe37b9097 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 9 Apr 2025 22:34:02 +0800 Subject: [PATCH 65/76] refactor(query): refactor aggreagte spill code --- .../aggregator/transform_partition_align.rs | 199 +++++++++++++----- .../aggregator/transform_partition_bucket.rs | 7 +- 2 files changed, 150 insertions(+), 56 deletions(-) diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_align.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_align.rs index 272037dcc1df3..91b64b292761c 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_align.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_align.rs @@ -12,7 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::any::Any; use std::collections::BTreeMap; +use std::collections::VecDeque; use std::sync::Arc; use bumpalo::Bump; @@ -27,7 +29,10 @@ use databend_common_expression::PartitionedPayload; use databend_common_expression::Payload; use databend_common_expression::PayloadFlushState; use databend_common_expression::ProbeState; -use databend_common_pipeline_transforms::AccumulatingTransform; +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_transforms::MemorySettings; use crate::pipelines::memory_settings::MemorySettingsExt; @@ -38,50 +43,66 @@ use crate::pipelines::processors::transforms::aggregator::AggregatorParams; use crate::sessions::QueryContext; pub struct TransformPartitionAlign { + input: Arc, + output: Arc, + + #[allow(dead_code)] settings: MemorySettings, params: Arc, max_partition: usize, working_partition: isize, partitions: Partitions, + + output_data: VecDeque, + input_data: Option<(AggregateMeta, DataBlock)>, } impl TransformPartitionAlign { - pub fn create(ctx: Arc, params: Arc) -> Result { + pub fn create( + ctx: Arc, + params: Arc, + input: Arc, + output: Arc, + ) -> Result { let settings = MemorySettings::from_aggregate_settings(&ctx)?; Ok(TransformPartitionAlign { + input, + output, params, settings, max_partition: 0, working_partition: 0, partitions: Partitions::create(), + input_data: None, + output_data: Default::default(), }) } fn ready_partition(&mut self) -> Option { let storage_min_partition = self.partitions.min_partition()?; - if storage_min_partition > self.working_partition { + if storage_min_partition >= self.working_partition { return None; } Some(storage_min_partition) } - fn fetch_ready_partition(&mut self) -> Result> { + fn fetch_ready_partition(&mut self) -> Result<()> { if let Some(ready_partition_id) = self.ready_partition() { let ready_partition = self.partitions.take_partition(ready_partition_id); - let mut ready_data = Vec::with_capacity(ready_partition.len()); for (meta, data_block) in ready_partition { - ready_data.push(data_block.add_meta(Some(Box::new(meta)))?); + self.output_data + .push_back(data_block.add_meta(Some(Box::new(meta)))?); } - ready_data.push(DataBlock::empty_with_meta(AggregateMeta::create_final())); - return Ok(ready_data); + self.output_data + .push_back(DataBlock::empty_with_meta(AggregateMeta::create_final())); } - Ok(vec![]) + Ok(()) } fn unpark_block(&self, mut data_block: DataBlock) -> Result<(AggregateMeta, DataBlock)> { @@ -204,69 +225,145 @@ impl TransformPartitionAlign { } } -#[async_trait::async_trait] -impl AccumulatingTransform for TransformPartitionAlign { - const NAME: &'static str = "TransformPartitionAlign"; +impl Processor for TransformPartitionAlign { + fn name(&self) -> String { + String::from("TransformPartitionAlign") + } - fn transform(&mut self, data_block: DataBlock) -> Result> { - let (meta, data_block) = self.unpark_block(data_block)?; - self.max_partition = meta.get_global_max_partition(); + fn as_any(&mut self) -> &mut dyn Any { + self + } - // need repartition - if meta.get_max_partition() != meta.get_global_max_partition() { - self.repartition(meta, data_block)?; - return Ok(vec![]); + fn event(&mut self) -> Result { + if self.output.is_finished() { + self.input.finish(); + return Ok(Event::Finished); } - let partition = meta.get_sorting_partition(); - self.partitions.add_data(meta, data_block); + if !self.output.can_push() { + self.input.set_not_need_data(); + return Ok(Event::NeedConsume); + } - if partition > SINGLE_LEVEL_BUCKET_NUM && partition != self.working_partition { - let ready_partition = self.fetch_ready_partition()?; - self.working_partition = partition; - return Ok(ready_partition); + if let Some(data_block) = self.output_data.pop_front() { + self.output.push_data(Ok(data_block)); + return Ok(Event::NeedConsume); } - Ok(vec![]) - } + if self.input.has_data() { + let data_block = self.input.pull_data().unwrap()?; - fn on_finish(&mut self, _output: bool) -> Result> { - let remain_size = self - .partitions - .data - .values() - .map(|x| x.len()) - .sum::(); + let (meta, data_block) = self.unpark_block(data_block)?; + self.max_partition = meta.get_global_max_partition(); - let mut remain_partitions = Vec::with_capacity(remain_size + self.partitions.data.len()); - self.working_partition = self.max_partition as isize; + // need repartition + if meta.get_max_partition() != meta.get_global_max_partition() { + self.input_data = Some((meta, data_block)); + return Ok(Event::Sync); + } - loop { - let ready_partition = self.fetch_ready_partition()?; + let partition = meta.get_sorting_partition(); + self.partitions.add_data(meta, data_block); - if !ready_partition.is_empty() { - remain_partitions.extend(ready_partition); - continue; + if partition > SINGLE_LEVEL_BUCKET_NUM && partition != self.working_partition { + self.working_partition = partition; } + } - return Ok(remain_partitions); + if self.input.is_finished() && self.working_partition as usize != self.max_partition { + self.working_partition = self.max_partition as isize; } - } - fn need_spill(&self) -> bool { - self.settings.check_spill() - } + if self.output_data.is_empty() { + self.fetch_ready_partition()?; + } - fn prepare_spill_payload(&mut self) -> Result { - // self.partitions.data.f - Ok(false) + if let Some(data_block) = self.output_data.pop_front() { + 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) } - async fn flush_spill_payload(&mut self) -> Result { - Ok(false) + fn process(&mut self) -> Result<()> { + if let Some((meta, data_block)) = self.input_data.take() { + self.repartition(meta, data_block)?; + } + + Ok(()) } } +// #[async_trait::async_trait] +// impl AccumulatingTransform for TransformPartitionAlign { +// const NAME: &'static str = "TransformPartitionAlign"; +// +// fn transform(&mut self, data_block: DataBlock) -> Result> { +// let (meta, data_block) = self.unpark_block(data_block)?; +// self.max_partition = meta.get_global_max_partition(); +// +// // need repartition +// if meta.get_max_partition() != meta.get_global_max_partition() { +// self.repartition(meta, data_block)?; +// return Ok(vec![]); +// } +// +// let partition = meta.get_sorting_partition(); +// self.partitions.add_data(meta, data_block); +// +// if partition > SINGLE_LEVEL_BUCKET_NUM && partition != self.working_partition { +// self.fetch_ready_partition()?; +// self.working_partition = partition; +// // return Ok(ready_partition); +// } +// +// Ok(vec![]) +// } +// +// fn on_finish(&mut self, _output: bool) -> Result> { +// let remain_size = self +// .partitions +// .data +// .values() +// .map(|x| x.len()) +// .sum::(); +// +// let mut remain_partitions = Vec::with_capacity(remain_size + self.partitions.data.len()); +// self.working_partition = self.max_partition as isize; +// +// loop { +// let ready_partition = self.fetch_ready_partition()?; +// +// if !ready_partition.is_empty() { +// remain_partitions.extend(ready_partition); +// continue; +// } +// +// return Ok(remain_partitions); +// } +// } +// +// fn need_spill(&self) -> bool { +// self.settings.check_spill() +// } +// +// fn prepare_spill_payload(&mut self) -> Result { +// // self.partitions.data.f +// Ok(false) +// } +// +// async fn flush_spill_payload(&mut self) -> Result { +// Ok(false) +// } +// } + #[derive(Debug)] struct Partitions { data: BTreeMap>, 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 index 81746b8e6fc64..0d6e98238c45f 100644 --- 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 @@ -20,7 +20,6 @@ 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_pipeline_transforms::AccumulatingTransformer; use databend_common_storage::DataOperator; use super::TransformFinalAggregate; @@ -47,10 +46,8 @@ pub fn build_final_aggregate( // 2. align partitions pipeline.add_transform(|input, output| { - Ok(ProcessorPtr::create(AccumulatingTransformer::create( - input, - output, - TransformPartitionAlign::create(ctx.clone(), params.clone())?, + Ok(ProcessorPtr::create(Box::new( + TransformPartitionAlign::create(ctx.clone(), params.clone(), input, output)?, ))) })?; From 66ade37a2ec414ff008314a584cc508df20b7b2d Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Thu, 10 Apr 2025 00:05:13 +0800 Subject: [PATCH 66/76] refactor(query): refactor aggreagte spill code --- .../core/src/processors/shuffle_processor.rs | 20 +++++++++++++++---- 1 file changed, 16 insertions(+), 4 deletions(-) diff --git a/src/query/pipeline/core/src/processors/shuffle_processor.rs b/src/query/pipeline/core/src/processors/shuffle_processor.rs index 6c8a77929facc..4590f376ed5b6 100644 --- a/src/query/pipeline/core/src/processors/shuffle_processor.rs +++ b/src/query/pipeline/core/src/processors/shuffle_processor.rs @@ -395,8 +395,14 @@ impl Processor for OnePartitionProcessor { return Ok(Event::Async); } - self.output.push_data(Ok(self.input_data.take().unwrap())); - return Ok(Event::NeedConsume); + let block = self.input_data.take().unwrap(); + let mut partitioned_data = self.exchange.partition(block, 1)?; + + if let Some(block) = partitioned_data.pop() { + debug_assert!(partitioned_data.is_empty()); + self.output.push_data(Ok(block)); + return Ok(Event::NeedConsume); + } } if self.input.has_data() { @@ -405,8 +411,14 @@ impl Processor for OnePartitionProcessor { return Ok(Event::Async); } - self.output.push_data(self.input.pull_data().unwrap()); - return Ok(Event::NeedConsume); + let data_block = self.input.pull_data().unwrap()?; + let mut partitioned_data = self.exchange.partition(data_block, 1)?; + + if let Some(block) = partitioned_data.pop() { + debug_assert!(partitioned_data.is_empty()); + self.output.push_data(Ok(block)); + return Ok(Event::NeedConsume); + } } if self.input.is_finished() { From 0b9305a0beb044fad14a6feac190544b2b50ab39 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Thu, 10 Apr 2025 00:41:51 +0800 Subject: [PATCH 67/76] refactor(query): refactor aggreagte spill code --- src/query/pipeline/core/src/processors/shuffle_processor.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/query/pipeline/core/src/processors/shuffle_processor.rs b/src/query/pipeline/core/src/processors/shuffle_processor.rs index 4590f376ed5b6..fca627b81d5f3 100644 --- a/src/query/pipeline/core/src/processors/shuffle_processor.rs +++ b/src/query/pipeline/core/src/processors/shuffle_processor.rs @@ -368,7 +368,7 @@ impl OnePartitionProcessor { #[async_trait::async_trait] impl Processor for OnePartitionProcessor { fn name(&self) -> String { - format!("ShuffleOnePartition({})", T::NAME) + format!("ShufflePartition({})", T::NAME) } fn as_any(&mut self) -> &mut dyn Any { From 87c911b40e4618446317639b4c7e9fc93668aae4 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sun, 13 Apr 2025 21:03:33 +0800 Subject: [PATCH 68/76] refactor(query): refactor aggreagte spill code --- .../core/src/processors/shuffle_processor.rs | 16 +- .../aggregator/transform_aggregate_final.rs | 207 +++++++++++------- 2 files changed, 142 insertions(+), 81 deletions(-) diff --git a/src/query/pipeline/core/src/processors/shuffle_processor.rs b/src/query/pipeline/core/src/processors/shuffle_processor.rs index fca627b81d5f3..9c435576eca23 100644 --- a/src/query/pipeline/core/src/processors/shuffle_processor.rs +++ b/src/query/pipeline/core/src/processors/shuffle_processor.rs @@ -195,6 +195,7 @@ pub struct PartitionProcessor { index: usize, initialized: bool, barrier: Arc, + hit: usize, } impl PartitionProcessor { @@ -206,6 +207,7 @@ impl PartitionProcessor { barrier: Arc, ) -> ProcessorPtr { let partitioned_data = vec![None; outputs.len()]; + let hit = index % outputs.len(); ProcessorPtr::create(Box::new(PartitionProcessor { input, outputs, @@ -215,6 +217,7 @@ impl PartitionProcessor { initialized: !T::MULTIWAY_SORT, index, barrier, + hit, })) } } @@ -233,7 +236,15 @@ impl Processor for PartitionProcessor { let mut all_output_finished = true; let mut all_data_pushed_output = true; - for (index, output) in self.outputs.iter().enumerate() { + for _index in 0..self.outputs.len() { + let index = self.hit; + let output = &self.outputs[self.hit]; + self.hit += 1; + + if self.hit == self.outputs.len() { + self.hit = 0; + } + if output.is_finished() { self.partitioned_data[index].take(); continue; @@ -245,9 +256,8 @@ impl Processor for PartitionProcessor { if let Some(block) = self.partitioned_data[index].take() { if !block.is_empty() || block.get_meta().is_some() { output.push_data(Ok(block)); + return Ok(Event::NeedConsume); } - - continue; } } 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 2b3e920d7c713..9a477b328de5f 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,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::any::Any; use std::sync::Arc; use bumpalo::Bump; @@ -25,11 +26,10 @@ use databend_common_expression::InputColumns; use databend_common_expression::Payload; use databend_common_expression::PayloadFlushState; use databend_common_expression::ProbeState; +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_transforms::AccumulatingTransform; -use databend_common_pipeline_transforms::AccumulatingTransformer; use crate::pipelines::processors::transforms::aggregator::aggregate_meta::AggregateMeta; use crate::pipelines::processors::transforms::aggregator::AggregatorParams; @@ -38,18 +38,110 @@ pub struct TransformFinalAggregate { params: Arc, flush_state: PayloadFlushState, hash_table: AggregateHashTable, - has_output: bool, working_partition: isize, - recv_final: usize, - init: bool, + + input: Arc, + output: Arc, + + next_partition_data: Option<(AggregateMeta, DataBlock)>, + + input_data: Vec<(AggregateMeta, DataBlock)>, + output_data: Option, } -impl AccumulatingTransform for TransformFinalAggregate { - const NAME: &'static str = "TransformFinalAggregate"; +#[async_trait::async_trait] +impl Processor for TransformFinalAggregate { + fn name(&self) -> String { + String::from("TransformFinalAggregate") + } + + fn as_any(&mut self) -> &mut dyn Any { + self + } + + fn event(&mut self) -> Result { + if self.output.is_finished() { + self.input_data.clear(); + self.next_partition_data.take(); + + 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(data_block) = self.output_data.take() { + self.output.push_data(Ok(data_block)); + return Ok(Event::NeedConsume); + } + + if self.input.has_data() { + return match self.add_data(self.input.pull_data().unwrap()?)? { + true => Ok(Event::Sync), + false => { + self.input.set_need_data(); + Ok(Event::NeedData) + } + }; + } + + if self.input.is_finished() { + return match self.input_data.is_empty() { + true => { + self.output.finish(); + Ok(Event::Finished) + } + false => Ok(Event::Sync), + }; + } + + self.input.set_need_data(); + Ok(Event::NeedData) + } + + fn process(&mut self) -> Result<()> { + for (meta, block) in std::mem::take(&mut self.input_data).into_iter() { + match meta { + AggregateMeta::SpilledPayload(_) => unreachable!(), + AggregateMeta::FinalPartition => unreachable!(), + AggregateMeta::InFlightPayload(_) => { + let payload = self.deserialize_flight(block)?; + + self.hash_table + .combine_payload(&payload, &mut self.flush_state)?; + } + AggregateMeta::AggregatePayload(payload) => { + self.hash_table + .combine_payload(&payload.payload, &mut self.flush_state)?; + } + }; + } - fn transform(&mut self, mut data: DataBlock) -> Result> { - let Some(meta) = data.take_meta() else { + if let Some(next_partition_data) = self.next_partition_data.take() { + self.input_data.push(next_partition_data); + } + + match self.hash_table.len() { + 0 => { + self.output_data = Some(self.params.empty_result_block()); + } + _ => { + let flush_blocks = self.flush_result_blocks()?; + self.output_data = Some(flush_blocks); + } + } + + Ok(()) + } +} + +impl TransformFinalAggregate { + pub fn add_data(&mut self, mut block: DataBlock) -> Result { + let Some(meta) = block.take_meta() else { return Err(ErrorCode::Internal( "Internal, TransformFinalAggregate only recv DataBlock with meta.", )); @@ -61,83 +153,45 @@ impl AccumulatingTransform for TransformFinalAggregate { )); }; - let mut flush_blocks = vec![]; - match aggregate_meta { - AggregateMeta::SpilledPayload(_) => unreachable!(), - AggregateMeta::FinalPartition => { - self.recv_final += 1; - } + AggregateMeta::SpilledPayload(_) => Ok(false), + AggregateMeta::FinalPartition => Ok(false), AggregateMeta::InFlightPayload(payload) => { debug_assert!(payload.partition >= self.working_partition); debug_assert_eq!(payload.max_partition, payload.global_max_partition); if self.working_partition != payload.partition { - assert!( - !self.init || self.recv_final > 0, - "payload partition: {}, working partition: {}", - payload.partition, - self.working_partition - ); - self.recv_final = 0; - if self.hash_table.len() != 0 { - flush_blocks = self.flush_result_blocks()?; - } - self.working_partition = payload.partition; + self.next_partition_data = + Some((AggregateMeta::InFlightPayload(payload), block)); + return Ok(true); } - self.init = true; - if !data.is_empty() { - let payload = self.deserialize_flight(data)?; - - self.hash_table - .combine_payload(&payload, &mut self.flush_state)?; + if !block.is_empty() { + self.input_data + .push((AggregateMeta::InFlightPayload(payload), block)); } + + Ok(false) } AggregateMeta::AggregatePayload(payload) => { debug_assert!(payload.partition >= self.working_partition); debug_assert_eq!(payload.max_partition, payload.global_max_partition); if self.working_partition != payload.partition { - assert!( - !self.init || self.recv_final > 0, - "payload partition: {}, working partition: {}", - payload.partition, - self.working_partition - ); - self.recv_final = 0; - if self.hash_table.len() != 0 { - flush_blocks = self.flush_result_blocks()?; - } - self.working_partition = payload.partition; + self.next_partition_data = + Some((AggregateMeta::AggregatePayload(payload), block)); + return Ok(true); } - self.init = true; if payload.payload.len() != 0 { - self.hash_table - .combine_payload(&payload.payload, &mut self.flush_state)?; + self.input_data + .push((AggregateMeta::AggregatePayload(payload), block)); } - } - } - Ok(flush_blocks) - } - - fn on_finish(&mut self, output: bool) -> Result> { - if !output { - return Ok(vec![]); - } - - let flush_blocks = match self.hash_table.len() == 0 { - true => vec![], - false => self.flush_result_blocks()?, - }; - - match self.has_output { - true => Ok(flush_blocks), - false => Ok(vec![self.params.empty_result_block()]), + Ok(false) + } } } } @@ -157,19 +211,17 @@ impl TransformFinalAggregate { Arc::new(Bump::new()), ); - Ok(AccumulatingTransformer::create( + Ok(Box::new(TransformFinalAggregate { + params, + hash_table, input, output, - TransformFinalAggregate { - params, - hash_table, - flush_state: PayloadFlushState::default(), - has_output: false, - working_partition: 0, - recv_final: 0, - init: false, - }, - )) + input_data: vec![], + output_data: None, + working_partition: 0, + next_partition_data: None, + flush_state: PayloadFlushState::default(), + })) } fn deserialize_flight(&mut self, data: DataBlock) -> Result { @@ -210,7 +262,7 @@ impl TransformFinalAggregate { Ok(hashtable.payload.payloads.pop().unwrap()) } - fn flush_result_blocks(&mut self) -> Result> { + fn flush_result_blocks(&mut self) -> Result { let mut blocks = vec![]; self.flush_state.clear(); @@ -228,7 +280,6 @@ impl TransformFinalAggregate { Arc::new(Bump::new()), ); - self.has_output |= !blocks.is_empty(); - Ok(blocks) + DataBlock::concat(&blocks) } } From d827b8c164986baaf8a97eb97faeaf31ffd40007 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 14 Apr 2025 01:12:57 +0800 Subject: [PATCH 69/76] refactor(query): refactor aggreagte spill code --- src/query/pipeline/core/Cargo.toml | 1 + src/query/pipeline/core/src/pipeline.rs | 36 ++ src/query/pipeline/core/src/processors/mod.rs | 3 + .../core/src/processors/shuffle_processor.rs | 482 ++++++++++++++++++ .../aggregator/transform_partition_bucket.rs | 2 +- .../transform_partition_exchange.rs | 4 + 6 files changed, 527 insertions(+), 1 deletion(-) diff --git a/src/query/pipeline/core/Cargo.toml b/src/query/pipeline/core/Cargo.toml index 64866ff49dc29..3464bd77c300d 100644 --- a/src/query/pipeline/core/Cargo.toml +++ b/src/query/pipeline/core/Cargo.toml @@ -17,6 +17,7 @@ futures = { workspace = true } log = { workspace = true } petgraph = { workspace = true } serde = { workspace = true } +typetag = { workspace = true } [dev-dependencies] serde = { workspace = true } diff --git a/src/query/pipeline/core/src/pipeline.rs b/src/query/pipeline/core/src/pipeline.rs index 801718d2e36e2..716c1226f6892 100644 --- a/src/query/pipeline/core/src/pipeline.rs +++ b/src/query/pipeline/core/src/pipeline.rs @@ -35,8 +35,11 @@ use crate::pipe::Pipe; use crate::pipe::PipeItem; use crate::processors::DuplicateProcessor; use crate::processors::Exchange; +use crate::processors::ExchangeShuffleProcessor; use crate::processors::InputPort; use crate::processors::MergePartitionProcessor; +use crate::processors::NewMergePartitionProcessor; +use crate::processors::NewPartitionProcessor; use crate::processors::OnePartitionProcessor; use crate::processors::OutputPort; use crate::processors::PartitionProcessor; @@ -514,6 +517,39 @@ impl Pipeline { } } + pub fn new_exchange(&mut self, n: usize, exchange: Arc) -> Result<()> { + self.add_transform(|input, output| { + Ok(NewPartitionProcessor::create( + input, + output, + n, + exchange.clone(), + )) + })?; + + let input_len = self.output_len(); + let inputs = (0..input_len) + .map(|_| InputPort::create()) + .collect::>(); + let outputs = (0..n).map(|_| OutputPort::create()).collect::>(); + + self.add_pipe(Pipe::create(input_len, n, vec![PipeItem::create( + ExchangeShuffleProcessor::create(inputs.clone(), outputs.clone(), exchange.clone()), + inputs, + outputs, + )])); + + self.add_transform(|input, output| { + Ok(NewMergePartitionProcessor::create( + input, + output, + exchange.clone(), + )) + })?; + + Ok(()) + } + #[track_caller] pub fn set_on_init Result<()> + Send + Sync + 'static>(&mut self, f: F) { let location = std::panic::Location::caller(); diff --git a/src/query/pipeline/core/src/processors/mod.rs b/src/query/pipeline/core/src/processors/mod.rs index 88baef3f1f5c0..583b61a4a5a68 100644 --- a/src/query/pipeline/core/src/processors/mod.rs +++ b/src/query/pipeline/core/src/processors/mod.rs @@ -38,8 +38,11 @@ pub use profile::PlanScopeGuard; pub use resize_processor::create_resize_item; pub use resize_processor::ResizeProcessor; pub use shuffle_processor::Exchange; +pub use shuffle_processor::ExchangeShuffleProcessor; pub use shuffle_processor::MergePartitionProcessor; pub use shuffle_processor::MultiwayStrategy; +pub use shuffle_processor::NewMergePartitionProcessor; +pub use shuffle_processor::NewPartitionProcessor; pub use shuffle_processor::OnePartitionProcessor; pub use shuffle_processor::PartitionProcessor; pub use shuffle_processor::ShuffleProcessor; diff --git a/src/query/pipeline/core/src/processors/shuffle_processor.rs b/src/query/pipeline/core/src/processors/shuffle_processor.rs index 9c435576eca23..1bb911b758914 100644 --- a/src/query/pipeline/core/src/processors/shuffle_processor.rs +++ b/src/query/pipeline/core/src/processors/shuffle_processor.rs @@ -19,6 +19,10 @@ use std::sync::Arc; use databend_common_base::base::tokio::sync::Barrier; use databend_common_exception::Result; +use databend_common_expression::local_block_meta_serde; +use databend_common_expression::BlockMetaInfo; +use databend_common_expression::BlockMetaInfoDowncast; +use databend_common_expression::BlockMetaInfoPtr; use databend_common_expression::DataBlock; use crate::processors::Event; @@ -65,6 +69,14 @@ pub trait Exchange: Send + Sync + 'static { position.map(|(idx, _)| idx) } + + fn output_window_size(&self) -> usize { + 3 + } + + fn merge_output(&self, data_blocks: Vec) -> Result> { + Ok(data_blocks) + } } pub struct ShuffleProcessor { @@ -635,3 +647,473 @@ impl Processor for MergePartitionProcessor { } } } + +#[derive(Debug)] +pub struct ShuffleMeta { + data_blocks: Vec, +} + +local_block_meta_serde!(ShuffleMeta); + +#[typetag::serde(name = "ShuffleMeta")] +impl BlockMetaInfo for ShuffleMeta {} + +impl ShuffleMeta { + pub fn create(blocks: Vec) -> BlockMetaInfoPtr { + Box::new(ShuffleMeta { + data_blocks: blocks, + }) + } +} + +pub struct NewPartitionProcessor { + input: Arc, + output: Arc, + + input_data: Option, + output_data: Option, + + exchange: Arc, + to_partition: usize, +} + +impl NewPartitionProcessor { + pub fn create( + input: Arc, + output: Arc, + to_partition: usize, + exchange: Arc, + ) -> ProcessorPtr { + ProcessorPtr::create(Box::new(NewPartitionProcessor { + input, + output, + exchange, + to_partition, + input_data: None, + output_data: None, + })) + } +} + +impl Processor for NewPartitionProcessor { + fn name(&self) -> String { + String::from("PartitionProcessor") + } + + 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(data_block) = self.output_data.take() { + self.output.push_data(Ok(data_block)); + return Ok(Event::NeedConsume); + } + + if self.input.has_data() { + self.input_data = Some(self.input.pull_data().unwrap()?); + return Ok(Event::Sync); + } + + 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(block) = self.input_data.take() { + let partitioned_data = self.exchange.partition(block, self.to_partition)?; + self.output_data = Some(DataBlock::empty_with_meta(ShuffleMeta::create( + partitioned_data, + ))); + } + + Ok(()) + } +} +// pub struct SortingExchangeShuffleProcessor { +// input: Vec>, +// output: Vec>, +// +// // output_finished: +// output_window_size: usize, +// matrix: Vec>>, +// matrix_size: usize, +// +// finished_input_size: usize, +// input_finish_status: Vec, +// +// initialize: bool, +// output_finish_status: Vec, +// waiting_outputs: VecDeque, +// +// output_matrix: Vec>, +// +// exchange: Arc, +// +// } +// +// fn multiway_pick(data_blocks: &mut [Option]) -> usize { +// let position = +// data_blocks +// .iter() +// .enumerate() +// .filter_map(|(idx, x)| x.as_ref().map(|d| (idx, d))) +// .min_by(|(left_idx, left_block), (right_idx, right_block)| { +// match T::sorting_function(left_block, right_block) { +// Ordering::Less => Ordering::Less, +// Ordering::Greater => Ordering::Greater, +// Ordering::Equal => left_idx.cmp(right_idx), +// } +// }); +// +// position.map(|(idx, _)| idx).unwrap_or(0) +// } +// +// impl Processor for SortingExchangeShuffleProcessor { +// fn name(&self) -> String { +// String::from("SortingExchangeShuffleProcessor") +// } +// +// fn as_any(&mut self) -> &mut dyn Any { +// self +// } +// +// fn event_with_cause(&mut self, cause: EventCause) -> Result { +// if !self.initialize && self.waiting_outputs.is_empty() { +// self.initialize = true; +// +// for input in &self.input { +// input.set_need_data(); +// } +// +// return Ok(Event::NeedData); +// } +// +// if let EventCause::Input(index) = cause { +// if self.input[index].is_finished() && !self.input_finish_status[index] { +// self.finished_input_size += 1; +// self.input_finish_status[index] = true; +// } +// +// if self.input[index].has_data() { +// let data_block = self.input[index].pull_data().unwrap()?; +// +// let meta = data_block.take_meta().unwrap(); +// let mut meta = ShuffleMeta::downcast_from(meta).unwrap(); +// +// self.matrix_size += 1; +// for (idx, block) in meta.data_blocks.into_iter().enumerate() { +// debug_assert!(self.matrix[idx][index].is_none()); +// +// if !self.output_finish_status[idx] { +// self.matrix[idx][index] = Some(block); +// } +// } +// } +// +// if self.matrix_size + self.finished_input_size != self.input.len() { +// return Ok(Event::NeedData); +// } +// +// if self.matrix_size != 0 { +// // matrix is full +// let Some(position) = self.output_finish_status.iter().position(|x| !x) else { +// for output in &self.output { +// output.finish(); +// } +// +// return Ok(Event::Finished); +// }; +// +// let position = multiway_pick::(&mut self.matrix[position]); +// +// for index in 0..self.output.len() { +// if let Some(data_block) = self.matrix[index][position] { +// self.output_matrix[index].push(data_block); +// } +// } +// +// self.matrix_size -= 1; +// self.output_window_size += 1; +// +// if self.output_window_size != self.exchange.output_window_size() { +// self.input[position].set_need_data(); +// return Ok(Event::NeedData); +// } +// +// // self.output_window_size +// } +// } +// +// if let EventCause::Output(index) = cause { +// if self.output[index].is_finished() && !self.output_finish_status[index] { +// self.output_finish_status[index] = true; +// } +// +// if self.output[index].can_push() { +// self.waiting_outputs.push_back(index); +// } +// } +// +// // while let Some(output) = self.waiting_outputs.pop_front() { +// // +// // } +// +// todo!() +// } +// } + +pub struct ExchangeShuffleProcessor { + input: Vec>, + output: Vec>, + + initialize: bool, + + finished_input_size: usize, + input_finish_status: Vec, + waiting_inputs: VecDeque, + + finished_output_size: usize, + pending_outputs: Vec, + output_finish_status: Vec, + + exchange: Arc, + matrix: Vec>, +} + +impl ExchangeShuffleProcessor { + pub fn create( + input: Vec>, + output: Vec>, + exchange: Arc, + ) -> ProcessorPtr { + let pending_outputs = vec![false; output.len()]; + let input_finish_status = vec![false; input.len()]; + let output_finish_status = vec![false; output.len()]; + + let mut matrix = Vec::with_capacity(output.len()); + + for _ in 0..output.capacity() { + matrix.push(VecDeque::new()); + } + + ProcessorPtr::create(Box::new(ExchangeShuffleProcessor { + input, + output, + matrix, + exchange, + pending_outputs, + input_finish_status, + output_finish_status, + + initialize: false, + finished_input_size: 0, + finished_output_size: 0, + waiting_inputs: VecDeque::new(), + })) + } +} + +impl Processor for ExchangeShuffleProcessor { + fn name(&self) -> String { + String::from("ExchangeShuffleProcessor") + } + + fn as_any(&mut self) -> &mut dyn Any { + self + } + + fn event_with_cause(&mut self, cause: EventCause) -> Result { + if let EventCause::Input(index) = cause { + if self.input[index].has_data() { + let mut data_block = self.input[index].pull_data().unwrap()?; + + let meta = data_block.take_meta().unwrap(); + let meta = ShuffleMeta::downcast_from(meta).unwrap(); + + for (idx, block) in meta.data_blocks.into_iter().enumerate() { + self.matrix[idx].push_back(block); + } + } + + if self.input[index].is_finished() { + if !self.input_finish_status[index] { + self.finished_input_size += 1; + self.input_finish_status[index] = true; + } + } else { + self.waiting_inputs.push_back(index); + } + } + + if let EventCause::Output(index) = cause { + if self.output[index].is_finished() && !self.output_finish_status[index] { + self.finished_output_size += 1; + self.output_finish_status[index] = true; + } + + if self.output[index].can_push() { + self.pending_outputs[index] = true; + } + } + + if !self.initialize { + self.initialize = true; + + for input in &self.input { + input.set_need_data(); + } + + return Ok(Event::NeedData); + } + + if self.finished_output_size == self.output.len() { + for input in &self.input { + input.finish(); + } + + return Ok(Event::Finished); + } + + let all_input_finished = self.finished_input_size == self.input.len(); + + let mut sent_all_data = true; + for (idx, data) in self.matrix.iter_mut().enumerate() { + if data.is_empty() || self.output_finish_status[idx] { + continue; + } + + sent_all_data = false; + if self.pending_outputs[idx] + && (all_input_finished || (data.len() >= self.exchange.output_window_size())) + { + self.pending_outputs[idx] = false; + let mut output_data = Vec::with_capacity(self.exchange.output_window_size()); + + for _index in 0..self.exchange.output_window_size() { + if let Some(data) = data.pop_front() { + output_data.push(data); + } + } + + self.output[idx].push_data(Ok(DataBlock::empty_with_meta(ShuffleMeta::create( + output_data, + )))); + return Ok(Event::NeedConsume); + } + } + + while let Some(index) = self.waiting_inputs.pop_front() { + if !self.input[index].is_finished() { + self.input[index].set_need_data(); + return Ok(Event::NeedData); + } else if !self.input_finish_status[index] { + self.input_finish_status[index] = true; + self.finished_input_size += 1; + } + } + + let all_input_finished = self.finished_input_size == self.input.len(); + if sent_all_data && all_input_finished { + for output in &self.output { + output.finish(); + } + + return Ok(Event::Finished); + } + + Ok(Event::NeedConsume) + } +} + +pub struct NewMergePartitionProcessor { + input: Arc, + output: Arc, + + input_data: Option, + output_data: VecDeque, + + exchange: Arc, +} + +impl NewMergePartitionProcessor { + pub fn create( + input: Arc, + output: Arc, + exchange: Arc, + ) -> ProcessorPtr { + ProcessorPtr::create(Box::new(NewMergePartitionProcessor { + input, + output, + input_data: None, + output_data: VecDeque::new(), + exchange, + })) + } +} + +impl Processor for NewMergePartitionProcessor { + fn name(&self) -> String { + String::from("MergePartitionProcessor") + } + + 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(data_block) = self.output_data.pop_front() { + self.output.push_data(Ok(data_block)); + return Ok(Event::NeedConsume); + } + + if self.input.has_data() { + self.input_data = Some(self.input.pull_data().unwrap()?); + return Ok(Event::Sync); + } + + 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(mut block) = self.input_data.take() { + let meta = block.take_meta().unwrap(); + let meta = ShuffleMeta::downcast_from(meta).unwrap(); + self.output_data + .extend(self.exchange.merge_output(meta.data_blocks)?); + } + + Ok(()) + } +} 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 index 0d6e98238c45f..bc6ea3ddcdf89 100644 --- 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 @@ -71,7 +71,7 @@ pub fn build_final_aggregate( // 5. exchange local let pipe_size = pipeline.output_len(); - pipeline.exchange(pipe_size, ExchangePartition::create(params.clone())); + pipeline.new_exchange(pipe_size, ExchangePartition::create(params.clone()))?; // 6. final aggregate pipeline.add_transform(|input, output| { diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_exchange.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_exchange.rs index 87c732f08a152..7eb8f125bd01f 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_exchange.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_exchange.rs @@ -191,4 +191,8 @@ impl Exchange for ExchangePartition { } } } + + // fn merge_output(&self, data_blocks: Vec) -> Result> { + // Ok(vec![DataBlock::concat(&data_blocks)?]) + // } } From d7e2a3691060d3b60dbdb28eb9adc9656089e048 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 14 Apr 2025 01:45:02 +0800 Subject: [PATCH 70/76] refactor(query): refactor aggreagte spill code --- .../aggregator/aggregate_exchange_injector.rs | 2 +- .../transforms/aggregator/aggregate_meta.rs | 20 ++++++------ .../aggregator/transform_aggregate_final.rs | 31 +++++++++++++------ .../aggregator/transform_partition_align.rs | 6 ++-- .../aggregator/transform_partition_bucket.rs | 5 ++- .../transform_partition_dispatch.rs | 12 +++---- .../transform_partition_exchange.rs | 24 +++++++++----- 7 files changed, 63 insertions(+), 37 deletions(-) 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 eea38704d82d7..5c91c2621275a 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 @@ -156,7 +156,7 @@ impl Exchange for FlightExchange { assert!(MULTIWAY_SORT); assert_eq!(self.node_list_lookup.len(), n); match AggregateMeta::downcast_from(meta).unwrap() { - AggregateMeta::FinalPartition => unreachable!(), + AggregateMeta::FinalPartition(_) => unreachable!(), AggregateMeta::InFlightPayload(_) => unreachable!(), AggregateMeta::SpilledPayload(v) => { let mut blocks = Vec::with_capacity(n); 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 7ebf7f86ebbeb..3baee05794962 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 @@ -140,7 +140,7 @@ pub struct InFlightPayload { } pub struct FinalPayload { - pub data: Arc>, + pub data: Vec, } #[derive(serde::Serialize, serde::Deserialize)] @@ -148,7 +148,7 @@ pub enum AggregateMeta { SpilledPayload(SpilledPayload), AggregatePayload(AggregatePayload), InFlightPayload(InFlightPayload), - FinalPartition, + FinalPartition(FinalPayload), } impl AggregateMeta { @@ -182,8 +182,8 @@ impl AggregateMeta { Box::new(AggregateMeta::SpilledPayload(payload)) } - pub fn create_final() -> BlockMetaInfoPtr { - Box::new(AggregateMeta::FinalPartition) + pub fn create_final(blocks: Vec) -> BlockMetaInfoPtr { + Box::new(AggregateMeta::FinalPartition(FinalPayload { data: blocks })) } pub fn get_global_max_partition(&self) -> usize { @@ -191,7 +191,7 @@ impl AggregateMeta { AggregateMeta::SpilledPayload(v) => v.global_max_partition, AggregateMeta::AggregatePayload(v) => v.global_max_partition, AggregateMeta::InFlightPayload(v) => v.global_max_partition, - AggregateMeta::FinalPartition => unreachable!(), + AggregateMeta::FinalPartition(_) => unreachable!(), } } @@ -200,7 +200,7 @@ impl AggregateMeta { AggregateMeta::SpilledPayload(v) => v.partition, AggregateMeta::AggregatePayload(v) => v.partition, AggregateMeta::InFlightPayload(v) => v.partition, - AggregateMeta::FinalPartition => unreachable!(), + AggregateMeta::FinalPartition(_) => unreachable!(), } } @@ -209,7 +209,7 @@ impl AggregateMeta { AggregateMeta::AggregatePayload(v) => v.partition, AggregateMeta::InFlightPayload(v) => v.partition, AggregateMeta::SpilledPayload(v) => v.get_sorting_partition(), - AggregateMeta::FinalPartition => unreachable!(), + AggregateMeta::FinalPartition(_) => unreachable!(), } } @@ -218,7 +218,7 @@ impl AggregateMeta { AggregateMeta::SpilledPayload(v) => v.max_partition, AggregateMeta::AggregatePayload(v) => v.max_partition, AggregateMeta::InFlightPayload(v) => v.max_partition, - AggregateMeta::FinalPartition => unreachable!(), + AggregateMeta::FinalPartition(_) => unreachable!(), } } @@ -233,7 +233,7 @@ impl AggregateMeta { AggregateMeta::InFlightPayload(v) => { v.global_max_partition = global_max_partition; } - AggregateMeta::FinalPartition => unreachable!(), + AggregateMeta::FinalPartition(_) => unreachable!(), } } } @@ -241,7 +241,7 @@ impl AggregateMeta { impl Debug for AggregateMeta { fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { match self { - AggregateMeta::FinalPartition => { + AggregateMeta::FinalPartition(_) => { f.debug_struct("AggregateMeta::FinalPartition").finish() } AggregateMeta::SpilledPayload(_) => { 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 9a477b328de5f..a0a0492a15aa1 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 @@ -13,6 +13,7 @@ // limitations under the License. use std::any::Any; +use std::collections::VecDeque; use std::sync::Arc; use bumpalo::Bump; @@ -44,7 +45,7 @@ pub struct TransformFinalAggregate { input: Arc, output: Arc, - next_partition_data: Option<(AggregateMeta, DataBlock)>, + next_partition_data: VecDeque<(AggregateMeta, DataBlock)>, input_data: Vec<(AggregateMeta, DataBlock)>, output_data: Option, @@ -63,7 +64,7 @@ impl Processor for TransformFinalAggregate { fn event(&mut self) -> Result { if self.output.is_finished() { self.input_data.clear(); - self.next_partition_data.take(); + self.next_partition_data.clear(); self.input.finish(); return Ok(Event::Finished); @@ -107,7 +108,7 @@ impl Processor for TransformFinalAggregate { for (meta, block) in std::mem::take(&mut self.input_data).into_iter() { match meta { AggregateMeta::SpilledPayload(_) => unreachable!(), - AggregateMeta::FinalPartition => unreachable!(), + AggregateMeta::FinalPartition(_) => unreachable!(), AggregateMeta::InFlightPayload(_) => { let payload = self.deserialize_flight(block)?; @@ -121,7 +122,7 @@ impl Processor for TransformFinalAggregate { }; } - if let Some(next_partition_data) = self.next_partition_data.take() { + while let Some(next_partition_data) = self.next_partition_data.pop_front() { self.input_data.push(next_partition_data); } @@ -155,15 +156,25 @@ impl TransformFinalAggregate { match aggregate_meta { AggregateMeta::SpilledPayload(_) => Ok(false), - AggregateMeta::FinalPartition => Ok(false), + AggregateMeta::FinalPartition(payload) => { + let mut need_final = false; + let working_partition = self.working_partition; + + for block in payload.data { + self.working_partition = working_partition; + need_final = self.add_data(block)?; + } + + Ok(need_final) + } AggregateMeta::InFlightPayload(payload) => { debug_assert!(payload.partition >= self.working_partition); debug_assert_eq!(payload.max_partition, payload.global_max_partition); if self.working_partition != payload.partition { self.working_partition = payload.partition; - self.next_partition_data = - Some((AggregateMeta::InFlightPayload(payload), block)); + self.next_partition_data + .push_back((AggregateMeta::InFlightPayload(payload), block)); return Ok(true); } @@ -180,8 +191,8 @@ impl TransformFinalAggregate { if self.working_partition != payload.partition { self.working_partition = payload.partition; - self.next_partition_data = - Some((AggregateMeta::AggregatePayload(payload), block)); + self.next_partition_data + .push_back((AggregateMeta::AggregatePayload(payload), block)); return Ok(true); } @@ -219,7 +230,7 @@ impl TransformFinalAggregate { input_data: vec![], output_data: None, working_partition: 0, - next_partition_data: None, + next_partition_data: VecDeque::new(), flush_state: PayloadFlushState::default(), })) } diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_align.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_align.rs index 91b64b292761c..226a844f36b3b 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_align.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_align.rs @@ -99,7 +99,9 @@ impl TransformPartitionAlign { } self.output_data - .push_back(DataBlock::empty_with_meta(AggregateMeta::create_final())); + .push_back(DataBlock::empty_with_meta(AggregateMeta::create_final( + vec![], + ))); } Ok(()) @@ -123,7 +125,7 @@ impl TransformPartitionAlign { fn repartition(&mut self, meta: AggregateMeta, data_block: DataBlock) -> Result<()> { match meta { - AggregateMeta::FinalPartition => unreachable!(), + AggregateMeta::FinalPartition(_) => unreachable!(), AggregateMeta::SpilledPayload(_payload) => unreachable!(), AggregateMeta::InFlightPayload(payload) => { if data_block.is_empty() { 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 index bc6ea3ddcdf89..cb9f98bbbf3d7 100644 --- 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 @@ -71,7 +71,10 @@ pub fn build_final_aggregate( // 5. exchange local let pipe_size = pipeline.output_len(); - pipeline.new_exchange(pipe_size, ExchangePartition::create(params.clone()))?; + pipeline.new_exchange( + pipe_size, + ExchangePartition::create(pipe_size, params.clone()), + )?; // 6. final aggregate pipeline.add_transform(|input, output| { diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_dispatch.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_dispatch.rs index 58149dbb1e5fe..bfe7e87258e75 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_dispatch.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_dispatch.rs @@ -135,7 +135,7 @@ impl Processor for TransformPartitionDispatch { } else { self.sent_final_partition[*output_index] = true; output.port.push_data(Ok(DataBlock::empty_with_meta( - AggregateMeta::create_final(), + AggregateMeta::create_final(vec![]), ))); } } else if output.status != PortStatus::NeedData { @@ -160,7 +160,7 @@ impl Processor for TransformPartitionDispatch { let (meta, data_block) = Self::unpark_block(data_block)?; match meta { - AggregateMeta::FinalPartition => { + AggregateMeta::FinalPartition(_) => { self.sync_final_partition = true; self.input.set_not_need_data(); } @@ -184,9 +184,9 @@ impl Processor for TransformPartitionDispatch { self.outputs[output_index] .port - .push_data(Ok( - DataBlock::empty_with_meta(AggregateMeta::create_final()), - )); + .push_data(Ok(DataBlock::empty_with_meta(AggregateMeta::create_final( + vec![], + )))); self.sent_final_partition[output_index] = true; self.outputs[output_index].status = PortStatus::Idle; } @@ -207,7 +207,7 @@ impl Processor for TransformPartitionDispatch { let (meta, data_block) = Self::unpark_block(data_block)?; match meta { - AggregateMeta::FinalPartition => { + AggregateMeta::FinalPartition(_) => { self.sync_final_partition = true; self.input.set_not_need_data(); continue; diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_exchange.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_exchange.rs index 7eb8f125bd01f..b4a7336a954aa 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_exchange.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_exchange.rs @@ -35,19 +35,23 @@ use crate::pipelines::processors::transforms::aggregator::InFlightPayload; const HASH_SEED: u64 = 9263883436177860930; pub struct ExchangePartition { + merge_window_size: usize, params: Arc, } impl ExchangePartition { - pub fn create(params: Arc) -> Arc { - Arc::new(ExchangePartition { params }) + pub fn create(merge_window_size: usize, params: Arc) -> Arc { + Arc::new(ExchangePartition { + merge_window_size, + params, + }) } } impl ExchangePartition { fn partition_final_payload(n: usize) -> Result> { Ok((0..n) - .map(|_| DataBlock::empty_with_meta(AggregateMeta::create_final())) + .map(|_| DataBlock::empty_with_meta(AggregateMeta::create_final(vec![]))) .collect()) } @@ -184,7 +188,7 @@ impl Exchange for ExchangePartition { // already restore in upstream AggregateMeta::SpilledPayload(_) => unreachable!(), // broadcast final partition to downstream - AggregateMeta::FinalPartition => Self::partition_final_payload(n), + AggregateMeta::FinalPartition(_) => Self::partition_final_payload(n), AggregateMeta::AggregatePayload(payload) => Self::partition_aggregate(payload, n), AggregateMeta::InFlightPayload(payload) => { self.partition_flight_payload(payload, data_block, n) @@ -192,7 +196,13 @@ impl Exchange for ExchangePartition { } } - // fn merge_output(&self, data_blocks: Vec) -> Result> { - // Ok(vec![DataBlock::concat(&data_blocks)?]) - // } + fn output_window_size(&self) -> usize { + self.merge_window_size + } + + fn merge_output(&self, data_blocks: Vec) -> Result> { + Ok(vec![DataBlock::empty_with_meta( + AggregateMeta::create_final(data_blocks), + )]) + } } From eb7cb159b36ffd79155fcaea608c5902d225220d Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 14 Apr 2025 09:40:57 +0800 Subject: [PATCH 71/76] refactor(query): refactor aggreagte spill code --- .../src/aggregate/partitioned_payload.rs | 2 +- src/query/expression/src/aggregate/payload.rs | 8 +- .../core/src/processors/shuffle_processor.rs | 132 ------------- .../aggregator/transform_aggregate_final.rs | 178 ++++-------------- .../aggregator/transform_partition_align.rs | 2 +- .../transform_partition_exchange.rs | 49 ++++- 6 files changed, 90 insertions(+), 281 deletions(-) diff --git a/src/query/expression/src/aggregate/partitioned_payload.rs b/src/query/expression/src/aggregate/partitioned_payload.rs index 46b666020453a..c5163811ffd90 100644 --- a/src/query/expression/src/aggregate/partitioned_payload.rs +++ b/src/query/expression/src/aggregate/partitioned_payload.rs @@ -83,7 +83,7 @@ impl PartitionedPayload { let payloads = (0..partition_count) .map(|_| { Payload::new( - arenas[0].clone(), + arenas.clone(), group_types.clone(), aggrs.clone(), states_layout.clone(), diff --git a/src/query/expression/src/aggregate/payload.rs b/src/query/expression/src/aggregate/payload.rs index 8e9d796da9833..28c6303aa3045 100644 --- a/src/query/expression/src/aggregate/payload.rs +++ b/src/query/expression/src/aggregate/payload.rs @@ -46,7 +46,7 @@ use crate::MAX_PAGE_SIZE; // [HASH] is the hash data of the groups // [STATE_ADDRS] is the state_addrs of the aggregate functions, 8 bytes each pub struct Payload { - pub arena: Arc, + pub arena: Vec>, // if true, the states are moved out of the payload into other payload, and will not be dropped pub state_move_out: bool, pub group_types: Vec, @@ -94,7 +94,7 @@ pub type Pages = Vec; impl Payload { pub fn new( - arena: Arc, + arena: Vec>, group_types: Vec, aggrs: Vec, states_layout: Option, @@ -267,7 +267,7 @@ impl Payload { unsafe { serialize_column_to_rowformat( - &self.arena, + &self.arena[0], col, select_vector, new_group_rows, @@ -297,7 +297,7 @@ impl Payload { // write states let (array_layout, padded_size) = layout.repeat(new_group_rows).unwrap(); // Bump only allocates but does not drop, so there is no use after free for any item. - let place = self.arena.alloc_layout(array_layout); + let place = self.arena[0].alloc_layout(array_layout); for (idx, place) in select_vector .iter() .take(new_group_rows) diff --git a/src/query/pipeline/core/src/processors/shuffle_processor.rs b/src/query/pipeline/core/src/processors/shuffle_processor.rs index 1bb911b758914..370528bbe7d65 100644 --- a/src/query/pipeline/core/src/processors/shuffle_processor.rs +++ b/src/query/pipeline/core/src/processors/shuffle_processor.rs @@ -745,138 +745,6 @@ impl Processor for NewPartitionProcessor { Ok(()) } } -// pub struct SortingExchangeShuffleProcessor { -// input: Vec>, -// output: Vec>, -// -// // output_finished: -// output_window_size: usize, -// matrix: Vec>>, -// matrix_size: usize, -// -// finished_input_size: usize, -// input_finish_status: Vec, -// -// initialize: bool, -// output_finish_status: Vec, -// waiting_outputs: VecDeque, -// -// output_matrix: Vec>, -// -// exchange: Arc, -// -// } -// -// fn multiway_pick(data_blocks: &mut [Option]) -> usize { -// let position = -// data_blocks -// .iter() -// .enumerate() -// .filter_map(|(idx, x)| x.as_ref().map(|d| (idx, d))) -// .min_by(|(left_idx, left_block), (right_idx, right_block)| { -// match T::sorting_function(left_block, right_block) { -// Ordering::Less => Ordering::Less, -// Ordering::Greater => Ordering::Greater, -// Ordering::Equal => left_idx.cmp(right_idx), -// } -// }); -// -// position.map(|(idx, _)| idx).unwrap_or(0) -// } -// -// impl Processor for SortingExchangeShuffleProcessor { -// fn name(&self) -> String { -// String::from("SortingExchangeShuffleProcessor") -// } -// -// fn as_any(&mut self) -> &mut dyn Any { -// self -// } -// -// fn event_with_cause(&mut self, cause: EventCause) -> Result { -// if !self.initialize && self.waiting_outputs.is_empty() { -// self.initialize = true; -// -// for input in &self.input { -// input.set_need_data(); -// } -// -// return Ok(Event::NeedData); -// } -// -// if let EventCause::Input(index) = cause { -// if self.input[index].is_finished() && !self.input_finish_status[index] { -// self.finished_input_size += 1; -// self.input_finish_status[index] = true; -// } -// -// if self.input[index].has_data() { -// let data_block = self.input[index].pull_data().unwrap()?; -// -// let meta = data_block.take_meta().unwrap(); -// let mut meta = ShuffleMeta::downcast_from(meta).unwrap(); -// -// self.matrix_size += 1; -// for (idx, block) in meta.data_blocks.into_iter().enumerate() { -// debug_assert!(self.matrix[idx][index].is_none()); -// -// if !self.output_finish_status[idx] { -// self.matrix[idx][index] = Some(block); -// } -// } -// } -// -// if self.matrix_size + self.finished_input_size != self.input.len() { -// return Ok(Event::NeedData); -// } -// -// if self.matrix_size != 0 { -// // matrix is full -// let Some(position) = self.output_finish_status.iter().position(|x| !x) else { -// for output in &self.output { -// output.finish(); -// } -// -// return Ok(Event::Finished); -// }; -// -// let position = multiway_pick::(&mut self.matrix[position]); -// -// for index in 0..self.output.len() { -// if let Some(data_block) = self.matrix[index][position] { -// self.output_matrix[index].push(data_block); -// } -// } -// -// self.matrix_size -= 1; -// self.output_window_size += 1; -// -// if self.output_window_size != self.exchange.output_window_size() { -// self.input[position].set_need_data(); -// return Ok(Event::NeedData); -// } -// -// // self.output_window_size -// } -// } -// -// if let EventCause::Output(index) = cause { -// if self.output[index].is_finished() && !self.output_finish_status[index] { -// self.output_finish_status[index] = true; -// } -// -// if self.output[index].can_push() { -// self.waiting_outputs.push_back(index); -// } -// } -// -// // while let Some(output) = self.waiting_outputs.pop_front() { -// // -// // } -// -// todo!() -// } -// } pub struct ExchangeShuffleProcessor { input: Vec>, 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 a0a0492a15aa1..4c044f74c2853 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,8 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::any::Any; -use std::collections::VecDeque; use std::sync::Arc; use bumpalo::Bump; @@ -27,10 +25,11 @@ use databend_common_expression::InputColumns; use databend_common_expression::Payload; use databend_common_expression::PayloadFlushState; use databend_common_expression::ProbeState; -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_transforms::AccumulatingTransform; +use databend_common_pipeline_transforms::AccumulatingTransformer; use crate::pipelines::processors::transforms::aggregator::aggregate_meta::AggregateMeta; use crate::pipelines::processors::transforms::aggregator::AggregatorParams; @@ -41,108 +40,13 @@ pub struct TransformFinalAggregate { hash_table: AggregateHashTable, working_partition: isize, - - input: Arc, - output: Arc, - - next_partition_data: VecDeque<(AggregateMeta, DataBlock)>, - - input_data: Vec<(AggregateMeta, DataBlock)>, - output_data: Option, } -#[async_trait::async_trait] -impl Processor for TransformFinalAggregate { - fn name(&self) -> String { - String::from("TransformFinalAggregate") - } - - fn as_any(&mut self) -> &mut dyn Any { - self - } - - fn event(&mut self) -> Result { - if self.output.is_finished() { - self.input_data.clear(); - self.next_partition_data.clear(); +impl AccumulatingTransform for TransformFinalAggregate { + const NAME: &'static str = "TransformFinalAggregate"; - 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(data_block) = self.output_data.take() { - self.output.push_data(Ok(data_block)); - return Ok(Event::NeedConsume); - } - - if self.input.has_data() { - return match self.add_data(self.input.pull_data().unwrap()?)? { - true => Ok(Event::Sync), - false => { - self.input.set_need_data(); - Ok(Event::NeedData) - } - }; - } - - if self.input.is_finished() { - return match self.input_data.is_empty() { - true => { - self.output.finish(); - Ok(Event::Finished) - } - false => Ok(Event::Sync), - }; - } - - self.input.set_need_data(); - Ok(Event::NeedData) - } - - fn process(&mut self) -> Result<()> { - for (meta, block) in std::mem::take(&mut self.input_data).into_iter() { - match meta { - AggregateMeta::SpilledPayload(_) => unreachable!(), - AggregateMeta::FinalPartition(_) => unreachable!(), - AggregateMeta::InFlightPayload(_) => { - let payload = self.deserialize_flight(block)?; - - self.hash_table - .combine_payload(&payload, &mut self.flush_state)?; - } - AggregateMeta::AggregatePayload(payload) => { - self.hash_table - .combine_payload(&payload.payload, &mut self.flush_state)?; - } - }; - } - - while let Some(next_partition_data) = self.next_partition_data.pop_front() { - self.input_data.push(next_partition_data); - } - - match self.hash_table.len() { - 0 => { - self.output_data = Some(self.params.empty_result_block()); - } - _ => { - let flush_blocks = self.flush_result_blocks()?; - self.output_data = Some(flush_blocks); - } - } - - Ok(()) - } -} - -impl TransformFinalAggregate { - pub fn add_data(&mut self, mut block: DataBlock) -> Result { - let Some(meta) = block.take_meta() else { + fn transform(&mut self, mut data: DataBlock) -> Result> { + let Some(meta) = data.take_meta() else { return Err(ErrorCode::Internal( "Internal, TransformFinalAggregate only recv DataBlock with meta.", )); @@ -154,36 +58,24 @@ impl TransformFinalAggregate { )); }; + let mut blocks = vec![]; match aggregate_meta { - AggregateMeta::SpilledPayload(_) => Ok(false), - AggregateMeta::FinalPartition(payload) => { - let mut need_final = false; - let working_partition = self.working_partition; - - for block in payload.data { - self.working_partition = working_partition; - need_final = self.add_data(block)?; - } - - Ok(need_final) - } + AggregateMeta::SpilledPayload(_) => unreachable!(), + AggregateMeta::FinalPartition(_) => unreachable!(), AggregateMeta::InFlightPayload(payload) => { debug_assert!(payload.partition >= self.working_partition); debug_assert_eq!(payload.max_partition, payload.global_max_partition); if self.working_partition != payload.partition { self.working_partition = payload.partition; - self.next_partition_data - .push_back((AggregateMeta::InFlightPayload(payload), block)); - return Ok(true); + blocks.push(self.flush_result_blocks()?); } - if !block.is_empty() { - self.input_data - .push((AggregateMeta::InFlightPayload(payload), block)); + if !data.is_empty() { + let payload = self.deserialize_flight(data)?; + self.hash_table + .combine_payload(&payload, &mut self.flush_state)?; } - - Ok(false) } AggregateMeta::AggregatePayload(payload) => { debug_assert!(payload.partition >= self.working_partition); @@ -191,19 +83,25 @@ impl TransformFinalAggregate { if self.working_partition != payload.partition { self.working_partition = payload.partition; - self.next_partition_data - .push_back((AggregateMeta::AggregatePayload(payload), block)); - return Ok(true); + blocks.push(self.flush_result_blocks()?); } if payload.payload.len() != 0 { - self.input_data - .push((AggregateMeta::AggregatePayload(payload), block)); + self.hash_table + .combine_payload(&payload.payload, &mut self.flush_state)?; } - - Ok(false) } + }; + + Ok(blocks) + } + + fn on_finish(&mut self, output: bool) -> Result> { + if !output { + return Ok(vec![]); } + + Ok(vec![self.flush_result_blocks()?]) } } @@ -222,17 +120,16 @@ impl TransformFinalAggregate { Arc::new(Bump::new()), ); - Ok(Box::new(TransformFinalAggregate { - params, - hash_table, + Ok(AccumulatingTransformer::create( input, output, - input_data: vec![], - output_data: None, - working_partition: 0, - next_partition_data: VecDeque::new(), - flush_state: PayloadFlushState::default(), - })) + TransformFinalAggregate { + params, + hash_table, + working_partition: 0, + flush_state: PayloadFlushState::default(), + }, + )) } fn deserialize_flight(&mut self, data: DataBlock) -> Result { @@ -291,6 +188,9 @@ impl TransformFinalAggregate { Arc::new(Bump::new()), ); - DataBlock::concat(&blocks) + match blocks.is_empty() { + true => Ok(self.params.empty_result_block()), + false => DataBlock::concat(&blocks), + } } } diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_align.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_align.rs index 226a844f36b3b..eb95601641217 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_align.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_align.rs @@ -208,7 +208,7 @@ impl TransformPartitionAlign { self.params.group_data_types.clone(), self.params.aggregate_functions.clone(), to as u64, - vec![from.payload.arena.clone()], + from.payload.arena.clone(), ); let mut flush_state = PayloadFlushState::default(); diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_exchange.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_exchange.rs index b4a7336a954aa..e6a86a4b483b5 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_exchange.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_exchange.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::btree_map::Entry; +use std::collections::BTreeMap; use std::sync::Arc; use bumpalo::Bump; @@ -188,7 +190,7 @@ impl Exchange for ExchangePartition { // already restore in upstream AggregateMeta::SpilledPayload(_) => unreachable!(), // broadcast final partition to downstream - AggregateMeta::FinalPartition(_) => Self::partition_final_payload(n), + AggregateMeta::FinalPartition(_) => Ok(vec![]), AggregateMeta::AggregatePayload(payload) => Self::partition_aggregate(payload, n), AggregateMeta::InFlightPayload(payload) => { self.partition_flight_payload(payload, data_block, n) @@ -201,8 +203,47 @@ impl Exchange for ExchangePartition { } fn merge_output(&self, data_blocks: Vec) -> Result> { - Ok(vec![DataBlock::empty_with_meta( - AggregateMeta::create_final(data_blocks), - )]) + let mut blocks = BTreeMap::::new(); + for mut data_block in data_blocks { + let Some(meta) = data_block.take_meta() else { + return Err(ErrorCode::Internal( + "Internal, ExchangePartition only recv DataBlock with meta.", + )); + }; + + let Some(aggregate_meta) = AggregateMeta::downcast_from(meta) else { + return Err(ErrorCode::Internal( + "Internal, ExchangePartition only recv DataBlock with meta.", + )); + }; + + let mut payload = match aggregate_meta { + AggregateMeta::SpilledPayload(_) => unreachable!(), + AggregateMeta::FinalPartition(_) => unreachable!(), + AggregateMeta::InFlightPayload(_) => unreachable!(), + AggregateMeta::AggregatePayload(payload) => payload, + }; + + match blocks.entry(payload.partition) { + Entry::Vacant(v) => { + v.insert(payload); + } + Entry::Occupied(mut v) => { + payload.payload.state_move_out = true; + v.get_mut() + .payload + .arena + .extend(payload.payload.arena.clone()); + v.get_mut().payload.combine(payload.payload); + } + } + } + + Ok(blocks + .into_values() + .map(|payload| { + DataBlock::empty_with_meta(Box::new(AggregateMeta::AggregatePayload(payload))) + }) + .collect()) } } From 5090728dee013254fe19aa354ea6c96622cd6e2e Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 14 Apr 2025 09:44:59 +0800 Subject: [PATCH 72/76] refactor(query): refactor aggreagte spill code --- .../transforms/aggregator/transform_partition_exchange.rs | 6 ------ 1 file changed, 6 deletions(-) diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_exchange.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_exchange.rs index e6a86a4b483b5..74ad55d95003b 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_exchange.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_exchange.rs @@ -51,12 +51,6 @@ impl ExchangePartition { } impl ExchangePartition { - fn partition_final_payload(n: usize) -> Result> { - Ok((0..n) - .map(|_| DataBlock::empty_with_meta(AggregateMeta::create_final(vec![]))) - .collect()) - } - fn partition_aggregate(mut payload: AggregatePayload, n: usize) -> Result> { if payload.payload.len() == 0 { return Ok(vec![]); From 580440b8d2b7bda6cc3a6c7f018f7d53cbaa06a2 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 14 Apr 2025 13:30:31 +0800 Subject: [PATCH 73/76] refactor(query): refactor aggreagte spill code --- src/query/pipeline/core/src/pipeline.rs | 26 +++++++---- src/query/pipeline/core/src/processors/mod.rs | 6 +-- .../core/src/processors/shuffle_processor.rs | 46 +++++++++---------- .../builders/builder_hilbert_partition.rs | 2 +- .../src/pipelines/builders/builder_window.rs | 4 +- .../aggregator/transform_partition_bucket.rs | 2 +- .../transform_partition_exchange.rs | 2 - .../flight/v1/exchange/exchange_sink.rs | 4 +- .../v1/exchange/exchange_transform_shuffle.rs | 4 +- 9 files changed, 50 insertions(+), 46 deletions(-) diff --git a/src/query/pipeline/core/src/pipeline.rs b/src/query/pipeline/core/src/pipeline.rs index 716c1226f6892..808849efa2bf7 100644 --- a/src/query/pipeline/core/src/pipeline.rs +++ b/src/query/pipeline/core/src/pipeline.rs @@ -33,13 +33,13 @@ use crate::finished_chain::ExecutionInfo; use crate::finished_chain::FinishedCallbackChain; use crate::pipe::Pipe; use crate::pipe::PipeItem; +use crate::processors::BatchExchangeProcessor; +use crate::processors::BatchMergePartitionProcessor; +use crate::processors::BatchPartitionProcessor; use crate::processors::DuplicateProcessor; use crate::processors::Exchange; -use crate::processors::ExchangeShuffleProcessor; use crate::processors::InputPort; use crate::processors::MergePartitionProcessor; -use crate::processors::NewMergePartitionProcessor; -use crate::processors::NewPartitionProcessor; use crate::processors::OnePartitionProcessor; use crate::processors::OutputPort; use crate::processors::PartitionProcessor; @@ -452,12 +452,16 @@ impl Pipeline { } } - pub fn exchange(&mut self, n: usize, exchange: Arc) { + pub fn exchange(&mut self, n: usize, exchange: Arc) -> Result<()> { debug_assert_ne!(n, 0); + if !T::MULTIWAY_SORT { + return self.batch_exchange(n, exchange); + } + if let Some(pipe) = self.pipes.last() { if pipe.output_length < 1 { - return; + return Ok(()); } let input_len = pipe.output_length; @@ -513,13 +517,15 @@ impl Pipeline { } // merge partition - self.add_pipe(Pipe::create(input_len * n, n, items)) + self.add_pipe(Pipe::create(input_len * n, n, items)); } + + Ok(()) } - pub fn new_exchange(&mut self, n: usize, exchange: Arc) -> Result<()> { + fn batch_exchange(&mut self, n: usize, exchange: Arc) -> Result<()> { self.add_transform(|input, output| { - Ok(NewPartitionProcessor::create( + Ok(BatchPartitionProcessor::create( input, output, n, @@ -534,13 +540,13 @@ impl Pipeline { let outputs = (0..n).map(|_| OutputPort::create()).collect::>(); self.add_pipe(Pipe::create(input_len, n, vec![PipeItem::create( - ExchangeShuffleProcessor::create(inputs.clone(), outputs.clone(), exchange.clone()), + BatchExchangeProcessor::create(inputs.clone(), outputs.clone(), exchange.clone()), inputs, outputs, )])); self.add_transform(|input, output| { - Ok(NewMergePartitionProcessor::create( + Ok(BatchMergePartitionProcessor::create( input, output, exchange.clone(), diff --git a/src/query/pipeline/core/src/processors/mod.rs b/src/query/pipeline/core/src/processors/mod.rs index 583b61a4a5a68..00023c709fd0d 100644 --- a/src/query/pipeline/core/src/processors/mod.rs +++ b/src/query/pipeline/core/src/processors/mod.rs @@ -37,12 +37,12 @@ pub use profile::PlanScope; pub use profile::PlanScopeGuard; pub use resize_processor::create_resize_item; pub use resize_processor::ResizeProcessor; +pub use shuffle_processor::BatchExchangeProcessor; +pub use shuffle_processor::BatchMergePartitionProcessor; +pub use shuffle_processor::BatchPartitionProcessor; pub use shuffle_processor::Exchange; -pub use shuffle_processor::ExchangeShuffleProcessor; pub use shuffle_processor::MergePartitionProcessor; pub use shuffle_processor::MultiwayStrategy; -pub use shuffle_processor::NewMergePartitionProcessor; -pub use shuffle_processor::NewPartitionProcessor; pub use shuffle_processor::OnePartitionProcessor; pub use shuffle_processor::PartitionProcessor; pub use shuffle_processor::ShuffleProcessor; diff --git a/src/query/pipeline/core/src/processors/shuffle_processor.rs b/src/query/pipeline/core/src/processors/shuffle_processor.rs index 370528bbe7d65..893fc77fdcb30 100644 --- a/src/query/pipeline/core/src/processors/shuffle_processor.rs +++ b/src/query/pipeline/core/src/processors/shuffle_processor.rs @@ -649,24 +649,24 @@ impl Processor for MergePartitionProcessor { } #[derive(Debug)] -pub struct ShuffleMeta { +pub struct ExchangeMeta { data_blocks: Vec, } -local_block_meta_serde!(ShuffleMeta); +local_block_meta_serde!(ExchangeMeta); -#[typetag::serde(name = "ShuffleMeta")] -impl BlockMetaInfo for ShuffleMeta {} +#[typetag::serde(name = "LocalExchangeMeta")] +impl BlockMetaInfo for ExchangeMeta {} -impl ShuffleMeta { +impl ExchangeMeta { pub fn create(blocks: Vec) -> BlockMetaInfoPtr { - Box::new(ShuffleMeta { + Box::new(ExchangeMeta { data_blocks: blocks, }) } } -pub struct NewPartitionProcessor { +pub struct BatchPartitionProcessor { input: Arc, output: Arc, @@ -677,14 +677,14 @@ pub struct NewPartitionProcessor { to_partition: usize, } -impl NewPartitionProcessor { +impl BatchPartitionProcessor { pub fn create( input: Arc, output: Arc, to_partition: usize, exchange: Arc, ) -> ProcessorPtr { - ProcessorPtr::create(Box::new(NewPartitionProcessor { + ProcessorPtr::create(Box::new(BatchPartitionProcessor { input, output, exchange, @@ -695,7 +695,7 @@ impl NewPartitionProcessor { } } -impl Processor for NewPartitionProcessor { +impl Processor for BatchPartitionProcessor { fn name(&self) -> String { String::from("PartitionProcessor") } @@ -737,7 +737,7 @@ impl Processor for NewPartitionProcessor { fn process(&mut self) -> Result<()> { if let Some(block) = self.input_data.take() { let partitioned_data = self.exchange.partition(block, self.to_partition)?; - self.output_data = Some(DataBlock::empty_with_meta(ShuffleMeta::create( + self.output_data = Some(DataBlock::empty_with_meta(ExchangeMeta::create( partitioned_data, ))); } @@ -746,7 +746,7 @@ impl Processor for NewPartitionProcessor { } } -pub struct ExchangeShuffleProcessor { +pub struct BatchExchangeProcessor { input: Vec>, output: Vec>, @@ -764,7 +764,7 @@ pub struct ExchangeShuffleProcessor { matrix: Vec>, } -impl ExchangeShuffleProcessor { +impl BatchExchangeProcessor { pub fn create( input: Vec>, output: Vec>, @@ -780,7 +780,7 @@ impl ExchangeShuffleProcessor { matrix.push(VecDeque::new()); } - ProcessorPtr::create(Box::new(ExchangeShuffleProcessor { + ProcessorPtr::create(Box::new(BatchExchangeProcessor { input, output, matrix, @@ -797,9 +797,9 @@ impl ExchangeShuffleProcessor { } } -impl Processor for ExchangeShuffleProcessor { +impl Processor for BatchExchangeProcessor { fn name(&self) -> String { - String::from("ExchangeShuffleProcessor") + String::from("BatchExchangeProcessor") } fn as_any(&mut self) -> &mut dyn Any { @@ -812,7 +812,7 @@ impl Processor for ExchangeShuffleProcessor { let mut data_block = self.input[index].pull_data().unwrap()?; let meta = data_block.take_meta().unwrap(); - let meta = ShuffleMeta::downcast_from(meta).unwrap(); + let meta = ExchangeMeta::downcast_from(meta).unwrap(); for (idx, block) in meta.data_blocks.into_iter().enumerate() { self.matrix[idx].push_back(block); @@ -879,7 +879,7 @@ impl Processor for ExchangeShuffleProcessor { } } - self.output[idx].push_data(Ok(DataBlock::empty_with_meta(ShuffleMeta::create( + self.output[idx].push_data(Ok(DataBlock::empty_with_meta(ExchangeMeta::create( output_data, )))); return Ok(Event::NeedConsume); @@ -909,7 +909,7 @@ impl Processor for ExchangeShuffleProcessor { } } -pub struct NewMergePartitionProcessor { +pub struct BatchMergePartitionProcessor { input: Arc, output: Arc, @@ -919,13 +919,13 @@ pub struct NewMergePartitionProcessor { exchange: Arc, } -impl NewMergePartitionProcessor { +impl BatchMergePartitionProcessor { pub fn create( input: Arc, output: Arc, exchange: Arc, ) -> ProcessorPtr { - ProcessorPtr::create(Box::new(NewMergePartitionProcessor { + ProcessorPtr::create(Box::new(BatchMergePartitionProcessor { input, output, input_data: None, @@ -935,7 +935,7 @@ impl NewMergePartitionProcessor { } } -impl Processor for NewMergePartitionProcessor { +impl Processor for BatchMergePartitionProcessor { fn name(&self) -> String { String::from("MergePartitionProcessor") } @@ -977,7 +977,7 @@ impl Processor for NewMergePartitionProcessor { fn process(&mut self) -> Result<()> { if let Some(mut block) = self.input_data.take() { let meta = block.take_meta().unwrap(); - let meta = ShuffleMeta::downcast_from(meta).unwrap(); + let meta = ExchangeMeta::downcast_from(meta).unwrap(); self.output_data .extend(self.exchange.merge_output(meta.data_blocks)?); } diff --git a/src/query/service/src/pipelines/builders/builder_hilbert_partition.rs b/src/query/service/src/pipelines/builders/builder_hilbert_partition.rs index dd9ab7edd4a38..cc3a2b2ab90ad 100644 --- a/src/query/service/src/pipelines/builders/builder_hilbert_partition.rs +++ b/src/query/service/src/pipelines/builders/builder_hilbert_partition.rs @@ -49,7 +49,7 @@ impl PipelineBuilder { self.main_pipeline.exchange( num_processors, HilbertPartitionExchange::create(partition.num_partitions), - ); + )?; let settings = self.ctx.get_settings(); let disk_bytes_limit = settings.get_window_partition_spilling_to_disk_bytes_limit()?; diff --git a/src/query/service/src/pipelines/builders/builder_window.rs b/src/query/service/src/pipelines/builders/builder_window.rs index ae0dffc1dc73b..6e531b5200c4b 100644 --- a/src/query/service/src/pipelines/builders/builder_window.rs +++ b/src/query/service/src/pipelines/builders/builder_window.rs @@ -184,12 +184,12 @@ impl PipelineBuilder { top_n.func, num_partitions as u64, ), - ) + )? } else { self.main_pipeline.exchange( num_processors, WindowPartitionExchange::create(partition_by.clone(), num_partitions), - ); + )?; } let disk_bytes_limit = settings.get_window_partition_spilling_to_disk_bytes_limit()?; 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 index cb9f98bbbf3d7..9b38fc2bbaa6e 100644 --- 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 @@ -42,7 +42,7 @@ pub fn build_final_aggregate( let pipe_size = settings.get_max_threads()? as usize; // 1. resorting partition - pipeline.exchange(1, Arc::new(ResortingPartition::create())); + pipeline.exchange(1, Arc::new(ResortingPartition::create()))?; // 2. align partitions pipeline.add_transform(|input, output| { diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_exchange.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_exchange.rs index 74ad55d95003b..67a500714be7e 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_exchange.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_exchange.rs @@ -57,7 +57,6 @@ impl ExchangePartition { } let mut repartition_payloads = Vec::with_capacity(n); - let group_types = payload.payload.group_types.clone(); let aggrs = payload.payload.aggrs.clone(); let mut state = PayloadFlushState::default(); @@ -183,7 +182,6 @@ impl Exchange for ExchangePartition { match meta { // already restore in upstream AggregateMeta::SpilledPayload(_) => unreachable!(), - // broadcast final partition to downstream AggregateMeta::FinalPartition(_) => Ok(vec![]), AggregateMeta::AggregatePayload(payload) => Self::partition_aggregate(payload, n), AggregateMeta::InFlightPayload(payload) => { diff --git a/src/query/service/src/servers/flight/v1/exchange/exchange_sink.rs b/src/query/service/src/servers/flight/v1/exchange/exchange_sink.rs index b7d9e4abfe164..ef606b21e3ac6 100644 --- a/src/query/service/src/servers/flight/v1/exchange/exchange_sink.rs +++ b/src/query/service/src/servers/flight/v1/exchange/exchange_sink.rs @@ -62,7 +62,7 @@ impl ExchangeSink { compression, Arc::new(Box::new(MergeFlightScatter)), ), - ), + )?, false => pipeline.exchange( 1, FlightExchange::::create( @@ -70,7 +70,7 @@ impl ExchangeSink { compression, Arc::new(Box::new(MergeFlightScatter)), ), - ), + )?, }; assert_eq!(senders.len(), 1); diff --git a/src/query/service/src/servers/flight/v1/exchange/exchange_transform_shuffle.rs b/src/query/service/src/servers/flight/v1/exchange/exchange_transform_shuffle.rs index 5ed1ce361589f..9cefcde59d441 100644 --- a/src/query/service/src/servers/flight/v1/exchange/exchange_transform_shuffle.rs +++ b/src/query/service/src/servers/flight/v1/exchange/exchange_transform_shuffle.rs @@ -72,7 +72,7 @@ pub fn exchange_shuffle( compression, params.shuffle_scatter.clone(), ), - ), + )?, false => pipeline.exchange( params.destination_ids.len(), FlightExchange::::create( @@ -80,7 +80,7 @@ pub fn exchange_shuffle( compression, params.shuffle_scatter.clone(), ), - ), + )?, }; Ok(()) From 963185515c81b0329e613512185d97219d4e9ed9 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 14 Apr 2025 13:38:49 +0800 Subject: [PATCH 74/76] refactor(query): refactor aggreagte spill code --- .../transforms/aggregator/transform_partition_bucket.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 index 9b38fc2bbaa6e..142c20e452acd 100644 --- 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 @@ -71,7 +71,7 @@ pub fn build_final_aggregate( // 5. exchange local let pipe_size = pipeline.output_len(); - pipeline.new_exchange( + pipeline.exchange( pipe_size, ExchangePartition::create(pipe_size, params.clone()), )?; From d8fe94def16af4ec3ccdb64254c7415f2e4e402f Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 14 Apr 2025 14:07:59 +0800 Subject: [PATCH 75/76] refactor(query): refactor aggreagte spill code --- .../core/src/processors/shuffle_processor.rs | 26 +++++++++++++++++++ 1 file changed, 26 insertions(+) diff --git a/src/query/pipeline/core/src/processors/shuffle_processor.rs b/src/query/pipeline/core/src/processors/shuffle_processor.rs index 893fc77fdcb30..4ca2c292510c6 100644 --- a/src/query/pipeline/core/src/processors/shuffle_processor.rs +++ b/src/query/pipeline/core/src/processors/shuffle_processor.rs @@ -736,6 +736,10 @@ impl Processor for BatchPartitionProcessor { fn process(&mut self) -> Result<()> { if let Some(block) = self.input_data.take() { + if T::SKIP_EMPTY_DATA_BLOCK && block.is_empty() { + return Ok(()); + } + let partitioned_data = self.exchange.partition(block, self.to_partition)?; self.output_data = Some(DataBlock::empty_with_meta(ExchangeMeta::create( partitioned_data, @@ -985,3 +989,25 @@ impl Processor for BatchMergePartitionProcessor { Ok(()) } } + +// +// pub struct BatchSortingExchangeProcessor { +// exchange: Arc, +// +// inputs: Vec>, +// outputs: Vec>, +// } +// +// impl Processor for BatchSortingExchangeProcessor { +// fn name(&self) -> String { +// String::from("BatchSortingShuffleProcessor") +// } +// +// fn as_any(&mut self) -> &mut dyn Any { +// self +// } +// +// fn event_with_cause(&mut self, _cause: EventCause) -> Result { +// todo!() +// } +// } From 68cf7c07ff1b9a340f399aa5a9aec39c3c657c6d Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 14 Apr 2025 14:12:18 +0800 Subject: [PATCH 76/76] refactor(query): refactor aggreagte spill code --- src/query/pipeline/core/src/processors/shuffle_processor.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/src/query/pipeline/core/src/processors/shuffle_processor.rs b/src/query/pipeline/core/src/processors/shuffle_processor.rs index 4ca2c292510c6..3ba0673135f34 100644 --- a/src/query/pipeline/core/src/processors/shuffle_processor.rs +++ b/src/query/pipeline/core/src/processors/shuffle_processor.rs @@ -990,7 +990,6 @@ impl Processor for BatchMergePartitionProcessor { } } -// // pub struct BatchSortingExchangeProcessor { // exchange: Arc, //