From 4da8c953754e4baa6e9a1352635828439a9d321d Mon Sep 17 00:00:00 2001 From: coldWater Date: Fri, 8 Nov 2024 15:00:50 +0800 Subject: [PATCH 01/61] reservoir_sampling Signed-off-by: coldWater --- src/query/expression/src/lib.rs | 1 + src/query/expression/src/simpler/mod.rs | 15 +++ .../src/simpler/reservoir_sampling.rs | 96 +++++++++++++++++++ 3 files changed, 112 insertions(+) create mode 100644 src/query/expression/src/simpler/mod.rs create mode 100644 src/query/expression/src/simpler/reservoir_sampling.rs diff --git a/src/query/expression/src/lib.rs b/src/query/expression/src/lib.rs index b87d5b15d6303..0175dba4e3cae 100755 --- a/src/query/expression/src/lib.rs +++ b/src/query/expression/src/lib.rs @@ -65,6 +65,7 @@ mod register_vectorize; pub mod row; pub mod sampler; pub mod schema; +pub mod simpler; pub mod type_check; pub mod types; pub mod utils; diff --git a/src/query/expression/src/simpler/mod.rs b/src/query/expression/src/simpler/mod.rs new file mode 100644 index 0000000000000..d6e99fdf5a2e5 --- /dev/null +++ b/src/query/expression/src/simpler/mod.rs @@ -0,0 +1,15 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +mod reservoir_sampling; diff --git a/src/query/expression/src/simpler/reservoir_sampling.rs b/src/query/expression/src/simpler/reservoir_sampling.rs new file mode 100644 index 0000000000000..898303894a396 --- /dev/null +++ b/src/query/expression/src/simpler/reservoir_sampling.rs @@ -0,0 +1,96 @@ +// 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 rand::Rng; + +/// An implementation of Algorithm `L` (https://en.wikipedia.org/wiki/Reservoir_sampling#An_optimal_algorithm) +pub struct AlgoL<'a, R: Rng + ?Sized> { + k: usize, + r: &'a mut R, + + i: usize, + w: f64, +} + +impl AlgoL<'_, R> { + pub fn new<'a>(k: usize, rng: &'a mut R) -> AlgoL<'a, R> { + assert!(k > 0); + let mut al = AlgoL::<'a, R> { + k, + i: k - 1, + w: 1.0, + r: rng, + }; + al.update_w(); + al + } + + pub fn next_index(&mut self) -> usize { + let i = (self.rng().log2() / (1.0 - self.w).log2()).floor() + 1.0 + self.i as f64; + if i.is_normal() && i < u64::MAX as f64 { + i as usize + } else { + usize::MAX + } + } + + pub fn pos(&mut self) -> usize { + self.r.sample(rand::distributions::Uniform::new(0, self.k)) + } + + pub fn update(&mut self, i: usize) { + self.i = i; + self.update_w() + } + + fn rng(&mut self) -> f64 { + self.r.sample(rand::distributions::Open01) + } + + fn update_w(&mut self) { + self.w *= (self.rng().log2() / self.k as f64).exp2(); // rng ^ (1/k) + } +} + +#[cfg(test)] +mod tests { + use rand::rngs::StdRng; + use rand::SeedableRng; + + use super::*; + + #[test] + fn test_algo_l() { + let mut rng = StdRng::seed_from_u64(0); + let mut sample = vec![0_u64; 10]; + + let mut al = AlgoL::new(10, &mut rng); + for (i, v) in sample.iter_mut().enumerate() { + *v = i as u64 + } + + loop { + let i = al.next_index(); + if i < 100 { + sample[al.pos()] = i as u64; + al.update(i) + } else { + break; + } + } + + let want: Vec = vec![69, 49, 53, 83, 4, 72, 88, 38, 45, 27]; + assert_eq!(want, sample) + } +} From 667bef3d30e6e01cfac7bf4886ee0dfabcc93829 Mon Sep 17 00:00:00 2001 From: coldWater Date: Fri, 8 Nov 2024 20:54:16 +0800 Subject: [PATCH 02/61] Simpler Signed-off-by: coldWater --- src/query/expression/src/simpler/mod.rs | 1 + .../src/simpler/reservoir_sampling.rs | 47 +++-- src/query/expression/src/simpler/simpler.rs | 176 ++++++++++++++++++ 3 files changed, 198 insertions(+), 26 deletions(-) create mode 100644 src/query/expression/src/simpler/simpler.rs diff --git a/src/query/expression/src/simpler/mod.rs b/src/query/expression/src/simpler/mod.rs index d6e99fdf5a2e5..1917509188299 100644 --- a/src/query/expression/src/simpler/mod.rs +++ b/src/query/expression/src/simpler/mod.rs @@ -13,3 +13,4 @@ // limitations under the License. mod reservoir_sampling; +mod simpler; diff --git a/src/query/expression/src/simpler/reservoir_sampling.rs b/src/query/expression/src/simpler/reservoir_sampling.rs index 898303894a396..2b9cd32d04d10 100644 --- a/src/query/expression/src/simpler/reservoir_sampling.rs +++ b/src/query/expression/src/simpler/reservoir_sampling.rs @@ -12,34 +12,33 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::num::NonZeroUsize; + use rand::Rng; /// An implementation of Algorithm `L` (https://en.wikipedia.org/wiki/Reservoir_sampling#An_optimal_algorithm) -pub struct AlgoL<'a, R: Rng + ?Sized> { +pub struct AlgoL { k: usize, - r: &'a mut R, - - i: usize, w: f64, + + r: R, } -impl AlgoL<'_, R> { - pub fn new<'a>(k: usize, rng: &'a mut R) -> AlgoL<'a, R> { - assert!(k > 0); - let mut al = AlgoL::<'a, R> { - k, - i: k - 1, +impl AlgoL { + pub fn new(k: NonZeroUsize, r: R) -> Self { + let mut al = Self { + k: k.into(), w: 1.0, - r: rng, + r, }; al.update_w(); al } - pub fn next_index(&mut self) -> usize { - let i = (self.rng().log2() / (1.0 - self.w).log2()).floor() + 1.0 + self.i as f64; - if i.is_normal() && i < u64::MAX as f64 { - i as usize + pub fn search(&mut self) -> usize { + let s = (self.rng().log2() / (1.0 - self.w).log2()).floor() + 1.0; + if s.is_normal() { + s as usize } else { usize::MAX } @@ -49,18 +48,13 @@ impl AlgoL<'_, R> { self.r.sample(rand::distributions::Uniform::new(0, self.k)) } - pub fn update(&mut self, i: usize) { - self.i = i; - self.update_w() + pub fn update_w(&mut self) { + self.w *= (self.rng().log2() / self.k as f64).exp2(); // rng ^ (1/k) } fn rng(&mut self) -> f64 { self.r.sample(rand::distributions::Open01) } - - fn update_w(&mut self) { - self.w *= (self.rng().log2() / self.k as f64).exp2(); // rng ^ (1/k) - } } #[cfg(test)] @@ -72,19 +66,20 @@ mod tests { #[test] fn test_algo_l() { - let mut rng = StdRng::seed_from_u64(0); + let rng = StdRng::seed_from_u64(0); let mut sample = vec![0_u64; 10]; - let mut al = AlgoL::new(10, &mut rng); + let mut al = AlgoL::new(10.try_into().unwrap(), rng); for (i, v) in sample.iter_mut().enumerate() { *v = i as u64 } + let mut i = 9; loop { - let i = al.next_index(); + i += al.search(); if i < 100 { sample[al.pos()] = i as u64; - al.update(i) + al.update_w() } else { break; } diff --git a/src/query/expression/src/simpler/simpler.rs b/src/query/expression/src/simpler/simpler.rs new file mode 100644 index 0000000000000..15b7dfa82a854 --- /dev/null +++ b/src/query/expression/src/simpler/simpler.rs @@ -0,0 +1,176 @@ +// 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::HashSet; + +use rand::Rng; + +use super::reservoir_sampling::AlgoL; +use crate::BlockRowIndex; +use crate::DataBlock; + +pub struct Simpler { + columns: Vec, + k: usize, + + blocks: Vec, + indices: Vec, + core: AlgoL, + + s: usize, +} + +impl Simpler { + pub fn new(columns: Vec, k: usize, rng: R) -> Self { + let core = AlgoL::new(k.try_into().unwrap(), rng); + Self { + columns, + blocks: Vec::new(), + indices: Vec::with_capacity(k), + k, + core, + s: usize::MAX, + } + } + + pub fn add_block(&mut self, data: DataBlock) { + let rows = data.num_rows(); + assert!(rows > 0); + let block_idx = self.blocks.len() as u32; + let change = self.add_indices(rows, block_idx); + if change { + let columns = self + .columns + .iter() + .map(|&offset| data.get_by_offset(offset).to_owned()) + .collect::>(); + + self.blocks.push(DataBlock::new(columns, rows)); + } + } + + fn add_indices(&mut self, rows: usize, block_idx: u32) -> bool { + let mut change = false; + let mut cur: usize = 0; + if self.indices.len() < self.k { + if rows + self.indices.len() <= self.k { + for i in 0..rows { + self.indices.push((block_idx, i as u32, 1)); + } + if self.indices.len() == self.k { + self.s = self.core.search() + } + return true; + } + while self.indices.len() < self.k { + self.indices.push((block_idx, cur as u32, 1)); + cur += 1; + } + self.s = self.core.search(); + change = true; + } + + while rows - cur > self.s { + change = true; + cur += self.s; + self.indices[self.core.pos()] = (block_idx, cur as u32, 1); + self.core.update_w(); + self.s = self.core.search(); + } + + self.s -= rows - cur; + change + } + + pub fn compact_indices(&mut self) { + let used_set: HashSet<_> = self.indices.iter().map(|&(b, _, _)| b).collect(); + if used_set.len() == self.blocks.len() { + return; + } + + let mut used: Vec<_> = used_set.iter().cloned().collect(); + used.sort(); + + self.indices = self + .indices + .drain(..) + .map(|(b, r, c)| (used.binary_search(&b).unwrap() as u32, r, c)) + .collect(); + + self.blocks = self + .blocks + .drain(..) + .enumerate() + .filter_map(|(i, block)| { + if used_set.contains(&(i as u32)) { + Some(block) + } else { + None + } + }) + .collect(); + } + + pub fn compact_blocks(&mut self) { + let rows = self.indices.len(); + let block = DataBlock::take_blocks(&self.blocks, &self.indices, rows); + self.blocks.clear(); + self.blocks.push(block); + + for (i, (b, r, _)) in self.indices.iter_mut().enumerate() { + *b = 0; + *r = i as u32; + } + } + + pub fn memory_size(self) -> usize { + self.blocks.iter().map(|b| b.memory_size()).sum() + } +} + +#[cfg(test)] +mod tests { + use rand::rngs::StdRng; + use rand::SeedableRng; + + use super::*; + + #[test] + fn test_add_indeces() { + let rng = StdRng::seed_from_u64(0); + let k = 5; + let core = AlgoL::new(k.try_into().unwrap(), rng); + let mut simpler = Simpler { + columns: vec![0], + blocks: Vec::new(), + indices: Vec::new(), + k, + core, + s: usize::MAX, + }; + + simpler.add_indices(15, 0); + + let want: Vec = + vec![(0, 10, 1), (0, 1, 1), (0, 2, 1), (0, 8, 1), (0, 12, 1)]; + assert_eq!(&want, &simpler.indices); + assert_eq!(0, simpler.s); + + simpler.add_indices(20, 1); + + let want: Vec = vec![(1, 0, 1), (0, 1, 1), (1, 6, 1), (0, 8, 1), (1, 9, 1)]; + assert_eq!(&want, &simpler.indices); + assert_eq!(1, simpler.s); + } +} From 48eafc8898d55edd3634a4905ce8ba91b0fae7dd Mon Sep 17 00:00:00 2001 From: coldWater Date: Tue, 12 Nov 2024 12:12:19 +0800 Subject: [PATCH 03/61] TransformSortSimple Signed-off-by: coldWater --- src/common/base/src/base/watch_notify.rs | 23 +- .../expression/src/{simpler => }/simpler.rs | 52 ++- src/query/pipeline/core/src/processors/mod.rs | 1 + .../core/src/processors/shuffle_processor.rs | 10 +- .../src/processors/transforms/mod.rs | 2 +- .../transforms/transform_multi_sort_merge.rs | 4 +- .../src/pipelines/builders/builder_sort.rs | 93 ++++- .../pipelines/processors/transforms/mod.rs | 1 + .../processors/transforms/sort}/mod.rs | 9 +- .../transforms/sort/sort_exchange.rs | 148 ++++++++ .../processors/transforms/sort/sort_merge.rs | 62 ++++ .../processors/transforms/sort/sort_simple.rs | 333 ++++++++++++++++++ .../processors/transforms/sort/sort_wait.rs | 113 ++++++ src/query/settings/src/settings_default.rs | 7 + .../settings/src/settings_getter_setter.rs | 4 + 15 files changed, 825 insertions(+), 37 deletions(-) rename src/query/expression/src/{simpler => }/simpler.rs (80%) rename src/query/{expression/src/simpler => service/src/pipelines/processors/transforms/sort}/mod.rs (83%) create mode 100644 src/query/service/src/pipelines/processors/transforms/sort/sort_exchange.rs create mode 100644 src/query/service/src/pipelines/processors/transforms/sort/sort_merge.rs create mode 100644 src/query/service/src/pipelines/processors/transforms/sort/sort_simple.rs create mode 100644 src/query/service/src/pipelines/processors/transforms/sort/sort_wait.rs diff --git a/src/common/base/src/base/watch_notify.rs b/src/common/base/src/base/watch_notify.rs index be05dfc9028c0..61d0528a94c77 100644 --- a/src/common/base/src/base/watch_notify.rs +++ b/src/common/base/src/base/watch_notify.rs @@ -41,6 +41,16 @@ impl WatchNotify { let _ = rx.changed().await; } + pub fn has_notified(&self) -> bool { + match self.rx.has_changed() { + Ok(b) => b, + Err(_) => { + // The sender has never dropped before + unreachable!() + } + } + } + pub fn notify_waiters(&self) { let _ = self.tx.send_replace(true); } @@ -61,11 +71,18 @@ mod tests { #[tokio::test] async fn test_notify_waiters_ahead() { let notify = WatchNotify::new(); + assert!(!notify.has_notified()); + let notified1 = notify.notified(); + assert!(!notify.has_notified()); + // notify_waiters ahead of notified being instantiated and awaited notify.notify_waiters(); - + assert!(notify.has_notified()); // this should not await indefinitely - let notified = notify.notified(); - notified.await; + let notified2 = notify.notified(); + notified2.await; + + notified1.await; + assert!(notify.has_notified()); } } diff --git a/src/query/expression/src/simpler/simpler.rs b/src/query/expression/src/simpler.rs similarity index 80% rename from src/query/expression/src/simpler/simpler.rs rename to src/query/expression/src/simpler.rs index 15b7dfa82a854..ab928d721ec77 100644 --- a/src/query/expression/src/simpler/simpler.rs +++ b/src/query/expression/src/simpler.rs @@ -12,17 +12,20 @@ // See the License for the specific language governing permissions and // limitations under the License. +mod reservoir_sampling; + use std::collections::HashSet; use rand::Rng; +use reservoir_sampling::AlgoL; -use super::reservoir_sampling::AlgoL; use crate::BlockRowIndex; use crate::DataBlock; pub struct Simpler { columns: Vec, k: usize, + block_size: usize, blocks: Vec, indices: Vec, @@ -32,19 +35,20 @@ pub struct Simpler { } impl Simpler { - pub fn new(columns: Vec, k: usize, rng: R) -> Self { + pub fn new(columns: Vec, block_size: usize, k: usize, rng: R) -> Self { let core = AlgoL::new(k.try_into().unwrap(), rng); Self { columns, blocks: Vec::new(), indices: Vec::with_capacity(k), k, + block_size, core, s: usize::MAX, } } - pub fn add_block(&mut self, data: DataBlock) { + pub fn add_block(&mut self, data: DataBlock) -> bool { let rows = data.num_rows(); assert!(rows > 0); let block_idx = self.blocks.len() as u32; @@ -57,7 +61,11 @@ impl Simpler { .collect::>(); self.blocks.push(DataBlock::new(columns, rows)); + if self.blocks.len() > self.k { + self.compact_blocks() + } } + change } fn add_indices(&mut self, rows: usize, block_idx: u32) -> bool { @@ -123,20 +131,35 @@ impl Simpler { } pub fn compact_blocks(&mut self) { - let rows = self.indices.len(); - let block = DataBlock::take_blocks(&self.blocks, &self.indices, rows); - self.blocks.clear(); - self.blocks.push(block); - - for (i, (b, r, _)) in self.indices.iter_mut().enumerate() { - *b = 0; - *r = i as u32; - } + self.blocks = self + .indices + .chunks_mut(self.block_size) + .enumerate() + .map(|(i, indices)| { + let rows = indices.len(); + let block = DataBlock::take_blocks(&self.blocks, indices, rows); + + for (j, (b, r, _)) in indices.iter_mut().enumerate() { + *b = i as u32; + *r = j as u32; + } + + block + }) + .collect::>(); } pub fn memory_size(self) -> usize { self.blocks.iter().map(|b| b.memory_size()).sum() } + + pub fn take_blocks(&mut self) -> Vec { + std::mem::take(&mut self.blocks) + } + + pub fn k(&self) -> usize { + self.k + } } #[cfg(test)] @@ -147,15 +170,16 @@ mod tests { use super::*; #[test] - fn test_add_indeces() { + fn test_add_indices() { let rng = StdRng::seed_from_u64(0); let k = 5; let core = AlgoL::new(k.try_into().unwrap(), rng); let mut simpler = Simpler { columns: vec![0], + k, + block_size: 65536, blocks: Vec::new(), indices: Vec::new(), - k, core, s: usize::MAX, }; diff --git a/src/query/pipeline/core/src/processors/mod.rs b/src/query/pipeline/core/src/processors/mod.rs index b9468d09fb984..add24f802c251 100644 --- a/src/query/pipeline/core/src/processors/mod.rs +++ b/src/query/pipeline/core/src/processors/mod.rs @@ -41,5 +41,6 @@ pub use resize_processor::ResizeProcessor; pub use sequence_group::SequenceGroupProcessor; 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..dac49ea50b79e 100644 --- a/src/query/pipeline/core/src/processors/shuffle_processor.rs +++ b/src/query/pipeline/core/src/processors/shuffle_processor.rs @@ -345,7 +345,10 @@ impl Processor for MergePartitionProcessor { input.set_need_data(); } - if all_inputs_finished { + if all_inputs_finished + && (!matches!(T::STRATEGY, MultiwayStrategy::Custom) + || self.inputs_data.iter().all(Option::is_none)) + { self.output.finish(); return Ok(Event::Finished); } @@ -357,6 +360,11 @@ impl Processor for MergePartitionProcessor { self.output.push_data(Ok(block)); return Ok(Event::NeedConsume); } + + if all_inputs_finished && self.inputs_data.iter().all(Option::is_none) { + self.output.finish(); + return Ok(Event::Finished); + } } Ok(Event::NeedData) diff --git a/src/query/pipeline/transforms/src/processors/transforms/mod.rs b/src/query/pipeline/transforms/src/processors/transforms/mod.rs index ec6ca0faf96a0..8fe951ce2c89a 100644 --- a/src/query/pipeline/transforms/src/processors/transforms/mod.rs +++ b/src/query/pipeline/transforms/src/processors/transforms/mod.rs @@ -41,7 +41,7 @@ pub use transform_compact_builder::*; pub use transform_compact_no_split_builder::*; pub use transform_dummy::*; pub use transform_k_way_merge_sort::*; -pub use transform_multi_sort_merge::try_add_multi_sort_merge; +pub use transform_multi_sort_merge::*; pub use transform_pipeline_helper::TransformPipelineHelper; pub use transform_retry_async::*; pub use transform_sort_merge::sort_merge; diff --git a/src/query/pipeline/transforms/src/processors/transforms/transform_multi_sort_merge.rs b/src/query/pipeline/transforms/src/processors/transforms/transform_multi_sort_merge.rs index 573315604e414..5ab82226a9940 100644 --- a/src/query/pipeline/transforms/src/processors/transforms/transform_multi_sort_merge.rs +++ b/src/query/pipeline/transforms/src/processors/transforms/transform_multi_sort_merge.rs @@ -77,7 +77,7 @@ pub fn try_add_multi_sort_merge( } let output_port = OutputPort::create(); - let processor = ProcessorPtr::create(create_processor( + let processor = ProcessorPtr::create(create_multi_sort_merge_processor( inputs_port.clone(), output_port.clone(), schema, @@ -98,7 +98,7 @@ pub fn try_add_multi_sort_merge( } } -fn create_processor( +pub fn create_multi_sort_merge_processor( inputs: Vec>, output: Arc, schema: DataSchemaRef, diff --git a/src/query/service/src/pipelines/builders/builder_sort.rs b/src/query/service/src/pipelines/builders/builder_sort.rs index 5665787f45543..a723eb8ddcf35 100644 --- a/src/query/service/src/pipelines/builders/builder_sort.rs +++ b/src/query/service/src/pipelines/builders/builder_sort.rs @@ -34,6 +34,11 @@ use databend_common_storages_fuse::TableContext; use databend_storages_common_cache::TempDirManager; use crate::pipelines::memory_settings::MemorySettingsExt; +use crate::pipelines::processors::transforms::sort::add_range_shuffle; +use crate::pipelines::processors::transforms::sort::add_range_shuffle_merge; +use crate::pipelines::processors::transforms::sort::add_sort_simple; +use crate::pipelines::processors::transforms::sort::SortSimpleState; +use crate::pipelines::processors::transforms::TransformLimit; use crate::pipelines::processors::transforms::TransformSortBuilder; use crate::pipelines::PipelineBuilder; use crate::sessions::QueryContext; @@ -135,9 +140,16 @@ impl PipelineBuilder { None => { // Build for single node mode. // We build the full sort pipeline for it. - builder - .remove_order_col_at_last() - .build_full_sort_pipeline(&mut self.main_pipeline) + let k = self.settings.get_range_shuffle_sort_simple_size()?; + if k > 0 && self.main_pipeline.output_len() > 1 { + builder + .remove_order_col_at_last() + .build_range_shuffle_sort_pipeline(&mut self.main_pipeline, k) + } else { + builder + .remove_order_col_at_last() + .build_full_sort_pipeline(&mut self.main_pipeline) + } } } } @@ -150,6 +162,7 @@ pub struct SortPipelineBuilder { limit: Option, block_size: usize, remove_order_col_at_last: bool, + enable_loser_tree: bool, } impl SortPipelineBuilder { @@ -158,7 +171,9 @@ impl SortPipelineBuilder { schema: DataSchemaRef, sort_desc: Arc<[SortColumnDescription]>, ) -> Result { - let block_size = ctx.get_settings().get_max_block_size()? as usize; + let settings = ctx.get_settings(); + let block_size = settings.get_max_block_size()? as usize; + let enable_loser_tree = settings.get_enable_loser_tree_merge_sort()?; Ok(Self { ctx, schema, @@ -166,6 +181,7 @@ impl SortPipelineBuilder { limit: None, block_size, remove_order_col_at_last: false, + enable_loser_tree, }) } @@ -197,11 +213,52 @@ impl SortPipelineBuilder { self.build_merge_sort_pipeline(pipeline, false) } - pub fn build_merge_sort_pipeline( - self, - pipeline: &mut Pipeline, - order_col_generated: bool, - ) -> Result<()> { + fn build_range_shuffle_sort_pipeline(self, pipeline: &mut Pipeline, k: usize) -> Result<()> { + let inputs = pipeline.output_len(); + let settings = self.ctx.get_settings(); + let max_threads = settings.get_max_threads()? as usize; + let simple = SortSimpleState::new( + inputs, + max_threads, + self.schema.clone(), + self.sort_desc.clone(), + ); + + add_sort_simple(pipeline, simple.clone(), self.sort_desc.clone(), k)?; + + // Partial sort + pipeline.add_transformer(|| { + TransformSortPartial::new( + LimitType::from_limit_rows(self.limit), + self.sort_desc.clone(), + ) + }); + + self.build_merge_sort(pipeline, false)?; + + add_range_shuffle( + pipeline, + simple.clone(), + self.sort_desc.clone(), + self.schema.clone(), + self.block_size, + self.limit, + self.remove_order_col_at_last, + self.enable_loser_tree, + )?; + + add_range_shuffle_merge(pipeline)?; + + if self.limit.is_none() { + return Ok(()); + } + + pipeline.add_transform(|input, output| { + TransformLimit::try_create(self.limit, 0, input, output).map(ProcessorPtr::create) + }) + } + + fn build_merge_sort(&self, pipeline: &mut Pipeline, order_col_generated: bool) -> Result<()> { // Merge sort let need_multi_merge = pipeline.output_len() > 1; let output_order_col = need_multi_merge || !self.remove_order_col_at_last; @@ -257,7 +314,16 @@ impl SortPipelineBuilder { .with_enable_loser_tree(enable_loser_tree); Ok(ProcessorPtr::create(builder.build()?)) - })?; + }) + } + + pub fn build_merge_sort_pipeline( + self, + pipeline: &mut Pipeline, + order_col_generated: bool, + ) -> Result<()> { + let need_multi_merge = pipeline.output_len() > 1; + self.build_merge_sort(pipeline, order_col_generated)?; if !need_multi_merge { return Ok(()); @@ -269,9 +335,8 @@ impl SortPipelineBuilder { pub fn build_multi_merge(self, pipeline: &mut Pipeline) -> Result<()> { // Multi-pipelines merge sort let settings = self.ctx.get_settings(); - let enable_loser_tree = settings.get_enable_loser_tree_merge_sort()?; - let max_threads = settings.get_max_threads()? as usize; if settings.get_enable_parallel_multi_merge_sort()? { + let max_threads = settings.get_max_threads()? as usize; add_k_way_merge_sort( pipeline, self.schema.clone(), @@ -280,7 +345,7 @@ impl SortPipelineBuilder { self.limit, self.sort_desc, self.remove_order_col_at_last, - enable_loser_tree, + self.enable_loser_tree, ) } else { try_add_multi_sort_merge( @@ -290,7 +355,7 @@ impl SortPipelineBuilder { self.limit, self.sort_desc, self.remove_order_col_at_last, - enable_loser_tree, + self.enable_loser_tree, ) } } diff --git a/src/query/service/src/pipelines/processors/transforms/mod.rs b/src/query/service/src/pipelines/processors/transforms/mod.rs index 7776ca90e1780..0b7c3241d0449 100644 --- a/src/query/service/src/pipelines/processors/transforms/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/mod.rs @@ -18,6 +18,7 @@ mod broadcast; mod hash_join; pub(crate) mod range_join; mod runtime_pool; +pub mod sort; mod transform_add_computed_columns; mod transform_add_const_columns; mod transform_add_internal_columns; diff --git a/src/query/expression/src/simpler/mod.rs b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs similarity index 83% rename from src/query/expression/src/simpler/mod.rs rename to src/query/service/src/pipelines/processors/transforms/sort/mod.rs index 1917509188299..8049256d68e67 100644 --- a/src/query/expression/src/simpler/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs @@ -12,5 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -mod reservoir_sampling; -mod simpler; +mod sort_exchange; +mod sort_merge; +mod sort_simple; +mod sort_wait; + +pub use sort_merge::*; +pub use sort_simple::*; diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange.rs new file mode 100644 index 0000000000000..1bdc66673ebb4 --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange.rs @@ -0,0 +1,148 @@ +// 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::iter; +use std::marker::PhantomData; +use std::sync::Arc; + +use databend_common_exception::Result; +use databend_common_expression::DataBlock; +use databend_common_expression::DataSchemaRef; +use databend_common_expression::SortColumnDescription; +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::Pipe; +use databend_common_pipeline_core::PipeItem; +use databend_common_pipeline_transforms::processors::sort::select_row_type; +use databend_common_pipeline_transforms::processors::sort::Rows; +use databend_common_pipeline_transforms::processors::sort::RowsTypeVisitor; +use databend_common_pipeline_transforms::sort::RowConverter; + +use super::sort_simple::SortSimpleState; +use crate::pipelines::processors::PartitionProcessor; + +pub struct SortRangeExchange { + state: Arc, + _r: PhantomData, +} + +unsafe impl Send for SortRangeExchange {} + +unsafe impl Sync for SortRangeExchange {} + +impl Exchange for SortRangeExchange { + const NAME: &'static str = "SortRange"; + fn partition(&self, data: DataBlock, n: usize) -> Result> { + let bounds = self.state.bounds().unwrap(); + debug_assert_eq!(n, self.state.partitions()); + debug_assert!(bounds.len() < n); + + if data.is_empty() { + return Ok(vec![]); + } + + if bounds.len() == 0 { + return Ok(vec![data]); + } + + let bounds = R::from_column(&bounds)?; + let rows = R::from_column(data.get_last_column())?; + + let mut i = 0; + let mut j = 0; + let mut bound = bounds.row(j); + let mut indices = Vec::new(); + while i < rows.len() { + match rows.row(i).cmp(&bound) { + Ordering::Less => indices.push(j as u32), + Ordering::Greater if j + 1 < bounds.len() => { + j += 1; + bound = bounds.row(j); + continue; + } + _ => indices.push(j as u32 + 1), + } + i += 1; + } + + DataBlock::scatter(&data, &indices, n) + } +} + +pub fn create_exchange_pipe( + inputs: usize, + partitions: usize, + schema: DataSchemaRef, + sort_desc: Arc<[SortColumnDescription]>, + state: Arc, +) -> Pipe { + let mut builder = Builder { + inputs, + partitions, + sort_desc, + schema, + state, + items: Vec::new(), + }; + + select_row_type(&mut builder); + + Pipe::create(inputs, inputs * partitions, builder.items) +} + +struct Builder { + inputs: usize, + partitions: usize, + sort_desc: Arc<[SortColumnDescription]>, + schema: DataSchemaRef, + state: Arc, + items: Vec, +} + +impl RowsTypeVisitor for Builder { + fn visit_type(&mut self) + where + R: Rows + 'static, + C: RowConverter + Send + 'static, + { + let exchange = Arc::new(SortRangeExchange:: { + state: self.state.clone(), + _r: PhantomData, + }); + self.items = iter::repeat_with(|| { + let input = InputPort::create(); + let outputs = iter::repeat_with(OutputPort::create) + .take(self.partitions) + .collect::>(); + + PipeItem::create( + PartitionProcessor::create(input.clone(), outputs.clone(), exchange.clone()), + vec![input], + outputs, + ) + }) + .take(self.inputs) + .collect::>(); + } + + fn schema(&self) -> DataSchemaRef { + self.schema.clone() + } + + fn sort_desc(&self) -> &[SortColumnDescription] { + &self.sort_desc + } +} diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_merge.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_merge.rs new file mode 100644 index 0000000000000..569a1e54e83b3 --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_merge.rs @@ -0,0 +1,62 @@ +// 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::Result; +use databend_common_expression::DataBlock; +use databend_common_pipeline_core::processors::Exchange; +use databend_common_pipeline_core::processors::InputPort; +use databend_common_pipeline_core::processors::MergePartitionProcessor; +use databend_common_pipeline_core::processors::MultiwayStrategy; +use databend_common_pipeline_core::processors::OutputPort; +use databend_common_pipeline_core::Pipe; +use databend_common_pipeline_core::PipeItem; +use databend_common_pipeline_core::Pipeline; + +pub struct TransformSortRangeMerge {} + +impl Exchange for TransformSortRangeMerge { + const NAME: &'static str = "SortRangeMerge"; + const STRATEGY: MultiwayStrategy = MultiwayStrategy::Custom; + + fn partition(&self, block: DataBlock, _: usize) -> Result> { + Ok(vec![block]) + } + + fn multiway_pick(&self, partitions: &[Option]) -> Result { + Ok(partitions.iter().position(Option::is_some).unwrap()) + } +} + +pub fn add_range_shuffle_merge(pipeline: &mut Pipeline) -> Result<()> { + let inputs = pipeline.output_len(); + let inputs_port = (0..inputs).map(|_| InputPort::create()).collect::>(); + let output = OutputPort::create(); + + let processor = MergePartitionProcessor::create( + inputs_port.clone(), + output.clone(), + Arc::new(TransformSortRangeMerge {}), + ); + + let pipe = Pipe::create(inputs, 1, vec![PipeItem::create( + processor, + inputs_port, + vec![output], + )]); + + pipeline.add_pipe(pipe); + Ok(()) +} diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_simple.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_simple.rs new file mode 100644 index 0000000000000..7195769842c42 --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_simple.rs @@ -0,0 +1,333 @@ +// 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::sync::RwLock; + +use databend_common_base::base::WatchNotify; +use databend_common_exception::Result; +use databend_common_expression::simpler::Simpler; +use databend_common_expression::visitor::ValueVisitor; +use databend_common_expression::Column; +use databend_common_expression::DataBlock; +use databend_common_expression::DataSchemaRef; +use databend_common_expression::SortColumnDescription; +use databend_common_expression::SortCompare; +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::processors::create_multi_sort_merge_processor; +use databend_common_pipeline_transforms::processors::sort::convert_rows; +use databend_common_pipeline_transforms::processors::Transform; +use databend_common_pipeline_transforms::TransformPipelineHelper; +use rand::rngs::StdRng; +use rand::SeedableRng; + +use super::sort_exchange::create_exchange_pipe; +use super::sort_wait::TransformSortSimpleWait; + +pub struct SortSimpleState { + inner: RwLock, + pub(super) done: WatchNotify, +} + +impl SortSimpleState { + pub fn partitions(&self) -> usize { + self.inner.read().unwrap().partitions + } +} + +struct StateInner { + partitions: usize, + // schema for bounds DataBlock + schema: DataSchemaRef, + // sort_desc for bounds DataBlock + sort_desc: Vec, + partial: Vec>, + bounds: Option, +} + +impl StateInner { + fn determine_bounds(&mut self) -> Result<()> { + let partial = std::mem::take(&mut self.partial) + .into_iter() + .filter_map(|b| { + let b = b.unwrap(); + if b.is_empty() { + None + } else { + Some(b) + } + }) + .collect::>(); + + if partial.is_empty() { + let bounds = convert_rows( + self.schema.clone(), + &self.sort_desc, + DataBlock::empty_with_schema(self.schema.clone()), + )?; + + self.bounds = Some(bounds); + return Ok(()); + } + + let candidates = DataBlock::concat(&partial)?; + let rows = candidates.num_rows(); + + let mut sort_compare = SortCompare::with_force_equality(self.sort_desc.clone(), rows); + + for desc in &self.sort_desc { + let array = candidates.get_by_offset(desc.offset).value.clone(); + sort_compare.visit_value(array)?; + sort_compare.increment_column_index(); + } + + let equality = sort_compare.equality_index().to_vec(); + let permutation = sort_compare.take_permutation(); + + let step = permutation.len() as f64 / self.partitions as f64; + let mut target = step; + let mut bounds = Vec::with_capacity(self.partitions - 1); + let mut equals = true; + for (i, (&pos, eq)) in permutation.iter().zip(equality).enumerate() { + if bounds.len() >= self.partitions - 1 { + break; + } + if equals && eq == 0 { + equals = false + } + if i as f64 >= target && (!equals || i != 0) { + bounds.push(pos); + target += step; + equals = true + } + } + + let bounds = convert_rows( + self.schema.clone(), + &self.sort_desc, + candidates.take(&bounds)?, + )?; + self.bounds = Some(bounds); + Ok(()) + } +} + +impl SortSimpleState { + pub fn new( + inputs: usize, + partitions: usize, + schema: DataSchemaRef, + sort_desc: Arc<[SortColumnDescription]>, + ) -> Arc { + let columns = sort_desc.iter().map(|desc| desc.offset).collect::>(); + let schema = schema.project(&columns).into(); + let sort_desc = sort_desc + .iter() + .enumerate() + .map(|(i, desc)| SortColumnDescription { + offset: i, + asc: desc.asc, + nulls_first: desc.nulls_first, + }) + .collect::>(); + Arc::new(SortSimpleState { + inner: RwLock::new(StateInner { + partitions, + schema, + sort_desc, + partial: vec![None; inputs], + bounds: None, + }), + done: WatchNotify::new(), + }) + } + + pub fn bounds(&self) -> Option { + if let Some(bounds) = &self.inner.read().unwrap().bounds { + return Some(bounds.clone()); + } + None + } + + pub fn commit_simple(&self, id: usize, block: Option) -> Result { + let mut inner = self.inner.write().unwrap(); + + let block = block.unwrap_or(DataBlock::empty_with_schema(inner.schema.clone())); + let x = inner.partial[id].replace(block); + debug_assert!(x.is_none()); + let done = inner.partial.iter().all(|x| x.is_some()); + if done { + inner.determine_bounds()?; + self.done.notify_waiters(); + } + Ok(done) + } +} + +pub struct TransformSortSimple { + id: usize, + simpler: Simpler, + state: Arc, +} + +unsafe impl Send for TransformSortSimple {} + +impl TransformSortSimple { + fn new(id: usize, k: usize, columns: Vec, state: Arc) -> Self { + let rng = StdRng::from_rng(rand::thread_rng()).unwrap(); + let simpler = Simpler::new(columns, 65536, k, rng); + TransformSortSimple { id, simpler, state } + } +} + +impl Transform for TransformSortSimple { + const NAME: &'static str = "TransformSortSimple"; + + fn transform(&mut self, data: DataBlock) -> Result { + self.simpler.add_block(data.clone()); + Ok(data) + } + + fn on_finish(&mut self) -> Result<()> { + self.simpler.compact_blocks(); + let mut simple = self.simpler.take_blocks(); + assert!(simple.len() <= 1); // Unlikely to sample rows greater than 65536 + self.state.commit_simple( + self.id, + if simple.is_empty() { + None + } else { + Some(simple.remove(0)) + }, + )?; + Ok(()) + } +} + +pub fn add_sort_simple( + pipeline: &mut Pipeline, + state: Arc, + sort_desc: Arc<[SortColumnDescription]>, + k: usize, +) -> Result<()> { + use std::sync::atomic; + let i = atomic::AtomicUsize::new(0); + let columns = sort_desc.iter().map(|desc| desc.offset).collect::>(); + pipeline.add_transformer(|| { + let id = i.fetch_add(1, atomic::Ordering::AcqRel); + TransformSortSimple::new(id, k, columns.clone(), state.clone()) + }); + Ok(()) +} + +pub fn add_range_shuffle( + pipeline: &mut Pipeline, + state: Arc, + sort_desc: Arc<[SortColumnDescription]>, + schema: DataSchemaRef, + block_size: usize, + limit: Option, + remove_order_col: bool, + enable_loser_tree: bool, +) -> Result<()> { + pipeline.add_transform(|input, output| { + Ok(ProcessorPtr::create(Box::new( + TransformSortSimpleWait::new(input, output, state.clone()), + ))) + })?; + + // partition data block + let input_len = pipeline.output_len(); + let n = state.partitions(); + let exchange = create_exchange_pipe(input_len, n, schema.clone(), sort_desc.clone(), state); + pipeline.add_pipe(exchange); + + let reorder_edges = (0..input_len * n) + .map(|index| (index % n) * input_len + (index / n)) + .collect::>(); + + pipeline.reorder_inputs(reorder_edges); + + let mut items = Vec::with_capacity(input_len); + for _ in 0..n { + let output = OutputPort::create(); + let inputs: Vec<_> = (0..input_len).map(|_| InputPort::create()).collect(); + + let proc = create_multi_sort_merge_processor( + inputs.clone(), + output.clone(), + schema.clone(), + block_size, + limit, + sort_desc.clone(), + remove_order_col, + enable_loser_tree, + )?; + + items.push(PipeItem::create(ProcessorPtr::create(proc), inputs, vec![ + output, + ])); + } + + // merge partition + pipeline.add_pipe(Pipe::create(input_len * n, n, items)); + + Ok(()) +} + +#[cfg(test)] +mod tests { + use databend_common_expression::types::ArgType; + use databend_common_expression::types::Int32Type; + use databend_common_expression::DataField; + use databend_common_expression::DataSchemaRefExt; + use databend_common_expression::FromData; + + use super::*; + + #[test] + fn test_determine_bounds() { + let partial = vec![vec![1, 2, 3, 4], vec![4, 5, 6, 7], vec![0, 2, 4, 5]] + .into_iter() + .map(|data| { + Some(DataBlock::new_from_columns(vec![Int32Type::from_data( + data, + )])) + }) + .collect::>(); + + let schema = DataSchemaRefExt::create(vec![DataField::new("a", Int32Type::data_type())]); + let mut inner = StateInner { + partitions: 3, + schema, + sort_desc: vec![SortColumnDescription { + offset: 0, + asc: true, + nulls_first: false, + }], + partial, + bounds: None, + }; + + inner.determine_bounds().unwrap(); + + // 0 1 2 2 | 3 4 4 4 | 5 5 6 7 + assert_eq!(Int32Type::from_data(vec![3, 5]), inner.bounds.unwrap()) + } +} diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_wait.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_wait.rs new file mode 100644 index 0000000000000..2fc6ec3cecea9 --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_wait.rs @@ -0,0 +1,113 @@ +// 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::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; + +use super::SortSimpleState; + +pub struct TransformSortSimpleWait { + input: Arc, + output: Arc, + output_data: VecDeque, + blocks: Vec, + state: Arc, +} + +impl TransformSortSimpleWait { + pub fn new( + input: Arc, + output: Arc, + state: Arc, + ) -> Self { + Self { + input, + output, + output_data: VecDeque::new(), + blocks: Vec::new(), + state, + } + } +} + +#[async_trait::async_trait] +impl Processor for TransformSortSimpleWait { + fn name(&self) -> String { + "TransformSortSimpleWait".to_string() + } + + 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.blocks.push(self.input.pull_data().unwrap()?); + self.input.set_need_data(); + return Ok(Event::NeedData); + } + + if self.input.is_finished() { + if self.blocks.is_empty() { + self.output.finish(); + return Ok(Event::Finished); + } + + return if self.state.done.has_notified() { + Ok(Event::Sync) + } else { + Ok(Event::Async) + }; + } + + self.input.set_need_data(); + Ok(Event::NeedData) + } + + fn process(&mut self) -> Result<()> { + debug_assert!(!self.blocks.is_empty()); + self.output_data = VecDeque::from(std::mem::take(&mut self.blocks)); + Ok(()) + } + + #[async_backtrace::framed] + async fn async_process(&mut self) -> Result<()> { + self.state.done.notified().await; + self.output_data = VecDeque::from(std::mem::take(&mut self.blocks)); + Ok(()) + } +} diff --git a/src/query/settings/src/settings_default.rs b/src/query/settings/src/settings_default.rs index 0c4b3a183f066..f8f14d4a237c5 100644 --- a/src/query/settings/src/settings_default.rs +++ b/src/query/settings/src/settings_default.rs @@ -654,6 +654,13 @@ impl DefaultSettings { scope: SettingScope::Both, range: Some(SettingRange::Numeric(4 * 1024..=u64::MAX)), }), + ("range_shuffle_sort_simple_size", DefaultSettingValue { + value: UserSettingValue::UInt64(20), + desc: "Sets the simple size per partition used for range shuffle sorting, 0 to disable range shuffle sorting.", + mode: SettingMode::Both, + scope: SettingScope::Both, + range: Some(SettingRange::Numeric(0..=500)), + }), ("group_by_shuffle_mode", DefaultSettingValue { value: UserSettingValue::String(String::from("before_merge")), desc: "Group by shuffle mode, 'before_partial' is more balanced, but more data needs to exchange.", diff --git a/src/query/settings/src/settings_getter_setter.rs b/src/query/settings/src/settings_getter_setter.rs index 1952a38a1c74c..bc2b1b661b0dc 100644 --- a/src/query/settings/src/settings_getter_setter.rs +++ b/src/query/settings/src/settings_getter_setter.rs @@ -512,6 +512,10 @@ impl Settings { Ok(self.try_get_u64("sort_spilling_to_disk_bytes_limit")? as usize) } + pub fn get_range_shuffle_sort_simple_size(&self) -> Result { + Ok(self.try_get_u64("range_shuffle_sort_simple_size")? as usize) + } + pub fn get_group_by_shuffle_mode(&self) -> Result { self.try_get_string("group_by_shuffle_mode") } From 6e5889a190ab65221015727b69790a1eae5a8c21 Mon Sep 17 00:00:00 2001 From: coldWater Date: Tue, 15 Apr 2025 17:25:34 +0800 Subject: [PATCH 04/61] rename --- src/query/expression/src/lib.rs | 1 - .../src/simpler/reservoir_sampling.rs | 91 ------------------- .../src/pipelines/builders/builder_sort.rs | 10 +- .../processors/transforms/sort/mod.rs | 4 +- .../transforms/sort/sort_exchange.rs | 8 +- .../sort/{sort_simple.rs => sort_sample.rs} | 54 +++++------ .../processors/transforms/sort/sort_wait.rs | 12 +-- 7 files changed, 44 insertions(+), 136 deletions(-) delete mode 100644 src/query/expression/src/simpler/reservoir_sampling.rs rename src/query/service/src/pipelines/processors/transforms/sort/{sort_simple.rs => sort_sample.rs} (89%) diff --git a/src/query/expression/src/lib.rs b/src/query/expression/src/lib.rs index 0175dba4e3cae..b87d5b15d6303 100755 --- a/src/query/expression/src/lib.rs +++ b/src/query/expression/src/lib.rs @@ -65,7 +65,6 @@ mod register_vectorize; pub mod row; pub mod sampler; pub mod schema; -pub mod simpler; pub mod type_check; pub mod types; pub mod utils; diff --git a/src/query/expression/src/simpler/reservoir_sampling.rs b/src/query/expression/src/simpler/reservoir_sampling.rs deleted file mode 100644 index 2b9cd32d04d10..0000000000000 --- a/src/query/expression/src/simpler/reservoir_sampling.rs +++ /dev/null @@ -1,91 +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::num::NonZeroUsize; - -use rand::Rng; - -/// An implementation of Algorithm `L` (https://en.wikipedia.org/wiki/Reservoir_sampling#An_optimal_algorithm) -pub struct AlgoL { - k: usize, - w: f64, - - r: R, -} - -impl AlgoL { - pub fn new(k: NonZeroUsize, r: R) -> Self { - let mut al = Self { - k: k.into(), - w: 1.0, - r, - }; - al.update_w(); - al - } - - pub fn search(&mut self) -> usize { - let s = (self.rng().log2() / (1.0 - self.w).log2()).floor() + 1.0; - if s.is_normal() { - s as usize - } else { - usize::MAX - } - } - - pub fn pos(&mut self) -> usize { - self.r.sample(rand::distributions::Uniform::new(0, self.k)) - } - - pub fn update_w(&mut self) { - self.w *= (self.rng().log2() / self.k as f64).exp2(); // rng ^ (1/k) - } - - fn rng(&mut self) -> f64 { - self.r.sample(rand::distributions::Open01) - } -} - -#[cfg(test)] -mod tests { - use rand::rngs::StdRng; - use rand::SeedableRng; - - use super::*; - - #[test] - fn test_algo_l() { - let rng = StdRng::seed_from_u64(0); - let mut sample = vec![0_u64; 10]; - - let mut al = AlgoL::new(10.try_into().unwrap(), rng); - for (i, v) in sample.iter_mut().enumerate() { - *v = i as u64 - } - - let mut i = 9; - loop { - i += al.search(); - if i < 100 { - sample[al.pos()] = i as u64; - al.update_w() - } else { - break; - } - } - - let want: Vec = vec![69, 49, 53, 83, 4, 72, 88, 38, 45, 27]; - assert_eq!(want, sample) - } -} diff --git a/src/query/service/src/pipelines/builders/builder_sort.rs b/src/query/service/src/pipelines/builders/builder_sort.rs index a723eb8ddcf35..b94880ff3e490 100644 --- a/src/query/service/src/pipelines/builders/builder_sort.rs +++ b/src/query/service/src/pipelines/builders/builder_sort.rs @@ -36,8 +36,8 @@ use databend_storages_common_cache::TempDirManager; use crate::pipelines::memory_settings::MemorySettingsExt; use crate::pipelines::processors::transforms::sort::add_range_shuffle; use crate::pipelines::processors::transforms::sort::add_range_shuffle_merge; -use crate::pipelines::processors::transforms::sort::add_sort_simple; -use crate::pipelines::processors::transforms::sort::SortSimpleState; +use crate::pipelines::processors::transforms::sort::add_sort_sample; +use crate::pipelines::processors::transforms::sort::SortSampleState; use crate::pipelines::processors::transforms::TransformLimit; use crate::pipelines::processors::transforms::TransformSortBuilder; use crate::pipelines::PipelineBuilder; @@ -217,14 +217,14 @@ impl SortPipelineBuilder { let inputs = pipeline.output_len(); let settings = self.ctx.get_settings(); let max_threads = settings.get_max_threads()? as usize; - let simple = SortSimpleState::new( + let sample = SortSampleState::new( inputs, max_threads, self.schema.clone(), self.sort_desc.clone(), ); - add_sort_simple(pipeline, simple.clone(), self.sort_desc.clone(), k)?; + add_sort_sample(pipeline, sample.clone(), self.sort_desc.clone(), k)?; // Partial sort pipeline.add_transformer(|| { @@ -238,7 +238,7 @@ impl SortPipelineBuilder { add_range_shuffle( pipeline, - simple.clone(), + sample.clone(), self.sort_desc.clone(), self.schema.clone(), self.block_size, diff --git a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs index 8049256d68e67..c237beb982d8e 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs @@ -14,8 +14,8 @@ mod sort_exchange; mod sort_merge; -mod sort_simple; +mod sort_sample; mod sort_wait; pub use sort_merge::*; -pub use sort_simple::*; +pub use sort_sample::*; diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange.rs index 1bdc66673ebb4..a528b5f3352f0 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange.rs @@ -31,11 +31,11 @@ use databend_common_pipeline_transforms::processors::sort::Rows; use databend_common_pipeline_transforms::processors::sort::RowsTypeVisitor; use databend_common_pipeline_transforms::sort::RowConverter; -use super::sort_simple::SortSimpleState; +use super::sort_sample::SortSampleState; use crate::pipelines::processors::PartitionProcessor; pub struct SortRangeExchange { - state: Arc, + state: Arc, _r: PhantomData, } @@ -87,7 +87,7 @@ pub fn create_exchange_pipe( partitions: usize, schema: DataSchemaRef, sort_desc: Arc<[SortColumnDescription]>, - state: Arc, + state: Arc, ) -> Pipe { let mut builder = Builder { inputs, @@ -108,7 +108,7 @@ struct Builder { partitions: usize, sort_desc: Arc<[SortColumnDescription]>, schema: DataSchemaRef, - state: Arc, + state: Arc, items: Vec, } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_simple.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_sample.rs similarity index 89% rename from src/query/service/src/pipelines/processors/transforms/sort/sort_simple.rs rename to src/query/service/src/pipelines/processors/transforms/sort/sort_sample.rs index 7195769842c42..6f5adb69a3c84 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_simple.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_sample.rs @@ -17,7 +17,7 @@ use std::sync::RwLock; use databend_common_base::base::WatchNotify; use databend_common_exception::Result; -use databend_common_expression::simpler::Simpler; +use databend_common_expression::sampler::FixedSizeSampler; use databend_common_expression::visitor::ValueVisitor; use databend_common_expression::Column; use databend_common_expression::DataBlock; @@ -38,14 +38,14 @@ use rand::rngs::StdRng; use rand::SeedableRng; use super::sort_exchange::create_exchange_pipe; -use super::sort_wait::TransformSortSimpleWait; +use super::sort_wait::TransformSortSampleWait; -pub struct SortSimpleState { +pub struct SortSampleState { inner: RwLock, pub(super) done: WatchNotify, } -impl SortSimpleState { +impl SortSampleState { pub fn partitions(&self) -> usize { self.inner.read().unwrap().partitions } @@ -128,13 +128,13 @@ impl StateInner { } } -impl SortSimpleState { +impl SortSampleState { pub fn new( inputs: usize, partitions: usize, schema: DataSchemaRef, sort_desc: Arc<[SortColumnDescription]>, - ) -> Arc { + ) -> Arc { let columns = sort_desc.iter().map(|desc| desc.offset).collect::>(); let schema = schema.project(&columns).into(); let sort_desc = sort_desc @@ -146,7 +146,7 @@ impl SortSimpleState { nulls_first: desc.nulls_first, }) .collect::>(); - Arc::new(SortSimpleState { + Arc::new(SortSampleState { inner: RwLock::new(StateInner { partitions, schema, @@ -165,7 +165,7 @@ impl SortSimpleState { None } - pub fn commit_simple(&self, id: usize, block: Option) -> Result { + pub fn commit_sample(&self, id: usize, block: Option) -> Result { let mut inner = self.inner.write().unwrap(); let block = block.unwrap_or(DataBlock::empty_with_schema(inner.schema.clone())); @@ -180,35 +180,35 @@ impl SortSimpleState { } } -pub struct TransformSortSimple { +pub struct TransformSortSample { id: usize, - simpler: Simpler, - state: Arc, + sampler: FixedSizeSampler, + state: Arc, } -unsafe impl Send for TransformSortSimple {} +unsafe impl Send for TransformSortSample {} -impl TransformSortSimple { - fn new(id: usize, k: usize, columns: Vec, state: Arc) -> Self { +impl TransformSortSample { + fn new(id: usize, k: usize, columns: Vec, state: Arc) -> Self { let rng = StdRng::from_rng(rand::thread_rng()).unwrap(); - let simpler = Simpler::new(columns, 65536, k, rng); - TransformSortSimple { id, simpler, state } + let sampler = FixedSizeSampler::new(columns, 65536, k, rng); + TransformSortSample { id, sampler, state } } } -impl Transform for TransformSortSimple { - const NAME: &'static str = "TransformSortSimple"; +impl Transform for TransformSortSample { + const NAME: &'static str = "TransformSortSample"; fn transform(&mut self, data: DataBlock) -> Result { - self.simpler.add_block(data.clone()); + self.sampler.add_block(data.clone()); Ok(data) } fn on_finish(&mut self) -> Result<()> { - self.simpler.compact_blocks(); - let mut simple = self.simpler.take_blocks(); + self.sampler.compact_blocks(); + let mut simple = self.sampler.take_blocks(); assert!(simple.len() <= 1); // Unlikely to sample rows greater than 65536 - self.state.commit_simple( + self.state.commit_sample( self.id, if simple.is_empty() { None @@ -220,9 +220,9 @@ impl Transform for TransformSortSimple { } } -pub fn add_sort_simple( +pub fn add_sort_sample( pipeline: &mut Pipeline, - state: Arc, + state: Arc, sort_desc: Arc<[SortColumnDescription]>, k: usize, ) -> Result<()> { @@ -231,14 +231,14 @@ pub fn add_sort_simple( let columns = sort_desc.iter().map(|desc| desc.offset).collect::>(); pipeline.add_transformer(|| { let id = i.fetch_add(1, atomic::Ordering::AcqRel); - TransformSortSimple::new(id, k, columns.clone(), state.clone()) + TransformSortSample::new(id, k, columns.clone(), state.clone()) }); Ok(()) } pub fn add_range_shuffle( pipeline: &mut Pipeline, - state: Arc, + state: Arc, sort_desc: Arc<[SortColumnDescription]>, schema: DataSchemaRef, block_size: usize, @@ -248,7 +248,7 @@ pub fn add_range_shuffle( ) -> Result<()> { pipeline.add_transform(|input, output| { Ok(ProcessorPtr::create(Box::new( - TransformSortSimpleWait::new(input, output, state.clone()), + TransformSortSampleWait::new(input, output, state.clone()), ))) })?; diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_wait.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_wait.rs index 2fc6ec3cecea9..3f1543d6b2760 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_wait.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_wait.rs @@ -23,21 +23,21 @@ use databend_common_pipeline_core::processors::InputPort; use databend_common_pipeline_core::processors::OutputPort; use databend_common_pipeline_core::processors::Processor; -use super::SortSimpleState; +use super::SortSampleState; -pub struct TransformSortSimpleWait { +pub struct TransformSortSampleWait { input: Arc, output: Arc, output_data: VecDeque, blocks: Vec, - state: Arc, + state: Arc, } -impl TransformSortSimpleWait { +impl TransformSortSampleWait { pub fn new( input: Arc, output: Arc, - state: Arc, + state: Arc, ) -> Self { Self { input, @@ -50,7 +50,7 @@ impl TransformSortSimpleWait { } #[async_trait::async_trait] -impl Processor for TransformSortSimpleWait { +impl Processor for TransformSortSampleWait { fn name(&self) -> String { "TransformSortSimpleWait".to_string() } From e23904ecbc7c511f646b541a60110b69d04c6a68 Mon Sep 17 00:00:00 2001 From: coldWater Date: Fri, 18 Apr 2025 19:12:24 +0800 Subject: [PATCH 05/61] fix --- src/query/service/src/pipelines/builders/builder_sort.rs | 2 +- .../pipelines/processors/transforms/transform_merge_sort.rs | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/src/query/service/src/pipelines/builders/builder_sort.rs b/src/query/service/src/pipelines/builders/builder_sort.rs index b94880ff3e490..f678a4e618782 100644 --- a/src/query/service/src/pipelines/builders/builder_sort.rs +++ b/src/query/service/src/pipelines/builders/builder_sort.rs @@ -295,7 +295,7 @@ impl SortPipelineBuilder { use_parquet: settings.get_spilling_file_format()?.is_parquet(), }; let op = DataOperator::instance().spill_operator(); - Arc::new(Spiller::create(self.ctx.clone(), op, config.clone())?) + Arc::new(Spiller::create(self.ctx.clone(), op, config)?) }; pipeline.add_transform(|input, output| { diff --git a/src/query/service/src/pipelines/processors/transforms/transform_merge_sort.rs b/src/query/service/src/pipelines/processors/transforms/transform_merge_sort.rs index 9b26a6eedc14d..d6f54ab2695bb 100644 --- a/src/query/service/src/pipelines/processors/transforms/transform_merge_sort.rs +++ b/src/query/service/src/pipelines/processors/transforms/transform_merge_sort.rs @@ -362,6 +362,7 @@ where return match self.state { State::Collect => { if self.check_spill() { + // delay the handle of input until the next call. Ok(Event::Async) } else { Ok(Event::Sync) @@ -437,7 +438,7 @@ where self.prepare_spill_limit()?; } Inner::Collect(input_data) => { - debug_assert!(!finished); + assert!(!finished); let input_data = std::mem::take(input_data); self.prepare_spill(input_data); } @@ -471,7 +472,7 @@ where let Inner::Spill(input_data, spill_sort) = &mut self.inner else { unreachable!() }; - debug_assert!(input_data.is_empty()); + assert!(input_data.is_empty()); let (block, finish) = spill_sort.on_restore().await?; self.output_data.extend(block); if finish { From 0bf671fc1ac210c4412c271ea4f05ad04b7860b1 Mon Sep 17 00:00:00 2001 From: coldWater Date: Mon, 21 Apr 2025 14:15:29 +0800 Subject: [PATCH 06/61] trait Spill --- .../processors/transforms/sort/sort_sample.rs | 2 +- .../transforms/transform_merge_sort.rs | 18 + .../transform_merge_sort/collect.rs | 373 ++++++++++++++++++ .../transform_merge_sort/sort_spill.rs | 156 +++++--- 4 files changed, 501 insertions(+), 48 deletions(-) create mode 100644 src/query/service/src/pipelines/processors/transforms/transform_merge_sort/collect.rs diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_sample.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_sample.rs index 6f5adb69a3c84..a2925d6a43a21 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_sample.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_sample.rs @@ -92,7 +92,7 @@ impl StateInner { let mut sort_compare = SortCompare::with_force_equality(self.sort_desc.clone(), rows); for desc in &self.sort_desc { - let array = candidates.get_by_offset(desc.offset).value.clone(); + let array = candidates.get_by_offset(desc.offset).value(); sort_compare.visit_value(array)?; sort_compare.increment_column_index(); } diff --git a/src/query/service/src/pipelines/processors/transforms/transform_merge_sort.rs b/src/query/service/src/pipelines/processors/transforms/transform_merge_sort.rs index d6f54ab2695bb..9001a90749f5f 100644 --- a/src/query/service/src/pipelines/processors/transforms/transform_merge_sort.rs +++ b/src/query/service/src/pipelines/processors/transforms/transform_merge_sort.rs @@ -20,6 +20,9 @@ use std::sync::atomic::AtomicBool; 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::Column; use databend_common_expression::DataBlock; use databend_common_expression::DataSchemaRef; use databend_common_expression::SortColumnDescription; @@ -34,6 +37,7 @@ use databend_common_pipeline_transforms::MemorySettings; use databend_common_pipeline_transforms::MergeSort; use databend_common_pipeline_transforms::SortSpillParams; use databend_common_pipeline_transforms::TransformSortMergeLimit; +use sort_spill::SpillableBlock; use crate::spillers::Spiller; @@ -45,6 +49,8 @@ use sort_spill::SortSpill; mod builder; pub use builder::TransformSortBuilder; +mod collect; + #[derive(Debug)] enum State { /// This state means the processor will collect incoming blocks. @@ -63,6 +69,18 @@ struct Base { limit: Option, } +#[derive(Debug)] +pub struct SortCollectedMeta { + batch_rows: usize, + bounds: Vec, + blocks: Vec>, +} + +local_block_meta_serde!(SortCollectedMeta); + +#[typetag::serde(name = "sort_collected")] +impl BlockMetaInfo for SortCollectedMeta {} + enum Inner { Collect(Vec), Limit(TransformSortMergeLimit), diff --git a/src/query/service/src/pipelines/processors/transforms/transform_merge_sort/collect.rs b/src/query/service/src/pipelines/processors/transforms/transform_merge_sort/collect.rs new file mode 100644 index 0000000000000..fdfa5671cde50 --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/transform_merge_sort/collect.rs @@ -0,0 +1,373 @@ +// 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::atomic; +use std::sync::atomic::AtomicBool; +use std::sync::Arc; + +use databend_common_exception::Result; +use databend_common_expression::DataBlock; +use databend_common_expression::DataSchemaRef; +use databend_common_expression::SortColumnDescription; +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::processors::sort::algorithm::SortAlgorithm; +use databend_common_pipeline_transforms::sort::RowConverter; +use databend_common_pipeline_transforms::sort::Rows; +use databend_common_pipeline_transforms::MemorySettings; +use databend_common_pipeline_transforms::MergeSort; +use databend_common_pipeline_transforms::SortSpillParams; +use databend_common_pipeline_transforms::TransformSortMergeLimit; + +use super::sort_spill::SortSpill; +use super::Base; +use crate::spillers::Spiller; + +enum Inner { + Collect(Vec), + Limit(TransformSortMergeLimit), + Spill(Vec, SortSpill), + None, +} + +pub struct TransformSortCollect { + name: &'static str, + input: Arc, + output: Arc, + output_data: Option, + + row_converter: C, + sort_desc: Arc<[SortColumnDescription]>, + /// If this transform is after an Exchange transform, + /// it means it will compact the data from cluster nodes. + /// And the order column is already generated in each cluster node, + /// so we don't need to generate the order column again. + order_col_generated: bool, + + base: Base, + inner: Inner, + + aborting: AtomicBool, + + memory_settings: MemorySettings, +} + +impl TransformSortCollect +where + A: SortAlgorithm, + C: RowConverter, +{ + #[allow(clippy::too_many_arguments)] + fn new( + input: Arc, + output: Arc, + schema: DataSchemaRef, + sort_desc: Arc<[SortColumnDescription]>, + limit: Option<(usize, bool)>, + spiller: Arc, + order_col_generated: bool, + memory_settings: MemorySettings, + ) -> Result { + let sort_row_offset = schema.fields().len() - 1; + let row_converter = C::create(&sort_desc, schema.clone())?; + let (name, inner, limit) = match limit { + Some((limit, true)) => ( + "TransformSortMergeCollectLimit", + Inner::Limit(TransformSortMergeLimit::create(max_block_size, limit)), + Some(limit), + ), + Some((limit, false)) => ( + "TransformSortMergeCollect", + Inner::Collect(vec![]), + Some(limit), + ), + None => ("TransformSortMergeCollect", Inner::Collect(vec![]), None), + }; + Ok(Self { + input, + output, + name, + row_converter, + output_data: None, + sort_desc, + order_col_generated, + base: Base { + schema, + spiller, + sort_row_offset, + limit, + }, + inner, + aborting: AtomicBool::new(false), + memory_settings, + }) + } + + fn generate_order_column(&self, mut block: DataBlock) -> Result<(A::Rows, DataBlock)> { + let order_by_cols = self + .sort_desc + .iter() + .map(|desc| block.get_by_offset(desc.offset).clone()) + .collect::>(); + let rows = self + .row_converter + .convert(&order_by_cols, block.num_rows())?; + block.add_column(rows.to_column()); + Ok((rows, block)) + } + + fn limit_trans_to_spill(&mut self) -> Result<()> { + let Inner::Limit(merger) = &self.inner else { + unreachable!() + }; + assert!(merger.num_rows() > 0); + let params = self.determine_params(merger.num_bytes(), merger.num_rows()); + let Inner::Limit(merger) = &mut self.inner else { + unreachable!() + }; + let blocks = merger.prepare_spill(params.batch_rows)?; + let spill_sort = SortSpill::new(self.base.clone(), params); + self.inner = Inner::Spill(blocks, spill_sort); + Ok(()) + } + + fn collect_trans_to_spill(&mut self, input_data: Vec) { + let (num_rows, num_bytes) = input_data + .iter() + .map(|block| (block.num_rows(), block.memory_size())) + .fold((0, 0), |(acc_rows, acc_bytes), (rows, bytes)| { + (acc_rows + rows, acc_bytes + bytes) + }); + assert!(num_rows > 0); + let params = self.determine_params(num_bytes, num_rows); + let spill_sort = SortSpill::new(self.base.clone(), params); + self.inner = Inner::Spill(input_data, spill_sort); + } + + fn trans_to_spill(&mut self) -> Result<()> { + match &mut self.inner { + Inner::Limit(_) => self.limit_trans_to_spill(), + Inner::Collect(input_data) => { + let input_data = std::mem::take(input_data); + self.collect_trans_to_spill(input_data); + Ok(()) + } + Inner::Spill(_, _) => Ok(()), + Inner::None => unreachable!(), + } + } + + fn determine_params(&self, bytes: usize, rows: usize) -> SortSpillParams { + // We use the first memory calculation to estimate the batch size and the number of merge. + let unit_size = self.memory_settings.spill_unit_size; + let num_merge = bytes.div_ceil(unit_size).max(2); + let batch_rows = rows.div_ceil(num_merge); + log::info!("determine sort spill params, buffer_bytes: {bytes}, buffer_rows: {rows}, spill_unit_size: {unit_size}, batch_rows: {batch_rows}, batch_num_merge {num_merge}"); + SortSpillParams { + batch_rows, + num_merge, + } + } + + fn collect_block(&mut self, block: DataBlock) -> Result<()> { + if self.order_col_generated { + return match &mut self.inner { + Inner::Limit(limit_sort) => { + let rows = A::Rows::from_column(block.get_last_column())?; + limit_sort.add_block(block, rows) + } + Inner::Collect(input_data) | Inner::Spill(input_data, _) => { + input_data.push(block); + Ok(()) + } + _ => unreachable!(), + }; + } + + let (rows, block) = self.generate_order_column(block)?; + match &mut self.inner { + Inner::Limit(limit_sort) => limit_sort.add_block(block, rows), + Inner::Collect(input_data) | Inner::Spill(input_data, _) => { + input_data.push(block); + Ok(()) + } + _ => unreachable!(), + } + } + + fn input_rows(&self) -> usize { + match &self.inner { + Inner::Collect(input_data) | Inner::Spill(input_data, _) => input_data.in_memory_rows(), + _ => 0, + } + } + + fn check_spill(&self) -> bool { + if !self.memory_settings.check_spill() { + return false; + } + + match &self.inner { + Inner::Limit(limit_sort) => { + limit_sort.num_bytes() > self.memory_settings.spill_unit_size * 2 + } + Inner::Collect(input_data) => { + input_data.iter().map(|b| b.memory_size()).sum::() + > self.memory_settings.spill_unit_size * 2 + } + Inner::Spill(input_data, sort_spill) => { + input_data.in_memory_rows() > sort_spill.max_rows() + } + _ => unreachable!(), + } + } + + fn create_output(&mut self) -> Result<()> { + let Inner::Spill(input_data, spill_sort) = std::mem::replace(&mut self.inner, Inner::None) + else { + unreachable!() + }; + assert!(input_data.is_empty()); + + let meta = spill_sort.dump_collect()?; + self.output_data = Some(DataBlock::empty_with_meta(Box::new(meta))); + Ok(()) + } +} + +trait MemoryRows { + fn in_memory_rows(&self) -> usize; +} + +impl MemoryRows for Vec { + fn in_memory_rows(&self) -> usize { + self.iter().map(|s| s.num_rows()).sum::() + } +} + +#[async_trait::async_trait] +impl Processor for TransformSortCollect +where + A: SortAlgorithm + 'static, + A::Rows: 'static, + C: RowConverter + Send + 'static, +{ + fn name(&self) -> String { + self.name.to_string() + } + + 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(block) = self.output_data.take() { + assert!(self.input.is_finished()); + self.output.push_data(Ok(block)); + self.output.finish(); + return Ok(Event::Finished); + } + + if self.input.has_data() { + return if self.check_spill() { + // delay the handle of input until the next call. + Ok(Event::Async) + } else { + Ok(Event::Sync) + }; + } + + if self.input.is_finished() { + return match &self.inner { + Inner::Limit(merger) => { + if merger.num_rows() == 0 { + self.output.finish(); + Ok(Event::Finished) + } else { + Ok(Event::Async) + } + } + Inner::Collect(input_data) => { + if input_data.is_empty() { + self.output.finish(); + Ok(Event::Finished) + } else { + Ok(Event::Async) + } + } + Inner::Spill(_, _) => Ok(Event::Async), + Inner::None => unreachable!(), + }; + } + + self.input.set_need_data(); + Ok(Event::NeedData) + } + + fn process(&mut self) -> Result<()> { + if let Some(block) = self.input.pull_data().transpose()? { + self.input.set_need_data(); + if !block.is_empty() { + self.collect_block(block)?; + } + } + Ok(()) + } + + #[async_backtrace::framed] + async fn async_process(&mut self) -> Result<()> { + let finished = self.input.is_finished(); + self.trans_to_spill()?; + + let input = self.input_rows(); + let Inner::Spill(input_data, spill_sort) = &mut self.inner else { + unreachable!() + }; + let memory_rows = spill_sort.collect_memory_rows(); + let max = spill_sort.max_rows(); + + if memory_rows > 0 && memory_rows + input > max { + spill_sort + .subsequent_spill_last(memory_rows + input - max) + .await?; + } + if input > max || finished && input > 0 { + spill_sort + .sort_input_data(std::mem::take(input_data), &self.aborting) + .await?; + } + if finished { + self.create_output() + } else { + Ok(()) + } + } + + fn interrupt(&self) { + self.aborting.store(true, atomic::Ordering::Release); + } +} diff --git a/src/query/service/src/pipelines/processors/transforms/transform_merge_sort/sort_spill.rs b/src/query/service/src/pipelines/processors/transforms/transform_merge_sort/sort_spill.rs index 53b885619ffb6..ba3a42b4e9fe7 100644 --- a/src/query/service/src/pipelines/processors/transforms/transform_merge_sort/sort_spill.rs +++ b/src/query/service/src/pipelines/processors/transforms/transform_merge_sort/sort_spill.rs @@ -40,6 +40,7 @@ use rand::SeedableRng; use super::Base; use super::MemoryRows; +use super::SortCollectedMeta; use crate::spillers::Location; use crate::spillers::Spiller; @@ -56,7 +57,7 @@ enum Step { struct StepCollect { params: SortSpillParams, sampler: FixedRateSampler, - streams: Vec>, + streams: Vec>>, } struct StepSort { @@ -66,10 +67,11 @@ struct StepSort { bounds: Vec, cur_bound: Option, - subsequent: Vec>, - current: Vec>, + subsequent: Vec>>, + current: Vec>>, - output_merger: Option>>, + #[allow(clippy::type_complexity)] + output_merger: Option>>>, } impl SortSpill @@ -157,6 +159,34 @@ where A: SortAlgorithm pub fn format_memory_usage(&self) -> FmtMemoryUsage<'_, A> { FmtMemoryUsage(self) } + + pub fn dump_collect(self) -> Result { + let Self { + base, + step: Step::Collect(mut collect), + } = self + else { + unreachable!() + }; + + let StepSort { + params, + bounds, + subsequent, + .. + } = collect.next_step(&base)?; + + let blocks = subsequent + .into_iter() + .map(|stream| Vec::from(stream.blocks).into_boxed_slice()) + .collect(); + + Ok(SortCollectedMeta { + batch_rows: params.batch_rows, + blocks, + bounds, + }) + } } impl StepCollect { @@ -449,8 +479,8 @@ impl Base { &self, blocks: VecDeque, bound: Option, - ) -> BoundBlockStream { - BoundBlockStream:: { + ) -> BoundBlockStream> { + BoundBlockStream { blocks, bound, sort_row_offset: self.sort_row_offset, @@ -517,7 +547,7 @@ impl Base { } } -impl MemoryRows for Vec> { +impl MemoryRows for Vec> { fn in_memory_rows(&self) -> usize { self.iter().map(|s| s.in_memory_rows()).sum::() } @@ -554,7 +584,7 @@ impl fmt::Debug for FmtMemoryUsage<'_, A> { } } -struct SpillableBlock { +pub struct SpillableBlock { data: Option, rows: usize, location: Option, @@ -592,10 +622,10 @@ impl SpillableBlock { R::from_column(&self.domain).unwrap() } - async fn spill(&mut self, spiller: &Spiller) -> Result<()> { + async fn spill(&mut self, spiller: &impl Spill) -> Result<()> { let data = self.data.take().unwrap(); if self.location.is_none() { - let location = spiller.spill(vec![data]).await?; + let location = spiller.spill(data).await?; self.location = Some(location); } Ok(()) @@ -618,15 +648,32 @@ fn sort_column(data: &DataBlock, sort_row_offset: usize) -> &Column { data.get_by_offset(sort_row_offset).as_column().unwrap() } +#[async_trait::async_trait] +pub trait Spill: Send { + async fn spill(&self, data_block: DataBlock) -> Result; + async fn resotre(&self, location: &Location) -> Result; +} + +#[async_trait::async_trait] +impl Spill for Arc { + async fn spill(&self, data_block: DataBlock) -> Result { + self.as_ref().spill(vec![data_block]).await + } + + async fn resotre(&self, location: &Location) -> Result { + self.read_spilled_file(location).await + } +} + /// BoundBlockStream is a stream of blocks that are cutoff less or equal than bound. -struct BoundBlockStream { +struct BoundBlockStream { blocks: VecDeque, bound: Option, sort_row_offset: usize, - spiller: Arc, + spiller: S, } -impl Debug for BoundBlockStream { +impl Debug for BoundBlockStream { fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { f.debug_struct("BoundBlockStream") .field("blocks", &self.blocks) @@ -637,7 +684,7 @@ impl Debug for BoundBlockStream { } #[async_trait::async_trait] -impl SortedStream for BoundBlockStream { +impl SortedStream for BoundBlockStream { async fn async_next(&mut self) -> Result<(Option<(DataBlock, Column)>, bool)> { if self.should_include_first() { self.restore_first().await?; @@ -650,7 +697,7 @@ impl SortedStream for BoundBlockStream { } } -impl BoundBlockStream { +impl BoundBlockStream { fn should_include_first(&self) -> bool { let Some(block) = self.blocks.front() else { return false; @@ -682,6 +729,23 @@ impl BoundBlockStream { block.data.take().unwrap() } + fn len(&self) -> usize { + self.blocks.len() + } + + fn is_empty(&self) -> bool { + self.len() == 0 + } + + fn in_memory_rows(&self) -> usize { + self.blocks + .iter() + .map(|b| if b.data.is_some() { b.rows } else { 0 }) + .sum() + } +} + +impl BoundBlockStream { async fn restore_first(&mut self) -> Result<()> { let block = self.blocks.front_mut().unwrap(); if block.data.is_some() { @@ -689,7 +753,7 @@ impl BoundBlockStream { } let location = block.location.as_ref().unwrap(); - let data = self.spiller.read_spilled_file(location).await?; + let data = self.spiller.resotre(location).await?; block.data = Some(if block.processed != 0 { debug_assert_eq!(block.rows + block.processed, data.num_rows()); data.slice(block.processed..data.num_rows()) @@ -706,21 +770,6 @@ impl BoundBlockStream { Ok(()) } - fn len(&self) -> usize { - self.blocks.len() - } - - fn is_empty(&self) -> bool { - self.len() == 0 - } - - fn in_memory_rows(&self) -> usize { - self.blocks - .iter() - .map(|b| if b.data.is_some() { b.rows } else { 0 }) - .sum() - } - async fn spill(&mut self, skip: usize) -> Result<()> { for b in &mut self .blocks @@ -815,6 +864,10 @@ fn get_domain(col: &Column) -> Column { #[cfg(test)] mod tests { + use std::collections::HashMap; + use std::sync::Mutex; + + use databend_common_base::base::GlobalUniqName; use databend_common_expression::types::DataType; use databend_common_expression::types::Int32Type; use databend_common_expression::types::NumberDataType; @@ -826,12 +879,8 @@ mod tests { use databend_common_pipeline_transforms::processors::sort::convert_rows; use databend_common_pipeline_transforms::processors::sort::SimpleRowsAsc; use databend_common_pipeline_transforms::sort::SimpleRowsDesc; - use databend_common_storage::DataOperator; use super::*; - use crate::spillers::SpillerConfig; - use crate::spillers::SpillerType; - use crate::test_kits::*; fn test_data() -> (DataSchemaRef, DataBlock) { let col1 = Int32Type::from_data(vec![7, 7, 8, 11, 3, 5, 10, 11]); @@ -848,7 +897,7 @@ mod tests { } async fn run_bound_block_stream( - spiller: Arc, + spiller: impl Spill + Clone, sort_desc: Arc>, bound: Column, block_part: usize, @@ -871,7 +920,7 @@ mod tests { }) .collect::>(); - let mut stream = BoundBlockStream:: { + let mut stream = BoundBlockStream:: { blocks, bound, sort_row_offset, @@ -887,17 +936,9 @@ mod tests { #[tokio::test] async fn test_bound_block_stream() -> Result<()> { - let fixture = TestFixture::setup().await?; - let ctx = fixture.new_query_ctx().await?; - - let op = DataOperator::instance().spill_operator(); - let spill_config = SpillerConfig { - spiller_type: SpillerType::OrderBy, - location_prefix: "_spill_test".to_string(), - disk_spill: None, - use_parquet: true, + let spiller = MockSpiller { + map: Arc::new(Mutex::new(HashMap::new())), }; - let spiller = Arc::new(Spiller::create(ctx.clone(), op, spill_config)?); { let sort_desc = Arc::new(vec![SortColumnDescription { @@ -944,4 +985,25 @@ mod tests { Ok(()) } + + #[derive(Clone)] + struct MockSpiller { + map: Arc>>, + } + + #[async_trait::async_trait] + impl Spill for MockSpiller { + async fn spill(&self, data_block: DataBlock) -> Result { + let name = GlobalUniqName::unique(); + self.map.lock().unwrap().insert(name.clone(), data_block); + Ok(Location::Remote(name)) + } + + async fn resotre(&self, location: &Location) -> Result { + match location { + Location::Remote(name) => Ok(self.map.lock().unwrap().get(name).unwrap().clone()), + _ => unreachable!(), + } + } + } } From 4094992afbc6bfaf67da9fbfca84f35bdb69ca0d Mon Sep 17 00:00:00 2001 From: coldWater Date: Mon, 21 Apr 2025 18:55:24 +0800 Subject: [PATCH 07/61] execute --- .../transforms/transform_merge_sort.rs | 3 +- .../transform_merge_sort/builder.rs | 118 ++++++++++++++- .../transform_merge_sort/collect.rs | 3 +- .../transform_merge_sort/execute.rs | 139 ++++++++++++++++++ .../transform_merge_sort/sort_spill.rs | 40 ++++- 5 files changed, 287 insertions(+), 16 deletions(-) create mode 100644 src/query/service/src/pipelines/processors/transforms/transform_merge_sort/execute.rs diff --git a/src/query/service/src/pipelines/processors/transforms/transform_merge_sort.rs b/src/query/service/src/pipelines/processors/transforms/transform_merge_sort.rs index 9001a90749f5f..4c3b8aa664d02 100644 --- a/src/query/service/src/pipelines/processors/transforms/transform_merge_sort.rs +++ b/src/query/service/src/pipelines/processors/transforms/transform_merge_sort.rs @@ -50,6 +50,7 @@ mod builder; pub use builder::TransformSortBuilder; mod collect; +mod execute; #[derive(Debug)] enum State { @@ -71,7 +72,7 @@ struct Base { #[derive(Debug)] pub struct SortCollectedMeta { - batch_rows: usize, + params: SortSpillParams, bounds: Vec, blocks: Vec>, } diff --git a/src/query/service/src/pipelines/processors/transforms/transform_merge_sort/builder.rs b/src/query/service/src/pipelines/processors/transforms/transform_merge_sort/builder.rs index 313e69d4412c6..e3a6712dc897e 100644 --- a/src/query/service/src/pipelines/processors/transforms/transform_merge_sort/builder.rs +++ b/src/query/service/src/pipelines/processors/transforms/transform_merge_sort/builder.rs @@ -31,9 +31,17 @@ use databend_common_pipeline_transforms::sort::Rows; use databend_common_pipeline_transforms::sort::RowsTypeVisitor; use databend_common_pipeline_transforms::MemorySettings; +use super::collect::TransformSortCollect; +use super::execute::TransformSortExecute; use super::TransformSort; use crate::spillers::Spiller; +enum SortType { + Sort, + Collect, + Execute, +} + pub struct TransformSortBuilder { input: Arc, output: Arc, @@ -47,6 +55,7 @@ pub struct TransformSortBuilder { enable_loser_tree: bool, limit: Option, processor: Option>>, + typ: SortType, } impl TransformSortBuilder { @@ -71,6 +80,7 @@ impl TransformSortBuilder { limit: None, memory_settings: MemorySettings::disable_spill(), processor: None, + typ: SortType::Sort, } } @@ -149,6 +159,83 @@ impl TransformSortBuilder { self.memory_settings.clone(), )?)) } + + pub fn build_collect(mut self) -> Result> { + debug_assert!(if self.output_order_col { + self.schema.has_field(ORDER_COL_NAME) + } else { + !self.schema.has_field(ORDER_COL_NAME) + }); + self.typ = SortType::Collect; + + select_row_type(&mut self); + self.processor.unwrap() + } + + fn build_sort_collect(&mut self) -> Result> + where + A: SortAlgorithm + 'static, + C: RowConverter + Send + 'static, + { + let schema = add_order_field(self.schema.clone(), &self.sort_desc); + + Ok(Box::new(TransformSortCollect::::new( + self.input.clone(), + self.output.clone(), + schema, + self.sort_desc.clone(), + self.block_size, + self.limit.map(|limit| (limit, false)), + self.spiller.clone(), + self.order_col_generated, + self.memory_settings.clone(), + )?)) + } + + fn build_sort_limit_collect(&mut self) -> Result> + where + A: SortAlgorithm + 'static, + C: RowConverter + Send + 'static, + { + let schema = add_order_field(self.schema.clone(), &self.sort_desc); + Ok(Box::new(TransformSortCollect::::new( + self.input.clone(), + self.output.clone(), + schema, + self.sort_desc.clone(), + self.block_size, + Some((self.limit.unwrap(), true)), + self.spiller.clone(), + self.order_col_generated, + self.memory_settings.clone(), + )?)) + } + + pub fn build_exec(mut self) -> Result> { + debug_assert!(if self.output_order_col { + self.schema.has_field(ORDER_COL_NAME) + } else { + !self.schema.has_field(ORDER_COL_NAME) + }); + self.typ = SortType::Execute; + + select_row_type(&mut self); + self.processor.unwrap() + } + + fn build_sort_exec(&mut self) -> Result> + where A: SortAlgorithm + 'static { + let schema = add_order_field(self.schema.clone(), &self.sort_desc); + + Ok(Box::new(TransformSortExecute::::new( + self.input.clone(), + self.output.clone(), + schema, + self.limit, + self.spiller.clone(), + self.output_order_col, + )?)) + } } impl RowsTypeVisitor for TransformSortBuilder { @@ -165,14 +252,29 @@ impl RowsTypeVisitor for TransformSortBuilder { R: Rows + 'static, C: RowConverter + Send + 'static, { - let processor = match ( - self.limit.map(|limit| limit < 10000).unwrap_or_default(), - self.enable_loser_tree, - ) { - (true, true) => self.build_sort_limit::, C>(), - (true, false) => self.build_sort_limit::, C>(), - (false, true) => self.build_sort::, C>(), - (false, false) => self.build_sort::, C>(), + let processor = match self.typ { + SortType::Sort => match ( + self.limit.map(|limit| limit < 10000).unwrap_or_default(), + self.enable_loser_tree, + ) { + (true, true) => self.build_sort_limit::, C>(), + (true, false) => self.build_sort_limit::, C>(), + (false, true) => self.build_sort::, C>(), + (false, false) => self.build_sort::, C>(), + }, + SortType::Collect => match ( + self.limit.map(|limit| limit < 10000).unwrap_or_default(), + self.enable_loser_tree, + ) { + (true, true) => self.build_sort_limit_collect::, C>(), + (true, false) => self.build_sort_limit_collect::, C>(), + (false, true) => self.build_sort_collect::, C>(), + (false, false) => self.build_sort_collect::, C>(), + }, + SortType::Execute => match self.enable_loser_tree { + true => self.build_sort_exec::>(), + false => self.build_sort_exec::>(), + }, }; self.processor = Some(processor) } diff --git a/src/query/service/src/pipelines/processors/transforms/transform_merge_sort/collect.rs b/src/query/service/src/pipelines/processors/transforms/transform_merge_sort/collect.rs index fdfa5671cde50..31a92b3933587 100644 --- a/src/query/service/src/pipelines/processors/transforms/transform_merge_sort/collect.rs +++ b/src/query/service/src/pipelines/processors/transforms/transform_merge_sort/collect.rs @@ -72,11 +72,12 @@ where C: RowConverter, { #[allow(clippy::too_many_arguments)] - fn new( + pub fn new( input: Arc, output: Arc, schema: DataSchemaRef, sort_desc: Arc<[SortColumnDescription]>, + max_block_size: usize, limit: Option<(usize, bool)>, spiller: Arc, order_col_generated: bool, diff --git a/src/query/service/src/pipelines/processors/transforms/transform_merge_sort/execute.rs b/src/query/service/src/pipelines/processors/transforms/transform_merge_sort/execute.rs new file mode 100644 index 0000000000000..3612af89d156d --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/transform_merge_sort/execute.rs @@ -0,0 +1,139 @@ +// 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::BlockMetaInfoDowncast; +use databend_common_expression::DataBlock; +use databend_common_expression::DataSchemaRef; +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::processors::sort::algorithm::SortAlgorithm; + +use super::sort_spill::SortSpill; +use super::Base; +use super::SortCollectedMeta; +use crate::spillers::Spiller; + +pub struct TransformSortExecute { + input: Arc, + output: Arc, + + /// If the next transform of current transform is [`super::transform_multi_sort_merge::MultiSortMergeProcessor`], + /// we can generate and output the order column to avoid the extra converting in the next transform. + remove_order_col: bool, + + base: Base, + inner: Option>, +} + +impl TransformSortExecute +where A: SortAlgorithm +{ + #[allow(clippy::too_many_arguments)] + pub fn new( + input: Arc, + output: Arc, + schema: DataSchemaRef, + limit: Option, + spiller: Arc, + output_order_col: bool, + ) -> Result { + let sort_row_offset = schema.fields().len() - 1; + Ok(Self { + input, + output, + remove_order_col: !output_order_col, + base: Base { + schema, + spiller, + sort_row_offset, + limit, + }, + inner: None, + }) + } + + fn output_block(&self, mut block: DataBlock) { + if self.remove_order_col { + block.pop_columns(1); + } + self.output.push_data(Ok(block)); + } +} + +#[async_trait::async_trait] +impl Processor for TransformSortExecute +where + A: SortAlgorithm + 'static, + A::Rows: 'static, +{ + fn name(&self) -> String { + "TransformSortExecute".to_string() + } + + 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(mut block) = self.input.pull_data().transpose()? { + assert!(self.inner.is_none()); + let meta = block + .take_meta() + .and_then(SortCollectedMeta::downcast_from) + .expect("require a SortCollectedMeta"); + + self.inner = Some(SortSpill::::from_meta(self.base.clone(), meta)); + return Ok(Event::Async); + } + + if self.input.is_finished() { + Ok(Event::Async) + } else { + self.input.set_need_data(); + Ok(Event::NeedData) + } + } + + #[async_backtrace::framed] + async fn async_process(&mut self) -> Result<()> { + let Some(spill_sort) = &mut self.inner else { + unreachable!() + }; + let (block, finish) = spill_sort.on_restore().await?; + if let Some(block) = block { + assert!(!self.output.has_data()); + self.output_block(block); + } + if finish { + self.output.finish(); + } + Ok(()) + } +} diff --git a/src/query/service/src/pipelines/processors/transforms/transform_merge_sort/sort_spill.rs b/src/query/service/src/pipelines/processors/transforms/transform_merge_sort/sort_spill.rs index ba3a42b4e9fe7..a45b18476c484 100644 --- a/src/query/service/src/pipelines/processors/transforms/transform_merge_sort/sort_spill.rs +++ b/src/query/service/src/pipelines/processors/transforms/transform_merge_sort/sort_spill.rs @@ -94,6 +94,31 @@ where A: SortAlgorithm Self { base, step } } + pub fn from_meta(base: Base, meta: SortCollectedMeta) -> Self { + let SortCollectedMeta { + params, + bounds, + blocks, + } = meta; + + let subsequent = blocks + .into_iter() + .map(|list| base.new_stream(Vec::from(list).into(), None)) + .collect(); + + Self { + base, + step: Step::Sort(StepSort { + params, + bounds, + cur_bound: None, + subsequent, + current: vec![], + output_merger: None, + }), + } + } + pub async fn sort_input_data( &mut self, input_data: Vec, @@ -178,11 +203,14 @@ where A: SortAlgorithm let blocks = subsequent .into_iter() - .map(|stream| Vec::from(stream.blocks).into_boxed_slice()) + .map(|stream| { + assert!(stream.bound.is_none()); + Vec::from(stream.blocks).into_boxed_slice() + }) .collect(); Ok(SortCollectedMeta { - batch_rows: params.batch_rows, + params, blocks, bounds, }) @@ -651,7 +679,7 @@ fn sort_column(data: &DataBlock, sort_row_offset: usize) -> &Column { #[async_trait::async_trait] pub trait Spill: Send { async fn spill(&self, data_block: DataBlock) -> Result; - async fn resotre(&self, location: &Location) -> Result; + async fn restore(&self, location: &Location) -> Result; } #[async_trait::async_trait] @@ -660,7 +688,7 @@ impl Spill for Arc { self.as_ref().spill(vec![data_block]).await } - async fn resotre(&self, location: &Location) -> Result { + async fn restore(&self, location: &Location) -> Result { self.read_spilled_file(location).await } } @@ -753,7 +781,7 @@ impl BoundBlockStream { } let location = block.location.as_ref().unwrap(); - let data = self.spiller.resotre(location).await?; + let data = self.spiller.restore(location).await?; block.data = Some(if block.processed != 0 { debug_assert_eq!(block.rows + block.processed, data.num_rows()); data.slice(block.processed..data.num_rows()) @@ -999,7 +1027,7 @@ mod tests { Ok(Location::Remote(name)) } - async fn resotre(&self, location: &Location) -> Result { + async fn restore(&self, location: &Location) -> Result { match location { Location::Remote(name) => Ok(self.map.lock().unwrap().get(name).unwrap().clone()), _ => unreachable!(), From ea6c9d9fa35e19d7473661af5b3a307e8c6fea9a Mon Sep 17 00:00:00 2001 From: coldWater Date: Mon, 21 Apr 2025 19:26:25 +0800 Subject: [PATCH 08/61] move --- .../pipelines/processors/transforms/mod.rs | 3 +- .../{transform_merge_sort => sort}/builder.rs | 2 +- .../{transform_merge_sort => sort}/collect.rs | 5 +-- .../{transform_merge_sort => sort}/execute.rs | 0 .../merge_sort.rs} | 41 +++--------------- .../processors/transforms/sort/mod.rs | 43 +++++++++++++++++++ .../sort_spill.rs | 0 7 files changed, 51 insertions(+), 43 deletions(-) rename src/query/service/src/pipelines/processors/transforms/{transform_merge_sort => sort}/builder.rs (99%) rename src/query/service/src/pipelines/processors/transforms/{transform_merge_sort => sort}/collect.rs (99%) rename src/query/service/src/pipelines/processors/transforms/{transform_merge_sort => sort}/execute.rs (100%) rename src/query/service/src/pipelines/processors/transforms/{transform_merge_sort.rs => sort/merge_sort.rs} (95%) rename src/query/service/src/pipelines/processors/transforms/{transform_merge_sort => sort}/sort_spill.rs (100%) diff --git a/src/query/service/src/pipelines/processors/transforms/mod.rs b/src/query/service/src/pipelines/processors/transforms/mod.rs index 0b7c3241d0449..7ad1bfb56df2b 100644 --- a/src/query/service/src/pipelines/processors/transforms/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/mod.rs @@ -33,7 +33,6 @@ mod transform_dictionary; mod transform_expression_scan; mod transform_filter; mod transform_limit; -mod transform_merge_sort; mod transform_recursive_cte_scan; mod transform_recursive_cte_source; mod transform_resort_addon; @@ -46,6 +45,7 @@ mod window; pub use broadcast::BroadcastSinkProcessor; pub use broadcast::BroadcastSourceProcessor; pub use hash_join::*; +pub use sort::*; pub use transform_add_computed_columns::TransformAddComputedColumns; pub use transform_add_const_columns::TransformAddConstColumns; pub use transform_add_internal_columns::TransformAddInternalColumns; @@ -65,7 +65,6 @@ pub use transform_create_sets::TransformCreateSets; pub use transform_expression_scan::TransformExpressionScan; pub use transform_filter::TransformFilter; pub use transform_limit::TransformLimit; -pub use transform_merge_sort::*; pub use transform_recursive_cte_scan::TransformRecursiveCteScan; pub use transform_recursive_cte_source::TransformRecursiveCteSource; pub use transform_resort_addon::TransformResortAddOn; diff --git a/src/query/service/src/pipelines/processors/transforms/transform_merge_sort/builder.rs b/src/query/service/src/pipelines/processors/transforms/sort/builder.rs similarity index 99% rename from src/query/service/src/pipelines/processors/transforms/transform_merge_sort/builder.rs rename to src/query/service/src/pipelines/processors/transforms/sort/builder.rs index e3a6712dc897e..b16686b4829fd 100644 --- a/src/query/service/src/pipelines/processors/transforms/transform_merge_sort/builder.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/builder.rs @@ -33,7 +33,7 @@ use databend_common_pipeline_transforms::MemorySettings; use super::collect::TransformSortCollect; use super::execute::TransformSortExecute; -use super::TransformSort; +use super::merge_sort::TransformSort; use crate::spillers::Spiller; enum SortType { diff --git a/src/query/service/src/pipelines/processors/transforms/transform_merge_sort/collect.rs b/src/query/service/src/pipelines/processors/transforms/sort/collect.rs similarity index 99% rename from src/query/service/src/pipelines/processors/transforms/transform_merge_sort/collect.rs rename to src/query/service/src/pipelines/processors/transforms/sort/collect.rs index 31a92b3933587..40adae710cf1b 100644 --- a/src/query/service/src/pipelines/processors/transforms/transform_merge_sort/collect.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/collect.rs @@ -35,6 +35,7 @@ use databend_common_pipeline_transforms::TransformSortMergeLimit; use super::sort_spill::SortSpill; use super::Base; +use super::MemoryRows; use crate::spillers::Spiller; enum Inner { @@ -250,10 +251,6 @@ where } } -trait MemoryRows { - fn in_memory_rows(&self) -> usize; -} - impl MemoryRows for Vec { fn in_memory_rows(&self) -> usize { self.iter().map(|s| s.num_rows()).sum::() diff --git a/src/query/service/src/pipelines/processors/transforms/transform_merge_sort/execute.rs b/src/query/service/src/pipelines/processors/transforms/sort/execute.rs similarity index 100% rename from src/query/service/src/pipelines/processors/transforms/transform_merge_sort/execute.rs rename to src/query/service/src/pipelines/processors/transforms/sort/execute.rs diff --git a/src/query/service/src/pipelines/processors/transforms/transform_merge_sort.rs b/src/query/service/src/pipelines/processors/transforms/sort/merge_sort.rs similarity index 95% rename from src/query/service/src/pipelines/processors/transforms/transform_merge_sort.rs rename to src/query/service/src/pipelines/processors/transforms/sort/merge_sort.rs index 4c3b8aa664d02..5c83175bb791a 100644 --- a/src/query/service/src/pipelines/processors/transforms/transform_merge_sort.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/merge_sort.rs @@ -20,9 +20,6 @@ use std::sync::atomic::AtomicBool; 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::Column; use databend_common_expression::DataBlock; use databend_common_expression::DataSchemaRef; use databend_common_expression::SortColumnDescription; @@ -37,21 +34,13 @@ use databend_common_pipeline_transforms::MemorySettings; use databend_common_pipeline_transforms::MergeSort; use databend_common_pipeline_transforms::SortSpillParams; use databend_common_pipeline_transforms::TransformSortMergeLimit; -use sort_spill::SpillableBlock; +use super::sort_spill::create_memory_merger; +use super::sort_spill::MemoryMerger; +use super::sort_spill::SortSpill; +use super::Base; use crate::spillers::Spiller; -mod sort_spill; -use sort_spill::create_memory_merger; -use sort_spill::MemoryMerger; -use sort_spill::SortSpill; - -mod builder; -pub use builder::TransformSortBuilder; - -mod collect; -mod execute; - #[derive(Debug)] enum State { /// This state means the processor will collect incoming blocks. @@ -62,26 +51,6 @@ enum State { Finish, } -#[derive(Clone)] -struct Base { - schema: DataSchemaRef, - spiller: Arc, - sort_row_offset: usize, - limit: Option, -} - -#[derive(Debug)] -pub struct SortCollectedMeta { - params: SortSpillParams, - bounds: Vec, - blocks: Vec>, -} - -local_block_meta_serde!(SortCollectedMeta); - -#[typetag::serde(name = "sort_collected")] -impl BlockMetaInfo for SortCollectedMeta {} - enum Inner { Collect(Vec), Limit(TransformSortMergeLimit), @@ -122,7 +91,7 @@ where C: RowConverter, { #[allow(clippy::too_many_arguments)] - fn new( + pub(super) fn new( input: Arc, output: Arc, schema: DataSchemaRef, diff --git a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs index c237beb982d8e..832f165d6341f 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs @@ -17,5 +17,48 @@ mod sort_merge; mod sort_sample; mod sort_wait; +use std::sync::Arc; + +use databend_common_expression::local_block_meta_serde; +use databend_common_expression::BlockMetaInfo; +use databend_common_expression::Column; +use databend_common_expression::DataSchemaRef; +use databend_common_pipeline_transforms::SortSpillParams; pub use sort_merge::*; pub use sort_sample::*; + +mod builder; +pub use builder::TransformSortBuilder; + +mod collect; +mod execute; +mod merge_sort; +mod sort_spill; + +use sort_spill::SpillableBlock; + +use crate::spillers::Spiller; + +#[derive(Clone)] +struct Base { + schema: DataSchemaRef, + spiller: Arc, + sort_row_offset: usize, + limit: Option, +} + +#[derive(Debug)] +pub struct SortCollectedMeta { + params: SortSpillParams, + bounds: Vec, + blocks: Vec>, +} + +local_block_meta_serde!(SortCollectedMeta); + +#[typetag::serde(name = "sort_collected")] +impl BlockMetaInfo for SortCollectedMeta {} + +trait MemoryRows { + fn in_memory_rows(&self) -> usize; +} diff --git a/src/query/service/src/pipelines/processors/transforms/transform_merge_sort/sort_spill.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs similarity index 100% rename from src/query/service/src/pipelines/processors/transforms/transform_merge_sort/sort_spill.rs rename to src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs From b891e6f1d2775e5b35d6ccce69c5fd75300eebed Mon Sep 17 00:00:00 2001 From: coldWater Date: Tue, 22 Apr 2025 10:50:14 +0800 Subject: [PATCH 09/61] update --- .../processors/transforms/sort/rows/mod.rs | 13 +++++ .../processors/transforms/sort/collect.rs | 15 +----- .../processors/transforms/sort/execute.rs | 2 +- .../processors/transforms/sort/merge_sort.rs | 48 +++++++------------ .../processors/transforms/sort/mod.rs | 9 +++- .../processors/transforms/sort/sort_spill.rs | 2 +- 6 files changed, 43 insertions(+), 46 deletions(-) diff --git a/src/query/pipeline/transforms/src/processors/transforms/sort/rows/mod.rs b/src/query/pipeline/transforms/src/processors/transforms/sort/rows/mod.rs index 479d2559c049e..64c2599fe4e79 100644 --- a/src/query/pipeline/transforms/src/processors/transforms/sort/rows/mod.rs +++ b/src/query/pipeline/transforms/src/processors/transforms/sort/rows/mod.rs @@ -25,6 +25,7 @@ use databend_common_expression::types::ArgType; use databend_common_expression::types::DataType; use databend_common_expression::BlockEntry; use databend_common_expression::Column; +use databend_common_expression::DataBlock; use databend_common_expression::DataSchemaRef; use databend_common_expression::SortColumnDescription; pub use simple::*; @@ -39,6 +40,18 @@ where Self: Sized + Debug output_schema: DataSchemaRef, ) -> Result; fn convert(&self, columns: &[BlockEntry], num_rows: usize) -> Result; + + fn convert_data_block( + &self, + sort_desc: &[SortColumnDescription], + data_block: &DataBlock, + ) -> Result { + let order_by_cols = sort_desc + .iter() + .map(|desc| block.get_by_offset(desc.offset).clone()) + .collect::>(); + self.convert(&order_by_cols, block.num_rows()) + } } /// Rows can be compared. diff --git a/src/query/service/src/pipelines/processors/transforms/sort/collect.rs b/src/query/service/src/pipelines/processors/transforms/sort/collect.rs index 40adae710cf1b..98c5445155b6d 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/collect.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/collect.rs @@ -73,7 +73,7 @@ where C: RowConverter, { #[allow(clippy::too_many_arguments)] - pub fn new( + pub(super) fn new( input: Arc, output: Arc, schema: DataSchemaRef, @@ -120,14 +120,9 @@ where } fn generate_order_column(&self, mut block: DataBlock) -> Result<(A::Rows, DataBlock)> { - let order_by_cols = self - .sort_desc - .iter() - .map(|desc| block.get_by_offset(desc.offset).clone()) - .collect::>(); let rows = self .row_converter - .convert(&order_by_cols, block.num_rows())?; + .convert_data_block(&self.sort_desc, &block); block.add_column(rows.to_column()); Ok((rows, block)) } @@ -251,12 +246,6 @@ where } } -impl MemoryRows for Vec { - fn in_memory_rows(&self) -> usize { - self.iter().map(|s| s.num_rows()).sum::() - } -} - #[async_trait::async_trait] impl Processor for TransformSortCollect where diff --git a/src/query/service/src/pipelines/processors/transforms/sort/execute.rs b/src/query/service/src/pipelines/processors/transforms/sort/execute.rs index 3612af89d156d..c8b8f12f46855 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/execute.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/execute.rs @@ -46,7 +46,7 @@ impl TransformSortExecute where A: SortAlgorithm { #[allow(clippy::too_many_arguments)] - pub fn new( + pub(super) fn new( input: Arc, output: Arc, schema: DataSchemaRef, diff --git a/src/query/service/src/pipelines/processors/transforms/sort/merge_sort.rs b/src/query/service/src/pipelines/processors/transforms/sort/merge_sort.rs index 5c83175bb791a..7a06bc3ab2f83 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/merge_sort.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/merge_sort.rs @@ -39,6 +39,7 @@ use super::sort_spill::create_memory_merger; use super::sort_spill::MemoryMerger; use super::sort_spill::SortSpill; use super::Base; +use super::MemoryRows; use crate::spillers::Spiller; #[derive(Debug)] @@ -138,20 +139,15 @@ where } fn generate_order_column(&self, mut block: DataBlock) -> Result<(A::Rows, DataBlock)> { - let order_by_cols = self - .sort_desc - .iter() - .map(|desc| block.get_by_offset(desc.offset).clone()) - .collect::>(); let rows = self .row_converter - .convert(&order_by_cols, block.num_rows())?; + .convert_data_block(&self.sort_desc, &block); let order_col = rows.to_column(); block.add_column(order_col); Ok((rows, block)) } - fn prepare_spill_limit(&mut self) -> Result<()> { + fn limit_trans_to_spill(&mut self) -> Result<()> { let Inner::Limit(merger) = &self.inner else { unreachable!() }; @@ -165,7 +161,7 @@ where Ok(()) } - fn prepare_spill(&mut self, input_data: Vec) { + fn collect_trans_to_spill(&mut self, input_data: Vec) { let (num_rows, num_bytes) = input_data .iter() .map(|block| (block.num_rows(), block.memory_size())) @@ -177,6 +173,19 @@ where self.inner = Inner::Spill(input_data, spill_sort); } + fn trans_to_spill(&mut self) -> Result<()> { + match &mut self.inner { + Inner::Limit(_) => self.limit_trans_to_spill(), + Inner::Collect(input_data) => { + let input_data = std::mem::take(input_data); + self.collect_trans_to_spill(input_data); + Ok(()) + } + Inner::Spill(_, _) => Ok(()), + Inner::Memory(_) => unreachable!(), + } + } + fn determine_params(&self, bytes: usize, rows: usize) -> SortSpillParams { // We use the first memory calculation to estimate the batch size and the number of merge. let unit_size = self.memory_settings.spill_unit_size; @@ -294,16 +303,6 @@ where } } -trait MemoryRows { - fn in_memory_rows(&self) -> usize; -} - -impl MemoryRows for Vec { - fn in_memory_rows(&self) -> usize { - self.iter().map(|s| s.num_rows()).sum::() - } -} - #[async_trait::async_trait] impl Processor for TransformSort where @@ -421,18 +420,7 @@ where match &self.state { State::Collect => { let finished = self.input.is_finished(); - match &mut self.inner { - Inner::Limit(_) => { - self.prepare_spill_limit()?; - } - Inner::Collect(input_data) => { - assert!(!finished); - let input_data = std::mem::take(input_data); - self.prepare_spill(input_data); - } - Inner::Spill(_, _) => (), - Inner::Memory(_) => unreachable!(), - }; + self.trans_to_spill(); let input = self.input_rows(); let Inner::Spill(input_data, spill_sort) = &mut self.inner else { diff --git a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs index 832f165d6341f..9f220cdf03c24 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs @@ -22,6 +22,7 @@ use std::sync::Arc; use databend_common_expression::local_block_meta_serde; use databend_common_expression::BlockMetaInfo; use databend_common_expression::Column; +use databend_common_expression::DataBlock; use databend_common_expression::DataSchemaRef; use databend_common_pipeline_transforms::SortSpillParams; pub use sort_merge::*; @@ -48,7 +49,7 @@ struct Base { } #[derive(Debug)] -pub struct SortCollectedMeta { +struct SortCollectedMeta { params: SortSpillParams, bounds: Vec, blocks: Vec>, @@ -62,3 +63,9 @@ impl BlockMetaInfo for SortCollectedMeta {} trait MemoryRows { fn in_memory_rows(&self) -> usize; } + +impl MemoryRows for Vec { + fn in_memory_rows(&self) -> usize { + self.iter().map(|s| s.num_rows()).sum::() + } +} diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs index a45b18476c484..3a30922c8cbf4 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs @@ -44,7 +44,7 @@ use super::SortCollectedMeta; use crate::spillers::Location; use crate::spillers::Spiller; -pub struct SortSpill { +pub(super) struct SortSpill { base: Base, step: Step, } From a58f3d56dcb86993f8fe023efe04d827fb0635bb Mon Sep 17 00:00:00 2001 From: coldWater Date: Tue, 22 Apr 2025 13:37:06 +0800 Subject: [PATCH 10/61] bounds --- .../processors/transforms/sort/rows/mod.rs | 4 +- .../processors/transforms/sort/bounds.rs | 171 ++++++++++++++++++ .../processors/transforms/sort/collect.rs | 3 +- .../processors/transforms/sort/execute.rs | 1 - .../processors/transforms/sort/merge_sort.rs | 5 +- .../processors/transforms/sort/mod.rs | 5 +- .../processors/transforms/sort/sort_spill.rs | 80 ++------ 7 files changed, 198 insertions(+), 71 deletions(-) create mode 100644 src/query/service/src/pipelines/processors/transforms/sort/bounds.rs diff --git a/src/query/pipeline/transforms/src/processors/transforms/sort/rows/mod.rs b/src/query/pipeline/transforms/src/processors/transforms/sort/rows/mod.rs index 64c2599fe4e79..b1498c183c1e3 100644 --- a/src/query/pipeline/transforms/src/processors/transforms/sort/rows/mod.rs +++ b/src/query/pipeline/transforms/src/processors/transforms/sort/rows/mod.rs @@ -48,9 +48,9 @@ where Self: Sized + Debug ) -> Result { let order_by_cols = sort_desc .iter() - .map(|desc| block.get_by_offset(desc.offset).clone()) + .map(|desc| data_block.get_by_offset(desc.offset).clone()) .collect::>(); - self.convert(&order_by_cols, block.num_rows()) + self.convert(&order_by_cols, data_block.num_rows()) } } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs b/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs new file mode 100644 index 0000000000000..fe9d357fd8ba1 --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs @@ -0,0 +1,171 @@ +// 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::Column; +use databend_common_expression::DataBlock; +use databend_common_expression::DataField; +use databend_common_expression::DataSchema; +use databend_common_expression::SortColumnDescription; +use databend_common_pipeline_transforms::sort::LoserTreeMerger; +use databend_common_pipeline_transforms::sort::Rows; +use databend_common_pipeline_transforms::sort::SortedStream; + +#[derive(Debug, PartialEq, Eq, Default)] +pub struct Bounds( + // stored in reverse order of Column. + Vec, +); + +impl Bounds { + pub fn from_column(column: Column) -> Result { + let block = DataBlock::sort( + &DataBlock::new_from_columns(vec![column]), + &[SortColumnDescription { + offset: 0, + asc: R::IS_ASC_COLUMN, + nulls_first: false, + }], + None, + )?; + + Ok(Bounds(vec![block.get_last_column().clone()])) + } + + pub fn merge(mut vector: Vec, batch_rows: usize) -> Result { + match vector.len() { + 0 => Ok(Bounds(vec![])), + 1 => Ok(vector.pop().unwrap()), + _ => { + let schema = DataSchema::new(vec![DataField::new("order_col", R::data_type())]); + let mut merger = + LoserTreeMerger::::create(schema.into(), vector, batch_rows, None); + + let mut blocks = Vec::new(); + while let Some(block) = merger.next_block()? { + blocks.push(block) + } + debug_assert!(merger.is_finished()); + + Ok(Bounds( + blocks + .iter() + .rev() + .map(|b| b.get_last_column().clone()) + .collect(), + )) + } + } + } + + pub fn next_bound(&mut self) -> Option { + let last = self.0.last_mut()?; + match last.len() { + 0 => unreachable!(), + 1 => Some(self.0.pop().unwrap()), + _ => { + let bound = last.slice(0..1).maybe_gc(); + *last = last.slice(1..last.len()); + Some(bound) + } + } + } + + #[expect(dead_code)] + pub fn len(&self) -> usize { + self.0.iter().map(Column::len).sum() + } + + #[expect(dead_code)] + pub fn is_empty(&self) -> bool { + self.0.iter().all(|col| col.len() == 0) + } +} + +impl SortedStream for Bounds { + fn next(&mut self) -> Result<(Option<(DataBlock, Column)>, bool)> { + match self.0.pop() { + Some(column) => Ok(( + Some((DataBlock::new_from_columns(vec![column.clone()]), column)), + false, + )), + None => Ok((None, false)), + } + } +} + +#[cfg(test)] +mod tests { + + use databend_common_expression::types::Int32Type; + use databend_common_expression::FromData; + use databend_common_pipeline_transforms::sort::SimpleRowsAsc; + use databend_common_pipeline_transforms::sort::SimpleRowsDesc; + + use super::*; + + #[test] + fn test_merge() -> Result<()> { + { + let column = Int32Type::from_data(vec![0, 7, 6, 6, 6]); + let bounds = Bounds::from_column::>(column)?; + assert_eq!( + bounds, + Bounds(vec![Int32Type::from_data(vec![0, 6, 6, 6, 7])]) + ); + + let vector = vec![ + bounds, + Bounds::default(), + Bounds::from_column::>(Int32Type::from_data(vec![ + 0, 1, 2, + ])) + .unwrap(), + ]; + let bounds = Bounds::merge::>(vector, 3)?; + + assert_eq!( + bounds, + Bounds(vec![ + Int32Type::from_data(vec![6, 7]), + Int32Type::from_data(vec![2, 6, 6]), + Int32Type::from_data(vec![0, 0, 1]), + ]) + ); + } + + { + let data = vec![vec![77, -2, 7], vec![3, 8, 6, 1, 1], vec![2]]; + + let data = data + .into_iter() + .map(|v| Bounds::from_column::>(Int32Type::from_data(v))) + .collect::>>()?; + let bounds = Bounds::merge::>(data, 2)?; + + assert_eq!( + bounds, + Bounds(vec![ + Int32Type::from_data(vec![-2]), + Int32Type::from_data(vec![1, 1]), + Int32Type::from_data(vec![3, 2]), + Int32Type::from_data(vec![7, 6]), + Int32Type::from_data(vec![77, 8]), + ]) + ); + } + + Ok(()) + } +} diff --git a/src/query/service/src/pipelines/processors/transforms/sort/collect.rs b/src/query/service/src/pipelines/processors/transforms/sort/collect.rs index 98c5445155b6d..d6640513f9844 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/collect.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/collect.rs @@ -72,7 +72,6 @@ where A: SortAlgorithm, C: RowConverter, { - #[allow(clippy::too_many_arguments)] pub(super) fn new( input: Arc, output: Arc, @@ -122,7 +121,7 @@ where fn generate_order_column(&self, mut block: DataBlock) -> Result<(A::Rows, DataBlock)> { let rows = self .row_converter - .convert_data_block(&self.sort_desc, &block); + .convert_data_block(&self.sort_desc, &block)?; block.add_column(rows.to_column()); Ok((rows, block)) } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/execute.rs b/src/query/service/src/pipelines/processors/transforms/sort/execute.rs index c8b8f12f46855..39f01dc0497e1 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/execute.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/execute.rs @@ -45,7 +45,6 @@ pub struct TransformSortExecute { impl TransformSortExecute where A: SortAlgorithm { - #[allow(clippy::too_many_arguments)] pub(super) fn new( input: Arc, output: Arc, diff --git a/src/query/service/src/pipelines/processors/transforms/sort/merge_sort.rs b/src/query/service/src/pipelines/processors/transforms/sort/merge_sort.rs index 7a06bc3ab2f83..c39c520f2c615 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/merge_sort.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/merge_sort.rs @@ -91,7 +91,6 @@ where A: SortAlgorithm, C: RowConverter, { - #[allow(clippy::too_many_arguments)] pub(super) fn new( input: Arc, output: Arc, @@ -141,7 +140,7 @@ where fn generate_order_column(&self, mut block: DataBlock) -> Result<(A::Rows, DataBlock)> { let rows = self .row_converter - .convert_data_block(&self.sort_desc, &block); + .convert_data_block(&self.sort_desc, &block)?; let order_col = rows.to_column(); block.add_column(order_col); Ok((rows, block)) @@ -420,7 +419,7 @@ where match &self.state { State::Collect => { let finished = self.input.is_finished(); - self.trans_to_spill(); + self.trans_to_spill()?; let input = self.input_rows(); let Inner::Spill(input_data, spill_sort) = &mut self.inner else { diff --git a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs index 9f220cdf03c24..128bd76f80a9e 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs @@ -19,9 +19,9 @@ mod sort_wait; use std::sync::Arc; +use bounds::Bounds; use databend_common_expression::local_block_meta_serde; use databend_common_expression::BlockMetaInfo; -use databend_common_expression::Column; use databend_common_expression::DataBlock; use databend_common_expression::DataSchemaRef; use databend_common_pipeline_transforms::SortSpillParams; @@ -31,6 +31,7 @@ pub use sort_sample::*; mod builder; pub use builder::TransformSortBuilder; +mod bounds; mod collect; mod execute; mod merge_sort; @@ -51,7 +52,7 @@ struct Base { #[derive(Debug)] struct SortCollectedMeta { params: SortSpillParams, - bounds: Vec, + bounds: Bounds, blocks: Vec>, } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs index 3a30922c8cbf4..873397621e433 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs @@ -29,7 +29,6 @@ use databend_common_expression::sampler::FixedRateSampler; use databend_common_expression::Column; use databend_common_expression::DataBlock; use databend_common_expression::DataSchemaRef; -use databend_common_expression::SortColumnDescription; use databend_common_pipeline_transforms::processors::sort::algorithm::SortAlgorithm; use databend_common_pipeline_transforms::processors::sort::Merger; use databend_common_pipeline_transforms::processors::sort::Rows; @@ -38,6 +37,7 @@ use databend_common_pipeline_transforms::processors::SortSpillParams; use rand::rngs::StdRng; use rand::SeedableRng; +use super::bounds::Bounds; use super::Base; use super::MemoryRows; use super::SortCollectedMeta; @@ -62,15 +62,15 @@ struct StepCollect { struct StepSort { params: SortSpillParams, - /// Partition boundaries for restoring and sorting blocks, stored in reverse order of Column. + /// Partition boundaries for restoring and sorting blocks. /// Each boundary represents a cutoff point where data less than or equal to it belongs to one partition. - bounds: Vec, + bounds: Bounds, cur_bound: Option, subsequent: Vec>>, current: Vec>>, - #[allow(clippy::type_complexity)] + #[expect(clippy::type_complexity)] output_merger: Option>>>, } @@ -180,7 +180,7 @@ where A: SortAlgorithm params.num_merge * params.batch_rows } - #[allow(unused)] + #[expect(unused)] pub fn format_memory_usage(&self) -> FmtMemoryUsage<'_, A> { FmtMemoryUsage(self) } @@ -303,20 +303,10 @@ impl StepCollect { impl StepSort { fn next_bound(&mut self) { - let Some(last) = self.bounds.last_mut() else { - self.cur_bound = None; - return; - }; - let bound = match last.len() { - 0 => unreachable!(), - 1 => self.bounds.pop().unwrap(), - _ => { - let bound = last.slice(0..1).maybe_gc(); - *last = last.slice(1..last.len()); - bound - } - }; - self.cur_bound = Some(A::Rows::from_column(&bound).unwrap()); + match self.bounds.next_bound() { + Some(bound) => self.cur_bound = Some(A::Rows::from_column(&bound).unwrap()), + None => self.cur_bound = None, + } } async fn merge_current(&mut self, base: &Base) -> Result<()> { @@ -524,52 +514,19 @@ impl Base { &self, sampled_rows: Vec, batch_rows: usize, - ) -> Result> { + ) -> Result { match sampled_rows.len() { - 0 => Ok(vec![]), - 1 => Ok(vec![DataBlock::sort( - &sampled_rows[0], - &[SortColumnDescription { - offset: 0, - asc: A::Rows::IS_ASC_COLUMN, - nulls_first: false, - }], - None, - )? - .get_last_column() - .clone()]), + 0 => Ok(Bounds::default()), + 1 => Bounds::from_column::(sampled_rows[0].get_last_column().clone()), _ => { - let streams = sampled_rows + let ls = sampled_rows .into_iter() .map(|data| { - let data = DataBlock::sort( - &data, - &[SortColumnDescription { - offset: 0, - asc: A::Rows::IS_ASC_COLUMN, - nulls_first: false, - }], - None, - ) - .unwrap(); - DataBlockStream::new(data, 0) + let col = data.get_last_column().clone(); + Bounds::from_column::(col) }) - .collect::>(); - - let schema = self.schema.project(&[self.sort_row_offset]); - let mut merger = Merger::::create(schema.into(), streams, batch_rows, None); - - let mut blocks = Vec::new(); - while let Some(block) = merger.next_block()? { - blocks.push(block) - } - debug_assert!(merger.is_finished()); - - Ok(blocks - .iter() - .rev() - .map(|b| b.get_last_column().clone()) - .collect::>()) + .collect::>>()?; + Bounds::merge::(ls, batch_rows) } } } @@ -853,7 +810,7 @@ impl SortedStream for DataBlockStream { } impl DataBlockStream { - fn new(data: DataBlock, sort_row_offset: usize) -> Self { + pub(super) fn new(data: DataBlock, sort_row_offset: usize) -> Self { let col = sort_column(&data, sort_row_offset).clone(); Self(Some((data, col))) } @@ -904,6 +861,7 @@ mod tests { use databend_common_expression::DataField; use databend_common_expression::DataSchemaRefExt; use databend_common_expression::FromData; + use databend_common_expression::SortColumnDescription; use databend_common_pipeline_transforms::processors::sort::convert_rows; use databend_common_pipeline_transforms::processors::sort::SimpleRowsAsc; use databend_common_pipeline_transforms::sort::SimpleRowsDesc; From fc04da7aa3aa14c89dc511f45efaae65e66a8fbc Mon Sep 17 00:00:00 2001 From: coldWater Date: Tue, 22 Apr 2025 18:46:28 +0800 Subject: [PATCH 11/61] wait --- .../processors/transforms/sort/bounds.rs | 67 +++++- .../processors/transforms/sort/mod.rs | 1 + .../processors/transforms/sort/sort_sample.rs | 8 +- .../processors/transforms/sort/wait.rs | 210 ++++++++++++++++++ 4 files changed, 279 insertions(+), 7 deletions(-) create mode 100644 src/query/service/src/pipelines/processors/transforms/sort/wait.rs diff --git a/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs b/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs index fe9d357fd8ba1..172a8fb1a80fc 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs @@ -13,6 +13,7 @@ // limitations under the License. use databend_common_exception::Result; +use databend_common_expression::types::DataType; use databend_common_expression::Column; use databend_common_expression::DataBlock; use databend_common_expression::DataField; @@ -22,7 +23,7 @@ use databend_common_pipeline_transforms::sort::LoserTreeMerger; use databend_common_pipeline_transforms::sort::Rows; use databend_common_pipeline_transforms::sort::SortedStream; -#[derive(Debug, PartialEq, Eq, Default)] +#[derive(Debug, PartialEq, Eq, Default, Clone)] pub struct Bounds( // stored in reverse order of Column. Vec, @@ -82,7 +83,6 @@ impl Bounds { } } - #[expect(dead_code)] pub fn len(&self) -> usize { self.0.iter().map(Column::len).sum() } @@ -91,6 +91,42 @@ impl Bounds { pub fn is_empty(&self) -> bool { self.0.iter().all(|col| col.len() == 0) } + + pub fn reduce(&self, n: usize, data_type: DataType) -> Option { + if n == 0 { + return Some(Self::default()); + } + let count = self.len(); + if n >= count { + return None; + } + + let step = count / n; + let offset = step / 2; + let indices = self + .0 + .iter() + .enumerate() + .rev() + .flat_map(|(b_idx, col)| std::iter::repeat_n(b_idx, col.len()).zip(0..col.len())) + .enumerate() + .take(step * n) + .filter_map(|(i, (block, row))| { + if i % step == offset { + Some((block as u32, row as u32, 1)) + } else { + None + } + }) + .collect::>(); + + Some(Bounds(vec![Column::take_column_indices( + &self.0, + data_type, + &indices, + indices.len(), + )])) + } } impl SortedStream for Bounds { @@ -107,7 +143,7 @@ impl SortedStream for Bounds { #[cfg(test)] mod tests { - + use databend_common_expression::types::ArgType; use databend_common_expression::types::Int32Type; use databend_common_expression::FromData; use databend_common_pipeline_transforms::sort::SimpleRowsAsc; @@ -168,4 +204,29 @@ mod tests { Ok(()) } + + #[test] + fn test_reduce() -> Result<()> { + let data = vec![vec![77, -2, 7], vec![3, 8, 6, 1, 1], vec![2]]; + + let data = data + .into_iter() + .map(|v| Bounds::from_column::>(Int32Type::from_data(v))) + .collect::>>()?; + let bounds = Bounds::merge::>(data, 2)?; + + let got = bounds.reduce(4, Int32Type::data_type()).unwrap(); + assert_eq!(got, Bounds(vec![Int32Type::from_data(vec![8, 6, 2, 1])])); // 77 _8 7 _6 3 _2 1 _1 -2 + + let got = bounds.reduce(3, Int32Type::data_type()).unwrap(); + assert_eq!(got, Bounds(vec![Int32Type::from_data(vec![8, 3, 1])])); // 77 _8 7 6 _3 2 1 _1 -2 + + let got = bounds.reduce(2, Int32Type::data_type()).unwrap(); + assert_eq!(got, Bounds(vec![Int32Type::from_data(vec![7, 1])])); // 77 8 _7 6 3 2 _1 1 -2 + + let got = bounds.reduce(1, Int32Type::data_type()).unwrap(); + assert_eq!(got, Bounds(vec![Int32Type::from_data(vec![3])])); // 77 8 7 6 _3 2 1 1 -2 + + Ok(()) + } } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs index 128bd76f80a9e..4223252dd2b34 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs @@ -36,6 +36,7 @@ mod collect; mod execute; mod merge_sort; mod sort_spill; +mod wait; use sort_spill::SpillableBlock; diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_sample.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_sample.rs index a2925d6a43a21..b868de0a6383d 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_sample.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_sample.rs @@ -206,14 +206,14 @@ impl Transform for TransformSortSample { fn on_finish(&mut self) -> Result<()> { self.sampler.compact_blocks(); - let mut simple = self.sampler.take_blocks(); - assert!(simple.len() <= 1); // Unlikely to sample rows greater than 65536 + let mut sample = self.sampler.take_blocks(); + assert!(sample.len() <= 1); // Unlikely to sample rows greater than 65536 self.state.commit_sample( self.id, - if simple.is_empty() { + if sample.is_empty() { None } else { - Some(simple.remove(0)) + Some(sample.remove(0)) }, )?; Ok(()) diff --git a/src/query/service/src/pipelines/processors/transforms/sort/wait.rs b/src/query/service/src/pipelines/processors/transforms/sort/wait.rs new file mode 100644 index 0000000000000..7029a4ae551fb --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/sort/wait.rs @@ -0,0 +1,210 @@ +// 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::sync::RwLock; + +use databend_common_base::base::WatchNotify; +use databend_common_exception::Result; +use databend_common_expression::BlockMetaInfoDowncast; +use databend_common_expression::DataBlock; +use databend_common_expression::DataSchemaRef; +use databend_common_expression::SortColumnDescription; +use databend_common_pipeline_transforms::processors::sort::select_row_type; +use databend_common_pipeline_transforms::processors::sort::Rows; +use databend_common_pipeline_transforms::processors::sort::RowsTypeVisitor; +use databend_common_pipeline_transforms::sort::RowConverter; + +use super::bounds::Bounds; +use super::SortCollectedMeta; +use crate::pipelines::processors::Event; +use crate::pipelines::processors::InputPort; +use crate::pipelines::processors::OutputPort; +use crate::pipelines::processors::Processor; + +pub struct TransformSortSampleWait { + input: Arc, + output: Arc, + schema: DataSchemaRef, + sort_desc: Arc<[SortColumnDescription]>, + id: usize, + meta: Option>, + state: Arc, +} + +impl TransformSortSampleWait { + pub fn new( + input: Arc, + output: Arc, + id: usize, + schema: DataSchemaRef, + sort_desc: Arc<[SortColumnDescription]>, + state: Arc, + ) -> Self { + Self { + input, + output, + id, + state, + schema, + sort_desc, + meta: None, + } + } +} + +#[async_trait::async_trait] +impl Processor for TransformSortSampleWait { + fn name(&self) -> String { + "TransformSortSimpleWait".to_string() + } + + 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(meta) = self.meta.take() { + self.output.push_data(Ok(DataBlock::empty_with_meta(meta))); + self.output.finish(); + return Ok(Event::Finished); + } + + if let Some(mut block) = self.input.pull_data().transpose()? { + assert!(self.meta.is_none()); + let meta = block + .take_meta() + .and_then(SortCollectedMeta::downcast_from) + .expect("require a SortCollectedMeta"); + + self.meta = Some(Box::new(meta)); + return Ok(Event::Async); + } + + if self.input.is_finished() { + if self.state.done.has_notified() { + self.output.finish(); + Ok(Event::Finished) + } else { + Ok(Event::Async) + } + } else { + self.input.set_need_data(); + Ok(Event::NeedData) + } + } + + #[async_backtrace::framed] + async fn async_process(&mut self) -> Result<()> { + let bounds = self + .meta + .as_ref() + .map(|meta| meta.bounds.clone()) + .unwrap_or_default(); + + let mut commit = CommitSample { + inner: self, + bounds: Some(bounds), + result: Ok(false), + }; + select_row_type(&mut commit); + commit.result?; + self.state.done.notified().await; + Ok(()) + } +} + +struct CommitSample<'a> { + inner: &'a TransformSortSampleWait, + bounds: Option, + result: Result, +} + +impl<'a> RowsTypeVisitor for CommitSample<'a> { + fn schema(&self) -> DataSchemaRef { + self.inner.schema.clone() + } + + fn sort_desc(&self) -> &[SortColumnDescription] { + &self.inner.sort_desc + } + + fn visit_type(&mut self) + where + R: Rows + 'static, + C: RowConverter + Send + 'static, + { + self.result = self + .inner + .state + .commit_sample::(self.inner.id, self.bounds.take().unwrap()); + } +} + +pub struct SortSampleState { + inner: RwLock, + pub(super) done: WatchNotify, +} + +impl SortSampleState { + pub fn commit_sample(&self, id: usize, bounds: Bounds) -> Result { + let mut inner = self.inner.write().unwrap(); + + let x = inner.partial[id].replace(bounds); + assert!(x.is_none()); + let done = inner.partial.iter().all(Option::is_some); + if done { + inner.determine_bounds::()?; + self.done.notify_waiters(); + } + Ok(done) + } +} + +struct StateInner { + // target partitions + partitions: usize, + // schema for bounds DataBlock + // schema: DataSchemaRef, + // sort_desc for bounds DataBlock + // sort_desc: Vec, + partial: Vec>, + bounds: Option, + batch_rows: usize, +} + +impl StateInner { + fn determine_bounds(&mut self) -> Result<()> { + let v = self.partial.drain(..).map(Option::unwrap).collect(); + let bounds = Bounds::merge::(v, self.batch_rows)?; + let bounds = bounds + .reduce(self.partitions - 1, R::data_type()) + .unwrap_or(bounds); + assert!(bounds.len() <= self.partitions - 1); + + self.bounds = Some(bounds); + Ok(()) + } +} From 40f7392eba42c210857787aa7211a3f380c08f0f Mon Sep 17 00:00:00 2001 From: coldWater Date: Wed, 23 Apr 2025 11:07:11 +0800 Subject: [PATCH 12/61] remove --- src/query/expression/src/simpler.rs | 200 ---------------------------- 1 file changed, 200 deletions(-) delete mode 100644 src/query/expression/src/simpler.rs diff --git a/src/query/expression/src/simpler.rs b/src/query/expression/src/simpler.rs deleted file mode 100644 index ab928d721ec77..0000000000000 --- a/src/query/expression/src/simpler.rs +++ /dev/null @@ -1,200 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -mod reservoir_sampling; - -use std::collections::HashSet; - -use rand::Rng; -use reservoir_sampling::AlgoL; - -use crate::BlockRowIndex; -use crate::DataBlock; - -pub struct Simpler { - columns: Vec, - k: usize, - block_size: usize, - - blocks: Vec, - indices: Vec, - core: AlgoL, - - s: usize, -} - -impl Simpler { - pub fn new(columns: Vec, block_size: usize, k: usize, rng: R) -> Self { - let core = AlgoL::new(k.try_into().unwrap(), rng); - Self { - columns, - blocks: Vec::new(), - indices: Vec::with_capacity(k), - k, - block_size, - core, - s: usize::MAX, - } - } - - pub fn add_block(&mut self, data: DataBlock) -> bool { - let rows = data.num_rows(); - assert!(rows > 0); - let block_idx = self.blocks.len() as u32; - let change = self.add_indices(rows, block_idx); - if change { - let columns = self - .columns - .iter() - .map(|&offset| data.get_by_offset(offset).to_owned()) - .collect::>(); - - self.blocks.push(DataBlock::new(columns, rows)); - if self.blocks.len() > self.k { - self.compact_blocks() - } - } - change - } - - fn add_indices(&mut self, rows: usize, block_idx: u32) -> bool { - let mut change = false; - let mut cur: usize = 0; - if self.indices.len() < self.k { - if rows + self.indices.len() <= self.k { - for i in 0..rows { - self.indices.push((block_idx, i as u32, 1)); - } - if self.indices.len() == self.k { - self.s = self.core.search() - } - return true; - } - while self.indices.len() < self.k { - self.indices.push((block_idx, cur as u32, 1)); - cur += 1; - } - self.s = self.core.search(); - change = true; - } - - while rows - cur > self.s { - change = true; - cur += self.s; - self.indices[self.core.pos()] = (block_idx, cur as u32, 1); - self.core.update_w(); - self.s = self.core.search(); - } - - self.s -= rows - cur; - change - } - - pub fn compact_indices(&mut self) { - let used_set: HashSet<_> = self.indices.iter().map(|&(b, _, _)| b).collect(); - if used_set.len() == self.blocks.len() { - return; - } - - let mut used: Vec<_> = used_set.iter().cloned().collect(); - used.sort(); - - self.indices = self - .indices - .drain(..) - .map(|(b, r, c)| (used.binary_search(&b).unwrap() as u32, r, c)) - .collect(); - - self.blocks = self - .blocks - .drain(..) - .enumerate() - .filter_map(|(i, block)| { - if used_set.contains(&(i as u32)) { - Some(block) - } else { - None - } - }) - .collect(); - } - - pub fn compact_blocks(&mut self) { - self.blocks = self - .indices - .chunks_mut(self.block_size) - .enumerate() - .map(|(i, indices)| { - let rows = indices.len(); - let block = DataBlock::take_blocks(&self.blocks, indices, rows); - - for (j, (b, r, _)) in indices.iter_mut().enumerate() { - *b = i as u32; - *r = j as u32; - } - - block - }) - .collect::>(); - } - - pub fn memory_size(self) -> usize { - self.blocks.iter().map(|b| b.memory_size()).sum() - } - - pub fn take_blocks(&mut self) -> Vec { - std::mem::take(&mut self.blocks) - } - - pub fn k(&self) -> usize { - self.k - } -} - -#[cfg(test)] -mod tests { - use rand::rngs::StdRng; - use rand::SeedableRng; - - use super::*; - - #[test] - fn test_add_indices() { - let rng = StdRng::seed_from_u64(0); - let k = 5; - let core = AlgoL::new(k.try_into().unwrap(), rng); - let mut simpler = Simpler { - columns: vec![0], - k, - block_size: 65536, - blocks: Vec::new(), - indices: Vec::new(), - core, - s: usize::MAX, - }; - - simpler.add_indices(15, 0); - - let want: Vec = - vec![(0, 10, 1), (0, 1, 1), (0, 2, 1), (0, 8, 1), (0, 12, 1)]; - assert_eq!(&want, &simpler.indices); - assert_eq!(0, simpler.s); - - simpler.add_indices(20, 1); - - let want: Vec = vec![(1, 0, 1), (0, 1, 1), (1, 6, 1), (0, 8, 1), (1, 9, 1)]; - assert_eq!(&want, &simpler.indices); - assert_eq!(1, simpler.s); - } -} From 7715bb44052a41f3f4139e7e835228e82bc3c0ef Mon Sep 17 00:00:00 2001 From: coldWater Date: Wed, 23 Apr 2025 11:53:52 +0800 Subject: [PATCH 13/61] scalar --- src/query/expression/src/types/array.rs | 2 +- src/query/expression/src/types/boolean.rs | 2 +- src/query/expression/src/types/map.rs | 2 +- .../processors/transforms/sort/rows/common.rs | 11 +++ .../processors/transforms/sort/rows/mod.rs | 8 +- .../processors/transforms/sort/rows/simple.rs | 19 +++++ .../processors/transforms/sort/bounds.rs | 11 ++- .../processors/transforms/sort/sort_spill.rs | 33 +++++---- .../processors/transforms/sort/wait.rs | 73 +++++++++---------- 9 files changed, 101 insertions(+), 60 deletions(-) diff --git a/src/query/expression/src/types/array.rs b/src/query/expression/src/types/array.rs index 04a050214e62a..6586103755b14 100755 --- a/src/query/expression/src/types/array.rs +++ b/src/query/expression/src/types/array.rs @@ -54,7 +54,7 @@ impl AccessType for ArrayType { scalar.clone() } - fn try_downcast_scalar<'a>(scalar: &'a ScalarRef) -> Option> { + fn try_downcast_scalar<'a>(scalar: &ScalarRef<'a>) -> Option> { match scalar { ScalarRef::Array(array) => T::try_downcast_column(array), _ => None, diff --git a/src/query/expression/src/types/boolean.rs b/src/query/expression/src/types/boolean.rs index 03cd92c8604ec..8635454287f0c 100644 --- a/src/query/expression/src/types/boolean.rs +++ b/src/query/expression/src/types/boolean.rs @@ -50,7 +50,7 @@ impl AccessType for BooleanType { *scalar } - fn try_downcast_scalar<'a>(scalar: &'a ScalarRef) -> Option> { + fn try_downcast_scalar<'a>(scalar: &ScalarRef<'a>) -> Option> { match scalar { ScalarRef::Boolean(scalar) => Some(*scalar), _ => None, diff --git a/src/query/expression/src/types/map.rs b/src/query/expression/src/types/map.rs index 9e3dd29849f4d..1f5b090b15885 100755 --- a/src/query/expression/src/types/map.rs +++ b/src/query/expression/src/types/map.rs @@ -451,7 +451,7 @@ impl AccessType for MapType { MapInternal::::to_scalar_ref(scalar) } - fn try_downcast_scalar<'a>(scalar: &'a ScalarRef) -> Option> { + fn try_downcast_scalar<'a>(scalar: &ScalarRef<'a>) -> Option> { match scalar { ScalarRef::Map(array) => KvPair::::try_downcast_column(array), _ => None, diff --git a/src/query/pipeline/transforms/src/processors/transforms/sort/rows/common.rs b/src/query/pipeline/transforms/src/processors/transforms/sort/rows/common.rs index bd837dabdbde3..358c1c0ce8d0c 100644 --- a/src/query/pipeline/transforms/src/processors/transforms/sort/rows/common.rs +++ b/src/query/pipeline/transforms/src/processors/transforms/sort/rows/common.rs @@ -58,6 +58,17 @@ impl Rows for BinaryColumn { fn slice(&self, range: Range) -> Self { self.slice(range) } + + fn scalar_as_item<'a>(s: &'a Scalar) -> Self::Item<'a> { + match s { + Scalar::Binary(s) => s, + _ => unreachable!(), + } + } + + fn owned_item(item: Self::Item<'_>) -> Scalar { + Scalar::Binary(Vec::from(item)) + } } impl RowConverter for CommonRowConverter { diff --git a/src/query/pipeline/transforms/src/processors/transforms/sort/rows/mod.rs b/src/query/pipeline/transforms/src/processors/transforms/sort/rows/mod.rs index b1498c183c1e3..5eb5036aec6c5 100644 --- a/src/query/pipeline/transforms/src/processors/transforms/sort/rows/mod.rs +++ b/src/query/pipeline/transforms/src/processors/transforms/sort/rows/mod.rs @@ -17,6 +17,7 @@ mod simple; mod utils; use std::fmt::Debug; +use std::ops::Range; pub use common::*; use databend_common_exception::ErrorCode; @@ -27,6 +28,7 @@ use databend_common_expression::BlockEntry; use databend_common_expression::Column; use databend_common_expression::DataBlock; use databend_common_expression::DataSchemaRef; +use databend_common_expression::Scalar; use databend_common_expression::SortColumnDescription; pub use simple::*; pub use utils::*; @@ -95,5 +97,9 @@ where Self: Sized + Clone + Debug + Send self.row(self.len() - 1) } - fn slice(&self, range: std::ops::Range) -> Self; + fn slice(&self, range: Range) -> Self; + + fn scalar_as_item<'a>(s: &'a Scalar) -> Self::Item<'a>; + + fn owned_item(item: Self::Item<'_>) -> Scalar; } diff --git a/src/query/pipeline/transforms/src/processors/transforms/sort/rows/simple.rs b/src/query/pipeline/transforms/src/processors/transforms/sort/rows/simple.rs index 9d1bd9e430ef8..91299a0eefd25 100644 --- a/src/query/pipeline/transforms/src/processors/transforms/sort/rows/simple.rs +++ b/src/query/pipeline/transforms/src/processors/transforms/sort/rows/simple.rs @@ -23,6 +23,7 @@ use databend_common_expression::types::ValueType; use databend_common_expression::BlockEntry; use databend_common_expression::Column; use databend_common_expression::DataSchemaRef; +use databend_common_expression::Scalar; use databend_common_expression::SortColumnDescription; use super::RowConverter; @@ -68,6 +69,15 @@ where inner: T::slice_column(&self.inner, range), } } + + fn scalar_as_item<'a>(s: &'a Scalar) -> Self::Item<'a> { + let s = &s.as_ref(); + T::try_downcast_scalar(s).unwrap() + } + + fn owned_item(item: Self::Item<'_>) -> Scalar { + T::upcast_scalar(T::to_owned_scalar(item)) + } } /// Rows structure for single simple types. (numbers, date, timestamp) @@ -111,6 +121,15 @@ where inner: T::slice_column(&self.inner, range), } } + + fn scalar_as_item<'a>(s: &'a Scalar) -> Self::Item<'a> { + let s = &s.as_ref(); + Reverse(T::try_downcast_scalar(s).unwrap()) + } + + fn owned_item(item: Self::Item<'_>) -> Scalar { + T::upcast_scalar(T::to_owned_scalar(item.0)) + } } /// If there is only one sort field and its type is a primitive type, diff --git a/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs b/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs index 172a8fb1a80fc..f139d6ec2a409 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs @@ -18,6 +18,7 @@ use databend_common_expression::Column; use databend_common_expression::DataBlock; use databend_common_expression::DataField; use databend_common_expression::DataSchema; +use databend_common_expression::Scalar; use databend_common_expression::SortColumnDescription; use databend_common_pipeline_transforms::sort::LoserTreeMerger; use databend_common_pipeline_transforms::sort::Rows; @@ -70,13 +71,17 @@ impl Bounds { } } - pub fn next_bound(&mut self) -> Option { + pub fn next_bound(&mut self) -> Option { let last = self.0.last_mut()?; match last.len() { 0 => unreachable!(), - 1 => Some(self.0.pop().unwrap()), + 1 => { + let bound = last.index(0).unwrap().to_owned(); + self.0.pop(); + Some(bound) + } _ => { - let bound = last.slice(0..1).maybe_gc(); + let bound = last.index(0).unwrap().to_owned(); *last = last.slice(1..last.len()); Some(bound) } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs index 873397621e433..e822d4097f835 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs @@ -17,6 +17,7 @@ use std::fmt; use std::fmt::Debug; use std::fmt::Formatter; use std::intrinsics::unlikely; +use std::marker::PhantomData; use std::mem; use std::sync::atomic; use std::sync::atomic::AtomicBool; @@ -29,6 +30,7 @@ use databend_common_expression::sampler::FixedRateSampler; use databend_common_expression::Column; use databend_common_expression::DataBlock; use databend_common_expression::DataSchemaRef; +use databend_common_expression::Scalar; use databend_common_pipeline_transforms::processors::sort::algorithm::SortAlgorithm; use databend_common_pipeline_transforms::processors::sort::Merger; use databend_common_pipeline_transforms::processors::sort::Rows; @@ -65,7 +67,7 @@ struct StepSort { /// Partition boundaries for restoring and sorting blocks. /// Each boundary represents a cutoff point where data less than or equal to it belongs to one partition. bounds: Bounds, - cur_bound: Option, + cur_bound: Option, subsequent: Vec>>, current: Vec>>, @@ -304,7 +306,7 @@ impl StepCollect { impl StepSort { fn next_bound(&mut self) { match self.bounds.next_bound() { - Some(bound) => self.cur_bound = Some(A::Rows::from_column(&bound).unwrap()), + Some(bound) => self.cur_bound = Some(bound), None => self.cur_bound = None, } } @@ -496,13 +498,14 @@ impl Base { fn new_stream( &self, blocks: VecDeque, - bound: Option, + bound: Option, ) -> BoundBlockStream> { BoundBlockStream { blocks, bound, sort_row_offset: self.sort_row_offset, spiller: self.spiller.clone(), + _r: Default::default(), } } @@ -653,9 +656,10 @@ impl Spill for Arc { /// BoundBlockStream is a stream of blocks that are cutoff less or equal than bound. struct BoundBlockStream { blocks: VecDeque, - bound: Option, + bound: Option, sort_row_offset: usize, spiller: S, + _r: PhantomData, } impl Debug for BoundBlockStream { @@ -689,7 +693,7 @@ impl BoundBlockStream { }; match &self.bound { - Some(bound) => block.domain::().first() <= bound.row(0), + Some(bound) => block.domain::().first() <= R::scalar_as_item(bound), None => true, } } @@ -701,7 +705,7 @@ impl BoundBlockStream { let block = self.blocks.front_mut().unwrap(); if let Some(pos) = - block_split_off_position(block.data.as_ref().unwrap(), bound, self.sort_row_offset) + block_split_off_position::(block.data.as_ref().unwrap(), bound, self.sort_row_offset) { block.slice(pos, self.sort_row_offset) } else { @@ -770,13 +774,12 @@ impl BoundBlockStream { fn block_split_off_position( data: &DataBlock, - bound: &R, + bound: &Scalar, sort_row_offset: usize, ) -> Option { let rows = R::from_column(sort_column(data, sort_row_offset)).unwrap(); debug_assert!(rows.len() > 0); - debug_assert!(bound.len() == 1); - let bound = bound.row(0); + let bound = R::scalar_as_item(bound); partition_point(&rows, &bound) } @@ -856,6 +859,7 @@ mod tests { use databend_common_expression::types::DataType; use databend_common_expression::types::Int32Type; use databend_common_expression::types::NumberDataType; + use databend_common_expression::types::NumberScalar; use databend_common_expression::types::StringType; use databend_common_expression::Column; use databend_common_expression::DataField; @@ -885,13 +889,13 @@ mod tests { async fn run_bound_block_stream( spiller: impl Spill + Clone, sort_desc: Arc>, - bound: Column, + bound: Scalar, block_part: usize, want: Column, ) -> Result<()> { let (schema, block) = test_data(); let block = DataBlock::sort(&block, &sort_desc, None)?; - let bound = Some(R::from_column(&bound)?); + let bound = Some(bound); let sort_row_offset = schema.fields().len(); let blocks = vec![ @@ -911,6 +915,7 @@ mod tests { bound, sort_row_offset, spiller: spiller.clone(), + _r: Default::default(), }; let data = stream.take_next_bounded_block(); @@ -936,7 +941,7 @@ mod tests { run_bound_block_stream::>( spiller.clone(), sort_desc.clone(), - Int32Type::from_data(vec![5]), + Scalar::Number(NumberScalar::Int32(5)), 4, Int32Type::from_data(vec![3, 5]), ) @@ -945,7 +950,7 @@ mod tests { run_bound_block_stream::>( spiller.clone(), sort_desc.clone(), - Int32Type::from_data(vec![8]), + Scalar::Number(NumberScalar::Int32(8)), 4, Int32Type::from_data(vec![3, 5, 7, 7]), ) @@ -962,7 +967,7 @@ mod tests { run_bound_block_stream::>( spiller.clone(), sort_desc.clone(), - StringType::from_data(vec!["f"]), + Scalar::String("f".to_string()), 4, StringType::from_data(vec!["w", "h", "g", "f"]), ) diff --git a/src/query/service/src/pipelines/processors/transforms/sort/wait.rs b/src/query/service/src/pipelines/processors/transforms/sort/wait.rs index 7029a4ae551fb..7e1fc53740541 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/wait.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/wait.rs @@ -124,59 +124,31 @@ impl Processor for TransformSortSampleWait { .map(|meta| meta.bounds.clone()) .unwrap_or_default(); - let mut commit = CommitSample { - inner: self, - bounds: Some(bounds), - result: Ok(false), - }; - select_row_type(&mut commit); - commit.result?; + self.state.commit_sample(self.id, bounds)?; self.state.done.notified().await; Ok(()) } } -struct CommitSample<'a> { - inner: &'a TransformSortSampleWait, - bounds: Option, - result: Result, -} - -impl<'a> RowsTypeVisitor for CommitSample<'a> { - fn schema(&self) -> DataSchemaRef { - self.inner.schema.clone() - } - - fn sort_desc(&self) -> &[SortColumnDescription] { - &self.inner.sort_desc - } - - fn visit_type(&mut self) - where - R: Rows + 'static, - C: RowConverter + Send + 'static, - { - self.result = self - .inner - .state - .commit_sample::(self.inner.id, self.bounds.take().unwrap()); - } -} - pub struct SortSampleState { inner: RwLock, pub(super) done: WatchNotify, } impl SortSampleState { - pub fn commit_sample(&self, id: usize, bounds: Bounds) -> Result { + pub fn commit_sample(&self, id: usize, bounds: Bounds) -> Result { let mut inner = self.inner.write().unwrap(); let x = inner.partial[id].replace(bounds); assert!(x.is_none()); let done = inner.partial.iter().all(Option::is_some); if done { - inner.determine_bounds::()?; + let mut visitor = DetermineBounds { + inner: &mut inner, + result: Ok(()), + }; + select_row_type(&mut visitor); + visitor.result?; self.done.notify_waiters(); } Ok(done) @@ -187,9 +159,9 @@ struct StateInner { // target partitions partitions: usize, // schema for bounds DataBlock - // schema: DataSchemaRef, + schema: DataSchemaRef, // sort_desc for bounds DataBlock - // sort_desc: Vec, + sort_desc: Arc<[SortColumnDescription]>, partial: Vec>, bounds: Option, batch_rows: usize, @@ -202,9 +174,32 @@ impl StateInner { let bounds = bounds .reduce(self.partitions - 1, R::data_type()) .unwrap_or(bounds); - assert!(bounds.len() <= self.partitions - 1); + assert!(bounds.len() < self.partitions); self.bounds = Some(bounds); Ok(()) } } + +struct DetermineBounds<'a> { + inner: &'a mut StateInner, + result: Result<()>, +} + +impl<'a> RowsTypeVisitor for DetermineBounds<'a> { + fn schema(&self) -> DataSchemaRef { + self.inner.schema.clone() + } + + fn sort_desc(&self) -> &[SortColumnDescription] { + &self.inner.sort_desc + } + + fn visit_type(&mut self) + where + R: Rows + 'static, + C: RowConverter + Send + 'static, + { + self.result = self.inner.determine_bounds::(); + } +} From 9089940ad78317bb062ef8c82ae2db09649027a5 Mon Sep 17 00:00:00 2001 From: coldWater Date: Wed, 23 Apr 2025 16:46:29 +0800 Subject: [PATCH 14/61] exchange --- .../processors/transforms/sort/exchange.rs | 72 +++++++++++++++++++ .../processors/transforms/sort/mod.rs | 1 + 2 files changed, 73 insertions(+) create mode 100644 src/query/service/src/pipelines/processors/transforms/sort/exchange.rs diff --git a/src/query/service/src/pipelines/processors/transforms/sort/exchange.rs b/src/query/service/src/pipelines/processors/transforms/sort/exchange.rs new file mode 100644 index 0000000000000..8f47822383d78 --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/sort/exchange.rs @@ -0,0 +1,72 @@ +// 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::marker::PhantomData; +use std::sync::Arc; + +use databend_common_exception::Result; +use databend_common_expression::DataBlock; +use databend_common_pipeline_core::processors::Exchange; +use databend_common_pipeline_transforms::processors::sort::Rows; + +use super::sort_sample::SortSampleState; + +pub struct SortRangeExchange { + state: Arc, + _r: PhantomData, +} + +unsafe impl Send for SortRangeExchange {} + +unsafe impl Sync for SortRangeExchange {} + +impl Exchange for SortRangeExchange { + const NAME: &'static str = "SortRange"; + fn partition(&self, data: DataBlock, n: usize) -> Result> { + let bounds = self.state.bounds().unwrap(); + debug_assert_eq!(n, self.state.partitions()); + debug_assert!(bounds.len() < n); + + if data.is_empty() { + return Ok(vec![]); + } + + if bounds.len() == 0 { + return Ok(vec![data]); + } + + let bounds = R::from_column(&bounds)?; + let rows = R::from_column(data.get_last_column())?; + + let mut i = 0; + let mut j = 0; + let mut bound = bounds.row(j); + let mut indices = Vec::new(); + while i < rows.len() { + match rows.row(i).cmp(&bound) { + Ordering::Less => indices.push(j as u32), + Ordering::Greater if j + 1 < bounds.len() => { + j += 1; + bound = bounds.row(j); + continue; + } + _ => indices.push(j as u32 + 1), + } + i += 1; + } + + DataBlock::scatter(&data, &indices, n) + } +} diff --git a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs index 4223252dd2b34..bb40e97ee8d6d 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs @@ -33,6 +33,7 @@ pub use builder::TransformSortBuilder; mod bounds; mod collect; +mod exchange; mod execute; mod merge_sort; mod sort_spill; From 129b7896ea83afe2cc4b24b913482db0b90ac1f0 Mon Sep 17 00:00:00 2001 From: coldWater Date: Wed, 23 Apr 2025 21:59:07 +0800 Subject: [PATCH 15/61] update --- .../processors/transforms/sort/bounds.rs | 1 - .../processors/transforms/sort/exchange.rs | 53 +++--- .../processors/transforms/sort/mod.rs | 8 + .../processors/transforms/sort/sort_spill.rs | 55 +++++- .../processors/transforms/sort/wait.rs | 180 +++++++++++------- 5 files changed, 198 insertions(+), 99 deletions(-) diff --git a/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs b/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs index f139d6ec2a409..741ccca55d66a 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs @@ -92,7 +92,6 @@ impl Bounds { self.0.iter().map(Column::len).sum() } - #[expect(dead_code)] pub fn is_empty(&self) -> bool { self.0.iter().all(|col| col.len() == 0) } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/exchange.rs b/src/query/service/src/pipelines/processors/transforms/sort/exchange.rs index 8f47822383d78..9aeeb29cc7414 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/exchange.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/exchange.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::cmp::Ordering; use std::marker::PhantomData; use std::sync::Arc; @@ -21,7 +20,7 @@ use databend_common_expression::DataBlock; use databend_common_pipeline_core::processors::Exchange; use databend_common_pipeline_transforms::processors::sort::Rows; -use super::sort_sample::SortSampleState; +use super::wait::SortSampleState; pub struct SortRangeExchange { state: Arc, @@ -35,38 +34,40 @@ unsafe impl Sync for SortRangeExchange {} impl Exchange for SortRangeExchange { const NAME: &'static str = "SortRange"; fn partition(&self, data: DataBlock, n: usize) -> Result> { - let bounds = self.state.bounds().unwrap(); - debug_assert_eq!(n, self.state.partitions()); - debug_assert!(bounds.len() < n); - if data.is_empty() { return Ok(vec![]); } - if bounds.len() == 0 { + let bounds = self.state.bounds(); + // debug_assert_eq!(n, self.state.partitions()); + debug_assert!(bounds.len() < n); + + if bounds.is_empty() { return Ok(vec![data]); } - let bounds = R::from_column(&bounds)?; - let rows = R::from_column(data.get_last_column())?; + todo!() - let mut i = 0; - let mut j = 0; - let mut bound = bounds.row(j); - let mut indices = Vec::new(); - while i < rows.len() { - match rows.row(i).cmp(&bound) { - Ordering::Less => indices.push(j as u32), - Ordering::Greater if j + 1 < bounds.len() => { - j += 1; - bound = bounds.row(j); - continue; - } - _ => indices.push(j as u32 + 1), - } - i += 1; - } + // let bounds = R::from_column(&bounds.0)?; + // let rows = R::from_column(data.get_last_column())?; + + // let mut i = 0; + // let mut j = 0; + // let mut bound = bounds.row(j); + // let mut indices = Vec::new(); + // while i < rows.len() { + // match rows.row(i).cmp(&bound) { + // Ordering::Less => indices.push(j as u32), + // Ordering::Greater if j + 1 < bounds.len() => { + // j += 1; + // bound = bounds.row(j); + // continue; + // } + // _ => indices.push(j as u32 + 1), + // } + // i += 1; + // } - DataBlock::scatter(&data, &indices, n) + // DataBlock::scatter(&data, &indices, n) } } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs index bb40e97ee8d6d..307c1cefdf4d9 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs @@ -63,6 +63,14 @@ local_block_meta_serde!(SortCollectedMeta); #[typetag::serde(name = "sort_collected")] impl BlockMetaInfo for SortCollectedMeta {} +#[derive(Debug)] +struct SortScatteredMeta(pub Vec); + +local_block_meta_serde!(SortScatteredMeta); + +#[typetag::serde(name = "sort_scattered")] +impl BlockMetaInfo for SortScatteredMeta {} + trait MemoryRows { fn in_memory_rows(&self) -> usize; } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs index e822d4097f835..b477dcc2a18e3 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs @@ -533,6 +533,27 @@ impl Base { } } } + + pub async fn scatter_stream( + &self, + mut blocks: VecDeque, + mut bounds: Bounds, + ) -> Result>> { + let mut scattered = Vec::with_capacity(bounds.len() + 1); + while !blocks.is_empty() { + let bound = bounds.next_bound(); + let mut stream = self.new_stream::(blocks, bound); + + let mut part = Vec::new(); + while let Some(block) = stream.take_next_bounded_spillable().await? { + part.push(block); + } + + scattered.push(part); + blocks = stream.blocks; + } + Ok(scattered) + } } impl MemoryRows for Vec> { @@ -693,8 +714,8 @@ impl BoundBlockStream { }; match &self.bound { - Some(bound) => block.domain::().first() <= R::scalar_as_item(bound), None => true, + Some(bound) => block.domain::().first() <= R::scalar_as_item(bound), } } @@ -770,6 +791,38 @@ impl BoundBlockStream { } Ok(()) } + + async fn take_next_bounded_spillable(&mut self) -> Result> { + let Some(bound) = &self.bound else { + return Ok(self.blocks.pop_front()); + }; + let Some(block) = self.blocks.front() else { + return Ok(None); + }; + { + let domain = block.domain::(); + let bound_item = R::scalar_as_item(bound); + if domain.first() > bound_item { + return Ok(None); + } + if domain.last() <= bound_item { + return Ok(self.blocks.pop_front()); + } + } + self.restore_first().await?; + + let block = self.blocks.front_mut().unwrap(); + if let Some(pos) = block_split_off_position::( + block.data.as_ref().unwrap(), + self.bound.as_ref().unwrap(), + self.sort_row_offset, + ) { + let data = block.slice(pos, self.sort_row_offset); + Ok(Some(SpillableBlock::new(data, self.sort_row_offset))) + } else { + Ok(self.blocks.pop_front()) + } + } } fn block_split_off_position( diff --git a/src/query/service/src/pipelines/processors/transforms/sort/wait.rs b/src/query/service/src/pipelines/processors/transforms/sort/wait.rs index 7e1fc53740541..1e2f04581c1ae 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/wait.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/wait.rs @@ -13,6 +13,8 @@ // limitations under the License. use std::any::Any; +use std::assert_matches::assert_matches; +use std::marker::PhantomData; use std::sync::Arc; use std::sync::RwLock; @@ -21,54 +23,108 @@ use databend_common_exception::Result; use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::DataBlock; use databend_common_expression::DataSchemaRef; -use databend_common_expression::SortColumnDescription; -use databend_common_pipeline_transforms::processors::sort::select_row_type; use databend_common_pipeline_transforms::processors::sort::Rows; -use databend_common_pipeline_transforms::processors::sort::RowsTypeVisitor; -use databend_common_pipeline_transforms::sort::RowConverter; use super::bounds::Bounds; +use super::Base; use super::SortCollectedMeta; +use super::SortScatteredMeta; use crate::pipelines::processors::Event; use crate::pipelines::processors::InputPort; use crate::pipelines::processors::OutputPort; use crate::pipelines::processors::Processor; +use crate::spillers::Spiller; -pub struct TransformSortSampleWait { +#[derive(Debug)] +enum Step { + None, + Meta(Box), + Scattered(Vec), +} + +pub struct TransformSortWait { input: Arc, output: Arc, - schema: DataSchemaRef, - sort_desc: Arc<[SortColumnDescription]>, id: usize, - meta: Option>, + step: Step, state: Arc, + spiller: Arc, + _r: PhantomData, } -impl TransformSortSampleWait { - pub fn new( - input: Arc, - output: Arc, - id: usize, - schema: DataSchemaRef, - sort_desc: Arc<[SortColumnDescription]>, - state: Arc, - ) -> Self { - Self { - input, - output, - id, - state, - schema, - sort_desc, - meta: None, +impl TransformSortWait { + // pub fn new( + // input: Arc, + // output: Arc, + // id: usize, + // state: Arc, + // ) -> Self { + // Self { + // input, + // output, + // id, + // state, + // meta: None, + // _r: Default::default(), + // } + // } + + async fn scatter(&mut self) -> Result<()> { + let scatter_bounds = self.state.bounds(); + + let Step::Meta(box SortCollectedMeta { + params, + bounds, + blocks, + }) = std::mem::replace(&mut self.step, Step::None) + else { + unreachable!() + }; + + if scatter_bounds.is_empty() { + Step::Scattered(vec![SortCollectedMeta { + params, + bounds, + blocks, + }]); + return Ok(()); + } + + let base = { + let inner = self.state.inner.read().unwrap(); + Base { + schema: inner.schema.clone(), + spiller: self.spiller.clone(), + sort_row_offset: inner.schema.fields.len() - 1, + limit: None, + } + }; + + let mut scattered_meta = std::iter::repeat_with(|| SortCollectedMeta { + params, + bounds: bounds.clone(), + blocks: vec![], + }) + .take(scatter_bounds.len() + 1) + .collect::>(); + for blocks in blocks { + let scattered = base + .scatter_stream::(Vec::from(blocks).into(), scatter_bounds.clone()) + .await?; + for (i, part) in scattered.into_iter().enumerate() { + scattered_meta[i].blocks.push(part.into_boxed_slice()); + } } + self.step = Step::Scattered(scattered_meta); + + Ok(()) } } #[async_trait::async_trait] -impl Processor for TransformSortSampleWait { +impl Processor for TransformSortWait { fn name(&self) -> String { - "TransformSortSimpleWait".to_string() + "TransformSortWait".to_string() } fn as_any(&mut self) -> &mut dyn Any { @@ -86,20 +142,25 @@ impl Processor for TransformSortSampleWait { return Ok(Event::NeedConsume); } - if let Some(meta) = self.meta.take() { - self.output.push_data(Ok(DataBlock::empty_with_meta(meta))); + if matches!(self.step, Step::Scattered(_)) { + let Step::Scattered(scattered) = std::mem::replace(&mut self.step, Step::None) else { + unreachable!() + }; + + let data = DataBlock::empty_with_meta(Box::new(SortScatteredMeta(scattered))); + self.output.push_data(Ok(data)); self.output.finish(); return Ok(Event::Finished); } if let Some(mut block) = self.input.pull_data().transpose()? { - assert!(self.meta.is_none()); + assert_matches!(self.step, Step::None); let meta = block .take_meta() .and_then(SortCollectedMeta::downcast_from) .expect("require a SortCollectedMeta"); - self.meta = Some(Box::new(meta)); + self.step = Step::Meta(Box::new(meta)); return Ok(Event::Async); } @@ -118,15 +179,13 @@ impl Processor for TransformSortSampleWait { #[async_backtrace::framed] async fn async_process(&mut self) -> Result<()> { - let bounds = self - .meta - .as_ref() - .map(|meta| meta.bounds.clone()) - .unwrap_or_default(); - - self.state.commit_sample(self.id, bounds)?; + let bounds = match &self.step { + Step::Meta(meta) => meta.bounds.clone(), + _ => unreachable!(), + }; + self.state.commit_sample::(self.id, bounds)?; self.state.done.notified().await; - Ok(()) + self.scatter().await } } @@ -136,23 +195,27 @@ pub struct SortSampleState { } impl SortSampleState { - pub fn commit_sample(&self, id: usize, bounds: Bounds) -> Result { + pub fn commit_sample(&self, id: usize, bounds: Bounds) -> Result { let mut inner = self.inner.write().unwrap(); let x = inner.partial[id].replace(bounds); assert!(x.is_none()); let done = inner.partial.iter().all(Option::is_some); if done { - let mut visitor = DetermineBounds { - inner: &mut inner, - result: Ok(()), - }; - select_row_type(&mut visitor); - visitor.result?; + inner.determine_bounds::()?; self.done.notify_waiters(); } Ok(done) } + + pub fn bounds(&self) -> Bounds { + self.inner + .read() + .unwrap() + .bounds + .clone() + .unwrap_or_default() + } } struct StateInner { @@ -160,8 +223,6 @@ struct StateInner { partitions: usize, // schema for bounds DataBlock schema: DataSchemaRef, - // sort_desc for bounds DataBlock - sort_desc: Arc<[SortColumnDescription]>, partial: Vec>, bounds: Option, batch_rows: usize, @@ -180,26 +241,3 @@ impl StateInner { Ok(()) } } - -struct DetermineBounds<'a> { - inner: &'a mut StateInner, - result: Result<()>, -} - -impl<'a> RowsTypeVisitor for DetermineBounds<'a> { - fn schema(&self) -> DataSchemaRef { - self.inner.schema.clone() - } - - fn sort_desc(&self) -> &[SortColumnDescription] { - &self.inner.sort_desc - } - - fn visit_type(&mut self) - where - R: Rows + 'static, - C: RowConverter + Send + 'static, - { - self.result = self.inner.determine_bounds::(); - } -} From 470e9730661114e00be8e1444846db34d94eb5ae Mon Sep 17 00:00:00 2001 From: coldWater Date: Thu, 24 Apr 2025 00:58:07 +0800 Subject: [PATCH 16/61] test Signed-off-by: coldWater --- .../processors/transforms/sort/sort_spill.rs | 294 ++++++++++++++++++ 1 file changed, 294 insertions(+) diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs index b477dcc2a18e3..06b3ced126956 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs @@ -1030,6 +1030,300 @@ mod tests { Ok(()) } + // Create test data with multiple blocks, including spilled and sliced blocks + async fn run_take_next_bounded_spillable( + spiller: impl Spill + Clone, + sort_desc: Arc>, + bound: Option, + expected_blocks: Vec, + // Flag to test with spilled blocks + with_spilled: bool, + // Flag to test with sliced blocks + with_sliced: bool, + ) -> Result<()> { + let (schema, block) = test_data(); + let block = DataBlock::sort(&block, &sort_desc, None)?; + let sort_row_offset = schema.fields().len(); + + // Create multiple blocks with different splits + let mut blocks = VecDeque::new(); + + // First block: 0..2 + let mut block1 = block.slice(0..2); + let col1 = convert_rows(schema.clone(), &sort_desc, block1.clone()).unwrap(); + block1.add_column(BlockEntry::new(col1.data_type(), Value::Column(col1))); + blocks.push_back(SpillableBlock::new(block1, sort_row_offset)); + + // Second block: 2..5 + let mut block2 = block.slice(2..5); + let col2 = convert_rows(schema.clone(), &sort_desc, block2.clone()).unwrap(); + block2.add_column(BlockEntry::new(col2.data_type(), Value::Column(col2))); + blocks.push_back(SpillableBlock::new(block2, sort_row_offset)); + + // We'll add the third block only if we're not using sliced blocks + // This is to avoid duplicating the data with additional_block + if !with_sliced { + // Third block: 5..8 + let mut block3 = block.slice(5..8); + let col3 = convert_rows(schema.clone(), &sort_desc, block3.clone()).unwrap(); + block3.add_column(BlockEntry::new(col3.data_type(), Value::Column(col3))); + blocks.push_back(SpillableBlock::new(block3, sort_row_offset)); + } + + // Spill some blocks if requested + if with_spilled { + // Spill the second block + blocks[1].spill(&spiller).await?; + } + + // Create a sliced block if requested + if with_sliced { + // Create a block for values 8..11 (the last part of the sorted data) + let mut additional_block = block.slice(5..8); + let col = convert_rows(schema.clone(), &sort_desc, additional_block.clone()).unwrap(); + additional_block.add_column(BlockEntry::new(col.data_type(), Value::Column(col))); + let mut spillable_block = SpillableBlock::new(additional_block, sort_row_offset); + + // Use SpillableBlock::slice to create a sliced block + // This tests the SpillableBlock::slice functionality by slicing at position 1 + // For ascending Int32: [8, 10, 11] -> [8] and [10, 11] + // For descending String: ["d", "e", "f"] -> ["d"] and ["e", "f"] + let sliced_data = spillable_block.slice(1, sort_row_offset); + let sliced_block = SpillableBlock::new(sliced_data, sort_row_offset); + + // Add both blocks to maintain the order + blocks.push_back(sliced_block); + blocks.push_back(spillable_block); + } + + let mut stream = BoundBlockStream:: { + blocks, + bound, + sort_row_offset, + spiller: spiller.clone(), + _r: Default::default(), + }; + + // Take blocks one by one and compare with expected + let mut result_blocks = Vec::new(); + while let Some(mut block) = stream.take_next_bounded_spillable().await? { + // If the block data is None (spilled), restore it first + if block.data.is_none() { + block.data = Some(spiller.restore(block.location.as_ref().unwrap()).await?); + } + + let data = block.data.unwrap(); + let col = sort_column(&data, sort_row_offset).clone(); + result_blocks.push(col); + } + + assert_eq!( + expected_blocks.len(), + result_blocks.len(), + "Number of blocks doesn't match" + ); + for (expected, actual) in expected_blocks.iter().zip(result_blocks.iter()) { + assert_eq!(expected, actual, "Block content doesn't match"); + } + + Ok(()) + } + + #[tokio::test] + async fn test_take_next_bounded_spillable() -> Result<()> { + let spiller = MockSpiller { + map: Arc::new(Mutex::new(HashMap::new())), + }; + + // Test with ascending Int32 type + { + let sort_desc = Arc::new(vec![SortColumnDescription { + offset: 0, + asc: true, + nulls_first: false, + }]); + + // Test 1: Basic test with bound = 5 (should return blocks with values <= 5) + // No spilled blocks, no sliced blocks + run_take_next_bounded_spillable::>( + spiller.clone(), + sort_desc.clone(), + Some(Scalar::Number(NumberScalar::Int32(5))), + vec![Int32Type::from_data(vec![3, 5])], + false, // no spilled blocks + false, // no sliced blocks + ) + .await?; + + // Test 2: With spilled blocks, bound = 8 (should return blocks with values <= 8) + run_take_next_bounded_spillable::>( + spiller.clone(), + sort_desc.clone(), + Some(Scalar::Number(NumberScalar::Int32(8))), + vec![ + Int32Type::from_data(vec![3, 5]), + Int32Type::from_data(vec![7, 7, 8]), + ], + true, // with spilled blocks + false, // no sliced blocks + ) + .await?; + + // Test 3: With sliced blocks, bound = 7 (should return blocks with values <= 7) + run_take_next_bounded_spillable::>( + spiller.clone(), + sort_desc.clone(), + Some(Scalar::Number(NumberScalar::Int32(7))), + vec![ + Int32Type::from_data(vec![3, 5]), + Int32Type::from_data(vec![7, 7]), + ], + false, // no spilled blocks + true, // with sliced blocks + ) + .await?; + + // Test 4: With both spilled and sliced blocks, bound = 10 + run_take_next_bounded_spillable::>( + spiller.clone(), + sort_desc.clone(), + Some(Scalar::Number(NumberScalar::Int32(10))), + vec![ + Int32Type::from_data(vec![3, 5]), + Int32Type::from_data(vec![7, 7, 8]), + Int32Type::from_data(vec![10]), + ], + true, // with spilled blocks + true, // with sliced blocks + ) + .await?; + + // Test 5: With bound = 2 (should return no blocks as all values > 2) + run_take_next_bounded_spillable::>( + spiller.clone(), + sort_desc.clone(), + Some(Scalar::Number(NumberScalar::Int32(2))), + vec![], + true, // with spilled blocks + true, // with sliced blocks + ) + .await?; + + // Test 6: With bound = 12 (should return all blocks as all values <= 12) + run_take_next_bounded_spillable::>( + spiller.clone(), + sort_desc.clone(), + Some(Scalar::Number(NumberScalar::Int32(12))), + vec![ + Int32Type::from_data(vec![3, 5]), + Int32Type::from_data(vec![7, 7, 8]), + Int32Type::from_data(vec![10, 11, 11]), + ], + true, // with spilled blocks + false, // no sliced blocks + ) + .await?; + + // Test 7: With no bound (should return all blocks) + run_take_next_bounded_spillable::>( + spiller.clone(), + sort_desc.clone(), + None, + vec![ + Int32Type::from_data(vec![3, 5]), + Int32Type::from_data(vec![7, 7, 8]), + Int32Type::from_data(vec![10, 11, 11]), + ], + true, // with spilled blocks + false, // no sliced blocks + ) + .await?; + } + + // Test with descending String type + { + let sort_desc = Arc::new(vec![SortColumnDescription { + offset: 1, + asc: false, + nulls_first: false, + }]); + + // Test 8: With bound = "f" (should return blocks with values >= "f") + run_take_next_bounded_spillable::>( + spiller.clone(), + sort_desc.clone(), + Some(Scalar::String("f".to_string())), + vec![ + StringType::from_data(vec!["w", "h"]), + StringType::from_data(vec!["g", "f"]), + ], + false, // no spilled blocks + false, // no sliced blocks + ) + .await?; + + // Test 9: With spilled blocks, bound = "e" (should return blocks with values >= "e") + run_take_next_bounded_spillable::>( + spiller.clone(), + sort_desc.clone(), + Some(Scalar::String("e".to_string())), + vec![ + StringType::from_data(vec!["w", "h"]), + StringType::from_data(vec!["g", "f", "e"]), + StringType::from_data(vec!["e"]), + ], + true, // with spilled blocks + false, // no sliced blocks + ) + .await?; + + // Test 10: With sliced blocks, bound = "d" (should return blocks with values >= "d") + run_take_next_bounded_spillable::>( + spiller.clone(), + sort_desc.clone(), + Some(Scalar::String("d".to_string())), + vec![ + StringType::from_data(vec!["w", "h"]), + StringType::from_data(vec!["g", "f", "e"]), + StringType::from_data(vec!["e"]), + StringType::from_data(vec!["d", "d"]), + ], + false, // no spilled blocks + true, // with sliced blocks + ) + .await?; + + // Test 11: With both spilled and sliced blocks, bound = "c" (should return all blocks) + run_take_next_bounded_spillable::>( + spiller.clone(), + sort_desc.clone(), + Some(Scalar::String("c".to_string())), + vec![ + StringType::from_data(vec!["w", "h"]), + StringType::from_data(vec!["g", "f", "e"]), + StringType::from_data(vec!["e"]), + StringType::from_data(vec!["d", "d"]), + ], + true, // with spilled blocks + true, // with sliced blocks + ) + .await?; + + // Test 12: With bound = "z" (should return no blocks as all values < "z") + run_take_next_bounded_spillable::>( + spiller.clone(), + sort_desc.clone(), + Some(Scalar::String("z".to_string())), + vec![], + true, // with spilled blocks + true, // with sliced blocks + ) + .await?; + } + + Ok(()) + } + #[derive(Clone)] struct MockSpiller { map: Arc>>, From c48f5631a0d374fa64f9938eaf272a8457815571 Mon Sep 17 00:00:00 2001 From: coldWater Date: Thu, 24 Apr 2025 12:06:05 +0800 Subject: [PATCH 17/61] x Signed-off-by: coldWater --- .../processors/transforms/sort/sort_spill.rs | 219 ++++++++++-------- .../processors/transforms/sort/wait.rs | 64 +++-- 2 files changed, 176 insertions(+), 107 deletions(-) diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs index 06b3ced126956..5eed43ed3b063 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs @@ -906,6 +906,7 @@ fn get_domain(col: &Column) -> Column { #[cfg(test)] mod tests { use std::collections::HashMap; + use std::ops::Range; use std::sync::Mutex; use databend_common_base::base::GlobalUniqName; @@ -941,7 +942,7 @@ mod tests { async fn run_bound_block_stream( spiller: impl Spill + Clone, - sort_desc: Arc>, + sort_desc: Arc<[SortColumnDescription]>, bound: Scalar, block_part: usize, want: Column, @@ -985,7 +986,7 @@ mod tests { }; { - let sort_desc = Arc::new(vec![SortColumnDescription { + let sort_desc = Arc::new([SortColumnDescription { offset: 0, asc: true, nulls_first: false, @@ -1011,7 +1012,7 @@ mod tests { } { - let sort_desc = Arc::new(vec![SortColumnDescription { + let sort_desc = Arc::new([SortColumnDescription { offset: 1, asc: false, nulls_first: false, @@ -1030,64 +1031,77 @@ mod tests { Ok(()) } - // Create test data with multiple blocks, including spilled and sliced blocks - async fn run_take_next_bounded_spillable( - spiller: impl Spill + Clone, - sort_desc: Arc>, - bound: Option, - expected_blocks: Vec, - // Flag to test with spilled blocks + fn create_spillable_block( + block: &DataBlock, + range: Range, + schema: &DataSchemaRef, + sort_desc: &[SortColumnDescription], + sort_row_offset: usize, + ) -> SpillableBlock { + let mut sliced_block = block.slice(range); + let col = convert_rows(schema.clone(), sort_desc, sliced_block.clone()).unwrap(); + sliced_block.add_column(BlockEntry::new(col.data_type(), Value::Column(col))); + SpillableBlock::new(sliced_block, sort_row_offset) + } + + async fn prepare_test_blocks( + spiller: &impl Spill, + sort_desc: &[SortColumnDescription], with_spilled: bool, - // Flag to test with sliced blocks with_sliced: bool, - ) -> Result<()> { + ) -> Result<(DataSchemaRef, VecDeque, usize)> { let (schema, block) = test_data(); - let block = DataBlock::sort(&block, &sort_desc, None)?; + let block = DataBlock::sort(&block, sort_desc, None)?; let sort_row_offset = schema.fields().len(); // Create multiple blocks with different splits let mut blocks = VecDeque::new(); // First block: 0..2 - let mut block1 = block.slice(0..2); - let col1 = convert_rows(schema.clone(), &sort_desc, block1.clone()).unwrap(); - block1.add_column(BlockEntry::new(col1.data_type(), Value::Column(col1))); - blocks.push_back(SpillableBlock::new(block1, sort_row_offset)); + blocks.push_back(create_spillable_block( + &block, + 0..2, + &schema, + sort_desc, + sort_row_offset, + )); // Second block: 2..5 - let mut block2 = block.slice(2..5); - let col2 = convert_rows(schema.clone(), &sort_desc, block2.clone()).unwrap(); - block2.add_column(BlockEntry::new(col2.data_type(), Value::Column(col2))); - blocks.push_back(SpillableBlock::new(block2, sort_row_offset)); - - // We'll add the third block only if we're not using sliced blocks - // This is to avoid duplicating the data with additional_block - if !with_sliced { - // Third block: 5..8 - let mut block3 = block.slice(5..8); - let col3 = convert_rows(schema.clone(), &sort_desc, block3.clone()).unwrap(); - block3.add_column(BlockEntry::new(col3.data_type(), Value::Column(col3))); - blocks.push_back(SpillableBlock::new(block3, sort_row_offset)); - } + blocks.push_back(create_spillable_block( + &block, + 2..5, + &schema, + sort_desc, + sort_row_offset, + )); // Spill some blocks if requested if with_spilled { // Spill the second block - blocks[1].spill(&spiller).await?; + blocks[1].spill(spiller).await?; } - // Create a sliced block if requested - if with_sliced { + if !with_sliced { + // Third block: 5..8 + blocks.push_back(create_spillable_block( + &block, + 5..8, + &schema, + sort_desc, + sort_row_offset, + )); + } else { // Create a block for values 8..11 (the last part of the sorted data) - let mut additional_block = block.slice(5..8); - let col = convert_rows(schema.clone(), &sort_desc, additional_block.clone()).unwrap(); - additional_block.add_column(BlockEntry::new(col.data_type(), Value::Column(col))); - let mut spillable_block = SpillableBlock::new(additional_block, sort_row_offset); - - // Use SpillableBlock::slice to create a sliced block - // This tests the SpillableBlock::slice functionality by slicing at position 1 - // For ascending Int32: [8, 10, 11] -> [8] and [10, 11] - // For descending String: ["d", "e", "f"] -> ["d"] and ["e", "f"] + let mut spillable_block = + create_spillable_block(&block, 5..8, &schema, sort_desc, sort_row_offset); + + spillable_block.spill(spiller).await?; + spillable_block.data = Some( + spiller + .restore(spillable_block.location.as_ref().unwrap()) + .await?, + ); + let sliced_data = spillable_block.slice(1, sort_row_offset); let sliced_block = SpillableBlock::new(sliced_data, sort_row_offset); @@ -1096,15 +1110,14 @@ mod tests { blocks.push_back(spillable_block); } - let mut stream = BoundBlockStream:: { - blocks, - bound, - sort_row_offset, - spiller: spiller.clone(), - _r: Default::default(), - }; + Ok((schema, blocks, sort_row_offset)) + } - // Take blocks one by one and compare with expected + async fn collect_and_verify_blocks( + stream: &mut BoundBlockStream, + spiller: &impl Spill, + expected_blocks: &[Column], + ) -> Result<()> { let mut result_blocks = Vec::new(); while let Some(mut block) = stream.take_next_bounded_spillable().await? { // If the block data is None (spilled), restore it first @@ -1113,7 +1126,7 @@ mod tests { } let data = block.data.unwrap(); - let col = sort_column(&data, sort_row_offset).clone(); + let col = sort_column(&data, stream.sort_row_offset).clone(); result_blocks.push(col); } @@ -1129,6 +1142,28 @@ mod tests { Ok(()) } + async fn run_take_next_bounded_spillable( + spiller: impl Spill + Clone, + sort_desc: &[SortColumnDescription], + bound: Option, + expected_blocks: Vec, + with_spilled: bool, + with_sliced: bool, + ) -> Result<()> { + let (_, blocks, sort_row_offset) = + prepare_test_blocks::(&spiller, sort_desc, with_spilled, with_sliced).await?; + + let mut stream = BoundBlockStream:: { + blocks, + bound, + sort_row_offset, + spiller: spiller.clone(), + _r: Default::default(), + }; + + collect_and_verify_blocks(&mut stream, &spiller, &expected_blocks).await + } + #[tokio::test] async fn test_take_next_bounded_spillable() -> Result<()> { let spiller = MockSpiller { @@ -1137,150 +1172,150 @@ mod tests { // Test with ascending Int32 type { - let sort_desc = Arc::new(vec![SortColumnDescription { + let sort_desc = [SortColumnDescription { offset: 0, asc: true, nulls_first: false, - }]); + }]; // Test 1: Basic test with bound = 5 (should return blocks with values <= 5) // No spilled blocks, no sliced blocks run_take_next_bounded_spillable::>( spiller.clone(), - sort_desc.clone(), + &sort_desc, Some(Scalar::Number(NumberScalar::Int32(5))), vec![Int32Type::from_data(vec![3, 5])], - false, // no spilled blocks - false, // no sliced blocks + false, + false, ) .await?; // Test 2: With spilled blocks, bound = 8 (should return blocks with values <= 8) run_take_next_bounded_spillable::>( spiller.clone(), - sort_desc.clone(), + &sort_desc, Some(Scalar::Number(NumberScalar::Int32(8))), vec![ Int32Type::from_data(vec![3, 5]), Int32Type::from_data(vec![7, 7, 8]), ], - true, // with spilled blocks - false, // no sliced blocks + true, + false, ) .await?; // Test 3: With sliced blocks, bound = 7 (should return blocks with values <= 7) run_take_next_bounded_spillable::>( spiller.clone(), - sort_desc.clone(), + &sort_desc, Some(Scalar::Number(NumberScalar::Int32(7))), vec![ Int32Type::from_data(vec![3, 5]), Int32Type::from_data(vec![7, 7]), ], - false, // no spilled blocks - true, // with sliced blocks + false, + true, ) .await?; // Test 4: With both spilled and sliced blocks, bound = 10 run_take_next_bounded_spillable::>( spiller.clone(), - sort_desc.clone(), + &sort_desc, Some(Scalar::Number(NumberScalar::Int32(10))), vec![ Int32Type::from_data(vec![3, 5]), Int32Type::from_data(vec![7, 7, 8]), Int32Type::from_data(vec![10]), ], - true, // with spilled blocks - true, // with sliced blocks + true, + true, ) .await?; // Test 5: With bound = 2 (should return no blocks as all values > 2) run_take_next_bounded_spillable::>( spiller.clone(), - sort_desc.clone(), + &sort_desc, Some(Scalar::Number(NumberScalar::Int32(2))), vec![], - true, // with spilled blocks - true, // with sliced blocks + true, + true, ) .await?; // Test 6: With bound = 12 (should return all blocks as all values <= 12) run_take_next_bounded_spillable::>( spiller.clone(), - sort_desc.clone(), + &sort_desc, Some(Scalar::Number(NumberScalar::Int32(12))), vec![ Int32Type::from_data(vec![3, 5]), Int32Type::from_data(vec![7, 7, 8]), Int32Type::from_data(vec![10, 11, 11]), ], - true, // with spilled blocks - false, // no sliced blocks + true, + false, ) .await?; // Test 7: With no bound (should return all blocks) run_take_next_bounded_spillable::>( spiller.clone(), - sort_desc.clone(), + &sort_desc, None, vec![ Int32Type::from_data(vec![3, 5]), Int32Type::from_data(vec![7, 7, 8]), Int32Type::from_data(vec![10, 11, 11]), ], - true, // with spilled blocks - false, // no sliced blocks + true, + false, ) .await?; } // Test with descending String type { - let sort_desc = Arc::new(vec![SortColumnDescription { + let sort_desc = [SortColumnDescription { offset: 1, asc: false, nulls_first: false, - }]); + }]; // Test 8: With bound = "f" (should return blocks with values >= "f") run_take_next_bounded_spillable::>( spiller.clone(), - sort_desc.clone(), + &sort_desc, Some(Scalar::String("f".to_string())), vec![ StringType::from_data(vec!["w", "h"]), StringType::from_data(vec!["g", "f"]), ], - false, // no spilled blocks - false, // no sliced blocks + false, + false, ) .await?; // Test 9: With spilled blocks, bound = "e" (should return blocks with values >= "e") run_take_next_bounded_spillable::>( spiller.clone(), - sort_desc.clone(), + &sort_desc, Some(Scalar::String("e".to_string())), vec![ StringType::from_data(vec!["w", "h"]), StringType::from_data(vec!["g", "f", "e"]), StringType::from_data(vec!["e"]), ], - true, // with spilled blocks - false, // no sliced blocks + true, + false, ) .await?; // Test 10: With sliced blocks, bound = "d" (should return blocks with values >= "d") run_take_next_bounded_spillable::>( spiller.clone(), - sort_desc.clone(), + &sort_desc, Some(Scalar::String("d".to_string())), vec![ StringType::from_data(vec!["w", "h"]), @@ -1288,15 +1323,15 @@ mod tests { StringType::from_data(vec!["e"]), StringType::from_data(vec!["d", "d"]), ], - false, // no spilled blocks - true, // with sliced blocks + false, + true, ) .await?; // Test 11: With both spilled and sliced blocks, bound = "c" (should return all blocks) run_take_next_bounded_spillable::>( spiller.clone(), - sort_desc.clone(), + &sort_desc, Some(Scalar::String("c".to_string())), vec![ StringType::from_data(vec!["w", "h"]), @@ -1304,19 +1339,19 @@ mod tests { StringType::from_data(vec!["e"]), StringType::from_data(vec!["d", "d"]), ], - true, // with spilled blocks - true, // with sliced blocks + true, + true, ) .await?; // Test 12: With bound = "z" (should return no blocks as all values < "z") run_take_next_bounded_spillable::>( spiller.clone(), - sort_desc.clone(), + &sort_desc, Some(Scalar::String("z".to_string())), vec![], - true, // with spilled blocks - true, // with sliced blocks + true, + true, ) .await?; } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/wait.rs b/src/query/service/src/pipelines/processors/transforms/sort/wait.rs index 1e2f04581c1ae..193dcaf4e89a6 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/wait.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/wait.rs @@ -53,21 +53,37 @@ pub struct TransformSortWait { } impl TransformSortWait { - // pub fn new( - // input: Arc, - // output: Arc, - // id: usize, - // state: Arc, - // ) -> Self { - // Self { - // input, - // output, - // id, - // state, - // meta: None, - // _r: Default::default(), - // } - // } + pub fn new( + input: Arc, + output: Arc, + id: usize, + state: Arc, + spiller: Arc, + ) -> Self { + Self { + input, + output, + id, + state, + spiller, + step: Step::None, + _r: PhantomData, + } + } + + pub fn create( + input: Arc, + output: Arc, + id: usize, + inputs: usize, + partitions: usize, + schema: DataSchemaRef, + batch_rows: usize, + spiller: Arc, + ) -> Self { + let state = SortSampleState::new(inputs, partitions, schema, batch_rows); + Self::new(input, output, id, state, spiller) + } async fn scatter(&mut self) -> Result<()> { let scatter_bounds = self.state.bounds(); @@ -195,6 +211,24 @@ pub struct SortSampleState { } impl SortSampleState { + pub fn new( + inputs: usize, + partitions: usize, + schema: DataSchemaRef, + batch_rows: usize, + ) -> Arc { + Arc::new(SortSampleState { + inner: RwLock::new(StateInner { + partitions, + schema, + partial: vec![None; inputs], + bounds: None, + batch_rows, + }), + done: WatchNotify::new(), + }) + } + pub fn commit_sample(&self, id: usize, bounds: Bounds) -> Result { let mut inner = self.inner.write().unwrap(); From 5aecc298b19fa192fa606e929ac19642aa7de4f3 Mon Sep 17 00:00:00 2001 From: coldWater Date: Thu, 24 Apr 2025 13:45:55 +0800 Subject: [PATCH 18/61] route Signed-off-by: coldWater --- .../processors/transforms/sort/exchange.rs | 50 +++------- .../processors/transforms/sort/mod.rs | 1 + .../processors/transforms/sort/route.rs | 96 +++++++++++++++++++ .../processors/transforms/sort/sort_spill.rs | 19 ++-- 4 files changed, 122 insertions(+), 44 deletions(-) create mode 100644 src/query/service/src/pipelines/processors/transforms/sort/route.rs diff --git a/src/query/service/src/pipelines/processors/transforms/sort/exchange.rs b/src/query/service/src/pipelines/processors/transforms/sort/exchange.rs index 9aeeb29cc7414..06b990e28a617 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/exchange.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/exchange.rs @@ -13,17 +13,16 @@ // limitations under the License. use std::marker::PhantomData; -use std::sync::Arc; use databend_common_exception::Result; +use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::DataBlock; use databend_common_pipeline_core::processors::Exchange; use databend_common_pipeline_transforms::processors::sort::Rows; -use super::wait::SortSampleState; +use super::SortScatteredMeta; pub struct SortRangeExchange { - state: Arc, _r: PhantomData, } @@ -33,41 +32,22 @@ unsafe impl Sync for SortRangeExchange {} impl Exchange for SortRangeExchange { const NAME: &'static str = "SortRange"; - fn partition(&self, data: DataBlock, n: usize) -> Result> { - if data.is_empty() { - return Ok(vec![]); - } + fn partition(&self, mut data: DataBlock, n: usize) -> Result> { + let Some(meta) = data.take_meta() else { + unreachable!(); + }; - let bounds = self.state.bounds(); - // debug_assert_eq!(n, self.state.partitions()); - debug_assert!(bounds.len() < n); + let Some(SortScatteredMeta(scattered)) = SortScatteredMeta::downcast_from(meta) else { + unreachable!(); + }; - if bounds.is_empty() { - return Ok(vec![data]); - } + assert!(scattered.len() <= n); - todo!() + let blocks = scattered + .into_iter() + .map(|meta| DataBlock::empty_with_meta(Box::new(meta))) + .collect(); - // let bounds = R::from_column(&bounds.0)?; - // let rows = R::from_column(data.get_last_column())?; - - // let mut i = 0; - // let mut j = 0; - // let mut bound = bounds.row(j); - // let mut indices = Vec::new(); - // while i < rows.len() { - // match rows.row(i).cmp(&bound) { - // Ordering::Less => indices.push(j as u32), - // Ordering::Greater if j + 1 < bounds.len() => { - // j += 1; - // bound = bounds.row(j); - // continue; - // } - // _ => indices.push(j as u32 + 1), - // } - // i += 1; - // } - - // DataBlock::scatter(&data, &indices, n) + Ok(blocks) } } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs index 307c1cefdf4d9..b897d2a262270 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs @@ -36,6 +36,7 @@ mod collect; mod exchange; mod execute; mod merge_sort; +mod route; mod sort_spill; mod wait; diff --git a/src/query/service/src/pipelines/processors/transforms/sort/route.rs b/src/query/service/src/pipelines/processors/transforms/sort/route.rs new file mode 100644 index 0000000000000..12d88fac8abfd --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/sort/route.rs @@ -0,0 +1,96 @@ +// 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_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 struct TransformSortRoute { + inputs: Vec>, + output: Arc, + cur_input: usize, +} + +impl TransformSortRoute { + pub fn new(inputs: Vec>, output: Arc) -> Self { + Self { + inputs, + output, + cur_input: 0, + } + } + + fn process_input(&mut self) -> Result<()> { + for (i, input) in self.inputs.iter().enumerate() { + if i != self.cur_input { + if !input.is_finished() && !input.has_data() { + input.set_need_data(); + } + continue; + } + + if input.is_finished() { + self.cur_input = i + 1; + continue; + } + + match input.pull_data() { + Some(data) => self.output.push_data(data), + None => input.set_need_data(), + } + } + + Ok(()) + } +} + +impl Processor for TransformSortRoute { + fn name(&self) -> String { + "SortRoute".to_string() + } + + 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); + } + + if !self.output.can_push() { + for input in &self.inputs { + input.set_not_need_data(); + } + return Ok(Event::NeedConsume); + } + + self.process_input()?; + + if self.inputs.iter().all(|input| input.is_finished()) { + self.output.finish(); + return Ok(Event::Finished); + } + + Ok(Event::NeedData) + } +} diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs index 5eed43ed3b063..9316ea59af04d 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs @@ -942,13 +942,13 @@ mod tests { async fn run_bound_block_stream( spiller: impl Spill + Clone, - sort_desc: Arc<[SortColumnDescription]>, + sort_desc: &[SortColumnDescription], bound: Scalar, block_part: usize, want: Column, ) -> Result<()> { let (schema, block) = test_data(); - let block = DataBlock::sort(&block, &sort_desc, None)?; + let block = DataBlock::sort(&block, sort_desc, None)?; let bound = Some(bound); let sort_row_offset = schema.fields().len(); @@ -960,6 +960,7 @@ mod tests { .map(|mut data| { let col = convert_rows(schema.clone(), &sort_desc, data.clone()).unwrap(); data.add_column(col); + SpillableBlock::new(data, sort_row_offset) }) .collect::>(); @@ -986,15 +987,15 @@ mod tests { }; { - let sort_desc = Arc::new([SortColumnDescription { + let sort_desc = [SortColumnDescription { offset: 0, asc: true, nulls_first: false, - }]); + }]; run_bound_block_stream::>( spiller.clone(), - sort_desc.clone(), + &sort_desc, Scalar::Number(NumberScalar::Int32(5)), 4, Int32Type::from_data(vec![3, 5]), @@ -1003,7 +1004,7 @@ mod tests { run_bound_block_stream::>( spiller.clone(), - sort_desc.clone(), + &sort_desc, Scalar::Number(NumberScalar::Int32(8)), 4, Int32Type::from_data(vec![3, 5, 7, 7]), @@ -1012,15 +1013,15 @@ mod tests { } { - let sort_desc = Arc::new([SortColumnDescription { + let sort_desc = [SortColumnDescription { offset: 1, asc: false, nulls_first: false, - }]); + }]; run_bound_block_stream::>( spiller.clone(), - sort_desc.clone(), + &sort_desc, Scalar::String("f".to_string()), 4, StringType::from_data(vec!["w", "h", "g", "f"]), From 63f743d304b8e82b45324eb0727b9b50e27edc48 Mon Sep 17 00:00:00 2001 From: coldWater Date: Thu, 24 Apr 2025 17:26:07 +0800 Subject: [PATCH 19/61] builder Signed-off-by: coldWater --- .../src/pipelines/builders/builder_sort.rs | 4 +- .../processors/transforms/sort/builder.rs | 244 ++++++++++++------ .../processors/transforms/sort/exchange.rs | 13 +- .../processors/transforms/sort/mod.rs | 2 +- .../transforms/sort/{wait.rs => shuffle.rs} | 22 +- 5 files changed, 170 insertions(+), 115 deletions(-) rename src/query/service/src/pipelines/processors/transforms/sort/{wait.rs => shuffle.rs} (92%) diff --git a/src/query/service/src/pipelines/builders/builder_sort.rs b/src/query/service/src/pipelines/builders/builder_sort.rs index f678a4e618782..be0913a779609 100644 --- a/src/query/service/src/pipelines/builders/builder_sort.rs +++ b/src/query/service/src/pipelines/builders/builder_sort.rs @@ -300,8 +300,6 @@ impl SortPipelineBuilder { pipeline.add_transform(|input, output| { let builder = TransformSortBuilder::create( - input, - output, sort_merge_output_schema.clone(), self.sort_desc.clone(), self.block_size, @@ -313,7 +311,7 @@ impl SortPipelineBuilder { .with_memory_settings(memory_settings.clone()) .with_enable_loser_tree(enable_loser_tree); - Ok(ProcessorPtr::create(builder.build()?)) + Ok(ProcessorPtr::create(builder.build(input, output)?)) }) } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/builder.rs b/src/query/service/src/pipelines/processors/transforms/sort/builder.rs index b16686b4829fd..80343cf5acca4 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/builder.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/builder.rs @@ -34,17 +34,18 @@ use databend_common_pipeline_transforms::MemorySettings; use super::collect::TransformSortCollect; use super::execute::TransformSortExecute; use super::merge_sort::TransformSort; +use super::shuffle::SortSampleState; +use super::shuffle::TransformSortShuffle; use crate::spillers::Spiller; enum SortType { Sort, Collect, Execute, + Shuffle, } pub struct TransformSortBuilder { - input: Arc, - output: Arc, schema: DataSchemaRef, block_size: usize, sort_desc: Arc<[SortColumnDescription]>, @@ -54,22 +55,16 @@ pub struct TransformSortBuilder { spiller: Arc, enable_loser_tree: bool, limit: Option, - processor: Option>>, - typ: SortType, } impl TransformSortBuilder { pub fn create( - input: Arc, - output: Arc, schema: DataSchemaRef, sort_desc: Arc<[SortColumnDescription]>, block_size: usize, spiller: Arc, ) -> Self { - Self { - input, - output, + TransformSortBuilder { block_size, schema, sort_desc, @@ -79,8 +74,6 @@ impl TransformSortBuilder { enable_loser_tree: false, limit: None, memory_settings: MemorySettings::disable_spill(), - processor: None, - typ: SortType::Sort, } } @@ -109,34 +102,133 @@ impl TransformSortBuilder { self } - pub fn build(mut self) -> Result> { - debug_assert!(if self.output_order_col { + pub fn build( + &self, + input: Arc, + output: Arc, + ) -> Result> { + self.check(); + + let mut build = Build { + params: self, + input, + output, + processor: None, + typ: SortType::Sort, + id: 0, + state: None, + }; + + select_row_type(&mut build); + build.processor.unwrap() + } + + pub fn build_collect( + &self, + input: Arc, + output: Arc, + ) -> Result> { + self.check(); + + let mut build = Build { + params: self, + input, + output, + processor: None, + typ: SortType::Collect, + id: 0, + state: None, + }; + + select_row_type(&mut build); + build.processor.unwrap() + } + + pub fn build_exec( + &self, + input: Arc, + output: Arc, + ) -> Result> { + self.check(); + + let mut build = Build { + params: self, + input, + output, + processor: None, + typ: SortType::Execute, + id: 0, + state: None, + }; + + select_row_type(&mut build); + build.processor.unwrap() + } + + pub fn build_shuffle( + &self, + input: Arc, + output: Arc, + id: usize, + state: Arc, + ) -> Result> { + self.check(); + + let mut build = Build { + params: self, + input, + output, + processor: None, + typ: SortType::Shuffle, + id, + state: Some(state), + }; + + select_row_type(&mut build); + build.processor.unwrap() + } + + fn should_use_sort_limit(&self) -> bool { + self.limit.map(|limit| limit < 10000).unwrap_or_default() + } + + fn check(&self) { + assert!(if self.output_order_col { self.schema.has_field(ORDER_COL_NAME) } else { !self.schema.has_field(ORDER_COL_NAME) }); - - select_row_type(&mut self); - self.processor.unwrap() } +} +pub struct Build<'a> { + params: &'a TransformSortBuilder, + typ: SortType, + input: Arc, + output: Arc, + processor: Option>>, + id: usize, + state: Option>, +} + +impl Build<'_> { fn build_sort(&mut self) -> Result> where A: SortAlgorithm + 'static, C: RowConverter + Send + 'static, { - let schema = add_order_field(self.schema.clone(), &self.sort_desc); + let schema = add_order_field(self.params.schema.clone(), &self.params.sort_desc); Ok(Box::new(TransformSort::::new( self.input.clone(), self.output.clone(), schema, - self.sort_desc.clone(), - self.block_size, - self.limit.map(|limit| (limit, false)), - self.spiller.clone(), - self.output_order_col, - self.order_col_generated, - self.memory_settings.clone(), + self.params.sort_desc.clone(), + self.params.block_size, + self.params.limit.map(|limit| (limit, false)), + self.params.spiller.clone(), + self.params.output_order_col, + self.params.order_col_generated, + self.params.memory_settings.clone(), )?)) } @@ -145,50 +237,38 @@ impl TransformSortBuilder { A: SortAlgorithm + 'static, C: RowConverter + Send + 'static, { - let schema = add_order_field(self.schema.clone(), &self.sort_desc); + let schema = add_order_field(self.params.schema.clone(), &self.params.sort_desc); Ok(Box::new(TransformSort::::new( self.input.clone(), self.output.clone(), schema, - self.sort_desc.clone(), - self.block_size, - Some((self.limit.unwrap(), true)), - self.spiller.clone(), - self.output_order_col, - self.order_col_generated, - self.memory_settings.clone(), + self.params.sort_desc.clone(), + self.params.block_size, + Some((self.params.limit.unwrap(), true)), + self.params.spiller.clone(), + self.params.output_order_col, + self.params.order_col_generated, + self.params.memory_settings.clone(), )?)) } - pub fn build_collect(mut self) -> Result> { - debug_assert!(if self.output_order_col { - self.schema.has_field(ORDER_COL_NAME) - } else { - !self.schema.has_field(ORDER_COL_NAME) - }); - self.typ = SortType::Collect; - - select_row_type(&mut self); - self.processor.unwrap() - } - fn build_sort_collect(&mut self) -> Result> where A: SortAlgorithm + 'static, C: RowConverter + Send + 'static, { - let schema = add_order_field(self.schema.clone(), &self.sort_desc); + let schema = add_order_field(self.params.schema.clone(), &self.params.sort_desc); Ok(Box::new(TransformSortCollect::::new( self.input.clone(), self.output.clone(), schema, - self.sort_desc.clone(), - self.block_size, - self.limit.map(|limit| (limit, false)), - self.spiller.clone(), - self.order_col_generated, - self.memory_settings.clone(), + self.params.sort_desc.clone(), + self.params.block_size, + self.params.limit.map(|limit| (limit, false)), + self.params.spiller.clone(), + self.params.order_col_generated, + self.params.memory_settings.clone(), )?)) } @@ -197,54 +277,53 @@ impl TransformSortBuilder { A: SortAlgorithm + 'static, C: RowConverter + Send + 'static, { - let schema = add_order_field(self.schema.clone(), &self.sort_desc); + let schema = add_order_field(self.params.schema.clone(), &self.params.sort_desc); Ok(Box::new(TransformSortCollect::::new( self.input.clone(), self.output.clone(), schema, - self.sort_desc.clone(), - self.block_size, - Some((self.limit.unwrap(), true)), - self.spiller.clone(), - self.order_col_generated, - self.memory_settings.clone(), + self.params.sort_desc.clone(), + self.params.block_size, + Some((self.params.limit.unwrap(), true)), + self.params.spiller.clone(), + self.params.order_col_generated, + self.params.memory_settings.clone(), )?)) } - pub fn build_exec(mut self) -> Result> { - debug_assert!(if self.output_order_col { - self.schema.has_field(ORDER_COL_NAME) - } else { - !self.schema.has_field(ORDER_COL_NAME) - }); - self.typ = SortType::Execute; - - select_row_type(&mut self); - self.processor.unwrap() - } - fn build_sort_exec(&mut self) -> Result> where A: SortAlgorithm + 'static { - let schema = add_order_field(self.schema.clone(), &self.sort_desc); + let schema = add_order_field(self.params.schema.clone(), &self.params.sort_desc); Ok(Box::new(TransformSortExecute::::new( self.input.clone(), self.output.clone(), schema, - self.limit, - self.spiller.clone(), - self.output_order_col, + self.params.limit, + self.params.spiller.clone(), + self.params.output_order_col, )?)) } + + fn build_sort_shuffle(&mut self) -> Result> + where R: Rows + 'static { + Ok(Box::new(TransformSortShuffle::::new( + self.input.clone(), + self.output.clone(), + self.id, + self.state.clone().unwrap(), + self.params.spiller.clone(), + ))) + } } -impl RowsTypeVisitor for TransformSortBuilder { +impl RowsTypeVisitor for Build<'_> { fn schema(&self) -> DataSchemaRef { - self.schema.clone() + self.params.schema.clone() } fn sort_desc(&self) -> &[SortColumnDescription] { - &self.sort_desc + &self.params.sort_desc } fn visit_type(&mut self) @@ -254,8 +333,8 @@ impl RowsTypeVisitor for TransformSortBuilder { { let processor = match self.typ { SortType::Sort => match ( - self.limit.map(|limit| limit < 10000).unwrap_or_default(), - self.enable_loser_tree, + self.params.should_use_sort_limit(), + self.params.enable_loser_tree, ) { (true, true) => self.build_sort_limit::, C>(), (true, false) => self.build_sort_limit::, C>(), @@ -263,18 +342,19 @@ impl RowsTypeVisitor for TransformSortBuilder { (false, false) => self.build_sort::, C>(), }, SortType::Collect => match ( - self.limit.map(|limit| limit < 10000).unwrap_or_default(), - self.enable_loser_tree, + self.params.should_use_sort_limit(), + self.params.enable_loser_tree, ) { (true, true) => self.build_sort_limit_collect::, C>(), (true, false) => self.build_sort_limit_collect::, C>(), (false, true) => self.build_sort_collect::, C>(), (false, false) => self.build_sort_collect::, C>(), }, - SortType::Execute => match self.enable_loser_tree { + SortType::Execute => match self.params.enable_loser_tree { true => self.build_sort_exec::>(), false => self.build_sort_exec::>(), }, + SortType::Shuffle => self.build_sort_shuffle::(), }; self.processor = Some(processor) } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/exchange.rs b/src/query/service/src/pipelines/processors/transforms/sort/exchange.rs index 06b990e28a617..53795063c4f61 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/exchange.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/exchange.rs @@ -12,25 +12,16 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::marker::PhantomData; - use databend_common_exception::Result; use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::DataBlock; use databend_common_pipeline_core::processors::Exchange; -use databend_common_pipeline_transforms::processors::sort::Rows; use super::SortScatteredMeta; -pub struct SortRangeExchange { - _r: PhantomData, -} - -unsafe impl Send for SortRangeExchange {} - -unsafe impl Sync for SortRangeExchange {} +pub struct SortRangeExchange; -impl Exchange for SortRangeExchange { +impl Exchange for SortRangeExchange { const NAME: &'static str = "SortRange"; fn partition(&self, mut data: DataBlock, n: usize) -> Result> { let Some(meta) = data.take_meta() else { diff --git a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs index b897d2a262270..120ad5a556b97 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs @@ -37,8 +37,8 @@ mod exchange; mod execute; mod merge_sort; mod route; +mod shuffle; mod sort_spill; -mod wait; use sort_spill::SpillableBlock; diff --git a/src/query/service/src/pipelines/processors/transforms/sort/wait.rs b/src/query/service/src/pipelines/processors/transforms/sort/shuffle.rs similarity index 92% rename from src/query/service/src/pipelines/processors/transforms/sort/wait.rs rename to src/query/service/src/pipelines/processors/transforms/sort/shuffle.rs index 193dcaf4e89a6..9c9a3b7067019 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/wait.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/shuffle.rs @@ -42,7 +42,7 @@ enum Step { Scattered(Vec), } -pub struct TransformSortWait { +pub struct TransformSortShuffle { input: Arc, output: Arc, id: usize, @@ -52,7 +52,7 @@ pub struct TransformSortWait { _r: PhantomData, } -impl TransformSortWait { +impl TransformSortShuffle { pub fn new( input: Arc, output: Arc, @@ -71,20 +71,6 @@ impl TransformSortWait { } } - pub fn create( - input: Arc, - output: Arc, - id: usize, - inputs: usize, - partitions: usize, - schema: DataSchemaRef, - batch_rows: usize, - spiller: Arc, - ) -> Self { - let state = SortSampleState::new(inputs, partitions, schema, batch_rows); - Self::new(input, output, id, state, spiller) - } - async fn scatter(&mut self) -> Result<()> { let scatter_bounds = self.state.bounds(); @@ -138,9 +124,9 @@ impl TransformSortWait { } #[async_trait::async_trait] -impl Processor for TransformSortWait { +impl Processor for TransformSortShuffle { fn name(&self) -> String { - "TransformSortWait".to_string() + "TransformSortShuffle".to_string() } fn as_any(&mut self) -> &mut dyn Any { From 1e0bf24dd605bc707eaf126a3b4dd70a1286922b Mon Sep 17 00:00:00 2001 From: coldWater Date: Thu, 24 Apr 2025 18:34:45 +0800 Subject: [PATCH 20/61] update Signed-off-by: coldWater --- .../processors/transforms/sort/rows/utils.rs | 125 ++++++++++-------- .../processors/transforms/sort/builder.rs | 93 ++++--------- .../processors/transforms/sort/collect.rs | 29 +--- .../processors/transforms/sort/execute.rs | 14 +- .../processors/transforms/sort/sort_spill.rs | 4 +- 5 files changed, 105 insertions(+), 160 deletions(-) diff --git a/src/query/pipeline/transforms/src/processors/transforms/sort/rows/utils.rs b/src/query/pipeline/transforms/src/processors/transforms/sort/rows/utils.rs index da1a63bf2deeb..49d0dc1c25ed8 100644 --- a/src/query/pipeline/transforms/src/processors/transforms/sort/rows/utils.rs +++ b/src/query/pipeline/transforms/src/processors/transforms/sort/rows/utils.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::row::RowConverter as CommonConverter; use databend_common_expression::types::DataType; @@ -21,7 +22,6 @@ use databend_common_expression::types::NumberType; use databend_common_expression::types::StringType; use databend_common_expression::types::TimestampType; use databend_common_expression::with_number_mapped_type; -use databend_common_expression::BlockEntry; use databend_common_expression::Column; use databend_common_expression::DataBlock; use databend_common_expression::DataSchema; @@ -41,55 +41,50 @@ pub fn convert_rows( sort_desc: &[SortColumnDescription], data: DataBlock, ) -> Result { - let num_rows = data.num_rows(); + struct ConvertRowsVisitor<'a> { + schema: DataSchemaRef, + sort_desc: &'a [SortColumnDescription], + data: DataBlock, + result: Result, + } - if sort_desc.len() == 1 { - let sort_type = schema.field(sort_desc[0].offset).data_type(); - let asc = sort_desc[0].asc; + impl RowsTypeVisitor for ConvertRowsVisitor<'_> { + fn schema(&self) -> DataSchemaRef { + self.schema.clone() + } - let offset = sort_desc[0].offset; - let columns = &data.columns()[offset..offset + 1]; + fn sort_desc(&self) -> &[SortColumnDescription] { + self.sort_desc + } - match_template! { - T = [ Date => DateType, Timestamp => TimestampType, String => StringType ], - match sort_type { - DataType::T => { - if asc { - convert_columns::,SimpleRowConverter<_>>(schema, sort_desc, columns, num_rows) - } else { - convert_columns::,SimpleRowConverter<_>>(schema, sort_desc, columns, num_rows) - } - }, - DataType::Number(num_ty) => with_number_mapped_type!(|NUM_TYPE| match num_ty { - NumberDataType::NUM_TYPE => { - if asc { - convert_columns::>,SimpleRowConverter<_>>(schema, sort_desc, columns, num_rows) - } else { - convert_columns::>,SimpleRowConverter<_>>(schema, sort_desc, columns, num_rows) - } - } - }), - _ => convert_columns::(schema, sort_desc, columns, num_rows), + fn visit_type(&mut self) + where + R: Rows + 'static, + C: RowConverter + Send + 'static, + { + let columns = self + .sort_desc + .iter() + .map(|desc| self.data.get_by_offset(desc.offset).to_owned()) + .collect::>(); + + self.result = try { + let converter = C::create(self.sort_desc, self.schema.clone())?; + let rows = C::convert(&converter, &columns, self.data.num_rows())?; + rows.to_column() } } - } else { - let columns = sort_desc - .iter() - .map(|desc| data.get_by_offset(desc.offset).to_owned()) - .collect::>(); - convert_columns::(schema, sort_desc, &columns, num_rows) } -} -fn convert_columns>( - schema: DataSchemaRef, - sort_desc: &[SortColumnDescription], - columns: &[BlockEntry], - num_rows: usize, -) -> Result { - let converter = C::create(sort_desc, schema)?; - let rows = C::convert(&converter, columns, num_rows)?; - Ok(rows.to_column()) + let mut visitor = ConvertRowsVisitor { + schema: schema.clone(), + sort_desc, + data, + result: Err(ErrorCode::Internal("unreachable")), + }; + + select_row_type(&mut visitor); + visitor.result } pub fn select_row_type(visitor: &mut impl RowsTypeVisitor) { @@ -138,19 +133,37 @@ pub trait RowsTypeVisitor { } pub fn order_field_type(schema: &DataSchema, desc: &[SortColumnDescription]) -> DataType { - debug_assert!(!desc.is_empty()); - if desc.len() == 1 { - let order_by_field = schema.field(desc[0].offset); - if matches!( - order_by_field.data_type(), - DataType::Number(_) - | DataType::Date - | DataType::Timestamp - | DataType::Binary - | DataType::String - ) { - return order_by_field.data_type().clone(); + struct OrderFieldTypeVisitor<'a> { + schema: DataSchemaRef, + sort_desc: &'a [SortColumnDescription], + result: Option, + } + + impl RowsTypeVisitor for OrderFieldTypeVisitor<'_> { + fn schema(&self) -> DataSchemaRef { + self.schema.clone() + } + + fn sort_desc(&self) -> &[SortColumnDescription] { + self.sort_desc + } + + fn visit_type(&mut self) + where + R: Rows + 'static, + C: RowConverter + Send + 'static, + { + self.result = Some(R::data_type()); } } - DataType::Binary + + assert!(!desc.is_empty()); + let mut visitor = OrderFieldTypeVisitor { + schema: schema.clone().into(), + sort_desc: desc, + result: None, + }; + + select_row_type(&mut visitor); + visitor.result.unwrap() } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/builder.rs b/src/query/service/src/pipelines/processors/transforms/sort/builder.rs index 80343cf5acca4..00bd45de5f434 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/builder.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/builder.rs @@ -36,6 +36,7 @@ use super::execute::TransformSortExecute; use super::merge_sort::TransformSort; use super::shuffle::SortSampleState; use super::shuffle::TransformSortShuffle; +use super::Base; use crate::spillers::Spiller; enum SortType { @@ -199,6 +200,17 @@ impl TransformSortBuilder { !self.schema.has_field(ORDER_COL_NAME) }); } + + fn new_base(&self) -> Base { + let schema = add_order_field(self.schema.clone(), &self.sort_desc); + let sort_row_offset = schema.fields().len() - 1; + Base { + sort_row_offset, + schema, + spiller: self.spiller.clone(), + limit: self.limit, + } + } } pub struct Build<'a> { @@ -212,27 +224,7 @@ pub struct Build<'a> { } impl Build<'_> { - fn build_sort(&mut self) -> Result> - where - A: SortAlgorithm + 'static, - C: RowConverter + Send + 'static, - { - let schema = add_order_field(self.params.schema.clone(), &self.params.sort_desc); - Ok(Box::new(TransformSort::::new( - self.input.clone(), - self.output.clone(), - schema, - self.params.sort_desc.clone(), - self.params.block_size, - self.params.limit.map(|limit| (limit, false)), - self.params.spiller.clone(), - self.params.output_order_col, - self.params.order_col_generated, - self.params.memory_settings.clone(), - )?)) - } - - fn build_sort_limit(&mut self) -> Result> + fn build_sort(&mut self, limit_sort: bool) -> Result> where A: SortAlgorithm + 'static, C: RowConverter + Send + 'static, @@ -244,7 +236,7 @@ impl Build<'_> { schema, self.params.sort_desc.clone(), self.params.block_size, - Some((self.params.limit.unwrap(), true)), + self.params.limit.map(|limit| (limit, limit_sort)), self.params.spiller.clone(), self.params.output_order_col, self.params.order_col_generated, @@ -252,40 +244,18 @@ impl Build<'_> { )?)) } - fn build_sort_collect(&mut self) -> Result> + fn build_sort_collect(&mut self, limit_sort: bool) -> Result> where A: SortAlgorithm + 'static, C: RowConverter + Send + 'static, { - let schema = add_order_field(self.params.schema.clone(), &self.params.sort_desc); - Ok(Box::new(TransformSortCollect::::new( self.input.clone(), self.output.clone(), - schema, + self.params.new_base(), self.params.sort_desc.clone(), self.params.block_size, - self.params.limit.map(|limit| (limit, false)), - self.params.spiller.clone(), - self.params.order_col_generated, - self.params.memory_settings.clone(), - )?)) - } - - fn build_sort_limit_collect(&mut self) -> Result> - where - A: SortAlgorithm + 'static, - C: RowConverter + Send + 'static, - { - let schema = add_order_field(self.params.schema.clone(), &self.params.sort_desc); - Ok(Box::new(TransformSortCollect::::new( - self.input.clone(), - self.output.clone(), - schema, - self.params.sort_desc.clone(), - self.params.block_size, - Some((self.params.limit.unwrap(), true)), - self.params.spiller.clone(), + limit_sort, self.params.order_col_generated, self.params.memory_settings.clone(), )?)) @@ -293,14 +263,10 @@ impl Build<'_> { fn build_sort_exec(&mut self) -> Result> where A: SortAlgorithm + 'static { - let schema = add_order_field(self.params.schema.clone(), &self.params.sort_desc); - Ok(Box::new(TransformSortExecute::::new( self.input.clone(), self.output.clone(), - schema, - self.params.limit, - self.params.spiller.clone(), + self.params.new_base(), self.params.output_order_col, )?)) } @@ -331,24 +297,15 @@ impl RowsTypeVisitor for Build<'_> { R: Rows + 'static, C: RowConverter + Send + 'static, { + let limit_sort = self.params.should_use_sort_limit(); let processor = match self.typ { - SortType::Sort => match ( - self.params.should_use_sort_limit(), - self.params.enable_loser_tree, - ) { - (true, true) => self.build_sort_limit::, C>(), - (true, false) => self.build_sort_limit::, C>(), - (false, true) => self.build_sort::, C>(), - (false, false) => self.build_sort::, C>(), + SortType::Sort => match self.params.enable_loser_tree { + true => self.build_sort::, C>(limit_sort), + false => self.build_sort::, C>(limit_sort), }, - SortType::Collect => match ( - self.params.should_use_sort_limit(), - self.params.enable_loser_tree, - ) { - (true, true) => self.build_sort_limit_collect::, C>(), - (true, false) => self.build_sort_limit_collect::, C>(), - (false, true) => self.build_sort_collect::, C>(), - (false, false) => self.build_sort_collect::, C>(), + SortType::Collect => match self.params.enable_loser_tree { + true => self.build_sort_collect::, C>(limit_sort), + false => self.build_sort_collect::, C>(limit_sort), }, SortType::Execute => match self.params.enable_loser_tree { true => self.build_sort_exec::>(), diff --git a/src/query/service/src/pipelines/processors/transforms/sort/collect.rs b/src/query/service/src/pipelines/processors/transforms/sort/collect.rs index d6640513f9844..b8546d32aebbd 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/collect.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/collect.rs @@ -19,7 +19,6 @@ use std::sync::Arc; use databend_common_exception::Result; use databend_common_expression::DataBlock; -use databend_common_expression::DataSchemaRef; use databend_common_expression::SortColumnDescription; use databend_common_pipeline_core::processors::Event; use databend_common_pipeline_core::processors::InputPort; @@ -36,7 +35,6 @@ use databend_common_pipeline_transforms::TransformSortMergeLimit; use super::sort_spill::SortSpill; use super::Base; use super::MemoryRows; -use crate::spillers::Spiller; enum Inner { Collect(Vec), @@ -75,28 +73,20 @@ where pub(super) fn new( input: Arc, output: Arc, - schema: DataSchemaRef, + base: Base, sort_desc: Arc<[SortColumnDescription]>, max_block_size: usize, - limit: Option<(usize, bool)>, - spiller: Arc, + sort_limit: bool, order_col_generated: bool, memory_settings: MemorySettings, ) -> Result { - let sort_row_offset = schema.fields().len() - 1; - let row_converter = C::create(&sort_desc, schema.clone())?; - let (name, inner, limit) = match limit { - Some((limit, true)) => ( + let row_converter = C::create(&sort_desc, base.schema.clone())?; + let (name, inner) = match base.limit { + Some(limit) if sort_limit => ( "TransformSortMergeCollectLimit", Inner::Limit(TransformSortMergeLimit::create(max_block_size, limit)), - Some(limit), ), - Some((limit, false)) => ( - "TransformSortMergeCollect", - Inner::Collect(vec![]), - Some(limit), - ), - None => ("TransformSortMergeCollect", Inner::Collect(vec![]), None), + _ => ("TransformSortMergeCollect", Inner::Collect(vec![])), }; Ok(Self { input, @@ -106,12 +96,7 @@ where output_data: None, sort_desc, order_col_generated, - base: Base { - schema, - spiller, - sort_row_offset, - limit, - }, + base, inner, aborting: AtomicBool::new(false), memory_settings, diff --git a/src/query/service/src/pipelines/processors/transforms/sort/execute.rs b/src/query/service/src/pipelines/processors/transforms/sort/execute.rs index 39f01dc0497e1..3c5d832aa5973 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/execute.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/execute.rs @@ -18,7 +18,6 @@ use std::sync::Arc; use databend_common_exception::Result; use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::DataBlock; -use databend_common_expression::DataSchemaRef; use databend_common_pipeline_core::processors::Event; use databend_common_pipeline_core::processors::InputPort; use databend_common_pipeline_core::processors::OutputPort; @@ -28,7 +27,6 @@ use databend_common_pipeline_transforms::processors::sort::algorithm::SortAlgori use super::sort_spill::SortSpill; use super::Base; use super::SortCollectedMeta; -use crate::spillers::Spiller; pub struct TransformSortExecute { input: Arc, @@ -48,22 +46,14 @@ where A: SortAlgorithm pub(super) fn new( input: Arc, output: Arc, - schema: DataSchemaRef, - limit: Option, - spiller: Arc, + base: Base, output_order_col: bool, ) -> Result { - let sort_row_offset = schema.fields().len() - 1; Ok(Self { input, output, remove_order_col: !output_order_col, - base: Base { - schema, - spiller, - sort_row_offset, - limit, - }, + base, inner: None, }) } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs index 9316ea59af04d..f14c124bf1364 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs @@ -1045,7 +1045,7 @@ mod tests { SpillableBlock::new(sliced_block, sort_row_offset) } - async fn prepare_test_blocks( + async fn prepare_test_blocks( spiller: &impl Spill, sort_desc: &[SortColumnDescription], with_spilled: bool, @@ -1152,7 +1152,7 @@ mod tests { with_sliced: bool, ) -> Result<()> { let (_, blocks, sort_row_offset) = - prepare_test_blocks::(&spiller, sort_desc, with_spilled, with_sliced).await?; + prepare_test_blocks(&spiller, sort_desc, with_spilled, with_sliced).await?; let mut stream = BoundBlockStream:: { blocks, From 8f1ba99820b0c2a9cdcdd067bfb49eaf4ec2cdec Mon Sep 17 00:00:00 2001 From: coldWater Date: Thu, 24 Apr 2025 21:20:25 +0800 Subject: [PATCH 21/61] build Signed-off-by: coldWater --- .../processors/transforms/sort/rows/utils.rs | 81 ++--- .../src/pipelines/builders/builder_sort.rs | 70 ++-- .../processors/transforms/sort/builder.rs | 43 ++- .../processors/transforms/sort/exchange.rs | 45 ++- .../processors/transforms/sort/mod.rs | 19 +- .../processors/transforms/sort/route.rs | 32 +- .../transforms/sort/sort_exchange.rs | 148 -------- .../processors/transforms/sort/sort_merge.rs | 62 ---- .../processors/transforms/sort/sort_sample.rs | 333 ------------------ .../processors/transforms/sort/sort_wait.rs | 113 ------ 10 files changed, 187 insertions(+), 759 deletions(-) delete mode 100644 src/query/service/src/pipelines/processors/transforms/sort/sort_exchange.rs delete mode 100644 src/query/service/src/pipelines/processors/transforms/sort/sort_merge.rs delete mode 100644 src/query/service/src/pipelines/processors/transforms/sort/sort_sample.rs delete mode 100644 src/query/service/src/pipelines/processors/transforms/sort/sort_wait.rs diff --git a/src/query/pipeline/transforms/src/processors/transforms/sort/rows/utils.rs b/src/query/pipeline/transforms/src/processors/transforms/sort/rows/utils.rs index 49d0dc1c25ed8..373e2e2e99281 100644 --- a/src/query/pipeline/transforms/src/processors/transforms/sort/rows/utils.rs +++ b/src/query/pipeline/transforms/src/processors/transforms/sort/rows/utils.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::row::RowConverter as CommonConverter; use databend_common_expression::types::DataType; @@ -45,10 +44,10 @@ pub fn convert_rows( schema: DataSchemaRef, sort_desc: &'a [SortColumnDescription], data: DataBlock, - result: Result, } impl RowsTypeVisitor for ConvertRowsVisitor<'_> { + type Result = Result; fn schema(&self) -> DataSchemaRef { self.schema.clone() } @@ -57,7 +56,7 @@ pub fn convert_rows( self.sort_desc } - fn visit_type(&mut self) + fn visit_type(&mut self) -> Self::Result where R: Rows + 'static, C: RowConverter + Send + 'static, @@ -68,11 +67,9 @@ pub fn convert_rows( .map(|desc| self.data.get_by_offset(desc.offset).to_owned()) .collect::>(); - self.result = try { - let converter = C::create(self.sort_desc, self.schema.clone())?; - let rows = C::convert(&converter, &columns, self.data.num_rows())?; - rows.to_column() - } + let converter = C::create(self.sort_desc, self.schema.clone())?; + let rows = C::convert(&converter, &columns, self.data.num_rows())?; + Ok(rows.to_column()) } } @@ -80,53 +77,53 @@ pub fn convert_rows( schema: schema.clone(), sort_desc, data, - result: Err(ErrorCode::Internal("unreachable")), }; - select_row_type(&mut visitor); - visitor.result + select_row_type(&mut visitor) } -pub fn select_row_type(visitor: &mut impl RowsTypeVisitor) { - let sort_desc = visitor.sort_desc(); - if sort_desc.len() == 1 { - let schema = visitor.schema(); - let sort_type = schema.field(sort_desc[0].offset).data_type(); - let asc = sort_desc[0].asc; - - match_template! { - T = [ Date => DateType, Timestamp => TimestampType, String => StringType ], - match sort_type { - DataType::T => { - if asc { - visitor.visit_type::, SimpleRowConverter>() - } else { - visitor.visit_type::, SimpleRowConverter>() - } - }, - DataType::Number(num_ty) => with_number_mapped_type!(|NUM_TYPE| match num_ty { - NumberDataType::NUM_TYPE => { +pub fn select_row_type(visitor: &mut V) -> V::Result +where V: RowsTypeVisitor { + match &visitor.sort_desc() { + &[desc] => { + let schema = visitor.schema(); + let sort_type = schema.field(desc.offset).data_type(); + let asc = desc.asc; + + match_template! { + T = [ Date => DateType, Timestamp => TimestampType, String => StringType ], + match sort_type { + DataType::T => { if asc { - visitor.visit_type::>, SimpleRowConverter>>() + visitor.visit_type::, SimpleRowConverter>() } else { - visitor.visit_type::>, SimpleRowConverter>>() + visitor.visit_type::, SimpleRowConverter>() + } + }, + DataType::Number(num_ty) => with_number_mapped_type!(|NUM_TYPE| match num_ty { + NumberDataType::NUM_TYPE => { + if asc { + visitor.visit_type::>, SimpleRowConverter>>() + } else { + visitor.visit_type::>, SimpleRowConverter>>() + } } + }), + _ => visitor.visit_type::() } - }), - _ => visitor.visit_type::() } } - } else { - visitor.visit_type::() + _ => visitor.visit_type::(), } } pub trait RowsTypeVisitor { + type Result; fn schema(&self) -> DataSchemaRef; fn sort_desc(&self) -> &[SortColumnDescription]; - fn visit_type(&mut self) + fn visit_type(&mut self) -> Self::Result where R: Rows + 'static, C: RowConverter + Send + 'static; @@ -136,10 +133,10 @@ pub fn order_field_type(schema: &DataSchema, desc: &[SortColumnDescription]) -> struct OrderFieldTypeVisitor<'a> { schema: DataSchemaRef, sort_desc: &'a [SortColumnDescription], - result: Option, } impl RowsTypeVisitor for OrderFieldTypeVisitor<'_> { + type Result = DataType; fn schema(&self) -> DataSchemaRef { self.schema.clone() } @@ -148,12 +145,12 @@ pub fn order_field_type(schema: &DataSchema, desc: &[SortColumnDescription]) -> self.sort_desc } - fn visit_type(&mut self) + fn visit_type(&mut self) -> Self::Result where R: Rows + 'static, C: RowConverter + Send + 'static, { - self.result = Some(R::data_type()); + R::data_type() } } @@ -161,9 +158,7 @@ pub fn order_field_type(schema: &DataSchema, desc: &[SortColumnDescription]) -> let mut visitor = OrderFieldTypeVisitor { schema: schema.clone().into(), sort_desc: desc, - result: None, }; - select_row_type(&mut visitor); - visitor.result.unwrap() + select_row_type(&mut visitor) } diff --git a/src/query/service/src/pipelines/builders/builder_sort.rs b/src/query/service/src/pipelines/builders/builder_sort.rs index be0913a779609..563128b0e169f 100644 --- a/src/query/service/src/pipelines/builders/builder_sort.rs +++ b/src/query/service/src/pipelines/builders/builder_sort.rs @@ -34,10 +34,9 @@ use databend_common_storages_fuse::TableContext; use databend_storages_common_cache::TempDirManager; use crate::pipelines::memory_settings::MemorySettingsExt; -use crate::pipelines::processors::transforms::sort::add_range_shuffle; -use crate::pipelines::processors::transforms::sort::add_range_shuffle_merge; -use crate::pipelines::processors::transforms::sort::add_sort_sample; -use crate::pipelines::processors::transforms::sort::SortSampleState; +use crate::pipelines::processors::transforms::add_range_shuffle_exchange; +use crate::pipelines::processors::transforms::add_range_shuffle_route; +use crate::pipelines::processors::transforms::SortSampleState; use crate::pipelines::processors::transforms::TransformLimit; use crate::pipelines::processors::transforms::TransformSortBuilder; use crate::pipelines::PipelineBuilder; @@ -144,7 +143,7 @@ impl PipelineBuilder { if k > 0 && self.main_pipeline.output_len() > 1 { builder .remove_order_col_at_last() - .build_range_shuffle_sort_pipeline(&mut self.main_pipeline, k) + .build_range_shuffle_sort_pipeline(&mut self.main_pipeline) } else { builder .remove_order_col_at_last() @@ -213,18 +212,11 @@ impl SortPipelineBuilder { self.build_merge_sort_pipeline(pipeline, false) } - fn build_range_shuffle_sort_pipeline(self, pipeline: &mut Pipeline, k: usize) -> Result<()> { + fn build_range_shuffle_sort_pipeline(self, pipeline: &mut Pipeline) -> Result<()> { let inputs = pipeline.output_len(); let settings = self.ctx.get_settings(); let max_threads = settings.get_max_threads()? as usize; - let sample = SortSampleState::new( - inputs, - max_threads, - self.schema.clone(), - self.sort_desc.clone(), - ); - - add_sort_sample(pipeline, sample.clone(), self.sort_desc.clone(), k)?; + let max_block_size = settings.get_max_block_size()? as usize; // Partial sort pipeline.add_transformer(|| { @@ -234,20 +226,48 @@ impl SortPipelineBuilder { ) }); - self.build_merge_sort(pipeline, false)?; + let spiller = { + let location_prefix = self.ctx.query_id_spill_prefix(); + let config = SpillerConfig { + spiller_type: SpillerType::OrderBy, + location_prefix, + disk_spill: None, + use_parquet: settings.get_spilling_file_format()?.is_parquet(), + }; + let op = DataOperator::instance().spill_operator(); + Arc::new(Spiller::create(self.ctx.clone(), op, config)?) + }; - add_range_shuffle( - pipeline, - sample.clone(), - self.sort_desc.clone(), + let memory_settings = MemorySettings::from_sort_settings(&self.ctx)?; + let enable_loser_tree = settings.get_enable_loser_tree_merge_sort()?; + + let builder = TransformSortBuilder::create( self.schema.clone(), - self.block_size, - self.limit, - self.remove_order_col_at_last, - self.enable_loser_tree, - )?; + self.sort_desc.clone(), + max_block_size, + spiller, + ) + .with_limit(self.limit) + .with_order_col_generated(false) + .with_output_order_col(false) + .with_memory_settings(memory_settings) + .with_enable_loser_tree(enable_loser_tree); + + pipeline.add_transform(|input, output| { + Ok(ProcessorPtr::create(builder.build_collect(input, output)?)) + })?; + + let state = SortSampleState::new(inputs, max_threads, self.schema.clone(), max_block_size); + + builder.add_shuffle(pipeline, state.clone())?; + + add_range_shuffle_exchange(pipeline, max_threads)?; + + pipeline.add_transform(|input, output| { + Ok(ProcessorPtr::create(builder.build_exec(input, output)?)) + })?; - add_range_shuffle_merge(pipeline)?; + add_range_shuffle_route(pipeline)?; if self.limit.is_none() { return Ok(()); diff --git a/src/query/service/src/pipelines/processors/transforms/sort/builder.rs b/src/query/service/src/pipelines/processors/transforms/sort/builder.rs index 00bd45de5f434..da9e0e232cf16 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/builder.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/builder.rs @@ -20,6 +20,8 @@ use databend_common_expression::SortColumnDescription; 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::Pipeline; use databend_common_pipeline_transforms::processors::sort::algorithm::SortAlgorithm; use databend_common_pipeline_transforms::sort::algorithm::HeapSort; use databend_common_pipeline_transforms::sort::algorithm::LoserTreeSort; @@ -114,14 +116,12 @@ impl TransformSortBuilder { params: self, input, output, - processor: None, typ: SortType::Sort, id: 0, state: None, }; - select_row_type(&mut build); - build.processor.unwrap() + select_row_type(&mut build) } pub fn build_collect( @@ -135,14 +135,12 @@ impl TransformSortBuilder { params: self, input, output, - processor: None, typ: SortType::Collect, id: 0, state: None, }; - select_row_type(&mut build); - build.processor.unwrap() + select_row_type(&mut build) } pub fn build_exec( @@ -156,14 +154,12 @@ impl TransformSortBuilder { params: self, input, output, - processor: None, typ: SortType::Execute, id: 0, state: None, }; - select_row_type(&mut build); - build.processor.unwrap() + select_row_type(&mut build) } pub fn build_shuffle( @@ -179,14 +175,12 @@ impl TransformSortBuilder { params: self, input, output, - processor: None, typ: SortType::Shuffle, id, state: Some(state), }; - select_row_type(&mut build); - build.processor.unwrap() + select_row_type(&mut build) } fn should_use_sort_limit(&self) -> bool { @@ -211,14 +205,27 @@ impl TransformSortBuilder { limit: self.limit, } } + + pub fn add_shuffle(&self, pipeline: &mut Pipeline, state: Arc) -> Result<()> { + use std::sync::atomic; + let i = atomic::AtomicUsize::new(0); + pipeline.add_transform(|input, output| { + let id = i.fetch_add(1, atomic::Ordering::AcqRel); + Ok(ProcessorPtr::create(self.build_shuffle( + input, + output, + id, + state.clone(), + )?)) + }) + } } -pub struct Build<'a> { +struct Build<'a> { params: &'a TransformSortBuilder, typ: SortType, input: Arc, output: Arc, - processor: Option>>, id: usize, state: Option>, } @@ -284,6 +291,7 @@ impl Build<'_> { } impl RowsTypeVisitor for Build<'_> { + type Result = Result>; fn schema(&self) -> DataSchemaRef { self.params.schema.clone() } @@ -292,13 +300,13 @@ impl RowsTypeVisitor for Build<'_> { &self.params.sort_desc } - fn visit_type(&mut self) + fn visit_type(&mut self) -> Self::Result where R: Rows + 'static, C: RowConverter + Send + 'static, { let limit_sort = self.params.should_use_sort_limit(); - let processor = match self.typ { + match self.typ { SortType::Sort => match self.params.enable_loser_tree { true => self.build_sort::, C>(limit_sort), false => self.build_sort::, C>(limit_sort), @@ -312,7 +320,6 @@ impl RowsTypeVisitor for Build<'_> { false => self.build_sort_exec::>(), }, SortType::Shuffle => self.build_sort_shuffle::(), - }; - self.processor = Some(processor) + } } } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/exchange.rs b/src/query/service/src/pipelines/processors/transforms/sort/exchange.rs index 53795063c4f61..cb378244c61d3 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/exchange.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/exchange.rs @@ -12,14 +12,23 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::iter; +use std::sync::Arc; + use databend_common_exception::Result; use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::DataBlock; 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::PartitionProcessor; +use databend_common_pipeline_core::Pipe; +use databend_common_pipeline_core::PipeItem; +use databend_common_pipeline_core::Pipeline; use super::SortScatteredMeta; -pub struct SortRangeExchange; +struct SortRangeExchange; impl Exchange for SortRangeExchange { const NAME: &'static str = "SortRange"; @@ -42,3 +51,37 @@ impl Exchange for SortRangeExchange { Ok(blocks) } } + +fn create_exchange_pipe(num_input: usize, num_output: usize) -> Pipe { + let items = iter::repeat_with(|| { + let input = InputPort::create(); + let outputs = iter::repeat_with(OutputPort::create) + .take(num_output) + .collect::>(); + + PipeItem::create( + PartitionProcessor::create(input.clone(), outputs.clone(), Arc::new(SortRangeExchange)), + vec![input], + outputs, + ) + }) + .take(num_input) + .collect::>(); + + Pipe::create(num_input, num_input * num_output, items) +} + +pub fn add_range_shuffle_exchange(pipeline: &mut Pipeline, num_output: usize) -> Result<()> { + let num_input = pipeline.output_len(); + + pipeline.add_pipe(create_exchange_pipe(num_input, num_output)); + + let n = num_output; + let reorder_edges = (0..num_input * n) + .map(|i| (i % n) * num_input + (i / n)) + .collect::>(); + + pipeline.reorder_inputs(reorder_edges); + + Ok(()) +} diff --git a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs index 120ad5a556b97..7e1e4fdff52bd 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs @@ -12,11 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -mod sort_exchange; -mod sort_merge; -mod sort_sample; -mod sort_wait; - use std::sync::Arc; use bounds::Bounds; @@ -25,13 +20,12 @@ use databend_common_expression::BlockMetaInfo; use databend_common_expression::DataBlock; use databend_common_expression::DataSchemaRef; use databend_common_pipeline_transforms::SortSpillParams; -pub use sort_merge::*; -pub use sort_sample::*; +use sort_spill::SpillableBlock; -mod builder; -pub use builder::TransformSortBuilder; +use crate::spillers::Spiller; mod bounds; +mod builder; mod collect; mod exchange; mod execute; @@ -40,9 +34,10 @@ mod route; mod shuffle; mod sort_spill; -use sort_spill::SpillableBlock; - -use crate::spillers::Spiller; +pub use builder::*; +pub use exchange::*; +pub use route::*; +pub use shuffle::*; #[derive(Clone)] struct Base { diff --git a/src/query/service/src/pipelines/processors/transforms/sort/route.rs b/src/query/service/src/pipelines/processors/transforms/sort/route.rs index 12d88fac8abfd..a9592f4cd405a 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/route.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/route.rs @@ -20,15 +20,19 @@ use databend_common_pipeline_core::processors::Event; use databend_common_pipeline_core::processors::InputPort; use databend_common_pipeline_core::processors::OutputPort; use databend_common_pipeline_core::processors::Processor; +use databend_common_pipeline_core::processors::ProcessorPtr; +use databend_common_pipeline_core::Pipe; +use databend_common_pipeline_core::PipeItem; +use databend_common_pipeline_core::Pipeline; -pub struct TransformSortRoute { +struct TransformSortRoute { inputs: Vec>, output: Arc, cur_input: usize, } impl TransformSortRoute { - pub fn new(inputs: Vec>, output: Arc) -> Self { + fn new(inputs: Vec>, output: Arc) -> Self { Self { inputs, output, @@ -36,7 +40,7 @@ impl TransformSortRoute { } } - fn process_input(&mut self) -> Result<()> { + fn process(&mut self) -> Result<()> { for (i, input) in self.inputs.iter().enumerate() { if i != self.cur_input { if !input.is_finished() && !input.has_data() { @@ -84,7 +88,7 @@ impl Processor for TransformSortRoute { return Ok(Event::NeedConsume); } - self.process_input()?; + self.process()?; if self.inputs.iter().all(|input| input.is_finished()) { self.output.finish(); @@ -94,3 +98,23 @@ impl Processor for TransformSortRoute { Ok(Event::NeedData) } } + +pub fn add_range_shuffle_route(pipeline: &mut Pipeline) -> Result<()> { + let inputs = pipeline.output_len(); + let inputs_port = (0..inputs).map(|_| InputPort::create()).collect::>(); + let output = OutputPort::create(); + + let processor = ProcessorPtr::create(Box::new(TransformSortRoute::new( + inputs_port.clone(), + output.clone(), + ))); + + let pipe = Pipe::create(inputs, 1, vec![PipeItem::create( + processor, + inputs_port, + vec![output], + )]); + + pipeline.add_pipe(pipe); + Ok(()) +} diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange.rs deleted file mode 100644 index a528b5f3352f0..0000000000000 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange.rs +++ /dev/null @@ -1,148 +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::cmp::Ordering; -use std::iter; -use std::marker::PhantomData; -use std::sync::Arc; - -use databend_common_exception::Result; -use databend_common_expression::DataBlock; -use databend_common_expression::DataSchemaRef; -use databend_common_expression::SortColumnDescription; -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::Pipe; -use databend_common_pipeline_core::PipeItem; -use databend_common_pipeline_transforms::processors::sort::select_row_type; -use databend_common_pipeline_transforms::processors::sort::Rows; -use databend_common_pipeline_transforms::processors::sort::RowsTypeVisitor; -use databend_common_pipeline_transforms::sort::RowConverter; - -use super::sort_sample::SortSampleState; -use crate::pipelines::processors::PartitionProcessor; - -pub struct SortRangeExchange { - state: Arc, - _r: PhantomData, -} - -unsafe impl Send for SortRangeExchange {} - -unsafe impl Sync for SortRangeExchange {} - -impl Exchange for SortRangeExchange { - const NAME: &'static str = "SortRange"; - fn partition(&self, data: DataBlock, n: usize) -> Result> { - let bounds = self.state.bounds().unwrap(); - debug_assert_eq!(n, self.state.partitions()); - debug_assert!(bounds.len() < n); - - if data.is_empty() { - return Ok(vec![]); - } - - if bounds.len() == 0 { - return Ok(vec![data]); - } - - let bounds = R::from_column(&bounds)?; - let rows = R::from_column(data.get_last_column())?; - - let mut i = 0; - let mut j = 0; - let mut bound = bounds.row(j); - let mut indices = Vec::new(); - while i < rows.len() { - match rows.row(i).cmp(&bound) { - Ordering::Less => indices.push(j as u32), - Ordering::Greater if j + 1 < bounds.len() => { - j += 1; - bound = bounds.row(j); - continue; - } - _ => indices.push(j as u32 + 1), - } - i += 1; - } - - DataBlock::scatter(&data, &indices, n) - } -} - -pub fn create_exchange_pipe( - inputs: usize, - partitions: usize, - schema: DataSchemaRef, - sort_desc: Arc<[SortColumnDescription]>, - state: Arc, -) -> Pipe { - let mut builder = Builder { - inputs, - partitions, - sort_desc, - schema, - state, - items: Vec::new(), - }; - - select_row_type(&mut builder); - - Pipe::create(inputs, inputs * partitions, builder.items) -} - -struct Builder { - inputs: usize, - partitions: usize, - sort_desc: Arc<[SortColumnDescription]>, - schema: DataSchemaRef, - state: Arc, - items: Vec, -} - -impl RowsTypeVisitor for Builder { - fn visit_type(&mut self) - where - R: Rows + 'static, - C: RowConverter + Send + 'static, - { - let exchange = Arc::new(SortRangeExchange:: { - state: self.state.clone(), - _r: PhantomData, - }); - self.items = iter::repeat_with(|| { - let input = InputPort::create(); - let outputs = iter::repeat_with(OutputPort::create) - .take(self.partitions) - .collect::>(); - - PipeItem::create( - PartitionProcessor::create(input.clone(), outputs.clone(), exchange.clone()), - vec![input], - outputs, - ) - }) - .take(self.inputs) - .collect::>(); - } - - fn schema(&self) -> DataSchemaRef { - self.schema.clone() - } - - fn sort_desc(&self) -> &[SortColumnDescription] { - &self.sort_desc - } -} diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_merge.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_merge.rs deleted file mode 100644 index 569a1e54e83b3..0000000000000 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_merge.rs +++ /dev/null @@ -1,62 +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::Result; -use databend_common_expression::DataBlock; -use databend_common_pipeline_core::processors::Exchange; -use databend_common_pipeline_core::processors::InputPort; -use databend_common_pipeline_core::processors::MergePartitionProcessor; -use databend_common_pipeline_core::processors::MultiwayStrategy; -use databend_common_pipeline_core::processors::OutputPort; -use databend_common_pipeline_core::Pipe; -use databend_common_pipeline_core::PipeItem; -use databend_common_pipeline_core::Pipeline; - -pub struct TransformSortRangeMerge {} - -impl Exchange for TransformSortRangeMerge { - const NAME: &'static str = "SortRangeMerge"; - const STRATEGY: MultiwayStrategy = MultiwayStrategy::Custom; - - fn partition(&self, block: DataBlock, _: usize) -> Result> { - Ok(vec![block]) - } - - fn multiway_pick(&self, partitions: &[Option]) -> Result { - Ok(partitions.iter().position(Option::is_some).unwrap()) - } -} - -pub fn add_range_shuffle_merge(pipeline: &mut Pipeline) -> Result<()> { - let inputs = pipeline.output_len(); - let inputs_port = (0..inputs).map(|_| InputPort::create()).collect::>(); - let output = OutputPort::create(); - - let processor = MergePartitionProcessor::create( - inputs_port.clone(), - output.clone(), - Arc::new(TransformSortRangeMerge {}), - ); - - let pipe = Pipe::create(inputs, 1, vec![PipeItem::create( - processor, - inputs_port, - vec![output], - )]); - - pipeline.add_pipe(pipe); - Ok(()) -} diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_sample.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_sample.rs deleted file mode 100644 index b868de0a6383d..0000000000000 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_sample.rs +++ /dev/null @@ -1,333 +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::sync::RwLock; - -use databend_common_base::base::WatchNotify; -use databend_common_exception::Result; -use databend_common_expression::sampler::FixedSizeSampler; -use databend_common_expression::visitor::ValueVisitor; -use databend_common_expression::Column; -use databend_common_expression::DataBlock; -use databend_common_expression::DataSchemaRef; -use databend_common_expression::SortColumnDescription; -use databend_common_expression::SortCompare; -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::processors::create_multi_sort_merge_processor; -use databend_common_pipeline_transforms::processors::sort::convert_rows; -use databend_common_pipeline_transforms::processors::Transform; -use databend_common_pipeline_transforms::TransformPipelineHelper; -use rand::rngs::StdRng; -use rand::SeedableRng; - -use super::sort_exchange::create_exchange_pipe; -use super::sort_wait::TransformSortSampleWait; - -pub struct SortSampleState { - inner: RwLock, - pub(super) done: WatchNotify, -} - -impl SortSampleState { - pub fn partitions(&self) -> usize { - self.inner.read().unwrap().partitions - } -} - -struct StateInner { - partitions: usize, - // schema for bounds DataBlock - schema: DataSchemaRef, - // sort_desc for bounds DataBlock - sort_desc: Vec, - partial: Vec>, - bounds: Option, -} - -impl StateInner { - fn determine_bounds(&mut self) -> Result<()> { - let partial = std::mem::take(&mut self.partial) - .into_iter() - .filter_map(|b| { - let b = b.unwrap(); - if b.is_empty() { - None - } else { - Some(b) - } - }) - .collect::>(); - - if partial.is_empty() { - let bounds = convert_rows( - self.schema.clone(), - &self.sort_desc, - DataBlock::empty_with_schema(self.schema.clone()), - )?; - - self.bounds = Some(bounds); - return Ok(()); - } - - let candidates = DataBlock::concat(&partial)?; - let rows = candidates.num_rows(); - - let mut sort_compare = SortCompare::with_force_equality(self.sort_desc.clone(), rows); - - for desc in &self.sort_desc { - let array = candidates.get_by_offset(desc.offset).value(); - sort_compare.visit_value(array)?; - sort_compare.increment_column_index(); - } - - let equality = sort_compare.equality_index().to_vec(); - let permutation = sort_compare.take_permutation(); - - let step = permutation.len() as f64 / self.partitions as f64; - let mut target = step; - let mut bounds = Vec::with_capacity(self.partitions - 1); - let mut equals = true; - for (i, (&pos, eq)) in permutation.iter().zip(equality).enumerate() { - if bounds.len() >= self.partitions - 1 { - break; - } - if equals && eq == 0 { - equals = false - } - if i as f64 >= target && (!equals || i != 0) { - bounds.push(pos); - target += step; - equals = true - } - } - - let bounds = convert_rows( - self.schema.clone(), - &self.sort_desc, - candidates.take(&bounds)?, - )?; - self.bounds = Some(bounds); - Ok(()) - } -} - -impl SortSampleState { - pub fn new( - inputs: usize, - partitions: usize, - schema: DataSchemaRef, - sort_desc: Arc<[SortColumnDescription]>, - ) -> Arc { - let columns = sort_desc.iter().map(|desc| desc.offset).collect::>(); - let schema = schema.project(&columns).into(); - let sort_desc = sort_desc - .iter() - .enumerate() - .map(|(i, desc)| SortColumnDescription { - offset: i, - asc: desc.asc, - nulls_first: desc.nulls_first, - }) - .collect::>(); - Arc::new(SortSampleState { - inner: RwLock::new(StateInner { - partitions, - schema, - sort_desc, - partial: vec![None; inputs], - bounds: None, - }), - done: WatchNotify::new(), - }) - } - - pub fn bounds(&self) -> Option { - if let Some(bounds) = &self.inner.read().unwrap().bounds { - return Some(bounds.clone()); - } - None - } - - pub fn commit_sample(&self, id: usize, block: Option) -> Result { - let mut inner = self.inner.write().unwrap(); - - let block = block.unwrap_or(DataBlock::empty_with_schema(inner.schema.clone())); - let x = inner.partial[id].replace(block); - debug_assert!(x.is_none()); - let done = inner.partial.iter().all(|x| x.is_some()); - if done { - inner.determine_bounds()?; - self.done.notify_waiters(); - } - Ok(done) - } -} - -pub struct TransformSortSample { - id: usize, - sampler: FixedSizeSampler, - state: Arc, -} - -unsafe impl Send for TransformSortSample {} - -impl TransformSortSample { - fn new(id: usize, k: usize, columns: Vec, state: Arc) -> Self { - let rng = StdRng::from_rng(rand::thread_rng()).unwrap(); - let sampler = FixedSizeSampler::new(columns, 65536, k, rng); - TransformSortSample { id, sampler, state } - } -} - -impl Transform for TransformSortSample { - const NAME: &'static str = "TransformSortSample"; - - fn transform(&mut self, data: DataBlock) -> Result { - self.sampler.add_block(data.clone()); - Ok(data) - } - - fn on_finish(&mut self) -> Result<()> { - self.sampler.compact_blocks(); - let mut sample = self.sampler.take_blocks(); - assert!(sample.len() <= 1); // Unlikely to sample rows greater than 65536 - self.state.commit_sample( - self.id, - if sample.is_empty() { - None - } else { - Some(sample.remove(0)) - }, - )?; - Ok(()) - } -} - -pub fn add_sort_sample( - pipeline: &mut Pipeline, - state: Arc, - sort_desc: Arc<[SortColumnDescription]>, - k: usize, -) -> Result<()> { - use std::sync::atomic; - let i = atomic::AtomicUsize::new(0); - let columns = sort_desc.iter().map(|desc| desc.offset).collect::>(); - pipeline.add_transformer(|| { - let id = i.fetch_add(1, atomic::Ordering::AcqRel); - TransformSortSample::new(id, k, columns.clone(), state.clone()) - }); - Ok(()) -} - -pub fn add_range_shuffle( - pipeline: &mut Pipeline, - state: Arc, - sort_desc: Arc<[SortColumnDescription]>, - schema: DataSchemaRef, - block_size: usize, - limit: Option, - remove_order_col: bool, - enable_loser_tree: bool, -) -> Result<()> { - pipeline.add_transform(|input, output| { - Ok(ProcessorPtr::create(Box::new( - TransformSortSampleWait::new(input, output, state.clone()), - ))) - })?; - - // partition data block - let input_len = pipeline.output_len(); - let n = state.partitions(); - let exchange = create_exchange_pipe(input_len, n, schema.clone(), sort_desc.clone(), state); - pipeline.add_pipe(exchange); - - let reorder_edges = (0..input_len * n) - .map(|index| (index % n) * input_len + (index / n)) - .collect::>(); - - pipeline.reorder_inputs(reorder_edges); - - let mut items = Vec::with_capacity(input_len); - for _ in 0..n { - let output = OutputPort::create(); - let inputs: Vec<_> = (0..input_len).map(|_| InputPort::create()).collect(); - - let proc = create_multi_sort_merge_processor( - inputs.clone(), - output.clone(), - schema.clone(), - block_size, - limit, - sort_desc.clone(), - remove_order_col, - enable_loser_tree, - )?; - - items.push(PipeItem::create(ProcessorPtr::create(proc), inputs, vec![ - output, - ])); - } - - // merge partition - pipeline.add_pipe(Pipe::create(input_len * n, n, items)); - - Ok(()) -} - -#[cfg(test)] -mod tests { - use databend_common_expression::types::ArgType; - use databend_common_expression::types::Int32Type; - use databend_common_expression::DataField; - use databend_common_expression::DataSchemaRefExt; - use databend_common_expression::FromData; - - use super::*; - - #[test] - fn test_determine_bounds() { - let partial = vec![vec![1, 2, 3, 4], vec![4, 5, 6, 7], vec![0, 2, 4, 5]] - .into_iter() - .map(|data| { - Some(DataBlock::new_from_columns(vec![Int32Type::from_data( - data, - )])) - }) - .collect::>(); - - let schema = DataSchemaRefExt::create(vec![DataField::new("a", Int32Type::data_type())]); - let mut inner = StateInner { - partitions: 3, - schema, - sort_desc: vec![SortColumnDescription { - offset: 0, - asc: true, - nulls_first: false, - }], - partial, - bounds: None, - }; - - inner.determine_bounds().unwrap(); - - // 0 1 2 2 | 3 4 4 4 | 5 5 6 7 - assert_eq!(Int32Type::from_data(vec![3, 5]), inner.bounds.unwrap()) - } -} diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_wait.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_wait.rs deleted file mode 100644 index 3f1543d6b2760..0000000000000 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_wait.rs +++ /dev/null @@ -1,113 +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::VecDeque; -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; - -use super::SortSampleState; - -pub struct TransformSortSampleWait { - input: Arc, - output: Arc, - output_data: VecDeque, - blocks: Vec, - state: Arc, -} - -impl TransformSortSampleWait { - pub fn new( - input: Arc, - output: Arc, - state: Arc, - ) -> Self { - Self { - input, - output, - output_data: VecDeque::new(), - blocks: Vec::new(), - state, - } - } -} - -#[async_trait::async_trait] -impl Processor for TransformSortSampleWait { - fn name(&self) -> String { - "TransformSortSimpleWait".to_string() - } - - 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.blocks.push(self.input.pull_data().unwrap()?); - self.input.set_need_data(); - return Ok(Event::NeedData); - } - - if self.input.is_finished() { - if self.blocks.is_empty() { - self.output.finish(); - return Ok(Event::Finished); - } - - return if self.state.done.has_notified() { - Ok(Event::Sync) - } else { - Ok(Event::Async) - }; - } - - self.input.set_need_data(); - Ok(Event::NeedData) - } - - fn process(&mut self) -> Result<()> { - debug_assert!(!self.blocks.is_empty()); - self.output_data = VecDeque::from(std::mem::take(&mut self.blocks)); - Ok(()) - } - - #[async_backtrace::framed] - async fn async_process(&mut self) -> Result<()> { - self.state.done.notified().await; - self.output_data = VecDeque::from(std::mem::take(&mut self.blocks)); - Ok(()) - } -} From 3575be8c1cbd991a16c1fc50f830879b044e3b89 Mon Sep 17 00:00:00 2001 From: coldWater Date: Thu, 24 Apr 2025 21:42:49 +0800 Subject: [PATCH 22/61] rename Signed-off-by: coldWater --- .../transforms/transform_multi_sort_merge.rs | 4 ++-- .../processors/transforms/sort/mod.rs | 20 +++++++++---------- .../sort/{builder.rs => sort_builder.rs} | 8 ++++---- .../sort/{collect.rs => sort_collect.rs} | 0 .../sort/{exchange.rs => sort_exchange.rs} | 0 .../sort/{execute.rs => sort_execute.rs} | 0 .../sort/{route.rs => sort_route.rs} | 0 .../sort/{shuffle.rs => sort_shuffle.rs} | 0 src/query/settings/src/settings_default.rs | 2 +- 9 files changed, 17 insertions(+), 17 deletions(-) rename src/query/service/src/pipelines/processors/transforms/sort/{builder.rs => sort_builder.rs} (98%) rename src/query/service/src/pipelines/processors/transforms/sort/{collect.rs => sort_collect.rs} (100%) rename src/query/service/src/pipelines/processors/transforms/sort/{exchange.rs => sort_exchange.rs} (100%) rename src/query/service/src/pipelines/processors/transforms/sort/{execute.rs => sort_execute.rs} (100%) rename src/query/service/src/pipelines/processors/transforms/sort/{route.rs => sort_route.rs} (100%) rename src/query/service/src/pipelines/processors/transforms/sort/{shuffle.rs => sort_shuffle.rs} (100%) diff --git a/src/query/pipeline/transforms/src/processors/transforms/transform_multi_sort_merge.rs b/src/query/pipeline/transforms/src/processors/transforms/transform_multi_sort_merge.rs index 5ab82226a9940..573315604e414 100644 --- a/src/query/pipeline/transforms/src/processors/transforms/transform_multi_sort_merge.rs +++ b/src/query/pipeline/transforms/src/processors/transforms/transform_multi_sort_merge.rs @@ -77,7 +77,7 @@ pub fn try_add_multi_sort_merge( } let output_port = OutputPort::create(); - let processor = ProcessorPtr::create(create_multi_sort_merge_processor( + let processor = ProcessorPtr::create(create_processor( inputs_port.clone(), output_port.clone(), schema, @@ -98,7 +98,7 @@ pub fn try_add_multi_sort_merge( } } -pub fn create_multi_sort_merge_processor( +fn create_processor( inputs: Vec>, output: Arc, schema: DataSchemaRef, diff --git a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs index 7e1e4fdff52bd..f59ae6a767d42 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs @@ -25,19 +25,19 @@ use sort_spill::SpillableBlock; use crate::spillers::Spiller; mod bounds; -mod builder; -mod collect; -mod exchange; -mod execute; mod merge_sort; -mod route; -mod shuffle; +mod sort_builder; +mod sort_collect; +mod sort_exchange; +mod sort_execute; +mod sort_route; +mod sort_shuffle; mod sort_spill; -pub use builder::*; -pub use exchange::*; -pub use route::*; -pub use shuffle::*; +pub use sort_builder::*; +pub use sort_exchange::*; +pub use sort_route::*; +pub use sort_shuffle::*; #[derive(Clone)] struct Base { diff --git a/src/query/service/src/pipelines/processors/transforms/sort/builder.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs similarity index 98% rename from src/query/service/src/pipelines/processors/transforms/sort/builder.rs rename to src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs index da9e0e232cf16..c7290b30af271 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/builder.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs @@ -33,11 +33,11 @@ use databend_common_pipeline_transforms::sort::Rows; use databend_common_pipeline_transforms::sort::RowsTypeVisitor; use databend_common_pipeline_transforms::MemorySettings; -use super::collect::TransformSortCollect; -use super::execute::TransformSortExecute; use super::merge_sort::TransformSort; -use super::shuffle::SortSampleState; -use super::shuffle::TransformSortShuffle; +use super::sort_collect::TransformSortCollect; +use super::sort_execute::TransformSortExecute; +use super::sort_shuffle::SortSampleState; +use super::sort_shuffle::TransformSortShuffle; use super::Base; use crate::spillers::Spiller; diff --git a/src/query/service/src/pipelines/processors/transforms/sort/collect.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_collect.rs similarity index 100% rename from src/query/service/src/pipelines/processors/transforms/sort/collect.rs rename to src/query/service/src/pipelines/processors/transforms/sort/sort_collect.rs diff --git a/src/query/service/src/pipelines/processors/transforms/sort/exchange.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange.rs similarity index 100% rename from src/query/service/src/pipelines/processors/transforms/sort/exchange.rs rename to src/query/service/src/pipelines/processors/transforms/sort/sort_exchange.rs diff --git a/src/query/service/src/pipelines/processors/transforms/sort/execute.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_execute.rs similarity index 100% rename from src/query/service/src/pipelines/processors/transforms/sort/execute.rs rename to src/query/service/src/pipelines/processors/transforms/sort/sort_execute.rs diff --git a/src/query/service/src/pipelines/processors/transforms/sort/route.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_route.rs similarity index 100% rename from src/query/service/src/pipelines/processors/transforms/sort/route.rs rename to src/query/service/src/pipelines/processors/transforms/sort/sort_route.rs diff --git a/src/query/service/src/pipelines/processors/transforms/sort/shuffle.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_shuffle.rs similarity index 100% rename from src/query/service/src/pipelines/processors/transforms/sort/shuffle.rs rename to src/query/service/src/pipelines/processors/transforms/sort/sort_shuffle.rs diff --git a/src/query/settings/src/settings_default.rs b/src/query/settings/src/settings_default.rs index f8f14d4a237c5..29e3c7c1ae012 100644 --- a/src/query/settings/src/settings_default.rs +++ b/src/query/settings/src/settings_default.rs @@ -655,7 +655,7 @@ impl DefaultSettings { range: Some(SettingRange::Numeric(4 * 1024..=u64::MAX)), }), ("range_shuffle_sort_simple_size", DefaultSettingValue { - value: UserSettingValue::UInt64(20), + value: UserSettingValue::UInt64(0), desc: "Sets the simple size per partition used for range shuffle sorting, 0 to disable range shuffle sorting.", mode: SettingMode::Both, scope: SettingScope::Both, From c150c345df84b933c15d7971fb6e26b955038617 Mon Sep 17 00:00:00 2001 From: coldWater Date: Fri, 25 Apr 2025 11:49:17 +0800 Subject: [PATCH 23/61] fix Signed-off-by: coldWater --- .../src/pipelines/builders/builder_sort.rs | 13 ++- .../src/pipelines/executor/executor_graph.rs | 10 ++- .../pipelines/executor/pipeline_executor.rs | 2 +- .../executor/query_pipeline_executor.rs | 2 +- .../processors/transforms/sort/bounds.rs | 11 ++- .../processors/transforms/sort/mod.rs | 3 +- .../transforms/sort/sort_builder.rs | 32 ++++++++ .../transforms/sort/sort_combine.rs | 80 +++++++++++++++++++ .../transforms/sort/sort_exchange.rs | 63 +++------------ .../transforms/sort/sort_execute.rs | 16 ++-- .../transforms/sort/sort_shuffle.rs | 58 ++++++++------ .../processors/transforms/sort/sort_spill.rs | 11 ++- src/query/settings/src/settings_default.rs | 8 +- .../settings/src/settings_getter_setter.rs | 4 +- 14 files changed, 205 insertions(+), 108 deletions(-) create mode 100644 src/query/service/src/pipelines/processors/transforms/sort/sort_combine.rs diff --git a/src/query/service/src/pipelines/builders/builder_sort.rs b/src/query/service/src/pipelines/builders/builder_sort.rs index 563128b0e169f..314a772503dca 100644 --- a/src/query/service/src/pipelines/builders/builder_sort.rs +++ b/src/query/service/src/pipelines/builders/builder_sort.rs @@ -34,8 +34,8 @@ use databend_common_storages_fuse::TableContext; use databend_storages_common_cache::TempDirManager; use crate::pipelines::memory_settings::MemorySettingsExt; -use crate::pipelines::processors::transforms::add_range_shuffle_exchange; use crate::pipelines::processors::transforms::add_range_shuffle_route; +use crate::pipelines::processors::transforms::SortRangeExchange; use crate::pipelines::processors::transforms::SortSampleState; use crate::pipelines::processors::transforms::TransformLimit; use crate::pipelines::processors::transforms::TransformSortBuilder; @@ -139,8 +139,9 @@ impl PipelineBuilder { None => { // Build for single node mode. // We build the full sort pipeline for it. - let k = self.settings.get_range_shuffle_sort_simple_size()?; - if k > 0 && self.main_pipeline.output_len() > 1 { + if self.settings.get_enable_range_shuffle_sort()? + && self.main_pipeline.output_len() > 1 + { builder .remove_order_col_at_last() .build_range_shuffle_sort_pipeline(&mut self.main_pipeline) @@ -261,7 +262,11 @@ impl SortPipelineBuilder { builder.add_shuffle(pipeline, state.clone())?; - add_range_shuffle_exchange(pipeline, max_threads)?; + pipeline.exchange(max_threads, Arc::new(SortRangeExchange)); + + pipeline.add_transform(|input, output| { + Ok(ProcessorPtr::create(builder.build_combine(input, output)?)) + })?; pipeline.add_transform(|input, output| { Ok(ProcessorPtr::create(builder.build_exec(input, output)?)) diff --git a/src/query/service/src/pipelines/executor/executor_graph.rs b/src/query/service/src/pipelines/executor/executor_graph.rs index 2d641e0096e29..cd2e92d36ba70 100644 --- a/src/query/service/src/pipelines/executor/executor_graph.rs +++ b/src/query/service/src/pipelines/executor/executor_graph.rs @@ -795,7 +795,7 @@ impl RunningGraph { true => Ok(()), false => Err(ErrorCode::Internal(format!( "Pipeline graph is not finished, details: {}", - self.format_graph_nodes() + self.format_graph_nodes(true) ))), } } @@ -861,7 +861,7 @@ impl RunningGraph { self.0.finished_notify.clone() } - pub fn format_graph_nodes(&self) -> String { + pub fn format_graph_nodes(&self, pretty: bool) -> String { pub struct NodeDisplay { id: usize, name: String, @@ -961,7 +961,11 @@ impl RunningGraph { } } - format!("{:?}", nodes_display) + if pretty { + format!("{:#?}", nodes_display) + } else { + format!("{:?}", nodes_display) + } } /// Change the priority diff --git a/src/query/service/src/pipelines/executor/pipeline_executor.rs b/src/query/service/src/pipelines/executor/pipeline_executor.rs index 53a82eba8da10..3d7ec2d3eba00 100644 --- a/src/query/service/src/pipelines/executor/pipeline_executor.rs +++ b/src/query/service/src/pipelines/executor/pipeline_executor.rs @@ -268,7 +268,7 @@ impl PipelineExecutor { pub fn format_graph_nodes(&self) -> String { match self { PipelineExecutor::QueryPipelineExecutor(executor) => executor.format_graph_nodes(), - PipelineExecutor::QueriesPipelineExecutor(v) => v.graph.format_graph_nodes(), + PipelineExecutor::QueriesPipelineExecutor(v) => v.graph.format_graph_nodes(false), } } diff --git a/src/query/service/src/pipelines/executor/query_pipeline_executor.rs b/src/query/service/src/pipelines/executor/query_pipeline_executor.rs index b45e707f30cc8..5e56b2830def6 100644 --- a/src/query/service/src/pipelines/executor/query_pipeline_executor.rs +++ b/src/query/service/src/pipelines/executor/query_pipeline_executor.rs @@ -434,7 +434,7 @@ impl QueryPipelineExecutor { } pub fn format_graph_nodes(&self) -> String { - self.graph.format_graph_nodes() + self.graph.format_graph_nodes(false) } pub fn fetch_plans_profile(&self, collect_metrics: bool) -> HashMap { diff --git a/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs b/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs index 741ccca55d66a..b9ee5cd80437b 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs @@ -96,7 +96,7 @@ impl Bounds { self.0.iter().all(|col| col.len() == 0) } - pub fn reduce(&self, n: usize, data_type: DataType) -> Option { + pub fn reduce(&self, n: usize) -> Option { if n == 0 { return Some(Self::default()); } @@ -126,7 +126,6 @@ impl Bounds { Some(Bounds(vec![Column::take_column_indices( &self.0, - data_type, &indices, indices.len(), )])) @@ -219,16 +218,16 @@ mod tests { .collect::>>()?; let bounds = Bounds::merge::>(data, 2)?; - let got = bounds.reduce(4, Int32Type::data_type()).unwrap(); + let got = bounds.reduce(4).unwrap(); assert_eq!(got, Bounds(vec![Int32Type::from_data(vec![8, 6, 2, 1])])); // 77 _8 7 _6 3 _2 1 _1 -2 - let got = bounds.reduce(3, Int32Type::data_type()).unwrap(); + let got = bounds.reduce(3).unwrap(); assert_eq!(got, Bounds(vec![Int32Type::from_data(vec![8, 3, 1])])); // 77 _8 7 6 _3 2 1 _1 -2 - let got = bounds.reduce(2, Int32Type::data_type()).unwrap(); + let got = bounds.reduce(2).unwrap(); assert_eq!(got, Bounds(vec![Int32Type::from_data(vec![7, 1])])); // 77 8 _7 6 3 2 _1 1 -2 - let got = bounds.reduce(1, Int32Type::data_type()).unwrap(); + let got = bounds.reduce(1).unwrap(); assert_eq!(got, Bounds(vec![Int32Type::from_data(vec![3])])); // 77 8 7 6 _3 2 1 1 -2 Ok(()) diff --git a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs index f59ae6a767d42..3d7c57e31beb0 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs @@ -28,6 +28,7 @@ mod bounds; mod merge_sort; mod sort_builder; mod sort_collect; +mod sort_combine; mod sort_exchange; mod sort_execute; mod sort_route; @@ -60,7 +61,7 @@ local_block_meta_serde!(SortCollectedMeta); impl BlockMetaInfo for SortCollectedMeta {} #[derive(Debug)] -struct SortScatteredMeta(pub Vec); +struct SortScatteredMeta(pub Vec>); local_block_meta_serde!(SortScatteredMeta); diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs index c7290b30af271..1eb032fb1bdc3 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs @@ -31,10 +31,12 @@ use databend_common_pipeline_transforms::sort::utils::ORDER_COL_NAME; use databend_common_pipeline_transforms::sort::RowConverter; use databend_common_pipeline_transforms::sort::Rows; use databend_common_pipeline_transforms::sort::RowsTypeVisitor; +use databend_common_pipeline_transforms::AccumulatingTransformer; use databend_common_pipeline_transforms::MemorySettings; use super::merge_sort::TransformSort; use super::sort_collect::TransformSortCollect; +use super::sort_combine::TransformSortCombine; use super::sort_execute::TransformSortExecute; use super::sort_shuffle::SortSampleState; use super::sort_shuffle::TransformSortShuffle; @@ -46,6 +48,7 @@ enum SortType { Collect, Execute, Shuffle, + Combine, } pub struct TransformSortBuilder { @@ -183,6 +186,25 @@ impl TransformSortBuilder { select_row_type(&mut build) } + pub fn build_combine( + &self, + input: Arc, + output: Arc, + ) -> Result> { + self.check(); + + let mut build = Build { + params: self, + input, + output, + typ: SortType::Combine, + id: 0, + state: None, + }; + + select_row_type(&mut build) + } + fn should_use_sort_limit(&self) -> bool { self.limit.map(|limit| limit < 10000).unwrap_or_default() } @@ -288,6 +310,15 @@ impl Build<'_> { self.params.spiller.clone(), ))) } + + fn build_sort_combine(&mut self) -> Result> + where R: Rows + 'static { + Ok(AccumulatingTransformer::create( + self.input.clone(), + self.output.clone(), + TransformSortCombine::::new(self.params.block_size), + )) + } } impl RowsTypeVisitor for Build<'_> { @@ -320,6 +351,7 @@ impl RowsTypeVisitor for Build<'_> { false => self.build_sort_exec::>(), }, SortType::Shuffle => self.build_sort_shuffle::(), + SortType::Combine => self.build_sort_combine::(), } } } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_combine.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_combine.rs new file mode 100644 index 0000000000000..538bd7aff390b --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_combine.rs @@ -0,0 +1,80 @@ +// 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::BlockMetaInfoDowncast; +use databend_common_expression::DataBlock; +use databend_common_pipeline_transforms::sort::Rows; +use databend_common_pipeline_transforms::AccumulatingTransform; + +use super::bounds::Bounds; +use super::SortCollectedMeta; + +pub struct TransformSortCombine { + batch_rows: usize, + metas: Vec, + _r: std::marker::PhantomData, +} + +impl TransformSortCombine { + pub fn new(batch_rows: usize) -> Self { + Self { + batch_rows, + metas: vec![], + _r: Default::default(), + } + } +} + +impl AccumulatingTransform for TransformSortCombine { + const NAME: &'static str = "TransformSortCombine"; + + fn transform(&mut self, mut data: DataBlock) -> Result> { + self.metas.push( + data.take_meta() + .and_then(SortCollectedMeta::downcast_from) + .expect("require a SortCollectedMeta"), + ); + Ok(vec![]) + } + + fn on_finish(&mut self, output: bool) -> Result> { + if !output || self.metas.is_empty() { + return Ok(vec![]); + } + + let params = self.metas.first().map(|meta| meta.params).unwrap(); + + let bounds = self + .metas + .iter_mut() + .map(|meta| std::mem::take(&mut meta.bounds)) + .collect(); + let bounds = Bounds::merge::(bounds, self.batch_rows)?; + + let blocks = self + .metas + .drain(..) + .flat_map(|meta| meta.blocks.into_iter()) + .collect(); + + Ok(vec![DataBlock::empty_with_meta(Box::new( + SortCollectedMeta { + params, + bounds, + blocks, + }, + ))]) + } +} diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange.rs index cb378244c61d3..700c5a3e0e81a 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange.rs @@ -12,76 +12,33 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::iter; -use std::sync::Arc; - use databend_common_exception::Result; use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::DataBlock; 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::PartitionProcessor; -use databend_common_pipeline_core::Pipe; -use databend_common_pipeline_core::PipeItem; -use databend_common_pipeline_core::Pipeline; use super::SortScatteredMeta; -struct SortRangeExchange; +pub struct SortRangeExchange; impl Exchange for SortRangeExchange { const NAME: &'static str = "SortRange"; fn partition(&self, mut data: DataBlock, n: usize) -> Result> { - let Some(meta) = data.take_meta() else { - unreachable!(); - }; - - let Some(SortScatteredMeta(scattered)) = SortScatteredMeta::downcast_from(meta) else { - unreachable!(); - }; - + let scattered = data + .take_meta() + .and_then(SortScatteredMeta::downcast_from) + .expect("require a SortScatteredMeta") + .0; assert!(scattered.len() <= n); let blocks = scattered .into_iter() - .map(|meta| DataBlock::empty_with_meta(Box::new(meta))) + .map(|meta| { + meta.map(|meta| DataBlock::empty_with_meta(Box::new(meta))) + .unwrap_or_else(DataBlock::empty) + }) .collect(); Ok(blocks) } } - -fn create_exchange_pipe(num_input: usize, num_output: usize) -> Pipe { - let items = iter::repeat_with(|| { - let input = InputPort::create(); - let outputs = iter::repeat_with(OutputPort::create) - .take(num_output) - .collect::>(); - - PipeItem::create( - PartitionProcessor::create(input.clone(), outputs.clone(), Arc::new(SortRangeExchange)), - vec![input], - outputs, - ) - }) - .take(num_input) - .collect::>(); - - Pipe::create(num_input, num_input * num_output, items) -} - -pub fn add_range_shuffle_exchange(pipeline: &mut Pipeline, num_output: usize) -> Result<()> { - let num_input = pipeline.output_len(); - - pipeline.add_pipe(create_exchange_pipe(num_input, num_output)); - - let n = num_output; - let reorder_edges = (0..num_input * n) - .map(|i| (i % n) * num_input + (i / n)) - .collect::>(); - - pipeline.reorder_inputs(reorder_edges); - - Ok(()) -} diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_execute.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_execute.rs index 3c5d832aa5973..044858dc4358a 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_execute.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_execute.rs @@ -91,6 +91,11 @@ where return Ok(Event::NeedConsume); } + if self.input.is_finished() && self.inner.is_none() { + self.output.finish(); + return Ok(Event::Finished); + } + if let Some(mut block) = self.input.pull_data().transpose()? { assert!(self.inner.is_none()); let meta = block @@ -102,12 +107,12 @@ where return Ok(Event::Async); } - if self.input.is_finished() { - Ok(Event::Async) - } else { - self.input.set_need_data(); - Ok(Event::NeedData) + if self.inner.is_some() { + return Ok(Event::Async); } + + self.input.set_need_data(); + Ok(Event::NeedData) } #[async_backtrace::framed] @@ -122,6 +127,7 @@ where } if finish { self.output.finish(); + self.inner = None; } Ok(()) } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_shuffle.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_shuffle.rs index 9c9a3b7067019..8f6cbb6155f42 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_shuffle.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_shuffle.rs @@ -39,7 +39,7 @@ use crate::spillers::Spiller; enum Step { None, Meta(Box), - Scattered(Vec), + Scattered(Vec>), } pub struct TransformSortShuffle { @@ -71,25 +71,26 @@ impl TransformSortShuffle { } } - async fn scatter(&mut self) -> Result<()> { - let scatter_bounds = self.state.bounds(); - - let Step::Meta(box SortCollectedMeta { + async fn scatter(&mut self) -> Result>> { + let SortCollectedMeta { params, bounds, blocks, - }) = std::mem::replace(&mut self.step, Step::None) - else { - unreachable!() + } = match std::mem::replace(&mut self.step, Step::None) { + Step::None => { + return Ok(vec![]); + } + Step::Meta(box meta) => meta, + _ => unreachable!(), }; + let scatter_bounds = self.state.bounds(); if scatter_bounds.is_empty() { - Step::Scattered(vec![SortCollectedMeta { + return Ok(vec![Some(SortCollectedMeta { params, bounds, blocks, - }]); - return Ok(()); + })]); } let base = { @@ -102,24 +103,31 @@ impl TransformSortShuffle { } }; - let mut scattered_meta = std::iter::repeat_with(|| SortCollectedMeta { - params, - bounds: bounds.clone(), - blocks: vec![], - }) - .take(scatter_bounds.len() + 1) - .collect::>(); + let mut scattered_blocks = std::iter::repeat_with(Vec::new) + .take(scatter_bounds.len() + 1) + .collect::>(); for blocks in blocks { let scattered = base .scatter_stream::(Vec::from(blocks).into(), scatter_bounds.clone()) .await?; for (i, part) in scattered.into_iter().enumerate() { - scattered_meta[i].blocks.push(part.into_boxed_slice()); + if !part.is_empty() { + scattered_blocks[i].push(part.into_boxed_slice()); + } } } - self.step = Step::Scattered(scattered_meta); - Ok(()) + let scattered_meta = scattered_blocks + .into_iter() + .map(|blocks| { + (!blocks.is_empty()).then_some(SortCollectedMeta { + params, + bounds: bounds.clone(), + blocks, + }) + }) + .collect(); + Ok(scattered_meta) } } @@ -182,12 +190,14 @@ impl Processor for TransformSortShuffle { #[async_backtrace::framed] async fn async_process(&mut self) -> Result<()> { let bounds = match &self.step { + Step::None if self.input.is_finished() => Bounds::default(), Step::Meta(meta) => meta.bounds.clone(), _ => unreachable!(), }; self.state.commit_sample::(self.id, bounds)?; self.state.done.notified().await; - self.scatter().await + self.step = Step::Scattered(self.scatter().await?); + Ok(()) } } @@ -252,9 +262,7 @@ impl StateInner { fn determine_bounds(&mut self) -> Result<()> { let v = self.partial.drain(..).map(Option::unwrap).collect(); let bounds = Bounds::merge::(v, self.batch_rows)?; - let bounds = bounds - .reduce(self.partitions - 1, R::data_type()) - .unwrap_or(bounds); + let bounds = bounds.reduce(self.partitions - 1).unwrap_or(bounds); assert!(bounds.len() < self.partitions); self.bounds = Some(bounds); diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs index f14c124bf1364..9e0fc64b6da2a 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs @@ -105,9 +105,11 @@ where A: SortAlgorithm let subsequent = blocks .into_iter() - .map(|list| base.new_stream(Vec::from(list).into(), None)) - .collect(); - + .filter_map(|list| { + (!list.is_empty()).then(|| base.new_stream(Vec::from(list).into(), None)) + }) + .collect::>(); + debug_assert!(!subsequent.is_empty()); Self { base, step: Step::Sort(StepSort { @@ -237,6 +239,8 @@ impl StepCollect { let data = input_data.pop().unwrap(); vec![base.new_block(data)].into() } else { + // todo: using multi-threaded cascade two-way merge sorting algorithm to obtain the best performance + // also see https://arxiv.org/pdf/1406.2628 let mut merger = create_memory_merger::( input_data, base.schema.clone(), @@ -500,6 +504,7 @@ impl Base { blocks: VecDeque, bound: Option, ) -> BoundBlockStream> { + assert!(!blocks.is_empty()); BoundBlockStream { blocks, bound, diff --git a/src/query/settings/src/settings_default.rs b/src/query/settings/src/settings_default.rs index 29e3c7c1ae012..808cb863f0079 100644 --- a/src/query/settings/src/settings_default.rs +++ b/src/query/settings/src/settings_default.rs @@ -654,12 +654,12 @@ impl DefaultSettings { scope: SettingScope::Both, range: Some(SettingRange::Numeric(4 * 1024..=u64::MAX)), }), - ("range_shuffle_sort_simple_size", DefaultSettingValue { - value: UserSettingValue::UInt64(0), - desc: "Sets the simple size per partition used for range shuffle sorting, 0 to disable range shuffle sorting.", + ("enable_range_shuffle_sort", DefaultSettingValue { + value: UserSettingValue::UInt64(1), + desc: "Enable range shuffle sort.", mode: SettingMode::Both, scope: SettingScope::Both, - range: Some(SettingRange::Numeric(0..=500)), + range: Some(SettingRange::Numeric(0..=1)), }), ("group_by_shuffle_mode", DefaultSettingValue { value: UserSettingValue::String(String::from("before_merge")), diff --git a/src/query/settings/src/settings_getter_setter.rs b/src/query/settings/src/settings_getter_setter.rs index bc2b1b661b0dc..60fcfda2a3c49 100644 --- a/src/query/settings/src/settings_getter_setter.rs +++ b/src/query/settings/src/settings_getter_setter.rs @@ -512,8 +512,8 @@ impl Settings { Ok(self.try_get_u64("sort_spilling_to_disk_bytes_limit")? as usize) } - pub fn get_range_shuffle_sort_simple_size(&self) -> Result { - Ok(self.try_get_u64("range_shuffle_sort_simple_size")? as usize) + pub fn get_enable_range_shuffle_sort(&self) -> Result { + Ok(self.try_get_u64("enable_range_shuffle_sort")? == 1) } pub fn get_group_by_shuffle_mode(&self) -> Result { From 2777a28528cc31dfe8f82c00b562ede4e8db58ec Mon Sep 17 00:00:00 2001 From: coldWater Date: Fri, 25 Apr 2025 15:54:25 +0800 Subject: [PATCH 24/61] fix Signed-off-by: coldWater --- src/query/pipeline/core/src/processors/mod.rs | 1 - .../pipeline/core/src/processors/shuffle_processor.rs | 10 +--------- 2 files changed, 1 insertion(+), 10 deletions(-) diff --git a/src/query/pipeline/core/src/processors/mod.rs b/src/query/pipeline/core/src/processors/mod.rs index add24f802c251..b9468d09fb984 100644 --- a/src/query/pipeline/core/src/processors/mod.rs +++ b/src/query/pipeline/core/src/processors/mod.rs @@ -41,6 +41,5 @@ pub use resize_processor::ResizeProcessor; pub use sequence_group::SequenceGroupProcessor; 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 dac49ea50b79e..2b57c3b3cc333 100644 --- a/src/query/pipeline/core/src/processors/shuffle_processor.rs +++ b/src/query/pipeline/core/src/processors/shuffle_processor.rs @@ -345,10 +345,7 @@ impl Processor for MergePartitionProcessor { input.set_need_data(); } - if all_inputs_finished - && (!matches!(T::STRATEGY, MultiwayStrategy::Custom) - || self.inputs_data.iter().all(Option::is_none)) - { + if all_inputs_finished { self.output.finish(); return Ok(Event::Finished); } @@ -360,11 +357,6 @@ impl Processor for MergePartitionProcessor { self.output.push_data(Ok(block)); return Ok(Event::NeedConsume); } - - if all_inputs_finished && self.inputs_data.iter().all(Option::is_none) { - self.output.finish(); - return Ok(Event::Finished); - } } Ok(Event::NeedData) From ffc888117e46e8c5c570e0b359e2760904ac76d2 Mon Sep 17 00:00:00 2001 From: coldWater Date: Fri, 25 Apr 2025 17:16:13 +0800 Subject: [PATCH 25/61] fix Signed-off-by: coldWater --- src/query/service/src/pipelines/builders/builder_sort.rs | 3 ++- .../pipelines/processors/transforms/sort/sort_builder.rs | 6 +++++- .../pipelines/processors/transforms/sort/sort_shuffle.rs | 1 - 3 files changed, 7 insertions(+), 3 deletions(-) diff --git a/src/query/service/src/pipelines/builders/builder_sort.rs b/src/query/service/src/pipelines/builders/builder_sort.rs index 314a772503dca..c66ce13cde9d2 100644 --- a/src/query/service/src/pipelines/builders/builder_sort.rs +++ b/src/query/service/src/pipelines/builders/builder_sort.rs @@ -258,7 +258,8 @@ impl SortPipelineBuilder { Ok(ProcessorPtr::create(builder.build_collect(input, output)?)) })?; - let state = SortSampleState::new(inputs, max_threads, self.schema.clone(), max_block_size); + let state = + SortSampleState::new(inputs, max_threads, builder.inner_schema(), max_block_size); builder.add_shuffle(pipeline, state.clone())?; diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs index 1eb032fb1bdc3..4bfbe9f9646ff 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs @@ -218,7 +218,7 @@ impl TransformSortBuilder { } fn new_base(&self) -> Base { - let schema = add_order_field(self.schema.clone(), &self.sort_desc); + let schema = self.inner_schema(); let sort_row_offset = schema.fields().len() - 1; Base { sort_row_offset, @@ -228,6 +228,10 @@ impl TransformSortBuilder { } } + pub fn inner_schema(&self) -> DataSchemaRef { + add_order_field(self.schema.clone(), &self.sort_desc) + } + pub fn add_shuffle(&self, pipeline: &mut Pipeline, state: Arc) -> Result<()> { use std::sync::atomic; let i = atomic::AtomicUsize::new(0); diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_shuffle.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_shuffle.rs index 8f6cbb6155f42..f453851d89cec 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_shuffle.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_shuffle.rs @@ -251,7 +251,6 @@ impl SortSampleState { struct StateInner { // target partitions partitions: usize, - // schema for bounds DataBlock schema: DataSchemaRef, partial: Vec>, bounds: Option, From bd484ceb0184387638691fb8b596b336a9c609b9 Mon Sep 17 00:00:00 2001 From: coldWater Date: Fri, 25 Apr 2025 17:50:05 +0800 Subject: [PATCH 26/61] fix Signed-off-by: coldWater --- tests/sqllogictests/suites/mode/standalone/explain/sort.test | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/sqllogictests/suites/mode/standalone/explain/sort.test b/tests/sqllogictests/suites/mode/standalone/explain/sort.test index dba340cebc735..ec3825fcf9011 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/sort.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/sort.test @@ -1,6 +1,9 @@ statement ok create or replace table t1(a int, b int); +statement ok +set enable_range_shuffle_sort = 0; + query T explain select a from (select * from t1 order by a) as t2 where a > 1; ---- From a87985463ce946207796b48ded363971f0fc7f07 Mon Sep 17 00:00:00 2001 From: coldWater Date: Fri, 25 Apr 2025 18:04:06 +0800 Subject: [PATCH 27/61] fix Signed-off-by: coldWater --- .../sqllogictests/suites/stage/formats/parquet/read_policy.test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/sqllogictests/suites/stage/formats/parquet/read_policy.test b/tests/sqllogictests/suites/stage/formats/parquet/read_policy.test index df51ce3bb1325..bc32805e36ad3 100644 --- a/tests/sqllogictests/suites/stage/formats/parquet/read_policy.test +++ b/tests/sqllogictests/suites/stage/formats/parquet/read_policy.test @@ -122,7 +122,7 @@ select id, t:a from @data/parquet/tuple.parquet where t:a > 1; # topk does not contain output query TT -select id, t:b from @data/parquet/tuple.parquet order by t:a desc limit 2; +select id, t:b from @data/parquet/tuple.parquet order by t:a desc, id desc limit 2; ---- 3 c 2 b From 1c98224bfc875422f45e43cc45b8dae916944d0d Mon Sep 17 00:00:00 2001 From: coldWater Date: Fri, 25 Apr 2025 23:05:42 +0800 Subject: [PATCH 28/61] update Signed-off-by: coldWater --- .../processors/transforms/sort/bounds.rs | 92 ++++++++++++++++++- .../transforms/sort/sort_shuffle.rs | 31 ++++++- .../processors/transforms/sort/sort_spill.rs | 2 +- .../mode/standalone/explain/window.test | 3 + .../stage/formats/parquet/read_policy.test | 2 +- 5 files changed, 121 insertions(+), 9 deletions(-) diff --git a/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs b/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs index b9ee5cd80437b..4e39779b92da5 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs @@ -13,7 +13,6 @@ // limitations under the License. use databend_common_exception::Result; -use databend_common_expression::types::DataType; use databend_common_expression::Column; use databend_common_expression::DataBlock; use databend_common_expression::DataField; @@ -31,6 +30,10 @@ pub struct Bounds( ); impl Bounds { + pub fn new_unchecked(column: Column) -> Bounds { + Bounds(vec![column]) + } + pub fn from_column(column: Column) -> Result { let block = DataBlock::sort( &DataBlock::new_from_columns(vec![column]), @@ -100,12 +103,12 @@ impl Bounds { if n == 0 { return Some(Self::default()); } - let count = self.len(); - if n >= count { + let total = self.len(); + if n >= total { return None; } - let step = count / n; + let step = total / n; let offset = step / 2; let indices = self .0 @@ -130,6 +133,55 @@ impl Bounds { indices.len(), )])) } + + pub fn dedup_reduce(&self, n: usize) -> Self { + if n == 0 { + return Self::default(); + } + let total = self.len(); + let mut step = total as f64 / n as f64; + let mut target = step / 2.0; + let mut indices = Vec::with_capacity(n); + let mut last: Option<(R, _)> = None; + for (i, (b_idx, r_idx)) in self + .0 + .iter() + .enumerate() + .rev() + .flat_map(|(b_idx, col)| std::iter::repeat_n(b_idx, col.len()).zip(0..col.len())) + .enumerate() + { + if indices.len() >= n { + break; + } + if (i as f64) < target { + continue; + } + + let cur_rows = R::from_column(&self.0[b_idx]).unwrap(); + if last + .as_ref() + .map(|(last_rows, last_idx)| cur_rows.row(r_idx) == last_rows.row(*last_idx)) + .unwrap_or_default() + { + continue; + } + + indices.push((b_idx as u32, r_idx as u32, 1)); + target += step; + if (i as f64) > target && indices.len() < n { + step = (total - i) as f64 / (n - indices.len()) as f64; + target = i as f64 + step / 2.0; + } + last = Some((cur_rows, r_idx)); + } + + Bounds(vec![Column::take_column_indices( + &self.0, + &indices, + indices.len(), + )]) + } } impl SortedStream for Bounds { @@ -146,7 +198,6 @@ impl SortedStream for Bounds { #[cfg(test)] mod tests { - use databend_common_expression::types::ArgType; use databend_common_expression::types::Int32Type; use databend_common_expression::FromData; use databend_common_pipeline_transforms::sort::SimpleRowsAsc; @@ -232,4 +283,35 @@ mod tests { Ok(()) } + + #[test] + fn test_dedup_reduce() -> Result<()> { + let column = Int32Type::from_data(vec![1, 2, 2, 3, 3, 3, 4, 5, 5]); + let bounds = Bounds::new_unchecked(column); + let reduced = bounds.dedup_reduce::>(3); + assert_eq!(reduced, Bounds(vec![Int32Type::from_data(vec![2, 3, 5])])); + + let column = Int32Type::from_data(vec![5, 5, 4, 3, 3, 3, 2, 2, 1]); + let bounds = Bounds::new_unchecked(column); + let reduced = bounds.dedup_reduce::>(3); + assert_eq!(reduced, Bounds(vec![Int32Type::from_data(vec![4, 3, 1])])); + + let bounds_vec = [vec![5, 6, 7, 7], vec![3, 3, 4, 5], vec![1, 2, 2, 3]] + .into_iter() + .map(Int32Type::from_data) + .collect::>(); + let bounds = Bounds(bounds_vec); + let reduced = bounds.dedup_reduce::>(5); + assert_eq!( + reduced, + Bounds(vec![Int32Type::from_data(vec![2, 3, 4, 6, 7])]) + ); + + let column = Int32Type::from_data(vec![1, 1, 1, 1, 1]); + let bounds = Bounds(vec![column]); + let reduced = bounds.dedup_reduce::>(3); + assert_eq!(reduced, Bounds(vec![Int32Type::from_data(vec![1])])); + + Ok(()) + } } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_shuffle.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_shuffle.rs index f453851d89cec..41c4ae5360132 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_shuffle.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_shuffle.rs @@ -191,7 +191,7 @@ impl Processor for TransformSortShuffle { async fn async_process(&mut self) -> Result<()> { let bounds = match &self.step { Step::None if self.input.is_finished() => Bounds::default(), - Step::Meta(meta) => meta.bounds.clone(), + Step::Meta(meta) => meta.generate_bounds(), _ => unreachable!(), }; self.state.commit_sample::(self.id, bounds)?; @@ -201,6 +201,27 @@ impl Processor for TransformSortShuffle { } } +impl SortCollectedMeta { + fn generate_bounds(&self) -> Bounds { + if self.bounds.len() > 1 { + return self.bounds.clone(); + } + + let Some(blocks) = self.blocks.get(self.blocks.len() / 2) else { + return Bounds::default(); + }; + + blocks + .get(blocks.len() / 2) + .map(|block| match block.domain.len() { + 0 => Bounds::default(), + 1 => Bounds::new_unchecked(block.domain.clone()), + _ => Bounds::new_unchecked(block.domain.slice(0..1)), + }) + .unwrap_or_default() + } +} + pub struct SortSampleState { inner: RwLock, pub(super) done: WatchNotify, @@ -261,7 +282,13 @@ impl StateInner { fn determine_bounds(&mut self) -> Result<()> { let v = self.partial.drain(..).map(Option::unwrap).collect(); let bounds = Bounds::merge::(v, self.batch_rows)?; - let bounds = bounds.reduce(self.partitions - 1).unwrap_or(bounds); + + let n = self.partitions - 1; + let bounds = if bounds.len() < n { + bounds + } else { + bounds.dedup_reduce::(n) + }; assert!(bounds.len() < self.partitions); self.bounds = Some(bounds); diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs index 9e0fc64b6da2a..87eddb9729e81 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs @@ -602,7 +602,7 @@ pub struct SpillableBlock { data: Option, rows: usize, location: Option, - domain: Column, + pub(super) domain: Column, processed: usize, } diff --git a/tests/sqllogictests/suites/mode/standalone/explain/window.test b/tests/sqllogictests/suites/mode/standalone/explain/window.test index 9846febbd9819..9c109e6ffa369 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/window.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/window.test @@ -50,6 +50,9 @@ set sort_spilling_memory_ratio = 0; statement ok set enable_parallel_multi_merge_sort = 0; +statement ok +set enable_range_shuffle_sort = 0; + query T explain pipeline SELECT depname, empno, salary, sum(salary) OVER (PARTITION BY depname ORDER BY empno) FROM empsalary ORDER BY depname, empno; ---- diff --git a/tests/sqllogictests/suites/stage/formats/parquet/read_policy.test b/tests/sqllogictests/suites/stage/formats/parquet/read_policy.test index bc32805e36ad3..075d2f24d28b9 100644 --- a/tests/sqllogictests/suites/stage/formats/parquet/read_policy.test +++ b/tests/sqllogictests/suites/stage/formats/parquet/read_policy.test @@ -135,7 +135,7 @@ select t, t:a from @data/parquet/tuple.parquet order by id desc limit 2; # topk contains output query TT -select id, t:b, t:a from @data/parquet/tuple.parquet order by t:a desc limit 2; +select id, t:b, t:a from @data/parquet/tuple.parquet order by t:a desc, id desc limit 2; ---- 3 c 3 2 b 3 From 885fe7b33b86043124152b87f8ab89f36261bc06 Mon Sep 17 00:00:00 2001 From: coldWater Date: Sat, 26 Apr 2025 18:12:46 +0800 Subject: [PATCH 29/61] fix Signed-off-by: coldWater --- .../processors/transforms/sort/bounds.rs | 55 +++++++++---------- .../transforms/sort/sort_collect.rs | 4 ++ 2 files changed, 29 insertions(+), 30 deletions(-) diff --git a/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs b/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs index 4e39779b92da5..dca8038f3315f 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs @@ -205,6 +205,11 @@ mod tests { use super::*; + fn int32_columns(data: T) -> Vec + where T: IntoIterator> { + data.into_iter().map(Int32Type::from_data).collect() + } + #[test] fn test_merge() -> Result<()> { { @@ -227,17 +232,12 @@ mod tests { assert_eq!( bounds, - Bounds(vec![ - Int32Type::from_data(vec![6, 7]), - Int32Type::from_data(vec![2, 6, 6]), - Int32Type::from_data(vec![0, 0, 1]), - ]) + Bounds(int32_columns([vec![6, 7], vec![2, 6, 6], vec![0, 0, 1]])) ); } { - let data = vec![vec![77, -2, 7], vec![3, 8, 6, 1, 1], vec![2]]; - + let data = [vec![77, -2, 7], vec![3, 8, 6, 1, 1], vec![2]]; let data = data .into_iter() .map(|v| Bounds::from_column::>(Int32Type::from_data(v))) @@ -246,13 +246,13 @@ mod tests { assert_eq!( bounds, - Bounds(vec![ - Int32Type::from_data(vec![-2]), - Int32Type::from_data(vec![1, 1]), - Int32Type::from_data(vec![3, 2]), - Int32Type::from_data(vec![7, 6]), - Int32Type::from_data(vec![77, 8]), - ]) + Bounds(int32_columns([ + vec![-2], + vec![1, 1], + vec![3, 2], + vec![7, 6], + vec![77, 8] + ])) ); } @@ -270,16 +270,16 @@ mod tests { let bounds = Bounds::merge::>(data, 2)?; let got = bounds.reduce(4).unwrap(); - assert_eq!(got, Bounds(vec![Int32Type::from_data(vec![8, 6, 2, 1])])); // 77 _8 7 _6 3 _2 1 _1 -2 + assert_eq!(got, Bounds(int32_columns([vec![8, 6, 2, 1]]))); // 77 _8 7 _6 3 _2 1 _1 -2 let got = bounds.reduce(3).unwrap(); - assert_eq!(got, Bounds(vec![Int32Type::from_data(vec![8, 3, 1])])); // 77 _8 7 6 _3 2 1 _1 -2 + assert_eq!(got, Bounds(int32_columns([vec![8, 3, 1]]))); // 77 _8 7 6 _3 2 1 _1 -2 let got = bounds.reduce(2).unwrap(); - assert_eq!(got, Bounds(vec![Int32Type::from_data(vec![7, 1])])); // 77 8 _7 6 3 2 _1 1 -2 + assert_eq!(got, Bounds(int32_columns([vec![7, 1]]))); // 77 8 _7 6 3 2 _1 1 -2 let got = bounds.reduce(1).unwrap(); - assert_eq!(got, Bounds(vec![Int32Type::from_data(vec![3])])); // 77 8 7 6 _3 2 1 1 -2 + assert_eq!(got, Bounds(int32_columns([vec![3]]))); // 77 8 7 6 _3 2 1 1 -2 Ok(()) } @@ -289,28 +289,23 @@ mod tests { let column = Int32Type::from_data(vec![1, 2, 2, 3, 3, 3, 4, 5, 5]); let bounds = Bounds::new_unchecked(column); let reduced = bounds.dedup_reduce::>(3); - assert_eq!(reduced, Bounds(vec![Int32Type::from_data(vec![2, 3, 5])])); + assert_eq!(reduced, Bounds(int32_columns([vec![2, 3, 5]]))); let column = Int32Type::from_data(vec![5, 5, 4, 3, 3, 3, 2, 2, 1]); let bounds = Bounds::new_unchecked(column); let reduced = bounds.dedup_reduce::>(3); - assert_eq!(reduced, Bounds(vec![Int32Type::from_data(vec![4, 3, 1])])); + assert_eq!(reduced, Bounds(int32_columns([vec![4, 3, 1]]))); - let bounds_vec = [vec![5, 6, 7, 7], vec![3, 3, 4, 5], vec![1, 2, 2, 3]] - .into_iter() - .map(Int32Type::from_data) - .collect::>(); - let bounds = Bounds(bounds_vec); + let bounds = Bounds(int32_columns([vec![5, 6, 7, 7], vec![3, 3, 4, 5], vec![ + 1, 2, 2, 3, + ]])); let reduced = bounds.dedup_reduce::>(5); - assert_eq!( - reduced, - Bounds(vec![Int32Type::from_data(vec![2, 3, 4, 6, 7])]) - ); + assert_eq!(reduced, Bounds(int32_columns([vec![2, 3, 4, 6, 7]]))); let column = Int32Type::from_data(vec![1, 1, 1, 1, 1]); let bounds = Bounds(vec![column]); let reduced = bounds.dedup_reduce::>(3); - assert_eq!(reduced, Bounds(vec![Int32Type::from_data(vec![1])])); + assert_eq!(reduced, Bounds(int32_columns([vec![1]]))); Ok(()) } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_collect.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_collect.rs index b8546d32aebbd..6a399dea7b31b 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_collect.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_collect.rs @@ -157,6 +157,10 @@ where let unit_size = self.memory_settings.spill_unit_size; let num_merge = bytes.div_ceil(unit_size).max(2); let batch_rows = rows.div_ceil(num_merge); + + /// The memory will be doubled during merging. + const MERGE_RATIO: usize = 2; + let num_merge = num_merge.div_ceil(MERGE_RATIO).max(2); log::info!("determine sort spill params, buffer_bytes: {bytes}, buffer_rows: {rows}, spill_unit_size: {unit_size}, batch_rows: {batch_rows}, batch_num_merge {num_merge}"); SortSpillParams { batch_rows, From f7a0373a2ef3ccbd5d9396764ce4743aa4d54b09 Mon Sep 17 00:00:00 2001 From: coldWater Date: Sun, 27 Apr 2025 16:33:00 +0800 Subject: [PATCH 30/61] fix Signed-off-by: coldWater --- .../src/pipelines/builders/builder_sort.rs | 7 ++-- .../processors/transforms/sort/merge_sort.rs | 2 +- .../transforms/sort/sort_collect.rs | 37 ++++++++++++++----- .../processors/transforms/sort/sort_spill.rs | 2 +- 4 files changed, 32 insertions(+), 16 deletions(-) diff --git a/src/query/service/src/pipelines/builders/builder_sort.rs b/src/query/service/src/pipelines/builders/builder_sort.rs index c66ce13cde9d2..f205d5c33675e 100644 --- a/src/query/service/src/pipelines/builders/builder_sort.rs +++ b/src/query/service/src/pipelines/builders/builder_sort.rs @@ -216,7 +216,7 @@ impl SortPipelineBuilder { fn build_range_shuffle_sort_pipeline(self, pipeline: &mut Pipeline) -> Result<()> { let inputs = pipeline.output_len(); let settings = self.ctx.get_settings(); - let max_threads = settings.get_max_threads()? as usize; + let num_exec = inputs; let max_block_size = settings.get_max_block_size()? as usize; // Partial sort @@ -258,12 +258,11 @@ impl SortPipelineBuilder { Ok(ProcessorPtr::create(builder.build_collect(input, output)?)) })?; - let state = - SortSampleState::new(inputs, max_threads, builder.inner_schema(), max_block_size); + let state = SortSampleState::new(inputs, num_exec, builder.inner_schema(), max_block_size); builder.add_shuffle(pipeline, state.clone())?; - pipeline.exchange(max_threads, Arc::new(SortRangeExchange)); + pipeline.exchange(num_exec, Arc::new(SortRangeExchange)); pipeline.add_transform(|input, output| { Ok(ProcessorPtr::create(builder.build_combine(input, output)?)) diff --git a/src/query/service/src/pipelines/processors/transforms/sort/merge_sort.rs b/src/query/service/src/pipelines/processors/transforms/sort/merge_sort.rs index c39c520f2c615..5c07cee78f795 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/merge_sort.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/merge_sort.rs @@ -430,7 +430,7 @@ where if memory_rows > 0 && memory_rows + input > max { spill_sort - .subsequent_spill_last(memory_rows + input - max) + .collect_spill_last(memory_rows + input - max) .await?; } if input > max || finished && input > 0 { diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_collect.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_collect.rs index 6a399dea7b31b..cf3be6e467fbe 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_collect.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_collect.rs @@ -49,6 +49,7 @@ pub struct TransformSortCollect { output: Arc, output_data: Option, + max_block_size: usize, row_converter: C, sort_desc: Arc<[SortColumnDescription]>, /// If this transform is after an Exchange transform, @@ -100,6 +101,7 @@ where inner, aborting: AtomicBool::new(false), memory_settings, + max_block_size, }) } @@ -111,12 +113,19 @@ where Ok((rows, block)) } - fn limit_trans_to_spill(&mut self) -> Result<()> { + fn limit_trans_to_spill(&mut self, no_spill: bool) -> Result<()> { let Inner::Limit(merger) = &self.inner else { unreachable!() }; assert!(merger.num_rows() > 0); - let params = self.determine_params(merger.num_bytes(), merger.num_rows()); + let params = if no_spill { + SortSpillParams { + batch_rows: self.max_block_size, + num_merge: merger.num_rows().div_ceil(self.max_block_size), + } + } else { + self.determine_params(merger.num_bytes(), merger.num_rows()) + }; let Inner::Limit(merger) = &mut self.inner else { unreachable!() }; @@ -126,7 +135,7 @@ where Ok(()) } - fn collect_trans_to_spill(&mut self, input_data: Vec) { + fn collect_trans_to_spill(&mut self, input_data: Vec, no_spill: bool) { let (num_rows, num_bytes) = input_data .iter() .map(|block| (block.num_rows(), block.memory_size())) @@ -134,17 +143,24 @@ where (acc_rows + rows, acc_bytes + bytes) }); assert!(num_rows > 0); - let params = self.determine_params(num_bytes, num_rows); + let params = if no_spill { + SortSpillParams { + batch_rows: self.max_block_size, + num_merge: num_rows.div_ceil(self.max_block_size), + } + } else { + self.determine_params(num_bytes, num_rows) + }; let spill_sort = SortSpill::new(self.base.clone(), params); self.inner = Inner::Spill(input_data, spill_sort); } - fn trans_to_spill(&mut self) -> Result<()> { + fn trans_to_spill(&mut self, no_spill: bool) -> Result<()> { match &mut self.inner { - Inner::Limit(_) => self.limit_trans_to_spill(), + Inner::Limit(_) => self.limit_trans_to_spill(no_spill), Inner::Collect(input_data) => { let input_data = std::mem::take(input_data); - self.collect_trans_to_spill(input_data); + self.collect_trans_to_spill(input_data, no_spill); Ok(()) } Inner::Spill(_, _) => Ok(()), @@ -316,18 +332,19 @@ where #[async_backtrace::framed] async fn async_process(&mut self) -> Result<()> { let finished = self.input.is_finished(); - self.trans_to_spill()?; + self.trans_to_spill(finished)?; - let input = self.input_rows(); let Inner::Spill(input_data, spill_sort) = &mut self.inner else { unreachable!() }; + + let input = input_data.in_memory_rows(); let memory_rows = spill_sort.collect_memory_rows(); let max = spill_sort.max_rows(); if memory_rows > 0 && memory_rows + input > max { spill_sort - .subsequent_spill_last(memory_rows + input - max) + .collect_spill_last(memory_rows + input - max) .await?; } if input > max || finished && input > 0 { diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs index 87eddb9729e81..5a7df43b383b8 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs @@ -136,7 +136,7 @@ where A: SortAlgorithm .await } - pub async fn subsequent_spill_last(&mut self, target_rows: usize) -> Result<()> { + pub async fn collect_spill_last(&mut self, target_rows: usize) -> Result<()> { let Step::Collect(collect) = &mut self.step else { unreachable!() }; From eb83630b16e72b4257ed33bf7660d7f1d31ae70a Mon Sep 17 00:00:00 2001 From: coldWater Date: Sun, 27 Apr 2025 17:22:16 +0800 Subject: [PATCH 31/61] fix Signed-off-by: coldWater --- .../processors/transforms/sort/sort_collect.rs | 11 ++--------- .../processors/transforms/sort/sort_spill.rs | 4 ++-- src/query/settings/src/settings_default.rs | 2 +- .../suites/mode/standalone/explain/sort.test | 3 --- .../suites/mode/standalone/explain/window.test | 3 --- 5 files changed, 5 insertions(+), 18 deletions(-) diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_collect.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_collect.rs index cf3be6e467fbe..ea29b4d2048c7 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_collect.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_collect.rs @@ -121,7 +121,7 @@ where let params = if no_spill { SortSpillParams { batch_rows: self.max_block_size, - num_merge: merger.num_rows().div_ceil(self.max_block_size), + num_merge: merger.num_rows().div_ceil(self.max_block_size).max(2), } } else { self.determine_params(merger.num_bytes(), merger.num_rows()) @@ -146,7 +146,7 @@ where let params = if no_spill { SortSpillParams { batch_rows: self.max_block_size, - num_merge: num_rows.div_ceil(self.max_block_size), + num_merge: num_rows.div_ceil(self.max_block_size).max(2), } } else { self.determine_params(num_bytes, num_rows) @@ -210,13 +210,6 @@ where } } - fn input_rows(&self) -> usize { - match &self.inner { - Inner::Collect(input_data) | Inner::Spill(input_data, _) => input_data.in_memory_rows(), - _ => 0, - } - } - fn check_spill(&self) -> bool { if !self.memory_settings.check_spill() { return false; diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs index 5a7df43b383b8..74e8965dfe3b4 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs @@ -963,7 +963,7 @@ mod tests { ] .into_iter() .map(|mut data| { - let col = convert_rows(schema.clone(), &sort_desc, data.clone()).unwrap(); + let col = convert_rows(schema.clone(), sort_desc, data.clone()).unwrap(); data.add_column(col); SpillableBlock::new(data, sort_row_offset) @@ -1046,7 +1046,7 @@ mod tests { ) -> SpillableBlock { let mut sliced_block = block.slice(range); let col = convert_rows(schema.clone(), sort_desc, sliced_block.clone()).unwrap(); - sliced_block.add_column(BlockEntry::new(col.data_type(), Value::Column(col))); + sliced_block.add_column(col); SpillableBlock::new(sliced_block, sort_row_offset) } diff --git a/src/query/settings/src/settings_default.rs b/src/query/settings/src/settings_default.rs index 808cb863f0079..6aa5701ebd525 100644 --- a/src/query/settings/src/settings_default.rs +++ b/src/query/settings/src/settings_default.rs @@ -655,7 +655,7 @@ impl DefaultSettings { range: Some(SettingRange::Numeric(4 * 1024..=u64::MAX)), }), ("enable_range_shuffle_sort", DefaultSettingValue { - value: UserSettingValue::UInt64(1), + value: UserSettingValue::UInt64(0), desc: "Enable range shuffle sort.", mode: SettingMode::Both, scope: SettingScope::Both, diff --git a/tests/sqllogictests/suites/mode/standalone/explain/sort.test b/tests/sqllogictests/suites/mode/standalone/explain/sort.test index ec3825fcf9011..dba340cebc735 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/sort.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/sort.test @@ -1,9 +1,6 @@ statement ok create or replace table t1(a int, b int); -statement ok -set enable_range_shuffle_sort = 0; - query T explain select a from (select * from t1 order by a) as t2 where a > 1; ---- diff --git a/tests/sqllogictests/suites/mode/standalone/explain/window.test b/tests/sqllogictests/suites/mode/standalone/explain/window.test index 9c109e6ffa369..b035741df72e9 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/window.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/window.test @@ -47,9 +47,6 @@ set max_threads=4; statement ok set sort_spilling_memory_ratio = 0; -statement ok -set enable_parallel_multi_merge_sort = 0; - statement ok set enable_range_shuffle_sort = 0; From 5330273a7e5128a36c449b12ae0a0c871b0bc45c Mon Sep 17 00:00:00 2001 From: coldWater Date: Mon, 30 Jun 2025 17:14:58 +0800 Subject: [PATCH 32/61] x --- .../interpreters/interpreter_table_analyze.rs | 46 ++-------- .../pipelines/builders/builder_aggregate.rs | 2 +- .../src/pipelines/builders/builder_sort.rs | 16 ++-- .../src/pipelines/builders/builder_window.rs | 7 +- .../processors/transforms/sort/mod.rs | 1 + .../transforms/sort/sort_exchange_injector.rs | 85 +++++++++++++++++++ .../sql/planner/optimizer/optimizer_test.rs | 11 ++- .../sql/src/executor/physical_plan_visitor.rs | 13 +-- .../sql/src/executor/physical_plans/mod.rs | 2 +- .../executor/physical_plans/physical_sort.rs | 40 +++++++-- .../physical_window_partition.rs | 3 +- .../src/planner/optimizer/ir/expr/s_expr.rs | 5 ++ .../planner/optimizer/optimizer_context.rs | 68 +++++++-------- .../optimizers/distributed/sort_and_limit.rs | 60 +++++++------ .../rule/agg_rules/rule_split_aggregate.rs | 27 ++++-- src/query/sql/src/planner/planner.rs | 5 +- src/query/sql/src/planner/plans/exchange.rs | 3 +- src/query/sql/src/planner/plans/operator.rs | 3 +- src/query/sql/src/planner/plans/recluster.rs | 4 +- src/query/sql/src/planner/plans/sort.rs | 2 - 20 files changed, 251 insertions(+), 152 deletions(-) create mode 100644 src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs diff --git a/src/query/service/src/interpreters/interpreter_table_analyze.rs b/src/query/service/src/interpreters/interpreter_table_analyze.rs index 3d784a9ad7cb0..bbd63893b33e4 100644 --- a/src/query/service/src/interpreters/interpreter_table_analyze.rs +++ b/src/query/service/src/interpreters/interpreter_table_analyze.rs @@ -283,66 +283,34 @@ fn remove_exchange(plan: PhysicalPlan) -> PhysicalPlan { fn traverse(plan: PhysicalPlan) -> PhysicalPlan { match plan { PhysicalPlan::Filter(plan) => PhysicalPlan::Filter(Filter { - plan_id: plan.plan_id, - projections: plan.projections, input: Box::new(traverse(*plan.input)), - predicates: plan.predicates, - stat_info: plan.stat_info, + ..plan }), PhysicalPlan::EvalScalar(plan) => PhysicalPlan::EvalScalar(EvalScalar { - plan_id: plan.plan_id, - projections: plan.projections, input: Box::new(traverse(*plan.input)), - exprs: plan.exprs, - stat_info: plan.stat_info, + ..plan }), PhysicalPlan::AggregateExpand(plan) => PhysicalPlan::AggregateExpand(AggregateExpand { - plan_id: plan.plan_id, input: Box::new(traverse(*plan.input)), - group_bys: plan.group_bys, - grouping_sets: plan.grouping_sets, - stat_info: plan.stat_info, + ..plan }), PhysicalPlan::AggregatePartial(plan) => { PhysicalPlan::AggregatePartial(AggregatePartial { - plan_id: plan.plan_id, input: Box::new(traverse(*plan.input)), - group_by: plan.group_by, - agg_funcs: plan.agg_funcs, - rank_limit: plan.rank_limit, - enable_experimental_aggregate_hashtable: plan - .enable_experimental_aggregate_hashtable, - group_by_display: plan.group_by_display, - stat_info: plan.stat_info, + ..plan }) } PhysicalPlan::AggregateFinal(plan) => PhysicalPlan::AggregateFinal(AggregateFinal { - plan_id: plan.plan_id, input: Box::new(traverse(*plan.input)), - group_by: plan.group_by, - agg_funcs: plan.agg_funcs, - before_group_by_schema: plan.before_group_by_schema, - group_by_display: plan.group_by_display, - stat_info: plan.stat_info, + ..plan }), PhysicalPlan::Window(plan) => PhysicalPlan::Window(Window { - plan_id: plan.plan_id, - index: plan.index, input: Box::new(traverse(*plan.input)), - func: plan.func, - partition_by: plan.partition_by, - order_by: plan.order_by, - window_frame: plan.window_frame, - limit: plan.limit, + ..plan }), PhysicalPlan::Sort(plan) => PhysicalPlan::Sort(Sort { - plan_id: plan.plan_id, input: Box::new(traverse(*plan.input)), - order_by: plan.order_by, - limit: plan.limit, - after_exchange: plan.after_exchange, - pre_projection: plan.pre_projection, - stat_info: plan.stat_info, + ..plan }), PhysicalPlan::Exchange(plan) => traverse(*plan.input), _ => plan, diff --git a/src/query/service/src/pipelines/builders/builder_aggregate.rs b/src/query/service/src/pipelines/builders/builder_aggregate.rs index e36bffb214907..20faed431fa56 100644 --- a/src/query/service/src/pipelines/builders/builder_aggregate.rs +++ b/src/query/service/src/pipelines/builders/builder_aggregate.rs @@ -227,7 +227,7 @@ impl PipelineBuilder { if matches!(input, PhysicalPlan::ExchangeSource(_)) { self.exchange_injector = AggregateInjector::create(self.ctx.clone(), params.clone()); } - self.build_pipeline(&aggregate.input)?; + self.build_pipeline(input)?; self.exchange_injector = old_inject; build_partition_bucket(&mut self.main_pipeline, params.clone(), max_restore_worker) } diff --git a/src/query/service/src/pipelines/builders/builder_sort.rs b/src/query/service/src/pipelines/builders/builder_sort.rs index f205d5c33675e..ea35f9943d17d 100644 --- a/src/query/service/src/pipelines/builders/builder_sort.rs +++ b/src/query/service/src/pipelines/builders/builder_sort.rs @@ -29,6 +29,7 @@ use databend_common_pipeline_transforms::MemorySettings; use databend_common_sql::evaluator::BlockOperator; use databend_common_sql::evaluator::CompoundBlockOperator; use databend_common_sql::executor::physical_plans::Sort; +use databend_common_sql::executor::physical_plans::SortStep; use databend_common_storage::DataOperator; use databend_common_storages_fuse::TableContext; use databend_storages_common_cache::TempDirManager; @@ -53,7 +54,7 @@ impl PipelineBuilder { let input_schema = sort.input.output_schema()?; - if !matches!(sort.after_exchange, Some(true)) { + if sort.step == SortStep::FinalMerge { // If the Sort plan is after exchange, we don't need to do a projection, // because the data is already projected in each cluster node. if let Some(proj) = &sort.pre_projection { @@ -93,7 +94,7 @@ impl PipelineBuilder { }) .collect::>>()?; - self.build_sort_pipeline(plan_schema, sort_desc, sort.limit, sort.after_exchange) + self.build_sort_pipeline(plan_schema, sort_desc, sort.limit, sort.step) } fn build_sort_pipeline( @@ -101,7 +102,7 @@ impl PipelineBuilder { plan_schema: DataSchemaRef, sort_desc: Vec, limit: Option, - after_exchange: Option, + sort_step: SortStep, ) -> Result<()> { let max_threads = self.settings.get_max_threads()? as usize; let sort_desc = sort_desc.into(); @@ -114,8 +115,8 @@ impl PipelineBuilder { let builder = SortPipelineBuilder::create(self.ctx.clone(), plan_schema, sort_desc)? .with_limit(limit); - match after_exchange { - Some(true) => { + match sort_step { + SortStep::FinalMerge => { // Build for the coordinator node. // We only build a `MultiSortMergeTransform`, // as the data is already sorted in each cluster node. @@ -130,13 +131,13 @@ impl PipelineBuilder { .build_merge_sort_pipeline(&mut self.main_pipeline, true) } } - Some(false) => { + SortStep::Partial => { // Build for each cluster node. // We build the full sort pipeline for it. // Don't remove the order column at last. builder.build_full_sort_pipeline(&mut self.main_pipeline) } - None => { + SortStep::Single => { // Build for single node mode. // We build the full sort pipeline for it. if self.settings.get_enable_range_shuffle_sort()? @@ -151,6 +152,7 @@ impl PipelineBuilder { .build_full_sort_pipeline(&mut self.main_pipeline) } } + _ => unimplemented!(), } } } diff --git a/src/query/service/src/pipelines/builders/builder_window.rs b/src/query/service/src/pipelines/builders/builder_window.rs index 187bb25d7dd77..cb82feca68ca3 100644 --- a/src/query/service/src/pipelines/builders/builder_window.rs +++ b/src/query/service/src/pipelines/builders/builder_window.rs @@ -198,7 +198,12 @@ impl PipelineBuilder { .map(|temp_dir| SpillerDiskConfig::new(temp_dir, enable_dio)) .transpose()?; - let have_order_col = window_partition.after_exchange.unwrap_or(false); + let have_order_col = match window_partition.sort_step { + SortStep::Single | SortStep::Partial => false, + SortStep::FinalMerge => true, + _ => unimplemented!(), + }; + let window_spill_settings = MemorySettings::from_window_settings(&self.ctx)?; let processor_id = AtomicUsize::new(0); diff --git a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs index 3d7c57e31beb0..1242f265c3ffc 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs @@ -30,6 +30,7 @@ mod sort_builder; mod sort_collect; mod sort_combine; mod sort_exchange; +mod sort_exchange_injector; mod sort_execute; mod sort_route; mod sort_shuffle; diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs new file mode 100644 index 0000000000000..e567af11fcaa0 --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs @@ -0,0 +1,85 @@ +// 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_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_settings::FlightCompression; +use databend_common_storage::DataOperator; + +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; + +pub struct SortInjector {} + +impl ExchangeInjector for SortInjector { + fn flight_scatter( + &self, + ctx: &Arc, + exchange: &DataExchange, + ) -> Result>> { + todo!() + } + + fn exchange_sorting(&self) -> Option> { + todo!() + } + + fn apply_merge_serializer( + &self, + params: &MergeExchangeParams, + compression: Option, + pipeline: &mut Pipeline, + ) -> Result<()> { + todo!() + } + + fn apply_shuffle_serializer( + &self, + params: &ShuffleExchangeParams, + compression: Option, + pipeline: &mut Pipeline, + ) -> Result<()> { + todo!() + } + + fn apply_merge_deserializer( + &self, + params: &MergeExchangeParams, + pipeline: &mut Pipeline, + ) -> Result<()> { + todo!() + } + + fn apply_shuffle_deserializer( + &self, + params: &ShuffleExchangeParams, + pipeline: &mut Pipeline, + ) -> Result<()> { + todo!() + } +} diff --git a/src/query/service/tests/it/sql/planner/optimizer/optimizer_test.rs b/src/query/service/tests/it/sql/planner/optimizer/optimizer_test.rs index 937b45289f183..f46648e1d9eb8 100644 --- a/src/query/service/tests/it/sql/planner/optimizer/optimizer_test.rs +++ b/src/query/service/tests/it/sql/planner/optimizer/optimizer_test.rs @@ -438,12 +438,10 @@ async fn optimize_plan(ctx: Arc, plan: Plan) -> Result { _ => Arc::new(parking_lot::RwLock::new(Metadata::default())), }; + let settings = ctx.get_settings(); let opt_ctx = OptimizerContext::new(ctx, metadata) + .with_settings(&settings)? .set_enable_distributed_optimization(true) - .set_enable_join_reorder(true) - .set_enable_dphyp(true) - .set_max_push_down_limit(10000) - .set_enable_trace(true) .clone(); optimize(opt_ctx, plan).await @@ -604,6 +602,11 @@ async fn run_test_case( fn configure_optimizer(ctx: &Arc, auto_stats: bool) -> Result<()> { let settings = ctx.get_settings(); + + settings.set_setting("enable_dphyp".to_string(), "1".to_string())?; + settings.set_setting("max_push_down_limit".to_string(), "10000".to_string())?; + settings.set_setting("enable_optimizer_trace".to_string(), "1".to_string())?; + if auto_stats { settings.set_optimizer_skip_list("".to_string()) } else { diff --git a/src/query/sql/src/executor/physical_plan_visitor.rs b/src/query/sql/src/executor/physical_plan_visitor.rs index 7267b7665e370..1d9948a131f81 100644 --- a/src/query/sql/src/executor/physical_plan_visitor.rs +++ b/src/query/sql/src/executor/physical_plan_visitor.rs @@ -260,7 +260,7 @@ pub trait PhysicalPlanReplacer { input: Box::new(input), partition_by: plan.partition_by.clone(), order_by: plan.order_by.clone(), - after_exchange: plan.after_exchange, + sort_step: plan.sort_step, top_n: plan.top_n.clone(), stat_info: plan.stat_info.clone(), })) @@ -313,16 +313,11 @@ pub trait PhysicalPlanReplacer { } fn replace_sort(&mut self, plan: &Sort) -> Result { - let input = self.replace(&plan.input)?; + let input = self.replace(&plan.input)?.into(); Ok(PhysicalPlan::Sort(Sort { - plan_id: plan.plan_id, - input: Box::new(input), - order_by: plan.order_by.clone(), - limit: plan.limit, - after_exchange: plan.after_exchange, - pre_projection: plan.pre_projection.clone(), - stat_info: plan.stat_info.clone(), + input, + ..plan.clone() })) } diff --git a/src/query/sql/src/executor/physical_plans/mod.rs b/src/query/sql/src/executor/physical_plans/mod.rs index 2f2afb69d3368..edf36a64edb31 100644 --- a/src/query/sql/src/executor/physical_plans/mod.rs +++ b/src/query/sql/src/executor/physical_plans/mod.rs @@ -106,7 +106,7 @@ pub use physical_replace_async_source::ReplaceAsyncSourcer; pub use physical_replace_deduplicate::*; pub use physical_replace_into::ReplaceInto; pub use physical_row_fetch::RowFetch; -pub use physical_sort::Sort; +pub use physical_sort::*; pub use physical_table_scan::TableScan; pub use physical_udf::Udf; pub use physical_udf::UdfFunctionDesc; diff --git a/src/query/sql/src/executor/physical_plans/physical_sort.rs b/src/query/sql/src/executor/physical_plans/physical_sort.rs index 94b765107f310..cdf7ab5c0cc53 100644 --- a/src/query/sql/src/executor/physical_plans/physical_sort.rs +++ b/src/query/sql/src/executor/physical_plans/physical_sort.rs @@ -40,15 +40,29 @@ pub struct Sort { pub order_by: Vec, /// limit = Limit.limit + Limit.offset pub limit: Option, - /// If the sort plan is after the exchange plan. - /// It's [None] if the sorting plan is in single node mode. - pub after_exchange: Option, + pub step: SortStep, pub pre_projection: Option>, + pub broadcast_id: Option, // Only used for explain pub stat_info: Option, } +#[derive(Debug, Hash, Clone, Copy, serde::Serialize, serde::Deserialize, PartialEq, Eq)] +pub enum SortStep { + // single node mode + Single, + + // cluster mode + Partial, // before the exchange plan + FinalMerge, // after the exchange plan + + // range shuffle mode + Sample, + RangeSort, + Route, +} + impl Sort { fn order_col_type(&self, schema: &DataSchema) -> Result { if self.order_by.len() == 1 { @@ -66,7 +80,7 @@ impl Sort { pub fn output_schema(&self) -> Result { let input_schema = self.input.output_schema()?; let mut fields = input_schema.fields().clone(); - if matches!(self.after_exchange, Some(true)) { + if self.step == SortStep::FinalMerge { // If the plan is after exchange plan in cluster mode, // the order column is at the last of the input schema. debug_assert_eq!(fields.last().unwrap().name(), ORDER_COL_NAME); @@ -88,7 +102,7 @@ impl Sort { } } - if matches!(self.after_exchange, Some(false)) { + if self.step == SortStep::Partial { // If the plan is before exchange plan in cluster mode, // the order column should be added to the output schema. fields.push(DataField::new( @@ -134,6 +148,17 @@ impl PhysicalPlanBuilder { }) .collect::>(); + let enable_range_shuffle_sort = self.ctx.get_settings().get_enable_range_shuffle_sort()?; + let sort_step = match sort.after_exchange { + Some(false) => SortStep::Partial, + Some(true) => SortStep::FinalMerge, + None => SortStep::Single, + }; + let broadcast_id = match sort.after_exchange { + Some(false) if enable_range_shuffle_sort => Some(self.ctx.get_next_broadcast_id()), + _ => None, + }; + // Add WindowPartition for parallel sort in window. if let Some(window) = &sort.window_partition { let window_partition = window @@ -147,7 +172,7 @@ impl PhysicalPlanBuilder { input: Box::new(input_plan.clone()), partition_by: window_partition.clone(), order_by: order_by.clone(), - after_exchange: sort.after_exchange, + sort_step, top_n: window.top.map(|top| WindowPartitionTopN { func: match window.func { WindowFuncType::RowNumber => WindowPartitionTopNFunc::RowNumber, @@ -167,8 +192,9 @@ impl PhysicalPlanBuilder { input: Box::new(input_plan), order_by, limit: sort.limit, - after_exchange: sort.after_exchange, + step: sort_step, pre_projection, + broadcast_id, stat_info: Some(stat_info), })) } diff --git a/src/query/sql/src/executor/physical_plans/physical_window_partition.rs b/src/query/sql/src/executor/physical_plans/physical_window_partition.rs index b0ff12d3f8685..21fa5e013f9fb 100644 --- a/src/query/sql/src/executor/physical_plans/physical_window_partition.rs +++ b/src/query/sql/src/executor/physical_plans/physical_window_partition.rs @@ -16,6 +16,7 @@ use databend_common_exception::Result; use databend_common_expression::DataSchemaRef; use crate::executor::explain::PlanStatsInfo; +use crate::executor::physical_plans::physical_sort::SortStep; use crate::executor::physical_plans::SortDesc; use crate::executor::PhysicalPlan; use crate::IndexType; @@ -26,7 +27,7 @@ pub struct WindowPartition { pub input: Box, pub partition_by: Vec, pub order_by: Vec, - pub after_exchange: Option, + pub sort_step: SortStep, pub top_n: Option, pub stat_info: Option, diff --git a/src/query/sql/src/planner/optimizer/ir/expr/s_expr.rs b/src/query/sql/src/planner/optimizer/ir/expr/s_expr.rs index 4900fe5167077..1adf9a993d0e2 100644 --- a/src/query/sql/src/planner/optimizer/ir/expr/s_expr.rs +++ b/src/query/sql/src/planner/optimizer/ir/expr/s_expr.rs @@ -135,6 +135,11 @@ impl SExpr { &self.children[0] } + pub fn unary_child_arc(&self) -> Arc { + assert_eq!(self.children.len(), 1); + self.children[0].clone() + } + pub fn left_child(&self) -> &SExpr { assert_eq!(self.children.len(), 2); &self.children[0] diff --git a/src/query/sql/src/planner/optimizer/optimizer_context.rs b/src/query/sql/src/planner/optimizer/optimizer_context.rs index 35d682b55901f..d75f9cea101e7 100644 --- a/src/query/sql/src/planner/optimizer/optimizer_context.rs +++ b/src/query/sql/src/planner/optimizer/optimizer_context.rs @@ -16,6 +16,8 @@ use std::collections::HashMap; use std::sync::Arc; use databend_common_catalog::table_context::TableContext; +use databend_common_exception::Result; +use databend_common_settings::Settings; use educe::Educe; use parking_lot::RwLock; @@ -64,93 +66,87 @@ impl OptimizerContext { }) } - pub fn get_table_ctx(self: &Arc) -> Arc { + pub fn with_settings(self: Arc, settings: &Settings) -> Result> { + self.set_enable_join_reorder(unsafe { !settings.get_disable_join_reorder()? }); + *self.enable_dphyp.write() = settings.get_enable_dphyp()?; + *self.max_push_down_limit.write() = settings.get_max_push_down_limit()?; + *self.enable_trace.write() = settings.get_enable_optimizer_trace()?; + + Ok(self) + } + + pub fn get_table_ctx(&self) -> Arc { self.table_ctx.clone() } - pub fn get_metadata(self: &Arc) -> MetadataRef { + pub fn get_metadata(&self) -> MetadataRef { self.metadata.clone() } - pub fn set_enable_distributed_optimization(self: &Arc, enable: bool) -> Arc { + pub fn set_enable_distributed_optimization(self: &Arc, enable: bool) -> &Arc { *self.enable_distributed_optimization.write() = enable; - self.clone() + self } - pub fn get_enable_distributed_optimization(self: &Arc) -> bool { + pub fn get_enable_distributed_optimization(&self) -> bool { *self.enable_distributed_optimization.read() } - pub fn set_enable_join_reorder(self: &Arc, enable: bool) -> Arc { + fn set_enable_join_reorder(self: &Arc, enable: bool) -> &Arc { *self.enable_join_reorder.write() = enable; - self.clone() + self } - pub fn get_enable_join_reorder(self: &Arc) -> bool { + pub fn get_enable_join_reorder(&self) -> bool { *self.enable_join_reorder.read() } - pub fn set_enable_dphyp(self: &Arc, enable: bool) -> Arc { - *self.enable_dphyp.write() = enable; - self.clone() - } - - pub fn get_enable_dphyp(self: &Arc) -> bool { + pub fn get_enable_dphyp(&self) -> bool { *self.enable_dphyp.read() } pub fn set_sample_executor( self: &Arc, sample_executor: Option>, - ) -> Arc { + ) -> &Arc { *self.sample_executor.write() = sample_executor; - self.clone() + self } - pub fn get_sample_executor(self: &Arc) -> Option> { + pub fn get_sample_executor(&self) -> Option> { self.sample_executor.read().clone() } - pub fn set_planning_agg_index(self: &Arc, enable: bool) -> Arc { + pub fn set_planning_agg_index(self: &Arc, enable: bool) -> &Arc { *self.planning_agg_index.write() = enable; - self.clone() + self } - pub fn get_planning_agg_index(self: &Arc) -> bool { + pub fn get_planning_agg_index(&self) -> bool { *self.planning_agg_index.read() } - pub fn set_max_push_down_limit(self: &Arc, max_push_down_limit: usize) -> Arc { - *self.max_push_down_limit.write() = max_push_down_limit; - self.clone() - } - - pub fn get_max_push_down_limit(self: &Arc) -> usize { + pub fn get_max_push_down_limit(&self) -> usize { *self.max_push_down_limit.read() } - pub fn set_flag(self: &Arc, name: &str, value: bool) -> Arc { + pub fn set_flag(self: &Arc, name: &str, value: bool) -> &Arc { let mut flags = self.flags.write(); flags.insert(name.to_string(), value); - self.clone() + self } - pub fn get_flag(self: &Arc, name: &str) -> bool { + pub fn get_flag(&self, name: &str) -> bool { let flags = self.flags.read(); *flags.get(name).unwrap_or(&false) } - pub fn set_enable_trace(self: &Arc, enable: bool) -> Arc { - *self.enable_trace.write() = enable; - self.clone() - } - - pub fn get_enable_trace(self: &Arc) -> bool { + pub fn get_enable_trace(&self) -> bool { *self.enable_trace.read() } /// Check if an optimizer or rule is disabled based on optimizer_skip_list setting - pub fn is_optimizer_disabled(self: &Arc, name: &str) -> bool { + pub fn is_optimizer_disabled(&self, name: &str) -> bool { let settings = self.get_table_ctx().get_settings(); match settings.get_optimizer_skip_list() { Ok(skip_list) if !skip_list.is_empty() => { diff --git a/src/query/sql/src/planner/optimizer/optimizers/distributed/sort_and_limit.rs b/src/query/sql/src/planner/optimizer/optimizers/distributed/sort_and_limit.rs index 65b286bc06b78..43287fcd8e97b 100644 --- a/src/query/sql/src/planner/optimizer/optimizers/distributed/sort_and_limit.rs +++ b/src/query/sql/src/planner/optimizer/optimizers/distributed/sort_and_limit.rs @@ -104,33 +104,43 @@ impl SortAndLimitPushDownOptimizer { return Ok(s_expr.clone()); } - let mut sort: Sort = s_expr.plan().clone().try_into()?; - sort.after_exchange = Some(false); - let exchange_sexpr = s_expr.child(0)?; - - // this is window shuffle sort - if matches!( - exchange_sexpr.plan.as_ref(), - RelOperator::Exchange(Exchange::Hash(_)) - ) { - return Ok(s_expr.clone()); + let exchange_sexpr = s_expr.unary_child(); + + match exchange_sexpr.plan() { + RelOperator::Exchange(exchange) => match exchange { + // this is window shuffle sort + Exchange::Hash(_) => return Ok(s_expr.clone()), + Exchange::Merge | Exchange::MergeSort => {} + Exchange::Broadcast => unreachable!(), + }, + _ => unreachable!(), } - debug_assert!(matches!( - exchange_sexpr.plan.as_ref(), - RelOperator::Exchange(Exchange::Merge) | RelOperator::Exchange(Exchange::MergeSort) - )); - - debug_assert!(exchange_sexpr.children.len() == 1); - let exchange_sexpr = exchange_sexpr.replace_plan(Arc::new(Exchange::MergeSort.into())); - - let child = exchange_sexpr.child(0)?.clone(); - let before_exchange_sort = - SExpr::create_unary(Arc::new(sort.clone().into()), Arc::new(child)); - let new_exchange = exchange_sexpr.replace_children(vec![Arc::new(before_exchange_sort)]); - sort.after_exchange = Some(true); - let new_plan = SExpr::create_unary(Arc::new(sort.into()), Arc::new(new_exchange)); - Ok(new_plan) + let sort = s_expr.plan.as_sort().unwrap(); + + let new_exchange = SExpr::create_unary( + Arc::new(Exchange::MergeSort.into()), + SExpr::create_unary( + Arc::new( + Sort { + after_exchange: Some(false), + ..sort.clone() + } + .into(), + ), + exchange_sexpr.unary_child_arc(), + ), + ); + Ok(SExpr::create_unary( + Arc::new( + Sort { + after_exchange: Some(true), + ..sort.clone() + } + .into(), + ), + Arc::new(new_exchange), + )) } fn apply_limit(&self, s_expr: &SExpr) -> Result { diff --git a/src/query/sql/src/planner/optimizer/optimizers/rule/agg_rules/rule_split_aggregate.rs b/src/query/sql/src/planner/optimizer/optimizers/rule/agg_rules/rule_split_aggregate.rs index ff8feb8da8780..5e6f08a0a4af3 100644 --- a/src/query/sql/src/planner/optimizer/optimizers/rule/agg_rules/rule_split_aggregate.rs +++ b/src/query/sql/src/planner/optimizer/optimizers/rule/agg_rules/rule_split_aggregate.rs @@ -52,20 +52,29 @@ impl Rule for RuleSplitAggregate { } fn apply(&self, s_expr: &SExpr, state: &mut TransformResult) -> Result<()> { - let mut agg: Aggregate = s_expr.plan().clone().try_into()?; + let agg = s_expr.plan().as_aggregate().unwrap(); if agg.mode != AggregateMode::Initial { return Ok(()); } - agg.mode = AggregateMode::Final; - let mut partial = agg.clone(); - partial.mode = AggregateMode::Partial; let result = SExpr::create_unary( - Arc::new(agg.into()), - Arc::new(SExpr::create_unary( - Arc::new(partial.into()), - Arc::new(s_expr.child(0)?.clone()), - )), + Arc::new( + Aggregate { + mode: AggregateMode::Final, + ..agg.clone() + } + .into(), + ), + SExpr::create_unary( + Arc::new( + Aggregate { + mode: AggregateMode::Partial, + ..agg.clone() + } + .into(), + ), + s_expr.unary_child_arc(), + ), ); state.add_result(result); Ok(()) diff --git a/src/query/sql/src/planner/planner.rs b/src/query/sql/src/planner/planner.rs index eb4456cdf76d4..8eb0423ded612 100644 --- a/src/query/sql/src/planner/planner.rs +++ b/src/query/sql/src/planner/planner.rs @@ -272,12 +272,9 @@ impl Planner { // Step 4: Optimize the SExpr with optimizers, and generate optimized physical SExpr let opt_ctx = OptimizerContext::new(self.ctx.clone(), metadata.clone()) + .with_settings(&settings)? .set_enable_distributed_optimization(!self.ctx.get_cluster().is_empty()) - .set_enable_join_reorder(unsafe { !settings.get_disable_join_reorder()? }) - .set_enable_dphyp(settings.get_enable_dphyp()?) - .set_max_push_down_limit(settings.get_max_push_down_limit()?) .set_sample_executor(self.query_executor.clone()) - .set_enable_trace(settings.get_enable_optimizer_trace()?) .clone(); let optimized_plan = optimize(opt_ctx, plan).await?; diff --git a/src/query/sql/src/planner/plans/exchange.rs b/src/query/sql/src/planner/plans/exchange.rs index a7aca885b2ed1..7bd4d27c4e8ca 100644 --- a/src/query/sql/src/planner/plans/exchange.rs +++ b/src/query/sql/src/planner/plans/exchange.rs @@ -47,8 +47,7 @@ impl Operator for Exchange { distribution: match self { Exchange::Hash(hash_keys) => Distribution::Hash(hash_keys.clone()), Exchange::Broadcast => Distribution::Broadcast, - Exchange::Merge => Distribution::Serial, - Exchange::MergeSort => Distribution::Serial, + Exchange::Merge | Exchange::MergeSort => Distribution::Serial, }, }) } diff --git a/src/query/sql/src/planner/plans/operator.rs b/src/query/sql/src/planner/plans/operator.rs index 3d3a67518fed0..a33b756ad4825 100644 --- a/src/query/sql/src/planner/plans/operator.rs +++ b/src/query/sql/src/planner/plans/operator.rs @@ -18,6 +18,7 @@ use databend_common_catalog::table_context::TableContext; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use educe::Educe; +use enum_as_inner::EnumAsInner; use super::MutationSource; use super::SubqueryExpr; @@ -125,7 +126,7 @@ pub enum RelOp { } /// Relational operators -#[derive(Educe)] +#[derive(Educe, EnumAsInner)] #[educe( PartialEq(bound = false, attrs = "#[recursive::recursive]"), Eq, diff --git a/src/query/sql/src/planner/plans/recluster.rs b/src/query/sql/src/planner/plans/recluster.rs index 2313d3320ea13..9a0e08636606b 100644 --- a/src/query/sql/src/planner/plans/recluster.rs +++ b/src/query/sql/src/planner/plans/recluster.rs @@ -86,10 +86,8 @@ pub async fn plan_hilbert_sql( let plan = binder.bind(&stmt).await?; let opt_ctx = OptimizerContext::new(ctx.clone(), metadata) + .with_settings(&settings)? .set_enable_distributed_optimization(!ctx.get_cluster().is_empty()) - .set_enable_join_reorder(unsafe { !settings.get_disable_join_reorder()? }) - .set_enable_dphyp(settings.get_enable_dphyp()?) - .set_max_push_down_limit(settings.get_max_push_down_limit()?) .clone(); optimize(opt_ctx, plan).await } diff --git a/src/query/sql/src/planner/plans/sort.rs b/src/query/sql/src/planner/plans/sort.rs index 7ef783ab3ac43..bcf18f814e5fb 100644 --- a/src/query/sql/src/planner/plans/sort.rs +++ b/src/query/sql/src/planner/plans/sort.rs @@ -34,8 +34,6 @@ pub struct Sort { pub items: Vec, pub limit: Option, - /// If the sort plan is after the exchange plan. - /// It's [None] if the sorting plan is in single node mode. pub after_exchange: Option, /// The columns needed by the plan after the sort plan. From 0eef223d59dbdcef668da52b6e3b8694876b2a72 Mon Sep 17 00:00:00 2001 From: coldWater Date: Tue, 1 Jul 2025 18:18:45 +0800 Subject: [PATCH 33/61] logical plan --- .../src/pipelines/builders/builder_sort.rs | 2 +- .../src/pipelines/builders/builder_window.rs | 1 + .../transforms/sort/sort_exchange_injector.rs | 1 + .../sql/src/executor/physical_plans/mod.rs | 2 +- .../executor/physical_plans/physical_sort.rs | 37 ++----- .../physical_window_partition.rs | 2 +- .../sql/src/planner/binder/bind_query/bind.rs | 3 +- src/query/sql/src/planner/binder/sort.rs | 3 +- src/query/sql/src/planner/binder/window.rs | 3 +- .../planner/optimizer/optimizer_context.rs | 7 ++ .../optimizer/optimizers/cascades/cascade.rs | 4 +- .../optimizers/distributed/distributed.rs | 4 +- .../optimizers/distributed/sort_and_limit.rs | 100 +++++++++++++++--- .../rule_push_down_limit_aggregate.rs | 3 +- src/query/sql/src/planner/plans/sort.rs | 17 ++- 15 files changed, 138 insertions(+), 51 deletions(-) diff --git a/src/query/service/src/pipelines/builders/builder_sort.rs b/src/query/service/src/pipelines/builders/builder_sort.rs index ea35f9943d17d..f1f21c1a62a2a 100644 --- a/src/query/service/src/pipelines/builders/builder_sort.rs +++ b/src/query/service/src/pipelines/builders/builder_sort.rs @@ -29,7 +29,7 @@ use databend_common_pipeline_transforms::MemorySettings; use databend_common_sql::evaluator::BlockOperator; use databend_common_sql::evaluator::CompoundBlockOperator; use databend_common_sql::executor::physical_plans::Sort; -use databend_common_sql::executor::physical_plans::SortStep; +use databend_common_sql::plans::SortStep; use databend_common_storage::DataOperator; use databend_common_storages_fuse::TableContext; use databend_storages_common_cache::TempDirManager; diff --git a/src/query/service/src/pipelines/builders/builder_window.rs b/src/query/service/src/pipelines/builders/builder_window.rs index cb82feca68ca3..9a26c677236df 100644 --- a/src/query/service/src/pipelines/builders/builder_window.rs +++ b/src/query/service/src/pipelines/builders/builder_window.rs @@ -26,6 +26,7 @@ use databend_common_pipeline_core::processors::ProcessorPtr; use databend_common_pipeline_transforms::MemorySettings; use databend_common_sql::executor::physical_plans::Window; use databend_common_sql::executor::physical_plans::WindowPartition; +use databend_common_sql::plans::SortStep; use databend_storages_common_cache::TempDirManager; use crate::pipelines::memory_settings::MemorySettingsExt; diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs index e567af11fcaa0..63303ce25c3e5 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs @@ -14,6 +14,7 @@ use std::sync::Arc; +use bumpalo::Bump; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::BlockMetaInfoDowncast; diff --git a/src/query/sql/src/executor/physical_plans/mod.rs b/src/query/sql/src/executor/physical_plans/mod.rs index edf36a64edb31..2f2afb69d3368 100644 --- a/src/query/sql/src/executor/physical_plans/mod.rs +++ b/src/query/sql/src/executor/physical_plans/mod.rs @@ -106,7 +106,7 @@ pub use physical_replace_async_source::ReplaceAsyncSourcer; pub use physical_replace_deduplicate::*; pub use physical_replace_into::ReplaceInto; pub use physical_row_fetch::RowFetch; -pub use physical_sort::*; +pub use physical_sort::Sort; pub use physical_table_scan::TableScan; pub use physical_udf::Udf; pub use physical_udf::UdfFunctionDesc; diff --git a/src/query/sql/src/executor/physical_plans/physical_sort.rs b/src/query/sql/src/executor/physical_plans/physical_sort.rs index cdf7ab5c0cc53..01c3fc0eee221 100644 --- a/src/query/sql/src/executor/physical_plans/physical_sort.rs +++ b/src/query/sql/src/executor/physical_plans/physical_sort.rs @@ -28,6 +28,7 @@ use crate::executor::physical_plans::WindowPartitionTopNFunc; use crate::executor::PhysicalPlan; use crate::executor::PhysicalPlanBuilder; use crate::optimizer::ir::SExpr; +use crate::plans::SortStep; use crate::plans::WindowFuncType; use crate::ColumnSet; use crate::IndexType; @@ -48,21 +49,6 @@ pub struct Sort { pub stat_info: Option, } -#[derive(Debug, Hash, Clone, Copy, serde::Serialize, serde::Deserialize, PartialEq, Eq)] -pub enum SortStep { - // single node mode - Single, - - // cluster mode - Partial, // before the exchange plan - FinalMerge, // after the exchange plan - - // range shuffle mode - Sample, - RangeSort, - Route, -} - impl Sort { fn order_col_type(&self, schema: &DataSchema) -> Result { if self.order_by.len() == 1 { @@ -148,17 +134,6 @@ impl PhysicalPlanBuilder { }) .collect::>(); - let enable_range_shuffle_sort = self.ctx.get_settings().get_enable_range_shuffle_sort()?; - let sort_step = match sort.after_exchange { - Some(false) => SortStep::Partial, - Some(true) => SortStep::FinalMerge, - None => SortStep::Single, - }; - let broadcast_id = match sort.after_exchange { - Some(false) if enable_range_shuffle_sort => Some(self.ctx.get_next_broadcast_id()), - _ => None, - }; - // Add WindowPartition for parallel sort in window. if let Some(window) = &sort.window_partition { let window_partition = window @@ -172,7 +147,7 @@ impl PhysicalPlanBuilder { input: Box::new(input_plan.clone()), partition_by: window_partition.clone(), order_by: order_by.clone(), - sort_step, + sort_step: sort.step, top_n: window.top.map(|top| WindowPartitionTopN { func: match window.func { WindowFuncType::RowNumber => WindowPartitionTopNFunc::RowNumber, @@ -186,13 +161,19 @@ impl PhysicalPlanBuilder { })); }; + let broadcast_id = if sort.step == SortStep::Sample { + Some(self.ctx.get_next_broadcast_id()) + } else { + None + }; + // 2. Build physical plan. Ok(PhysicalPlan::Sort(Sort { plan_id: 0, input: Box::new(input_plan), order_by, limit: sort.limit, - step: sort_step, + step: sort.step, pre_projection, broadcast_id, stat_info: Some(stat_info), diff --git a/src/query/sql/src/executor/physical_plans/physical_window_partition.rs b/src/query/sql/src/executor/physical_plans/physical_window_partition.rs index 21fa5e013f9fb..d879d72f4d8ec 100644 --- a/src/query/sql/src/executor/physical_plans/physical_window_partition.rs +++ b/src/query/sql/src/executor/physical_plans/physical_window_partition.rs @@ -16,9 +16,9 @@ use databend_common_exception::Result; use databend_common_expression::DataSchemaRef; use crate::executor::explain::PlanStatsInfo; -use crate::executor::physical_plans::physical_sort::SortStep; use crate::executor::physical_plans::SortDesc; use crate::executor::PhysicalPlan; +use crate::plans::SortStep; use crate::IndexType; #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] diff --git a/src/query/sql/src/planner/binder/bind_query/bind.rs b/src/query/sql/src/planner/binder/bind_query/bind.rs index 46867955cb746..bcd665aff4d9f 100644 --- a/src/query/sql/src/planner/binder/bind_query/bind.rs +++ b/src/query/sql/src/planner/binder/bind_query/bind.rs @@ -48,6 +48,7 @@ use crate::plans::BoundColumnRef; use crate::plans::ScalarExpr; use crate::plans::Sort; use crate::plans::SortItem; +use crate::plans::SortStep; use crate::NameResolutionContext; #[derive(Debug, Default, Visitor)] @@ -234,7 +235,7 @@ impl Binder { let sort_plan = Sort { items: order_by_items, limit: None, - after_exchange: None, + step: SortStep::Single, pre_projection: None, window_partition: None, }; diff --git a/src/query/sql/src/planner/binder/sort.rs b/src/query/sql/src/planner/binder/sort.rs index 617aad8308f0b..212af23ce1fa6 100644 --- a/src/query/sql/src/planner/binder/sort.rs +++ b/src/query/sql/src/planner/binder/sort.rs @@ -38,6 +38,7 @@ use crate::plans::ScalarExpr; use crate::plans::ScalarItem; use crate::plans::Sort; use crate::plans::SortItem; +use crate::plans::SortStep; use crate::plans::UDFCall; use crate::plans::VisitorMut as _; use crate::BindContext; @@ -221,7 +222,7 @@ impl Binder { let sort_plan = Sort { items: order_by_items, limit: None, - after_exchange: None, + step: SortStep::Single, pre_projection: None, window_partition: None, }; diff --git a/src/query/sql/src/planner/binder/window.rs b/src/query/sql/src/planner/binder/window.rs index db6140b55129d..8e78916375df3 100644 --- a/src/query/sql/src/planner/binder/window.rs +++ b/src/query/sql/src/planner/binder/window.rs @@ -38,6 +38,7 @@ use crate::plans::ScalarExpr; use crate::plans::ScalarItem; use crate::plans::Sort; use crate::plans::SortItem; +use crate::plans::SortStep; use crate::plans::SubqueryExpr; use crate::plans::VisitorMut; use crate::plans::Window; @@ -655,7 +656,7 @@ pub fn bind_window_function_info( let sort_plan = Sort { items: sort_items, limit: None, - after_exchange: None, + step: SortStep::Single, pre_projection: None, window_partition: if window_plan.partition_by.is_empty() { None diff --git a/src/query/sql/src/planner/optimizer/optimizer_context.rs b/src/query/sql/src/planner/optimizer/optimizer_context.rs index d75f9cea101e7..f28baee21db48 100644 --- a/src/query/sql/src/planner/optimizer/optimizer_context.rs +++ b/src/query/sql/src/planner/optimizer/optimizer_context.rs @@ -33,6 +33,7 @@ pub struct OptimizerContext { // Optimizer configurations enable_distributed_optimization: RwLock, + enable_range_shuffle_sort: RwLock, enable_join_reorder: RwLock, enable_dphyp: RwLock, max_push_down_limit: RwLock, @@ -56,6 +57,7 @@ impl OptimizerContext { metadata, enable_distributed_optimization: RwLock::new(false), + enable_range_shuffle_sort: RwLock::new(false), enable_join_reorder: RwLock::new(true), enable_dphyp: RwLock::new(true), max_push_down_limit: RwLock::new(10000), @@ -71,6 +73,7 @@ impl OptimizerContext { *self.enable_dphyp.write() = settings.get_enable_dphyp()?; *self.max_push_down_limit.write() = settings.get_max_push_down_limit()?; *self.enable_trace.write() = settings.get_enable_optimizer_trace()?; + *self.enable_range_shuffle_sort.write() = settings.get_enable_range_shuffle_sort()?; Ok(self) } @@ -97,6 +100,10 @@ impl OptimizerContext { self } + pub fn get_enable_range_shuffle_sort(&self) -> bool { + *self.enable_range_shuffle_sort.read() + } + pub fn get_enable_join_reorder(&self) -> bool { *self.enable_join_reorder.read() } diff --git a/src/query/sql/src/planner/optimizer/optimizers/cascades/cascade.rs b/src/query/sql/src/planner/optimizer/optimizers/cascades/cascade.rs index a79f76dba9497..1e08336a8c031 100644 --- a/src/query/sql/src/planner/optimizer/optimizers/cascades/cascade.rs +++ b/src/query/sql/src/planner/optimizer/optimizers/cascades/cascade.rs @@ -95,7 +95,9 @@ impl CascadesOptimizer { Ok(expr) => { // After successful optimization, apply sort and limit push down if distributed optimization is enabled if opt_ctx.get_enable_distributed_optimization() { - let sort_and_limit_optimizer = SortAndLimitPushDownOptimizer::create(); + let sort_and_limit_optimizer = SortAndLimitPushDownOptimizer::create( + opt_ctx.get_enable_range_shuffle_sort(), + ); sort_and_limit_optimizer.optimize(&expr)? } else { expr diff --git a/src/query/sql/src/planner/optimizer/optimizers/distributed/distributed.rs b/src/query/sql/src/planner/optimizer/optimizers/distributed/distributed.rs index c317c56ed5ed0..abd713fab2ec0 100644 --- a/src/query/sql/src/planner/optimizer/optimizers/distributed/distributed.rs +++ b/src/query/sql/src/planner/optimizer/optimizers/distributed/distributed.rs @@ -40,7 +40,9 @@ impl DistributedOptimizer { pub fn new(opt_ctx: Arc) -> Self { Self { ctx: opt_ctx.get_table_ctx(), - sort_limit_optimizer: SortAndLimitPushDownOptimizer::create(), + sort_limit_optimizer: SortAndLimitPushDownOptimizer::create( + opt_ctx.get_enable_range_shuffle_sort(), + ), } } diff --git a/src/query/sql/src/planner/optimizer/optimizers/distributed/sort_and_limit.rs b/src/query/sql/src/planner/optimizer/optimizers/distributed/sort_and_limit.rs index 43287fcd8e97b..5445fb4ee0580 100644 --- a/src/query/sql/src/planner/optimizer/optimizers/distributed/sort_and_limit.rs +++ b/src/query/sql/src/planner/optimizer/optimizers/distributed/sort_and_limit.rs @@ -23,15 +23,18 @@ use crate::plans::Limit; use crate::plans::RelOp; use crate::plans::RelOperator; use crate::plans::Sort; +use crate::plans::SortStep; pub struct SortAndLimitPushDownOptimizer { + range_shuffle: bool, sort_matcher: Matcher, limit_matcher: Matcher, } impl SortAndLimitPushDownOptimizer { - pub fn create() -> Self { + pub fn create(range_shuffle: bool) -> Self { Self { + range_shuffle, sort_matcher: Self::sort_matcher(), limit_matcher: Self::limit_matcher(), } @@ -46,14 +49,6 @@ impl SortAndLimitPushDownOptimizer { // Exchange // \ // * - // Output: - // Sort (after_exchange = true) - // \ - // Exchange - // \ - // Sort (after_exchange = false) - // \ - // * Matcher::MatchOp { op_type: RelOp::Sort, children: vec![Matcher::MatchOp { @@ -95,11 +90,24 @@ impl SortAndLimitPushDownOptimizer { replaced_children.push(Arc::new(new_child)); } let new_sexpr = s_expr.replace_children(replaced_children); - let apply_topn_res = self.apply_sort(&new_sexpr)?; + + let apply_topn_res = if self.range_shuffle { + self.apply_shuffle_sort(&new_sexpr)? + } else { + self.apply_dist_sort(&new_sexpr)? + }; self.apply_limit(&apply_topn_res) } - fn apply_sort(&self, s_expr: &SExpr) -> Result { + fn apply_dist_sort(&self, s_expr: &SExpr) -> Result { + // Output: + // Sort (FinalMerge) + // \ + // Exchange + // \ + // Sort (Partial) + // \ + // * if !self.sort_matcher.matches(s_expr) { return Ok(s_expr.clone()); } @@ -123,7 +131,7 @@ impl SortAndLimitPushDownOptimizer { SExpr::create_unary( Arc::new( Sort { - after_exchange: Some(false), + step: SortStep::Partial, ..sort.clone() } .into(), @@ -134,7 +142,7 @@ impl SortAndLimitPushDownOptimizer { Ok(SExpr::create_unary( Arc::new( Sort { - after_exchange: Some(true), + step: SortStep::FinalMerge, ..sort.clone() } .into(), @@ -143,6 +151,72 @@ impl SortAndLimitPushDownOptimizer { )) } + fn apply_shuffle_sort(&self, s_expr: &SExpr) -> Result { + // Output: + // Sort (Route) + // \ + // Exchange + // \ + // Sort (RangeSort) + // \ + // Exchange + // \ + // Sort (Sample) + // \ + // * + if !self.sort_matcher.matches(s_expr) { + return Ok(s_expr.clone()); + } + + let exchange_sexpr = s_expr.unary_child(); + match exchange_sexpr.plan() { + RelOperator::Exchange(exchange) => match exchange { + // this is window shuffle sort + Exchange::Hash(_) => return Ok(s_expr.clone()), + Exchange::Merge | Exchange::MergeSort => {} + Exchange::Broadcast => unreachable!(), + }, + _ => unreachable!(), + } + + let sort = s_expr.plan.as_sort().unwrap(); + + let input = exchange_sexpr.unary_child_arc(); + + let sort_simple = SExpr::create_unary( + Arc::new( + Sort { + step: SortStep::Sample, + ..sort.clone() + } + .into(), + ), + input, + ); + let exchange1 = SExpr::create_unary(Arc::new(Exchange::MergeSort.into()), sort_simple); + let sort_range = SExpr::create_unary( + Arc::new( + Sort { + step: SortStep::RangeSort, + ..sort.clone() + } + .into(), + ), + exchange1, + ); + let exchange2 = SExpr::create_unary(Arc::new(Exchange::MergeSort.into()), sort_range); + Ok(SExpr::create_unary( + Arc::new( + Sort { + step: SortStep::Route, + ..sort.clone() + } + .into(), + ), + exchange2, + )) + } + fn apply_limit(&self, s_expr: &SExpr) -> Result { if !self.limit_matcher.matches(s_expr) { return Ok(s_expr.clone()); diff --git a/src/query/sql/src/planner/optimizer/optimizers/rule/agg_rules/rule_push_down_limit_aggregate.rs b/src/query/sql/src/planner/optimizer/optimizers/rule/agg_rules/rule_push_down_limit_aggregate.rs index bf2f1c1eb05ab..08b21b4811650 100644 --- a/src/query/sql/src/planner/optimizer/optimizers/rule/agg_rules/rule_push_down_limit_aggregate.rs +++ b/src/query/sql/src/planner/optimizer/optimizers/rule/agg_rules/rule_push_down_limit_aggregate.rs @@ -26,6 +26,7 @@ use crate::plans::RelOp; use crate::plans::RelOperator; use crate::plans::Sort; use crate::plans::SortItem; +use crate::plans::SortStep; /// Input: Limit | Sort /// \ @@ -110,7 +111,7 @@ impl RulePushDownRankLimitAggregate { let sort = Sort { items: sort_items.clone(), limit: Some(count), - after_exchange: None, + step: SortStep::Single, pre_projection: None, window_partition: None, }; diff --git a/src/query/sql/src/planner/plans/sort.rs b/src/query/sql/src/planner/plans/sort.rs index bcf18f814e5fb..f11ffa775f0d2 100644 --- a/src/query/sql/src/planner/plans/sort.rs +++ b/src/query/sql/src/planner/plans/sort.rs @@ -34,7 +34,7 @@ pub struct Sort { pub items: Vec, pub limit: Option, - pub after_exchange: Option, + pub step: SortStep, /// The columns needed by the plan after the sort plan. /// It's used to build a projection operation before building the sort operator. @@ -44,6 +44,21 @@ pub struct Sort { pub window_partition: Option, } +#[derive(Debug, Hash, Clone, Copy, serde::Serialize, serde::Deserialize, PartialEq, Eq)] +pub enum SortStep { + // single node mode + Single, + + // cluster mode + Partial, // before the exchange plan + FinalMerge, // after the exchange plan + + // range shuffle mode + Sample, + RangeSort, + Route, +} + impl Sort { pub fn used_columns(&self) -> ColumnSet { self.items.iter().map(|item| item.index).collect() From d80e0258254fffd602a71e3f7bffb6c03dc7ac4d Mon Sep 17 00:00:00 2001 From: coldWater Date: Tue, 1 Jul 2025 18:19:57 +0800 Subject: [PATCH 34/61] x --- .../src/pipelines/builders/builder_sort.rs | 2 +- .../src/pipelines/builders/builder_window.rs | 1 - .../transforms/sort/sort_exchange_injector.rs | 1 - .../sql/src/executor/physical_plans/mod.rs | 2 +- .../executor/physical_plans/physical_sort.rs | 37 +++++-- .../physical_window_partition.rs | 2 +- .../sql/src/planner/binder/bind_query/bind.rs | 3 +- src/query/sql/src/planner/binder/sort.rs | 3 +- src/query/sql/src/planner/binder/window.rs | 3 +- .../planner/optimizer/optimizer_context.rs | 7 -- .../optimizer/optimizers/cascades/cascade.rs | 4 +- .../optimizers/distributed/distributed.rs | 4 +- .../optimizers/distributed/sort_and_limit.rs | 100 +++--------------- .../rule_push_down_limit_aggregate.rs | 3 +- src/query/sql/src/planner/plans/sort.rs | 17 +-- 15 files changed, 51 insertions(+), 138 deletions(-) diff --git a/src/query/service/src/pipelines/builders/builder_sort.rs b/src/query/service/src/pipelines/builders/builder_sort.rs index f1f21c1a62a2a..ea35f9943d17d 100644 --- a/src/query/service/src/pipelines/builders/builder_sort.rs +++ b/src/query/service/src/pipelines/builders/builder_sort.rs @@ -29,7 +29,7 @@ use databend_common_pipeline_transforms::MemorySettings; use databend_common_sql::evaluator::BlockOperator; use databend_common_sql::evaluator::CompoundBlockOperator; use databend_common_sql::executor::physical_plans::Sort; -use databend_common_sql::plans::SortStep; +use databend_common_sql::executor::physical_plans::SortStep; use databend_common_storage::DataOperator; use databend_common_storages_fuse::TableContext; use databend_storages_common_cache::TempDirManager; diff --git a/src/query/service/src/pipelines/builders/builder_window.rs b/src/query/service/src/pipelines/builders/builder_window.rs index 9a26c677236df..cb82feca68ca3 100644 --- a/src/query/service/src/pipelines/builders/builder_window.rs +++ b/src/query/service/src/pipelines/builders/builder_window.rs @@ -26,7 +26,6 @@ use databend_common_pipeline_core::processors::ProcessorPtr; use databend_common_pipeline_transforms::MemorySettings; use databend_common_sql::executor::physical_plans::Window; use databend_common_sql::executor::physical_plans::WindowPartition; -use databend_common_sql::plans::SortStep; use databend_storages_common_cache::TempDirManager; use crate::pipelines::memory_settings::MemorySettingsExt; diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs index 63303ce25c3e5..e567af11fcaa0 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs @@ -14,7 +14,6 @@ use std::sync::Arc; -use bumpalo::Bump; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::BlockMetaInfoDowncast; diff --git a/src/query/sql/src/executor/physical_plans/mod.rs b/src/query/sql/src/executor/physical_plans/mod.rs index 2f2afb69d3368..edf36a64edb31 100644 --- a/src/query/sql/src/executor/physical_plans/mod.rs +++ b/src/query/sql/src/executor/physical_plans/mod.rs @@ -106,7 +106,7 @@ pub use physical_replace_async_source::ReplaceAsyncSourcer; pub use physical_replace_deduplicate::*; pub use physical_replace_into::ReplaceInto; pub use physical_row_fetch::RowFetch; -pub use physical_sort::Sort; +pub use physical_sort::*; pub use physical_table_scan::TableScan; pub use physical_udf::Udf; pub use physical_udf::UdfFunctionDesc; diff --git a/src/query/sql/src/executor/physical_plans/physical_sort.rs b/src/query/sql/src/executor/physical_plans/physical_sort.rs index 01c3fc0eee221..cdf7ab5c0cc53 100644 --- a/src/query/sql/src/executor/physical_plans/physical_sort.rs +++ b/src/query/sql/src/executor/physical_plans/physical_sort.rs @@ -28,7 +28,6 @@ use crate::executor::physical_plans::WindowPartitionTopNFunc; use crate::executor::PhysicalPlan; use crate::executor::PhysicalPlanBuilder; use crate::optimizer::ir::SExpr; -use crate::plans::SortStep; use crate::plans::WindowFuncType; use crate::ColumnSet; use crate::IndexType; @@ -49,6 +48,21 @@ pub struct Sort { pub stat_info: Option, } +#[derive(Debug, Hash, Clone, Copy, serde::Serialize, serde::Deserialize, PartialEq, Eq)] +pub enum SortStep { + // single node mode + Single, + + // cluster mode + Partial, // before the exchange plan + FinalMerge, // after the exchange plan + + // range shuffle mode + Sample, + RangeSort, + Route, +} + impl Sort { fn order_col_type(&self, schema: &DataSchema) -> Result { if self.order_by.len() == 1 { @@ -134,6 +148,17 @@ impl PhysicalPlanBuilder { }) .collect::>(); + let enable_range_shuffle_sort = self.ctx.get_settings().get_enable_range_shuffle_sort()?; + let sort_step = match sort.after_exchange { + Some(false) => SortStep::Partial, + Some(true) => SortStep::FinalMerge, + None => SortStep::Single, + }; + let broadcast_id = match sort.after_exchange { + Some(false) if enable_range_shuffle_sort => Some(self.ctx.get_next_broadcast_id()), + _ => None, + }; + // Add WindowPartition for parallel sort in window. if let Some(window) = &sort.window_partition { let window_partition = window @@ -147,7 +172,7 @@ impl PhysicalPlanBuilder { input: Box::new(input_plan.clone()), partition_by: window_partition.clone(), order_by: order_by.clone(), - sort_step: sort.step, + sort_step, top_n: window.top.map(|top| WindowPartitionTopN { func: match window.func { WindowFuncType::RowNumber => WindowPartitionTopNFunc::RowNumber, @@ -161,19 +186,13 @@ impl PhysicalPlanBuilder { })); }; - let broadcast_id = if sort.step == SortStep::Sample { - Some(self.ctx.get_next_broadcast_id()) - } else { - None - }; - // 2. Build physical plan. Ok(PhysicalPlan::Sort(Sort { plan_id: 0, input: Box::new(input_plan), order_by, limit: sort.limit, - step: sort.step, + step: sort_step, pre_projection, broadcast_id, stat_info: Some(stat_info), diff --git a/src/query/sql/src/executor/physical_plans/physical_window_partition.rs b/src/query/sql/src/executor/physical_plans/physical_window_partition.rs index d879d72f4d8ec..21fa5e013f9fb 100644 --- a/src/query/sql/src/executor/physical_plans/physical_window_partition.rs +++ b/src/query/sql/src/executor/physical_plans/physical_window_partition.rs @@ -16,9 +16,9 @@ use databend_common_exception::Result; use databend_common_expression::DataSchemaRef; use crate::executor::explain::PlanStatsInfo; +use crate::executor::physical_plans::physical_sort::SortStep; use crate::executor::physical_plans::SortDesc; use crate::executor::PhysicalPlan; -use crate::plans::SortStep; use crate::IndexType; #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] diff --git a/src/query/sql/src/planner/binder/bind_query/bind.rs b/src/query/sql/src/planner/binder/bind_query/bind.rs index bcd665aff4d9f..46867955cb746 100644 --- a/src/query/sql/src/planner/binder/bind_query/bind.rs +++ b/src/query/sql/src/planner/binder/bind_query/bind.rs @@ -48,7 +48,6 @@ use crate::plans::BoundColumnRef; use crate::plans::ScalarExpr; use crate::plans::Sort; use crate::plans::SortItem; -use crate::plans::SortStep; use crate::NameResolutionContext; #[derive(Debug, Default, Visitor)] @@ -235,7 +234,7 @@ impl Binder { let sort_plan = Sort { items: order_by_items, limit: None, - step: SortStep::Single, + after_exchange: None, pre_projection: None, window_partition: None, }; diff --git a/src/query/sql/src/planner/binder/sort.rs b/src/query/sql/src/planner/binder/sort.rs index 212af23ce1fa6..617aad8308f0b 100644 --- a/src/query/sql/src/planner/binder/sort.rs +++ b/src/query/sql/src/planner/binder/sort.rs @@ -38,7 +38,6 @@ use crate::plans::ScalarExpr; use crate::plans::ScalarItem; use crate::plans::Sort; use crate::plans::SortItem; -use crate::plans::SortStep; use crate::plans::UDFCall; use crate::plans::VisitorMut as _; use crate::BindContext; @@ -222,7 +221,7 @@ impl Binder { let sort_plan = Sort { items: order_by_items, limit: None, - step: SortStep::Single, + after_exchange: None, pre_projection: None, window_partition: None, }; diff --git a/src/query/sql/src/planner/binder/window.rs b/src/query/sql/src/planner/binder/window.rs index 8e78916375df3..db6140b55129d 100644 --- a/src/query/sql/src/planner/binder/window.rs +++ b/src/query/sql/src/planner/binder/window.rs @@ -38,7 +38,6 @@ use crate::plans::ScalarExpr; use crate::plans::ScalarItem; use crate::plans::Sort; use crate::plans::SortItem; -use crate::plans::SortStep; use crate::plans::SubqueryExpr; use crate::plans::VisitorMut; use crate::plans::Window; @@ -656,7 +655,7 @@ pub fn bind_window_function_info( let sort_plan = Sort { items: sort_items, limit: None, - step: SortStep::Single, + after_exchange: None, pre_projection: None, window_partition: if window_plan.partition_by.is_empty() { None diff --git a/src/query/sql/src/planner/optimizer/optimizer_context.rs b/src/query/sql/src/planner/optimizer/optimizer_context.rs index f28baee21db48..d75f9cea101e7 100644 --- a/src/query/sql/src/planner/optimizer/optimizer_context.rs +++ b/src/query/sql/src/planner/optimizer/optimizer_context.rs @@ -33,7 +33,6 @@ pub struct OptimizerContext { // Optimizer configurations enable_distributed_optimization: RwLock, - enable_range_shuffle_sort: RwLock, enable_join_reorder: RwLock, enable_dphyp: RwLock, max_push_down_limit: RwLock, @@ -57,7 +56,6 @@ impl OptimizerContext { metadata, enable_distributed_optimization: RwLock::new(false), - enable_range_shuffle_sort: RwLock::new(false), enable_join_reorder: RwLock::new(true), enable_dphyp: RwLock::new(true), max_push_down_limit: RwLock::new(10000), @@ -73,7 +71,6 @@ impl OptimizerContext { *self.enable_dphyp.write() = settings.get_enable_dphyp()?; *self.max_push_down_limit.write() = settings.get_max_push_down_limit()?; *self.enable_trace.write() = settings.get_enable_optimizer_trace()?; - *self.enable_range_shuffle_sort.write() = settings.get_enable_range_shuffle_sort()?; Ok(self) } @@ -100,10 +97,6 @@ impl OptimizerContext { self } - pub fn get_enable_range_shuffle_sort(&self) -> bool { - *self.enable_range_shuffle_sort.read() - } - pub fn get_enable_join_reorder(&self) -> bool { *self.enable_join_reorder.read() } diff --git a/src/query/sql/src/planner/optimizer/optimizers/cascades/cascade.rs b/src/query/sql/src/planner/optimizer/optimizers/cascades/cascade.rs index 1e08336a8c031..a79f76dba9497 100644 --- a/src/query/sql/src/planner/optimizer/optimizers/cascades/cascade.rs +++ b/src/query/sql/src/planner/optimizer/optimizers/cascades/cascade.rs @@ -95,9 +95,7 @@ impl CascadesOptimizer { Ok(expr) => { // After successful optimization, apply sort and limit push down if distributed optimization is enabled if opt_ctx.get_enable_distributed_optimization() { - let sort_and_limit_optimizer = SortAndLimitPushDownOptimizer::create( - opt_ctx.get_enable_range_shuffle_sort(), - ); + let sort_and_limit_optimizer = SortAndLimitPushDownOptimizer::create(); sort_and_limit_optimizer.optimize(&expr)? } else { expr diff --git a/src/query/sql/src/planner/optimizer/optimizers/distributed/distributed.rs b/src/query/sql/src/planner/optimizer/optimizers/distributed/distributed.rs index abd713fab2ec0..c317c56ed5ed0 100644 --- a/src/query/sql/src/planner/optimizer/optimizers/distributed/distributed.rs +++ b/src/query/sql/src/planner/optimizer/optimizers/distributed/distributed.rs @@ -40,9 +40,7 @@ impl DistributedOptimizer { pub fn new(opt_ctx: Arc) -> Self { Self { ctx: opt_ctx.get_table_ctx(), - sort_limit_optimizer: SortAndLimitPushDownOptimizer::create( - opt_ctx.get_enable_range_shuffle_sort(), - ), + sort_limit_optimizer: SortAndLimitPushDownOptimizer::create(), } } diff --git a/src/query/sql/src/planner/optimizer/optimizers/distributed/sort_and_limit.rs b/src/query/sql/src/planner/optimizer/optimizers/distributed/sort_and_limit.rs index 5445fb4ee0580..43287fcd8e97b 100644 --- a/src/query/sql/src/planner/optimizer/optimizers/distributed/sort_and_limit.rs +++ b/src/query/sql/src/planner/optimizer/optimizers/distributed/sort_and_limit.rs @@ -23,18 +23,15 @@ use crate::plans::Limit; use crate::plans::RelOp; use crate::plans::RelOperator; use crate::plans::Sort; -use crate::plans::SortStep; pub struct SortAndLimitPushDownOptimizer { - range_shuffle: bool, sort_matcher: Matcher, limit_matcher: Matcher, } impl SortAndLimitPushDownOptimizer { - pub fn create(range_shuffle: bool) -> Self { + pub fn create() -> Self { Self { - range_shuffle, sort_matcher: Self::sort_matcher(), limit_matcher: Self::limit_matcher(), } @@ -49,6 +46,14 @@ impl SortAndLimitPushDownOptimizer { // Exchange // \ // * + // Output: + // Sort (after_exchange = true) + // \ + // Exchange + // \ + // Sort (after_exchange = false) + // \ + // * Matcher::MatchOp { op_type: RelOp::Sort, children: vec![Matcher::MatchOp { @@ -90,24 +95,11 @@ impl SortAndLimitPushDownOptimizer { replaced_children.push(Arc::new(new_child)); } let new_sexpr = s_expr.replace_children(replaced_children); - - let apply_topn_res = if self.range_shuffle { - self.apply_shuffle_sort(&new_sexpr)? - } else { - self.apply_dist_sort(&new_sexpr)? - }; + let apply_topn_res = self.apply_sort(&new_sexpr)?; self.apply_limit(&apply_topn_res) } - fn apply_dist_sort(&self, s_expr: &SExpr) -> Result { - // Output: - // Sort (FinalMerge) - // \ - // Exchange - // \ - // Sort (Partial) - // \ - // * + fn apply_sort(&self, s_expr: &SExpr) -> Result { if !self.sort_matcher.matches(s_expr) { return Ok(s_expr.clone()); } @@ -131,7 +123,7 @@ impl SortAndLimitPushDownOptimizer { SExpr::create_unary( Arc::new( Sort { - step: SortStep::Partial, + after_exchange: Some(false), ..sort.clone() } .into(), @@ -142,7 +134,7 @@ impl SortAndLimitPushDownOptimizer { Ok(SExpr::create_unary( Arc::new( Sort { - step: SortStep::FinalMerge, + after_exchange: Some(true), ..sort.clone() } .into(), @@ -151,72 +143,6 @@ impl SortAndLimitPushDownOptimizer { )) } - fn apply_shuffle_sort(&self, s_expr: &SExpr) -> Result { - // Output: - // Sort (Route) - // \ - // Exchange - // \ - // Sort (RangeSort) - // \ - // Exchange - // \ - // Sort (Sample) - // \ - // * - if !self.sort_matcher.matches(s_expr) { - return Ok(s_expr.clone()); - } - - let exchange_sexpr = s_expr.unary_child(); - match exchange_sexpr.plan() { - RelOperator::Exchange(exchange) => match exchange { - // this is window shuffle sort - Exchange::Hash(_) => return Ok(s_expr.clone()), - Exchange::Merge | Exchange::MergeSort => {} - Exchange::Broadcast => unreachable!(), - }, - _ => unreachable!(), - } - - let sort = s_expr.plan.as_sort().unwrap(); - - let input = exchange_sexpr.unary_child_arc(); - - let sort_simple = SExpr::create_unary( - Arc::new( - Sort { - step: SortStep::Sample, - ..sort.clone() - } - .into(), - ), - input, - ); - let exchange1 = SExpr::create_unary(Arc::new(Exchange::MergeSort.into()), sort_simple); - let sort_range = SExpr::create_unary( - Arc::new( - Sort { - step: SortStep::RangeSort, - ..sort.clone() - } - .into(), - ), - exchange1, - ); - let exchange2 = SExpr::create_unary(Arc::new(Exchange::MergeSort.into()), sort_range); - Ok(SExpr::create_unary( - Arc::new( - Sort { - step: SortStep::Route, - ..sort.clone() - } - .into(), - ), - exchange2, - )) - } - fn apply_limit(&self, s_expr: &SExpr) -> Result { if !self.limit_matcher.matches(s_expr) { return Ok(s_expr.clone()); diff --git a/src/query/sql/src/planner/optimizer/optimizers/rule/agg_rules/rule_push_down_limit_aggregate.rs b/src/query/sql/src/planner/optimizer/optimizers/rule/agg_rules/rule_push_down_limit_aggregate.rs index 08b21b4811650..bf2f1c1eb05ab 100644 --- a/src/query/sql/src/planner/optimizer/optimizers/rule/agg_rules/rule_push_down_limit_aggregate.rs +++ b/src/query/sql/src/planner/optimizer/optimizers/rule/agg_rules/rule_push_down_limit_aggregate.rs @@ -26,7 +26,6 @@ use crate::plans::RelOp; use crate::plans::RelOperator; use crate::plans::Sort; use crate::plans::SortItem; -use crate::plans::SortStep; /// Input: Limit | Sort /// \ @@ -111,7 +110,7 @@ impl RulePushDownRankLimitAggregate { let sort = Sort { items: sort_items.clone(), limit: Some(count), - step: SortStep::Single, + after_exchange: None, pre_projection: None, window_partition: None, }; diff --git a/src/query/sql/src/planner/plans/sort.rs b/src/query/sql/src/planner/plans/sort.rs index f11ffa775f0d2..bcf18f814e5fb 100644 --- a/src/query/sql/src/planner/plans/sort.rs +++ b/src/query/sql/src/planner/plans/sort.rs @@ -34,7 +34,7 @@ pub struct Sort { pub items: Vec, pub limit: Option, - pub step: SortStep, + pub after_exchange: Option, /// The columns needed by the plan after the sort plan. /// It's used to build a projection operation before building the sort operator. @@ -44,21 +44,6 @@ pub struct Sort { pub window_partition: Option, } -#[derive(Debug, Hash, Clone, Copy, serde::Serialize, serde::Deserialize, PartialEq, Eq)] -pub enum SortStep { - // single node mode - Single, - - // cluster mode - Partial, // before the exchange plan - FinalMerge, // after the exchange plan - - // range shuffle mode - Sample, - RangeSort, - Route, -} - impl Sort { pub fn used_columns(&self) -> ColumnSet { self.items.iter().map(|item| item.index).collect() From cfa2edeb2aea0aa7115b3e11b995d06fa43477f4 Mon Sep 17 00:00:00 2001 From: coldWater Date: Tue, 1 Jul 2025 19:00:46 +0800 Subject: [PATCH 35/61] x --- .../src/pipelines/builders/builder_sort.rs | 69 +++---- .../src/pipelines/builders/builder_window.rs | 1 + .../transforms/sort/sort_exchange_injector.rs | 8 - .../executor/physical_plans/physical_sort.rs | 180 +++++++++++++----- 4 files changed, 169 insertions(+), 89 deletions(-) diff --git a/src/query/service/src/pipelines/builders/builder_sort.rs b/src/query/service/src/pipelines/builders/builder_sort.rs index ea35f9943d17d..70caa58fb5039 100644 --- a/src/query/service/src/pipelines/builders/builder_sort.rs +++ b/src/query/service/src/pipelines/builders/builder_sort.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::assert_matches::debug_assert_matches; use std::sync::Arc; use databend_common_exception::Result; @@ -54,28 +55,29 @@ impl PipelineBuilder { let input_schema = sort.input.output_schema()?; - if sort.step == SortStep::FinalMerge { - // If the Sort plan is after exchange, we don't need to do a projection, - // because the data is already projected in each cluster node. - if let Some(proj) = &sort.pre_projection { - // Do projection to reduce useless data copying during sorting. - let projection = proj - .iter() - .filter_map(|i| input_schema.index_of(&i.to_string()).ok()) - .collect::>(); - - if projection.len() < input_schema.fields().len() { - // Only if the projection is not a full projection, we need to add a projection transform. - self.main_pipeline.add_transformer(|| { - CompoundBlockOperator::new( - vec![BlockOperator::Project { - projection: projection.clone(), - }], - self.func_ctx.clone(), - input_schema.num_fields(), - ) - }); - } + if let Some(proj) = &sort.pre_projection { + debug_assert_matches!( + sort.step, + SortStep::Single | SortStep::Partial | SortStep::Sample + ); + + // Do projection to reduce useless data copying during sorting. + let projection = proj + .iter() + .filter_map(|i| input_schema.index_of(&i.to_string()).ok()) + .collect::>(); + + if projection.len() < input_schema.fields().len() { + // Only if the projection is not a full projection, we need to add a projection transform. + self.main_pipeline.add_transformer(|| { + CompoundBlockOperator::new( + vec![BlockOperator::Project { + projection: projection.clone(), + }], + self.func_ctx.clone(), + input_schema.num_fields(), + ) + }); } } @@ -140,19 +142,18 @@ impl PipelineBuilder { SortStep::Single => { // Build for single node mode. // We build the full sort pipeline for it. - if self.settings.get_enable_range_shuffle_sort()? - && self.main_pipeline.output_len() > 1 - { - builder - .remove_order_col_at_last() - .build_range_shuffle_sort_pipeline(&mut self.main_pipeline) - } else { - builder - .remove_order_col_at_last() - .build_full_sort_pipeline(&mut self.main_pipeline) - } + builder + .remove_order_col_at_last() + .build_full_sort_pipeline(&mut self.main_pipeline) + } + SortStep::Sample => { + builder + .remove_order_col_at_last() + .build_range_shuffle_sort_pipeline(&mut self.main_pipeline); + todo!() } - _ => unimplemented!(), + SortStep::RangeSort => todo!(), + SortStep::Route => todo!(), } } } diff --git a/src/query/service/src/pipelines/builders/builder_window.rs b/src/query/service/src/pipelines/builders/builder_window.rs index cb82feca68ca3..7906dfd96f164 100644 --- a/src/query/service/src/pipelines/builders/builder_window.rs +++ b/src/query/service/src/pipelines/builders/builder_window.rs @@ -24,6 +24,7 @@ use databend_common_expression::SortColumnDescription; use databend_common_pipeline_core::processors::Processor; use databend_common_pipeline_core::processors::ProcessorPtr; use databend_common_pipeline_transforms::MemorySettings; +use databend_common_sql::executor::physical_plans::SortStep; use databend_common_sql::executor::physical_plans::Window; use databend_common_sql::executor::physical_plans::WindowPartition; use databend_storages_common_cache::TempDirManager; diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs index e567af11fcaa0..a42454de0be45 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs @@ -14,17 +14,9 @@ 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_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_settings::FlightCompression; -use databend_common_storage::DataOperator; use crate::servers::flight::v1::exchange::DataExchange; use crate::servers::flight::v1::exchange::ExchangeInjector; diff --git a/src/query/sql/src/executor/physical_plans/physical_sort.rs b/src/query/sql/src/executor/physical_plans/physical_sort.rs index cdf7ab5c0cc53..d064f17673202 100644 --- a/src/query/sql/src/executor/physical_plans/physical_sort.rs +++ b/src/query/sql/src/executor/physical_plans/physical_sort.rs @@ -22,6 +22,8 @@ use databend_common_pipeline_transforms::processors::sort::utils::ORDER_COL_NAME use crate::executor::explain::PlanStatsInfo; use crate::executor::physical_plans::common::SortDesc; +use crate::executor::physical_plans::Exchange; +use crate::executor::physical_plans::FragmentKind; use crate::executor::physical_plans::WindowPartition; use crate::executor::physical_plans::WindowPartitionTopN; use crate::executor::physical_plans::WindowPartitionTopNFunc; @@ -79,40 +81,54 @@ impl Sort { pub fn output_schema(&self) -> Result { let input_schema = self.input.output_schema()?; - let mut fields = input_schema.fields().clone(); - if self.step == SortStep::FinalMerge { - // If the plan is after exchange plan in cluster mode, - // the order column is at the last of the input schema. - debug_assert_eq!(fields.last().unwrap().name(), ORDER_COL_NAME); - debug_assert_eq!( - fields.last().unwrap().data_type(), - &self.order_col_type(&input_schema)? - ); - fields.pop(); - } else { - if let Some(proj) = &self.pre_projection { - let fileted_fields = proj - .iter() - .filter_map(|index| input_schema.field_with_name(&index.to_string()).ok()) - .cloned() - .collect::>(); - if fileted_fields.len() < fields.len() { - // Only if the projection is not a full projection, we need to add a projection transform. - fields = fileted_fields - } + match self.step { + SortStep::FinalMerge | SortStep::Route => { + let mut fields = input_schema.fields().clone(); + // If the plan is after exchange plan in cluster mode, + // the order column is at the last of the input schema. + debug_assert_eq!(fields.last().unwrap().name(), ORDER_COL_NAME); + debug_assert_eq!( + fields.last().unwrap().data_type(), + &self.order_col_type(&input_schema)? + ); + fields.pop(); + Ok(DataSchemaRefExt::create(fields)) } + SortStep::RangeSort => Ok(input_schema), + SortStep::Single | SortStep::Partial | SortStep::Sample => { + let mut fields = self + .pre_projection + .as_ref() + .and_then(|proj| { + let fileted_fields = proj + .iter() + .map(|index| { + input_schema + .field_with_name(&index.to_string()) + .unwrap() + .clone() + }) + .collect::>(); - if self.step == SortStep::Partial { - // If the plan is before exchange plan in cluster mode, - // the order column should be added to the output schema. - fields.push(DataField::new( - ORDER_COL_NAME, - self.order_col_type(&input_schema)?, - )); + if fileted_fields.len() < input_schema.fields().len() { + // Only if the projection is not a full projection, we need to add a projection transform. + Some(fileted_fields) + } else { + None + } + }) + .unwrap_or_else(|| input_schema.fields().clone()); + if self.step != SortStep::Single { + // If the plan is before exchange plan in cluster mode, + // the order column should be added to the output schema. + fields.push(DataField::new( + ORDER_COL_NAME, + self.order_col_type(&input_schema)?, + )); + } + Ok(DataSchemaRefExt::create(fields)) } } - - Ok(DataSchemaRefExt::create(fields)) } } @@ -135,7 +151,6 @@ impl PhysicalPlanBuilder { } else { None }; - let input_plan = self.build(s_expr.child(0)?, required).await?; let order_by = sort .items @@ -148,17 +163,6 @@ impl PhysicalPlanBuilder { }) .collect::>(); - let enable_range_shuffle_sort = self.ctx.get_settings().get_enable_range_shuffle_sort()?; - let sort_step = match sort.after_exchange { - Some(false) => SortStep::Partial, - Some(true) => SortStep::FinalMerge, - None => SortStep::Single, - }; - let broadcast_id = match sort.after_exchange { - Some(false) if enable_range_shuffle_sort => Some(self.ctx.get_next_broadcast_id()), - _ => None, - }; - // Add WindowPartition for parallel sort in window. if let Some(window) = &sort.window_partition { let window_partition = window @@ -167,9 +171,17 @@ impl PhysicalPlanBuilder { .map(|v| v.index) .collect::>(); + let sort_step = match sort.after_exchange { + Some(false) => SortStep::Partial, + Some(true) => SortStep::FinalMerge, + None => SortStep::Single, + }; + + let input_plan = self.build(s_expr.unary_child(), required).await?; + return Ok(PhysicalPlan::WindowPartition(WindowPartition { plan_id: 0, - input: Box::new(input_plan.clone()), + input: Box::new(input_plan), partition_by: window_partition.clone(), order_by: order_by.clone(), sort_step, @@ -187,14 +199,88 @@ impl PhysicalPlanBuilder { }; // 2. Build physical plan. - Ok(PhysicalPlan::Sort(Sort { + let Some(after_exchange) = sort.after_exchange else { + let input_plan = self.build(s_expr.unary_child(), required).await?; + return Ok(PhysicalPlan::Sort(Sort { + plan_id: 0, + input: Box::new(input_plan), + order_by, + limit: sort.limit, + step: SortStep::Single, + pre_projection, + broadcast_id: None, + stat_info: Some(stat_info), + })); + }; + + if !self.ctx.get_settings().get_enable_range_shuffle_sort()? { + let input_plan = self.build(s_expr.unary_child(), required).await?; + return if !after_exchange { + Ok(PhysicalPlan::Sort(Sort { + plan_id: 0, + input: Box::new(input_plan), + order_by, + limit: sort.limit, + step: SortStep::Partial, + pre_projection, + broadcast_id: None, + stat_info: Some(stat_info), + })) + } else { + Ok(PhysicalPlan::Sort(Sort { + plan_id: 0, + input: Box::new(input_plan), + order_by, + limit: sort.limit, + step: SortStep::FinalMerge, + pre_projection: None, + broadcast_id: None, + stat_info: Some(stat_info), + })) + }; + } + + if after_exchange { + let input_plan = self.build(s_expr.unary_child(), required).await?; + return Ok(PhysicalPlan::Sort(Sort { + plan_id: 0, + input: Box::new(input_plan), + order_by, + limit: sort.limit, + step: SortStep::Route, + pre_projection: None, + broadcast_id: None, + stat_info: Some(stat_info), + })); + } + + let input_plan = self.build(s_expr.unary_child(), required).await?; + let sample = PhysicalPlan::Sort(Sort { plan_id: 0, input: Box::new(input_plan), - order_by, + order_by: order_by.clone(), limit: sort.limit, - step: sort_step, + step: SortStep::Sample, pre_projection, - broadcast_id, + broadcast_id: Some(self.ctx.get_next_broadcast_id()), + stat_info: Some(stat_info.clone()), + }); + let exchange = PhysicalPlan::Exchange(Exchange { + plan_id: 0, + input: Box::new(sample), + kind: FragmentKind::Normal, + keys: vec![], + ignore_exchange: false, + allow_adjust_parallelism: false, + }); + Ok(PhysicalPlan::Sort(Sort { + plan_id: 0, + input: Box::new(exchange), + order_by, + limit: sort.limit, + step: SortStep::RangeSort, + pre_projection: None, + broadcast_id: None, stat_info: Some(stat_info), })) } From e0b17b52e51b396d63bc68fc4235af83fd1720c8 Mon Sep 17 00:00:00 2001 From: coldWater Date: Wed, 2 Jul 2025 11:30:37 +0800 Subject: [PATCH 36/61] x --- .../pipelines/builders/builder_recluster.rs | 2 +- .../src/pipelines/builders/builder_sort.rs | 79 +++++++++++++++++- .../processors/transforms/sort/bounds.rs | 15 +++- .../transforms/sort/sort_builder.rs | 13 +-- .../transforms/sort/sort_shuffle.rs | 80 ++++++++++--------- 5 files changed, 134 insertions(+), 55 deletions(-) diff --git a/src/query/service/src/pipelines/builders/builder_recluster.rs b/src/query/service/src/pipelines/builders/builder_recluster.rs index 05d2d63dd55d7..32884655711a8 100644 --- a/src/query/service/src/pipelines/builders/builder_recluster.rs +++ b/src/query/service/src/pipelines/builders/builder_recluster.rs @@ -156,7 +156,7 @@ impl PipelineBuilder { ); let sort_pipeline_builder = - SortPipelineBuilder::create(self.ctx.clone(), schema, sort_descs.into())? + SortPipelineBuilder::create(self.ctx.clone(), schema, sort_descs.into(), None)? .with_block_size_hit(sort_block_size) .remove_order_col_at_last(); // Todo(zhyass): Recluster will no longer perform sort in the near future. diff --git a/src/query/service/src/pipelines/builders/builder_sort.rs b/src/query/service/src/pipelines/builders/builder_sort.rs index 70caa58fb5039..9b5b9521c6b1e 100644 --- a/src/query/service/src/pipelines/builders/builder_sort.rs +++ b/src/query/service/src/pipelines/builders/builder_sort.rs @@ -114,7 +114,7 @@ impl PipelineBuilder { self.main_pipeline.try_resize(max_threads)?; } - let builder = SortPipelineBuilder::create(self.ctx.clone(), plan_schema, sort_desc)? + let builder = SortPipelineBuilder::create(self.ctx.clone(), plan_schema, sort_desc, None)? .with_limit(limit); match sort_step { @@ -149,7 +149,7 @@ impl PipelineBuilder { SortStep::Sample => { builder .remove_order_col_at_last() - .build_range_shuffle_sort_pipeline(&mut self.main_pipeline); + .build_range_shuffle_sort_pipeline(&mut self.main_pipeline)?; todo!() } SortStep::RangeSort => todo!(), @@ -166,6 +166,7 @@ pub struct SortPipelineBuilder { block_size: usize, remove_order_col_at_last: bool, enable_loser_tree: bool, + broadcast_id: Option, } impl SortPipelineBuilder { @@ -173,6 +174,7 @@ impl SortPipelineBuilder { ctx: Arc, schema: DataSchemaRef, sort_desc: Arc<[SortColumnDescription]>, + broadcast_id: Option, ) -> Result { let settings = ctx.get_settings(); let block_size = settings.get_max_block_size()? as usize; @@ -185,6 +187,7 @@ impl SortPipelineBuilder { block_size, remove_order_col_at_last: false, enable_loser_tree, + broadcast_id, }) } @@ -261,9 +264,16 @@ impl SortPipelineBuilder { Ok(ProcessorPtr::create(builder.build_collect(input, output)?)) })?; - let state = SortSampleState::new(inputs, num_exec, builder.inner_schema(), max_block_size); + let state = SortSampleState::new( + inputs, + num_exec, + builder.inner_schema(), + max_block_size, + self.ctx.clone(), + self.broadcast_id.unwrap(), + ); - builder.add_shuffle(pipeline, state.clone())?; + builder.add_shuffle(pipeline, state)?; pipeline.exchange(num_exec, Arc::new(SortRangeExchange)); @@ -286,6 +296,67 @@ impl SortPipelineBuilder { }) } + fn build_sort_part(self, pipeline: &mut Pipeline) -> Result<()> { + let inputs = pipeline.output_len(); + let settings = self.ctx.get_settings(); + let num_exec = inputs; + let max_block_size = settings.get_max_block_size()? as usize; + + // Partial sort + pipeline.add_transformer(|| { + TransformSortPartial::new( + LimitType::from_limit_rows(self.limit), + self.sort_desc.clone(), + ) + }); + + let spiller = { + let location_prefix = self.ctx.query_id_spill_prefix(); + let config = SpillerConfig { + spiller_type: SpillerType::OrderBy, + location_prefix, + disk_spill: None, + use_parquet: settings.get_spilling_file_format()?.is_parquet(), + }; + let op = DataOperator::instance().spill_operator(); + Arc::new(Spiller::create(self.ctx.clone(), op, config)?) + }; + + let memory_settings = MemorySettings::from_sort_settings(&self.ctx)?; + let enable_loser_tree = settings.get_enable_loser_tree_merge_sort()?; + + let builder = TransformSortBuilder::create( + self.schema.clone(), + self.sort_desc.clone(), + max_block_size, + spiller, + ) + .with_limit(self.limit) + .with_order_col_generated(false) + .with_output_order_col(false) + .with_memory_settings(memory_settings) + .with_enable_loser_tree(enable_loser_tree); + + pipeline.add_transform(|input, output| { + Ok(ProcessorPtr::create(builder.build_collect(input, output)?)) + })?; + + let state = SortSampleState::new( + inputs, + num_exec, + builder.inner_schema(), + max_block_size, + self.ctx.clone(), + self.broadcast_id.unwrap(), + ); + + builder.add_shuffle(pipeline, state.clone())?; + + // pipeline.exchange(num_exec, Arc::new(SortRangeExchange)); + + Ok(()) + } + fn build_merge_sort(&self, pipeline: &mut Pipeline, order_col_generated: bool) -> Result<()> { // Merge sort let need_multi_merge = pipeline.output_len() > 1; diff --git a/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs b/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs index dca8038f3315f..e56fe49fdbf51 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs @@ -13,6 +13,8 @@ // limitations under the License. use databend_common_exception::Result; +use databend_common_expression::BlockMetaInfo; +use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::Column; use databend_common_expression::DataBlock; use databend_common_expression::DataField; @@ -23,7 +25,7 @@ use databend_common_pipeline_transforms::sort::LoserTreeMerger; use databend_common_pipeline_transforms::sort::Rows; use databend_common_pipeline_transforms::sort::SortedStream; -#[derive(Debug, PartialEq, Eq, Default, Clone)] +#[derive(Debug, PartialEq, Eq, Default, Clone, serde::Serialize, serde::Deserialize)] pub struct Bounds( // stored in reverse order of Column. Vec, @@ -196,6 +198,17 @@ impl SortedStream for Bounds { } } +#[typetag::serde(name = "sort_bounds")] +impl BlockMetaInfo for Bounds { + fn equals(&self, info: &Box) -> bool { + Bounds::downcast_ref_from(info).is_some_and(|other| self == other) + } + + fn clone_self(&self) -> Box { + Box::new(self.clone()) + } +} + #[cfg(test)] mod tests { use databend_common_expression::types::Int32Type; diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs index 4bfbe9f9646ff..27be48d2555c7 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs @@ -120,7 +120,6 @@ impl TransformSortBuilder { input, output, typ: SortType::Sort, - id: 0, state: None, }; @@ -139,7 +138,6 @@ impl TransformSortBuilder { input, output, typ: SortType::Collect, - id: 0, state: None, }; @@ -158,7 +156,6 @@ impl TransformSortBuilder { input, output, typ: SortType::Execute, - id: 0, state: None, }; @@ -170,7 +167,7 @@ impl TransformSortBuilder { input: Arc, output: Arc, id: usize, - state: Arc, + state: SortSampleState, ) -> Result> { self.check(); @@ -179,7 +176,6 @@ impl TransformSortBuilder { input, output, typ: SortType::Shuffle, - id, state: Some(state), }; @@ -198,7 +194,6 @@ impl TransformSortBuilder { input, output, typ: SortType::Combine, - id: 0, state: None, }; @@ -232,7 +227,7 @@ impl TransformSortBuilder { add_order_field(self.schema.clone(), &self.sort_desc) } - pub fn add_shuffle(&self, pipeline: &mut Pipeline, state: Arc) -> Result<()> { + pub fn add_shuffle(&self, pipeline: &mut Pipeline, state: SortSampleState) -> Result<()> { use std::sync::atomic; let i = atomic::AtomicUsize::new(0); pipeline.add_transform(|input, output| { @@ -252,8 +247,7 @@ struct Build<'a> { typ: SortType, input: Arc, output: Arc, - id: usize, - state: Option>, + state: Option, } impl Build<'_> { @@ -309,7 +303,6 @@ impl Build<'_> { Ok(Box::new(TransformSortShuffle::::new( self.input.clone(), self.output.clone(), - self.id, self.state.clone().unwrap(), self.params.spiller.clone(), ))) diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_shuffle.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_shuffle.rs index 41c4ae5360132..b39a612c67095 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_shuffle.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_shuffle.rs @@ -16,9 +16,8 @@ use std::any::Any; use std::assert_matches::assert_matches; use std::marker::PhantomData; use std::sync::Arc; -use std::sync::RwLock; -use databend_common_base::base::WatchNotify; +use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::DataBlock; @@ -33,6 +32,7 @@ use crate::pipelines::processors::Event; use crate::pipelines::processors::InputPort; use crate::pipelines::processors::OutputPort; use crate::pipelines::processors::Processor; +use crate::sessions::QueryContext; use crate::spillers::Spiller; #[derive(Debug)] @@ -45,9 +45,8 @@ enum Step { pub struct TransformSortShuffle { input: Arc, output: Arc, - id: usize, step: Step, - state: Arc, + state: SortSampleState, spiller: Arc, _r: PhantomData, } @@ -56,14 +55,12 @@ impl TransformSortShuffle { pub fn new( input: Arc, output: Arc, - id: usize, - state: Arc, + state: SortSampleState, spiller: Arc, ) -> Self { Self { input, output, - id, state, spiller, step: Step::None, @@ -94,7 +91,7 @@ impl TransformSortShuffle { } let base = { - let inner = self.state.inner.read().unwrap(); + let inner = &self.state.inner; Base { schema: inner.schema.clone(), spiller: self.spiller.clone(), @@ -175,7 +172,7 @@ impl Processor for TransformSortShuffle { } if self.input.is_finished() { - if self.state.done.has_notified() { + if self.state.inner.bounds.is_some() { self.output.finish(); Ok(Event::Finished) } else { @@ -194,8 +191,8 @@ impl Processor for TransformSortShuffle { Step::Meta(meta) => meta.generate_bounds(), _ => unreachable!(), }; - self.state.commit_sample::(self.id, bounds)?; - self.state.done.notified().await; + + self.state.commit_sample::(bounds).await?; self.step = Step::Scattered(self.scatter().await?); Ok(()) } @@ -222,9 +219,11 @@ impl SortCollectedMeta { } } +#[derive(Clone)] pub struct SortSampleState { - inner: RwLock, - pub(super) done: WatchNotify, + inner: StateInner, + ctx: Arc, + broadcast_id: u32, } impl SortSampleState { @@ -233,55 +232,58 @@ impl SortSampleState { partitions: usize, schema: DataSchemaRef, batch_rows: usize, - ) -> Arc { - Arc::new(SortSampleState { - inner: RwLock::new(StateInner { + ctx: Arc, + broadcast_id: u32, + ) -> SortSampleState { + SortSampleState { + inner: StateInner { partitions, schema, - partial: vec![None; inputs], + partial: vec![], bounds: None, batch_rows, - }), - done: WatchNotify::new(), - }) + }, + ctx, + broadcast_id, + } } - pub fn commit_sample(&self, id: usize, bounds: Bounds) -> Result { - let mut inner = self.inner.write().unwrap(); - - let x = inner.partial[id].replace(bounds); - assert!(x.is_none()); - let done = inner.partial.iter().all(Option::is_some); - if done { - inner.determine_bounds::()?; - self.done.notify_waiters(); + pub async fn commit_sample(&mut self, bounds: Bounds) -> Result<()> { + let sender = self.ctx.broadcast_source_sender(self.broadcast_id); + sender + .send(Box::new(bounds)) + .await + .map_err(|_| ErrorCode::TokioError("send sort bounds failed"))?; + sender.close(); + + let receiver = self.ctx.broadcast_sink_receiver(self.broadcast_id); + while let Ok(r) = receiver.recv().await { + self.inner.partial.push(Bounds::downcast_from(r).unwrap()); } - Ok(done) + + self.inner.determine_bounds::()?; + Ok(()) } pub fn bounds(&self) -> Bounds { - self.inner - .read() - .unwrap() - .bounds - .clone() - .unwrap_or_default() + self.inner.bounds.clone().unwrap_or_default() } } +#[derive(Clone)] + struct StateInner { // target partitions partitions: usize, schema: DataSchemaRef, - partial: Vec>, + partial: Vec, bounds: Option, batch_rows: usize, } impl StateInner { fn determine_bounds(&mut self) -> Result<()> { - let v = self.partial.drain(..).map(Option::unwrap).collect(); - let bounds = Bounds::merge::(v, self.batch_rows)?; + let bounds = Bounds::merge::(std::mem::take(&mut self.partial), self.batch_rows)?; let n = self.partitions - 1; let bounds = if bounds.len() < n { From 6fb293d115b67f759f874cea4b9660360d9f5d58 Mon Sep 17 00:00:00 2001 From: coldWater Date: Thu, 3 Jul 2025 13:07:21 +0800 Subject: [PATCH 37/61] x --- .../src/pipelines/builders/builder_sort.rs | 117 +++--- .../processors/transforms/sort/bounds.rs | 4 + .../processors/transforms/sort/mod.rs | 1 + .../transforms/sort/sort_builder.rs | 40 +- .../transforms/sort/sort_exchange_injector.rs | 121 +++++- .../transforms/sort/sort_shuffle.rs | 370 ++++++++---------- 6 files changed, 349 insertions(+), 304 deletions(-) diff --git a/src/query/service/src/pipelines/builders/builder_sort.rs b/src/query/service/src/pipelines/builders/builder_sort.rs index 9b5b9521c6b1e..78913984e1a84 100644 --- a/src/query/service/src/pipelines/builders/builder_sort.rs +++ b/src/query/service/src/pipelines/builders/builder_sort.rs @@ -31,17 +31,19 @@ use databend_common_sql::evaluator::BlockOperator; use databend_common_sql::evaluator::CompoundBlockOperator; use databend_common_sql::executor::physical_plans::Sort; use databend_common_sql::executor::physical_plans::SortStep; +use databend_common_sql::executor::PhysicalPlan; use databend_common_storage::DataOperator; use databend_common_storages_fuse::TableContext; use databend_storages_common_cache::TempDirManager; use crate::pipelines::memory_settings::MemorySettingsExt; use crate::pipelines::processors::transforms::add_range_shuffle_route; +use crate::pipelines::processors::transforms::SortInjector; use crate::pipelines::processors::transforms::SortRangeExchange; -use crate::pipelines::processors::transforms::SortSampleState; use crate::pipelines::processors::transforms::TransformLimit; use crate::pipelines::processors::transforms::TransformSortBuilder; use crate::pipelines::PipelineBuilder; +use crate::servers::flight::v1::exchange::ExchangeInjector; use crate::sessions::QueryContext; use crate::spillers::Spiller; use crate::spillers::SpillerConfig; @@ -49,40 +51,8 @@ use crate::spillers::SpillerDiskConfig; use crate::spillers::SpillerType; impl PipelineBuilder { - // The pipeline graph of distributed sort can be found in https://github.com/datafuselabs/databend/pull/13881 pub(crate) fn build_sort(&mut self, sort: &Sort) -> Result<()> { - self.build_pipeline(&sort.input)?; - - let input_schema = sort.input.output_schema()?; - - if let Some(proj) = &sort.pre_projection { - debug_assert_matches!( - sort.step, - SortStep::Single | SortStep::Partial | SortStep::Sample - ); - - // Do projection to reduce useless data copying during sorting. - let projection = proj - .iter() - .filter_map(|i| input_schema.index_of(&i.to_string()).ok()) - .collect::>(); - - if projection.len() < input_schema.fields().len() { - // Only if the projection is not a full projection, we need to add a projection transform. - self.main_pipeline.add_transformer(|| { - CompoundBlockOperator::new( - vec![BlockOperator::Project { - projection: projection.clone(), - }], - self.func_ctx.clone(), - input_schema.num_fields(), - ) - }); - } - } - let plan_schema = sort.output_schema()?; - let sort_desc = sort .order_by .iter() @@ -95,19 +65,37 @@ impl PipelineBuilder { }) }) .collect::>>()?; + let sort_desc = sort_desc.into(); - self.build_sort_pipeline(plan_schema, sort_desc, sort.limit, sort.step) - } + if sort.step != SortStep::RangeSort { + self.build_pipeline(&sort.input)?; + } + + if let Some(proj) = &sort.pre_projection { + debug_assert_matches!( + sort.step, + SortStep::Single | SortStep::Partial | SortStep::Sample + ); + + let input_schema = sort.input.output_schema()?; + // Do projection to reduce useless data copying during sorting. + let projection = proj + .iter() + .map(|i| input_schema.index_of(&i.to_string()).unwrap()) + .collect::>(); + + self.main_pipeline.add_transformer(|| { + CompoundBlockOperator::new( + vec![BlockOperator::Project { + projection: projection.clone(), + }], + self.func_ctx.clone(), + input_schema.num_fields(), + ) + }); + } - fn build_sort_pipeline( - &mut self, - plan_schema: DataSchemaRef, - sort_desc: Vec, - limit: Option, - sort_step: SortStep, - ) -> Result<()> { let max_threads = self.settings.get_max_threads()? as usize; - let sort_desc = sort_desc.into(); // TODO(Winter): the query will hang in MultiSortMergeProcessor when max_threads == 1 and output_len != 1 if self.main_pipeline.output_len() == 1 || max_threads == 1 { @@ -115,9 +103,9 @@ impl PipelineBuilder { } let builder = SortPipelineBuilder::create(self.ctx.clone(), plan_schema, sort_desc, None)? - .with_limit(limit); + .with_limit(sort.limit); - match sort_step { + match sort.step { SortStep::FinalMerge => { // Build for the coordinator node. // We only build a `MultiSortMergeTransform`, @@ -146,13 +134,18 @@ impl PipelineBuilder { .remove_order_col_at_last() .build_full_sort_pipeline(&mut self.main_pipeline) } - SortStep::Sample => { - builder - .remove_order_col_at_last() - .build_range_shuffle_sort_pipeline(&mut self.main_pipeline)?; + SortStep::Sample => builder.build_sort_part(&mut self.main_pipeline), + SortStep::RangeSort => { + if matches!(*sort.input, PhysicalPlan::ExchangeSource(_)) { + let exchange = builder.exchange_injector(); + let old_inject = std::mem::replace(&mut self.exchange_injector, exchange); + self.build_pipeline(&sort.input)?; + self.exchange_injector = old_inject; + } else { + self.build_pipeline(&sort.input)?; + } todo!() } - SortStep::RangeSort => todo!(), SortStep::Route => todo!(), } } @@ -264,16 +257,13 @@ impl SortPipelineBuilder { Ok(ProcessorPtr::create(builder.build_collect(input, output)?)) })?; - let state = SortSampleState::new( - inputs, - num_exec, + builder.add_bound_broadcast( + pipeline, builder.inner_schema(), max_block_size, self.ctx.clone(), self.broadcast_id.unwrap(), - ); - - builder.add_shuffle(pipeline, state)?; + )?; pipeline.exchange(num_exec, Arc::new(SortRangeExchange)); @@ -341,18 +331,13 @@ impl SortPipelineBuilder { Ok(ProcessorPtr::create(builder.build_collect(input, output)?)) })?; - let state = SortSampleState::new( - inputs, - num_exec, + builder.add_bound_broadcast( + pipeline, builder.inner_schema(), max_block_size, self.ctx.clone(), self.broadcast_id.unwrap(), - ); - - builder.add_shuffle(pipeline, state.clone())?; - - // pipeline.exchange(num_exec, Arc::new(SortRangeExchange)); + )?; Ok(()) } @@ -456,4 +441,8 @@ impl SortPipelineBuilder { ) } } + + pub fn exchange_injector(self) -> Arc { + Arc::new(SortInjector {}) + } } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs b/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs index e56fe49fdbf51..8ec18f1bc7d9a 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs @@ -184,6 +184,10 @@ impl Bounds { indices.len(), )]) } + + pub fn dedup(&self) -> Self { + self.dedup_reduce::(self.len()) + } } impl SortedStream for Bounds { diff --git a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs index 1242f265c3ffc..d41aecd9e4ee7 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs @@ -38,6 +38,7 @@ mod sort_spill; pub use sort_builder::*; pub use sort_exchange::*; +pub use sort_exchange_injector::*; pub use sort_route::*; pub use sort_shuffle::*; diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs index 27be48d2555c7..3e3b9d61d0d24 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs @@ -39,16 +39,18 @@ use super::sort_collect::TransformSortCollect; use super::sort_combine::TransformSortCombine; use super::sort_execute::TransformSortExecute; use super::sort_shuffle::SortSampleState; -use super::sort_shuffle::TransformSortShuffle; +use super::sort_shuffle::TransformSortBoundBroadcast; use super::Base; +use crate::servers::flight::v1::exchange::ExchangeInjector; +use crate::sessions::QueryContext; use crate::spillers::Spiller; enum SortType { Sort, Collect, - Execute, - Shuffle, + BoundBroadcast, Combine, + Execute, } pub struct TransformSortBuilder { @@ -162,11 +164,10 @@ impl TransformSortBuilder { select_row_type(&mut build) } - pub fn build_shuffle( + pub fn build_bound_broadcast( &self, input: Arc, output: Arc, - id: usize, state: SortSampleState, ) -> Result> { self.check(); @@ -175,7 +176,7 @@ impl TransformSortBuilder { params: self, input, output, - typ: SortType::Shuffle, + typ: SortType::BoundBroadcast, state: Some(state), }; @@ -227,19 +228,27 @@ impl TransformSortBuilder { add_order_field(self.schema.clone(), &self.sort_desc) } - pub fn add_shuffle(&self, pipeline: &mut Pipeline, state: SortSampleState) -> Result<()> { - use std::sync::atomic; - let i = atomic::AtomicUsize::new(0); + pub fn add_bound_broadcast( + &self, + pipeline: &mut Pipeline, + schema: DataSchemaRef, + batch_rows: usize, + ctx: Arc, + broadcast_id: u32, + ) -> Result<()> { + let state = SortSampleState::new(schema, batch_rows, ctx, broadcast_id); pipeline.add_transform(|input, output| { - let id = i.fetch_add(1, atomic::Ordering::AcqRel); - Ok(ProcessorPtr::create(self.build_shuffle( + Ok(ProcessorPtr::create(self.build_bound_broadcast( input, output, - id, state.clone(), )?)) }) } + + pub fn exchange_injector(&self) -> Arc { + todo!() + } } struct Build<'a> { @@ -300,12 +309,11 @@ impl Build<'_> { fn build_sort_shuffle(&mut self) -> Result> where R: Rows + 'static { - Ok(Box::new(TransformSortShuffle::::new( + Ok(TransformSortBoundBroadcast::::create( self.input.clone(), self.output.clone(), self.state.clone().unwrap(), - self.params.spiller.clone(), - ))) + )) } fn build_sort_combine(&mut self) -> Result> @@ -347,7 +355,7 @@ impl RowsTypeVisitor for Build<'_> { true => self.build_sort_exec::>(), false => self.build_sort_exec::>(), }, - SortType::Shuffle => self.build_sort_shuffle::(), + SortType::BoundBroadcast => self.build_sort_shuffle::(), SortType::Combine => self.build_sort_combine::(), } } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs index a42454de0be45..31be6944b0dd2 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs @@ -14,17 +14,25 @@ use std::sync::Arc; +use arrow_ipc::writer::IpcWriteOptions; use databend_common_exception::Result; +use databend_common_expression::BlockMetaInfoDowncast; +use databend_common_expression::DataBlock; use databend_common_pipeline_core::Pipeline; +use databend_common_pipeline_transforms::BlockMetaTransform; use databend_common_settings::FlightCompression; +use crate::pipelines::processors::transforms::aggregator::FlightSerializedMeta; +use crate::pipelines::processors::transforms::sort::SortCollectedMeta; use crate::servers::flight::v1::exchange::DataExchange; use crate::servers::flight::v1::exchange::ExchangeInjector; +use crate::servers::flight::v1::exchange::ExchangeShuffleMeta; 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; +use crate::spillers::Spiller; pub struct SortInjector {} @@ -34,11 +42,14 @@ impl ExchangeInjector for SortInjector { ctx: &Arc, exchange: &DataExchange, ) -> Result>> { - todo!() + match exchange { + DataExchange::Merge(_) | DataExchange::Broadcast(_) => unreachable!(), + DataExchange::ShuffleDataExchange(exchange) => Ok(Arc::new(Box::new(DummyScatter {}))), + } } fn exchange_sorting(&self) -> Option> { - todo!() + None } fn apply_merge_serializer( @@ -47,21 +58,21 @@ impl ExchangeInjector for SortInjector { compression: Option, pipeline: &mut Pipeline, ) -> Result<()> { - todo!() + unreachable!() } - fn apply_shuffle_serializer( + fn apply_merge_deserializer( &self, - params: &ShuffleExchangeParams, - compression: Option, + params: &MergeExchangeParams, pipeline: &mut Pipeline, ) -> Result<()> { - todo!() + unreachable!() } - fn apply_merge_deserializer( + fn apply_shuffle_serializer( &self, - params: &MergeExchangeParams, + params: &ShuffleExchangeParams, + compression: Option, pipeline: &mut Pipeline, ) -> Result<()> { todo!() @@ -75,3 +86,95 @@ impl ExchangeInjector for SortInjector { todo!() } } + +pub struct DummyScatter {} + +impl FlightScatter for DummyScatter { + fn execute(&self, mut data_block: DataBlock) -> Result> { + Ok(vec![data_block]) + } +} + +pub struct TransformExchangeSortSerializer { + ctx: Arc, + local_pos: usize, + options: IpcWriteOptions, + + // params: Arc, + spiller: Arc, +} + +impl BlockMetaTransform for TransformExchangeSortSerializer { + const NAME: &'static str = "TransformExchangeSortSerializer"; + + fn transform(&mut self, meta: ExchangeShuffleMeta) -> Result> { + let serialized_blocks = meta + .blocks + .into_iter() + .map(|mut block| { + let SortCollectedMeta { + params, + bounds, + blocks, + } = SortCollectedMeta::downcast_from(block.take_meta().unwrap()).unwrap(); + + // 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, + // )?, + // }, + }) + .collect(); + + // let meta = SortCollectedMeta::downcast_from(block.take_meta().unwrap()).unwrap(); + + // match AggregateMeta::downcast_from(block.take_meta().unwrap()) { + // None => unreachable!(), + // Some(AggregateMeta::Spilled(_)) => unreachable!(), + // Some(AggregateMeta::Serialized(_)) => unreachable!(), + // Some(AggregateMeta::BucketSpilled(_)) => unreachable!(), + // Some(AggregateMeta::Partitioned { .. }) => unreachable!(), + // Some(AggregateMeta::AggregateSpilling(payload)) => { + // serialized_blocks.push(FlightSerialized::Future( + + // )); + // } + + // 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), + )]) + } +} diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_shuffle.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_shuffle.rs index b39a612c67095..bdee0aac93223 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_shuffle.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_shuffle.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::assert_matches::assert_matches; use std::marker::PhantomData; use std::sync::Arc; @@ -22,233 +20,61 @@ use databend_common_exception::Result; use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::DataBlock; use databend_common_expression::DataSchemaRef; +use databend_common_pipeline_core::processors::Processor; use databend_common_pipeline_transforms::processors::sort::Rows; +use databend_common_pipeline_transforms::AsyncAccumulatingTransform; +use databend_common_pipeline_transforms::AsyncAccumulatingTransformer; use super::bounds::Bounds; -use super::Base; use super::SortCollectedMeta; -use super::SortScatteredMeta; -use crate::pipelines::processors::Event; use crate::pipelines::processors::InputPort; use crate::pipelines::processors::OutputPort; -use crate::pipelines::processors::Processor; use crate::sessions::QueryContext; -use crate::spillers::Spiller; -#[derive(Debug)] -enum Step { - None, - Meta(Box), - Scattered(Vec>), -} - -pub struct TransformSortShuffle { - input: Arc, - output: Arc, - step: Step, +pub struct TransformSortBoundBroadcast { + buffer: Vec, state: SortSampleState, - spiller: Arc, _r: PhantomData, } -impl TransformSortShuffle { - pub fn new( +impl TransformSortBoundBroadcast { + pub fn create( input: Arc, output: Arc, state: SortSampleState, - spiller: Arc, - ) -> Self { - Self { - input, - output, + ) -> Box { + AsyncAccumulatingTransformer::create(input, output, Self { + buffer: Vec::new(), state, - spiller, - step: Step::None, _r: PhantomData, - } - } - - async fn scatter(&mut self) -> Result>> { - let SortCollectedMeta { - params, - bounds, - blocks, - } = match std::mem::replace(&mut self.step, Step::None) { - Step::None => { - return Ok(vec![]); - } - Step::Meta(box meta) => meta, - _ => unreachable!(), - }; - - let scatter_bounds = self.state.bounds(); - if scatter_bounds.is_empty() { - return Ok(vec![Some(SortCollectedMeta { - params, - bounds, - blocks, - })]); - } - - let base = { - let inner = &self.state.inner; - Base { - schema: inner.schema.clone(), - spiller: self.spiller.clone(), - sort_row_offset: inner.schema.fields.len() - 1, - limit: None, - } - }; - - let mut scattered_blocks = std::iter::repeat_with(Vec::new) - .take(scatter_bounds.len() + 1) - .collect::>(); - for blocks in blocks { - let scattered = base - .scatter_stream::(Vec::from(blocks).into(), scatter_bounds.clone()) - .await?; - for (i, part) in scattered.into_iter().enumerate() { - if !part.is_empty() { - scattered_blocks[i].push(part.into_boxed_slice()); - } - } - } - - let scattered_meta = scattered_blocks - .into_iter() - .map(|blocks| { - (!blocks.is_empty()).then_some(SortCollectedMeta { - params, - bounds: bounds.clone(), - blocks, - }) - }) - .collect(); - Ok(scattered_meta) - } -} - -#[async_trait::async_trait] -impl Processor for TransformSortShuffle { - fn name(&self) -> String { - "TransformSortShuffle".to_string() - } - - 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 matches!(self.step, Step::Scattered(_)) { - let Step::Scattered(scattered) = std::mem::replace(&mut self.step, Step::None) else { - unreachable!() - }; - - let data = DataBlock::empty_with_meta(Box::new(SortScatteredMeta(scattered))); - self.output.push_data(Ok(data)); - self.output.finish(); - return Ok(Event::Finished); - } - - if let Some(mut block) = self.input.pull_data().transpose()? { - assert_matches!(self.step, Step::None); - let meta = block - .take_meta() - .and_then(SortCollectedMeta::downcast_from) - .expect("require a SortCollectedMeta"); - - self.step = Step::Meta(Box::new(meta)); - return Ok(Event::Async); - } - - if self.input.is_finished() { - if self.state.inner.bounds.is_some() { - self.output.finish(); - Ok(Event::Finished) - } else { - Ok(Event::Async) - } - } else { - self.input.set_need_data(); - Ok(Event::NeedData) - } - } - - #[async_backtrace::framed] - async fn async_process(&mut self) -> Result<()> { - let bounds = match &self.step { - Step::None if self.input.is_finished() => Bounds::default(), - Step::Meta(meta) => meta.generate_bounds(), - _ => unreachable!(), - }; - - self.state.commit_sample::(bounds).await?; - self.step = Step::Scattered(self.scatter().await?); - Ok(()) - } -} - -impl SortCollectedMeta { - fn generate_bounds(&self) -> Bounds { - if self.bounds.len() > 1 { - return self.bounds.clone(); - } - - let Some(blocks) = self.blocks.get(self.blocks.len() / 2) else { - return Bounds::default(); - }; - - blocks - .get(blocks.len() / 2) - .map(|block| match block.domain.len() { - 0 => Bounds::default(), - 1 => Bounds::new_unchecked(block.domain.clone()), - _ => Bounds::new_unchecked(block.domain.slice(0..1)), - }) - .unwrap_or_default() + }) } } #[derive(Clone)] pub struct SortSampleState { - inner: StateInner, ctx: Arc, broadcast_id: u32, + schema: DataSchemaRef, + batch_rows: usize, } impl SortSampleState { pub fn new( - inputs: usize, - partitions: usize, schema: DataSchemaRef, batch_rows: usize, ctx: Arc, broadcast_id: u32, ) -> SortSampleState { SortSampleState { - inner: StateInner { - partitions, - schema, - partial: vec![], - bounds: None, - batch_rows, - }, ctx, broadcast_id, + schema, + batch_rows, } } - pub async fn commit_sample(&mut self, bounds: Bounds) -> Result<()> { + pub async fn commit_sample(&mut self, bounds: Bounds) -> Result { let sender = self.ctx.broadcast_source_sender(self.broadcast_id); sender .send(Box::new(bounds)) @@ -257,43 +83,157 @@ impl SortSampleState { sender.close(); let receiver = self.ctx.broadcast_sink_receiver(self.broadcast_id); + let mut partials = Vec::new(); while let Ok(r) = receiver.recv().await { - self.inner.partial.push(Bounds::downcast_from(r).unwrap()); + partials.push(Bounds::downcast_from(r).unwrap()); } - self.inner.determine_bounds::()?; - Ok(()) + Bounds::merge::(partials, self.batch_rows) } +} - pub fn bounds(&self) -> Bounds { - self.inner.bounds.clone().unwrap_or_default() +#[async_trait::async_trait] +impl AsyncAccumulatingTransform for TransformSortBoundBroadcast { + const NAME: &'static str = "TransformSortBoundBroadcast"; + + async fn transform(&mut self, mut data: DataBlock) -> Result> { + let meta = data + .take_meta() + .and_then(SortCollectedMeta::downcast_from) + .expect("require a SortCollectedMeta"); + self.buffer.push(meta); + Ok(None) } -} -#[derive(Clone)] + async fn on_finish(&mut self, output: bool) -> Result> { + if !output { + return Ok(None); + } -struct StateInner { - // target partitions - partitions: usize, - schema: DataSchemaRef, - partial: Vec, - bounds: Option, - batch_rows: usize, + let Some(params) = self.buffer.first().map(|meta| meta.params.clone()) else { + return Ok(None); + }; + + let bounds_vec = self + .buffer + .iter() + .map(|meta| meta.bounds.dedup::()) + .collect(); + let bounds = Bounds::merge::(bounds_vec, self.state.batch_rows)?; + + let blocks = self + .buffer + .into_iter() + .flat_map(|meta| meta.blocks.into_iter()) + .collect(); + + let local = SortCollectedMeta { + params, + bounds, + blocks, + }; + + let global_bounds = self + .state + .commit_sample::(local.generate_bounds()) + .await?; + + Ok(Some(DataBlock::empty_with_meta(Box::new( + SortCollectedMeta { + bounds: global_bounds, + ..local + }, + )))) + } } -impl StateInner { - fn determine_bounds(&mut self) -> Result<()> { - let bounds = Bounds::merge::(std::mem::take(&mut self.partial), self.batch_rows)?; +impl SortCollectedMeta { + fn generate_bounds(&self) -> Bounds { + if self.bounds.len() > 1 { + return self.bounds.clone(); + } - let n = self.partitions - 1; - let bounds = if bounds.len() < n { - bounds - } else { - bounds.dedup_reduce::(n) + let Some(blocks) = self.blocks.get(self.blocks.len() / 2) else { + return Bounds::default(); }; - assert!(bounds.len() < self.partitions); - self.bounds = Some(bounds); - Ok(()) + blocks + .get(blocks.len() / 2) + .map(|block| match block.domain.len() { + 0 => Bounds::default(), + 1 => Bounds::new_unchecked(block.domain.clone()), + _ => Bounds::new_unchecked(block.domain.slice(0..1)), + }) + .unwrap_or_default() } } + +// fn determine_bounds(&self, bounds: Bounds) -> Bounds { +// let n = self.partitions - 1; +// let bounds = if bounds.len() < n { +// bounds +// } else { +// bounds.dedup_reduce::(n) +// }; +// assert!(bounds.len() < self.partitions); +// bounds +// } + +// async fn scatter(&mut self) -> Result>> { +// let SortCollectedMeta { +// params, +// bounds, +// blocks, +// } = match std::mem::replace(&mut self.step, Step::None) { +// Step::None => { +// return Ok(vec![]); +// } +// Step::Local(box meta) => meta, +// _ => unreachable!(), +// }; + +// let scatter_bounds = self.state.bounds(); +// if scatter_bounds.is_empty() { +// return Ok(vec![Some(SortCollectedMeta { +// params, +// bounds, +// blocks, +// })]); +// } + +// let base = { +// let inner = &self.state.inner; +// Base { +// schema: inner.schema.clone(), +// spiller: self.spiller.clone(), +// sort_row_offset: inner.schema.fields.len() - 1, +// limit: None, +// } +// }; + +// let mut scattered_blocks = std::iter::repeat_with(Vec::new) +// .take(scatter_bounds.len() + 1) +// .collect::>(); +// for blocks in blocks { +// let scattered = base +// .scatter_stream::(Vec::from(blocks).into(), scatter_bounds.clone()) +// .await?; +// for (i, part) in scattered.into_iter().enumerate() { +// if !part.is_empty() { +// scattered_blocks[i].push(part.into_boxed_slice()); +// } +// } +// } + +// let scattered_meta = scattered_blocks +// .into_iter() +// .map(|blocks| { +// (!blocks.is_empty()).then_some(SortCollectedMeta { +// params, +// bounds: bounds.clone(), +// blocks, +// }) +// }) +// .collect(); +// Ok(scattered_meta) +// } From 182cdde769c5fc0f1952423140b65dfcd30406e4 Mon Sep 17 00:00:00 2001 From: coldWater Date: Fri, 4 Jul 2025 11:22:37 +0800 Subject: [PATCH 38/61] x --- .../src/processors/transforms/mod.rs | 4 + .../transforms/transform_blocking_async.rs | 120 +++++++++++ .../processors/transforms/transform_hook.rs | 115 +++++++++++ .../transforms/transform_sort_merge_base.rs | 2 +- .../src/pipelines/builders/builder_sort.rs | 4 +- .../processors/transforms/sort/mod.rs | 18 ++ .../transforms/sort/sort_builder.rs | 2 +- .../transforms/sort/sort_exchange_injector.rs | 190 +++++++++++++----- .../transforms/sort/sort_execute.rs | 90 +++------ .../transforms/sort/sort_shuffle.rs | 76 +------ 10 files changed, 440 insertions(+), 181 deletions(-) create mode 100644 src/query/pipeline/transforms/src/processors/transforms/transform_blocking_async.rs create mode 100644 src/query/pipeline/transforms/src/processors/transforms/transform_hook.rs diff --git a/src/query/pipeline/transforms/src/processors/transforms/mod.rs b/src/query/pipeline/transforms/src/processors/transforms/mod.rs index 8fe951ce2c89a..5278fbfaae32b 100644 --- a/src/query/pipeline/transforms/src/processors/transforms/mod.rs +++ b/src/query/pipeline/transforms/src/processors/transforms/mod.rs @@ -18,10 +18,12 @@ mod transform_accumulating; mod transform_accumulating_async; mod transform_async; mod transform_blocking; +mod transform_blocking_async; mod transform_compact_block; mod transform_compact_builder; mod transform_compact_no_split_builder; mod transform_dummy; +mod transform_hook; mod transform_k_way_merge_sort; mod transform_multi_sort_merge; mod transform_pipeline_helper; @@ -36,10 +38,12 @@ pub use transform_accumulating::*; pub use transform_accumulating_async::*; pub use transform_async::*; pub use transform_blocking::*; +pub use transform_blocking_async::*; pub use transform_compact_block::*; pub use transform_compact_builder::*; pub use transform_compact_no_split_builder::*; pub use transform_dummy::*; +pub use transform_hook::*; pub use transform_k_way_merge_sort::*; pub use transform_multi_sort_merge::*; pub use transform_pipeline_helper::TransformPipelineHelper; diff --git a/src/query/pipeline/transforms/src/processors/transforms/transform_blocking_async.rs b/src/query/pipeline/transforms/src/processors/transforms/transform_blocking_async.rs new file mode 100644 index 0000000000000..801a955fe992f --- /dev/null +++ b/src/query/pipeline/transforms/src/processors/transforms/transform_blocking_async.rs @@ -0,0 +1,120 @@ +// 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; + +#[async_trait::async_trait] +pub trait AsyncBlockingTransform: Send { + const NAME: &'static str; + + async fn consume(&mut self, block: DataBlock) -> Result<()>; + + async fn transform(&mut self) -> Result>; +} + +/// A transform may be blocked on a certain input. +/// This transform will not pull new data from the input until the inner transform returns [None]. +pub struct AsyncBlockingTransformer { + inner: T, + input: Arc, + output: Arc, + input_data: Option, + output_data: Option, + need_data: bool, +} + +impl AsyncBlockingTransformer { + pub fn create(input: Arc, output: Arc, inner: T) -> Box { + Box::new(Self { + inner, + input, + output, + input_data: None, + output_data: None, + need_data: true, + }) + } +} + +#[async_trait::async_trait] +impl Processor for AsyncBlockingTransformer { + fn name(&self) -> String { + String::from(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 Ok(Event::Finished); + } + + if !self.output.can_push() { + self.input.set_not_need_data(); + return Ok(Event::NeedConsume); + } + + if let Some(output) = self.output_data.take() { + self.output.push_data(Ok(output)); + return Ok(Event::NeedConsume); + } + + if !self.need_data { + // There is data needed to be transformed. + return Ok(Event::Async); + } + + // The data is fully consumed, we can begin to consume new data. + if self.input.has_data() { + let data = self.input.pull_data().unwrap()?; + self.input_data = Some(data); + return Ok(Event::Async); + } + + if self.input.is_finished() { + self.output.finish(); + return Ok(Event::Finished); + } + + self.input.set_need_data(); + Ok(Event::NeedData) + } + + async fn async_process(&mut self) -> Result<()> { + if let Some(input) = self.input_data.take() { + debug_assert!(self.need_data); + self.inner.consume(input).await?; + } + + if let Some(block) = self.inner.transform().await? { + self.output_data = Some(block); + self.need_data = false; + } else { + self.need_data = true; + } + + Ok(()) + } +} diff --git a/src/query/pipeline/transforms/src/processors/transforms/transform_hook.rs b/src/query/pipeline/transforms/src/processors/transforms/transform_hook.rs new file mode 100644 index 0000000000000..e200b55515ab7 --- /dev/null +++ b/src/query/pipeline/transforms/src/processors/transforms/transform_hook.rs @@ -0,0 +1,115 @@ +// 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; + +#[async_trait::async_trait] +pub trait HookTransform: Send + 'static { + const NAME: &'static str; + + fn on_input(&mut self, data: DataBlock) -> Result<()>; + + fn on_output(&mut self) -> Result>; + + fn need_process(&self) -> Option; + + fn process(&mut self) -> Result<()> { + unimplemented!() + } + + async fn async_process(&mut self) -> Result<()> { + unimplemented!() + } +} + +pub struct HookTransformer { + inner: T, + input: Arc, + output: Arc, +} + +impl HookTransformer { + pub fn new(input: Arc, output: Arc, inner: T) -> Self { + Self { + inner, + input, + output, + } + } +} + +#[async_trait::async_trait] +impl Processor for HookTransformer { + fn name(&self) -> String { + String::from(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 Ok(Event::Finished); + } + + if !self.output.can_push() { + self.input.set_not_need_data(); + return Ok(Event::NeedConsume); + } + + if let Some(output) = self.inner.on_output()? { + self.output.push_data(Ok(output)); + return Ok(Event::NeedConsume); + } + + if let Some(event) = self.inner.need_process() { + return Ok(event); + } + + if self.input.has_data() { + let data = self.input.pull_data().unwrap()?; + self.inner.on_input(data)?; + } + + if let Some(event) = self.inner.need_process() { + return Ok(event); + } + + if self.input.is_finished() { + self.output.finish(); + Ok(Event::Finished) + } else { + self.input.set_need_data(); + Ok(Event::NeedData) + } + } + + fn process(&mut self) -> Result<()> { + self.inner.process() + } + + async fn async_process(&mut self) -> Result<()> { + self.inner.async_process().await + } +} diff --git a/src/query/pipeline/transforms/src/processors/transforms/transform_sort_merge_base.rs b/src/query/pipeline/transforms/src/processors/transforms/transform_sort_merge_base.rs index 5b44807cf30a6..c3c4a8180099f 100644 --- a/src/query/pipeline/transforms/src/processors/transforms/transform_sort_merge_base.rs +++ b/src/query/pipeline/transforms/src/processors/transforms/transform_sort_merge_base.rs @@ -23,7 +23,7 @@ use databend_common_expression::SortColumnDescription; use super::sort::RowConverter; use super::sort::Rows; -#[derive(serde::Serialize, serde::Deserialize, Debug, Clone, Copy)] +#[derive(Debug, Clone, Copy, PartialEq, Eq, serde::Serialize, serde::Deserialize)] pub struct SortSpillParams { /// The number of rows of each spilled block. pub batch_rows: usize, diff --git a/src/query/service/src/pipelines/builders/builder_sort.rs b/src/query/service/src/pipelines/builders/builder_sort.rs index 78913984e1a84..6cb688c28e508 100644 --- a/src/query/service/src/pipelines/builders/builder_sort.rs +++ b/src/query/service/src/pipelines/builders/builder_sort.rs @@ -212,7 +212,7 @@ impl SortPipelineBuilder { self.build_merge_sort_pipeline(pipeline, false) } - fn build_range_shuffle_sort_pipeline(self, pipeline: &mut Pipeline) -> Result<()> { + pub fn build_range_shuffle_sort_pipeline(self, pipeline: &mut Pipeline) -> Result<()> { let inputs = pipeline.output_len(); let settings = self.ctx.get_settings(); let num_exec = inputs; @@ -287,9 +287,7 @@ impl SortPipelineBuilder { } fn build_sort_part(self, pipeline: &mut Pipeline) -> Result<()> { - let inputs = pipeline.output_len(); let settings = self.ctx.get_settings(); - let num_exec = inputs; let max_block_size = settings.get_max_block_size()? as usize; // Partial sort diff --git a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs index d41aecd9e4ee7..79b52890b8b47 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs @@ -17,6 +17,7 @@ use std::sync::Arc; use bounds::Bounds; use databend_common_expression::local_block_meta_serde; use databend_common_expression::BlockMetaInfo; +use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::DataBlock; use databend_common_expression::DataSchemaRef; use databend_common_pipeline_transforms::SortSpillParams; @@ -79,3 +80,20 @@ impl MemoryRows for Vec { self.iter().map(|s| s.num_rows()).sum::() } } + +#[derive(Debug, Clone, PartialEq, Eq, serde::Serialize, serde::Deserialize)] +struct SortExchangeMeta { + params: SortSpillParams, + bounds: Bounds, +} + +#[typetag::serde(name = "sort_exchange")] +impl BlockMetaInfo for SortExchangeMeta { + fn equals(&self, info: &Box) -> bool { + SortExchangeMeta::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/sort/sort_builder.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs index 3e3b9d61d0d24..354e3d25cc41a 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs @@ -299,7 +299,7 @@ impl Build<'_> { fn build_sort_exec(&mut self) -> Result> where A: SortAlgorithm + 'static { - Ok(Box::new(TransformSortExecute::::new( + Ok(Box::new(TransformSortExecute::::create( self.input.clone(), self.output.clone(), self.params.new_base(), diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs index 31be6944b0dd2..5bb842c42935c 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs @@ -12,17 +12,22 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::marker::PhantomData; use std::sync::Arc; use arrow_ipc::writer::IpcWriteOptions; use databend_common_exception::Result; use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::DataBlock; +use databend_common_expression::DataSchemaRef; use databend_common_pipeline_core::Pipeline; +use databend_common_pipeline_transforms::sort::Rows; +use databend_common_pipeline_transforms::AsyncBlockingTransform; use databend_common_pipeline_transforms::BlockMetaTransform; use databend_common_settings::FlightCompression; -use crate::pipelines::processors::transforms::aggregator::FlightSerializedMeta; +use crate::pipelines::processors::transforms::sort::bounds::Bounds; +use crate::pipelines::processors::transforms::sort::sort_spill::SpillableBlock; use crate::pipelines::processors::transforms::sort::SortCollectedMeta; use crate::servers::flight::v1::exchange::DataExchange; use crate::servers::flight::v1::exchange::ExchangeInjector; @@ -39,12 +44,12 @@ pub struct SortInjector {} impl ExchangeInjector for SortInjector { fn flight_scatter( &self, - ctx: &Arc, + _: &Arc, exchange: &DataExchange, ) -> Result>> { match exchange { DataExchange::Merge(_) | DataExchange::Broadcast(_) => unreachable!(), - DataExchange::ShuffleDataExchange(exchange) => Ok(Arc::new(Box::new(DummyScatter {}))), + DataExchange::ShuffleDataExchange(_) => Ok(Arc::new(Box::new(DummyScatter {}))), } } @@ -54,34 +59,30 @@ impl ExchangeInjector for SortInjector { fn apply_merge_serializer( &self, - params: &MergeExchangeParams, - compression: Option, - pipeline: &mut Pipeline, + _: &MergeExchangeParams, + _: Option, + _: &mut Pipeline, ) -> Result<()> { unreachable!() } - fn apply_merge_deserializer( - &self, - params: &MergeExchangeParams, - pipeline: &mut Pipeline, - ) -> Result<()> { + fn apply_merge_deserializer(&self, _: &MergeExchangeParams, _: &mut Pipeline) -> Result<()> { unreachable!() } fn apply_shuffle_serializer( &self, - params: &ShuffleExchangeParams, - compression: Option, - pipeline: &mut Pipeline, + _params: &ShuffleExchangeParams, + _compression: Option, + _pipeline: &mut Pipeline, ) -> Result<()> { todo!() } fn apply_shuffle_deserializer( &self, - params: &ShuffleExchangeParams, - pipeline: &mut Pipeline, + _params: &ShuffleExchangeParams, + _pipeline: &mut Pipeline, ) -> Result<()> { todo!() } @@ -90,7 +91,7 @@ impl ExchangeInjector for SortInjector { pub struct DummyScatter {} impl FlightScatter for DummyScatter { - fn execute(&self, mut data_block: DataBlock) -> Result> { + fn execute(&self, data_block: DataBlock) -> Result> { Ok(vec![data_block]) } } @@ -107,31 +108,34 @@ pub struct TransformExchangeSortSerializer { impl BlockMetaTransform for TransformExchangeSortSerializer { const NAME: &'static str = "TransformExchangeSortSerializer"; - fn transform(&mut self, meta: ExchangeShuffleMeta) -> Result> { - let serialized_blocks = meta - .blocks - .into_iter() - .map(|mut block| { - let SortCollectedMeta { - params, - bounds, - blocks, - } = SortCollectedMeta::downcast_from(block.take_meta().unwrap()).unwrap(); - - // 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, - // )?, - // }, - }) - .collect(); + fn transform(&mut self, _meta: ExchangeShuffleMeta) -> Result> { + // let serialized_blocks = meta + // .blocks + // .into_iter() + // .map(|mut block| { + // let SortCollectedMeta { + // params, + // bounds, + // blocks, + // } = block + // .take_meta() + // .and_then(SortCollectedMeta::downcast_from) + // .unwrap(); + + // 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, + // )?, + // }, + // }) + // .collect(); // let meta = SortCollectedMeta::downcast_from(block.take_meta().unwrap()).unwrap(); @@ -173,8 +177,104 @@ impl BlockMetaTransform for TransformExchangeSortSerializer // } // }; - Ok(vec![DataBlock::empty_with_meta( - FlightSerializedMeta::create(serialized_blocks), - )]) + todo!() + + // Ok(vec![DataBlock::empty_with_meta( + // FlightSerializedMeta::create(serialized_blocks), + // )]) + } +} + +struct SortScatter { + ctx: Arc, + local_pos: usize, + options: IpcWriteOptions, + schema: DataSchemaRef, + + partitions: usize, + spiller: Arc, + data: Option, + scatter_bounds: Bounds, + blocks: Vec>, + + _r: PhantomData, +} + +#[async_trait::async_trait] +impl AsyncBlockingTransform for SortScatter { + const NAME: &'static str = "TransformExchangeSortSerializer"; + + async fn consume(&mut self, mut block: DataBlock) -> Result<()> { + let meta = block + .take_meta() + .and_then(SortCollectedMeta::downcast_from) + .unwrap(); + self.data = Some(meta); + Ok(()) + } + + async fn transform(&mut self) -> Result> { + todo!() + } +} + +impl SortScatter { + fn scatter_bounds(&self, bounds: Bounds) -> Bounds { + let n = self.partitions - 1; + let bounds = if bounds.len() < n { + bounds + } else { + bounds.dedup_reduce::(n) + }; + assert!(bounds.len() < self.partitions); + bounds + } + + async fn scatter(&mut self) -> Result>> { + // if self.scatter_bounds.is_empty() { + // return Ok(vec![Some(SortCollectedMeta { + // params, + // bounds, + // blocks, + // })]); + // } + + // let base = { + // Base { + // schema: self.schema.clone(), + // spiller: self.spiller.clone(), + // sort_row_offset: self.schema.fields.len() - 1, + // limit: None, + // } + // }; + + // let mut scattered_blocks = Vec::with_capacity(self.scatter_bounds.len() + 1); + + // let Some(list) = self.blocks.pop() else { + // todo!() + // }; + // let scattered = base + // .scatter_stream::(Vec::from(list).into(), self.scatter_bounds.clone()) + // .await?; + + // ExchangeShuffleMeta::create(blocks); + + // for list in { + + // } + + // let scattered_meta = scattered_blocks + // .into_iter() + // .map(|blocks| { + // (!blocks.is_empty()).then_some(SortCollectedMeta { + // params: todo!(), + // bounds: todo!(), + // blocks, + // }) + // }) + // .collect(); + // Ok(scattered_meta) + + todo!() } } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_execute.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_execute.rs index 044858dc4358a..f6e5b1dcef01d 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_execute.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_execute.rs @@ -12,7 +12,6 @@ // 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; @@ -21,16 +20,16 @@ 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_transforms::processors::sort::algorithm::SortAlgorithm; +use databend_common_pipeline_transforms::HookTransform; +use databend_common_pipeline_transforms::HookTransformer; use super::sort_spill::SortSpill; use super::Base; use super::SortCollectedMeta; pub struct TransformSortExecute { - input: Arc, - output: Arc, + output: Option, /// If the next transform of current transform is [`super::transform_multi_sort_merge::MultiSortMergeProcessor`], /// we can generate and output the order column to avoid the extra converting in the next transform. @@ -41,78 +40,52 @@ pub struct TransformSortExecute { } impl TransformSortExecute -where A: SortAlgorithm +where A: SortAlgorithm + Send + 'static { - pub(super) fn new( + pub(super) fn create( input: Arc, output: Arc, base: Base, output_order_col: bool, - ) -> Result { - Ok(Self { - input, - output, + ) -> Result> { + Ok(HookTransformer::new(input, output, Self { + output: None, remove_order_col: !output_order_col, base, inner: None, - }) - } - - fn output_block(&self, mut block: DataBlock) { - if self.remove_order_col { - block.pop_columns(1); - } - self.output.push_data(Ok(block)); + })) } } #[async_trait::async_trait] -impl Processor for TransformSortExecute +impl HookTransform for TransformSortExecute where A: SortAlgorithm + 'static, A::Rows: 'static, { - fn name(&self) -> String { - "TransformSortExecute".to_string() - } - - 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); - } + const NAME: &'static str = "TransformSortExecute"; - if !self.output.can_push() { - self.input.set_not_need_data(); - return Ok(Event::NeedConsume); - } - - if self.input.is_finished() && self.inner.is_none() { - self.output.finish(); - return Ok(Event::Finished); - } + fn on_input(&mut self, mut block: DataBlock) -> Result<()> { + assert!(self.inner.is_none()); + let meta = block + .take_meta() + .and_then(SortCollectedMeta::downcast_from) + .expect("require a SortCollectedMeta"); - if let Some(mut block) = self.input.pull_data().transpose()? { - assert!(self.inner.is_none()); - let meta = block - .take_meta() - .and_then(SortCollectedMeta::downcast_from) - .expect("require a SortCollectedMeta"); + self.inner = Some(SortSpill::::from_meta(self.base.clone(), meta)); + Ok(()) + } - self.inner = Some(SortSpill::::from_meta(self.base.clone(), meta)); - return Ok(Event::Async); - } + fn on_output(&mut self) -> Result> { + Ok(self.output.take()) + } + fn need_process(&self) -> Option { if self.inner.is_some() { - return Ok(Event::Async); + Some(Event::Async) + } else { + None } - - self.input.set_need_data(); - Ok(Event::NeedData) } #[async_backtrace::framed] @@ -121,12 +94,13 @@ where unreachable!() }; let (block, finish) = spill_sort.on_restore().await?; - if let Some(block) = block { - assert!(!self.output.has_data()); - self.output_block(block); + if let Some(mut block) = block { + if self.remove_order_col { + block.pop_columns(1); + } + self.output = Some(block); } if finish { - self.output.finish(); self.inner = None; } Ok(()) diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_shuffle.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_shuffle.rs index bdee0aac93223..a6ef018d8c6ed 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_shuffle.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_shuffle.rs @@ -37,7 +37,7 @@ pub struct TransformSortBoundBroadcast { _r: PhantomData, } -impl TransformSortBoundBroadcast { +impl TransformSortBoundBroadcast { pub fn create( input: Arc, output: Arc, @@ -110,7 +110,7 @@ impl AsyncAccumulatingTransform for TransformSortBoundBroadca return Ok(None); } - let Some(params) = self.buffer.first().map(|meta| meta.params.clone()) else { + let Some(params) = self.buffer.first().map(|meta| meta.params) else { return Ok(None); }; @@ -123,7 +123,7 @@ impl AsyncAccumulatingTransform for TransformSortBoundBroadca let blocks = self .buffer - .into_iter() + .drain(..) .flat_map(|meta| meta.blocks.into_iter()) .collect(); @@ -167,73 +167,3 @@ impl SortCollectedMeta { .unwrap_or_default() } } - -// fn determine_bounds(&self, bounds: Bounds) -> Bounds { -// let n = self.partitions - 1; -// let bounds = if bounds.len() < n { -// bounds -// } else { -// bounds.dedup_reduce::(n) -// }; -// assert!(bounds.len() < self.partitions); -// bounds -// } - -// async fn scatter(&mut self) -> Result>> { -// let SortCollectedMeta { -// params, -// bounds, -// blocks, -// } = match std::mem::replace(&mut self.step, Step::None) { -// Step::None => { -// return Ok(vec![]); -// } -// Step::Local(box meta) => meta, -// _ => unreachable!(), -// }; - -// let scatter_bounds = self.state.bounds(); -// if scatter_bounds.is_empty() { -// return Ok(vec![Some(SortCollectedMeta { -// params, -// bounds, -// blocks, -// })]); -// } - -// let base = { -// let inner = &self.state.inner; -// Base { -// schema: inner.schema.clone(), -// spiller: self.spiller.clone(), -// sort_row_offset: inner.schema.fields.len() - 1, -// limit: None, -// } -// }; - -// let mut scattered_blocks = std::iter::repeat_with(Vec::new) -// .take(scatter_bounds.len() + 1) -// .collect::>(); -// for blocks in blocks { -// let scattered = base -// .scatter_stream::(Vec::from(blocks).into(), scatter_bounds.clone()) -// .await?; -// for (i, part) in scattered.into_iter().enumerate() { -// if !part.is_empty() { -// scattered_blocks[i].push(part.into_boxed_slice()); -// } -// } -// } - -// let scattered_meta = scattered_blocks -// .into_iter() -// .map(|blocks| { -// (!blocks.is_empty()).then_some(SortCollectedMeta { -// params, -// bounds: bounds.clone(), -// blocks, -// }) -// }) -// .collect(); -// Ok(scattered_meta) -// } From 79194ca3e6dfbf19ee7288f323598cda8b78494d Mon Sep 17 00:00:00 2001 From: coldWater Date: Fri, 4 Jul 2025 17:20:08 +0800 Subject: [PATCH 39/61] x --- src/query/expression/src/block.rs | 19 +++ .../processors/transforms/transform_hook.rs | 6 +- .../processors/transforms/sort/bounds.rs | 13 -- .../processors/transforms/sort/merge_sort.rs | 3 +- .../processors/transforms/sort/mod.rs | 17 +++ .../transforms/sort/sort_builder.rs | 4 +- .../transforms/sort/sort_execute.rs | 19 ++- .../transforms/sort/sort_shuffle.rs | 127 +++++++++++------- .../processors/transforms/sort/sort_spill.rs | 56 ++++++-- 9 files changed, 177 insertions(+), 87 deletions(-) diff --git a/src/query/expression/src/block.rs b/src/query/expression/src/block.rs index 70db35bb31695..eb0cddbdcbeca 100644 --- a/src/query/expression/src/block.rs +++ b/src/query/expression/src/block.rs @@ -332,10 +332,29 @@ pub trait BlockMetaInfoDowncast: Sized + BlockMetaInfo { let boxed = boxed.as_ref() as &dyn Any; boxed.downcast_ref() } + + fn downcast_from_err(boxed: BlockMetaInfoPtr) -> std::result::Result { + if (boxed.as_ref() as &dyn Any).is::() { + Ok(*(boxed as Box).downcast().unwrap()) + } else { + Err(boxed) + } + } } impl BlockMetaInfoDowncast for T {} +#[typetag::serde(name = "empty")] +impl BlockMetaInfo for () { + fn equals(&self, info: &Box) -> bool { + <() as BlockMetaInfoDowncast>::downcast_ref_from(info).is_some() + } + + fn clone_self(&self) -> Box { + Box::new(()) + } +} + impl DataBlock { #[inline] pub fn new(entries: Vec, num_rows: usize) -> Self { diff --git a/src/query/pipeline/transforms/src/processors/transforms/transform_hook.rs b/src/query/pipeline/transforms/src/processors/transforms/transform_hook.rs index e200b55515ab7..ca6babb9696fd 100644 --- a/src/query/pipeline/transforms/src/processors/transforms/transform_hook.rs +++ b/src/query/pipeline/transforms/src/processors/transforms/transform_hook.rs @@ -30,7 +30,7 @@ pub trait HookTransform: Send + 'static { fn on_output(&mut self) -> Result>; - fn need_process(&self) -> Option; + fn need_process(&self, input_finished: bool) -> Option; fn process(&mut self) -> Result<()> { unimplemented!() @@ -83,7 +83,7 @@ impl Processor for HookTransformer { return Ok(Event::NeedConsume); } - if let Some(event) = self.inner.need_process() { + if let Some(event) = self.inner.need_process(self.input.is_finished()) { return Ok(event); } @@ -92,7 +92,7 @@ impl Processor for HookTransformer { self.inner.on_input(data)?; } - if let Some(event) = self.inner.need_process() { + if let Some(event) = self.inner.need_process(self.input.is_finished()) { return Ok(event); } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs b/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs index 8ec18f1bc7d9a..8029ff96bdf0d 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs @@ -13,8 +13,6 @@ // limitations under the License. use databend_common_exception::Result; -use databend_common_expression::BlockMetaInfo; -use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::Column; use databend_common_expression::DataBlock; use databend_common_expression::DataField; @@ -202,17 +200,6 @@ impl SortedStream for Bounds { } } -#[typetag::serde(name = "sort_bounds")] -impl BlockMetaInfo for Bounds { - fn equals(&self, info: &Box) -> bool { - Bounds::downcast_ref_from(info).is_some_and(|other| self == other) - } - - fn clone_self(&self) -> Box { - Box::new(self.clone()) - } -} - #[cfg(test)] mod tests { use databend_common_expression::types::Int32Type; diff --git a/src/query/service/src/pipelines/processors/transforms/sort/merge_sort.rs b/src/query/service/src/pipelines/processors/transforms/sort/merge_sort.rs index 5c07cee78f795..96fd0a955899e 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/merge_sort.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/merge_sort.rs @@ -40,6 +40,7 @@ use super::sort_spill::MemoryMerger; use super::sort_spill::SortSpill; use super::Base; use super::MemoryRows; +use crate::pipelines::processors::transforms::sort::sort_spill::OutputData; use crate::spillers::Spiller; #[derive(Debug)] @@ -448,7 +449,7 @@ where unreachable!() }; assert!(input_data.is_empty()); - let (block, finish) = spill_sort.on_restore().await?; + let OutputData { block, finish, .. } = spill_sort.on_restore().await?; self.output_data.extend(block); if finish { self.state = State::Finish diff --git a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs index 79b52890b8b47..6dc9fac578658 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs @@ -20,6 +20,7 @@ use databend_common_expression::BlockMetaInfo; use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::DataBlock; use databend_common_expression::DataSchemaRef; +use databend_common_expression::Scalar; use databend_common_pipeline_transforms::SortSpillParams; use sort_spill::SpillableBlock; @@ -97,3 +98,19 @@ impl BlockMetaInfo for SortExchangeMeta { Box::new(self.clone()) } } + +#[derive(Debug, Clone, PartialEq, Eq, serde::Serialize, serde::Deserialize)] +pub struct SortBound { + bound: Option, +} + +#[typetag::serde(name = "sort_bound")] +impl BlockMetaInfo for SortBound { + fn equals(&self, info: &Box) -> bool { + SortBound::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/sort/sort_builder.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs index 354e3d25cc41a..b3aef0f6fa806 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs @@ -37,7 +37,7 @@ use databend_common_pipeline_transforms::MemorySettings; use super::merge_sort::TransformSort; use super::sort_collect::TransformSortCollect; use super::sort_combine::TransformSortCombine; -use super::sort_execute::TransformSortExecute; +use super::sort_execute::TransformSortRestore; use super::sort_shuffle::SortSampleState; use super::sort_shuffle::TransformSortBoundBroadcast; use super::Base; @@ -299,7 +299,7 @@ impl Build<'_> { fn build_sort_exec(&mut self) -> Result> where A: SortAlgorithm + 'static { - Ok(Box::new(TransformSortExecute::::create( + Ok(Box::new(TransformSortRestore::::create( self.input.clone(), self.output.clone(), self.params.new_base(), diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_execute.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_execute.rs index f6e5b1dcef01d..70c7aa5c3c6a4 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_execute.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_execute.rs @@ -26,9 +26,11 @@ use databend_common_pipeline_transforms::HookTransformer; use super::sort_spill::SortSpill; use super::Base; +use super::SortBound; use super::SortCollectedMeta; +use crate::pipelines::processors::transforms::sort::sort_spill::OutputData; -pub struct TransformSortExecute { +pub struct TransformSortRestore { output: Option, /// If the next transform of current transform is [`super::transform_multi_sort_merge::MultiSortMergeProcessor`], @@ -39,7 +41,7 @@ pub struct TransformSortExecute { inner: Option>, } -impl TransformSortExecute +impl TransformSortRestore where A: SortAlgorithm + Send + 'static { pub(super) fn create( @@ -58,7 +60,7 @@ where A: SortAlgorithm + Send + 'static } #[async_trait::async_trait] -impl HookTransform for TransformSortExecute +impl HookTransform for TransformSortRestore where A: SortAlgorithm + 'static, A::Rows: 'static, @@ -80,7 +82,7 @@ where Ok(self.output.take()) } - fn need_process(&self) -> Option { + fn need_process(&self, _: bool) -> Option { if self.inner.is_some() { Some(Event::Async) } else { @@ -93,8 +95,13 @@ where let Some(spill_sort) = &mut self.inner else { unreachable!() }; - let (block, finish) = spill_sort.on_restore().await?; - if let Some(mut block) = block { + let OutputData { + block, + bound, + finish, + } = spill_sort.on_restore().await?; + if let Some(block) = block { + let mut block = block.add_meta(Some(SortBound { bound }.boxed()))?; if self.remove_order_col { block.pop_columns(1); } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_shuffle.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_shuffle.rs index a6ef018d8c6ed..3c6506dcfee6e 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_shuffle.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_shuffle.rs @@ -20,20 +20,25 @@ use databend_common_exception::Result; use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::DataBlock; use databend_common_expression::DataSchemaRef; +use databend_common_pipeline_core::processors::Event; use databend_common_pipeline_core::processors::Processor; use databend_common_pipeline_transforms::processors::sort::Rows; -use databend_common_pipeline_transforms::AsyncAccumulatingTransform; -use databend_common_pipeline_transforms::AsyncAccumulatingTransformer; +use databend_common_pipeline_transforms::HookTransform; +use databend_common_pipeline_transforms::HookTransformer; use super::bounds::Bounds; use super::SortCollectedMeta; +use crate::pipelines::processors::transforms::sort::SortExchangeMeta; use crate::pipelines::processors::InputPort; use crate::pipelines::processors::OutputPort; use crate::sessions::QueryContext; pub struct TransformSortBoundBroadcast { - buffer: Vec, state: SortSampleState, + input_data: Option, + output_data: Option, + called_on_finish: bool, + _r: PhantomData, } @@ -43,11 +48,13 @@ impl TransformSortBoundBroadcast { output: Arc, state: SortSampleState, ) -> Box { - AsyncAccumulatingTransformer::create(input, output, Self { - buffer: Vec::new(), + Box::new(HookTransformer::new(input, output, Self { state, + input_data: None, + output_data: None, + called_on_finish: false, _r: PhantomData, - }) + })) } } @@ -74,83 +81,101 @@ impl SortSampleState { } } - pub async fn commit_sample(&mut self, bounds: Bounds) -> Result { + async fn commit_sample( + &mut self, + meta: Option, + ) -> Result> { let sender = self.ctx.broadcast_source_sender(self.broadcast_id); + let meta = meta.map(|meta| meta.boxed()).unwrap_or(().boxed()); sender - .send(Box::new(bounds)) + .send(meta) .await .map_err(|_| ErrorCode::TokioError("send sort bounds failed"))?; sender.close(); let receiver = self.ctx.broadcast_sink_receiver(self.broadcast_id); - let mut partials = Vec::new(); + let mut all = Vec::new(); while let Ok(r) = receiver.recv().await { - partials.push(Bounds::downcast_from(r).unwrap()); + match SortExchangeMeta::downcast_from_err(r) { + Ok(meta) => all.push(meta), + Err(r) => { + debug_assert!(().boxed().equals(&r)) + } + }; } - - Bounds::merge::(partials, self.batch_rows) + Ok(all) } } #[async_trait::async_trait] -impl AsyncAccumulatingTransform for TransformSortBoundBroadcast { +impl HookTransform for TransformSortBoundBroadcast { const NAME: &'static str = "TransformSortBoundBroadcast"; - async fn transform(&mut self, mut data: DataBlock) -> Result> { + fn on_input(&mut self, mut data: DataBlock) -> Result<()> { let meta = data .take_meta() .and_then(SortCollectedMeta::downcast_from) .expect("require a SortCollectedMeta"); - self.buffer.push(meta); - Ok(None) + debug_assert!(self.input_data.is_none()); + self.input_data = Some(meta); + Ok(()) + } + + fn on_output(&mut self) -> Result> { + Ok(self.output_data.take()) } - async fn on_finish(&mut self, output: bool) -> Result> { - if !output { - return Ok(None); + fn need_process(&self, input_finished: bool) -> Option { + if input_finished && !self.called_on_finish { + Some(Event::Async) + } else { + None } + } - let Some(params) = self.buffer.first().map(|meta| meta.params) else { - return Ok(None); + async fn async_process(&mut self) -> Result<()> { + self.called_on_finish = true; + let (metas, local) = match self.input_data.take() { + Some(meta) => { + let bounds = meta.normalize_bounds::(); + let metas = self + .state + .commit_sample(Some(SortExchangeMeta { + params: meta.params, + bounds, + })) + .await?; + (metas, Some(meta)) + } + None => (self.state.commit_sample(None).await?, None), }; - let bounds_vec = self - .buffer - .iter() - .map(|meta| meta.bounds.dedup::()) - .collect(); - let bounds = Bounds::merge::(bounds_vec, self.state.batch_rows)?; + let params = match local.as_ref().map(|meta| meta.params) { + Some(params) => params, + None => { + let Some(meta) = metas.first() else { + return Ok(()); + }; + meta.params + } + }; - let blocks = self - .buffer - .drain(..) - .flat_map(|meta| meta.blocks.into_iter()) - .collect(); + let bounds_vec = metas.iter().map(|meta| meta.bounds.clone()).collect(); + let global_bounds = Bounds::merge::(bounds_vec, self.state.batch_rows)?.dedup::(); - let local = SortCollectedMeta { + self.output_data = Some(DataBlock::empty_with_meta(Box::new(SortCollectedMeta { params, - bounds, - blocks, - }; - - let global_bounds = self - .state - .commit_sample::(local.generate_bounds()) - .await?; - - Ok(Some(DataBlock::empty_with_meta(Box::new( - SortCollectedMeta { - bounds: global_bounds, - ..local - }, - )))) + bounds: global_bounds, + blocks: local.map(|meta| meta.blocks).unwrap_or_default(), + }))); + Ok(()) } } impl SortCollectedMeta { - fn generate_bounds(&self) -> Bounds { + fn normalize_bounds(&self) -> Bounds { if self.bounds.len() > 1 { - return self.bounds.clone(); + return self.bounds.dedup::(); } let Some(blocks) = self.blocks.get(self.blocks.len() / 2) else { @@ -162,7 +187,7 @@ impl SortCollectedMeta { .map(|block| match block.domain.len() { 0 => Bounds::default(), 1 => Bounds::new_unchecked(block.domain.clone()), - _ => Bounds::new_unchecked(block.domain.slice(0..1)), + _ => Bounds::new_unchecked(block.domain.slice(0..1)).dedup::(), }) .unwrap_or_default() } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs index 74e8965dfe3b4..91929135f95e6 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs @@ -150,7 +150,7 @@ where A: SortAlgorithm } } - pub async fn on_restore(&mut self) -> Result<(Option, bool)> { + pub async fn on_restore(&mut self) -> Result { match &mut self.step { Step::Collect(collect) => self.step = Step::Sort(collect.next_step(&self.base)?), Step::Sort(_) => (), @@ -170,7 +170,11 @@ where A: SortAlgorithm if sort.current.len() > sort.params.num_merge { sort.merge_current(&self.base).await?; - Ok((None, false)) + Ok(OutputData { + block: None, + bound: None, + finish: false, + }) } else { sort.restore_and_output(&self.base).await } @@ -307,6 +311,12 @@ impl StepCollect { } } +pub struct OutputData { + pub block: Option, + pub bound: Option, + pub finish: bool, +} + impl StepSort { fn next_bound(&mut self) { match self.bounds.next_bound() { @@ -353,7 +363,7 @@ impl StepSort { Ok(()) } - async fn restore_and_output(&mut self, base: &Base) -> Result<(Option, bool)> { + async fn restore_and_output(&mut self, base: &Base) -> Result { let merger = match self.output_merger.as_mut() { Some(merger) => merger, None => { @@ -362,6 +372,7 @@ impl StepSort { let mut s = self.current.pop().unwrap(); s.restore_first().await?; let block = Some(s.take_next_bounded_block()); + let bound = s.bound.clone(); if !s.is_empty() { if s.should_include_first() { @@ -369,10 +380,18 @@ impl StepSort { } else { self.subsequent.push(s); } - return Ok((block, false)); + return Ok(OutputData { + block, + bound, + finish: false, + }); } - return Ok((block, self.subsequent.is_empty())); + return Ok(OutputData { + block, + bound, + finish: self.subsequent.is_empty(), + }); } self.sort_spill(base, self.params).await?; @@ -393,18 +412,28 @@ impl StepSort { let streams = self.output_merger.take().unwrap().streams(); self.subsequent .extend(streams.into_iter().filter(|s| !s.is_empty())); - return Ok((None, self.subsequent.is_empty())); + + return Ok(OutputData { + block: None, + bound: None, + finish: self.subsequent.is_empty(), + }); }; let mut sorted = base.new_stream([base.new_block(data)].into(), self.cur_bound.clone()); - let block = if sorted.should_include_first() { + let (block, bound) = if sorted.should_include_first() { let block = Some(sorted.take_next_bounded_block()); + let bound = sorted.bound.clone(); if sorted.is_empty() { - return Ok((block, false)); + return Ok(OutputData { + block, + bound, + finish: false, + }); } - block + (block, bound) } else { - None + (None, None) }; while let Some(data) = merger.async_next_block().await? { @@ -420,7 +449,12 @@ impl StepSort { let streams = self.output_merger.take().unwrap().streams(); self.subsequent .extend(streams.into_iter().filter(|s| !s.is_empty())); - Ok((block, self.subsequent.is_empty())) + + Ok(OutputData { + block, + bound, + finish: self.subsequent.is_empty(), + }) } async fn sort_spill( From 8f9e75dc8349592b53b944c66b217810f0a5e27f Mon Sep 17 00:00:00 2001 From: coldWater Date: Fri, 4 Jul 2025 18:56:16 +0800 Subject: [PATCH 40/61] x --- .../transforms/sort/sort_execute.rs | 23 +++++++++++++++++++ 1 file changed, 23 insertions(+) diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_execute.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_execute.rs index 70c7aa5c3c6a4..be08c793c53f9 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_execute.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_execute.rs @@ -16,7 +16,9 @@ use std::sync::Arc; use databend_common_exception::Result; use databend_common_expression::BlockMetaInfoDowncast; +use databend_common_expression::ColumnBuilder; use databend_common_expression::DataBlock; +use databend_common_expression::Scalar; use databend_common_pipeline_core::processors::Event; use databend_common_pipeline_core::processors::InputPort; use databend_common_pipeline_core::processors::OutputPort; @@ -105,6 +107,7 @@ where if self.remove_order_col { block.pop_columns(1); } + self.base.append_exchange_key(block, bound.as_ref()); self.output = Some(block); } if finish { @@ -113,3 +116,23 @@ where Ok(()) } } + +impl Base { + fn append_exchange_key(&self, &mut block: DataBlock, bound: Option<&Scalar>) { + let data_type = self + .schema + .field(self.base.sort_row_offset) + .data_type() + .wrap_nullable(); + let mut builder = ColumnBuilder::with_capacity(&data_type, 1); + match bound { + Some(bound) => { + builder.push(bound.as_ref()); + } + None => { + builder.push(Scalar::Null.as_ref()); + } + } + block.add_const_column(builder.build_scalar(), data_type); + } +} From d6ad39a4743bdf901220d31f0991f227a52aef31 Mon Sep 17 00:00:00 2001 From: coldWater Date: Sat, 5 Jul 2025 00:05:15 +0800 Subject: [PATCH 41/61] x --- .../transforms/sort/sort_exchange_injector.rs | 15 ++++++++------- .../processors/transforms/sort/sort_execute.rs | 14 ++++++-------- 2 files changed, 14 insertions(+), 15 deletions(-) diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs index 5bb842c42935c..9e3dc6b258c91 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs @@ -30,6 +30,7 @@ use crate::pipelines::processors::transforms::sort::bounds::Bounds; use crate::pipelines::processors::transforms::sort::sort_spill::SpillableBlock; use crate::pipelines::processors::transforms::sort::SortCollectedMeta; 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::exchange::ExchangeShuffleMeta; use crate::servers::flight::v1::exchange::ExchangeSorting; @@ -72,19 +73,19 @@ impl ExchangeInjector for SortInjector { fn apply_shuffle_serializer( &self, - _params: &ShuffleExchangeParams, - _compression: Option, - _pipeline: &mut Pipeline, + params: &ShuffleExchangeParams, + compression: Option, + pipeline: &mut Pipeline, ) -> Result<()> { - todo!() + DefaultExchangeInjector::create().apply_shuffle_serializer(params, compression, pipeline) } fn apply_shuffle_deserializer( &self, - _params: &ShuffleExchangeParams, - _pipeline: &mut Pipeline, + params: &ShuffleExchangeParams, + pipeline: &mut Pipeline, ) -> Result<()> { - todo!() + DefaultExchangeInjector::create().apply_shuffle_deserializer(params, pipeline) } } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_execute.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_execute.rs index be08c793c53f9..a3dc5aa97776d 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_execute.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_execute.rs @@ -124,15 +124,13 @@ impl Base { .field(self.base.sort_row_offset) .data_type() .wrap_nullable(); + let mut builder = ColumnBuilder::with_capacity(&data_type, 1); - match bound { - Some(bound) => { - builder.push(bound.as_ref()); - } - None => { - builder.push(Scalar::Null.as_ref()); - } - } + let bound = match bound { + Some(bound) => bound.as_ref(), + None => Scalar::Null.as_ref(), + }; + builder.push(bound); block.add_const_column(builder.build_scalar(), data_type); } } From 9a036b923c412eb8485fdf545773d638d596ff07 Mon Sep 17 00:00:00 2001 From: coldWater Date: Mon, 7 Jul 2025 13:03:42 +0800 Subject: [PATCH 42/61] x --- .../transforms/sort/sort_execute.rs | 21 ------------------- 1 file changed, 21 deletions(-) diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_execute.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_execute.rs index a3dc5aa97776d..70c7aa5c3c6a4 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_execute.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_execute.rs @@ -16,9 +16,7 @@ use std::sync::Arc; use databend_common_exception::Result; use databend_common_expression::BlockMetaInfoDowncast; -use databend_common_expression::ColumnBuilder; use databend_common_expression::DataBlock; -use databend_common_expression::Scalar; use databend_common_pipeline_core::processors::Event; use databend_common_pipeline_core::processors::InputPort; use databend_common_pipeline_core::processors::OutputPort; @@ -107,7 +105,6 @@ where if self.remove_order_col { block.pop_columns(1); } - self.base.append_exchange_key(block, bound.as_ref()); self.output = Some(block); } if finish { @@ -116,21 +113,3 @@ where Ok(()) } } - -impl Base { - fn append_exchange_key(&self, &mut block: DataBlock, bound: Option<&Scalar>) { - let data_type = self - .schema - .field(self.base.sort_row_offset) - .data_type() - .wrap_nullable(); - - let mut builder = ColumnBuilder::with_capacity(&data_type, 1); - let bound = match bound { - Some(bound) => bound.as_ref(), - None => Scalar::Null.as_ref(), - }; - builder.push(bound); - block.add_const_column(builder.build_scalar(), data_type); - } -} From 14f7281bf5d06158c13eb9e604782a42808fcf4b Mon Sep 17 00:00:00 2001 From: coldWater Date: Mon, 7 Jul 2025 14:13:02 +0800 Subject: [PATCH 43/61] x --- Cargo.lock | 1 + src/query/service/Cargo.toml | 1 + .../transforms/sort/sort_exchange_injector.rs | 406 +++++++++--------- 3 files changed, 214 insertions(+), 194 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 78c94a905684e..30a98291f58ce 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -5366,6 +5366,7 @@ dependencies = [ "toml 0.8.22", "tonic", "tower 0.5.2", + "twox-hash 1.6.3", "typetag", "url", "uuid", diff --git a/src/query/service/Cargo.toml b/src/query/service/Cargo.toml index 8f264f45a2e09..fa66832451efc 100644 --- a/src/query/service/Cargo.toml +++ b/src/query/service/Cargo.toml @@ -171,6 +171,7 @@ tokio = { workspace = true } tokio-stream = { workspace = true, features = ["net"] } toml = { workspace = true, features = ["parse"] } tonic = { workspace = true } +twox-hash = { workspace = true } typetag = { workspace = true } url = { workspace = true } uuid = { workspace = true } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs index 9e3dc6b258c91..10263541fdbea 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs @@ -12,33 +12,27 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::marker::PhantomData; +use std::hash::Hash; +use std::hash::Hasher; use std::sync::Arc; -use arrow_ipc::writer::IpcWriteOptions; use databend_common_exception::Result; use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::DataBlock; -use databend_common_expression::DataSchemaRef; +use databend_common_expression::Scalar; use databend_common_pipeline_core::Pipeline; -use databend_common_pipeline_transforms::sort::Rows; -use databend_common_pipeline_transforms::AsyncBlockingTransform; -use databend_common_pipeline_transforms::BlockMetaTransform; use databend_common_settings::FlightCompression; +use twox_hash::XxHash64; -use crate::pipelines::processors::transforms::sort::bounds::Bounds; -use crate::pipelines::processors::transforms::sort::sort_spill::SpillableBlock; -use crate::pipelines::processors::transforms::sort::SortCollectedMeta; +use crate::pipelines::processors::transforms::SortBound; 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::exchange::ExchangeShuffleMeta; 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; -use crate::spillers::Spiller; pub struct SortInjector {} @@ -50,7 +44,11 @@ impl ExchangeInjector for SortInjector { ) -> Result>> { match exchange { DataExchange::Merge(_) | DataExchange::Broadcast(_) => unreachable!(), - DataExchange::ShuffleDataExchange(_) => Ok(Arc::new(Box::new(DummyScatter {}))), + DataExchange::ShuffleDataExchange(exchange) => { + Ok(Arc::new(Box::new(BoundHashScatter { + paritions: exchange.destination_ids.len() as _, + }))) + } } } @@ -89,193 +87,213 @@ impl ExchangeInjector for SortInjector { } } -pub struct DummyScatter {} - -impl FlightScatter for DummyScatter { - fn execute(&self, data_block: DataBlock) -> Result> { - Ok(vec![data_block]) - } -} - -pub struct TransformExchangeSortSerializer { - ctx: Arc, - local_pos: usize, - options: IpcWriteOptions, - - // params: Arc, - spiller: Arc, -} - -impl BlockMetaTransform for TransformExchangeSortSerializer { - const NAME: &'static str = "TransformExchangeSortSerializer"; - - fn transform(&mut self, _meta: ExchangeShuffleMeta) -> Result> { - // let serialized_blocks = meta - // .blocks - // .into_iter() - // .map(|mut block| { - // let SortCollectedMeta { - // params, - // bounds, - // blocks, - // } = block - // .take_meta() - // .and_then(SortCollectedMeta::downcast_from) - // .unwrap(); - - // 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, - // )?, - // }, - // }) - // .collect(); - - // let meta = SortCollectedMeta::downcast_from(block.take_meta().unwrap()).unwrap(); - - // match AggregateMeta::downcast_from(block.take_meta().unwrap()) { - // None => unreachable!(), - // Some(AggregateMeta::Spilled(_)) => unreachable!(), - // Some(AggregateMeta::Serialized(_)) => unreachable!(), - // Some(AggregateMeta::BucketSpilled(_)) => unreachable!(), - // Some(AggregateMeta::Partitioned { .. }) => unreachable!(), - // Some(AggregateMeta::AggregateSpilling(payload)) => { - // serialized_blocks.push(FlightSerialized::Future( - - // )); - // } - - // 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)); - // } - // }; - - todo!() - - // Ok(vec![DataBlock::empty_with_meta( - // FlightSerializedMeta::create(serialized_blocks), - // )]) - } -} - -struct SortScatter { - ctx: Arc, - local_pos: usize, - options: IpcWriteOptions, - schema: DataSchemaRef, - - partitions: usize, - spiller: Arc, - data: Option, - scatter_bounds: Bounds, - blocks: Vec>, - - _r: PhantomData, +pub struct BoundHashScatter { + paritions: u64, } -#[async_trait::async_trait] -impl AsyncBlockingTransform for SortScatter { - const NAME: &'static str = "TransformExchangeSortSerializer"; - - async fn consume(&mut self, mut block: DataBlock) -> Result<()> { - let meta = block - .take_meta() - .and_then(SortCollectedMeta::downcast_from) +impl FlightScatter for BoundHashScatter { + fn execute(&self, data_block: DataBlock) -> Result> { + let meta = data_block + .get_meta() + .and_then(SortBound::downcast_ref_from) .unwrap(); - self.data = Some(meta); - Ok(()) - } - async fn transform(&mut self) -> Result> { - todo!() - } -} - -impl SortScatter { - fn scatter_bounds(&self, bounds: Bounds) -> Bounds { - let n = self.partitions - 1; - let bounds = if bounds.len() < n { - bounds - } else { - bounds.dedup_reduce::(n) + let bound = match &meta.bound { + Some(bound) => { + debug_assert!(!bound.is_null()); + bound + } + None => &Scalar::Null, }; - assert!(bounds.len() < self.partitions); - bounds - } - async fn scatter(&mut self) -> Result>> { - // if self.scatter_bounds.is_empty() { - // return Ok(vec![Some(SortCollectedMeta { - // params, - // bounds, - // blocks, - // })]); - // } - - // let base = { - // Base { - // schema: self.schema.clone(), - // spiller: self.spiller.clone(), - // sort_row_offset: self.schema.fields.len() - 1, - // limit: None, - // } - // }; - - // let mut scattered_blocks = Vec::with_capacity(self.scatter_bounds.len() + 1); - - // let Some(list) = self.blocks.pop() else { - // todo!() - // }; - // let scattered = base - // .scatter_stream::(Vec::from(list).into(), self.scatter_bounds.clone()) - // .await?; - - // ExchangeShuffleMeta::create(blocks); - - // for list in { - - // } - - // let scattered_meta = scattered_blocks - // .into_iter() - // .map(|blocks| { - // (!blocks.is_empty()).then_some(SortCollectedMeta { - // params: todo!(), - // bounds: todo!(), - // blocks, - // }) - // }) - // .collect(); - // Ok(scattered_meta) - - todo!() + let mut hasher = XxHash64::default(); + bound.hash(&mut hasher); + let index = hasher.finish() % self.paritions; + + Ok(std::iter::repeat_n(DataBlock::empty(), index as _) + .chain(Some(data_block)) + .collect()) } } + +// pub struct TransformExchangeSortSerializer { +// ctx: Arc, +// local_pos: usize, +// options: IpcWriteOptions, + +// spiller: Arc, +// } + +// impl BlockMetaTransform for TransformExchangeSortSerializer { +// const NAME: &'static str = "TransformExchangeSortSerializer"; + +// fn transform(&mut self, _meta: ExchangeShuffleMeta) -> Result> { +// // let serialized_blocks = meta +// // .blocks +// // .into_iter() +// // .map(|mut block| { +// // let SortCollectedMeta { +// // params, +// // bounds, +// // blocks, +// // } = block +// // .take_meta() +// // .and_then(SortCollectedMeta::downcast_from) +// // .unwrap(); + +// // 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, +// // )?, +// // }, +// // }) +// // .collect(); + +// // let meta = SortCollectedMeta::downcast_from(block.take_meta().unwrap()).unwrap(); + +// // match AggregateMeta::downcast_from(block.take_meta().unwrap()) { +// // None => unreachable!(), +// // Some(AggregateMeta::Spilled(_)) => unreachable!(), +// // Some(AggregateMeta::Serialized(_)) => unreachable!(), +// // Some(AggregateMeta::BucketSpilled(_)) => unreachable!(), +// // Some(AggregateMeta::Partitioned { .. }) => unreachable!(), +// // Some(AggregateMeta::AggregateSpilling(payload)) => { +// // serialized_blocks.push(FlightSerialized::Future( + +// // )); +// // } + +// // 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)); +// // } +// // }; + +// todo!() + +// // Ok(vec![DataBlock::empty_with_meta( +// // FlightSerializedMeta::create(serialized_blocks), +// // )]) +// } +// } + +// struct SortScatter { +// ctx: Arc, +// local_pos: usize, +// options: IpcWriteOptions, +// schema: DataSchemaRef, + +// partitions: usize, +// spiller: Arc, +// data: Option, +// scatter_bounds: Bounds, +// blocks: Vec>, + +// _r: PhantomData, +// } + +// #[async_trait::async_trait] +// impl AsyncBlockingTransform for SortScatter { +// const NAME: &'static str = "TransformExchangeSortSerializer"; + +// async fn consume(&mut self, mut block: DataBlock) -> Result<()> { +// let meta = block +// .take_meta() +// .and_then(SortCollectedMeta::downcast_from) +// .unwrap(); +// self.data = Some(meta); +// Ok(()) +// } + +// async fn transform(&mut self) -> Result> { +// todo!() +// } +// } + +// impl SortScatter { +// fn scatter_bounds(&self, bounds: Bounds) -> Bounds { +// let n = self.partitions - 1; +// let bounds = if bounds.len() < n { +// bounds +// } else { +// bounds.dedup_reduce::(n) +// }; +// assert!(bounds.len() < self.partitions); +// bounds +// } + +// async fn scatter(&mut self) -> Result>> { +// // if self.scatter_bounds.is_empty() { +// // return Ok(vec![Some(SortCollectedMeta { +// // params, +// // bounds, +// // blocks, +// // })]); +// // } + +// // let base = { +// // Base { +// // schema: self.schema.clone(), +// // spiller: self.spiller.clone(), +// // sort_row_offset: self.schema.fields.len() - 1, +// // limit: None, +// // } +// // }; + +// // let mut scattered_blocks = Vec::with_capacity(self.scatter_bounds.len() + 1); + +// // let Some(list) = self.blocks.pop() else { +// // todo!() +// // }; +// // let scattered = base +// // .scatter_stream::(Vec::from(list).into(), self.scatter_bounds.clone()) +// // .await?; + +// // ExchangeShuffleMeta::create(blocks); + +// // for list in { + +// // } + +// // let scattered_meta = scattered_blocks +// // .into_iter() +// // .map(|blocks| { +// // (!blocks.is_empty()).then_some(SortCollectedMeta { +// // params: todo!(), +// // bounds: todo!(), +// // blocks, +// // }) +// // }) +// // .collect(); +// // Ok(scattered_meta) + +// todo!() +// } +// } From c5679826fecac15d7de22821fe986f6bfe927e96 Mon Sep 17 00:00:00 2001 From: coldWater Date: Mon, 7 Jul 2025 14:48:15 +0800 Subject: [PATCH 44/61] x --- .../src/pipelines/builders/builder_sort.rs | 54 ++++++++++--------- .../processors/transforms/sort/bounds.rs | 2 + .../processors/transforms/sort/sort_spill.rs | 2 + .../executor/physical_plans/physical_sort.rs | 6 +-- 4 files changed, 37 insertions(+), 27 deletions(-) diff --git a/src/query/service/src/pipelines/builders/builder_sort.rs b/src/query/service/src/pipelines/builders/builder_sort.rs index 6cb688c28e508..9b2b71c192f4a 100644 --- a/src/query/service/src/pipelines/builders/builder_sort.rs +++ b/src/query/service/src/pipelines/builders/builder_sort.rs @@ -67,7 +67,7 @@ impl PipelineBuilder { .collect::>>()?; let sort_desc = sort_desc.into(); - if sort.step != SortStep::RangeSort { + if sort.step != SortStep::SortShuffled { self.build_pipeline(&sort.input)?; } @@ -102,10 +102,29 @@ impl PipelineBuilder { self.main_pipeline.try_resize(max_threads)?; } - let builder = SortPipelineBuilder::create(self.ctx.clone(), plan_schema, sort_desc, None)? - .with_limit(sort.limit); + let builder = SortPipelineBuilder::create( + self.ctx.clone(), + plan_schema, + sort_desc, + sort.broadcast_id, + )? + .with_limit(sort.limit); match sort.step { + SortStep::Single => { + // Build for single node mode. + // We build the full sort pipeline for it. + builder + .remove_order_col_at_last() + .build_full_sort_pipeline(&mut self.main_pipeline) + } + + SortStep::Partial => { + // Build for each cluster node. + // We build the full sort pipeline for it. + // Don't remove the order column at last. + builder.build_full_sort_pipeline(&mut self.main_pipeline) + } SortStep::FinalMerge => { // Build for the coordinator node. // We only build a `MultiSortMergeTransform`, @@ -121,21 +140,9 @@ impl PipelineBuilder { .build_merge_sort_pipeline(&mut self.main_pipeline, true) } } - SortStep::Partial => { - // Build for each cluster node. - // We build the full sort pipeline for it. - // Don't remove the order column at last. - builder.build_full_sort_pipeline(&mut self.main_pipeline) - } - SortStep::Single => { - // Build for single node mode. - // We build the full sort pipeline for it. - builder - .remove_order_col_at_last() - .build_full_sort_pipeline(&mut self.main_pipeline) - } - SortStep::Sample => builder.build_sort_part(&mut self.main_pipeline), - SortStep::RangeSort => { + + SortStep::Sample => builder.build_sample(&mut self.main_pipeline), + SortStep::SortShuffled => { if matches!(*sort.input, PhysicalPlan::ExchangeSource(_)) { let exchange = builder.exchange_injector(); let old_inject = std::mem::replace(&mut self.exchange_injector, exchange); @@ -144,9 +151,10 @@ impl PipelineBuilder { } else { self.build_pipeline(&sort.input)?; } - todo!() + + self.main_pipeline.resize(1, false) } - SortStep::Route => todo!(), + SortStep::Route => self.main_pipeline.resize(1, false), } } } @@ -286,7 +294,7 @@ impl SortPipelineBuilder { }) } - fn build_sort_part(self, pipeline: &mut Pipeline) -> Result<()> { + fn build_sample(self, pipeline: &mut Pipeline) -> Result<()> { let settings = self.ctx.get_settings(); let max_block_size = settings.get_max_block_size()? as usize; @@ -335,9 +343,7 @@ impl SortPipelineBuilder { max_block_size, self.ctx.clone(), self.broadcast_id.unwrap(), - )?; - - Ok(()) + ) } fn build_merge_sort(&self, pipeline: &mut Pipeline, order_col_generated: bool) -> Result<()> { diff --git a/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs b/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs index 8029ff96bdf0d..f4f4c8052022b 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs @@ -95,10 +95,12 @@ impl Bounds { self.0.iter().map(Column::len).sum() } + #[expect(dead_code)] pub fn is_empty(&self) -> bool { self.0.iter().all(|col| col.len() == 0) } + #[expect(dead_code)] pub fn reduce(&self, n: usize) -> Option { if n == 0 { return Some(Self::default()); diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs index 91929135f95e6..d374db4ffcf4d 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs @@ -573,6 +573,7 @@ impl Base { } } + #[expect(dead_code)] pub async fn scatter_stream( &self, mut blocks: VecDeque, @@ -831,6 +832,7 @@ impl BoundBlockStream { Ok(()) } + #[expect(dead_code)] async fn take_next_bounded_spillable(&mut self) -> Result> { let Some(bound) = &self.bound else { return Ok(self.blocks.pop_front()); diff --git a/src/query/sql/src/executor/physical_plans/physical_sort.rs b/src/query/sql/src/executor/physical_plans/physical_sort.rs index d064f17673202..855f3a774e45f 100644 --- a/src/query/sql/src/executor/physical_plans/physical_sort.rs +++ b/src/query/sql/src/executor/physical_plans/physical_sort.rs @@ -61,7 +61,7 @@ pub enum SortStep { // range shuffle mode Sample, - RangeSort, + SortShuffled, Route, } @@ -94,7 +94,7 @@ impl Sort { fields.pop(); Ok(DataSchemaRefExt::create(fields)) } - SortStep::RangeSort => Ok(input_schema), + SortStep::SortShuffled => Ok(input_schema), SortStep::Single | SortStep::Partial | SortStep::Sample => { let mut fields = self .pre_projection @@ -278,7 +278,7 @@ impl PhysicalPlanBuilder { input: Box::new(exchange), order_by, limit: sort.limit, - step: SortStep::RangeSort, + step: SortStep::SortShuffled, pre_projection: None, broadcast_id: None, stat_info: Some(stat_info), From 881e07f7940ec6f8206e7306f136e3236d35b52f Mon Sep 17 00:00:00 2001 From: coldWater Date: Mon, 7 Jul 2025 16:36:04 +0800 Subject: [PATCH 45/61] format --- .../src/pipelines/builders/builder_sort.rs | 10 +++++++--- .../aggregator/aggregate_exchange_injector.rs | 4 ++++ .../processors/transforms/sort/bounds.rs | 2 +- .../transforms/sort/sort_exchange_injector.rs | 10 +++++++--- .../processors/transforms/sort/sort_shuffle.rs | 1 + .../processors/transforms/sort/sort_spill.rs | 1 - .../v1/exchange/exchange_transform_scatter.rs | 4 ++++ .../servers/flight/v1/scatter/flight_scatter.rs | 2 ++ .../flight/v1/scatter/flight_scatter_broadcast.rs | 4 ++++ .../flight/v1/scatter/flight_scatter_hash.rs | 8 ++++++++ src/query/sql/src/executor/format.rs | 9 ++++++++- .../src/executor/physical_plans/physical_sort.rs | 15 +++++++++++++++ 12 files changed, 61 insertions(+), 9 deletions(-) diff --git a/src/query/service/src/pipelines/builders/builder_sort.rs b/src/query/service/src/pipelines/builders/builder_sort.rs index 9b2b71c192f4a..ad3b48782543d 100644 --- a/src/query/service/src/pipelines/builders/builder_sort.rs +++ b/src/query/service/src/pipelines/builders/builder_sort.rs @@ -141,10 +141,14 @@ impl PipelineBuilder { } } - SortStep::Sample => builder.build_sample(&mut self.main_pipeline), + SortStep::Sample => { + builder.build_sample(&mut self.main_pipeline)?; + self.exchange_injector = Arc::new(SortInjector {}); + Ok(()) + } SortStep::SortShuffled => { if matches!(*sort.input, PhysicalPlan::ExchangeSource(_)) { - let exchange = builder.exchange_injector(); + let exchange = Arc::new(SortInjector {}); let old_inject = std::mem::replace(&mut self.exchange_injector, exchange); self.build_pipeline(&sort.input)?; self.exchange_injector = old_inject; @@ -329,7 +333,7 @@ impl SortPipelineBuilder { ) .with_limit(self.limit) .with_order_col_generated(false) - .with_output_order_col(false) + .with_output_order_col(true) .with_memory_settings(memory_settings) .with_enable_loser_tree(enable_loser_tree); 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..11ae2b014a6ad 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,6 +168,10 @@ fn scatter_partitioned_payload( } impl FlightScatter for HashTableHashScatter { + fn name(&self) -> &'static str { + "HashTableHash" + } + 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) { diff --git a/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs b/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs index f4f4c8052022b..e1a924c8debf6 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs @@ -100,7 +100,7 @@ impl Bounds { self.0.iter().all(|col| col.len() == 0) } - #[expect(dead_code)] + #[allow(dead_code)] pub fn reduce(&self, n: usize) -> Option { if n == 0 { return Some(Self::default()); diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs index 10263541fdbea..4150f703c266f 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs @@ -45,7 +45,7 @@ impl ExchangeInjector for SortInjector { match exchange { DataExchange::Merge(_) | DataExchange::Broadcast(_) => unreachable!(), DataExchange::ShuffleDataExchange(exchange) => { - Ok(Arc::new(Box::new(BoundHashScatter { + Ok(Arc::new(Box::new(SortBoundScatter { paritions: exchange.destination_ids.len() as _, }))) } @@ -87,11 +87,15 @@ impl ExchangeInjector for SortInjector { } } -pub struct BoundHashScatter { +pub struct SortBoundScatter { paritions: u64, } -impl FlightScatter for BoundHashScatter { +impl FlightScatter for SortBoundScatter { + fn name(&self) -> &'static str { + "SortBound" + } + fn execute(&self, data_block: DataBlock) -> Result> { let meta = data_block .get_meta() diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_shuffle.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_shuffle.rs index 3c6506dcfee6e..313953201068c 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_shuffle.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_shuffle.rs @@ -62,6 +62,7 @@ impl TransformSortBoundBroadcast { pub struct SortSampleState { ctx: Arc, broadcast_id: u32, + #[expect(dead_code)] schema: DataSchemaRef, batch_rows: usize, } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs index d374db4ffcf4d..a602699405a5c 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs @@ -832,7 +832,6 @@ impl BoundBlockStream { Ok(()) } - #[expect(dead_code)] async fn take_next_bounded_spillable(&mut self) -> Result> { let Some(bound) = &self.bound else { return Ok(self.blocks.pop_front()); 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 index 0b69270eab8e3..9afbe2504b26b 100644 --- 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 @@ -43,6 +43,10 @@ impl ScatterTransform { impl Transform for ScatterTransform { const NAME: &'static str = "ScatterTransform"; + fn name(&self) -> String { + format!("ScatterTransform({})", self.scatter.name()) + } + fn transform(&mut self, data: DataBlock) -> databend_common_exception::Result { let blocks = self.scatter.execute(data)?; diff --git a/src/query/service/src/servers/flight/v1/scatter/flight_scatter.rs b/src/query/service/src/servers/flight/v1/scatter/flight_scatter.rs index 4a29a487b3898..192d1027ef04d 100644 --- a/src/query/service/src/servers/flight/v1/scatter/flight_scatter.rs +++ b/src/query/service/src/servers/flight/v1/scatter/flight_scatter.rs @@ -16,5 +16,7 @@ use databend_common_exception::Result; use databend_common_expression::DataBlock; pub trait FlightScatter: Sync + Send { + fn name(&self) -> &'static str; + fn execute(&self, data_block: DataBlock) -> Result>; } diff --git a/src/query/service/src/servers/flight/v1/scatter/flight_scatter_broadcast.rs b/src/query/service/src/servers/flight/v1/scatter/flight_scatter_broadcast.rs index 89368238b0a57..70afa5d1fad6c 100644 --- a/src/query/service/src/servers/flight/v1/scatter/flight_scatter_broadcast.rs +++ b/src/query/service/src/servers/flight/v1/scatter/flight_scatter_broadcast.rs @@ -28,6 +28,10 @@ impl BroadcastFlightScatter { } impl FlightScatter for BroadcastFlightScatter { + fn name(&self) -> &'static str { + "Broadcast" + } + fn execute(&self, data_block: DataBlock) -> Result> { let mut data_blocks = vec![]; for _ in 0..self.scattered_size { 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 3be83e078c0f4..e6c03d09abd29 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 @@ -131,6 +131,10 @@ impl OneHashKeyFlightScatter { } impl FlightScatter for OneHashKeyFlightScatter { + fn name(&self) -> &'static str { + "OneHashKey" + } + fn execute(&self, data_block: DataBlock) -> Result> { let evaluator = Evaluator::new(&data_block, &self.func_ctx, &BUILTIN_FUNCTIONS); let num = data_block.num_rows(); @@ -150,6 +154,10 @@ impl FlightScatter for OneHashKeyFlightScatter { } impl FlightScatter for HashFlightScatter { + fn name(&self) -> &'static str { + "Hash" + } + fn execute(&self, data_block: DataBlock) -> Result> { let evaluator = Evaluator::new(&data_block, &self.func_ctx, &BUILTIN_FUNCTIONS); let num = data_block.num_rows(); diff --git a/src/query/sql/src/executor/format.rs b/src/query/sql/src/executor/format.rs index 684607c500e4e..53c8c9c4aaed9 100644 --- a/src/query/sql/src/executor/format.rs +++ b/src/query/sql/src/executor/format.rs @@ -1321,6 +1321,10 @@ fn sort_to_format_tree( FormatTreeNode::new(format!("sort keys: [{sort_keys}]")), ]; + if let Some(id) = plan.broadcast_id { + children.push(FormatTreeNode::new(format!("broadcast id: {id}"))); + } + if let Some(info) = &plan.stat_info { let items = plan_stats_info_to_format_tree(info); children.extend(items); @@ -1335,7 +1339,10 @@ fn sort_to_format_tree( context, )?); - Ok(FormatTreeNode::with_children("Sort".to_string(), children)) + Ok(FormatTreeNode::with_children( + format!("Sort({})", plan.step), + children, + )) } fn window_partition_to_format_tree( diff --git a/src/query/sql/src/executor/physical_plans/physical_sort.rs b/src/query/sql/src/executor/physical_plans/physical_sort.rs index 855f3a774e45f..c984f0238b789 100644 --- a/src/query/sql/src/executor/physical_plans/physical_sort.rs +++ b/src/query/sql/src/executor/physical_plans/physical_sort.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::fmt::Display; + use databend_common_exception::Result; use databend_common_expression::types::DataType; use databend_common_expression::DataField; @@ -65,6 +67,19 @@ pub enum SortStep { Route, } +impl Display for SortStep { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + match self { + SortStep::Single => write!(f, "Single"), + SortStep::Partial => write!(f, "Partial"), + SortStep::FinalMerge => write!(f, "FinalMerge"), + SortStep::Sample => write!(f, "Sample"), + SortStep::SortShuffled => write!(f, "SortShuffled"), + SortStep::Route => write!(f, "Route"), + } + } +} + impl Sort { fn order_col_type(&self, schema: &DataSchema) -> Result { if self.order_by.len() == 1 { From d791ccb06960cec2c540439a52dac86d1491c3b0 Mon Sep 17 00:00:00 2001 From: coldWater Date: Mon, 7 Jul 2025 19:08:36 +0800 Subject: [PATCH 46/61] x --- .../src/pipelines/builders/builder_sort.rs | 2 +- .../processors/transforms/sort/mod.rs | 2 +- .../transforms/sort/sort_builder.rs | 10 +- .../transforms/sort/sort_combine.rs | 6 +- .../transforms/sort/sort_exchange_injector.rs | 6 +- .../transforms/sort/sort_shuffle.rs | 91 +++++++++++-------- .../processors/transforms/sort/sort_spill.rs | 8 +- 7 files changed, 71 insertions(+), 54 deletions(-) diff --git a/src/query/service/src/pipelines/builders/builder_sort.rs b/src/query/service/src/pipelines/builders/builder_sort.rs index ad3b48782543d..25b94125c63af 100644 --- a/src/query/service/src/pipelines/builders/builder_sort.rs +++ b/src/query/service/src/pipelines/builders/builder_sort.rs @@ -450,7 +450,7 @@ impl SortPipelineBuilder { } } - pub fn exchange_injector(self) -> Arc { + pub fn exchange_injector(&self) -> Arc { Arc::new(SortInjector {}) } } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs index 6dc9fac578658..f7f0a398583f3 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs @@ -56,7 +56,7 @@ struct Base { struct SortCollectedMeta { params: SortSpillParams, bounds: Bounds, - blocks: Vec>, + sequences: Vec>, } local_block_meta_serde!(SortCollectedMeta); diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs index b3aef0f6fa806..4eb6d00d39428 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs @@ -237,13 +237,17 @@ impl TransformSortBuilder { broadcast_id: u32, ) -> Result<()> { let state = SortSampleState::new(schema, batch_rows, ctx, broadcast_id); + + let n = pipeline.output_len(); + pipeline.resize(1, false)?; pipeline.add_transform(|input, output| { Ok(ProcessorPtr::create(self.build_bound_broadcast( input, output, state.clone(), )?)) - }) + })?; + pipeline.resize(n, false) } pub fn exchange_injector(&self) -> Arc { @@ -307,7 +311,7 @@ impl Build<'_> { )?)) } - fn build_sort_shuffle(&mut self) -> Result> + fn build_bound_broadcast(&mut self) -> Result> where R: Rows + 'static { Ok(TransformSortBoundBroadcast::::create( self.input.clone(), @@ -355,7 +359,7 @@ impl RowsTypeVisitor for Build<'_> { true => self.build_sort_exec::>(), false => self.build_sort_exec::>(), }, - SortType::BoundBroadcast => self.build_sort_shuffle::(), + SortType::BoundBroadcast => self.build_bound_broadcast::(), SortType::Combine => self.build_sort_combine::(), } } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_combine.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_combine.rs index 538bd7aff390b..c3d2854e82bab 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_combine.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_combine.rs @@ -63,17 +63,17 @@ impl AccumulatingTransform for TransformSortCombine { .collect(); let bounds = Bounds::merge::(bounds, self.batch_rows)?; - let blocks = self + let sequences = self .metas .drain(..) - .flat_map(|meta| meta.blocks.into_iter()) + .flat_map(|meta| meta.sequences.into_iter()) .collect(); Ok(vec![DataBlock::empty_with_meta(Box::new( SortCollectedMeta { params, bounds, - blocks, + sequences, }, ))]) } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs index 4150f703c266f..795d0da63914e 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs @@ -46,7 +46,7 @@ impl ExchangeInjector for SortInjector { DataExchange::Merge(_) | DataExchange::Broadcast(_) => unreachable!(), DataExchange::ShuffleDataExchange(exchange) => { Ok(Arc::new(Box::new(SortBoundScatter { - paritions: exchange.destination_ids.len() as _, + partitions: exchange.destination_ids.len() as _, }))) } } @@ -88,7 +88,7 @@ impl ExchangeInjector for SortInjector { } pub struct SortBoundScatter { - paritions: u64, + partitions: u64, } impl FlightScatter for SortBoundScatter { @@ -112,7 +112,7 @@ impl FlightScatter for SortBoundScatter { let mut hasher = XxHash64::default(); bound.hash(&mut hasher); - let index = hasher.finish() % self.paritions; + let index = hasher.finish() % self.partitions; Ok(std::iter::repeat_n(DataBlock::empty(), index as _) .chain(Some(data_block)) diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_shuffle.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_shuffle.rs index 313953201068c..b4152c0985c68 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_shuffle.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_shuffle.rs @@ -35,8 +35,8 @@ use crate::sessions::QueryContext; pub struct TransformSortBoundBroadcast { state: SortSampleState, - input_data: Option, - output_data: Option, + input_data: Vec, + output_data: Option, called_on_finish: bool, _r: PhantomData, @@ -50,7 +50,7 @@ impl TransformSortBoundBroadcast { ) -> Box { Box::new(HookTransformer::new(input, output, Self { state, - input_data: None, + input_data: Vec::new(), output_data: None, called_on_finish: false, _r: PhantomData, @@ -117,13 +117,20 @@ impl HookTransform for TransformSortBoundBroadcast { .take_meta() .and_then(SortCollectedMeta::downcast_from) .expect("require a SortCollectedMeta"); - debug_assert!(self.input_data.is_none()); - self.input_data = Some(meta); + self.input_data.push(meta); Ok(()) } fn on_output(&mut self) -> Result> { - Ok(self.output_data.take()) + Ok(self.output_data.as_mut().and_then(|meta| { + meta.sequences.pop().map(|seq| { + DataBlock::empty_with_meta(Box::new(SortCollectedMeta { + params: meta.params, + bounds: meta.bounds.clone(), + sequences: vec![seq], + })) + }) + })) } fn need_process(&self, input_finished: bool) -> Option { @@ -136,39 +143,46 @@ impl HookTransform for TransformSortBoundBroadcast { async fn async_process(&mut self) -> Result<()> { self.called_on_finish = true; - let (metas, local) = match self.input_data.take() { - Some(meta) => { - let bounds = meta.normalize_bounds::(); - let metas = self - .state - .commit_sample(Some(SortExchangeMeta { - params: meta.params, - bounds, - })) - .await?; - (metas, Some(meta)) - } - None => (self.state.commit_sample(None).await?, None), - }; - - let params = match local.as_ref().map(|meta| meta.params) { - Some(params) => params, - None => { - let Some(meta) = metas.first() else { - return Ok(()); - }; - meta.params - } - }; - let bounds_vec = metas.iter().map(|meta| meta.bounds.clone()).collect(); - let global_bounds = Bounds::merge::(bounds_vec, self.state.batch_rows)?.dedup::(); + let bounds = Bounds::merge::( + self.input_data + .iter_mut() + .map(|meta| std::mem::take(&mut meta.bounds)) + .collect(), + self.state.batch_rows, + )?; + + let sequences: Vec<_> = self + .input_data + .iter_mut() + .flat_map(|meta| meta.sequences.drain(..)) + .collect(); + + if sequences.is_empty() { + self.state.commit_sample(None).await?; + return Ok(()); + } - self.output_data = Some(DataBlock::empty_with_meta(Box::new(SortCollectedMeta { + let params = self.input_data.first().unwrap().params; + let local = SortCollectedMeta { params, - bounds: global_bounds, - blocks: local.map(|meta| meta.blocks).unwrap_or_default(), - }))); + bounds, + sequences, + }; + + let global = self + .state + .commit_sample(Some(SortExchangeMeta { + params, + bounds: local.normalize_bounds::(), + })) + .await?; + + let bounds_vec = global.into_iter().map(|meta| meta.bounds).collect(); + self.output_data = Some(SortCollectedMeta { + bounds: Bounds::merge::(bounds_vec, self.state.batch_rows)?.dedup::(), + ..local + }); Ok(()) } } @@ -179,12 +193,11 @@ impl SortCollectedMeta { return self.bounds.dedup::(); } - let Some(blocks) = self.blocks.get(self.blocks.len() / 2) else { + let Some(seq) = self.sequences.get(self.sequences.len() / 2) else { return Bounds::default(); }; - blocks - .get(blocks.len() / 2) + seq.get(seq.len() / 2) .map(|block| match block.domain.len() { 0 => Bounds::default(), 1 => Bounds::new_unchecked(block.domain.clone()), diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs index a602699405a5c..8173c40eadc8f 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs @@ -100,10 +100,10 @@ where A: SortAlgorithm let SortCollectedMeta { params, bounds, - blocks, + sequences, } = meta; - let subsequent = blocks + let subsequent = sequences .into_iter() .filter_map(|list| { (!list.is_empty()).then(|| base.new_stream(Vec::from(list).into(), None)) @@ -209,7 +209,7 @@ where A: SortAlgorithm .. } = collect.next_step(&base)?; - let blocks = subsequent + let sequences = subsequent .into_iter() .map(|stream| { assert!(stream.bound.is_none()); @@ -219,7 +219,7 @@ where A: SortAlgorithm Ok(SortCollectedMeta { params, - blocks, + sequences, bounds, }) } From 932600b5c5aa5fb5087c2b76b41e088d30cb5e0d Mon Sep 17 00:00:00 2001 From: coldWater Date: Mon, 7 Jul 2025 20:12:57 +0800 Subject: [PATCH 47/61] x --- .../src/pipelines/builders/builder_sort.rs | 8 +++-- .../transforms/sort/sort_builder.rs | 19 +++++------- .../transforms/sort/sort_execute.rs | 30 +++++++++++++++---- .../processors/transforms/sort/sort_spill.rs | 4 +-- 4 files changed, 39 insertions(+), 22 deletions(-) diff --git a/src/query/service/src/pipelines/builders/builder_sort.rs b/src/query/service/src/pipelines/builders/builder_sort.rs index 25b94125c63af..b88ac0010c175 100644 --- a/src/query/service/src/pipelines/builders/builder_sort.rs +++ b/src/query/service/src/pipelines/builders/builder_sort.rs @@ -284,7 +284,7 @@ impl SortPipelineBuilder { })?; pipeline.add_transform(|input, output| { - Ok(ProcessorPtr::create(builder.build_exec(input, output)?)) + Ok(ProcessorPtr::create(builder.build_restore(input, output)?)) })?; add_range_shuffle_route(pipeline)?; @@ -347,7 +347,11 @@ impl SortPipelineBuilder { max_block_size, self.ctx.clone(), self.broadcast_id.unwrap(), - ) + )?; + + pipeline.add_transform(|input, output| { + Ok(ProcessorPtr::create(builder.build_restore(input, output)?)) + }) } fn build_merge_sort(&self, pipeline: &mut Pipeline, order_col_generated: bool) -> Result<()> { diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs index 4eb6d00d39428..6a1eeddc753a0 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs @@ -41,7 +41,6 @@ use super::sort_execute::TransformSortRestore; use super::sort_shuffle::SortSampleState; use super::sort_shuffle::TransformSortBoundBroadcast; use super::Base; -use crate::servers::flight::v1::exchange::ExchangeInjector; use crate::sessions::QueryContext; use crate::spillers::Spiller; @@ -50,7 +49,7 @@ enum SortType { Collect, BoundBroadcast, Combine, - Execute, + Restore, } pub struct TransformSortBuilder { @@ -146,7 +145,7 @@ impl TransformSortBuilder { select_row_type(&mut build) } - pub fn build_exec( + pub fn build_restore( &self, input: Arc, output: Arc, @@ -157,7 +156,7 @@ impl TransformSortBuilder { params: self, input, output, - typ: SortType::Execute, + typ: SortType::Restore, state: None, }; @@ -249,10 +248,6 @@ impl TransformSortBuilder { })?; pipeline.resize(n, false) } - - pub fn exchange_injector(&self) -> Arc { - todo!() - } } struct Build<'a> { @@ -301,7 +296,7 @@ impl Build<'_> { )?)) } - fn build_sort_exec(&mut self) -> Result> + fn build_sort_restore(&mut self) -> Result> where A: SortAlgorithm + 'static { Ok(Box::new(TransformSortRestore::::create( self.input.clone(), @@ -355,9 +350,9 @@ impl RowsTypeVisitor for Build<'_> { true => self.build_sort_collect::, C>(limit_sort), false => self.build_sort_collect::, C>(limit_sort), }, - SortType::Execute => match self.params.enable_loser_tree { - true => self.build_sort_exec::>(), - false => self.build_sort_exec::>(), + SortType::Restore => match self.params.enable_loser_tree { + true => self.build_sort_restore::>(), + false => self.build_sort_restore::>(), }, SortType::BoundBroadcast => self.build_bound_broadcast::(), SortType::Combine => self.build_sort_combine::(), diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_execute.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_execute.rs index 70c7aa5c3c6a4..8326ea501119b 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_execute.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_execute.rs @@ -31,6 +31,7 @@ use super::SortCollectedMeta; use crate::pipelines::processors::transforms::sort::sort_spill::OutputData; pub struct TransformSortRestore { + input: Vec, output: Option, /// If the next transform of current transform is [`super::transform_multi_sort_merge::MultiSortMergeProcessor`], @@ -51,6 +52,7 @@ where A: SortAlgorithm + Send + 'static output_order_col: bool, ) -> Result> { Ok(HookTransformer::new(input, output, Self { + input: Vec::new(), output: None, remove_order_col: !output_order_col, base, @@ -73,8 +75,7 @@ where .take_meta() .and_then(SortCollectedMeta::downcast_from) .expect("require a SortCollectedMeta"); - - self.inner = Some(SortSpill::::from_meta(self.base.clone(), meta)); + self.input.push(meta); Ok(()) } @@ -82,8 +83,8 @@ where Ok(self.output.take()) } - fn need_process(&self, _: bool) -> Option { - if self.inner.is_some() { + fn need_process(&self, input_finished: bool) -> Option { + if input_finished && (self.inner.is_some() || !self.input.is_empty()) { Some(Event::Async) } else { None @@ -92,9 +93,26 @@ where #[async_backtrace::framed] async fn async_process(&mut self) -> Result<()> { - let Some(spill_sort) = &mut self.inner else { - unreachable!() + let spill_sort = match &mut self.inner { + Some(inner) => inner, + None => { + debug_assert!(!self.input.is_empty()); + let sequences = self + .input + .iter_mut() + .flat_map(|meta| meta.sequences.drain(..)) + .collect(); + + let meta = self.input.pop().unwrap(); + self.input.clear(); + self.inner + .insert(SortSpill::from_meta(self.base.clone(), SortCollectedMeta { + sequences, + ..meta + })) + } }; + let OutputData { block, bound, diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs index 8173c40eadc8f..f74c1f73f0691 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs @@ -105,8 +105,8 @@ where A: SortAlgorithm let subsequent = sequences .into_iter() - .filter_map(|list| { - (!list.is_empty()).then(|| base.new_stream(Vec::from(list).into(), None)) + .filter_map(|seq| { + (!seq.is_empty()).then(|| base.new_stream(Vec::from(seq).into(), None)) }) .collect::>(); debug_assert!(!subsequent.is_empty()); From 56ae36ecddc77e3c907e97b018c66c15cc133211 Mon Sep 17 00:00:00 2001 From: coldWater Date: Tue, 8 Jul 2025 16:21:50 +0800 Subject: [PATCH 48/61] allow_adjust_parallelism --- src/query/pipeline/core/src/pipeline.rs | 25 ++++++++--- .../transforms/transform_multi_sort_merge.rs | 4 +- .../builders/builder_hilbert_partition.rs | 2 +- .../src/pipelines/builders/builder_sort.rs | 23 +++++++++- .../src/pipelines/builders/builder_window.rs | 4 +- .../processors/transforms/sort/mod.rs | 2 +- .../transforms/sort/sort_builder.rs | 6 +-- .../transforms/sort/sort_exchange.rs | 11 +++++ .../transforms/sort/sort_exchange_injector.rs | 42 ++++++++++--------- .../sort/{sort_execute.rs => sort_restore.rs} | 2 +- .../src/schedulers/fragments/fragmenter.rs | 37 ++++++++-------- .../flight/v1/exchange/data_exchange.rs | 10 +---- .../flight/v1/exchange/exchange_manager.rs | 2 + .../flight/v1/exchange/exchange_params.rs | 1 + .../flight/v1/exchange/exchange_transform.rs | 23 +++++++--- 15 files changed, 125 insertions(+), 69 deletions(-) rename src/query/service/src/pipelines/processors/transforms/sort/{sort_execute.rs => sort_restore.rs} (98%) diff --git a/src/query/pipeline/core/src/pipeline.rs b/src/query/pipeline/core/src/pipeline.rs index 2f1ff886df989..9600adf0b78c0 100644 --- a/src/query/pipeline/core/src/pipeline.rs +++ b/src/query/pipeline/core/src/pipeline.rs @@ -458,15 +458,29 @@ impl Pipeline { self.sinks = new_sinks; } - pub fn exchange(&mut self, n: usize, exchange: Arc) { + pub fn exchange(&mut self, n: usize, exchange: Arc) -> Result<()> { + self.exchange_with_merge(n, exchange.clone(), |inputs, output| { + Ok(MergePartitionProcessor::create( + inputs, + output, + exchange.clone(), + )) + }) + } + + pub fn exchange_with_merge(&mut self, n: usize, exchange: Arc, f: F) -> Result<()> + where + T: Exchange, + F: Fn(Vec>, Arc) -> Result, + { if self.sinks.is_empty() { - return; + return Ok(()); } let input_len = self.sinks.len(); let mut items = Vec::with_capacity(input_len); - for _index in 0..input_len { + for _ in 0..input_len { let input = InputPort::create(); let outputs: Vec<_> = (0..n).map(|_| OutputPort::create()).collect(); items.push(PipeItem::create( @@ -491,14 +505,15 @@ 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()), + f(inputs.clone(), output.clone())?, inputs, vec![output], )); } // merge partition - self.add_pipe(Pipe::create(input_len * n, n, items)) + self.add_pipe(Pipe::create(input_len * n, n, items)); + Ok(()) } #[track_caller] diff --git a/src/query/pipeline/transforms/src/processors/transforms/transform_multi_sort_merge.rs b/src/query/pipeline/transforms/src/processors/transforms/transform_multi_sort_merge.rs index 573315604e414..632deef1d0ba4 100644 --- a/src/query/pipeline/transforms/src/processors/transforms/transform_multi_sort_merge.rs +++ b/src/query/pipeline/transforms/src/processors/transforms/transform_multi_sort_merge.rs @@ -77,7 +77,7 @@ pub fn try_add_multi_sort_merge( } let output_port = OutputPort::create(); - let processor = ProcessorPtr::create(create_processor( + let processor = ProcessorPtr::create(create_multi_sort_processor( inputs_port.clone(), output_port.clone(), schema, @@ -98,7 +98,7 @@ pub fn try_add_multi_sort_merge( } } -fn create_processor( +pub fn create_multi_sort_processor( inputs: Vec>, output: Arc, schema: DataSchemaRef, 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 86b23bfca9238..d7ef224108129 100644 --- a/src/query/service/src/pipelines/builders/builder_hilbert_partition.rs +++ b/src/query/service/src/pipelines/builders/builder_hilbert_partition.rs @@ -47,7 +47,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_sort.rs b/src/query/service/src/pipelines/builders/builder_sort.rs index b88ac0010c175..e8340d64f13bb 100644 --- a/src/query/service/src/pipelines/builders/builder_sort.rs +++ b/src/query/service/src/pipelines/builders/builder_sort.rs @@ -277,7 +277,7 @@ impl SortPipelineBuilder { self.broadcast_id.unwrap(), )?; - pipeline.exchange(num_exec, Arc::new(SortRangeExchange)); + pipeline.exchange(num_exec, Arc::new(SortRangeExchange))?; pipeline.add_transform(|input, output| { Ok(ProcessorPtr::create(builder.build_combine(input, output)?)) @@ -351,7 +351,26 @@ impl SortPipelineBuilder { pipeline.add_transform(|input, output| { Ok(ProcessorPtr::create(builder.build_restore(input, output)?)) - }) + })?; + + // pipeline.exchange_with_merge( + // pipeline.output_len(), + // Arc::new(SortBoundExchange {}), + // |inputs, output| { + // Ok(ProcessorPtr::create(create_multi_sort_processor( + // inputs, + // output, + // self.schema.clone(), + // self.block_size, + // self.limit, + // self.sort_desc.clone(), + // self.remove_order_col_at_last, + // self.enable_loser_tree, + // )?)) + // }, + // )?; + + Ok(()) } fn build_merge_sort(&self, pipeline: &mut Pipeline, order_col_generated: bool) -> Result<()> { diff --git a/src/query/service/src/pipelines/builders/builder_window.rs b/src/query/service/src/pipelines/builders/builder_window.rs index 7906dfd96f164..c25228606717d 100644 --- a/src/query/service/src/pipelines/builders/builder_window.rs +++ b/src/query/service/src/pipelines/builders/builder_window.rs @@ -183,12 +183,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 temp_dir_manager = TempDirManager::instance(); diff --git a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs index f7f0a398583f3..798d995d03b07 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs @@ -33,7 +33,7 @@ mod sort_collect; mod sort_combine; mod sort_exchange; mod sort_exchange_injector; -mod sort_execute; +mod sort_restore; mod sort_route; mod sort_shuffle; mod sort_spill; diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs index 6a1eeddc753a0..5c5f560e5be04 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs @@ -37,7 +37,7 @@ use databend_common_pipeline_transforms::MemorySettings; use super::merge_sort::TransformSort; use super::sort_collect::TransformSortCollect; use super::sort_combine::TransformSortCombine; -use super::sort_execute::TransformSortRestore; +use super::sort_restore::TransformSortRestore; use super::sort_shuffle::SortSampleState; use super::sort_shuffle::TransformSortBoundBroadcast; use super::Base; @@ -237,7 +237,6 @@ impl TransformSortBuilder { ) -> Result<()> { let state = SortSampleState::new(schema, batch_rows, ctx, broadcast_id); - let n = pipeline.output_len(); pipeline.resize(1, false)?; pipeline.add_transform(|input, output| { Ok(ProcessorPtr::create(self.build_bound_broadcast( @@ -245,8 +244,7 @@ impl TransformSortBuilder { output, state.clone(), )?)) - })?; - pipeline.resize(n, false) + }) } } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange.rs index 700c5a3e0e81a..08a3a2adf1b7f 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange.rs @@ -17,6 +17,7 @@ use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::DataBlock; use databend_common_pipeline_core::processors::Exchange; +use super::bound_scatter; use super::SortScatteredMeta; pub struct SortRangeExchange; @@ -42,3 +43,13 @@ impl Exchange for SortRangeExchange { Ok(blocks) } } + +pub struct SortBoundExchange; + +impl Exchange for SortBoundExchange { + const NAME: &'static str = "SortBound"; + + fn partition(&self, data_block: DataBlock, n: usize) -> Result> { + bound_scatter(data_block, n as _) + } +} diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs index 795d0da63914e..fe4199b837bef 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs @@ -97,27 +97,31 @@ impl FlightScatter for SortBoundScatter { } fn execute(&self, data_block: DataBlock) -> Result> { - let meta = data_block - .get_meta() - .and_then(SortBound::downcast_ref_from) - .unwrap(); - - let bound = match &meta.bound { - Some(bound) => { - debug_assert!(!bound.is_null()); - bound - } - None => &Scalar::Null, - }; + bound_scatter(data_block, self.partitions) + } +} - let mut hasher = XxHash64::default(); - bound.hash(&mut hasher); - let index = hasher.finish() % self.partitions; +pub(super) fn bound_scatter(data_block: DataBlock, n: u64) -> Result> { + let meta = data_block + .get_meta() + .and_then(SortBound::downcast_ref_from) + .unwrap(); - Ok(std::iter::repeat_n(DataBlock::empty(), index as _) - .chain(Some(data_block)) - .collect()) - } + let bound = match &meta.bound { + Some(bound) => { + debug_assert!(!bound.is_null()); + bound + } + None => &Scalar::Null, + }; + + let mut hasher = XxHash64::default(); + bound.hash(&mut hasher); + let index = hasher.finish() % n; + + Ok(std::iter::repeat_n(DataBlock::empty(), index as _) + .chain(Some(data_block)) + .collect()) } // pub struct TransformExchangeSortSerializer { diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_execute.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_restore.rs similarity index 98% rename from src/query/service/src/pipelines/processors/transforms/sort/sort_execute.rs rename to src/query/service/src/pipelines/processors/transforms/sort/sort_restore.rs index 8326ea501119b..c6599601e0bbc 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_execute.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_restore.rs @@ -67,7 +67,7 @@ where A: SortAlgorithm + 'static, A::Rows: 'static, { - const NAME: &'static str = "TransformSortExecute"; + const NAME: &'static str = "TransformSortRestore"; fn on_input(&mut self, mut block: DataBlock) -> Result<()> { assert!(self.inner.is_none()); diff --git a/src/query/service/src/schedulers/fragments/fragmenter.rs b/src/query/service/src/schedulers/fragments/fragmenter.rs index 2e7a6e878b819..18152bacdcd2a 100644 --- a/src/query/service/src/schedulers/fragments/fragmenter.rs +++ b/src/query/service/src/schedulers/fragments/fragmenter.rs @@ -101,23 +101,26 @@ impl Fragmenter { ctx: Arc, plan: &PhysicalPlan, ) -> Result> { - match plan { - PhysicalPlan::ExchangeSink(plan) => match plan.kind { - FragmentKind::Normal => Ok(Some(ShuffleDataExchange::create( - Self::get_executors(ctx), - plan.keys.clone(), - ))), - FragmentKind::Merge => Ok(Some(MergeExchange::create( - Self::get_local_executor(ctx), - plan.ignore_exchange, - plan.allow_adjust_parallelism, - ))), - FragmentKind::Expansive => { - Ok(Some(BroadcastExchange::create(Self::get_executors(ctx)))) - } - _ => Ok(None), - }, - _ => Ok(None), + let PhysicalPlan::ExchangeSink(exchange) = plan else { + return Ok(None); + }; + match exchange.kind { + FragmentKind::Init => Ok(None), + FragmentKind::Normal => Ok(Some(DataExchange::ShuffleDataExchange( + ShuffleDataExchange { + destination_ids: Self::get_executors(ctx), + shuffle_keys: exchange.keys.clone(), + allow_adjust_parallelism: exchange.allow_adjust_parallelism, + }, + ))), + FragmentKind::Merge => Ok(Some(MergeExchange::create( + Self::get_local_executor(ctx), + exchange.ignore_exchange, + exchange.allow_adjust_parallelism, + ))), + FragmentKind::Expansive => { + Ok(Some(BroadcastExchange::create(Self::get_executors(ctx)))) + } } } diff --git a/src/query/service/src/servers/flight/v1/exchange/data_exchange.rs b/src/query/service/src/servers/flight/v1/exchange/data_exchange.rs index f23c7582559a7..0e6997aca9157 100644 --- a/src/query/service/src/servers/flight/v1/exchange/data_exchange.rs +++ b/src/query/service/src/servers/flight/v1/exchange/data_exchange.rs @@ -35,15 +35,7 @@ impl DataExchange { pub struct ShuffleDataExchange { pub destination_ids: Vec, pub shuffle_keys: Vec, -} - -impl ShuffleDataExchange { - pub fn create(destination_ids: Vec, shuffle_keys: Vec) -> DataExchange { - DataExchange::ShuffleDataExchange(ShuffleDataExchange { - destination_ids, - shuffle_keys, - }) - } + pub allow_adjust_parallelism: bool, } #[derive(Debug, Clone, serde::Serialize, serde::Deserialize)] 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 1bb5af570df33..09a009eaaaa50 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 @@ -1010,6 +1010,7 @@ impl FragmentCoordinator { destination_ids: exchange.destination_ids.to_owned(), shuffle_scatter: exchange_injector .flight_scatter(&info.query_ctx, data_exchange)?, + allow_adjust_parallelism: true, }, ))), DataExchange::ShuffleDataExchange(exchange) => Ok(Some( @@ -1022,6 +1023,7 @@ impl FragmentCoordinator { destination_ids: exchange.destination_ids.to_owned(), shuffle_scatter: exchange_injector .flight_scatter(&info.query_ctx, data_exchange)?, + allow_adjust_parallelism: exchange.allow_adjust_parallelism, }), )), } 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..64298c28b2218 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 @@ -28,6 +28,7 @@ pub struct ShuffleExchangeParams { pub destination_ids: Vec, pub shuffle_scatter: Arc>, pub exchange_injector: Arc, + pub allow_adjust_parallelism: bool, } #[derive(Clone)] 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..c8db2ccf17f90 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 @@ -48,7 +48,11 @@ impl ExchangeTransform { // exchange writer sink and resize and exchange reader let len = params.destination_ids.len(); - let max_threads = ctx.get_settings().get_max_threads()? as usize; + let local_pipe = if params.allow_adjust_parallelism { + ctx.get_settings().get_max_threads()? as usize + } else { + 1 + }; let mut items = Vec::with_capacity(len); let exchange_params = ExchangeParams::ShuffleExchange(params.clone()); @@ -58,8 +62,13 @@ 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 => { + if local_pipe == 1 { + create_dummy_item() + } else { + create_resize_item(1, local_pipe) + } + } false => create_writer_item( sender, false, @@ -84,11 +93,13 @@ impl ExchangeTransform { } } - let new_outputs = max_threads + nodes_source; + let new_outputs = local_pipe + nodes_source; pipeline.add_pipe(Pipe::create(len, new_outputs, items)); - if params.exchange_injector.exchange_sorting().is_none() { - pipeline.try_resize(max_threads)?; + if params.exchange_injector.exchange_sorting().is_none() + && params.allow_adjust_parallelism + { + pipeline.try_resize(ctx.get_settings().get_max_threads()? as usize)?; } injector.apply_shuffle_deserializer(params, pipeline) From c672b695446c5a54ff30337ff95fc002145d68ff Mon Sep 17 00:00:00 2001 From: coldWater Date: Tue, 8 Jul 2025 17:51:46 +0800 Subject: [PATCH 49/61] InputBoundStream --- .../processors/transforms/sort/mod.rs | 1 + .../transforms/sort/sort_merge_stream.rs | 99 +++++++++++++++++++ .../processors/transforms/sort/sort_spill.rs | 2 +- 3 files changed, 101 insertions(+), 1 deletion(-) create mode 100644 src/query/service/src/pipelines/processors/transforms/sort/sort_merge_stream.rs diff --git a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs index 798d995d03b07..c406de5bd1542 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs @@ -33,6 +33,7 @@ mod sort_collect; mod sort_combine; mod sort_exchange; mod sort_exchange_injector; +mod sort_merge_stream; mod sort_restore; mod sort_route; mod sort_shuffle; diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_merge_stream.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_merge_stream.rs new file mode 100644 index 0000000000000..9e649e640a725 --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_merge_stream.rs @@ -0,0 +1,99 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::marker::PhantomData; +use std::sync::Arc; + +use databend_common_exception::Result; +use databend_common_expression::BlockMetaInfoDowncast; +use databend_common_expression::Column; +use databend_common_expression::DataBlock; +use databend_common_expression::Scalar; +use databend_common_pipeline_core::processors::InputPort; +use databend_common_pipeline_transforms::sort::Rows; +use databend_common_pipeline_transforms::sort::SortedStream; + +use crate::pipelines::processors::transforms::SortBound; + +/// InputBoundStream is a stream of blocks that are cutoff less or equal than bound. +struct InputBoundStream { + data: Option, + input: Arc, + remove_order_col: bool, + bound: Option, + sort_row_offset: usize, + _r: PhantomData, +} + +impl SortedStream for InputBoundStream { + fn next(&mut self) -> Result<(Option<(DataBlock, Column)>, bool)> { + if self.pull()? { + return Ok((None, true)); + } + + match self.take_next_bounded_block() { + None => Ok((None, false)), + Some(mut block) => { + let col = sort_column(&block, self.sort_row_offset).clone(); + if self.remove_order_col { + block.remove_column(self.sort_row_offset); + } + Ok((Some((block, col)), false)) + } + } + } +} + +fn sort_column(data: &DataBlock, sort_row_offset: usize) -> &Column { + data.get_by_offset(sort_row_offset).as_column().unwrap() +} + +impl InputBoundStream { + fn pull(&mut self) -> Result { + if self.data.is_some() { + return Ok(false); + } + + if self.input.has_data() { + let block = self.input.pull_data().unwrap()?; + self.input.set_need_data(); + self.data = Some(block); + Ok(false) + } else if self.input.is_finished() { + Ok(false) + } else { + self.input.set_need_data(); + Ok(true) + } + } + + fn take_next_bounded_block(&mut self) -> Option { + let Some(bound) = &self.bound else { + return self.data.take(); + }; + + let meta = self + .data + .as_ref()? + .get_meta() + .and_then(SortBound::downcast_ref_from) + .expect("require a SortBound"); + + if meta.bound.as_ref() == Some(bound) { + self.data.take() + } else { + None + } + } +} diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs index f74c1f73f0691..6faf66265a2f2 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs @@ -865,7 +865,7 @@ impl BoundBlockStream { } } -fn block_split_off_position( +pub fn block_split_off_position( data: &DataBlock, bound: &Scalar, sort_row_offset: usize, From c5db85b0d4b681487d5c01059b57c904cfc5f5d7 Mon Sep 17 00:00:00 2001 From: coldWater Date: Tue, 8 Jul 2025 20:29:06 +0800 Subject: [PATCH 50/61] BoundedMultiSortMergeProcessor --- .../transforms/sort/sort_merge_stream.rs | 177 +++++++++++++++++- 1 file changed, 167 insertions(+), 10 deletions(-) diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_merge_stream.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_merge_stream.rs index 9e649e640a725..cd9294c9096e8 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_merge_stream.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_merge_stream.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::any::Any; +use std::cmp::Ordering; use std::marker::PhantomData; use std::sync::Arc; @@ -19,15 +21,171 @@ use databend_common_exception::Result; use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::Column; use databend_common_expression::DataBlock; +use databend_common_expression::DataSchemaRef; use databend_common_expression::Scalar; +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::sort::algorithm::SortAlgorithm; +use databend_common_pipeline_transforms::sort::Merger; use databend_common_pipeline_transforms::sort::Rows; use databend_common_pipeline_transforms::sort::SortedStream; use crate::pipelines::processors::transforms::SortBound; -/// InputBoundStream is a stream of blocks that are cutoff less or equal than bound. -struct InputBoundStream { +type Stream = BoundedInputStream<::Rows>; + +pub struct BoundedMultiSortMergeProcessor +where A: SortAlgorithm +{ + inputs: Vec>, + output: Arc, + schema: DataSchemaRef, + block_size: usize, + limit: Option, + + output_data: Option, + bound: Option, + inner: std::result::Result>, Vec>>, +} + +impl BoundedMultiSortMergeProcessor +where A: SortAlgorithm +{ + pub fn create( + inputs: Vec>, + output: Arc, + schema: DataSchemaRef, + block_size: usize, + limit: Option, + remove_order_col: bool, + ) -> Result { + let streams = inputs + .iter() + .map(|input| BoundedInputStream { + data: None, + input: input.clone(), + remove_order_col, + bound: None, + sort_row_offset: schema.fields().len() - 1, + _r: PhantomData, + }) + .collect(); + + Ok(Self { + inputs, + output, + schema, + block_size, + limit, + output_data: None, + bound: None, + inner: Err(streams), + }) + } +} + +impl Processor for BoundedMultiSortMergeProcessor +where A: SortAlgorithm + 'static +{ + fn name(&self) -> String { + "BoundedMultiSortMerge".to_string() + } + + fn as_any(&mut self) -> &mut dyn Any { + self + } + + fn event(&mut self) -> Result { + if self.output.is_finished() { + for input in self.inputs.iter() { + input.finish(); + } + return Ok(Event::Finished); + } + + if !self.output.can_push() { + return Ok(Event::NeedConsume); + } + + if let Some(block) = self.output_data.take() { + self.output.push_data(Ok(block)); + return Ok(Event::NeedConsume); + } + + self.next_event() + } + + fn process(&mut self) -> Result<()> { + if let Some(block) = self.inner.as_mut().ok().unwrap().next_block()? { + let meta = SortBound { + bound: self.bound.clone(), + } + .boxed(); + self.output_data = Some(block.add_meta(Some(meta))?); + }; + Ok(()) + } +} + +impl BoundedMultiSortMergeProcessor +where A: SortAlgorithm + 'static +{ + fn next_event(&mut self) -> Result { + let streams = match &mut self.inner { + inner @ Ok(_) => { + let merger = inner.as_ref().ok().unwrap(); + if !merger.is_finished() { + return Ok(Event::Sync); + } + let merger = std::mem::replace(inner, Err(vec![])).ok().unwrap(); + self.inner = Err(merger.streams()); + self.inner.as_mut().err().unwrap() + } + Err(streams) => streams, + }; + + let mut bounds = Vec::with_capacity(streams.len()); + for stream in streams.iter_mut() { + if stream.pull()? { + return Ok(Event::NeedData); + } + let Some(data) = &stream.data else { + continue; + }; + let meta = data + .get_meta() + .and_then(SortBound::downcast_ref_from) + .expect("require a SortBound"); + bounds.push(&meta.bound) + } + + self.bound = match bounds.iter().min_by(|a, b| match (a, b) { + (None, None) => Ordering::Equal, + (None, Some(_)) => Ordering::Greater, + (Some(_), None) => Ordering::Less, + (Some(a), Some(b)) => A::Rows::scalar_as_item(a).cmp(&A::Rows::scalar_as_item(b)), + }) { + Some(bound) => (*bound).clone(), + None => return Ok(Event::Finished), + }; + for stream in streams.iter_mut() { + stream.bound = self.bound.clone(); + } + + let streams = std::mem::take(streams); + self.inner = Ok(Merger::create( + self.schema.clone(), + streams, + self.block_size, + self.limit, + )); + Ok(Event::Sync) + } +} + +struct BoundedInputStream { data: Option, input: Arc, remove_order_col: bool, @@ -36,7 +194,7 @@ struct InputBoundStream { _r: PhantomData, } -impl SortedStream for InputBoundStream { +impl SortedStream for BoundedInputStream { fn next(&mut self) -> Result<(Option<(DataBlock, Column)>, bool)> { if self.pull()? { return Ok((None, true)); @@ -59,7 +217,7 @@ fn sort_column(data: &DataBlock, sort_row_offset: usize) -> &Column { data.get_by_offset(sort_row_offset).as_column().unwrap() } -impl InputBoundStream { +impl BoundedInputStream { fn pull(&mut self) -> Result { if self.data.is_some() { return Ok(false); @@ -79,10 +237,6 @@ impl InputBoundStream { } fn take_next_bounded_block(&mut self) -> Option { - let Some(bound) = &self.bound else { - return self.data.take(); - }; - let meta = self .data .as_ref()? @@ -90,8 +244,11 @@ impl InputBoundStream { .and_then(SortBound::downcast_ref_from) .expect("require a SortBound"); - if meta.bound.as_ref() == Some(bound) { - self.data.take() + if meta.bound == self.bound { + self.data.take().map(|mut data| { + let _ = data.take_meta(); + data + }) } else { None } From 9cbe20750f767b7ba4b971eb3159223db6e7c2d1 Mon Sep 17 00:00:00 2001 From: coldWater Date: Wed, 9 Jul 2025 11:11:58 +0800 Subject: [PATCH 51/61] build_bounded_merge_sort --- .../src/pipelines/builders/builder_sort.rs | 49 ++++++++-- .../processors/transforms/sort/mod.rs | 5 ++ .../transforms/sort/sort_builder.rs | 90 +++++++++++++++++-- .../transforms/sort/sort_merge_stream.rs | 2 +- 4 files changed, 128 insertions(+), 18 deletions(-) diff --git a/src/query/service/src/pipelines/builders/builder_sort.rs b/src/query/service/src/pipelines/builders/builder_sort.rs index e8340d64f13bb..5e808ecf8a727 100644 --- a/src/query/service/src/pipelines/builders/builder_sort.rs +++ b/src/query/service/src/pipelines/builders/builder_sort.rs @@ -19,7 +19,11 @@ use databend_common_exception::Result; use databend_common_expression::DataSchemaRef; use databend_common_expression::LimitType; use databend_common_expression::SortColumnDescription; +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::processors::add_k_way_merge_sort; use databend_common_pipeline_transforms::processors::sort::utils::add_order_field; @@ -38,6 +42,7 @@ use databend_storages_common_cache::TempDirManager; use crate::pipelines::memory_settings::MemorySettingsExt; use crate::pipelines::processors::transforms::add_range_shuffle_route; +use crate::pipelines::processors::transforms::BoundedMergeSortBuilder; use crate::pipelines::processors::transforms::SortInjector; use crate::pipelines::processors::transforms::SortRangeExchange; use crate::pipelines::processors::transforms::TransformLimit; @@ -156,7 +161,7 @@ impl PipelineBuilder { self.build_pipeline(&sort.input)?; } - self.main_pipeline.resize(1, false) + builder.build_bounded_merge_sort(&mut self.main_pipeline) } SortStep::Route => self.main_pipeline.resize(1, false), } @@ -253,7 +258,7 @@ impl SortPipelineBuilder { let memory_settings = MemorySettings::from_sort_settings(&self.ctx)?; let enable_loser_tree = settings.get_enable_loser_tree_merge_sort()?; - let builder = TransformSortBuilder::create( + let builder = TransformSortBuilder::new( self.schema.clone(), self.sort_desc.clone(), max_block_size, @@ -325,7 +330,7 @@ impl SortPipelineBuilder { let memory_settings = MemorySettings::from_sort_settings(&self.ctx)?; let enable_loser_tree = settings.get_enable_loser_tree_merge_sort()?; - let builder = TransformSortBuilder::create( + let builder = TransformSortBuilder::new( self.schema.clone(), self.sort_desc.clone(), max_block_size, @@ -377,12 +382,10 @@ impl SortPipelineBuilder { // Merge sort let need_multi_merge = pipeline.output_len() > 1; let output_order_col = need_multi_merge || !self.remove_order_col_at_last; - debug_assert!(if order_col_generated { + debug_assert!( // If `order_col_generated`, it means this transform is the last processor in the distributed sort pipeline. - !output_order_col - } else { - true - }); + !order_col_generated || !output_order_col + ); let memory_settings = MemorySettings::from_sort_settings(&self.ctx)?; let sort_merge_output_schema = match output_order_col { @@ -414,7 +417,7 @@ impl SortPipelineBuilder { }; pipeline.add_transform(|input, output| { - let builder = TransformSortBuilder::create( + let builder = TransformSortBuilder::new( sort_merge_output_schema.clone(), self.sort_desc.clone(), self.block_size, @@ -476,4 +479,32 @@ impl SortPipelineBuilder { pub fn exchange_injector(&self) -> Arc { Arc::new(SortInjector {}) } + + pub fn build_bounded_merge_sort(self, pipeline: &mut Pipeline) -> Result<()> { + let inputs_port: Vec<_> = (0..pipeline.output_len()) + .map(|_| InputPort::create()) + .collect(); + let output_port = OutputPort::create(); + + let processor = ProcessorPtr::create( + BoundedMergeSortBuilder::new( + inputs_port.clone(), + output_port.clone(), + self.schema.clone(), + self.sort_desc.clone(), + self.block_size, + self.limit, + self.remove_order_col_at_last, + self.enable_loser_tree, + ) + .build()?, + ); + + pipeline.add_pipe(Pipe::create(inputs_port.len(), 1, vec![PipeItem::create( + processor, + inputs_port, + vec![output_port], + )])); + Ok(()) + } } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs index c406de5bd1542..a8947eb399d19 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs @@ -39,9 +39,14 @@ mod sort_route; mod sort_shuffle; mod sort_spill; +pub use merge_sort::*; pub use sort_builder::*; +pub use sort_collect::*; +pub use sort_combine::*; pub use sort_exchange::*; pub use sort_exchange_injector::*; +pub use sort_merge_stream::*; +pub use sort_restore::*; pub use sort_route::*; pub use sort_shuffle::*; diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs index 5c5f560e5be04..3a68dd52e2c6b 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs @@ -34,13 +34,7 @@ use databend_common_pipeline_transforms::sort::RowsTypeVisitor; use databend_common_pipeline_transforms::AccumulatingTransformer; use databend_common_pipeline_transforms::MemorySettings; -use super::merge_sort::TransformSort; -use super::sort_collect::TransformSortCollect; -use super::sort_combine::TransformSortCombine; -use super::sort_restore::TransformSortRestore; -use super::sort_shuffle::SortSampleState; -use super::sort_shuffle::TransformSortBoundBroadcast; -use super::Base; +use super::*; use crate::sessions::QueryContext; use crate::spillers::Spiller; @@ -65,7 +59,7 @@ pub struct TransformSortBuilder { } impl TransformSortBuilder { - pub fn create( + pub fn new( schema: DataSchemaRef, sort_desc: Arc<[SortColumnDescription]>, block_size: usize, @@ -357,3 +351,83 @@ impl RowsTypeVisitor for Build<'_> { } } } + +pub struct BoundedMergeSortBuilder { + inputs: Vec>, + output: Arc, + schema: DataSchemaRef, + sort_desc: Arc<[SortColumnDescription]>, + block_size: usize, + limit: Option, + remove_order_col: bool, + enable_loser_tree: bool, +} + +impl BoundedMergeSortBuilder { + pub fn new( + inputs: Vec>, + output: Arc, + schema: DataSchemaRef, + sort_desc: Arc<[SortColumnDescription]>, + block_size: usize, + limit: Option, + remove_order_col: bool, + enable_loser_tree: bool, + ) -> Self { + Self { + inputs, + output, + schema, + sort_desc, + block_size, + limit, + remove_order_col, + enable_loser_tree, + } + } + + pub fn build(mut self) -> Result> { + select_row_type(&mut self) + } +} + +impl RowsTypeVisitor for BoundedMergeSortBuilder { + type Result = Result>; + + fn schema(&self) -> DataSchemaRef { + self.schema.clone() + } + + fn sort_desc(&self) -> &[SortColumnDescription] { + &self.sort_desc + } + + fn visit_type(&mut self) -> Self::Result + where + R: Rows + 'static, + C: RowConverter + Send + 'static, + { + match self.enable_loser_tree { + true => Ok(Box::new( + BoundedMultiSortMergeProcessor::>::new( + self.inputs.clone(), + self.output.clone(), + self.schema.clone(), + self.block_size, + self.limit, + self.remove_order_col, + )?, + )), + false => Ok(Box::new( + BoundedMultiSortMergeProcessor::>::new( + self.inputs.clone(), + self.output.clone(), + self.schema.clone(), + self.block_size, + self.limit, + self.remove_order_col, + )?, + )), + } + } +} diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_merge_stream.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_merge_stream.rs index cd9294c9096e8..46743a09d8f58 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_merge_stream.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_merge_stream.rs @@ -53,7 +53,7 @@ where A: SortAlgorithm impl BoundedMultiSortMergeProcessor where A: SortAlgorithm { - pub fn create( + pub fn new( inputs: Vec>, output: Arc, schema: DataSchemaRef, From 0944c51f78dce1cdc11d6f61967f8feec2938e04 Mon Sep 17 00:00:00 2001 From: coldWater Date: Wed, 9 Jul 2025 14:07:53 +0800 Subject: [PATCH 52/61] bound_index --- Cargo.lock | 1 - src/query/service/Cargo.toml | 1 - .../processors/transforms/sort/mod.rs | 10 ++++-- .../transforms/sort/sort_exchange_injector.rs | 20 ++--------- .../transforms/sort/sort_merge_stream.rs | 35 +++++++------------ .../transforms/sort/sort_restore.rs | 4 +-- .../processors/transforms/sort/sort_spill.rs | 18 ++++++---- 7 files changed, 38 insertions(+), 51 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 30a98291f58ce..78c94a905684e 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -5366,7 +5366,6 @@ dependencies = [ "toml 0.8.22", "tonic", "tower 0.5.2", - "twox-hash 1.6.3", "typetag", "url", "uuid", diff --git a/src/query/service/Cargo.toml b/src/query/service/Cargo.toml index fa66832451efc..8f264f45a2e09 100644 --- a/src/query/service/Cargo.toml +++ b/src/query/service/Cargo.toml @@ -171,7 +171,6 @@ tokio = { workspace = true } tokio-stream = { workspace = true, features = ["net"] } toml = { workspace = true, features = ["parse"] } tonic = { workspace = true } -twox-hash = { workspace = true } typetag = { workspace = true } url = { workspace = true } uuid = { workspace = true } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs index a8947eb399d19..8e888c34c2b83 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs @@ -20,7 +20,6 @@ use databend_common_expression::BlockMetaInfo; use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::DataBlock; use databend_common_expression::DataSchemaRef; -use databend_common_expression::Scalar; use databend_common_pipeline_transforms::SortSpillParams; use sort_spill::SpillableBlock; @@ -107,7 +106,14 @@ impl BlockMetaInfo for SortExchangeMeta { #[derive(Debug, Clone, PartialEq, Eq, serde::Serialize, serde::Deserialize)] pub struct SortBound { - bound: Option, + bound_index: u32, +} + +impl SortBound { + fn create(bound_index: u32) -> Box { + debug_assert!(bound_index != u32::MAX); + SortBound { bound_index }.boxed() + } } #[typetag::serde(name = "sort_bound")] diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs index fe4199b837bef..496dcae3b7bdc 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs @@ -12,17 +12,13 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::hash::Hash; -use std::hash::Hasher; use std::sync::Arc; use databend_common_exception::Result; use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::DataBlock; -use databend_common_expression::Scalar; use databend_common_pipeline_core::Pipeline; use databend_common_settings::FlightCompression; -use twox_hash::XxHash64; use crate::pipelines::processors::transforms::SortBound; use crate::servers::flight::v1::exchange::DataExchange; @@ -88,7 +84,7 @@ impl ExchangeInjector for SortInjector { } pub struct SortBoundScatter { - partitions: u64, + partitions: u32, } impl FlightScatter for SortBoundScatter { @@ -101,23 +97,13 @@ impl FlightScatter for SortBoundScatter { } } -pub(super) fn bound_scatter(data_block: DataBlock, n: u64) -> Result> { +pub(super) fn bound_scatter(data_block: DataBlock, n: u32) -> Result> { let meta = data_block .get_meta() .and_then(SortBound::downcast_ref_from) .unwrap(); - let bound = match &meta.bound { - Some(bound) => { - debug_assert!(!bound.is_null()); - bound - } - None => &Scalar::Null, - }; - - let mut hasher = XxHash64::default(); - bound.hash(&mut hasher); - let index = hasher.finish() % n; + let index = meta.bound_index % n; Ok(std::iter::repeat_n(DataBlock::empty(), index as _) .chain(Some(data_block)) diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_merge_stream.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_merge_stream.rs index 46743a09d8f58..5a7a08214e3f5 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_merge_stream.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_merge_stream.rs @@ -13,7 +13,6 @@ // limitations under the License. use std::any::Any; -use std::cmp::Ordering; use std::marker::PhantomData; use std::sync::Arc; @@ -22,7 +21,6 @@ use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::Column; use databend_common_expression::DataBlock; use databend_common_expression::DataSchemaRef; -use databend_common_expression::Scalar; use databend_common_pipeline_core::processors::Event; use databend_common_pipeline_core::processors::InputPort; use databend_common_pipeline_core::processors::OutputPort; @@ -46,7 +44,7 @@ where A: SortAlgorithm limit: Option, output_data: Option, - bound: Option, + bound_index: u32, inner: std::result::Result>, Vec>>, } @@ -67,7 +65,7 @@ where A: SortAlgorithm data: None, input: input.clone(), remove_order_col, - bound: None, + bound_index: None, sort_row_offset: schema.fields().len() - 1, _r: PhantomData, }) @@ -80,7 +78,7 @@ where A: SortAlgorithm block_size, limit, output_data: None, - bound: None, + bound_index: u32::MAX, inner: Err(streams), }) } @@ -119,11 +117,7 @@ where A: SortAlgorithm + 'static fn process(&mut self) -> Result<()> { if let Some(block) = self.inner.as_mut().ok().unwrap().next_block()? { - let meta = SortBound { - bound: self.bound.clone(), - } - .boxed(); - self.output_data = Some(block.add_meta(Some(meta))?); + self.output_data = Some(block.add_meta(Some(SortBound::create(self.bound_index)))?); }; Ok(()) } @@ -158,20 +152,17 @@ where A: SortAlgorithm + 'static .get_meta() .and_then(SortBound::downcast_ref_from) .expect("require a SortBound"); - bounds.push(&meta.bound) + bounds.push(meta.bound_index) } - self.bound = match bounds.iter().min_by(|a, b| match (a, b) { - (None, None) => Ordering::Equal, - (None, Some(_)) => Ordering::Greater, - (Some(_), None) => Ordering::Less, - (Some(a), Some(b)) => A::Rows::scalar_as_item(a).cmp(&A::Rows::scalar_as_item(b)), - }) { - Some(bound) => (*bound).clone(), + let bound_index = match bounds.iter().min() { + Some(index) => *index, None => return Ok(Event::Finished), }; + assert!(self.bound_index != u32::MAX || bound_index > self.bound_index); + self.bound_index = bound_index; for stream in streams.iter_mut() { - stream.bound = self.bound.clone(); + stream.bound_index = Some(self.bound_index); } let streams = std::mem::take(streams); @@ -189,7 +180,7 @@ struct BoundedInputStream { data: Option, input: Arc, remove_order_col: bool, - bound: Option, + bound_index: Option, sort_row_offset: usize, _r: PhantomData, } @@ -244,9 +235,9 @@ impl BoundedInputStream { .and_then(SortBound::downcast_ref_from) .expect("require a SortBound"); - if meta.bound == self.bound { + if meta.bound_index == self.bound_index.unwrap() { self.data.take().map(|mut data| { - let _ = data.take_meta(); + data.take_meta().unwrap(); data }) } else { diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_restore.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_restore.rs index c6599601e0bbc..8c27d47c08576 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_restore.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_restore.rs @@ -115,11 +115,11 @@ where let OutputData { block, - bound, + bound: (bound_index, _), finish, } = spill_sort.on_restore().await?; if let Some(block) = block { - let mut block = block.add_meta(Some(SortBound { bound }.boxed()))?; + let mut block = block.add_meta(Some(SortBound::create(bound_index)))?; if self.remove_order_col { block.pop_columns(1); } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs index 6faf66265a2f2..9bbfd97e388b2 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs @@ -68,6 +68,7 @@ struct StepSort { /// Each boundary represents a cutoff point where data less than or equal to it belongs to one partition. bounds: Bounds, cur_bound: Option, + bound_index: i32, subsequent: Vec>>, current: Vec>>, @@ -116,6 +117,7 @@ where A: SortAlgorithm params, bounds, cur_bound: None, + bound_index: -1, subsequent, current: vec![], output_merger: None, @@ -172,7 +174,7 @@ where A: SortAlgorithm sort.merge_current(&self.base).await?; Ok(OutputData { block: None, - bound: None, + bound: (u32::MAX, None), finish: false, }) } else { @@ -303,6 +305,7 @@ impl StepCollect { Ok(StepSort { bounds, cur_bound: None, + bound_index: -1, subsequent: std::mem::take(&mut self.streams), current: vec![], output_merger: None, @@ -313,7 +316,7 @@ impl StepCollect { pub struct OutputData { pub block: Option, - pub bound: Option, + pub bound: (u32, Option), pub finish: bool, } @@ -323,6 +326,7 @@ impl StepSort { Some(bound) => self.cur_bound = Some(bound), None => self.cur_bound = None, } + self.bound_index += 1; } async fn merge_current(&mut self, base: &Base) -> Result<()> { @@ -372,7 +376,8 @@ impl StepSort { let mut s = self.current.pop().unwrap(); s.restore_first().await?; let block = Some(s.take_next_bounded_block()); - let bound = s.bound.clone(); + assert!(self.bound_index >= 0); + let bound = (self.bound_index as _, s.bound.clone()); if !s.is_empty() { if s.should_include_first() { @@ -415,7 +420,7 @@ impl StepSort { return Ok(OutputData { block: None, - bound: None, + bound: (u32::MAX, None), finish: self.subsequent.is_empty(), }); }; @@ -423,7 +428,8 @@ impl StepSort { let mut sorted = base.new_stream([base.new_block(data)].into(), self.cur_bound.clone()); let (block, bound) = if sorted.should_include_first() { let block = Some(sorted.take_next_bounded_block()); - let bound = sorted.bound.clone(); + debug_assert!(self.bound_index >= 0); + let bound = (self.bound_index as _, sorted.bound.clone()); if sorted.is_empty() { return Ok(OutputData { block, @@ -433,7 +439,7 @@ impl StepSort { } (block, bound) } else { - (None, None) + (None, (u32::MAX, None)) }; while let Some(data) = merger.async_next_block().await? { From 43a11e968e255b2ac0b618d4ef2670a7af4f5f75 Mon Sep 17 00:00:00 2001 From: coldWater Date: Wed, 9 Jul 2025 16:29:47 +0800 Subject: [PATCH 53/61] SortBoundEdge --- src/query/expression/src/block.rs | 20 +++- .../processors/transforms/sort/mod.rs | 5 +- .../transforms/sort/sort_merge_stream.rs | 3 +- .../transforms/sort/sort_restore.rs | 91 ++++++++++++++++++- .../processors/transforms/sort/sort_route.rs | 51 +++++++---- .../processors/transforms/sort/sort_spill.rs | 25 +++-- 6 files changed, 153 insertions(+), 42 deletions(-) diff --git a/src/query/expression/src/block.rs b/src/query/expression/src/block.rs index eb0cddbdcbeca..830cc3e89a0c5 100644 --- a/src/query/expression/src/block.rs +++ b/src/query/expression/src/block.rs @@ -340,6 +340,11 @@ pub trait BlockMetaInfoDowncast: Sized + BlockMetaInfo { Err(boxed) } } + + fn downcast_mut_from(boxed: &mut BlockMetaInfoPtr) -> Option<&mut Self> { + let boxed = boxed.as_mut() as &mut dyn Any; + boxed.downcast_mut() + } } impl BlockMetaInfoDowncast for T {} @@ -450,11 +455,6 @@ impl DataBlock { DataBlock::new_with_meta(vec![], 0, Some(meta)) } - #[inline] - pub fn take_meta(&mut self) -> Option { - self.meta.take() - } - #[inline] pub fn columns(&self) -> &[BlockEntry] { &self.entries @@ -721,6 +721,16 @@ impl DataBlock { }) } + #[inline] + pub fn take_meta(&mut self) -> Option { + self.meta.take() + } + + #[inline] + pub fn mut_meta(&mut self) -> Option<&mut BlockMetaInfoPtr> { + self.meta.as_mut() + } + #[inline] pub fn replace_meta(&mut self, meta: BlockMetaInfoPtr) { self.meta.replace(meta); diff --git a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs index 8e888c34c2b83..dee24068e6986 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs @@ -107,12 +107,13 @@ impl BlockMetaInfo for SortExchangeMeta { #[derive(Debug, Clone, PartialEq, Eq, serde::Serialize, serde::Deserialize)] pub struct SortBound { bound_index: u32, + more: bool, } impl SortBound { - fn create(bound_index: u32) -> Box { + fn create(bound_index: u32, more: bool) -> Box { debug_assert!(bound_index != u32::MAX); - SortBound { bound_index }.boxed() + SortBound { bound_index, more }.boxed() } } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_merge_stream.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_merge_stream.rs index 5a7a08214e3f5..e6c01727adcd5 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_merge_stream.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_merge_stream.rs @@ -117,7 +117,8 @@ where A: SortAlgorithm + 'static fn process(&mut self) -> Result<()> { if let Some(block) = self.inner.as_mut().ok().unwrap().next_block()? { - self.output_data = Some(block.add_meta(Some(SortBound::create(self.bound_index)))?); + self.output_data = + Some(block.add_meta(Some(SortBound::create(self.bound_index, todo!())))?); }; Ok(()) } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_restore.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_restore.rs index 8c27d47c08576..3d63c9e076186 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_restore.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_restore.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 databend_common_exception::Result; @@ -20,6 +21,7 @@ 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_transforms::processors::sort::algorithm::SortAlgorithm; use databend_common_pipeline_transforms::HookTransform; use databend_common_pipeline_transforms::HookTransformer; @@ -119,7 +121,7 @@ where finish, } = spill_sort.on_restore().await?; if let Some(block) = block { - let mut block = block.add_meta(Some(SortBound::create(bound_index)))?; + let mut block = block.add_meta(Some(SortBound::create(bound_index, true)))?; if self.remove_order_col { block.pop_columns(1); } @@ -131,3 +133,90 @@ where Ok(()) } } + +struct SortBoundEdge { + input: Arc, + output: Arc, + data: Option, +} + +impl SortBoundEdge { + pub fn new(input: Arc, output: Arc) -> Self { + Self { + input, + output, + data: None, + } + } +} + +impl Processor for SortBoundEdge { + fn name(&self) -> String { + String::from("SortBoundEdgeTransform") + } + + 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.data.is_none() { + if self.input.is_finished() { + self.output.finish(); + return Ok(Event::Finished); + } + let Some(block) = self.input.pull_data().transpose()? else { + self.input.set_need_data(); + return Ok(Event::NeedData); + }; + self.data = Some(block); + } + + if self.input.is_finished() { + let mut block = self.data.take().unwrap(); + let mut meta = block + .take_meta() + .and_then(SortBound::downcast_from) + .expect("require a SortBound"); + meta.more = false; + self.output + .push_data(Ok(block.add_meta(Some(meta.boxed()))?)); + self.output.finish(); + return Ok(Event::Finished); + } + + let Some(incoming) = self.input.pull_data().transpose()? else { + self.input.set_need_data(); + return Ok(Event::NeedData); + }; + + let incoming_index = incoming + .get_meta() + .and_then(SortBound::downcast_ref_from) + .expect("require a SortBound") + .bound_index; + + let mut output = self.data.replace(incoming).unwrap(); + let mut output_meta = output + .mut_meta() + .and_then(SortBound::downcast_mut_from) + .expect("require a SortBound"); + + if output_meta.bound_index != incoming_index { + output_meta.more = false; + } + + self.output.push_data(Ok(output)); + Ok(Event::NeedConsume) + } +} diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_route.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_route.rs index a9592f4cd405a..251bf6cefc89c 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_route.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_route.rs @@ -16,6 +16,8 @@ use std::any::Any; use std::sync::Arc; 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::InputPort; use databend_common_pipeline_core::processors::OutputPort; @@ -25,42 +27,51 @@ use databend_common_pipeline_core::Pipe; use databend_common_pipeline_core::PipeItem; use databend_common_pipeline_core::Pipeline; +use crate::pipelines::processors::transforms::SortBound; + struct TransformSortRoute { inputs: Vec>, output: Arc, - cur_input: usize, + + input_data: Vec>, + cur_index: u32, } impl TransformSortRoute { fn new(inputs: Vec>, output: Arc) -> Self { Self { + input_data: vec![None; inputs.len()], + cur_index: 0, inputs, output, - cur_input: 0, } } - fn process(&mut self) -> Result<()> { - for (i, input) in self.inputs.iter().enumerate() { - if i != self.cur_input { - if !input.is_finished() && !input.has_data() { - input.set_need_data(); - } - continue; - } - - if input.is_finished() { - self.cur_input = i + 1; - continue; - } + fn process(&mut self) -> Result { + for input in &self.inputs { + input.set_need_data(); + } - match input.pull_data() { - Some(data) => self.output.push_data(data), - None => input.set_need_data(), - } + for (input, data) in self.inputs.iter().zip(self.input_data.iter_mut()) { + if data.is_none() { + let Some(mut block) = input.pull_data().transpose()? else { + continue; + }; + + let bound_index = block + .take_meta() + .and_then(SortBound::downcast_from) + .expect("require a SortBound") + .bound_index; + if bound_index == self.cur_index { + self.output.push_data(Ok(block)); + return Ok(Event::NeedConsume); + } + *data = Some((block, bound_index)); + }; } - Ok(()) + Ok(Event::NeedData) } } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs index 9bbfd97e388b2..bad2fb89f476c 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs @@ -170,16 +170,16 @@ where A: SortAlgorithm sort.choice_streams_by_bound(); } - if sort.current.len() > sort.params.num_merge { - sort.merge_current(&self.base).await?; - Ok(OutputData { - block: None, - bound: (u32::MAX, None), - finish: false, - }) - } else { - sort.restore_and_output(&self.base).await + if sort.current.len() <= sort.params.num_merge { + return sort.restore_and_output(&self.base).await; } + + sort.merge_current(&self.base).await?; + Ok(OutputData { + block: None, + bound: (u32::MAX, None), + finish: false, + }) } pub fn max_rows(&self) -> usize { @@ -380,10 +380,9 @@ impl StepSort { let bound = (self.bound_index as _, s.bound.clone()); if !s.is_empty() { - if s.should_include_first() { - self.current.push(s); - } else { - self.subsequent.push(s); + match s.should_include_first() { + true => self.current.push(s), + false => self.subsequent.push(s), } return Ok(OutputData { block, From a4fc5855e320ac2ea0c132def95ea0f71eeb452e Mon Sep 17 00:00:00 2001 From: coldWater Date: Thu, 10 Jul 2025 15:19:02 +0800 Subject: [PATCH 54/61] route --- src/query/expression/src/block.rs | 2 +- .../processors/transforms/sort/mod.rs | 14 +-- .../transforms/sort/sort_exchange_injector.rs | 16 ++-- .../transforms/sort/sort_merge_stream.rs | 92 +++++++++++++------ .../transforms/sort/sort_restore.rs | 14 ++- .../processors/transforms/sort/sort_route.rs | 38 +++++--- 6 files changed, 107 insertions(+), 69 deletions(-) diff --git a/src/query/expression/src/block.rs b/src/query/expression/src/block.rs index 830cc3e89a0c5..83697beea302a 100644 --- a/src/query/expression/src/block.rs +++ b/src/query/expression/src/block.rs @@ -341,7 +341,7 @@ pub trait BlockMetaInfoDowncast: Sized + BlockMetaInfo { } } - fn downcast_mut_from(boxed: &mut BlockMetaInfoPtr) -> Option<&mut Self> { + fn downcast_mut(boxed: &mut BlockMetaInfoPtr) -> Option<&mut Self> { let boxed = boxed.as_mut() as &mut dyn Any; boxed.downcast_mut() } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs index dee24068e6986..0e4d219c2ee4b 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs @@ -104,16 +104,16 @@ impl BlockMetaInfo for SortExchangeMeta { } } -#[derive(Debug, Clone, PartialEq, Eq, serde::Serialize, serde::Deserialize)] +#[derive(Debug, Clone, Copy, PartialEq, Eq, serde::Serialize, serde::Deserialize)] pub struct SortBound { - bound_index: u32, - more: bool, + index: u32, + next: Option, } impl SortBound { - fn create(bound_index: u32, more: bool) -> Box { - debug_assert!(bound_index != u32::MAX); - SortBound { bound_index, more }.boxed() + fn create(index: u32, next: Option) -> Box { + debug_assert!(index != u32::MAX); + SortBound { index, next }.boxed() } } @@ -124,6 +124,6 @@ impl BlockMetaInfo for SortBound { } fn clone_self(&self) -> Box { - Box::new(self.clone()) + Box::new(*self) } } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs index 496dcae3b7bdc..50bf8a12367b2 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs @@ -42,7 +42,7 @@ impl ExchangeInjector for SortInjector { DataExchange::Merge(_) | DataExchange::Broadcast(_) => unreachable!(), DataExchange::ShuffleDataExchange(exchange) => { Ok(Arc::new(Box::new(SortBoundScatter { - partitions: exchange.destination_ids.len() as _, + partitions: exchange.destination_ids.len(), }))) } } @@ -84,7 +84,7 @@ impl ExchangeInjector for SortInjector { } pub struct SortBoundScatter { - partitions: u32, + partitions: usize, } impl FlightScatter for SortBoundScatter { @@ -97,17 +97,17 @@ impl FlightScatter for SortBoundScatter { } } -pub(super) fn bound_scatter(data_block: DataBlock, n: u32) -> Result> { - let meta = data_block +pub(super) fn bound_scatter(data_block: DataBlock, n: usize) -> Result> { + let meta = *data_block .get_meta() .and_then(SortBound::downcast_ref_from) .unwrap(); - let index = meta.bound_index % n; + let empty = data_block.slice(0..0); + let mut result = vec![empty; n]; + result[meta.index as usize % n] = data_block; - Ok(std::iter::repeat_n(DataBlock::empty(), index as _) - .chain(Some(data_block)) - .collect()) + Ok(result) } // pub struct TransformExchangeSortSerializer { diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_merge_stream.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_merge_stream.rs index e6c01727adcd5..aa406ec06cb54 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_merge_stream.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_merge_stream.rs @@ -65,7 +65,7 @@ where A: SortAlgorithm data: None, input: input.clone(), remove_order_col, - bound_index: None, + bound: None, sort_row_offset: schema.fields().len() - 1, _r: PhantomData, }) @@ -78,7 +78,7 @@ where A: SortAlgorithm block_size, limit, output_data: None, - bound_index: u32::MAX, + bound_index: 0, inner: Err(streams), }) } @@ -116,9 +116,12 @@ where A: SortAlgorithm + 'static } fn process(&mut self) -> Result<()> { - if let Some(block) = self.inner.as_mut().ok().unwrap().next_block()? { - self.output_data = - Some(block.add_meta(Some(SortBound::create(self.bound_index, todo!())))?); + let merger = self.inner.as_mut().ok().unwrap(); + if let Some(block) = merger.next_block()? { + self.output_data = Some(block.add_meta(Some(SortBound::create( + self.bound_index, + (!merger.is_finished()).then_some(self.bound_index), + )))?); }; Ok(()) } @@ -134,6 +137,7 @@ where A: SortAlgorithm + 'static if !merger.is_finished() { return Ok(Event::Sync); } + self.bound_index += 1; let merger = std::mem::replace(inner, Err(vec![])).ok().unwrap(); self.inner = Err(merger.streams()); self.inner.as_mut().err().unwrap() @@ -141,35 +145,44 @@ where A: SortAlgorithm + 'static Err(streams) => streams, }; - let mut bounds = Vec::with_capacity(streams.len()); - for stream in streams.iter_mut() { - if stream.pull()? { - return Ok(Event::NeedData); - } - let Some(data) = &stream.data else { - continue; - }; - let meta = data - .get_meta() - .and_then(SortBound::downcast_ref_from) - .expect("require a SortBound"); - bounds.push(meta.bound_index) + if streams.iter().all(|stream| stream.input.is_finished()) { + return Ok(Event::Finished); } - let bound_index = match bounds.iter().min() { - Some(index) => *index, - None => return Ok(Event::Finished), - }; - assert!(self.bound_index != u32::MAX || bound_index > self.bound_index); - self.bound_index = bound_index; + // { + // for stream in streams.iter_mut() { + // stream.pull()?; + // } + + // if streams + // .iter_mut() + // .map(|stream| stream.pull()) + // .try_fold(false, |acc, pending| acc || pending?)? + // { + // return Ok(Event::NeedData); + // } + + // if bounds.is_empty() { + // return Ok(Event::Finished); + // } + + // if bounds.iter().all(|meta| meta.index != self.bound_index) { + // let meta = match bounds.iter().min_by_key(|meta| meta.index) { + // Some(index) => *index, + // None => return Ok(Event::Finished), + // }; + // assert!(meta.index > self.bound_index); + // self.bound_index = meta.index; + // } + // } + for stream in streams.iter_mut() { - stream.bound_index = Some(self.bound_index); + stream.update_bound_index(self.bound_index); } - let streams = std::mem::take(streams); self.inner = Ok(Merger::create( self.schema.clone(), - streams, + std::mem::take(streams), self.block_size, self.limit, )); @@ -181,20 +194,29 @@ struct BoundedInputStream { data: Option, input: Arc, remove_order_col: bool, - bound_index: Option, sort_row_offset: usize, + bound: Option, _r: PhantomData, } +#[derive(Debug, Clone, Copy)] +struct Bound { + bound_index: u32, + more: bool, +} + impl SortedStream for BoundedInputStream { fn next(&mut self) -> Result<(Option<(DataBlock, Column)>, bool)> { - if self.pull()? { + if self.bound.unwrap().more && self.pull()? { return Ok((None, true)); } match self.take_next_bounded_block() { None => Ok((None, false)), Some(mut block) => { + if block.is_empty() { + return Ok((None, true)); + } let col = sort_column(&block, self.sort_row_offset).clone(); if self.remove_order_col { block.remove_column(self.sort_row_offset); @@ -236,13 +258,23 @@ impl BoundedInputStream { .and_then(SortBound::downcast_ref_from) .expect("require a SortBound"); - if meta.bound_index == self.bound_index.unwrap() { + let bound = self.bound.as_mut().unwrap(); + if meta.index == bound.bound_index { + bound.more = meta.next.is_some_and(|next| next == meta.index); self.data.take().map(|mut data| { data.take_meta().unwrap(); data }) } else { + assert!(meta.index > bound.bound_index); None } } + + fn update_bound_index(&mut self, bound_index: u32) { + self.bound = Some(Bound { + bound_index, + more: true, + }); + } } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_restore.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_restore.rs index 3d63c9e076186..c46d35a8e9c3b 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_restore.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_restore.rs @@ -121,7 +121,7 @@ where finish, } = spill_sort.on_restore().await?; if let Some(block) = block { - let mut block = block.add_meta(Some(SortBound::create(bound_index, true)))?; + let mut block = block.add_meta(Some(SortBound::create(bound_index, None)))?; if self.remove_order_col { block.pop_columns(1); } @@ -188,7 +188,7 @@ impl Processor for SortBoundEdge { .take_meta() .and_then(SortBound::downcast_from) .expect("require a SortBound"); - meta.more = false; + meta.next = None; self.output .push_data(Ok(block.add_meta(Some(meta.boxed()))?)); self.output.finish(); @@ -204,17 +204,15 @@ impl Processor for SortBoundEdge { .get_meta() .and_then(SortBound::downcast_ref_from) .expect("require a SortBound") - .bound_index; + .index; let mut output = self.data.replace(incoming).unwrap(); - let mut output_meta = output + let output_meta = output .mut_meta() - .and_then(SortBound::downcast_mut_from) + .and_then(SortBound::downcast_mut) .expect("require a SortBound"); - if output_meta.bound_index != incoming_index { - output_meta.more = false; - } + output_meta.next = Some(incoming_index); self.output.push_data(Ok(output)); Ok(Event::NeedConsume) diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_route.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_route.rs index 251bf6cefc89c..4963025f48753 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_route.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_route.rs @@ -33,7 +33,7 @@ struct TransformSortRoute { inputs: Vec>, output: Arc, - input_data: Vec>, + input_data: Vec>, cur_index: u32, } @@ -53,22 +53,30 @@ impl TransformSortRoute { } for (input, data) in self.inputs.iter().zip(self.input_data.iter_mut()) { - if data.is_none() { - let Some(mut block) = input.pull_data().transpose()? else { - continue; - }; - - let bound_index = block - .take_meta() - .and_then(SortBound::downcast_from) - .expect("require a SortBound") - .bound_index; - if bound_index == self.cur_index { - self.output.push_data(Ok(block)); - return Ok(Event::NeedConsume); + let meta = match data { + Some((_, meta)) => *meta, + None => { + let Some(mut block) = input.pull_data().transpose()? else { + continue; + }; + + let meta = block + .take_meta() + .and_then(SortBound::downcast_from) + .expect("require a SortBound"); + + data.insert((block, meta)).1 } - *data = Some((block, bound_index)); }; + + if meta.index == self.cur_index { + let (block, meta) = data.take().unwrap(); + self.output.push_data(Ok(block)); + if meta.next.is_none() { + self.cur_index += 1; + } + return Ok(Event::NeedConsume); + } } Ok(Event::NeedData) From 75ac0062ec3303126ca30ab485dc04c3453d559a Mon Sep 17 00:00:00 2001 From: coldWater Date: Thu, 10 Jul 2025 17:51:55 +0800 Subject: [PATCH 55/61] clean up --- .../src/pipelines/builders/builder_sort.rs | 151 ++-------- .../processors/transforms/sort/mod.rs | 12 - .../transforms/sort/sort_builder.rs | 262 ++++++++---------- .../transforms/sort/sort_combine.rs | 80 ------ .../transforms/sort/sort_exchange.rs | 55 ---- .../transforms/sort/sort_exchange_injector.rs | 184 +----------- .../transforms/sort/sort_restore.rs | 2 +- .../processors/transforms/sort/sort_route.rs | 28 +- src/query/settings/src/settings_default.rs | 4 +- .../settings/src/settings_getter_setter.rs | 4 +- .../executor/physical_plans/physical_sort.rs | 2 +- 11 files changed, 147 insertions(+), 637 deletions(-) delete mode 100644 src/query/service/src/pipelines/processors/transforms/sort/sort_combine.rs delete mode 100644 src/query/service/src/pipelines/processors/transforms/sort/sort_exchange.rs diff --git a/src/query/service/src/pipelines/builders/builder_sort.rs b/src/query/service/src/pipelines/builders/builder_sort.rs index 5e808ecf8a727..793f20a60a110 100644 --- a/src/query/service/src/pipelines/builders/builder_sort.rs +++ b/src/query/service/src/pipelines/builders/builder_sort.rs @@ -41,14 +41,8 @@ use databend_common_storages_fuse::TableContext; use databend_storages_common_cache::TempDirManager; use crate::pipelines::memory_settings::MemorySettingsExt; -use crate::pipelines::processors::transforms::add_range_shuffle_route; -use crate::pipelines::processors::transforms::BoundedMergeSortBuilder; -use crate::pipelines::processors::transforms::SortInjector; -use crate::pipelines::processors::transforms::SortRangeExchange; -use crate::pipelines::processors::transforms::TransformLimit; use crate::pipelines::processors::transforms::TransformSortBuilder; use crate::pipelines::PipelineBuilder; -use crate::servers::flight::v1::exchange::ExchangeInjector; use crate::sessions::QueryContext; use crate::spillers::Spiller; use crate::spillers::SpillerConfig; @@ -148,12 +142,12 @@ impl PipelineBuilder { SortStep::Sample => { builder.build_sample(&mut self.main_pipeline)?; - self.exchange_injector = Arc::new(SortInjector {}); + self.exchange_injector = TransformSortBuilder::exchange_injector(); Ok(()) } SortStep::SortShuffled => { if matches!(*sort.input, PhysicalPlan::ExchangeSource(_)) { - let exchange = Arc::new(SortInjector {}); + let exchange = TransformSortBuilder::exchange_injector(); let old_inject = std::mem::replace(&mut self.exchange_injector, exchange); self.build_pipeline(&sort.input)?; self.exchange_injector = old_inject; @@ -163,7 +157,7 @@ impl PipelineBuilder { builder.build_bounded_merge_sort(&mut self.main_pipeline) } - SortStep::Route => self.main_pipeline.resize(1, false), + SortStep::Route => TransformSortBuilder::add_route(&mut self.main_pipeline), } } } @@ -229,10 +223,8 @@ impl SortPipelineBuilder { self.build_merge_sort_pipeline(pipeline, false) } - pub fn build_range_shuffle_sort_pipeline(self, pipeline: &mut Pipeline) -> Result<()> { - let inputs = pipeline.output_len(); + fn build_sample(self, pipeline: &mut Pipeline) -> Result<()> { let settings = self.ctx.get_settings(); - let num_exec = inputs; let max_block_size = settings.get_max_block_size()? as usize; // Partial sort @@ -258,17 +250,13 @@ impl SortPipelineBuilder { let memory_settings = MemorySettings::from_sort_settings(&self.ctx)?; let enable_loser_tree = settings.get_enable_loser_tree_merge_sort()?; - let builder = TransformSortBuilder::new( - self.schema.clone(), - self.sort_desc.clone(), - max_block_size, - spiller, - ) - .with_limit(self.limit) - .with_order_col_generated(false) - .with_output_order_col(false) - .with_memory_settings(memory_settings) - .with_enable_loser_tree(enable_loser_tree); + let builder = + TransformSortBuilder::new(self.schema.clone(), self.sort_desc.clone(), max_block_size) + .with_spiller(spiller) + .with_limit(self.limit) + .with_order_column(false, true) + .with_memory_settings(memory_settings) + .with_enable_loser_tree(enable_loser_tree); pipeline.add_transform(|input, output| { Ok(ProcessorPtr::create(builder.build_collect(input, output)?)) @@ -276,105 +264,21 @@ impl SortPipelineBuilder { builder.add_bound_broadcast( pipeline, - builder.inner_schema(), max_block_size, self.ctx.clone(), self.broadcast_id.unwrap(), )?; - pipeline.exchange(num_exec, Arc::new(SortRangeExchange))?; - - pipeline.add_transform(|input, output| { - Ok(ProcessorPtr::create(builder.build_combine(input, output)?)) - })?; - pipeline.add_transform(|input, output| { Ok(ProcessorPtr::create(builder.build_restore(input, output)?)) })?; - add_range_shuffle_route(pipeline)?; - - if self.limit.is_none() { - return Ok(()); - } - - pipeline.add_transform(|input, output| { - TransformLimit::try_create(self.limit, 0, input, output).map(ProcessorPtr::create) - }) - } - - fn build_sample(self, pipeline: &mut Pipeline) -> Result<()> { - let settings = self.ctx.get_settings(); - let max_block_size = settings.get_max_block_size()? as usize; - - // Partial sort - pipeline.add_transformer(|| { - TransformSortPartial::new( - LimitType::from_limit_rows(self.limit), - self.sort_desc.clone(), - ) - }); - - let spiller = { - let location_prefix = self.ctx.query_id_spill_prefix(); - let config = SpillerConfig { - spiller_type: SpillerType::OrderBy, - location_prefix, - disk_spill: None, - use_parquet: settings.get_spilling_file_format()?.is_parquet(), - }; - let op = DataOperator::instance().spill_operator(); - Arc::new(Spiller::create(self.ctx.clone(), op, config)?) - }; - - let memory_settings = MemorySettings::from_sort_settings(&self.ctx)?; - let enable_loser_tree = settings.get_enable_loser_tree_merge_sort()?; - - let builder = TransformSortBuilder::new( - self.schema.clone(), - self.sort_desc.clone(), - max_block_size, - spiller, - ) - .with_limit(self.limit) - .with_order_col_generated(false) - .with_output_order_col(true) - .with_memory_settings(memory_settings) - .with_enable_loser_tree(enable_loser_tree); - - pipeline.add_transform(|input, output| { - Ok(ProcessorPtr::create(builder.build_collect(input, output)?)) - })?; - - builder.add_bound_broadcast( - pipeline, - builder.inner_schema(), - max_block_size, - self.ctx.clone(), - self.broadcast_id.unwrap(), - )?; - pipeline.add_transform(|input, output| { - Ok(ProcessorPtr::create(builder.build_restore(input, output)?)) + Ok(ProcessorPtr::create( + builder.build_bound_edge(input, output)?, + )) })?; - // pipeline.exchange_with_merge( - // pipeline.output_len(), - // Arc::new(SortBoundExchange {}), - // |inputs, output| { - // Ok(ProcessorPtr::create(create_multi_sort_processor( - // inputs, - // output, - // self.schema.clone(), - // self.block_size, - // self.limit, - // self.sort_desc.clone(), - // self.remove_order_col_at_last, - // self.enable_loser_tree, - // )?)) - // }, - // )?; - Ok(()) } @@ -421,11 +325,10 @@ impl SortPipelineBuilder { sort_merge_output_schema.clone(), self.sort_desc.clone(), self.block_size, - spiller.clone(), ) + .with_spiller(spiller.clone()) .with_limit(self.limit) - .with_order_col_generated(order_col_generated) - .with_output_order_col(output_order_col) + .with_order_column(order_col_generated, output_order_col) .with_memory_settings(memory_settings.clone()) .with_enable_loser_tree(enable_loser_tree); @@ -476,28 +379,20 @@ impl SortPipelineBuilder { } } - pub fn exchange_injector(&self) -> Arc { - Arc::new(SortInjector {}) - } - pub fn build_bounded_merge_sort(self, pipeline: &mut Pipeline) -> Result<()> { + let builder = + TransformSortBuilder::new(self.schema.clone(), self.sort_desc.clone(), self.block_size) + .with_limit(self.limit) + .with_order_column(true, self.remove_order_col_at_last) + .with_enable_loser_tree(self.enable_loser_tree); + let inputs_port: Vec<_> = (0..pipeline.output_len()) .map(|_| InputPort::create()) .collect(); let output_port = OutputPort::create(); let processor = ProcessorPtr::create( - BoundedMergeSortBuilder::new( - inputs_port.clone(), - output_port.clone(), - self.schema.clone(), - self.sort_desc.clone(), - self.block_size, - self.limit, - self.remove_order_col_at_last, - self.enable_loser_tree, - ) - .build()?, + builder.build_bounded_merge_sort(inputs_port.clone(), output_port.clone())?, ); pipeline.add_pipe(Pipe::create(inputs_port.len(), 1, vec![PipeItem::create( diff --git a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs index 0e4d219c2ee4b..13f464f14bac1 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs @@ -29,8 +29,6 @@ mod bounds; mod merge_sort; mod sort_builder; mod sort_collect; -mod sort_combine; -mod sort_exchange; mod sort_exchange_injector; mod sort_merge_stream; mod sort_restore; @@ -41,8 +39,6 @@ mod sort_spill; pub use merge_sort::*; pub use sort_builder::*; pub use sort_collect::*; -pub use sort_combine::*; -pub use sort_exchange::*; pub use sort_exchange_injector::*; pub use sort_merge_stream::*; pub use sort_restore::*; @@ -69,14 +65,6 @@ local_block_meta_serde!(SortCollectedMeta); #[typetag::serde(name = "sort_collected")] impl BlockMetaInfo for SortCollectedMeta {} -#[derive(Debug)] -struct SortScatteredMeta(pub Vec>); - -local_block_meta_serde!(SortScatteredMeta); - -#[typetag::serde(name = "sort_scattered")] -impl BlockMetaInfo for SortScatteredMeta {} - trait MemoryRows { fn in_memory_rows(&self) -> usize; } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs index 3a68dd52e2c6b..24f545ed7887e 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs @@ -21,6 +21,8 @@ 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::processors::sort::algorithm::SortAlgorithm; use databend_common_pipeline_transforms::sort::algorithm::HeapSort; @@ -31,19 +33,21 @@ use databend_common_pipeline_transforms::sort::utils::ORDER_COL_NAME; use databend_common_pipeline_transforms::sort::RowConverter; use databend_common_pipeline_transforms::sort::Rows; use databend_common_pipeline_transforms::sort::RowsTypeVisitor; -use databend_common_pipeline_transforms::AccumulatingTransformer; use databend_common_pipeline_transforms::MemorySettings; use super::*; +use crate::servers::flight::v1::exchange::ExchangeInjector; use crate::sessions::QueryContext; use crate::spillers::Spiller; enum SortType { - Sort, - Collect, - BoundBroadcast, - Combine, - Restore, + Sort(Arc), + + Collect(Arc), + BoundBroadcast(Arc), + Restore(Arc), + + BoundedMergeSort(Vec>), } pub struct TransformSortBuilder { @@ -53,7 +57,7 @@ pub struct TransformSortBuilder { order_col_generated: bool, output_order_col: bool, memory_settings: MemorySettings, - spiller: Arc, + spiller: Option>, enable_loser_tree: bool, limit: Option, } @@ -63,13 +67,12 @@ impl TransformSortBuilder { schema: DataSchemaRef, sort_desc: Arc<[SortColumnDescription]>, block_size: usize, - spiller: Arc, ) -> Self { TransformSortBuilder { block_size, schema, sort_desc, - spiller, + spiller: None, order_col_generated: false, output_order_col: false, enable_loser_tree: false, @@ -78,13 +81,14 @@ impl TransformSortBuilder { } } - pub fn with_order_col_generated(mut self, order_col_generated: bool) -> Self { - self.order_col_generated = order_col_generated; + pub fn with_spiller(mut self, spiller: Arc) -> Self { + self.spiller = Some(spiller); self } - pub fn with_output_order_col(mut self, output_order_col: bool) -> Self { - self.output_order_col = output_order_col; + pub fn with_order_column(mut self, generated: bool, output: bool) -> Self { + self.order_col_generated = generated; + self.output_order_col = output; self } @@ -112,9 +116,8 @@ impl TransformSortBuilder { let mut build = Build { params: self, - input, output, - typ: SortType::Sort, + typ: Some(SortType::Sort(input)), state: None, }; @@ -130,64 +133,70 @@ impl TransformSortBuilder { let mut build = Build { params: self, - input, output, - typ: SortType::Collect, + typ: Some(SortType::Collect(input)), state: None, }; select_row_type(&mut build) } - pub fn build_restore( + pub fn build_bound_broadcast( &self, input: Arc, output: Arc, + state: SortSampleState, ) -> Result> { self.check(); let mut build = Build { params: self, - input, output, - typ: SortType::Restore, - state: None, + typ: Some(SortType::BoundBroadcast(input)), + state: Some(state), }; select_row_type(&mut build) } - pub fn build_bound_broadcast( + pub fn build_restore( &self, input: Arc, output: Arc, - state: SortSampleState, ) -> Result> { self.check(); let mut build = Build { params: self, - input, output, - typ: SortType::BoundBroadcast, - state: Some(state), + typ: Some(SortType::Restore(input)), + state: None, }; select_row_type(&mut build) } - pub fn build_combine( + pub fn build_bound_edge( &self, input: Arc, output: Arc, ) -> Result> { self.check(); + Ok(Box::new(SortBoundEdge::new(input, output))) + } + + pub fn build_bounded_merge_sort( + &self, + inputs: Vec>, + output: Arc, + ) -> Result> { + self.check(); + let mut build = Build { params: self, - input, output, - typ: SortType::Combine, + typ: Some(SortType::BoundedMergeSort(inputs)), state: None, }; @@ -199,11 +208,7 @@ impl TransformSortBuilder { } fn check(&self) { - assert!(if self.output_order_col { - self.schema.has_field(ORDER_COL_NAME) - } else { - !self.schema.has_field(ORDER_COL_NAME) - }); + assert_eq!(self.schema.has_field(ORDER_COL_NAME), self.output_order_col) } fn new_base(&self) -> Base { @@ -212,24 +217,23 @@ impl TransformSortBuilder { Base { sort_row_offset, schema, - spiller: self.spiller.clone(), + spiller: self.spiller.clone().unwrap(), limit: self.limit, } } - pub fn inner_schema(&self) -> DataSchemaRef { + fn inner_schema(&self) -> DataSchemaRef { add_order_field(self.schema.clone(), &self.sort_desc) } pub fn add_bound_broadcast( &self, pipeline: &mut Pipeline, - schema: DataSchemaRef, batch_rows: usize, ctx: Arc, broadcast_id: u32, ) -> Result<()> { - let state = SortSampleState::new(schema, batch_rows, ctx, broadcast_id); + let state = SortSampleState::new(self.inner_schema(), batch_rows, ctx, broadcast_id); pipeline.resize(1, false)?; pipeline.add_transform(|input, output| { @@ -240,80 +244,119 @@ impl TransformSortBuilder { )?)) }) } + + pub fn add_route(pipeline: &mut Pipeline) -> Result<()> { + let inputs = pipeline.output_len(); + let inputs_port: Vec<_> = (0..inputs).map(|_| InputPort::create()).collect(); + let output = OutputPort::create(); + + let processor = ProcessorPtr::create(Box::new(TransformSortRoute::new( + inputs_port.clone(), + output.clone(), + ))); + + let pipe = Pipe::create(inputs, 1, vec![PipeItem::create( + processor, + inputs_port, + vec![output], + )]); + + pipeline.add_pipe(pipe); + Ok(()) + } + + pub fn exchange_injector() -> Arc { + Arc::new(SortInjector {}) + } } struct Build<'a> { params: &'a TransformSortBuilder, - typ: SortType, - input: Arc, + typ: Option, output: Arc, state: Option, } impl Build<'_> { - fn build_sort(&mut self, limit_sort: bool) -> Result> + fn build_sort( + &mut self, + limit_sort: bool, + input: Arc, + ) -> Result> where A: SortAlgorithm + 'static, C: RowConverter + Send + 'static, { let schema = add_order_field(self.params.schema.clone(), &self.params.sort_desc); Ok(Box::new(TransformSort::::new( - self.input.clone(), + input, self.output.clone(), schema, self.params.sort_desc.clone(), self.params.block_size, self.params.limit.map(|limit| (limit, limit_sort)), - self.params.spiller.clone(), + self.params.spiller.clone().unwrap(), self.params.output_order_col, self.params.order_col_generated, self.params.memory_settings.clone(), )?)) } - fn build_sort_collect(&mut self, limit_sort: bool) -> Result> + fn build_sort_collect( + &mut self, + sort_limit: bool, + input: Arc, + ) -> Result> where A: SortAlgorithm + 'static, C: RowConverter + Send + 'static, { Ok(Box::new(TransformSortCollect::::new( - self.input.clone(), + input, self.output.clone(), self.params.new_base(), self.params.sort_desc.clone(), self.params.block_size, - limit_sort, + sort_limit, self.params.order_col_generated, self.params.memory_settings.clone(), )?)) } - fn build_sort_restore(&mut self) -> Result> + fn build_sort_restore(&mut self, input: Arc) -> Result> where A: SortAlgorithm + 'static { Ok(Box::new(TransformSortRestore::::create( - self.input.clone(), + input, self.output.clone(), self.params.new_base(), self.params.output_order_col, )?)) } - fn build_bound_broadcast(&mut self) -> Result> + fn build_bound_broadcast(&mut self, input: Arc) -> Result> where R: Rows + 'static { Ok(TransformSortBoundBroadcast::::create( - self.input.clone(), + input, self.output.clone(), self.state.clone().unwrap(), )) } - fn build_sort_combine(&mut self) -> Result> - where R: Rows + 'static { - Ok(AccumulatingTransformer::create( - self.input.clone(), + fn build_bounded_merge_sort( + &mut self, + inputs: Vec>, + ) -> Result> + where + A: SortAlgorithm + 'static, + { + Ok(Box::new(BoundedMultiSortMergeProcessor::::new( + inputs, self.output.clone(), - TransformSortCombine::::new(self.params.block_size), - )) + self.schema().clone(), + self.params.block_size, + self.params.limit, + !self.params.output_order_col, + )?)) } } @@ -333,101 +376,26 @@ impl RowsTypeVisitor for Build<'_> { C: RowConverter + Send + 'static, { let limit_sort = self.params.should_use_sort_limit(); - match self.typ { - SortType::Sort => match self.params.enable_loser_tree { - true => self.build_sort::, C>(limit_sort), - false => self.build_sort::, C>(limit_sort), + match self.typ.take().unwrap() { + SortType::Sort(input) => match self.params.enable_loser_tree { + true => self.build_sort::, C>(limit_sort, input), + false => self.build_sort::, C>(limit_sort, input), }, - SortType::Collect => match self.params.enable_loser_tree { - true => self.build_sort_collect::, C>(limit_sort), - false => self.build_sort_collect::, C>(limit_sort), + + SortType::Collect(input) => match self.params.enable_loser_tree { + true => self.build_sort_collect::, C>(limit_sort, input), + false => self.build_sort_collect::, C>(limit_sort, input), }, - SortType::Restore => match self.params.enable_loser_tree { - true => self.build_sort_restore::>(), - false => self.build_sort_restore::>(), + SortType::BoundBroadcast(input) => self.build_bound_broadcast::(input), + SortType::Restore(input) => match self.params.enable_loser_tree { + true => self.build_sort_restore::>(input), + false => self.build_sort_restore::>(input), }, - SortType::BoundBroadcast => self.build_bound_broadcast::(), - SortType::Combine => self.build_sort_combine::(), - } - } -} - -pub struct BoundedMergeSortBuilder { - inputs: Vec>, - output: Arc, - schema: DataSchemaRef, - sort_desc: Arc<[SortColumnDescription]>, - block_size: usize, - limit: Option, - remove_order_col: bool, - enable_loser_tree: bool, -} - -impl BoundedMergeSortBuilder { - pub fn new( - inputs: Vec>, - output: Arc, - schema: DataSchemaRef, - sort_desc: Arc<[SortColumnDescription]>, - block_size: usize, - limit: Option, - remove_order_col: bool, - enable_loser_tree: bool, - ) -> Self { - Self { - inputs, - output, - schema, - sort_desc, - block_size, - limit, - remove_order_col, - enable_loser_tree, - } - } - pub fn build(mut self) -> Result> { - select_row_type(&mut self) - } -} - -impl RowsTypeVisitor for BoundedMergeSortBuilder { - type Result = Result>; - - fn schema(&self) -> DataSchemaRef { - self.schema.clone() - } - - fn sort_desc(&self) -> &[SortColumnDescription] { - &self.sort_desc - } - - fn visit_type(&mut self) -> Self::Result - where - R: Rows + 'static, - C: RowConverter + Send + 'static, - { - match self.enable_loser_tree { - true => Ok(Box::new( - BoundedMultiSortMergeProcessor::>::new( - self.inputs.clone(), - self.output.clone(), - self.schema.clone(), - self.block_size, - self.limit, - self.remove_order_col, - )?, - )), - false => Ok(Box::new( - BoundedMultiSortMergeProcessor::>::new( - self.inputs.clone(), - self.output.clone(), - self.schema.clone(), - self.block_size, - self.limit, - self.remove_order_col, - )?, - )), + SortType::BoundedMergeSort(inputs) => match self.params.enable_loser_tree { + true => self.build_bounded_merge_sort::>(inputs), + false => self.build_bounded_merge_sort::>(inputs), + }, } } } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_combine.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_combine.rs deleted file mode 100644 index c3d2854e82bab..0000000000000 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_combine.rs +++ /dev/null @@ -1,80 +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 databend_common_exception::Result; -use databend_common_expression::BlockMetaInfoDowncast; -use databend_common_expression::DataBlock; -use databend_common_pipeline_transforms::sort::Rows; -use databend_common_pipeline_transforms::AccumulatingTransform; - -use super::bounds::Bounds; -use super::SortCollectedMeta; - -pub struct TransformSortCombine { - batch_rows: usize, - metas: Vec, - _r: std::marker::PhantomData, -} - -impl TransformSortCombine { - pub fn new(batch_rows: usize) -> Self { - Self { - batch_rows, - metas: vec![], - _r: Default::default(), - } - } -} - -impl AccumulatingTransform for TransformSortCombine { - const NAME: &'static str = "TransformSortCombine"; - - fn transform(&mut self, mut data: DataBlock) -> Result> { - self.metas.push( - data.take_meta() - .and_then(SortCollectedMeta::downcast_from) - .expect("require a SortCollectedMeta"), - ); - Ok(vec![]) - } - - fn on_finish(&mut self, output: bool) -> Result> { - if !output || self.metas.is_empty() { - return Ok(vec![]); - } - - let params = self.metas.first().map(|meta| meta.params).unwrap(); - - let bounds = self - .metas - .iter_mut() - .map(|meta| std::mem::take(&mut meta.bounds)) - .collect(); - let bounds = Bounds::merge::(bounds, self.batch_rows)?; - - let sequences = self - .metas - .drain(..) - .flat_map(|meta| meta.sequences.into_iter()) - .collect(); - - Ok(vec![DataBlock::empty_with_meta(Box::new( - SortCollectedMeta { - params, - bounds, - sequences, - }, - ))]) - } -} diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange.rs deleted file mode 100644 index 08a3a2adf1b7f..0000000000000 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange.rs +++ /dev/null @@ -1,55 +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 databend_common_exception::Result; -use databend_common_expression::BlockMetaInfoDowncast; -use databend_common_expression::DataBlock; -use databend_common_pipeline_core::processors::Exchange; - -use super::bound_scatter; -use super::SortScatteredMeta; - -pub struct SortRangeExchange; - -impl Exchange for SortRangeExchange { - const NAME: &'static str = "SortRange"; - fn partition(&self, mut data: DataBlock, n: usize) -> Result> { - let scattered = data - .take_meta() - .and_then(SortScatteredMeta::downcast_from) - .expect("require a SortScatteredMeta") - .0; - assert!(scattered.len() <= n); - - let blocks = scattered - .into_iter() - .map(|meta| { - meta.map(|meta| DataBlock::empty_with_meta(Box::new(meta))) - .unwrap_or_else(DataBlock::empty) - }) - .collect(); - - Ok(blocks) - } -} - -pub struct SortBoundExchange; - -impl Exchange for SortBoundExchange { - const NAME: &'static str = "SortBound"; - - fn partition(&self, data_block: DataBlock, n: usize) -> Result> { - bound_scatter(data_block, n as _) - } -} diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs index 50bf8a12367b2..43cff0f1e205a 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs @@ -97,7 +97,7 @@ impl FlightScatter for SortBoundScatter { } } -pub(super) fn bound_scatter(data_block: DataBlock, n: usize) -> Result> { +fn bound_scatter(data_block: DataBlock, n: usize) -> Result> { let meta = *data_block .get_meta() .and_then(SortBound::downcast_ref_from) @@ -109,185 +109,3 @@ pub(super) fn bound_scatter(data_block: DataBlock, n: usize) -> Result, -// local_pos: usize, -// options: IpcWriteOptions, - -// spiller: Arc, -// } - -// impl BlockMetaTransform for TransformExchangeSortSerializer { -// const NAME: &'static str = "TransformExchangeSortSerializer"; - -// fn transform(&mut self, _meta: ExchangeShuffleMeta) -> Result> { -// // let serialized_blocks = meta -// // .blocks -// // .into_iter() -// // .map(|mut block| { -// // let SortCollectedMeta { -// // params, -// // bounds, -// // blocks, -// // } = block -// // .take_meta() -// // .and_then(SortCollectedMeta::downcast_from) -// // .unwrap(); - -// // 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, -// // )?, -// // }, -// // }) -// // .collect(); - -// // let meta = SortCollectedMeta::downcast_from(block.take_meta().unwrap()).unwrap(); - -// // match AggregateMeta::downcast_from(block.take_meta().unwrap()) { -// // None => unreachable!(), -// // Some(AggregateMeta::Spilled(_)) => unreachable!(), -// // Some(AggregateMeta::Serialized(_)) => unreachable!(), -// // Some(AggregateMeta::BucketSpilled(_)) => unreachable!(), -// // Some(AggregateMeta::Partitioned { .. }) => unreachable!(), -// // Some(AggregateMeta::AggregateSpilling(payload)) => { -// // serialized_blocks.push(FlightSerialized::Future( - -// // )); -// // } - -// // 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)); -// // } -// // }; - -// todo!() - -// // Ok(vec![DataBlock::empty_with_meta( -// // FlightSerializedMeta::create(serialized_blocks), -// // )]) -// } -// } - -// struct SortScatter { -// ctx: Arc, -// local_pos: usize, -// options: IpcWriteOptions, -// schema: DataSchemaRef, - -// partitions: usize, -// spiller: Arc, -// data: Option, -// scatter_bounds: Bounds, -// blocks: Vec>, - -// _r: PhantomData, -// } - -// #[async_trait::async_trait] -// impl AsyncBlockingTransform for SortScatter { -// const NAME: &'static str = "TransformExchangeSortSerializer"; - -// async fn consume(&mut self, mut block: DataBlock) -> Result<()> { -// let meta = block -// .take_meta() -// .and_then(SortCollectedMeta::downcast_from) -// .unwrap(); -// self.data = Some(meta); -// Ok(()) -// } - -// async fn transform(&mut self) -> Result> { -// todo!() -// } -// } - -// impl SortScatter { -// fn scatter_bounds(&self, bounds: Bounds) -> Bounds { -// let n = self.partitions - 1; -// let bounds = if bounds.len() < n { -// bounds -// } else { -// bounds.dedup_reduce::(n) -// }; -// assert!(bounds.len() < self.partitions); -// bounds -// } - -// async fn scatter(&mut self) -> Result>> { -// // if self.scatter_bounds.is_empty() { -// // return Ok(vec![Some(SortCollectedMeta { -// // params, -// // bounds, -// // blocks, -// // })]); -// // } - -// // let base = { -// // Base { -// // schema: self.schema.clone(), -// // spiller: self.spiller.clone(), -// // sort_row_offset: self.schema.fields.len() - 1, -// // limit: None, -// // } -// // }; - -// // let mut scattered_blocks = Vec::with_capacity(self.scatter_bounds.len() + 1); - -// // let Some(list) = self.blocks.pop() else { -// // todo!() -// // }; -// // let scattered = base -// // .scatter_stream::(Vec::from(list).into(), self.scatter_bounds.clone()) -// // .await?; - -// // ExchangeShuffleMeta::create(blocks); - -// // for list in { - -// // } - -// // let scattered_meta = scattered_blocks -// // .into_iter() -// // .map(|blocks| { -// // (!blocks.is_empty()).then_some(SortCollectedMeta { -// // params: todo!(), -// // bounds: todo!(), -// // blocks, -// // }) -// // }) -// // .collect(); -// // Ok(scattered_meta) - -// todo!() -// } -// } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_restore.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_restore.rs index c46d35a8e9c3b..2d02e5b5d4135 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_restore.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_restore.rs @@ -134,7 +134,7 @@ where } } -struct SortBoundEdge { +pub struct SortBoundEdge { input: Arc, output: Arc, data: Option, diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_route.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_route.rs index 4963025f48753..1022feec21389 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_route.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_route.rs @@ -22,14 +22,10 @@ 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 crate::pipelines::processors::transforms::SortBound; -struct TransformSortRoute { +pub struct TransformSortRoute { inputs: Vec>, output: Arc, @@ -38,7 +34,7 @@ struct TransformSortRoute { } impl TransformSortRoute { - fn new(inputs: Vec>, output: Arc) -> Self { + pub(super) fn new(inputs: Vec>, output: Arc) -> Self { Self { input_data: vec![None; inputs.len()], cur_index: 0, @@ -117,23 +113,3 @@ impl Processor for TransformSortRoute { Ok(Event::NeedData) } } - -pub fn add_range_shuffle_route(pipeline: &mut Pipeline) -> Result<()> { - let inputs = pipeline.output_len(); - let inputs_port = (0..inputs).map(|_| InputPort::create()).collect::>(); - let output = OutputPort::create(); - - let processor = ProcessorPtr::create(Box::new(TransformSortRoute::new( - inputs_port.clone(), - output.clone(), - ))); - - let pipe = Pipe::create(inputs, 1, vec![PipeItem::create( - processor, - inputs_port, - vec![output], - )]); - - pipeline.add_pipe(pipe); - Ok(()) -} diff --git a/src/query/settings/src/settings_default.rs b/src/query/settings/src/settings_default.rs index 2b66e402163e1..089d9d4569d83 100644 --- a/src/query/settings/src/settings_default.rs +++ b/src/query/settings/src/settings_default.rs @@ -654,9 +654,9 @@ impl DefaultSettings { scope: SettingScope::Both, range: Some(SettingRange::Numeric(4 * 1024..=u64::MAX)), }), - ("enable_range_shuffle_sort", DefaultSettingValue { + ("enable_shuffle_sort", DefaultSettingValue { value: UserSettingValue::UInt64(0), - desc: "Enable range shuffle sort.", + desc: "Enable shuffle sort.", mode: SettingMode::Both, scope: SettingScope::Both, range: Some(SettingRange::Numeric(0..=1)), diff --git a/src/query/settings/src/settings_getter_setter.rs b/src/query/settings/src/settings_getter_setter.rs index 1fc585b84ff9c..6f93e4f258e42 100644 --- a/src/query/settings/src/settings_getter_setter.rs +++ b/src/query/settings/src/settings_getter_setter.rs @@ -512,8 +512,8 @@ impl Settings { Ok(self.try_get_u64("sort_spilling_to_disk_bytes_limit")? as usize) } - pub fn get_enable_range_shuffle_sort(&self) -> Result { - Ok(self.try_get_u64("enable_range_shuffle_sort")? == 1) + pub fn get_enable_shuffle_sort(&self) -> Result { + Ok(self.try_get_u64("enable_shuffle_sort")? == 1) } pub fn get_group_by_shuffle_mode(&self) -> Result { diff --git a/src/query/sql/src/executor/physical_plans/physical_sort.rs b/src/query/sql/src/executor/physical_plans/physical_sort.rs index c984f0238b789..9f93eb89dad70 100644 --- a/src/query/sql/src/executor/physical_plans/physical_sort.rs +++ b/src/query/sql/src/executor/physical_plans/physical_sort.rs @@ -228,7 +228,7 @@ impl PhysicalPlanBuilder { })); }; - if !self.ctx.get_settings().get_enable_range_shuffle_sort()? { + if !self.ctx.get_settings().get_enable_shuffle_sort()? { let input_plan = self.build(s_expr.unary_child(), required).await?; return if !after_exchange { Ok(PhysicalPlan::Sort(Sort { From fdac25123fdcf472e2a532534ca48a10d1c93320 Mon Sep 17 00:00:00 2001 From: coldWater Date: Thu, 10 Jul 2025 17:58:40 +0800 Subject: [PATCH 56/61] enable --- src/query/settings/src/settings_default.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/query/settings/src/settings_default.rs b/src/query/settings/src/settings_default.rs index 089d9d4569d83..762d404ba16d0 100644 --- a/src/query/settings/src/settings_default.rs +++ b/src/query/settings/src/settings_default.rs @@ -655,7 +655,7 @@ impl DefaultSettings { range: Some(SettingRange::Numeric(4 * 1024..=u64::MAX)), }), ("enable_shuffle_sort", DefaultSettingValue { - value: UserSettingValue::UInt64(0), + value: UserSettingValue::UInt64(1), desc: "Enable shuffle sort.", mode: SettingMode::Both, scope: SettingScope::Both, From 89dd8851e3a789fa7762739e24a1be0a968449a9 Mon Sep 17 00:00:00 2001 From: coldWater Date: Thu, 10 Jul 2025 18:43:40 +0800 Subject: [PATCH 57/61] fix --- .../src/pipelines/builders/builder_sort.rs | 8 ++--- .../src/pipelines/builders/builder_window.rs | 2 +- .../processors/transforms/sort/mod.rs | 4 +-- .../{sort_shuffle.rs => sort_broadcast.rs} | 13 ++------ .../transforms/sort/sort_builder.rs | 31 +++++++++++-------- .../transforms/sort/sort_restore.rs | 2 +- .../data/results/tpcds/Q01_physical.txt | 4 +-- .../data/results/tpcds/Q03_physical.txt | 4 +-- .../sql/planner/optimizer/optimizer_test.rs | 1 + .../executor/physical_plans/physical_sort.rs | 20 ++++++------ 10 files changed, 43 insertions(+), 46 deletions(-) rename src/query/service/src/pipelines/processors/transforms/sort/{sort_shuffle.rs => sort_broadcast.rs} (94%) diff --git a/src/query/service/src/pipelines/builders/builder_sort.rs b/src/query/service/src/pipelines/builders/builder_sort.rs index 793f20a60a110..81b9bb04ccb0e 100644 --- a/src/query/service/src/pipelines/builders/builder_sort.rs +++ b/src/query/service/src/pipelines/builders/builder_sort.rs @@ -66,7 +66,7 @@ impl PipelineBuilder { .collect::>>()?; let sort_desc = sort_desc.into(); - if sort.step != SortStep::SortShuffled { + if sort.step != SortStep::Shuffled { self.build_pipeline(&sort.input)?; } @@ -124,7 +124,7 @@ impl PipelineBuilder { // Don't remove the order column at last. builder.build_full_sort_pipeline(&mut self.main_pipeline) } - SortStep::FinalMerge => { + SortStep::Final => { // Build for the coordinator node. // We only build a `MultiSortMergeTransform`, // as the data is already sorted in each cluster node. @@ -145,7 +145,7 @@ impl PipelineBuilder { self.exchange_injector = TransformSortBuilder::exchange_injector(); Ok(()) } - SortStep::SortShuffled => { + SortStep::Shuffled => { if matches!(*sort.input, PhysicalPlan::ExchangeSource(_)) { let exchange = TransformSortBuilder::exchange_injector(); let old_inject = std::mem::replace(&mut self.exchange_injector, exchange); @@ -383,7 +383,7 @@ impl SortPipelineBuilder { let builder = TransformSortBuilder::new(self.schema.clone(), self.sort_desc.clone(), self.block_size) .with_limit(self.limit) - .with_order_column(true, self.remove_order_col_at_last) + .with_order_column(true, !self.remove_order_col_at_last) .with_enable_loser_tree(self.enable_loser_tree); let inputs_port: Vec<_> = (0..pipeline.output_len()) diff --git a/src/query/service/src/pipelines/builders/builder_window.rs b/src/query/service/src/pipelines/builders/builder_window.rs index c25228606717d..f4cd950a76d1a 100644 --- a/src/query/service/src/pipelines/builders/builder_window.rs +++ b/src/query/service/src/pipelines/builders/builder_window.rs @@ -201,7 +201,7 @@ impl PipelineBuilder { let have_order_col = match window_partition.sort_step { SortStep::Single | SortStep::Partial => false, - SortStep::FinalMerge => true, + SortStep::Final => true, _ => unimplemented!(), }; diff --git a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs index 13f464f14bac1..c8a98290e24bc 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs @@ -27,23 +27,23 @@ use crate::spillers::Spiller; mod bounds; mod merge_sort; +mod sort_broadcast; mod sort_builder; mod sort_collect; mod sort_exchange_injector; mod sort_merge_stream; mod sort_restore; mod sort_route; -mod sort_shuffle; mod sort_spill; pub use merge_sort::*; +pub use sort_broadcast::*; pub use sort_builder::*; pub use sort_collect::*; pub use sort_exchange_injector::*; pub use sort_merge_stream::*; pub use sort_restore::*; pub use sort_route::*; -pub use sort_shuffle::*; #[derive(Clone)] struct Base { diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_shuffle.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_broadcast.rs similarity index 94% rename from src/query/service/src/pipelines/processors/transforms/sort/sort_shuffle.rs rename to src/query/service/src/pipelines/processors/transforms/sort/sort_broadcast.rs index b4152c0985c68..023460458f6f4 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_shuffle.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_broadcast.rs @@ -19,7 +19,6 @@ use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::DataBlock; -use databend_common_expression::DataSchemaRef; use databend_common_pipeline_core::processors::Event; use databend_common_pipeline_core::processors::Processor; use databend_common_pipeline_transforms::processors::sort::Rows; @@ -62,22 +61,14 @@ impl TransformSortBoundBroadcast { pub struct SortSampleState { ctx: Arc, broadcast_id: u32, - #[expect(dead_code)] - schema: DataSchemaRef, batch_rows: usize, } impl SortSampleState { - pub fn new( - schema: DataSchemaRef, - batch_rows: usize, - ctx: Arc, - broadcast_id: u32, - ) -> SortSampleState { + pub fn new(batch_rows: usize, ctx: Arc, broadcast_id: u32) -> SortSampleState { SortSampleState { ctx, broadcast_id, - schema, batch_rows, } } @@ -110,7 +101,7 @@ impl SortSampleState { #[async_trait::async_trait] impl HookTransform for TransformSortBoundBroadcast { - const NAME: &'static str = "TransformSortBoundBroadcast"; + const NAME: &'static str = "SortBoundBroadcast"; fn on_input(&mut self, mut data: DataBlock) -> Result<()> { let meta = data diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs index 24f545ed7887e..1dfccce364e86 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs @@ -44,7 +44,10 @@ enum SortType { Sort(Arc), Collect(Arc), - BoundBroadcast(Arc), + BoundBroadcast { + input: Arc, + state: SortSampleState, + }, Restore(Arc), BoundedMergeSort(Vec>), @@ -118,7 +121,6 @@ impl TransformSortBuilder { params: self, output, typ: Some(SortType::Sort(input)), - state: None, }; select_row_type(&mut build) @@ -135,7 +137,6 @@ impl TransformSortBuilder { params: self, output, typ: Some(SortType::Collect(input)), - state: None, }; select_row_type(&mut build) @@ -152,8 +153,7 @@ impl TransformSortBuilder { let mut build = Build { params: self, output, - typ: Some(SortType::BoundBroadcast(input)), - state: Some(state), + typ: Some(SortType::BoundBroadcast { input, state }), }; select_row_type(&mut build) @@ -170,7 +170,6 @@ impl TransformSortBuilder { params: self, output, typ: Some(SortType::Restore(input)), - state: None, }; select_row_type(&mut build) @@ -197,7 +196,6 @@ impl TransformSortBuilder { params: self, output, typ: Some(SortType::BoundedMergeSort(inputs)), - state: None, }; select_row_type(&mut build) @@ -233,7 +231,7 @@ impl TransformSortBuilder { ctx: Arc, broadcast_id: u32, ) -> Result<()> { - let state = SortSampleState::new(self.inner_schema(), batch_rows, ctx, broadcast_id); + let state = SortSampleState::new(batch_rows, ctx, broadcast_id); pipeline.resize(1, false)?; pipeline.add_transform(|input, output| { @@ -274,7 +272,6 @@ struct Build<'a> { params: &'a TransformSortBuilder, typ: Option, output: Arc, - state: Option, } impl Build<'_> { @@ -333,12 +330,18 @@ impl Build<'_> { )?)) } - fn build_bound_broadcast(&mut self, input: Arc) -> Result> - where R: Rows + 'static { + fn build_bound_broadcast( + &mut self, + input: Arc, + state: SortSampleState, + ) -> Result> + where + R: Rows + 'static, + { Ok(TransformSortBoundBroadcast::::create( input, self.output.clone(), - self.state.clone().unwrap(), + state, )) } @@ -386,7 +389,9 @@ impl RowsTypeVisitor for Build<'_> { true => self.build_sort_collect::, C>(limit_sort, input), false => self.build_sort_collect::, C>(limit_sort, input), }, - SortType::BoundBroadcast(input) => self.build_bound_broadcast::(input), + SortType::BoundBroadcast { input, state } => { + self.build_bound_broadcast::(input, state) + } SortType::Restore(input) => match self.params.enable_loser_tree { true => self.build_sort_restore::>(input), false => self.build_sort_restore::>(input), diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_restore.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_restore.rs index 2d02e5b5d4135..49c084aa9d30a 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_restore.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_restore.rs @@ -152,7 +152,7 @@ impl SortBoundEdge { impl Processor for SortBoundEdge { fn name(&self) -> String { - String::from("SortBoundEdgeTransform") + String::from("SortBoundEdge") } fn as_any(&mut self) -> &mut dyn Any { diff --git a/src/query/service/tests/it/sql/planner/optimizer/data/results/tpcds/Q01_physical.txt b/src/query/service/tests/it/sql/planner/optimizer/data/results/tpcds/Q01_physical.txt index bf34e3e48aa1e..44eb30514a715 100644 --- a/src/query/service/tests/it/sql/planner/optimizer/data/results/tpcds/Q01_physical.txt +++ b/src/query/service/tests/it/sql/planner/optimizer/data/results/tpcds/Q01_physical.txt @@ -3,14 +3,14 @@ Limit ├── limit: 100 ├── offset: 0 ├── estimated rows: 0.00 -└── Sort +└── Sort(Final) ├── output columns: [customer.c_customer_id (#79)] ├── sort keys: [c_customer_id ASC NULLS LAST] ├── estimated rows: 0.00 └── Exchange ├── output columns: [customer.c_customer_id (#79), #_order_col] ├── exchange type: Merge - └── Sort + └── Sort(Partial) ├── output columns: [customer.c_customer_id (#79), #_order_col] ├── sort keys: [c_customer_id ASC NULLS LAST] ├── estimated rows: 0.00 diff --git a/src/query/service/tests/it/sql/planner/optimizer/data/results/tpcds/Q03_physical.txt b/src/query/service/tests/it/sql/planner/optimizer/data/results/tpcds/Q03_physical.txt index 9bc7008a03981..1ee7ed611a957 100644 --- a/src/query/service/tests/it/sql/planner/optimizer/data/results/tpcds/Q03_physical.txt +++ b/src/query/service/tests/it/sql/planner/optimizer/data/results/tpcds/Q03_physical.txt @@ -3,14 +3,14 @@ Limit ├── limit: 100 ├── offset: 0 ├── estimated rows: 100.00 -└── Sort +└── Sort(Final) ├── output columns: [SUM(ss_ext_sales_price) (#73), dt.d_year (#6), item.i_brand (#59), item.i_brand_id (#58)] ├── sort keys: [d_year ASC NULLS LAST, SUM(ss_ext_sales_price) DESC NULLS LAST, i_brand_id ASC NULLS LAST] ├── estimated rows: 143057683321996.78 └── Exchange ├── output columns: [SUM(ss_ext_sales_price) (#73), dt.d_year (#6), item.i_brand (#59), item.i_brand_id (#58), #_order_col] ├── exchange type: Merge - └── Sort + └── Sort(Partial) ├── output columns: [SUM(ss_ext_sales_price) (#73), dt.d_year (#6), item.i_brand (#59), item.i_brand_id (#58), #_order_col] ├── sort keys: [d_year ASC NULLS LAST, SUM(ss_ext_sales_price) DESC NULLS LAST, i_brand_id ASC NULLS LAST] ├── estimated rows: 143057683321996.78 diff --git a/src/query/service/tests/it/sql/planner/optimizer/optimizer_test.rs b/src/query/service/tests/it/sql/planner/optimizer/optimizer_test.rs index f46648e1d9eb8..7b38248ad5da4 100644 --- a/src/query/service/tests/it/sql/planner/optimizer/optimizer_test.rs +++ b/src/query/service/tests/it/sql/planner/optimizer/optimizer_test.rs @@ -606,6 +606,7 @@ fn configure_optimizer(ctx: &Arc, auto_stats: bool) -> Result<()> settings.set_setting("enable_dphyp".to_string(), "1".to_string())?; settings.set_setting("max_push_down_limit".to_string(), "10000".to_string())?; settings.set_setting("enable_optimizer_trace".to_string(), "1".to_string())?; + settings.set_setting("enable_shuffle_sort".to_string(), "0".to_string())?; if auto_stats { settings.set_optimizer_skip_list("".to_string()) diff --git a/src/query/sql/src/executor/physical_plans/physical_sort.rs b/src/query/sql/src/executor/physical_plans/physical_sort.rs index 9f93eb89dad70..6465e490b29eb 100644 --- a/src/query/sql/src/executor/physical_plans/physical_sort.rs +++ b/src/query/sql/src/executor/physical_plans/physical_sort.rs @@ -58,12 +58,12 @@ pub enum SortStep { Single, // cluster mode - Partial, // before the exchange plan - FinalMerge, // after the exchange plan + Partial, // before the exchange plan + Final, // after the exchange plan // range shuffle mode Sample, - SortShuffled, + Shuffled, Route, } @@ -72,9 +72,9 @@ impl Display for SortStep { match self { SortStep::Single => write!(f, "Single"), SortStep::Partial => write!(f, "Partial"), - SortStep::FinalMerge => write!(f, "FinalMerge"), + SortStep::Final => write!(f, "Final"), SortStep::Sample => write!(f, "Sample"), - SortStep::SortShuffled => write!(f, "SortShuffled"), + SortStep::Shuffled => write!(f, "Shuffled"), SortStep::Route => write!(f, "Route"), } } @@ -97,7 +97,7 @@ impl Sort { pub fn output_schema(&self) -> Result { let input_schema = self.input.output_schema()?; match self.step { - SortStep::FinalMerge | SortStep::Route => { + SortStep::Final | SortStep::Route => { let mut fields = input_schema.fields().clone(); // If the plan is after exchange plan in cluster mode, // the order column is at the last of the input schema. @@ -109,7 +109,7 @@ impl Sort { fields.pop(); Ok(DataSchemaRefExt::create(fields)) } - SortStep::SortShuffled => Ok(input_schema), + SortStep::Shuffled => Ok(input_schema), SortStep::Single | SortStep::Partial | SortStep::Sample => { let mut fields = self .pre_projection @@ -188,7 +188,7 @@ impl PhysicalPlanBuilder { let sort_step = match sort.after_exchange { Some(false) => SortStep::Partial, - Some(true) => SortStep::FinalMerge, + Some(true) => SortStep::Final, None => SortStep::Single, }; @@ -247,7 +247,7 @@ impl PhysicalPlanBuilder { input: Box::new(input_plan), order_by, limit: sort.limit, - step: SortStep::FinalMerge, + step: SortStep::Final, pre_projection: None, broadcast_id: None, stat_info: Some(stat_info), @@ -293,7 +293,7 @@ impl PhysicalPlanBuilder { input: Box::new(exchange), order_by, limit: sort.limit, - step: SortStep::SortShuffled, + step: SortStep::Shuffled, pre_projection: None, broadcast_id: None, stat_info: Some(stat_info), From d8c64bbf05f5ee00bc032278f6e33bbe99a362ad Mon Sep 17 00:00:00 2001 From: coldWater Date: Fri, 11 Jul 2025 10:28:25 +0800 Subject: [PATCH 58/61] fix --- .../processors/transforms/sort/bounds.rs | 7 +- .../transforms/sort/sort_merge_stream.rs | 28 +--- .../processors/transforms/sort/sort_route.rs | 6 +- .../mode/standalone/explain/aggregate.test | 14 +- .../standalone/explain/eliminate_sort.test | 16 +- .../mode/standalone/explain/explain.test | 10 +- .../mode/standalone/explain/explain_like.test | 4 +- .../mode/standalone/explain/lazy_read.test | 14 +- .../suites/mode/standalone/explain/limit.test | 12 +- .../mode/standalone/explain/prune_column.test | 4 +- .../explain/selectivity/is_not_null.test | 2 +- .../suites/mode/standalone/explain/sort.test | 8 +- .../mode/standalone/explain/window.test | 144 +++++++++++++----- 13 files changed, 153 insertions(+), 116 deletions(-) diff --git a/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs b/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs index e1a924c8debf6..4c2de885c8c50 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs @@ -48,7 +48,11 @@ impl Bounds { Ok(Bounds(vec![block.get_last_column().clone()])) } - pub fn merge(mut vector: Vec, batch_rows: usize) -> Result { + pub fn merge(vector: Vec, batch_rows: usize) -> Result { + let mut vector: Vec<_> = vector + .into_iter() + .filter(|bounds| !bounds.is_empty()) + .collect(); match vector.len() { 0 => Ok(Bounds(vec![])), 1 => Ok(vector.pop().unwrap()), @@ -95,7 +99,6 @@ impl Bounds { self.0.iter().map(Column::len).sum() } - #[expect(dead_code)] pub fn is_empty(&self) -> bool { self.0.iter().all(|col| col.len() == 0) } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_merge_stream.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_merge_stream.rs index aa406ec06cb54..356710ea455bb 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_merge_stream.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_merge_stream.rs @@ -146,36 +146,10 @@ where A: SortAlgorithm + 'static }; if streams.iter().all(|stream| stream.input.is_finished()) { + self.output.finish(); return Ok(Event::Finished); } - // { - // for stream in streams.iter_mut() { - // stream.pull()?; - // } - - // if streams - // .iter_mut() - // .map(|stream| stream.pull()) - // .try_fold(false, |acc, pending| acc || pending?)? - // { - // return Ok(Event::NeedData); - // } - - // if bounds.is_empty() { - // return Ok(Event::Finished); - // } - - // if bounds.iter().all(|meta| meta.index != self.bound_index) { - // let meta = match bounds.iter().min_by_key(|meta| meta.index) { - // Some(index) => *index, - // None => return Ok(Event::Finished), - // }; - // assert!(meta.index > self.bound_index); - // self.bound_index = meta.index; - // } - // } - for stream in streams.iter_mut() { stream.update_bound_index(self.bound_index); } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_route.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_route.rs index 1022feec21389..ca88b1b7ff0f2 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_route.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_route.rs @@ -44,17 +44,15 @@ impl TransformSortRoute { } fn process(&mut self) -> Result { - for input in &self.inputs { - input.set_need_data(); - } - for (input, data) in self.inputs.iter().zip(self.input_data.iter_mut()) { let meta = match data { Some((_, meta)) => *meta, None => { let Some(mut block) = input.pull_data().transpose()? else { + input.set_need_data(); continue; }; + input.set_need_data(); let meta = block .take_meta() diff --git a/tests/sqllogictests/suites/mode/standalone/explain/aggregate.test b/tests/sqllogictests/suites/mode/standalone/explain/aggregate.test index 30954b9fdd187..3000316916869 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/aggregate.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/aggregate.test @@ -227,7 +227,7 @@ EvalScalar query T explain select avg(b) from explain_agg_t1 group by a order by avg(b); ---- -Sort +Sort(Single) ├── output columns: [sum(b) / if(count(b) = 0, 1, count(b)) (#4), sum(b) / if(count(b) = 0, 1, count(b)) (#5)] ├── sort keys: [sum(b) / if(count(b) = 0, 1, count(b)) ASC NULLS LAST] ├── estimated rows: 0.00 @@ -258,7 +258,7 @@ Sort query T explain select avg(b) + 1 from explain_agg_t1 group by a order by avg(b); ---- -Sort +Sort(Single) ├── output columns: [sum(b) / if(count(b) = 0, 1, count(b)) + 1 (#4), sum(b) / if(count(b) = 0, 1, count(b)) (#5)] ├── sort keys: [sum(b) / if(count(b) = 0, 1, count(b)) ASC NULLS LAST] ├── estimated rows: 0.00 @@ -288,7 +288,7 @@ Sort query T explain select avg(b), avg(b) + 1 from explain_agg_t1 group by a order by avg(b); ---- -Sort +Sort(Single) ├── output columns: [sum(b) / if(count(b) = 0, 1, count(b)) (#4), sum(b) / if(count(b) = 0, 1, count(b)) + 1 (#5), sum(b) / if(count(b) = 0, 1, count(b)) (#6)] ├── sort keys: [sum(b) / if(count(b) = 0, 1, count(b)) ASC NULLS LAST] ├── estimated rows: 0.00 @@ -318,7 +318,7 @@ Sort query T explain select avg(b) + 1, avg(b) from explain_agg_t1 group by a order by avg(b); ---- -Sort +Sort(Single) ├── output columns: [sum(b) / if(count(b) = 0, 1, count(b)) + 1 (#4), sum(b) / if(count(b) = 0, 1, count(b)) (#5), sum(b) / if(count(b) = 0, 1, count(b)) (#6)] ├── sort keys: [sum(b) / if(count(b) = 0, 1, count(b)) ASC NULLS LAST] ├── estimated rows: 0.00 @@ -348,7 +348,7 @@ Sort query T explain select avg(b), avg(b) + 1 from explain_agg_t1 group by a order by avg(b) + 1; ---- -Sort +Sort(Single) ├── output columns: [sum(b) / if(count(b) = 0, 1, count(b)) (#4), sum(b) / if(count(b) = 0, 1, count(b)) + 1 (#5), sum(b) / if(count(b) = 0, 1, count(b)) + 1 (#6)] ├── sort keys: [sum(b) / if(count(b) = 0, 1, count(b)) + 1 ASC NULLS LAST] ├── estimated rows: 0.00 @@ -472,7 +472,7 @@ Limit ├── limit: 10 ├── offset: 0 ├── estimated rows: 0.00 -└── Sort +└── Sort(Single) ├── output columns: [count() (#2), t.referer (#0), t.isrefresh (#1)] ├── sort keys: [referer ASC NULLS LAST, isrefresh DESC NULLS LAST] ├── estimated rows: 0.00 @@ -504,7 +504,7 @@ Limit ├── limit: 10 ├── offset: 3 ├── estimated rows: 0.00 -└── Sort +└── Sort(Single) ├── output columns: [count() (#2), t.referer (#0), t.isrefresh (#1)] ├── sort keys: [referer ASC NULLS LAST, isrefresh ASC NULLS LAST] ├── estimated rows: 0.00 diff --git a/tests/sqllogictests/suites/mode/standalone/explain/eliminate_sort.test b/tests/sqllogictests/suites/mode/standalone/explain/eliminate_sort.test index 51fc761efaac0..3fbd6e428c2de 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/eliminate_sort.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/eliminate_sort.test @@ -1,7 +1,7 @@ query T explain select * from numbers(10) t order by t.number desc ---- -Sort +Sort(Single) ├── output columns: [t.number (#0)] ├── sort keys: [number DESC NULLS LAST] ├── estimated rows: 10.00 @@ -18,7 +18,7 @@ Sort query T explain select * from (select * from numbers(10) t order by t.number desc) order by number desc ---- -Sort +Sort(Single) ├── output columns: [t.number (#0)] ├── sort keys: [number DESC NULLS LAST] ├── estimated rows: 10.00 @@ -35,11 +35,11 @@ Sort query T explain select * from (select * from numbers(10) t order by t.number desc) order by t.number asc ---- -Sort +Sort(Single) ├── output columns: [t.number (#0)] ├── sort keys: [number ASC NULLS LAST] ├── estimated rows: 10.00 -└── Sort +└── Sort(Single) ├── output columns: [t.number (#0)] ├── sort keys: [number DESC NULLS LAST] ├── estimated rows: 10.00 @@ -56,11 +56,11 @@ Sort query T explain select * from (select * from numbers(10) t order by t.number desc) order by t.number desc, t.number desc ---- -Sort +Sort(Single) ├── output columns: [t.number (#0)] ├── sort keys: [number DESC NULLS LAST, number DESC NULLS LAST] ├── estimated rows: 10.00 -└── Sort +└── Sort(Single) ├── output columns: [t.number (#0)] ├── sort keys: [number DESC NULLS LAST] ├── estimated rows: 10.00 @@ -77,7 +77,7 @@ Sort query T explain select * from (select * from numbers(10) t order by t.number desc) order by t.number+1 desc ---- -Sort +Sort(Single) ├── output columns: [t.number (#0), number + 1 (#1)] ├── sort keys: [number + 1 DESC NULLS LAST] ├── estimated rows: 10.00 @@ -85,7 +85,7 @@ Sort ├── output columns: [t.number (#0), number + 1 (#1)] ├── expressions: [t.number (#0) + 1] ├── estimated rows: 10.00 - └── Sort + └── Sort(Single) ├── output columns: [t.number (#0)] ├── sort keys: [number DESC NULLS LAST] ├── estimated rows: 10.00 diff --git a/tests/sqllogictests/suites/mode/standalone/explain/explain.test b/tests/sqllogictests/suites/mode/standalone/explain/explain.test index f669955d88d69..e3079621dcfd4 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/explain.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/explain.test @@ -535,7 +535,7 @@ Limit ├── limit: 3 ├── offset: 0 ├── estimated rows: 3.00 -└── Sort +└── Sort(Single) ├── output columns: [t2.a (#2), t2.b (#3), t1.a (#0), t1.b (#1)] ├── sort keys: [a DESC NULLS LAST] ├── estimated rows: 3.85 @@ -868,7 +868,7 @@ from numbers(10) where number > 5 ) a join ( select number , to_yyyymmdd(to_timestamp(number)) as register_at from numbers(10) where number > 5 ) b on a.number=b.number order by a.number) where pt = register_at; ---- -Sort +Sort(Single) ├── output columns: [numbers.number (#0), pt (#1), register_at (#3)] ├── sort keys: [number ASC NULLS LAST] ├── estimated rows: 0.00 @@ -1079,7 +1079,7 @@ INSERT INTO t2 VALUES (1, 10), (2, 20); query T EXPLAIN SELECT * FROM t1 LEFT OUTER JOIN t2 ON TRUE AND t1.i = t2.k AND FALSE order by i, j; ---- -Sort +Sort(Single) ├── output columns: [t1.i (#0), t1.j (#1), t2.k (#2), t2.l (#3)] ├── sort keys: [i ASC NULLS LAST, j ASC NULLS LAST] ├── estimated rows: 3.00 @@ -1362,7 +1362,7 @@ HashJoin ├── build join filters: │ └── filter id:0, build key:a (#3), probe key:a (#0), filter type:bloom,inlist,min_max ├── estimated rows: 0.00 -├── Sort(Build) +├── Sort(Single)(Build) │ ├── output columns: [count(a) (#6), t1.c (#5), t1.a (#3)] │ ├── sort keys: [count(a) ASC NULLS LAST] │ ├── estimated rows: 0.00 @@ -1412,7 +1412,7 @@ HashJoin ├── build join filters: │ └── filter id:0, build key:a (#3), probe key:a (#0), filter type:bloom,inlist,min_max ├── estimated rows: 0.00 -├── Sort(Build) +├── Sort(Single)(Build) │ ├── output columns: [COUNT(*) (#6), t1.c (#5), t1.a (#3)] │ ├── sort keys: [COUNT(*) ASC NULLS LAST] │ ├── estimated rows: 0.00 diff --git a/tests/sqllogictests/suites/mode/standalone/explain/explain_like.test b/tests/sqllogictests/suites/mode/standalone/explain/explain_like.test index daf5d6459ac7b..6f638534c5404 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/explain_like.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/explain_like.test @@ -35,7 +35,7 @@ Filter query T explain select * from t1 where s like 'abcd%' order by s; ---- -Sort +Sort(Single) ├── output columns: [t1.s (#0)] ├── sort keys: [s ASC NULLS LAST] ├── estimated rows: 1.00 @@ -62,7 +62,7 @@ abcd query T explain select * from t1 where s like 'abcd' order by s; ---- -Sort +Sort(Single) ├── output columns: [t1.s (#0)] ├── sort keys: [s ASC NULLS LAST] ├── estimated rows: 1.00 diff --git a/tests/sqllogictests/suites/mode/standalone/explain/lazy_read.test b/tests/sqllogictests/suites/mode/standalone/explain/lazy_read.test index 89f4f18f4a908..19bc89cc854dd 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/lazy_read.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/lazy_read.test @@ -19,7 +19,7 @@ RowFetch ├── limit: 2 ├── offset: 0 ├── estimated rows: 0.00 - └── Sort + └── Sort(Single) ├── output columns: [t_lazy.a (#0), t_lazy._row_id (#7)] ├── sort keys: [a DESC NULLS LAST] ├── estimated rows: 0.00 @@ -94,7 +94,7 @@ RowFetch ├── limit: 10 ├── offset: 0 ├── estimated rows: 0.00 - └── Sort + └── Sort(Single) ├── output columns: [t_11831.uid (#0), t_11831.time (#3), t_11831._row_id (#4)] ├── sort keys: [time DESC NULLS LAST] ├── estimated rows: 0.00 @@ -126,7 +126,7 @@ Limit ├── limit: 2 ├── offset: 0 ├── estimated rows: 0.00 -└── Sort +└── Sort(Single) ├── output columns: [t_lazy.a (#0), t_lazy.b (#1), t_lazy.c (#2), t_lazy.d (#3), t_lazy.e (#6)] ├── sort keys: [a DESC NULLS LAST] ├── estimated rows: 0.00 @@ -194,7 +194,7 @@ Limit ├── limit: 1 ├── offset: 0 ├── estimated rows: 0.00 -└── Sort +└── Sort(Single) ├── output columns: [t_lazy.a (#0), x (#7), y (#8)] ├── sort keys: [x ASC NULLS LAST] ├── estimated rows: 0.00 @@ -221,7 +221,7 @@ Limit ├── limit: 1 ├── offset: 0 ├── estimated rows: 0.00 -└── Sort +└── Sort(Single) ├── output columns: [t_lazy.a (#0), t_lazy._row_id (#9), x (#7), y (#8), t_lazy.b (#1)] ├── sort keys: [x ASC NULLS LAST] ├── estimated rows: 0.00 @@ -234,7 +234,7 @@ Limit ├── limit: 2 ├── offset: 0 ├── estimated rows: 0.00 - └── Sort + └── Sort(Single) ├── output columns: [t_lazy.a (#0), t_lazy._row_id (#9), x (#7), y (#8)] ├── sort keys: [y ASC NULLS LAST] ├── estimated rows: 0.00 @@ -251,7 +251,7 @@ Limit ├── limit: 3 ├── offset: 0 ├── estimated rows: 0.00 - └── Sort + └── Sort(Single) ├── output columns: [t_lazy.a (#0), t_lazy._row_id (#9)] ├── sort keys: [a ASC NULLS LAST] ├── estimated rows: 0.00 diff --git a/tests/sqllogictests/suites/mode/standalone/explain/limit.test b/tests/sqllogictests/suites/mode/standalone/explain/limit.test index 697169bc4048b..a77e114390b49 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/limit.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/limit.test @@ -24,11 +24,11 @@ Limit query T explain select * from (select t.number from numbers(10) as t order by number desc) order by number asc ---- -Sort +Sort(Single) ├── output columns: [t.number (#0)] ├── sort keys: [number ASC NULLS LAST] ├── estimated rows: 10.00 -└── Sort +└── Sort(Single) ├── output columns: [t.number (#0)] ├── sort keys: [number DESC NULLS LAST] ├── estimated rows: 10.00 @@ -50,7 +50,7 @@ Limit ├── limit: 9 ├── offset: 0 ├── estimated rows: 8.00 -└── Sort +└── Sort(Single) ├── output columns: [t.number (#0)] ├── sort keys: [number ASC NULLS LAST] ├── estimated rows: 8.00 @@ -59,7 +59,7 @@ Limit ├── limit: 8 ├── offset: 0 ├── estimated rows: 8.00 - └── Sort + └── Sort(Single) ├── output columns: [t.number (#0)] ├── sort keys: [number DESC NULLS LAST] ├── estimated rows: 10.00 @@ -81,7 +81,7 @@ Limit ├── limit: 3 ├── offset: 0 ├── estimated rows: 0.20 -└── Sort +└── Sort(Single) ├── output columns: [t.number (#0)] ├── sort keys: [number DESC NULLS LAST] ├── estimated rows: 0.20 @@ -177,7 +177,7 @@ Limit ├── limit: 1 ├── offset: 0 ├── estimated rows: 1.00 -└── Sort +└── Sort(Single) ├── output columns: [c (#4), count(t1.number) (#1)] ├── sort keys: [count(t1.number) ASC NULLS LAST] ├── estimated rows: 2.00 diff --git a/tests/sqllogictests/suites/mode/standalone/explain/prune_column.test b/tests/sqllogictests/suites/mode/standalone/explain/prune_column.test index 7f88c308160d8..67543ec537e2c 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/prune_column.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/prune_column.test @@ -41,7 +41,7 @@ Limit ├── limit: 1 ├── offset: 0 ├── estimated rows: 0.00 -└── Sort +└── Sort(Single) ├── output columns: [numbers.number (#0)] ├── sort keys: [number ASC NULLS LAST] ├── estimated rows: 0.00 @@ -186,7 +186,7 @@ HashJoin query T explain select name from system.functions order by example ---- -Sort +Sort(Single) ├── output columns: [functions.name (#0), functions.example (#4)] ├── sort keys: [example ASC NULLS LAST] ├── estimated rows: 0.00 diff --git a/tests/sqllogictests/suites/mode/standalone/explain/selectivity/is_not_null.test b/tests/sqllogictests/suites/mode/standalone/explain/selectivity/is_not_null.test index a4c42da235e36..5c41e4a9a5e4c 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/selectivity/is_not_null.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/selectivity/is_not_null.test @@ -50,4 +50,4 @@ Filter └── estimated rows: 2.00 statement ok -drop table t_user; \ No newline at end of file +drop table t_user; diff --git a/tests/sqllogictests/suites/mode/standalone/explain/sort.test b/tests/sqllogictests/suites/mode/standalone/explain/sort.test index dba340cebc735..8ac291342a258 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/sort.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/sort.test @@ -4,7 +4,7 @@ create or replace table t1(a int, b int); query T explain select a from (select * from t1 order by a) as t2 where a > 1; ---- -Sort +Sort(Single) ├── output columns: [t1.a (#0)] ├── sort keys: [a ASC NULLS LAST] ├── estimated rows: 0.00 @@ -25,7 +25,7 @@ Sort query T explain select a from (select * from t1) where a > 1 order by a; ---- -Sort +Sort(Single) ├── output columns: [t1.a (#0)] ├── sort keys: [a ASC NULLS LAST] ├── estimated rows: 0.00 @@ -46,7 +46,7 @@ Sort query T explain select a from (select * from t1) as t2 where t2.a > 1 order by a; ---- -Sort +Sort(Single) ├── output columns: [t1.a (#0)] ├── sort keys: [a ASC NULLS LAST] ├── estimated rows: 0.00 @@ -72,7 +72,7 @@ Limit ├── limit: 1 ├── offset: 0 ├── estimated rows: 0.00 -└── Sort +└── Sort(Single) ├── output columns: [t1.a (#0), a::STRING (#2)] ├── sort keys: [a::STRING ASC NULLS LAST] ├── estimated rows: 0.00 diff --git a/tests/sqllogictests/suites/mode/standalone/explain/window.test b/tests/sqllogictests/suites/mode/standalone/explain/window.test index b035741df72e9..f8bb761689b5e 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/window.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/window.test @@ -16,7 +16,7 @@ CREATE TABLE empsalary (depname string, empno bigint, salary int, enroll_date da query T explain SELECT depname, empno, salary, sum(salary) OVER (PARTITION BY depname ORDER BY empno) FROM empsalary ORDER BY depname, empno ---- -Sort +Sort(Single) ├── output columns: [empsalary.depname (#0), empsalary.empno (#1), empsalary.salary (#2), sum(salary) OVER (PARTITION BY depname ORDER BY empno) (#4)] ├── sort keys: [depname ASC NULLS LAST, empno ASC NULLS LAST] ├── estimated rows: 0.00 @@ -47,9 +47,6 @@ set max_threads=4; statement ok set sort_spilling_memory_ratio = 0; -statement ok -set enable_range_shuffle_sort = 0; - query T explain pipeline SELECT depname, empno, salary, sum(salary) OVER (PARTITION BY depname ORDER BY empno) FROM empsalary ORDER BY depname, empno; ---- @@ -70,8 +67,13 @@ digraph { 13 [ label = "TransformSortMerge" ] 14 [ label = "TransformSortMerge" ] 15 [ label = "TransformSortMerge" ] - 16 [ label = "MultiSortMerge" ] - 17 [ label = "CompoundBlockOperator(Project)" ] + 16 [ label = "KWayMergePartitioner" ] + 17 [ label = "KWayMergeWorker" ] + 18 [ label = "KWayMergeWorker" ] + 19 [ label = "KWayMergeWorker" ] + 20 [ label = "KWayMergeWorker" ] + 21 [ label = "KWayMergeCombiner" ] + 22 [ label = "CompoundBlockOperator(Project)" ] 0 -> 1 [ label = "" ] 1 -> 2 [ label = "" ] 2 -> 3 [ label = "" ] @@ -91,7 +93,15 @@ digraph { 13 -> 16 [ label = "from: 0, to: 1" ] 14 -> 16 [ label = "from: 0, to: 2" ] 15 -> 16 [ label = "from: 0, to: 3" ] - 16 -> 17 [ label = "" ] + 16 -> 17 [ label = "from: 0, to: 0" ] + 16 -> 18 [ label = "from: 1, to: 0" ] + 16 -> 19 [ label = "from: 2, to: 0" ] + 16 -> 20 [ label = "from: 3, to: 0" ] + 17 -> 21 [ label = "from: 0, to: 0" ] + 18 -> 21 [ label = "from: 0, to: 1" ] + 19 -> 21 [ label = "from: 0, to: 2" ] + 20 -> 21 [ label = "from: 0, to: 3" ] + 21 -> 22 [ label = "" ] } @@ -119,8 +129,13 @@ digraph { 13 [ label = "TransformSortMerge" ] 14 [ label = "TransformSortMerge" ] 15 [ label = "TransformSortMerge" ] - 16 [ label = "MultiSortMerge" ] - 17 [ label = "CompoundBlockOperator(Project)" ] + 16 [ label = "KWayMergePartitioner" ] + 17 [ label = "KWayMergeWorker" ] + 18 [ label = "KWayMergeWorker" ] + 19 [ label = "KWayMergeWorker" ] + 20 [ label = "KWayMergeWorker" ] + 21 [ label = "KWayMergeCombiner" ] + 22 [ label = "CompoundBlockOperator(Project)" ] 0 -> 1 [ label = "" ] 1 -> 2 [ label = "" ] 2 -> 3 [ label = "" ] @@ -140,7 +155,15 @@ digraph { 13 -> 16 [ label = "from: 0, to: 1" ] 14 -> 16 [ label = "from: 0, to: 2" ] 15 -> 16 [ label = "from: 0, to: 3" ] - 16 -> 17 [ label = "" ] + 16 -> 17 [ label = "from: 0, to: 0" ] + 16 -> 18 [ label = "from: 1, to: 0" ] + 16 -> 19 [ label = "from: 2, to: 0" ] + 16 -> 20 [ label = "from: 3, to: 0" ] + 17 -> 21 [ label = "from: 0, to: 0" ] + 18 -> 21 [ label = "from: 0, to: 1" ] + 19 -> 21 [ label = "from: 0, to: 2" ] + 20 -> 21 [ label = "from: 0, to: 3" ] + 21 -> 22 [ label = "" ] } @@ -254,7 +277,7 @@ Filter ├── partition by: [] ├── order by: [v] ├── frame: [Range: Preceding(None) ~ CurrentRow] - └── Sort + └── Sort(Single) ├── output columns: [k (#4), v (#5)] ├── sort keys: [v DESC NULLS LAST] ├── estimated rows: 0.00 @@ -335,7 +358,7 @@ Window ├── partition by: [] ├── order by: [b] ├── frame: [Range: Preceding(None) ~ CurrentRow] -└── Sort +└── Sort(Single) ├── output columns: [tbpush.b (#0)] ├── sort keys: [b ASC NULLS LAST] ├── estimated rows: 0.00 @@ -366,7 +389,7 @@ Filter ├── partition by: [] ├── order by: [b] ├── frame: [Range: Preceding(None) ~ CurrentRow] - └── Sort + └── Sort(Single) ├── output columns: [tbpush.b (#0)] ├── sort keys: [b ASC NULLS LAST] ├── estimated rows: 0.00 @@ -393,7 +416,7 @@ Filter ├── partition by: [] ├── order by: [b] ├── frame: [Range: Preceding(None) ~ CurrentRow] - └── Sort + └── Sort(Single) ├── output columns: [tbpush.b (#0)] ├── sort keys: [b ASC NULLS LAST] ├── estimated rows: 0.00 @@ -570,10 +593,15 @@ digraph { 13 [ label = "TransformSortMerge" ] 14 [ label = "TransformSortMerge" ] 15 [ label = "TransformSortMerge" ] - 16 [ label = "MultiSortMerge" ] - 17 [ label = "Transform Window" ] - 18 [ label = "LimitTransform" ] - 19 [ label = "CompoundBlockOperator(Project)" ] + 16 [ label = "KWayMergePartitioner" ] + 17 [ label = "KWayMergeWorker" ] + 18 [ label = "KWayMergeWorker" ] + 19 [ label = "KWayMergeWorker" ] + 20 [ label = "KWayMergeWorker" ] + 21 [ label = "KWayMergeCombiner" ] + 22 [ label = "Transform Window" ] + 23 [ label = "LimitTransform" ] + 24 [ label = "CompoundBlockOperator(Project)" ] 0 -> 1 [ label = "" ] 1 -> 2 [ label = "" ] 2 -> 3 [ label = "" ] @@ -593,9 +621,17 @@ digraph { 13 -> 16 [ label = "from: 0, to: 1" ] 14 -> 16 [ label = "from: 0, to: 2" ] 15 -> 16 [ label = "from: 0, to: 3" ] - 16 -> 17 [ label = "" ] - 17 -> 18 [ label = "" ] - 18 -> 19 [ label = "" ] + 16 -> 17 [ label = "from: 0, to: 0" ] + 16 -> 18 [ label = "from: 1, to: 0" ] + 16 -> 19 [ label = "from: 2, to: 0" ] + 16 -> 20 [ label = "from: 3, to: 0" ] + 17 -> 21 [ label = "from: 0, to: 0" ] + 18 -> 21 [ label = "from: 0, to: 1" ] + 19 -> 21 [ label = "from: 0, to: 2" ] + 20 -> 21 [ label = "from: 0, to: 3" ] + 21 -> 22 [ label = "" ] + 22 -> 23 [ label = "" ] + 23 -> 24 [ label = "" ] } # row fetch with window function(pipeline explain) @@ -621,10 +657,15 @@ digraph { 15 [ label = "TransformSortMergeLimit" ] 16 [ label = "TransformSortMergeLimit" ] 17 [ label = "TransformSortMergeLimit" ] - 18 [ label = "MultiSortMerge" ] - 19 [ label = "LimitTransform" ] - 20 [ label = "TransformRowsFetcher" ] - 21 [ label = "CompoundBlockOperator(Project)" ] + 18 [ label = "KWayMergePartitioner" ] + 19 [ label = "KWayMergeWorker" ] + 20 [ label = "KWayMergeWorker" ] + 21 [ label = "KWayMergeWorker" ] + 22 [ label = "KWayMergeWorker" ] + 23 [ label = "KWayMergeCombiner" ] + 24 [ label = "LimitTransform" ] + 25 [ label = "TransformRowsFetcher" ] + 26 [ label = "CompoundBlockOperator(Project)" ] 0 -> 1 [ label = "" ] 1 -> 2 [ label = "" ] 2 -> 3 [ label = "" ] @@ -646,9 +687,17 @@ digraph { 15 -> 18 [ label = "from: 0, to: 1" ] 16 -> 18 [ label = "from: 0, to: 2" ] 17 -> 18 [ label = "from: 0, to: 3" ] - 18 -> 19 [ label = "" ] - 19 -> 20 [ label = "" ] - 20 -> 21 [ label = "" ] + 18 -> 19 [ label = "from: 0, to: 0" ] + 18 -> 20 [ label = "from: 1, to: 0" ] + 18 -> 21 [ label = "from: 2, to: 0" ] + 18 -> 22 [ label = "from: 3, to: 0" ] + 19 -> 23 [ label = "from: 0, to: 0" ] + 20 -> 23 [ label = "from: 0, to: 1" ] + 21 -> 23 [ label = "from: 0, to: 2" ] + 22 -> 23 [ label = "from: 0, to: 3" ] + 23 -> 24 [ label = "" ] + 24 -> 25 [ label = "" ] + 25 -> 26 [ label = "" ] } # row fetch with window function(plan explain) @@ -664,7 +713,7 @@ RowFetch ├── limit: 3 ├── offset: 0 ├── estimated rows: 0.00 - └── Sort + └── Sort(Single) ├── output columns: [t.a (#0), t.b (#1), t._row_id (#7), sum(a) OVER (PARTITION BY a ORDER BY a DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) (#6)] ├── sort keys: [b ASC NULLS LAST] ├── estimated rows: 0.00 @@ -748,13 +797,18 @@ digraph { 8 [ label = "TransformSortMerge" ] 9 [ label = "TransformSortMerge" ] 10 [ label = "TransformSortMerge" ] - 11 [ label = "MultiSortMerge" ] - 12 [ label = "Transform Window" ] - 13 [ label = "CompoundBlockOperator(Map)" ] - 14 [ label = "Transform Window" ] - 15 [ label = "CompoundBlockOperator(Map)" ] - 16 [ label = "Transform Window" ] - 17 [ label = "CompoundBlockOperator(Project)" ] + 11 [ label = "KWayMergePartitioner" ] + 12 [ label = "KWayMergeWorker" ] + 13 [ label = "KWayMergeWorker" ] + 14 [ label = "KWayMergeWorker" ] + 15 [ label = "KWayMergeWorker" ] + 16 [ label = "KWayMergeCombiner" ] + 17 [ label = "Transform Window" ] + 18 [ label = "CompoundBlockOperator(Map)" ] + 19 [ label = "Transform Window" ] + 20 [ label = "CompoundBlockOperator(Map)" ] + 21 [ label = "Transform Window" ] + 22 [ label = "CompoundBlockOperator(Project)" ] 0 -> 1 [ label = "" ] 1 -> 2 [ label = "" ] 2 -> 3 [ label = "from: 0, to: 0" ] @@ -769,12 +823,20 @@ digraph { 8 -> 11 [ label = "from: 0, to: 1" ] 9 -> 11 [ label = "from: 0, to: 2" ] 10 -> 11 [ label = "from: 0, to: 3" ] - 11 -> 12 [ label = "" ] - 12 -> 13 [ label = "" ] - 13 -> 14 [ label = "" ] - 14 -> 15 [ label = "" ] - 15 -> 16 [ label = "" ] + 11 -> 12 [ label = "from: 0, to: 0" ] + 11 -> 13 [ label = "from: 1, to: 0" ] + 11 -> 14 [ label = "from: 2, to: 0" ] + 11 -> 15 [ label = "from: 3, to: 0" ] + 12 -> 16 [ label = "from: 0, to: 0" ] + 13 -> 16 [ label = "from: 0, to: 1" ] + 14 -> 16 [ label = "from: 0, to: 2" ] + 15 -> 16 [ label = "from: 0, to: 3" ] 16 -> 17 [ label = "" ] + 17 -> 18 [ label = "" ] + 18 -> 19 [ label = "" ] + 19 -> 20 [ label = "" ] + 20 -> 21 [ label = "" ] + 21 -> 22 [ label = "" ] } # same order same partiton by multi window From 90636a530152a3969b25ab2dd2d9add21be3aae9 Mon Sep 17 00:00:00 2001 From: coldWater Date: Fri, 11 Jul 2025 10:45:43 +0800 Subject: [PATCH 59/61] fix --- .../src/pipelines/processors/transforms/sort/bounds.rs | 7 ++----- .../processors/transforms/sort/sort_broadcast.rs | 9 +++++++-- .../02_0000_async_agg_index_base.test | 2 +- 3 files changed, 10 insertions(+), 8 deletions(-) diff --git a/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs b/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs index 4c2de885c8c50..d7450313fb768 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs @@ -48,11 +48,8 @@ impl Bounds { Ok(Bounds(vec![block.get_last_column().clone()])) } - pub fn merge(vector: Vec, batch_rows: usize) -> Result { - let mut vector: Vec<_> = vector - .into_iter() - .filter(|bounds| !bounds.is_empty()) - .collect(); + pub fn merge(mut vector: Vec, batch_rows: usize) -> Result { + debug_assert!(vector.iter().all(|bounds| !bounds.is_empty())); match vector.len() { 0 => Ok(Bounds(vec![])), 1 => Ok(vector.pop().unwrap()), diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_broadcast.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_broadcast.rs index 023460458f6f4..ab597a2665346 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_broadcast.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_broadcast.rs @@ -138,7 +138,9 @@ impl HookTransform for TransformSortBoundBroadcast { let bounds = Bounds::merge::( self.input_data .iter_mut() - .map(|meta| std::mem::take(&mut meta.bounds)) + .filter_map(|meta| { + (!meta.bounds.is_empty()).then(|| std::mem::take(&mut meta.bounds)) + }) .collect(), self.state.batch_rows, )?; @@ -169,7 +171,10 @@ impl HookTransform for TransformSortBoundBroadcast { })) .await?; - let bounds_vec = global.into_iter().map(|meta| meta.bounds).collect(); + let bounds_vec = global + .into_iter() + .filter_map(|meta| (!meta.bounds.is_empty()).then_some(meta.bounds)) + .collect(); self.output_data = Some(SortCollectedMeta { bounds: Bounds::merge::(bounds_vec, self.state.batch_rows)?.dedup::(), ..local diff --git a/tests/sqllogictests/suites/ee/02_ee_aggregating_index/02_0000_async_agg_index_base.test b/tests/sqllogictests/suites/ee/02_ee_aggregating_index/02_0000_async_agg_index_base.test index cb3b7adb2d683..b1d64350dd820 100644 --- a/tests/sqllogictests/suites/ee/02_ee_aggregating_index/02_0000_async_agg_index_base.test +++ b/tests/sqllogictests/suites/ee/02_ee_aggregating_index/02_0000_async_agg_index_base.test @@ -414,7 +414,7 @@ Logout 2 2 5.0 query T explain select event_name, user_id, max(user_id), avg(id) from t where user_id > 1 group by event_name, user_id order by event_name, user_id desc ---- -Sort +Sort(Single) ├── output columns: [max(user_id) (#3), t.event_name (#2), t.user_id (#1), sum(id) / if(count(id) = 0, 1, count(id)) (#6)] ├── sort keys: [event_name ASC NULLS LAST, user_id DESC NULLS LAST] ├── estimated rows: 6.00 From 87ffa7d53042381f821255df9342546cedc2271c Mon Sep 17 00:00:00 2001 From: coldWater Date: Fri, 11 Jul 2025 13:55:03 +0800 Subject: [PATCH 60/61] test --- .../transforms/sort/sort_merge_stream.rs | 119 ++++++++++++++++-- 1 file changed, 111 insertions(+), 8 deletions(-) diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_merge_stream.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_merge_stream.rs index 356710ea455bb..f41519db5c90f 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_merge_stream.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_merge_stream.rs @@ -30,7 +30,7 @@ use databend_common_pipeline_transforms::sort::Merger; use databend_common_pipeline_transforms::sort::Rows; use databend_common_pipeline_transforms::sort::SortedStream; -use crate::pipelines::processors::transforms::SortBound; +use super::SortBound; type Stream = BoundedInputStream<::Rows>; @@ -44,7 +44,7 @@ where A: SortAlgorithm limit: Option, output_data: Option, - bound_index: u32, + cur_index: u32, inner: std::result::Result>, Vec>>, } @@ -78,7 +78,7 @@ where A: SortAlgorithm block_size, limit, output_data: None, - bound_index: 0, + cur_index: 0, inner: Err(streams), }) } @@ -119,8 +119,8 @@ where A: SortAlgorithm + 'static let merger = self.inner.as_mut().ok().unwrap(); if let Some(block) = merger.next_block()? { self.output_data = Some(block.add_meta(Some(SortBound::create( - self.bound_index, - (!merger.is_finished()).then_some(self.bound_index), + self.cur_index, + (!merger.is_finished()).then_some(self.cur_index), )))?); }; Ok(()) @@ -137,7 +137,7 @@ where A: SortAlgorithm + 'static if !merger.is_finished() { return Ok(Event::Sync); } - self.bound_index += 1; + self.cur_index += 1; let merger = std::mem::replace(inner, Err(vec![])).ok().unwrap(); self.inner = Err(merger.streams()); self.inner.as_mut().err().unwrap() @@ -151,7 +151,7 @@ where A: SortAlgorithm + 'static } for stream in streams.iter_mut() { - stream.update_bound_index(self.bound_index); + stream.update_bound_index(self.cur_index); } self.inner = Ok(Merger::create( @@ -233,6 +233,10 @@ impl BoundedInputStream { .expect("require a SortBound"); let bound = self.bound.as_mut().unwrap(); + assert!( + meta.index >= bound.bound_index, + "meta: {meta:?}, bound: {bound:?}", + ); if meta.index == bound.bound_index { bound.more = meta.next.is_some_and(|next| next == meta.index); self.data.take().map(|mut data| { @@ -240,7 +244,6 @@ impl BoundedInputStream { data }) } else { - assert!(meta.index > bound.bound_index); None } } @@ -252,3 +255,103 @@ impl BoundedInputStream { }); } } + +#[cfg(test)] +mod tests { + use std::sync::Arc; + + use databend_common_expression::types::Int32Type; + use databend_common_expression::FromData; + use databend_common_pipeline_core::processors::connect; + use databend_common_pipeline_transforms::sort::SimpleRowsAsc; + + use super::*; + + fn create_block(empty: bool, index: u32, next: Option) -> DataBlock { + let block = DataBlock::new_from_columns(vec![Int32Type::from_data(vec![1, 2, 3])]); + let block = if empty { block.slice(0..0) } else { block }; + block + .add_meta(Some(SortBound::create(index, next))) + .unwrap() + } + + fn create_stream() -> ( + BoundedInputStream>, + Arc, + ) { + let output = OutputPort::create(); + let input = InputPort::create(); + unsafe { + connect(&input, &output); + } + + let stream = BoundedInputStream { + data: None, + input, + remove_order_col: false, + sort_row_offset: 0, + bound: None, + _r: PhantomData, + }; + (stream, output) + } + + #[test] + fn test_bounded_input_stream() { + let (mut stream, output) = create_stream(); + + stream.update_bound_index(0); + + { + let (_, pending) = stream.next().unwrap(); + + assert!(stream.bound.unwrap().more); + assert!(pending); + } + + { + let block = create_block(true, 0, Some(0)); + output.push_data(Ok(block)); + + let (_, pending) = stream.next().unwrap(); + + assert!(stream.bound.unwrap().more); + assert!(pending); + } + + { + let block = create_block(false, 0, Some(0)); + output.push_data(Ok(block)); + + let (data, pending) = stream.next().unwrap(); + assert!(!pending); + let data = data.unwrap(); + assert!(data.0.get_meta().is_none()); + assert_eq!(data.1.len(), 3); + } + + { + let block = create_block(true, 0, Some(1)); + output.push_data(Ok(block)); + + let (data, pending) = stream.next().unwrap(); + + assert!(data.is_none()); + assert!(!stream.bound.unwrap().more); + assert!(pending); + + let block = create_block(false, 1, Some(1)); + output.push_data(Ok(block)); + + let (data, pending) = stream.next().unwrap(); + assert!(data.is_none()); + assert!(!stream.bound.unwrap().more); + assert!(!pending); + + let (data, pending) = stream.next().unwrap(); + assert!(data.is_none()); + assert!(!stream.bound.unwrap().more); + assert!(!pending); + } + } +} From 3670ee1c0d726c3d111bb8a9e5a54135a3ba99eb Mon Sep 17 00:00:00 2001 From: coldWater Date: Fri, 11 Jul 2025 14:56:45 +0800 Subject: [PATCH 61/61] fix --- .../processors/transforms/sort/sort_builder.rs | 1 - .../transforms/sort/sort_merge_stream.rs | 15 ++++++++------- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs index 1dfccce364e86..7f324e9ef28d2 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs @@ -357,7 +357,6 @@ impl Build<'_> { self.output.clone(), self.schema().clone(), self.params.block_size, - self.params.limit, !self.params.output_order_col, )?)) } diff --git a/src/query/service/src/pipelines/processors/transforms/sort/sort_merge_stream.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_merge_stream.rs index f41519db5c90f..fd9a64b060143 100644 --- a/src/query/service/src/pipelines/processors/transforms/sort/sort_merge_stream.rs +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_merge_stream.rs @@ -41,7 +41,6 @@ where A: SortAlgorithm output: Arc, schema: DataSchemaRef, block_size: usize, - limit: Option, output_data: Option, cur_index: u32, @@ -56,7 +55,6 @@ where A: SortAlgorithm output: Arc, schema: DataSchemaRef, block_size: usize, - limit: Option, remove_order_col: bool, ) -> Result { let streams = inputs @@ -76,7 +74,6 @@ where A: SortAlgorithm output, schema, block_size, - limit, output_data: None, cur_index: 0, inner: Err(streams), @@ -145,11 +142,12 @@ where A: SortAlgorithm + 'static Err(streams) => streams, }; - if streams.iter().all(|stream| stream.input.is_finished()) { + if streams.iter().all(|stream| stream.is_finished()) { self.output.finish(); return Ok(Event::Finished); } + log::debug!("create merger cur_index {}", self.cur_index); for stream in streams.iter_mut() { stream.update_bound_index(self.cur_index); } @@ -158,7 +156,7 @@ where A: SortAlgorithm + 'static self.schema.clone(), std::mem::take(streams), self.block_size, - self.limit, + None, )); Ok(Event::Sync) } @@ -211,8 +209,7 @@ impl BoundedInputStream { return Ok(false); } - if self.input.has_data() { - let block = self.input.pull_data().unwrap()?; + if let Some(block) = self.input.pull_data().transpose()? { self.input.set_need_data(); self.data = Some(block); Ok(false) @@ -254,6 +251,10 @@ impl BoundedInputStream { more: true, }); } + + fn is_finished(&self) -> bool { + self.input.is_finished() && self.data.is_none() + } } #[cfg(test)]