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/block.rs b/src/query/expression/src/block.rs index 70db35bb31695..83697beea302a 100644 --- a/src/query/expression/src/block.rs +++ b/src/query/expression/src/block.rs @@ -332,10 +332,34 @@ 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) + } + } + + fn downcast_mut(boxed: &mut BlockMetaInfoPtr) -> Option<&mut Self> { + let boxed = boxed.as_mut() as &mut dyn Any; + boxed.downcast_mut() + } } 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 { @@ -431,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 @@ -702,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/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/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/mod.rs b/src/query/pipeline/transforms/src/processors/transforms/mod.rs index ec6ca0faf96a0..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,12 +38,14 @@ 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::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/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 479d2559c049e..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; @@ -25,7 +26,9 @@ 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::Scalar; use databend_common_expression::SortColumnDescription; pub use simple::*; pub use utils::*; @@ -39,6 +42,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| data_block.get_by_offset(desc.offset).clone()) + .collect::>(); + self.convert(&order_by_cols, data_block.num_rows()) + } } /// Rows can be compared. @@ -82,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/pipeline/transforms/src/processors/transforms/sort/rows/utils.rs b/src/query/pipeline/transforms/src/processors/transforms/sort/rows/utils.rs index da1a63bf2deeb..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 @@ -21,7 +21,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,116 +40,125 @@ pub fn convert_rows( sort_desc: &[SortColumnDescription], data: DataBlock, ) -> Result { - let num_rows = data.num_rows(); - - if sort_desc.len() == 1 { - let sort_type = schema.field(sort_desc[0].offset).data_type(); - let asc = sort_desc[0].asc; - - let offset = sort_desc[0].offset; - let columns = &data.columns()[offset..offset + 1]; - - 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), - } + struct ConvertRowsVisitor<'a> { + schema: DataSchemaRef, + sort_desc: &'a [SortColumnDescription], + data: DataBlock, + } + + impl RowsTypeVisitor for ConvertRowsVisitor<'_> { + 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, + { + let columns = self + .sort_desc + .iter() + .map(|desc| self.data.get_by_offset(desc.offset).to_owned()) + .collect::>(); + + let converter = C::create(self.sort_desc, self.schema.clone())?; + let rows = C::convert(&converter, &columns, self.data.num_rows())?; + Ok(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, + }; + + 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; } 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], + } + + impl RowsTypeVisitor for OrderFieldTypeVisitor<'_> { + type Result = DataType; + 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, + { + R::data_type() } } - DataType::Binary + + assert!(!desc.is_empty()); + let mut visitor = OrderFieldTypeVisitor { + schema: schema.clone().into(), + sort_desc: desc, + }; + + select_row_type(&mut visitor) } 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..ca6babb9696fd --- /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, input_finished: bool) -> 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(self.input.is_finished()) { + 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(self.input.is_finished()) { + 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_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/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/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_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_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 5665787f45543..81b9bb04ccb0e 100644 --- a/src/query/service/src/pipelines/builders/builder_sort.rs +++ b/src/query/service/src/pipelines/builders/builder_sort.rs @@ -12,13 +12,18 @@ // 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; 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; @@ -29,6 +34,8 @@ 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::executor::PhysicalPlan; use databend_common_storage::DataOperator; use databend_common_storages_fuse::TableContext; use databend_storages_common_cache::TempDirManager; @@ -43,39 +50,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 !matches!(sort.after_exchange, Some(true)) { - // 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(), - ) - }); - } - } - } - let plan_schema = sort.output_schema()?; - let sort_desc = sort .order_by .iter() @@ -88,30 +64,67 @@ impl PipelineBuilder { }) }) .collect::>>()?; + let sort_desc = sort_desc.into(); - self.build_sort_pipeline(plan_schema, sort_desc, sort.limit, sort.after_exchange) - } + if sort.step != SortStep::Shuffled { + 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, - after_exchange: Option, - ) -> 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 { self.main_pipeline.try_resize(max_threads)?; } - let builder = SortPipelineBuilder::create(self.ctx.clone(), plan_schema, sort_desc)? - .with_limit(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) + } - match after_exchange { - Some(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::Final => { // Build for the coordinator node. // We only build a `MultiSortMergeTransform`, // as the data is already sorted in each cluster node. @@ -126,19 +139,25 @@ impl PipelineBuilder { .build_merge_sort_pipeline(&mut self.main_pipeline, true) } } - Some(false) => { - // 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::Sample => { + builder.build_sample(&mut self.main_pipeline)?; + self.exchange_injector = TransformSortBuilder::exchange_injector(); + Ok(()) } - 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) + SortStep::Shuffled => { + if matches!(*sort.input, PhysicalPlan::ExchangeSource(_)) { + 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; + } else { + self.build_pipeline(&sort.input)?; + } + + builder.build_bounded_merge_sort(&mut self.main_pipeline) } + SortStep::Route => TransformSortBuilder::add_route(&mut self.main_pipeline), } } } @@ -150,6 +169,8 @@ pub struct SortPipelineBuilder { limit: Option, block_size: usize, remove_order_col_at_last: bool, + enable_loser_tree: bool, + broadcast_id: Option, } impl SortPipelineBuilder { @@ -157,8 +178,11 @@ impl SortPipelineBuilder { ctx: Arc, schema: DataSchemaRef, sort_desc: Arc<[SortColumnDescription]>, + broadcast_id: Option, ) -> 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 +190,8 @@ impl SortPipelineBuilder { limit: None, block_size, remove_order_col_at_last: false, + enable_loser_tree, + broadcast_id, }) } @@ -197,20 +223,73 @@ 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_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) + .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)?)) + })?; + + builder.add_bound_broadcast( + pipeline, + max_block_size, + self.ctx.clone(), + self.broadcast_id.unwrap(), + )?; + + pipeline.add_transform(|input, output| { + Ok(ProcessorPtr::create(builder.build_restore(input, output)?)) + })?; + + pipeline.add_transform(|input, output| { + Ok(ProcessorPtr::create( + builder.build_bound_edge(input, output)?, + )) + })?; + + Ok(()) + } + + 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; - 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 { @@ -238,26 +317,32 @@ 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| { - let builder = TransformSortBuilder::create( - input, - output, + let builder = TransformSortBuilder::new( 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); - Ok(ProcessorPtr::create(builder.build()?)) - })?; + Ok(ProcessorPtr::create(builder.build(input, output)?)) + }) + } + + 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 +354,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 +364,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,8 +374,32 @@ impl SortPipelineBuilder { self.limit, self.sort_desc, self.remove_order_col_at_last, - enable_loser_tree, + self.enable_loser_tree, ) } } + + 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( + builder.build_bounded_merge_sort(inputs_port.clone(), output_port.clone())?, + ); + + 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/builders/builder_window.rs b/src/query/service/src/pipelines/builders/builder_window.rs index 187bb25d7dd77..f4cd950a76d1a 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; @@ -182,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(); @@ -198,7 +199,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::Final => 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/executor/executor_graph.rs b/src/query/service/src/pipelines/executor/executor_graph.rs index eae85cbdd53fa..d3f3bdc43de4d 100644 --- a/src/query/service/src/pipelines/executor/executor_graph.rs +++ b/src/query/service/src/pipelines/executor/executor_graph.rs @@ -797,7 +797,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) ))), } } @@ -863,7 +863,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, @@ -963,7 +963,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/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/mod.rs b/src/query/service/src/pipelines/processors/transforms/mod.rs index 7776ca90e1780..7ad1bfb56df2b 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; @@ -32,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; @@ -45,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; @@ -64,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/sort/bounds.rs b/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs new file mode 100644 index 0000000000000..d7450313fb768 --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/sort/bounds.rs @@ -0,0 +1,318 @@ +// 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::Scalar; +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, Clone, serde::Serialize, serde::Deserialize)] +pub struct Bounds( + // stored in reverse order of Column. + Vec, +); + +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]), + &[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 { + debug_assert!(vector.iter().all(|bounds| !bounds.is_empty())); + 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 => { + let bound = last.index(0).unwrap().to_owned(); + self.0.pop(); + Some(bound) + } + _ => { + let bound = last.index(0).unwrap().to_owned(); + *last = last.slice(1..last.len()); + Some(bound) + } + } + } + + pub fn len(&self) -> usize { + self.0.iter().map(Column::len).sum() + } + + pub fn is_empty(&self) -> bool { + self.0.iter().all(|col| col.len() == 0) + } + + #[allow(dead_code)] + pub fn reduce(&self, n: usize) -> Option { + if n == 0 { + return Some(Self::default()); + } + let total = self.len(); + if n >= total { + return None; + } + + let step = total / 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, + &indices, + 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(), + )]) + } + + pub fn dedup(&self) -> Self { + self.dedup_reduce::(self.len()) + } +} + +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::*; + + fn int32_columns(data: T) -> Vec + where T: IntoIterator> { + data.into_iter().map(Int32Type::from_data).collect() + } + + #[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(int32_columns([vec![6, 7], vec![2, 6, 6], vec![0, 0, 1]])) + ); + } + + { + 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))) + .collect::>>()?; + let bounds = Bounds::merge::>(data, 2)?; + + assert_eq!( + bounds, + Bounds(int32_columns([ + vec![-2], + vec![1, 1], + vec![3, 2], + vec![7, 6], + vec![77, 8] + ])) + ); + } + + 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).unwrap(); + 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(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(int32_columns([vec![7, 1]]))); // 77 8 _7 6 3 2 _1 1 -2 + + let got = bounds.reduce(1).unwrap(); + assert_eq!(got, Bounds(int32_columns([vec![3]]))); // 77 8 7 6 _3 2 1 1 -2 + + 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(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(int32_columns([vec![4, 3, 1]]))); + + 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(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(int32_columns([vec![1]]))); + + Ok(()) + } +} 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 90% 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 9b26a6eedc14d..96fd0a955899e 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 @@ -35,16 +35,14 @@ use databend_common_pipeline_transforms::MergeSort; use databend_common_pipeline_transforms::SortSpillParams; use databend_common_pipeline_transforms::TransformSortMergeLimit; +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::pipelines::processors::transforms::sort::sort_spill::OutputData; 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; - #[derive(Debug)] enum State { /// This state means the processor will collect incoming blocks. @@ -55,14 +53,6 @@ enum State { Finish, } -#[derive(Clone)] -struct Base { - schema: DataSchemaRef, - spiller: Arc, - sort_row_offset: usize, - limit: Option, -} - enum Inner { Collect(Vec), Limit(TransformSortMergeLimit), @@ -102,8 +92,7 @@ where A: SortAlgorithm, C: RowConverter, { - #[allow(clippy::too_many_arguments)] - fn new( + pub(super) fn new( input: Arc, output: Arc, schema: DataSchemaRef, @@ -150,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!() }; @@ -177,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())) @@ -189,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; @@ -306,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 @@ -362,6 +349,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) @@ -432,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) => { - debug_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 { @@ -454,7 +431,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 { @@ -471,8 +448,8 @@ where let Inner::Spill(input_data, spill_sort) = &mut self.inner else { unreachable!() }; - debug_assert!(input_data.is_empty()); - let (block, finish) = spill_sort.on_restore().await?; + assert!(input_data.is_empty()); + 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 new file mode 100644 index 0000000000000..c8a98290e24bc --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/sort/mod.rs @@ -0,0 +1,117 @@ +// 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 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; +use sort_spill::SpillableBlock; + +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_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::*; + +#[derive(Clone)] +struct Base { + schema: DataSchemaRef, + spiller: Arc, + sort_row_offset: usize, + limit: Option, +} + +#[derive(Debug)] +struct SortCollectedMeta { + params: SortSpillParams, + bounds: Bounds, + sequences: Vec>, +} + +local_block_meta_serde!(SortCollectedMeta); + +#[typetag::serde(name = "sort_collected")] +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::() + } +} + +#[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()) + } +} + +#[derive(Debug, Clone, Copy, PartialEq, Eq, serde::Serialize, serde::Deserialize)] +pub struct SortBound { + index: u32, + next: Option, +} + +impl SortBound { + fn create(index: u32, next: Option) -> Box { + debug_assert!(index != u32::MAX); + SortBound { index, next }.boxed() + } +} + +#[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) + } +} 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 new file mode 100644 index 0000000000000..ab597a2665346 --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_broadcast.rs @@ -0,0 +1,204 @@ +// 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::ErrorCode; +use databend_common_exception::Result; +use databend_common_expression::BlockMetaInfoDowncast; +use databend_common_expression::DataBlock; +use databend_common_pipeline_core::processors::Event; +use databend_common_pipeline_core::processors::Processor; +use databend_common_pipeline_transforms::processors::sort::Rows; +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 { + state: SortSampleState, + input_data: Vec, + output_data: Option, + called_on_finish: bool, + + _r: PhantomData, +} + +impl TransformSortBoundBroadcast { + pub fn create( + input: Arc, + output: Arc, + state: SortSampleState, + ) -> Box { + Box::new(HookTransformer::new(input, output, Self { + state, + input_data: Vec::new(), + output_data: None, + called_on_finish: false, + _r: PhantomData, + })) + } +} + +#[derive(Clone)] +pub struct SortSampleState { + ctx: Arc, + broadcast_id: u32, + batch_rows: usize, +} + +impl SortSampleState { + pub fn new(batch_rows: usize, ctx: Arc, broadcast_id: u32) -> SortSampleState { + SortSampleState { + ctx, + broadcast_id, + batch_rows, + } + } + + 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(meta) + .await + .map_err(|_| ErrorCode::TokioError("send sort bounds failed"))?; + sender.close(); + + let receiver = self.ctx.broadcast_sink_receiver(self.broadcast_id); + let mut all = Vec::new(); + while let Ok(r) = receiver.recv().await { + match SortExchangeMeta::downcast_from_err(r) { + Ok(meta) => all.push(meta), + Err(r) => { + debug_assert!(().boxed().equals(&r)) + } + }; + } + Ok(all) + } +} + +#[async_trait::async_trait] +impl HookTransform for TransformSortBoundBroadcast { + const NAME: &'static str = "SortBoundBroadcast"; + + fn on_input(&mut self, mut data: DataBlock) -> Result<()> { + let meta = data + .take_meta() + .and_then(SortCollectedMeta::downcast_from) + .expect("require a SortCollectedMeta"); + self.input_data.push(meta); + Ok(()) + } + + fn on_output(&mut self) -> Result> { + 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 { + if input_finished && !self.called_on_finish { + Some(Event::Async) + } else { + None + } + } + + async fn async_process(&mut self) -> Result<()> { + self.called_on_finish = true; + + let bounds = Bounds::merge::( + self.input_data + .iter_mut() + .filter_map(|meta| { + (!meta.bounds.is_empty()).then(|| 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(()); + } + + let params = self.input_data.first().unwrap().params; + let local = SortCollectedMeta { + params, + bounds, + sequences, + }; + + let global = self + .state + .commit_sample(Some(SortExchangeMeta { + params, + bounds: local.normalize_bounds::(), + })) + .await?; + + 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 + }); + Ok(()) + } +} + +impl SortCollectedMeta { + fn normalize_bounds(&self) -> Bounds { + if self.bounds.len() > 1 { + return self.bounds.dedup::(); + } + + let Some(seq) = self.sequences.get(self.sequences.len() / 2) else { + return Bounds::default(); + }; + + seq.get(seq.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)).dedup::(), + }) + .unwrap_or_default() + } +} 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 new file mode 100644 index 0000000000000..7f324e9ef28d2 --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_builder.rs @@ -0,0 +1,405 @@ +// 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::DataSchemaRef; +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::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; +use databend_common_pipeline_transforms::sort::algorithm::LoserTreeSort; +use databend_common_pipeline_transforms::sort::select_row_type; +use databend_common_pipeline_transforms::sort::utils::add_order_field; +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::MemorySettings; + +use super::*; +use crate::servers::flight::v1::exchange::ExchangeInjector; +use crate::sessions::QueryContext; +use crate::spillers::Spiller; + +enum SortType { + Sort(Arc), + + Collect(Arc), + BoundBroadcast { + input: Arc, + state: SortSampleState, + }, + Restore(Arc), + + BoundedMergeSort(Vec>), +} + +pub struct TransformSortBuilder { + schema: DataSchemaRef, + block_size: usize, + sort_desc: Arc<[SortColumnDescription]>, + order_col_generated: bool, + output_order_col: bool, + memory_settings: MemorySettings, + spiller: Option>, + enable_loser_tree: bool, + limit: Option, +} + +impl TransformSortBuilder { + pub fn new( + schema: DataSchemaRef, + sort_desc: Arc<[SortColumnDescription]>, + block_size: usize, + ) -> Self { + TransformSortBuilder { + block_size, + schema, + sort_desc, + spiller: None, + order_col_generated: false, + output_order_col: false, + enable_loser_tree: false, + limit: None, + memory_settings: MemorySettings::disable_spill(), + } + } + + pub fn with_spiller(mut self, spiller: Arc) -> Self { + self.spiller = Some(spiller); + self + } + + pub fn with_order_column(mut self, generated: bool, output: bool) -> Self { + self.order_col_generated = generated; + self.output_order_col = output; + self + } + + pub fn with_limit(mut self, limit: Option) -> Self { + self.limit = limit; + self + } + + pub fn with_memory_settings(mut self, memory_settings: MemorySettings) -> Self { + self.memory_settings = memory_settings; + self + } + + pub fn with_enable_loser_tree(mut self, enable_loser_tree: bool) -> Self { + self.enable_loser_tree = enable_loser_tree; + self + } + + pub fn build( + &self, + input: Arc, + output: Arc, + ) -> Result> { + self.check(); + + let mut build = Build { + params: self, + output, + typ: Some(SortType::Sort(input)), + }; + + select_row_type(&mut build) + } + + pub fn build_collect( + &self, + input: Arc, + output: Arc, + ) -> Result> { + self.check(); + + let mut build = Build { + params: self, + output, + typ: Some(SortType::Collect(input)), + }; + + select_row_type(&mut build) + } + + pub fn build_bound_broadcast( + &self, + input: Arc, + output: Arc, + state: SortSampleState, + ) -> Result> { + self.check(); + + let mut build = Build { + params: self, + output, + typ: Some(SortType::BoundBroadcast { input, state }), + }; + + select_row_type(&mut build) + } + + pub fn build_restore( + &self, + input: Arc, + output: Arc, + ) -> Result> { + self.check(); + + let mut build = Build { + params: self, + output, + typ: Some(SortType::Restore(input)), + }; + + select_row_type(&mut build) + } + + 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, + output, + typ: Some(SortType::BoundedMergeSort(inputs)), + }; + + select_row_type(&mut build) + } + + fn should_use_sort_limit(&self) -> bool { + self.limit.map(|limit| limit < 10000).unwrap_or_default() + } + + fn check(&self) { + assert_eq!(self.schema.has_field(ORDER_COL_NAME), self.output_order_col) + } + + fn new_base(&self) -> Base { + let schema = self.inner_schema(); + let sort_row_offset = schema.fields().len() - 1; + Base { + sort_row_offset, + schema, + spiller: self.spiller.clone().unwrap(), + limit: self.limit, + } + } + + fn inner_schema(&self) -> DataSchemaRef { + add_order_field(self.schema.clone(), &self.sort_desc) + } + + pub fn add_bound_broadcast( + &self, + pipeline: &mut Pipeline, + batch_rows: usize, + ctx: Arc, + broadcast_id: u32, + ) -> Result<()> { + let state = SortSampleState::new(batch_rows, ctx, broadcast_id); + + pipeline.resize(1, false)?; + pipeline.add_transform(|input, output| { + Ok(ProcessorPtr::create(self.build_bound_broadcast( + input, + output, + state.clone(), + )?)) + }) + } + + 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: Option, + output: Arc, +} + +impl Build<'_> { + 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( + 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().unwrap(), + self.params.output_order_col, + self.params.order_col_generated, + self.params.memory_settings.clone(), + )?)) + } + + 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( + input, + self.output.clone(), + self.params.new_base(), + self.params.sort_desc.clone(), + self.params.block_size, + sort_limit, + self.params.order_col_generated, + self.params.memory_settings.clone(), + )?)) + } + + fn build_sort_restore(&mut self, input: Arc) -> Result> + where A: SortAlgorithm + 'static { + Ok(Box::new(TransformSortRestore::::create( + input, + self.output.clone(), + self.params.new_base(), + self.params.output_order_col, + )?)) + } + + fn build_bound_broadcast( + &mut self, + input: Arc, + state: SortSampleState, + ) -> Result> + where + R: Rows + 'static, + { + Ok(TransformSortBoundBroadcast::::create( + input, + self.output.clone(), + state, + )) + } + + fn build_bounded_merge_sort( + &mut self, + inputs: Vec>, + ) -> Result> + where + A: SortAlgorithm + 'static, + { + Ok(Box::new(BoundedMultiSortMergeProcessor::::new( + inputs, + self.output.clone(), + self.schema().clone(), + self.params.block_size, + !self.params.output_order_col, + )?)) + } +} + +impl RowsTypeVisitor for Build<'_> { + type Result = Result>; + fn schema(&self) -> DataSchemaRef { + self.params.schema.clone() + } + + fn sort_desc(&self) -> &[SortColumnDescription] { + &self.params.sort_desc + } + + fn visit_type(&mut self) -> Self::Result + where + R: Rows + 'static, + C: RowConverter + Send + 'static, + { + let limit_sort = self.params.should_use_sort_limit(); + 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(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::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), + }, + + 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_collect.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_collect.rs new file mode 100644 index 0000000000000..ea29b4d2048c7 --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_collect.rs @@ -0,0 +1,358 @@ +// 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::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 super::MemoryRows; + +enum Inner { + Collect(Vec), + Limit(TransformSortMergeLimit), + Spill(Vec, SortSpill), + None, +} + +pub struct TransformSortCollect { + name: &'static str, + input: Arc, + output: Arc, + output_data: Option, + + max_block_size: usize, + 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, +{ + pub(super) fn new( + input: Arc, + output: Arc, + base: Base, + sort_desc: Arc<[SortColumnDescription]>, + max_block_size: usize, + sort_limit: bool, + order_col_generated: bool, + memory_settings: MemorySettings, + ) -> Result { + 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)), + ), + _ => ("TransformSortMergeCollect", Inner::Collect(vec![])), + }; + Ok(Self { + input, + output, + name, + row_converter, + output_data: None, + sort_desc, + order_col_generated, + base, + inner, + aborting: AtomicBool::new(false), + memory_settings, + max_block_size, + }) + } + + fn generate_order_column(&self, mut block: DataBlock) -> Result<(A::Rows, DataBlock)> { + let rows = self + .row_converter + .convert_data_block(&self.sort_desc, &block)?; + block.add_column(rows.to_column()); + Ok((rows, block)) + } + + 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 = if no_spill { + SortSpillParams { + batch_rows: 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()) + }; + 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, no_spill: bool) { + 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 = if no_spill { + SortSpillParams { + batch_rows: self.max_block_size, + num_merge: num_rows.div_ceil(self.max_block_size).max(2), + } + } 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, no_spill: bool) -> Result<()> { + match &mut self.inner { + 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, no_spill); + 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); + + /// 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, + 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 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(()) + } +} + +#[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(finished)?; + + 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 + .collect_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/sort/sort_exchange_injector.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs new file mode 100644 index 0000000000000..43cff0f1e205a --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_exchange_injector.rs @@ -0,0 +1,111 @@ +// 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::BlockMetaInfoDowncast; +use databend_common_expression::DataBlock; +use databend_common_pipeline_core::Pipeline; +use databend_common_settings::FlightCompression; + +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::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, + _: &Arc, + exchange: &DataExchange, + ) -> Result>> { + match exchange { + DataExchange::Merge(_) | DataExchange::Broadcast(_) => unreachable!(), + DataExchange::ShuffleDataExchange(exchange) => { + Ok(Arc::new(Box::new(SortBoundScatter { + partitions: exchange.destination_ids.len(), + }))) + } + } + } + + fn exchange_sorting(&self) -> Option> { + None + } + + fn apply_merge_serializer( + &self, + _: &MergeExchangeParams, + _: Option, + _: &mut Pipeline, + ) -> Result<()> { + unreachable!() + } + + fn apply_merge_deserializer(&self, _: &MergeExchangeParams, _: &mut Pipeline) -> Result<()> { + unreachable!() + } + + fn apply_shuffle_serializer( + &self, + params: &ShuffleExchangeParams, + compression: Option, + pipeline: &mut Pipeline, + ) -> Result<()> { + DefaultExchangeInjector::create().apply_shuffle_serializer(params, compression, pipeline) + } + + fn apply_shuffle_deserializer( + &self, + params: &ShuffleExchangeParams, + pipeline: &mut Pipeline, + ) -> Result<()> { + DefaultExchangeInjector::create().apply_shuffle_deserializer(params, pipeline) + } +} + +pub struct SortBoundScatter { + partitions: usize, +} + +impl FlightScatter for SortBoundScatter { + fn name(&self) -> &'static str { + "SortBound" + } + + fn execute(&self, data_block: DataBlock) -> Result> { + bound_scatter(data_block, self.partitions) + } +} + +fn bound_scatter(data_block: DataBlock, n: usize) -> Result> { + let meta = *data_block + .get_meta() + .and_then(SortBound::downcast_ref_from) + .unwrap(); + + let empty = data_block.slice(0..0); + let mut result = vec![empty; n]; + result[meta.index as usize % n] = data_block; + + Ok(result) +} 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..fd9a64b060143 --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_merge_stream.rs @@ -0,0 +1,358 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::any::Any; +use std::marker::PhantomData; +use std::sync::Arc; + +use databend_common_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_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 super::SortBound; + +type Stream = BoundedInputStream<::Rows>; + +pub struct BoundedMultiSortMergeProcessor +where A: SortAlgorithm +{ + inputs: Vec>, + output: Arc, + schema: DataSchemaRef, + block_size: usize, + + output_data: Option, + cur_index: u32, + inner: std::result::Result>, Vec>>, +} + +impl BoundedMultiSortMergeProcessor +where A: SortAlgorithm +{ + pub fn new( + inputs: Vec>, + output: Arc, + schema: DataSchemaRef, + block_size: usize, + 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, + output_data: None, + cur_index: 0, + 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<()> { + 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.cur_index, + (!merger.is_finished()).then_some(self.cur_index), + )))?); + }; + 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); + } + 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() + } + Err(streams) => streams, + }; + + 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); + } + + self.inner = Ok(Merger::create( + self.schema.clone(), + std::mem::take(streams), + self.block_size, + None, + )); + Ok(Event::Sync) + } +} + +struct BoundedInputStream { + data: Option, + input: Arc, + remove_order_col: bool, + 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.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); + } + 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 BoundedInputStream { + fn pull(&mut self) -> Result { + if self.data.is_some() { + return Ok(false); + } + + if let Some(block) = self.input.pull_data().transpose()? { + 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 meta = self + .data + .as_ref()? + .get_meta() + .and_then(SortBound::downcast_ref_from) + .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| { + data.take_meta().unwrap(); + data + }) + } else { + None + } + } + + fn update_bound_index(&mut self, bound_index: u32) { + self.bound = Some(Bound { + bound_index, + more: true, + }); + } + + fn is_finished(&self) -> bool { + self.input.is_finished() && self.data.is_none() + } +} + +#[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); + } + } +} 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 new file mode 100644 index 0000000000000..49c084aa9d30a --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_restore.rs @@ -0,0 +1,220 @@ +// 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_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::SortBound; +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`], + /// 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 TransformSortRestore +where A: SortAlgorithm + Send + 'static +{ + pub(super) fn create( + input: Arc, + output: Arc, + base: Base, + output_order_col: bool, + ) -> Result> { + Ok(HookTransformer::new(input, output, Self { + input: Vec::new(), + output: None, + remove_order_col: !output_order_col, + base, + inner: None, + })) + } +} + +#[async_trait::async_trait] +impl HookTransform for TransformSortRestore +where + A: SortAlgorithm + 'static, + A::Rows: 'static, +{ + const NAME: &'static str = "TransformSortRestore"; + + 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"); + self.input.push(meta); + Ok(()) + } + + fn on_output(&mut self) -> Result> { + Ok(self.output.take()) + } + + fn need_process(&self, input_finished: bool) -> Option { + if input_finished && (self.inner.is_some() || !self.input.is_empty()) { + Some(Event::Async) + } else { + None + } + } + + #[async_backtrace::framed] + async fn async_process(&mut self) -> Result<()> { + 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: (bound_index, _), + finish, + } = spill_sort.on_restore().await?; + if let Some(block) = block { + let mut block = block.add_meta(Some(SortBound::create(bound_index, None)))?; + if self.remove_order_col { + block.pop_columns(1); + } + self.output = Some(block); + } + if finish { + self.inner = None; + } + Ok(()) + } +} + +pub 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("SortBoundEdge") + } + + 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.next = None; + 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") + .index; + + let mut output = self.data.replace(incoming).unwrap(); + let output_meta = output + .mut_meta() + .and_then(SortBound::downcast_mut) + .expect("require a SortBound"); + + 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 new file mode 100644 index 0000000000000..ca88b1b7ff0f2 --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/sort/sort_route.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::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; +use databend_common_pipeline_core::processors::Processor; + +use crate::pipelines::processors::transforms::SortBound; + +pub struct TransformSortRoute { + inputs: Vec>, + output: Arc, + + input_data: Vec>, + cur_index: u32, +} + +impl TransformSortRoute { + pub(super) fn new(inputs: Vec>, output: Arc) -> Self { + Self { + input_data: vec![None; inputs.len()], + cur_index: 0, + inputs, + output, + } + } + + fn process(&mut self) -> Result { + 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() + .and_then(SortBound::downcast_from) + .expect("require a SortBound"); + + data.insert((block, meta)).1 + } + }; + + 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) + } +} + +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()?; + + 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/transform_merge_sort/sort_spill.rs b/src/query/service/src/pipelines/processors/transforms/sort/sort_spill.rs similarity index 52% 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 index 53b885619ffb6..bad2fb89f476c 100644 --- 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 @@ -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,7 +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::SortColumnDescription; +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; @@ -38,12 +39,14 @@ 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; use crate::spillers::Location; use crate::spillers::Spiller; -pub struct SortSpill { +pub(super) struct SortSpill { base: Base, step: Step, } @@ -56,20 +59,22 @@ enum Step { struct StepCollect { params: SortSpillParams, sampler: FixedRateSampler, - streams: Vec>, + streams: Vec>>, } 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, - cur_bound: Option, + bounds: Bounds, + cur_bound: Option, + bound_index: i32, - subsequent: Vec>, - current: Vec>, + subsequent: Vec>>, + current: Vec>>, - output_merger: Option>>, + #[expect(clippy::type_complexity)] + output_merger: Option>>>, } impl SortSpill @@ -92,6 +97,34 @@ where A: SortAlgorithm Self { base, step } } + pub fn from_meta(base: Base, meta: SortCollectedMeta) -> Self { + let SortCollectedMeta { + params, + bounds, + sequences, + } = meta; + + let subsequent = sequences + .into_iter() + .filter_map(|seq| { + (!seq.is_empty()).then(|| base.new_stream(Vec::from(seq).into(), None)) + }) + .collect::>(); + debug_assert!(!subsequent.is_empty()); + Self { + base, + step: Step::Sort(StepSort { + params, + bounds, + cur_bound: None, + bound_index: -1, + subsequent, + current: vec![], + output_merger: None, + }), + } + } + pub async fn sort_input_data( &mut self, input_data: Vec, @@ -105,7 +138,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!() }; @@ -119,7 +152,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(_) => (), @@ -137,12 +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((None, 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 { @@ -153,10 +190,41 @@ where A: SortAlgorithm params.num_merge * params.batch_rows } - #[allow(unused)] + #[expect(unused)] 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 sequences = subsequent + .into_iter() + .map(|stream| { + assert!(stream.bound.is_none()); + Vec::from(stream.blocks).into_boxed_slice() + }) + .collect(); + + Ok(SortCollectedMeta { + params, + sequences, + bounds, + }) + } } impl StepCollect { @@ -177,6 +245,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(), @@ -235,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, @@ -243,22 +314,19 @@ impl StepCollect { } } +pub struct OutputData { + pub block: Option, + pub bound: (u32, Option), + pub finish: bool, +} + 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(bound), + None => self.cur_bound = None, + } + self.bound_index += 1; } async fn merge_current(&mut self, base: &Base) -> Result<()> { @@ -299,7 +367,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 => { @@ -308,17 +376,26 @@ impl StepSort { let mut s = self.current.pop().unwrap(); s.restore_first().await?; let block = Some(s.take_next_bounded_block()); + assert!(self.bound_index >= 0); + 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((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?; @@ -339,18 +416,29 @@ 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: (u32::MAX, 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()); + debug_assert!(self.bound_index >= 0); + let bound = (self.bound_index as _, sorted.bound.clone()); if sorted.is_empty() { - return Ok((block, false)); + return Ok(OutputData { + block, + bound, + finish: false, + }); } - block + (block, bound) } else { - None + (None, (u32::MAX, None)) }; while let Some(data) = merger.async_next_block().await? { @@ -366,7 +454,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( @@ -448,13 +541,15 @@ impl Base { fn new_stream( &self, blocks: VecDeque, - bound: Option, - ) -> BoundBlockStream { - BoundBlockStream:: { + bound: Option, + ) -> BoundBlockStream> { + assert!(!blocks.is_empty()); + BoundBlockStream { blocks, bound, sort_row_offset: self.sort_row_offset, spiller: self.spiller.clone(), + _r: Default::default(), } } @@ -466,58 +561,47 @@ 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()); + .collect::>>()?; + Bounds::merge::(ls, batch_rows) + } + } + } - Ok(blocks - .iter() - .rev() - .map(|b| b.get_last_column().clone()) - .collect::>()) + #[expect(dead_code)] + 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> { +impl MemoryRows for Vec> { fn in_memory_rows(&self) -> usize { self.iter().map(|s| s.in_memory_rows()).sum::() } @@ -554,11 +638,11 @@ impl fmt::Debug for FmtMemoryUsage<'_, A> { } } -struct SpillableBlock { +pub struct SpillableBlock { data: Option, rows: usize, location: Option, - domain: Column, + pub(super) domain: Column, processed: usize, } @@ -592,10 +676,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 +702,33 @@ 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 restore(&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 restore(&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, + bound: Option, sort_row_offset: usize, - spiller: Arc, + spiller: S, + _r: PhantomData, } -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 +739,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,15 +752,15 @@ impl SortedStream for BoundBlockStream { } } -impl BoundBlockStream { +impl BoundBlockStream { fn should_include_first(&self) -> bool { let Some(block) = self.blocks.front() else { return false; }; match &self.bound { - Some(bound) => block.domain::().first() <= bound.row(0), None => true, + Some(bound) => block.domain::().first() <= R::scalar_as_item(bound), } } @@ -669,7 +771,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 { @@ -682,6 +784,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 +808,7 @@ impl BoundBlockStream { } let location = block.location.as_ref().unwrap(); - let data = self.spiller.read_spilled_file(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()) @@ -706,21 +825,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 @@ -732,17 +836,48 @@ 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( +pub 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) } @@ -776,7 +911,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))) } @@ -815,23 +950,26 @@ 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; 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; 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; - 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,15 +986,15 @@ mod tests { } async fn run_bound_block_stream( - spiller: Arc, - sort_desc: Arc>, - bound: Column, + spiller: impl Spill + Clone, + 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 bound = Some(R::from_column(&bound)?); + let block = DataBlock::sort(&block, sort_desc, None)?; + let bound = Some(bound); let sort_row_offset = schema.fields().len(); let blocks = vec![ @@ -865,17 +1003,19 @@ 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) }) .collect::>(); - let mut stream = BoundBlockStream:: { + let mut stream = BoundBlockStream:: { blocks, bound, sort_row_offset, spiller: spiller.clone(), + _r: Default::default(), }; let data = stream.take_next_bounded_block(); @@ -887,29 +1027,21 @@ 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 { + let sort_desc = [SortColumnDescription { offset: 0, asc: true, nulls_first: false, - }]); + }]; run_bound_block_stream::>( spiller.clone(), - sort_desc.clone(), - Int32Type::from_data(vec![5]), + &sort_desc, + Scalar::Number(NumberScalar::Int32(5)), 4, Int32Type::from_data(vec![3, 5]), ) @@ -917,8 +1049,8 @@ mod tests { run_bound_block_stream::>( spiller.clone(), - sort_desc.clone(), - Int32Type::from_data(vec![8]), + &sort_desc, + Scalar::Number(NumberScalar::Int32(8)), 4, Int32Type::from_data(vec![3, 5, 7, 7]), ) @@ -926,16 +1058,16 @@ mod tests { } { - let sort_desc = Arc::new(vec![SortColumnDescription { + let sort_desc = [SortColumnDescription { offset: 1, asc: false, nulls_first: false, - }]); + }]; run_bound_block_stream::>( spiller.clone(), - sort_desc.clone(), - StringType::from_data(vec!["f"]), + &sort_desc, + Scalar::String("f".to_string()), 4, StringType::from_data(vec!["w", "h", "g", "f"]), ) @@ -944,4 +1076,353 @@ mod tests { Ok(()) } + + 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(col); + SpillableBlock::new(sliced_block, sort_row_offset) + } + + async fn prepare_test_blocks( + spiller: &impl Spill, + sort_desc: &[SortColumnDescription], + with_spilled: bool, + with_sliced: bool, + ) -> Result<(DataSchemaRef, VecDeque, usize)> { + 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 + blocks.push_back(create_spillable_block( + &block, + 0..2, + &schema, + sort_desc, + sort_row_offset, + )); + + // Second block: 2..5 + 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?; + } + + 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 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); + + // Add both blocks to maintain the order + blocks.push_back(sliced_block); + blocks.push_back(spillable_block); + } + + Ok((schema, blocks, sort_row_offset)) + } + + 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 + 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, stream.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(()) + } + + 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 { + map: Arc::new(Mutex::new(HashMap::new())), + }; + + // Test with ascending Int32 type + { + 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, + Some(Scalar::Number(NumberScalar::Int32(5))), + vec![Int32Type::from_data(vec![3, 5])], + 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, + Some(Scalar::Number(NumberScalar::Int32(8))), + vec![ + Int32Type::from_data(vec![3, 5]), + Int32Type::from_data(vec![7, 7, 8]), + ], + 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, + Some(Scalar::Number(NumberScalar::Int32(7))), + vec![ + Int32Type::from_data(vec![3, 5]), + Int32Type::from_data(vec![7, 7]), + ], + false, + true, + ) + .await?; + + // Test 4: With both spilled and sliced blocks, bound = 10 + run_take_next_bounded_spillable::>( + spiller.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, + true, + ) + .await?; + + // Test 5: With bound = 2 (should return no blocks as all values > 2) + run_take_next_bounded_spillable::>( + spiller.clone(), + &sort_desc, + Some(Scalar::Number(NumberScalar::Int32(2))), + vec![], + 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, + 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, + false, + ) + .await?; + + // Test 7: With no bound (should return all blocks) + run_take_next_bounded_spillable::>( + spiller.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, + false, + ) + .await?; + } + + // Test with descending String type + { + 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, + Some(Scalar::String("f".to_string())), + vec![ + StringType::from_data(vec!["w", "h"]), + StringType::from_data(vec!["g", "f"]), + ], + 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, + 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, + false, + ) + .await?; + + // Test 10: With sliced blocks, bound = "d" (should return blocks with values >= "d") + run_take_next_bounded_spillable::>( + spiller.clone(), + &sort_desc, + 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, + 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, + 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, + true, + ) + .await?; + + // Test 12: With bound = "z" (should return no blocks as all values < "z") + run_take_next_bounded_spillable::>( + spiller.clone(), + &sort_desc, + Some(Scalar::String("z".to_string())), + vec![], + true, + true, + ) + .await?; + } + + 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 restore(&self, location: &Location) -> Result { + match location { + Location::Remote(name) => Ok(self.map.lock().unwrap().get(name).unwrap().clone()), + _ => unreachable!(), + } + } + } } 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 deleted file mode 100644 index 313e69d4412c6..0000000000000 --- a/src/query/service/src/pipelines/processors/transforms/transform_merge_sort/builder.rs +++ /dev/null @@ -1,179 +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::DataSchemaRef; -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_transforms::processors::sort::algorithm::SortAlgorithm; -use databend_common_pipeline_transforms::sort::algorithm::HeapSort; -use databend_common_pipeline_transforms::sort::algorithm::LoserTreeSort; -use databend_common_pipeline_transforms::sort::select_row_type; -use databend_common_pipeline_transforms::sort::utils::add_order_field; -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::MemorySettings; - -use super::TransformSort; -use crate::spillers::Spiller; - -pub struct TransformSortBuilder { - input: Arc, - output: Arc, - schema: DataSchemaRef, - block_size: usize, - sort_desc: Arc<[SortColumnDescription]>, - order_col_generated: bool, - output_order_col: bool, - memory_settings: MemorySettings, - spiller: Arc, - enable_loser_tree: bool, - limit: Option, - processor: Option>>, -} - -impl TransformSortBuilder { - pub fn create( - input: Arc, - output: Arc, - schema: DataSchemaRef, - sort_desc: Arc<[SortColumnDescription]>, - block_size: usize, - spiller: Arc, - ) -> Self { - Self { - input, - output, - block_size, - schema, - sort_desc, - spiller, - order_col_generated: false, - output_order_col: false, - enable_loser_tree: false, - limit: None, - memory_settings: MemorySettings::disable_spill(), - processor: None, - } - } - - pub fn with_order_col_generated(mut self, order_col_generated: bool) -> Self { - self.order_col_generated = order_col_generated; - self - } - - pub fn with_output_order_col(mut self, output_order_col: bool) -> Self { - self.output_order_col = output_order_col; - self - } - - pub fn with_limit(mut self, limit: Option) -> Self { - self.limit = limit; - self - } - - pub fn with_memory_settings(mut self, memory_settings: MemorySettings) -> Self { - self.memory_settings = memory_settings; - self - } - - pub fn with_enable_loser_tree(mut self, enable_loser_tree: bool) -> Self { - self.enable_loser_tree = enable_loser_tree; - self - } - - pub fn build(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) - }); - - select_row_type(&mut self); - self.processor.unwrap() - } - - 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); - 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(), - )?)) - } - - fn build_sort_limit(&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(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(), - )?)) - } -} - -impl RowsTypeVisitor for TransformSortBuilder { - 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, - { - 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>(), - }; - self.processor = Some(processor) - } -} 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) 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/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 937b45289f183..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 @@ -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,12 @@ 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())?; + settings.set_setting("enable_shuffle_sort".to_string(), "0".to_string())?; + if auto_stats { settings.set_optimizer_skip_list("".to_string()) } else { diff --git a/src/query/settings/src/settings_default.rs b/src/query/settings/src/settings_default.rs index 9b32bbbc233fd..762d404ba16d0 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)), }), + ("enable_shuffle_sort", DefaultSettingValue { + value: UserSettingValue::UInt64(1), + desc: "Enable shuffle sort.", + mode: SettingMode::Both, + scope: SettingScope::Both, + range: Some(SettingRange::Numeric(0..=1)), + }), ("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 0e9a7617a1ac9..6f93e4f258e42 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_enable_shuffle_sort(&self) -> Result { + Ok(self.try_get_u64("enable_shuffle_sort")? == 1) + } + pub fn get_group_by_shuffle_mode(&self) -> Result { self.try_get_string("group_by_shuffle_mode") } 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_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..6465e490b29eb 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; @@ -22,6 +24,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; @@ -40,15 +44,42 @@ 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 + Final, // after the exchange plan + + // range shuffle mode + Sample, + Shuffled, + 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::Final => write!(f, "Final"), + SortStep::Sample => write!(f, "Sample"), + SortStep::Shuffled => write!(f, "Shuffled"), + SortStep::Route => write!(f, "Route"), + } + } +} + impl Sort { fn order_col_type(&self, schema: &DataSchema) -> Result { if self.order_by.len() == 1 { @@ -65,40 +96,54 @@ 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 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::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. + 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::Shuffled => 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 matches!(self.after_exchange, Some(false)) { - // 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)) } } @@ -121,7 +166,6 @@ impl PhysicalPlanBuilder { } else { None }; - let input_plan = self.build(s_expr.child(0)?, required).await?; let order_by = sort .items @@ -142,12 +186,20 @@ impl PhysicalPlanBuilder { .map(|v| v.index) .collect::>(); + let sort_step = match sort.after_exchange { + Some(false) => SortStep::Partial, + Some(true) => SortStep::Final, + 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(), - after_exchange: sort.after_exchange, + sort_step, top_n: window.top.map(|top| WindowPartitionTopN { func: match window.func { WindowFuncType::RowNumber => WindowPartitionTopNFunc::RowNumber, @@ -162,13 +214,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_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::Final, + 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, - after_exchange: sort.after_exchange, + step: SortStep::Sample, pre_projection, + 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::Shuffled, + pre_projection: None, + broadcast_id: None, 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. 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 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 9846febbd9819..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_parallel_multi_merge_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 diff --git a/tests/sqllogictests/suites/stage/formats/parquet/read_policy.test b/tests/sqllogictests/suites/stage/formats/parquet/read_policy.test index df51ce3bb1325..075d2f24d28b9 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 @@ -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