From 3f092204faf26c9687fbccbf26eba0184a7ad48d Mon Sep 17 00:00:00 2001 From: coldWater Date: Thu, 13 Nov 2025 12:34:37 +0800 Subject: [PATCH 01/19] loop join --- src/common/exception/src/exception_into.rs | 7 + src/query/expression/src/block.rs | 12 + src/query/expression/src/schema.rs | 4 + .../format/format_nested_loop_join.rs | 125 +++++++++ .../service/src/physical_plans/format/mod.rs | 2 + src/query/service/src/physical_plans/mod.rs | 2 + .../src/physical_plans/physical_hash_join.rs | 14 +- .../src/physical_plans/physical_join.rs | 88 ++++-- .../physical_nested_loop_join.rs | 211 ++++++++++++++ .../src/physical_plans/physical_range_join.rs | 14 +- .../pipelines/processors/transforms/mod.rs | 2 + .../transforms/nested_loop_join/mod.rs | 16 ++ .../nested_loop_join/transform_loop_join.rs | 257 ++++++++++++++++++ .../transforms/range_join/range_join_state.rs | 45 +-- .../range_join/transform_range_join.rs | 2 +- src/query/settings/src/settings_default.rs | 7 + .../settings/src/settings_getter_setter.rs | 4 + .../join/inner/equality_join_limits.test | 10 +- 18 files changed, 754 insertions(+), 68 deletions(-) create mode 100644 src/query/service/src/physical_plans/format/format_nested_loop_join.rs create mode 100644 src/query/service/src/physical_plans/physical_nested_loop_join.rs create mode 100644 src/query/service/src/pipelines/processors/transforms/nested_loop_join/mod.rs create mode 100644 src/query/service/src/pipelines/processors/transforms/nested_loop_join/transform_loop_join.rs diff --git a/src/common/exception/src/exception_into.rs b/src/common/exception/src/exception_into.rs index df84a23c7ee79..574191b4c73b1 100644 --- a/src/common/exception/src/exception_into.rs +++ b/src/common/exception/src/exception_into.rs @@ -16,6 +16,7 @@ use std::error::Error; use std::fmt::Debug; use std::fmt::Display; use std::fmt::Formatter; +use std::sync::PoisonError; use geozero::error::GeozeroError; @@ -436,3 +437,9 @@ impl From for ErrorCode { ErrorCode::DictionarySourceError(format!("Dictionary Redis Error, cause: {}", error)) } } + +impl From> for ErrorCode { + fn from(error: PoisonError) -> Self { + ErrorCode::Internal(format!("{error}")) + } +} diff --git a/src/query/expression/src/block.rs b/src/query/expression/src/block.rs index cbeede2fec333..c8322beebd795 100644 --- a/src/query/expression/src/block.rs +++ b/src/query/expression/src/block.rs @@ -191,6 +191,18 @@ impl BlockEntry { BlockEntry::Column(column) => Ok(ColumnView::Column(T::try_downcast_column(column)?)), } } + + pub fn into_nullable(self) -> BlockEntry { + match self { + BlockEntry::Const(scalar, data_type, n) if !data_type.is_nullable_or_null() => { + BlockEntry::Const(scalar, DataType::Nullable(Box::new(data_type)), n) + } + entry @ BlockEntry::Const(_, _, _) + | entry @ BlockEntry::Column(Column::Nullable(_)) + | entry @ BlockEntry::Column(Column::Null { .. }) => entry, + BlockEntry::Column(column) => column.wrap_nullable(None).into(), + } + } } impl From for BlockEntry { diff --git a/src/query/expression/src/schema.rs b/src/query/expression/src/schema.rs index 63a3bd00008b6..38d0500d5b35d 100644 --- a/src/query/expression/src/schema.rs +++ b/src/query/expression/src/schema.rs @@ -362,6 +362,10 @@ impl DataSchema { } } + pub fn new_ref(fields: Vec) -> Arc { + Self::new(fields).into() + } + pub fn new_from(fields: Vec, metadata: BTreeMap) -> Self { Self { fields, metadata } } diff --git a/src/query/service/src/physical_plans/format/format_nested_loop_join.rs b/src/query/service/src/physical_plans/format/format_nested_loop_join.rs new file mode 100644 index 0000000000000..8259bc99a6b0d --- /dev/null +++ b/src/query/service/src/physical_plans/format/format_nested_loop_join.rs @@ -0,0 +1,125 @@ +// 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_ast::ast::FormatTreeNode; +use databend_common_exception::Result; +use databend_common_functions::BUILTIN_FUNCTIONS; + +use crate::physical_plans::format::append_output_rows_info; +use crate::physical_plans::format::format_output_columns; +use crate::physical_plans::format::plan_stats_info_to_format_tree; +use crate::physical_plans::format::FormatContext; +use crate::physical_plans::format::PhysicalFormat; +use crate::physical_plans::IPhysicalPlan; +use crate::physical_plans::NestedLoopJoin; +use crate::physical_plans::PhysicalPlanMeta; + +pub struct NestedLoopJoinFormatter<'a> { + inner: &'a NestedLoopJoin, +} + +impl<'a> NestedLoopJoinFormatter<'a> { + pub fn create(inner: &'a NestedLoopJoin) -> Box { + Box::new(NestedLoopJoinFormatter { inner }) + } +} + +impl<'a> PhysicalFormat for NestedLoopJoinFormatter<'a> { + fn get_meta(&self) -> &PhysicalPlanMeta { + self.inner.get_meta() + } + + #[recursive::recursive] + fn format(&self, ctx: &mut FormatContext<'_>) -> Result> { + let conditions = self + .inner + .conditions + .iter() + .map(|expr| expr.as_expr(&BUILTIN_FUNCTIONS).sql_display()) + .collect::>() + .join(", "); + + let mut node_children = vec![ + FormatTreeNode::new(format!( + "output columns: [{}]", + format_output_columns(self.inner.output_schema()?, ctx.metadata, true) + )), + FormatTreeNode::new(format!("join type: {}", self.inner.join_type)), + FormatTreeNode::new(format!("conditions: [{conditions}]")), + ]; + + if let Some(info) = &self.inner.stat_info { + let items = plan_stats_info_to_format_tree(info); + node_children.extend(items); + } + + let left_formatter = self.inner.left.formatter()?; + let mut left_child = left_formatter.dispatch(ctx)?; + left_child.payload = format!("{}(Left)", left_child.payload); + + let right_formatter = self.inner.right.formatter()?; + let mut right_child = right_formatter.dispatch(ctx)?; + right_child.payload = format!("{}(Right)", right_child.payload); + + node_children.push(left_child); + node_children.push(right_child); + + Ok(FormatTreeNode::with_children( + "NestedLoopJoin".to_string(), + node_children, + )) + } + + #[recursive::recursive] + fn format_join(&self, ctx: &mut FormatContext<'_>) -> Result> { + let left_child = self.inner.left.formatter()?.format_join(ctx)?; + let right_child = self.inner.right.formatter()?.format_join(ctx)?; + + let children = vec![ + FormatTreeNode::with_children("Left".to_string(), vec![left_child]), + FormatTreeNode::with_children("Right".to_string(), vec![right_child]), + ]; + + Ok(FormatTreeNode::with_children( + format!("NestedLoopJoin: {}", self.inner.join_type), + children, + )) + } + + #[recursive::recursive] + fn partial_format(&self, ctx: &mut FormatContext<'_>) -> Result> { + let left_child = self.inner.left.formatter()?.partial_format(ctx)?; + let right_child = self.inner.right.formatter()?.partial_format(ctx)?; + + let mut children = vec![]; + if let Some(info) = &self.inner.stat_info { + let items = plan_stats_info_to_format_tree(info); + children.extend(items); + } + + append_output_rows_info(&mut children, &ctx.profs, self.inner.get_id()); + + children.push(FormatTreeNode::with_children("Left".to_string(), vec![ + left_child, + ])); + children.push(FormatTreeNode::with_children("Right".to_string(), vec![ + right_child, + ])); + + Ok(FormatTreeNode::with_children( + format!("NestedLoopJoin: {}", self.inner.join_type), + children, + )) + } +} diff --git a/src/query/service/src/physical_plans/format/mod.rs b/src/query/service/src/physical_plans/format/mod.rs index 8166b60bc64d8..39b5b3138d07b 100644 --- a/src/query/service/src/physical_plans/format/mod.rs +++ b/src/query/service/src/physical_plans/format/mod.rs @@ -45,6 +45,7 @@ mod format_mutation_into_organize; mod format_mutation_into_split; mod format_mutation_manipulate; mod format_mutation_source; +mod format_nested_loop_join; mod format_project_set; mod format_range_join; mod format_replace_into; @@ -93,6 +94,7 @@ pub use format_mutation_into_organize::*; pub use format_mutation_into_split::*; pub use format_mutation_manipulate::*; pub use format_mutation_source::*; +pub use format_nested_loop_join::*; pub use format_project_set::*; pub use format_range_join::*; pub use format_replace_into::*; diff --git a/src/query/service/src/physical_plans/mod.rs b/src/query/service/src/physical_plans/mod.rs index a6008acbace85..2c607ce257598 100644 --- a/src/query/service/src/physical_plans/mod.rs +++ b/src/query/service/src/physical_plans/mod.rs @@ -42,6 +42,7 @@ mod physical_mutation_into_organize; mod physical_mutation_into_split; mod physical_mutation_manipulate; mod physical_mutation_source; +mod physical_nested_loop_join; mod physical_project_set; mod physical_r_cte_scan; mod physical_range_join; @@ -90,6 +91,7 @@ pub use physical_mutation_into_organize::MutationOrganize; pub use physical_mutation_into_split::MutationSplit; pub use physical_mutation_manipulate::MutationManipulate; pub use physical_mutation_source::*; +pub use physical_nested_loop_join::NestedLoopJoin; pub use physical_project_set::ProjectSet; pub use physical_r_cte_scan::RecursiveCteScan; pub use physical_range_join::*; diff --git a/src/query/service/src/physical_plans/physical_hash_join.rs b/src/query/service/src/physical_plans/physical_hash_join.rs index 553a462ba688e..a1f7ca9403ec2 100644 --- a/src/query/service/src/physical_plans/physical_hash_join.rs +++ b/src/query/service/src/physical_plans/physical_hash_join.rs @@ -527,13 +527,15 @@ impl PhysicalPlanBuilder { required: &mut ColumnSet, others_required: &mut ColumnSet, ) -> (Vec, Vec) { - let retained_columns = self.metadata.read().get_retained_column().clone(); - *required = required.union(&retained_columns).cloned().collect(); - let column_projections = required.clone().into_iter().collect::>(); - - *others_required = others_required.union(&retained_columns).cloned().collect(); - let pre_column_projections = others_required.clone().into_iter().collect::>(); + { + let metadata = self.metadata.read(); + let retained_columns = metadata.get_retained_column(); + required.extend(retained_columns); + others_required.extend(retained_columns); + } + let column_projections = required.iter().copied().collect(); + let pre_column_projections = others_required.iter().copied().collect(); (column_projections, pre_column_projections) } diff --git a/src/query/service/src/physical_plans/physical_join.rs b/src/query/service/src/physical_plans/physical_join.rs index bc48246312e38..347b5ef48e6b1 100644 --- a/src/query/service/src/physical_plans/physical_join.rs +++ b/src/query/service/src/physical_plans/physical_join.rs @@ -14,6 +14,7 @@ use databend_common_exception::ErrorCode; use databend_common_exception::Result; +use databend_common_settings::Settings; use databend_common_sql::binder::is_range_join_condition; use databend_common_sql::optimizer::ir::RelExpr; use databend_common_sql::optimizer::ir::SExpr; @@ -30,17 +31,52 @@ use crate::physical_plans::PhysicalPlanBuilder; enum PhysicalJoinType { Hash, // The first arg is range conditions, the second arg is other conditions - RangeJoin(Vec, Vec), + RangeJoin { + range: Vec, + other: Vec, + }, + LoopJoin { + conditions: Vec, + }, } // Choose physical join type by join conditions -fn physical_join(join: &Join, s_expr: &SExpr) -> Result { +fn physical_join(join: &Join, s_expr: &SExpr, settings: &Settings) -> Result { if join.equi_conditions.is_empty() && join.join_type.is_any_join() { return Err(ErrorCode::SemanticError( "ANY JOIN only supports equality-based hash joins", )); } + let left_rel_expr = RelExpr::with_s_expr(s_expr.left_child()); + let right_rel_expr = RelExpr::with_s_expr(s_expr.right_child()); + let right_stat_info = right_rel_expr.derive_cardinality()?; + let nested_loop_join_threshold = settings.get_nested_loop_join_threshold()?; + if matches!(join.join_type, JoinType::Inner | JoinType::Cross) + && (right_stat_info + .statistics + .precise_cardinality + .map(|n| n < nested_loop_join_threshold) + .unwrap_or(false) + || right_stat_info.cardinality < nested_loop_join_threshold as _) + { + let conditions = join + .non_equi_conditions + .iter() + .cloned() + .chain(join.equi_conditions.iter().cloned().map(|condition| { + FunctionCall { + span: condition.left.span(), + func_name: "eq".to_string(), + params: vec![], + arguments: vec![condition.left, condition.right], + } + .into() + })) + .collect(); + return Ok(PhysicalJoinType::LoopJoin { conditions }); + }; + if !join.equi_conditions.is_empty() { // Contain equi condition, use hash join return Ok(PhysicalJoinType::Hash); @@ -51,9 +87,6 @@ fn physical_join(join: &Join, s_expr: &SExpr) -> Result { return Ok(PhysicalJoinType::Hash); } - let left_rel_expr = RelExpr::with_s_expr(s_expr.child(0)?); - let right_rel_expr = RelExpr::with_s_expr(s_expr.child(1)?); - let right_stat_info = right_rel_expr.derive_cardinality()?; if matches!(right_stat_info.statistics.precise_cardinality, Some(1)) || right_stat_info.cardinality == 1.0 { @@ -61,22 +94,22 @@ fn physical_join(join: &Join, s_expr: &SExpr) -> Result { return Ok(PhysicalJoinType::Hash); } - let left_prop = left_rel_expr.derive_relational_prop()?; - let right_prop = right_rel_expr.derive_relational_prop()?; - let (range_conditions, other_conditions) = join - .non_equi_conditions - .iter() - .cloned() - .partition::, _>(|condition| { - is_range_join_condition(condition, &left_prop, &right_prop).is_some() - }); - - if !range_conditions.is_empty() && matches!(join.join_type, JoinType::Inner | JoinType::Cross) { - return Ok(PhysicalJoinType::RangeJoin( - range_conditions, - other_conditions, - )); + if matches!(join.join_type, JoinType::Inner | JoinType::Cross) { + let left_prop = left_rel_expr.derive_relational_prop()?; + let right_prop = right_rel_expr.derive_relational_prop()?; + let (range, other) = join + .non_equi_conditions + .iter() + .cloned() + .partition::, _>(|condition| { + is_range_join_condition(condition, &left_prop, &right_prop).is_some() + }); + + if !range.is_empty() { + return Ok(PhysicalJoinType::RangeJoin { range, other }); + } } + // Leverage hash join to execute nested loop join Ok(PhysicalJoinType::Hash) } @@ -157,7 +190,8 @@ impl PhysicalPlanBuilder { ) .await } else { - match physical_join(join, s_expr)? { + let settings = self.ctx.get_settings(); + match physical_join(join, s_expr, &settings)? { PhysicalJoinType::Hash => { self.build_hash_join( join, @@ -170,7 +204,7 @@ impl PhysicalPlanBuilder { ) .await } - PhysicalJoinType::RangeJoin(range, other) => { + PhysicalJoinType::RangeJoin { range, other } => { self.build_range_join( join.join_type, s_expr, @@ -181,6 +215,16 @@ impl PhysicalPlanBuilder { ) .await } + PhysicalJoinType::LoopJoin { conditions } => { + self.build_loop_join( + join.join_type, + s_expr, + left_required, + right_required, + conditions, + ) + .await + } } } } diff --git a/src/query/service/src/physical_plans/physical_nested_loop_join.rs b/src/query/service/src/physical_plans/physical_nested_loop_join.rs new file mode 100644 index 0000000000000..91267b84b2615 --- /dev/null +++ b/src/query/service/src/physical_plans/physical_nested_loop_join.rs @@ -0,0 +1,211 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::any::Any; +use std::sync::Arc; + +use databend_common_exception::Result; +use databend_common_expression::DataSchemaRef; +use databend_common_expression::DataSchemaRefExt; +use databend_common_expression::RemoteExpr; +use databend_common_functions::BUILTIN_FUNCTIONS; +use databend_common_pipeline::core::ProcessorPtr; +use databend_common_pipeline::sinks::Sinker; +use databend_common_sql::executor::cast_expr_to_non_null_boolean; +use databend_common_sql::optimizer::ir::SExpr; +use databend_common_sql::plans::JoinType; +use databend_common_sql::ColumnSet; +use databend_common_sql::ScalarExpr; +use itertools::Itertools; + +use super::explain::PlanStatsInfo; +use super::format::NestedLoopJoinFormatter; +use super::format::PhysicalFormat; +use super::resolve_scalar; +use super::IPhysicalPlan; +use super::PhysicalPlan; +use super::PhysicalPlanBuilder; +use super::PhysicalPlanMeta; +use crate::pipelines::processors::transforms::LoopJoinState; +use crate::pipelines::processors::transforms::TransformLoopJoinLeft; +use crate::pipelines::processors::transforms::TransformLoopJoinRight; +use crate::pipelines::PipelineBuilder; + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct NestedLoopJoin { + pub meta: PhysicalPlanMeta, + pub left: PhysicalPlan, + pub right: PhysicalPlan, + pub conditions: Vec, + pub join_type: JoinType, + pub output_schema: DataSchemaRef, + + // Only used for explain + pub stat_info: Option, +} + +#[typetag::serde] +impl IPhysicalPlan for NestedLoopJoin { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + #[recursive::recursive] + fn output_schema(&self) -> Result { + Ok(self.output_schema.clone()) + } + + fn children<'a>(&'a self) -> Box + 'a> { + Box::new([&self.left, &self.right].into_iter()) + } + + fn children_mut<'a>(&'a mut self) -> Box + 'a> { + Box::new([&mut self.left, &mut self.right].into_iter()) + } + + fn formatter(&self) -> Result> { + Ok(NestedLoopJoinFormatter::create(self)) + } + + fn get_desc(&self) -> Result { + Ok(self + .conditions + .iter() + .map(|x| x.as_expr(&BUILTIN_FUNCTIONS).sql_display()) + .join(" AND ")) + } + + fn derive(&self, mut children: Vec) -> PhysicalPlan { + assert_eq!(children.len(), 2); + let right_child = children.pop().unwrap(); + let left_child = children.pop().unwrap(); + + PhysicalPlan::new(NestedLoopJoin { + meta: self.meta.clone(), + left: left_child, + right: right_child, + conditions: self.conditions.clone(), + join_type: self.join_type, + output_schema: self.output_schema.clone(), + stat_info: self.stat_info.clone(), + }) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + let state = Arc::new(LoopJoinState::new(builder.ctx.clone(), self)); + self.build_right(state.clone(), builder)?; + self.build_left(state, builder) + } +} + +impl NestedLoopJoin { + fn build_left(&self, state: Arc, builder: &mut PipelineBuilder) -> Result<()> { + self.left.build_pipeline(builder)?; + + let max_threads = builder.settings.get_max_threads()? as usize; + builder.main_pipeline.try_resize(max_threads)?; + builder.main_pipeline.add_transform(|input, output| { + Ok(ProcessorPtr::create(TransformLoopJoinLeft::create( + input, + output, + state.clone(), + ))) + })?; + + match self.conditions.len() { + 0 => Ok(()), + 1 => { + let expr = + cast_expr_to_non_null_boolean(self.conditions[0].as_expr(&BUILTIN_FUNCTIONS))?; + let projections = (0..self.output_schema.num_fields()).collect(); + builder.main_pipeline.add_transform( + builder.filter_transform_builder(&[expr.as_remote_expr()], projections)?, + ) + } + _ => { + let projections = (0..self.output_schema.num_fields()).collect(); + builder + .main_pipeline + .add_transform(builder.filter_transform_builder(&self.conditions, projections)?) + } + } + } + + fn build_right(&self, state: Arc, builder: &mut PipelineBuilder) -> Result<()> { + let right_side_builder = builder.create_sub_pipeline_builder(); + + let mut right_res = right_side_builder.finalize(&self.right)?; + right_res.main_pipeline.add_sink(|input| { + Ok(ProcessorPtr::create(Sinker::create( + input, + TransformLoopJoinRight::create(state.clone())?, + ))) + })?; + + builder + .pipelines + .push(right_res.main_pipeline.finalize(None)); + builder.pipelines.extend(right_res.sources_pipelines); + Ok(()) + } +} + +impl PhysicalPlanBuilder { + pub async fn build_loop_join( + &mut self, + join_type: JoinType, + s_expr: &SExpr, + left_required: ColumnSet, + right_required: ColumnSet, + conditions: Vec, + ) -> Result { + let (left, right) = self + .build_join_sides(s_expr, left_required, right_required) + .await?; + + let left_schema = self.prepare_probe_schema(join_type, &left)?; + let right_schema = self.prepare_build_schema(join_type, &right)?; + + let output_schema = DataSchemaRefExt::create( + left_schema + .fields() + .iter() + .chain(right_schema.fields()) + .cloned() + .collect::>(), + ); + + let conditions = conditions + .iter() + .map(|scalar| resolve_scalar(scalar, &output_schema)) + .collect::>()?; + + Ok(PhysicalPlan::new(NestedLoopJoin { + left, + right, + meta: PhysicalPlanMeta::new("NestedLoopJoin"), + conditions, + join_type, + output_schema, + stat_info: Some(self.build_plan_stat_info(s_expr)?), + })) + } +} diff --git a/src/query/service/src/physical_plans/physical_range_join.rs b/src/query/service/src/physical_plans/physical_range_join.rs index 9c9723edf6e73..1be812e3f26d0 100644 --- a/src/query/service/src/physical_plans/physical_range_join.rs +++ b/src/query/service/src/physical_plans/physical_range_join.rs @@ -20,7 +20,6 @@ use databend_common_exception::Result; use databend_common_expression::type_check::common_super_type; use databend_common_expression::DataSchema; use databend_common_expression::DataSchemaRef; -use databend_common_expression::DataSchemaRefExt; use databend_common_expression::RemoteExpr; use databend_common_functions::BUILTIN_FUNCTIONS; use databend_common_pipeline::core::ProcessorPtr; @@ -232,10 +231,7 @@ impl PhysicalPlanBuilder { let left_schema = self.prepare_probe_schema(join_type, &left_side)?; let right_schema = self.prepare_build_schema(join_type, &right_side)?; - let mut output_schema = Vec::clone(left_schema.fields()); - output_schema.extend_from_slice(right_schema.fields()); - - let merged_schema = DataSchemaRefExt::create( + let output_schema = DataSchema::new_ref( left_schema .fields() .iter() @@ -262,11 +258,11 @@ impl PhysicalPlanBuilder { .collect::>()?, other_conditions: other_conditions .iter() - .map(|scalar| resolve_scalar(scalar, &merged_schema)) + .map(|scalar| resolve_scalar(scalar, &output_schema)) .collect::>()?, join_type, range_join_type, - output_schema: Arc::new(DataSchema::new(output_schema)), + output_schema, stat_info: Some(self.build_plan_stat_info(s_expr)?), })) } @@ -343,9 +339,9 @@ fn resolve_range_condition( } } -fn resolve_scalar(scalar: &ScalarExpr, schema: &DataSchemaRef) -> Result { +pub fn resolve_scalar(scalar: &ScalarExpr, schema: &DataSchema) -> Result { let expr = scalar - .type_check(schema.as_ref())? + .type_check(schema)? .project_column_ref(|index| schema.index_of(&index.to_string()))?; Ok(expr.as_remote_expr()) } diff --git a/src/query/service/src/pipelines/processors/transforms/mod.rs b/src/query/service/src/pipelines/processors/transforms/mod.rs index f1d74da1383a3..c56bf9dd853ff 100644 --- a/src/query/service/src/pipelines/processors/transforms/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/mod.rs @@ -16,6 +16,7 @@ pub mod aggregator; mod broadcast; mod hash_join; mod materialized_cte; +mod nested_loop_join; mod new_hash_join; pub(crate) mod range_join; mod runtime_pool; @@ -40,6 +41,7 @@ pub use broadcast::BroadcastSourceProcessor; pub use hash_join::*; pub use materialized_cte::CTESource; pub use materialized_cte::MaterializedCteSink; +pub use nested_loop_join::*; pub use new_hash_join::Join; pub use new_hash_join::TransformHashJoin; pub use new_hash_join::*; diff --git a/src/query/service/src/pipelines/processors/transforms/nested_loop_join/mod.rs b/src/query/service/src/pipelines/processors/transforms/nested_loop_join/mod.rs new file mode 100644 index 0000000000000..ab67f4854b88a --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/nested_loop_join/mod.rs @@ -0,0 +1,16 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +mod transform_loop_join; +pub use transform_loop_join::*; diff --git a/src/query/service/src/pipelines/processors/transforms/nested_loop_join/transform_loop_join.rs b/src/query/service/src/pipelines/processors/transforms/nested_loop_join/transform_loop_join.rs new file mode 100644 index 0000000000000..def08d92ffa98 --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/nested_loop_join/transform_loop_join.rs @@ -0,0 +1,257 @@ +// 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 std::sync::Mutex; +use std::sync::RwLock; + +use databend_common_exception::Result; +use databend_common_expression::BlockEntry; +use databend_common_expression::DataBlock; +use databend_common_pipeline::core::Event; +use databend_common_pipeline::core::InputPort; +use databend_common_pipeline::core::OutputPort; +use databend_common_pipeline::core::Processor; +use databend_common_pipeline::sinks::Sink; +use databend_common_sql::plans::JoinType; + +use crate::physical_plans::NestedLoopJoin; +use crate::pipelines::executor::WatchNotify; +use crate::sessions::QueryContext; + +pub struct TransformLoopJoinLeft { + input_port: Arc, + output_port: Arc, + output_data_blocks: VecDeque, + state: Arc, +} + +impl TransformLoopJoinLeft { + pub fn create( + input_port: Arc, + output_port: Arc, + state: Arc, + ) -> Box { + Box::new(TransformLoopJoinLeft { + input_port, + output_port, + output_data_blocks: Default::default(), + state, + }) + } +} + +#[async_trait::async_trait] +impl Processor for TransformLoopJoinLeft { + fn name(&self) -> String { + "TransformLoopJoinLeft".to_string() + } + + fn as_any(&mut self) -> &mut dyn Any { + self + } + + fn event(&mut self) -> Result { + if self.output_port.is_finished() { + return Ok(Event::Finished); + } + if !self.output_port.can_push() { + return Ok(Event::NeedConsume); + } + if let Some(data) = self.output_data_blocks.pop_front() { + self.output_port.push_data(Ok(data)); + return Ok(Event::NeedConsume); + } + + if !self.state.is_right_finish()? { + return Ok(Event::Async); + } + + if self.input_port.is_finished() { + // todo!() + self.output_port.finish(); + return Ok(Event::Finished); + } + if !self.input_port.has_data() { + self.input_port.set_need_data(); + return Ok(Event::NeedData); + } + Ok(Event::Sync) + } + + fn process(&mut self) -> Result<()> { + if let Some(data) = self.input_port.pull_data() { + self.output_data_blocks = self.state.loop_join(data?)?; + } + Ok(()) + } + + #[async_backtrace::framed] + async fn async_process(&mut self) -> Result<()> { + self.state.wait_right_finish().await?; + Ok(()) + } +} + +pub struct TransformLoopJoinRight { + state: Arc, +} + +impl TransformLoopJoinRight { + pub fn create(state: Arc) -> Result { + state.attach_right()?; + Ok(TransformLoopJoinRight { state }) + } +} + +impl Sink for TransformLoopJoinRight { + const NAME: &'static str = "TransformLoopJoinRight"; + + fn consume(&mut self, data_block: DataBlock) -> Result<()> { + self.state.sink_right(data_block) + } + + fn on_finish(&mut self) -> Result<()> { + self.state.detach_right()?; + Ok(()) + } +} + +pub struct LoopJoinState { + right_table: RwLock>, + right_finished: Mutex, + finished_notify: Arc, + + right_sinker_count: RwLock, + + #[allow(dead_code)] + join_type: JoinType, +} + +impl LoopJoinState { + pub fn new(_ctx: Arc, join: &NestedLoopJoin) -> Self { + Self { + right_table: RwLock::new(vec![]), + right_finished: Mutex::new(false), + finished_notify: Arc::new(WatchNotify::new()), + right_sinker_count: RwLock::new(0), + join_type: join.join_type, + } + } + + fn attach_right(&self) -> Result<()> { + let mut right_sinker_count = self.right_sinker_count.write()?; + *right_sinker_count += 1; + Ok(()) + } + + fn sink_right(&self, right_block: DataBlock) -> Result<()> { + let right = if matches!(self.join_type, JoinType::Left | JoinType::Full) { + let rows = right_block.num_rows(); + let entries = right_block + .take_columns() + .into_iter() + .map(|entry| entry.into_nullable()) + .collect::>(); + DataBlock::new(entries, rows) + } else { + right_block + }; + self.right_table.write()?.push(right); + Ok(()) + } + + fn detach_right(&self) -> Result<()> { + let finished = { + let mut right_sinker_count = self.right_sinker_count.write()?; + *right_sinker_count -= 1; + *right_sinker_count == 0 + }; + if finished { + // todo + *self.right_finished.lock()? = true; + self.finished_notify.notify_waiters(); + } + Ok(()) + } + + fn is_right_finish(&self) -> Result { + Ok(*self.right_finished.lock()?) + } + + async fn wait_right_finish(&self) -> Result<()> { + if !*self.right_finished.lock()? { + self.finished_notify.notified().await + } + Ok(()) + } + + fn loop_join(&self, left_block: DataBlock) -> Result> { + let left = if matches!(self.join_type, JoinType::Right | JoinType::Full) { + let rows = left_block.num_rows(); + let entries = left_block + .take_columns() + .into_iter() + .map(|entry| entry.into_nullable()) + .collect::>(); + DataBlock::new(entries, rows) + } else { + left_block + }; + + let right_table = self.right_table.read()?; + let mut blocks = VecDeque::with_capacity(right_table.len() * left.num_rows()); + for right in right_table.iter() { + blocks.extend(self.single_loop_join(&left, right)?); + } + Ok(blocks) + } + + fn single_loop_join<'a>( + &self, + left: &'a DataBlock, + right: &'a DataBlock, + ) -> Result + use<'a>> { + let mut left_blocks = vec![Vec::new(); left.num_rows()]; + for entry in left.columns().iter() { + match entry { + BlockEntry::Const(scalar, _, _) => { + for left_entries in &mut left_blocks { + left_entries.push(scalar.as_ref()); + } + } + BlockEntry::Column(column) => { + for (left_entries, scalar) in left_blocks.iter_mut().zip(column.iter()) { + left_entries.push(scalar); + } + } + } + } + + let iter = left_blocks.into_iter().map(|left_entries| { + let entries = left_entries + .iter() + .zip(left.columns().iter().map(|entry| entry.data_type())) + .map(|(scalar, data_type)| { + BlockEntry::Const(scalar.to_owned(), data_type, right.num_rows()) + }) + .chain(right.columns().iter().cloned()) + .collect(); + DataBlock::new(entries, right.num_rows()) + }); + Ok(iter) + } +} diff --git a/src/query/service/src/pipelines/processors/transforms/range_join/range_join_state.rs b/src/query/service/src/pipelines/processors/transforms/range_join/range_join_state.rs index 45b5acea21e5e..19b88f044cd7d 100644 --- a/src/query/service/src/pipelines/processors/transforms/range_join/range_join_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/range_join/range_join_state.rs @@ -17,12 +17,12 @@ use std::sync::atomic::AtomicU64; use std::sync::Arc; use databend_common_catalog::table_context::TableContext; -use databend_common_column::bitmap::Bitmap; use databend_common_column::bitmap::MutableBitmap; use databend_common_exception::Result; use databend_common_expression::types::DataType; use databend_common_expression::types::NumberDataType; use databend_common_expression::types::NumberScalar; +use databend_common_expression::BlockEntry; use databend_common_expression::ColumnBuilder; use databend_common_expression::DataBlock; use databend_common_expression::Evaluator; @@ -39,7 +39,6 @@ use crate::physical_plans::RangeJoinCondition; use crate::physical_plans::RangeJoinType; use crate::pipelines::executor::WatchNotify; use crate::pipelines::processors::transforms::range_join::IEJoinState; -use crate::pipelines::processors::transforms::wrap_true_validity; use crate::sessions::QueryContext; pub struct RangeJoinState { @@ -109,16 +108,17 @@ impl RangeJoinState { pub(crate) fn sink_right(&self, block: DataBlock) -> Result<()> { // Sink block to right table let mut right_table = self.right_table.write(); - let mut right_block = block; - if matches!(self.join_type, JoinType::Left | JoinType::LeftAsof) { - let validity = Bitmap::new_constant(true, right_block.num_rows()); - let nullable_right_columns = right_block - .columns() - .iter() - .map(|c| wrap_true_validity(c, right_block.num_rows(), &validity)) - .collect::>(); - right_block = DataBlock::new(nullable_right_columns, right_block.num_rows()); - } + let right_block = if matches!(self.join_type, JoinType::Left | JoinType::LeftAsof) { + let rows = block.num_rows(); + let nullable_right_columns = block + .take_columns() + .into_iter() + .map(BlockEntry::into_nullable) + .collect(); + DataBlock::new(nullable_right_columns, rows) + } else { + block + }; right_table.push(right_block); Ok(()) } @@ -126,16 +126,17 @@ impl RangeJoinState { pub(crate) fn sink_left(&self, block: DataBlock) -> Result<()> { // Sink block to left table let mut left_table = self.left_table.write(); - let mut left_block = block; - if matches!(self.join_type, JoinType::Right | JoinType::RightAsof) { - let validity = Bitmap::new_constant(true, left_block.num_rows()); - let nullable_left_columns = left_block - .columns() - .iter() - .map(|c| wrap_true_validity(c, left_block.num_rows(), &validity)) - .collect::>(); - left_block = DataBlock::new(nullable_left_columns, left_block.num_rows()); - } + let left_block = if matches!(self.join_type, JoinType::Right | JoinType::RightAsof) { + let rows = block.num_rows(); + let nullable_left_columns = block + .take_columns() + .into_iter() + .map(BlockEntry::into_nullable) + .collect(); + DataBlock::new(nullable_left_columns, rows) + } else { + block + }; left_table.push(left_block); Ok(()) } diff --git a/src/query/service/src/pipelines/processors/transforms/range_join/transform_range_join.rs b/src/query/service/src/pipelines/processors/transforms/range_join/transform_range_join.rs index bda89435842c7..d62382fcd3e05 100644 --- a/src/query/service/src/pipelines/processors/transforms/range_join/transform_range_join.rs +++ b/src/query/service/src/pipelines/processors/transforms/range_join/transform_range_join.rs @@ -24,7 +24,7 @@ use databend_common_pipeline::core::OutputPort; use databend_common_pipeline::core::Processor; use databend_common_pipeline::sinks::Sink; -use crate::pipelines::processors::transforms::range_join::RangeJoinState; +use super::RangeJoinState; enum RangeJoinStep { Sink, diff --git a/src/query/settings/src/settings_default.rs b/src/query/settings/src/settings_default.rs index 4a341ba6e2597..9089ced456a96 100644 --- a/src/query/settings/src/settings_default.rs +++ b/src/query/settings/src/settings_default.rs @@ -495,6 +495,13 @@ impl DefaultSettings { scope: SettingScope::Both, range: Some(SettingRange::Numeric(0..=u64::MAX)), }), + ("nested_loop_join_threshold", DefaultSettingValue { + value: UserSettingValue::UInt64(1024), + desc: "Set the threshold for use nested loop join. Setting it to 0 disable nested loop join.", + mode: SettingMode::Both, + scope: SettingScope::Both, + range: Some(SettingRange::Numeric(0..=u64::MAX)), + }), ("enable_bloom_runtime_filter", DefaultSettingValue { value: UserSettingValue::UInt64(1), desc: "Enables bloom runtime filter optimization for JOIN.", diff --git a/src/query/settings/src/settings_getter_setter.rs b/src/query/settings/src/settings_getter_setter.rs index 6057058270e25..074c652743d05 100644 --- a/src/query/settings/src/settings_getter_setter.rs +++ b/src/query/settings/src/settings_getter_setter.rs @@ -380,6 +380,10 @@ impl Settings { Ok(self.try_get_u64("inlist_to_join_threshold")? as usize) } + pub fn get_nested_loop_join_threshold(&self) -> Result { + Ok(self.try_get_u64("nested_loop_join_threshold")?) + } + pub fn get_bloom_runtime_filter(&self) -> Result { Ok(self.try_get_u64("enable_bloom_runtime_filter")? != 0) } diff --git a/tests/sqllogictests/suites/duckdb/join/inner/equality_join_limits.test b/tests/sqllogictests/suites/duckdb/join/inner/equality_join_limits.test index 70be64638f413..1ef504ff4fe9d 100644 --- a/tests/sqllogictests/suites/duckdb/join/inner/equality_join_limits.test +++ b/tests/sqllogictests/suites/duckdb/join/inner/equality_join_limits.test @@ -1,18 +1,12 @@ -statement ok -drop table if exists t; - -statement ok -drop table if exists u; - # TINYINT limits statement ok -CREATE TABLE t(t_k0 TINYINT); +CREATE OR REPLACE TABLE t(t_k0 TINYINT); statement ok INSERT INTO t VALUES (-128), (127); statement ok -CREATE TABLE u(u_k0 TINYINT); +CREATE OR REPLACE TABLE u(u_k0 TINYINT); statement ok INSERT INTO u VALUES (-128), (127); From f822514c9b545359b71442c6c70f1762d2cc7649 Mon Sep 17 00:00:00 2001 From: coldWater Date: Thu, 13 Nov 2025 12:43:00 +0800 Subject: [PATCH 02/19] refine --- .../src/pipelines/processors/transforms/mod.rs | 4 ++-- .../transforms/nested_loop_join/mod.rs | 16 ---------------- .../transform_loop_join.rs | 0 src/query/settings/src/settings_getter_setter.rs | 2 +- 4 files changed, 3 insertions(+), 19 deletions(-) delete mode 100644 src/query/service/src/pipelines/processors/transforms/nested_loop_join/mod.rs rename src/query/service/src/pipelines/processors/transforms/{nested_loop_join => }/transform_loop_join.rs (100%) diff --git a/src/query/service/src/pipelines/processors/transforms/mod.rs b/src/query/service/src/pipelines/processors/transforms/mod.rs index c56bf9dd853ff..b3e98f3479033 100644 --- a/src/query/service/src/pipelines/processors/transforms/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/mod.rs @@ -16,7 +16,6 @@ pub mod aggregator; mod broadcast; mod hash_join; mod materialized_cte; -mod nested_loop_join; mod new_hash_join; pub(crate) mod range_join; mod runtime_pool; @@ -26,6 +25,7 @@ mod transform_branched_async_function; mod transform_cache_scan; mod transform_dictionary; mod transform_expression_scan; +mod transform_loop_join; mod transform_recursive_cte_scan; mod transform_recursive_cte_source; @@ -41,7 +41,6 @@ pub use broadcast::BroadcastSourceProcessor; pub use hash_join::*; pub use materialized_cte::CTESource; pub use materialized_cte::MaterializedCteSink; -pub use nested_loop_join::*; pub use new_hash_join::Join; pub use new_hash_join::TransformHashJoin; pub use new_hash_join::*; @@ -55,6 +54,7 @@ pub use transform_cache_scan::HashJoinCacheState; pub use transform_cache_scan::NewHashJoinCacheState; pub use transform_cache_scan::TransformCacheScan; pub use transform_expression_scan::TransformExpressionScan; +pub use transform_loop_join::*; pub use transform_recursive_cte_scan::TransformRecursiveCteScan; pub use transform_recursive_cte_source::TransformRecursiveCteSource; pub use transform_resort_addon::TransformResortAddOn; diff --git a/src/query/service/src/pipelines/processors/transforms/nested_loop_join/mod.rs b/src/query/service/src/pipelines/processors/transforms/nested_loop_join/mod.rs deleted file mode 100644 index ab67f4854b88a..0000000000000 --- a/src/query/service/src/pipelines/processors/transforms/nested_loop_join/mod.rs +++ /dev/null @@ -1,16 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -mod transform_loop_join; -pub use transform_loop_join::*; diff --git a/src/query/service/src/pipelines/processors/transforms/nested_loop_join/transform_loop_join.rs b/src/query/service/src/pipelines/processors/transforms/transform_loop_join.rs similarity index 100% rename from src/query/service/src/pipelines/processors/transforms/nested_loop_join/transform_loop_join.rs rename to src/query/service/src/pipelines/processors/transforms/transform_loop_join.rs diff --git a/src/query/settings/src/settings_getter_setter.rs b/src/query/settings/src/settings_getter_setter.rs index 074c652743d05..e0cb508822177 100644 --- a/src/query/settings/src/settings_getter_setter.rs +++ b/src/query/settings/src/settings_getter_setter.rs @@ -381,7 +381,7 @@ impl Settings { } pub fn get_nested_loop_join_threshold(&self) -> Result { - Ok(self.try_get_u64("nested_loop_join_threshold")?) + self.try_get_u64("nested_loop_join_threshold") } pub fn get_bloom_runtime_filter(&self) -> Result { From be7ccb04d1664e8bdc9558f3977262cf4f14a294 Mon Sep 17 00:00:00 2001 From: coldWater Date: Thu, 13 Nov 2025 16:09:20 +0800 Subject: [PATCH 03/19] fix --- .../src/physical_plans/physical_join.rs | 41 +++++++++++---- .../transforms/new_hash_join/memory/basic.rs | 10 ++-- .../transforms/new_hash_join/mod.rs | 1 + .../transforms/transform_loop_join.rs | 50 ++++++++++++++++++- src/query/sql/src/planner/plans/join.rs | 6 ++- 5 files changed, 92 insertions(+), 16 deletions(-) diff --git a/src/query/service/src/physical_plans/physical_join.rs b/src/query/service/src/physical_plans/physical_join.rs index 347b5ef48e6b1..6f619551974a8 100644 --- a/src/query/service/src/physical_plans/physical_join.rs +++ b/src/query/service/src/physical_plans/physical_join.rs @@ -14,12 +14,14 @@ use databend_common_exception::ErrorCode; use databend_common_exception::Result; +use databend_common_expression::types::DataType; use databend_common_settings::Settings; use databend_common_sql::binder::is_range_join_condition; use databend_common_sql::optimizer::ir::RelExpr; use databend_common_sql::optimizer::ir::SExpr; use databend_common_sql::plans::FunctionCall; use databend_common_sql::plans::Join; +use databend_common_sql::plans::JoinEquiCondition; use databend_common_sql::plans::JoinType; use databend_common_sql::ColumnSet; use databend_common_sql::ScalarExpr; @@ -63,17 +65,9 @@ fn physical_join(join: &Join, s_expr: &SExpr, settings: &Settings) -> Result>()?; return Ok(PhysicalJoinType::LoopJoin { conditions }); }; @@ -229,3 +223,28 @@ impl PhysicalPlanBuilder { } } } + +fn condition_to_expr(condition: &JoinEquiCondition) -> Result { + let left_type = condition.left.data_type()?; + let right_type = condition.right.data_type()?; + + let arguments = match (&left_type, &right_type) { + (DataType::Nullable(left), right) if **left == *right => vec![ + condition.left.clone(), + condition.right.clone().unify_to_data_type(&left_type), + ], + (left, DataType::Nullable(right)) if *left == **right => vec![ + condition.left.clone().unify_to_data_type(&right_type), + condition.right.clone(), + ], + _ => vec![condition.left.clone(), condition.right.clone()], + }; + + Ok(FunctionCall { + span: condition.left.span(), + func_name: "eq".to_string(), + params: vec![], + arguments, + } + .into()) +} diff --git a/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/basic.rs b/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/basic.rs index 8168fbc081173..ffa47f6d079a9 100644 --- a/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/basic.rs +++ b/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/basic.rs @@ -60,17 +60,20 @@ impl BasicHashJoin { state: Arc, ) -> Result { let settings = ctx.get_settings(); - let block_size = settings.get_max_block_size()? as usize; - let block_bytes = settings.get_max_block_size()? as usize; + let squash_block = SquashBlocks::new( + settings.get_max_block_size()? as _, + settings.get_max_block_bytes()? as _, + ); Ok(BasicHashJoin { desc, state, method, function_ctx, - squash_block: SquashBlocks::new(block_size, block_bytes), + squash_block, }) } + pub(crate) fn add_block(&mut self, mut data: Option) -> Result<()> { let mut squashed_block = match data.take() { None => self.squash_block.finalize()?, @@ -128,6 +131,7 @@ impl BasicHashJoin { std::mem::swap(&mut chunks[chunk_index], &mut chunk_block); } + log::info!("build_hash_table chunk_index{chunk_index}"); self.build_hash_table(keys_block, chunk_index)?; Ok(Some(ProgressValues { diff --git a/src/query/service/src/pipelines/processors/transforms/new_hash_join/mod.rs b/src/query/service/src/pipelines/processors/transforms/new_hash_join/mod.rs index 002e21b578204..30a71a4a0ee72 100644 --- a/src/query/service/src/pipelines/processors/transforms/new_hash_join/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/new_hash_join/mod.rs @@ -26,6 +26,7 @@ mod transform_hash_join; pub use grace::GraceHashJoin; pub use hash_join_factory::HashJoinFactory; pub use join::Join; +pub use join::JoinStream; pub use memory::BasicHashJoinState; pub use memory::InnerHashJoin; pub use runtime_filter::RuntimeFiltersDesc; diff --git a/src/query/service/src/pipelines/processors/transforms/transform_loop_join.rs b/src/query/service/src/pipelines/processors/transforms/transform_loop_join.rs index def08d92ffa98..8f1b755342e3d 100644 --- a/src/query/service/src/pipelines/processors/transforms/transform_loop_join.rs +++ b/src/query/service/src/pipelines/processors/transforms/transform_loop_join.rs @@ -18,6 +18,7 @@ use std::sync::Arc; use std::sync::Mutex; use std::sync::RwLock; +use databend_common_base::base::ProgressValues; use databend_common_exception::Result; use databend_common_expression::BlockEntry; use databend_common_expression::DataBlock; @@ -28,6 +29,8 @@ use databend_common_pipeline::core::Processor; use databend_common_pipeline::sinks::Sink; use databend_common_sql::plans::JoinType; +use super::Join; +use super::JoinStream; use crate::physical_plans::NestedLoopJoin; use crate::pipelines::executor::WatchNotify; use crate::sessions::QueryContext; @@ -137,7 +140,6 @@ pub struct LoopJoinState { right_sinker_count: RwLock, - #[allow(dead_code)] join_type: JoinType, } @@ -255,3 +257,49 @@ impl LoopJoinState { Ok(iter) } } + +impl Join for LoopJoinState { + fn add_block(&mut self, data: Option) -> Result<()> { + let Some(right_block) = data else { + return Ok(()); + }; + + let right = if matches!(self.join_type, JoinType::Left | JoinType::Full) { + let rows = right_block.num_rows(); + let entries = right_block + .take_columns() + .into_iter() + .map(|entry| entry.into_nullable()) + .collect::>(); + DataBlock::new(entries, rows) + } else { + right_block + }; + self.right_table.write()?.push(right); + Ok(()) + } + + fn final_build(&mut self) -> Result> { + let progress = self.right_table.read()?.iter().fold( + ProgressValues::default(), + |mut progress, block| { + progress.rows += block.num_rows(); + progress.bytes += block.memory_size(); + progress + }, + ); + Ok(Some(progress)) + } + + fn probe_block(&mut self, data: DataBlock) -> Result> { + todo!(); + } +} + +struct LoopJoinStream {} + +impl JoinStream for LoopJoinStream { + fn next(&mut self) -> Result> { + todo!() + } +} diff --git a/src/query/sql/src/planner/plans/join.rs b/src/query/sql/src/planner/plans/join.rs index 7cc07b8b3909d..c7d6147379bc8 100644 --- a/src/query/sql/src/planner/plans/join.rs +++ b/src/query/sql/src/planner/plans/join.rs @@ -222,7 +222,11 @@ impl JoinEquiCondition { left.into_iter() .zip(right) .enumerate() - .map(|(index, (l, r))| JoinEquiCondition::new(l, r, is_null_equal.contains(&index))) + .map(|(index, (left, right))| Self { + left, + right, + is_null_equal: is_null_equal.contains(&index), + }) .collect() } } From d44a8e7b35a54067105283747ef352ea5b5a8d14 Mon Sep 17 00:00:00 2001 From: coldWater Date: Tue, 18 Nov 2025 13:26:43 +0800 Subject: [PATCH 04/19] x --- src/query/service/src/lib.rs | 1 + src/query/service/src/physical_plans/mod.rs | 2 +- .../src/physical_plans/physical_hash_join.rs | 160 ++++--- .../src/physical_plans/physical_join.rs | 4 +- .../physical_nested_loop_join.rs | 90 ++-- .../processors/transforms/hash_join/desc.rs | 33 +- .../hash_join/hash_join_build_state.rs | 1 + .../hash_join/hash_join_probe_state.rs | 2 + .../transforms/hash_join/hash_join_state.rs | 15 +- .../transforms/hash_join/result_blocks.rs | 4 +- .../transforms/new_hash_join/join.rs | 6 + .../transforms/new_hash_join/memory/basic.rs | 294 +++++++------ .../new_hash_join/memory/basic_state.rs | 17 + .../new_hash_join/memory/inner_join.rs | 112 ++--- .../transforms/new_hash_join/memory/mod.rs | 2 + .../new_hash_join/memory/nested_loop.rs | 90 ++++ .../new_hash_join/memory/outer_left_join.rs | 10 +- .../transforms/transform_loop_join.rs | 414 +++++++++++------- 18 files changed, 775 insertions(+), 482 deletions(-) create mode 100644 src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/nested_loop.rs diff --git a/src/query/service/src/lib.rs b/src/query/service/src/lib.rs index d7684a3d87788..949b678e04414 100644 --- a/src/query/service/src/lib.rs +++ b/src/query/service/src/lib.rs @@ -41,6 +41,7 @@ #![allow(clippy::diverging_sub_expression)] #![allow(clippy::arc_with_non_send_sync)] #![feature(debug_closure_helpers)] +#![feature(stmt_expr_attributes)] extern crate core; diff --git a/src/query/service/src/physical_plans/mod.rs b/src/query/service/src/physical_plans/mod.rs index 2c607ce257598..520834d696345 100644 --- a/src/query/service/src/physical_plans/mod.rs +++ b/src/query/service/src/physical_plans/mod.rs @@ -82,7 +82,7 @@ pub use physical_exchange::Exchange; pub use physical_exchange_sink::ExchangeSink; pub use physical_exchange_source::ExchangeSource; pub use physical_filter::Filter; -pub use physical_hash_join::HashJoin; +pub use physical_hash_join::*; pub use physical_limit::Limit; pub use physical_materialized_cte::*; pub use physical_multi_table_insert::*; diff --git a/src/query/service/src/physical_plans/physical_hash_join.rs b/src/query/service/src/physical_plans/physical_hash_join.rs index a1f7ca9403ec2..7d18c665e3879 100644 --- a/src/query/service/src/physical_plans/physical_hash_join.rs +++ b/src/query/service/src/physical_plans/physical_hash_join.rs @@ -35,7 +35,9 @@ use databend_common_pipeline::core::Pipe; use databend_common_pipeline::core::PipeItem; use databend_common_pipeline::core::ProcessorPtr; use databend_common_sql::optimizer::ir::SExpr; +use databend_common_sql::plans::FunctionCall; use databend_common_sql::plans::Join; +use databend_common_sql::plans::JoinEquiCondition; use databend_common_sql::plans::JoinType; use databend_common_sql::ColumnEntry; use databend_common_sql::ColumnSet; @@ -52,6 +54,7 @@ use crate::physical_plans::format::PhysicalFormat; use crate::physical_plans::physical_plan::IPhysicalPlan; use crate::physical_plans::physical_plan::PhysicalPlan; use crate::physical_plans::physical_plan::PhysicalPlanMeta; +use crate::physical_plans::resolve_scalar; use crate::physical_plans::runtime_filter::build_runtime_filter; use crate::physical_plans::Exchange; use crate::physical_plans::PhysicalPlanBuilder; @@ -99,6 +102,12 @@ type MergedFieldsResult = ( Vec<(usize, (bool, bool))>, ); +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct NestedLoopFilterInfo { + pub predicates: Vec, + pub projection: Vec, +} + #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct HashJoin { pub meta: PhysicalPlanMeta, @@ -140,6 +149,7 @@ pub struct HashJoin { pub runtime_filter: PhysicalRuntimeFilters, pub broadcast_id: Option, + pub nested_loop_filter: NestedLoopFilterInfo, } #[typetag::serde] @@ -261,6 +271,7 @@ impl IPhysicalPlan for HashJoin { build_side_cache_info: self.build_side_cache_info.clone(), runtime_filter: self.runtime_filter.clone(), broadcast_id: self.broadcast_id, + nested_loop_filter: self.nested_loop_filter.clone(), }) } @@ -1184,80 +1195,23 @@ impl PhysicalPlanBuilder { .collect::>() } - /// Creates a HashJoin physical plan - /// - /// # Arguments - /// * `join` - Join operation - /// * `probe_side` - Probe side physical plan - /// * `build_side` - Build side physical plan - /// * `is_broadcast` - Whether this is a broadcast join - /// * `projections` - Column projections - /// * `probe_projections` - Probe side projections - /// * `build_projections` - Build side projections - /// * `left_join_conditions` - Left join conditions - /// * `right_join_conditions` - Right join conditions - /// * `is_null_equal` - Null equality flags - /// * `non_equi_conditions` - Non-equi conditions - /// * `probe_to_build` - Probe to build mapping - /// * `output_schema` - Output schema - /// * `build_side_cache_info` - Build side cache info - /// * `runtime_filter` - Runtime filter - /// * `stat_info` - Statistics info - /// - /// # Returns - /// * `Result` - The HashJoin physical plan - #[allow(clippy::too_many_arguments)] - fn create_hash_join( + fn build_nested_loop_filter_info( &self, - s_expr: &SExpr, join: &Join, - probe_side: PhysicalPlan, - build_side: PhysicalPlan, - projections: ColumnSet, - probe_projections: ColumnSet, - build_projections: ColumnSet, - left_join_conditions: Vec, - right_join_conditions: Vec, - is_null_equal: Vec, - non_equi_conditions: Vec, - probe_to_build: Vec<(usize, (bool, bool))>, - output_schema: DataSchemaRef, - build_side_cache_info: Option<(usize, HashMap)>, - runtime_filter: PhysicalRuntimeFilters, - stat_info: PlanStatsInfo, - ) -> Result { - let build_side_data_distribution = s_expr.build_side_child().get_data_distribution()?; - let broadcast_id = if build_side_data_distribution - .as_ref() - .is_some_and(|e| matches!(e, databend_common_sql::plans::Exchange::NodeToNodeHash(_))) - { - Some(self.ctx.get_next_broadcast_id()) - } else { - None - }; - Ok(PhysicalPlan::new(HashJoin { - projections, - build_projections, - probe_projections, - build: build_side, - probe: probe_side, - join_type: join.join_type, - build_keys: right_join_conditions, - probe_keys: left_join_conditions, - is_null_equal, - non_equi_conditions, - marker_index: join.marker_index, - meta: PhysicalPlanMeta::new("HashJoin"), - from_correlated_subquery: join.from_correlated_subquery, - probe_to_build, - output_schema, - need_hold_hash_table: join.need_hold_hash_table, - stat_info: Some(stat_info), - single_to_inner: join.single_to_inner, - build_side_cache_info, - runtime_filter, - broadcast_id, - })) + merged_schema: &DataSchemaRef, + ) -> Result { + let predicates = join + .non_equi_conditions + .iter() + .map(|c| Ok(c.clone())) + .chain(join.equi_conditions.iter().map(condition_to_expr)) + .map(|scalar| resolve_scalar(&scalar?, merged_schema)) + .collect::>()?; + + Ok(NestedLoopFilterInfo { + predicates, + projection: vec![], + }) } pub async fn build_hash_join( @@ -1332,6 +1286,8 @@ impl PhysicalPlanBuilder { // Step 10: Process non-equi conditions let non_equi_conditions = self.process_non_equi_conditions(join, &merged_schema)?; + let nested_loop_filter = self.build_nested_loop_filter_info(join, &merged_schema)?; + // Step 11: Build runtime filter let runtime_filter = build_runtime_filter( self.ctx.clone(), @@ -1345,23 +1301,63 @@ impl PhysicalPlanBuilder { .await?; // Step 12: Create and return the HashJoin - self.create_hash_join( - s_expr, - join, - probe_side, - build_side, + let build_side_data_distribution = s_expr.build_side_child().get_data_distribution()?; + let broadcast_id = if build_side_data_distribution + .as_ref() + .is_some_and(|e| matches!(e, databend_common_sql::plans::Exchange::NodeToNodeHash(_))) + { + Some(self.ctx.get_next_broadcast_id()) + } else { + None + }; + Ok(PhysicalPlan::new(HashJoin { projections, - probe_projections, build_projections, - left_join_conditions, - right_join_conditions, + probe_projections, + build: build_side, + probe: probe_side, + join_type: join.join_type, + build_keys: right_join_conditions, + probe_keys: left_join_conditions, is_null_equal, non_equi_conditions, + marker_index: join.marker_index, + meta: PhysicalPlanMeta::new("HashJoin"), + from_correlated_subquery: join.from_correlated_subquery, probe_to_build, output_schema, + need_hold_hash_table: join.need_hold_hash_table, + stat_info: Some(stat_info), + single_to_inner: join.single_to_inner, build_side_cache_info, runtime_filter, - stat_info, - ) + broadcast_id, + nested_loop_filter, + })) + } +} + +fn condition_to_expr(condition: &JoinEquiCondition) -> Result { + let left_type = condition.left.data_type()?; + let right_type = condition.right.data_type()?; + + let arguments = match (&left_type, &right_type) { + (DataType::Nullable(left), right) if **left == *right => vec![ + condition.left.clone(), + condition.right.clone().unify_to_data_type(&left_type), + ], + (left, DataType::Nullable(right)) if *left == **right => vec![ + condition.left.clone().unify_to_data_type(&right_type), + condition.right.clone(), + ], + _ => vec![condition.left.clone(), condition.right.clone()], + }; + + Ok(FunctionCall { + span: condition.left.span(), + func_name: "eq".to_string(), + params: vec![], + arguments, } + .into()) } diff --git a/src/query/service/src/physical_plans/physical_join.rs b/src/query/service/src/physical_plans/physical_join.rs index 6f619551974a8..662ff26b76a87 100644 --- a/src/query/service/src/physical_plans/physical_join.rs +++ b/src/query/service/src/physical_plans/physical_join.rs @@ -43,7 +43,7 @@ enum PhysicalJoinType { } // Choose physical join type by join conditions -fn physical_join(join: &Join, s_expr: &SExpr, settings: &Settings) -> Result { +fn physical_join(join: &Join, s_expr: &SExpr, _settings: &Settings) -> Result { if join.equi_conditions.is_empty() && join.join_type.is_any_join() { return Err(ErrorCode::SemanticError( "ANY JOIN only supports equality-based hash joins", @@ -53,7 +53,7 @@ fn physical_join(join: &Join, s_expr: &SExpr, settings: &Settings) -> Result Result<()> { - let state = Arc::new(LoopJoinState::new(builder.ctx.clone(), self)); - self.build_right(state.clone(), builder)?; - self.build_left(state, builder) - } -} + // Build right side (build side) + let right_side_builder = builder.create_sub_pipeline_builder(); + let mut right_res = right_side_builder.finalize(&self.right)?; + let mut build_sinks = right_res.main_pipeline.take_sinks(); + builder + .pipelines + .push(right_res.main_pipeline.finalize(None)); + builder.pipelines.extend(right_res.sources_pipelines); -impl NestedLoopJoin { - fn build_left(&self, state: Arc, builder: &mut PipelineBuilder) -> Result<()> { + // Build left side (probe side) self.left.build_pipeline(builder)?; - let max_threads = builder.settings.get_max_threads()? as usize; - builder.main_pipeline.try_resize(max_threads)?; - builder.main_pipeline.add_transform(|input, output| { - Ok(ProcessorPtr::create(TransformLoopJoinLeft::create( - input, - output, - state.clone(), - ))) - })?; + let output_len = std::cmp::max(build_sinks.len(), builder.main_pipeline.output_len()); + builder.main_pipeline.resize(output_len, false)?; + let probe_sinks = builder.main_pipeline.take_sinks(); + + if output_len != build_sinks.len() { + builder.main_pipeline.extend_sinks(build_sinks); + builder.main_pipeline.resize(output_len, false)?; + build_sinks = builder.main_pipeline.take_sinks(); + } + + debug_assert_eq!(build_sinks.len(), probe_sinks.len()); + + let join_pipe_items = build_sinks + .into_iter() + .zip(probe_sinks) + .map(|(build_sink, probe_sink)| { + builder.main_pipeline.extend_sinks([build_sink, probe_sink]); + + let build_input = InputPort::create(); + let probe_input = InputPort::create(); + let joined_output = OutputPort::create(); + + let join_state = LoopJoinState::new(builder.ctx.clone(), self); + let loop_join = ProcessorPtr::create(TransformLoopJoin::create( + build_input.clone(), + probe_input.clone(), + joined_output.clone(), + Box::new(join_state), + )); + + PipeItem::create(loop_join, vec![build_input, probe_input], vec![ + joined_output, + ]) + }) + .collect(); + + let join_pipe = Pipe::create(output_len * 2, output_len, join_pipe_items); + builder.main_pipeline.add_pipe(join_pipe); match self.conditions.len() { 0 => Ok(()), @@ -148,24 +180,6 @@ impl NestedLoopJoin { } } } - - fn build_right(&self, state: Arc, builder: &mut PipelineBuilder) -> Result<()> { - let right_side_builder = builder.create_sub_pipeline_builder(); - - let mut right_res = right_side_builder.finalize(&self.right)?; - right_res.main_pipeline.add_sink(|input| { - Ok(ProcessorPtr::create(Sinker::create( - input, - TransformLoopJoinRight::create(state.clone())?, - ))) - })?; - - builder - .pipelines - .push(right_res.main_pipeline.finalize(None)); - builder.pipelines.extend(right_res.sources_pipelines); - Ok(()) - } } impl PhysicalPlanBuilder { 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 ae2a25d06733b..f5a470c7d5334 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 @@ -22,6 +22,7 @@ use databend_common_expression::DataBlock; use databend_common_expression::DataSchemaRef; use databend_common_expression::Evaluator; use databend_common_expression::Expr; +use databend_common_expression::FilterExecutor; use databend_common_expression::FunctionContext; use databend_common_expression::RemoteExpr; use databend_common_functions::BUILTIN_FUNCTIONS; @@ -30,6 +31,7 @@ use databend_common_sql::ColumnSet; use parking_lot::RwLock; use crate::physical_plans::HashJoin; +use crate::physical_plans::NestedLoopFilterInfo; use crate::physical_plans::PhysicalRuntimeFilter; use crate::physical_plans::PhysicalRuntimeFilters; use crate::pipelines::processors::transforms::wrap_true_validity; @@ -60,9 +62,10 @@ pub struct HashJoinDesc { pub(crate) runtime_filter: RuntimeFiltersDesc, pub(crate) build_projection: ColumnSet, - pub(crate) probe_projections: ColumnSet, + pub(crate) probe_projection: ColumnSet, pub(crate) probe_to_build: Vec<(usize, (bool, bool))>, pub(crate) build_schema: DataSchemaRef, + pub(crate) nested_loop_filter: NestedLoopFilterInfo, } #[derive(Debug, Clone)] @@ -136,8 +139,9 @@ impl HashJoinDesc { runtime_filter: (&join.runtime_filter).into(), probe_to_build: join.probe_to_build.clone(), build_projection: join.build_projections.clone(), - probe_projections: join.probe_projections.clone(), + probe_projection: join.probe_projections.clone(), build_schema: join.build.output_schema()?, + nested_loop_filter: join.nested_loop_filter.clone(), }) } @@ -259,4 +263,29 @@ impl HashJoinDesc { } } } + + pub fn create_nested_loop_filter( + &self, + function_ctx: &FunctionContext, + block_size: usize, + ) -> Result { + let predicates = self + .nested_loop_filter + .predicates + .iter() + .map(|x| Ok(x.as_expr(&BUILTIN_FUNCTIONS))) + .reduce(|lhs, rhs| { + check_function(None, "and_filters", &[], &[lhs?, rhs?], &BUILTIN_FUNCTIONS) + }) + .unwrap()?; + + Ok(FilterExecutor::new( + predicates, + function_ctx.clone(), + block_size, + None, // Some(self.nested_loop_filter.projection.iter().copied().collect()), + &BUILTIN_FUNCTIONS, + false, + )) + } } 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 380977cee104e..78380489e3fdf 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 @@ -744,6 +744,7 @@ impl HashJoinBuildState { "Aborted query, because the hash table is uninitialized.", )); } + HashJoinHashTable::NestedLoop(_) => unreachable!(), HashJoinHashTable::SkipDuplicatesSerializer(table) => insert_binary_key! { &mut table.hash_table, &table.hash_method, chunk, build_keys, valids, chunk_index as u32, entry_size, &mut local_raw_entry_spaces, }, 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 1632c7dab8371..0dd15dd68df6f 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 @@ -165,6 +165,7 @@ impl HashJoinProbeState { // Continue to probe hash table and process data blocks. self.result_blocks(probe_state, keys, &table.hash_table) } + HashJoinHashTable::NestedLoop(_) => unreachable!(), HashJoinHashTable::Null => Err(ErrorCode::AbortedQuery( "Aborted query, because the hash table is uninitialized.", )), @@ -376,6 +377,7 @@ impl HashJoinProbeState { // Continue to probe hash table and process data blocks. self.result_blocks(probe_state, keys, &table.hash_table) } + HashJoinHashTable::NestedLoop(_) => unreachable!(), HashJoinHashTable::Null => Err(ErrorCode::AbortedQuery( "Aborted query, because the hash table is uninitialized.", )), 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 d696ca2304e7b..9107d9ea12965 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 @@ -78,6 +78,7 @@ pub struct FixedKeyHashJoinHashTable< pub enum HashJoinHashTable { Null, + NestedLoop(Vec), Serializer(SerializerHashJoinHashTable), SkipDuplicatesSerializer(SkipDuplicatesSerializerHashJoinHashTable), SingleBinary(SingleBinaryHashJoinHashTable), @@ -348,9 +349,9 @@ impl HashJoinState { } impl HashJoinHashTable { - pub fn len(&self) -> usize { - match self { - HashJoinHashTable::Null => 0, + pub fn size(&self) -> Option { + let n = match self { + HashJoinHashTable::Null | HashJoinHashTable::NestedLoop(_) => return None, HashJoinHashTable::Serializer(table) => table.hash_table.len(), HashJoinHashTable::SingleBinary(table) => table.hash_table.len(), HashJoinHashTable::KeysU8(table) => table.hash_table.len(), @@ -367,11 +368,7 @@ impl HashJoinHashTable { HashJoinHashTable::SkipDuplicatesKeysU64(table) => table.hash_table.len(), HashJoinHashTable::SkipDuplicatesKeysU128(table) => table.hash_table.len(), HashJoinHashTable::SkipDuplicatesKeysU256(table) => table.hash_table.len(), - } - } - - #[allow(dead_code)] - pub fn is_empty(&self) -> bool { - self.len() == 0 + }; + Some(n) } } diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/result_blocks.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/result_blocks.rs index 114f019669287..f7950e3948366 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/result_blocks.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/result_blocks.rs @@ -54,7 +54,9 @@ impl HashJoinProbeState { JoinType::InnerAny | JoinType::RightAny ) { let hash_table = unsafe { &*self.hash_join_state.hash_table.get() }; - probe_state.used_once = Some(MutableBitmap::from_len_zeroed(hash_table.len())) + probe_state.used_once = Some(MutableBitmap::from_len_zeroed( + hash_table.size().unwrap_or_default(), + )) } let no_other_predicate = self .hash_join_state diff --git a/src/query/service/src/pipelines/processors/transforms/new_hash_join/join.rs b/src/query/service/src/pipelines/processors/transforms/new_hash_join/join.rs index d1a44c0992b28..a4006838900f7 100644 --- a/src/query/service/src/pipelines/processors/transforms/new_hash_join/join.rs +++ b/src/query/service/src/pipelines/processors/transforms/new_hash_join/join.rs @@ -24,16 +24,22 @@ pub trait JoinStream: Send + Sync { } pub trait Join: Send + Sync + 'static { + /// Push one block into the build side. `None` signals the end of input. fn add_block(&mut self, data: Option) -> Result<()>; + /// Finalize build phase in chunks; each call processes the next pending build batch and + /// returns its progress. Once all batches are consumed it returns `None` to signal completion. fn final_build(&mut self) -> Result>; + /// Generate runtime filter packet for the given filter description. fn build_runtime_filter(&self, _: &RuntimeFiltersDesc) -> Result { Ok(JoinRuntimeFilterPacket::default()) } + /// Probe with a single block and return a streaming iterator over results. fn probe_block(&mut self, data: DataBlock) -> Result>; + /// Final steps after probing all blocks; used when more output is pending. fn final_probe(&mut self) -> Result>> { Ok(None) } diff --git a/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/basic.rs b/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/basic.rs index ffa47f6d079a9..8b6ab97247953 100644 --- a/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/basic.rs +++ b/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/basic.rs @@ -17,7 +17,6 @@ use std::sync::Arc; use std::sync::PoisonError; use databend_common_base::base::ProgressValues; -use databend_common_catalog::table_context::TableContext; use databend_common_exception::Result; use databend_common_expression::Column; use databend_common_expression::DataBlock; @@ -27,6 +26,7 @@ use databend_common_expression::HashMethodSerializer; use databend_common_expression::HashMethodSingleBinary; use databend_common_hashtable::BinaryHashJoinHashMap; use databend_common_hashtable::HashJoinHashMap; +use databend_common_settings::Settings; use databend_common_sql::plans::JoinType; use ethnum::U256; @@ -40,7 +40,6 @@ use crate::pipelines::processors::transforms::SkipDuplicatesFixedKeyHashJoinHash use crate::pipelines::processors::transforms::SkipDuplicatesSerializerHashJoinHashTable; use crate::pipelines::processors::transforms::SkipDuplicatesSingleBinaryHashJoinHashTable; use crate::pipelines::processors::HashJoinDesc; -use crate::sessions::QueryContext; pub struct BasicHashJoin { pub(crate) desc: Arc, @@ -49,17 +48,18 @@ pub struct BasicHashJoin { pub(crate) method: HashMethodKind, pub(crate) function_ctx: FunctionContext, pub(crate) state: Arc, + nested_loop_join_threshold: usize, } impl BasicHashJoin { pub fn create( - ctx: &QueryContext, + settings: &Settings, function_ctx: FunctionContext, method: HashMethodKind, desc: Arc, state: Arc, + nested_loop_join_threshold: usize, ) -> Result { - let settings = ctx.get_settings(); let squash_block = SquashBlocks::new( settings.get_max_block_size()? as _, settings.get_max_block_bytes()? as _, @@ -71,6 +71,7 @@ impl BasicHashJoin { method, function_ctx, squash_block, + nested_loop_join_threshold, }) } @@ -81,22 +82,18 @@ impl BasicHashJoin { }; if let Some(squashed_block) = squashed_block.take() { - let locked = self.state.mutex.lock(); - let _locked = locked.unwrap_or_else(PoisonError::into_inner); - - *self.state.build_rows.as_mut() += squashed_block.num_rows(); - let chunk_index = self.state.chunks.len(); - self.state.chunks.as_mut().push(squashed_block); - self.state.build_queue.as_mut().push_back(chunk_index); + self.state.push_chunk(squashed_block); } Ok(()) } pub(crate) fn final_build(&mut self) -> Result> { - self.init_memory_hash_table(); + if let Some(true) = self.init_memory_hash_table() { + return Ok(Some(self.build_nested_loop())); + }; - let Some(chunk_index) = self.steal_chunk_index() else { + let Some(chunk_index) = self.state.steal_chunk_index() else { return Ok(None); }; @@ -131,7 +128,6 @@ impl BasicHashJoin { std::mem::swap(&mut chunks[chunk_index], &mut chunk_block); } - log::info!("build_hash_table chunk_index{chunk_index}"); self.build_hash_table(keys_block, chunk_index)?; Ok(Some(ProgressValues { @@ -139,20 +135,19 @@ impl BasicHashJoin { bytes: num_bytes, })) } -} - -impl BasicHashJoin { - fn steal_chunk_index(&self) -> Option { - let locked = self.state.mutex.lock(); - let _locked = locked.unwrap_or_else(PoisonError::into_inner); - self.state.build_queue.as_mut().pop_front() - } pub(crate) fn finalize_chunks(&mut self) { if self.desc.build_projection.is_empty() || !self.state.columns.is_empty() { return; } + if matches!( + self.state.hash_table.deref(), + HashJoinHashTable::NestedLoop(_) + ) { + return; + } + let locked = self.state.mutex.lock(); let _locked = locked.unwrap_or_else(PoisonError::into_inner); @@ -179,135 +174,139 @@ impl BasicHashJoin { columns.push(Column::take_downcast_column_vec(&full_columns)); } - std::mem::swap(&mut columns, self.state.columns.as_mut()); + *self.state.columns.as_mut() = columns; } - fn init_memory_hash_table(&mut self) { + fn init_memory_hash_table(&mut self) -> Option { if !matches!(self.state.hash_table.deref(), HashJoinHashTable::Null) { - return; + return None; } let skip_duplicates = matches!(self.desc.join_type, JoinType::InnerAny | JoinType::LeftAny); let locked = self.state.mutex.lock(); let _locked = locked.unwrap_or_else(PoisonError::into_inner); - if matches!(self.state.hash_table.deref(), HashJoinHashTable::Null) { - let build_num_rows = *self.state.build_rows.deref(); - *self.state.hash_table.as_mut() = match (self.method.clone(), skip_duplicates) { - (HashMethodKind::Serializer(_), false) => { - HashJoinHashTable::Serializer(SerializerHashJoinHashTable::new( - BinaryHashJoinHashMap::with_build_row_num(build_num_rows), - HashMethodSerializer::default(), - )) - } - (HashMethodKind::Serializer(_), true) => { - HashJoinHashTable::SkipDuplicatesSerializer( - SkipDuplicatesSerializerHashJoinHashTable::new( - BinaryHashJoinHashMap::with_build_row_num(build_num_rows), - HashMethodSerializer::default(), - ), - ) - } - (HashMethodKind::SingleBinary(_), false) => { - HashJoinHashTable::SingleBinary(SingleBinaryHashJoinHashTable::new( + if !matches!(self.state.hash_table.deref(), HashJoinHashTable::Null) { + return None; + } + + let build_num_rows = *self.state.build_rows.deref(); + if build_num_rows < self.nested_loop_join_threshold { + *self.state.hash_table.as_mut() = HashJoinHashTable::NestedLoop(vec![]); + return Some(true); + } + + *self.state.hash_table.as_mut() = match (self.method.clone(), skip_duplicates) { + (HashMethodKind::Serializer(_), false) => { + HashJoinHashTable::Serializer(SerializerHashJoinHashTable::new( + BinaryHashJoinHashMap::with_build_row_num(build_num_rows), + HashMethodSerializer::default(), + )) + } + (HashMethodKind::Serializer(_), true) => HashJoinHashTable::SkipDuplicatesSerializer( + SkipDuplicatesSerializerHashJoinHashTable::new( + BinaryHashJoinHashMap::with_build_row_num(build_num_rows), + HashMethodSerializer::default(), + ), + ), + (HashMethodKind::SingleBinary(_), false) => { + HashJoinHashTable::SingleBinary(SingleBinaryHashJoinHashTable::new( + BinaryHashJoinHashMap::with_build_row_num(build_num_rows), + HashMethodSingleBinary::default(), + )) + } + (HashMethodKind::SingleBinary(_), true) => { + HashJoinHashTable::SkipDuplicatesSingleBinary( + SkipDuplicatesSingleBinaryHashJoinHashTable::new( BinaryHashJoinHashMap::with_build_row_num(build_num_rows), HashMethodSingleBinary::default(), - )) - } - (HashMethodKind::SingleBinary(_), true) => { - HashJoinHashTable::SkipDuplicatesSingleBinary( - SkipDuplicatesSingleBinaryHashJoinHashTable::new( - BinaryHashJoinHashMap::with_build_row_num(build_num_rows), - HashMethodSingleBinary::default(), - ), - ) - } - (HashMethodKind::KeysU8(hash_method), false) => { - HashJoinHashTable::KeysU8(FixedKeyHashJoinHashTable::new( - HashJoinHashMap::::with_build_row_num(build_num_rows), - hash_method, - )) - } - (HashMethodKind::KeysU8(hash_method), true) => { - HashJoinHashTable::SkipDuplicatesKeysU8( - SkipDuplicatesFixedKeyHashJoinHashTable::new( - HashJoinHashMap::::with_build_row_num(build_num_rows), - hash_method, - ), - ) - } - (HashMethodKind::KeysU16(hash_method), false) => { - HashJoinHashTable::KeysU16(FixedKeyHashJoinHashTable::new( - HashJoinHashMap::::with_build_row_num(build_num_rows), + ), + ) + } + (HashMethodKind::KeysU8(hash_method), false) => { + HashJoinHashTable::KeysU8(FixedKeyHashJoinHashTable::new( + HashJoinHashMap::::with_build_row_num(build_num_rows), + hash_method, + )) + } + (HashMethodKind::KeysU8(hash_method), true) => HashJoinHashTable::SkipDuplicatesKeysU8( + SkipDuplicatesFixedKeyHashJoinHashTable::new( + HashJoinHashMap::::with_build_row_num(build_num_rows), + hash_method, + ), + ), + (HashMethodKind::KeysU16(hash_method), false) => { + HashJoinHashTable::KeysU16(FixedKeyHashJoinHashTable::new( + HashJoinHashMap::::with_build_row_num(build_num_rows), + hash_method, + )) + } + (HashMethodKind::KeysU16(hash_method), true) => { + HashJoinHashTable::SkipDuplicatesKeysU16( + SkipDuplicatesFixedKeyHashJoinHashTable::new( + HashJoinHashMap::::with_build_row_num(build_num_rows), hash_method, - )) - } - (HashMethodKind::KeysU16(hash_method), true) => { - HashJoinHashTable::SkipDuplicatesKeysU16( - SkipDuplicatesFixedKeyHashJoinHashTable::new( - HashJoinHashMap::::with_build_row_num(build_num_rows), - hash_method, - ), - ) - } - (HashMethodKind::KeysU32(hash_method), false) => { - HashJoinHashTable::KeysU32(FixedKeyHashJoinHashTable::new( - HashJoinHashMap::::with_build_row_num(build_num_rows), + ), + ) + } + (HashMethodKind::KeysU32(hash_method), false) => { + HashJoinHashTable::KeysU32(FixedKeyHashJoinHashTable::new( + HashJoinHashMap::::with_build_row_num(build_num_rows), + hash_method, + )) + } + (HashMethodKind::KeysU32(hash_method), true) => { + HashJoinHashTable::SkipDuplicatesKeysU32( + SkipDuplicatesFixedKeyHashJoinHashTable::new( + HashJoinHashMap::::with_build_row_num(build_num_rows), hash_method, - )) - } - (HashMethodKind::KeysU32(hash_method), true) => { - HashJoinHashTable::SkipDuplicatesKeysU32( - SkipDuplicatesFixedKeyHashJoinHashTable::new( - HashJoinHashMap::::with_build_row_num(build_num_rows), - hash_method, - ), - ) - } - (HashMethodKind::KeysU64(hash_method), false) => { - HashJoinHashTable::KeysU64(FixedKeyHashJoinHashTable::new( - HashJoinHashMap::::with_build_row_num(build_num_rows), + ), + ) + } + (HashMethodKind::KeysU64(hash_method), false) => { + HashJoinHashTable::KeysU64(FixedKeyHashJoinHashTable::new( + HashJoinHashMap::::with_build_row_num(build_num_rows), + hash_method, + )) + } + (HashMethodKind::KeysU64(hash_method), true) => { + HashJoinHashTable::SkipDuplicatesKeysU64( + SkipDuplicatesFixedKeyHashJoinHashTable::new( + HashJoinHashMap::::with_build_row_num(build_num_rows), hash_method, - )) - } - (HashMethodKind::KeysU64(hash_method), true) => { - HashJoinHashTable::SkipDuplicatesKeysU64( - SkipDuplicatesFixedKeyHashJoinHashTable::new( - HashJoinHashMap::::with_build_row_num(build_num_rows), - hash_method, - ), - ) - } - (HashMethodKind::KeysU128(hash_method), false) => { - HashJoinHashTable::KeysU128(FixedKeyHashJoinHashTable::new( - HashJoinHashMap::::with_build_row_num(build_num_rows), + ), + ) + } + (HashMethodKind::KeysU128(hash_method), false) => { + HashJoinHashTable::KeysU128(FixedKeyHashJoinHashTable::new( + HashJoinHashMap::::with_build_row_num(build_num_rows), + hash_method, + )) + } + (HashMethodKind::KeysU128(hash_method), true) => { + HashJoinHashTable::SkipDuplicatesKeysU128( + SkipDuplicatesFixedKeyHashJoinHashTable::new( + HashJoinHashMap::::with_build_row_num(build_num_rows), hash_method, - )) - } - (HashMethodKind::KeysU128(hash_method), true) => { - HashJoinHashTable::SkipDuplicatesKeysU128( - SkipDuplicatesFixedKeyHashJoinHashTable::new( - HashJoinHashMap::::with_build_row_num(build_num_rows), - hash_method, - ), - ) - } - (HashMethodKind::KeysU256(hash_method), false) => { - HashJoinHashTable::KeysU256(FixedKeyHashJoinHashTable::new( - HashJoinHashMap::::with_build_row_num(build_num_rows), + ), + ) + } + (HashMethodKind::KeysU256(hash_method), false) => { + HashJoinHashTable::KeysU256(FixedKeyHashJoinHashTable::new( + HashJoinHashMap::::with_build_row_num(build_num_rows), + hash_method, + )) + } + (HashMethodKind::KeysU256(hash_method), true) => { + HashJoinHashTable::SkipDuplicatesKeysU256( + SkipDuplicatesFixedKeyHashJoinHashTable::new( + HashJoinHashMap::::with_build_row_num(build_num_rows), hash_method, - )) - } - (HashMethodKind::KeysU256(hash_method), true) => { - HashJoinHashTable::SkipDuplicatesKeysU256( - SkipDuplicatesFixedKeyHashJoinHashTable::new( - HashJoinHashMap::::with_build_row_num(build_num_rows), - hash_method, - ), - ) - } - }; - } + ), + ) + } + }; + Some(false) } fn build_hash_table(&self, keys: DataBlock, chunk_idx: usize) -> Result<()> { @@ -315,6 +314,7 @@ impl BasicHashJoin { match self.state.hash_table.deref() { HashJoinHashTable::Null => (), + HashJoinHashTable::NestedLoop(_) => unreachable!(), HashJoinHashTable::Serializer(v) => v.insert(keys, chunk_idx, &mut arena)?, HashJoinHashTable::SingleBinary(v) => v.insert(keys, chunk_idx, &mut arena)?, HashJoinHashTable::KeysU8(v) => v.insert(keys, chunk_idx, &mut arena)?, @@ -349,4 +349,28 @@ impl BasicHashJoin { Ok(()) } + + fn build_nested_loop(&self) -> ProgressValues { + let mut progress = ProgressValues::default(); + let mut plain = vec![]; + while let Some(chunk_index) = self.state.steal_chunk_index() { + let chunk_mut = &mut self.state.chunks.as_mut()[chunk_index]; + + let mut chunk_block = DataBlock::empty(); + std::mem::swap(chunk_mut, &mut chunk_block); + + progress.rows += chunk_block.num_rows(); + progress.bytes += chunk_block.memory_size(); + + *chunk_mut = chunk_block.clone(); + + plain.push(chunk_block); + } + debug_assert!(matches!( + *self.state.hash_table, + HashJoinHashTable::NestedLoop(_) + )); + *self.state.hash_table.as_mut() = HashJoinHashTable::NestedLoop(plain); + progress + } } diff --git a/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/basic_state.rs b/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/basic_state.rs index 897ed42c27eee..40fc675bc7834 100644 --- a/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/basic_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/basic_state.rs @@ -15,6 +15,7 @@ use std::collections::VecDeque; use std::sync::Arc; use std::sync::Mutex; +use std::sync::PoisonError; use databend_common_expression::types::DataType; use databend_common_expression::ColumnVec; @@ -54,6 +55,22 @@ impl BasicHashJoinState { hash_table: CStyleCell::new(HashJoinHashTable::Null), } } + + pub(super) fn push_chunk(&self, chunk: DataBlock) { + let locked = self.mutex.lock(); + let _locked = locked.unwrap_or_else(PoisonError::into_inner); + + *self.build_rows.as_mut() += chunk.num_rows(); + let chunk_index = self.chunks.len(); + self.chunks.as_mut().push(chunk); + self.build_queue.as_mut().push_back(chunk_index); + } + + pub(super) fn steal_chunk_index(&self) -> Option { + let locked = self.mutex.lock(); + let _locked = locked.unwrap_or_else(PoisonError::into_inner); + self.build_queue.as_mut().pop_front() + } } impl Drop for BasicHashJoinState { diff --git a/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/inner_join.rs b/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/inner_join.rs index 3b1dab84b39a7..279a064937fc2 100644 --- a/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/inner_join.rs +++ b/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/inner_join.rs @@ -28,6 +28,9 @@ use databend_common_expression::FilterExecutor; use databend_common_expression::FunctionContext; use databend_common_expression::HashMethodKind; +use super::basic::BasicHashJoin; +use super::basic_state::BasicHashJoinState; +use super::LoopJoinStream; use crate::pipelines::processors::transforms::build_runtime_filter_packet; use crate::pipelines::processors::transforms::new_hash_join::hashtable::basic::ProbeStream; use crate::pipelines::processors::transforms::new_hash_join::hashtable::basic::ProbedRows; @@ -35,8 +38,6 @@ use crate::pipelines::processors::transforms::new_hash_join::hashtable::ProbeDat use crate::pipelines::processors::transforms::new_hash_join::join::EmptyJoinStream; use crate::pipelines::processors::transforms::new_hash_join::join::Join; use crate::pipelines::processors::transforms::new_hash_join::join::JoinStream; -use crate::pipelines::processors::transforms::new_hash_join::memory::basic::BasicHashJoin; -use crate::pipelines::processors::transforms::new_hash_join::memory::basic_state::BasicHashJoinState; use crate::pipelines::processors::transforms::new_hash_join::performance::PerformanceContext; use crate::pipelines::processors::transforms::HashJoinHashTable; use crate::pipelines::processors::transforms::JoinRuntimeFilterPacket; @@ -51,6 +52,7 @@ pub struct InnerHashJoin { pub(crate) function_ctx: FunctionContext, pub(crate) basic_state: Arc, pub(crate) performance_context: PerformanceContext, + nested_loop_filter: FilterExecutor, } impl InnerHashJoin { @@ -67,19 +69,22 @@ impl InnerHashJoin { let context = PerformanceContext::create(block_size, desc.clone(), function_ctx.clone()); let basic_hash_join = BasicHashJoin::create( - ctx, + &settings, function_ctx.clone(), method, desc.clone(), state.clone(), + settings.get_nested_loop_join_threshold()? as _, )?; + let nested_loop_filter = desc.create_nested_loop_filter(&function_ctx, block_size)?; Ok(InnerHashJoin { desc, basic_hash_join, function_ctx, basic_state: state, performance_context: context, + nested_loop_filter, }) } } @@ -114,6 +119,22 @@ impl Join for InnerHashJoin { self.basic_hash_join.finalize_chunks(); + match &*self.basic_state.hash_table { + HashJoinHashTable::Null => { + return Err(ErrorCode::AbortedQuery( + "Aborted query, because the hash table is uninitialized.", + )) + } + HashJoinHashTable::NestedLoop(build_blocks) => { + let nested = Box::new(LoopJoinStream::new(data, build_blocks)); + return Ok(InnerHashJoinFilterStream::create( + nested, + &mut self.nested_loop_filter, + )); + } + _ => (), + } + let probe_keys = self.desc.probe_key(&data, &self.function_ctx)?; let mut keys = DataBlock::new(probe_keys, data.num_rows()); @@ -123,29 +144,26 @@ impl Join for InnerHashJoin { }; self.desc.remove_keys_nullable(&mut keys); - let probe_block = data.project(&self.desc.probe_projections); - - let joined_stream = - with_join_hash_method!(|T| match self.basic_state.hash_table.deref() { - HashJoinHashTable::T(table) => { - let probe_hash_statistics = &mut self.performance_context.probe_hash_statistics; - probe_hash_statistics.clear(probe_block.num_rows()); - - let probe_data = ProbeData::new(keys, valids, probe_hash_statistics); - let probe_keys_stream = table.probe_matched(probe_data)?; - - Ok(InnerHashJoinStream::create( - probe_block, - self.basic_state.clone(), - probe_keys_stream, - self.desc.clone(), - &mut self.performance_context.probe_result, - )) - } - HashJoinHashTable::Null => Err(ErrorCode::AbortedQuery( - "Aborted query, because the hash table is uninitialized.", - )), - })?; + let probe_block = data.project(&self.desc.probe_projection); + + let joined_stream = with_join_hash_method!(|T| match self.basic_state.hash_table.deref() { + HashJoinHashTable::T(table) => { + let probe_hash_statistics = &mut self.performance_context.probe_hash_statistics; + probe_hash_statistics.clear(probe_block.num_rows()); + + let probe_data = ProbeData::new(keys, valids, probe_hash_statistics); + let probe_keys_stream = table.probe_matched(probe_data)?; + + InnerHashJoinStream::create( + probe_block, + self.basic_state.clone(), + probe_keys_stream, + self.desc.clone(), + &mut self.performance_context.probe_result, + ) + } + HashJoinHashTable::Null | HashJoinHashTable::NestedLoop(_) => unreachable!(), + }); match &mut self.performance_context.filter_executor { None => Ok(joined_stream), @@ -232,31 +250,27 @@ impl<'a> JoinStream for InnerHashJoinStream<'a> { (None, None) => DataBlock::new(vec![], self.probed_rows.matched_build.len()), }; - if !self.desc.probe_to_build.is_empty() { - for (index, (is_probe_nullable, is_build_nullable)) in - self.desc.probe_to_build.iter() - { - let entry = match (is_probe_nullable, is_build_nullable) { - (true, true) | (false, false) => result_block.get_by_offset(*index).clone(), - (true, false) => { - result_block.get_by_offset(*index).clone().remove_nullable() + for (index, (is_probe_nullable, is_build_nullable)) in + self.desc.probe_to_build.iter().cloned() + { + let entry = match (is_probe_nullable, is_build_nullable) { + (true, true) | (false, false) => result_block.get_by_offset(index).clone(), + (true, false) => result_block.get_by_offset(index).clone().remove_nullable(), + (false, true) => { + let entry = result_block.get_by_offset(index); + let col = entry.to_column(); + + match col.is_null() || col.is_nullable() { + true => entry.clone(), + false => BlockEntry::from(NullableColumn::new_column( + col, + Bitmap::new_constant(true, result_block.num_rows()), + )), } - (false, true) => { - let entry = result_block.get_by_offset(*index); - let col = entry.to_column(); - - match col.is_null() || col.is_nullable() { - true => entry.clone(), - false => BlockEntry::from(NullableColumn::new_column( - col, - Bitmap::new_constant(true, result_block.num_rows()), - )), - } - } - }; + } + }; - result_block.add_entry(entry); - } + result_block.add_entry(entry); } return Ok(Some(result_block)); diff --git a/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/mod.rs b/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/mod.rs index 4979c37245fca..e8eb9fba13b79 100644 --- a/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/mod.rs @@ -15,7 +15,9 @@ mod basic; mod basic_state; mod inner_join; +mod nested_loop; pub mod outer_left_join; pub use basic_state::BasicHashJoinState; pub use inner_join::InnerHashJoin; +pub use nested_loop::*; diff --git a/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/nested_loop.rs b/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/nested_loop.rs new file mode 100644 index 0000000000000..f22158fa08e7c --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/nested_loop.rs @@ -0,0 +1,90 @@ +// 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 databend_common_exception::Result; +use databend_common_expression::types::DataType; +use databend_common_expression::BlockEntry; +use databend_common_expression::DataBlock; +use databend_common_expression::Scalar; + +use crate::pipelines::processors::transforms::JoinStream; + +pub struct LoopJoinStream<'a> { + probe_rows: VecDeque>, + probe_types: Vec, + build_blocks: &'a [DataBlock], + build_index: usize, +} + +impl<'a> LoopJoinStream<'a> { + pub fn new(probe: DataBlock, build_blocks: &'a [DataBlock]) -> Self { + let mut probe_rows = vec![Vec::new(); probe.num_rows()]; + for entry in probe.columns().iter() { + match entry { + BlockEntry::Const(scalar, _, _) => { + for row in &mut probe_rows { + row.push(scalar.to_owned()); + } + } + BlockEntry::Column(column) => { + for (row, scalar) in probe_rows.iter_mut().zip(column.iter()) { + row.push(scalar.to_owned()); + } + } + } + } + + let left_types = probe + .columns() + .iter() + .map(|entry| entry.data_type()) + .collect(); + + LoopJoinStream { + probe_rows: probe_rows.into(), + probe_types: left_types, + build_blocks, + build_index: 0, + } + } +} + +impl<'a> JoinStream for LoopJoinStream<'a> { + fn next(&mut self) -> Result> { + let Some(probe_entries) = self.probe_rows.front() else { + return Ok(None); + }; + + let build_block = &self.build_blocks[self.build_index]; + + let entries = probe_entries + .iter() + .zip(self.probe_types.iter()) + .map(|(scalar, data_type)| { + BlockEntry::Const(scalar.clone(), data_type.clone(), build_block.num_rows()) + }) + .chain(build_block.columns().iter().cloned()) + .collect(); + + self.build_index += 1; + if self.build_index >= self.build_blocks.len() { + self.build_index = 0; + self.probe_rows.pop_front(); + } + + Ok(Some(DataBlock::new(entries, build_block.num_rows()))) + } +} diff --git a/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/outer_left_join.rs b/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/outer_left_join.rs index 8210b495bfca3..e548178a06646 100644 --- a/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/outer_left_join.rs +++ b/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/outer_left_join.rs @@ -68,11 +68,12 @@ impl OuterLeftHashJoin { let context = PerformanceContext::create(block_size, desc.clone(), function_ctx.clone()); let basic_hash_join = BasicHashJoin::create( - ctx, + &settings, function_ctx.clone(), method, desc.clone(), state.clone(), + 0, )?; Ok(OuterLeftHashJoin { @@ -111,7 +112,7 @@ impl Join for OuterLeftHashJoin { .collect::>(); let build_block = null_build_block(&types, data.num_rows()); - let probe_block = Some(data.project(&self.desc.probe_projections)); + let probe_block = Some(data.project(&self.desc.probe_projection)); let result_block = final_result_block(&self.desc, probe_block, build_block, num_rows); return Ok(Box::new(OneBlockJoinStream(Some(result_block)))); } @@ -127,7 +128,7 @@ impl Join for OuterLeftHashJoin { }; self.desc.remove_keys_nullable(&mut keys); - let probe_block = data.project(&self.desc.probe_projections); + let probe_block = data.project(&self.desc.probe_projection); let probe_stream = with_join_hash_method!(|T| match self.basic_state.hash_table.deref() { HashJoinHashTable::T(table) => { @@ -137,6 +138,9 @@ impl Join for OuterLeftHashJoin { let probe_data = ProbeData::new(keys, valids, probe_hash_statistics); table.probe(probe_data) } + HashJoinHashTable::NestedLoop(_) => { + todo!() + } HashJoinHashTable::Null => Err(ErrorCode::AbortedQuery( "Aborted query, because the hash table is uninitialized.", )), diff --git a/src/query/service/src/pipelines/processors/transforms/transform_loop_join.rs b/src/query/service/src/pipelines/processors/transforms/transform_loop_join.rs index 8f1b755342e3d..d841be2fadb11 100644 --- a/src/query/service/src/pipelines/processors/transforms/transform_loop_join.rs +++ b/src/query/service/src/pipelines/processors/transforms/transform_loop_join.rs @@ -13,54 +13,63 @@ // limitations under the License. use std::any::Any; -use std::collections::VecDeque; +use std::sync::atomic::AtomicBool; +use std::sync::atomic::Ordering; use std::sync::Arc; use std::sync::Mutex; use std::sync::RwLock; use databend_common_base::base::ProgressValues; use databend_common_exception::Result; +use databend_common_expression::types::DataType; use databend_common_expression::BlockEntry; use databend_common_expression::DataBlock; +use databend_common_expression::Scalar; use databend_common_pipeline::core::Event; use databend_common_pipeline::core::InputPort; use databend_common_pipeline::core::OutputPort; use databend_common_pipeline::core::Processor; -use databend_common_pipeline::sinks::Sink; use databend_common_sql::plans::JoinType; use super::Join; use super::JoinStream; use crate::physical_plans::NestedLoopJoin; -use crate::pipelines::executor::WatchNotify; use crate::sessions::QueryContext; -pub struct TransformLoopJoinLeft { - input_port: Arc, +pub struct TransformLoopJoin { + build_port: Arc, + probe_port: Arc, output_port: Arc, - output_data_blocks: VecDeque, - state: Arc, + joined_data: Option, + join: Box, + stage: Stage, } -impl TransformLoopJoinLeft { +impl TransformLoopJoin { pub fn create( - input_port: Arc, + build_port: Arc, + probe_port: Arc, output_port: Arc, - state: Arc, + join: Box, ) -> Box { - Box::new(TransformLoopJoinLeft { - input_port, + Box::new(TransformLoopJoin { + build_port, + probe_port, output_port, - output_data_blocks: Default::default(), - state, + join, + joined_data: None, + stage: Stage::Build(BuildState { + finished: false, + build_data: None, + }), }) } } #[async_trait::async_trait] -impl Processor for TransformLoopJoinLeft { +impl Processor for TransformLoopJoin { fn name(&self) -> String { - "TransformLoopJoinLeft".to_string() + String::from("TransformLoopJoinLeft") } fn as_any(&mut self) -> &mut dyn Any { @@ -69,98 +78,194 @@ impl Processor for TransformLoopJoinLeft { fn event(&mut self) -> Result { if self.output_port.is_finished() { + self.build_port.finish(); + self.probe_port.finish(); return Ok(Event::Finished); } + if !self.output_port.can_push() { - return Ok(Event::NeedConsume); - } - if let Some(data) = self.output_data_blocks.pop_front() { - self.output_port.push_data(Ok(data)); + match self.stage { + Stage::Build(_) => self.build_port.set_not_need_data(), + Stage::Probe(_) => self.probe_port.set_not_need_data(), + Stage::BuildFinal(_) | Stage::Finished => (), + } return Ok(Event::NeedConsume); } - if !self.state.is_right_finish()? { - return Ok(Event::Async); + if let Some(joined_data) = self.joined_data.take() { + self.output_port.push_data(Ok(joined_data)); + return Ok(Event::NeedConsume); } - if self.input_port.is_finished() { - // todo!() - self.output_port.finish(); - return Ok(Event::Finished); - } - if !self.input_port.has_data() { - self.input_port.set_need_data(); - return Ok(Event::NeedData); + match &mut self.stage { + Stage::Build(state) => { + self.probe_port.set_not_need_data(); + state.event(&self.build_port) + } + Stage::BuildFinal(state) => state.event(), + Stage::Probe(state) => state.event(&self.probe_port), + Stage::Finished => Ok(Event::Finished), } - Ok(Event::Sync) } fn process(&mut self) -> Result<()> { - if let Some(data) = self.input_port.pull_data() { - self.output_data_blocks = self.state.loop_join(data?)?; + match &mut self.stage { + Stage::Finished => Ok(()), + Stage::Build(state) => { + let Some(data_block) = state.build_data.take() else { + if !state.finished { + state.finished = true; + } + self.stage = Stage::BuildFinal(BuildFinalState::new()); + return Ok(()); + }; + + if !data_block.is_empty() { + self.join.add_block(Some(data_block))?; + } + + Ok(()) + } + Stage::BuildFinal(state) => { + if self.join.final_build()?.is_none() { + state.finished = true; + self.stage = Stage::Probe(ProbeState::new()); + } + Ok(()) + } + Stage::Probe(state) => { + if let Some(probe_data) = state.input_data.take() { + let _stream = self.join.probe_block(probe_data)?; + todo!(); + // join and stream share the same lifetime within processor + // #[allow(clippy::missing_transmute_annotations)] + // state.stream = Some(unsafe { std::mem::transmute(stream) }); + } + + if let Some(mut stream) = state.stream.take() { + if let Some(joined_data) = stream.next()? { + self.joined_data = Some(joined_data); + state.stream = Some(stream); + } else if self.probe_port.is_finished() { + self.output_port.finish(); + self.stage = Stage::Finished; + } + } else if self.probe_port.is_finished() { + self.output_port.finish(); + self.stage = Stage::Finished; + } + + Ok(()) + } } - Ok(()) } +} - #[async_backtrace::framed] - async fn async_process(&mut self) -> Result<()> { - self.state.wait_right_finish().await?; - Ok(()) - } +enum Stage { + Build(BuildState), + BuildFinal(BuildFinalState), + Probe(ProbeState), + Finished, } -pub struct TransformLoopJoinRight { - state: Arc, +#[derive(Debug)] +struct BuildState { + finished: bool, + build_data: Option, } -impl TransformLoopJoinRight { - pub fn create(state: Arc) -> Result { - state.attach_right()?; - Ok(TransformLoopJoinRight { state }) +impl BuildState { + pub fn event(&mut self, input: &InputPort) -> Result { + if self.build_data.is_some() { + return Ok(Event::Sync); + } + + if input.has_data() { + self.build_data = Some(input.pull_data().unwrap()?); + return Ok(Event::Sync); + } + + if input.is_finished() { + return match self.finished { + true => Ok(Event::Finished), + false => Ok(Event::Sync), + }; + } + + input.set_need_data(); + Ok(Event::NeedData) } } -impl Sink for TransformLoopJoinRight { - const NAME: &'static str = "TransformLoopJoinRight"; +#[derive(Debug)] +struct BuildFinalState { + finished: bool, +} - fn consume(&mut self, data_block: DataBlock) -> Result<()> { - self.state.sink_right(data_block) +impl BuildFinalState { + pub fn new() -> BuildFinalState { + BuildFinalState { finished: false } } - fn on_finish(&mut self) -> Result<()> { - self.state.detach_right()?; - Ok(()) + pub fn event(&mut self) -> Result { + match self.finished { + true => Ok(Event::Async), + false => Ok(Event::Sync), + } } } -pub struct LoopJoinState { - right_table: RwLock>, - right_finished: Mutex, - finished_notify: Arc, +struct ProbeState { + input_data: Option, + stream: Option>, +} - right_sinker_count: RwLock, +impl ProbeState { + pub fn new() -> ProbeState { + ProbeState { + input_data: None, + stream: None, + } + } + pub fn event(&mut self, input: &InputPort) -> Result { + if self.input_data.is_some() || self.stream.is_some() { + return Ok(Event::Sync); + } + + if input.has_data() { + self.input_data = Some(input.pull_data().unwrap()?); + return Ok(Event::Sync); + } + + if input.is_finished() { + return Ok(Event::Sync); + } + + input.set_need_data(); + Ok(Event::NeedData) + } +} + +#[derive(Clone)] +pub struct LoopJoinState { + build_table: Arc>>, + build_progress: Arc>>, + build_finished: Arc, join_type: JoinType, } impl LoopJoinState { pub fn new(_ctx: Arc, join: &NestedLoopJoin) -> Self { Self { - right_table: RwLock::new(vec![]), - right_finished: Mutex::new(false), - finished_notify: Arc::new(WatchNotify::new()), - right_sinker_count: RwLock::new(0), + build_table: Arc::new(RwLock::new(vec![])), + build_progress: Arc::new(Mutex::new(None)), + build_finished: Arc::new(AtomicBool::new(false)), join_type: join.join_type, } } - fn attach_right(&self) -> Result<()> { - let mut right_sinker_count = self.right_sinker_count.write()?; - *right_sinker_count += 1; - Ok(()) - } - - fn sink_right(&self, right_block: DataBlock) -> Result<()> { + fn push_build_block(&self, right_block: DataBlock) -> Result<()> { let right = if matches!(self.join_type, JoinType::Left | JoinType::Full) { let rows = right_block.num_rows(); let entries = right_block @@ -172,134 +277,123 @@ impl LoopJoinState { } else { right_block }; - self.right_table.write()?.push(right); + self.build_table.write()?.push(right); Ok(()) } +} - fn detach_right(&self) -> Result<()> { - let finished = { - let mut right_sinker_count = self.right_sinker_count.write()?; - *right_sinker_count -= 1; - *right_sinker_count == 0 +impl Join for LoopJoinState { + fn add_block(&mut self, data: Option) -> Result<()> { + let Some(right_block) = data else { + return Ok(()); }; - if finished { - // todo - *self.right_finished.lock()? = true; - self.finished_notify.notify_waiters(); - } - Ok(()) - } - fn is_right_finish(&self) -> Result { - Ok(*self.right_finished.lock()?) + self.push_build_block(right_block) } - async fn wait_right_finish(&self) -> Result<()> { - if !*self.right_finished.lock()? { - self.finished_notify.notified().await + fn final_build(&mut self) -> Result> { + if self.build_finished.swap(true, Ordering::SeqCst) { + return Ok(None); } - Ok(()) + + let progress = self.build_table.read()?.iter().fold( + ProgressValues::default(), + |mut progress, block| { + progress.rows += block.num_rows(); + progress.bytes += block.memory_size(); + progress + }, + ); + + let mut guard = self.build_progress.lock()?; + *guard = Some(progress.clone()); + Ok(Some(progress)) } - fn loop_join(&self, left_block: DataBlock) -> Result> { + fn probe_block(&mut self, data: DataBlock) -> Result> { let left = if matches!(self.join_type, JoinType::Right | JoinType::Full) { - let rows = left_block.num_rows(); - let entries = left_block + let rows = data.num_rows(); + let entries = data .take_columns() .into_iter() .map(|entry| entry.into_nullable()) .collect::>(); DataBlock::new(entries, rows) } else { - left_block + data }; - let right_table = self.right_table.read()?; - let mut blocks = VecDeque::with_capacity(right_table.len() * left.num_rows()); - for right in right_table.iter() { - blocks.extend(self.single_loop_join(&left, right)?); - } - Ok(blocks) + let right_table = self.build_table.read()?.clone(); + Ok(Box::new(LoopJoinStream::new(left, right_table))) } +} - fn single_loop_join<'a>( - &self, - left: &'a DataBlock, - right: &'a DataBlock, - ) -> Result + use<'a>> { - let mut left_blocks = vec![Vec::new(); left.num_rows()]; +struct LoopJoinStream { + left_rows: Vec>, + left_types: Vec, + right_blocks: Vec, + left_row: usize, + right_index: usize, +} + +impl LoopJoinStream { + fn new(left: DataBlock, right_blocks: Vec) -> Self { + let mut left_rows = vec![Vec::new(); left.num_rows()]; for entry in left.columns().iter() { match entry { BlockEntry::Const(scalar, _, _) => { - for left_entries in &mut left_blocks { - left_entries.push(scalar.as_ref()); + for row in &mut left_rows { + row.push(scalar.to_owned()); } } BlockEntry::Column(column) => { - for (left_entries, scalar) in left_blocks.iter_mut().zip(column.iter()) { - left_entries.push(scalar); + for (row, scalar) in left_rows.iter_mut().zip(column.iter()) { + row.push(scalar.to_owned()); } } } } - let iter = left_blocks.into_iter().map(|left_entries| { - let entries = left_entries - .iter() - .zip(left.columns().iter().map(|entry| entry.data_type())) - .map(|(scalar, data_type)| { - BlockEntry::Const(scalar.to_owned(), data_type, right.num_rows()) - }) - .chain(right.columns().iter().cloned()) - .collect(); - DataBlock::new(entries, right.num_rows()) - }); - Ok(iter) - } -} - -impl Join for LoopJoinState { - fn add_block(&mut self, data: Option) -> Result<()> { - let Some(right_block) = data else { - return Ok(()); - }; - - let right = if matches!(self.join_type, JoinType::Left | JoinType::Full) { - let rows = right_block.num_rows(); - let entries = right_block - .take_columns() - .into_iter() - .map(|entry| entry.into_nullable()) - .collect::>(); - DataBlock::new(entries, rows) - } else { - right_block - }; - self.right_table.write()?.push(right); - Ok(()) - } - - fn final_build(&mut self) -> Result> { - let progress = self.right_table.read()?.iter().fold( - ProgressValues::default(), - |mut progress, block| { - progress.rows += block.num_rows(); - progress.bytes += block.memory_size(); - progress - }, - ); - Ok(Some(progress)) - } - - fn probe_block(&mut self, data: DataBlock) -> Result> { - todo!(); + let left_types = left + .columns() + .iter() + .map(|entry| entry.data_type()) + .collect(); + + LoopJoinStream { + left_rows, + left_types, + right_blocks, + left_row: 0, + right_index: 0, + } } } -struct LoopJoinStream {} - impl JoinStream for LoopJoinStream { fn next(&mut self) -> Result> { - todo!() + if self.right_blocks.is_empty() || self.left_row >= self.left_rows.len() { + return Ok(None); + } + + let right_block = &self.right_blocks[self.right_index]; + let left_entries = &self.left_rows[self.left_row]; + + let entries = left_entries + .iter() + .zip(self.left_types.iter()) + .map(|(scalar, data_type)| { + BlockEntry::Const(scalar.clone(), data_type.clone(), right_block.num_rows()) + }) + .chain(right_block.columns().iter().cloned()) + .collect(); + + self.right_index += 1; + if self.right_index >= self.right_blocks.len() { + self.right_index = 0; + self.left_row += 1; + } + + Ok(Some(DataBlock::new(entries, right_block.num_rows()))) } } From 1ac9e5af925160c7385c97d775b3e6eed0f9f424 Mon Sep 17 00:00:00 2001 From: coldWater Date: Wed, 19 Nov 2025 18:27:26 +0800 Subject: [PATCH 05/19] clear --- .../format/format_nested_loop_join.rs | 125 ------ .../service/src/physical_plans/format/mod.rs | 2 - src/query/service/src/physical_plans/mod.rs | 2 - .../src/physical_plans/physical_hash_join.rs | 9 +- .../src/physical_plans/physical_join.rs | 63 +-- .../physical_nested_loop_join.rs | 225 ---------- .../pipelines/processors/transforms/mod.rs | 2 - .../transforms/transform_loop_join.rs | 399 ------------------ 8 files changed, 7 insertions(+), 820 deletions(-) delete mode 100644 src/query/service/src/physical_plans/format/format_nested_loop_join.rs delete mode 100644 src/query/service/src/physical_plans/physical_nested_loop_join.rs delete mode 100644 src/query/service/src/pipelines/processors/transforms/transform_loop_join.rs diff --git a/src/query/service/src/physical_plans/format/format_nested_loop_join.rs b/src/query/service/src/physical_plans/format/format_nested_loop_join.rs deleted file mode 100644 index 8259bc99a6b0d..0000000000000 --- a/src/query/service/src/physical_plans/format/format_nested_loop_join.rs +++ /dev/null @@ -1,125 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_ast::ast::FormatTreeNode; -use databend_common_exception::Result; -use databend_common_functions::BUILTIN_FUNCTIONS; - -use crate::physical_plans::format::append_output_rows_info; -use crate::physical_plans::format::format_output_columns; -use crate::physical_plans::format::plan_stats_info_to_format_tree; -use crate::physical_plans::format::FormatContext; -use crate::physical_plans::format::PhysicalFormat; -use crate::physical_plans::IPhysicalPlan; -use crate::physical_plans::NestedLoopJoin; -use crate::physical_plans::PhysicalPlanMeta; - -pub struct NestedLoopJoinFormatter<'a> { - inner: &'a NestedLoopJoin, -} - -impl<'a> NestedLoopJoinFormatter<'a> { - pub fn create(inner: &'a NestedLoopJoin) -> Box { - Box::new(NestedLoopJoinFormatter { inner }) - } -} - -impl<'a> PhysicalFormat for NestedLoopJoinFormatter<'a> { - fn get_meta(&self) -> &PhysicalPlanMeta { - self.inner.get_meta() - } - - #[recursive::recursive] - fn format(&self, ctx: &mut FormatContext<'_>) -> Result> { - let conditions = self - .inner - .conditions - .iter() - .map(|expr| expr.as_expr(&BUILTIN_FUNCTIONS).sql_display()) - .collect::>() - .join(", "); - - let mut node_children = vec![ - FormatTreeNode::new(format!( - "output columns: [{}]", - format_output_columns(self.inner.output_schema()?, ctx.metadata, true) - )), - FormatTreeNode::new(format!("join type: {}", self.inner.join_type)), - FormatTreeNode::new(format!("conditions: [{conditions}]")), - ]; - - if let Some(info) = &self.inner.stat_info { - let items = plan_stats_info_to_format_tree(info); - node_children.extend(items); - } - - let left_formatter = self.inner.left.formatter()?; - let mut left_child = left_formatter.dispatch(ctx)?; - left_child.payload = format!("{}(Left)", left_child.payload); - - let right_formatter = self.inner.right.formatter()?; - let mut right_child = right_formatter.dispatch(ctx)?; - right_child.payload = format!("{}(Right)", right_child.payload); - - node_children.push(left_child); - node_children.push(right_child); - - Ok(FormatTreeNode::with_children( - "NestedLoopJoin".to_string(), - node_children, - )) - } - - #[recursive::recursive] - fn format_join(&self, ctx: &mut FormatContext<'_>) -> Result> { - let left_child = self.inner.left.formatter()?.format_join(ctx)?; - let right_child = self.inner.right.formatter()?.format_join(ctx)?; - - let children = vec![ - FormatTreeNode::with_children("Left".to_string(), vec![left_child]), - FormatTreeNode::with_children("Right".to_string(), vec![right_child]), - ]; - - Ok(FormatTreeNode::with_children( - format!("NestedLoopJoin: {}", self.inner.join_type), - children, - )) - } - - #[recursive::recursive] - fn partial_format(&self, ctx: &mut FormatContext<'_>) -> Result> { - let left_child = self.inner.left.formatter()?.partial_format(ctx)?; - let right_child = self.inner.right.formatter()?.partial_format(ctx)?; - - let mut children = vec![]; - if let Some(info) = &self.inner.stat_info { - let items = plan_stats_info_to_format_tree(info); - children.extend(items); - } - - append_output_rows_info(&mut children, &ctx.profs, self.inner.get_id()); - - children.push(FormatTreeNode::with_children("Left".to_string(), vec![ - left_child, - ])); - children.push(FormatTreeNode::with_children("Right".to_string(), vec![ - right_child, - ])); - - Ok(FormatTreeNode::with_children( - format!("NestedLoopJoin: {}", self.inner.join_type), - children, - )) - } -} diff --git a/src/query/service/src/physical_plans/format/mod.rs b/src/query/service/src/physical_plans/format/mod.rs index 39b5b3138d07b..8166b60bc64d8 100644 --- a/src/query/service/src/physical_plans/format/mod.rs +++ b/src/query/service/src/physical_plans/format/mod.rs @@ -45,7 +45,6 @@ mod format_mutation_into_organize; mod format_mutation_into_split; mod format_mutation_manipulate; mod format_mutation_source; -mod format_nested_loop_join; mod format_project_set; mod format_range_join; mod format_replace_into; @@ -94,7 +93,6 @@ pub use format_mutation_into_organize::*; pub use format_mutation_into_split::*; pub use format_mutation_manipulate::*; pub use format_mutation_source::*; -pub use format_nested_loop_join::*; pub use format_project_set::*; pub use format_range_join::*; pub use format_replace_into::*; diff --git a/src/query/service/src/physical_plans/mod.rs b/src/query/service/src/physical_plans/mod.rs index 520834d696345..9a0d5f7e4a2e4 100644 --- a/src/query/service/src/physical_plans/mod.rs +++ b/src/query/service/src/physical_plans/mod.rs @@ -42,7 +42,6 @@ mod physical_mutation_into_organize; mod physical_mutation_into_split; mod physical_mutation_manipulate; mod physical_mutation_source; -mod physical_nested_loop_join; mod physical_project_set; mod physical_r_cte_scan; mod physical_range_join; @@ -91,7 +90,6 @@ pub use physical_mutation_into_organize::MutationOrganize; pub use physical_mutation_into_split::MutationSplit; pub use physical_mutation_manipulate::MutationManipulate; pub use physical_mutation_source::*; -pub use physical_nested_loop_join::NestedLoopJoin; pub use physical_project_set::ProjectSet; pub use physical_r_cte_scan::RecursiveCteScan; pub use physical_range_join::*; diff --git a/src/query/service/src/physical_plans/physical_hash_join.rs b/src/query/service/src/physical_plans/physical_hash_join.rs index 7d18c665e3879..3136ba7eff702 100644 --- a/src/query/service/src/physical_plans/physical_hash_join.rs +++ b/src/query/service/src/physical_plans/physical_hash_join.rs @@ -25,6 +25,7 @@ use databend_common_expression::types::DataType; use databend_common_expression::ConstantFolder; use databend_common_expression::DataBlock; use databend_common_expression::DataField; +use databend_common_expression::DataSchema; use databend_common_expression::DataSchemaRef; use databend_common_expression::DataSchemaRefExt; use databend_common_expression::RemoteExpr; @@ -1150,7 +1151,7 @@ impl PhysicalPlanBuilder { // Create projections and output schema let mut projections = ColumnSet::new(); - let projected_schema = DataSchemaRefExt::create(merged_fields.clone()); + let projected_schema = DataSchema::new(merged_fields.clone()); for column in column_projections.iter() { if let Some((index, _)) = projected_schema.column_with_name(&column.to_string()) { @@ -1198,7 +1199,7 @@ impl PhysicalPlanBuilder { fn build_nested_loop_filter_info( &self, join: &Join, - merged_schema: &DataSchemaRef, + merged_schema: &DataSchema, ) -> Result { let predicates = join .non_equi_conditions @@ -1286,8 +1287,6 @@ impl PhysicalPlanBuilder { // Step 10: Process non-equi conditions let non_equi_conditions = self.process_non_equi_conditions(join, &merged_schema)?; - let nested_loop_filter = self.build_nested_loop_filter_info(join, &merged_schema)?; - // Step 11: Build runtime filter let runtime_filter = build_runtime_filter( self.ctx.clone(), @@ -1300,6 +1299,8 @@ impl PhysicalPlanBuilder { ) .await?; + let nested_loop_filter = self.build_nested_loop_filter_info(join, &merged_schema)?; + // Step 12: Create and return the HashJoin let build_side_data_distribution = s_expr.build_side_child().get_data_distribution()?; let broadcast_id = if build_side_data_distribution diff --git a/src/query/service/src/physical_plans/physical_join.rs b/src/query/service/src/physical_plans/physical_join.rs index 662ff26b76a87..b8910faf02664 100644 --- a/src/query/service/src/physical_plans/physical_join.rs +++ b/src/query/service/src/physical_plans/physical_join.rs @@ -14,14 +14,11 @@ use databend_common_exception::ErrorCode; use databend_common_exception::Result; -use databend_common_expression::types::DataType; -use databend_common_settings::Settings; use databend_common_sql::binder::is_range_join_condition; use databend_common_sql::optimizer::ir::RelExpr; use databend_common_sql::optimizer::ir::SExpr; use databend_common_sql::plans::FunctionCall; use databend_common_sql::plans::Join; -use databend_common_sql::plans::JoinEquiCondition; use databend_common_sql::plans::JoinType; use databend_common_sql::ColumnSet; use databend_common_sql::ScalarExpr; @@ -37,13 +34,10 @@ enum PhysicalJoinType { range: Vec, other: Vec, }, - LoopJoin { - conditions: Vec, - }, } // Choose physical join type by join conditions -fn physical_join(join: &Join, s_expr: &SExpr, _settings: &Settings) -> Result { +fn physical_join(join: &Join, s_expr: &SExpr) -> Result { if join.equi_conditions.is_empty() && join.join_type.is_any_join() { return Err(ErrorCode::SemanticError( "ANY JOIN only supports equality-based hash joins", @@ -53,23 +47,6 @@ fn physical_join(join: &Join, s_expr: &SExpr, _settings: &Settings) -> Result>()?; - return Ok(PhysicalJoinType::LoopJoin { conditions }); - }; if !join.equi_conditions.is_empty() { // Contain equi condition, use hash join @@ -184,8 +161,7 @@ impl PhysicalPlanBuilder { ) .await } else { - let settings = self.ctx.get_settings(); - match physical_join(join, s_expr, &settings)? { + match physical_join(join, s_expr)? { PhysicalJoinType::Hash => { self.build_hash_join( join, @@ -209,42 +185,7 @@ impl PhysicalPlanBuilder { ) .await } - PhysicalJoinType::LoopJoin { conditions } => { - self.build_loop_join( - join.join_type, - s_expr, - left_required, - right_required, - conditions, - ) - .await - } } } } } - -fn condition_to_expr(condition: &JoinEquiCondition) -> Result { - let left_type = condition.left.data_type()?; - let right_type = condition.right.data_type()?; - - let arguments = match (&left_type, &right_type) { - (DataType::Nullable(left), right) if **left == *right => vec![ - condition.left.clone(), - condition.right.clone().unify_to_data_type(&left_type), - ], - (left, DataType::Nullable(right)) if *left == **right => vec![ - condition.left.clone().unify_to_data_type(&right_type), - condition.right.clone(), - ], - _ => vec![condition.left.clone(), condition.right.clone()], - }; - - Ok(FunctionCall { - span: condition.left.span(), - func_name: "eq".to_string(), - params: vec![], - arguments, - } - .into()) -} diff --git a/src/query/service/src/physical_plans/physical_nested_loop_join.rs b/src/query/service/src/physical_plans/physical_nested_loop_join.rs deleted file mode 100644 index b0fbbb6090049..0000000000000 --- a/src/query/service/src/physical_plans/physical_nested_loop_join.rs +++ /dev/null @@ -1,225 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::any::Any; - -use databend_common_exception::Result; -use databend_common_expression::DataSchemaRef; -use databend_common_expression::DataSchemaRefExt; -use databend_common_expression::RemoteExpr; -use databend_common_functions::BUILTIN_FUNCTIONS; -use databend_common_pipeline::core::InputPort; -use databend_common_pipeline::core::OutputPort; -use databend_common_pipeline::core::Pipe; -use databend_common_pipeline::core::PipeItem; -use databend_common_pipeline::core::ProcessorPtr; -use databend_common_sql::executor::cast_expr_to_non_null_boolean; -use databend_common_sql::optimizer::ir::SExpr; -use databend_common_sql::plans::JoinType; -use databend_common_sql::ColumnSet; -use databend_common_sql::ScalarExpr; -use itertools::Itertools; - -use super::explain::PlanStatsInfo; -use super::format::NestedLoopJoinFormatter; -use super::format::PhysicalFormat; -use super::resolve_scalar; -use super::IPhysicalPlan; -use super::PhysicalPlan; -use super::PhysicalPlanBuilder; -use super::PhysicalPlanMeta; -use crate::pipelines::processors::transforms::LoopJoinState; -use crate::pipelines::processors::transforms::TransformLoopJoin; -use crate::pipelines::PipelineBuilder; - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct NestedLoopJoin { - pub meta: PhysicalPlanMeta, - pub left: PhysicalPlan, - pub right: PhysicalPlan, - pub conditions: Vec, - pub join_type: JoinType, - pub output_schema: DataSchemaRef, - - // Only used for explain - pub stat_info: Option, -} - -#[typetag::serde] -impl IPhysicalPlan for NestedLoopJoin { - fn as_any(&self) -> &dyn Any { - self - } - fn get_meta(&self) -> &PhysicalPlanMeta { - &self.meta - } - - fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { - &mut self.meta - } - - #[recursive::recursive] - fn output_schema(&self) -> Result { - Ok(self.output_schema.clone()) - } - - fn children<'a>(&'a self) -> Box + 'a> { - Box::new([&self.left, &self.right].into_iter()) - } - - fn children_mut<'a>(&'a mut self) -> Box + 'a> { - Box::new([&mut self.left, &mut self.right].into_iter()) - } - - fn formatter(&self) -> Result> { - Ok(NestedLoopJoinFormatter::create(self)) - } - - fn get_desc(&self) -> Result { - Ok(self - .conditions - .iter() - .map(|x| x.as_expr(&BUILTIN_FUNCTIONS).sql_display()) - .join(" AND ")) - } - - fn derive(&self, mut children: Vec) -> PhysicalPlan { - assert_eq!(children.len(), 2); - let right_child = children.pop().unwrap(); - let left_child = children.pop().unwrap(); - - PhysicalPlan::new(NestedLoopJoin { - meta: self.meta.clone(), - left: left_child, - right: right_child, - conditions: self.conditions.clone(), - join_type: self.join_type, - output_schema: self.output_schema.clone(), - stat_info: self.stat_info.clone(), - }) - } - - fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { - // Build right side (build side) - let right_side_builder = builder.create_sub_pipeline_builder(); - let mut right_res = right_side_builder.finalize(&self.right)?; - let mut build_sinks = right_res.main_pipeline.take_sinks(); - builder - .pipelines - .push(right_res.main_pipeline.finalize(None)); - builder.pipelines.extend(right_res.sources_pipelines); - - // Build left side (probe side) - self.left.build_pipeline(builder)?; - - let output_len = std::cmp::max(build_sinks.len(), builder.main_pipeline.output_len()); - builder.main_pipeline.resize(output_len, false)?; - let probe_sinks = builder.main_pipeline.take_sinks(); - - if output_len != build_sinks.len() { - builder.main_pipeline.extend_sinks(build_sinks); - builder.main_pipeline.resize(output_len, false)?; - build_sinks = builder.main_pipeline.take_sinks(); - } - - debug_assert_eq!(build_sinks.len(), probe_sinks.len()); - - let join_pipe_items = build_sinks - .into_iter() - .zip(probe_sinks) - .map(|(build_sink, probe_sink)| { - builder.main_pipeline.extend_sinks([build_sink, probe_sink]); - - let build_input = InputPort::create(); - let probe_input = InputPort::create(); - let joined_output = OutputPort::create(); - - let join_state = LoopJoinState::new(builder.ctx.clone(), self); - let loop_join = ProcessorPtr::create(TransformLoopJoin::create( - build_input.clone(), - probe_input.clone(), - joined_output.clone(), - Box::new(join_state), - )); - - PipeItem::create(loop_join, vec![build_input, probe_input], vec![ - joined_output, - ]) - }) - .collect(); - - let join_pipe = Pipe::create(output_len * 2, output_len, join_pipe_items); - builder.main_pipeline.add_pipe(join_pipe); - - match self.conditions.len() { - 0 => Ok(()), - 1 => { - let expr = - cast_expr_to_non_null_boolean(self.conditions[0].as_expr(&BUILTIN_FUNCTIONS))?; - let projections = (0..self.output_schema.num_fields()).collect(); - builder.main_pipeline.add_transform( - builder.filter_transform_builder(&[expr.as_remote_expr()], projections)?, - ) - } - _ => { - let projections = (0..self.output_schema.num_fields()).collect(); - builder - .main_pipeline - .add_transform(builder.filter_transform_builder(&self.conditions, projections)?) - } - } - } -} - -impl PhysicalPlanBuilder { - pub async fn build_loop_join( - &mut self, - join_type: JoinType, - s_expr: &SExpr, - left_required: ColumnSet, - right_required: ColumnSet, - conditions: Vec, - ) -> Result { - let (left, right) = self - .build_join_sides(s_expr, left_required, right_required) - .await?; - - let left_schema = self.prepare_probe_schema(join_type, &left)?; - let right_schema = self.prepare_build_schema(join_type, &right)?; - - let output_schema = DataSchemaRefExt::create( - left_schema - .fields() - .iter() - .chain(right_schema.fields()) - .cloned() - .collect::>(), - ); - - let conditions = conditions - .iter() - .map(|scalar| resolve_scalar(scalar, &output_schema)) - .collect::>()?; - - Ok(PhysicalPlan::new(NestedLoopJoin { - left, - right, - meta: PhysicalPlanMeta::new("NestedLoopJoin"), - conditions, - join_type, - output_schema, - stat_info: Some(self.build_plan_stat_info(s_expr)?), - })) - } -} diff --git a/src/query/service/src/pipelines/processors/transforms/mod.rs b/src/query/service/src/pipelines/processors/transforms/mod.rs index b3e98f3479033..f1d74da1383a3 100644 --- a/src/query/service/src/pipelines/processors/transforms/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/mod.rs @@ -25,7 +25,6 @@ mod transform_branched_async_function; mod transform_cache_scan; mod transform_dictionary; mod transform_expression_scan; -mod transform_loop_join; mod transform_recursive_cte_scan; mod transform_recursive_cte_source; @@ -54,7 +53,6 @@ pub use transform_cache_scan::HashJoinCacheState; pub use transform_cache_scan::NewHashJoinCacheState; pub use transform_cache_scan::TransformCacheScan; pub use transform_expression_scan::TransformExpressionScan; -pub use transform_loop_join::*; pub use transform_recursive_cte_scan::TransformRecursiveCteScan; pub use transform_recursive_cte_source::TransformRecursiveCteSource; pub use transform_resort_addon::TransformResortAddOn; diff --git a/src/query/service/src/pipelines/processors/transforms/transform_loop_join.rs b/src/query/service/src/pipelines/processors/transforms/transform_loop_join.rs deleted file mode 100644 index d841be2fadb11..0000000000000 --- a/src/query/service/src/pipelines/processors/transforms/transform_loop_join.rs +++ /dev/null @@ -1,399 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::any::Any; -use std::sync::atomic::AtomicBool; -use std::sync::atomic::Ordering; -use std::sync::Arc; -use std::sync::Mutex; -use std::sync::RwLock; - -use databend_common_base::base::ProgressValues; -use databend_common_exception::Result; -use databend_common_expression::types::DataType; -use databend_common_expression::BlockEntry; -use databend_common_expression::DataBlock; -use databend_common_expression::Scalar; -use databend_common_pipeline::core::Event; -use databend_common_pipeline::core::InputPort; -use databend_common_pipeline::core::OutputPort; -use databend_common_pipeline::core::Processor; -use databend_common_sql::plans::JoinType; - -use super::Join; -use super::JoinStream; -use crate::physical_plans::NestedLoopJoin; -use crate::sessions::QueryContext; - -pub struct TransformLoopJoin { - build_port: Arc, - probe_port: Arc, - output_port: Arc, - joined_data: Option, - join: Box, - stage: Stage, -} - -impl TransformLoopJoin { - pub fn create( - build_port: Arc, - probe_port: Arc, - output_port: Arc, - join: Box, - ) -> Box { - Box::new(TransformLoopJoin { - build_port, - probe_port, - output_port, - join, - joined_data: None, - stage: Stage::Build(BuildState { - finished: false, - build_data: None, - }), - }) - } -} - -#[async_trait::async_trait] -impl Processor for TransformLoopJoin { - fn name(&self) -> String { - String::from("TransformLoopJoinLeft") - } - - fn as_any(&mut self) -> &mut dyn Any { - self - } - - fn event(&mut self) -> Result { - if self.output_port.is_finished() { - self.build_port.finish(); - self.probe_port.finish(); - return Ok(Event::Finished); - } - - if !self.output_port.can_push() { - match self.stage { - Stage::Build(_) => self.build_port.set_not_need_data(), - Stage::Probe(_) => self.probe_port.set_not_need_data(), - Stage::BuildFinal(_) | Stage::Finished => (), - } - return Ok(Event::NeedConsume); - } - - if let Some(joined_data) = self.joined_data.take() { - self.output_port.push_data(Ok(joined_data)); - return Ok(Event::NeedConsume); - } - - match &mut self.stage { - Stage::Build(state) => { - self.probe_port.set_not_need_data(); - state.event(&self.build_port) - } - Stage::BuildFinal(state) => state.event(), - Stage::Probe(state) => state.event(&self.probe_port), - Stage::Finished => Ok(Event::Finished), - } - } - - fn process(&mut self) -> Result<()> { - match &mut self.stage { - Stage::Finished => Ok(()), - Stage::Build(state) => { - let Some(data_block) = state.build_data.take() else { - if !state.finished { - state.finished = true; - } - self.stage = Stage::BuildFinal(BuildFinalState::new()); - return Ok(()); - }; - - if !data_block.is_empty() { - self.join.add_block(Some(data_block))?; - } - - Ok(()) - } - Stage::BuildFinal(state) => { - if self.join.final_build()?.is_none() { - state.finished = true; - self.stage = Stage::Probe(ProbeState::new()); - } - Ok(()) - } - Stage::Probe(state) => { - if let Some(probe_data) = state.input_data.take() { - let _stream = self.join.probe_block(probe_data)?; - todo!(); - // join and stream share the same lifetime within processor - // #[allow(clippy::missing_transmute_annotations)] - // state.stream = Some(unsafe { std::mem::transmute(stream) }); - } - - if let Some(mut stream) = state.stream.take() { - if let Some(joined_data) = stream.next()? { - self.joined_data = Some(joined_data); - state.stream = Some(stream); - } else if self.probe_port.is_finished() { - self.output_port.finish(); - self.stage = Stage::Finished; - } - } else if self.probe_port.is_finished() { - self.output_port.finish(); - self.stage = Stage::Finished; - } - - Ok(()) - } - } - } -} - -enum Stage { - Build(BuildState), - BuildFinal(BuildFinalState), - Probe(ProbeState), - Finished, -} - -#[derive(Debug)] -struct BuildState { - finished: bool, - build_data: Option, -} - -impl BuildState { - pub fn event(&mut self, input: &InputPort) -> Result { - if self.build_data.is_some() { - return Ok(Event::Sync); - } - - if input.has_data() { - self.build_data = Some(input.pull_data().unwrap()?); - return Ok(Event::Sync); - } - - if input.is_finished() { - return match self.finished { - true => Ok(Event::Finished), - false => Ok(Event::Sync), - }; - } - - input.set_need_data(); - Ok(Event::NeedData) - } -} - -#[derive(Debug)] -struct BuildFinalState { - finished: bool, -} - -impl BuildFinalState { - pub fn new() -> BuildFinalState { - BuildFinalState { finished: false } - } - - pub fn event(&mut self) -> Result { - match self.finished { - true => Ok(Event::Async), - false => Ok(Event::Sync), - } - } -} - -struct ProbeState { - input_data: Option, - stream: Option>, -} - -impl ProbeState { - pub fn new() -> ProbeState { - ProbeState { - input_data: None, - stream: None, - } - } - - pub fn event(&mut self, input: &InputPort) -> Result { - if self.input_data.is_some() || self.stream.is_some() { - return Ok(Event::Sync); - } - - if input.has_data() { - self.input_data = Some(input.pull_data().unwrap()?); - return Ok(Event::Sync); - } - - if input.is_finished() { - return Ok(Event::Sync); - } - - input.set_need_data(); - Ok(Event::NeedData) - } -} - -#[derive(Clone)] -pub struct LoopJoinState { - build_table: Arc>>, - build_progress: Arc>>, - build_finished: Arc, - join_type: JoinType, -} - -impl LoopJoinState { - pub fn new(_ctx: Arc, join: &NestedLoopJoin) -> Self { - Self { - build_table: Arc::new(RwLock::new(vec![])), - build_progress: Arc::new(Mutex::new(None)), - build_finished: Arc::new(AtomicBool::new(false)), - join_type: join.join_type, - } - } - - fn push_build_block(&self, right_block: DataBlock) -> Result<()> { - let right = if matches!(self.join_type, JoinType::Left | JoinType::Full) { - let rows = right_block.num_rows(); - let entries = right_block - .take_columns() - .into_iter() - .map(|entry| entry.into_nullable()) - .collect::>(); - DataBlock::new(entries, rows) - } else { - right_block - }; - self.build_table.write()?.push(right); - Ok(()) - } -} - -impl Join for LoopJoinState { - fn add_block(&mut self, data: Option) -> Result<()> { - let Some(right_block) = data else { - return Ok(()); - }; - - self.push_build_block(right_block) - } - - fn final_build(&mut self) -> Result> { - if self.build_finished.swap(true, Ordering::SeqCst) { - return Ok(None); - } - - let progress = self.build_table.read()?.iter().fold( - ProgressValues::default(), - |mut progress, block| { - progress.rows += block.num_rows(); - progress.bytes += block.memory_size(); - progress - }, - ); - - let mut guard = self.build_progress.lock()?; - *guard = Some(progress.clone()); - Ok(Some(progress)) - } - - fn probe_block(&mut self, data: DataBlock) -> Result> { - let left = if matches!(self.join_type, JoinType::Right | JoinType::Full) { - let rows = data.num_rows(); - let entries = data - .take_columns() - .into_iter() - .map(|entry| entry.into_nullable()) - .collect::>(); - DataBlock::new(entries, rows) - } else { - data - }; - - let right_table = self.build_table.read()?.clone(); - Ok(Box::new(LoopJoinStream::new(left, right_table))) - } -} - -struct LoopJoinStream { - left_rows: Vec>, - left_types: Vec, - right_blocks: Vec, - left_row: usize, - right_index: usize, -} - -impl LoopJoinStream { - fn new(left: DataBlock, right_blocks: Vec) -> Self { - let mut left_rows = vec![Vec::new(); left.num_rows()]; - for entry in left.columns().iter() { - match entry { - BlockEntry::Const(scalar, _, _) => { - for row in &mut left_rows { - row.push(scalar.to_owned()); - } - } - BlockEntry::Column(column) => { - for (row, scalar) in left_rows.iter_mut().zip(column.iter()) { - row.push(scalar.to_owned()); - } - } - } - } - - let left_types = left - .columns() - .iter() - .map(|entry| entry.data_type()) - .collect(); - - LoopJoinStream { - left_rows, - left_types, - right_blocks, - left_row: 0, - right_index: 0, - } - } -} - -impl JoinStream for LoopJoinStream { - fn next(&mut self) -> Result> { - if self.right_blocks.is_empty() || self.left_row >= self.left_rows.len() { - return Ok(None); - } - - let right_block = &self.right_blocks[self.right_index]; - let left_entries = &self.left_rows[self.left_row]; - - let entries = left_entries - .iter() - .zip(self.left_types.iter()) - .map(|(scalar, data_type)| { - BlockEntry::Const(scalar.clone(), data_type.clone(), right_block.num_rows()) - }) - .chain(right_block.columns().iter().cloned()) - .collect(); - - self.right_index += 1; - if self.right_index >= self.right_blocks.len() { - self.right_index = 0; - self.left_row += 1; - } - - Ok(Some(DataBlock::new(entries, right_block.num_rows()))) - } -} From 0fb419e0ba83d0234d0f7c0d3da08e545cc8e577 Mon Sep 17 00:00:00 2001 From: coldWater Date: Thu, 20 Nov 2025 09:53:58 +0800 Subject: [PATCH 06/19] x --- src/query/expression/src/schema.rs | 3 +- .../src/physical_plans/physical_hash_join.rs | 86 +++++++++++++++---- .../processors/transforms/hash_join/desc.rs | 70 +++++++++++---- .../processors/transforms/hash_join/mod.rs | 1 + .../new_hash_join/memory/inner_join.rs | 35 +++++++- 5 files changed, 156 insertions(+), 39 deletions(-) diff --git a/src/query/expression/src/schema.rs b/src/query/expression/src/schema.rs index 38d0500d5b35d..ca227df6bea42 100644 --- a/src/query/expression/src/schema.rs +++ b/src/query/expression/src/schema.rs @@ -420,8 +420,7 @@ impl DataSchema { let mut valid_fields: Vec = self.fields.iter().map(|f| f.name().clone()).collect(); valid_fields.truncate(16); Err(ErrorCode::BadArguments(format!( - "Unable to get field named \"{}\". Valid fields: {:?} ...", - name, valid_fields + "Unable to get field named {name:?}. Valid fields: {valid_fields:?} ...", ))) } diff --git a/src/query/service/src/physical_plans/physical_hash_join.rs b/src/query/service/src/physical_plans/physical_hash_join.rs index 3136ba7eff702..4b1a24eac109f 100644 --- a/src/query/service/src/physical_plans/physical_hash_join.rs +++ b/src/query/service/src/physical_plans/physical_hash_join.rs @@ -150,7 +150,7 @@ pub struct HashJoin { pub runtime_filter: PhysicalRuntimeFilters, pub broadcast_id: Option, - pub nested_loop_filter: NestedLoopFilterInfo, + pub nested_loop_filter: Option, } #[typetag::serde] @@ -1043,7 +1043,7 @@ impl PhysicalPlanBuilder { } // Add tail fields - build_fields.extend(tail_fields.clone()); + build_fields.extend_from_slice(&tail_fields); merged_fields.extend(tail_fields); Ok((merged_fields, probe_fields, build_fields, probe_to_build)) @@ -1199,20 +1199,73 @@ impl PhysicalPlanBuilder { fn build_nested_loop_filter_info( &self, join: &Join, - merged_schema: &DataSchema, - ) -> Result { - let predicates = join - .non_equi_conditions + probe_schema: &DataSchema, + build_schema: &DataSchema, + target_schema: &DataSchema, + ) -> Result> { + if matches!(join.join_type, JoinType::Inner) { + return Ok(None); + } + + let merged = DataSchema::new( + probe_schema + .fields + .iter() + .cloned() + .chain(build_schema.fields.iter().cloned()) + .collect(), + ); + + let mut predicates = + Vec::with_capacity(join.equi_conditions.len() + join.non_equi_conditions.len()); + + for condition in &join.equi_conditions { + let scalar = condition_to_expr(condition)?; + match resolve_scalar(&scalar, &merged) { + Ok(expr) => predicates.push(expr), + Err(_) + if condition + .left + .data_type() + .map(|data_type| data_type.remove_nullable().is_bitmap()) + .unwrap_or_default() => + { + // no function matches signature `eq(Bitmap NULL, Bitmap NULL) + return Ok(None); + } + Err(err) => { + return Err(err.add_message(format!( + "Failed build nested loop filter schema: {merged:#?} equi_conditions: {:#?}", + join.equi_conditions + ))) + } + } + } + + for scalar in &join.non_equi_conditions { + predicates.push(resolve_scalar(scalar, &merged).map_err(|err|{ + err.add_message(format!( + "Failed build nested loop filter schema: {merged:#?} non_equi_conditions: {:#?}", + join.non_equi_conditions + )) + })?); + } + + let projection = target_schema + .fields .iter() - .map(|c| Ok(c.clone())) - .chain(join.equi_conditions.iter().map(condition_to_expr)) - .map(|scalar| resolve_scalar(&scalar?, merged_schema)) - .collect::>()?; + .map(|column| merged.index_of(column.name())) + .collect::>>() + .map_err(|err| { + err.add_message(format!( + "Failed build nested loop filter schema: {merged:#?} target: {target_schema:#?}", + )) + })?; - Ok(NestedLoopFilterInfo { + Ok(Some(NestedLoopFilterInfo { predicates, - projection: vec![], - }) + projection, + })) } pub async fn build_hash_join( @@ -1299,7 +1352,8 @@ impl PhysicalPlanBuilder { ) .await?; - let nested_loop_filter = self.build_nested_loop_filter_info(join, &merged_schema)?; + let nested_loop_filter = + self.build_nested_loop_filter_info(join, &probe_schema, &build_schema, &merged_schema)?; // Step 12: Create and return the HashJoin let build_side_data_distribution = s_expr.build_side_child().get_data_distribution()?; @@ -1343,11 +1397,11 @@ fn condition_to_expr(condition: &JoinEquiCondition) -> Result { let right_type = condition.right.data_type()?; let arguments = match (&left_type, &right_type) { - (DataType::Nullable(left), right) if **left == *right => vec![ + (DataType::Nullable(box left), right) if left == right => vec![ condition.left.clone(), condition.right.clone().unify_to_data_type(&left_type), ], - (left, DataType::Nullable(right)) if *left == **right => vec![ + (left, DataType::Nullable(box right)) if left == right => vec![ condition.left.clone().unify_to_data_type(&right_type), condition.right.clone(), ], 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 f5a470c7d5334..a0b09a97e49bf 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 @@ -22,10 +22,12 @@ use databend_common_expression::DataBlock; use databend_common_expression::DataSchemaRef; use databend_common_expression::Evaluator; use databend_common_expression::Expr; +use databend_common_expression::FieldIndex; use databend_common_expression::FilterExecutor; use databend_common_expression::FunctionContext; use databend_common_expression::RemoteExpr; use databend_common_functions::BUILTIN_FUNCTIONS; +use databend_common_settings::Settings; use databend_common_sql::executor::cast_expr_to_non_null_boolean; use databend_common_sql::ColumnSet; use parking_lot::RwLock; @@ -65,7 +67,7 @@ pub struct HashJoinDesc { pub(crate) probe_projection: ColumnSet, pub(crate) probe_to_build: Vec<(usize, (bool, bool))>, pub(crate) build_schema: DataSchemaRef, - pub(crate) nested_loop_filter: NestedLoopFilterInfo, + pub(crate) nested_loop_filter: Option, } #[derive(Debug, Clone)] @@ -79,7 +81,7 @@ pub struct RuntimeFilterDesc { pub enable_min_max_runtime_filter: bool, } -pub struct RuntimeFiltersDesc { +pub(crate) struct RuntimeFiltersDesc { pub filters: Vec, } @@ -264,28 +266,62 @@ impl HashJoinDesc { } } - pub fn create_nested_loop_filter( + pub fn create_nested_loop_desc( &self, + settings: &Settings, function_ctx: &FunctionContext, - block_size: usize, - ) -> Result { - let predicates = self - .nested_loop_filter - .predicates + ) -> Result> { + let nested_loop_join_threshold = settings.get_nested_loop_join_threshold()? as usize; + let block_size = settings.get_max_block_size()? as usize; + if nested_loop_join_threshold == 0 { + return Ok(None); + } + + let Some(NestedLoopFilterInfo { + predicates, + projection, + }) = &self.nested_loop_filter + else { + return Ok(None); + }; + + let predicates = predicates .iter() .map(|x| Ok(x.as_expr(&BUILTIN_FUNCTIONS))) .reduce(|lhs, rhs| { check_function(None, "and_filters", &[], &[lhs?, rhs?], &BUILTIN_FUNCTIONS) }) - .unwrap()?; + .transpose()?; + let Some(predicates) = predicates else { + return Ok(None); + }; - Ok(FilterExecutor::new( - predicates, - function_ctx.clone(), - block_size, - None, // Some(self.nested_loop_filter.projection.iter().copied().collect()), - &BUILTIN_FUNCTIONS, - false, - )) + let field_reorder = if !projection.is_sorted() { + let mut mapper = projection.iter().cloned().enumerate().collect::>(); + mapper.sort_by_key(|(_, field)| *field); + let reorder = mapper.iter().map(|(i, _)| *i).collect::>(); + Some(reorder) + } else { + None + }; + + Ok(Some(NestedLoopDesc { + filter: FilterExecutor::new( + predicates, + function_ctx.clone(), + block_size, + Some(projection.iter().copied().collect()), + &BUILTIN_FUNCTIONS, + false, + ), + field_reorder, + nested_loop_join_threshold, + })) } } + +pub struct NestedLoopDesc { + pub filter: FilterExecutor, + pub field_reorder: Option>, + pub nested_loop_join_threshold: usize, +} 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 32b46295ccb1f..2f5caba0e8474 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 @@ -32,6 +32,7 @@ mod util; pub use common::wrap_true_validity; pub use desc::HashJoinDesc; +pub use desc::NestedLoopDesc; pub use desc::RuntimeFilterDesc; pub use hash_join_build_state::HashJoinBuildState; pub use hash_join_probe_state::HashJoinProbeState; diff --git a/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/inner_join.rs b/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/inner_join.rs index 279a064937fc2..36475c0560f61 100644 --- a/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/inner_join.rs +++ b/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/inner_join.rs @@ -41,6 +41,7 @@ use crate::pipelines::processors::transforms::new_hash_join::join::JoinStream; use crate::pipelines::processors::transforms::new_hash_join::performance::PerformanceContext; use crate::pipelines::processors::transforms::HashJoinHashTable; use crate::pipelines::processors::transforms::JoinRuntimeFilterPacket; +use crate::pipelines::processors::transforms::NestedLoopDesc; use crate::pipelines::processors::transforms::RuntimeFiltersDesc; use crate::pipelines::processors::HashJoinDesc; use crate::sessions::QueryContext; @@ -52,7 +53,8 @@ pub struct InnerHashJoin { pub(crate) function_ctx: FunctionContext, pub(crate) basic_state: Arc, pub(crate) performance_context: PerformanceContext, - nested_loop_filter: FilterExecutor, + nested_loop_filter: Option, + nested_loop_field_reorder: Option>, } impl InnerHashJoin { @@ -68,16 +70,25 @@ impl InnerHashJoin { let context = PerformanceContext::create(block_size, desc.clone(), function_ctx.clone()); + let (nested_loop_filter, nested_loop_field_reorder, nested_loop_join_threshold) = + match desc.create_nested_loop_desc(&settings, &function_ctx)? { + Some(NestedLoopDesc { + filter, + field_reorder, + nested_loop_join_threshold, + }) => (Some(filter), field_reorder, nested_loop_join_threshold), + None => (None, None, 0), + }; + let basic_hash_join = BasicHashJoin::create( &settings, function_ctx.clone(), method, desc.clone(), state.clone(), - settings.get_nested_loop_join_threshold()? as _, + nested_loop_join_threshold, )?; - let nested_loop_filter = desc.create_nested_loop_filter(&function_ctx, block_size)?; Ok(InnerHashJoin { desc, basic_hash_join, @@ -85,6 +96,7 @@ impl InnerHashJoin { basic_state: state, performance_context: context, nested_loop_filter, + nested_loop_field_reorder, }) } } @@ -129,7 +141,8 @@ impl Join for InnerHashJoin { let nested = Box::new(LoopJoinStream::new(data, build_blocks)); return Ok(InnerHashJoinFilterStream::create( nested, - &mut self.nested_loop_filter, + self.nested_loop_filter.as_mut().unwrap(), + self.nested_loop_field_reorder.as_deref(), )); } _ => (), @@ -170,6 +183,7 @@ impl Join for InnerHashJoin { Some(filter_executor) => Ok(InnerHashJoinFilterStream::create( joined_stream, filter_executor, + None, )), } } @@ -281,16 +295,19 @@ impl<'a> JoinStream for InnerHashJoinStream<'a> { struct InnerHashJoinFilterStream<'a> { inner: Box, filter_executor: &'a mut FilterExecutor, + field_reorder: Option<&'a [usize]>, } impl<'a> InnerHashJoinFilterStream<'a> { pub fn create( inner: Box, filter_executor: &'a mut FilterExecutor, + field_reorder: Option<&'a [usize]>, ) -> Box { Box::new(InnerHashJoinFilterStream { inner, filter_executor, + field_reorder, }) } } @@ -312,6 +329,16 @@ impl<'a> JoinStream for InnerHashJoinFilterStream<'a> { continue; } + let data_block = if let Some(field_reorder) = self.field_reorder { + DataBlock::from_iter( + field_reorder + .iter() + .map(|offset| data_block.get_by_offset(*offset).clone()), + data_block.num_rows(), + ) + } else { + data_block + }; return Ok(Some(data_block)); } } From 76e5c0d5ef44afce14d403b314f4d3a39fa587da Mon Sep 17 00:00:00 2001 From: coldWater Date: Fri, 21 Nov 2025 10:54:13 +0800 Subject: [PATCH 07/19] refine --- .../new_hash_join/hash_join_factory.rs | 40 +++++++++---- .../transforms/new_hash_join/memory/basic.rs | 11 +--- .../new_hash_join/memory/inner_join.rs | 51 ++++------------ .../new_hash_join/memory/nested_loop.rs | 58 ++++++++++++++++++- .../new_hash_join/transform_hash_join.rs | 16 +++-- 5 files changed, 111 insertions(+), 65 deletions(-) diff --git a/src/query/service/src/pipelines/processors/transforms/new_hash_join/hash_join_factory.rs b/src/query/service/src/pipelines/processors/transforms/new_hash_join/hash_join_factory.rs index bc7aad3e4a36f..c643526d15156 100644 --- a/src/query/service/src/pipelines/processors/transforms/new_hash_join/hash_join_factory.rs +++ b/src/query/service/src/pipelines/processors/transforms/new_hash_join/hash_join_factory.rs @@ -26,9 +26,10 @@ use databend_common_expression::FunctionContext; use databend_common_expression::HashMethodKind; use databend_common_sql::plans::JoinType; -use crate::pipelines::processors::transforms::memory::outer_left_join::OuterLeftHashJoin; -use crate::pipelines::processors::transforms::new_hash_join::common::CStyleCell; -use crate::pipelines::processors::transforms::new_hash_join::grace::GraceHashJoinState; +use super::common::CStyleCell; +use super::grace::GraceHashJoinState; +use super::memory::outer_left_join::OuterLeftHashJoin; +use super::memory::NestedLoopJoin; use crate::pipelines::processors::transforms::BasicHashJoinState; use crate::pipelines::processors::transforms::GraceHashJoin; use crate::pipelines::processors::transforms::InnerHashJoin; @@ -126,13 +127,29 @@ impl HashJoinFactory { } match typ { - JoinType::Inner => Ok(Box::new(InnerHashJoin::create( - &self.ctx, - self.function_ctx.clone(), - self.hash_method.clone(), - self.desc.clone(), - self.create_basic_state(id)?, - )?)), + JoinType::Inner => { + let state = self.create_basic_state(id)?; + let nested_loop_desc = self + .desc + .create_nested_loop_desc(&settings, &self.function_ctx)?; + + let inner = InnerHashJoin::create( + &settings, + self.function_ctx.clone(), + self.hash_method.clone(), + self.desc.clone(), + state.clone(), + nested_loop_desc + .as_ref() + .map(|desc| desc.nested_loop_join_threshold) + .unwrap_or_default(), + )?; + + match nested_loop_desc { + Some(desc) => Ok(Box::new(NestedLoopJoin::create(inner, state, desc))), + None => Ok(Box::new(inner)), + } + } JoinType::Left => Ok(Box::new(OuterLeftHashJoin::create( &self.ctx, self.function_ctx.clone(), @@ -148,11 +165,12 @@ impl HashJoinFactory { match typ { JoinType::Inner => { let inner_hash_join = InnerHashJoin::create( - &self.ctx, + &self.ctx.get_settings(), self.function_ctx.clone(), self.hash_method.clone(), self.desc.clone(), self.create_basic_state(id)?, + 0, )?; Ok(Box::new(GraceHashJoin::create( diff --git a/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/basic.rs b/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/basic.rs index 8b6ab97247953..3c15a7ff8d3c7 100644 --- a/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/basic.rs +++ b/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/basic.rs @@ -354,17 +354,10 @@ impl BasicHashJoin { let mut progress = ProgressValues::default(); let mut plain = vec![]; while let Some(chunk_index) = self.state.steal_chunk_index() { - let chunk_mut = &mut self.state.chunks.as_mut()[chunk_index]; - - let mut chunk_block = DataBlock::empty(); - std::mem::swap(chunk_mut, &mut chunk_block); - + let chunk_block = &self.state.chunks[chunk_index]; progress.rows += chunk_block.num_rows(); progress.bytes += chunk_block.memory_size(); - - *chunk_mut = chunk_block.clone(); - - plain.push(chunk_block); + plain.push(chunk_block.clone()); } debug_assert!(matches!( *self.state.hash_table, diff --git a/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/inner_join.rs b/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/inner_join.rs index 36475c0560f61..a5bebc663062f 100644 --- a/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/inner_join.rs +++ b/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/inner_join.rs @@ -16,7 +16,6 @@ use std::ops::Deref; use std::sync::Arc; use databend_common_base::base::ProgressValues; -use databend_common_catalog::table_context::TableContext; use databend_common_column::bitmap::Bitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; @@ -27,10 +26,10 @@ use databend_common_expression::DataBlock; use databend_common_expression::FilterExecutor; use databend_common_expression::FunctionContext; use databend_common_expression::HashMethodKind; +use databend_common_settings::Settings; use super::basic::BasicHashJoin; use super::basic_state::BasicHashJoinState; -use super::LoopJoinStream; use crate::pipelines::processors::transforms::build_runtime_filter_packet; use crate::pipelines::processors::transforms::new_hash_join::hashtable::basic::ProbeStream; use crate::pipelines::processors::transforms::new_hash_join::hashtable::basic::ProbedRows; @@ -41,10 +40,8 @@ use crate::pipelines::processors::transforms::new_hash_join::join::JoinStream; use crate::pipelines::processors::transforms::new_hash_join::performance::PerformanceContext; use crate::pipelines::processors::transforms::HashJoinHashTable; use crate::pipelines::processors::transforms::JoinRuntimeFilterPacket; -use crate::pipelines::processors::transforms::NestedLoopDesc; use crate::pipelines::processors::transforms::RuntimeFiltersDesc; use crate::pipelines::processors::HashJoinDesc; -use crate::sessions::QueryContext; pub struct InnerHashJoin { pub(crate) basic_hash_join: BasicHashJoin, @@ -53,35 +50,23 @@ pub struct InnerHashJoin { pub(crate) function_ctx: FunctionContext, pub(crate) basic_state: Arc, pub(crate) performance_context: PerformanceContext, - nested_loop_filter: Option, - nested_loop_field_reorder: Option>, } impl InnerHashJoin { pub fn create( - ctx: &QueryContext, + settings: &Settings, function_ctx: FunctionContext, method: HashMethodKind, desc: Arc, state: Arc, + nested_loop_join_threshold: usize, ) -> Result { - let settings = ctx.get_settings(); let block_size = settings.get_max_block_size()? as usize; let context = PerformanceContext::create(block_size, desc.clone(), function_ctx.clone()); - let (nested_loop_filter, nested_loop_field_reorder, nested_loop_join_threshold) = - match desc.create_nested_loop_desc(&settings, &function_ctx)? { - Some(NestedLoopDesc { - filter, - field_reorder, - nested_loop_join_threshold, - }) => (Some(filter), field_reorder, nested_loop_join_threshold), - None => (None, None, 0), - }; - let basic_hash_join = BasicHashJoin::create( - &settings, + settings, function_ctx.clone(), method, desc.clone(), @@ -95,8 +80,6 @@ impl InnerHashJoin { function_ctx, basic_state: state, performance_context: context, - nested_loop_filter, - nested_loop_field_reorder, }) } } @@ -131,23 +114,6 @@ impl Join for InnerHashJoin { self.basic_hash_join.finalize_chunks(); - match &*self.basic_state.hash_table { - HashJoinHashTable::Null => { - return Err(ErrorCode::AbortedQuery( - "Aborted query, because the hash table is uninitialized.", - )) - } - HashJoinHashTable::NestedLoop(build_blocks) => { - let nested = Box::new(LoopJoinStream::new(data, build_blocks)); - return Ok(InnerHashJoinFilterStream::create( - nested, - self.nested_loop_filter.as_mut().unwrap(), - self.nested_loop_field_reorder.as_deref(), - )); - } - _ => (), - } - let probe_keys = self.desc.probe_key(&data, &self.function_ctx)?; let mut keys = DataBlock::new(probe_keys, data.num_rows()); @@ -175,7 +141,12 @@ impl Join for InnerHashJoin { &mut self.performance_context.probe_result, ) } - HashJoinHashTable::Null | HashJoinHashTable::NestedLoop(_) => unreachable!(), + HashJoinHashTable::Null => { + return Err(ErrorCode::AbortedQuery( + "Aborted query, because the hash table is uninitialized.", + )); + } + HashJoinHashTable::NestedLoop(_) => unreachable!(), }); match &mut self.performance_context.filter_executor { @@ -292,7 +263,7 @@ impl<'a> JoinStream for InnerHashJoinStream<'a> { } } -struct InnerHashJoinFilterStream<'a> { +pub(super) struct InnerHashJoinFilterStream<'a> { inner: Box, filter_executor: &'a mut FilterExecutor, field_reorder: Option<&'a [usize]>, diff --git a/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/nested_loop.rs b/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/nested_loop.rs index f22158fa08e7c..ce157e2622516 100644 --- a/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/nested_loop.rs +++ b/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/nested_loop.rs @@ -13,16 +13,72 @@ // limitations under the License. use std::collections::VecDeque; +use std::sync::Arc; +use databend_common_base::base::ProgressValues; use databend_common_exception::Result; use databend_common_expression::types::DataType; use databend_common_expression::BlockEntry; use databend_common_expression::DataBlock; use databend_common_expression::Scalar; +use super::inner_join::InnerHashJoinFilterStream; +use crate::pipelines::processors::transforms::new_hash_join::join::EmptyJoinStream; +use crate::pipelines::processors::transforms::BasicHashJoinState; +use crate::pipelines::processors::transforms::HashJoinHashTable; +use crate::pipelines::processors::transforms::Join; +use crate::pipelines::processors::transforms::JoinRuntimeFilterPacket; use crate::pipelines::processors::transforms::JoinStream; +use crate::pipelines::processors::transforms::NestedLoopDesc; +use crate::pipelines::processors::transforms::RuntimeFiltersDesc; -pub struct LoopJoinStream<'a> { +pub struct NestedLoopJoin { + inner: T, + basic_state: Arc, + desc: NestedLoopDesc, +} + +impl NestedLoopJoin { + pub fn create(inner: T, basic_state: Arc, desc: NestedLoopDesc) -> Self { + Self { + inner, + basic_state, + desc, + } + } +} + +impl Join for NestedLoopJoin { + fn add_block(&mut self, data: Option) -> Result<()> { + self.inner.add_block(data) + } + + fn final_build(&mut self) -> Result> { + self.inner.final_build() + } + + fn build_runtime_filter(&self, desc: &RuntimeFiltersDesc) -> Result { + self.inner.build_runtime_filter(desc) + } + + fn probe_block(&mut self, data: DataBlock) -> Result> { + if data.is_empty() || *self.basic_state.build_rows == 0 { + return Ok(Box::new(EmptyJoinStream)); + } + let HashJoinHashTable::NestedLoop(build_blocks) = &*self.basic_state.hash_table else { + return self.inner.probe_block(data); + }; + + let nested = Box::new(LoopJoinStream::new(data, build_blocks)); + Ok(InnerHashJoinFilterStream::create( + nested, + &mut self.desc.filter, + self.desc.field_reorder.as_deref(), + )) + } +} + +struct LoopJoinStream<'a> { probe_rows: VecDeque>, probe_types: Vec, build_blocks: &'a [DataBlock], diff --git a/src/query/service/src/pipelines/processors/transforms/new_hash_join/transform_hash_join.rs b/src/query/service/src/pipelines/processors/transforms/new_hash_join/transform_hash_join.rs index dce2ca6ecceb7..cd73b20d14599 100644 --- a/src/query/service/src/pipelines/processors/transforms/new_hash_join/transform_hash_join.rs +++ b/src/query/service/src/pipelines/processors/transforms/new_hash_join/transform_hash_join.rs @@ -15,6 +15,7 @@ use std::any::Any; use std::fmt::Debug; use std::fmt::Formatter; +use std::marker::PhantomPinned; use std::sync::Arc; use databend_common_exception::Result; @@ -42,6 +43,7 @@ pub struct TransformHashJoin { stage_sync_barrier: Arc, projection: ColumnSet, rf_desc: Arc, + _p: PhantomPinned, } impl TransformHashJoin { @@ -67,6 +69,7 @@ impl TransformHashJoin { finished: false, build_data: None, }), + _p: PhantomPinned, })) } } @@ -117,8 +120,7 @@ impl Processor for TransformHashJoin { } } - #[allow(clippy::missing_transmute_annotations)] - fn process(&mut self) -> Result<()> { + fn process<'a>(&'a mut self) -> Result<()> { match &mut self.stage { Stage::Finished => Ok(()), Stage::Build(state) => { @@ -144,7 +146,9 @@ impl Processor for TransformHashJoin { if let Some(probe_data) = state.input_data.take() { let stream = self.join.probe_block(probe_data)?; // This is safe because both join and stream are properties of the struct. - state.stream = Some(unsafe { std::mem::transmute(stream) }); + state.stream = Some(unsafe { + std::mem::transmute::, Box>(stream) + }); } if let Some(mut stream) = state.stream.take() { @@ -161,7 +165,11 @@ impl Processor for TransformHashJoin { if let Some(final_stream) = self.join.final_probe()? { state.initialize = true; // This is safe because both join and stream are properties of the struct. - state.stream = Some(unsafe { std::mem::transmute(final_stream) }); + state.stream = Some(unsafe { + std::mem::transmute::, Box>( + final_stream, + ) + }); } else { state.finished = true; } From 2d3932fd69641927e7ddfd2b19cc3a8c46f36bc5 Mon Sep 17 00:00:00 2001 From: coldWater Date: Fri, 21 Nov 2025 11:04:06 +0800 Subject: [PATCH 08/19] fix --- src/query/service/src/physical_plans/physical_hash_join.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/query/service/src/physical_plans/physical_hash_join.rs b/src/query/service/src/physical_plans/physical_hash_join.rs index 4b1a24eac109f..fdeacfae28117 100644 --- a/src/query/service/src/physical_plans/physical_hash_join.rs +++ b/src/query/service/src/physical_plans/physical_hash_join.rs @@ -1203,7 +1203,7 @@ impl PhysicalPlanBuilder { build_schema: &DataSchema, target_schema: &DataSchema, ) -> Result> { - if matches!(join.join_type, JoinType::Inner) { + if !matches!(join.join_type, JoinType::Inner) { return Ok(None); } From 3c64236359b32327a5945917cda4f6644faac38a Mon Sep 17 00:00:00 2001 From: coldWater Date: Fri, 21 Nov 2025 11:48:13 +0800 Subject: [PATCH 09/19] x --- .../transforms/new_hash_join/memory/outer_left_join.rs | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/outer_left_join.rs b/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/outer_left_join.rs index e548178a06646..f6faa9db4f60c 100644 --- a/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/outer_left_join.rs +++ b/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/outer_left_join.rs @@ -138,9 +138,7 @@ impl Join for OuterLeftHashJoin { let probe_data = ProbeData::new(keys, valids, probe_hash_statistics); table.probe(probe_data) } - HashJoinHashTable::NestedLoop(_) => { - todo!() - } + HashJoinHashTable::NestedLoop(_) => unreachable!(), HashJoinHashTable::Null => Err(ErrorCode::AbortedQuery( "Aborted query, because the hash table is uninitialized.", )), From e6e9bfbc6e12dfdafa9e0374d20fee97b33b4895 Mon Sep 17 00:00:00 2001 From: coldWater Date: Mon, 24 Nov 2025 09:53:07 +0800 Subject: [PATCH 10/19] fix --- src/common/exception/src/exception_into.rs | 7 ----- .../transforms/new_hash_join/memory/basic.rs | 28 +++++++++---------- .../new_hash_join/transform_hash_join.rs | 6 ++-- 3 files changed, 17 insertions(+), 24 deletions(-) diff --git a/src/common/exception/src/exception_into.rs b/src/common/exception/src/exception_into.rs index 574191b4c73b1..df84a23c7ee79 100644 --- a/src/common/exception/src/exception_into.rs +++ b/src/common/exception/src/exception_into.rs @@ -16,7 +16,6 @@ use std::error::Error; use std::fmt::Debug; use std::fmt::Display; use std::fmt::Formatter; -use std::sync::PoisonError; use geozero::error::GeozeroError; @@ -437,9 +436,3 @@ impl From for ErrorCode { ErrorCode::DictionarySourceError(format!("Dictionary Redis Error, cause: {}", error)) } } - -impl From> for ErrorCode { - fn from(error: PoisonError) -> Self { - ErrorCode::Internal(format!("{error}")) - } -} diff --git a/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/basic.rs b/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/basic.rs index 3c15a7ff8d3c7..e110d20eca480 100644 --- a/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/basic.rs +++ b/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/basic.rs @@ -141,26 +141,26 @@ impl BasicHashJoin { return; } - if matches!( - self.state.hash_table.deref(), - HashJoinHashTable::NestedLoop(_) - ) { - return; - } - let locked = self.state.mutex.lock(); let _locked = locked.unwrap_or_else(PoisonError::into_inner); - if self.state.chunks.is_empty() || !self.state.columns.is_empty() { + debug_assert!(!matches!( + self.state.hash_table.deref(), + HashJoinHashTable::NestedLoop(_) + )); + + if !self.state.columns.is_empty() { return; } - if let Some(block) = self.state.chunks.first() { - for offset in 0..self.desc.build_projection.len() { - let column_type = self.state.column_types.as_mut(); - column_type.push(block.get_by_offset(offset).data_type()); - } - } + let column_type = self.state.column_types.as_mut(); + column_type.extend( + (0..self.desc.build_projection.len()) + .map(|offset| block.get_by_offset(offset).data_type()), + ); + } else { + return; + }; let mut columns = Vec::with_capacity(self.desc.build_projection.len()); for offset in 0..self.desc.build_projection.len() { diff --git a/src/query/service/src/pipelines/processors/transforms/new_hash_join/transform_hash_join.rs b/src/query/service/src/pipelines/processors/transforms/new_hash_join/transform_hash_join.rs index cd73b20d14599..b9d50285c2a6a 100644 --- a/src/query/service/src/pipelines/processors/transforms/new_hash_join/transform_hash_join.rs +++ b/src/query/service/src/pipelines/processors/transforms/new_hash_join/transform_hash_join.rs @@ -120,7 +120,7 @@ impl Processor for TransformHashJoin { } } - fn process<'a>(&'a mut self) -> Result<()> { + fn process(&mut self) -> Result<()> { match &mut self.stage { Stage::Finished => Ok(()), Stage::Build(state) => { @@ -147,7 +147,7 @@ impl Processor for TransformHashJoin { let stream = self.join.probe_block(probe_data)?; // This is safe because both join and stream are properties of the struct. state.stream = Some(unsafe { - std::mem::transmute::, Box>(stream) + std::mem::transmute::, Box>(stream) }); } @@ -166,7 +166,7 @@ impl Processor for TransformHashJoin { state.initialize = true; // This is safe because both join and stream are properties of the struct. state.stream = Some(unsafe { - std::mem::transmute::, Box>( + std::mem::transmute::, Box>( final_stream, ) }); From 321f36d1d2391a390be146578759b75fb9f08528 Mon Sep 17 00:00:00 2001 From: coldWater Date: Mon, 24 Nov 2025 14:45:26 +0800 Subject: [PATCH 11/19] update --- .../base/src/runtime/perf/query_perf.rs | 2 +- .../src/physical_plans/physical_hash_join.rs | 14 ++++ .../new_hash_join/memory/nested_loop.rs | 70 ++++++++++++++++--- 3 files changed, 77 insertions(+), 9 deletions(-) diff --git a/src/common/base/src/runtime/perf/query_perf.rs b/src/common/base/src/runtime/perf/query_perf.rs index 1d20c387fab9d..15084a3350b0c 100644 --- a/src/common/base/src/runtime/perf/query_perf.rs +++ b/src/common/base/src/runtime/perf/query_perf.rs @@ -54,7 +54,7 @@ impl QueryPerf { .blocklist(&["libc", "libgcc", "pthread", "vdso"]) .set_filter_func(filter_closure) .build() - .map_err(|_e| ErrorCode::Internal("Failed to create profiler"))?; + .map_err(|e| ErrorCode::Internal(format!("Failed to create profiler, {e}")))?; debug!("starting perf with frequency: {}", frequency); let mut payload = ThreadTracker::new_tracking_payload(); payload.perf_enabled = true; diff --git a/src/query/service/src/physical_plans/physical_hash_join.rs b/src/query/service/src/physical_plans/physical_hash_join.rs index fdeacfae28117..873e48d923c86 100644 --- a/src/query/service/src/physical_plans/physical_hash_join.rs +++ b/src/query/service/src/physical_plans/physical_hash_join.rs @@ -1219,7 +1219,21 @@ impl PhysicalPlanBuilder { let mut predicates = Vec::with_capacity(join.equi_conditions.len() + join.non_equi_conditions.len()); + let is_simple_expr = |expr: &ScalarExpr| { + matches!( + expr, + ScalarExpr::BoundColumnRef(_) + | ScalarExpr::ConstantExpr(_) + | ScalarExpr::TypedConstantExpr(_, _) + ) + }; + for condition in &join.equi_conditions { + if !is_simple_expr(&condition.left) || !is_simple_expr(&condition.right) { + // todo: Filtering after cross join cause expression to be evaluated multiple times + return Ok(None); + } + let scalar = condition_to_expr(condition)?; match resolve_scalar(&scalar, &merged) { Ok(expr) => predicates.push(expr), diff --git a/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/nested_loop.rs b/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/nested_loop.rs index ce157e2622516..8fda0798c9627 100644 --- a/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/nested_loop.rs +++ b/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/nested_loop.rs @@ -69,7 +69,12 @@ impl Join for NestedLoopJoin { return self.inner.probe_block(data); }; - let nested = Box::new(LoopJoinStream::new(data, build_blocks)); + let nested: Box = if data.num_rows() <= *self.basic_state.build_rows { + Box::new(ConstProbeLoopJoinStream::new(data, build_blocks)) + } else { + Box::new(ConstBuildLoopJoinStream::new(data, build_blocks)) + }; + Ok(InnerHashJoinFilterStream::create( nested, &mut self.desc.filter, @@ -78,15 +83,15 @@ impl Join for NestedLoopJoin { } } -struct LoopJoinStream<'a> { +struct ConstProbeLoopJoinStream<'a> { probe_rows: VecDeque>, probe_types: Vec, build_blocks: &'a [DataBlock], build_index: usize, } -impl<'a> LoopJoinStream<'a> { - pub fn new(probe: DataBlock, build_blocks: &'a [DataBlock]) -> Self { +impl<'a> ConstProbeLoopJoinStream<'a> { + fn new(probe: DataBlock, build_blocks: &'a [DataBlock]) -> Self { let mut probe_rows = vec![Vec::new(); probe.num_rows()]; for entry in probe.columns().iter() { match entry { @@ -103,22 +108,22 @@ impl<'a> LoopJoinStream<'a> { } } - let left_types = probe + let probe_types = probe .columns() .iter() .map(|entry| entry.data_type()) .collect(); - LoopJoinStream { + ConstProbeLoopJoinStream { probe_rows: probe_rows.into(), - probe_types: left_types, + probe_types, build_blocks, build_index: 0, } } } -impl<'a> JoinStream for LoopJoinStream<'a> { +impl<'a> JoinStream for ConstProbeLoopJoinStream<'a> { fn next(&mut self) -> Result> { let Some(probe_entries) = self.probe_rows.front() else { return Ok(None); @@ -144,3 +149,52 @@ impl<'a> JoinStream for LoopJoinStream<'a> { Ok(Some(DataBlock::new(entries, build_block.num_rows()))) } } + +struct ConstBuildLoopJoinStream<'a> { + probe: DataBlock, + build_blocks: &'a [DataBlock], + block_index: usize, + row_index: usize, +} + +impl<'a> ConstBuildLoopJoinStream<'a> { + fn new(probe: DataBlock, build_blocks: &'a [DataBlock]) -> Self { + ConstBuildLoopJoinStream { + probe, + build_blocks, + block_index: 0, + row_index: 0, + } + } +} + +impl<'a> JoinStream for ConstBuildLoopJoinStream<'a> { + fn next(&mut self) -> Result> { + let Some(build_block) = self.build_blocks.get(self.block_index) else { + return Ok(None); + }; + + let num_rows = self.probe.num_rows(); + let entries = self + .probe + .columns() + .iter() + .cloned() + .chain(build_block.columns().iter().map(|entry| { + BlockEntry::Const( + entry.index(self.row_index).unwrap().to_owned(), + entry.data_type(), + num_rows, + ) + })) + .collect(); + + self.row_index += 1; + if self.row_index >= build_block.num_rows() { + self.row_index = 0; + self.block_index += 1; + } + + Ok(Some(DataBlock::new(entries, num_rows))) + } +} From 07fcd448802b005961d6d875cf9f940b496561bb Mon Sep 17 00:00:00 2001 From: coldWater Date: Tue, 25 Nov 2025 12:31:05 +0800 Subject: [PATCH 12/19] update --- .../transforms/new_hash_join/memory/basic.rs | 8 +- .../new_hash_join/memory/nested_loop.rs | 257 +++++++++--------- 2 files changed, 127 insertions(+), 138 deletions(-) diff --git a/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/basic.rs b/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/basic.rs index e110d20eca480..babd8dc28a226 100644 --- a/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/basic.rs +++ b/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/basic.rs @@ -153,11 +153,9 @@ impl BasicHashJoin { return; } if let Some(block) = self.state.chunks.first() { - let column_type = self.state.column_types.as_mut(); - column_type.extend( - (0..self.desc.build_projection.len()) - .map(|offset| block.get_by_offset(offset).data_type()), - ); + *self.state.column_types.as_mut() = (0..self.desc.build_projection.len()) + .map(|offset| block.get_by_offset(offset).data_type()) + .collect(); } else { return; }; diff --git a/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/nested_loop.rs b/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/nested_loop.rs index 8fda0798c9627..4fbe861f20730 100644 --- a/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/nested_loop.rs +++ b/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/nested_loop.rs @@ -12,18 +12,19 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::VecDeque; use std::sync::Arc; +use std::sync::PoisonError; use databend_common_base::base::ProgressValues; +use databend_common_column::bitmap::MutableBitmap; 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_hashtable::RowPtr; -use super::inner_join::InnerHashJoinFilterStream; use crate::pipelines::processors::transforms::new_hash_join::join::EmptyJoinStream; +use crate::pipelines::processors::transforms::new_hash_join::join::OneBlockJoinStream; use crate::pipelines::processors::transforms::BasicHashJoinState; use crate::pipelines::processors::transforms::HashJoinHashTable; use crate::pipelines::processors::transforms::Join; @@ -34,167 +35,157 @@ use crate::pipelines::processors::transforms::RuntimeFiltersDesc; pub struct NestedLoopJoin { inner: T, - basic_state: Arc, + state: Arc, desc: NestedLoopDesc, } impl NestedLoopJoin { - pub fn create(inner: T, basic_state: Arc, desc: NestedLoopDesc) -> Self { - Self { - inner, - basic_state, - desc, - } - } -} - -impl Join for NestedLoopJoin { - fn add_block(&mut self, data: Option) -> Result<()> { - self.inner.add_block(data) + pub fn create(inner: T, state: Arc, desc: NestedLoopDesc) -> Self { + Self { inner, state, desc } } - fn final_build(&mut self) -> Result> { - self.inner.final_build() - } + fn finalize_chunks(&mut self) { + if !self.state.columns.is_empty() { + return; + } - fn build_runtime_filter(&self, desc: &RuntimeFiltersDesc) -> Result { - self.inner.build_runtime_filter(desc) - } + let locked = self.state.mutex.lock(); + let _locked = locked.unwrap_or_else(PoisonError::into_inner); - fn probe_block(&mut self, data: DataBlock) -> Result> { - if data.is_empty() || *self.basic_state.build_rows == 0 { - return Ok(Box::new(EmptyJoinStream)); + if !self.state.columns.is_empty() { + return; } - let HashJoinHashTable::NestedLoop(build_blocks) = &*self.basic_state.hash_table else { - return self.inner.probe_block(data); - }; - - let nested: Box = if data.num_rows() <= *self.basic_state.build_rows { - Box::new(ConstProbeLoopJoinStream::new(data, build_blocks)) + let num_columns = if let Some(block) = self.state.chunks.first() { + *self.state.column_types.as_mut() = block + .columns() + .iter() + .map(|entry| entry.data_type()) + .collect(); + block.num_columns() } else { - Box::new(ConstBuildLoopJoinStream::new(data, build_blocks)) + return; }; - Ok(InnerHashJoinFilterStream::create( - nested, - &mut self.desc.filter, - self.desc.field_reorder.as_deref(), - )) + *self.state.columns.as_mut() = (0..num_columns) + .map(|offset| { + let full_columns = self + .state + .chunks + .iter() + .map(|block| block.get_by_offset(offset).to_column()) + .collect::>(); + + Column::take_downcast_column_vec(&full_columns) + }) + .collect(); } -} -struct ConstProbeLoopJoinStream<'a> { - probe_rows: VecDeque>, - probe_types: Vec, - build_blocks: &'a [DataBlock], - build_index: usize, -} + fn handle_block(&mut self, data: DataBlock) -> Result> { + let HashJoinHashTable::NestedLoop(build_blocks) = &*self.state.hash_table else { + unreachable!() + }; -impl<'a> ConstProbeLoopJoinStream<'a> { - fn new(probe: DataBlock, build_blocks: &'a [DataBlock]) -> Self { - let mut probe_rows = vec![Vec::new(); probe.num_rows()]; - for entry in probe.columns().iter() { - match entry { - BlockEntry::Const(scalar, _, _) => { - for row in &mut probe_rows { - row.push(scalar.to_owned()); - } - } - BlockEntry::Column(column) => { - for (row, scalar) in probe_rows.iter_mut().zip(column.iter()) { - row.push(scalar.to_owned()); - } - } + let probe_rows = data.num_rows(); + let mut matched = Vec::with_capacity(probe_rows); + for (chunk_index, build) in build_blocks.iter().enumerate() { + for row_index in 0..build.num_rows() { + let entries = data + .columns() + .iter() + .cloned() + .chain(build.columns().iter().map(|entry| { + BlockEntry::Const( + entry.index(row_index).unwrap().to_owned(), + entry.data_type(), + probe_rows, + ) + })) + .collect(); + let result_count = self + .desc + .filter + .select(&DataBlock::new(entries, probe_rows))?; + + matched.extend( + self.desc.filter.true_selection()[..result_count] + .iter() + .copied() + .map(|probe| { + (probe, RowPtr { + chunk_index: chunk_index as _, + row_index: row_index as _, + }) + }), + ); } } - let probe_types = probe - .columns() - .iter() - .map(|entry| entry.data_type()) - .collect(); - - ConstProbeLoopJoinStream { - probe_rows: probe_rows.into(), - probe_types, - build_blocks, - build_index: 0, + if matched.is_empty() { + return Ok(None); } - } -} -impl<'a> JoinStream for ConstProbeLoopJoinStream<'a> { - fn next(&mut self) -> Result> { - let Some(probe_entries) = self.probe_rows.front() else { - return Ok(None); - }; + let mut bitmap = MutableBitmap::with_capacity(matched.len()); + for (i, _) in &matched { + bitmap.set(*i as _, true); + } + let probe = data.filter_with_bitmap(&bitmap.freeze())?; - let build_block = &self.build_blocks[self.build_index]; + matched.sort_by_key(|(v, _)| *v); + let indices = matched.into_iter().map(|(_, row)| row).collect::>(); - let entries = probe_entries + let build_entries = self + .state + .columns .iter() - .zip(self.probe_types.iter()) - .map(|(scalar, data_type)| { - BlockEntry::Const(scalar.clone(), data_type.clone(), build_block.num_rows()) - }) - .chain(build_block.columns().iter().cloned()) - .collect(); - - self.build_index += 1; - if self.build_index >= self.build_blocks.len() { - self.build_index = 0; - self.probe_rows.pop_front(); - } - - Ok(Some(DataBlock::new(entries, build_block.num_rows()))) + .zip(&*self.state.column_types) + .map(|(columns, data_type)| { + Column::take_column_vec_indices(columns, data_type.clone(), &indices, indices.len()) + .into() + }); + + let data_block = DataBlock::from_iter( + probe.take_columns().into_iter().chain(build_entries), + indices.len(), + ); + + let Some(field_reorder) = &self.desc.field_reorder else { + return Ok(Some(data_block)); + }; + let data_block = DataBlock::from_iter( + field_reorder + .iter() + .map(|offset| data_block.get_by_offset(*offset).clone()), + data_block.num_rows(), + ); + + Ok(Some(data_block)) } } -struct ConstBuildLoopJoinStream<'a> { - probe: DataBlock, - build_blocks: &'a [DataBlock], - block_index: usize, - row_index: usize, -} +impl Join for NestedLoopJoin { + fn add_block(&mut self, data: Option) -> Result<()> { + self.inner.add_block(data) + } -impl<'a> ConstBuildLoopJoinStream<'a> { - fn new(probe: DataBlock, build_blocks: &'a [DataBlock]) -> Self { - ConstBuildLoopJoinStream { - probe, - build_blocks, - block_index: 0, - row_index: 0, - } + fn final_build(&mut self) -> Result> { + self.inner.final_build() } -} -impl<'a> JoinStream for ConstBuildLoopJoinStream<'a> { - fn next(&mut self) -> Result> { - let Some(build_block) = self.build_blocks.get(self.block_index) else { - return Ok(None); - }; + fn build_runtime_filter(&self, desc: &RuntimeFiltersDesc) -> Result { + self.inner.build_runtime_filter(desc) + } - let num_rows = self.probe.num_rows(); - let entries = self - .probe - .columns() - .iter() - .cloned() - .chain(build_block.columns().iter().map(|entry| { - BlockEntry::Const( - entry.index(self.row_index).unwrap().to_owned(), - entry.data_type(), - num_rows, - ) - })) - .collect(); + fn probe_block(&mut self, data: DataBlock) -> Result> { + if data.is_empty() || *self.state.build_rows == 0 { + return Ok(Box::new(EmptyJoinStream)); + } - self.row_index += 1; - if self.row_index >= build_block.num_rows() { - self.row_index = 0; - self.block_index += 1; + if !matches!(*self.state.hash_table, HashJoinHashTable::NestedLoop(_)) { + return self.inner.probe_block(data); } - Ok(Some(DataBlock::new(entries, num_rows))) + self.finalize_chunks(); + + Ok(Box::new(OneBlockJoinStream(self.handle_block(data)?))) } } From 166832ca66d44db822e402f1697ad80c8a9b1619 Mon Sep 17 00:00:00 2001 From: coldWater Date: Tue, 25 Nov 2025 14:49:59 +0800 Subject: [PATCH 13/19] update --- src/query/expression/src/block.rs | 5 +- .../expression/src/filter/filter_executor.rs | 4 + .../processors/transforms/hash_join/desc.rs | 4 +- .../new_hash_join/hash_join_factory.rs | 2 +- .../new_hash_join/memory/nested_loop.rs | 224 +++++++++++------- 5 files changed, 146 insertions(+), 93 deletions(-) diff --git a/src/query/expression/src/block.rs b/src/query/expression/src/block.rs index c8322beebd795..2f4bb95062ca7 100644 --- a/src/query/expression/src/block.rs +++ b/src/query/expression/src/block.rs @@ -858,10 +858,9 @@ impl DataBlock { pub fn project(mut self, projections: &ColumnSet) -> Self { let mut entries = Vec::with_capacity(projections.len()); for (index, column) in self.entries.into_iter().enumerate() { - if !projections.contains(&index) { - continue; + if projections.contains(&index) { + entries.push(column); } - entries.push(column); } self.entries = entries; self diff --git a/src/query/expression/src/filter/filter_executor.rs b/src/query/expression/src/filter/filter_executor.rs index e9011f72ea03d..a365e375a6c15 100644 --- a/src/query/expression/src/filter/filter_executor.rs +++ b/src/query/expression/src/filter/filter_executor.rs @@ -208,4 +208,8 @@ impl FilterExecutor { pub fn mutable_true_selection(&mut self) -> &mut [u32] { &mut self.true_selection } + + pub fn max_block_size(&self) -> usize { + self.max_block_size + } } 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 a0b09a97e49bf..3f910af5c0ca4 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 @@ -310,10 +310,11 @@ impl HashJoinDesc { predicates, function_ctx.clone(), block_size, - Some(projection.iter().copied().collect()), + None, &BUILTIN_FUNCTIONS, false, ), + projections: projection.iter().copied().collect(), field_reorder, nested_loop_join_threshold, })) @@ -323,5 +324,6 @@ impl HashJoinDesc { pub struct NestedLoopDesc { pub filter: FilterExecutor, pub field_reorder: Option>, + pub projections: ColumnSet, pub nested_loop_join_threshold: usize, } diff --git a/src/query/service/src/pipelines/processors/transforms/new_hash_join/hash_join_factory.rs b/src/query/service/src/pipelines/processors/transforms/new_hash_join/hash_join_factory.rs index c643526d15156..ad75c32761383 100644 --- a/src/query/service/src/pipelines/processors/transforms/new_hash_join/hash_join_factory.rs +++ b/src/query/service/src/pipelines/processors/transforms/new_hash_join/hash_join_factory.rs @@ -146,7 +146,7 @@ impl HashJoinFactory { )?; match nested_loop_desc { - Some(desc) => Ok(Box::new(NestedLoopJoin::create(inner, state, desc))), + Some(desc) => Ok(Box::new(NestedLoopJoin::new(inner, state, desc))), None => Ok(Box::new(inner)), } } diff --git a/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/nested_loop.rs b/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/nested_loop.rs index 4fbe861f20730..1727df0b54145 100644 --- a/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/nested_loop.rs +++ b/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/nested_loop.rs @@ -16,7 +16,6 @@ use std::sync::Arc; use std::sync::PoisonError; use databend_common_base::base::ProgressValues; -use databend_common_column::bitmap::MutableBitmap; use databend_common_exception::Result; use databend_common_expression::BlockEntry; use databend_common_expression::Column; @@ -24,7 +23,6 @@ use databend_common_expression::DataBlock; use databend_common_hashtable::RowPtr; use crate::pipelines::processors::transforms::new_hash_join::join::EmptyJoinStream; -use crate::pipelines::processors::transforms::new_hash_join::join::OneBlockJoinStream; use crate::pipelines::processors::transforms::BasicHashJoinState; use crate::pipelines::processors::transforms::HashJoinHashTable; use crate::pipelines::processors::transforms::Join; @@ -40,11 +38,11 @@ pub struct NestedLoopJoin { } impl NestedLoopJoin { - pub fn create(inner: T, state: Arc, desc: NestedLoopDesc) -> Self { + pub fn new(inner: T, state: Arc, desc: NestedLoopDesc) -> Self { Self { inner, state, desc } } - fn finalize_chunks(&mut self) { + fn finalize_chunks(&self) { if !self.state.columns.is_empty() { return; } @@ -79,87 +77,6 @@ impl NestedLoopJoin { }) .collect(); } - - fn handle_block(&mut self, data: DataBlock) -> Result> { - let HashJoinHashTable::NestedLoop(build_blocks) = &*self.state.hash_table else { - unreachable!() - }; - - let probe_rows = data.num_rows(); - let mut matched = Vec::with_capacity(probe_rows); - for (chunk_index, build) in build_blocks.iter().enumerate() { - for row_index in 0..build.num_rows() { - let entries = data - .columns() - .iter() - .cloned() - .chain(build.columns().iter().map(|entry| { - BlockEntry::Const( - entry.index(row_index).unwrap().to_owned(), - entry.data_type(), - probe_rows, - ) - })) - .collect(); - let result_count = self - .desc - .filter - .select(&DataBlock::new(entries, probe_rows))?; - - matched.extend( - self.desc.filter.true_selection()[..result_count] - .iter() - .copied() - .map(|probe| { - (probe, RowPtr { - chunk_index: chunk_index as _, - row_index: row_index as _, - }) - }), - ); - } - } - - if matched.is_empty() { - return Ok(None); - } - - let mut bitmap = MutableBitmap::with_capacity(matched.len()); - for (i, _) in &matched { - bitmap.set(*i as _, true); - } - let probe = data.filter_with_bitmap(&bitmap.freeze())?; - - matched.sort_by_key(|(v, _)| *v); - let indices = matched.into_iter().map(|(_, row)| row).collect::>(); - - let build_entries = self - .state - .columns - .iter() - .zip(&*self.state.column_types) - .map(|(columns, data_type)| { - Column::take_column_vec_indices(columns, data_type.clone(), &indices, indices.len()) - .into() - }); - - let data_block = DataBlock::from_iter( - probe.take_columns().into_iter().chain(build_entries), - indices.len(), - ); - - let Some(field_reorder) = &self.desc.field_reorder else { - return Ok(Some(data_block)); - }; - let data_block = DataBlock::from_iter( - field_reorder - .iter() - .map(|offset| data_block.get_by_offset(*offset).clone()), - data_block.num_rows(), - ); - - Ok(Some(data_block)) - } } impl Join for NestedLoopJoin { @@ -180,12 +97,143 @@ impl Join for NestedLoopJoin { return Ok(Box::new(EmptyJoinStream)); } - if !matches!(*self.state.hash_table, HashJoinHashTable::NestedLoop(_)) { + let HashJoinHashTable::NestedLoop(build_blocks) = &*self.state.hash_table else { return self.inner.probe_block(data); + }; + self.finalize_chunks(); + + let max_block_size = self.desc.filter.max_block_size(); + Ok(Box::new(NestedLoopJoinStream { + probe_block: data, + build_blocks, + state: &self.state, + max_block_size, + desc: &mut self.desc, + matches: Vec::with_capacity(max_block_size), + build_block_index: 0, + build_row_index: 0, + })) + } +} + +struct NestedLoopJoinStream<'a> { + probe_block: DataBlock, + build_blocks: &'a [DataBlock], + state: &'a BasicHashJoinState, + desc: &'a mut NestedLoopDesc, + max_block_size: usize, + build_block_index: usize, + build_row_index: usize, + matches: Vec<(u32, RowPtr)>, +} + +impl<'a> NestedLoopJoinStream<'a> { + fn process_next_row(&mut self) -> Result<()> { + let build_block = &self.build_blocks[self.build_block_index]; + + let probe_rows = self.probe_block.num_rows(); + let entries = self + .probe_block + .columns() + .iter() + .cloned() + .chain(build_block.columns().iter().map(|entry| { + BlockEntry::Const( + entry.index(self.build_row_index).unwrap().to_owned(), + entry.data_type(), + probe_rows, + ) + })) + .collect(); + + let result_count = self + .desc + .filter + .select(&DataBlock::new(entries, probe_rows))?; + let row_ptr = RowPtr { + chunk_index: self.build_block_index as u32, + row_index: self.build_row_index as u32, + }; + self.matches.extend( + self.desc.filter.true_selection()[..result_count] + .iter() + .map(|probe| (*probe, row_ptr)), + ); + + self.build_row_index += 1; + if self.build_row_index >= build_block.num_rows() { + self.build_row_index = 0; + self.build_block_index += 1; } - self.finalize_chunks(); + Ok(()) + } + + fn emit_block(&mut self, count: usize) -> Result { + self.matches.sort_by_key(|(probe, _)| *probe); + + let block = { + let (probe_indices, build_indices): (Vec<_>, Vec<_>) = + self.matches.drain(..count).unzip(); + + let probe = self + .probe_block + .clone() + .project(&self.desc.projections) + .take(&probe_indices)?; + + let build_entries = self + .state + .columns + .iter() + .zip(self.state.column_types.as_slice()) + .enumerate() + .filter_map(|(i, x)| { + let i = self.probe_block.num_columns() + i; + self.desc.projections.contains(&i).then_some(x) + }) + .map(|(columns, data_type)| { + Column::take_column_vec_indices( + columns, + data_type.clone(), + &build_indices, + count, + ) + .into() + }); + + DataBlock::from_iter(probe.take_columns().into_iter().chain(build_entries), count) + }; - Ok(Box::new(OneBlockJoinStream(self.handle_block(data)?))) + if let Some(field_reorder) = &self.desc.field_reorder { + Ok(DataBlock::from_iter( + field_reorder + .iter() + .map(|offset| block.get_by_offset(*offset).clone()), + block.num_rows(), + )) + } else { + Ok(block) + } + } +} + +impl<'a> JoinStream for NestedLoopJoinStream<'a> { + fn next(&mut self) -> Result> { + loop { + if self.matches.len() >= self.max_block_size { + return Ok(Some(self.emit_block(self.max_block_size)?)); + } + + if self.build_block_index >= self.build_blocks.len() { + return if self.matches.is_empty() { + Ok(None) + } else { + Ok(Some(self.emit_block(self.matches.len())?)) + }; + } + + self.process_next_row()?; + } } } From 83218552ece4eb71281e0e8369d891093ee60bee Mon Sep 17 00:00:00 2001 From: coldWater Date: Thu, 27 Nov 2025 15:01:30 +0800 Subject: [PATCH 14/19] fix --- .../transforms/new_hash_join/memory/basic.rs | 23 +++++++++++-------- 1 file changed, 14 insertions(+), 9 deletions(-) diff --git a/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/basic.rs b/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/basic.rs index babd8dc28a226..e2d755d8425ae 100644 --- a/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/basic.rs +++ b/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/basic.rs @@ -89,9 +89,15 @@ impl BasicHashJoin { } pub(crate) fn final_build(&mut self) -> Result> { - if let Some(true) = self.init_memory_hash_table() { - return Ok(Some(self.build_nested_loop())); - }; + match self.state.hash_table.deref() { + HashJoinHashTable::Null => match self.init_memory_hash_table() { + Some(true) => return Ok(Some(self.build_nested_loop())), + Some(false) => return Ok(None), + None => {} + }, + HashJoinHashTable::NestedLoop(_) => return Ok(None), + _ => {} + } let Some(chunk_index) = self.state.steal_chunk_index() else { return Ok(None); @@ -176,16 +182,15 @@ impl BasicHashJoin { } fn init_memory_hash_table(&mut self) -> Option { - if !matches!(self.state.hash_table.deref(), HashJoinHashTable::Null) { - return None; - } let skip_duplicates = matches!(self.desc.join_type, JoinType::InnerAny | JoinType::LeftAny); let locked = self.state.mutex.lock(); let _locked = locked.unwrap_or_else(PoisonError::into_inner); - if !matches!(self.state.hash_table.deref(), HashJoinHashTable::Null) { - return None; + match self.state.hash_table.deref() { + HashJoinHashTable::Null => {} + HashJoinHashTable::NestedLoop(_) => return Some(false), + _ => return None, } let build_num_rows = *self.state.build_rows.deref(); @@ -304,7 +309,7 @@ impl BasicHashJoin { ) } }; - Some(false) + None } fn build_hash_table(&self, keys: DataBlock, chunk_idx: usize) -> Result<()> { From 0d4339fdf5e9d4180be07901b24206ce9faa0f75 Mon Sep 17 00:00:00 2001 From: coldWater Date: Thu, 27 Nov 2025 17:46:46 +0800 Subject: [PATCH 15/19] use range take --- .../expression/src/kernels/take_ranges.rs | 15 +++++++++++++++ .../new_hash_join/memory/nested_loop.rs | 19 ++++++++++++------- 2 files changed, 27 insertions(+), 7 deletions(-) diff --git a/src/query/expression/src/kernels/take_ranges.rs b/src/query/expression/src/kernels/take_ranges.rs index e11ae845560a6..b214965dfae37 100644 --- a/src/query/expression/src/kernels/take_ranges.rs +++ b/src/query/expression/src/kernels/take_ranges.rs @@ -60,6 +60,21 @@ impl DataBlock { self.get_meta().cloned(), )) } + + pub fn merge_indices_to_ranges(indices: &[u32]) -> Vec> { + debug_assert!(indices.is_sorted()); + let mut ranges: Vec> = Vec::with_capacity(indices.len() / 2); + for &index in indices { + if let Some(cur) = ranges.last_mut() + && cur.end == index + { + cur.end += 1; + } else { + ranges.push(index..index + 1) + } + } + ranges + } } struct TakeRangeVisitor<'a> { diff --git a/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/nested_loop.rs b/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/nested_loop.rs index 1727df0b54145..389ce1bbbf318 100644 --- a/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/nested_loop.rs +++ b/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/nested_loop.rs @@ -20,6 +20,7 @@ use databend_common_exception::Result; use databend_common_expression::BlockEntry; use databend_common_expression::Column; use databend_common_expression::DataBlock; +use databend_common_expression::SELECTIVITY_THRESHOLD; use databend_common_hashtable::RowPtr; use crate::pipelines::processors::transforms::new_hash_join::join::EmptyJoinStream; @@ -170,17 +171,21 @@ impl<'a> NestedLoopJoinStream<'a> { } fn emit_block(&mut self, count: usize) -> Result { - self.matches.sort_by_key(|(probe, _)| *probe); - + let use_range = count as f64 > SELECTIVITY_THRESHOLD * self.max_block_size as f64; let block = { + if use_range { + self.matches.sort_unstable_by_key(|(probe, _)| *probe); + } let (probe_indices, build_indices): (Vec<_>, Vec<_>) = self.matches.drain(..count).unzip(); - let probe = self - .probe_block - .clone() - .project(&self.desc.projections) - .take(&probe_indices)?; + let probe = self.probe_block.clone().project(&self.desc.projections); + let probe = if use_range { + let ranges = DataBlock::merge_indices_to_ranges(&probe_indices); + probe.take_ranges(&ranges, count)? + } else { + probe.take_with_optimize_size(&probe_indices)? + }; let build_entries = self .state From 591d4f65bcb69dddd357bb9d95aaae29485e56c5 Mon Sep 17 00:00:00 2001 From: coldWater Date: Thu, 27 Nov 2025 20:21:15 +0800 Subject: [PATCH 16/19] update --- .../new_hash_join/memory/nested_loop.rs | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/nested_loop.rs b/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/nested_loop.rs index 389ce1bbbf318..9929d64d8228a 100644 --- a/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/nested_loop.rs +++ b/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/nested_loop.rs @@ -113,6 +113,7 @@ impl Join for NestedLoopJoin { matches: Vec::with_capacity(max_block_size), build_block_index: 0, build_row_index: 0, + use_range: false, })) } } @@ -126,6 +127,7 @@ struct NestedLoopJoinStream<'a> { build_block_index: usize, build_row_index: usize, matches: Vec<(u32, RowPtr)>, + use_range: bool, } impl<'a> NestedLoopJoinStream<'a> { @@ -171,16 +173,23 @@ impl<'a> NestedLoopJoinStream<'a> { } fn emit_block(&mut self, count: usize) -> Result { - let use_range = count as f64 > SELECTIVITY_THRESHOLD * self.max_block_size as f64; + if !self.use_range + && self.matches.len() as f64 + > SELECTIVITY_THRESHOLD * self.probe_block.num_rows() as f64 + { + // Need to test the scenario where a probe matches multiple builds + self.use_range = true; + } + let block = { - if use_range { + if self.use_range { self.matches.sort_unstable_by_key(|(probe, _)| *probe); } let (probe_indices, build_indices): (Vec<_>, Vec<_>) = self.matches.drain(..count).unzip(); let probe = self.probe_block.clone().project(&self.desc.projections); - let probe = if use_range { + let probe = if self.use_range { let ranges = DataBlock::merge_indices_to_ranges(&probe_indices); probe.take_ranges(&ranges, count)? } else { From a0766b2706356391606b52d99f46e8c23dec7bd9 Mon Sep 17 00:00:00 2001 From: coldWater Date: Fri, 28 Nov 2025 09:12:33 +0800 Subject: [PATCH 17/19] settings --- src/query/settings/src/settings_default.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/query/settings/src/settings_default.rs b/src/query/settings/src/settings_default.rs index a31ad1c1dc6d0..29c2ac09fe272 100644 --- a/src/query/settings/src/settings_default.rs +++ b/src/query/settings/src/settings_default.rs @@ -496,7 +496,7 @@ impl DefaultSettings { range: Some(SettingRange::Numeric(0..=u64::MAX)), }), ("nested_loop_join_threshold", DefaultSettingValue { - value: UserSettingValue::UInt64(1024), + value: UserSettingValue::UInt64(10000), desc: "Set the threshold for use nested loop join. Setting it to 0 disable nested loop join.", mode: SettingMode::Both, scope: SettingScope::Both, @@ -1498,7 +1498,7 @@ impl DefaultSettings { range: Some(SettingRange::Numeric(0..=1)), }), ("enable_experimental_new_join", DefaultSettingValue { - value: UserSettingValue::UInt64(0), + value: UserSettingValue::UInt64(1), desc: "Enables the experimental new join implement", mode: SettingMode::Both, scope: SettingScope::Both, From e50390e2049654eda21256c399b77f1b6c1bead4 Mon Sep 17 00:00:00 2001 From: coldWater Date: Fri, 28 Nov 2025 16:40:58 +0800 Subject: [PATCH 18/19] fix --- .../processors/transforms/hash_join/desc.rs | 9 +- .../test_full_outer_join_issue_4252.test | 4 +- .../join/iejoin/test_iejoin_east_west.test | 8 +- .../sqllogictests/suites/query/join/join.test | 24 +-- .../suites/query/join/large_query.test | 202 ++++-------------- tests/sqllogictests/suites/tpch/join.test | 8 +- 6 files changed, 70 insertions(+), 185 deletions(-) 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 3f910af5c0ca4..5eee7bd77acc7 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 @@ -43,11 +43,13 @@ pub const MARKER_KIND_TRUE: u8 = 0; pub const MARKER_KIND_FALSE: u8 = 1; pub const MARKER_KIND_NULL: u8 = 2; +#[derive(Debug)] pub struct MarkJoinDesc { // pub(crate) marker_index: Option, pub(crate) has_null: RwLock, } +#[derive(Debug)] pub struct HashJoinDesc { pub(crate) build_keys: Vec, pub(crate) probe_keys: Vec, @@ -81,6 +83,7 @@ pub struct RuntimeFilterDesc { pub enable_min_max_runtime_filter: bool, } +#[derive(Debug)] pub(crate) struct RuntimeFiltersDesc { pub filters: Vec, } @@ -299,7 +302,9 @@ impl HashJoinDesc { let field_reorder = if !projection.is_sorted() { let mut mapper = projection.iter().cloned().enumerate().collect::>(); mapper.sort_by_key(|(_, field)| *field); - let reorder = mapper.iter().map(|(i, _)| *i).collect::>(); + let reorder = (0..projection.len()) + .map(|j| mapper.iter().position(|(i, _)| *i == j).unwrap()) + .collect::>(); Some(reorder) } else { None @@ -323,7 +328,7 @@ impl HashJoinDesc { pub struct NestedLoopDesc { pub filter: FilterExecutor, - pub field_reorder: Option>, pub projections: ColumnSet, + pub field_reorder: Option>, pub nested_loop_join_threshold: usize, } diff --git a/tests/sqllogictests/suites/duckdb/join/full_outer/test_full_outer_join_issue_4252.test b/tests/sqllogictests/suites/duckdb/join/full_outer/test_full_outer_join_issue_4252.test index 37c4131f55009..66195dfe18dce 100644 --- a/tests/sqllogictests/suites/duckdb/join/full_outer/test_full_outer_join_issue_4252.test +++ b/tests/sqllogictests/suites/duckdb/join/full_outer/test_full_outer_join_issue_4252.test @@ -48,7 +48,7 @@ INSERT INTO df3 VALUES ('2022-02-03', 2000, 'org2'), ('2022-04-01', 3000, 'org3'); -query II +query ?R SELECT coalesce(anon_1.month, anon_2.month) AS month, coalesce(coalesce(CAST(anon_1.value AS float), 0.0) + coalesce(CAST(anon_2.value AS float), 0.0), 0.0) AS value @@ -93,7 +93,7 @@ FULL OUTER JOIN ( ---- 2022-01-01 1100.0 -query II +query ?R SELECT coalesce(anon_1.month, anon_2.month) AS month, coalesce(coalesce(CAST(anon_1.value AS float), 0.0) + coalesce(CAST(anon_2.value AS float), 0.0), 0.0) AS value diff --git a/tests/sqllogictests/suites/duckdb/join/iejoin/test_iejoin_east_west.test b/tests/sqllogictests/suites/duckdb/join/iejoin/test_iejoin_east_west.test index b193a7c74faf3..f7f9918cad0fc 100644 --- a/tests/sqllogictests/suites/duckdb/join/iejoin/test_iejoin_east_west.test +++ b/tests/sqllogictests/suites/duckdb/join/iejoin/test_iejoin_east_west.test @@ -21,7 +21,7 @@ CREATE TABLE west AS SELECT * FROM (VALUES ) west(rid, t_id, time, cost, cores) # Qs -query II +query TT SELECT s1.rid, s2.rid FROM west s1, west s2 WHERE s1.time > s2.time @@ -35,7 +35,7 @@ s2 s4 s4 s3 # Qp -query II +query TT SELECT s1.rid, s2.rid FROM west s1, west s2 WHERE s1.time > s2.time AND s1.cost < s2.cost @@ -45,7 +45,7 @@ s1 s3 s4 s3 # Qt -query II +query TT SELECT east.rid, west.rid FROM east, west WHERE east.dur < west.time AND east.rev > west.cost @@ -54,7 +54,7 @@ ORDER BY 1, 2 r2 s2 # Test string comparisons -query II +query TT WITH weststr AS ( SELECT rid, time::VARCHAR AS time, cost::VARCHAR as cost FROM west diff --git a/tests/sqllogictests/suites/query/join/join.test b/tests/sqllogictests/suites/query/join/join.test index 5b58536f71ba1..b68c0cfa5e47a 100644 --- a/tests/sqllogictests/suites/query/join/join.test +++ b/tests/sqllogictests/suites/query/join/join.test @@ -15,17 +15,17 @@ select * from (select * from numbers(100)) n join t1 on n.number = t1.a; ---- # right semi with empty build side -query II +query I select * from (select * from numbers(100)) n right semi join t1 on n.number = t1.a; ---- # right anti with empty build side -query II +query I select * from (select * from numbers(100)) n right anti join t1 on n.number = t1.a; ---- # left semi with empty build side -query II +query I select * from (select * from numbers(100)) n left semi join t1 on n.number = t1.a; ---- @@ -97,7 +97,7 @@ insert into t1 values(2, 2); statement ok insert into t2 values(2, 6), (2, 8); -query I +query III select t1.a, t2.b, t1.b from t1 inner join t2 on t1.a = t2.a order by t1.a, t2.b, t1.b; ---- 1 2 2 @@ -139,7 +139,7 @@ select t1.id from t1 left join t2 on t1.id = t2.id where t1.val >= t2.val; 2 2 -query I +query II select t1.id, t1.val from t1 left join t2 on t1.id = t2.id and t1.val = t2.val where t1.val >= t2.val; ---- 2 1696549154013 @@ -162,7 +162,7 @@ create or replace table t1(id int, col1 varchar); statement ok insert into t1 values(1, 'c'), (3, 'd'); -query I rowsort +query IITIIT rowsort SELECT * FROM t JOIN t1, t as t2 JOIN t1 as t3; ---- 1 1 c 1 1 c @@ -265,7 +265,7 @@ select * from (select a, 'A' as name from t) t1 full outer join (select number f 2 A 2 3 A 3 -query ITI +query IIT select * from (select number from numbers(5)) t2 full outer join (select a, 'A' as name from t) t1 on t1.a = t2.number where t1.name is not null order by a; ---- 1 1 A @@ -291,15 +291,15 @@ select * from t join t1 on t.a = t1.a 1 1.0 0 2 2.0 1 -query ITB +query ITB rowsort select * from t join t1 on t.a = t1.b ---- -3 1 1 -2 1 1 1 1 1 -3 2.0 1 -2 2.0 1 1 2.0 1 +2 1 1 +2 2.0 1 +3 1 1 +3 2.0 1 statement ok drop table if exists t; diff --git a/tests/sqllogictests/suites/query/join/large_query.test b/tests/sqllogictests/suites/query/join/large_query.test index 14e35ed68676a..6ddac845443ad 100644 --- a/tests/sqllogictests/suites/query/join/large_query.test +++ b/tests/sqllogictests/suites/query/join/large_query.test @@ -1,242 +1,122 @@ statement ok -drop table if exists t1; +create or replace table t1 (id int, c1 int); statement ok -drop table if exists t2; +create or replace table t2 (id int, c1 int); statement ok -drop table if exists t3; +create or replace table t3 (id int, c1 int); statement ok -drop table if exists t4; +create or replace table t4 (id int, c1 int); statement ok -drop table if exists t5; +create or replace table t5 (id int, c1 int); statement ok -drop table if exists t6; +create or replace table t6 (id int, c1 int); statement ok -drop table if exists t7; +create or replace table t7 (id int, c1 int); statement ok -drop table if exists t8; +create or replace table t8 (id int, c1 int); statement ok -drop table if exists t9; +create or replace table t9 (id int, c1 int); statement ok -drop table if exists t10; +create or replace table t10 (id int, c1 int); statement ok -drop table if exists t11; +create or replace table t11 (id int, c1 int); statement ok -drop table if exists t12; +create or replace table t12 (id int, c1 int); statement ok -drop table if exists t13; +create or replace table t13 (id int, c1 int); statement ok -drop table if exists t14; +create or replace table t14 (id int, c1 int); statement ok -drop table if exists t15; +create or replace table t15 (id int, c1 int); statement ok -drop table if exists t16; +create or replace table t16 (id int, c1 int); statement ok -drop table if exists t17; +create or replace table t17 (id int, c1 int); statement ok -drop table if exists t18; +create or replace table t18 (id int, c1 int); statement ok -drop table if exists t19; +create or replace table t19 (id int, c1 int); statement ok -drop table if exists t20; +create or replace table t20 (id int, c1 int); statement ok -drop table if exists t21; +create or replace table t21 (id int, c1 int); statement ok -drop table if exists t22; +create or replace table t22 (id int, c1 int); statement ok -drop table if exists t23; +create or replace table t23 (id int, c1 int); statement ok -drop table if exists t24; +create or replace table t24 (id int, c1 int); statement ok -drop table if exists t25; +create or replace table t25 (id int, c1 int); statement ok -drop table if exists t26; +create or replace table t26 (id int, c1 int); statement ok -drop table if exists t27; +create or replace table t27 (id int, c1 int); statement ok -drop table if exists t28; +create or replace table t28 (id int, c1 int); statement ok -drop table if exists t29; +create or replace table t29 (id int, c1 int); statement ok -drop table if exists t30; +create or replace table t30 (id int, c1 int); statement ok -drop table if exists t31; +create or replace table t31 (id int, c1 int); statement ok -drop table if exists t32; +create or replace table t32 (id int, c1 int); statement ok -drop table if exists t33; +create or replace table t33 (id int, c1 int); statement ok -drop table if exists t34; +create or replace table t34 (id int, c1 int); statement ok -drop table if exists t35; +create or replace table t35 (id int, c1 int); statement ok -drop table if exists t36; +create or replace table t36 (id int, c1 int); statement ok -drop table if exists t37; +create or replace table t37 (id int, c1 int); statement ok -drop table if exists t38; +create or replace table t38 (id int, c1 int); statement ok -drop table if exists t39; +create or replace table t39 (id int, c1 int); statement ok -drop table if exists t40; - -statement ok -create table t1 (id int, c1 int); - -statement ok -create table t2 (id int, c1 int); - -statement ok -create table t3 (id int, c1 int); - -statement ok -create table t4 (id int, c1 int); - -statement ok -create table t5 (id int, c1 int); - -statement ok -create table t6 (id int, c1 int); - -statement ok -create table t7 (id int, c1 int); - -statement ok -create table t8 (id int, c1 int); - -statement ok -create table t9 (id int, c1 int); - -statement ok -create table t10 (id int, c1 int); - -statement ok -create table t11 (id int, c1 int); - -statement ok -create table t12 (id int, c1 int); - -statement ok -create table t13 (id int, c1 int); - -statement ok -create table t14 (id int, c1 int); - -statement ok -create table t15 (id int, c1 int); - -statement ok -create table t16 (id int, c1 int); - -statement ok -create table t17 (id int, c1 int); - -statement ok -create table t18 (id int, c1 int); - -statement ok -create table t19 (id int, c1 int); - -statement ok -create table t20 (id int, c1 int); - -statement ok -create table t21 (id int, c1 int); - -statement ok -create table t22 (id int, c1 int); - -statement ok -create table t23 (id int, c1 int); - -statement ok -create table t24 (id int, c1 int); - -statement ok -create table t25 (id int, c1 int); - -statement ok -create table t26 (id int, c1 int); - -statement ok -create table t27 (id int, c1 int); - -statement ok -create table t28 (id int, c1 int); - -statement ok -create table t29 (id int, c1 int); - -statement ok -create table t30 (id int, c1 int); - -statement ok -create table t31 (id int, c1 int); - -statement ok -create table t32 (id int, c1 int); - -statement ok -create table t33 (id int, c1 int); - -statement ok -create table t34 (id int, c1 int); - -statement ok -create table t35 (id int, c1 int); - -statement ok -create table t36 (id int, c1 int); - -statement ok -create table t37 (id int, c1 int); - -statement ok -create table t38 (id int, c1 int); - -statement ok -create table t39 (id int, c1 int); - -statement ok -create table t40 (id int, c1 int); +create or replace table t40 (id int, c1 int); statement ok insert into t1 (id, c1) VALUES (0, 36); @@ -299,7 +179,7 @@ statement ok insert into t20 (id, c1) VALUES (0, 1); # Large query -query I +query IIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIII select * from t1 join t2 on t1.id = t2.id join t3 on t2.id = t3.id diff --git a/tests/sqllogictests/suites/tpch/join.test b/tests/sqllogictests/suites/tpch/join.test index 63db102dbc85a..2ebb83b6dab07 100644 --- a/tests/sqllogictests/suites/tpch/join.test +++ b/tests/sqllogictests/suites/tpch/join.test @@ -4,7 +4,7 @@ set sandbox_tenant = 'test_tenant'; statement ok use tpch_test; -query I +query II select c_custkey, count(o_orderkey) as c_count from @@ -71,7 +71,7 @@ from 103 103 -query I +query II select c_custkey, count(o_orderkey) as c_count from @@ -107,7 +107,7 @@ order by c_custkey 20 0 -query I +query II select c_custkey, count(o_orderkey) as c_count from @@ -330,7 +330,7 @@ select l_orderkey from (select * from lineitem order by l_orderkey limit 5000) a 3 # LEFT OUTER / LEFT SINGEL / FULL -query I +query I?I select l_orderkey, o_orderdate, o_shippriority from lineitem left join orders on l_orderkey = o_orderkey and o_orderdate < to_date('1995-03-15') order by o_orderdate, l_orderkey limit 5; ---- 3271 1992-01-01 0 From c774fcb9dfabf9725dc9fb22f5fc4d8473eb018c Mon Sep 17 00:00:00 2001 From: coldWater Date: Fri, 28 Nov 2025 22:43:19 +0800 Subject: [PATCH 19/19] fix --- .../expression/src/filter/filter_executor.rs | 2 + .../new_hash_join/memory/nested_loop.rs | 35 ++++++++++++----- .../suites/crdb/join_small_block_size.test | 30 +++++++-------- .../suites/query/join_small_block_size.test | 38 ++++++++----------- 4 files changed, 58 insertions(+), 47 deletions(-) diff --git a/src/query/expression/src/filter/filter_executor.rs b/src/query/expression/src/filter/filter_executor.rs index a365e375a6c15..ce361f1bc3a64 100644 --- a/src/query/expression/src/filter/filter_executor.rs +++ b/src/query/expression/src/filter/filter_executor.rs @@ -81,6 +81,7 @@ impl FilterExecutor { // Filter a DataBlock, return the filtered DataBlock. pub fn filter(&mut self, data_block: DataBlock) -> Result { if self.func_ctx.enable_selector_executor { + debug_assert!(data_block.num_rows() <= self.max_block_size); let origin_count = data_block.num_rows(); let result_count = self.select(&data_block)?; self.take(data_block, origin_count, result_count) @@ -100,6 +101,7 @@ impl FilterExecutor { // Store the filtered indices of data_block in `true_selection` and return the number of filtered indices. pub fn select(&mut self, data_block: &DataBlock) -> Result { + debug_assert!(data_block.num_rows() <= self.max_block_size); let evaluator = Evaluator::new(data_block, &self.func_ctx, self.fn_registry); let selector = Selector::new(evaluator, data_block.num_rows()); selector.select( diff --git a/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/nested_loop.rs b/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/nested_loop.rs index 9929d64d8228a..d432791e44019 100644 --- a/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/nested_loop.rs +++ b/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/nested_loop.rs @@ -148,20 +148,35 @@ impl<'a> NestedLoopJoinStream<'a> { ) })) .collect(); - - let result_count = self - .desc - .filter - .select(&DataBlock::new(entries, probe_rows))?; + let merged = DataBlock::new(entries, probe_rows); let row_ptr = RowPtr { chunk_index: self.build_block_index as u32, row_index: self.build_row_index as u32, }; - self.matches.extend( - self.desc.filter.true_selection()[..result_count] - .iter() - .map(|probe| (*probe, row_ptr)), - ); + + let max_block_size = self.desc.filter.max_block_size(); + if merged.num_rows() <= max_block_size { + let result_count = self.desc.filter.select(&merged)?; + self.matches.extend( + self.desc.filter.true_selection()[..result_count] + .iter() + .map(|probe| (*probe, row_ptr)), + ); + } else { + for (i, block) in merged + .split_by_rows_no_tail(max_block_size) + .into_iter() + .enumerate() + { + let offset = (i * max_block_size) as u32; + let result_count = self.desc.filter.select(&block)?; + self.matches.extend( + self.desc.filter.true_selection()[..result_count] + .iter() + .map(|probe| (*probe + offset, row_ptr)), + ); + } + } self.build_row_index += 1; if self.build_row_index >= build_block.num_rows() { diff --git a/tests/sqllogictests/suites/crdb/join_small_block_size.test b/tests/sqllogictests/suites/crdb/join_small_block_size.test index e615a6616f191..9606b5d7a94df 100644 --- a/tests/sqllogictests/suites/crdb/join_small_block_size.test +++ b/tests/sqllogictests/suites/crdb/join_small_block_size.test @@ -207,7 +207,7 @@ SELECT * FROM empty AS a(x) JOIN onecolumn AS b(y) ON a.x = b.y statement ok SELECT * FROM empty AS a JOIN onecolumn AS b USING(x) -query IT +query II SELECT * FROM onecolumn AS a(x) LEFT OUTER JOIN empty AS b(y) ON a.x = b.y ORDER BY a.x ---- 42 NULL @@ -291,7 +291,7 @@ SELECT o.x, t.y FROM onecolumn o INNER JOIN twocolumn t ON (o.x=t.x AND t.y=53) ---- 42 53 -query IT +query II SELECT o.x, t.y FROM onecolumn o LEFT OUTER JOIN twocolumn t ON (o.x=t.x AND t.y=53) order by o.x ---- 42 53 @@ -333,27 +333,27 @@ CREATE TABLE b (i int, b bool) statement ok INSERT INTO b VALUES (2, true), (3, true), (4, false) -query III +query IIB SELECT * FROM a INNER JOIN b ON a.i = b.i ---- 2 2 1 3 3 1 -query ITT +query IIB SELECT * FROM a LEFT OUTER JOIN b ON a.i = b.i ---- 1 NULL NULL 2 2 1 3 3 1 -query III +query IIB SELECT * FROM a RIGHT OUTER JOIN b ON a.i = b.i order by b ---- 2 2 1 3 3 1 NULL 4 0 -query III +query IIB SELECT * FROM a FULL OUTER JOIN b ON a.i = b.i order by b ---- 1 NULL NULL @@ -361,7 +361,7 @@ SELECT * FROM a FULL OUTER JOIN b ON a.i = b.i order by b 3 3 1 NULL 4 0 -query III +query IIB SELECT * FROM a FULL OUTER JOIN b ON (a.i = b.i and a.i>2) ORDER BY a.i, b.i ---- 1 NULL NULL @@ -374,7 +374,7 @@ NULL 4 0 statement ok INSERT INTO b VALUES (3, false) -query III +query IIB SELECT * FROM a RIGHT OUTER JOIN b ON a.i=b.i ORDER BY b.i, b.b ---- 2 2 1 @@ -382,7 +382,7 @@ SELECT * FROM a RIGHT OUTER JOIN b ON a.i=b.i ORDER BY b.i, b.b 3 3 1 NULL 4 0 -query III +query IIB SELECT * FROM a FULL OUTER JOIN b ON a.i=b.i ORDER BY b.i, b.b ---- 1 NULL NULL @@ -414,7 +414,7 @@ SELECT * FROM onecolumn JOIN (SELECT x + 2 AS x FROM onecolumn) USING(x) ---- 44 -query IIIII +query IIII SELECT * FROM (twocolumn AS a JOIN twocolumn AS b USING(x) JOIN twocolumn AS c USING(x)) ORDER BY x LIMIT 1 ---- 42 53 53 53 @@ -548,7 +548,7 @@ SELECT * FROM pairs FULL OUTER JOIN square ON pairs.a + pairs.b = square.sq WHER 1 3 2 4 3 6 3 9 -query IITT +query IIII SELECT * FROM (SELECT * FROM pairs LEFT JOIN square ON b = sq AND a > 1 AND n < 6) WHERE b > 1 AND (n IS NULL OR n > 1) AND (n IS NULL OR a < sq) ---- 1 2 NULL NULL @@ -772,7 +772,7 @@ SELECT * FROM xyu INNER JOIN xyv ON xyu.x = xyv.x AND xyu.y = xyv.y AND xyu.x = ---- 1 1 1 1 1 1 -query IIITTT +query IIIIII SELECT * FROM xyu LEFT OUTER JOIN xyv ON xyu.x = xyv.x AND xyu.y = xyv.y AND xyu.x = 1 AND xyu.y < 10 ---- 0 0 0 NULL NULL NULL @@ -807,7 +807,7 @@ NULL 5 5 55 # query # SELECT * FROM (SELECT * FROM xyu ORDER BY x, y) AS xyu FULL OUTER JOIN (SELECT * FROM xyv ORDER BY x, y) AS xyv USING(x, y) WHERE x > 2 -query IIITTT +query IIIIII SELECT * FROM (SELECT * FROM xyu ORDER BY x, y) AS xyu LEFT OUTER JOIN (SELECT * FROM xyv ORDER BY x, y) AS xyv ON xyu.x = xyv.x AND xyu.y = xyv.y AND xyu.x = 1 AND xyu.y < 10 ---- 0 0 0 NULL NULL NULL @@ -888,7 +888,7 @@ SELECT * FROM foo NATURAL JOIN bar 2 2 2.0 2.0 3 3 3.0 3.0 -query II??I?I +query IIRRIRI SELECT * FROM foo JOIN bar USING (b) ---- 1 1 1.0 1.0 1 1.0 1 @@ -944,7 +944,7 @@ SELECT * FROM foo JOIN bar USING (a, b) WHERE foo.c = bar.c AND foo.d = bar.d 2 2 2.0 2.0 2.0 2 3 3 3.0 3.0 3.0 3 -query TII +query III SELECT * FROM onecolumn AS a(x) RIGHT JOIN twocolumn ON false order by y ---- NULL 42 53 diff --git a/tests/sqllogictests/suites/query/join_small_block_size.test b/tests/sqllogictests/suites/query/join_small_block_size.test index 789e04c831741..fa71d57cfa89e 100644 --- a/tests/sqllogictests/suites/query/join_small_block_size.test +++ b/tests/sqllogictests/suites/query/join_small_block_size.test @@ -26,7 +26,7 @@ create table t2(a int, b int) statement ok insert into t2 values(1, 2), (1, 4), (2, 6), (2, 8), (3, 10); -query I +query IIII select * from t1 left join t2 on t1.a = t2.a order by t2.b, t1.b; ---- 1 NULL 1 2 @@ -64,19 +64,13 @@ statement ok use default statement ok -drop table if exists t1 - -statement ok -create table t1(a int, b int) +create or replace table t1(a int, b int) statement ok insert into t1 values(7, 8), (3, 4), (5, 6) statement ok -drop table if exists t2 - -statement ok -create table t2(a int, d int) +create or replace table t2(a int, d int) statement ok insert into t2 values(1, 2), (3, 4), (5, 6) @@ -205,17 +199,17 @@ select * from (select * from numbers(100)) n join t1 on n.number = t1.a; ---- # right semi with empty build side -query II +query I select * from (select * from numbers(100)) n right semi join t1 on n.number = t1.a; ---- # right anti with empty build side -query II +query I select * from (select * from numbers(100)) n right anti join t1 on n.number = t1.a; ---- # left semi with empty build side -query II +query I select * from (select * from numbers(100)) n left semi join t1 on n.number = t1.a; ---- @@ -293,7 +287,7 @@ insert into t1 values(2, 2); statement ok insert into t2 values(2, 6), (2, 8); -query I +query III select t1.a, t2.b, t1.b from t1 inner join t2 on t1.a = t2.a order by t1.a, t2.b, t1.b; ---- 1 2 2 @@ -331,7 +325,7 @@ select t1.id from t1 left join t2 on t1.id = t2.id where t1.val >= t2.val; 2 2 -query I +query II select t1.id, t1.val from t1 left join t2 on t1.id = t2.id and t1.val = t2.val where t1.val >= t2.val; ---- 2 1696549154013 @@ -354,7 +348,7 @@ create table t1(id int, col1 varchar); statement ok insert into t1 values(1, 'c'), (3, 'd'); -query I rowsort +query IITIIT rowsort SELECT * FROM t JOIN t1, t as t2 JOIN t1 as t3; ---- 1 1 c 1 1 c @@ -554,7 +548,7 @@ INSERT INTO t2(c1float) VALUES (0.9702655076980591); statement ok INSERT INTO t2(c1float, c2varchar) VALUES (0.5340723991394043, '02'), (0.4661566913127899, '1261837'); -query IIR +query BBR SELECT t0.c0boolean, t1.c0boolean, t1.c1float FROM t0, t1 RIGHT JOIN t2 ON t1.c0boolean order by t0.c0boolean; ---- 0 NULL NULL @@ -600,7 +594,7 @@ create table t2(a int, b int) statement ok insert into t2 values(1, 2), (1, 4), (1, 6), (1, 8), (1, 10); -query I +query IIII select * from t1 left join t2 on t1.a = t2.a order by t2.b, t1.b; ---- 1 2 1 2 @@ -614,7 +608,7 @@ select * from t1 left join t2 on t1.a = t2.a order by t2.b, t1.b; 5 10 NULL NULL # left join with conjunct -query II +query IIII select * from t1 left join t2 on t1.a = t2.a and t1.b > t2.b order by t1.a, t2.a ---- 1 2 NULL NULL @@ -642,7 +636,7 @@ INSERT INTO t1 VALUES (1, 2), (2, 3), (3, 4); statement ok INSERT INTO t2 VALUES (1, 10), (2, 20); -query I +query IIII SELECT * FROM t1 LEFT OUTER JOIN t2 ON TRUE AND t1.i=t2.k AND FALSE order by i, j; ---- 1 2 NULL NULL @@ -706,14 +700,14 @@ create table t2(b int null); statement ok insert into t2 values(1), (NULL), (3); -query I +query II select * from t1 join t2 on t1.a < t2.b order by t1.a; ---- 1 3 2 3 -query I +query II select * from t1 join t2 on t1.a <= t2.b order by t1.a, t2.b desc; ---- 1 3 @@ -744,7 +738,7 @@ CREATE TABLE twocolumn (x INT NULL, y INT NULL); statement ok INSERT INTO twocolumn(x, y) VALUES (44,51), (NULL,52), (42,53), (45,45); -query I +query II select * from onecolumn as a right anti join twocolumn as b on a.x = b.x and a.x > 42 order by b.x; ---- 42 53