diff --git a/src/common/storage/src/lib.rs b/src/common/storage/src/lib.rs index 4777206435530..0b7a781f6aff0 100644 --- a/src/common/storage/src/lib.rs +++ b/src/common/storage/src/lib.rs @@ -30,6 +30,7 @@ //! - Intermediate data generated by query could be stored by temporary operator. #![allow(clippy::uninlined_format_args)] +#![feature(let_chains)] mod config; pub use config::ShareTableConfig; diff --git a/src/common/storage/src/statistics.rs b/src/common/storage/src/statistics.rs index 917d84f46d51d..d22761433a685 100644 --- a/src/common/storage/src/statistics.rs +++ b/src/common/storage/src/statistics.rs @@ -19,6 +19,8 @@ use databend_common_base::base::OrderedFloat; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::types::number::NumberScalar; +use databend_common_expression::types::DataType; +use databend_common_expression::types::NumberDataType; use databend_common_expression::Scalar; pub type F64 = OrderedFloat; @@ -55,6 +57,123 @@ impl Datum { } } + pub fn to_scalar(&self, data_type: &DataType) -> Result> { + let scalar = match self { + Datum::Bool(v) => Some(Scalar::Boolean(*v)), + Datum::Int(v) => match data_type { + DataType::Number(NumberDataType::Int8) => { + Some(Scalar::Number(NumberScalar::Int8(*v as i8))) + } + DataType::Number(NumberDataType::Int16) => { + Some(Scalar::Number(NumberScalar::Int16(*v as i16))) + } + DataType::Number(NumberDataType::Int32) => { + Some(Scalar::Number(NumberScalar::Int32(*v as i32))) + } + DataType::Number(NumberDataType::Int64) => { + Some(Scalar::Number(NumberScalar::Int64(*v))) + } + DataType::Number(NumberDataType::UInt8) => { + if *v > 0 { + Some(Scalar::Number(NumberScalar::UInt8(*v as u8))) + } else { + None + } + } + DataType::Number(NumberDataType::UInt16) => { + if *v > 0 { + Some(Scalar::Number(NumberScalar::UInt16(*v as u16))) + } else { + None + } + } + DataType::Number(NumberDataType::UInt32) => { + if *v > 0 { + Some(Scalar::Number(NumberScalar::UInt32(*v as u32))) + } else { + None + } + } + DataType::Number(NumberDataType::UInt64) => { + if *v > 0 { + Some(Scalar::Number(NumberScalar::UInt64(*v as u64))) + } else { + None + } + } + _ => None, + }, + Datum::UInt(v) => match data_type { + DataType::Number(NumberDataType::Int8) => { + if *v <= i8::MAX as u64 { + Some(Scalar::Number(NumberScalar::Int8(*v as i8))) + } else { + None + } + } + DataType::Number(NumberDataType::Int16) => { + if *v <= i16::MAX as u64 { + Some(Scalar::Number(NumberScalar::Int16(*v as i16))) + } else { + None + } + } + DataType::Number(NumberDataType::Int32) => { + if *v <= i32::MAX as u64 { + Some(Scalar::Number(NumberScalar::Int32(*v as i32))) + } else { + None + } + } + DataType::Number(NumberDataType::Int64) => { + if *v <= i64::MAX as u64 { + Some(Scalar::Number(NumberScalar::Int64(*v as i64))) + } else { + None + } + } + DataType::Number(NumberDataType::UInt8) => { + Some(Scalar::Number(NumberScalar::UInt8(*v as u8))) + } + DataType::Number(NumberDataType::UInt16) => { + Some(Scalar::Number(NumberScalar::UInt16(*v as u16))) + } + DataType::Number(NumberDataType::UInt32) => { + Some(Scalar::Number(NumberScalar::UInt32(*v as u32))) + } + DataType::Number(NumberDataType::UInt64) => { + Some(Scalar::Number(NumberScalar::UInt64(*v))) + } + _ => None, + }, + Datum::Float(v) => match data_type { + DataType::Number(NumberDataType::Float32) => { + if v.into_inner() <= f32::MAX as f64 { + Some(Scalar::Number(NumberScalar::Float32(OrderedFloat::from( + v.into_inner() as f32, + )))) + } else { + None + } + } + DataType::Number(NumberDataType::Float64) => { + Some(Scalar::Number(NumberScalar::Float64(*v))) + } + _ => None, + }, + Datum::Bytes(v) => match data_type { + DataType::String => { + let s = String::from_utf8(v.clone())?; + Some(Scalar::String(s)) + } + DataType::Binary => Some(Scalar::Binary(v.clone())), + _ => None, + }, + }; + + Ok(scalar) + } + pub fn is_bytes(&self) -> bool { matches!(self, Datum::Bytes(_)) } @@ -109,6 +228,84 @@ impl Datum { _ => None, } } + + pub fn sub(x: &Datum, y: &Datum) -> Option { + match (x, y) { + (Datum::Int(x), Datum::Int(y)) => Some(Datum::Int(x - y)), + (Datum::UInt(x), Datum::UInt(y)) => Some(Datum::UInt(x - y)), + (Datum::Float(x), Datum::Float(y)) => { + Some(Datum::Float(F64::from(x.into_inner() - y.into_inner()))) + } + _ => None, + } + } + + pub fn add(&self, other: &Datum) -> Option { + match (self, other) { + (Datum::Int(x), Datum::Int(y)) => Some(Datum::Int(x + y)), + (Datum::UInt(x), Datum::UInt(y)) => Some(Datum::UInt(x + y)), + (Datum::Float(x), Datum::Float(y)) => { + Some(Datum::Float(F64::from(x.into_inner() + y.into_inner()))) + } + _ => None, + } + } + + pub fn div(x: &Datum, y: &Datum) -> Option { + match (x, y) { + (Datum::Int(x), Datum::Int(y)) => { + if *y == 0 { + return None; + } + Some(Datum::Int(x / y)) + } + (Datum::UInt(x), Datum::UInt(y)) => { + if *y == 0 { + return None; + } + Some(Datum::UInt(x / y)) + } + (Datum::Float(x), Datum::Float(y)) => { + if y.into_inner() == 0.0 { + return None; + } + Some(Datum::Float(F64::from(x.into_inner() / y.into_inner()))) + } + _ => None, + } + } + + pub fn build_range_info( + min: Datum, + max: Datum, + num_segments: usize, + data_type: &DataType, + ) -> Result>> { + let mut result = Vec::with_capacity(num_segments); + let num_segments_datum = match min { + Datum::Int(_) => Datum::Int(num_segments as i64), + Datum::UInt(_) => Datum::UInt(num_segments as u64), + Datum::Float(_) => Datum::Float(OrderedFloat::from(num_segments as f64)), + _ => return Ok(None), + }; + if let Some(range) = Self::sub(&max, &min) + && let Some(step) = Self::div(&range, &num_segments_datum) + { + let mut start = min; + for _ in 0..num_segments { + let end = Self::add(&start, &step).unwrap(); + if let Some(start) = start.to_scalar(data_type)? + && let Some(end) = end.to_scalar(data_type)? + { + result.push((start, end)); + } else { + return Ok(None); + } + start = end; + } + } + Ok(Some(result)) + } } impl Display for Datum { diff --git a/src/query/catalog/src/runtime_filter_info.rs b/src/query/catalog/src/runtime_filter_info.rs index 7c29f193879f0..c09f0472a8533 100644 --- a/src/query/catalog/src/runtime_filter_info.rs +++ b/src/query/catalog/src/runtime_filter_info.rs @@ -12,54 +12,85 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::sync::atomic::AtomicU64; +use std::sync::atomic::Ordering; +use std::sync::Arc; + use databend_common_expression::Expr; use xorf::BinaryFuse16; #[derive(Clone, Debug, Default)] pub struct RuntimeFilterInfo { - inlist: Vec>, - min_max: Vec>, - bloom: Vec<(String, BinaryFuse16)>, + min_max: Vec<(String, Expr)>, + inlist: Vec<(String, Expr)>, + bloom: Vec<(String, Arc)>, } impl RuntimeFilterInfo { - pub fn add_inlist(&mut self, expr: Expr) { - self.inlist.push(expr); + pub fn add_min_max(&mut self, filter: (String, Expr)) { + self.min_max.push(filter); } - pub fn add_bloom(&mut self, bloom: (String, BinaryFuse16)) { - self.bloom.push(bloom); + pub fn add_inlist(&mut self, filter: (String, Expr)) { + self.inlist.push(filter); } - pub fn add_min_max(&mut self, expr: Expr) { - self.min_max.push(expr); + pub fn add_bloom(&mut self, filter: (String, Arc)) { + self.bloom.push(filter); } - pub fn get_inlist(&self) -> &Vec> { - &self.inlist + pub fn get_min_max(&self) -> &Vec<(String, Expr)> { + &self.min_max } - pub fn get_bloom(&self) -> &Vec<(String, BinaryFuse16)> { - &self.bloom + pub fn get_inlist(&self) -> &Vec<(String, Expr)> { + &self.inlist } - pub fn get_min_max(&self) -> &Vec> { - &self.min_max + pub fn get_bloom(&self) -> &Vec<(String, Arc)> { + &self.bloom } - pub fn blooms(self) -> Vec<(String, BinaryFuse16)> { - self.bloom + pub fn min_maxs(self) -> Vec<(String, Expr)> { + self.min_max } - pub fn inlists(self) -> Vec> { + pub fn inlists(self) -> Vec<(String, Expr)> { self.inlist } - pub fn min_maxs(self) -> Vec> { - self.min_max + pub fn blooms(self) -> Vec<(String, Arc)> { + self.bloom } pub fn is_empty(&self) -> bool { self.inlist.is_empty() && self.bloom.is_empty() && self.min_max.is_empty() } } + +#[derive(Debug, Default)] +pub struct HashJoinProbeStatistics { + // Statistics for runtime filter, the `num_rows` indicates the number of valid rows in probe side. + // the `num_hash_matched_rows` indicates the number of keys which matched by hash. + pub num_rows: AtomicU64, + pub num_hash_matched_rows: AtomicU64, +} + +impl HashJoinProbeStatistics { + pub fn increment_num_rows(&self, num_rows: u64) { + self.num_rows.fetch_add(num_rows, Ordering::AcqRel); + } + + pub fn increment_num_hash_matched_rows(&self, num_hash_matched_rows: u64) { + self.num_hash_matched_rows + .fetch_add(num_hash_matched_rows, Ordering::AcqRel); + } + + // Check whether to use runtime filter in table scan. + pub fn prefer_runtime_filter(&self) -> bool { + // If the number of valid rows in probe side is less than 1/2 of the number + // of rows which matched by hash, we prefer to use runtime filter. + self.num_hash_matched_rows.load(Ordering::Acquire) * 2 + < self.num_rows.load(Ordering::Acquire) + } +} diff --git a/src/query/catalog/src/table_context.rs b/src/query/catalog/src/table_context.rs index e7a5e74f60e6e..11171e37d19ca 100644 --- a/src/query/catalog/src/table_context.rs +++ b/src/query/catalog/src/table_context.rs @@ -74,6 +74,7 @@ use crate::plan::DataSourcePlan; use crate::plan::PartInfoPtr; use crate::plan::Partitions; use crate::query_kind::QueryKind; +use crate::runtime_filter_info::HashJoinProbeStatistics; use crate::runtime_filter_info::RuntimeFilterInfo; use crate::statistics::data_cache_statistics::DataCacheMetrics; use crate::table::Table; @@ -322,17 +323,29 @@ pub trait TableContext: Send + Sync { fn set_runtime_filter(&self, filters: (usize, RuntimeFilterInfo)); + fn set_runtime_filter_columns(&self, table_index: usize, columns: Vec<(usize, String)>); + + fn get_runtime_filter_columns(&self, table_index: usize) -> Vec<(usize, String)>; + + fn set_hash_join_probe_statistics( + &self, + join_id: usize, + statistics: Arc, + ); + + fn get_hash_join_probe_statistics(&self, join_id: usize) -> Arc; + fn clear_runtime_filter(&self); fn set_merge_into_join(&self, join: MergeIntoJoin); fn get_merge_into_join(&self) -> MergeIntoJoin; - fn get_bloom_runtime_filter_with_id(&self, id: usize) -> Vec<(String, BinaryFuse16)>; + fn get_bloom_runtime_filter_with_id(&self, id: usize) -> Vec<(String, Arc)>; - fn get_inlist_runtime_filter_with_id(&self, id: usize) -> Vec>; + fn get_inlist_runtime_filter_with_id(&self, id: usize) -> Vec<(String, Expr)>; - fn get_min_max_runtime_filter_with_id(&self, id: usize) -> Vec>; + fn get_min_max_runtime_filter_with_id(&self, id: usize) -> Vec<(String, Expr)>; fn has_bloom_runtime_filters(&self, id: usize) -> bool; fn txn_mgr(&self) -> TxnManagerRef; diff --git a/src/query/expression/src/block.rs b/src/query/expression/src/block.rs index 47715b137db7c..633ac4fba1799 100644 --- a/src/query/expression/src/block.rs +++ b/src/query/expression/src/block.rs @@ -425,6 +425,11 @@ impl DataBlock { }) } + #[inline] + pub fn into_columns(self) -> Vec { + self.columns + } + #[inline] pub fn add_meta(self, meta: Option) -> Result { if self.meta.is_some() { diff --git a/src/query/expression/src/expression.rs b/src/query/expression/src/expression.rs index 92c84c7ac540c..8427786c79c5f 100644 --- a/src/query/expression/src/expression.rs +++ b/src/query/expression/src/expression.rs @@ -428,12 +428,6 @@ impl Expr { } } - pub fn runtime_filter_supported_types(&self) -> bool { - self.data_type().remove_nullable().is_numeric() - || self.data_type().remove_nullable().is_string() - || self.data_type().remove_nullable().is_date() - } - pub fn data_type(&self) -> &DataType { match self { Expr::Constant { data_type, .. } => data_type, @@ -707,6 +701,21 @@ impl Expr { Expr::LambdaFunctionCall { args, .. } => args.iter().any(Expr::contains_column_ref), } } + + pub fn column_id(column_expr: &Expr) -> Option { + if let Expr::ColumnRef { id, .. } = column_expr { + Some(id.to_string()) + } else if let Expr::Cast { + expr, dest_type, .. + } = column_expr + && let Expr::ColumnRef { id, .. } = expr.as_ref() + && dest_type.is_nullable() + { + Some(id.to_string()) + } else { + None + } + } } impl RemoteExpr { diff --git a/src/query/pipeline/transforms/src/processors/transforms/transform_async.rs b/src/query/pipeline/transforms/src/processors/transforms/transform_async.rs index 8a27d0774ec2d..3f11fe3b7365b 100644 --- a/src/query/pipeline/transforms/src/processors/transforms/transform_async.rs +++ b/src/query/pipeline/transforms/src/processors/transforms/transform_async.rs @@ -13,6 +13,7 @@ // limitations under the License. use std::any::Any; +use std::collections::VecDeque; use std::sync::Arc; use databend_common_exception::Result; @@ -48,8 +49,11 @@ pub struct AsyncTransformer { called_on_start: bool, called_on_finish: bool, - input_data: Option, - output_data: Option, + + batch_size: Option, + num_input_rows: usize, + input_data_blocks: Vec, + output_data_blocks: VecDeque, } impl AsyncTransformer { @@ -58,8 +62,29 @@ impl AsyncTransformer { input, output, transform: inner, - input_data: None, - output_data: None, + batch_size: None, + num_input_rows: 0, + input_data_blocks: Vec::new(), + output_data_blocks: VecDeque::new(), + called_on_start: false, + called_on_finish: false, + }) + } + + pub fn create_with_batch_size( + input: Arc, + output: Arc, + batch_size: usize, + inner: T, + ) -> Box { + Box::new(Self { + input, + output, + transform: inner, + batch_size: Some(batch_size * 65536), + num_input_rows: 0, + input_data_blocks: Vec::new(), + output_data_blocks: VecDeque::new(), called_on_start: false, called_on_finish: false, }) @@ -85,11 +110,15 @@ impl Processor for AsyncTransformer { true => self.finish_input(), false if !self.output.can_push() => self.not_need_data(), false => { - if let Some(data) = self.output_data.take() { + if let Some(data) = self.output_data_blocks.pop_front() { self.output.push_data(Ok(data)); } - if self.input_data.is_some() { + if let Some(batch_size) = self.batch_size { + if self.num_input_rows >= batch_size { + return Ok(Event::Async); + } + } else if !self.input_data_blocks.is_empty() { return Ok(Event::Async); } @@ -106,9 +135,21 @@ impl Processor for AsyncTransformer { return Ok(()); } - if let Some(data_block) = self.input_data.take() { - let data_block = self.transform.transform(data_block).await?; - self.output_data = Some(data_block); + if !self.input_data_blocks.is_empty() { + let input_data_blocks = std::mem::take(&mut self.input_data_blocks); + if self.batch_size.is_some() { + let data_block = self + .transform + .transform(DataBlock::concat(&input_data_blocks)?) + .await?; + self.output_data_blocks.push_back(data_block); + } else { + for data_block in input_data_blocks { + let data_block = self.transform.transform(data_block).await?; + self.output_data_blocks.push_back(data_block); + } + } + self.num_input_rows = 0; return Ok(()); } @@ -124,8 +165,16 @@ impl Processor for AsyncTransformer { impl AsyncTransformer { fn pull_data(&mut self) -> Result { if self.input.has_data() { - self.input_data = Some(self.input.pull_data().unwrap()?); - return Ok(Event::Async); + let input_data = self.input.pull_data().unwrap()?; + self.num_input_rows += input_data.num_rows(); + self.input_data_blocks.push(input_data); + if let Some(batch_size) = self.batch_size { + if self.num_input_rows >= batch_size { + return Ok(Event::Async); + } + } else { + return Ok(Event::Async); + } } if self.input.is_finished() { diff --git a/src/query/service/src/interpreters/interpreter_copy_into_table.rs b/src/query/service/src/interpreters/interpreter_copy_into_table.rs index e03d5500c87ce..61aaf40f64abf 100644 --- a/src/query/service/src/interpreters/interpreter_copy_into_table.rs +++ b/src/query/service/src/interpreters/interpreter_copy_into_table.rs @@ -142,6 +142,7 @@ impl CopyIntoTableInterpreter { table_index: None, internal_column: None, source: Box::new(data_source_plan), + runtime_filter_columns: vec![], }))), None, ) diff --git a/src/query/service/src/pipelines/builders/builder_exchange.rs b/src/query/service/src/pipelines/builders/builder_exchange.rs index 6c27b81ae366e..40555712ab47b 100644 --- a/src/query/service/src/pipelines/builders/builder_exchange.rs +++ b/src/query/service/src/pipelines/builders/builder_exchange.rs @@ -25,6 +25,7 @@ impl PipelineBuilder { &exchange_source.query_id, exchange_source.source_fragment_id, self.exchange_injector.clone(), + self.runtime_filter_hash_join_state.clone(), )?; // add profile diff --git a/src/query/service/src/pipelines/builders/builder_join.rs b/src/query/service/src/pipelines/builders/builder_join.rs index fd5ef8c2f906a..07014fda7386e 100644 --- a/src/query/service/src/pipelines/builders/builder_join.rs +++ b/src/query/service/src/pipelines/builders/builder_join.rs @@ -12,15 +12,19 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::HashMap; use std::sync::Arc; use databend_common_base::base::tokio::sync::Barrier; +use databend_common_catalog::table_context::TableContext; use databend_common_exception::Result; use databend_common_pipeline_core::processors::ProcessorPtr; use databend_common_pipeline_sinks::Sinker; use databend_common_sql::executor::physical_plans::HashJoin; use databend_common_sql::executor::physical_plans::MaterializedCte; use databend_common_sql::executor::physical_plans::RangeJoin; +use databend_common_sql::executor::physical_plans::RuntimeFilterSink; +use databend_common_sql::executor::physical_plans::RuntimeFilterSource; use databend_common_sql::executor::PhysicalPlan; use databend_common_sql::ColumnBinding; use databend_common_sql::IndexType; @@ -34,6 +38,8 @@ use crate::pipelines::processors::transforms::MaterializedCteSink; use crate::pipelines::processors::transforms::MaterializedCteState; use crate::pipelines::processors::transforms::TransformHashJoinBuild; use crate::pipelines::processors::transforms::TransformHashJoinProbe; +use crate::pipelines::processors::transforms::TransformRuntimeFilterSink; +use crate::pipelines::processors::transforms::TransformRuntimeFilterSource; use crate::pipelines::processors::HashJoinDesc; use crate::pipelines::processors::HashJoinState; use crate::pipelines::PipelineBuilder; @@ -80,6 +86,7 @@ impl PipelineBuilder { right_side_builder.cte_state = self.cte_state.clone(); right_side_builder.cte_scan_offsets = self.cte_scan_offsets.clone(); right_side_builder.hash_join_states = self.hash_join_states.clone(); + right_side_builder.runtime_filter_columns = self.runtime_filter_columns.clone(); let mut right_res = right_side_builder.finalize(&range_join.right)?; right_res.main_pipeline.add_sink(|input| { @@ -109,6 +116,9 @@ impl PipelineBuilder { self.hash_join_states .insert(build_cache_index, state.clone()); } + if let Some(runtime_filter) = &join.runtime_filter { + self.expand_runtime_filter_pipeline(runtime_filter, state.clone())?; + } self.expand_build_side_pipeline(&join.build, join, state.clone())?; self.build_join_probe(join, state)?; @@ -123,7 +133,7 @@ impl PipelineBuilder { merge_into_is_distributed: bool, enable_merge_into_optimization: bool, ) -> Result> { - HashJoinState::try_create( + let hash_join_state = HashJoinState::create( self.ctx.clone(), join.build.output_schema()?, &join.build_projections, @@ -132,7 +142,13 @@ impl PipelineBuilder { merge_into_is_distributed, enable_merge_into_optimization, join.build_side_cache_info.clone(), - ) + join.runtime_filter_source_fields.clone(), + )?; + self.ctx.set_hash_join_probe_statistics( + join.hash_join_id, + hash_join_state.probe_statistics.clone(), + ); + Ok(hash_join_state) } fn expand_build_side_pipeline( @@ -151,6 +167,7 @@ impl PipelineBuilder { build_side_builder.cte_state = self.cte_state.clone(); build_side_builder.cte_scan_offsets = self.cte_scan_offsets.clone(); build_side_builder.hash_join_states = self.hash_join_states.clone(); + build_side_builder.runtime_filter_columns = self.runtime_filter_columns.clone(); let mut build_res = build_side_builder.finalize(build)?; assert!(build_res.main_pipeline.is_pulling_pipeline()?); @@ -170,10 +187,6 @@ impl PipelineBuilder { build_state.clone(), )?)) }; - // for distributed merge into when source as build side. - if hash_join_plan.need_hold_hash_table { - self.join_state = Some(build_state.clone()) - } build_res.main_pipeline.add_sink(create_sink_processor)?; self.pipelines.push(build_res.main_pipeline.finalize()); @@ -181,6 +194,26 @@ impl PipelineBuilder { Ok(()) } + fn expand_runtime_filter_pipeline( + &mut self, + runtime_filter: &PhysicalPlan, + hash_join_state: Arc, + ) -> Result<()> { + let context = QueryContext::create_from(self.ctx.clone()); + let mut builder = PipelineBuilder::create( + self.func_ctx.clone(), + self.settings.clone(), + context, + self.main_pipeline.get_scopes(), + ); + builder.runtime_filter_hash_join_state = Some(hash_join_state); + let build_res = builder.finalize(runtime_filter)?; + + self.pipelines.push(build_res.main_pipeline.finalize()); + self.pipelines.extend(build_res.sources_pipelines); + Ok(()) + } + fn build_join_probe(&mut self, join: &HashJoin, state: Arc) -> Result<()> { self.build_pipeline(&join.probe)?; @@ -258,6 +291,7 @@ impl PipelineBuilder { materialized_side_builder.cte_state = self.cte_state.clone(); materialized_side_builder.cte_scan_offsets = self.cte_scan_offsets.clone(); materialized_side_builder.hash_join_states = self.hash_join_states.clone(); + materialized_side_builder.runtime_filter_columns = self.runtime_filter_columns.clone(); let mut materialized_side_pipeline = materialized_side_builder.finalize(materialized_side)?; assert!( @@ -287,4 +321,53 @@ impl PipelineBuilder { .extend(materialized_side_pipeline.sources_pipelines); Ok(()) } + + pub(crate) fn build_runtime_filter_source( + &mut self, + _runtime_filter_source: &RuntimeFilterSource, + ) -> Result<()> { + let node_id = self.ctx.get_cluster().local_id.clone(); + let hash_join_state = self.runtime_filter_hash_join_state.clone().unwrap(); + self.main_pipeline.add_source( + |output| { + TransformRuntimeFilterSource::create( + output, + node_id.clone(), + hash_join_state.clone(), + ) + }, + 1, + ) + } + + pub(crate) fn build_runtime_filter_sink( + &mut self, + runtime_filter_sink: &RuntimeFilterSink, + ) -> Result<()> { + self.build_pipeline(&runtime_filter_sink.input)?; + self.main_pipeline.try_resize(1)?; + + let local_id = self.ctx.get_cluster().local_id.clone(); + let num_cluster_nodes = self.ctx.get_cluster().nodes.len(); + let mut is_collected = self + .ctx + .get_cluster() + .nodes + .iter() + .map(|node| (node.id.clone(), false)) + .collect::>(); + is_collected.insert(local_id, true); + + let hash_join_state = self.runtime_filter_hash_join_state.clone().unwrap(); + let create_sink_processor = |input| { + TransformRuntimeFilterSink::create( + input, + hash_join_state.clone(), + num_cluster_nodes, + is_collected.clone(), + ) + }; + + self.main_pipeline.add_sink(create_sink_processor) + } } diff --git a/src/query/service/src/pipelines/builders/builder_scan.rs b/src/query/service/src/pipelines/builders/builder_scan.rs index 2ba6f9ce135ef..d514838c1b30a 100644 --- a/src/query/service/src/pipelines/builders/builder_scan.rs +++ b/src/query/service/src/pipelines/builders/builder_scan.rs @@ -40,6 +40,11 @@ use crate::pipelines::PipelineBuilder; impl PipelineBuilder { pub(crate) fn build_table_scan(&mut self, scan: &TableScan) -> Result<()> { let table = self.ctx.build_table_from_source_plan(&scan.source)?; + if let Some(table_index) = scan.table_index { + let runtime_filter_columns = scan.runtime_filter_columns.clone(); + self.ctx + .set_runtime_filter_columns(table_index, runtime_filter_columns); + } self.ctx.set_partitions(scan.source.parts.clone())?; table.read_data( self.ctx.clone(), diff --git a/src/query/service/src/pipelines/builders/builder_union_all.rs b/src/query/service/src/pipelines/builders/builder_union_all.rs index d0584e311f4ff..e494eadfe92d6 100644 --- a/src/query/service/src/pipelines/builders/builder_union_all.rs +++ b/src/query/service/src/pipelines/builders/builder_union_all.rs @@ -81,6 +81,7 @@ impl PipelineBuilder { pipeline_builder.cte_state = self.cte_state.clone(); pipeline_builder.cte_scan_offsets = self.cte_scan_offsets.clone(); pipeline_builder.hash_join_states = self.hash_join_states.clone(); + pipeline_builder.runtime_filter_columns = self.runtime_filter_columns.clone(); let mut build_res = pipeline_builder.finalize(input)?; diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs index 653324ffbdab3..f383fb34b13ec 100644 --- a/src/query/service/src/pipelines/pipeline_builder.rs +++ b/src/query/service/src/pipelines/pipeline_builder.rs @@ -16,6 +16,7 @@ use std::collections::HashMap; use std::sync::Arc; use databend_common_base::runtime::profile::ProfileLabel; +use databend_common_catalog::runtime_filter_info::HashJoinProbeStatistics; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::DataField; @@ -57,6 +58,8 @@ pub struct PipelineBuilder { pub(crate) exchange_injector: Arc, pub hash_join_states: HashMap>, + pub runtime_filter_columns: HashMap)>>, + pub runtime_filter_hash_join_state: Option>, pub r_cte_scan_interpreters: Vec, pub(crate) is_exchange_neighbor: bool, @@ -83,6 +86,8 @@ impl PipelineBuilder { hash_join_states: HashMap::new(), r_cte_scan_interpreters: vec![], is_exchange_neighbor: false, + runtime_filter_columns: HashMap::new(), + runtime_filter_hash_join_state: None, } } @@ -174,6 +179,12 @@ impl PipelineBuilder { PhysicalPlan::Limit(limit) => self.build_limit(limit), PhysicalPlan::RowFetch(row_fetch) => self.build_row_fetch(row_fetch), PhysicalPlan::HashJoin(join) => self.build_join(join), + PhysicalPlan::RuntimeFilterSource(runtime_filter_source) => { + self.build_runtime_filter_source(runtime_filter_source) + } + PhysicalPlan::RuntimeFilterSink(runtime_filter_sink) => { + self.build_runtime_filter_sink(runtime_filter_sink) + } PhysicalPlan::ExchangeSink(sink) => self.build_exchange_sink(sink), PhysicalPlan::ExchangeSource(source) => self.build_exchange_source(source), PhysicalPlan::UnionAll(union_all) => self.build_union_all(union_all), diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/build_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/build_state.rs index bddb87c0cf2a4..322f288cdb0e9 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/build_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/build_state.rs @@ -13,6 +13,7 @@ // limitations under the License. use databend_common_expression::types::DataType; +use databend_common_expression::Column; use databend_common_expression::ColumnVec; use databend_common_expression::DataBlock; @@ -23,6 +24,7 @@ pub struct BuildState { pub(crate) outer_scan_map: Vec>, /// LeftMarkScan map, initialized at `HashJoinBuildState`, used in `HashJoinProbeState` pub(crate) mark_scan_map: Vec>, + pub(crate) runtime_filter_columns: Vec, } impl BuildState { @@ -31,6 +33,7 @@ impl BuildState { generation_state: BuildBlockGenerationState::new(), outer_scan_map: Vec::new(), mark_scan_map: Vec::new(), + runtime_filter_columns: Vec::new(), } } } diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/desc.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/desc.rs index c4d6a7ac7a4bf..92b9683fbb6eb 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/desc.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/desc.rs @@ -45,11 +45,11 @@ pub struct HashJoinDesc { pub(crate) marker_join_desc: MarkJoinDesc, /// Whether the Join are derived from correlated subquery. pub(crate) from_correlated_subquery: bool, - pub(crate) probe_keys_rt: Vec, IndexType)>>, + pub(crate) runtime_filter_exprs: Vec, IndexType)>>, // Under cluster, mark if the join is broadcast join. pub broadcast: bool, - // If enable bloom runtime filter - pub enable_bloom_runtime_filter: bool, + // If support runtime filter. + pub support_runtime_filter: bool, } impl HashJoinDesc { @@ -67,13 +67,13 @@ impl HashJoinDesc { .map(|k| k.as_expr(&BUILTIN_FUNCTIONS)) .collect(); - let probe_keys_rt: Vec, IndexType)>> = join - .probe_keys_rt + let runtime_filter_exprs: Vec, IndexType)>> = join + .runtime_filter_exprs .iter() - .map(|probe_key_rt| { - probe_key_rt + .map(|runtime_filter_expr| { + runtime_filter_expr .as_ref() - .map(|(expr, idx)| (expr.as_expr(&BUILTIN_FUNCTIONS), *idx)) + .map(|(expr, table_index)| (expr.as_expr(&BUILTIN_FUNCTIONS), *table_index)) }) .collect(); @@ -88,10 +88,10 @@ impl HashJoinDesc { // marker_index: join.marker_index, }, from_correlated_subquery: join.from_correlated_subquery, - probe_keys_rt, + runtime_filter_exprs, broadcast: join.broadcast, single_to_inner: join.single_to_inner.clone(), - enable_bloom_runtime_filter: join.enable_bloom_runtime_filter, + support_runtime_filter: join.support_runtime_filter, }) } diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs index 3af426c9401ac..655a8a361759b 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::HashSet; use std::collections::VecDeque; use std::ops::ControlFlow; use std::sync::atomic::AtomicU32; @@ -60,13 +59,12 @@ use itertools::Itertools; use log::info; use parking_lot::Mutex; use parking_lot::RwLock; -use xorf::BinaryFuse16; use crate::pipelines::processors::transforms::hash_join::common::wrap_true_validity; use crate::pipelines::processors::transforms::hash_join::desc::MARKER_KIND_FALSE; use crate::pipelines::processors::transforms::hash_join::transform_hash_join_build::HashTableType; +use crate::pipelines::processors::transforms::hash_join::util::build_key_data_block; use crate::pipelines::processors::transforms::hash_join::util::dedup_build_key_column; -use crate::pipelines::processors::transforms::hash_join::util::hash_by_method; use crate::pipelines::processors::transforms::hash_join::util::inlist_filter; use crate::pipelines::processors::transforms::hash_join::util::min_max_filter; use crate::pipelines::processors::transforms::hash_join::FixedKeyHashJoinHashTable; @@ -115,12 +113,6 @@ pub struct HashJoinBuildState { pub(crate) global_memory_threshold: usize, /// Max memory usage threshold for each processor. pub(crate) processor_memory_threshold: usize, - - /// Runtime filter related states - pub(crate) enable_inlist_runtime_filter: bool, - pub(crate) enable_min_max_runtime_filter: bool, - /// Need to open runtime filter setting. - pub(crate) enable_bloom_runtime_filter: bool, } impl HashJoinBuildState { @@ -143,20 +135,6 @@ impl HashJoinBuildState { }) .collect::>(); let method = DataBlock::choose_hash_method_with_types(&hash_key_types, false)?; - let mut enable_bloom_runtime_filter = false; - let mut enable_inlist_runtime_filter = false; - let mut enable_min_max_runtime_filter = false; - if supported_join_type_for_runtime_filter(&hash_join_state.hash_join_desc.join_type) { - let is_cluster = !ctx.get_cluster().is_empty(); - // For cluster, only support runtime filter for broadcast join. - let is_broadcast_join = hash_join_state.hash_join_desc.broadcast; - if !is_cluster || is_broadcast_join { - enable_inlist_runtime_filter = true; - enable_min_max_runtime_filter = true; - enable_bloom_runtime_filter = - hash_join_state.hash_join_desc.enable_bloom_runtime_filter; - } - } let settings = ctx.get_settings(); let chunk_size_limit = settings.get_max_block_size()? as usize * 16; @@ -181,12 +159,29 @@ impl HashJoinBuildState { mutex: Default::default(), global_memory_threshold, processor_memory_threshold, - enable_bloom_runtime_filter, - enable_inlist_runtime_filter, - enable_min_max_runtime_filter, })) } + pub fn runtime_filter_columns(&self) -> Vec<(usize, String)> { + let mut columns = Vec::new(); + for (probe_key, table_index) in self + .hash_join_state + .hash_join_desc + .runtime_filter_exprs + .iter() + .filter_map(|runtime_filter_expr| { + runtime_filter_expr + .as_ref() + .map(|(probe_key, table_index)| (probe_key, table_index)) + }) + { + if let Some(column_name) = Expr::::column_id(probe_key) { + columns.push((*table_index, column_name)); + } + } + columns + } + // Get max memory usage for settings fn get_memory_threshold(ctx: Arc, num_threads: usize) -> Result<(usize, usize)> { debug_assert!(num_threads != 0); @@ -346,8 +341,8 @@ impl HashJoinBuildState { }; // If spilling happened, skip adding runtime filter, because probe data is ready and spilled. - if self.hash_join_state.spilled_partitions.read().is_empty() { - self.add_runtime_filter(&build_chunks, build_num_rows)?; + if self.support_runtime_filter() { + self.build_runtime_filter(&build_chunks, build_num_rows)?; } // Divide the finalize phase into multiple tasks. @@ -848,207 +843,214 @@ impl HashJoinBuildState { Ok(()) } - fn add_runtime_filter(&self, build_chunks: &[DataBlock], build_num_rows: usize) -> Result<()> { + fn build_runtime_filter(&self, data_blocks: &[DataBlock], build_num_rows: usize) -> Result<()> { + if build_num_rows == 0 { + return Ok(()); + } for (build_key, probe_key, table_index) in self .hash_join_state .hash_join_desc .build_keys .iter() - .zip(self.hash_join_state.hash_join_desc.probe_keys_rt.iter()) - .filter_map(|(b, p)| p.as_ref().map(|(p, index)| (b, p, index))) + .zip( + self.hash_join_state + .hash_join_desc + .runtime_filter_exprs + .iter(), + ) + .filter_map(|(build_key, runtime_filter_expr)| { + runtime_filter_expr + .as_ref() + .map(|(probe_key, table_index)| (build_key, probe_key, table_index)) + }) { - let mut runtime_filter = RuntimeFilterInfo::default(); - if self.enable_inlist_runtime_filter && build_num_rows < INLIST_RUNTIME_FILTER_THRESHOLD - { - self.inlist_runtime_filter( - &mut runtime_filter, - build_chunks, - build_key, - probe_key, - )?; - } - if self.enable_bloom_runtime_filter { - self.bloom_runtime_filter(build_chunks, &mut runtime_filter, build_key, probe_key)?; - } - if self.enable_min_max_runtime_filter { - self.min_max_runtime_filter( - build_chunks, - &mut runtime_filter, - build_key, - probe_key, - )?; + let Some(probe_key_column_id) = Expr::::column_id(probe_key) else { + // Unsupported expression. + continue; + }; + + // Collect build key columns. + let mut build_key_columns = Vec::with_capacity(data_blocks.len()); + for data_block in data_blocks.iter() { + let data_block = build_key_data_block( + &self.hash_join_state.hash_join_desc.join_type, + data_block, + ); + let column = Evaluator::new(&data_block, &self.func_ctx, &BUILTIN_FUNCTIONS) + .run(build_key)? + .convert_to_full_column(build_key.data_type(), data_block.num_rows()); + build_key_columns.push(column); } + let column = Column::concat_columns(build_key_columns.into_iter())?; + + // Build runtime filter. + let mut runtime_filter = RuntimeFilterInfo::default(); + self.build_inlist_runtime_filter(&mut runtime_filter, column.clone(), probe_key)?; + self.build_min_max_runtime_filter( + &mut runtime_filter, + column, + probe_key, + &probe_key_column_id, + )?; if !runtime_filter.is_empty() { self.ctx.set_runtime_filter((*table_index, runtime_filter)); } } + + self.prepare_bloom_filter_data()?; + Ok(()) } - fn bloom_runtime_filter( - &self, - data_blocks: &[DataBlock], - runtime_filter: &mut RuntimeFilterInfo, - build_key: &Expr, - probe_key: &Expr, - ) -> Result<()> { - if !build_key.data_type().remove_nullable().is_numeric() - && !build_key.data_type().remove_nullable().is_string() - { + pub fn prepare_bloom_filter_data(&self) -> Result<()> { + let build_state = unsafe { &mut *self.hash_join_state.build_state.get() }; + let num_rows = build_state.generation_state.build_num_rows; + if num_rows == 0 { return Ok(()); } - if let Expr::ColumnRef { id, .. } = probe_key { - let mut columns = Vec::with_capacity(data_blocks.len()); - for block in data_blocks.iter() { - if block.num_columns() == 0 { - continue; - } - let evaluator = Evaluator::new(block, &self.func_ctx, &BUILTIN_FUNCTIONS); - let column = evaluator + + let data_blocks = unsafe { + (*self.hash_join_state.build_state.get()) + .generation_state + .chunks + .clone() + }; + let bloom_filter_columns = &mut build_state.runtime_filter_columns; + + for (_, build_key) in self + .hash_join_state + .hash_join_desc + .build_keys + .iter() + .zip( + self.hash_join_state + .hash_join_desc + .runtime_filter_exprs + .iter(), + ) + .filter_map(|(build_key, runtime_filter_expr)| { + runtime_filter_expr + .as_ref() + .map(|(probe_key, _)| (probe_key, build_key)) + }) + .filter(|(probe_key, _)| Expr::::column_id(probe_key).is_some()) + { + // Collect build key columns. + let mut columns = Vec::new(); + for data_block in data_blocks.iter() { + let data_block = build_key_data_block( + &self.hash_join_state.hash_join_desc.join_type, + data_block, + ); + let column = Evaluator::new(&data_block, &self.func_ctx, &BUILTIN_FUNCTIONS) .run(build_key)? - .convert_to_full_column(build_key.data_type(), block.num_rows()); + .convert_to_full_column(build_key.data_type(), data_block.num_rows()); columns.push(column); } - if columns.is_empty() { - return Ok(()); - } - let build_key_column = Column::concat_columns(columns.into_iter())?; - // Generate bloom filter using build column - let data_type = build_key.data_type(); - let num_rows = build_key_column.len(); - let method = DataBlock::choose_hash_method_with_types(&[data_type.clone()], false)?; - let mut hashes = HashSet::with_capacity(num_rows); - let key_columns = &[build_key_column]; - hash_by_method(&method, key_columns.into(), num_rows, &mut hashes)?; - let mut hashes_vec = Vec::with_capacity(num_rows); - hashes.into_iter().for_each(|hash| { - hashes_vec.push(hash); - }); - let filter = BinaryFuse16::try_from(&hashes_vec)?; - runtime_filter.add_bloom((id.to_string(), filter)); + bloom_filter_columns.push(Column::concat_columns(columns.into_iter())?); } + Ok(()) } - fn inlist_runtime_filter( + fn build_inlist_runtime_filter( &self, runtime_filter: &mut RuntimeFilterInfo, - data_blocks: &[DataBlock], - build_key: &Expr, + column: Column, probe_key: &Expr, ) -> Result<()> { - if let Some(distinct_build_column) = - dedup_build_key_column(&self.func_ctx, data_blocks, build_key)? - { - if let Some(filter) = inlist_filter(probe_key, distinct_build_column.clone())? { - info!("inlist_filter: {:?}", filter.sql_display()); - runtime_filter.add_inlist(filter); - } + if column.len() > INLIST_RUNTIME_FILTER_THRESHOLD { + return Ok(()); } + + let column = dedup_build_key_column(&self.func_ctx, column)?; + if let Some(filter) = inlist_filter(probe_key, column)? { + info!("inlist_filter: {:?}", filter.1.sql_display()); + runtime_filter.add_inlist(filter); + } + Ok(()) } - fn min_max_runtime_filter( + fn build_min_max_runtime_filter( &self, - data_blocks: &[DataBlock], runtime_filter: &mut RuntimeFilterInfo, - build_key: &Expr, + column: Column, probe_key: &Expr, + probe_key_column_id: &String, ) -> Result<()> { - if !build_key.runtime_filter_supported_types() { - return Ok(()); - } - if let Expr::ColumnRef { .. } = probe_key { - let mut columns = Vec::with_capacity(data_blocks.len()); - for block in data_blocks.iter() { - if block.num_columns() == 0 { - continue; + // Generate min max filter using build column + let min_max = column.remove_nullable().domain(); + let min_max_filter = match min_max { + Domain::Number(domain) => match domain { + NumberDomain::UInt8(simple_domain) => { + let min = Scalar::Number(NumberScalar::from(simple_domain.min)); + let max = Scalar::Number(NumberScalar::from(simple_domain.max)); + min_max_filter(min, max, probe_key)? } - let evaluator = Evaluator::new(block, &self.func_ctx, &BUILTIN_FUNCTIONS); - let column = evaluator - .run(build_key)? - .convert_to_full_column(build_key.data_type(), block.num_rows()); - columns.push(column); - } - if columns.is_empty() { - return Ok(()); - } - let build_key_column = Column::concat_columns(columns.into_iter())?; - if build_key_column.len() == 0 { - return Ok(()); - } - // Generate min max filter using build column - let min_max = build_key_column.remove_nullable().domain(); - let min_max_filter = match min_max { - Domain::Number(domain) => match domain { - NumberDomain::UInt8(simple_domain) => { - let min = Scalar::Number(NumberScalar::from(simple_domain.min)); - let max = Scalar::Number(NumberScalar::from(simple_domain.max)); - min_max_filter(min, max, probe_key)? - } - NumberDomain::UInt16(simple_domain) => { - let min = Scalar::Number(NumberScalar::from(simple_domain.min)); - let max = Scalar::Number(NumberScalar::from(simple_domain.max)); - min_max_filter(min, max, probe_key)? - } - NumberDomain::UInt32(simple_domain) => { - let min = Scalar::Number(NumberScalar::from(simple_domain.min)); - let max = Scalar::Number(NumberScalar::from(simple_domain.max)); - min_max_filter(min, max, probe_key)? - } - NumberDomain::UInt64(simple_domain) => { - let min = Scalar::Number(NumberScalar::from(simple_domain.min)); - let max = Scalar::Number(NumberScalar::from(simple_domain.max)); - min_max_filter(min, max, probe_key)? - } - NumberDomain::Int8(simple_domain) => { - let min = Scalar::Number(NumberScalar::from(simple_domain.min)); - let max = Scalar::Number(NumberScalar::from(simple_domain.max)); - min_max_filter(min, max, probe_key)? - } - NumberDomain::Int16(simple_domain) => { - let min = Scalar::Number(NumberScalar::from(simple_domain.min)); - let max = Scalar::Number(NumberScalar::from(simple_domain.max)); - min_max_filter(min, max, probe_key)? - } - NumberDomain::Int32(simple_domain) => { - let min = Scalar::Number(NumberScalar::from(simple_domain.min)); - let max = Scalar::Number(NumberScalar::from(simple_domain.max)); - min_max_filter(min, max, probe_key)? - } - NumberDomain::Int64(simple_domain) => { - let min = Scalar::Number(NumberScalar::from(simple_domain.min)); - let max = Scalar::Number(NumberScalar::from(simple_domain.max)); - min_max_filter(min, max, probe_key)? - } - NumberDomain::Float32(simple_domain) => { - let min = Scalar::Number(NumberScalar::from(simple_domain.min)); - let max = Scalar::Number(NumberScalar::from(simple_domain.max)); - min_max_filter(min, max, probe_key)? - } - NumberDomain::Float64(simple_domain) => { - let min = Scalar::Number(NumberScalar::from(simple_domain.min)); - let max = Scalar::Number(NumberScalar::from(simple_domain.max)); - min_max_filter(min, max, probe_key)? - } - }, - Domain::String(domain) => { - let min = Scalar::String(domain.min); - let max = Scalar::String(domain.max.unwrap()); + NumberDomain::UInt16(simple_domain) => { + let min = Scalar::Number(NumberScalar::from(simple_domain.min)); + let max = Scalar::Number(NumberScalar::from(simple_domain.max)); min_max_filter(min, max, probe_key)? } - Domain::Date(date_domain) => { - let min = Scalar::Date(date_domain.min); - let max = Scalar::Date(date_domain.max); + NumberDomain::UInt32(simple_domain) => { + let min = Scalar::Number(NumberScalar::from(simple_domain.min)); + let max = Scalar::Number(NumberScalar::from(simple_domain.max)); min_max_filter(min, max, probe_key)? } - _ => unreachable!(), - }; - if let Some(min_max_filter) = min_max_filter { - info!("min_max_filter: {:?}", min_max_filter.sql_display()); - runtime_filter.add_min_max(min_max_filter); + NumberDomain::UInt64(simple_domain) => { + let min = Scalar::Number(NumberScalar::from(simple_domain.min)); + let max = Scalar::Number(NumberScalar::from(simple_domain.max)); + min_max_filter(min, max, probe_key)? + } + NumberDomain::Int8(simple_domain) => { + let min = Scalar::Number(NumberScalar::from(simple_domain.min)); + let max = Scalar::Number(NumberScalar::from(simple_domain.max)); + min_max_filter(min, max, probe_key)? + } + NumberDomain::Int16(simple_domain) => { + let min = Scalar::Number(NumberScalar::from(simple_domain.min)); + let max = Scalar::Number(NumberScalar::from(simple_domain.max)); + min_max_filter(min, max, probe_key)? + } + NumberDomain::Int32(simple_domain) => { + let min = Scalar::Number(NumberScalar::from(simple_domain.min)); + let max = Scalar::Number(NumberScalar::from(simple_domain.max)); + min_max_filter(min, max, probe_key)? + } + NumberDomain::Int64(simple_domain) => { + let min = Scalar::Number(NumberScalar::from(simple_domain.min)); + let max = Scalar::Number(NumberScalar::from(simple_domain.max)); + min_max_filter(min, max, probe_key)? + } + NumberDomain::Float32(simple_domain) => { + let min = Scalar::Number(NumberScalar::from(simple_domain.min)); + let max = Scalar::Number(NumberScalar::from(simple_domain.max)); + min_max_filter(min, max, probe_key)? + } + NumberDomain::Float64(simple_domain) => { + let min = Scalar::Number(NumberScalar::from(simple_domain.min)); + let max = Scalar::Number(NumberScalar::from(simple_domain.max)); + min_max_filter(min, max, probe_key)? + } + }, + Domain::String(domain) => { + let min = Scalar::String(domain.min); + let max = Scalar::String(domain.max.unwrap()); + min_max_filter(min, max, probe_key)? + } + Domain::Date(date_domain) => { + let min = Scalar::Date(date_domain.min); + let max = Scalar::Date(date_domain.max); + min_max_filter(min, max, probe_key)? } + _ => unreachable!(), + }; + if let Some(min_max_filter) = min_max_filter { + info!("min_max_filter: {:?}", min_max_filter.sql_display()); + runtime_filter.add_min_max((probe_key_column_id.to_string(), min_max_filter)); } + Ok(()) } @@ -1056,22 +1058,8 @@ impl HashJoinBuildState { self.hash_join_state.hash_join_desc.join_type.clone() } - pub fn get_enable_bloom_runtime_filter(&self) -> bool { - self.enable_bloom_runtime_filter - } - - pub fn get_enable_min_max_runtime_filter(&self) -> bool { - self.enable_min_max_runtime_filter + pub fn support_runtime_filter(&self) -> bool { + self.hash_join_state.hash_join_desc.support_runtime_filter + && self.hash_join_state.spilled_partitions.read().is_empty() } } - -pub fn supported_join_type_for_runtime_filter(join_type: &JoinType) -> bool { - matches!( - join_type, - JoinType::Inner - | JoinType::Right - | JoinType::RightSemi - | JoinType::RightAnti - | JoinType::LeftMark - ) -} diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs index 5e27be5eb94ac..1df45522014f1 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::HashSet; use std::collections::VecDeque; use std::ops::ControlFlow; use std::sync::atomic::AtomicUsize; @@ -21,16 +22,19 @@ use std::sync::Arc; use databend_common_arrow::arrow::bitmap::Bitmap; use databend_common_arrow::arrow::bitmap::MutableBitmap; use databend_common_base::base::tokio::sync::Barrier; +use databend_common_catalog::runtime_filter_info::RuntimeFilterInfo; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::arrow::and_validities; use databend_common_expression::types::nullable::NullableColumn; +use databend_common_expression::types::DataType; use databend_common_expression::with_join_hash_method; use databend_common_expression::BlockEntry; use databend_common_expression::Column; use databend_common_expression::DataBlock; use databend_common_expression::DataSchemaRef; use databend_common_expression::Evaluator; +use databend_common_expression::Expr; use databend_common_expression::FunctionContext; use databend_common_expression::HashMethod; use databend_common_expression::HashMethodKind; @@ -40,10 +44,13 @@ use databend_common_expression::Value; use databend_common_functions::BUILTIN_FUNCTIONS; use databend_common_hashtable::HashJoinHashtableLike; use databend_common_hashtable::Interval; +use databend_common_sql::plans::JoinType; use databend_common_sql::ColumnSet; +use databend_common_storages_fuse::TableContext; use itertools::Itertools; use parking_lot::Mutex; use parking_lot::RwLock; +use xorf::BinaryFuse16; use super::ProbeState; use super::ProcessState; @@ -52,10 +59,10 @@ use crate::pipelines::processors::transforms::hash_join::desc::MARKER_KIND_FALSE use crate::pipelines::processors::transforms::hash_join::desc::MARKER_KIND_NULL; use crate::pipelines::processors::transforms::hash_join::desc::MARKER_KIND_TRUE; use crate::pipelines::processors::transforms::hash_join::hash_join_state::HashJoinHashTable; +use crate::pipelines::processors::transforms::hash_join::util::hash_by_method; use crate::pipelines::processors::transforms::hash_join::util::probe_schema_wrap_nullable; use crate::pipelines::processors::HashJoinState; use crate::sessions::QueryContext; -use crate::sql::planner::plans::JoinType; // ({(Interval,prefix),(Interval,repfix),...},chunk_idx) // 1.The Interval is the partial unmodified interval offset in chunks. @@ -91,6 +98,8 @@ pub struct HashJoinProbeState { pub(crate) mark_scan_map_lock: Mutex<()>, /// Hash method pub(crate) hash_method: HashMethodKind, + /// Runtime filter + pub(crate) build_runtime_filter_worker: AtomicUsize, } impl HashJoinProbeState { @@ -136,6 +145,7 @@ impl HashJoinProbeState { merge_into_final_partial_unmodified_scan_tasks: RwLock::new(VecDeque::new()), mark_scan_map_lock: Mutex::new(()), hash_method: method, + build_runtime_filter_worker: AtomicUsize::new(0), }) } @@ -165,6 +175,20 @@ impl HashJoinProbeState { }) } + fn increase_probe_num_rows(&self, num_rows: u64) { + self.hash_join_state + .probe_statistics + .num_rows + .fetch_add(num_rows, Ordering::AcqRel); + } + + fn increase_probe_num_hash_matched_rows(&self, num_hash_matched_rows: u64) { + self.hash_join_state + .probe_statistics + .num_hash_matched_rows + .fetch_add(num_hash_matched_rows, Ordering::AcqRel); + } + pub fn probe_join( &self, mut input: DataBlock, @@ -277,11 +301,13 @@ impl HashJoinProbeState { // Adaptive early filtering. // Thanks to the **adaptive** execution strategy of early filtering, we don't experience a performance decrease // when all keys have matches. This allows us to achieve the same performance as before. - probe_state.num_keys += if let Some(valids) = &valids { + let num_keys = if let Some(valids) = &valids { (valids.len() - valids.unset_bits()) as u64 } else { input_num_rows as u64 }; + probe_state.num_keys += num_keys; + self.increase_probe_num_rows(num_keys); // We use the information from the probed data to predict the matching state of this probe. let prefer_early_filtering = (probe_state.num_keys_hash_matched as f64) / (probe_state.num_keys as f64) < 0.8; @@ -348,6 +374,7 @@ impl HashJoinProbeState { } }; probe_state.num_keys_hash_matched += probe_state.selection_count as u64; + self.increase_probe_num_hash_matched_rows(probe_state.selection_count as u64); // Continue to probe hash table and process data blocks. self.result_blocks(probe_state, keys, &table.hash_table) @@ -358,6 +385,79 @@ impl HashJoinProbeState { }) } + fn construct_binary_fuse( + &self, + column: Column, + validity: Option, + data_type: DataType, + ) -> Result { + // Generate bloom filter using build column + let num_rows = column.len(); + let method = DataBlock::choose_hash_method_with_types(&[data_type.clone()], false)?; + let mut column_hashes = HashSet::with_capacity(num_rows); + let column = if let Some(validity) = validity { + column.filter(&validity) + } else { + column + }; + let key_columns = &[column]; + hash_by_method(&method, key_columns.into(), num_rows, &mut column_hashes)?; + let mut hashes = Vec::with_capacity(column_hashes.len()); + column_hashes.into_iter().for_each(|hash| { + hashes.push(hash); + }); + Ok(BinaryFuse16::try_from(&hashes)?) + } + + pub fn build_bloom_filter(&self) -> Result<()> { + let build_state = unsafe { &mut *self.hash_join_state.build_state.get() }; + let bloom_filter_columns = &mut build_state.runtime_filter_columns; + + for ((build_key, probe_key, table_index), column) in self + .hash_join_state + .hash_join_desc + .build_keys + .iter() + .zip( + self.hash_join_state + .hash_join_desc + .runtime_filter_exprs + .iter(), + ) + .filter_map(|(build_key, runtime_filter_expr)| { + runtime_filter_expr + .as_ref() + .map(|(probe_key, table_index)| (build_key, probe_key, table_index)) + }) + .filter(|(_, probe_key, _)| Expr::::column_id(probe_key).is_some()) + .zip(bloom_filter_columns.iter()) + { + // Build runtime filter. + let mut runtime_filter = RuntimeFilterInfo::default(); + let (column, validity) = if let Column::Nullable(inner_column) = column.clone() { + if inner_column.validity.unset_bits() == 0 { + (inner_column.column, None) + } else { + (inner_column.column, Some(inner_column.validity)) + } + } else { + (column.clone(), None) + }; + + // Construct bloom filter. + let data_type = build_key.data_type().remove_nullable(); + let bloom_filter = self.construct_binary_fuse(column, validity, data_type)?; + let probe_key_column_id = Expr::::column_id(probe_key).unwrap(); + runtime_filter.add_bloom((probe_key_column_id, Arc::new(bloom_filter))); + if !runtime_filter.is_empty() { + self.ctx.set_runtime_filter((*table_index, runtime_filter)); + } + } + + bloom_filter_columns.clear(); + Ok(()) + } + /// Checks if a join type can eliminate valids. pub fn check_for_eliminate_valids( from_correlated_subquery: bool, @@ -390,6 +490,8 @@ impl HashJoinProbeState { pub fn probe_attach(&self) { self.wait_probe_counter.fetch_add(1, Ordering::AcqRel); self.next_round_counter.fetch_add(1, Ordering::AcqRel); + self.build_runtime_filter_worker + .fetch_add(1, Ordering::AcqRel); } pub fn probe_done(&self) -> Result<()> { diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_state.rs index 747e6994a953a..d0ea0effc586e 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_state.rs @@ -23,10 +23,12 @@ use std::sync::Arc; use databend_common_base::base::tokio::sync::watch; use databend_common_base::base::tokio::sync::watch::Receiver; use databend_common_base::base::tokio::sync::watch::Sender; +use databend_common_catalog::runtime_filter_info::HashJoinProbeStatistics; use databend_common_catalog::table_context::TableContext; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::BlockEntry; +use databend_common_expression::DataField; use databend_common_expression::DataSchemaRef; use databend_common_expression::HashMethodFixedKeys; use databend_common_expression::HashMethodSerializer; @@ -130,10 +132,20 @@ pub struct HashJoinState { pub(crate) column_map: HashMap, // The index of the next cache block to be read. pub(crate) next_cache_block_index: AtomicUsize, + + /// Runtime filter + pub(crate) runtime_filter_source_fields: Vec, + /// Sender message to notify the runtime filter source processor. + pub(crate) build_runtime_filter_watcher: Sender>, + pub(crate) _build_runtime_filter_receiver: Receiver>, + pub(crate) is_runtime_filter_data_ready: AtomicBool, + pub(crate) need_to_check_runtime_filter_data: AtomicBool, + /// Statistics + pub(crate) probe_statistics: Arc, } impl HashJoinState { - pub fn try_create( + pub fn create( ctx: Arc, mut build_schema: DataSchemaRef, build_projections: &ColumnSet, @@ -142,6 +154,7 @@ impl HashJoinState { merge_into_is_distributed: bool, enable_merge_into_optimization: bool, build_side_cache_info: Option<(usize, HashMap)>, + runtime_filter_source_fields: Vec, ) -> Result> { if matches!( hash_join_desc.join_type, @@ -151,6 +164,7 @@ impl HashJoinState { }; let (build_watcher, _build_done_dummy_receiver) = watch::channel(HashTableType::UnFinished); let (continue_build_watcher, _continue_build_dummy_receiver) = watch::channel(false); + let (build_runtime_filter_watcher, _build_runtime_filter_receiver) = watch::channel(None); let settings = ctx.get_settings(); let enable_spill = settings.get_join_spilling_memory_ratio()? != 0; @@ -189,6 +203,12 @@ impl HashJoinState { }, column_map, next_cache_block_index: AtomicUsize::new(0), + runtime_filter_source_fields, + build_runtime_filter_watcher, + _build_runtime_filter_receiver, + is_runtime_filter_data_ready: AtomicBool::new(false), + need_to_check_runtime_filter_data: AtomicBool::new(true), + probe_statistics: Arc::new(HashJoinProbeStatistics::default()), })) } diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/mod.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/mod.rs index d50ee45b166eb..774bc31279650 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/mod.rs @@ -24,9 +24,12 @@ mod probe_join; mod probe_state; mod result_blocks; pub(crate) mod row; +mod runtime_filter; mod spill_common; mod transform_hash_join_build; mod transform_hash_join_probe; +mod transform_runtime_filter_sink; +mod transform_runtime_filter_source; mod util; pub use desc::HashJoinDesc; @@ -38,3 +41,5 @@ pub use probe_state::ProbeState; pub use probe_state::ProcessState; pub use transform_hash_join_build::TransformHashJoinBuild; pub use transform_hash_join_probe::TransformHashJoinProbe; +pub use transform_runtime_filter_sink::TransformRuntimeFilterSink; +pub use transform_runtime_filter_source::TransformRuntimeFilterSource; diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/runtime_filter.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/runtime_filter.rs new file mode 100644 index 0000000000000..80d020ecbdddc --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/runtime_filter.rs @@ -0,0 +1,55 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::fmt::Debug; +use std::fmt::Formatter; + +use databend_common_expression::BlockMetaInfo; +use databend_common_expression::BlockMetaInfoDowncast; +use databend_common_expression::BlockMetaInfoPtr; + +#[derive(serde::Serialize, serde::Deserialize, Clone, PartialEq)] +pub struct RuntimeFilterMeta { + pub node_id: String, + pub need_to_build: bool, +} + +impl RuntimeFilterMeta { + pub fn create(node_id: String, need_to_build: bool) -> BlockMetaInfoPtr { + Box::new(RuntimeFilterMeta { + node_id, + need_to_build, + }) + } +} + +impl Debug for RuntimeFilterMeta { + fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { + f.debug_struct("RuntimeFilterMeta") + .field("node_id", &self.node_id) + .field("need_to_build", &self.need_to_build) + .finish() + } +} + +#[typetag::serde(name = "runtime_filter_meta")] +impl BlockMetaInfo for RuntimeFilterMeta { + fn equals(&self, info: &Box) -> bool { + Self::downcast_ref_from(info).is_some_and(|other| self == other) + } + + fn clone_self(&self) -> Box { + Box::new(self.clone()) + } +} diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/transform_hash_join_probe.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/transform_hash_join_probe.rs index cd113272c8b31..89dfc88bfed6e 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/transform_hash_join_probe.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/transform_hash_join_probe.rs @@ -17,6 +17,7 @@ use std::collections::VecDeque; use std::sync::atomic::Ordering; use std::sync::Arc; +use databend_common_catalog::table_context::TableContext; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::DataBlock; @@ -51,6 +52,8 @@ pub enum SyncStep { Probe, // Final scan for right-related join or merge into. FinalScan, + // Build runtime filter. + BuildRuntimeFilter, } #[derive(Clone, Debug, Eq, PartialEq)] @@ -111,6 +114,12 @@ pub struct TransformHashJoinProbe { // The next partition id to restore. partition_id_to_restore: usize, + // Bloom filter related states. + support_runtime_filter: bool, + need_to_check_is_build_processor: bool, + is_build_runtime_filter_processor: bool, + need_to_notify_runtime_filter_source: bool, + step: Step, step_logs: Vec, } @@ -156,6 +165,10 @@ impl TransformHashJoinProbe { other_predicate, ); + let support_runtime_filter = join_probe_state + .hash_join_state + .hash_join_desc + .support_runtime_filter; Ok(Box::new(TransformHashJoinProbe { input_port, output_port, @@ -177,6 +190,10 @@ impl TransformHashJoinProbe { partition_id_to_restore: 0, step: Step::Async(AsyncStep::WaitBuild), step_logs: vec![Step::Async(AsyncStep::WaitBuild)], + support_runtime_filter, + need_to_check_is_build_processor: true, + is_build_runtime_filter_processor: true, + need_to_notify_runtime_filter_source: true, })) } @@ -187,6 +204,11 @@ impl TransformHashJoinProbe { Step::Finish => { self.input_port.finish(); self.output_port.finish(); + self.join_probe_state + .hash_join_state + .build_runtime_filter_watcher + .send(Some(false)) + .map_err(|_| ErrorCode::TokioError("watcher's sender is dropped"))?; self.finish_build()?; Event::Finished } @@ -196,6 +218,74 @@ impl TransformHashJoinProbe { Ok(event) } + fn need_build_runtime_filter(&mut self) -> Result { + if !self.support_runtime_filter + || !self.is_build_runtime_filter_processor + || self.is_spill_happened + { + return Ok(false); + } + + if self.is_build_runtime_filter_processor() { + if !self.prefer_runtime_filter() { + return Ok(false); + } + + if self.join_probe_state.ctx.get_cluster().is_empty() { + self.is_build_runtime_filter_processor = false; + return Ok(true); + } + + if self.need_to_notify_runtime_filter_source { + self.join_probe_state + .hash_join_state + .build_runtime_filter_watcher + .send(Some(true)) + .map_err(|_| ErrorCode::TokioError("watcher's sender is dropped"))?; + self.need_to_notify_runtime_filter_source = false; + return Ok(false); + } + + if self + .join_probe_state + .hash_join_state + .need_to_check_runtime_filter_data + .load(Ordering::Acquire) + { + self.is_build_runtime_filter_processor = false; + if self + .join_probe_state + .hash_join_state + .is_runtime_filter_data_ready + .load(Ordering::Acquire) + { + return Ok(true); + } + } + } + Ok(false) + } + + fn is_build_runtime_filter_processor(&mut self) -> bool { + if !self.need_to_check_is_build_processor { + return self.is_build_runtime_filter_processor; + } + self.need_to_check_is_build_processor = false; + self.is_build_runtime_filter_processor = self + .join_probe_state + .build_runtime_filter_worker + .fetch_sub(1, Ordering::AcqRel) + == 1; + self.is_build_runtime_filter_processor + } + + fn prefer_runtime_filter(&self) -> bool { + self.join_probe_state + .hash_join_state + .probe_statistics + .prefer_runtime_filter() + } + fn probe(&mut self) -> Result { if self.output_port.is_finished() { if self.need_final_scan() { @@ -219,6 +309,10 @@ impl TransformHashJoinProbe { return self.next_step(Step::Async(AsyncStep::Spill)); } + if self.need_build_runtime_filter()? { + return self.next_step(Step::Sync(SyncStep::BuildRuntimeFilter)); + } + if self.input_port.has_data() { let data_block = self.input_port.pull_data().unwrap()?; self.add_data_block(data_block); @@ -315,7 +409,7 @@ impl Processor for TransformHashJoinProbe { fn event(&mut self) -> Result { match &self.step { Step::Sync(step) => match step { - SyncStep::Probe => self.probe(), + SyncStep::Probe | SyncStep::BuildRuntimeFilter => self.probe(), SyncStep::FinalScan => self.final_scan(), }, Step::Async(step) => match step { @@ -385,6 +479,7 @@ impl Processor for TransformHashJoinProbe { self.is_final_scan_finished = true; Ok(()) } + Step::Sync(SyncStep::BuildRuntimeFilter) => self.join_probe_state.build_bloom_filter(), _ => unreachable!(), } } diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/transform_runtime_filter_sink.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/transform_runtime_filter_sink.rs new file mode 100644 index 0000000000000..5a52620f4937a --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/transform_runtime_filter_sink.rs @@ -0,0 +1,133 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::any::Any; +use std::collections::HashMap; +use std::sync::atomic::Ordering; +use std::sync::Arc; + +use databend_common_exception::Result; +use databend_common_expression::Column; +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::Processor; +use databend_common_pipeline_core::processors::ProcessorPtr; + +use crate::pipelines::processors::HashJoinState; + +pub struct TransformRuntimeFilterSink { + input: Arc, + hash_join_state: Arc, + num_collected_nodes: usize, + num_cluster_nodes: usize, + is_collected: HashMap, + data_blocks: Vec, +} + +impl TransformRuntimeFilterSink { + pub fn create( + input: Arc, + hash_join_state: Arc, + num_cluster_nodes: usize, + is_collected: HashMap, + ) -> Result { + Ok(ProcessorPtr::create(Box::new(TransformRuntimeFilterSink { + input, + hash_join_state, + num_collected_nodes: 0, + num_cluster_nodes, + is_collected, + data_blocks: Vec::new(), + }))) + } + + pub fn get_meta(data_block: &DataBlock) -> (String, bool) { + let num_columns = data_block.num_columns(); + let node_id_value = data_block.get_by_offset(num_columns - 2).value.clone(); + let need_to_build_value = data_block.get_by_offset(num_columns - 1).value.clone(); + let node_id = node_id_value + .index(0) + .unwrap() + .into_string() + .unwrap() + .to_string(); + let need_to_build = need_to_build_value + .index(0) + .unwrap() + .into_boolean() + .unwrap(); + (node_id, need_to_build) + } +} + +#[async_trait::async_trait] +impl Processor for TransformRuntimeFilterSink { + fn name(&self) -> String { + String::from("TransformRuntimeFilterSink") + } + + fn as_any(&mut self) -> &mut dyn Any { + self + } + + fn event(&mut self) -> Result { + if self.input.is_finished() { + self.data_blocks.clear(); + return Ok(Event::Finished); + } else if self.input.has_data() { + let data_block = self.input.pull_data().unwrap()?; + let (node_id, need_to_build) = Self::get_meta(&data_block); + if need_to_build { + let num_columns = data_block.num_columns() - 2; + self.data_blocks.push(data_block); + if let Some(is_collected) = self.is_collected.get_mut(&node_id) { + if !*is_collected { + self.num_collected_nodes += 1; + *is_collected = true; + } + } + + if self.num_collected_nodes == self.num_cluster_nodes { + let build_state = unsafe { &mut *self.hash_join_state.build_state.get() }; + let bloom_filter_columns = &mut build_state.runtime_filter_columns; + for column_index in 0..num_columns { + let mut columns = Vec::new(); + for data_block in self.data_blocks.iter() { + let num_rows = data_block.num_rows(); + columns + .push(data_block.get_by_offset(column_index).to_column(num_rows)); + } + bloom_filter_columns.push(Column::concat_columns(columns.into_iter())?); + } + self.hash_join_state + .is_runtime_filter_data_ready + .store(true, Ordering::Release); + self.hash_join_state + .need_to_check_runtime_filter_data + .store(true, Ordering::Release); + } + } else { + self.hash_join_state + .is_runtime_filter_data_ready + .store(false, Ordering::Release); + self.hash_join_state + .need_to_check_runtime_filter_data + .store(true, Ordering::Release); + } + } + self.input.set_need_data(); + Ok(Event::NeedData) + } +} diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/transform_runtime_filter_source.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/transform_runtime_filter_source.rs new file mode 100644 index 0000000000000..d5f41b9adf3fb --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/transform_runtime_filter_source.rs @@ -0,0 +1,151 @@ +// 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::ErrorCode; +use databend_common_exception::Result; +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::Scalar; +use databend_common_expression::Value; +use databend_common_pipeline_core::processors::Event; +use databend_common_pipeline_core::processors::OutputPort; +use databend_common_pipeline_core::processors::Processor; +use databend_common_pipeline_core::processors::ProcessorPtr; + +use crate::pipelines::processors::HashJoinState; + +pub struct TransformRuntimeFilterSource { + output: Arc, + output_data: Option, + node_id: String, + hash_join_state: Arc, + is_runtime_filter_readded: bool, +} + +impl TransformRuntimeFilterSource { + pub fn create( + output: Arc, + node_id: String, + hash_join_state: Arc, + ) -> Result { + Ok(ProcessorPtr::create(Box::new( + TransformRuntimeFilterSource { + output, + output_data: None, + node_id, + hash_join_state, + is_runtime_filter_readded: false, + }, + ))) + } +} + +impl TransformRuntimeFilterSource { + #[async_backtrace::framed] + async fn wait_runtime_filter_notify(&mut self) -> Result { + let mut rx = self + .hash_join_state + .build_runtime_filter_watcher + .subscribe(); + if (*rx.borrow()).is_some() { + return Ok((*rx.borrow()).unwrap()); + } + rx.changed() + .await + .map_err(|_| ErrorCode::TokioError("watcher's sender is dropped"))?; + let need_to_build_runtime_filter = (*rx.borrow()).unwrap(); + Ok(need_to_build_runtime_filter) + } +} + +#[async_trait::async_trait] +impl Processor for TransformRuntimeFilterSource { + fn name(&self) -> String { + String::from("TransformRuntimeFilterSource") + } + + fn as_any(&mut self) -> &mut dyn Any { + self + } + + fn event(&mut self) -> Result { + if self.output.is_finished() { + return Ok(Event::Finished); + } + + if !self.output.can_push() { + return Ok(Event::NeedConsume); + } + + if let Some(data_block) = self.output_data.take() { + self.output.push_data(Ok(data_block)); + } + + if !self.is_runtime_filter_readded { + Ok(Event::Async) + } else { + self.output.finish(); + Ok(Event::Finished) + } + } + + #[async_backtrace::framed] + async fn async_process(&mut self) -> Result<()> { + let need_to_build_runtime_filter = self.wait_runtime_filter_notify().await?; + + let data_block = if need_to_build_runtime_filter { + let build_state = unsafe { &mut *self.hash_join_state.build_state.get() }; + let bloom_filter_columns = std::mem::take(&mut build_state.runtime_filter_columns); + let mut data_block = DataBlock::new_from_columns(bloom_filter_columns); + data_block.add_column(BlockEntry::new( + DataType::String, + Value::Scalar(Scalar::String(self.node_id.clone())), + )); + data_block.add_column(BlockEntry::new( + DataType::Boolean, + Value::Scalar(Scalar::Boolean(true)), + )); + data_block + } else { + let runtime_filter_source_fields = &self.hash_join_state.runtime_filter_source_fields; + let mut block_entries = Vec::with_capacity(runtime_filter_source_fields.len()); + for field in runtime_filter_source_fields + .iter() + .take(runtime_filter_source_fields.len() - 2) + { + let data_type = field.data_type().clone(); + let column = Value::Column(Column::random(&data_type, 1, None)); + block_entries.push(BlockEntry::new(data_type, column)); + } + block_entries.push(BlockEntry::new( + DataType::String, + Value::Scalar(Scalar::String(self.node_id.clone())), + )); + block_entries.push(BlockEntry::new( + DataType::Boolean, + Value::Scalar(Scalar::Boolean(false)), + )); + DataBlock::new(block_entries, 1) + }; + + self.output_data = Some(data_block); + self.is_runtime_filter_readded = true; + Ok(()) + } +} diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/util.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/util.rs index fb24d7f951a31..673a8363a5e60 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/util.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/util.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use databend_common_arrow::arrow::bitmap::Bitmap; use databend_common_exception::Result; use databend_common_expression::type_check; use databend_common_expression::types::AnyType; @@ -31,6 +32,9 @@ use databend_common_expression::Scalar; use databend_common_expression::Value; use databend_common_functions::BUILTIN_FUNCTIONS; use databend_common_hashtable::FastHash; +use databend_common_sql::plans::JoinType; + +use crate::pipelines::processors::transforms::hash_join::common::wrap_true_validity; pub(crate) fn build_schema_wrap_nullable(build_schema: &DataSchemaRef) -> DataSchemaRef { let mut nullable_field = Vec::with_capacity(build_schema.fields().len()); @@ -54,11 +58,28 @@ pub(crate) fn probe_schema_wrap_nullable(probe_schema: &DataSchemaRef) -> DataSc DataSchemaRefExt::create(nullable_field) } +pub fn build_key_data_block(join_type: &JoinType, data_block: &DataBlock) -> DataBlock { + if matches!( + join_type, + JoinType::Left | JoinType::LeftSingle | JoinType::Full + ) { + let validity = Bitmap::new_constant(true, data_block.num_rows()); + let nullable_columns = data_block + .columns() + .iter() + .map(|c| wrap_true_validity(c, data_block.num_rows(), &validity)) + .collect::>(); + DataBlock::new(nullable_columns, data_block.num_rows()) + } else { + data_block.clone() + } +} + // Construct inlist runtime filter pub(crate) fn inlist_filter( probe_key: &Expr, build_column: Value, -) -> Result>> { +) -> Result)>> { // Currently, only support key is a column, will support more later. // Such as t1.a + 1 = t2.a, or t1.a + t1.b = t2.a (left side is probe side) if let Expr::ColumnRef { @@ -88,7 +109,7 @@ pub(crate) fn inlist_filter( args, }; let expr = type_check::check(&contain_func, &BUILTIN_FUNCTIONS)?; - return Ok(Some(expr)); + return Ok(Some((id.to_string(), expr))); } Ok(None) } @@ -96,27 +117,11 @@ pub(crate) fn inlist_filter( // Deduplicate build key column pub(crate) fn dedup_build_key_column( func_ctx: &FunctionContext, - data_blocks: &[DataBlock], - build_key: &Expr, -) -> Result>> { - // Dedup build key column - let mut columns = Vec::with_capacity(data_blocks.len()); - for block in data_blocks.iter() { - if block.num_columns() == 0 { - continue; - } - let evaluator = Evaluator::new(block, func_ctx, &BUILTIN_FUNCTIONS); - let column = evaluator - .run(build_key)? - .convert_to_full_column(build_key.data_type(), block.num_rows()); - columns.push(column); - } - if columns.is_empty() { - return Ok(None); - } - let build_key_column = Column::concat_columns(columns.into_iter())?; - let mut list = Vec::with_capacity(build_key_column.len()); - for value in build_key_column.iter() { + column: Column, +) -> Result> { + // Deduplicate build key column. + let mut list = Vec::with_capacity(column.len()); + for value in column.iter() { list.push(RawExpr::Constant { span: None, scalar: value.to_owned(), @@ -135,13 +140,9 @@ pub(crate) fn dedup_build_key_column( args: vec![array], }; - // Deduplicate build key column let empty_key_block = DataBlock::empty(); let evaluator = Evaluator::new(&empty_key_block, func_ctx, &BUILTIN_FUNCTIONS); - Ok(Some(evaluator.run(&type_check::check( - &distinct_list, - &BUILTIN_FUNCTIONS, - )?)?)) + evaluator.run(&type_check::check(&distinct_list, &BUILTIN_FUNCTIONS)?) } // Get row hash by HashMethod diff --git a/src/query/service/src/pipelines/processors/transforms/transform_recursive_cte_source.rs b/src/query/service/src/pipelines/processors/transforms/transform_recursive_cte_source.rs index e86b7b10c3b6f..e2c7f8239a67f 100644 --- a/src/query/service/src/pipelines/processors/transforms/transform_recursive_cte_source.rs +++ b/src/query/service/src/pipelines/processors/transforms/transform_recursive_cte_source.rs @@ -324,6 +324,8 @@ async fn create_memory_table_for_cte_scan( | PhysicalPlan::ExpressionScan(_) | PhysicalPlan::CacheScan(_) | PhysicalPlan::DistributedInsertSelect(_) + | PhysicalPlan::RuntimeFilterSource(_) + | PhysicalPlan::RuntimeFilterSink(_) | PhysicalPlan::ExchangeSource(_) | PhysicalPlan::ExchangeSink(_) | PhysicalPlan::CopyIntoTable(_) diff --git a/src/query/service/src/schedulers/fragments/fragmenter.rs b/src/query/service/src/schedulers/fragments/fragmenter.rs index c00a11e7403d2..ab55d482ec1ea 100644 --- a/src/query/service/src/schedulers/fragments/fragmenter.rs +++ b/src/query/service/src/schedulers/fragments/fragmenter.rs @@ -214,6 +214,13 @@ impl PhysicalPlanReplacer for Fragmenter { fragments.append(&mut self.fragments); let probe_input = self.replace(plan.probe.as_ref())?; fragments.append(&mut self.fragments); + let runtime_filter = if let Some(runtime_filter) = &plan.runtime_filter { + let runtime_filter = self.replace(runtime_filter)?; + fragments.append(&mut self.fragments); + Some(Box::new(runtime_filter)) + } else { + None + }; self.fragments = fragments; Ok(PhysicalPlan::HashJoin(HashJoin { @@ -221,8 +228,10 @@ impl PhysicalPlanReplacer for Fragmenter { projections: plan.projections.clone(), probe_projections: plan.probe_projections.clone(), build_projections: plan.build_projections.clone(), + hash_join_id: plan.hash_join_id, build: Box::new(build_input), probe: Box::new(probe_input), + runtime_filter, build_keys: plan.build_keys.clone(), probe_keys: plan.probe_keys.clone(), is_null_equal: plan.is_null_equal.clone(), @@ -234,8 +243,9 @@ impl PhysicalPlanReplacer for Fragmenter { output_schema: plan.output_schema.clone(), need_hold_hash_table: plan.need_hold_hash_table, stat_info: plan.stat_info.clone(), - probe_keys_rt: plan.probe_keys_rt.clone(), - enable_bloom_runtime_filter: plan.enable_bloom_runtime_filter, + runtime_filter_exprs: plan.runtime_filter_exprs.clone(), + runtime_filter_source_fields: plan.runtime_filter_source_fields.clone(), + support_runtime_filter: plan.support_runtime_filter, broadcast: plan.broadcast, single_to_inner: plan.single_to_inner.clone(), build_side_cache_info: plan.build_side_cache_info.clone(), diff --git a/src/query/service/src/schedulers/fragments/plan_fragment.rs b/src/query/service/src/schedulers/fragments/plan_fragment.rs index db415a5bc29c9..77739922273f2 100644 --- a/src/query/service/src/schedulers/fragments/plan_fragment.rs +++ b/src/query/service/src/schedulers/fragments/plan_fragment.rs @@ -514,6 +514,7 @@ impl PhysicalPlanReplacer for ReplaceReadSource { table_index: plan.table_index, stat_info: plan.stat_info.clone(), internal_column: plan.internal_column.clone(), + runtime_filter_columns: plan.runtime_filter_columns.clone(), })) } 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 a05b9a48c306e..e58a2f92e53a2 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 @@ -54,6 +54,7 @@ use super::statistics_sender::StatisticsSender; use crate::clusters::ClusterHelper; use crate::pipelines::executor::ExecutorSettings; use crate::pipelines::executor::PipelineCompleteExecutor; +use crate::pipelines::processors::HashJoinState; use crate::pipelines::PipelineBuildResult; use crate::pipelines::PipelineBuilder; use crate::schedulers::QueryFragmentActions; @@ -461,7 +462,7 @@ impl DataExchangeManager { assert!(query_coordinator.fragment_exchanges.is_empty()); let injector = DefaultExchangeInjector::create(); let mut build_res = - query_coordinator.subscribe_fragment(&ctx, fragment_id, injector)?; + query_coordinator.subscribe_fragment(&ctx, fragment_id, injector, None)?; let exchanges = std::mem::take(&mut query_coordinator.statistics_exchanges); let statistics_receiver = StatisticsReceiver::spawn_receiver(&ctx, exchanges)?; @@ -522,6 +523,7 @@ impl DataExchangeManager { query_id: &str, fragment_id: usize, injector: Arc, + runtime_filter_hash_join_state: Option>, ) -> Result { let queries_coordinator_guard = self.queries_coordinator.lock(); let queries_coordinator = unsafe { &mut *queries_coordinator_guard.deref().get() }; @@ -536,7 +538,12 @@ impl DataExchangeManager { .query_ctx .clone(); - query_coordinator.subscribe_fragment(&query_ctx, fragment_id, injector) + query_coordinator.subscribe_fragment( + &query_ctx, + fragment_id, + injector, + runtime_filter_hash_join_state, + ) } } } @@ -711,8 +718,11 @@ impl QueryCoordinator { for fragment in &fragments.fragments { let fragment_id = fragment.fragment_id; + if fragment.physical_plan.contain_runtime_filter_source() { + continue; + } if let Some(coordinator) = self.fragments_coordinator.get_mut(&fragment_id) { - coordinator.prepare_pipeline(query_context.clone())?; + coordinator.prepare_pipeline(query_context.clone(), None)?; } } @@ -724,11 +734,12 @@ impl QueryCoordinator { ctx: &Arc, fragment_id: usize, injector: Arc, + runtime_filter_hash_join_state: Option>, ) -> Result { // Merge pipelines if exist locally pipeline if let Some(mut fragment_coordinator) = self.fragments_coordinator.remove(&fragment_id) { let info = self.info.as_ref().expect("QueryInfo is none"); - fragment_coordinator.prepare_pipeline(ctx.clone())?; + fragment_coordinator.prepare_pipeline(ctx.clone(), runtime_filter_hash_join_state)?; if fragment_coordinator.pipeline_build_res.is_none() { return Err(ErrorCode::Internal( @@ -811,6 +822,10 @@ impl QueryCoordinator { .as_ref() .map(|x| x.exchange_injector.clone()) .ok_or_else(|| { + dbg!( + "BB fragment_coordinator.plan = {:?}", + &coordinator.physical_plan + ); ErrorCode::Internal("Pipeline build result is none, It's a bug") })?, )?, @@ -948,7 +963,11 @@ impl FragmentCoordinator { Err(ErrorCode::Internal("Cannot find data exchange.")) } - pub fn prepare_pipeline(&mut self, ctx: Arc) -> Result<()> { + pub fn prepare_pipeline( + &mut self, + ctx: Arc, + runtime_filter_hash_join_state: Option>, + ) -> Result<()> { if !self.initialized { self.initialized = true; @@ -962,12 +981,13 @@ impl FragmentCoordinator { drop(ctx); } - let pipeline_builder = PipelineBuilder::create( + let mut pipeline_builder = PipelineBuilder::create( pipeline_ctx.get_function_context()?, pipeline_ctx.get_settings(), pipeline_ctx, vec![], ); + pipeline_builder.runtime_filter_hash_join_state = runtime_filter_hash_join_state; let res = pipeline_builder.finalize(&self.physical_plan)?; diff --git a/src/query/service/src/sessions/query_ctx.rs b/src/query/service/src/sessions/query_ctx.rs index bdf4b7866eda6..b906418023fe5 100644 --- a/src/query/service/src/sessions/query_ctx.rs +++ b/src/query/service/src/sessions/query_ctx.rs @@ -49,6 +49,7 @@ use databend_common_catalog::plan::PartInfoPtr; use databend_common_catalog::plan::Partitions; use databend_common_catalog::plan::StageTableInfo; use databend_common_catalog::query_kind::QueryKind; +use databend_common_catalog::runtime_filter_info::HashJoinProbeStatistics; use databend_common_catalog::runtime_filter_info::RuntimeFilterInfo; use databend_common_catalog::statistics::data_cache_statistics::DataCacheMetrics; use databend_common_catalog::table_args::TableArgs; @@ -637,7 +638,7 @@ impl TableContext for QueryContext { } fn get_fragment_id(&self) -> usize { - self.fragment_id.fetch_add(1, Ordering::Release) + self.fragment_id.fetch_add(1, Ordering::AcqRel) } #[async_backtrace::framed] @@ -1191,13 +1192,53 @@ impl TableContext for QueryContext { for filter in filters.1.get_min_max() { v.get_mut().add_min_max(filter.clone()); } - for filter in filters.1.blooms() { - v.get_mut().add_bloom(filter); + for (column_name, filter) in filters.1.blooms() { + v.get_mut().add_bloom((column_name, filter)); } } } } + fn set_runtime_filter_columns(&self, table_index: usize, columns: Vec<(usize, String)>) { + let mut runtime_filter_columns = self.shared.runtime_filter_columns.write(); + match runtime_filter_columns.entry(table_index) { + Entry::Vacant(v) => { + v.insert(columns); + } + Entry::Occupied(mut v) => { + v.get_mut().extend(columns); + } + } + } + + fn get_runtime_filter_columns(&self, table_index: usize) -> Vec<(usize, String)> { + let runtime_filter_columns = self.shared.runtime_filter_columns.read(); + match runtime_filter_columns.get(&table_index) { + Some(v) => v.clone(), + None => vec![], + } + } + + fn set_hash_join_probe_statistics( + &self, + join_id: usize, + statistics: Arc, + ) { + self.shared + .hash_join_probe_statistics + .write() + .insert(join_id, statistics); + } + + fn get_hash_join_probe_statistics(&self, join_id: usize) -> Arc { + self.shared + .hash_join_probe_statistics + .read() + .get(&join_id) + .cloned() + .unwrap() + } + fn get_merge_into_join(&self) -> MergeIntoJoin { let merge_into_join = self.shared.merge_into_join.read(); MergeIntoJoin { @@ -1207,7 +1248,7 @@ impl TableContext for QueryContext { } } - fn get_bloom_runtime_filter_with_id(&self, id: IndexType) -> Vec<(String, BinaryFuse16)> { + fn get_bloom_runtime_filter_with_id(&self, id: IndexType) -> Vec<(String, Arc)> { let runtime_filters = self.shared.runtime_filters.read(); match runtime_filters.get(&id) { Some(v) => (v.get_bloom()).clone(), @@ -1215,7 +1256,7 @@ impl TableContext for QueryContext { } } - fn get_inlist_runtime_filter_with_id(&self, id: IndexType) -> Vec> { + fn get_inlist_runtime_filter_with_id(&self, id: IndexType) -> Vec<(String, Expr)> { let runtime_filters = self.shared.runtime_filters.read(); match runtime_filters.get(&id) { Some(v) => (v.get_inlist()).clone(), @@ -1223,7 +1264,7 @@ impl TableContext for QueryContext { } } - fn get_min_max_runtime_filter_with_id(&self, id: IndexType) -> Vec> { + fn get_min_max_runtime_filter_with_id(&self, id: IndexType) -> Vec<(String, Expr)> { let runtime_filters = self.shared.runtime_filters.read(); match runtime_filters.get(&id) { Some(v) => (v.get_min_max()).clone(), diff --git a/src/query/service/src/sessions/query_ctx_shared.rs b/src/query/service/src/sessions/query_ctx_shared.rs index 547babc3b6061..ed52a3792728a 100644 --- a/src/query/service/src/sessions/query_ctx_shared.rs +++ b/src/query/service/src/sessions/query_ctx_shared.rs @@ -30,6 +30,7 @@ use databend_common_catalog::catalog::Catalog; use databend_common_catalog::catalog::CatalogManager; use databend_common_catalog::merge_into_join::MergeIntoJoin; use databend_common_catalog::query_kind::QueryKind; +use databend_common_catalog::runtime_filter_info::HashJoinProbeStatistics; use databend_common_catalog::runtime_filter_info::RuntimeFilterInfo; use databend_common_catalog::statistics::data_cache_statistics::DataCacheMetrics; use databend_common_catalog::table_context::ContextError; @@ -64,6 +65,7 @@ use crate::sessions::Session; use crate::storages::Table; type DatabaseAndTable = (String, String, String); +type HashJoinRuntimeFilterColumn = HashMap>; /// Data that needs to be shared in a query context. pub struct QueryContextShared { @@ -132,6 +134,11 @@ pub struct QueryContextShared { pub(in crate::sessions) runtime_filters: Arc>>, + pub(in crate::sessions) runtime_filter_columns: Arc>, + + pub(in crate::sessions) hash_join_probe_statistics: + Arc>>>, + pub(in crate::sessions) merge_into_join: Arc>, // Records query level data cache metrics @@ -189,6 +196,8 @@ impl QueryContextShared { query_cache_metrics: DataCacheMetrics::new(), query_profiles: Arc::new(RwLock::new(HashMap::new())), runtime_filters: Default::default(), + runtime_filter_columns: Default::default(), + hash_join_probe_statistics: Default::default(), merge_into_join: Default::default(), multi_table_insert_status: Default::default(), query_queued_duration: Arc::new(RwLock::new(Duration::from_secs(0))), diff --git a/src/query/service/tests/it/pipelines/builders/runtime_filter.rs b/src/query/service/tests/it/pipelines/builders/runtime_filter.rs index 811153ec118bd..5b66d8371d1c2 100644 --- a/src/query/service/tests/it/pipelines/builders/runtime_filter.rs +++ b/src/query/service/tests/it/pipelines/builders/runtime_filter.rs @@ -83,7 +83,7 @@ async fn join_build_state( ) -> Result> { let func_ctx = ctx.get_function_context()?; - let join_state = HashJoinState::try_create( + let join_state = HashJoinState::create( ctx.clone(), join.build.output_schema()?, &join.build_projections, @@ -92,6 +92,7 @@ async fn join_build_state( false, true, None, + vec![], )?; let build_state = HashJoinBuildState::try_create( ctx.clone(), @@ -126,9 +127,7 @@ async fn test_generate_runtime_filter() -> Result<()> { ) .await?; let join = find_join(&plan)?; - assert!(join.enable_bloom_runtime_filter); let join_build_state = join_build_state(&fixture.new_query_ctx().await?, &join).await?; - assert!(join_build_state.get_enable_bloom_runtime_filter()); - assert!(join_build_state.get_enable_min_max_runtime_filter()); + assert!(join_build_state.support_runtime_filter()); Ok(()) } diff --git a/src/query/service/tests/it/sql/exec/get_table_bind_test.rs b/src/query/service/tests/it/sql/exec/get_table_bind_test.rs index a1cc8fce6f386..f6c12896cbf85 100644 --- a/src/query/service/tests/it/sql/exec/get_table_bind_test.rs +++ b/src/query/service/tests/it/sql/exec/get_table_bind_test.rs @@ -32,6 +32,7 @@ use databend_common_catalog::plan::DataSourcePlan; use databend_common_catalog::plan::PartInfoPtr; use databend_common_catalog::plan::Partitions; use databend_common_catalog::query_kind::QueryKind; +use databend_common_catalog::runtime_filter_info::HashJoinProbeStatistics; use databend_common_catalog::runtime_filter_info::RuntimeFilterInfo; use databend_common_catalog::statistics::data_cache_statistics::DataCacheMetrics; use databend_common_catalog::table::Table; @@ -941,19 +942,39 @@ impl TableContext for CtxDelegation { todo!() } + fn set_runtime_filter_columns(&self, _table_index: usize, _columns: Vec<(usize, String)>) { + todo!() + } + + fn get_runtime_filter_columns(&self, _table_index: usize) -> Vec<(usize, String)> { + todo!() + } + + fn set_hash_join_probe_statistics( + &self, + _join_id: usize, + _statistics: Arc, + ) { + todo!() + } + + fn get_hash_join_probe_statistics(&self, _join_id: usize) -> Arc { + todo!() + } + fn clear_runtime_filter(&self) { todo!() } - fn get_bloom_runtime_filter_with_id(&self, _id: usize) -> Vec<(String, BinaryFuse16)> { + fn get_bloom_runtime_filter_with_id(&self, _id: usize) -> Vec<(String, Arc)> { todo!() } - fn get_inlist_runtime_filter_with_id(&self, _id: usize) -> Vec> { + fn get_inlist_runtime_filter_with_id(&self, _id: usize) -> Vec<(String, Expr)> { todo!() } - fn get_min_max_runtime_filter_with_id(&self, _id: usize) -> Vec> { + fn get_min_max_runtime_filter_with_id(&self, _id: usize) -> Vec<(String, Expr)> { todo!() } diff --git a/src/query/service/tests/it/storages/fuse/operations/commit.rs b/src/query/service/tests/it/storages/fuse/operations/commit.rs index fe5b5e69a2077..8cadaaeac4ba6 100644 --- a/src/query/service/tests/it/storages/fuse/operations/commit.rs +++ b/src/query/service/tests/it/storages/fuse/operations/commit.rs @@ -31,6 +31,7 @@ use databend_common_catalog::plan::DataSourcePlan; use databend_common_catalog::plan::PartInfoPtr; use databend_common_catalog::plan::Partitions; use databend_common_catalog::query_kind::QueryKind; +use databend_common_catalog::runtime_filter_info::HashJoinProbeStatistics; use databend_common_catalog::runtime_filter_info::RuntimeFilterInfo; use databend_common_catalog::statistics::data_cache_statistics::DataCacheMetrics; use databend_common_catalog::table::Table; @@ -824,19 +825,39 @@ impl TableContext for CtxDelegation { todo!() } + fn set_runtime_filter_columns(&self, _table_index: usize, _columns: Vec<(usize, String)>) { + todo!() + } + + fn get_runtime_filter_columns(&self, _table_index: usize) -> Vec<(usize, String)> { + todo!() + } + + fn set_hash_join_probe_statistics( + &self, + _join_id: usize, + _statistics: Arc, + ) { + todo!() + } + + fn get_hash_join_probe_statistics(&self, _join_id: usize) -> Arc { + todo!() + } + fn clear_runtime_filter(&self) { todo!() } - fn get_bloom_runtime_filter_with_id(&self, _id: usize) -> Vec<(String, BinaryFuse16)> { + fn get_bloom_runtime_filter_with_id(&self, _id: usize) -> Vec<(String, Arc)> { todo!() } - fn get_inlist_runtime_filter_with_id(&self, _id: usize) -> Vec> { + fn get_inlist_runtime_filter_with_id(&self, _id: usize) -> Vec<(String, Expr)> { todo!() } - fn get_min_max_runtime_filter_with_id(&self, _id: usize) -> Vec> { + fn get_min_max_runtime_filter_with_id(&self, _id: usize) -> Vec<(String, Expr)> { todo!() } diff --git a/src/query/settings/src/settings_default.rs b/src/query/settings/src/settings_default.rs index 35e74afbb3122..5bdba1b642944 100644 --- a/src/query/settings/src/settings_default.rs +++ b/src/query/settings/src/settings_default.rs @@ -341,12 +341,6 @@ impl DefaultSettings { mode: SettingMode::Both, range: Some(SettingRange::Numeric(0..=u64::MAX)), }), - ("enable_bloom_runtime_filter", DefaultSettingValue { - value: UserSettingValue::UInt64(1), - desc: "Enables runtime filter optimization for JOIN.", - mode: SettingMode::Both, - range: Some(SettingRange::Numeric(0..=1)), - }), ("max_execute_time_in_seconds", DefaultSettingValue { value: UserSettingValue::UInt64(0), desc: "Sets the maximum query execution time in seconds. Setting it to 0 means no limit.", diff --git a/src/query/settings/src/settings_getter_setter.rs b/src/query/settings/src/settings_getter_setter.rs index 8938fcd616ddb..24c50ca21c25c 100644 --- a/src/query/settings/src/settings_getter_setter.rs +++ b/src/query/settings/src/settings_getter_setter.rs @@ -339,10 +339,6 @@ impl Settings { Ok(self.try_get_u64("inlist_to_join_threshold")? as usize) } - pub fn get_bloom_runtime_filter(&self) -> Result { - Ok(self.try_get_u64("enable_bloom_runtime_filter")? != 0) - } - pub fn get_prefer_broadcast_join(&self) -> Result { Ok(self.try_get_u64("prefer_broadcast_join")? != 0) } diff --git a/src/query/sql/src/executor/format.rs b/src/query/sql/src/executor/format.rs index e0463d9aa8a35..90994d8fd3cf2 100644 --- a/src/query/sql/src/executor/format.rs +++ b/src/query/sql/src/executor/format.rs @@ -25,8 +25,8 @@ use databend_common_functions::BUILTIN_FUNCTIONS; use databend_common_pipeline_core::processors::PlanProfile; use itertools::Itertools; -use super::physical_plans::AddStreamColumn; use crate::executor::explain::PlanStatsInfo; +use crate::executor::physical_plans::AddStreamColumn; use crate::executor::physical_plans::AggregateExpand; use crate::executor::physical_plans::AggregateFinal; use crate::executor::physical_plans::AggregateFunctionDesc; @@ -59,6 +59,8 @@ use crate::executor::physical_plans::ProjectSet; use crate::executor::physical_plans::RangeJoin; use crate::executor::physical_plans::RangeJoinType; use crate::executor::physical_plans::RowFetch; +use crate::executor::physical_plans::RuntimeFilterSink; +use crate::executor::physical_plans::RuntimeFilterSource; use crate::executor::physical_plans::Sort; use crate::executor::physical_plans::TableScan; use crate::executor::physical_plans::Udf; @@ -358,6 +360,12 @@ fn to_format_tree( PhysicalPlan::Limit(plan) => limit_to_format_tree(plan, metadata, profs), PhysicalPlan::RowFetch(plan) => row_fetch_to_format_tree(plan, metadata, profs), PhysicalPlan::HashJoin(plan) => hash_join_to_format_tree(plan, metadata, profs), + PhysicalPlan::RuntimeFilterSource(plan) => { + runtime_filter_source_to_format_tree(plan, metadata, profs) + } + PhysicalPlan::RuntimeFilterSink(plan) => { + runtime_filter_sink_to_format_tree(plan, metadata, profs) + } PhysicalPlan::Exchange(plan) => exchange_to_format_tree(plan, metadata, profs), PhysicalPlan::UnionAll(plan) => union_all_to_format_tree(plan, metadata, profs), PhysicalPlan::ExchangeSource(plan) => exchange_source_to_format_tree(plan, metadata), @@ -1503,6 +1511,25 @@ fn hash_join_to_format_tree( )) } +fn runtime_filter_source_to_format_tree( + _plan: &RuntimeFilterSource, + _metadata: &Metadata, + _profs: &HashMap, +) -> Result> { + Ok(FormatTreeNode::with_children( + "RuntimeFilterSource".to_string(), + vec![], + )) +} + +fn runtime_filter_sink_to_format_tree( + plan: &RuntimeFilterSink, + metadata: &Metadata, + profs: &HashMap, +) -> Result> { + to_format_tree(&plan.input, metadata, profs) +} + fn exchange_to_format_tree( plan: &Exchange, metadata: &Metadata, diff --git a/src/query/sql/src/executor/physical_plan.rs b/src/query/sql/src/executor/physical_plan.rs index 4bbe18f59e0b7..4c9f0542942cf 100644 --- a/src/query/sql/src/executor/physical_plan.rs +++ b/src/query/sql/src/executor/physical_plan.rs @@ -67,6 +67,8 @@ use crate::executor::physical_plans::ReplaceAsyncSourcer; use crate::executor::physical_plans::ReplaceDeduplicate; use crate::executor::physical_plans::ReplaceInto; use crate::executor::physical_plans::RowFetch; +use crate::executor::physical_plans::RuntimeFilterSink; +use crate::executor::physical_plans::RuntimeFilterSource; use crate::executor::physical_plans::Shuffle; use crate::executor::physical_plans::Sort; use crate::executor::physical_plans::TableScan; @@ -102,6 +104,10 @@ pub enum PhysicalPlan { Udf(Udf), RecursiveCteScan(RecursiveCteScan), + /// Runtime filter. + RuntimeFilterSource(RuntimeFilterSource), + RuntimeFilterSink(RuntimeFilterSink), + /// For insert into ... select ... in cluster DistributedInsertSelect(Box), @@ -226,6 +232,18 @@ impl PhysicalPlan { *next_id += 1; plan.probe.adjust_plan_id(next_id); plan.build.adjust_plan_id(next_id); + if let Some(plan) = plan.runtime_filter.as_mut() { + plan.adjust_plan_id(next_id); + } + } + PhysicalPlan::RuntimeFilterSource(plan) => { + plan.plan_id = *next_id; + *next_id += 1; + } + PhysicalPlan::RuntimeFilterSink(plan) => { + plan.plan_id = *next_id; + *next_id += 1; + plan.input.adjust_plan_id(next_id); } PhysicalPlan::RangeJoin(plan) => { plan.plan_id = *next_id; @@ -425,6 +443,8 @@ impl PhysicalPlan { PhysicalPlan::Limit(v) => v.plan_id, PhysicalPlan::RowFetch(v) => v.plan_id, PhysicalPlan::HashJoin(v) => v.plan_id, + PhysicalPlan::RuntimeFilterSource(v) => v.plan_id, + PhysicalPlan::RuntimeFilterSink(v) => v.plan_id, PhysicalPlan::RangeJoin(v) => v.plan_id, PhysicalPlan::Exchange(v) => v.plan_id, PhysicalPlan::UnionAll(v) => v.plan_id, @@ -480,6 +500,7 @@ impl PhysicalPlan { PhysicalPlan::Limit(plan) => plan.output_schema(), PhysicalPlan::RowFetch(plan) => plan.output_schema(), PhysicalPlan::HashJoin(plan) => plan.output_schema(), + PhysicalPlan::RuntimeFilterSource(plan) => plan.output_schema(), PhysicalPlan::Exchange(plan) => plan.output_schema(), PhysicalPlan::ExchangeSource(plan) => plan.output_schema(), PhysicalPlan::ExchangeSink(plan) => plan.output_schema(), @@ -508,6 +529,7 @@ impl PhysicalPlan { | PhysicalPlan::CompactSource(_) | PhysicalPlan::CommitSink(_) | PhysicalPlan::DistributedInsertSelect(_) + | PhysicalPlan::RuntimeFilterSink(_) | PhysicalPlan::Recluster(_) => Ok(DataSchemaRef::default()), PhysicalPlan::Duplicate(plan) => plan.input.output_schema(), PhysicalPlan::Shuffle(plan) => plan.input.output_schema(), @@ -542,6 +564,8 @@ impl PhysicalPlan { PhysicalPlan::Limit(_) => "Limit".to_string(), PhysicalPlan::RowFetch(_) => "RowFetch".to_string(), PhysicalPlan::HashJoin(_) => "HashJoin".to_string(), + PhysicalPlan::RuntimeFilterSource(_) => "RuntimeFilterSource".to_string(), + PhysicalPlan::RuntimeFilterSink(_) => "RuntimeFilterSink".to_string(), PhysicalPlan::Exchange(_) => "Exchange".to_string(), PhysicalPlan::UnionAll(_) => "UnionAll".to_string(), PhysicalPlan::DistributedInsertSelect(_) => "DistributedInsertSelect".to_string(), @@ -607,6 +631,8 @@ impl PhysicalPlan { PhysicalPlan::HashJoin(plan) => Box::new( std::iter::once(plan.probe.as_ref()).chain(std::iter::once(plan.build.as_ref())), ), + PhysicalPlan::RuntimeFilterSource(_) => Box::new(std::iter::empty()), + PhysicalPlan::RuntimeFilterSink(plan) => Box::new(std::iter::once(plan.input.as_ref())), PhysicalPlan::ExpressionScan(plan) => Box::new(std::iter::once(plan.input.as_ref())), PhysicalPlan::Exchange(plan) => Box::new(std::iter::once(plan.input.as_ref())), PhysicalPlan::ExchangeSink(plan) => Box::new(std::iter::once(plan.input.as_ref())), @@ -679,6 +705,8 @@ impl PhysicalPlan { PhysicalPlan::UnionAll(_) | PhysicalPlan::ExchangeSource(_) | PhysicalPlan::HashJoin(_) + | PhysicalPlan::RuntimeFilterSource(_) + | PhysicalPlan::RuntimeFilterSink(_) | PhysicalPlan::RangeJoin(_) | PhysicalPlan::MaterializedCte(_) | PhysicalPlan::AggregateExpand(_) @@ -715,6 +743,12 @@ impl PhysicalPlan { } } + pub fn contain_runtime_filter_source(&self) -> bool { + self.children() + .any(|child| child.contain_runtime_filter_source()) + || matches!(self, Self::RuntimeFilterSource(_)) + } + pub fn is_distributed_plan(&self) -> bool { self.children().any(|child| child.is_distributed_plan()) || matches!( diff --git a/src/query/sql/src/executor/physical_plan_builder.rs b/src/query/sql/src/executor/physical_plan_builder.rs index e386d1d4b5d3d..0bbebaf97a3cd 100644 --- a/src/query/sql/src/executor/physical_plan_builder.rs +++ b/src/query/sql/src/executor/physical_plan_builder.rs @@ -45,6 +45,9 @@ pub struct PhysicalPlanBuilder { pub(crate) cet_used_column_offsets: HashMap>, // DataMutation info, used to build MergeInto physical plan pub(crate) mutation_build_info: Option, + // Runtime filter. + pub(crate) hash_join_id: usize, + pub(crate) runtime_filter_columns: HashMap>, } impl PhysicalPlanBuilder { @@ -58,6 +61,8 @@ impl PhysicalPlanBuilder { cte_output_columns: Default::default(), cet_used_column_offsets: Default::default(), mutation_build_info: None, + hash_join_id: 0, + runtime_filter_columns: HashMap::new(), } } @@ -148,6 +153,12 @@ impl PhysicalPlanBuilder { pub fn set_mutation_build_info(&mut self, mutation_build_info: MutationBuildInfo) { self.mutation_build_info = Some(mutation_build_info); } + + pub fn next_hash_join_id(&mut self) -> usize { + let id = self.hash_join_id; + self.hash_join_id += 1; + id + } } #[derive(Clone)] diff --git a/src/query/sql/src/executor/physical_plan_visitor.rs b/src/query/sql/src/executor/physical_plan_visitor.rs index 6dcab582c7f34..ee377465b0360 100644 --- a/src/query/sql/src/executor/physical_plan_visitor.rs +++ b/src/query/sql/src/executor/physical_plan_visitor.rs @@ -60,6 +60,8 @@ use crate::executor::physical_plans::ReplaceAsyncSourcer; use crate::executor::physical_plans::ReplaceDeduplicate; use crate::executor::physical_plans::ReplaceInto; use crate::executor::physical_plans::RowFetch; +use crate::executor::physical_plans::RuntimeFilterSink; +use crate::executor::physical_plans::RuntimeFilterSource; use crate::executor::physical_plans::Shuffle; use crate::executor::physical_plans::Sort; use crate::executor::physical_plans::TableScan; @@ -85,6 +87,8 @@ pub trait PhysicalPlanReplacer { PhysicalPlan::Limit(plan) => self.replace_limit(plan), PhysicalPlan::RowFetch(plan) => self.replace_row_fetch(plan), PhysicalPlan::HashJoin(plan) => self.replace_hash_join(plan), + PhysicalPlan::RuntimeFilterSource(_) => Ok(plan.clone()), + PhysicalPlan::RuntimeFilterSink(plan) => self.replace_runtime_filter_sink(plan), PhysicalPlan::Exchange(plan) => self.replace_exchange(plan), PhysicalPlan::ExchangeSource(plan) => self.replace_exchange_source(plan), PhysicalPlan::ExchangeSink(plan) => self.replace_exchange_sink(plan), @@ -249,14 +253,21 @@ pub trait PhysicalPlanReplacer { fn replace_hash_join(&mut self, plan: &HashJoin) -> Result { let build = self.replace(&plan.build)?; let probe = self.replace(&plan.probe)?; + let runtime_filter = if let Some(runtime_filter) = &plan.runtime_filter { + Some(Box::new(self.replace(runtime_filter)?)) + } else { + None + }; Ok(PhysicalPlan::HashJoin(HashJoin { plan_id: plan.plan_id, projections: plan.projections.clone(), probe_projections: plan.probe_projections.clone(), build_projections: plan.build_projections.clone(), + hash_join_id: plan.hash_join_id, build: Box::new(build), probe: Box::new(probe), + runtime_filter, build_keys: plan.build_keys.clone(), probe_keys: plan.probe_keys.clone(), is_null_equal: plan.is_null_equal.clone(), @@ -268,8 +279,9 @@ pub trait PhysicalPlanReplacer { output_schema: plan.output_schema.clone(), need_hold_hash_table: plan.need_hold_hash_table, stat_info: plan.stat_info.clone(), - probe_keys_rt: plan.probe_keys_rt.clone(), - enable_bloom_runtime_filter: plan.enable_bloom_runtime_filter, + runtime_filter_exprs: plan.runtime_filter_exprs.clone(), + support_runtime_filter: plan.support_runtime_filter, + runtime_filter_source_fields: plan.runtime_filter_source_fields.clone(), broadcast: plan.broadcast, single_to_inner: plan.single_to_inner.clone(), build_side_cache_info: plan.build_side_cache_info.clone(), @@ -445,6 +457,13 @@ pub trait PhysicalPlanReplacer { Ok(PhysicalPlan::CompactSource(Box::new(plan.clone()))) } + fn replace_runtime_filter_source( + &mut self, + plan: &RuntimeFilterSource, + ) -> Result { + Ok(PhysicalPlan::RuntimeFilterSource(plan.clone())) + } + fn replace_commit_sink(&mut self, plan: &CommitSink) -> Result { let input = self.replace(&plan.input)?; Ok(PhysicalPlan::CommitSink(Box::new(CommitSink { @@ -495,6 +514,14 @@ pub trait PhysicalPlanReplacer { }))) } + fn replace_runtime_filter_sink(&mut self, plan: &RuntimeFilterSink) -> Result { + let input = self.replace(&plan.input)?; + Ok(PhysicalPlan::RuntimeFilterSink(RuntimeFilterSink { + input: Box::new(input), + ..plan.clone() + })) + } + fn replace_mutation_split(&mut self, plan: &MutationSplit) -> Result { let input = self.replace(&plan.input)?; Ok(PhysicalPlan::MutationSplit(Box::new(MutationSplit { @@ -663,6 +690,8 @@ impl PhysicalPlan { | PhysicalPlan::Recluster(_) | PhysicalPlan::ExchangeSource(_) | PhysicalPlan::CompactSource(_) + | PhysicalPlan::RuntimeFilterSource(_) + | PhysicalPlan::RuntimeFilterSink(_) | PhysicalPlan::MutationSource(_) => {} PhysicalPlan::Filter(plan) => { Self::traverse(&plan.input, pre_visit, visit, post_visit); diff --git a/src/query/sql/src/executor/physical_plans/mod.rs b/src/query/sql/src/executor/physical_plans/mod.rs index f8b1c86702d72..0ad65a87c1714 100644 --- a/src/query/sql/src/executor/physical_plans/mod.rs +++ b/src/query/sql/src/executor/physical_plans/mod.rs @@ -52,6 +52,8 @@ mod physical_replace_async_source; mod physical_replace_deduplicate; mod physical_replace_into; mod physical_row_fetch; +mod physical_runtime_filter_sink; +mod physical_runtime_filter_source; mod physical_sort; mod physical_table_scan; mod physical_udf; @@ -100,6 +102,8 @@ 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_runtime_filter_sink::RuntimeFilterSink; +pub use physical_runtime_filter_source::RuntimeFilterSource; pub use physical_sort::Sort; mod physical_window_partition; pub use physical_table_scan::TableScan; diff --git a/src/query/sql/src/executor/physical_plans/physical_hash_join.rs b/src/query/sql/src/executor/physical_plans/physical_hash_join.rs index 0d947551bd4bb..c812e02160246 100644 --- a/src/query/sql/src/executor/physical_plans/physical_hash_join.rs +++ b/src/query/sql/src/executor/physical_plans/physical_hash_join.rs @@ -14,9 +14,7 @@ use std::collections::HashMap; use std::collections::HashSet; -use std::sync::Arc; -use databend_common_catalog::table_context::TableContext; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::type_check::check_cast; @@ -26,10 +24,10 @@ use databend_common_expression::ConstantFolder; use databend_common_expression::DataField; use databend_common_expression::DataSchemaRef; use databend_common_expression::DataSchemaRefExt; +use databend_common_expression::Expr; use databend_common_expression::RemoteExpr; use databend_common_expression::ROW_NUMBER_COL_NAME; use databend_common_functions::BUILTIN_FUNCTIONS; -use databend_storages_common_table_meta::table::get_change_type; use crate::executor::explain::PlanStatsInfo; use crate::executor::physical_plans::Exchange; @@ -37,13 +35,11 @@ use crate::executor::physical_plans::FragmentKind; use crate::executor::PhysicalPlan; use crate::executor::PhysicalPlanBuilder; use crate::optimizer::ColumnSet; -use crate::optimizer::RelExpr; use crate::optimizer::SExpr; use crate::plans::Join; use crate::plans::JoinType; use crate::ColumnEntry; use crate::IndexType; -use crate::MetadataRef; use crate::ScalarExpr; use crate::TypeCheck; @@ -59,8 +55,10 @@ pub struct HashJoin { pub probe_projections: ColumnSet, pub build_projections: ColumnSet, + pub hash_join_id: usize, pub build: Box, pub probe: Box, + pub runtime_filter: Option>, pub build_keys: Vec, pub probe_keys: Vec, pub is_null_equal: Vec, @@ -80,9 +78,10 @@ pub struct HashJoin { pub stat_info: Option, // probe keys for runtime filter, and record the index of table that used in probe keys. - pub probe_keys_rt: Vec, IndexType)>>, - // If enable bloom runtime filter - pub enable_bloom_runtime_filter: bool, + pub runtime_filter_exprs: Vec, IndexType)>>, + pub runtime_filter_source_fields: Vec, + // If support runtime filter. + pub support_runtime_filter: bool, // Under cluster, mark if the join is broadcast join. pub broadcast: bool, // When left/right single join converted to inner join, record the original join type @@ -109,8 +108,13 @@ impl PhysicalPlanBuilder { mut others_required: ColumnSet, left_required: ColumnSet, right_required: ColumnSet, - stat_info: PlanStatsInfo, + plan_stat_info: PlanStatsInfo, ) -> Result { + let hash_join_id = self.next_hash_join_id(); + let runtime_filter_exprs = self.runtime_filter_exprs(join)?; + self.runtime_filter_columns + .insert(hash_join_id, runtime_filter_columns(&runtime_filter_exprs)); + let mut probe_side = Box::new(self.build(s_expr.child(0)?, left_required).await?); let mut build_side = Box::new(self.build(s_expr.child(1)?, right_required).await?); @@ -121,16 +125,17 @@ impl PhysicalPlanBuilder { others_required = others_required.union(&retained_columns).cloned().collect(); let mut pre_column_projections = others_required.clone().into_iter().collect::>(); - let mut is_broadcast = false; - // Check if join is broadcast join + // Check whether it is a distributed hash join. + let mut is_broadcast_join = false; if let PhysicalPlan::Exchange(Exchange { kind: FragmentKind::Expansive, .. }) = build_side.as_ref() { - is_broadcast = true; + // Broadcast join. + is_broadcast_join = true; } - // Unify the data types of the left and right exchange keys. + let mut is_shuffle_join = false; if let ( PhysicalPlan::Exchange(Exchange { keys: probe_keys, .. @@ -140,127 +145,57 @@ impl PhysicalPlanBuilder { }), ) = (probe_side.as_mut(), build_side.as_mut()) { - for (probe_key, build_key) in probe_keys.iter_mut().zip(build_keys.iter_mut()) { - let probe_expr = probe_key.as_expr(&BUILTIN_FUNCTIONS); - let build_expr = build_key.as_expr(&BUILTIN_FUNCTIONS); - let common_ty = common_super_type( - probe_expr.data_type().clone(), - build_expr.data_type().clone(), - &BUILTIN_FUNCTIONS.default_cast_rules, - ) - .ok_or_else(|| { - ErrorCode::IllegalDataType(format!( - "Cannot find common type for probe key {:?} and build key {:?}", - &probe_expr, &build_expr - )) - })?; - *probe_key = check_cast( - probe_expr.span(), - false, - probe_expr, - &common_ty, - &BUILTIN_FUNCTIONS, - )? - .as_remote_expr(); - *build_key = check_cast( - build_expr.span(), - false, - build_expr, - &common_ty, - &BUILTIN_FUNCTIONS, - )? - .as_remote_expr(); - } + // Shuffle join, unify the data types of the left and right exchange keys. + is_shuffle_join = true; + self.unify_keys_data_type(probe_keys, build_keys)?; } - let build_schema = match join.join_type { - JoinType::Left | JoinType::LeftSingle | JoinType::Full => { - let build_schema = build_side.output_schema()?; - // Wrap nullable type for columns in build side. - let build_schema = DataSchemaRefExt::create( - build_schema - .fields() - .iter() - .map(|field| { - DataField::new(field.name(), field.data_type().wrap_nullable()) - }) - .collect::>(), - ); - build_schema - } - _ => build_side.output_schema()?, - }; - - let probe_schema = match join.join_type { - JoinType::Right | JoinType::RightSingle | JoinType::Full => { - let probe_schema = probe_side.output_schema()?; - // Wrap nullable type for columns in probe side. - let probe_schema = DataSchemaRefExt::create( - probe_schema - .fields() - .iter() - .map(|field| { - DataField::new(field.name(), field.data_type().wrap_nullable()) - }) - .collect::>(), - ); - probe_schema - } - _ => probe_side.output_schema()?, - }; + // The output schema of build and probe side. + let (build_schema, probe_schema) = + self.build_and_probe_output_schema(join, &build_side, &probe_side)?; - let mut left_join_conditions = Vec::new(); - let mut right_join_conditions = Vec::new(); + let mut probe_keys = Vec::new(); + let mut build_keys = Vec::new(); let mut is_null_equal = Vec::new(); - let mut left_join_conditions_rt = Vec::new(); let mut probe_to_build_index = Vec::new(); - let mut table_index = None; + let mut runtime_filter_source_fields = Vec::new(); for condition in join.equi_conditions.iter() { - let left_condition = &condition.left; - let right_condition = &condition.right; - let right_expr = right_condition + let build_condition = &condition.right; + let probe_condition = &condition.left; + let build_expr = build_condition .type_check(build_schema.as_ref())? .project_column_ref(|index| build_schema.index_of(&index.to_string()).unwrap()); - let left_expr = left_condition + let probe_expr = probe_condition .type_check(probe_schema.as_ref())? .project_column_ref(|index| probe_schema.index_of(&index.to_string()).unwrap()); - let left_expr_for_runtime_filter = if left_condition.used_columns().iter().all(|idx| { - // Runtime filter only support column in base table. It's possible to use a wrong derived column with - // the same name as a base table column, so we need to check if the column is a base table column. - matches!( - self.metadata.read().column(*idx), - ColumnEntry::BaseTableColumn(_) - ) - }) { - if let Some(column_idx) = left_condition.used_columns().iter().next() { - // Safe to unwrap because we have checked the column is a base table column. - if table_index.is_none() { - table_index = Some( - self.metadata - .read() - .column(*column_idx) - .table_index() - .unwrap(), - ); - } - Some(( - left_condition - .as_raw_expr() - .type_check(&*self.metadata.read())? - .project_column_ref(|col| col.column_name.clone()), - table_index.unwrap(), - )) - } else { - None - } - } else { - None - }; + // Unify the data types of the probe and right expressions. + let probe_type = probe_expr.data_type(); + let build_type = build_expr.data_type(); + let common_data_type = common_super_type( + probe_type.clone(), + build_type.clone(), + &BUILTIN_FUNCTIONS.default_cast_rules, + ) + .ok_or_else(|| { + ErrorCode::IllegalDataType(format!( + "Cannot find common type for {:?} and {:?}", + probe_type, build_type + )) + })?; + + if let Some((_, build_index)) = + self.support_runtime_filter(probe_condition, build_condition)? + { + let build_index = build_schema.index_of(&build_index.to_string())?; + let build_field = build_schema.field(build_index); + runtime_filter_source_fields + .push(DataField::new(build_field.name(), common_data_type.clone())); + } if join.join_type == JoinType::Inner { if let (ScalarExpr::BoundColumnRef(left), ScalarExpr::BoundColumnRef(right)) = - (left_condition, right_condition) + (probe_condition, build_condition) { if column_projections.contains(&right.column.index) { if let (Ok(probe_index), Ok(build_index)) = ( @@ -285,67 +220,57 @@ impl PhysicalPlanBuilder { } } } - // Unify the data types of the left and right expressions. - let left_type = left_expr.data_type(); - let right_type = right_expr.data_type(); - let common_ty = common_super_type( - left_type.clone(), - right_type.clone(), - &BUILTIN_FUNCTIONS.default_cast_rules, - ) - .ok_or_else(|| { - ErrorCode::IllegalDataType(format!( - "Cannot find common type for {:?} and {:?}", - left_type, right_type - )) - })?; - let left_expr = check_cast( - left_expr.span(), + + let probe_expr = check_cast( + probe_expr.span(), false, - left_expr, - &common_ty, + probe_expr, + &common_data_type, &BUILTIN_FUNCTIONS, )?; - let right_expr = check_cast( - right_expr.span(), + let build_expr = check_cast( + build_expr.span(), false, - right_expr, - &common_ty, + build_expr, + &common_data_type, &BUILTIN_FUNCTIONS, )?; - let left_expr_for_runtime_filter = left_expr_for_runtime_filter - .map(|(expr, idx)| { - check_cast(expr.span(), false, expr, &common_ty, &BUILTIN_FUNCTIONS) - .map(|casted_expr| (casted_expr, idx)) - }) - .transpose()?; - - let (left_expr, _) = - ConstantFolder::fold(&left_expr, &self.func_ctx, &BUILTIN_FUNCTIONS); - let (right_expr, _) = - ConstantFolder::fold(&right_expr, &self.func_ctx, &BUILTIN_FUNCTIONS); - - let left_expr_for_runtime_filter = left_expr_for_runtime_filter.map(|(expr, idx)| { - ( - ConstantFolder::fold(&expr, &self.func_ctx, &BUILTIN_FUNCTIONS).0, - idx, - ) - }); + let (probe_expr, _) = + ConstantFolder::fold(&probe_expr, &self.func_ctx, &BUILTIN_FUNCTIONS); + let (build_expr, _) = + ConstantFolder::fold(&build_expr, &self.func_ctx, &BUILTIN_FUNCTIONS); - left_join_conditions.push(left_expr.as_remote_expr()); - right_join_conditions.push(right_expr.as_remote_expr()); + probe_keys.push(probe_expr.as_remote_expr()); + build_keys.push(build_expr.as_remote_expr()); is_null_equal.push(condition.is_null_equal); - left_join_conditions_rt - .push(left_expr_for_runtime_filter.map(|(expr, idx)| (expr.as_remote_expr(), idx))); } + // For shuffle join, we need to build global runtime filter. + let support_runtime_filter = supported_runtime_filter_join_type(&join.join_type); + let runtime_filter_plan = if support_runtime_filter + && !self.ctx.get_cluster().is_empty() + && is_shuffle_join + && !runtime_filter_source_fields.is_empty() + { + runtime_filter_source_fields.push(DataField::new("node_id", DataType::String)); + runtime_filter_source_fields.push(DataField::new("need_to_build", DataType::Boolean)); + Some(self.build_runtime_filter_plan( + hash_join_id, + DataSchemaRefExt::create(runtime_filter_source_fields.clone()), + )?) + } else { + None + }; + + // Cache scan info. let mut cache_column_map = HashMap::new(); let cached_column = if let Some(cache_info) = &join.build_side_cache_info { cache_info.columns.clone().into_iter().collect() } else { HashSet::new() }; + pre_column_projections.extend(cached_column.iter()); let mut probe_projections = ColumnSet::new(); let mut build_projections = ColumnSet::new(); @@ -368,12 +293,6 @@ impl PhysicalPlanBuilder { None }; - // for distributed merge into, there is a field called "_row_number", but - // it's not an internal row_number, we need to add it here - if let Some((index, _)) = build_schema.column_with_name(ROW_NUMBER_COL_NAME) { - build_projections.insert(index); - } - let mut merged_fields = Vec::with_capacity(probe_projections.len() + build_projections.len()); let mut probe_fields = Vec::with_capacity(probe_projections.len()); @@ -514,13 +433,16 @@ impl PhysicalPlanBuilder { projections, build_projections, probe_projections, + hash_join_id, build: build_side, probe: probe_side, + runtime_filter: runtime_filter_plan, join_type: join.join_type.clone(), - build_keys: right_join_conditions, - probe_keys: left_join_conditions, + build_keys, + probe_keys, is_null_equal, - probe_keys_rt: left_join_conditions_rt, + runtime_filter_exprs, + runtime_filter_source_fields, non_equi_conditions: join .non_equi_conditions .iter() @@ -539,48 +461,252 @@ impl PhysicalPlanBuilder { probe_to_build, output_schema, need_hold_hash_table: join.need_hold_hash_table, - stat_info: Some(stat_info), - broadcast: is_broadcast, + stat_info: Some(plan_stat_info), + broadcast: is_broadcast_join, single_to_inner: join.single_to_inner.clone(), - enable_bloom_runtime_filter: adjust_bloom_runtime_filter( - self.ctx.clone(), - &self.metadata, - table_index, - s_expr, - ) - .await?, + support_runtime_filter, build_side_cache_info, })) } -} -// Check if enable bloom runtime filter -async fn adjust_bloom_runtime_filter( - ctx: Arc, - metadata: &MetadataRef, - table_index: Option, - s_expr: &SExpr, -) -> Result { - // The setting of `enable_bloom_runtime_filter` is true by default. - if !ctx.get_settings().get_bloom_runtime_filter()? { - return Ok(false); + fn unify_keys_data_type( + &self, + probe_keys: &mut [RemoteExpr], + build_keys: &mut [RemoteExpr], + ) -> Result<()> { + for (probe_key, build_key) in probe_keys.iter_mut().zip(build_keys.iter_mut()) { + let probe_expr = probe_key.as_expr(&BUILTIN_FUNCTIONS); + let build_expr = build_key.as_expr(&BUILTIN_FUNCTIONS); + let common_ty = common_super_type( + probe_expr.data_type().clone(), + build_expr.data_type().clone(), + &BUILTIN_FUNCTIONS.default_cast_rules, + ) + .ok_or_else(|| { + ErrorCode::IllegalDataType(format!( + "Cannot find common type for probe key {:?} and build key {:?}", + &probe_expr, &build_expr + )) + })?; + *probe_key = check_cast( + probe_expr.span(), + false, + probe_expr, + &common_ty, + &BUILTIN_FUNCTIONS, + )? + .as_remote_expr(); + *build_key = check_cast( + build_expr.span(), + false, + build_expr, + &common_ty, + &BUILTIN_FUNCTIONS, + )? + .as_remote_expr(); + } + + Ok(()) } - if let Some(table_index) = table_index { - let table_entry = metadata.read().table(table_index).clone(); - let change_type = get_change_type(table_entry.alias_name()); - let table = table_entry.table(); - if let Some(stats) = table - .table_statistics(ctx.clone(), true, change_type) - .await? - { - if let Some(num_rows) = stats.num_rows { - let join_cardinality = RelExpr::with_s_expr(s_expr) - .derive_cardinality()? - .cardinality; - // If the filtered data reduces to less than 1/1000 of the original dataset, we will enable bloom runtime filter. - return Ok(join_cardinality <= (num_rows / 1000) as f64); + + fn build_and_probe_output_schema( + &self, + join: &Join, + build: &PhysicalPlan, + probe: &PhysicalPlan, + ) -> Result<(DataSchemaRef, DataSchemaRef)> { + let build_schema = match join.join_type { + JoinType::Left | JoinType::LeftSingle | JoinType::Full => { + // Wrap nullable type for columns in build side. + DataSchemaRefExt::create( + build + .output_schema()? + .fields() + .iter() + .map(|field| { + DataField::new(field.name(), field.data_type().wrap_nullable()) + }) + .collect::>(), + ) + } + _ => build.output_schema()?, + }; + + let probe_schema = match join.join_type { + JoinType::Right | JoinType::RightSingle | JoinType::Full => { + // Wrap nullable type for columns in probe side. + DataSchemaRefExt::create( + probe + .output_schema()? + .fields() + .iter() + .map(|field| { + DataField::new(field.name(), field.data_type().wrap_nullable()) + }) + .collect::>(), + ) } + _ => probe.output_schema()?, + }; + + Ok((build_schema, probe_schema)) + } + + fn build_runtime_filter_plan( + &mut self, + hash_join_id: usize, + runtime_filter_source_output_schema: DataSchemaRef, + ) -> Result> { + let runtime_filter_source = + self.build_runtime_filter_source(hash_join_id, runtime_filter_source_output_schema)?; + Ok(Box::new(self.build_runtime_filter_sink( + hash_join_id, + Box::new(PhysicalPlan::Exchange(Exchange { + plan_id: 0, + input: Box::new(runtime_filter_source), + kind: FragmentKind::Expansive, + keys: vec![], + allow_adjust_parallelism: true, + ignore_exchange: false, + })), + )?)) + } + + pub fn support_runtime_filter( + &self, + probe_condition: &ScalarExpr, + build_condition: &ScalarExpr, + ) -> Result> { + // Runtime filter only support column in base table. It's possible to use a wrong derived column with + // the same name as a base table column, so we need to check if the column is a base table column. + if let (ScalarExpr::BoundColumnRef(probe_column), ScalarExpr::BoundColumnRef(build_column)) = + (probe_condition, build_condition) + && matches!( + self.metadata.read().column(probe_column.column.index), + ColumnEntry::BaseTableColumn(_) + ) + && matches!( + self.metadata.read().column(build_column.column.index), + ColumnEntry::BaseTableColumn(_) + ) + && supported_runtime_filter_data_type(&probe_condition.data_type()?) + { + Ok(Some((probe_column.column.index, build_column.column.index))) + } else { + Ok(None) + } + } + + #[allow(clippy::type_complexity)] + pub fn runtime_filter_exprs( + &self, + join: &Join, + ) -> Result, IndexType)>>> { + let mut runtime_filter_exprs = Vec::with_capacity(join.equi_conditions.len()); + for condition in join.equi_conditions.iter() { + let build_condition = &condition.right; + let probe_condition = &condition.left; + // Runtime filter only support column in base table. It's possible to use a wrong derived column with + // the same name as a base table column, so we need to check if the column is a base table column. + let runtime_filter_expr = if let Some((probe_index, _)) = + self.support_runtime_filter(probe_condition, build_condition)? + { + // Safe to unwrap because we have checked the column is a base table column. + let table_index = self + .metadata + .read() + .column(probe_index) + .table_index() + .unwrap(); + Some(( + probe_condition + .as_raw_expr() + .type_check(&*self.metadata.read())? + .project_column_ref(|col| col.column_name.clone()), + table_index, + )) + } else { + None + }; + + let probe_data_type = probe_condition.data_type()?; + let build_data_type = build_condition.data_type()?; + let common_data_type = common_super_type( + probe_data_type.clone(), + build_data_type.clone(), + &BUILTIN_FUNCTIONS.default_cast_rules, + ) + .ok_or_else(|| { + ErrorCode::IllegalDataType(format!( + "Cannot find common type for {:?} and {:?}", + probe_data_type, build_data_type + )) + })?; + let runtime_filter_expr = runtime_filter_expr + .map(|(expr, idx)| { + check_cast( + expr.span(), + false, + expr, + &common_data_type, + &BUILTIN_FUNCTIONS, + ) + .map(|casted_expr| (casted_expr, idx)) + }) + .transpose()?; + + let runtime_filter_expr = runtime_filter_expr.map(|(expr, table_index)| { + ( + ConstantFolder::fold(&expr, &self.func_ctx, &BUILTIN_FUNCTIONS).0, + table_index, + ) + }); + + runtime_filter_exprs + .push(runtime_filter_expr.map(|(expr, index)| (expr.as_remote_expr(), index))); + } + Ok(runtime_filter_exprs) + } +} + +pub fn runtime_filter_columns( + runtime_filter_exprs: &[Option<(RemoteExpr, IndexType)>], +) -> Vec<(usize, String)> { + let runtime_filter_exprs: Vec, IndexType)>> = runtime_filter_exprs + .iter() + .map(|runtime_filter_expr| { + runtime_filter_expr + .as_ref() + .map(|(expr, table_index)| (expr.as_expr(&BUILTIN_FUNCTIONS), *table_index)) + }) + .collect(); + let mut columns = Vec::new(); + for (probe_key, table_index) in runtime_filter_exprs + .iter() + .filter_map(|runtime_filter_expr| { + runtime_filter_expr + .as_ref() + .map(|(probe_key, table_index)| (probe_key, table_index)) + }) + { + if let Some(column_name) = Expr::::column_id(probe_key) { + columns.push((*table_index, column_name)); } } - Ok(false) + columns +} + +pub fn supported_runtime_filter_data_type(data_type: &DataType) -> bool { + let data_type = data_type.remove_nullable(); + data_type.is_numeric() + || data_type.is_string() + || data_type.is_date() + || data_type.is_timestamp() +} + +pub fn supported_runtime_filter_join_type(join_type: &JoinType) -> bool { + matches!( + join_type, + JoinType::Inner | JoinType::Right | JoinType::RightSemi | JoinType::LeftMark + ) } diff --git a/src/query/sql/src/executor/physical_plans/physical_join.rs b/src/query/sql/src/executor/physical_plans/physical_join.rs index 9ef68c7a53018..7313dd8e4b2af 100644 --- a/src/query/sql/src/executor/physical_plans/physical_join.rs +++ b/src/query/sql/src/executor/physical_plans/physical_join.rs @@ -111,7 +111,7 @@ impl PhysicalPlanBuilder { s_expr: &SExpr, join: &crate::plans::Join, required: ColumnSet, - stat_info: PlanStatsInfo, + plan_stat_info: PlanStatsInfo, ) -> Result { // 1. Prune unused Columns. let mut others_required = join @@ -160,7 +160,7 @@ impl PhysicalPlanBuilder { others_required, left_required, right_required, - stat_info, + plan_stat_info, ) .await } diff --git a/src/query/sql/src/executor/physical_plans/physical_runtime_filter_sink.rs b/src/query/sql/src/executor/physical_plans/physical_runtime_filter_sink.rs new file mode 100644 index 0000000000000..26f4f3ea9446b --- /dev/null +++ b/src/query/sql/src/executor/physical_plans/physical_runtime_filter_sink.rs @@ -0,0 +1,40 @@ +// 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 crate::executor::PhysicalPlan; +use crate::executor::PhysicalPlanBuilder; + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct RuntimeFilterSink { + // A unique id of operator in a `PhysicalPlan` tree, only used for display. + pub plan_id: u32, + pub hash_join_id: usize, + pub input: Box, +} + +impl PhysicalPlanBuilder { + pub(crate) fn build_runtime_filter_sink( + &mut self, + hash_join_id: usize, + input: Box, + ) -> Result { + Ok(PhysicalPlan::RuntimeFilterSink(RuntimeFilterSink { + plan_id: 0, + hash_join_id, + input, + })) + } +} diff --git a/src/query/sql/src/executor/physical_plans/physical_runtime_filter_source.rs b/src/query/sql/src/executor/physical_plans/physical_runtime_filter_source.rs new file mode 100644 index 0000000000000..5ce00ace09275 --- /dev/null +++ b/src/query/sql/src/executor/physical_plans/physical_runtime_filter_source.rs @@ -0,0 +1,47 @@ +// 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::DataSchemaRef; + +use crate::executor::PhysicalPlan; +use crate::executor::PhysicalPlanBuilder; + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct RuntimeFilterSource { + // A unique id of operator in a `PhysicalPlan` tree, only used for display. + pub plan_id: u32, + pub hash_join_id: usize, + pub output_schema: DataSchemaRef, +} + +impl RuntimeFilterSource { + pub fn output_schema(&self) -> Result { + Ok(self.output_schema.clone()) + } +} + +impl PhysicalPlanBuilder { + pub(crate) fn build_runtime_filter_source( + &mut self, + hash_join_id: usize, + output_schema: DataSchemaRef, + ) -> Result { + Ok(PhysicalPlan::RuntimeFilterSource(RuntimeFilterSource { + plan_id: 0, + hash_join_id, + output_schema, + })) + } +} diff --git a/src/query/sql/src/executor/physical_plans/physical_table_scan.rs b/src/query/sql/src/executor/physical_plans/physical_table_scan.rs index 09d3bc58b6a4b..d5b35903cd414 100644 --- a/src/query/sql/src/executor/physical_plans/physical_table_scan.rs +++ b/src/query/sql/src/executor/physical_plans/physical_table_scan.rs @@ -77,6 +77,8 @@ pub struct TableScan { pub table_index: Option, pub stat_info: Option, + + pub runtime_filter_columns: Vec<(usize, String)>, } impl TableScan { @@ -281,6 +283,18 @@ impl PhysicalPlanBuilder { metadata.set_table_source(scan.table_index, source.clone()); } + let runtime_filter_columns = if scan.is_merge_into_target { + let mut runtime_filter_columns = Vec::new(); + for (hash_join_id, columns) in self.runtime_filter_columns.iter() { + for (_, column_name) in columns.iter() { + runtime_filter_columns.push((*hash_join_id, column_name.clone())) + } + } + runtime_filter_columns + } else { + vec![] + }; + let mut plan = PhysicalPlan::TableScan(TableScan { plan_id: 0, name_mapping, @@ -288,6 +302,7 @@ impl PhysicalPlanBuilder { table_index: Some(scan.table_index), stat_info: Some(stat_info), internal_column, + runtime_filter_columns, }); // Update stream columns if needed. @@ -326,6 +341,7 @@ impl PhysicalPlanBuilder { estimated_rows: 1.0, }), internal_column: None, + runtime_filter_columns: vec![], })) } diff --git a/src/query/sql/src/planner/binder/bind_mutation/mutation_expression.rs b/src/query/sql/src/planner/binder/bind_mutation/mutation_expression.rs index 9634e7ae0cc07..9547d4eb67e4e 100644 --- a/src/query/sql/src/planner/binder/bind_mutation/mutation_expression.rs +++ b/src/query/sql/src/planner/binder/bind_mutation/mutation_expression.rs @@ -383,6 +383,7 @@ impl MutationExpression { RelOperator::Scan(scan) => { let mut scan = scan.clone(); scan.is_lazy_table = is_lazy_table; + scan.is_merge_into_target = true; scan.set_update_stream_columns(update_stream_columns); Ok(SExpr::create_leaf(Arc::new(scan.into()))) } diff --git a/src/query/sql/src/planner/binder/bind_table_reference/bind_join.rs b/src/query/sql/src/planner/binder/bind_table_reference/bind_join.rs index 87f428aa55913..c4da053a55b79 100644 --- a/src/query/sql/src/planner/binder/bind_table_reference/bind_join.rs +++ b/src/query/sql/src/planner/binder/bind_table_reference/bind_join.rs @@ -120,6 +120,7 @@ impl Binder { left_child, right_child, build_side_cache_info, + false, )?; let bind_context = join_bind_context(&join_type, bind_context, left_context, right_context); @@ -163,6 +164,7 @@ impl Binder { left_child, right_child, None, + true, )?; let bind_context = join_bind_context(&join_type, bind_context, left_context, right_context); @@ -217,6 +219,7 @@ impl Binder { mut left_child: SExpr, mut right_child: SExpr, build_side_cache_info: Option, + is_merge_into_join: bool, ) -> Result { let mut left_conditions = join_conditions.left_conditions; let mut right_conditions = join_conditions.right_conditions; @@ -291,7 +294,7 @@ impl Binder { right_conditions, ) }; - let logical_join = Join { + let join = Join { equi_conditions: JoinEquiCondition::new_conditions( left_conditions, right_conditions, @@ -305,9 +308,10 @@ impl Binder { is_lateral, single_to_inner: None, build_side_cache_info, + is_merge_into_join, }; Ok(SExpr::create_binary( - Arc::new(logical_join.into()), + Arc::new(join.into()), Arc::new(left_child), Arc::new(right_child), )) diff --git a/src/query/sql/src/planner/binder/select.rs b/src/query/sql/src/planner/binder/select.rs index c52f5679a6ce2..d54b666a40fda 100644 --- a/src/query/sql/src/planner/binder/select.rs +++ b/src/query/sql/src/planner/binder/select.rs @@ -382,8 +382,14 @@ impl Binder { non_equi_conditions: vec![], other_conditions: vec![], }; - let s_expr = - self.bind_join_with_type(join_type, join_conditions, left_expr, right_expr, None)?; + let s_expr = self.bind_join_with_type( + join_type, + join_conditions, + left_expr, + right_expr, + None, + false, + )?; Ok((s_expr, left_context)) } diff --git a/src/query/sql/src/planner/optimizer/decorrelate/decorrelate.rs b/src/query/sql/src/planner/optimizer/decorrelate/decorrelate.rs index 6a3f074260fb6..9b5c353ae4c24 100644 --- a/src/query/sql/src/planner/optimizer/decorrelate/decorrelate.rs +++ b/src/query/sql/src/planner/optimizer/decorrelate/decorrelate.rs @@ -201,6 +201,7 @@ impl SubqueryRewriter { is_lateral: false, single_to_inner: None, build_side_cache_info: None, + is_merge_into_join: false, }; // Rewrite plan to semi-join. @@ -295,6 +296,7 @@ impl SubqueryRewriter { is_lateral: false, single_to_inner: None, build_side_cache_info: None, + is_merge_into_join: false, }; let s_expr = SExpr::create_binary( Arc::new(join_plan.into()), @@ -349,6 +351,7 @@ impl SubqueryRewriter { is_lateral: false, single_to_inner: None, build_side_cache_info: None, + is_merge_into_join: false, }; let s_expr = SExpr::create_binary( Arc::new(join_plan.into()), @@ -418,6 +421,7 @@ impl SubqueryRewriter { is_lateral: false, single_to_inner: None, build_side_cache_info: None, + is_merge_into_join: false, } .into(); Ok(( diff --git a/src/query/sql/src/planner/optimizer/decorrelate/flatten_plan.rs b/src/query/sql/src/planner/optimizer/decorrelate/flatten_plan.rs index 3d18059714edd..0fb423ff378a4 100644 --- a/src/query/sql/src/planner/optimizer/decorrelate/flatten_plan.rs +++ b/src/query/sql/src/planner/optimizer/decorrelate/flatten_plan.rs @@ -164,6 +164,7 @@ impl SubqueryRewriter { is_lateral: false, single_to_inner: None, build_side_cache_info: None, + is_merge_into_join: false, } .into(); @@ -522,6 +523,7 @@ impl SubqueryRewriter { is_lateral: false, single_to_inner: None, build_side_cache_info: None, + is_merge_into_join: false, } .into(), ), diff --git a/src/query/sql/src/planner/optimizer/decorrelate/subquery_rewriter.rs b/src/query/sql/src/planner/optimizer/decorrelate/subquery_rewriter.rs index d973b91a69447..35a9a72940c87 100644 --- a/src/query/sql/src/planner/optimizer/decorrelate/subquery_rewriter.rs +++ b/src/query/sql/src/planner/optimizer/decorrelate/subquery_rewriter.rs @@ -435,6 +435,7 @@ impl SubqueryRewriter { is_lateral: false, single_to_inner: None, build_side_cache_info: None, + is_merge_into_join: false, } .into(); let s_expr = SExpr::create_binary( @@ -550,6 +551,7 @@ impl SubqueryRewriter { is_lateral: false, single_to_inner: None, build_side_cache_info: None, + is_merge_into_join: false, } .into(); Ok(( @@ -625,6 +627,7 @@ impl SubqueryRewriter { is_lateral: false, single_to_inner: None, build_side_cache_info: None, + is_merge_into_join: false, } .into(); let s_expr = SExpr::create_binary( diff --git a/src/query/sql/src/planner/optimizer/hyper_dp/join_node.rs b/src/query/sql/src/planner/optimizer/hyper_dp/join_node.rs index 15ca730ae0818..6396524efe9ea 100644 --- a/src/query/sql/src/planner/optimizer/hyper_dp/join_node.rs +++ b/src/query/sql/src/planner/optimizer/hyper_dp/join_node.rs @@ -95,6 +95,7 @@ impl JoinNode { is_lateral: false, single_to_inner: None, build_side_cache_info: None, + is_merge_into_join: false, }); let children = self .children diff --git a/src/query/sql/src/planner/optimizer/s_expr.rs b/src/query/sql/src/planner/optimizer/s_expr.rs index d56f921e50536..1adb04d380275 100644 --- a/src/query/sql/src/planner/optimizer/s_expr.rs +++ b/src/query/sql/src/planner/optimizer/s_expr.rs @@ -119,6 +119,10 @@ impl SExpr { self.original_group } + pub fn stat_info(&self) -> Option> { + self.stat_info.lock().unwrap().clone() + } + /// Replace children with given new `children`. /// Note that this method will keep the `applied_rules` of /// current `SExpr` unchanged. diff --git a/src/query/sql/src/planner/plans/join.rs b/src/query/sql/src/planner/plans/join.rs index 643d44058972c..9dae30439f044 100644 --- a/src/query/sql/src/planner/plans/join.rs +++ b/src/query/sql/src/planner/plans/join.rs @@ -201,6 +201,7 @@ pub struct Join { pub single_to_inner: Option, // Cache info for ExpressionScan. pub build_side_cache_info: Option, + pub is_merge_into_join: bool, } impl Default for Join { @@ -215,6 +216,7 @@ impl Default for Join { is_lateral: false, single_to_inner: None, build_side_cache_info: None, + is_merge_into_join: false, } } } diff --git a/src/query/sql/src/planner/plans/scan.rs b/src/query/sql/src/planner/plans/scan.rs index 72124bf43945d..6dd0d78cd52be 100644 --- a/src/query/sql/src/planner/plans/scan.rs +++ b/src/query/sql/src/planner/plans/scan.rs @@ -107,6 +107,8 @@ pub struct Scan { // Lazy row fetch. pub is_lazy_table: bool, pub sample: Option, + // Merge into target table + pub is_merge_into_target: bool, pub statistics: Arc, } @@ -146,6 +148,7 @@ impl Scan { update_stream_columns: self.update_stream_columns, inverted_index: self.inverted_index.clone(), is_lazy_table: self.is_lazy_table, + is_merge_into_target: self.is_merge_into_target, sample: self.sample.clone(), } } diff --git a/src/query/storages/common/io/src/merge_io_result.rs b/src/query/storages/common/io/src/merge_io_result.rs index bdb6062f6e2b6..ab39cd9e2bb42 100644 --- a/src/query/storages/common/io/src/merge_io_result.rs +++ b/src/query/storages/common/io/src/merge_io_result.rs @@ -21,6 +21,7 @@ use databend_common_exception::Result; use databend_common_expression::ColumnId; type ChunkIndex = usize; +#[derive(Clone)] pub struct OwnerMemory { pub chunks: HashMap, } @@ -45,6 +46,7 @@ impl OwnerMemory { } } +#[derive(Clone)] pub struct MergeIOReadResult { pub block_path: String, pub columns_chunk_offsets: HashMap)>, diff --git a/src/query/storages/fuse/src/io/read/block/block_reader.rs b/src/query/storages/fuse/src/io/read/block/block_reader.rs index 975bcd1e8d777..6829819886576 100644 --- a/src/query/storages/fuse/src/io/read/block/block_reader.rs +++ b/src/query/storages/fuse/src/io/read/block/block_reader.rs @@ -55,7 +55,10 @@ pub struct BlockReader { pub native_columns_reader: NativeColumnsReader, } -fn inner_project_field_default_values(default_vals: &[Scalar], paths: &[usize]) -> Result { +pub fn inner_project_field_default_values( + default_vals: &[Scalar], + paths: &[usize], +) -> Result { if paths.is_empty() { return Err(ErrorCode::BadArguments( "path should not be empty".to_string(), diff --git a/src/query/storages/fuse/src/io/read/block/block_reader_merge_io.rs b/src/query/storages/fuse/src/io/read/block/block_reader_merge_io.rs index 5882015807b3c..da52d87c9f0b3 100644 --- a/src/query/storages/fuse/src/io/read/block/block_reader_merge_io.rs +++ b/src/query/storages/fuse/src/io/read/block/block_reader_merge_io.rs @@ -31,6 +31,7 @@ pub enum DataItem<'a> { ColumnArray(&'a Arc), } +#[derive(Clone)] pub struct BlockReadResult { merge_io_result: MergeIOReadResult, pub(crate) cached_column_data: CachedColumnData, diff --git a/src/query/storages/fuse/src/io/read/block/mod.rs b/src/query/storages/fuse/src/io/read/block/mod.rs index 09d37197dcc28..04c7ac3340e0f 100644 --- a/src/query/storages/fuse/src/io/read/block/mod.rs +++ b/src/query/storages/fuse/src/io/read/block/mod.rs @@ -22,7 +22,9 @@ mod block_reader_native_deserialize; mod block_reader_parquet_deserialize; pub mod parquet; +pub use block_reader::inner_project_field_default_values; pub use block_reader::BlockReader; pub use block_reader_merge_io::BlockReadResult; +pub use block_reader_merge_io::DataItem; pub use block_reader_native::NativeReaderExt; pub use block_reader_native::NativeSourceData; diff --git a/src/query/storages/fuse/src/io/read/mod.rs b/src/query/storages/fuse/src/io/read/mod.rs index e08107c547fa9..ec3214a8e0018 100644 --- a/src/query/storages/fuse/src/io/read/mod.rs +++ b/src/query/storages/fuse/src/io/read/mod.rs @@ -22,8 +22,10 @@ mod utils; mod virtual_column; pub use agg_index::AggIndexReader; +pub use block::inner_project_field_default_values; pub use block::BlockReadResult; pub use block::BlockReader; +pub use block::DataItem; pub use block::NativeReaderExt; pub use block::NativeSourceData; pub use bloom::BloomBlockFilterReader; diff --git a/src/query/storages/fuse/src/operations/read/fuse_rows_fetcher.rs b/src/query/storages/fuse/src/operations/read/fuse_rows_fetcher.rs index a68f9d49d3752..1774b1310cb4a 100644 --- a/src/query/storages/fuse/src/operations/read/fuse_rows_fetcher.rs +++ b/src/query/storages/fuse/src/operations/read/fuse_rows_fetcher.rs @@ -203,11 +203,16 @@ where F: RowsFetcher + Send + Sync + 'static fetcher: F, need_wrap_nullable: bool, ) -> ProcessorPtr { - ProcessorPtr::create(AsyncTransformer::create(input, output, Self { - row_id_col_offset, - fetcher, - need_wrap_nullable, - })) + ProcessorPtr::create(AsyncTransformer::create_with_batch_size( + input, + output, + 8, + Self { + row_id_col_offset, + fetcher, + need_wrap_nullable, + }, + )) } } diff --git a/src/query/storages/fuse/src/operations/read/fuse_source.rs b/src/query/storages/fuse/src/operations/read/fuse_source.rs index 214df13b9f8cc..7974bbb9d001a 100644 --- a/src/query/storages/fuse/src/operations/read/fuse_source.rs +++ b/src/query/storages/fuse/src/operations/read/fuse_source.rs @@ -28,7 +28,9 @@ use databend_common_pipeline_core::processors::OutputPort; use databend_common_pipeline_core::Pipe; use databend_common_pipeline_core::Pipeline; use databend_common_pipeline_core::SourcePipeBuilder; +use databend_storages_common_io::ReadSettings; use log::info; +use opendal::Operator; use crate::fuse_part::FuseBlockPartInfo; use crate::io::AggIndexReader; @@ -40,6 +42,9 @@ use crate::operations::read::native_data_transform_reader::ReadNativeDataTransfo use crate::operations::read::parquet_data_transform_reader::ReadParquetDataTransform; use crate::operations::read::DeserializeDataTransform; use crate::operations::read::NativeDeserializeDataTransform; +use crate::operations::read::PartitionDeserializer; +use crate::operations::read::PartitionReader; +use crate::operations::read::PartitionScanState; #[allow(clippy::too_many_arguments)] pub fn build_fuse_native_source_pipeline( @@ -253,9 +258,76 @@ pub fn build_fuse_parquet_source_pipeline( index_reader.clone(), virtual_reader.clone(), ) - })?; + }) +} - Ok(()) +#[allow(clippy::too_many_arguments)] +pub fn build_partition_source_pipeline( + pipeline: &mut Pipeline, + plan: &DataSourcePlan, + ctx: Arc, + table_schema: Arc, + max_threads: usize, + max_io_requests: usize, + bloom_filter_columns: Vec<(usize, String)>, + column_indices: Vec, + operator: Operator, + put_cache: bool, +) -> Result<()> { + let (max_threads, max_io_requests) = + adjust_threads_and_request(false, max_threads, max_io_requests, plan); + + let partitions = dispatch_partitions(ctx.clone(), plan, max_io_requests); + let partitions = StealablePartitions::new(partitions, ctx.clone()); + let partition_scan_state = Arc::new(PartitionScanState::new()); + let read_settings = ReadSettings::from_ctx(&partitions.ctx)?; + let (sender, receiver) = async_channel::unbounded(); + + let mut source_builder = SourcePipeBuilder::create(); + for i in 0..max_io_requests { + let output = OutputPort::create(); + source_builder.add_source( + output.clone(), + PartitionReader::create( + output, + i, + ctx.clone(), + plan.table_index, + table_schema.clone(), + read_settings, + partitions.clone(), + bloom_filter_columns.clone(), + column_indices.clone(), + partition_scan_state.clone(), + receiver.clone(), + operator.clone(), + plan.query_internal_columns, + plan.update_stream_columns, + put_cache, + )?, + ); + } + pipeline.add_pipe(source_builder.finalize()); + + pipeline.try_resize(std::cmp::min(max_threads, max_io_requests))?; + if max_threads < max_io_requests { + info!( + "read block pipeline resize from:{} to:{}", + max_io_requests, + pipeline.output_len() + ); + } + + pipeline.add_transform(|transform_input, transform_output| { + PartitionDeserializer::create( + plan, + ctx.clone(), + transform_input, + transform_output, + partition_scan_state.clone(), + sender.clone(), + ) + }) } pub fn dispatch_partitions( diff --git a/src/query/storages/fuse/src/operations/read/mod.rs b/src/query/storages/fuse/src/operations/read/mod.rs index 26494de63cbe9..7fce1dfb01cc8 100644 --- a/src/query/storages/fuse/src/operations/read/mod.rs +++ b/src/query/storages/fuse/src/operations/read/mod.rs @@ -22,7 +22,11 @@ mod parquet_data_source; mod parquet_data_source_deserializer; mod parquet_data_transform_reader; mod parquet_rows_fetcher; +mod partition_deserializer; +mod partition_reader; +mod partition_scan_meta; mod runtime_filter_prunner; +mod source_reader; mod block_partition_meta; mod block_partition_receiver_source; @@ -32,6 +36,14 @@ mod util; pub use fuse_rows_fetcher::row_fetch_processor; pub use fuse_source::build_fuse_parquet_source_pipeline; +pub use fuse_source::build_partition_source_pipeline; pub use native_data_source_deserializer::NativeDeserializeDataTransform; pub use parquet_data_source_deserializer::DeserializeDataTransform; +pub use partition_deserializer::PartitionDeserializer; +pub use partition_reader::PartitionReader; +pub use partition_reader::PartitionScanState; +pub use partition_scan_meta::PartitionScanMeta; +pub use source_reader::ReaderState; +pub use source_reader::SourceBlockReader; +pub use source_reader::SourceReader; pub use util::need_reserve_block_info; diff --git a/src/query/storages/fuse/src/operations/read/native_data_source_deserializer.rs b/src/query/storages/fuse/src/operations/read/native_data_source_deserializer.rs index db5387aa92add..bb329e8e6c913 100644 --- a/src/query/storages/fuse/src/operations/read/native_data_source_deserializer.rs +++ b/src/query/storages/fuse/src/operations/read/native_data_source_deserializer.rs @@ -223,7 +223,7 @@ pub struct NativeDeserializeDataTransform { // Structures for the bloom runtime filter: ctx: Arc, - bloom_runtime_filter: Option>, + bloom_runtime_filter: Option)>>, // Structures for aggregating index: index_reader: Arc>, diff --git a/src/query/storages/fuse/src/operations/read/parquet_data_source_deserializer.rs b/src/query/storages/fuse/src/operations/read/parquet_data_source_deserializer.rs index c4c6d9d916f63..e75e830dace23 100644 --- a/src/query/storages/fuse/src/operations/read/parquet_data_source_deserializer.rs +++ b/src/query/storages/fuse/src/operations/read/parquet_data_source_deserializer.rs @@ -71,7 +71,7 @@ pub struct DeserializeDataTransform { virtual_reader: Arc>, base_block_ids: Option, - cached_runtime_filter: Option>, + cached_runtime_filter: Option)>>, // for merge_into target build. need_reserve_block_info: bool, } @@ -141,7 +141,7 @@ impl DeserializeDataTransform { .ok() .map(|idx| (idx, filter.1.clone())) }) - .collect::>(); + .collect::)>>(); if bloom_filters.is_empty() { return Ok(None); } diff --git a/src/query/storages/fuse/src/operations/read/partition_deserializer.rs b/src/query/storages/fuse/src/operations/read/partition_deserializer.rs new file mode 100644 index 0000000000000..48cb58ce2fb6b --- /dev/null +++ b/src/query/storages/fuse/src/operations/read/partition_deserializer.rs @@ -0,0 +1,348 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::any::Any; +use std::collections::VecDeque; +use std::sync::Arc; + +use async_channel::Sender; +use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_arrow::arrow::bitmap::MutableBitmap; +use databend_common_base::base::Progress; +use databend_common_base::base::ProgressValues; +use databend_common_base::runtime::profile::Profile; +use databend_common_base::runtime::profile::ProfileStatisticsName; +use databend_common_catalog::plan::gen_mutation_stream_meta; +use databend_common_catalog::plan::DataSourcePlan; +use databend_common_catalog::plan::InternalColumnMeta; +use databend_common_catalog::table_context::TableContext; +use databend_common_exception::ErrorCode; +use databend_common_exception::Result; +use databend_common_expression::BlockEntry; +use databend_common_expression::BlockMetaInfoDowncast; +use databend_common_expression::BlockMetaInfoPtr; +use databend_common_expression::DataBlock; +use databend_common_expression::Scalar; +use databend_common_pipeline_core::processors::Event; +use databend_common_pipeline_core::processors::InputPort; +use databend_common_pipeline_core::processors::OutputPort; +use databend_common_pipeline_core::processors::Processor; +use databend_common_pipeline_core::processors::ProcessorPtr; +use xorf::BinaryFuse16; + +use super::runtime_filter_prunner::update_bitmap_with_bloom_filter; +use crate::fuse_part::FuseBlockPartInfo; +use crate::operations::read::PartitionScanMeta; +use crate::operations::read::PartitionScanState; +use crate::operations::read::ReaderState; +use crate::operations::read::SourceBlockReader; +use crate::operations::read::SourceReader; + +pub struct PartitionDeserializer { + input: Arc, + output: Arc, + output_data_blocks: VecDeque, + partition_scan_meta: Option, + partition_scan_state: Arc, + base_block_ids: Option, + next_partition_id: usize, + task_sender: Sender>, + scan_progress: Arc, +} + +impl PartitionDeserializer { + #[allow(clippy::too_many_arguments)] + pub fn create( + plan: &DataSourcePlan, + ctx: Arc, + input: Arc, + output: Arc, + partition_scan_state: Arc, + task_sender: Sender>, + ) -> Result { + let scan_progress = ctx.get_scan_progress(); + Ok(ProcessorPtr::create(Box::new(PartitionDeserializer { + input, + output, + output_data_blocks: VecDeque::new(), + partition_scan_meta: None, + partition_scan_state, + next_partition_id: 0, + task_sender, + base_block_ids: plan.base_block_ids.clone(), + scan_progress, + }))) + } +} + +impl PartitionDeserializer { + fn deserialize( + &self, + partition_id: usize, + meta: &mut PartitionScanMeta, + source_block_reader: &SourceBlockReader, + ) -> Result { + let partition = &meta.partitions[partition_id]; + let part = FuseBlockPartInfo::from_part(partition)?; + let io_result = meta.io_results.pop_front().unwrap(); + let columns_chunks = io_result.columns_chunks()?; + match source_block_reader { + SourceBlockReader::Parquet { + block_reader, + bloom_filter, + } => { + let data_block = block_reader.deserialize_parquet_chunks( + part.nums_rows, + &part.columns_meta, + columns_chunks, + &part.compression, + &part.location, + )?; + + if let Some(bloom_filter) = bloom_filter { + let bitmap = self.runtime_filter(&data_block, bloom_filter)?; + if bitmap.unset_bits() == bitmap.len() { + Ok(DataBlock::empty()) + } else { + if bitmap.unset_bits() != 0 { + meta.bitmaps[partition_id] = if let Some(partition_bitmap) = + meta.bitmaps[partition_id].as_ref() + { + Some((partition_bitmap) & (&bitmap)) + } else { + Some(bitmap) + }; + } + Ok(data_block) + } + } else { + Ok(data_block) + } + } + } + } + + fn reorder_columns(columns: Vec, column_positions: &[usize]) -> Vec { + let mut columns_with_position = Vec::with_capacity(columns.len()); + for (column, position) in columns.into_iter().zip(column_positions.iter()) { + columns_with_position.push((column, position)); + } + columns_with_position.sort_by(|a, b| a.1.cmp(b.1)); + + columns_with_position + .into_iter() + .map(|(column, _)| column.clone()) + .collect() + } + + pub(crate) fn construct_data_blocks( + &mut self, + mut meta: PartitionScanMeta, + ) -> Result> { + let mut data_blocks = Vec::with_capacity(meta.partitions.len()); + let partitions = std::mem::take(&mut meta.partitions); + let partition_columns = std::mem::take(&mut meta.columns); + let bitmap = std::mem::take(&mut meta.bitmaps); + for (partition, ((mut columns, num_rows), bitmap)) in partitions.into_iter().zip( + partition_columns + .into_iter() + .zip(meta.num_rows.iter()) + .zip(bitmap.into_iter()), + ) { + let part = FuseBlockPartInfo::from_part(&partition)?; + let data_block = match &meta.source_reader { + SourceReader::Parquet { + block_reader, + filter_readers, + column_positions, + .. + } => { + let mut meta: Option = if block_reader.update_stream_columns() + { + // Fill `BlockMetaInfoPtr` if update stream columns + let stream_meta = gen_mutation_stream_meta(None, &part.location)?; + Some(Box::new(stream_meta)) + } else { + None + }; + + if block_reader.query_internal_columns() { + // Fill `BlockMetaInfoPtr` if query internal columns + let block_meta = part.block_meta_index().unwrap(); + let internal_column_meta = InternalColumnMeta { + segment_idx: block_meta.segment_idx, + block_id: block_meta.block_id, + block_location: block_meta.block_location.clone(), + segment_location: block_meta.segment_location.clone(), + snapshot_location: block_meta.snapshot_location.clone(), + offsets: None, + base_block_ids: self.base_block_ids.clone(), + inner: meta, + matched_rows: block_meta.matched_rows.clone(), + }; + meta = Some(Box::new(internal_column_meta)); + } + + if !filter_readers.is_empty() { + columns = Self::reorder_columns(columns, column_positions); + } + let mut data_block = DataBlock::new_with_meta(columns, *num_rows, meta); + if let Some(bitmap) = bitmap { + data_block = data_block.filter_with_bitmap(&bitmap)?; + } + + self.record_scan_statistics(data_block.num_rows(), data_block.memory_size()); + + data_block + } + }; + + data_blocks.push(data_block); + } + Ok(data_blocks) + } + + fn runtime_filter( + &self, + data_block: &DataBlock, + bloom_filter: &Arc, + ) -> Result { + let mut bitmap = MutableBitmap::from_len_zeroed(data_block.num_rows()); + let probe_block_entry = data_block.get_by_offset(0); + let probe_column = probe_block_entry + .value + .convert_to_full_column(&probe_block_entry.data_type, data_block.num_rows()); + update_bitmap_with_bloom_filter(probe_column, bloom_filter, &mut bitmap)?; + Ok(bitmap.into()) + } + + fn prune_partitions( + &mut self, + mut meta: PartitionScanMeta, + partition_id: usize, + ) -> PartitionScanMeta { + meta.partitions.remove(partition_id); + meta.columns.remove(partition_id); + meta.num_rows.remove(partition_id); + meta.bitmaps.remove(partition_id); + meta + } + + fn record_scan_statistics(&self, num_rows: usize, bytes: usize) { + // Record scan statistics. + let progress_values = ProgressValues { + rows: num_rows, + bytes, + }; + self.scan_progress.incr(&progress_values); + Profile::record_usize_profile(ProfileStatisticsName::ScanBytes, bytes); + } +} + +#[async_trait::async_trait] +impl Processor for PartitionDeserializer { + fn name(&self) -> String { + String::from("PartitionDeserializer") + } + + fn as_any(&mut self) -> &mut dyn Any { + self + } + + fn event(&mut self) -> Result { + if self.output.is_finished() { + self.input.finish(); + return Ok(Event::Finished); + } + + if !self.output.can_push() { + self.input.set_not_need_data(); + return Ok(Event::NeedConsume); + } + + if let Some(data_block) = self.output_data_blocks.pop_front() { + self.output.push_data(Ok(data_block)); + return Ok(Event::NeedConsume); + } + + if self.partition_scan_meta.is_some() { + if !self.input.has_data() { + self.input.set_need_data(); + } + return Ok(Event::Sync); + } + + if self.input.has_data() { + let mut data_block = self.input.pull_data().unwrap()?; + if let Some(meta) = data_block.take_meta() { + if let Some(source_meta) = PartitionScanMeta::downcast_from(meta) { + self.partition_scan_meta = Some(source_meta); + return Ok(Event::Sync); + } + } + } + + if self.input.is_finished() { + self.output.finish(); + return Ok(Event::Finished); + } + + self.input.set_need_data(); + Ok(Event::NeedData) + } + + fn process(&mut self) -> Result<()> { + let Some(mut meta) = self.partition_scan_meta.take() else { + return Ok(()); + }; + + let partition_id = self.next_partition_id; + let source_block_reader = meta.source_reader.source_block_reader(&meta.reader_state); + let data_block = self.deserialize(partition_id, &mut meta, &source_block_reader)?; + + if data_block.is_empty() { + meta = self.prune_partitions(meta, partition_id); + self.partition_scan_state.inc_num_deserialized_partitions(1); + } else { + meta.num_rows[partition_id] = data_block.num_rows(); + meta.columns[partition_id].extend(data_block.into_columns()); + self.next_partition_id += 1; + } + + if self.next_partition_id >= meta.partitions.len() { + self.next_partition_id = 0; + let is_partitions_finished = matches!( + meta.reader_state.next_reader_state(&meta.source_reader), + ReaderState::Finish + ); + if !is_partitions_finished && !meta.partitions.is_empty() { + let data_block = Arc::new(DataBlock::empty_with_meta(Box::new(meta))); + self.task_sender + .send_blocking(data_block) + .map_err(|_| ErrorCode::Internal("channel is closed"))?; + } else { + let num_partitions = meta.partitions.len(); + let data_blocks = self.construct_data_blocks(meta)?; + self.output_data_blocks.extend(data_blocks); + if self.partition_scan_state.finished(num_partitions) { + self.task_sender.close(); + } + } + } else { + self.partition_scan_meta = Some(meta); + } + + Ok(()) + } +} diff --git a/src/query/storages/fuse/src/operations/read/partition_reader.rs b/src/query/storages/fuse/src/operations/read/partition_reader.rs new file mode 100644 index 0000000000000..7b33cbed895e0 --- /dev/null +++ b/src/query/storages/fuse/src/operations/read/partition_reader.rs @@ -0,0 +1,627 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::any::Any; +use std::collections::BTreeMap; +use std::sync::atomic::AtomicBool; +use std::sync::atomic::AtomicUsize; +use std::sync::atomic::Ordering; +use std::sync::Arc; +use std::sync::Mutex; + +use async_channel::Receiver; +use databend_common_arrow::arrow::datatypes::Field; +use databend_common_arrow::arrow::datatypes::Schema; +use databend_common_arrow::native::read::NativeColumnsReader; +use databend_common_catalog::plan::PartInfoPtr; +use databend_common_catalog::plan::Projection; +use databend_common_catalog::plan::StealablePartitions; +use databend_common_catalog::runtime_filter_info::HashJoinProbeStatistics; +use databend_common_catalog::table_context::TableContext; +use databend_common_exception::Result; +use databend_common_expression::types::DataType; +use databend_common_expression::BlockMetaInfoDowncast; +use databend_common_expression::ColumnId; +use databend_common_expression::DataBlock; +use databend_common_expression::DataField; +use databend_common_expression::FieldIndex; +use databend_common_expression::FunctionContext; +use databend_common_expression::TableSchema; +use databend_common_expression::TableSchemaRef; +use databend_common_pipeline_core::processors::Event; +use databend_common_pipeline_core::processors::OutputPort; +use databend_common_pipeline_core::processors::Processor; +use databend_common_pipeline_core::processors::ProcessorPtr; +use databend_common_sql::field_default_value; +use databend_common_sql::IndexType; +use databend_common_storage::ColumnNode; +use databend_common_storage::ColumnNodes; +use databend_storages_common_io::ReadSettings; +use opendal::Operator; +use xorf::BinaryFuse16; + +use crate::fuse_part::FuseBlockPartInfo; +use crate::io::read::inner_project_field_default_values; +use crate::io::BlockReader; +use crate::operations::read::runtime_filter_prunner::runtime_filter_pruner; +use crate::operations::read::PartitionScanMeta; +use crate::operations::read::SourceBlockReader; +use crate::operations::read::SourceReader; + +#[derive(Copy, Clone, Debug)] +enum Step { + Sync(SyncStep), + Async(AsyncStep), + Finish, +} + +#[derive(Copy, Clone, Debug)] +enum SyncStep { + SyncRead, +} + +#[derive(Copy, Clone, Debug)] +enum AsyncStep { + WaitTask, + AsyncRead, +} + +pub struct PartitionReader { + // Processor state. + output: Arc, + output_data: Option, + is_finished: bool, + step: Step, + + // Read source related variables. + id: usize, + ctx: Arc, + func_ctx: FunctionContext, + table_index: IndexType, + schema: Arc, + batch_size: usize, + is_blocking_io: bool, + read_settings: ReadSettings, + read_task: Option, + stealable_partitions: StealablePartitions, + + // Block reader related variables. + operator: Operator, + bloom_filter_columns: Vec<(usize, String)>, + bloom_filter_statistics: Vec<(String, Arc)>, + is_bloom_filter_statistics_fetched: bool, + column_indices: Vec, + query_internal_columns: bool, + update_stream_columns: bool, + put_cache: bool, + + // Sync with deserializer processors. + partition_scan_state: Arc, + read_tasks: Receiver>, +} + +impl PartitionReader { + #[allow(clippy::too_many_arguments)] + pub fn create( + output: Arc, + id: usize, + ctx: Arc, + table_index: IndexType, + schema: Arc, + read_settings: ReadSettings, + stealable_partitions: StealablePartitions, + bloom_filter_columns: Vec<(usize, String)>, + column_indices: Vec, + partition_scan_state: Arc, + read_tasks: Receiver>, + operator: Operator, + query_internal_columns: bool, + update_stream_columns: bool, + put_cache: bool, + ) -> Result { + let is_blocking_io = operator.info().native_capability().blocking; + let (batch_size, step) = if is_blocking_io { + (1, Step::Sync(SyncStep::SyncRead)) + } else { + ( + ctx.get_settings().get_storage_fetch_part_num()? as usize, + Step::Async(AsyncStep::AsyncRead), + ) + }; + let func_ctx = ctx.get_function_context()?; + Ok(ProcessorPtr::create(Box::new(PartitionReader { + output, + output_data: None, + is_finished: false, + step, + id, + ctx, + func_ctx, + table_index, + schema, + batch_size, + is_blocking_io, + read_settings, + read_task: None, + stealable_partitions, + bloom_filter_columns, + bloom_filter_statistics: vec![], + is_bloom_filter_statistics_fetched: false, + column_indices, + partition_scan_state, + read_tasks, + operator, + query_internal_columns, + update_stream_columns, + put_cache, + }))) + } + + fn next_step(&mut self, step: Step) -> Result { + let event = match step { + Step::Sync(_) => Event::Sync, + Step::Async(_) => Event::Async, + Step::Finish => { + self.read_tasks.close(); + self.output.finish(); + Event::Finished + } + }; + self.step = step; + Ok(event) + } + + fn output_data_block(&mut self) -> Result { + if self.output.is_finished() { + return Ok(Event::Finished); + } + + if !self.output.can_push() { + return Ok(Event::NeedConsume); + } + + if let Some(read_result) = self.output_data.take() { + self.output.push_data(Ok(read_result)); + } + + if self.is_blocking_io { + self.next_step(Step::Async(AsyncStep::WaitTask)) + } else { + self.next_step(Step::Async(AsyncStep::AsyncRead)) + } + } + + async fn read_task(&mut self) -> Result> { + if let Ok(task) = self.read_tasks.try_recv() { + return Ok(Some(task.as_ref().clone())); + } + + let partitions = self.steal_partitions()?; + if !partitions.is_empty() { + let source_reader = self.create_source_reader()?; + return Ok(Some(DataBlock::empty_with_meta(PartitionScanMeta::create( + partitions, + source_reader, + )))); + } else if self.partition_scan_state.num_readded_partitions() == 0 { + self.is_finished = true; + return Ok(None); + } + + if self.partition_scan_state.num_readded_partitions() + == self.partition_scan_state.num_deserialized_partitions() + { + self.is_finished = true; + return Ok(None); + } + match self.read_tasks.recv().await { + Ok(task) => Ok(Some(task.as_ref().clone())), + Err(_) => { + self.is_finished = true; + Ok(None) + } + } + } + + fn steal_partitions(&mut self) -> Result> { + let _guard = self.partition_scan_state.mutex.lock().unwrap(); + let mut partitions = Vec::with_capacity(self.batch_size); + while partitions.len() < self.batch_size { + let parts = if let Some(parts) = self + .stealable_partitions + .steal(self.id, self.batch_size - partitions.len()) + { + parts + } else { + let parts = self.ctx.get_partitions(self.batch_size - partitions.len()); + if !parts.is_empty() { + parts + } else { + self.partition_scan_state.set_stealable_partitions_empty(); + break; + } + }; + + for part in parts.into_iter() { + let mut filters = self + .stealable_partitions + .ctx + .get_inlist_runtime_filter_with_id(self.table_index); + filters.extend( + self.stealable_partitions + .ctx + .get_min_max_runtime_filter_with_id(self.table_index), + ); + + if !runtime_filter_pruner(self.schema.clone(), &part, &filters, &self.func_ctx)? { + partitions.push(part); + }; + } + } + self.partition_scan_state + .inc_num_readded_partitions(partitions.len()); + Ok(partitions) + } + + fn sync_read( + &self, + mut meta: PartitionScanMeta, + source_block_reader: SourceBlockReader, + ) -> Result { + let data_block = match source_block_reader { + SourceBlockReader::Parquet { block_reader, .. } => { + for partition in meta.partitions.iter() { + let merge_io_read_result = block_reader.sync_read_columns_data_by_merge_io( + &self.read_settings, + partition, + &None, + )?; + meta.io_results.push_back(merge_io_read_result); + } + DataBlock::empty_with_meta(Box::new(meta)) + } + }; + Ok(data_block) + } + + async fn async_read( + &self, + mut meta: PartitionScanMeta, + source_block_reader: SourceBlockReader, + ) -> Result { + let data_block = match source_block_reader { + SourceBlockReader::Parquet { block_reader, .. } => { + let mut block_read_results = Vec::with_capacity(meta.partitions.len()); + for partition in meta.partitions.iter() { + let reader = block_reader.clone(); + let settings = ReadSettings::from_ctx(&self.stealable_partitions.ctx)?; + let partition = partition.clone(); + block_read_results.push(async move { + databend_common_base::runtime::spawn(async move { + let part = FuseBlockPartInfo::from_part(&partition)?; + reader + .read_columns_data_by_merge_io( + &settings, + &part.location, + &part.columns_meta, + &None, + ) + .await + }) + .await + .unwrap() + }); + } + + meta.io_results + .extend(futures::future::try_join_all(block_read_results).await?); + DataBlock::empty_with_meta(Box::new(meta)) + } + }; + Ok(data_block) + } + + fn fetch_bloom_filters(&self) -> Vec>> { + let mut bloom_filters = self.ctx.get_bloom_runtime_filter_with_id(self.table_index); + bloom_filters.sort_by(|a, b| { + let a_pos = self + .bloom_filter_statistics + .iter() + .position(|(name, _)| name == &a.0) + .unwrap_or(usize::MAX); + let b_pos = self + .bloom_filter_statistics + .iter() + .position(|(name, _)| name == &b.0) + .unwrap_or(usize::MAX); + a_pos.cmp(&b_pos) + }); + + self.bloom_filter_statistics + .iter() + .map(|(name, _)| { + bloom_filters + .iter() + .find(|(filter_name, _)| filter_name == name) + .map(|(_, filter)| filter.clone()) + }) + .collect() + } + + fn create_source_reader(&self) -> Result { + let bloom_filters = self.fetch_bloom_filters(); + let mut column_indices = self.column_indices.clone(); + let mut column_positions = Vec::with_capacity(column_indices.len()); + let mut filter_readers = Vec::new(); + let block_reader = self.create_block_reader(Projection::Columns(column_indices.clone()))?; + + for ((column_name, hash_join_probe_statistics), bloom_filter) in self + .bloom_filter_statistics + .iter() + .zip(bloom_filters.into_iter()) + { + if !hash_join_probe_statistics.prefer_runtime_filter() { + continue; + } + + let Some(bloom_filter) = bloom_filter else { + continue; + }; + + let position = self.schema.index_of(column_name)?; + let block_reader = self.create_block_reader(Projection::Columns(vec![position]))?; + column_positions.push(position); + filter_readers.push((block_reader, bloom_filter)); + column_indices = column_indices + .into_iter() + .filter(|&x| x != position) + .collect::>(); + } + + let remaining_reader = if !filter_readers.is_empty() && !column_indices.is_empty() { + for column_index in column_indices.iter() { + column_positions.push(*column_index); + } + let remain_column_projection = Projection::Columns(column_indices); + Some(self.create_block_reader(remain_column_projection)?) + } else { + None + }; + + Ok(SourceReader::Parquet { + block_reader, + filter_readers, + remaining_reader, + column_positions, + }) + } + + pub fn create_block_reader(&self, projection: Projection) -> Result> { + // init projected_schema and default_vals of schema.fields + let (projected_schema, default_vals) = match projection { + Projection::Columns(ref indices) => { + let projected_schema = TableSchemaRef::new(self.schema.project(indices)); + // If projection by Columns, just calc default values by projected fields. + let mut default_vals = Vec::with_capacity(projected_schema.fields().len()); + for field in projected_schema.fields() { + let default_val = field_default_value(self.ctx.clone(), field)?; + default_vals.push(default_val); + } + + (projected_schema, default_vals) + } + Projection::InnerColumns(ref path_indices) => { + let projected_schema = TableSchemaRef::new(self.schema.inner_project(path_indices)); + let mut field_default_vals = Vec::with_capacity(self.schema.fields().len()); + + // If projection by InnerColumns, first calc default value of all schema fields. + for field in self.schema.fields() { + field_default_vals.push(field_default_value(self.ctx.clone(), field)?); + } + + // Then calc project scalars by path_indices + let mut default_vals = Vec::with_capacity(self.schema.fields().len()); + path_indices.values().for_each(|path| { + default_vals.push( + inner_project_field_default_values(&field_default_vals, path).unwrap(), + ); + }); + + (projected_schema, default_vals) + } + }; + + let arrow_schema: Schema = self.schema.as_ref().into(); + let native_columns_reader = NativeColumnsReader::new(arrow_schema.clone())?; + let column_nodes = ColumnNodes::new_from_schema(&arrow_schema, Some(&self.schema)); + + let project_column_nodes: Vec = projection + .project_column_nodes(&column_nodes)? + .iter() + .map(|c| (*c).clone()) + .collect(); + let project_indices = Self::build_projection_indices(&project_column_nodes); + + Ok(Arc::new(BlockReader { + ctx: self.ctx.clone(), + operator: self.operator.clone(), + projection, + projected_schema, + project_indices, + project_column_nodes, + default_vals, + query_internal_columns: self.query_internal_columns, + update_stream_columns: self.update_stream_columns, + put_cache: self.put_cache, + original_schema: self.schema.clone(), + native_columns_reader, + })) + } + + // Build non duplicate leaf_indices to avoid repeated read column from parquet + pub(crate) fn build_projection_indices( + columns: &[ColumnNode], + ) -> BTreeMap { + let mut indices = BTreeMap::new(); + for column in columns { + for (i, index) in column.leaf_indices.iter().enumerate() { + let f = DataField::try_from(&column.field).unwrap(); + indices.insert( + *index, + ( + column.leaf_column_ids[i], + column.field.clone(), + f.data_type().clone(), + ), + ); + } + } + indices + } + + fn fetche_bloom_filter_statistics(&mut self) { + for (hash_join_id, column_name) in self.bloom_filter_columns.iter() { + let statistics = self.ctx.get_hash_join_probe_statistics(*hash_join_id); + self.bloom_filter_statistics + .push((column_name.clone(), statistics)) + } + self.is_bloom_filter_statistics_fetched = true; + } +} + +#[async_trait::async_trait] +impl Processor for PartitionReader { + fn name(&self) -> String { + String::from("PartitionReader") + } + + fn as_any(&mut self) -> &mut dyn Any { + self + } + + fn event(&mut self) -> Result { + if self.is_finished { + return self.next_step(Step::Finish); + } + if !self.is_bloom_filter_statistics_fetched { + self.fetche_bloom_filter_statistics(); + } + match self.step { + Step::Sync(step) => match step { + SyncStep::SyncRead => self.output_data_block(), + }, + Step::Async(step) => match step { + AsyncStep::AsyncRead => self.output_data_block(), + AsyncStep::WaitTask => self.next_step(Step::Sync(SyncStep::SyncRead)), + }, + Step::Finish => self.next_step(Step::Finish), + } + } + + fn process(&mut self) -> Result<()> { + let Some(mut read_task) = self.read_task.take() else { + return Ok(()); + }; + + if let Some(meta) = read_task.take_meta() { + if let Some(mut meta) = PartitionScanMeta::downcast_from(meta) { + meta.reader_state = meta.reader_state.next_reader_state(&meta.source_reader); + let source_block_reader = + meta.source_reader.source_block_reader(&meta.reader_state); + self.output_data = Some(self.sync_read(meta, source_block_reader)?); + } + } + + Ok(()) + } + + #[async_backtrace::framed] + async fn async_process(&mut self) -> Result<()> { + match self.step { + Step::Async(AsyncStep::WaitTask) => { + self.read_task = self.read_task().await?; + } + Step::Async(AsyncStep::AsyncRead) => { + let Some(mut read_task) = self.read_task().await?.take() else { + return Ok(()); + }; + + if let Some(meta) = read_task.take_meta() { + if let Some(mut meta) = PartitionScanMeta::downcast_from(meta) { + meta.reader_state = + meta.reader_state.next_reader_state(&meta.source_reader); + let source_block_reader = + meta.source_reader.source_block_reader(&meta.reader_state); + self.output_data = Some(self.async_read(meta, source_block_reader).await?); + } + } + } + _ => unreachable!(), + } + Ok(()) + } +} + +#[derive(Debug)] +pub struct PartitionScanState { + mutex: Mutex<()>, + is_stealable_partitions_empty: AtomicBool, + num_readded_partitions: AtomicUsize, + num_deserialized_partitions: AtomicUsize, +} + +impl PartitionScanState { + pub fn new() -> Self { + Self { + mutex: Mutex::new(()), + is_stealable_partitions_empty: AtomicBool::new(false), + num_readded_partitions: AtomicUsize::new(0), + num_deserialized_partitions: AtomicUsize::new(0), + } + } + + pub fn is_stealable_partitions_empty(&self) -> bool { + self.is_stealable_partitions_empty.load(Ordering::Acquire) + } + + pub fn num_readded_partitions(&self) -> usize { + self.num_readded_partitions.load(Ordering::Acquire) + } + + pub fn num_deserialized_partitions(&self) -> usize { + self.num_deserialized_partitions.load(Ordering::Acquire) + } + + pub fn set_stealable_partitions_empty(&self) { + self.is_stealable_partitions_empty + .store(true, Ordering::Release); + } + + pub fn inc_num_readded_partitions(&self, num: usize) { + self.num_readded_partitions.fetch_add(num, Ordering::AcqRel); + } + + pub fn inc_num_deserialized_partitions(&self, num: usize) { + self.num_deserialized_partitions + .fetch_add(num, Ordering::AcqRel); + } + + pub fn finished(&self, num: usize) -> bool { + let num_deserialized_partitions = self + .num_deserialized_partitions + .fetch_add(num, Ordering::AcqRel); + self.is_stealable_partitions_empty() + && self.num_readded_partitions() == num_deserialized_partitions + num + } +} diff --git a/src/query/storages/fuse/src/operations/read/partition_scan_meta.rs b/src/query/storages/fuse/src/operations/read/partition_scan_meta.rs new file mode 100644 index 0000000000000..ea51c6a7f1795 --- /dev/null +++ b/src/query/storages/fuse/src/operations/read/partition_scan_meta.rs @@ -0,0 +1,93 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::VecDeque; +use std::fmt::Debug; +use std::fmt::Formatter; + +use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_catalog::plan::PartInfoPtr; +use databend_common_expression::BlockEntry; +use databend_common_expression::BlockMetaInfo; +use databend_common_expression::BlockMetaInfoPtr; + +use crate::operations::read::ReaderState; +use crate::operations::read::SourceReader; +use crate::BlockReadResult; + +#[derive(Clone)] +pub struct PartitionScanMeta { + pub partitions: Vec, + pub source_reader: SourceReader, + pub reader_state: ReaderState, + pub num_rows: Vec, + pub bitmaps: Vec>, + pub columns: Vec>, + pub io_results: VecDeque, +} + +impl PartitionScanMeta { + pub fn create(partitions: Vec, source_reader: SourceReader) -> BlockMetaInfoPtr { + let num_partitions = partitions.len(); + Box::new(PartitionScanMeta { + partitions, + source_reader, + reader_state: ReaderState::Uninitialized, + num_rows: vec![0; num_partitions], + bitmaps: vec![None; num_partitions], + columns: vec![vec![]; num_partitions], + io_results: VecDeque::with_capacity(num_partitions), + }) + } +} + +impl Debug for PartitionScanMeta { + fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { + f.debug_struct("PartitionScanMeta") + .field("partitions", &self.partitions) + .finish() + } +} + +impl serde::Serialize for PartitionScanMeta { + fn serialize(&self, _: S) -> Result + where S: serde::Serializer { + unreachable!("Unimplemented serialize PartitionScanMeta") + } +} + +impl<'de> serde::Deserialize<'de> for PartitionScanMeta { + fn deserialize(_: D) -> Result + where D: serde::Deserializer<'de> { + unreachable!("Unimplemented deserialize PartitionScanMeta") + } +} + +impl BlockMetaInfo for PartitionScanMeta { + fn typetag_deserialize(&self) { + unimplemented!("PartitionScanMeta does not support exchanging between multiple nodes") + } + + fn typetag_name(&self) -> &'static str { + unimplemented!("PartitionScanMeta does not support exchanging between multiple nodes") + } + + fn equals(&self, _: &Box) -> bool { + unimplemented!("Unimplemented equals PartitionScanMeta") + } + + fn clone_self(&self) -> Box { + Box::new(self.clone()) + } +} diff --git a/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs b/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs index 066a418b626df..b3a7432fda0c7 100644 --- a/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs +++ b/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs @@ -15,12 +15,16 @@ use std::collections::HashMap; use std::sync::Arc; +use databend_common_arrow::arrow::bitmap::Bitmap; use databend_common_arrow::arrow::bitmap::MutableBitmap; +use databend_common_arrow::arrow::buffer::Buffer; use databend_common_base::runtime::profile::Profile; use databend_common_base::runtime::profile::ProfileStatisticsName; use databend_common_catalog::plan::PartInfoPtr; use databend_common_exception::Result; +use databend_common_expression::types::BinaryColumn; use databend_common_expression::types::NumberColumn; +use databend_common_expression::types::StringColumn; use databend_common_expression::Column; use databend_common_expression::ConstantFolder; use databend_common_expression::DataBlock; @@ -34,6 +38,7 @@ use databend_common_expression::KeysState::U256; use databend_common_expression::Scalar; use databend_common_expression::TableSchema; use databend_common_functions::BUILTIN_FUNCTIONS; +use databend_common_hashtable::DictionaryKeys; use databend_common_hashtable::FastHash; use databend_storages_common_index::statistics_to_domain; use log::debug; @@ -46,31 +51,30 @@ use crate::FuseBlockPartInfo; pub fn runtime_filter_pruner( table_schema: Arc, part: &PartInfoPtr, - filters: &[Expr], + filters: &[(String, Expr)], func_ctx: &FunctionContext, ) -> Result { if filters.is_empty() { return Ok(false); } let part = FuseBlockPartInfo::from_part(part)?; - let pruned = filters.iter().any(|filter| { + let pruned = filters.iter().any(|(_, filter)| { let column_refs = filter.column_refs(); // Currently only support filter with one column(probe key). debug_assert!(column_refs.len() == 1); - let ty = column_refs.values().last().unwrap(); - let name = column_refs.keys().last().unwrap(); + let data_type = column_refs.values().last().unwrap(); + let column_name = column_refs.keys().last().unwrap(); if let Some(stats) = &part.columns_stat { - let column_ids = table_schema.leaf_columns_of(name); + let column_ids = table_schema.leaf_columns_of(column_name); if column_ids.len() != 1 { return false; } - debug_assert!(column_ids.len() == 1); if let Some(stat) = stats.get(&column_ids[0]) { let stats = vec![stat]; - let domain = statistics_to_domain(stats, ty); + let domain = statistics_to_domain(stats, data_type); let mut input_domains = HashMap::new(); - input_domains.insert(name.to_string(), domain.clone()); + input_domains.insert(column_name.clone(), domain.clone()); let (new_expr, _) = ConstantFolder::fold_with_domain( filter, @@ -102,150 +106,209 @@ pub fn runtime_filter_pruner( pub(crate) fn update_bitmap_with_bloom_filter( column: Column, - filter: &BinaryFuse16, + bloom_filter: &BinaryFuse16, bitmap: &mut MutableBitmap, ) -> Result<()> { - let data_type = column.data_type(); + let data_type = column.data_type().remove_nullable(); let num_rows = column.len(); let method = DataBlock::choose_hash_method_with_types(&[data_type.clone()], false)?; + + let (column, validity) = if let Column::Nullable(inner_column) = column { + if inner_column.validity.unset_bits() == 0 { + (inner_column.column, None) + } else { + (inner_column.column, Some(inner_column.validity)) + } + } else { + (column, None) + }; + let columns = &[column]; let group_columns = columns.into(); - let mut idx = 0; match method { - HashMethodKind::Serializer(method) => { - let key_state = method.build_keys_state(group_columns, num_rows)?; + HashMethodKind::KeysU8(hash_method) => { + let key_state = hash_method.build_keys_state(group_columns, num_rows)?; match key_state { - KeysState::Column(Column::Binary(col)) => col.iter().for_each(|key| { - let hash = key.fast_hash(); - if filter.contains(&hash) { - bitmap.set(idx, true); - } - idx += 1; - }), + KeysState::Column(Column::Number(NumberColumn::UInt8(keys))) => { + update_bitmap_primitive_types(keys, bitmap, bloom_filter, &validity) + } _ => unreachable!(), } } - HashMethodKind::DictionarySerializer(method) => { - let key_state = method.build_keys_state(group_columns, num_rows)?; + HashMethodKind::KeysU16(hash_method) => { + let key_state = hash_method.build_keys_state(group_columns, num_rows)?; match key_state { - KeysState::Dictionary { dictionaries, .. } => dictionaries.iter().for_each(|key| { - let hash = key.fast_hash(); - if filter.contains(&hash) { - bitmap.set(idx, true); - } - idx += 1; - }), + KeysState::Column(Column::Number(NumberColumn::UInt16(keys))) => { + update_bitmap_primitive_types(keys, bitmap, bloom_filter, &validity) + } _ => unreachable!(), } } - HashMethodKind::SingleBinary(method) => { - let key_state = method.build_keys_state(group_columns, num_rows)?; + HashMethodKind::KeysU32(hash_method) => { + let key_state = hash_method.build_keys_state(group_columns, num_rows)?; match key_state { - KeysState::Column(Column::Binary(col)) - | KeysState::Column(Column::Variant(col)) - | KeysState::Column(Column::Bitmap(col)) => col.iter().for_each(|key| { - let hash = key.fast_hash(); - if filter.contains(&hash) { - bitmap.set(idx, true); - } - idx += 1; - }), - KeysState::Column(Column::String(col)) => col.iter().for_each(|key| { - let hash = key.as_bytes().fast_hash(); - if filter.contains(&hash) { - bitmap.set(idx, true); - } - idx += 1; - }), + KeysState::Column(Column::Number(NumberColumn::UInt32(keys))) => { + update_bitmap_primitive_types(keys, bitmap, bloom_filter, &validity) + } _ => unreachable!(), } } - HashMethodKind::KeysU8(hash_method) => { + HashMethodKind::KeysU64(hash_method) => { let key_state = hash_method.build_keys_state(group_columns, num_rows)?; match key_state { - KeysState::Column(Column::Number(NumberColumn::UInt8(c))) => { - c.iter().for_each(|key| { - let hash = key.fast_hash(); - if filter.contains(&hash) { - bitmap.set(idx, true); - } - idx += 1; - }) + KeysState::Column(Column::Number(NumberColumn::UInt64(keys))) => { + update_bitmap_primitive_types(keys, bitmap, bloom_filter, &validity) } _ => unreachable!(), } } - HashMethodKind::KeysU16(hash_method) => { + HashMethodKind::KeysU128(hash_method) => { let key_state = hash_method.build_keys_state(group_columns, num_rows)?; match key_state { - KeysState::Column(Column::Number(NumberColumn::UInt16(c))) => { - c.iter().for_each(|key| { - let hash = key.fast_hash(); - if filter.contains(&hash) { - bitmap.set(idx, true); - } - idx += 1; - }) - } + U128(keys) => update_bitmap_primitive_types(keys, bitmap, bloom_filter, &validity), _ => unreachable!(), } } - HashMethodKind::KeysU32(hash_method) => { + HashMethodKind::KeysU256(hash_method) => { let key_state = hash_method.build_keys_state(group_columns, num_rows)?; match key_state { - KeysState::Column(Column::Number(NumberColumn::UInt32(c))) => { - c.iter().for_each(|key| { - let hash = key.fast_hash(); - if filter.contains(&hash) { - bitmap.set(idx, true); - } - idx += 1; - }) - } + U256(keys) => update_bitmap_primitive_types(keys, bitmap, bloom_filter, &validity), _ => unreachable!(), } } - HashMethodKind::KeysU64(hash_method) => { - let key_state = hash_method.build_keys_state(group_columns, num_rows)?; + HashMethodKind::SingleBinary(method) => { + let key_state = method.build_keys_state(group_columns, num_rows)?; match key_state { - KeysState::Column(Column::Number(NumberColumn::UInt64(c))) => { - c.iter().for_each(|key| { - let hash = key.fast_hash(); - if filter.contains(&hash) { - bitmap.set(idx, true); - } - idx += 1; - }) + KeysState::Column(Column::Binary(keys)) + | KeysState::Column(Column::Variant(keys)) + | KeysState::Column(Column::Bitmap(keys)) => { + update_bitmap_binary(keys, bitmap, bloom_filter, &validity) + } + KeysState::Column(Column::String(keys)) => { + update_bitmap_string(keys, bitmap, bloom_filter, &validity) } _ => unreachable!(), } } - HashMethodKind::KeysU128(hash_method) => { - let key_state = hash_method.build_keys_state(group_columns, num_rows)?; + HashMethodKind::Serializer(method) => { + let key_state = method.build_keys_state(group_columns, num_rows)?; match key_state { - U128(c) => c.iter().for_each(|key| { - let hash = key.fast_hash(); - if filter.contains(&hash) { - bitmap.set(idx, true); - } - idx += 1; - }), + KeysState::Column(Column::Binary(keys)) => { + update_bitmap_binary(keys, bitmap, bloom_filter, &validity) + } _ => unreachable!(), } } - HashMethodKind::KeysU256(hash_method) => { - let key_state = hash_method.build_keys_state(group_columns, num_rows)?; + HashMethodKind::DictionarySerializer(method) => { + let key_state = method.build_keys_state(group_columns, num_rows)?; match key_state { - U256(c) => c.iter().for_each(|key| { - let hash = key.fast_hash(); - if filter.contains(&hash) { - bitmap.set(idx, true); - } - idx += 1; - }), + KeysState::Dictionary { dictionaries, .. } => { + update_bitmap_dictionary(dictionaries, bitmap, bloom_filter, &validity) + } _ => unreachable!(), } } } + Ok(()) } + +fn update_bitmap_primitive_types( + keys: Buffer, + bitmap: &mut MutableBitmap, + bloom_filter: &BinaryFuse16, + validity: &Option, +) { + if let Some(validity) = validity { + for (index, key) in keys.iter().enumerate() { + if validity.get_bit(index) { + let hash = key.fast_hash(); + if bloom_filter.contains(&hash) { + bitmap.set(index, true); + } + } + } + } else { + for (index, key) in keys.iter().enumerate() { + let hash = key.fast_hash(); + if bloom_filter.contains(&hash) { + bitmap.set(index, true); + } + } + } +} + +fn update_bitmap_string( + keys: StringColumn, + bitmap: &mut MutableBitmap, + bloom_filter: &BinaryFuse16, + validity: &Option, +) { + if let Some(validity) = validity { + for (index, key) in keys.iter().enumerate() { + if validity.get_bit(index) { + let hash = key.fast_hash(); + if bloom_filter.contains(&hash) { + bitmap.set(index, true); + } + } + } + } else { + for (index, key) in keys.iter().enumerate() { + let hash = key.fast_hash(); + if bloom_filter.contains(&hash) { + bitmap.set(index, true); + } + } + } +} + +fn update_bitmap_binary( + keys: BinaryColumn, + bitmap: &mut MutableBitmap, + bloom_filter: &BinaryFuse16, + validity: &Option, +) { + if let Some(validity) = validity { + for (index, key) in keys.iter().enumerate() { + if validity.get_bit(index) { + let hash = key.fast_hash(); + if bloom_filter.contains(&hash) { + bitmap.set(index, true); + } + } + } + } else { + for (index, key) in keys.iter().enumerate() { + let hash = key.fast_hash(); + if bloom_filter.contains(&hash) { + bitmap.set(index, true); + } + } + } +} + +fn update_bitmap_dictionary( + keys: Vec, + bitmap: &mut MutableBitmap, + bloom_filter: &BinaryFuse16, + validity: &Option, +) { + if let Some(validity) = validity { + for (index, key) in keys.iter().enumerate() { + if validity.get_bit(index) { + let hash = key.fast_hash(); + if bloom_filter.contains(&hash) { + bitmap.set(index, true); + } + } + } + } else { + for (index, key) in keys.iter().enumerate() { + let hash = key.fast_hash(); + if bloom_filter.contains(&hash) { + bitmap.set(index, true); + } + } + } +} diff --git a/src/query/storages/fuse/src/operations/read/source_reader.rs b/src/query/storages/fuse/src/operations/read/source_reader.rs new file mode 100644 index 0000000000000..bb38ec1a637b7 --- /dev/null +++ b/src/query/storages/fuse/src/operations/read/source_reader.rs @@ -0,0 +1,104 @@ +// 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 xorf::BinaryFuse16; + +use crate::io::BlockReader; + +#[derive(Copy, Clone, Debug, Eq, PartialEq)] +pub enum ReaderState { + Uninitialized, + All, + Filter(usize), + Remaining, + Finish, +} + +impl ReaderState { + pub fn next_reader_state(&self, source_reader: &SourceReader) -> ReaderState { + match self { + ReaderState::Uninitialized => match source_reader { + SourceReader::Parquet { filter_readers, .. } => { + if filter_readers.is_empty() { + ReaderState::All + } else { + ReaderState::Filter(0) + } + } + }, + ReaderState::Filter(index) => match source_reader { + SourceReader::Parquet { + filter_readers, + remaining_reader, + .. + } => { + if index + 1 < filter_readers.len() { + ReaderState::Filter(*index + 1) + } else if remaining_reader.is_some() { + ReaderState::Remaining + } else { + ReaderState::Finish + } + } + }, + ReaderState::All | ReaderState::Remaining | ReaderState::Finish => ReaderState::Finish, + } + } +} + +#[derive(Clone)] +pub enum SourceReader { + Parquet { + block_reader: Arc, + filter_readers: Vec<(Arc, Arc)>, + remaining_reader: Option>, + column_positions: Vec, + }, +} + +impl SourceReader { + pub fn source_block_reader(&self, reader_state: &ReaderState) -> SourceBlockReader { + match self { + SourceReader::Parquet { + block_reader, + filter_readers, + remaining_reader, + .. + } => { + let (source_block_reader, bloom_filter) = match reader_state { + ReaderState::All => (block_reader.clone(), None), + ReaderState::Filter(index) => ( + filter_readers[*index].0.clone(), + Some(filter_readers[*index].1.clone()), + ), + ReaderState::Remaining => (remaining_reader.as_ref().unwrap().clone(), None), + _ => unreachable!(), + }; + SourceBlockReader::Parquet { + block_reader: source_block_reader, + bloom_filter, + } + } + } + } +} + +pub enum SourceBlockReader { + Parquet { + block_reader: Arc, + bloom_filter: Option>, + }, +} diff --git a/src/query/storages/fuse/src/operations/read_data.rs b/src/query/storages/fuse/src/operations/read_data.rs index b95061fe32879..b23ea05cdcfbe 100644 --- a/src/query/storages/fuse/src/operations/read_data.rs +++ b/src/query/storages/fuse/src/operations/read_data.rs @@ -15,6 +15,7 @@ use std::sync::Arc; use async_channel::Receiver; +use databend_common_base::runtime::Runtime; use databend_common_base::runtime::TrySpawn; use databend_common_catalog::plan::DataSourcePlan; use databend_common_catalog::plan::PartInfoPtr; @@ -35,6 +36,7 @@ use crate::io::AggIndexReader; use crate::io::BlockReader; use crate::io::VirtualColumnReader; use crate::operations::read::build_fuse_parquet_source_pipeline; +use crate::operations::read::build_partition_source_pipeline; use crate::operations::read::fuse_source::build_fuse_native_source_pipeline; use crate::pruning::SegmentLocation; use crate::FuseLazyPartInfo; @@ -203,56 +205,138 @@ impl FuseTable { (None, None) }; - self.build_fuse_source_pipeline( - ctx.clone(), - pipeline, - self.storage_format, - block_reader, - plan, - topk, - max_io_requests, - index_reader, - virtual_reader, - rx, - )?; + let schema = self.schema_with_stream(); + let projection_column_indices = if let Some(PushDownInfo { + projection: Some(projection), + .. + }) = &plan.push_downs + { + if let Projection::Columns(indices) = projection { + Some(indices.clone()) + } else { + None + } + } else { + let indices = (0..schema.fields().len()).collect::>(); + Some(indices) + }; + + let runtime_filter_columns = ctx.get_runtime_filter_columns(plan.table_index); + let mut partition_scan = !runtime_filter_columns.is_empty(); + for (_, column_name) in runtime_filter_columns.iter() { + if schema.index_of(column_name).is_err() { + partition_scan = false; + } + } + + if partition_scan + && let Some(column_indices) = projection_column_indices + && index_reader.is_none() + && virtual_reader.is_none() + && matches!(self.storage_format, FuseStorageFormat::Parquet) + { + if !lazy_init_segments.is_empty() { + let table = self.clone(); + let table_schema = self.schema_with_stream(); + let push_downs = plan.push_downs.clone(); + let query_ctx = ctx.clone(); + + let lazy_init_segments = lazy_init_segments.clone(); + pipeline.set_on_init(move || { + let table = table.clone(); + let table_schema = table_schema.clone(); + let ctx = query_ctx.clone(); + let push_downs = push_downs.clone(); + + let partitions = + Runtime::with_worker_threads(2, None)?.block_on(async move { + let (_statistics, partitions) = table + .prune_snapshot_blocks( + ctx, + push_downs, + table_schema, + lazy_init_segments, + 0, + ) + .await?; + + Result::<_>::Ok(partitions) + })?; + + query_ctx.set_partitions(partitions)?; + Ok(()) + }); + } + + let max_threads = ctx.get_settings().get_max_threads()? as usize; + let table_schema = self.schema_with_stream(); + build_partition_source_pipeline( + pipeline, + plan, + ctx.clone(), + table_schema, + max_threads, + max_io_requests, + runtime_filter_columns, + column_indices, + self.operator.clone(), + put_cache, + )?; + } else { + partition_scan = false; + self.build_fuse_source_pipeline( + ctx.clone(), + pipeline, + self.storage_format, + block_reader, + plan, + topk, + max_io_requests, + index_reader, + virtual_reader, + rx, + )?; + } // replace the column which has data mask if needed self.apply_data_mask_policy_if_needed(ctx.clone(), plan, pipeline)?; - if let Some(sender) = tx { - let table = self.clone(); - let table_schema = self.schema_with_stream(); - let push_downs = plan.push_downs.clone(); - pipeline.set_on_init(move || { - ctx.get_runtime()?.try_spawn( - async move { - match table - .prune_snapshot_blocks( - ctx, - push_downs, - table_schema, - lazy_init_segments, - 0, - ) - .await - { - Ok((_, partitions)) => { - for part in partitions.partitions { - // ignore the error, the sql may be killed or early stop - let _ = sender.send(Ok(part)).await; + if !partition_scan { + if let Some(sender) = tx { + let table = self.clone(); + let table_schema = self.schema_with_stream(); + let push_downs = plan.push_downs.clone(); + pipeline.set_on_init(move || { + ctx.get_runtime()?.try_spawn( + async move { + match table + .prune_snapshot_blocks( + ctx, + push_downs, + table_schema, + lazy_init_segments, + 0, + ) + .await + { + Ok((_, partitions)) => { + for part in partitions.partitions { + // ignore the error, the sql may be killed or early stop + let _ = sender.send(Ok(part)).await; + } + } + Err(err) => { + let _ = sender.send(Err(err)).await; } } - Err(err) => { - let _ = sender.send(Err(err)).await; - } - } - Ok::<_, ErrorCode>(()) - }, - None, - )?; + Ok::<_, ErrorCode>(()) + }, + None, + )?; - Ok(()) - }); + Ok(()) + }); + } } Ok(())