From 1f1359771b975853eab65758bc7a089e8dd0e903 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Mon, 9 Oct 2023 22:32:01 +0800 Subject: [PATCH 01/39] add settings --- .../service/tests/it/storages/testdata/settings_table.txt | 3 ++- src/query/settings/src/settings_default.rs | 8 +++++++- src/query/settings/src/settings_getter_setter.rs | 4 ++++ 3 files changed, 13 insertions(+), 2 deletions(-) diff --git a/src/query/service/tests/it/storages/testdata/settings_table.txt b/src/query/service/tests/it/storages/testdata/settings_table.txt index eed6e8a9701b..afdb8ab67b64 100644 --- a/src/query/service/tests/it/storages/testdata/settings_table.txt +++ b/src/query/service/tests/it/storages/testdata/settings_table.txt @@ -14,7 +14,8 @@ DB.Table: 'system'.'settings', Table: settings-table_id:1, ver:0, Engine: System | 'enable_distributed_copy_into' | '0' | '0' | 'SESSION' | 'Enable distributed execution of copy into.' | 'UInt64' | | 'enable_distributed_replace_into' | '0' | '0' | 'SESSION' | 'Enable distributed execution of replace into.' | 'UInt64' | | 'enable_dphyp' | '1' | '1' | 'SESSION' | 'Enables dphyp join order algorithm.' | 'UInt64' | -| 'enable_experimental_merge_into' | '0' | '0' | 'SESSION' | 'Enable unstable merge into.' | 'UInt64' | +| 'enable_experimental_merge_into' | '0' | '0' | 'SESSION' | 'Enable experimental merge into.' | 'UInt64' | +| 'enable_distributed_merge_into' | '0' | '0' | 'SESSION' | 'Enable distributed merge into.' | 'UInt64' | | 'enable_hive_parquet_predict_pushdown' | '1' | '1' | 'SESSION' | 'Enable hive parquet predict pushdown by setting this variable to 1, default value: 1' | 'UInt64' | | 'enable_query_profiling' | '0' | '0' | 'SESSION' | 'Enables recording query profile' | 'UInt64' | | 'enable_query_result_cache' | '0' | '0' | 'SESSION' | 'Enables caching query results to improve performance for identical queries.' | 'UInt64' | diff --git a/src/query/settings/src/settings_default.rs b/src/query/settings/src/settings_default.rs index 6d6504ed1462..5b09caf970cb 100644 --- a/src/query/settings/src/settings_default.rs +++ b/src/query/settings/src/settings_default.rs @@ -340,7 +340,13 @@ impl DefaultSettings { }), ("enable_experimental_merge_into", DefaultSettingValue { value: UserSettingValue::UInt64(0), - desc: "Enable unstable merge into.", + desc: "Enable experimental merge into.", + possible_values: None, + display_in_show_settings: true, + }), + ("enable_distributed_merge_into", DefaultSettingValue { + value: UserSettingValue::UInt64(0), + desc: "Enable distributed merge into.", possible_values: None, display_in_show_settings: true, }), diff --git a/src/query/settings/src/settings_getter_setter.rs b/src/query/settings/src/settings_getter_setter.rs index c6473214f627..6484b42778a1 100644 --- a/src/query/settings/src/settings_getter_setter.rs +++ b/src/query/settings/src/settings_getter_setter.rs @@ -320,6 +320,10 @@ impl Settings { Ok(self.try_get_u64("enable_experimental_merge_into")? != 0) } + pub fn get_enable_distributed_merge_into(&self) -> Result { + Ok(self.try_get_u64("enable_distributed_merge_into")? != 0) + } + pub fn get_enable_distributed_replace(&self) -> Result { Ok(self.try_get_u64("enable_distributed_replace_into")? != 0) } From 879ee6ee585f6f5b3c02dd5a309d65ca9c84438e Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Tue, 10 Oct 2023 19:35:16 +0800 Subject: [PATCH 02/39] right join for merge into first --- .../sql/src/planner/binder/merge_into.rs | 27 ++++++++++--------- 1 file changed, 14 insertions(+), 13 deletions(-) diff --git a/src/query/sql/src/planner/binder/merge_into.rs b/src/query/sql/src/planner/binder/merge_into.rs index ac7c1e914417..edce6364e6af 100644 --- a/src/query/sql/src/planner/binder/merge_into.rs +++ b/src/query/sql/src/planner/binder/merge_into.rs @@ -18,7 +18,7 @@ use std::sync::Arc; use common_ast::ast::Join; use common_ast::ast::JoinCondition; -use common_ast::ast::JoinOperator::LeftOuter; +use common_ast::ast::JoinOperator::RightOuter; use common_ast::ast::MatchOperation; use common_ast::ast::MatchedClause; use common_ast::ast::MergeIntoStmt; @@ -124,12 +124,12 @@ impl Binder { let source_data = source.transform_table_reference(source_alias.clone()); // bind source data - let (source_expr, mut left_context) = + let (source_expr, mut source_context) = self.bind_single_table(bind_context, &source_data).await?; // add all left source columns for read // todo: (JackTan25) do column prune after finish "split expr for target and source" - let mut columns_set = left_context.column_set(); + let mut columns_set = source_context.column_set(); let update_columns_star = if self.has_star_clause(&matched_clauses, &unmatched_clauses) { // when there are "update *"/"insert *", we need to get the index of correlated columns in source. @@ -139,7 +139,7 @@ impl Binder { .remove_computed_fields() .num_fields(), ); - let source_output_columns = &left_context.columns; + let source_output_columns = &source_context.columns; // we use Vec as the value, because if there could be duplicate names let mut name_map = HashMap::>::new(); for column in source_output_columns { @@ -189,8 +189,8 @@ impl Binder { // Todo: (JackTan25) Maybe we can remove bind target_table // when the target table has been binded in bind_merge_into_source // bind table for target table - let (mut target_expr, mut right_context) = self - .bind_single_table(&mut left_context, &target_table) + let (mut target_expr, mut target_context) = self + .bind_single_table(&mut source_context, &target_table) .await?; // add internal_column (_row_id) @@ -209,7 +209,7 @@ impl Binder { }, }; - let column_binding = right_context + let column_binding = target_context .add_internal_column_binding(&row_id_column_binding, self.metadata.clone())?; target_expr = @@ -223,19 +223,20 @@ impl Binder { // add join,use left outer join in V1, we use _row_id to check_duplicate join row. let join = Join { - op: LeftOuter, + op: RightOuter, condition: JoinCondition::On(Box::new(join_expr.clone())), - left: Box::new(source_data.clone()), - right: Box::new(target_table), + left: Box::new(target_table), + // use source as build table + right: Box::new(source_data.clone()), }; let (join_sexpr, mut bind_ctx) = self .bind_join( bind_context, - left_context, - right_context.clone(), - source_expr, + target_context.clone(), + source_context, target_expr, + source_expr, &join, ) .await?; From 92538e190a6a140f7f7afd74f008105926c21f2f Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Sat, 14 Oct 2023 00:23:43 +0800 Subject: [PATCH 03/39] add distribution optimization for merge into join --- .../sql/src/planner/optimizer/optimizer.rs | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/src/query/sql/src/planner/optimizer/optimizer.rs b/src/query/sql/src/planner/optimizer/optimizer.rs index 6ca4fa5e745e..96c81c651c2f 100644 --- a/src/query/sql/src/planner/optimizer/optimizer.rs +++ b/src/query/sql/src/planner/optimizer/optimizer.rs @@ -35,6 +35,7 @@ use crate::optimizer::SExpr; use crate::optimizer::DEFAULT_REWRITE_RULES; use crate::optimizer::RESIDUAL_RULES; use crate::plans::CopyIntoLocationPlan; +use crate::plans::MergeInto; use crate::plans::Plan; use crate::IndexType; use crate::MetadataRef; @@ -123,6 +124,24 @@ pub fn optimize( ); Ok(Plan::CopyIntoTable(plan)) } + Plan::MergeInto(plan) => { + // try to optimize distributed join + if opt_ctx.config.enable_distributed_optimization + && ctx.get_settings().get_enable_distributed_merge_into()? + { + // Todo(JackTan25): We should use optimizer to make a decision to use + // left join and right join + // input is a Join_SExpr + let optimized_distributed_join_sexpr = + optimize_distributed_query(ctx.clone(), &*plan.input)?; + Ok(Plan::MergeInto(Box::new(MergeInto { + input: Box::new(optimized_distributed_join_sexpr), + ..*plan + }))) + } else { + Ok(Plan::MergeInto(plan)) + } + } // Passthrough statements. _ => Ok(plan), } From 19a199979768a4cb7621329901f003ec448b0858 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Mon, 16 Oct 2023 17:39:14 +0800 Subject: [PATCH 04/39] split merge into plan --- .../interpreters/interpreter_merge_into.rs | 79 ++++++++++++++----- .../service/src/pipelines/pipeline_builder.rs | 19 +++++ src/query/sql/src/executor/format.rs | 4 + src/query/sql/src/executor/mod.rs | 2 + src/query/sql/src/executor/physical_plan.rs | 16 ++++ .../sql/src/executor/physical_plan_display.rs | 18 +++++ .../sql/src/executor/physical_plan_visitor.rs | 31 ++++++++ .../physical_plans/physical_merge_into.rs | 22 ++++++ src/query/sql/src/executor/profile.rs | 2 + 9 files changed, 174 insertions(+), 19 deletions(-) diff --git a/src/query/service/src/interpreters/interpreter_merge_into.rs b/src/query/service/src/interpreters/interpreter_merge_into.rs index 5cc4e9dd61c1..2497812fd02d 100644 --- a/src/query/service/src/interpreters/interpreter_merge_into.rs +++ b/src/query/service/src/interpreters/interpreter_merge_into.rs @@ -27,7 +27,10 @@ use common_expression::RemoteExpr; use common_functions::BUILTIN_FUNCTIONS; use common_meta_app::schema::TableInfo; use common_sql::executor::CommitSink; +use common_sql::executor::Exchange; use common_sql::executor::MergeInto; +use common_sql::executor::MergeIntoAppend; +use common_sql::executor::MergeIntoRowIdApply; use common_sql::executor::MergeIntoSource; use common_sql::executor::MutationKind; use common_sql::executor::PhysicalPlan; @@ -139,11 +142,21 @@ impl MergeIntoInterpreter { field_index_map, .. } = &self.plan; + let table_name = table.clone(); let mut builder = PhysicalPlanBuilder::new(meta_data.clone(), self.ctx.clone(), false); // build source for MergeInto - let join_input = builder.build(input, *columns_set.clone()).await?; + let mut join_input = builder.build(input, *columns_set.clone()).await?; + + // check distributed execution + let exchange = if let PhysicalPlan::Exchange(exchange) = join_input { + // remove top exchange + join_input = *exchange.input.clone(); + Some(exchange) + } else { + None + }; // find row_id column index let join_output_schema = join_input.output_schema()?; @@ -309,27 +322,55 @@ impl MergeIntoInterpreter { .insert(*field_index, join_output_schema.index_of(value).unwrap()); } - // recv datablocks from matched upstream and unmatched upstream - // transform and append dat - let merge_into = PhysicalPlan::MergeInto(Box::new(MergeInto { - input: Box::new(merge_into_source), - table_info: table_info.clone(), - catalog_info: catalog_.info(), - unmatched, - matched, - field_index_of_input_schema, - row_id_idx, - segments: base_snapshot - .segments - .clone() - .into_iter() - .enumerate() - .collect(), - })); + let commit_input = if exchange.is_none() { + // recv datablocks from matched upstream and unmatched upstream + // transform and append dat + PhysicalPlan::MergeInto(Box::new(MergeInto { + input: Box::new(merge_into_source), + table_info: table_info.clone(), + catalog_info: catalog_.info(), + unmatched, + matched, + field_index_of_input_schema, + row_id_idx, + segments: base_snapshot + .segments + .clone() + .into_iter() + .enumerate() + .collect(), + })) + } else { + let merge_append = PhysicalPlan::MergeIntoAppend(Box::new(MergeIntoAppend { + input: Box::new(merge_into_source), + table_info: table_info.clone(), + catalog_info: catalog_.info(), + unmatched, + matched, + field_index_of_input_schema, + row_id_idx, + segments: base_snapshot + .segments + .clone() + .into_iter() + .enumerate() + .collect(), + })); + let exchange = exchange.unwrap(); + PhysicalPlan::MergeIntoRowIdApply(Box::new(MergeIntoRowIdApply { + input: Box::new(PhysicalPlan::Exchange(Exchange { + plan_id: 0, + input: Box::new(merge_append), + kind: exchange.kind, + keys: exchange.keys, + ignore_exchange: exchange.ignore_exchange, + })), + })) + }; // build mutation_aggregate let physical_plan = PhysicalPlan::CommitSink(Box::new(CommitSink { - input: Box::new(merge_into), + input: Box::new(commit_input), snapshot: base_snapshot, table_info: table_info.clone(), catalog_info: catalog_.info(), diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs index e9d217846337..730379bd5a95 100644 --- a/src/query/service/src/pipelines/pipeline_builder.rs +++ b/src/query/service/src/pipelines/pipeline_builder.rs @@ -94,6 +94,8 @@ use common_sql::executor::Lambda; use common_sql::executor::Limit; use common_sql::executor::MaterializedCte; use common_sql::executor::MergeInto; +use common_sql::executor::MergeIntoAppend; +use common_sql::executor::MergeIntoRowIdApply; use common_sql::executor::MergeIntoSource; use common_sql::executor::PhysicalPlan; use common_sql::executor::Project; @@ -289,6 +291,12 @@ impl PipelineBuilder { PhysicalPlan::MergeIntoSource(merge_into_source) => { self.build_merge_into_source(merge_into_source) } + PhysicalPlan::MergeIntoAppend(merge_into_append) => { + self.build_merge_into_append(merge_into_append) + } + PhysicalPlan::MergeIntoRowIdApply(merge_into_row_id_apply) => { + self.build_merge_into_row_id_apply(merge_into_row_id_apply) + } } } @@ -301,6 +309,17 @@ impl PipelineBuilder { Ok(cast_needed) } + fn build_merge_into_append(&mut self, _merge_into_append: &MergeIntoAppend) -> Result<()> { + todo!() + } + + fn build_merge_into_row_id_apply( + &mut self, + _merge_into_row_id_apply: &MergeIntoRowIdApply, + ) -> Result<()> { + todo!() + } + fn build_merge_into_source(&mut self, merge_into_source: &MergeIntoSource) -> Result<()> { let MergeIntoSource { input, row_id_idx } = merge_into_source; diff --git a/src/query/sql/src/executor/format.rs b/src/query/sql/src/executor/format.rs index 4654889a12df..e269e2bd1009 100644 --- a/src/query/sql/src/executor/format.rs +++ b/src/query/sql/src/executor/format.rs @@ -212,6 +212,10 @@ fn to_format_tree( PhysicalPlan::ReplaceInto(_) => Ok(FormatTreeNode::new("Replace".to_string())), PhysicalPlan::MergeInto(_) => Ok(FormatTreeNode::new("MergeInto".to_string())), PhysicalPlan::MergeIntoSource(_) => Ok(FormatTreeNode::new("MergeIntoSource".to_string())), + PhysicalPlan::MergeIntoAppend(_) => Ok(FormatTreeNode::new("MergeIntoAppend".to_string())), + PhysicalPlan::MergeIntoRowIdApply(_) => { + Ok(FormatTreeNode::new("MergeIntoRowIdApply".to_string())) + } PhysicalPlan::CteScan(plan) => cte_scan_to_format_tree(plan), PhysicalPlan::MaterializedCte(plan) => { materialized_cte_to_format_tree(plan, metadata, profs) diff --git a/src/query/sql/src/executor/mod.rs b/src/query/sql/src/executor/mod.rs index 1db769dcca77..e88f29c31574 100644 --- a/src/query/sql/src/executor/mod.rs +++ b/src/query/sql/src/executor/mod.rs @@ -57,6 +57,8 @@ pub use physical_plans::physical_limit::Limit; pub use physical_plans::physical_materialized_cte::MaterializedCte; pub use physical_plans::physical_merge_into::MatchExpr; pub use physical_plans::physical_merge_into::MergeInto; +pub use physical_plans::physical_merge_into::MergeIntoAppend; +pub use physical_plans::physical_merge_into::MergeIntoRowIdApply; pub use physical_plans::physical_merge_into::MergeIntoSource; pub use physical_plans::physical_project::Project; pub use physical_plans::physical_project_set::ProjectSet; diff --git a/src/query/sql/src/executor/physical_plan.rs b/src/query/sql/src/executor/physical_plan.rs index a76e7390daa8..841197d45b8b 100644 --- a/src/query/sql/src/executor/physical_plan.rs +++ b/src/query/sql/src/executor/physical_plan.rs @@ -17,6 +17,8 @@ use common_exception::Result; use common_expression::DataSchemaRef; use enum_as_inner::EnumAsInner; +use super::MergeIntoAppend; +use super::MergeIntoRowIdApply; use crate::executor::physical_plans::physical_aggregate_expand::AggregateExpand; use crate::executor::physical_plans::physical_aggregate_final::AggregateFinal; use crate::executor::physical_plans::physical_aggregate_partial::AggregatePartial; @@ -97,6 +99,8 @@ pub enum PhysicalPlan { /// MergeInto MergeIntoSource(MergeIntoSource), MergeInto(Box), + MergeIntoAppend(Box), + MergeIntoRowIdApply(Box), /// Compact CompactPartial(Box), @@ -134,6 +138,8 @@ impl PhysicalPlan { PhysicalPlan::DeletePartial(_) | PhysicalPlan::MergeInto(_) | PhysicalPlan::MergeIntoSource(_) + | PhysicalPlan::MergeIntoAppend(_) + | PhysicalPlan::MergeIntoRowIdApply(_) | PhysicalPlan::CommitSink(_) | PhysicalPlan::CopyIntoTable(_) | PhysicalPlan::AsyncSourcer(_) @@ -175,6 +181,8 @@ impl PhysicalPlan { PhysicalPlan::MaterializedCte(plan) => plan.output_schema(), PhysicalPlan::ConstantTableScan(plan) => plan.output_schema(), PhysicalPlan::MergeIntoSource(plan) => plan.input.output_schema(), + PhysicalPlan::MergeIntoRowIdApply(_) => todo!(), + PhysicalPlan::MergeIntoAppend(_) => todo!(), PhysicalPlan::AsyncSourcer(_) | PhysicalPlan::MergeInto(_) | PhysicalPlan::Deduplicate(_) @@ -215,6 +223,8 @@ impl PhysicalPlan { PhysicalPlan::ReplaceInto(_) => "Replace".to_string(), PhysicalPlan::MergeInto(_) => "MergeInto".to_string(), PhysicalPlan::MergeIntoSource(_) => "MergeIntoSource".to_string(), + PhysicalPlan::MergeIntoAppend(_) => "MergeIntoAppend".to_string(), + PhysicalPlan::MergeIntoRowIdApply(_) => "MergeIntoRowIdApply".to_string(), PhysicalPlan::CteScan(_) => "PhysicalCteScan".to_string(), PhysicalPlan::MaterializedCte(_) => "PhysicalMaterializedCte".to_string(), PhysicalPlan::ConstantTableScan(_) => "PhysicalConstantTableScan".to_string(), @@ -266,6 +276,10 @@ impl PhysicalPlan { PhysicalPlan::ReplaceInto(plan) => Box::new(std::iter::once(plan.input.as_ref())), PhysicalPlan::MergeInto(plan) => Box::new(std::iter::once(plan.input.as_ref())), PhysicalPlan::MergeIntoSource(plan) => Box::new(std::iter::once(plan.input.as_ref())), + PhysicalPlan::MergeIntoAppend(plan) => Box::new(std::iter::once(plan.input.as_ref())), + PhysicalPlan::MergeIntoRowIdApply(plan) => { + Box::new(std::iter::once(plan.input.as_ref())) + } PhysicalPlan::MaterializedCte(plan) => Box::new( std::iter::once(plan.left.as_ref()).chain(std::iter::once(plan.right.as_ref())), ), @@ -305,6 +319,8 @@ impl PhysicalPlan { | PhysicalPlan::Deduplicate(_) | PhysicalPlan::ReplaceInto(_) | PhysicalPlan::MergeInto(_) + | PhysicalPlan::MergeIntoAppend(_) + | PhysicalPlan::MergeIntoRowIdApply(_) | PhysicalPlan::MergeIntoSource(_) | PhysicalPlan::ConstantTableScan(_) | PhysicalPlan::CteScan(_) => None, diff --git a/src/query/sql/src/executor/physical_plan_display.rs b/src/query/sql/src/executor/physical_plan_display.rs index 273a330cad2b..3df4aebbf05a 100644 --- a/src/query/sql/src/executor/physical_plan_display.rs +++ b/src/query/sql/src/executor/physical_plan_display.rs @@ -18,6 +18,8 @@ use std::fmt::Formatter; use common_functions::BUILTIN_FUNCTIONS; use itertools::Itertools; +use super::MergeIntoAppend; +use super::MergeIntoRowIdApply; use crate::executor::physical_plans::physical_aggregate_expand::AggregateExpand; use crate::executor::physical_plans::physical_aggregate_final::AggregateFinal; use crate::executor::physical_plans::physical_aggregate_partial::AggregatePartial; @@ -100,6 +102,10 @@ impl<'a> Display for PhysicalPlanIndentFormatDisplay<'a> { PhysicalPlan::ReplaceInto(replace) => write!(f, "{}", replace)?, PhysicalPlan::MergeIntoSource(merge_into_source) => write!(f, "{}", merge_into_source)?, PhysicalPlan::MergeInto(merge_into) => write!(f, "{}", merge_into)?, + PhysicalPlan::MergeIntoAppend(merge_into_append) => write!(f, "{}", merge_into_append)?, + PhysicalPlan::MergeIntoRowIdApply(merge_into_row_id_apply) => { + write!(f, "{}", merge_into_row_id_apply)? + } PhysicalPlan::CteScan(cte_scan) => write!(f, "{}", cte_scan)?, PhysicalPlan::MaterializedCte(plan) => write!(f, "{}", plan)?, PhysicalPlan::ConstantTableScan(scan) => write!(f, "{}", scan)?, @@ -470,6 +476,18 @@ impl Display for MergeInto { } } +impl Display for MergeIntoAppend { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!(f, "MergeIntoAppend") + } +} + +impl Display for MergeIntoRowIdApply { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!(f, "MergeIntoRowIdApply") + } +} + impl Display for MergeIntoSource { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { write!(f, "MergeIntoSource") diff --git a/src/query/sql/src/executor/physical_plan_visitor.rs b/src/query/sql/src/executor/physical_plan_visitor.rs index 26a8a76570fa..2be2507b8a90 100644 --- a/src/query/sql/src/executor/physical_plan_visitor.rs +++ b/src/query/sql/src/executor/physical_plan_visitor.rs @@ -14,6 +14,8 @@ use common_exception::Result; +use super::MergeIntoAppend; +use super::MergeIntoRowIdApply; use crate::executor::physical_plans::physical_aggregate_expand::AggregateExpand; use crate::executor::physical_plans::physical_aggregate_final::AggregateFinal; use crate::executor::physical_plans::physical_aggregate_partial::AggregatePartial; @@ -85,6 +87,8 @@ pub trait PhysicalPlanReplacer { PhysicalPlan::ReplaceInto(plan) => self.replace_replace_into(plan), PhysicalPlan::MergeInto(plan) => self.replace_merge_into(plan), PhysicalPlan::MergeIntoSource(plan) => self.replace_merge_into_source(plan), + PhysicalPlan::MergeIntoAppend(plan) => self.replace_merge_into_append(plan), + PhysicalPlan::MergeIntoRowIdApply(plan) => self.replace_merge_into_row_id_apply(plan), PhysicalPlan::MaterializedCte(plan) => self.replace_materialized_cte(plan), PhysicalPlan::ConstantTableScan(plan) => self.replace_constant_table_scan(plan), } @@ -422,6 +426,27 @@ pub trait PhysicalPlanReplacer { })) } + fn replace_merge_into_append(&mut self, plan: &MergeIntoAppend) -> Result { + let input = self.replace(&plan.input)?; + Ok(PhysicalPlan::MergeIntoAppend(Box::new(MergeIntoAppend { + input: Box::new(input), + ..plan.clone() + }))) + } + + fn replace_merge_into_row_id_apply( + &mut self, + plan: &MergeIntoRowIdApply, + ) -> Result { + let input = self.replace(&plan.input)?; + Ok(PhysicalPlan::MergeIntoRowIdApply(Box::new( + MergeIntoRowIdApply { + input: Box::new(input), + ..plan.clone() + }, + ))) + } + fn replace_project_set(&mut self, plan: &ProjectSet) -> Result { let input = self.replace(&plan.input)?; Ok(PhysicalPlan::ProjectSet(ProjectSet { @@ -558,6 +583,12 @@ impl PhysicalPlan { PhysicalPlan::MergeInto(plan) => { Self::traverse(&plan.input, pre_visit, visit, post_visit); } + PhysicalPlan::MergeIntoAppend(plan) => { + Self::traverse(&plan.input, pre_visit, visit, post_visit); + } + PhysicalPlan::MergeIntoRowIdApply(plan) => { + Self::traverse(&plan.input, pre_visit, visit, post_visit); + } PhysicalPlan::MaterializedCte(plan) => { Self::traverse(&plan.left, pre_visit, visit, post_visit); Self::traverse(&plan.right, pre_visit, visit, post_visit); diff --git a/src/query/sql/src/executor/physical_plans/physical_merge_into.rs b/src/query/sql/src/executor/physical_plans/physical_merge_into.rs index 8330c44cee07..ad496ba06912 100644 --- a/src/query/sql/src/executor/physical_plans/physical_merge_into.rs +++ b/src/query/sql/src/executor/physical_plans/physical_merge_into.rs @@ -32,6 +32,7 @@ pub struct MergeIntoSource { pub row_id_idx: u32, } +// Todo:(JackTan25) replace MergeInto with MergeIntoAppend and MergeIntoRowIdApply #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct MergeInto { pub input: Box, @@ -47,3 +48,24 @@ pub struct MergeInto { pub row_id_idx: usize, pub segments: Vec<(usize, Location)>, } + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct MergeIntoAppend { + pub input: Box, + pub table_info: TableInfo, + pub catalog_info: CatalogInfo, + // (DataSchemaRef, Option, Vec,Vec) => (source_schema, condition, value_exprs) + pub unmatched: Vec<(DataSchemaRef, Option, Vec)>, + // the first option stands for the condition + // the second option stands for update/delete + pub matched: MatchExpr, + // used to record the index of target table's field in merge_source_schema + pub field_index_of_input_schema: HashMap, + pub row_id_idx: usize, + pub segments: Vec<(usize, Location)>, +} + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct MergeIntoRowIdApply { + pub input: Box, +} diff --git a/src/query/sql/src/executor/profile.rs b/src/query/sql/src/executor/profile.rs index 296d022789f6..8dfc21350aff 100644 --- a/src/query/sql/src/executor/profile.rs +++ b/src/query/sql/src/executor/profile.rs @@ -510,6 +510,8 @@ fn flatten_plan_node_profile( | PhysicalPlan::CopyIntoTable(_) | PhysicalPlan::AsyncSourcer(_) | PhysicalPlan::MergeInto(_) + | PhysicalPlan::MergeIntoAppend(_) + | PhysicalPlan::MergeIntoRowIdApply(_) | PhysicalPlan::MergeIntoSource(_) | PhysicalPlan::Deduplicate(_) | PhysicalPlan::ReplaceInto(_) From 60f79f7e713a9222fbc0a839d52ff3bb958d3b5a Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Mon, 16 Oct 2023 19:38:21 +0800 Subject: [PATCH 05/39] fix update identify error --- .../ast/src/ast/statements/merge_into.rs | 7 +------ src/query/ast/src/parser/statement.rs | 9 ++------ .../sql/src/planner/binder/merge_into.rs | 21 +++++++++++++++++++ .../base/09_fuse_engine/09_0026_merge_into | 7 +++++++ 4 files changed, 31 insertions(+), 13 deletions(-) diff --git a/src/query/ast/src/ast/statements/merge_into.rs b/src/query/ast/src/ast/statements/merge_into.rs index 576a59db64ca..fd0fd141d0c8 100644 --- a/src/query/ast/src/ast/statements/merge_into.rs +++ b/src/query/ast/src/ast/statements/merge_into.rs @@ -31,7 +31,6 @@ use crate::ast::TableReference; #[derive(Debug, Clone, PartialEq)] pub struct MergeUpdateExpr { - pub catalog: Option, pub table: Option, pub name: Identifier, pub expr: Expr, @@ -39,10 +38,6 @@ pub struct MergeUpdateExpr { impl Display for MergeUpdateExpr { fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { - if self.catalog.is_some() { - write!(f, "{}.", self.catalog.clone().unwrap())?; - } - if self.table.is_some() { write!(f, "{}.", self.table.clone().unwrap())?; } @@ -104,7 +99,7 @@ impl Display for MergeIntoStmt { write!(f, "MERGE INTO ")?; write_dot_separated_list( f, - self.catalog + self.database .iter() .chain(&self.database) .chain(Some(&self.table_ident)), diff --git a/src/query/ast/src/parser/statement.rs b/src/query/ast/src/parser/statement.rs index 17b63c8b5a76..013e0951f77c 100644 --- a/src/query/ast/src/parser/statement.rs +++ b/src/query/ast/src/parser/statement.rs @@ -2629,12 +2629,7 @@ pub fn udf_definition(i: Input) -> IResult { pub fn merge_update_expr(i: Input) -> IResult { map( - rule! { ( #dot_separated_idents_1_to_3 ~ "=" ~ ^#expr ) }, - |((catalog, table, name), _, expr)| MergeUpdateExpr { - catalog, - table, - name, - expr, - }, + rule! { ( #dot_separated_idents_1_to_2 ~ "=" ~ ^#expr ) }, + |((table, name), _, expr)| MergeUpdateExpr { table, name, expr }, )(i) } diff --git a/src/query/sql/src/planner/binder/merge_into.rs b/src/query/sql/src/planner/binder/merge_into.rs index edce6364e6af..4ecce28fa128 100644 --- a/src/query/sql/src/planner/binder/merge_into.rs +++ b/src/query/sql/src/planner/binder/merge_into.rs @@ -269,6 +269,14 @@ impl Binder { } } + let target_name = if let Some(target_identify) = target_alias { + normalize_identifier(&target_identify.name, &self.name_resolution_ctx) + .name + .clone() + } else { + table_name.clone() + }; + // bind matched clause columns and add update fields and exprs for clause in &matched_clauses { matched_evaluators.push( @@ -278,6 +286,7 @@ impl Binder { &mut columns_set, table_schema.clone(), update_columns_star.clone(), + target_name.as_ref(), ) .await?, ); @@ -321,6 +330,7 @@ impl Binder { columns: &mut HashSet, schema: TableSchemaRef, update_columns_star: Option>, + target_name: &str, ) -> Result { let condition = if let Some(expr) = &clause.selection { let (scalar_expr, _) = scalar_binder.bind(expr).await?; @@ -348,6 +358,17 @@ impl Binder { let (scalar_expr, _) = scalar_binder.bind(&update_expr.expr).await?; let col_name = normalize_identifier(&update_expr.name, &self.name_resolution_ctx).name; + if let Some(tbl_identify) = &update_expr.table { + let update_table_name = + normalize_identifier(tbl_identify, &self.name_resolution_ctx).name; + if &update_table_name != target_name { + return Err(ErrorCode::BadArguments(format!( + "Update Indentify's `{}` should be `{}`", + update_table_name, target_name + ))); + } + } + let index = schema.index_of(&col_name)?; if update_columns.contains_key(&index) { diff --git a/tests/sqllogictests/suites/base/09_fuse_engine/09_0026_merge_into b/tests/sqllogictests/suites/base/09_fuse_engine/09_0026_merge_into index 81b24a93b422..f3d6433c2b41 100644 --- a/tests/sqllogictests/suites/base/09_fuse_engine/09_0026_merge_into +++ b/tests/sqllogictests/suites/base/09_fuse_engine/09_0026_merge_into @@ -491,6 +491,13 @@ create table cluster_source(a int,b string,c int); statement ok insert into cluster_source values(12,'b',1),(1,'a',2),(2,'b',3),(2,'a',4),(3,'a',3); +## test update indetify error +statement error 1065 +merge into cluster_target as t1 using (select * from cluster_source) as t2 on t1.a = t2.a when matched then update cluster_target.a = t2.a; + +statement error 1065 +merge into cluster_target as t1 using (select * from cluster_source) as t2 on t1.a = t2.a when matched then update t2.a = t2.a; + statement ok merge into cluster_target as t1 using (select * from cluster_source) as t2 on t1.a = t2.a when not matched then insert *; From 662af1063c73d3baa64394c0d2d198b4b1383d51 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Mon, 16 Oct 2023 22:06:21 +0800 Subject: [PATCH 06/39] finish distibuted baisc codes --- .../interpreters/interpreter_merge_into.rs | 27 +++--- .../service/src/pipelines/pipeline_builder.rs | 88 +++++++++++++++---- src/query/sql/src/executor/format.rs | 1 - src/query/sql/src/executor/mod.rs | 1 - src/query/sql/src/executor/physical_plan.rs | 10 +-- .../sql/src/executor/physical_plan_display.rs | 8 -- .../sql/src/executor/physical_plan_visitor.rs | 14 +-- .../physical_plans/physical_merge_into.rs | 18 +--- src/query/sql/src/executor/profile.rs | 1 - .../storages/fuse/src/operations/merge.rs | 2 + .../merge_into/mutator/matched_mutator.rs | 18 +++- 11 files changed, 106 insertions(+), 82 deletions(-) diff --git a/src/query/service/src/interpreters/interpreter_merge_into.rs b/src/query/service/src/interpreters/interpreter_merge_into.rs index 2497812fd02d..c863a963437a 100644 --- a/src/query/service/src/interpreters/interpreter_merge_into.rs +++ b/src/query/service/src/interpreters/interpreter_merge_into.rs @@ -29,7 +29,6 @@ use common_meta_app::schema::TableInfo; use common_sql::executor::CommitSink; use common_sql::executor::Exchange; use common_sql::executor::MergeInto; -use common_sql::executor::MergeIntoAppend; use common_sql::executor::MergeIntoRowIdApply; use common_sql::executor::MergeIntoSource; use common_sql::executor::MutationKind; @@ -322,6 +321,13 @@ impl MergeIntoInterpreter { .insert(*field_index, join_output_schema.index_of(value).unwrap()); } + let segments = base_snapshot + .segments + .clone() + .into_iter() + .enumerate() + .collect(); + let commit_input = if exchange.is_none() { // recv datablocks from matched upstream and unmatched upstream // transform and append dat @@ -333,15 +339,10 @@ impl MergeIntoInterpreter { matched, field_index_of_input_schema, row_id_idx, - segments: base_snapshot - .segments - .clone() - .into_iter() - .enumerate() - .collect(), + segments: Some(segments), })) } else { - let merge_append = PhysicalPlan::MergeIntoAppend(Box::new(MergeIntoAppend { + let merge_append = PhysicalPlan::MergeInto(Box::new(MergeInto { input: Box::new(merge_into_source), table_info: table_info.clone(), catalog_info: catalog_.info(), @@ -349,12 +350,7 @@ impl MergeIntoInterpreter { matched, field_index_of_input_schema, row_id_idx, - segments: base_snapshot - .segments - .clone() - .into_iter() - .enumerate() - .collect(), + segments: None, })); let exchange = exchange.unwrap(); PhysicalPlan::MergeIntoRowIdApply(Box::new(MergeIntoRowIdApply { @@ -365,6 +361,9 @@ impl MergeIntoInterpreter { keys: exchange.keys, ignore_exchange: exchange.ignore_exchange, })), + table_info: table_info.clone(), + catalog_info: catalog_.info(), + segments, })) }; diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs index 730379bd5a95..0d7088e38ead 100644 --- a/src/query/service/src/pipelines/pipeline_builder.rs +++ b/src/query/service/src/pipelines/pipeline_builder.rs @@ -94,7 +94,6 @@ use common_sql::executor::Lambda; use common_sql::executor::Limit; use common_sql::executor::MaterializedCte; use common_sql::executor::MergeInto; -use common_sql::executor::MergeIntoAppend; use common_sql::executor::MergeIntoRowIdApply; use common_sql::executor::MergeIntoSource; use common_sql::executor::PhysicalPlan; @@ -291,9 +290,6 @@ impl PipelineBuilder { PhysicalPlan::MergeIntoSource(merge_into_source) => { self.build_merge_into_source(merge_into_source) } - PhysicalPlan::MergeIntoAppend(merge_into_append) => { - self.build_merge_into_append(merge_into_append) - } PhysicalPlan::MergeIntoRowIdApply(merge_into_row_id_apply) => { self.build_merge_into_row_id_apply(merge_into_row_id_apply) } @@ -309,14 +305,52 @@ impl PipelineBuilder { Ok(cast_needed) } - fn build_merge_into_append(&mut self, _merge_into_append: &MergeIntoAppend) -> Result<()> { - todo!() - } - fn build_merge_into_row_id_apply( &mut self, - _merge_into_row_id_apply: &MergeIntoRowIdApply, + merge_into_row_id_apply: &MergeIntoRowIdApply, ) -> Result<()> { + let MergeIntoRowIdApply { + input, + table_info, + catalog_info, + + segments, + } = merge_into_row_id_apply; + // recieve rowids and MutationLogs + self.build_pipeline(input)?; + let mut pipe_items = Vec::with_capacity(self.main_pipeline.output_len()); + let tbl = self + .ctx + .build_table_by_table_info(catalog_info, table_info, None)?; + + let table = FuseTable::try_from_table(tbl.as_ref())?; + + let block_thresholds = table.get_block_thresholds(); + + let cluster_stats_gen = + table.get_cluster_stats_gen(self.ctx.clone(), 0, block_thresholds, None)?; + + // this TransformSerializeBlock is just used to get block_builder + let block_builder = TransformSerializeBlock::try_create( + self.ctx.clone(), + InputPort::create(), + OutputPort::create(), + table, + cluster_stats_gen.clone(), + )? + .get_block_builder(); + + let max_threads = self.settings.get_max_threads()?; + let io_request_semaphore = Arc::new(Semaphore::new(max_threads as usize)); + + pipe_items.push(table.rowid_aggregate_mutator( + self.ctx.clone(), + block_builder, + io_request_semaphore, + segments.clone(), + true, + )?); + todo!() } @@ -485,6 +519,8 @@ impl PipelineBuilder { } = merge_into; self.build_pipeline(input)?; + + let apply_row_id = segments.is_some(); let tbl = self .ctx .build_table_by_table_info(catalog_info, table_info, None)?; @@ -512,6 +548,7 @@ impl PipelineBuilder { table, block_thresholds, ); + let get_output_len = |pipe_items: &Vec| -> usize { let mut output_len = 0; for item in pipe_items.iter() { @@ -661,12 +698,20 @@ impl PipelineBuilder { )?; pipe_items.clear(); - pipe_items.push(table.rowid_aggregate_mutator( - self.ctx.clone(), - block_builder, - io_request_semaphore, - segments.clone(), - )?); + + // standalone execution + if apply_row_id { + pipe_items.push(table.rowid_aggregate_mutator( + self.ctx.clone(), + block_builder, + io_request_semaphore, + segments.clone().unwrap(), + false, + )?); + } else { + // distributed exectution + pipe_items.push(create_dummy_item()) + } for _ in 0..self.main_pipeline.output_len() - 1 { let serialize_block_transform = TransformSerializeBlock::try_create( @@ -686,9 +731,9 @@ impl PipelineBuilder { )); // resize block ports - // aggregate_mutator port aggregate_mutator port - // serialize_block port0 ======> - // serialize_block port1 serialize_block port + // aggregate_mutator port/dummy_item port aggregate_mutator port/ dummy_item (this depends on apply_row_id) + // serialize_block port0 ======> + // serialize_block port1 serialize_block port // ....... ranges.clear(); ranges.push(vec![0]); @@ -704,6 +749,13 @@ impl PipelineBuilder { serialize_segment_transform.into_pipe_item(), ]; + // apply_row_id: true + // output_port0: MutationLogs + // output_port1: MutationLogs + // + // apply_row_id: false + // output_port0: row_ids + // output_port1: MutationLogs self.main_pipeline.add_pipe(Pipe::create( self.main_pipeline.output_len(), get_output_len(&pipe_items), diff --git a/src/query/sql/src/executor/format.rs b/src/query/sql/src/executor/format.rs index e269e2bd1009..314740c0abbe 100644 --- a/src/query/sql/src/executor/format.rs +++ b/src/query/sql/src/executor/format.rs @@ -212,7 +212,6 @@ fn to_format_tree( PhysicalPlan::ReplaceInto(_) => Ok(FormatTreeNode::new("Replace".to_string())), PhysicalPlan::MergeInto(_) => Ok(FormatTreeNode::new("MergeInto".to_string())), PhysicalPlan::MergeIntoSource(_) => Ok(FormatTreeNode::new("MergeIntoSource".to_string())), - PhysicalPlan::MergeIntoAppend(_) => Ok(FormatTreeNode::new("MergeIntoAppend".to_string())), PhysicalPlan::MergeIntoRowIdApply(_) => { Ok(FormatTreeNode::new("MergeIntoRowIdApply".to_string())) } diff --git a/src/query/sql/src/executor/mod.rs b/src/query/sql/src/executor/mod.rs index e88f29c31574..28ce154f62e7 100644 --- a/src/query/sql/src/executor/mod.rs +++ b/src/query/sql/src/executor/mod.rs @@ -57,7 +57,6 @@ pub use physical_plans::physical_limit::Limit; pub use physical_plans::physical_materialized_cte::MaterializedCte; pub use physical_plans::physical_merge_into::MatchExpr; pub use physical_plans::physical_merge_into::MergeInto; -pub use physical_plans::physical_merge_into::MergeIntoAppend; pub use physical_plans::physical_merge_into::MergeIntoRowIdApply; pub use physical_plans::physical_merge_into::MergeIntoSource; pub use physical_plans::physical_project::Project; diff --git a/src/query/sql/src/executor/physical_plan.rs b/src/query/sql/src/executor/physical_plan.rs index 841197d45b8b..1c42b06a92a2 100644 --- a/src/query/sql/src/executor/physical_plan.rs +++ b/src/query/sql/src/executor/physical_plan.rs @@ -17,7 +17,6 @@ use common_exception::Result; use common_expression::DataSchemaRef; use enum_as_inner::EnumAsInner; -use super::MergeIntoAppend; use super::MergeIntoRowIdApply; use crate::executor::physical_plans::physical_aggregate_expand::AggregateExpand; use crate::executor::physical_plans::physical_aggregate_final::AggregateFinal; @@ -99,7 +98,6 @@ pub enum PhysicalPlan { /// MergeInto MergeIntoSource(MergeIntoSource), MergeInto(Box), - MergeIntoAppend(Box), MergeIntoRowIdApply(Box), /// Compact @@ -138,7 +136,6 @@ impl PhysicalPlan { PhysicalPlan::DeletePartial(_) | PhysicalPlan::MergeInto(_) | PhysicalPlan::MergeIntoSource(_) - | PhysicalPlan::MergeIntoAppend(_) | PhysicalPlan::MergeIntoRowIdApply(_) | PhysicalPlan::CommitSink(_) | PhysicalPlan::CopyIntoTable(_) @@ -181,12 +178,12 @@ impl PhysicalPlan { PhysicalPlan::MaterializedCte(plan) => plan.output_schema(), PhysicalPlan::ConstantTableScan(plan) => plan.output_schema(), PhysicalPlan::MergeIntoSource(plan) => plan.input.output_schema(), - PhysicalPlan::MergeIntoRowIdApply(_) => todo!(), - PhysicalPlan::MergeIntoAppend(_) => todo!(), + PhysicalPlan::AsyncSourcer(_) | PhysicalPlan::MergeInto(_) | PhysicalPlan::Deduplicate(_) | PhysicalPlan::ReplaceInto(_) + | PhysicalPlan::MergeIntoRowIdApply(_) | PhysicalPlan::CompactPartial(_) => Ok(DataSchemaRef::default()), } } @@ -223,7 +220,6 @@ impl PhysicalPlan { PhysicalPlan::ReplaceInto(_) => "Replace".to_string(), PhysicalPlan::MergeInto(_) => "MergeInto".to_string(), PhysicalPlan::MergeIntoSource(_) => "MergeIntoSource".to_string(), - PhysicalPlan::MergeIntoAppend(_) => "MergeIntoAppend".to_string(), PhysicalPlan::MergeIntoRowIdApply(_) => "MergeIntoRowIdApply".to_string(), PhysicalPlan::CteScan(_) => "PhysicalCteScan".to_string(), PhysicalPlan::MaterializedCte(_) => "PhysicalMaterializedCte".to_string(), @@ -276,7 +272,6 @@ impl PhysicalPlan { PhysicalPlan::ReplaceInto(plan) => Box::new(std::iter::once(plan.input.as_ref())), PhysicalPlan::MergeInto(plan) => Box::new(std::iter::once(plan.input.as_ref())), PhysicalPlan::MergeIntoSource(plan) => Box::new(std::iter::once(plan.input.as_ref())), - PhysicalPlan::MergeIntoAppend(plan) => Box::new(std::iter::once(plan.input.as_ref())), PhysicalPlan::MergeIntoRowIdApply(plan) => { Box::new(std::iter::once(plan.input.as_ref())) } @@ -319,7 +314,6 @@ impl PhysicalPlan { | PhysicalPlan::Deduplicate(_) | PhysicalPlan::ReplaceInto(_) | PhysicalPlan::MergeInto(_) - | PhysicalPlan::MergeIntoAppend(_) | PhysicalPlan::MergeIntoRowIdApply(_) | PhysicalPlan::MergeIntoSource(_) | PhysicalPlan::ConstantTableScan(_) diff --git a/src/query/sql/src/executor/physical_plan_display.rs b/src/query/sql/src/executor/physical_plan_display.rs index 3df4aebbf05a..7f0acb5b4b7e 100644 --- a/src/query/sql/src/executor/physical_plan_display.rs +++ b/src/query/sql/src/executor/physical_plan_display.rs @@ -18,7 +18,6 @@ use std::fmt::Formatter; use common_functions::BUILTIN_FUNCTIONS; use itertools::Itertools; -use super::MergeIntoAppend; use super::MergeIntoRowIdApply; use crate::executor::physical_plans::physical_aggregate_expand::AggregateExpand; use crate::executor::physical_plans::physical_aggregate_final::AggregateFinal; @@ -102,7 +101,6 @@ impl<'a> Display for PhysicalPlanIndentFormatDisplay<'a> { PhysicalPlan::ReplaceInto(replace) => write!(f, "{}", replace)?, PhysicalPlan::MergeIntoSource(merge_into_source) => write!(f, "{}", merge_into_source)?, PhysicalPlan::MergeInto(merge_into) => write!(f, "{}", merge_into)?, - PhysicalPlan::MergeIntoAppend(merge_into_append) => write!(f, "{}", merge_into_append)?, PhysicalPlan::MergeIntoRowIdApply(merge_into_row_id_apply) => { write!(f, "{}", merge_into_row_id_apply)? } @@ -476,12 +474,6 @@ impl Display for MergeInto { } } -impl Display for MergeIntoAppend { - fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { - write!(f, "MergeIntoAppend") - } -} - impl Display for MergeIntoRowIdApply { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { write!(f, "MergeIntoRowIdApply") diff --git a/src/query/sql/src/executor/physical_plan_visitor.rs b/src/query/sql/src/executor/physical_plan_visitor.rs index 2be2507b8a90..3eefe8064271 100644 --- a/src/query/sql/src/executor/physical_plan_visitor.rs +++ b/src/query/sql/src/executor/physical_plan_visitor.rs @@ -14,7 +14,6 @@ use common_exception::Result; -use super::MergeIntoAppend; use super::MergeIntoRowIdApply; use crate::executor::physical_plans::physical_aggregate_expand::AggregateExpand; use crate::executor::physical_plans::physical_aggregate_final::AggregateFinal; @@ -87,7 +86,6 @@ pub trait PhysicalPlanReplacer { PhysicalPlan::ReplaceInto(plan) => self.replace_replace_into(plan), PhysicalPlan::MergeInto(plan) => self.replace_merge_into(plan), PhysicalPlan::MergeIntoSource(plan) => self.replace_merge_into_source(plan), - PhysicalPlan::MergeIntoAppend(plan) => self.replace_merge_into_append(plan), PhysicalPlan::MergeIntoRowIdApply(plan) => self.replace_merge_into_row_id_apply(plan), PhysicalPlan::MaterializedCte(plan) => self.replace_materialized_cte(plan), PhysicalPlan::ConstantTableScan(plan) => self.replace_constant_table_scan(plan), @@ -426,14 +424,6 @@ pub trait PhysicalPlanReplacer { })) } - fn replace_merge_into_append(&mut self, plan: &MergeIntoAppend) -> Result { - let input = self.replace(&plan.input)?; - Ok(PhysicalPlan::MergeIntoAppend(Box::new(MergeIntoAppend { - input: Box::new(input), - ..plan.clone() - }))) - } - fn replace_merge_into_row_id_apply( &mut self, plan: &MergeIntoRowIdApply, @@ -583,9 +573,7 @@ impl PhysicalPlan { PhysicalPlan::MergeInto(plan) => { Self::traverse(&plan.input, pre_visit, visit, post_visit); } - PhysicalPlan::MergeIntoAppend(plan) => { - Self::traverse(&plan.input, pre_visit, visit, post_visit); - } + PhysicalPlan::MergeIntoRowIdApply(plan) => { Self::traverse(&plan.input, pre_visit, visit, post_visit); } diff --git a/src/query/sql/src/executor/physical_plans/physical_merge_into.rs b/src/query/sql/src/executor/physical_plans/physical_merge_into.rs index ad496ba06912..0a5b84b5c04b 100644 --- a/src/query/sql/src/executor/physical_plans/physical_merge_into.rs +++ b/src/query/sql/src/executor/physical_plans/physical_merge_into.rs @@ -32,7 +32,6 @@ pub struct MergeIntoSource { pub row_id_idx: u32, } -// Todo:(JackTan25) replace MergeInto with MergeIntoAppend and MergeIntoRowIdApply #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct MergeInto { pub input: Box, @@ -46,26 +45,13 @@ pub struct MergeInto { // used to record the index of target table's field in merge_source_schema pub field_index_of_input_schema: HashMap, pub row_id_idx: usize, - pub segments: Vec<(usize, Location)>, + pub segments: Option>, } #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct MergeIntoAppend { +pub struct MergeIntoRowIdApply { pub input: Box, pub table_info: TableInfo, pub catalog_info: CatalogInfo, - // (DataSchemaRef, Option, Vec,Vec) => (source_schema, condition, value_exprs) - pub unmatched: Vec<(DataSchemaRef, Option, Vec)>, - // the first option stands for the condition - // the second option stands for update/delete - pub matched: MatchExpr, - // used to record the index of target table's field in merge_source_schema - pub field_index_of_input_schema: HashMap, - pub row_id_idx: usize, pub segments: Vec<(usize, Location)>, } - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct MergeIntoRowIdApply { - pub input: Box, -} diff --git a/src/query/sql/src/executor/profile.rs b/src/query/sql/src/executor/profile.rs index 8dfc21350aff..f48dd83f0982 100644 --- a/src/query/sql/src/executor/profile.rs +++ b/src/query/sql/src/executor/profile.rs @@ -510,7 +510,6 @@ fn flatten_plan_node_profile( | PhysicalPlan::CopyIntoTable(_) | PhysicalPlan::AsyncSourcer(_) | PhysicalPlan::MergeInto(_) - | PhysicalPlan::MergeIntoAppend(_) | PhysicalPlan::MergeIntoRowIdApply(_) | PhysicalPlan::MergeIntoSource(_) | PhysicalPlan::Deduplicate(_) diff --git a/src/query/storages/fuse/src/operations/merge.rs b/src/query/storages/fuse/src/operations/merge.rs index 0724a322df60..5994b25918fe 100644 --- a/src/query/storages/fuse/src/operations/merge.rs +++ b/src/query/storages/fuse/src/operations/merge.rs @@ -77,6 +77,7 @@ impl FuseTable { block_builder: BlockBuilder, io_request_semaphore: Arc, segment_locations: Vec<(SegmentIndex, Location)>, + distributed_recieve: bool, ) -> Result { let read_settings = ReadSettings::from_ctx(&ctx)?; let aggregator = MatchedAggregator::create( @@ -88,6 +89,7 @@ impl FuseTable { block_builder, io_request_semaphore, segment_locations, + distributed_recieve, )?; Ok(aggregator.into_pipe_item()) } diff --git a/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs b/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs index bbdb8fbec0b3..9e011c6327e0 100644 --- a/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs +++ b/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs @@ -81,6 +81,8 @@ pub struct MatchedAggregator { segment_locations: AHashMap, block_mutation_row_offset: HashMap, HashSet)>, aggregation_ctx: Arc, + entries: Vec, + distributed_recieve: bool, } impl MatchedAggregator { @@ -94,9 +96,11 @@ impl MatchedAggregator { block_builder: BlockBuilder, io_request_semaphore: Arc, segment_locations: Vec<(SegmentIndex, Location)>, + distributed_recieve: bool, ) -> Result { let segment_reader = MetaReaders::segment_info_reader(data_accessor.clone(), target_table_schema.clone()); + let block_reader = { let projection = Projection::Columns((0..target_table_schema.num_fields()).collect_vec()); @@ -122,15 +126,25 @@ impl MatchedAggregator { segment_reader, block_mutation_row_offset: HashMap::new(), segment_locations: AHashMap::from_iter(segment_locations.into_iter()), + entries: Vec::new(), + distributed_recieve, }) } #[async_backtrace::framed] pub async fn accumulate(&mut self, data_block: DataBlock) -> Result<()> { - let start = Instant::now(); + // we need to distinct MutationLogs and RowIds + // this operation is lightweight. + let logs = MutationLogs::try_from(data_block.clone()); + if self.distributed_recieve && logs.is_ok() { + self.entries.extend(logs.unwrap().entries); + return Ok(()); + } + if data_block.is_empty() { return Ok(()); } + let start = Instant::now(); // data_block is from matched_split, so there is only one column. // that's row_id let row_ids = get_row_id(&data_block, 0)?; @@ -258,7 +272,7 @@ impl MatchedAggregator { ErrorCode::Internal("unexpected, failed to join apply-deletion tasks.") .add_message_back(e.to_string()) })?; - let mut mutation_logs = Vec::new(); + let mut mutation_logs: Vec = self.entries.drain(..).into_iter().collect(); for maybe_log_entry in log_entries { if let Some(segment_mutation_log) = maybe_log_entry? { mutation_logs.push(segment_mutation_log); From 76e135213355931cb588e6dd5c364215b762b105 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Mon, 16 Oct 2023 22:09:10 +0800 Subject: [PATCH 07/39] fix typo --- src/query/service/src/pipelines/pipeline_builder.rs | 4 ++-- src/query/sql/src/planner/binder/merge_into.rs | 2 +- src/query/storages/fuse/src/operations/merge.rs | 4 ++-- .../src/operations/merge_into/mutator/matched_mutator.rs | 8 ++++---- 4 files changed, 9 insertions(+), 9 deletions(-) diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs index 0d7088e38ead..8d16c27ad0de 100644 --- a/src/query/service/src/pipelines/pipeline_builder.rs +++ b/src/query/service/src/pipelines/pipeline_builder.rs @@ -316,7 +316,7 @@ impl PipelineBuilder { segments, } = merge_into_row_id_apply; - // recieve rowids and MutationLogs + // receive rowids and MutationLogs self.build_pipeline(input)?; let mut pipe_items = Vec::with_capacity(self.main_pipeline.output_len()); let tbl = self @@ -709,7 +709,7 @@ impl PipelineBuilder { false, )?); } else { - // distributed exectution + // distributed execution pipe_items.push(create_dummy_item()) } diff --git a/src/query/sql/src/planner/binder/merge_into.rs b/src/query/sql/src/planner/binder/merge_into.rs index 4ecce28fa128..cc4185245a42 100644 --- a/src/query/sql/src/planner/binder/merge_into.rs +++ b/src/query/sql/src/planner/binder/merge_into.rs @@ -363,7 +363,7 @@ impl Binder { normalize_identifier(tbl_identify, &self.name_resolution_ctx).name; if &update_table_name != target_name { return Err(ErrorCode::BadArguments(format!( - "Update Indentify's `{}` should be `{}`", + "Update Identify's `{}` should be `{}`", update_table_name, target_name ))); } diff --git a/src/query/storages/fuse/src/operations/merge.rs b/src/query/storages/fuse/src/operations/merge.rs index 5994b25918fe..e404561ea96f 100644 --- a/src/query/storages/fuse/src/operations/merge.rs +++ b/src/query/storages/fuse/src/operations/merge.rs @@ -77,7 +77,7 @@ impl FuseTable { block_builder: BlockBuilder, io_request_semaphore: Arc, segment_locations: Vec<(SegmentIndex, Location)>, - distributed_recieve: bool, + distributed_receive: bool, ) -> Result { let read_settings = ReadSettings::from_ctx(&ctx)?; let aggregator = MatchedAggregator::create( @@ -89,7 +89,7 @@ impl FuseTable { block_builder, io_request_semaphore, segment_locations, - distributed_recieve, + distributed_receive, )?; Ok(aggregator.into_pipe_item()) } diff --git a/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs b/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs index 9e011c6327e0..f418ed8e7976 100644 --- a/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs +++ b/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs @@ -82,7 +82,7 @@ pub struct MatchedAggregator { block_mutation_row_offset: HashMap, HashSet)>, aggregation_ctx: Arc, entries: Vec, - distributed_recieve: bool, + distributed_receive: bool, } impl MatchedAggregator { @@ -96,7 +96,7 @@ impl MatchedAggregator { block_builder: BlockBuilder, io_request_semaphore: Arc, segment_locations: Vec<(SegmentIndex, Location)>, - distributed_recieve: bool, + distributed_receive: bool, ) -> Result { let segment_reader = MetaReaders::segment_info_reader(data_accessor.clone(), target_table_schema.clone()); @@ -127,7 +127,7 @@ impl MatchedAggregator { block_mutation_row_offset: HashMap::new(), segment_locations: AHashMap::from_iter(segment_locations.into_iter()), entries: Vec::new(), - distributed_recieve, + distributed_receive, }) } @@ -136,7 +136,7 @@ impl MatchedAggregator { // we need to distinct MutationLogs and RowIds // this operation is lightweight. let logs = MutationLogs::try_from(data_block.clone()); - if self.distributed_recieve && logs.is_ok() { + if self.distributed_receive && logs.is_ok() { self.entries.extend(logs.unwrap().entries); return Ok(()); } From e4f7450ffeb224c6bac5e7ffe2cac4cb902c5192 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Wed, 18 Oct 2023 00:10:27 +0800 Subject: [PATCH 08/39] uniform row_kind and mutation_log --- .../interpreters/interpreter_merge_into.rs | 5 + .../service/src/pipelines/pipeline_builder.rs | 20 +++- src/query/sql/src/executor/physical_plan.rs | 3 +- .../physical_plans/physical_merge_into.rs | 1 + .../fuse/src/operations/merge_into/mod.rs | 2 + .../operations/merge_into/processors/mod.rs | 4 + ...istributed_merge_into_block_deserialize.rs | 82 ++++++++++++++++ ..._distributed_merge_into_block_serialize.rs | 94 +++++++++++++++++++ .../processor_merge_into_matched_and_split.rs | 20 +++- 9 files changed, 226 insertions(+), 5 deletions(-) create mode 100644 src/query/storages/fuse/src/operations/merge_into/processors/processor_distributed_merge_into_block_deserialize.rs create mode 100644 src/query/storages/fuse/src/operations/merge_into/processors/processor_distributed_merge_into_block_serialize.rs diff --git a/src/query/service/src/interpreters/interpreter_merge_into.rs b/src/query/service/src/interpreters/interpreter_merge_into.rs index c863a963437a..9e4f07b8c5e9 100644 --- a/src/query/service/src/interpreters/interpreter_merge_into.rs +++ b/src/query/service/src/interpreters/interpreter_merge_into.rs @@ -21,6 +21,7 @@ use common_base::runtime::GlobalIORuntime; use common_exception::ErrorCode; use common_exception::Result; use common_expression::ConstantFolder; +use common_expression::DataSchema; use common_expression::DataSchemaRef; use common_expression::FieldIndex; use common_expression::RemoteExpr; @@ -340,6 +341,7 @@ impl MergeIntoInterpreter { field_index_of_input_schema, row_id_idx, segments: Some(segments), + output_schema: DataSchemaRef::default(), })) } else { let merge_append = PhysicalPlan::MergeInto(Box::new(MergeInto { @@ -351,6 +353,9 @@ impl MergeIntoInterpreter { field_index_of_input_schema, row_id_idx, segments: None, + output_schema: DataSchemaRef::new(DataSchema::new(vec![ + join_output_schema.fields[row_id_idx].clone(), + ])), })); let exchange = exchange.unwrap(); PhysicalPlan::MergeIntoRowIdApply(Box::new(MergeIntoRowIdApply { diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs index 8d16c27ad0de..e276142ec0d8 100644 --- a/src/query/service/src/pipelines/pipeline_builder.rs +++ b/src/query/service/src/pipelines/pipeline_builder.rs @@ -121,6 +121,8 @@ use common_storages_fuse::operations::common::TransformSerializeSegment; use common_storages_fuse::operations::merge_into::MatchedSplitProcessor; use common_storages_fuse::operations::merge_into::MergeIntoNotMatchedProcessor; use common_storages_fuse::operations::merge_into::MergeIntoSplitProcessor; +use common_storages_fuse::operations::merge_into::TransformDistributedMergeIntoBlockDeserialize; +use common_storages_fuse::operations::merge_into::TransformDistributedMergeIntoBlockSerialize; use common_storages_fuse::operations::replace_into::BroadcastProcessor; use common_storages_fuse::operations::replace_into::ReplaceIntoProcessor; use common_storages_fuse::operations::replace_into::UnbranchedReplaceIntoProcessor; @@ -318,7 +320,12 @@ impl PipelineBuilder { } = merge_into_row_id_apply; // receive rowids and MutationLogs self.build_pipeline(input)?; - let mut pipe_items = Vec::with_capacity(self.main_pipeline.output_len()); + self.main_pipeline.try_resize(1)?; + + self.main_pipeline + .add_pipe(TransformDistributedMergeIntoBlockDeserialize::into_pipe()); + + let mut pipe_items = Vec::with_capacity(1); let tbl = self .ctx .build_table_by_table_info(catalog_info, table_info, None)?; @@ -351,7 +358,9 @@ impl PipelineBuilder { true, )?); - todo!() + self.main_pipeline + .add_pipe(Pipe::create(self.main_pipeline.output_len(), 1, pipe_items)); + Ok(()) } fn build_merge_into_source(&mut self, merge_into_source: &MergeIntoSource) -> Result<()> { @@ -516,6 +525,7 @@ impl PipelineBuilder { field_index_of_input_schema, row_id_idx, segments, + .. } = merge_into; self.build_pipeline(input)?; @@ -762,6 +772,12 @@ impl PipelineBuilder { pipe_items, )); + // distributed execution + if !apply_row_id { + self.main_pipeline.try_resize(1)?; + self.main_pipeline + .add_pipe(TransformDistributedMergeIntoBlockSerialize::into_pipe()) + } Ok(()) } diff --git a/src/query/sql/src/executor/physical_plan.rs b/src/query/sql/src/executor/physical_plan.rs index 1c42b06a92a2..687e49ecfb45 100644 --- a/src/query/sql/src/executor/physical_plan.rs +++ b/src/query/sql/src/executor/physical_plan.rs @@ -178,9 +178,8 @@ impl PhysicalPlan { PhysicalPlan::MaterializedCte(plan) => plan.output_schema(), PhysicalPlan::ConstantTableScan(plan) => plan.output_schema(), PhysicalPlan::MergeIntoSource(plan) => plan.input.output_schema(), - + PhysicalPlan::MergeInto(plan) => Ok(plan.output_schema.clone()), PhysicalPlan::AsyncSourcer(_) - | PhysicalPlan::MergeInto(_) | PhysicalPlan::Deduplicate(_) | PhysicalPlan::ReplaceInto(_) | PhysicalPlan::MergeIntoRowIdApply(_) diff --git a/src/query/sql/src/executor/physical_plans/physical_merge_into.rs b/src/query/sql/src/executor/physical_plans/physical_merge_into.rs index 0a5b84b5c04b..31ae6a1bb3f3 100644 --- a/src/query/sql/src/executor/physical_plans/physical_merge_into.rs +++ b/src/query/sql/src/executor/physical_plans/physical_merge_into.rs @@ -46,6 +46,7 @@ pub struct MergeInto { pub field_index_of_input_schema: HashMap, pub row_id_idx: usize, pub segments: Option>, + pub output_schema: DataSchemaRef, } #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] diff --git a/src/query/storages/fuse/src/operations/merge_into/mod.rs b/src/query/storages/fuse/src/operations/merge_into/mod.rs index 36b207cd1900..fc68baac7203 100644 --- a/src/query/storages/fuse/src/operations/merge_into/mod.rs +++ b/src/query/storages/fuse/src/operations/merge_into/mod.rs @@ -19,3 +19,5 @@ pub use mutator::MatchedAggregator; pub use processors::MatchedSplitProcessor; pub use processors::MergeIntoNotMatchedProcessor; pub use processors::MergeIntoSplitProcessor; +pub use processors::TransformDistributedMergeIntoBlockDeserialize; +pub use processors::TransformDistributedMergeIntoBlockSerialize; diff --git a/src/query/storages/fuse/src/operations/merge_into/processors/mod.rs b/src/query/storages/fuse/src/operations/merge_into/processors/mod.rs index 6425672d538f..7e6fe208619e 100644 --- a/src/query/storages/fuse/src/operations/merge_into/processors/mod.rs +++ b/src/query/storages/fuse/src/operations/merge_into/processors/mod.rs @@ -12,10 +12,14 @@ // See the License for the specific language governing permissions and // limitations under the License. +mod processor_distributed_merge_into_block_deserialize; +mod processor_distributed_merge_into_block_serialize; mod processor_merge_into_matched_and_split; mod processor_merge_into_not_matched; mod processor_merge_into_split; mod transform_matched_mutation_aggregator; +pub use processor_distributed_merge_into_block_deserialize::TransformDistributedMergeIntoBlockDeserialize; +pub use processor_distributed_merge_into_block_serialize::TransformDistributedMergeIntoBlockSerialize; pub use processor_merge_into_matched_and_split::MatchedSplitProcessor; pub(crate) use processor_merge_into_matched_and_split::RowIdKind; pub use processor_merge_into_not_matched::MergeIntoNotMatchedProcessor; diff --git a/src/query/storages/fuse/src/operations/merge_into/processors/processor_distributed_merge_into_block_deserialize.rs b/src/query/storages/fuse/src/operations/merge_into/processors/processor_distributed_merge_into_block_deserialize.rs new file mode 100644 index 000000000000..9d1570f3bff5 --- /dev/null +++ b/src/query/storages/fuse/src/operations/merge_into/processors/processor_distributed_merge_into_block_deserialize.rs @@ -0,0 +1,82 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::sync::Arc; + +use common_exception::Result; +use common_expression::BlockMetaInfoDowncast; +use common_expression::DataBlock; +use common_pipeline_core::pipe::Pipe; +use common_pipeline_core::pipe::PipeItem; +use common_pipeline_core::processors::port::InputPort; +use common_pipeline_core::processors::port::OutputPort; +use common_pipeline_core::processors::processor::ProcessorPtr; +use common_pipeline_transforms::processors::transforms::Transform; +use common_pipeline_transforms::processors::transforms::Transformer; + +use super::processor_merge_into_matched_and_split::MixRowIdKindAndLog; + +// It will recieve MutationLogs Or RowIds. +// But for MutationLogs, it's a empty block +// we will add a fake BlockEntry to make it consistent with +// RowIds, because arrow-flight requires this. +pub struct TransformDistributedMergeIntoBlockDeserialize; + +impl TransformDistributedMergeIntoBlockDeserialize { + #[allow(dead_code)] + fn create(input: Arc, output: Arc) -> ProcessorPtr { + ProcessorPtr::create(Transformer::create( + input, + output, + TransformDistributedMergeIntoBlockDeserialize {}, + )) + } + + fn create_distributed_merge_into_transform_item() -> PipeItem { + let input = InputPort::create(); + let output = OutputPort::create(); + PipeItem::create( + TransformDistributedMergeIntoBlockDeserialize::create(input.clone(), output.clone()), + vec![input], + vec![output], + ) + } + + pub fn into_pipe() -> Pipe { + let pipe_item = Self::create_distributed_merge_into_transform_item(); + Pipe::create(1, 1, vec![pipe_item]) + } +} + +#[async_trait::async_trait] +impl Transform for TransformDistributedMergeIntoBlockDeserialize { + const NAME: &'static str = "TransformDistributedMergeIntoBlockDeserialize"; + + fn transform(&mut self, data: DataBlock) -> Result { + let kind = MixRowIdKindAndLog::downcast_ref_from(data.get_meta().unwrap()).unwrap(); + let data_block = match kind { + MixRowIdKindAndLog::MutationLogs(logs) => DataBlock::new_with_meta( + data.columns().to_vec(), + data.num_rows(), + Some(Box::new(logs.clone())), + ), + MixRowIdKindAndLog::RowIdKind(row_id_kind) => DataBlock::new_with_meta( + data.columns().to_vec(), + data.num_rows(), + Some(Box::new(row_id_kind.clone())), + ), + }; + Ok(data_block) + } +} diff --git a/src/query/storages/fuse/src/operations/merge_into/processors/processor_distributed_merge_into_block_serialize.rs b/src/query/storages/fuse/src/operations/merge_into/processors/processor_distributed_merge_into_block_serialize.rs new file mode 100644 index 000000000000..3fed939d9e9d --- /dev/null +++ b/src/query/storages/fuse/src/operations/merge_into/processors/processor_distributed_merge_into_block_serialize.rs @@ -0,0 +1,94 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::sync::Arc; + +use common_exception::Result; +use common_expression::types::DataType; +use common_expression::types::NumberDataType::UInt64; +use common_expression::types::NumberType; +use common_expression::BlockEntry; +use common_expression::BlockMetaInfoDowncast; +use common_expression::DataBlock; +use common_expression::Value; +use common_pipeline_core::pipe::Pipe; +use common_pipeline_core::pipe::PipeItem; +use common_pipeline_core::processors::port::InputPort; +use common_pipeline_core::processors::port::OutputPort; +use common_pipeline_core::processors::processor::ProcessorPtr; +use common_pipeline_transforms::processors::transforms::Transform; +use common_pipeline_transforms::processors::transforms::Transformer; + +use super::processor_merge_into_matched_and_split::MixRowIdKindAndLog; +use super::RowIdKind; +use crate::operations::common::MutationLogs; + +// It will recieve MutationLogs Or RowIds. +// But for MutationLogs, it's a empty block +// we will add a fake BlockEntry to make it consistent with +// RowIds, because arrow-flight requires this. +pub struct TransformDistributedMergeIntoBlockSerialize; + +impl TransformDistributedMergeIntoBlockSerialize { + #[allow(dead_code)] + fn create(input: Arc, output: Arc) -> ProcessorPtr { + ProcessorPtr::create(Transformer::create( + input, + output, + TransformDistributedMergeIntoBlockSerialize {}, + )) + } + + fn create_distributed_merge_into_transform_item() -> PipeItem { + let input = InputPort::create(); + let output = OutputPort::create(); + PipeItem::create( + TransformDistributedMergeIntoBlockSerialize::create(input.clone(), output.clone()), + vec![input], + vec![output], + ) + } + + pub fn into_pipe() -> Pipe { + let pipe_item = Self::create_distributed_merge_into_transform_item(); + Pipe::create(1, 1, vec![pipe_item]) + } +} + +#[async_trait::async_trait] +impl Transform for TransformDistributedMergeIntoBlockSerialize { + const NAME: &'static str = "TransformDistributedMergeIntoBlockSerialize"; + + fn transform(&mut self, data: DataBlock) -> Result { + // 1. MutationLogs + if data.is_empty() { + let scalar_value = Value::>::Scalar(0); + let entry = BlockEntry::new(DataType::Number(UInt64), scalar_value.upcast()); + let log = MutationLogs::try_from(data)?; + Ok(DataBlock::new_with_meta( + vec![entry], + 1, + Some(Box::new(MixRowIdKindAndLog::MutationLogs(log))), + )) + } else { + // RowIdKind + let row_id_kind = RowIdKind::downcast_ref_from(data.get_meta().unwrap()).unwrap(); + Ok(DataBlock::new_with_meta( + data.columns().to_vec(), + 1, + Some(Box::new(MixRowIdKindAndLog::RowIdKind(row_id_kind.clone()))), + )) + } + } +} diff --git a/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_matched_and_split.rs b/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_matched_and_split.rs index 79e0a7dead5b..a16b28c2b20e 100644 --- a/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_matched_and_split.rs +++ b/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_matched_and_split.rs @@ -39,6 +39,7 @@ use common_storage::metrics::merge_into::merge_into_matched_operation_millisecon use common_storage::metrics::merge_into::metrics_inc_merge_into_append_blocks_counter; use common_storage::metrics::merge_into::metrics_inc_merge_into_append_blocks_rows_counter; +use crate::operations::common::MutationLogs; use crate::operations::merge_into::mutator::DeleteByExprMutator; use crate::operations::merge_into::mutator::UpdateByExprMutator; @@ -47,7 +48,24 @@ enum MutationKind { Delete(DeleteDataBlockMutation), } -#[derive(serde::Serialize, serde::Deserialize, Clone, Debug, PartialEq, Eq)] +#[derive(serde::Serialize, serde::Deserialize, Clone, Debug, PartialEq)] +pub enum MixRowIdKindAndLog { + MutationLogs(MutationLogs), + RowIdKind(RowIdKind), +} + +#[typetag::serde(name = "mix_row_id_kind_and_log")] +impl BlockMetaInfo for MixRowIdKindAndLog { + fn equals(&self, info: &Box) -> bool { + MixRowIdKindAndLog::downcast_ref_from(info).is_some_and(|other| self == other) + } + + fn clone_self(&self) -> Box { + Box::new(self.clone()) + } +} + +#[derive(serde::Serialize, serde::Deserialize, Clone, Debug, PartialEq)] pub enum RowIdKind { Update, Delete, From 93a1cdf31dcba10c96afb602a15ee9f13a8687c5 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Wed, 18 Oct 2023 14:17:25 +0800 Subject: [PATCH 09/39] fix MixRowKindAndLog serialize and deserialize --- .../fuse/src/operations/merge_into/mod.rs | 1 + .../operations/merge_into/processors/mod.rs | 1 + ...istributed_merge_into_block_deserialize.rs | 30 ++++++++++++------- ..._distributed_merge_into_block_serialize.rs | 17 ++++++++--- .../processor_merge_into_matched_and_split.rs | 8 +++-- 5 files changed, 39 insertions(+), 18 deletions(-) diff --git a/src/query/storages/fuse/src/operations/merge_into/mod.rs b/src/query/storages/fuse/src/operations/merge_into/mod.rs index fc68baac7203..aee696b8c3a3 100644 --- a/src/query/storages/fuse/src/operations/merge_into/mod.rs +++ b/src/query/storages/fuse/src/operations/merge_into/mod.rs @@ -19,5 +19,6 @@ pub use mutator::MatchedAggregator; pub use processors::MatchedSplitProcessor; pub use processors::MergeIntoNotMatchedProcessor; pub use processors::MergeIntoSplitProcessor; +pub use processors::MixRowIdKindAndLog; pub use processors::TransformDistributedMergeIntoBlockDeserialize; pub use processors::TransformDistributedMergeIntoBlockSerialize; diff --git a/src/query/storages/fuse/src/operations/merge_into/processors/mod.rs b/src/query/storages/fuse/src/operations/merge_into/processors/mod.rs index 7e6fe208619e..0a2a2654b7d9 100644 --- a/src/query/storages/fuse/src/operations/merge_into/processors/mod.rs +++ b/src/query/storages/fuse/src/operations/merge_into/processors/mod.rs @@ -21,6 +21,7 @@ mod transform_matched_mutation_aggregator; pub use processor_distributed_merge_into_block_deserialize::TransformDistributedMergeIntoBlockDeserialize; pub use processor_distributed_merge_into_block_serialize::TransformDistributedMergeIntoBlockSerialize; pub use processor_merge_into_matched_and_split::MatchedSplitProcessor; +pub use processor_merge_into_matched_and_split::MixRowIdKindAndLog; pub(crate) use processor_merge_into_matched_and_split::RowIdKind; pub use processor_merge_into_not_matched::MergeIntoNotMatchedProcessor; pub use processor_merge_into_split::MergeIntoSplitProcessor; diff --git a/src/query/storages/fuse/src/operations/merge_into/processors/processor_distributed_merge_into_block_deserialize.rs b/src/query/storages/fuse/src/operations/merge_into/processors/processor_distributed_merge_into_block_deserialize.rs index 9d1570f3bff5..e9d1211c2ca4 100644 --- a/src/query/storages/fuse/src/operations/merge_into/processors/processor_distributed_merge_into_block_deserialize.rs +++ b/src/query/storages/fuse/src/operations/merge_into/processors/processor_distributed_merge_into_block_deserialize.rs @@ -14,6 +14,7 @@ use std::sync::Arc; +use common_exception::ErrorCode; use common_exception::Result; use common_expression::BlockMetaInfoDowncast; use common_expression::DataBlock; @@ -26,8 +27,9 @@ use common_pipeline_transforms::processors::transforms::Transform; use common_pipeline_transforms::processors::transforms::Transformer; use super::processor_merge_into_matched_and_split::MixRowIdKindAndLog; +use super::RowIdKind; -// It will recieve MutationLogs Or RowIds. +// It will receive MutationLogs Or RowIds. // But for MutationLogs, it's a empty block // we will add a fake BlockEntry to make it consistent with // RowIds, because arrow-flight requires this. @@ -64,19 +66,25 @@ impl Transform for TransformDistributedMergeIntoBlockDeserialize { const NAME: &'static str = "TransformDistributedMergeIntoBlockDeserialize"; fn transform(&mut self, data: DataBlock) -> Result { - let kind = MixRowIdKindAndLog::downcast_ref_from(data.get_meta().unwrap()).unwrap(); - let data_block = match kind { - MixRowIdKindAndLog::MutationLogs(logs) => DataBlock::new_with_meta( + let mix_kind = MixRowIdKindAndLog::downcast_ref_from(data.get_meta().unwrap()).unwrap(); + match mix_kind.kind { + 0 => Ok(DataBlock::new_with_meta( data.columns().to_vec(), data.num_rows(), - Some(Box::new(logs.clone())), - ), - MixRowIdKindAndLog::RowIdKind(row_id_kind) => DataBlock::new_with_meta( + Some(Box::new(mix_kind.log.clone().unwrap())), + )), + + 1 => Ok(DataBlock::new_with_meta( + data.columns().to_vec(), + data.num_rows(), + Some(Box::new(RowIdKind::Update)), + )), + 2 => Ok(DataBlock::new_with_meta( data.columns().to_vec(), data.num_rows(), - Some(Box::new(row_id_kind.clone())), - ), - }; - Ok(data_block) + Some(Box::new(RowIdKind::Delete)), + )), + _ => Err(ErrorCode::BadBytes("get error MixRowIdKindAndLog kind")), + } } } diff --git a/src/query/storages/fuse/src/operations/merge_into/processors/processor_distributed_merge_into_block_serialize.rs b/src/query/storages/fuse/src/operations/merge_into/processors/processor_distributed_merge_into_block_serialize.rs index 3fed939d9e9d..da3214a6db10 100644 --- a/src/query/storages/fuse/src/operations/merge_into/processors/processor_distributed_merge_into_block_serialize.rs +++ b/src/query/storages/fuse/src/operations/merge_into/processors/processor_distributed_merge_into_block_serialize.rs @@ -34,7 +34,7 @@ use super::processor_merge_into_matched_and_split::MixRowIdKindAndLog; use super::RowIdKind; use crate::operations::common::MutationLogs; -// It will recieve MutationLogs Or RowIds. +// It will receive MutationLogs Or RowIds. // But for MutationLogs, it's a empty block // we will add a fake BlockEntry to make it consistent with // RowIds, because arrow-flight requires this. @@ -79,15 +79,24 @@ impl Transform for TransformDistributedMergeIntoBlockSerialize { Ok(DataBlock::new_with_meta( vec![entry], 1, - Some(Box::new(MixRowIdKindAndLog::MutationLogs(log))), + Some(Box::new(MixRowIdKindAndLog { + log: Some(log), + kind: 0, + })), )) } else { // RowIdKind let row_id_kind = RowIdKind::downcast_ref_from(data.get_meta().unwrap()).unwrap(); Ok(DataBlock::new_with_meta( data.columns().to_vec(), - 1, - Some(Box::new(MixRowIdKindAndLog::RowIdKind(row_id_kind.clone()))), + data.num_rows(), + Some(Box::new(MixRowIdKindAndLog { + log: None, + kind: match row_id_kind { + RowIdKind::Update => 1, + RowIdKind::Delete => 2, + }, + })), )) } } diff --git a/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_matched_and_split.rs b/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_matched_and_split.rs index a16b28c2b20e..9a9e750ed40e 100644 --- a/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_matched_and_split.rs +++ b/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_matched_and_split.rs @@ -49,9 +49,11 @@ enum MutationKind { } #[derive(serde::Serialize, serde::Deserialize, Clone, Debug, PartialEq)] -pub enum MixRowIdKindAndLog { - MutationLogs(MutationLogs), - RowIdKind(RowIdKind), +pub struct MixRowIdKindAndLog { + pub log: Option, + // kind's range is [0,1,2], 0 stands for log + // 1 stands for row_id_update, 2 stands for row_id_delete, + pub kind: usize, } #[typetag::serde(name = "mix_row_id_kind_and_log")] From 0a070b187b865465790701f2a9a9537dc9baf60b Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Wed, 18 Oct 2023 14:26:59 +0800 Subject: [PATCH 10/39] add tests --- .../09_0028_distributed_merge_into | 67 +++++++++++++++++++ 1 file changed, 67 insertions(+) create mode 100644 tests/sqllogictests/suites/base/09_fuse_engine/09_0028_distributed_merge_into diff --git a/tests/sqllogictests/suites/base/09_fuse_engine/09_0028_distributed_merge_into b/tests/sqllogictests/suites/base/09_fuse_engine/09_0028_distributed_merge_into new file mode 100644 index 000000000000..656e7029275d --- /dev/null +++ b/tests/sqllogictests/suites/base/09_fuse_engine/09_0028_distributed_merge_into @@ -0,0 +1,67 @@ +statement ok +set enable_experimental_merge_into = 1; + +statement ok +set enable_distributed_merge_into = 1; + +## distributed test +statement ok +drop table if exists distributed_target_test; + +statement ok +drop table if exists distributed_source_test; + +statement ok +create table distributed_target_test(a int,b string); + +statement ok +insert into distributed_target_test values(1,'a'),(2,'b'),(3,'c'); + +statement ok +insert into distributed_target_test values(6,'z'),(7,'y'),(8,'x'); + +query TT +select * from distributed_target_test order by a; +---- +1 a +2 b +3 c +6 z +7 y +8 x + +statement ok +create table distributed_source_test(a int,b string,is_databend_deleted bool); + +statement ok +insert into distributed_source_test values(1,'d',true),(2,'e',true),(3,'f',false),(4,'e',true),(5,'f',false); + +statement ok +insert into distributed_source_test values(6,'h',true),(7,'i',true),(8,'j',false),(9,'k',true),(10,'l',false); + +statement ok +create stage source_parquet file_format = (type = parquet); + +statement ok +remove @source_parquet; + +statement ok +copy into @source_parquet from (select * from distributed_source_test); + +statement ok +merge into `distributed_target_test` as tt using (select `a`,`b`,`is_databend_deleted` from @source_parquet (pattern => '.*[.]parquet')) as ss on (ss.`a` = tt.`a`) +when matched and ss.`is_databend_deleted` = true then delete when matched then update * when not matched and ss.`is_databend_deleted` = false then insert *; + +query TT +select * from distributed_target_test order by a; +---- +3 f +5 f +8 j +10 l + +statement ok +set enable_experimental_merge_into = 0; + +statement ok +set enable_distributed_merge_into = 0; \ No newline at end of file From f258ac0f8e554a68f661b33582fc268eae1f5737 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Wed, 18 Oct 2023 14:29:32 +0800 Subject: [PATCH 11/39] fix check --- src/query/sql/src/planner/binder/merge_into.rs | 2 +- src/query/sql/src/planner/optimizer/optimizer.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/query/sql/src/planner/binder/merge_into.rs b/src/query/sql/src/planner/binder/merge_into.rs index 5dba45a121d9..9ac7baf0e966 100644 --- a/src/query/sql/src/planner/binder/merge_into.rs +++ b/src/query/sql/src/planner/binder/merge_into.rs @@ -360,7 +360,7 @@ impl Binder { if let Some(tbl_identify) = &update_expr.table { let update_table_name = normalize_identifier(tbl_identify, &self.name_resolution_ctx).name; - if &update_table_name != target_name { + if update_table_name != target_name { return Err(ErrorCode::BadArguments(format!( "Update Identify's `{}` should be `{}`", update_table_name, target_name diff --git a/src/query/sql/src/planner/optimizer/optimizer.rs b/src/query/sql/src/planner/optimizer/optimizer.rs index 96c81c651c2f..982cc909c88c 100644 --- a/src/query/sql/src/planner/optimizer/optimizer.rs +++ b/src/query/sql/src/planner/optimizer/optimizer.rs @@ -133,7 +133,7 @@ pub fn optimize( // left join and right join // input is a Join_SExpr let optimized_distributed_join_sexpr = - optimize_distributed_query(ctx.clone(), &*plan.input)?; + optimize_distributed_query(ctx.clone(), &plan.input)?; Ok(Plan::MergeInto(Box::new(MergeInto { input: Box::new(optimized_distributed_join_sexpr), ..*plan From f292341cfef72d7335c74bb7a43943b1f2bd1628 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Wed, 18 Oct 2023 15:03:26 +0800 Subject: [PATCH 12/39] fix check --- .../fuse/src/operations/merge_into/mutator/matched_mutator.rs | 2 +- src/tests/sqlsmith/src/sql_gen/dml.rs | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs b/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs index f418ed8e7976..b6faa4d13052 100644 --- a/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs +++ b/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs @@ -272,7 +272,7 @@ impl MatchedAggregator { ErrorCode::Internal("unexpected, failed to join apply-deletion tasks.") .add_message_back(e.to_string()) })?; - let mut mutation_logs: Vec = self.entries.drain(..).into_iter().collect(); + let mut mutation_logs: Vec = self.entries.drain(..).collect(); for maybe_log_entry in log_entries { if let Some(segment_mutation_log) = maybe_log_entry? { mutation_logs.push(segment_mutation_log); diff --git a/src/tests/sqlsmith/src/sql_gen/dml.rs b/src/tests/sqlsmith/src/sql_gen/dml.rs index ff6d4e8618db..d1a785b23968 100644 --- a/src/tests/sqlsmith/src/sql_gen/dml.rs +++ b/src/tests/sqlsmith/src/sql_gen/dml.rs @@ -203,7 +203,6 @@ impl<'a, R: Rng + 'a> SqlGenerator<'a, R> { for field in fields { self.only_scalar_expr = true; let update_expr = MergeUpdateExpr { - catalog: None, table: None, name: Identifier::from_name(field.name().clone()), expr: self.gen_expr(&DataType::from(field.data_type())), From 137b52f2cd52d83149cc84694ef605f7b9c1482a Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Wed, 18 Oct 2023 17:38:35 +0800 Subject: [PATCH 13/39] fix check --- src/query/service/src/pipelines/pipeline_builder.rs | 2 +- .../service/tests/it/storages/testdata/settings_table.txt | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs index e276142ec0d8..6679f5c6fea6 100644 --- a/src/query/service/src/pipelines/pipeline_builder.rs +++ b/src/query/service/src/pipelines/pipeline_builder.rs @@ -343,7 +343,7 @@ impl PipelineBuilder { InputPort::create(), OutputPort::create(), table, - cluster_stats_gen.clone(), + cluster_stats_gen, )? .get_block_builder(); diff --git a/src/query/service/tests/it/storages/testdata/settings_table.txt b/src/query/service/tests/it/storages/testdata/settings_table.txt index afdb8ab67b64..44ac34a64410 100644 --- a/src/query/service/tests/it/storages/testdata/settings_table.txt +++ b/src/query/service/tests/it/storages/testdata/settings_table.txt @@ -12,10 +12,10 @@ DB.Table: 'system'.'settings', Table: settings-table_id:1, ver:0, Engine: System | 'enable_cbo' | '1' | '1' | 'SESSION' | 'Enables cost-based optimization.' | 'UInt64' | | 'enable_distributed_compact' | '0' | '0' | 'SESSION' | 'Enable distributed execution of table compaction.' | 'UInt64' | | 'enable_distributed_copy_into' | '0' | '0' | 'SESSION' | 'Enable distributed execution of copy into.' | 'UInt64' | +| 'enable_distributed_merge_into' | '0' | '0' | 'SESSION' | 'Enable distributed merge into.' | 'UInt64' | | 'enable_distributed_replace_into' | '0' | '0' | 'SESSION' | 'Enable distributed execution of replace into.' | 'UInt64' | | 'enable_dphyp' | '1' | '1' | 'SESSION' | 'Enables dphyp join order algorithm.' | 'UInt64' | -| 'enable_experimental_merge_into' | '0' | '0' | 'SESSION' | 'Enable experimental merge into.' | 'UInt64' | -| 'enable_distributed_merge_into' | '0' | '0' | 'SESSION' | 'Enable distributed merge into.' | 'UInt64' | +| 'enable_experimental_merge_into' | '0' | '0' | 'SESSION' | 'Enable experimental merge into.' | 'UInt64' | | 'enable_hive_parquet_predict_pushdown' | '1' | '1' | 'SESSION' | 'Enable hive parquet predict pushdown by setting this variable to 1, default value: 1' | 'UInt64' | | 'enable_query_profiling' | '0' | '0' | 'SESSION' | 'Enables recording query profile' | 'UInt64' | | 'enable_query_result_cache' | '0' | '0' | 'SESSION' | 'Enables caching query results to improve performance for identical queries.' | 'UInt64' | From f39bfd1d74724ca7434c1aa79f329d75b25d3faf Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Wed, 18 Oct 2023 18:28:38 +0800 Subject: [PATCH 14/39] fix test --- .../suites/base/09_fuse_engine/09_0026_merge_into | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/sqllogictests/suites/base/09_fuse_engine/09_0026_merge_into b/tests/sqllogictests/suites/base/09_fuse_engine/09_0026_merge_into index 38aed6e683e9..2375205b0baa 100644 --- a/tests/sqllogictests/suites/base/09_fuse_engine/09_0026_merge_into +++ b/tests/sqllogictests/suites/base/09_fuse_engine/09_0026_merge_into @@ -496,10 +496,10 @@ statement ok insert into cluster_source values(12,'b',1),(1,'a',2),(2,'b',3),(2,'a',4),(3,'a',3); ## test update indetify error -statement error 1065 -merge into cluster_target as t1 using (select * from cluster_source) as t2 on t1.a = t2.a when matched then update cluster_target.a = t2.a; +statement error 1006 +merge into cluster_target as t1 using (select * from cluster_source) as t2 on t1.a = t2.a when matched then update set cluster_target.a = t2.a; -statement error 1065 +statement error 1006 merge into cluster_target as t1 using (select * from cluster_source) as t2 on t1.a = t2.a when matched then update t2.a = t2.a; statement ok From 9b114f41f610f819b0f0e87be42d860b7f980870 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Wed, 18 Oct 2023 19:35:39 +0800 Subject: [PATCH 15/39] fix test --- .../sqllogictests/suites/base/09_fuse_engine/09_0026_merge_into | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/sqllogictests/suites/base/09_fuse_engine/09_0026_merge_into b/tests/sqllogictests/suites/base/09_fuse_engine/09_0026_merge_into index 2375205b0baa..0f9bd76f3463 100644 --- a/tests/sqllogictests/suites/base/09_fuse_engine/09_0026_merge_into +++ b/tests/sqllogictests/suites/base/09_fuse_engine/09_0026_merge_into @@ -500,7 +500,7 @@ statement error 1006 merge into cluster_target as t1 using (select * from cluster_source) as t2 on t1.a = t2.a when matched then update set cluster_target.a = t2.a; statement error 1006 -merge into cluster_target as t1 using (select * from cluster_source) as t2 on t1.a = t2.a when matched then update t2.a = t2.a; +merge into cluster_target as t1 using (select * from cluster_source) as t2 on t1.a = t2.a when matched then update set t2.a = t2.a; statement ok merge into cluster_target as t1 using (select * from cluster_source) as t2 on t1.a = t2.a when not matched then insert *; From 7bcf7605f8e7a540d5b1c2e4af8549b5a97450cf Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Wed, 18 Oct 2023 20:27:01 +0800 Subject: [PATCH 16/39] fix --- src/query/sql/src/executor/physical_plan.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/src/query/sql/src/executor/physical_plan.rs b/src/query/sql/src/executor/physical_plan.rs index f8f82a12e716..3aa6786ff5e6 100644 --- a/src/query/sql/src/executor/physical_plan.rs +++ b/src/query/sql/src/executor/physical_plan.rs @@ -183,7 +183,6 @@ impl PhysicalPlan { | PhysicalPlan::Deduplicate(_) | PhysicalPlan::ReplaceInto(_) | PhysicalPlan::MergeIntoRowIdApply(_) - | PhysicalPlan::CompactPartial(_) | PhysicalPlan::CompactSource(_) => Ok(DataSchemaRef::default()), } } From 6bb03e6f87194228600f0dc5fa0b97cca4bd7bff Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Thu, 19 Oct 2023 09:55:12 +0800 Subject: [PATCH 17/39] remove memory size limit --- src/binaries/query/entry.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/binaries/query/entry.rs b/src/binaries/query/entry.rs index 5d0e3758c2a9..a12c49c3f590 100644 --- a/src/binaries/query/entry.rs +++ b/src/binaries/query/entry.rs @@ -82,9 +82,9 @@ pub async fn init_services(conf: &InnerConfig) -> Result<()> { async fn precheck_services(conf: &InnerConfig) -> Result<()> { if conf.query.max_memory_limit_enabled { - let size = conf.query.max_server_memory_usage as i64; - info!("Set memory limit: {}", size); - GLOBAL_MEM_STAT.set_limit(size); + // let size = conf.query.max_server_memory_usage as i64; + // info!("Set memory limit: {}", size); + // GLOBAL_MEM_STAT.set_limit(size); } let tenant = conf.query.tenant_id.clone(); From 0d0eba9fabcd7d967264ca4b2a3708d3aceaa728 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Mon, 23 Oct 2023 22:15:44 +0800 Subject: [PATCH 18/39] optmizie merge source and add row_number processor --- .../interpreters/interpreter_merge_into.rs | 38 +++++- .../distributed/distributed_merge.rs | 127 ++++++++++++++++++ .../src/planner/optimizer/distributed/mod.rs | 2 + .../sql/src/planner/optimizer/optimizer.rs | 11 +- .../fuse/src/operations/merge_into/mod.rs | 1 + .../operations/merge_into/processors/mod.rs | 2 + .../transform_add_rownumber_column.rs | 119 ++++++++++++++++ 7 files changed, 292 insertions(+), 8 deletions(-) create mode 100644 src/query/sql/src/planner/optimizer/distributed/distributed_merge.rs create mode 100644 src/query/storages/fuse/src/operations/merge_into/processors/transform_add_rownumber_column.rs diff --git a/src/query/service/src/interpreters/interpreter_merge_into.rs b/src/query/service/src/interpreters/interpreter_merge_into.rs index b09ce0ed514d..936ac9bbdbe6 100644 --- a/src/query/service/src/interpreters/interpreter_merge_into.rs +++ b/src/query/service/src/interpreters/interpreter_merge_into.rs @@ -29,6 +29,7 @@ use common_functions::BUILTIN_FUNCTIONS; use common_meta_app::schema::TableInfo; use common_sql::executor::CommitSink; use common_sql::executor::Exchange; +use common_sql::executor::FragmentKind::Merge; use common_sql::executor::MergeInto; use common_sql::executor::MergeIntoRowIdApply; use common_sql::executor::MergeIntoSource; @@ -204,6 +205,35 @@ impl MergeIntoInterpreter { let table_info = fuse_table.get_table_info(); let catalog_ = self.ctx.get_catalog(catalog).await?; + // // distributed execution + // if exchange.is_some() { + // // rewrite plan: for now, we use right join, and the default + // // distributed right join will use shuffle hash join, but its + // // performance is very slow and poor. So we need to rewrite it. + // // In new distributed plan, target partitions will be shuffled + // // to query nodes, and source will be broadcasted to all nodes + // // and build hashtable. It means all nodes hold the same hashtable. + // match &mut join_input { + // PhysicalPlan::HashJoin(join) => match &join.probe { + // PhysicalPlan::Exchange(probe_exchange) => { + // join.probe = probe_exchange.input; + // } + // _ => { + // return Err(ErrorCode::BadArguments(format!( + // "find error plan {},it should be hash join", + // join_input.name() + // ))); + // } + // }, + // _ => { + // return Err(ErrorCode::BadArguments(format!( + // "find error plan {},it should be hash join", + // join_input.name() + // ))); + // } + // } + // } + // merge_into_source is used to recv join's datablocks and split them into macthed and not matched // datablocks. let merge_into_source = PhysicalPlan::MergeIntoSource(MergeIntoSource { @@ -359,14 +389,14 @@ impl MergeIntoInterpreter { join_output_schema.fields[row_id_idx].clone(), ])), })); - let exchange = exchange.unwrap(); + PhysicalPlan::MergeIntoRowIdApply(Box::new(MergeIntoRowIdApply { input: Box::new(PhysicalPlan::Exchange(Exchange { plan_id: 0, input: Box::new(merge_append), - kind: exchange.kind, - keys: exchange.keys, - ignore_exchange: exchange.ignore_exchange, + kind: Merge, + keys: vec![], + ignore_exchange: false, })), table_info: table_info.clone(), catalog_info: catalog_.info(), diff --git a/src/query/sql/src/planner/optimizer/distributed/distributed_merge.rs b/src/query/sql/src/planner/optimizer/distributed/distributed_merge.rs new file mode 100644 index 000000000000..7b5c846c7083 --- /dev/null +++ b/src/query/sql/src/planner/optimizer/distributed/distributed_merge.rs @@ -0,0 +1,127 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::sync::Arc; + +use common_exception::ErrorCode; +use common_exception::Result; + +use crate::optimizer::SExpr; +use crate::plans::Exchange::Broadcast; +use crate::plans::PatternPlan; +use crate::plans::RelOp; +use crate::plans::RelOperator; +pub struct MergeSourceOptimizer { + merge_source_pattern: SExpr, +} + +impl MergeSourceOptimizer { + pub fn create() -> Self { + Self { + merge_source_pattern: Self::merge_source_pattern(), + } + } + + pub fn optimize(&self, s_expr: &SExpr) -> Result { + if !s_expr.match_pattern(&self.merge_source_pattern) { + return Err(ErrorCode::BadArguments(format!( + "pattern not match for dirstributed merge source" + ))); + } else { + let join_s_expr = s_expr.child(0)?; + + let left_exchange = join_s_expr.child(0)?; + assert!(left_exchange.children.len() == 1); + let left_exchange_input = left_exchange.child(0)?; + + let right_exchange = join_s_expr.child(1)?; + assert!(right_exchange.children.len() == 1); + let right_exchange_input = join_s_expr.child(0)?; + + let new_join_children = vec![ + Arc::new(left_exchange_input.clone()), + Arc::new(SExpr::create_unary( + Arc::new(RelOperator::Exchange(Broadcast)), + Arc::new(right_exchange_input.clone()), + )), + ]; + Ok(join_s_expr.replace_children(new_join_children)) + } + } + + fn merge_source_pattern() -> SExpr { + // Input: + // Exchange(Merge) + // | + // Join + // / \ + // / \ + // Exchange Exchange(Shuffle) + // | | + // * * + // Output: + // Exchange + // | + // Join + // / \ + // / \ + // * Exchange(Broadcast) + // | + // * + SExpr::create_unary( + Arc::new( + PatternPlan { + plan_type: RelOp::Exchange, + } + .into(), + ), + Arc::new(SExpr::create_binary( + Arc::new( + PatternPlan { + plan_type: RelOp::Join, + } + .into(), + ), + Arc::new(SExpr::create_unary( + Arc::new( + PatternPlan { + plan_type: RelOp::Exchange, + } + .into(), + ), + Arc::new(SExpr::create_leaf(Arc::new( + PatternPlan { + plan_type: RelOp::Pattern, + } + .into(), + ))), + )), + Arc::new(SExpr::create_unary( + Arc::new( + PatternPlan { + plan_type: RelOp::Exchange, + } + .into(), + ), + Arc::new(SExpr::create_leaf(Arc::new( + PatternPlan { + plan_type: RelOp::Pattern, + } + .into(), + ))), + )), + )), + ) + } +} diff --git a/src/query/sql/src/planner/optimizer/distributed/mod.rs b/src/query/sql/src/planner/optimizer/distributed/mod.rs index a71cb7971743..88619549edc5 100644 --- a/src/query/sql/src/planner/optimizer/distributed/mod.rs +++ b/src/query/sql/src/planner/optimizer/distributed/mod.rs @@ -14,6 +14,8 @@ #[allow(clippy::module_inception)] mod distributed; +mod distributed_merge; mod topn; pub use distributed::optimize_distributed_query; +pub use distributed_merge::MergeSourceOptimizer; diff --git a/src/query/sql/src/planner/optimizer/optimizer.rs b/src/query/sql/src/planner/optimizer/optimizer.rs index 982cc909c88c..8e2ee8649e39 100644 --- a/src/query/sql/src/planner/optimizer/optimizer.rs +++ b/src/query/sql/src/planner/optimizer/optimizer.rs @@ -22,6 +22,7 @@ use common_exception::Result; use log::info; use super::cost::CostContext; +use super::distributed::MergeSourceOptimizer; use super::format::display_memo; use super::Memo; use crate::optimizer::cascades::CascadesOptimizer; @@ -130,12 +131,14 @@ pub fn optimize( && ctx.get_settings().get_enable_distributed_merge_into()? { // Todo(JackTan25): We should use optimizer to make a decision to use - // left join and right join + // left join and right join. // input is a Join_SExpr - let optimized_distributed_join_sexpr = - optimize_distributed_query(ctx.clone(), &plan.input)?; + let merge_into_join_sexpr = optimize_distributed_query(ctx.clone(), &plan.input)?; + let merge_source_optimizer = MergeSourceOptimizer::create(); + let optimized_distributed_merge_into_join_sexpr = + merge_source_optimizer.optimize(&merge_into_join_sexpr)?; Ok(Plan::MergeInto(Box::new(MergeInto { - input: Box::new(optimized_distributed_join_sexpr), + input: Box::new(optimized_distributed_merge_into_join_sexpr), ..*plan }))) } else { diff --git a/src/query/storages/fuse/src/operations/merge_into/mod.rs b/src/query/storages/fuse/src/operations/merge_into/mod.rs index aee696b8c3a3..10b1c3206ec7 100644 --- a/src/query/storages/fuse/src/operations/merge_into/mod.rs +++ b/src/query/storages/fuse/src/operations/merge_into/mod.rs @@ -20,5 +20,6 @@ pub use processors::MatchedSplitProcessor; pub use processors::MergeIntoNotMatchedProcessor; pub use processors::MergeIntoSplitProcessor; pub use processors::MixRowIdKindAndLog; +pub use processors::TransformAddRowNumberColumnProcessor; pub use processors::TransformDistributedMergeIntoBlockDeserialize; pub use processors::TransformDistributedMergeIntoBlockSerialize; diff --git a/src/query/storages/fuse/src/operations/merge_into/processors/mod.rs b/src/query/storages/fuse/src/operations/merge_into/processors/mod.rs index 0a2a2654b7d9..cb679611f47f 100644 --- a/src/query/storages/fuse/src/operations/merge_into/processors/mod.rs +++ b/src/query/storages/fuse/src/operations/merge_into/processors/mod.rs @@ -17,6 +17,7 @@ mod processor_distributed_merge_into_block_serialize; mod processor_merge_into_matched_and_split; mod processor_merge_into_not_matched; mod processor_merge_into_split; +mod transform_add_rownumber_column; mod transform_matched_mutation_aggregator; pub use processor_distributed_merge_into_block_deserialize::TransformDistributedMergeIntoBlockDeserialize; pub use processor_distributed_merge_into_block_serialize::TransformDistributedMergeIntoBlockSerialize; @@ -25,3 +26,4 @@ pub use processor_merge_into_matched_and_split::MixRowIdKindAndLog; pub(crate) use processor_merge_into_matched_and_split::RowIdKind; pub use processor_merge_into_not_matched::MergeIntoNotMatchedProcessor; pub use processor_merge_into_split::MergeIntoSplitProcessor; +pub use transform_add_rownumber_column::TransformAddRowNumberColumnProcessor; diff --git a/src/query/storages/fuse/src/operations/merge_into/processors/transform_add_rownumber_column.rs b/src/query/storages/fuse/src/operations/merge_into/processors/transform_add_rownumber_column.rs new file mode 100644 index 000000000000..7fe42618767f --- /dev/null +++ b/src/query/storages/fuse/src/operations/merge_into/processors/transform_add_rownumber_column.rs @@ -0,0 +1,119 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::sync::atomic::AtomicU64; +use std::sync::atomic::Ordering; +use std::sync::Arc; + +use common_exception::Result; +use common_expression::types::DataType; +use common_expression::types::NumberDataType; +use common_expression::types::UInt64Type; +use common_expression::BlockEntry; +use common_expression::DataBlock; +use common_expression::FromData; +use common_expression::Value; +use common_pipeline_core::pipe::Pipe; +use common_pipeline_core::pipe::PipeItem; +use common_pipeline_core::processors::port::InputPort; +use common_pipeline_core::processors::port::OutputPort; +use common_pipeline_core::processors::processor::ProcessorPtr; +use common_pipeline_transforms::processors::transforms::Transform; +use common_pipeline_transforms::processors::transforms::Transformer; + +const PREFIX_OFFSET: usize = 48; + +pub struct TransformAddRowNumberColumnProcessor { + // node_id + prefix: u16, + // current row number, row_number shouldn't be overflow 48bits + row_number: Arc, +} + +impl TransformAddRowNumberColumnProcessor { + #[allow(dead_code)] + fn create( + input: Arc, + output: Arc, + node_id: u16, + row_number: Arc, + ) -> ProcessorPtr { + ProcessorPtr::create(Transformer::create( + input, + output, + TransformAddRowNumberColumnProcessor { + prefix: node_id, + row_number, + }, + )) + } + + fn create_row_number_column_transform_item( + node_id: u16, + row_number: Arc, + ) -> PipeItem { + let input = InputPort::create(); + let output = OutputPort::create(); + PipeItem::create( + TransformAddRowNumberColumnProcessor::create( + input.clone(), + output.clone(), + node_id, + row_number, + ), + vec![input], + vec![output], + ) + } + + pub fn into_pipe(node_id: u16, num_threads: usize) -> Pipe { + let mut pipe_items = Vec::with_capacity(num_threads); + let row_number = Arc::new(AtomicU64::new(0)); + for _ in 0..num_threads { + let pipe_item = + Self::create_row_number_column_transform_item(node_id, row_number.clone()); + pipe_items.push(pipe_item); + } + Pipe::create(num_threads, num_threads, pipe_items) + } +} + +impl TransformAddRowNumberColumnProcessor { + fn generate_row_number(&mut self, num_rows: u64) -> u64 { + let row_number = self.row_number.fetch_add(num_rows, Ordering::SeqCst); + let mut prefix_u64 = self.prefix as u64; + prefix_u64 = prefix_u64 << PREFIX_OFFSET; + prefix_u64 & row_number + } +} + +#[async_trait::async_trait] +impl Transform for TransformAddRowNumberColumnProcessor { + const NAME: &'static str = "TransformAddRowNumberColumnProcessor"; + fn transform(&mut self, data: DataBlock) -> Result { + let num_rows = data.num_rows() as u64; + let last_row_number = self.generate_row_number(num_rows); + let mut row_ids = Vec::with_capacity(data.num_rows()); + for number in (last_row_number - num_rows + 1)..=last_row_number { + row_ids.push(number); + } + let mut data_block = data; + let row_number_entry = BlockEntry::new( + DataType::Number(NumberDataType::UInt64), + Value::Column(UInt64Type::from_data(row_ids)), + ); + data_block.add_column(row_number_entry); + Ok(data_block) + } +} From f1f2f94995f516a1e8edbf7d5498d1c2d89b2e49 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Tue, 24 Oct 2023 20:17:37 +0800 Subject: [PATCH 19/39] fix delete bug --- .../merge_into/mutator/delete_by_expr_mutator.rs | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/src/query/storages/fuse/src/operations/merge_into/mutator/delete_by_expr_mutator.rs b/src/query/storages/fuse/src/operations/merge_into/mutator/delete_by_expr_mutator.rs index 97cf45bda5e6..04abc9909bc8 100644 --- a/src/query/storages/fuse/src/operations/merge_into/mutator/delete_by_expr_mutator.rs +++ b/src/query/storages/fuse/src/operations/merge_into/mutator/delete_by_expr_mutator.rs @@ -15,6 +15,7 @@ use std::ops::Not; use common_exception::Result; +use common_expression::eval_function; use common_expression::types::BooleanType; use common_expression::types::DataType; use common_expression::DataBlock; @@ -60,11 +61,18 @@ impl DeleteByExprMutator { .map_err(|e| e.add_message("eval filter failed:"))? .try_downcast::() .unwrap(); - let filter = predicates.into_column().unwrap(); - let filtered_block = data_block.clone().filter_with_bitmap(&filter)?; + let (predicates_not, _) = eval_function( + None, + "not", + [(predicates.clone().upcast(), DataType::Boolean)], + &self.func_ctx, + data_block.num_rows(), + &BUILTIN_FUNCTIONS, + )?; + let filtered_block = data_block.clone().filter_boolean_value(&predicates)?; Ok(( - data_block.filter_with_bitmap(&filter.not())?, + data_block.filter_boolean_value(&predicates_not.try_downcast().unwrap())?, DataBlock::new( vec![filtered_block.get_by_offset(self.row_id_idx).clone()], filtered_block.num_rows(), From bd3be353c124bff6eedba24fa8de126a445d8d15 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Wed, 25 Oct 2023 00:19:56 +0800 Subject: [PATCH 20/39] add row number plan --- src/query/catalog/src/plan/internal_column.rs | 1 + src/query/expression/src/schema.rs | 4 +- .../interpreters/interpreter_merge_into.rs | 29 -------- src/query/sql/src/executor/format.rs | 1 + src/query/sql/src/executor/physical_plan.rs | 7 ++ .../sql/src/executor/physical_plan_builder.rs | 1 + .../sql/src/executor/physical_plan_display.rs | 8 +++ .../sql/src/executor/physical_plan_visitor.rs | 14 +++- .../sql/src/executor/physical_plans/mod.rs | 1 + .../physical_plans/physical_add_row_number.rs | 60 ++++++++++++++++ src/query/sql/src/executor/profile.rs | 1 + .../sql/src/planner/binder/merge_into.rs | 3 +- .../planner/format/display_rel_operator.rs | 1 + .../distributed/distributed_merge.rs | 30 +++++--- src/query/sql/src/planner/optimizer/format.rs | 1 + .../src/planner/optimizer/hyper_dp/dphyp.rs | 1 + src/query/sql/src/planner/optimizer/s_expr.rs | 1 + .../sql/src/planner/plans/add_row_number.rs | 69 +++++++++++++++++++ src/query/sql/src/planner/plans/mod.rs | 1 + src/query/sql/src/planner/plans/operator.rs | 10 +++ .../mutator/delete_by_expr_mutator.rs | 2 - 21 files changed, 204 insertions(+), 42 deletions(-) create mode 100644 src/query/sql/src/executor/physical_plans/physical_add_row_number.rs create mode 100644 src/query/sql/src/planner/plans/add_row_number.rs diff --git a/src/query/catalog/src/plan/internal_column.rs b/src/query/catalog/src/plan/internal_column.rs index 2a04b4185b45..58b977787103 100644 --- a/src/query/catalog/src/plan/internal_column.rs +++ b/src/query/catalog/src/plan/internal_column.rs @@ -29,6 +29,7 @@ use common_expression::TableDataType; use common_expression::Value; use common_expression::BLOCK_NAME_COLUMN_ID; use common_expression::ROW_ID_COLUMN_ID; +use common_expression::ROW_NUMBER_COLUMN_ID; use common_expression::SEGMENT_NAME_COLUMN_ID; use common_expression::SNAPSHOT_NAME_COLUMN_ID; use storages_common_table_meta::meta::NUM_BLOCK_ID_BITS; diff --git a/src/query/expression/src/schema.rs b/src/query/expression/src/schema.rs index 25754c21943b..7c6599ca9333 100644 --- a/src/query/expression/src/schema.rs +++ b/src/query/expression/src/schema.rs @@ -48,9 +48,11 @@ pub type FieldIndex = usize; pub const ROW_ID_COLUMN_ID: u32 = u32::MAX; pub const BLOCK_NAME_COLUMN_ID: u32 = u32::MAX - 1; pub const SEGMENT_NAME_COLUMN_ID: u32 = u32::MAX - 2; -pub const SNAPSHOT_NAME_COLUMN_ID: u32 = u32::MAX - 3; +pub const ROW_NUMBER_COLUMN_ID: u32 = u32::MAX - 3; +pub const SNAPSHOT_NAME_COLUMN_ID: u32 = u32::MAX - 4; pub const ROW_ID_COL_NAME: &str = "_row_id"; +pub const ROW_NUMBER_COL_NAME: &str = "_row_number"; pub const SNAPSHOT_NAME_COL_NAME: &str = "_snapshot_name"; pub const SEGMENT_NAME_COL_NAME: &str = "_segment_name"; pub const BLOCK_NAME_COL_NAME: &str = "_block_name"; diff --git a/src/query/service/src/interpreters/interpreter_merge_into.rs b/src/query/service/src/interpreters/interpreter_merge_into.rs index 936ac9bbdbe6..e40385e29079 100644 --- a/src/query/service/src/interpreters/interpreter_merge_into.rs +++ b/src/query/service/src/interpreters/interpreter_merge_into.rs @@ -205,35 +205,6 @@ impl MergeIntoInterpreter { let table_info = fuse_table.get_table_info(); let catalog_ = self.ctx.get_catalog(catalog).await?; - // // distributed execution - // if exchange.is_some() { - // // rewrite plan: for now, we use right join, and the default - // // distributed right join will use shuffle hash join, but its - // // performance is very slow and poor. So we need to rewrite it. - // // In new distributed plan, target partitions will be shuffled - // // to query nodes, and source will be broadcasted to all nodes - // // and build hashtable. It means all nodes hold the same hashtable. - // match &mut join_input { - // PhysicalPlan::HashJoin(join) => match &join.probe { - // PhysicalPlan::Exchange(probe_exchange) => { - // join.probe = probe_exchange.input; - // } - // _ => { - // return Err(ErrorCode::BadArguments(format!( - // "find error plan {},it should be hash join", - // join_input.name() - // ))); - // } - // }, - // _ => { - // return Err(ErrorCode::BadArguments(format!( - // "find error plan {},it should be hash join", - // join_input.name() - // ))); - // } - // } - // } - // merge_into_source is used to recv join's datablocks and split them into macthed and not matched // datablocks. let merge_into_source = PhysicalPlan::MergeIntoSource(MergeIntoSource { diff --git a/src/query/sql/src/executor/format.rs b/src/query/sql/src/executor/format.rs index ce5ad7445fdf..bd7c6f30b85a 100644 --- a/src/query/sql/src/executor/format.rs +++ b/src/query/sql/src/executor/format.rs @@ -212,6 +212,7 @@ fn to_format_tree( PhysicalPlan::ReplaceInto(_) => Ok(FormatTreeNode::new("Replace".to_string())), PhysicalPlan::MergeInto(_) => Ok(FormatTreeNode::new("MergeInto".to_string())), PhysicalPlan::MergeIntoSource(_) => Ok(FormatTreeNode::new("MergeIntoSource".to_string())), + PhysicalPlan::AddRowNumber(_) => Ok(FormatTreeNode::new("AddRowNumber".to_string())), PhysicalPlan::MergeIntoRowIdApply(_) => { Ok(FormatTreeNode::new("MergeIntoRowIdApply".to_string())) } diff --git a/src/query/sql/src/executor/physical_plan.rs b/src/query/sql/src/executor/physical_plan.rs index 3aa6786ff5e6..685a1f04724b 100644 --- a/src/query/sql/src/executor/physical_plan.rs +++ b/src/query/sql/src/executor/physical_plan.rs @@ -17,6 +17,7 @@ use common_exception::Result; use common_expression::DataSchemaRef; use enum_as_inner::EnumAsInner; +use super::physical_plans::physical_add_row_number::AddRowNumber; use super::MergeIntoRowIdApply; use crate::executor::physical_plans::physical_aggregate_expand::AggregateExpand; use crate::executor::physical_plans::physical_aggregate_final::AggregateFinal; @@ -99,6 +100,7 @@ pub enum PhysicalPlan { MergeIntoSource(MergeIntoSource), MergeInto(Box), MergeIntoRowIdApply(Box), + AddRowNumber(Box), /// Compact CompactSource(Box), @@ -135,6 +137,7 @@ impl PhysicalPlan { PhysicalPlan::ConstantTableScan(v) => v.plan_id, PhysicalPlan::DeleteSource(_) | PhysicalPlan::MergeInto(_) + | PhysicalPlan::AddRowNumber(_) | PhysicalPlan::MergeIntoSource(_) | PhysicalPlan::MergeIntoRowIdApply(_) | PhysicalPlan::CommitSink(_) @@ -179,6 +182,7 @@ impl PhysicalPlan { PhysicalPlan::ConstantTableScan(plan) => plan.output_schema(), PhysicalPlan::MergeIntoSource(plan) => plan.input.output_schema(), PhysicalPlan::MergeInto(plan) => Ok(plan.output_schema.clone()), + PhysicalPlan::AddRowNumber(plan) => plan.input.output_schema(), PhysicalPlan::AsyncSourcer(_) | PhysicalPlan::Deduplicate(_) | PhysicalPlan::ReplaceInto(_) @@ -223,6 +227,7 @@ impl PhysicalPlan { PhysicalPlan::CteScan(_) => "PhysicalCteScan".to_string(), PhysicalPlan::MaterializedCte(_) => "PhysicalMaterializedCte".to_string(), PhysicalPlan::ConstantTableScan(_) => "PhysicalConstantTableScan".to_string(), + PhysicalPlan::AddRowNumber(_) => "AddRowNumber".to_string(), } } @@ -270,6 +275,7 @@ impl PhysicalPlan { PhysicalPlan::Deduplicate(plan) => Box::new(std::iter::once(plan.input.as_ref())), PhysicalPlan::ReplaceInto(plan) => Box::new(std::iter::once(plan.input.as_ref())), PhysicalPlan::MergeInto(plan) => Box::new(std::iter::once(plan.input.as_ref())), + PhysicalPlan::AddRowNumber(plan) => Box::new(std::iter::once(plan.input.as_ref())), PhysicalPlan::MergeIntoSource(plan) => Box::new(std::iter::once(plan.input.as_ref())), PhysicalPlan::MergeIntoRowIdApply(plan) => { Box::new(std::iter::once(plan.input.as_ref())) @@ -313,6 +319,7 @@ impl PhysicalPlan { | PhysicalPlan::Deduplicate(_) | PhysicalPlan::ReplaceInto(_) | PhysicalPlan::MergeInto(_) + | PhysicalPlan::AddRowNumber(_) | PhysicalPlan::MergeIntoRowIdApply(_) | PhysicalPlan::MergeIntoSource(_) | PhysicalPlan::ConstantTableScan(_) diff --git a/src/query/sql/src/executor/physical_plan_builder.rs b/src/query/sql/src/executor/physical_plan_builder.rs index 980c7d06bdcd..6fed0028565e 100644 --- a/src/query/sql/src/executor/physical_plan_builder.rs +++ b/src/query/sql/src/executor/physical_plan_builder.rs @@ -117,6 +117,7 @@ impl PhysicalPlanBuilder { RelOperator::ConstantTableScan(scan) => { self.build_constant_table_scan(scan, required).await } + RelOperator::AddRowNumber(_) => self.build_add_row_number(s_expr, required).await, _ => Err(ErrorCode::Internal(format!( "Unsupported physical plan: {:?}", s_expr.plan() diff --git a/src/query/sql/src/executor/physical_plan_display.rs b/src/query/sql/src/executor/physical_plan_display.rs index bdf603e676a1..d7832447c5d3 100644 --- a/src/query/sql/src/executor/physical_plan_display.rs +++ b/src/query/sql/src/executor/physical_plan_display.rs @@ -18,6 +18,7 @@ use std::fmt::Formatter; use common_functions::BUILTIN_FUNCTIONS; use itertools::Itertools; +use super::physical_plans::physical_add_row_number::AddRowNumber; use super::MergeIntoRowIdApply; use crate::executor::physical_plans::physical_aggregate_expand::AggregateExpand; use crate::executor::physical_plans::physical_aggregate_final::AggregateFinal; @@ -104,6 +105,7 @@ impl<'a> Display for PhysicalPlanIndentFormatDisplay<'a> { PhysicalPlan::MergeIntoRowIdApply(merge_into_row_id_apply) => { write!(f, "{}", merge_into_row_id_apply)? } + PhysicalPlan::AddRowNumber(add_row_number) => write!(f, "{}", add_row_number)?, PhysicalPlan::CteScan(cte_scan) => write!(f, "{}", cte_scan)?, PhysicalPlan::MaterializedCte(plan) => write!(f, "{}", plan)?, PhysicalPlan::ConstantTableScan(scan) => write!(f, "{}", scan)?, @@ -474,6 +476,12 @@ impl Display for MergeInto { } } +impl Display for AddRowNumber { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!(f, "AddRowNumber") + } +} + impl Display for MergeIntoRowIdApply { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { write!(f, "MergeIntoRowIdApply") diff --git a/src/query/sql/src/executor/physical_plan_visitor.rs b/src/query/sql/src/executor/physical_plan_visitor.rs index cbc9cf2ea4db..dcfa7349f6ed 100644 --- a/src/query/sql/src/executor/physical_plan_visitor.rs +++ b/src/query/sql/src/executor/physical_plan_visitor.rs @@ -14,6 +14,7 @@ use common_exception::Result; +use super::physical_plans::physical_add_row_number::AddRowNumber; use super::MergeIntoRowIdApply; use crate::executor::physical_plans::physical_aggregate_expand::AggregateExpand; use crate::executor::physical_plans::physical_aggregate_final::AggregateFinal; @@ -85,6 +86,7 @@ pub trait PhysicalPlanReplacer { PhysicalPlan::Deduplicate(plan) => self.replace_deduplicate(plan), PhysicalPlan::ReplaceInto(plan) => self.replace_replace_into(plan), PhysicalPlan::MergeInto(plan) => self.replace_merge_into(plan), + PhysicalPlan::AddRowNumber(plan) => self.replace_add_row_number(&plan), PhysicalPlan::MergeIntoSource(plan) => self.replace_merge_into_source(plan), PhysicalPlan::MergeIntoRowIdApply(plan) => self.replace_merge_into_row_id_apply(plan), PhysicalPlan::MaterializedCte(plan) => self.replace_materialized_cte(plan), @@ -416,6 +418,14 @@ pub trait PhysicalPlanReplacer { }))) } + fn replace_add_row_number(&mut self, plan: &AddRowNumber) -> Result { + let input = self.replace(&plan.input)?; + Ok(PhysicalPlan::AddRowNumber(Box::new(AddRowNumber { + input: Box::new(input), + ..plan.clone() + }))) + } + fn replace_merge_into_source(&mut self, plan: &MergeIntoSource) -> Result { let input = self.replace(&plan.input)?; Ok(PhysicalPlan::MergeIntoSource(MergeIntoSource { @@ -573,7 +583,9 @@ impl PhysicalPlan { PhysicalPlan::MergeInto(plan) => { Self::traverse(&plan.input, pre_visit, visit, post_visit); } - + PhysicalPlan::AddRowNumber(plan) => { + Self::traverse(&plan.input, pre_visit, visit, post_visit); + } PhysicalPlan::MergeIntoRowIdApply(plan) => { Self::traverse(&plan.input, pre_visit, visit, post_visit); } diff --git a/src/query/sql/src/executor/physical_plans/mod.rs b/src/query/sql/src/executor/physical_plans/mod.rs index 7ccd857105cb..b17d8ab06cf7 100644 --- a/src/query/sql/src/executor/physical_plans/mod.rs +++ b/src/query/sql/src/executor/physical_plans/mod.rs @@ -13,6 +13,7 @@ // limitations under the License. pub mod common; +pub mod physical_add_row_number; pub mod physical_aggregate_expand; pub mod physical_aggregate_final; pub mod physical_aggregate_partial; diff --git a/src/query/sql/src/executor/physical_plans/physical_add_row_number.rs b/src/query/sql/src/executor/physical_plans/physical_add_row_number.rs new file mode 100644 index 000000000000..d31e34120ea3 --- /dev/null +++ b/src/query/sql/src/executor/physical_plans/physical_add_row_number.rs @@ -0,0 +1,60 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::BTreeMap; + +use common_exception::ErrorCode; +use common_exception::Result; +use common_expression::DataSchemaRef; + +use crate::executor::PhysicalPlan; +use crate::executor::PhysicalPlanBuilder; +use crate::optimizer::ColumnSet; +use crate::optimizer::SExpr; + +// add row_number for distributed merge into +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct AddRowNumber { + pub cluster_index: BTreeMap, + pub input: Box, +} + +impl AddRowNumber { + pub fn output_schema(&self) -> Result { + todo!() + } +} + +impl PhysicalPlanBuilder { + pub(crate) async fn build_add_row_number( + &mut self, + s_expr: &SExpr, + mut required: ColumnSet, + ) -> Result { + let input_plan = self.build(s_expr.child(0)?, required).await?; + if self.ctx.get_cluster().is_empty() { + return Err(ErrorCode::CannotConnectNode( + "there is only one node when build distributed merge into", + )); + } + let mut cluster_index = BTreeMap::new(); + for (id, node) in self.ctx.get_cluster().nodes.iter().enumerate() { + cluster_index.insert(node.id.clone(), id); + } + Ok(PhysicalPlan::AddRowNumber(Box::new(AddRowNumber { + cluster_index, + input: Box::new(input_plan), + }))) + } +} diff --git a/src/query/sql/src/executor/profile.rs b/src/query/sql/src/executor/profile.rs index c431d78e1218..1a006140afa9 100644 --- a/src/query/sql/src/executor/profile.rs +++ b/src/query/sql/src/executor/profile.rs @@ -510,6 +510,7 @@ fn flatten_plan_node_profile( | PhysicalPlan::CopyIntoTable(_) | PhysicalPlan::AsyncSourcer(_) | PhysicalPlan::MergeInto(_) + | PhysicalPlan::AddRowNumber(_) | PhysicalPlan::MergeIntoRowIdApply(_) | PhysicalPlan::MergeIntoSource(_) | PhysicalPlan::Deduplicate(_) diff --git a/src/query/sql/src/planner/binder/merge_into.rs b/src/query/sql/src/planner/binder/merge_into.rs index 9c59b9c70b8a..b0c6c4d05d82 100644 --- a/src/query/sql/src/planner/binder/merge_into.rs +++ b/src/query/sql/src/planner/binder/merge_into.rs @@ -32,6 +32,7 @@ use common_expression::types::DataType; use common_expression::FieldIndex; use common_expression::TableSchemaRef; use common_expression::ROW_ID_COL_NAME; +use common_expression::ROW_NUMBER_COL_NAME; use indexmap::IndexMap; use super::wrap_cast_scalar; @@ -70,7 +71,7 @@ impl Binder { .unwrap_or_default() { return Err(ErrorCode::Unimplemented( - "merge into is unstable for now, you can use 'set enable_experimental_merge_into = 1' to set up it", + "merge into is experimental for now, you can use 'set enable_experimental_merge_into = 1' to set it up", )); } let MergeIntoStmt { diff --git a/src/query/sql/src/planner/format/display_rel_operator.rs b/src/query/sql/src/planner/format/display_rel_operator.rs index 3acc9c5575e4..6ee519ac51e7 100644 --- a/src/query/sql/src/planner/format/display_rel_operator.rs +++ b/src/query/sql/src/planner/format/display_rel_operator.rs @@ -80,6 +80,7 @@ impl Display for FormatContext { RelOperator::MaterializedCte(_) => write!(f, "MaterializedCte"), RelOperator::Lambda(_) => write!(f, "Lambda"), RelOperator::ConstantTableScan(_) => write!(f, "ConstantTableScan"), + RelOperator::AddRowNumber(_) => write!(f, "AddRowNumber"), }, Self::Text(text) => write!(f, "{}", text), } diff --git a/src/query/sql/src/planner/optimizer/distributed/distributed_merge.rs b/src/query/sql/src/planner/optimizer/distributed/distributed_merge.rs index 7b5c846c7083..d11e14409b11 100644 --- a/src/query/sql/src/planner/optimizer/distributed/distributed_merge.rs +++ b/src/query/sql/src/planner/optimizer/distributed/distributed_merge.rs @@ -33,6 +33,12 @@ impl MergeSourceOptimizer { } } + // rewrite plan: for now, we use right join, and the default + // distributed right join will use shuffle hash join, but its + // performance is very slow and poor. So we need to rewrite it. + // In new distributed plan, target partitions will be shuffled + // to query nodes, and source will be broadcasted to all nodes + // and build hashtable. It means all nodes hold the same hashtable. pub fn optimize(&self, s_expr: &SExpr) -> Result { if !s_expr.match_pattern(&self.merge_source_pattern) { return Err(ErrorCode::BadArguments(format!( @@ -77,8 +83,8 @@ impl MergeSourceOptimizer { // / \ // / \ // * Exchange(Broadcast) - // | - // * + // | + // AddRowNumber SExpr::create_unary( Arc::new( PatternPlan { @@ -114,12 +120,20 @@ impl MergeSourceOptimizer { } .into(), ), - Arc::new(SExpr::create_leaf(Arc::new( - PatternPlan { - plan_type: RelOp::Pattern, - } - .into(), - ))), + Arc::new(SExpr::create_unary( + Arc::new( + PatternPlan { + plan_type: RelOp::AddRowNumber, + } + .into(), + ), + Arc::new(SExpr::create_leaf(Arc::new( + PatternPlan { + plan_type: RelOp::Pattern, + } + .into(), + ))), + )), )), )), ) diff --git a/src/query/sql/src/planner/optimizer/format.rs b/src/query/sql/src/planner/optimizer/format.rs index c2129749ed9d..685e394e5c07 100644 --- a/src/query/sql/src/planner/optimizer/format.rs +++ b/src/query/sql/src/planner/optimizer/format.rs @@ -53,6 +53,7 @@ pub fn display_rel_op(rel_op: &RelOperator) -> String { RelOperator::MaterializedCte(_) => "MaterializedCte".to_string(), RelOperator::Lambda(_) => "LambdaFunc".to_string(), RelOperator::ConstantTableScan(_) => "ConstantTableScan".to_string(), + RelOperator::AddRowNumber(_) => "AddRowNumber".to_string(), } } diff --git a/src/query/sql/src/planner/optimizer/hyper_dp/dphyp.rs b/src/query/sql/src/planner/optimizer/hyper_dp/dphyp.rs index 49f905953cd3..4755af4930ae 100644 --- a/src/query/sql/src/planner/optimizer/hyper_dp/dphyp.rs +++ b/src/query/sql/src/planner/optimizer/hyper_dp/dphyp.rs @@ -253,6 +253,7 @@ impl DPhpy { Ok((new_s_expr, optimized)) } RelOperator::Exchange(_) + | RelOperator::AddRowNumber(_) | RelOperator::Pattern(_) | RelOperator::RuntimeFilterSource(_) => unreachable!(), RelOperator::DummyTableScan(_) diff --git a/src/query/sql/src/planner/optimizer/s_expr.rs b/src/query/sql/src/planner/optimizer/s_expr.rs index ca726cea325b..1f8352fe8b59 100644 --- a/src/query/sql/src/planner/optimizer/s_expr.rs +++ b/src/query/sql/src/planner/optimizer/s_expr.rs @@ -251,6 +251,7 @@ fn find_subquery(rel_op: &RelOperator) -> bool { | RelOperator::Sort(_) | RelOperator::DummyTableScan(_) | RelOperator::CteScan(_) + | RelOperator::AddRowNumber(_) | RelOperator::RuntimeFilterSource(_) | RelOperator::Pattern(_) | RelOperator::MaterializedCte(_) diff --git a/src/query/sql/src/planner/plans/add_row_number.rs b/src/query/sql/src/planner/plans/add_row_number.rs new file mode 100644 index 000000000000..d9a7715b5aea --- /dev/null +++ b/src/query/sql/src/planner/plans/add_row_number.rs @@ -0,0 +1,69 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::sync::Arc; + +use common_catalog::table_context::TableContext; +use common_exception::ErrorCode; + +use super::Operator; +use super::RelOp; +use crate::optimizer::PhysicalProperty; +use crate::optimizer::RelExpr; +use crate::optimizer::RelationalProperty; +use crate::optimizer::RequiredProperty; +use crate::optimizer::StatInfo; + +#[derive(Clone, Debug, PartialEq, Eq, Hash)] +pub struct AddRowNumber; + +impl Operator for AddRowNumber { + fn rel_op(&self) -> RelOp { + RelOp::AddRowNumber + } + + fn derive_relational_prop( + &self, + _rel_expr: &RelExpr, + ) -> common_exception::Result> { + Err(ErrorCode::Internal( + "Cannot derive relational property for add_row_number plan", + )) + } + + fn derive_physical_prop( + &self, + _rel_expr: &RelExpr, + ) -> common_exception::Result { + Err(ErrorCode::Internal( + "Cannot derive physical property for add_row_number plan", + )) + } + + fn derive_cardinality(&self, _rel_expr: &RelExpr) -> common_exception::Result> { + Err(ErrorCode::Internal( + "Cannot derive cardinality for add_row_number plan", + )) + } + + fn compute_required_prop_child( + &self, + _ctx: Arc, + _rel_expr: &RelExpr, + _child_index: usize, + _required: &RequiredProperty, + ) -> common_exception::Result { + unreachable!() + } +} diff --git a/src/query/sql/src/planner/plans/mod.rs b/src/query/sql/src/planner/plans/mod.rs index f6968964ba5f..a7788918456c 100644 --- a/src/query/sql/src/planner/plans/mod.rs +++ b/src/query/sql/src/planner/plans/mod.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +mod add_row_number; mod aggregate; mod call; mod constant_table_scan; diff --git a/src/query/sql/src/planner/plans/operator.rs b/src/query/sql/src/planner/plans/operator.rs index dc5ba621174d..5241444630ea 100644 --- a/src/query/sql/src/planner/plans/operator.rs +++ b/src/query/sql/src/planner/plans/operator.rs @@ -18,6 +18,7 @@ use common_catalog::table_context::TableContext; use common_exception::ErrorCode; use common_exception::Result; +use super::add_row_number::AddRowNumber; use super::aggregate::Aggregate; use super::dummy_table_scan::DummyTableScan; use super::eval_scalar::EvalScalar; @@ -84,6 +85,7 @@ pub enum RelOp { MaterializedCte, Lambda, ConstantTableScan, + AddRowNumber, // Pattern Pattern, @@ -101,6 +103,7 @@ pub enum RelOperator { Sort(Sort), Limit(Limit), Exchange(Exchange), + AddRowNumber(AddRowNumber), UnionAll(UnionAll), DummyTableScan(DummyTableScan), RuntimeFilterSource(RuntimeFilterSource), @@ -133,6 +136,7 @@ impl Operator for RelOperator { RelOperator::MaterializedCte(rel_op) => rel_op.rel_op(), RelOperator::Lambda(rel_op) => rel_op.rel_op(), RelOperator::ConstantTableScan(rel_op) => rel_op.rel_op(), + RelOperator::AddRowNumber(rel_op) => rel_op.rel_op(), } } @@ -156,6 +160,7 @@ impl Operator for RelOperator { RelOperator::MaterializedCte(rel_op) => rel_op.derive_relational_prop(rel_expr), RelOperator::Lambda(rel_op) => rel_op.derive_relational_prop(rel_expr), RelOperator::ConstantTableScan(rel_op) => rel_op.derive_relational_prop(rel_expr), + RelOperator::AddRowNumber(rel_op) => rel_op.derive_relational_prop(rel_expr), } } @@ -179,6 +184,7 @@ impl Operator for RelOperator { RelOperator::MaterializedCte(rel_op) => rel_op.derive_physical_prop(rel_expr), RelOperator::Lambda(rel_op) => rel_op.derive_physical_prop(rel_expr), RelOperator::ConstantTableScan(rel_op) => rel_op.derive_physical_prop(rel_expr), + RelOperator::AddRowNumber(rel_op) => rel_op.derive_physical_prop(rel_expr), } } @@ -202,6 +208,7 @@ impl Operator for RelOperator { RelOperator::MaterializedCte(rel_op) => rel_op.derive_cardinality(rel_expr), RelOperator::Lambda(rel_op) => rel_op.derive_cardinality(rel_expr), RelOperator::ConstantTableScan(rel_op) => rel_op.derive_cardinality(rel_expr), + RelOperator::AddRowNumber(rel_op) => rel_op.derive_cardinality(rel_expr), } } @@ -267,6 +274,9 @@ impl Operator for RelOperator { RelOperator::ConstantTableScan(rel_op) => { rel_op.compute_required_prop_child(ctx, rel_expr, child_index, required) } + RelOperator::AddRowNumber(rel_op) => { + rel_op.compute_required_prop_child(ctx, rel_expr, child_index, required) + } } } } diff --git a/src/query/storages/fuse/src/operations/merge_into/mutator/delete_by_expr_mutator.rs b/src/query/storages/fuse/src/operations/merge_into/mutator/delete_by_expr_mutator.rs index 04abc9909bc8..8b937c88f786 100644 --- a/src/query/storages/fuse/src/operations/merge_into/mutator/delete_by_expr_mutator.rs +++ b/src/query/storages/fuse/src/operations/merge_into/mutator/delete_by_expr_mutator.rs @@ -12,8 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::ops::Not; - use common_exception::Result; use common_expression::eval_function; use common_expression::types::BooleanType; From b60479ea842aa3d90c2abf4bc9e2be571786ec00 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Wed, 25 Oct 2023 11:02:38 +0800 Subject: [PATCH 21/39] fix row number --- .../service/src/pipelines/pipeline_builder.rs | 31 ++++++++++++++ src/query/sql/src/executor/mod.rs | 1 + .../physical_plans/physical_add_row_number.rs | 2 +- .../transform_add_rownumber_column.rs | 40 ++----------------- 4 files changed, 37 insertions(+), 37 deletions(-) diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs index 05cbc2c928fb..f524cf905b1a 100644 --- a/src/query/service/src/pipelines/pipeline_builder.rs +++ b/src/query/service/src/pipelines/pipeline_builder.rs @@ -14,6 +14,7 @@ use std::collections::HashMap; use std::convert::TryFrom; +use std::sync::atomic::AtomicU64; use std::sync::Arc; use std::sync::Mutex; use std::time::Instant; @@ -72,6 +73,7 @@ use common_profile::SharedProcessorProfiles; use common_settings::Settings; use common_sql::evaluator::BlockOperator; use common_sql::evaluator::CompoundBlockOperator; +use common_sql::executor::AddRowNumber; use common_sql::executor::AggregateExpand; use common_sql::executor::AggregateFinal; use common_sql::executor::AggregateFunctionDesc; @@ -122,6 +124,7 @@ use common_storages_fuse::operations::common::TransformSerializeSegment; use common_storages_fuse::operations::merge_into::MatchedSplitProcessor; use common_storages_fuse::operations::merge_into::MergeIntoNotMatchedProcessor; use common_storages_fuse::operations::merge_into::MergeIntoSplitProcessor; +use common_storages_fuse::operations::merge_into::TransformAddRowNumberColumnProcessor; use common_storages_fuse::operations::merge_into::TransformDistributedMergeIntoBlockDeserialize; use common_storages_fuse::operations::merge_into::TransformDistributedMergeIntoBlockSerialize; use common_storages_fuse::operations::replace_into::BroadcastProcessor; @@ -294,6 +297,9 @@ impl PipelineBuilder { PhysicalPlan::MergeIntoRowIdApply(merge_into_row_id_apply) => { self.build_merge_into_row_id_apply(merge_into_row_id_apply) } + PhysicalPlan::AddRowNumber(add_row_number) => { + self.build_add_row_number(&add_row_number) + } } } @@ -306,6 +312,31 @@ impl PipelineBuilder { Ok(cast_needed) } + fn build_add_row_number(&mut self, add_row_number: &AddRowNumber) -> Result<()> { + // it must be distributed merge into execution + let node_index = add_row_number + .cluster_index + .get(&self.ctx.get_cluster().local_id); + if node_index.is_none() { + return Err(ErrorCode::NotFoundClusterNode(format!( + "can't find out {} when build distributed merge into pipeline", + self.ctx.get_cluster().local_id + ))); + } + let node_index = *node_index.unwrap() as u16; + let row_number = Arc::new(AtomicU64::new(0)); + self.main_pipeline + .add_transform(|transform_input_port, transform_output_port| { + TransformAddRowNumberColumnProcessor::create( + transform_input_port, + transform_output_port, + node_index, + row_number.clone(), + ) + })?; + todo!() + } + fn build_merge_into_row_id_apply( &mut self, merge_into_row_id_apply: &MergeIntoRowIdApply, diff --git a/src/query/sql/src/executor/mod.rs b/src/query/sql/src/executor/mod.rs index 57be14fdf14a..14f5f2b164d7 100644 --- a/src/query/sql/src/executor/mod.rs +++ b/src/query/sql/src/executor/mod.rs @@ -30,6 +30,7 @@ pub use physical_plans::common::AggregateFunctionDesc; pub use physical_plans::common::FragmentKind; pub use physical_plans::common::MutationKind; pub use physical_plans::common::OnConflictField; +pub use physical_plans::physical_add_row_number::AddRowNumber; pub use physical_plans::physical_aggregate_expand::AggregateExpand; pub use physical_plans::physical_aggregate_final::AggregateFinal; pub use physical_plans::physical_aggregate_partial::AggregatePartial; diff --git a/src/query/sql/src/executor/physical_plans/physical_add_row_number.rs b/src/query/sql/src/executor/physical_plans/physical_add_row_number.rs index d31e34120ea3..e2e04d5cf55a 100644 --- a/src/query/sql/src/executor/physical_plans/physical_add_row_number.rs +++ b/src/query/sql/src/executor/physical_plans/physical_add_row_number.rs @@ -40,7 +40,7 @@ impl PhysicalPlanBuilder { pub(crate) async fn build_add_row_number( &mut self, s_expr: &SExpr, - mut required: ColumnSet, + required: ColumnSet, ) -> Result { let input_plan = self.build(s_expr.child(0)?, required).await?; if self.ctx.get_cluster().is_empty() { diff --git a/src/query/storages/fuse/src/operations/merge_into/processors/transform_add_rownumber_column.rs b/src/query/storages/fuse/src/operations/merge_into/processors/transform_add_rownumber_column.rs index 7fe42618767f..5ce8d9f781f6 100644 --- a/src/query/storages/fuse/src/operations/merge_into/processors/transform_add_rownumber_column.rs +++ b/src/query/storages/fuse/src/operations/merge_into/processors/transform_add_rownumber_column.rs @@ -24,8 +24,6 @@ use common_expression::BlockEntry; use common_expression::DataBlock; use common_expression::FromData; use common_expression::Value; -use common_pipeline_core::pipe::Pipe; -use common_pipeline_core::pipe::PipeItem; use common_pipeline_core::processors::port::InputPort; use common_pipeline_core::processors::port::OutputPort; use common_pipeline_core::processors::processor::ProcessorPtr; @@ -42,50 +40,20 @@ pub struct TransformAddRowNumberColumnProcessor { } impl TransformAddRowNumberColumnProcessor { - #[allow(dead_code)] - fn create( + pub fn create( input: Arc, output: Arc, node_id: u16, row_number: Arc, - ) -> ProcessorPtr { - ProcessorPtr::create(Transformer::create( + ) -> Result { + Ok(ProcessorPtr::create(Transformer::create( input, output, TransformAddRowNumberColumnProcessor { prefix: node_id, row_number, }, - )) - } - - fn create_row_number_column_transform_item( - node_id: u16, - row_number: Arc, - ) -> PipeItem { - let input = InputPort::create(); - let output = OutputPort::create(); - PipeItem::create( - TransformAddRowNumberColumnProcessor::create( - input.clone(), - output.clone(), - node_id, - row_number, - ), - vec![input], - vec![output], - ) - } - - pub fn into_pipe(node_id: u16, num_threads: usize) -> Pipe { - let mut pipe_items = Vec::with_capacity(num_threads); - let row_number = Arc::new(AtomicU64::new(0)); - for _ in 0..num_threads { - let pipe_item = - Self::create_row_number_column_transform_item(node_id, row_number.clone()); - pipe_items.push(pipe_item); - } - Pipe::create(num_threads, num_threads, pipe_items) + ))) } } From c79a04b8788b6788bcc0e52cb13574c4a9b0c5b7 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Wed, 25 Oct 2023 18:12:28 +0800 Subject: [PATCH 22/39] refactor merge into pipeline --- src/query/catalog/src/plan/internal_column.rs | 1 - src/query/expression/src/schema.rs | 3 +- src/query/pipeline/core/src/pipe.rs | 6 + .../interpreters/interpreter_merge_into.rs | 27 +- .../service/src/pipelines/pipeline_builder.rs | 322 ++++++++++++------ src/query/sql/src/executor/format.rs | 4 +- src/query/sql/src/executor/mod.rs | 2 +- src/query/sql/src/executor/physical_plan.rs | 16 +- .../sql/src/executor/physical_plan_display.rs | 8 +- .../sql/src/executor/physical_plan_visitor.rs | 14 +- .../physical_plans/physical_add_row_number.rs | 16 + .../physical_plans/physical_merge_into.rs | 6 +- src/query/sql/src/executor/profile.rs | 2 +- .../sql/src/planner/binder/merge_into.rs | 1 - .../storages/fuse/src/operations/append.rs | 20 +- 15 files changed, 305 insertions(+), 143 deletions(-) diff --git a/src/query/catalog/src/plan/internal_column.rs b/src/query/catalog/src/plan/internal_column.rs index 58b977787103..2a04b4185b45 100644 --- a/src/query/catalog/src/plan/internal_column.rs +++ b/src/query/catalog/src/plan/internal_column.rs @@ -29,7 +29,6 @@ use common_expression::TableDataType; use common_expression::Value; use common_expression::BLOCK_NAME_COLUMN_ID; use common_expression::ROW_ID_COLUMN_ID; -use common_expression::ROW_NUMBER_COLUMN_ID; use common_expression::SEGMENT_NAME_COLUMN_ID; use common_expression::SNAPSHOT_NAME_COLUMN_ID; use storages_common_table_meta::meta::NUM_BLOCK_ID_BITS; diff --git a/src/query/expression/src/schema.rs b/src/query/expression/src/schema.rs index 7c6599ca9333..0a3af16daee5 100644 --- a/src/query/expression/src/schema.rs +++ b/src/query/expression/src/schema.rs @@ -48,8 +48,7 @@ pub type FieldIndex = usize; pub const ROW_ID_COLUMN_ID: u32 = u32::MAX; pub const BLOCK_NAME_COLUMN_ID: u32 = u32::MAX - 1; pub const SEGMENT_NAME_COLUMN_ID: u32 = u32::MAX - 2; -pub const ROW_NUMBER_COLUMN_ID: u32 = u32::MAX - 3; -pub const SNAPSHOT_NAME_COLUMN_ID: u32 = u32::MAX - 4; +pub const SNAPSHOT_NAME_COLUMN_ID: u32 = u32::MAX - 3; pub const ROW_ID_COL_NAME: &str = "_row_id"; pub const ROW_NUMBER_COL_NAME: &str = "_row_number"; diff --git a/src/query/pipeline/core/src/pipe.rs b/src/query/pipeline/core/src/pipe.rs index a64d836f5397..83219e2b636b 100644 --- a/src/query/pipeline/core/src/pipe.rs +++ b/src/query/pipeline/core/src/pipe.rs @@ -144,4 +144,10 @@ impl TransformPipeBuilder { } self.items = items } + + pub fn add_items(&mut self, items: Vec) { + for item in items { + self.items.push(item) + } + } } diff --git a/src/query/service/src/interpreters/interpreter_merge_into.rs b/src/query/service/src/interpreters/interpreter_merge_into.rs index e40385e29079..b6a4499fa174 100644 --- a/src/query/service/src/interpreters/interpreter_merge_into.rs +++ b/src/query/service/src/interpreters/interpreter_merge_into.rs @@ -25,13 +25,14 @@ use common_expression::DataSchema; use common_expression::DataSchemaRef; use common_expression::FieldIndex; use common_expression::RemoteExpr; +use common_expression::ROW_NUMBER_COL_NAME; use common_functions::BUILTIN_FUNCTIONS; use common_meta_app::schema::TableInfo; use common_sql::executor::CommitSink; use common_sql::executor::Exchange; use common_sql::executor::FragmentKind::Merge; use common_sql::executor::MergeInto; -use common_sql::executor::MergeIntoRowIdApply; +use common_sql::executor::MergeIntoAppendNotMatched; use common_sql::executor::MergeIntoSource; use common_sql::executor::MutationKind; use common_sql::executor::PhysicalPlan; @@ -177,12 +178,17 @@ impl MergeIntoInterpreter { }; let mut found_row_id = false; + let mut row_number_idx = -1; for (idx, data_field) in join_output_schema.fields().iter().enumerate() { if *data_field.name() == row_id_idx.to_string() { row_id_idx = idx; found_row_id = true; break; } + + if exchange.is_some() && data_field.name() == ROW_NUMBER_COL_NAME { + row_number_idx = idx as i32; + } } // we can't get row_id_idx, throw an exception @@ -192,6 +198,12 @@ impl MergeIntoInterpreter { )); } + if exchange.is_some() && row_number_idx == -1 { + return Err(ErrorCode::InvalidRowIdIndex( + "can't get internal row_number_idx when running merge into", + )); + } + let table = self.ctx.get_table(catalog, database, &table_name).await?; let fuse_table = table @@ -325,7 +337,7 @@ impl MergeIntoInterpreter { .insert(*field_index, join_output_schema.index_of(value).unwrap()); } - let segments = base_snapshot + let segments: Vec<_> = base_snapshot .segments .clone() .into_iter() @@ -343,7 +355,8 @@ impl MergeIntoInterpreter { matched, field_index_of_input_schema, row_id_idx, - segments: Some(segments), + segments: segments.clone(), + distributed: false, output_schema: DataSchemaRef::default(), })) } else { @@ -355,13 +368,14 @@ impl MergeIntoInterpreter { matched, field_index_of_input_schema, row_id_idx, - segments: None, + segments, + distributed: true, output_schema: DataSchemaRef::new(DataSchema::new(vec![ - join_output_schema.fields[row_id_idx].clone(), + join_output_schema.fields[row_number_idx as usize].clone(), ])), })); - PhysicalPlan::MergeIntoRowIdApply(Box::new(MergeIntoRowIdApply { + PhysicalPlan::MergeIntoAppendNotMatched(Box::new(MergeIntoAppendNotMatched { input: Box::new(PhysicalPlan::Exchange(Exchange { plan_id: 0, input: Box::new(merge_append), @@ -371,7 +385,6 @@ impl MergeIntoInterpreter { })), table_info: table_info.clone(), catalog_info: catalog_.info(), - segments, })) }; diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs index f524cf905b1a..70462f4e4445 100644 --- a/src/query/service/src/pipelines/pipeline_builder.rs +++ b/src/query/service/src/pipelines/pipeline_builder.rs @@ -43,6 +43,7 @@ use common_expression::FunctionContext; use common_expression::HashMethodKind; use common_expression::Scalar; use common_expression::SortColumnDescription; +use common_expression::ROW_NUMBER_COL_NAME; use common_formats::FastFieldDecoderValues; use common_formats::FastValuesDecodeFallback; use common_formats::FastValuesDecoder; @@ -97,7 +98,7 @@ use common_sql::executor::Lambda; use common_sql::executor::Limit; use common_sql::executor::MaterializedCte; use common_sql::executor::MergeInto; -use common_sql::executor::MergeIntoRowIdApply; +use common_sql::executor::MergeIntoAppendNotMatched; use common_sql::executor::MergeIntoSource; use common_sql::executor::PhysicalPlan; use common_sql::executor::Project; @@ -294,7 +295,7 @@ impl PipelineBuilder { PhysicalPlan::MergeIntoSource(merge_into_source) => { self.build_merge_into_source(merge_into_source) } - PhysicalPlan::MergeIntoRowIdApply(merge_into_row_id_apply) => { + PhysicalPlan::MergeIntoAppendNotMatched(merge_into_row_id_apply) => { self.build_merge_into_row_id_apply(merge_into_row_id_apply) } PhysicalPlan::AddRowNumber(add_row_number) => { @@ -334,19 +335,17 @@ impl PipelineBuilder { row_number.clone(), ) })?; - todo!() + Ok(()) } fn build_merge_into_row_id_apply( &mut self, - merge_into_row_id_apply: &MergeIntoRowIdApply, + merge_into_row_id_apply: &MergeIntoAppendNotMatched, ) -> Result<()> { - let MergeIntoRowIdApply { + let MergeIntoAppendNotMatched { input, table_info, catalog_info, - - segments, } = merge_into_row_id_apply; // receive rowids and MutationLogs self.build_pipeline(input)?; @@ -355,7 +354,7 @@ impl PipelineBuilder { self.main_pipeline .add_pipe(TransformDistributedMergeIntoBlockDeserialize::into_pipe()); - let mut pipe_items = Vec::with_capacity(1); + // let mut pipe_items = Vec::with_capacity(1); let tbl = self .ctx .build_table_by_table_info(catalog_info, table_info, None)?; @@ -367,30 +366,31 @@ impl PipelineBuilder { let cluster_stats_gen = table.get_cluster_stats_gen(self.ctx.clone(), 0, block_thresholds, None)?; - // this TransformSerializeBlock is just used to get block_builder - let block_builder = TransformSerializeBlock::try_create( - self.ctx.clone(), - InputPort::create(), - OutputPort::create(), - table, - cluster_stats_gen, - )? - .get_block_builder(); - - let max_threads = self.settings.get_max_threads()?; - let io_request_semaphore = Arc::new(Semaphore::new(max_threads as usize)); - - pipe_items.push(table.rowid_aggregate_mutator( - self.ctx.clone(), - block_builder, - io_request_semaphore, - segments.clone(), - true, - )?); - - self.main_pipeline - .add_pipe(Pipe::create(self.main_pipeline.output_len(), 1, pipe_items)); - Ok(()) + // // this TransformSerializeBlock is just used to get block_builder + // let block_builder = TransformSerializeBlock::try_create( + // self.ctx.clone(), + // InputPort::create(), + // OutputPort::create(), + // table, + // cluster_stats_gen, + // )? + // .get_block_builder(); + + // let max_threads = self.settings.get_max_threads()?; + // let io_request_semaphore = Arc::new(Semaphore::new(max_threads as usize)); + + // pipe_items.push(table.rowid_aggregate_mutator( + // self.ctx.clone(), + // block_builder, + // io_request_semaphore, + // segments.clone(), + // true, + // )?); + + // self.main_pipeline + // .add_pipe(Pipe::create(self.main_pipeline.output_len(), 1, pipe_items)); + // Ok(()) + todo!() } fn build_merge_into_source(&mut self, merge_into_source: &MergeIntoSource) -> Result<()> { @@ -555,12 +555,12 @@ impl PipelineBuilder { field_index_of_input_schema, row_id_idx, segments, + distributed, .. } = merge_into; self.build_pipeline(input)?; - let apply_row_id = segments.is_some(); let tbl = self .ctx .build_table_by_table_info(catalog_info, table_info, None)?; @@ -650,43 +650,88 @@ impl PipelineBuilder { // not macthed insert data port1_3 // ...... assert_eq!(self.main_pipeline.output_len() % 3, 0); - - // merge rowid and serialize_blocks let mut ranges = Vec::with_capacity(self.main_pipeline.output_len() / 3 * 2); - for idx in (0..self.main_pipeline.output_len()).step_by(3) { - ranges.push(vec![idx]); - ranges.push(vec![idx + 1, idx + 2]); - } - self.main_pipeline.resize_partial_one(ranges.clone())?; - assert_eq!(self.main_pipeline.output_len() % 2, 0); - - // shuffle outputs and resize row_id - // ---------------------------------------------------------------------- - // row_id port0_1 row_id port0_1 - // data port0_2 row_id port1_1 row_id port - // row_id port1_1 ======> ...... ======> - // data port1_2 data port0_2 data port0 - // ...... data port1_2 data port1 - // ...... ..... ..... - // ---------------------------------------------------------------------- - let mut rules = Vec::with_capacity(self.main_pipeline.output_len()); - for idx in 0..(self.main_pipeline.output_len() / 2) { - rules.push(idx); - rules.push(idx + self.main_pipeline.output_len() / 2); - } - self.main_pipeline.reorder_inputs(rules); - // resize row_id - ranges.clear(); - let mut vec = Vec::with_capacity(self.main_pipeline.output_len() / 2); - for idx in 0..(self.main_pipeline.output_len() / 2) { - vec.push(idx); - } - ranges.push(vec.clone()); - for idx in 0..(self.main_pipeline.output_len() / 2) { - ranges.push(vec![idx + self.main_pipeline.output_len() / 2]); + if !*distributed { + // merge rowid and serialize_blocks + for idx in (0..self.main_pipeline.output_len()).step_by(3) { + ranges.push(vec![idx]); + ranges.push(vec![idx + 1, idx + 2]); + } + self.main_pipeline.resize_partial_one(ranges.clone())?; + assert_eq!(self.main_pipeline.output_len() % 2, 0); + + // shuffle outputs and resize row_id + // ---------------------------------------------------------------------- + // row_id port0_1 row_id port0_1 + // data port0_2 row_id port1_1 row_id port + // row_id port1_1 ======> ...... ======> + // data port1_2 data port0_2 data port0 + // ...... data port1_2 data port1 + // ...... ..... ..... + // ---------------------------------------------------------------------- + let mut rules = Vec::with_capacity(self.main_pipeline.output_len()); + for idx in 0..(self.main_pipeline.output_len() / 2) { + rules.push(idx); + rules.push(idx + self.main_pipeline.output_len() / 2); + } + self.main_pipeline.reorder_inputs(rules); + // resize row_id + ranges.clear(); + let mut vec = Vec::with_capacity(self.main_pipeline.output_len() / 2); + for idx in 0..(self.main_pipeline.output_len() / 2) { + vec.push(idx); + } + ranges.push(vec.clone()); + for idx in 0..(self.main_pipeline.output_len() / 2) { + ranges.push(vec![idx + self.main_pipeline.output_len() / 2]); + } + self.main_pipeline.resize_partial_one(ranges.clone())?; + } else { + // shuffle outputs and resize row_id + // ---------------------------------------------------------------------- + // row_id port0_1 row_id port0_1 row_id port + // matched data port0_2 row_id port1_1 matched data port0_2 + // unmatched data port0_3 matched data port0_2 matched data port1_2 + // row_id port1_1 matched data port1_2 ...... + // matched data port1_2 ===> ..... ====> ...... + // unmatched data port1_2 ..... ...... + // unmatched data port0_3 unmatched data port0_3 unmatched data port + // ...... unmatched data port1_3 + // ...... ..... + // ---------------------------------------------------------------------- + // do shuffle + let mut rules = Vec::with_capacity(self.main_pipeline.output_len()); + for idx in 0..(self.main_pipeline.output_len() / 3) { + rules.push(idx); + rules.push(idx + self.main_pipeline.output_len() / 3); + rules.push(idx + self.main_pipeline.output_len() / 3 * 2); + } + self.main_pipeline.reorder_inputs(rules); + + // resize row_id + ranges.clear(); + let mut vec = Vec::with_capacity(self.main_pipeline.output_len() / 3); + for idx in 0..(self.main_pipeline.output_len() / 3) { + vec.push(idx); + } + ranges.push(vec.clone()); + for idx in 0..(self.main_pipeline.output_len() / 3) { + ranges.push(vec![idx + self.main_pipeline.output_len() / 3]); + } + vec.clear(); + for idx in 0..(self.main_pipeline.output_len() / 3) { + vec.push(idx + self.main_pipeline.output_len() / 3 * 2); + } + self.main_pipeline.resize_partial_one(ranges.clone())?; } - self.main_pipeline.resize_partial_one(ranges.clone())?; + let fill_default_len = if !*distributed { + // remove first row_id port + self.main_pipeline.output_len() - 1 + } else { + // remove first row_id port and last unmatched_port + self.main_pipeline.output_len() - 2 + }; // fill default columns let table_default_schema = &table.schema().remove_computed_fields(); let mut builder = self.main_pipeline.add_transform_with_specified_len( @@ -699,10 +744,42 @@ impl PipelineBuilder { tbl.clone(), ) }, - self.main_pipeline.output_len() - 1, + fill_default_len, )?; - builder.add_items_prepend(vec![create_dummy_item()]); - self.main_pipeline.add_pipe(builder.finalize()); + + if !*distributed { + builder.add_items_prepend(vec![create_dummy_item()]); + self.main_pipeline.add_pipe(builder.finalize()); + } else { + let input_num_columns = input.output_schema()?.num_fields(); + assert_eq!( + input.output_schema()?.field(input_num_columns - 1).name(), + ROW_NUMBER_COL_NAME + ); + let input_port = InputPort::create(); + let output_port = OutputPort::create(); + // project row number column + let proc = ProcessorPtr::create(CompoundBlockOperator::create( + input_port.clone(), + output_port.clone(), + input_num_columns, + self.func_ctx.clone(), + vec![BlockOperator::Project { + projection: vec![input_num_columns - 1], + }], + )); + builder.add_items_prepend(vec![create_dummy_item()]); + builder.add_transform(input_port, output_port, proc); + self.main_pipeline.add_pipe(builder.finalize()); + } + + let fill_computed_len = if !*distributed { + // remove first row_id port + self.main_pipeline.output_len() - 1 + } else { + // remove first row_id port and last row_number_port + self.main_pipeline.output_len() - 2 + }; // fill computed columns let table_computed_schema = &table.schema().remove_virtual_computed_fields(); @@ -719,10 +796,17 @@ impl PipelineBuilder { computed_schema.clone(), ) }, - self.main_pipeline.output_len() - 1, + fill_computed_len, )?; - builder.add_items_prepend(vec![create_dummy_item()]); - self.main_pipeline.add_pipe(builder.finalize()); + if !*distributed { + builder.add_items_prepend(vec![create_dummy_item()]); + self.main_pipeline.add_pipe(builder.finalize()); + } else { + builder.add_items_prepend(vec![create_dummy_item()]); + // recieve row_number + builder.add_items(vec![create_dummy_item()]); + self.main_pipeline.add_pipe(builder.finalize()); + } } let max_threads = self.settings.get_max_threads()?; @@ -730,30 +814,41 @@ impl PipelineBuilder { // after filling default columns, we need to add cluster‘s blocksort if it's a cluster table let output_lens = self.main_pipeline.output_len(); - table.cluster_gen_for_append_with_specified_last_len( - self.ctx.clone(), - &mut self.main_pipeline, - block_thresholds, - output_lens - 1, - )?; + if !*distributed { + table.cluster_gen_for_append_with_specified_len( + self.ctx.clone(), + &mut self.main_pipeline, + block_thresholds, + output_lens - 1, + 0, + )?; + } else { + table.cluster_gen_for_append_with_specified_len( + self.ctx.clone(), + &mut self.main_pipeline, + block_thresholds, + output_lens - 2, + 1, + )?; + } pipe_items.clear(); - // standalone execution - if apply_row_id { - pipe_items.push(table.rowid_aggregate_mutator( - self.ctx.clone(), - block_builder, - io_request_semaphore, - segments.clone().unwrap(), - false, - )?); + pipe_items.push(table.rowid_aggregate_mutator( + self.ctx.clone(), + block_builder, + io_request_semaphore, + segments.clone(), + false, + )?); + + let serialize_len = if !*distributed { + self.main_pipeline.output_len() - 1 } else { - // distributed execution - pipe_items.push(create_dummy_item()) - } + self.main_pipeline.output_len() - 2 + }; - for _ in 0..self.main_pipeline.output_len() - 1 { + for _ in 0..serialize_len { let serialize_block_transform = TransformSerializeBlock::try_create( self.ctx.clone(), InputPort::create(), @@ -764,6 +859,11 @@ impl PipelineBuilder { pipe_items.push(serialize_block_transform.into_pipe_item()); } + // recieve row_number + if *distributed { + pipe_items.push(create_dummy_item()); + } + self.main_pipeline.add_pipe(Pipe::create( self.main_pipeline.output_len(), get_output_len(&pipe_items), @@ -775,27 +875,45 @@ impl PipelineBuilder { // serialize_block port0 ======> // serialize_block port1 serialize_block port // ....... + // row_number_port (distributed) row_number_port(distributed) ranges.clear(); ranges.push(vec![0]); - vec.clear(); - for idx in 0..self.main_pipeline.output_len() - 1 { + let mut vec = Vec::with_capacity(self.main_pipeline.output_len()); + let output_lens = if !*distributed { + self.main_pipeline.output_len() - 1 + } else { + self.main_pipeline.output_len() - 2 + }; + for idx in 0..output_lens { vec.push(idx + 1); } + if *distributed { + ranges.push(vec![self.main_pipeline.output_len() - 1]); + } ranges.push(vec); self.main_pipeline.resize_partial_one(ranges)?; - let pipe_items = vec![ - create_dummy_item(), - serialize_segment_transform.into_pipe_item(), - ]; + let pipe_items = if !distributed { + vec![ + create_dummy_item(), + serialize_segment_transform.into_pipe_item(), + ] + } else { + vec![ + create_dummy_item(), + serialize_segment_transform.into_pipe_item(), + create_dummy_item(), + ] + }; - // apply_row_id: true + // distributed: false // output_port0: MutationLogs // output_port1: MutationLogs // - // apply_row_id: false - // output_port0: row_ids + // distributed: true + // output_port0: MutationLogs // output_port1: MutationLogs + // output_port2: row_numbers self.main_pipeline.add_pipe(Pipe::create( self.main_pipeline.output_len(), get_output_len(&pipe_items), @@ -803,7 +921,7 @@ impl PipelineBuilder { )); // distributed execution - if !apply_row_id { + if *distributed { self.main_pipeline.try_resize(1)?; self.main_pipeline .add_pipe(TransformDistributedMergeIntoBlockSerialize::into_pipe()) diff --git a/src/query/sql/src/executor/format.rs b/src/query/sql/src/executor/format.rs index bd7c6f30b85a..6d113fb4c5e4 100644 --- a/src/query/sql/src/executor/format.rs +++ b/src/query/sql/src/executor/format.rs @@ -213,8 +213,8 @@ fn to_format_tree( PhysicalPlan::MergeInto(_) => Ok(FormatTreeNode::new("MergeInto".to_string())), PhysicalPlan::MergeIntoSource(_) => Ok(FormatTreeNode::new("MergeIntoSource".to_string())), PhysicalPlan::AddRowNumber(_) => Ok(FormatTreeNode::new("AddRowNumber".to_string())), - PhysicalPlan::MergeIntoRowIdApply(_) => { - Ok(FormatTreeNode::new("MergeIntoRowIdApply".to_string())) + PhysicalPlan::MergeIntoAppendNotMatched(_) => { + Ok(FormatTreeNode::new("MergeIntoAppendNotMatched".to_string())) } PhysicalPlan::CteScan(plan) => cte_scan_to_format_tree(plan), PhysicalPlan::MaterializedCte(plan) => { diff --git a/src/query/sql/src/executor/mod.rs b/src/query/sql/src/executor/mod.rs index 14f5f2b164d7..561599f7f968 100644 --- a/src/query/sql/src/executor/mod.rs +++ b/src/query/sql/src/executor/mod.rs @@ -58,7 +58,7 @@ pub use physical_plans::physical_limit::Limit; pub use physical_plans::physical_materialized_cte::MaterializedCte; pub use physical_plans::physical_merge_into::MatchExpr; pub use physical_plans::physical_merge_into::MergeInto; -pub use physical_plans::physical_merge_into::MergeIntoRowIdApply; +pub use physical_plans::physical_merge_into::MergeIntoAppendNotMatched; pub use physical_plans::physical_merge_into::MergeIntoSource; pub use physical_plans::physical_project::Project; pub use physical_plans::physical_project_set::ProjectSet; diff --git a/src/query/sql/src/executor/physical_plan.rs b/src/query/sql/src/executor/physical_plan.rs index 685a1f04724b..f461a9a1fad2 100644 --- a/src/query/sql/src/executor/physical_plan.rs +++ b/src/query/sql/src/executor/physical_plan.rs @@ -18,7 +18,7 @@ use common_expression::DataSchemaRef; use enum_as_inner::EnumAsInner; use super::physical_plans::physical_add_row_number::AddRowNumber; -use super::MergeIntoRowIdApply; +use super::MergeIntoAppendNotMatched; use crate::executor::physical_plans::physical_aggregate_expand::AggregateExpand; use crate::executor::physical_plans::physical_aggregate_final::AggregateFinal; use crate::executor::physical_plans::physical_aggregate_partial::AggregatePartial; @@ -99,7 +99,7 @@ pub enum PhysicalPlan { /// MergeInto MergeIntoSource(MergeIntoSource), MergeInto(Box), - MergeIntoRowIdApply(Box), + MergeIntoAppendNotMatched(Box), AddRowNumber(Box), /// Compact @@ -139,7 +139,7 @@ impl PhysicalPlan { | PhysicalPlan::MergeInto(_) | PhysicalPlan::AddRowNumber(_) | PhysicalPlan::MergeIntoSource(_) - | PhysicalPlan::MergeIntoRowIdApply(_) + | PhysicalPlan::MergeIntoAppendNotMatched(_) | PhysicalPlan::CommitSink(_) | PhysicalPlan::CopyIntoTable(_) | PhysicalPlan::AsyncSourcer(_) @@ -182,11 +182,11 @@ impl PhysicalPlan { PhysicalPlan::ConstantTableScan(plan) => plan.output_schema(), PhysicalPlan::MergeIntoSource(plan) => plan.input.output_schema(), PhysicalPlan::MergeInto(plan) => Ok(plan.output_schema.clone()), - PhysicalPlan::AddRowNumber(plan) => plan.input.output_schema(), + PhysicalPlan::AddRowNumber(plan) => plan.output_schema(), PhysicalPlan::AsyncSourcer(_) | PhysicalPlan::Deduplicate(_) | PhysicalPlan::ReplaceInto(_) - | PhysicalPlan::MergeIntoRowIdApply(_) + | PhysicalPlan::MergeIntoAppendNotMatched(_) | PhysicalPlan::CompactSource(_) => Ok(DataSchemaRef::default()), } } @@ -223,7 +223,7 @@ impl PhysicalPlan { PhysicalPlan::ReplaceInto(_) => "Replace".to_string(), PhysicalPlan::MergeInto(_) => "MergeInto".to_string(), PhysicalPlan::MergeIntoSource(_) => "MergeIntoSource".to_string(), - PhysicalPlan::MergeIntoRowIdApply(_) => "MergeIntoRowIdApply".to_string(), + PhysicalPlan::MergeIntoAppendNotMatched(_) => "MergeIntoAppendNotMatched".to_string(), PhysicalPlan::CteScan(_) => "PhysicalCteScan".to_string(), PhysicalPlan::MaterializedCte(_) => "PhysicalMaterializedCte".to_string(), PhysicalPlan::ConstantTableScan(_) => "PhysicalConstantTableScan".to_string(), @@ -277,7 +277,7 @@ impl PhysicalPlan { PhysicalPlan::MergeInto(plan) => Box::new(std::iter::once(plan.input.as_ref())), PhysicalPlan::AddRowNumber(plan) => Box::new(std::iter::once(plan.input.as_ref())), PhysicalPlan::MergeIntoSource(plan) => Box::new(std::iter::once(plan.input.as_ref())), - PhysicalPlan::MergeIntoRowIdApply(plan) => { + PhysicalPlan::MergeIntoAppendNotMatched(plan) => { Box::new(std::iter::once(plan.input.as_ref())) } PhysicalPlan::MaterializedCte(plan) => Box::new( @@ -320,7 +320,7 @@ impl PhysicalPlan { | PhysicalPlan::ReplaceInto(_) | PhysicalPlan::MergeInto(_) | PhysicalPlan::AddRowNumber(_) - | PhysicalPlan::MergeIntoRowIdApply(_) + | PhysicalPlan::MergeIntoAppendNotMatched(_) | PhysicalPlan::MergeIntoSource(_) | PhysicalPlan::ConstantTableScan(_) | PhysicalPlan::CteScan(_) => None, diff --git a/src/query/sql/src/executor/physical_plan_display.rs b/src/query/sql/src/executor/physical_plan_display.rs index d7832447c5d3..ef01f044dc39 100644 --- a/src/query/sql/src/executor/physical_plan_display.rs +++ b/src/query/sql/src/executor/physical_plan_display.rs @@ -19,7 +19,7 @@ use common_functions::BUILTIN_FUNCTIONS; use itertools::Itertools; use super::physical_plans::physical_add_row_number::AddRowNumber; -use super::MergeIntoRowIdApply; +use super::MergeIntoAppendNotMatched; use crate::executor::physical_plans::physical_aggregate_expand::AggregateExpand; use crate::executor::physical_plans::physical_aggregate_final::AggregateFinal; use crate::executor::physical_plans::physical_aggregate_partial::AggregatePartial; @@ -102,7 +102,7 @@ impl<'a> Display for PhysicalPlanIndentFormatDisplay<'a> { PhysicalPlan::ReplaceInto(replace) => write!(f, "{}", replace)?, PhysicalPlan::MergeIntoSource(merge_into_source) => write!(f, "{}", merge_into_source)?, PhysicalPlan::MergeInto(merge_into) => write!(f, "{}", merge_into)?, - PhysicalPlan::MergeIntoRowIdApply(merge_into_row_id_apply) => { + PhysicalPlan::MergeIntoAppendNotMatched(merge_into_row_id_apply) => { write!(f, "{}", merge_into_row_id_apply)? } PhysicalPlan::AddRowNumber(add_row_number) => write!(f, "{}", add_row_number)?, @@ -482,9 +482,9 @@ impl Display for AddRowNumber { } } -impl Display for MergeIntoRowIdApply { +impl Display for MergeIntoAppendNotMatched { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { - write!(f, "MergeIntoRowIdApply") + write!(f, "MergeIntoAppendNotMatched") } } diff --git a/src/query/sql/src/executor/physical_plan_visitor.rs b/src/query/sql/src/executor/physical_plan_visitor.rs index dcfa7349f6ed..b36e74a00eac 100644 --- a/src/query/sql/src/executor/physical_plan_visitor.rs +++ b/src/query/sql/src/executor/physical_plan_visitor.rs @@ -15,7 +15,7 @@ use common_exception::Result; use super::physical_plans::physical_add_row_number::AddRowNumber; -use super::MergeIntoRowIdApply; +use super::MergeIntoAppendNotMatched; use crate::executor::physical_plans::physical_aggregate_expand::AggregateExpand; use crate::executor::physical_plans::physical_aggregate_final::AggregateFinal; use crate::executor::physical_plans::physical_aggregate_partial::AggregatePartial; @@ -88,7 +88,9 @@ pub trait PhysicalPlanReplacer { PhysicalPlan::MergeInto(plan) => self.replace_merge_into(plan), PhysicalPlan::AddRowNumber(plan) => self.replace_add_row_number(&plan), PhysicalPlan::MergeIntoSource(plan) => self.replace_merge_into_source(plan), - PhysicalPlan::MergeIntoRowIdApply(plan) => self.replace_merge_into_row_id_apply(plan), + PhysicalPlan::MergeIntoAppendNotMatched(plan) => { + self.replace_merge_into_row_id_apply(plan) + } PhysicalPlan::MaterializedCte(plan) => self.replace_materialized_cte(plan), PhysicalPlan::ConstantTableScan(plan) => self.replace_constant_table_scan(plan), } @@ -436,11 +438,11 @@ pub trait PhysicalPlanReplacer { fn replace_merge_into_row_id_apply( &mut self, - plan: &MergeIntoRowIdApply, + plan: &MergeIntoAppendNotMatched, ) -> Result { let input = self.replace(&plan.input)?; - Ok(PhysicalPlan::MergeIntoRowIdApply(Box::new( - MergeIntoRowIdApply { + Ok(PhysicalPlan::MergeIntoAppendNotMatched(Box::new( + MergeIntoAppendNotMatched { input: Box::new(input), ..plan.clone() }, @@ -586,7 +588,7 @@ impl PhysicalPlan { PhysicalPlan::AddRowNumber(plan) => { Self::traverse(&plan.input, pre_visit, visit, post_visit); } - PhysicalPlan::MergeIntoRowIdApply(plan) => { + PhysicalPlan::MergeIntoAppendNotMatched(plan) => { Self::traverse(&plan.input, pre_visit, visit, post_visit); } PhysicalPlan::MaterializedCte(plan) => { diff --git a/src/query/sql/src/executor/physical_plans/physical_add_row_number.rs b/src/query/sql/src/executor/physical_plans/physical_add_row_number.rs index e2e04d5cf55a..b0cd38d913ff 100644 --- a/src/query/sql/src/executor/physical_plans/physical_add_row_number.rs +++ b/src/query/sql/src/executor/physical_plans/physical_add_row_number.rs @@ -13,10 +13,16 @@ // limitations under the License. use std::collections::BTreeMap; +use std::sync::Arc; use common_exception::ErrorCode; use common_exception::Result; +use common_expression::types::DataType; +use common_expression::types::NumberDataType; +use common_expression::DataField; +use common_expression::DataSchema; use common_expression::DataSchemaRef; +use common_expression::ROW_NUMBER_COL_NAME; use crate::executor::PhysicalPlan; use crate::executor::PhysicalPlanBuilder; @@ -28,6 +34,7 @@ use crate::optimizer::SExpr; pub struct AddRowNumber { pub cluster_index: BTreeMap, pub input: Box, + pub output_schema: DataSchemaRef, } impl AddRowNumber { @@ -52,9 +59,18 @@ impl PhysicalPlanBuilder { for (id, node) in self.ctx.get_cluster().nodes.iter().enumerate() { cluster_index.insert(node.id.clone(), id); } + let input_schema = input_plan.output_schema()?; + let mut fields = input_schema.fields.clone(); + fields.push(DataField::new( + ROW_NUMBER_COL_NAME, + DataType::Number(NumberDataType::UInt64), + )); + let meta = input_schema.meta().clone(); + Ok(PhysicalPlan::AddRowNumber(Box::new(AddRowNumber { cluster_index, input: Box::new(input_plan), + output_schema: Arc::new(DataSchema::new_from(fields, meta)), }))) } } diff --git a/src/query/sql/src/executor/physical_plans/physical_merge_into.rs b/src/query/sql/src/executor/physical_plans/physical_merge_into.rs index 31ae6a1bb3f3..8c6dc864dc2b 100644 --- a/src/query/sql/src/executor/physical_plans/physical_merge_into.rs +++ b/src/query/sql/src/executor/physical_plans/physical_merge_into.rs @@ -45,14 +45,14 @@ pub struct MergeInto { // used to record the index of target table's field in merge_source_schema pub field_index_of_input_schema: HashMap, pub row_id_idx: usize, - pub segments: Option>, + pub segments: Vec<(usize, Location)>, pub output_schema: DataSchemaRef, + pub distributed: bool, } #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct MergeIntoRowIdApply { +pub struct MergeIntoAppendNotMatched { pub input: Box, pub table_info: TableInfo, pub catalog_info: CatalogInfo, - pub segments: Vec<(usize, Location)>, } diff --git a/src/query/sql/src/executor/profile.rs b/src/query/sql/src/executor/profile.rs index 1a006140afa9..8a24b99b9e4c 100644 --- a/src/query/sql/src/executor/profile.rs +++ b/src/query/sql/src/executor/profile.rs @@ -511,7 +511,7 @@ fn flatten_plan_node_profile( | PhysicalPlan::AsyncSourcer(_) | PhysicalPlan::MergeInto(_) | PhysicalPlan::AddRowNumber(_) - | PhysicalPlan::MergeIntoRowIdApply(_) + | PhysicalPlan::MergeIntoAppendNotMatched(_) | PhysicalPlan::MergeIntoSource(_) | PhysicalPlan::Deduplicate(_) | PhysicalPlan::ReplaceInto(_) diff --git a/src/query/sql/src/planner/binder/merge_into.rs b/src/query/sql/src/planner/binder/merge_into.rs index b0c6c4d05d82..e3b62ff56a2b 100644 --- a/src/query/sql/src/planner/binder/merge_into.rs +++ b/src/query/sql/src/planner/binder/merge_into.rs @@ -32,7 +32,6 @@ use common_expression::types::DataType; use common_expression::FieldIndex; use common_expression::TableSchemaRef; use common_expression::ROW_ID_COL_NAME; -use common_expression::ROW_NUMBER_COL_NAME; use indexmap::IndexMap; use super::wrap_cast_scalar; diff --git a/src/query/storages/fuse/src/operations/append.rs b/src/query/storages/fuse/src/operations/append.rs index 4a557137f8f7..dfb01e56257a 100644 --- a/src/query/storages/fuse/src/operations/append.rs +++ b/src/query/storages/fuse/src/operations/append.rs @@ -27,6 +27,7 @@ use common_expression::SortColumnDescription; use common_functions::BUILTIN_FUNCTIONS; use common_pipeline_core::processors::processor::ProcessorPtr; use common_pipeline_core::Pipeline; +use common_pipeline_transforms::processors::transforms::create_dummy_item; use common_pipeline_transforms::processors::transforms::create_dummy_items; use common_pipeline_transforms::processors::transforms::transform_block_compact_for_copy::BlockCompactorForCopy; use common_pipeline_transforms::processors::transforms::BlockCompactor; @@ -87,18 +88,25 @@ impl FuseTable { Ok(()) } - pub fn cluster_gen_for_append_with_specified_last_len( + pub fn cluster_gen_for_append_with_specified_len( &self, ctx: Arc, pipeline: &mut Pipeline, block_thresholds: BlockThresholds, + specified_mid_len: usize, specified_last_len: usize, ) -> Result { let cluster_stats_gen = self.get_cluster_stats_gen(ctx.clone(), 0, block_thresholds, None)?; let output_lens = pipeline.output_len(); - let items1 = create_dummy_items(output_lens - specified_last_len, output_lens); - let items2 = create_dummy_items(output_lens - specified_last_len, output_lens); + let items1 = create_dummy_items( + output_lens - specified_mid_len - specified_last_len, + output_lens, + ); + let items2 = create_dummy_items( + output_lens - specified_mid_len - specified_last_len, + output_lens, + ); let operators = cluster_stats_gen.operators.clone(); if !operators.is_empty() { let num_input_columns = self.table_info.schema().fields().len(); @@ -113,9 +121,10 @@ impl FuseTable { operators.clone(), ))) }, - specified_last_len, + specified_mid_len, )?; builder.add_items_prepend(items1); + builder.add_items(create_dummy_items(specified_last_len, specified_last_len)); pipeline.add_pipe(builder.finalize()); } @@ -140,9 +149,10 @@ impl FuseTable { sort_descs.clone(), )?)) }, - specified_last_len, + specified_mid_len, )?; builder.add_items_prepend(items2); + builder.add_items(create_dummy_items(specified_last_len, specified_last_len)); pipeline.add_pipe(builder.finalize()); } Ok(cluster_stats_gen) From 126da66326a51bee32668d5a5ed742a6127325b7 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Thu, 26 Oct 2023 13:00:35 +0800 Subject: [PATCH 23/39] split row_number and log, try to get hash table source data --- .../service/src/pipelines/pipeline_builder.rs | 29 ++-- .../fuse/src/operations/merge_into/mod.rs | 3 +- .../operations/merge_into/processors/mod.rs | 4 +- ...istributed_merge_into_block_deserialize.rs | 6 +- ..._distributed_merge_into_block_serialize.rs | 6 +- .../processor_merge_into_matched_and_split.rs | 6 +- ...sor_merge_into_split_row_number_and_log.rs | 140 ++++++++++++++++++ 7 files changed, 172 insertions(+), 22 deletions(-) create mode 100644 src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_split_row_number_and_log.rs diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs index d3303163388a..02e8e780c9b9 100644 --- a/src/query/service/src/pipelines/pipeline_builder.rs +++ b/src/query/service/src/pipelines/pipeline_builder.rs @@ -127,6 +127,7 @@ use common_storages_fuse::operations::common::TransformSerializeSegment; use common_storages_fuse::operations::merge_into::MatchedSplitProcessor; use common_storages_fuse::operations::merge_into::MergeIntoNotMatchedProcessor; use common_storages_fuse::operations::merge_into::MergeIntoSplitProcessor; +use common_storages_fuse::operations::merge_into::RowNumberAndLogSplitProcessor; use common_storages_fuse::operations::merge_into::TransformAddRowNumberColumnProcessor; use common_storages_fuse::operations::merge_into::TransformDistributedMergeIntoBlockDeserialize; use common_storages_fuse::operations::merge_into::TransformDistributedMergeIntoBlockSerialize; @@ -297,8 +298,8 @@ impl PipelineBuilder { PhysicalPlan::MergeIntoSource(merge_into_source) => { self.build_merge_into_source(merge_into_source) } - PhysicalPlan::MergeIntoAppendNotMatched(merge_into_row_id_apply) => { - self.build_merge_into_row_id_apply(merge_into_row_id_apply) + PhysicalPlan::MergeIntoAppendNotMatched(merge_into_append_not_matched) => { + self.build_merge_into_append_not_matched(merge_into_append_not_matched) } PhysicalPlan::AddRowNumber(add_row_number) => { self.build_add_row_number(&add_row_number) @@ -346,7 +347,7 @@ impl PipelineBuilder { Ok(()) } - fn build_merge_into_row_id_apply( + fn build_merge_into_append_not_matched( &mut self, merge_into_row_id_apply: &MergeIntoAppendNotMatched, ) -> Result<()> { @@ -355,12 +356,18 @@ impl PipelineBuilder { table_info, catalog_info, } = merge_into_row_id_apply; - // receive rowids and MutationLogs + // receive row numbers and MutationLogs self.build_pipeline(input)?; self.main_pipeline.try_resize(1)?; + // split row_number and log + // output_port_row_number + // output_port_log self.main_pipeline - .add_pipe(TransformDistributedMergeIntoBlockDeserialize::into_pipe()); + .add_pipe(RowNumberAndLogSplitProcessor::create()?.into_pipe()); + + // self.main_pipeline + // .add_pipe(TransformDistributedMergeIntoBlockDeserialize::into_pipe()); // let mut pipe_items = Vec::with_capacity(1); let tbl = self @@ -374,7 +381,7 @@ impl PipelineBuilder { let cluster_stats_gen = table.get_cluster_stats_gen(self.ctx.clone(), 0, block_thresholds, None)?; - // // this TransformSerializeBlock is just used to get block_builder + // this TransformSerializeBlock is just used to get block_builder // let block_builder = TransformSerializeBlock::try_create( // self.ctx.clone(), // InputPort::create(), @@ -928,11 +935,11 @@ impl PipelineBuilder { )); // distributed execution - if *distributed { - self.main_pipeline.try_resize(1)?; - self.main_pipeline - .add_pipe(TransformDistributedMergeIntoBlockSerialize::into_pipe()) - } + // if *distributed { + // self.main_pipeline.try_resize(1)?; + // self.main_pipeline + // .add_pipe(TransformDistributedMergeIntoBlockSerialize::into_pipe()) + // } Ok(()) } diff --git a/src/query/storages/fuse/src/operations/merge_into/mod.rs b/src/query/storages/fuse/src/operations/merge_into/mod.rs index 10b1c3206ec7..85402b3e1cbd 100644 --- a/src/query/storages/fuse/src/operations/merge_into/mod.rs +++ b/src/query/storages/fuse/src/operations/merge_into/mod.rs @@ -19,7 +19,8 @@ pub use mutator::MatchedAggregator; pub use processors::MatchedSplitProcessor; pub use processors::MergeIntoNotMatchedProcessor; pub use processors::MergeIntoSplitProcessor; -pub use processors::MixRowIdKindAndLog; +pub use processors::MixRowNumberKindAndLog; +pub use processors::RowNumberAndLogSplitProcessor; pub use processors::TransformAddRowNumberColumnProcessor; pub use processors::TransformDistributedMergeIntoBlockDeserialize; pub use processors::TransformDistributedMergeIntoBlockSerialize; diff --git a/src/query/storages/fuse/src/operations/merge_into/processors/mod.rs b/src/query/storages/fuse/src/operations/merge_into/processors/mod.rs index cb679611f47f..d6f139ae3c30 100644 --- a/src/query/storages/fuse/src/operations/merge_into/processors/mod.rs +++ b/src/query/storages/fuse/src/operations/merge_into/processors/mod.rs @@ -17,13 +17,15 @@ mod processor_distributed_merge_into_block_serialize; mod processor_merge_into_matched_and_split; mod processor_merge_into_not_matched; mod processor_merge_into_split; +mod processor_merge_into_split_row_number_and_log; mod transform_add_rownumber_column; mod transform_matched_mutation_aggregator; pub use processor_distributed_merge_into_block_deserialize::TransformDistributedMergeIntoBlockDeserialize; pub use processor_distributed_merge_into_block_serialize::TransformDistributedMergeIntoBlockSerialize; pub use processor_merge_into_matched_and_split::MatchedSplitProcessor; -pub use processor_merge_into_matched_and_split::MixRowIdKindAndLog; +pub use processor_merge_into_matched_and_split::MixRowNumberKindAndLog; pub(crate) use processor_merge_into_matched_and_split::RowIdKind; pub use processor_merge_into_not_matched::MergeIntoNotMatchedProcessor; pub use processor_merge_into_split::MergeIntoSplitProcessor; +pub use processor_merge_into_split_row_number_and_log::RowNumberAndLogSplitProcessor; pub use transform_add_rownumber_column::TransformAddRowNumberColumnProcessor; diff --git a/src/query/storages/fuse/src/operations/merge_into/processors/processor_distributed_merge_into_block_deserialize.rs b/src/query/storages/fuse/src/operations/merge_into/processors/processor_distributed_merge_into_block_deserialize.rs index e9d1211c2ca4..d7cd7851ff70 100644 --- a/src/query/storages/fuse/src/operations/merge_into/processors/processor_distributed_merge_into_block_deserialize.rs +++ b/src/query/storages/fuse/src/operations/merge_into/processors/processor_distributed_merge_into_block_deserialize.rs @@ -26,7 +26,7 @@ use common_pipeline_core::processors::processor::ProcessorPtr; use common_pipeline_transforms::processors::transforms::Transform; use common_pipeline_transforms::processors::transforms::Transformer; -use super::processor_merge_into_matched_and_split::MixRowIdKindAndLog; +use super::processor_merge_into_matched_and_split::MixRowNumberKindAndLog; use super::RowIdKind; // It will receive MutationLogs Or RowIds. @@ -66,7 +66,7 @@ impl Transform for TransformDistributedMergeIntoBlockDeserialize { const NAME: &'static str = "TransformDistributedMergeIntoBlockDeserialize"; fn transform(&mut self, data: DataBlock) -> Result { - let mix_kind = MixRowIdKindAndLog::downcast_ref_from(data.get_meta().unwrap()).unwrap(); + let mix_kind = MixRowNumberKindAndLog::downcast_ref_from(data.get_meta().unwrap()).unwrap(); match mix_kind.kind { 0 => Ok(DataBlock::new_with_meta( data.columns().to_vec(), @@ -84,7 +84,7 @@ impl Transform for TransformDistributedMergeIntoBlockDeserialize { data.num_rows(), Some(Box::new(RowIdKind::Delete)), )), - _ => Err(ErrorCode::BadBytes("get error MixRowIdKindAndLog kind")), + _ => Err(ErrorCode::BadBytes("get error MixRowNumberKindAndLog kind")), } } } diff --git a/src/query/storages/fuse/src/operations/merge_into/processors/processor_distributed_merge_into_block_serialize.rs b/src/query/storages/fuse/src/operations/merge_into/processors/processor_distributed_merge_into_block_serialize.rs index da3214a6db10..0a4097bf19b4 100644 --- a/src/query/storages/fuse/src/operations/merge_into/processors/processor_distributed_merge_into_block_serialize.rs +++ b/src/query/storages/fuse/src/operations/merge_into/processors/processor_distributed_merge_into_block_serialize.rs @@ -30,7 +30,7 @@ use common_pipeline_core::processors::processor::ProcessorPtr; use common_pipeline_transforms::processors::transforms::Transform; use common_pipeline_transforms::processors::transforms::Transformer; -use super::processor_merge_into_matched_and_split::MixRowIdKindAndLog; +use super::processor_merge_into_matched_and_split::MixRowNumberKindAndLog; use super::RowIdKind; use crate::operations::common::MutationLogs; @@ -79,7 +79,7 @@ impl Transform for TransformDistributedMergeIntoBlockSerialize { Ok(DataBlock::new_with_meta( vec![entry], 1, - Some(Box::new(MixRowIdKindAndLog { + Some(Box::new(MixRowNumberKindAndLog { log: Some(log), kind: 0, })), @@ -90,7 +90,7 @@ impl Transform for TransformDistributedMergeIntoBlockSerialize { Ok(DataBlock::new_with_meta( data.columns().to_vec(), data.num_rows(), - Some(Box::new(MixRowIdKindAndLog { + Some(Box::new(MixRowNumberKindAndLog { log: None, kind: match row_id_kind { RowIdKind::Update => 1, diff --git a/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_matched_and_split.rs b/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_matched_and_split.rs index 9a9e750ed40e..72d85a089a7e 100644 --- a/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_matched_and_split.rs +++ b/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_matched_and_split.rs @@ -49,7 +49,7 @@ enum MutationKind { } #[derive(serde::Serialize, serde::Deserialize, Clone, Debug, PartialEq)] -pub struct MixRowIdKindAndLog { +pub struct MixRowNumberKindAndLog { pub log: Option, // kind's range is [0,1,2], 0 stands for log // 1 stands for row_id_update, 2 stands for row_id_delete, @@ -57,9 +57,9 @@ pub struct MixRowIdKindAndLog { } #[typetag::serde(name = "mix_row_id_kind_and_log")] -impl BlockMetaInfo for MixRowIdKindAndLog { +impl BlockMetaInfo for MixRowNumberKindAndLog { fn equals(&self, info: &Box) -> bool { - MixRowIdKindAndLog::downcast_ref_from(info).is_some_and(|other| self == other) + MixRowNumberKindAndLog::downcast_ref_from(info).is_some_and(|other| self == other) } fn clone_self(&self) -> Box { diff --git a/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_split_row_number_and_log.rs b/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_split_row_number_and_log.rs new file mode 100644 index 000000000000..7cd88ae1059d --- /dev/null +++ b/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_split_row_number_and_log.rs @@ -0,0 +1,140 @@ +// 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 common_exception::Result; +use common_expression::DataBlock; +use common_pipeline_core::pipe::Pipe; +use common_pipeline_core::pipe::PipeItem; +use common_pipeline_core::processors::port::InputPort; +use common_pipeline_core::processors::port::OutputPort; +use common_pipeline_core::processors::processor::Event; +use common_pipeline_core::processors::processor::ProcessorPtr; +use common_pipeline_core::processors::Processor; + +pub struct RowNumberAndLogSplitProcessor { + input_port: Arc, + output_port_row_number: Arc, + output_port_log: Arc, + + input_data: Option, + output_data_row_number: Option, + output_data_log: Option, +} + +impl RowNumberAndLogSplitProcessor { + pub fn create() -> Result { + Ok(Self { + input_port: InputPort::create(), + output_port_log: OutputPort::create(), + output_port_row_number: OutputPort::create(), + input_data: None, + output_data_log: None, + output_data_row_number: None, + }) + } + + pub fn into_pipe(self) -> Pipe { + let pipe_item = self.into_pipe_item(); + Pipe::create(1, 2, vec![pipe_item]) + } + + pub fn into_pipe_item(self) -> PipeItem { + let input = self.input_port.clone(); + let output_port_row_number = self.output_port_row_number.clone(); + let output_port_log = self.output_port_log.clone(); + let processor_ptr = ProcessorPtr::create(Box::new(self)); + PipeItem::create(processor_ptr, vec![input], vec![ + output_port_row_number, + output_port_log, + ]) + } +} + +impl Processor for RowNumberAndLogSplitProcessor { + fn name(&self) -> String { + "RowNumberAndLogSplit".to_owned() + } + + #[doc = " Reference used for downcast."] + fn as_any(&mut self) -> &mut dyn Any { + self + } + + fn event(&mut self) -> Result { + // 1. if there is no data and input_port is finished, this processor has finished + // it's work + let finished = self.input_port.is_finished() + && self.output_data_row_number.is_none() + && self.output_data_log.is_none(); + if finished { + self.output_port_row_number.finish(); + self.output_port_log.finish(); + return Ok(Event::Finished); + } + + let mut pushed_something = false; + + // 2. process data stage here + if self.output_port_row_number.can_push() { + if let Some(row_number_data) = self.output_data_row_number.take() { + self.output_port_row_number.push_data(Ok(row_number_data)); + pushed_something = true + } + } + + if self.output_port_log.can_push() { + if let Some(log_data) = self.output_data_log.take() { + self.output_port_log.push_data(Ok(log_data)); + pushed_something = true + } + } + + // 3. trigger down stream pipeItem to consume if we pushed data + if pushed_something { + Ok(Event::NeedConsume) + } else { + // 4. we can't pushed data ,so the down stream is not prepared or we have no data at all + // we need to make sure only when the all out_pudt_data are empty ,and we start to split + // datablock held by input_data + if self.input_port.has_data() { + if self.output_data_row_number.is_none() && self.output_data_log.is_none() { + // no pending data (being sent to down streams) + self.input_data = Some(self.input_port.pull_data().unwrap()?); + Ok(Event::Sync) + } else { + // data pending + Ok(Event::NeedConsume) + } + } else { + self.input_port.set_need_data(); + Ok(Event::NeedData) + } + } + } + + fn process(&mut self) -> Result<()> { + if let Some(data_block) = self.input_data.take() { + // mutation logs + if data_block.is_empty() { + self.output_data_log = Some(data_block); + } else { + self.output_data_row_number = Some(data_block) + } + } + Ok(()) + } +} From 4112417491242aa5eaa70a3103a717122315d290 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Thu, 26 Oct 2023 19:57:50 +0800 Subject: [PATCH 24/39] finish distributed codes, need to get data from hashtable --- .../service/src/pipelines/pipeline_builder.rs | 125 +++++++++++++----- .../service/src/pipelines/processors/mod.rs | 1 + .../pipelines/processors/transforms/mod.rs | 2 + .../processor_deduplicate_row_number.rs | 118 +++++++++++++++++ .../src/schedulers/fragments/fragmenter.rs | 1 + .../tests/it/sql/planner/format/mod.rs | 1 + .../sql/src/executor/physical_plan_visitor.rs | 1 + .../physical_plans/physical_hash_join.rs | 4 + src/query/sql/src/planner/binder/join.rs | 1 + .../distributed/distributed_merge.rs | 5 + .../optimizer/heuristic/decorrelate.rs | 6 + .../optimizer/heuristic/subquery_rewriter.rs | 3 + .../planner/optimizer/hyper_dp/join_node.rs | 1 + src/query/sql/src/planner/plans/join.rs | 4 + 14 files changed, 242 insertions(+), 31 deletions(-) create mode 100644 src/query/service/src/pipelines/processors/transforms/processor_deduplicate_row_number.rs diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs index 02e8e780c9b9..9b93dd67f73f 100644 --- a/src/query/service/src/pipelines/pipeline_builder.rs +++ b/src/query/service/src/pipelines/pipeline_builder.rs @@ -129,8 +129,6 @@ use common_storages_fuse::operations::merge_into::MergeIntoNotMatchedProcessor; use common_storages_fuse::operations::merge_into::MergeIntoSplitProcessor; use common_storages_fuse::operations::merge_into::RowNumberAndLogSplitProcessor; use common_storages_fuse::operations::merge_into::TransformAddRowNumberColumnProcessor; -use common_storages_fuse::operations::merge_into::TransformDistributedMergeIntoBlockDeserialize; -use common_storages_fuse::operations::merge_into::TransformDistributedMergeIntoBlockSerialize; use common_storages_fuse::operations::replace_into::BroadcastProcessor; use common_storages_fuse::operations::replace_into::ReplaceIntoProcessor; use common_storages_fuse::operations::replace_into::UnbranchedReplaceIntoProcessor; @@ -179,6 +177,7 @@ use crate::pipelines::processors::transforms::TransformPartialAggregate; use crate::pipelines::processors::transforms::TransformPartialGroupBy; use crate::pipelines::processors::transforms::TransformWindow; use crate::pipelines::processors::AggregatorParams; +use crate::pipelines::processors::DeduplicateRowNumber; use crate::pipelines::processors::HashJoinState; use crate::pipelines::processors::SinkRuntimeFilterSource; use crate::pipelines::processors::TransformCastSchema; @@ -351,6 +350,9 @@ impl PipelineBuilder { &mut self, merge_into_row_id_apply: &MergeIntoAppendNotMatched, ) -> Result<()> { + // self.main_pipeline + // .add_pipe(TransformDistributedMergeIntoBlockDeserialize::into_pipe()); + let MergeIntoAppendNotMatched { input, table_info, @@ -360,52 +362,110 @@ impl PipelineBuilder { self.build_pipeline(input)?; self.main_pipeline.try_resize(1)?; + assert!(self.join_state.is_some()); + let join_state = self.join_state.clone().unwrap(); // split row_number and log // output_port_row_number // output_port_log self.main_pipeline .add_pipe(RowNumberAndLogSplitProcessor::create()?.into_pipe()); - // self.main_pipeline - // .add_pipe(TransformDistributedMergeIntoBlockDeserialize::into_pipe()); + // accumulate source data which is not matched from hashstate + let mut pipe_items = Vec::with_capacity(2); + pipe_items.push(DeduplicateRowNumber::create(join_state.clone())?.into_pipe_item()); + pipe_items.push(create_dummy_item()); + self.main_pipeline.add_pipe(Pipe::create(2, 2, pipe_items)); - // let mut pipe_items = Vec::with_capacity(1); + // split row_number and log + // output_port_not_matched_data + // output_port_log + // start to append data let tbl = self .ctx .build_table_by_table_info(catalog_info, table_info, None)?; + // 1.fill default columns + let table_default_schema = &tbl.schema().remove_computed_fields(); + let mut builder = self.main_pipeline.add_transform_with_specified_len( + |transform_input_port, transform_output_port| { + TransformResortAddOnWithoutSourceSchema::try_create( + self.ctx.clone(), + transform_input_port, + transform_output_port, + Arc::new(DataSchema::from(table_default_schema)), + tbl.clone(), + ) + }, + 1, + )?; + builder.add_items(vec![create_dummy_item()]); + self.main_pipeline.add_pipe(builder.finalize()); - let table = FuseTable::try_from_table(tbl.as_ref())?; + // 2.fill computed columns + // fill computed columns + let table_computed_schema = &tbl.schema().remove_virtual_computed_fields(); + let default_schema: DataSchemaRef = Arc::new(table_default_schema.into()); + let computed_schema: DataSchemaRef = Arc::new(table_computed_schema.into()); + if default_schema != computed_schema { + builder = self.main_pipeline.add_transform_with_specified_len( + |transform_input_port, transform_output_port| { + TransformAddComputedColumns::try_create( + self.ctx.clone(), + transform_input_port, + transform_output_port, + default_schema.clone(), + computed_schema.clone(), + ) + }, + 1, + )?; + builder.add_items(vec![create_dummy_item()]); + self.main_pipeline.add_pipe(builder.finalize()); + } + // 3. cluster sort + let table = FuseTable::try_from_table(tbl.as_ref())?; let block_thresholds = table.get_block_thresholds(); + table.cluster_gen_for_append_with_specified_len( + self.ctx.clone(), + &mut self.main_pipeline, + block_thresholds, + 1, + 1, + )?; + // 4. serialize block let cluster_stats_gen = table.get_cluster_stats_gen(self.ctx.clone(), 0, block_thresholds, None)?; + let serialize_block_transform = TransformSerializeBlock::try_create( + self.ctx.clone(), + InputPort::create(), + OutputPort::create(), + table, + cluster_stats_gen.clone(), + )?; - // this TransformSerializeBlock is just used to get block_builder - // let block_builder = TransformSerializeBlock::try_create( - // self.ctx.clone(), - // InputPort::create(), - // OutputPort::create(), - // table, - // cluster_stats_gen, - // )? - // .get_block_builder(); - - // let max_threads = self.settings.get_max_threads()?; - // let io_request_semaphore = Arc::new(Semaphore::new(max_threads as usize)); - - // pipe_items.push(table.rowid_aggregate_mutator( - // self.ctx.clone(), - // block_builder, - // io_request_semaphore, - // segments.clone(), - // true, - // )?); + let mut pipe_items = Vec::with_capacity(2); + pipe_items.push(serialize_block_transform.into_pipe_item()); + pipe_items.push(create_dummy_item()); + self.main_pipeline.add_pipe(Pipe::create(2, 2, pipe_items)); - // self.main_pipeline - // .add_pipe(Pipe::create(self.main_pipeline.output_len(), 1, pipe_items)); - // Ok(()) - todo!() + // 5. serialize segment + let serialize_segment_transform = TransformSerializeSegment::new( + self.ctx.clone(), + InputPort::create(), + OutputPort::create(), + table, + block_thresholds, + ); + let mut pipe_items = Vec::with_capacity(2); + pipe_items.push(serialize_segment_transform.into_pipe_item()); + pipe_items.push(create_dummy_item()); + self.main_pipeline.add_pipe(Pipe::create(2, 2, pipe_items)); + + // resize to one, because they are all mutation logs now. + self.main_pipeline.try_resize(1)?; + + Ok(()) } fn build_merge_into_source(&mut self, merge_into_source: &MergeIntoSource) -> Result<()> { @@ -1469,9 +1529,12 @@ impl PipelineBuilder { }; if hash_join_plan.contain_runtime_filter { build_res.main_pipeline.duplicate(false)?; - self.join_state = Some(build_state); + self.join_state = Some(build_state.clone()); self.index = Some(self.pipelines.len()); } else { + if hash_join_plan.need_hold_hash_table { + self.join_state = Some(build_state.clone()) + } build_res.main_pipeline.add_sink(create_sink_processor)?; } diff --git a/src/query/service/src/pipelines/processors/mod.rs b/src/query/service/src/pipelines/processors/mod.rs index 30915dd69fc4..fdf924ce53ba 100644 --- a/src/query/service/src/pipelines/processors/mod.rs +++ b/src/query/service/src/pipelines/processors/mod.rs @@ -19,6 +19,7 @@ pub use transforms::hash_join::TransformHashJoinBuild; pub use transforms::hash_join::TransformHashJoinProbe; pub use transforms::AggregatorParams; pub use transforms::BlockCompactor; +pub use transforms::DeduplicateRowNumber; pub use transforms::HashJoinDesc; pub use transforms::HashJoinState; pub use transforms::SinkRuntimeFilterSource; diff --git a/src/query/service/src/pipelines/processors/transforms/mod.rs b/src/query/service/src/pipelines/processors/transforms/mod.rs index 48897393502a..c867166671b4 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 @@ mod aggregator; pub mod group_by; pub(crate) mod hash_join; mod metrics; +mod processor_deduplicate_row_number; pub(crate) mod range_join; mod runtime_filter; mod transform_add_computed_columns; @@ -57,6 +58,7 @@ use common_pipeline_transforms::processors::transforms::transform_sort_partial; pub use hash_join::FixedKeyHashJoinHashTable; pub use hash_join::HashJoinDesc; pub use hash_join::HashJoinState; +pub use processor_deduplicate_row_number::DeduplicateRowNumber; pub use range_join::RangeJoinState; pub use runtime_filter::RuntimeFilterState; pub use transform_add_computed_columns::TransformAddComputedColumns; diff --git a/src/query/service/src/pipelines/processors/transforms/processor_deduplicate_row_number.rs b/src/query/service/src/pipelines/processors/transforms/processor_deduplicate_row_number.rs new file mode 100644 index 000000000000..adda0b7e28a6 --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/processor_deduplicate_row_number.rs @@ -0,0 +1,118 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::HashSet; +use std::sync::Arc; + +use common_exception::Result; +use common_expression::types::NumberType; +use common_expression::DataBlock; +use common_pipeline_core::pipe::PipeItem; +use common_pipeline_core::processors::port::InputPort; +use common_pipeline_core::processors::port::OutputPort; +use common_pipeline_core::processors::processor::ProcessorPtr; +use common_pipeline_transforms::processors::transforms::AsyncAccumulatingTransform; +use common_pipeline_transforms::processors::transforms::AsyncAccumulatingTransformer; + +use super::hash_join::HashJoinBuildState; + +pub struct DeduplicateRowNumber { + hashstate: Arc, + unique_row_number: HashSet, + accepted_data: bool, +} + +#[async_trait::async_trait] +impl AsyncAccumulatingTransform for DeduplicateRowNumber { + const NAME: &'static str = "DeduplicateRowNumber"; + + #[async_backtrace::framed] + async fn transform(&mut self, data: DataBlock) -> Result> { + self.accumulate(data).await?; + // no partial output + Ok(None) + } + + #[async_backtrace::framed] + async fn on_finish(&mut self, _output: bool) -> Result> { + if self.unique_row_number.is_empty() { + Ok(Some(DataBlock::empty())) + } else { + let block = self.apply().await?; + Ok(block) + } + } +} + +impl DeduplicateRowNumber { + #[async_backtrace::framed] + pub async fn accumulate(&mut self, data_block: DataBlock) -> Result<()> { + // warning!!!: if all source data is matched, will + // we receive a empty block as expected? + if !self.accepted_data { + self.unique_row_number = get_row_number(&data_block, 0)?; + self.accepted_data = true; + return Ok(()); + } + let row_number = get_row_number(&data_block, 0)?; + self.unique_row_number = self + .unique_row_number + .intersection(&row_number) + .cloned() + .collect(); + Ok(()) + } + + #[async_backtrace::framed] + pub async fn apply(&mut self) -> Result> { + // todo! get datablock from hashstate. + todo!() + } +} + +fn get_row_number(data_block: &DataBlock, row_number_idx: usize) -> Result> { + let row_number_col = data_block.get_by_offset(row_number_idx); + let value = row_number_col + .value + .try_downcast::>() + .unwrap(); + match value { + common_expression::Value::Scalar(scalar) => { + let mut set = HashSet::new(); + set.insert(scalar); + Ok(set) + } + common_expression::Value::Column(column) => Ok(column.iter().cloned().collect()), + } +} + +impl DeduplicateRowNumber { + pub fn create(hashstate: Arc) -> Result { + Ok(Self { + hashstate, + unique_row_number: HashSet::new(), + accepted_data: false, + }) + } + + pub fn into_pipe_item(self) -> PipeItem { + let input = InputPort::create(); + let output = OutputPort::create(); + let processor_ptr = + AsyncAccumulatingTransformer::create(input.clone(), output.clone(), self); + PipeItem::create(ProcessorPtr::create(processor_ptr), vec![input], vec![ + output, + ]) + } +} diff --git a/src/query/service/src/schedulers/fragments/fragmenter.rs b/src/query/service/src/schedulers/fragments/fragmenter.rs index c1e5e9a09173..c1676b54ae5e 100644 --- a/src/query/service/src/schedulers/fragments/fragmenter.rs +++ b/src/query/service/src/schedulers/fragments/fragmenter.rs @@ -240,6 +240,7 @@ impl PhysicalPlanReplacer for Fragmenter { probe_to_build: plan.probe_to_build.clone(), output_schema: plan.output_schema.clone(), contain_runtime_filter: plan.contain_runtime_filter, + need_hold_hash_table: plan.need_hold_hash_table, stat_info: plan.stat_info.clone(), })) } diff --git a/src/query/service/tests/it/sql/planner/format/mod.rs b/src/query/service/tests/it/sql/planner/format/mod.rs index c88bacae1bd0..87d15c80dc16 100644 --- a/src/query/service/tests/it/sql/planner/format/mod.rs +++ b/src/query/service/tests/it/sql/planner/format/mod.rs @@ -152,6 +152,7 @@ fn test_format() { marker_index: None, from_correlated_subquery: false, contain_runtime_filter: false, + need_hold_hash_table: false, } .into(), ), diff --git a/src/query/sql/src/executor/physical_plan_visitor.rs b/src/query/sql/src/executor/physical_plan_visitor.rs index b07740174599..70cba5c1aa6b 100644 --- a/src/query/sql/src/executor/physical_plan_visitor.rs +++ b/src/query/sql/src/executor/physical_plan_visitor.rs @@ -232,6 +232,7 @@ pub trait PhysicalPlanReplacer { probe_to_build: plan.probe_to_build.clone(), output_schema: plan.output_schema.clone(), contain_runtime_filter: plan.contain_runtime_filter, + need_hold_hash_table: plan.need_hold_hash_table, stat_info: plan.stat_info.clone(), })) } diff --git a/src/query/sql/src/executor/physical_plans/physical_hash_join.rs b/src/query/sql/src/executor/physical_plans/physical_hash_join.rs index c7ecc42497f0..c0658f41ec7f 100644 --- a/src/query/sql/src/executor/physical_plans/physical_hash_join.rs +++ b/src/query/sql/src/executor/physical_plans/physical_hash_join.rs @@ -62,6 +62,9 @@ pub struct HashJoin { pub output_schema: DataSchemaRef, // It means that join has a corresponding runtime filter pub contain_runtime_filter: bool, + // if we execute distributed merge into, we need to hold the + // hash table to get not match data from source. + pub need_hold_hash_table: bool, // Only used for explain pub stat_info: Option, @@ -383,6 +386,7 @@ impl PhysicalPlanBuilder { probe_to_build, output_schema, contain_runtime_filter: join.contain_runtime_filter, + need_hold_hash_table: join.need_hold_hash_table, stat_info: Some(stat_info), })) } diff --git a/src/query/sql/src/planner/binder/join.rs b/src/query/sql/src/planner/binder/join.rs index 80660f6263d2..00d15a670bfb 100644 --- a/src/query/sql/src/planner/binder/join.rs +++ b/src/query/sql/src/planner/binder/join.rs @@ -211,6 +211,7 @@ impl Binder { marker_index: None, from_correlated_subquery: false, contain_runtime_filter: false, + need_hold_hash_table: false, }; Ok(SExpr::create_binary( Arc::new(logical_join.into()), diff --git a/src/query/sql/src/planner/optimizer/distributed/distributed_merge.rs b/src/query/sql/src/planner/optimizer/distributed/distributed_merge.rs index d11e14409b11..6b680904e47c 100644 --- a/src/query/sql/src/planner/optimizer/distributed/distributed_merge.rs +++ b/src/query/sql/src/planner/optimizer/distributed/distributed_merge.rs @@ -19,6 +19,7 @@ use common_exception::Result; use crate::optimizer::SExpr; use crate::plans::Exchange::Broadcast; +use crate::plans::Join; use crate::plans::PatternPlan; use crate::plans::RelOp; use crate::plans::RelOperator; @@ -62,6 +63,10 @@ impl MergeSourceOptimizer { Arc::new(right_exchange_input.clone()), )), ]; + + let mut join: Join = join_s_expr.plan().clone().try_into()?; + join.need_hold_hash_table = true; + let join_s_expr = join_s_expr.replace_plan(Arc::new(RelOperator::Join(join))); Ok(join_s_expr.replace_children(new_join_children)) } } diff --git a/src/query/sql/src/planner/optimizer/heuristic/decorrelate.rs b/src/query/sql/src/planner/optimizer/heuristic/decorrelate.rs index 57700be978cb..d3cbc9b167ec 100644 --- a/src/query/sql/src/planner/optimizer/heuristic/decorrelate.rs +++ b/src/query/sql/src/planner/optimizer/heuristic/decorrelate.rs @@ -236,6 +236,7 @@ impl SubqueryRewriter { marker_index: None, from_correlated_subquery: true, contain_runtime_filter: false, + need_hold_hash_table: false, }; // Rewrite plan to semi-join. @@ -306,6 +307,7 @@ impl SubqueryRewriter { marker_index: None, from_correlated_subquery: true, contain_runtime_filter: false, + need_hold_hash_table: false, }; let s_expr = SExpr::create_binary( Arc::new(join_plan.into()), @@ -349,6 +351,7 @@ impl SubqueryRewriter { marker_index: Some(marker_index), from_correlated_subquery: true, contain_runtime_filter: false, + need_hold_hash_table: false, }; let s_expr = SExpr::create_binary( Arc::new(join_plan.into()), @@ -407,6 +410,7 @@ impl SubqueryRewriter { marker_index: Some(marker_index), from_correlated_subquery: true, contain_runtime_filter: false, + need_hold_hash_table: false, } .into(); Ok(( @@ -502,6 +506,7 @@ impl SubqueryRewriter { marker_index: None, from_correlated_subquery: false, contain_runtime_filter: false, + need_hold_hash_table: false, } .into(); return Ok(SExpr::create_binary( @@ -612,6 +617,7 @@ impl SubqueryRewriter { marker_index: join.marker_index, from_correlated_subquery: false, contain_runtime_filter: false, + need_hold_hash_table: false, } .into(), ), diff --git a/src/query/sql/src/planner/optimizer/heuristic/subquery_rewriter.rs b/src/query/sql/src/planner/optimizer/heuristic/subquery_rewriter.rs index 4dc29456b0fd..8f96b30f2fbb 100644 --- a/src/query/sql/src/planner/optimizer/heuristic/subquery_rewriter.rs +++ b/src/query/sql/src/planner/optimizer/heuristic/subquery_rewriter.rs @@ -459,6 +459,7 @@ impl SubqueryRewriter { marker_index: None, from_correlated_subquery: false, contain_runtime_filter: false, + need_hold_hash_table: false, } .into(); Ok(( @@ -526,6 +527,7 @@ impl SubqueryRewriter { marker_index: Some(marker_index), from_correlated_subquery: false, contain_runtime_filter: false, + need_hold_hash_table: false, } .into(); let s_expr = SExpr::create_binary( @@ -555,6 +557,7 @@ impl SubqueryRewriter { marker_index: None, from_correlated_subquery: false, contain_runtime_filter: false, + need_hold_hash_table: false, } .into(); diff --git a/src/query/sql/src/planner/optimizer/hyper_dp/join_node.rs b/src/query/sql/src/planner/optimizer/hyper_dp/join_node.rs index 8198e5445574..9c33c99b37d5 100644 --- a/src/query/sql/src/planner/optimizer/hyper_dp/join_node.rs +++ b/src/query/sql/src/planner/optimizer/hyper_dp/join_node.rs @@ -87,6 +87,7 @@ impl JoinNode { marker_index: None, from_correlated_subquery: false, contain_runtime_filter: false, + need_hold_hash_table: false, }); let children = self .children diff --git a/src/query/sql/src/planner/plans/join.rs b/src/query/sql/src/planner/plans/join.rs index b637a4196770..588375a54bbe 100644 --- a/src/query/sql/src/planner/plans/join.rs +++ b/src/query/sql/src/planner/plans/join.rs @@ -150,6 +150,9 @@ pub struct Join { pub from_correlated_subquery: bool, // It means that join has a corresponding runtime filter pub contain_runtime_filter: bool, + // if we execute distributed merge into, we need to hold the + // hash table to get not match data from source. + pub need_hold_hash_table: bool, } impl Default for Join { @@ -162,6 +165,7 @@ impl Default for Join { marker_index: Default::default(), from_correlated_subquery: Default::default(), contain_runtime_filter: false, + need_hold_hash_table: false, } } } From 622a13a7e44a3b02ecee24df1e12ebd4021b9f8a Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Fri, 27 Oct 2023 18:10:37 +0800 Subject: [PATCH 25/39] finish not macthed append data --- .../service/src/pipelines/pipeline_builder.rs | 31 +++- .../pipelines/processors/transforms/mod.rs | 2 + .../processor_deduplicate_row_number.rs | 40 ++--- ...cessor_extract_hash_table_by_row_number.rs | 156 ++++++++++++++++++ 4 files changed, 209 insertions(+), 20 deletions(-) create mode 100644 src/query/service/src/pipelines/processors/transforms/processor_extract_hash_table_by_row_number.rs diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs index 9b93dd67f73f..47acd50d1d35 100644 --- a/src/query/service/src/pipelines/pipeline_builder.rs +++ b/src/query/service/src/pipelines/pipeline_builder.rs @@ -37,6 +37,7 @@ use common_expression::with_mappedhash_method; use common_expression::with_number_mapped_type; use common_expression::ColumnBuilder; use common_expression::DataBlock; +use common_expression::DataField; use common_expression::DataSchema; use common_expression::DataSchemaRef; use common_expression::FunctionContext; @@ -162,6 +163,7 @@ use crate::pipelines::processors::transforms::hash_join::TransformHashJoinProbe; use crate::pipelines::processors::transforms::range_join::TransformRangeJoinLeft; use crate::pipelines::processors::transforms::range_join::TransformRangeJoinRight; use crate::pipelines::processors::transforms::AggregateInjector; +use crate::pipelines::processors::transforms::ExtractHashTableByRowNumber; use crate::pipelines::processors::transforms::FinalSingleStateAggregator; use crate::pipelines::processors::transforms::HashJoinDesc; use crate::pipelines::processors::transforms::MaterializedCteSink; @@ -197,6 +199,8 @@ pub struct PipelineBuilder { func_ctx: FunctionContext, settings: Arc, + // probe data_fields for merge into + pub probe_data_fields: Option>, // Used in runtime filter source pub join_state: Option>, // record the index of join build side pipeline in `pipelines` @@ -230,6 +234,7 @@ impl PipelineBuilder { exchange_injector: DefaultExchangeInjector::create(), index: None, cte_state: HashMap::new(), + probe_data_fields: None, } } @@ -363,6 +368,8 @@ impl PipelineBuilder { self.main_pipeline.try_resize(1)?; assert!(self.join_state.is_some()); + assert!(self.probe_data_fields.is_some()); + let join_state = self.join_state.clone().unwrap(); // split row_number and log // output_port_row_number @@ -372,7 +379,18 @@ impl PipelineBuilder { // accumulate source data which is not matched from hashstate let mut pipe_items = Vec::with_capacity(2); - pipe_items.push(DeduplicateRowNumber::create(join_state.clone())?.into_pipe_item()); + pipe_items.push(DeduplicateRowNumber::create()?.into_pipe_item()); + pipe_items.push(create_dummy_item()); + self.main_pipeline.add_pipe(Pipe::create(2, 2, pipe_items)); + + let mut pipe_items = Vec::with_capacity(2); + pipe_items.push( + ExtractHashTableByRowNumber::create( + join_state.clone(), + self.probe_data_fields.clone().unwrap(), + )? + .into_pipe_item(), + ); pipe_items.push(create_dummy_item()); self.main_pipeline.add_pipe(Pipe::create(2, 2, pipe_items)); @@ -1532,6 +1550,7 @@ impl PipelineBuilder { self.join_state = Some(build_state.clone()); self.index = Some(self.pipelines.len()); } else { + // for merge into if hash_join_plan.need_hold_hash_table { self.join_state = Some(build_state.clone()) } @@ -2485,6 +2504,16 @@ impl PipelineBuilder { })?; } + if join.need_hold_hash_table { + let mut projected_probe_fields = vec![]; + for (i, field) in probe_state.probe_schema.fields().iter().enumerate() { + if probe_state.probe_projections.contains(&i) { + projected_probe_fields.push(field.clone()); + } + } + self.probe_data_fields = Some(projected_probe_fields); + } + Ok(()) } diff --git a/src/query/service/src/pipelines/processors/transforms/mod.rs b/src/query/service/src/pipelines/processors/transforms/mod.rs index c867166671b4..ce31f3141394 100644 --- a/src/query/service/src/pipelines/processors/transforms/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/mod.rs @@ -17,6 +17,7 @@ pub mod group_by; pub(crate) mod hash_join; mod metrics; mod processor_deduplicate_row_number; +mod processor_extract_hash_table_by_row_number; pub(crate) mod range_join; mod runtime_filter; mod transform_add_computed_columns; @@ -59,6 +60,7 @@ pub use hash_join::FixedKeyHashJoinHashTable; pub use hash_join::HashJoinDesc; pub use hash_join::HashJoinState; pub use processor_deduplicate_row_number::DeduplicateRowNumber; +pub use processor_extract_hash_table_by_row_number::ExtractHashTableByRowNumber; pub use range_join::RangeJoinState; pub use runtime_filter::RuntimeFilterState; pub use transform_add_computed_columns::TransformAddComputedColumns; diff --git a/src/query/service/src/pipelines/processors/transforms/processor_deduplicate_row_number.rs b/src/query/service/src/pipelines/processors/transforms/processor_deduplicate_row_number.rs index adda0b7e28a6..c2db03fd52a8 100644 --- a/src/query/service/src/pipelines/processors/transforms/processor_deduplicate_row_number.rs +++ b/src/query/service/src/pipelines/processors/transforms/processor_deduplicate_row_number.rs @@ -13,22 +13,21 @@ // limitations under the License. use std::collections::HashSet; -use std::sync::Arc; use common_exception::Result; use common_expression::types::NumberType; +use common_expression::types::UInt64Type; use common_expression::DataBlock; +use common_expression::FromData; use common_pipeline_core::pipe::PipeItem; use common_pipeline_core::processors::port::InputPort; use common_pipeline_core::processors::port::OutputPort; use common_pipeline_core::processors::processor::ProcessorPtr; use common_pipeline_transforms::processors::transforms::AsyncAccumulatingTransform; use common_pipeline_transforms::processors::transforms::AsyncAccumulatingTransformer; - -use super::hash_join::HashJoinBuildState; +use itertools::Itertools; pub struct DeduplicateRowNumber { - hashstate: Arc, unique_row_number: HashSet, accepted_data: bool, } @@ -59,16 +58,22 @@ impl DeduplicateRowNumber { #[async_backtrace::framed] pub async fn accumulate(&mut self, data_block: DataBlock) -> Result<()> { // warning!!!: if all source data is matched, will - // we receive a empty block as expected? + // we receive a empty block as expected? the answer is yes. + // but if there is still also some data unmatched, we won't receive + // an empty block. + let row_number_vec = get_row_number(&data_block, 0)?; + let row_number_set: HashSet = row_number_vec.iter().cloned().collect(); + assert_eq!(row_number_set.len(), row_number_vec.len()); + if !self.accepted_data { - self.unique_row_number = get_row_number(&data_block, 0)?; + self.unique_row_number = row_number_set; self.accepted_data = true; return Ok(()); } - let row_number = get_row_number(&data_block, 0)?; + self.unique_row_number = self .unique_row_number - .intersection(&row_number) + .intersection(&row_number_set) .cloned() .collect(); Ok(()) @@ -76,31 +81,28 @@ impl DeduplicateRowNumber { #[async_backtrace::framed] pub async fn apply(&mut self) -> Result> { - // todo! get datablock from hashstate. - todo!() + let row_number_vecs = self.unique_row_number.clone().into_iter().collect_vec(); + Ok(Some(DataBlock::new_from_columns(vec![ + UInt64Type::from_data(row_number_vecs), + ]))) } } -fn get_row_number(data_block: &DataBlock, row_number_idx: usize) -> Result> { +pub(crate) fn get_row_number(data_block: &DataBlock, row_number_idx: usize) -> Result> { let row_number_col = data_block.get_by_offset(row_number_idx); let value = row_number_col .value .try_downcast::>() .unwrap(); match value { - common_expression::Value::Scalar(scalar) => { - let mut set = HashSet::new(); - set.insert(scalar); - Ok(set) - } - common_expression::Value::Column(column) => Ok(column.iter().cloned().collect()), + common_expression::Value::Scalar(scalar) => Ok(vec![scalar]), + common_expression::Value::Column(column) => Ok(column.into_iter().collect_vec()), } } impl DeduplicateRowNumber { - pub fn create(hashstate: Arc) -> Result { + pub fn create() -> Result { Ok(Self { - hashstate, unique_row_number: HashSet::new(), accepted_data: false, }) diff --git a/src/query/service/src/pipelines/processors/transforms/processor_extract_hash_table_by_row_number.rs b/src/query/service/src/pipelines/processors/transforms/processor_extract_hash_table_by_row_number.rs new file mode 100644 index 000000000000..ad3c1c60f9f4 --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/processor_extract_hash_table_by_row_number.rs @@ -0,0 +1,156 @@ +// 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::HashSet; +use std::sync::Arc; + +use common_arrow::arrow::bitmap::MutableBitmap; +use common_exception::Result; +use common_expression::types::DataType; +use common_expression::types::NumberDataType; +use common_expression::BlockEntry; +use common_expression::DataBlock; +use common_expression::DataField; +use common_expression::Scalar; +use common_expression::Value; +use common_pipeline_core::pipe::PipeItem; +use common_pipeline_core::processors::port::InputPort; +use common_pipeline_core::processors::port::OutputPort; +use common_pipeline_core::processors::processor::Event; +use common_pipeline_core::processors::processor::ProcessorPtr; +use common_pipeline_core::processors::Processor; + +use super::hash_join::HashJoinBuildState; +use super::processor_deduplicate_row_number::get_row_number; + +pub struct ExtractHashTableByRowNumber { + input_port: Arc, + output_port: Arc, + input_data: Option, + output_data: Vec, + probe_data_fields: Vec, + hashstate: Arc, +} + +impl ExtractHashTableByRowNumber { + pub fn create( + hashstate: Arc, + probe_data_fields: Vec, + ) -> Result { + Ok(Self { + input_port: InputPort::create(), + output_port: OutputPort::create(), + hashstate, + probe_data_fields, + input_data: None, + output_data: Vec::new(), + }) + } + + pub fn into_pipe_item(self) -> PipeItem { + let input = self.input_port.clone(); + let output_port = self.output_port.clone(); + let processor_ptr = ProcessorPtr::create(Box::new(self)); + PipeItem::create(processor_ptr, vec![input], vec![output_port]) + } +} + +impl Processor for ExtractHashTableByRowNumber { + fn name(&self) -> String { + "ExtractHashTableByRowNumber".to_owned() + } + + #[doc = " Reference used for downcast."] + fn as_any(&mut self) -> &mut dyn Any { + self + } + + fn event(&mut self) -> Result { + let finished = self.input_port.is_finished() && self.output_data.is_empty(); + if finished { + self.output_port.finish(); + return Ok(Event::Finished); + } + + let mut pushed_something = false; + + if self.output_port.can_push() && !self.output_data.is_empty() { + self.output_port + .push_data(Ok(self.output_data.pop().unwrap())); + pushed_something = true + } + + if pushed_something { + return Ok(Event::NeedConsume); + } + + if self.input_port.has_data() { + if self.output_data.is_empty() { + self.input_data = Some(self.input_port.pull_data().unwrap()?); + Ok(Event::Sync) + } else { + Ok(Event::NeedConsume) + } + } else { + self.input_port.set_need_data(); + Ok(Event::NeedData) + } + } + + fn process(&mut self) -> Result<()> { + if let Some(data_block) = self.input_data.take() { + if data_block.is_empty() { + return Ok(()); + } + + let row_number_vec = get_row_number(&data_block, 0)?; + let row_number_set: HashSet = row_number_vec.iter().cloned().collect(); + assert_eq!(row_number_set.len(), row_number_vec.len()); + + // get datablocks from hashstate. + unsafe { + for block in &*self.hashstate.hash_join_state.chunks.get() { + assert_eq!( + block.columns()[block.num_columns() - 1].data_type, + DataType::Number(NumberDataType::UInt64) + ); + let mut bitmap = MutableBitmap::new(); + let row_numbers = get_row_number(&block, 0)?; + for row_number in row_numbers.iter() { + if row_number_set.contains(row_number) { + bitmap.push(true); + } else { + bitmap.push(false); + } + } + let filtered_block = block.clone().filter_with_bitmap(&bitmap.into())?; + // Create null chunk for unmatched rows in probe side + let mut null_block = DataBlock::new( + self.probe_data_fields + .iter() + .map(|df| { + BlockEntry::new(df.data_type().clone(), Value::Scalar(Scalar::Null)) + }) + .collect(), + filtered_block.num_rows(), + ); + null_block.merge_block(filtered_block); + self.output_data.push(null_block); + } + } + } + Ok(()) + } +} From a15df12b13457717cf40898a402994ea21d7aeeb Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Fri, 27 Oct 2023 18:25:31 +0800 Subject: [PATCH 26/39] fix filter --- src/query/service/src/interpreters/interpreter_merge_into.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/query/service/src/interpreters/interpreter_merge_into.rs b/src/query/service/src/interpreters/interpreter_merge_into.rs index 181bf2d6e573..81e8a36c650a 100644 --- a/src/query/service/src/interpreters/interpreter_merge_into.rs +++ b/src/query/service/src/interpreters/interpreter_merge_into.rs @@ -446,7 +446,7 @@ impl MergeIntoInterpreter { // EvalScalar(source_join_side_expr) // \ // SourcePlan - let source_plan = join.child(0)?; + let source_plan = join.child(1)?; let join_op = match join.plan() { RelOperator::Join(j) => j, _ => unreachable!(), @@ -603,7 +603,7 @@ impl MergeIntoInterpreter { }); let filters = vec![gte_min, lte_max]; - let mut target_plan = join.child(1)?.clone(); + let mut target_plan = join.child(0)?.clone(); Self::push_down_filters(&mut target_plan, &filters)?; let new_sexpr = join.replace_children(vec![Arc::new(source_plan.clone()), Arc::new(target_plan)]); From 672218ea412f8cd71d7ddcd2b976f0ea53c32c42 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Fri, 27 Oct 2023 22:59:06 +0800 Subject: [PATCH 27/39] fix filter --- .../service/src/interpreters/interpreter_merge_into.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/query/service/src/interpreters/interpreter_merge_into.rs b/src/query/service/src/interpreters/interpreter_merge_into.rs index 81e8a36c650a..030477858583 100644 --- a/src/query/service/src/interpreters/interpreter_merge_into.rs +++ b/src/query/service/src/interpreters/interpreter_merge_into.rs @@ -454,8 +454,8 @@ impl MergeIntoInterpreter { if join_op.left_conditions.len() != 1 || join_op.right_conditions.len() != 1 { return Ok(Box::new(join.clone())); } - let source_side_expr = &join_op.left_conditions[0]; - let target_side_expr = &join_op.right_conditions[0]; + let source_side_expr = &join_op.right_conditions[0]; + let target_side_expr = &join_op.left_conditions[0]; // eval source side join expr let source_side_join_expr_index = metadata.write().add_derived_column( @@ -606,7 +606,7 @@ impl MergeIntoInterpreter { let mut target_plan = join.child(0)?.clone(); Self::push_down_filters(&mut target_plan, &filters)?; let new_sexpr = - join.replace_children(vec![Arc::new(source_plan.clone()), Arc::new(target_plan)]); + join.replace_children(vec![Arc::new(target_plan), Arc::new(source_plan.clone())]); Ok(Box::new(new_sexpr)) } From 2ba7b7d46de97f459529746412290134c556606d Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Sat, 28 Oct 2023 23:40:46 +0800 Subject: [PATCH 28/39] fix distributed bugs,many bugs, need to support insert --- .../src/api/rpc/exchange/exchange_manager.rs | 107 +++++++++++++++++- .../interpreters/interpreter_merge_into.rs | 29 +++-- .../src/pipelines/executor/executor_graph.rs | 23 ++++ src/query/service/src/pipelines/mod.rs | 1 + .../src/pipelines/pipeline_build_res.rs | 7 ++ .../service/src/pipelines/pipeline_builder.rs | 51 ++++++++- .../src/schedulers/fragments/fragmenter.rs | 10 ++ .../physical_plans/physical_add_row_number.rs | 2 +- .../physical_plans/physical_hash_join.rs | 13 +++ .../physical_plans/physical_merge_into.rs | 3 + .../distributed/distributed_merge.rs | 33 +++--- .../sql/src/planner/optimizer/optimizer.rs | 1 + .../sql/src/planner/plans/add_row_number.rs | 23 ++-- src/query/sql/src/planner/plans/mod.rs | 1 + .../merge_into/mutator/matched_mutator.rs | 1 + ...sor_merge_into_split_row_number_and_log.rs | 7 ++ .../transform_add_rownumber_column.rs | 4 +- 17 files changed, 269 insertions(+), 47 deletions(-) diff --git a/src/query/service/src/api/rpc/exchange/exchange_manager.rs b/src/query/service/src/api/rpc/exchange/exchange_manager.rs index b693f935ad8c..8da8c4b6a12b 100644 --- a/src/query/service/src/api/rpc/exchange/exchange_manager.rs +++ b/src/query/service/src/api/rpc/exchange/exchange_manager.rs @@ -58,6 +58,7 @@ use crate::pipelines::executor::ExecutorSettings; use crate::pipelines::executor::PipelineCompleteExecutor; use crate::pipelines::PipelineBuildResult; use crate::pipelines::PipelineBuilder; +use crate::pipelines::PipelineBuilderData; use crate::schedulers::QueryFragmentActions; use crate::schedulers::QueryFragmentsActions; use crate::sessions::QueryContext; @@ -210,6 +211,28 @@ impl DataExchangeManager { } } + // Create a pipeline based on query plan + #[minitrace::trace] + pub fn init_query_fragments_plan_local( + &self, + ctx: &Arc, + packet: &QueryFragmentsPlanPacket, + ) -> Result<()> { + let queries_coordinator_guard = self.queries_coordinator.lock(); + let queries_coordinator = unsafe { &mut *queries_coordinator_guard.deref().get() }; + + // TODO: When the query is not executed for a long time after submission, we need to remove it + match queries_coordinator.get_mut(&packet.query_id) { + None => Err(ErrorCode::Internal(format!( + "Query {} not found in cluster.", + packet.query_id + ))), + Some(query_coordinator) => { + query_coordinator.prepare_pipeline_local(ctx, packet.enable_profiling, packet) + } + } + } + #[minitrace::trace] pub fn handle_statistics_exchange( &self, @@ -297,7 +320,7 @@ impl DataExchangeManager { .await?; // Submit tasks to localhost - self.init_query_fragments_plan(&ctx, &local_query_fragments_plan_packet)?; + self.init_query_fragments_plan_local(&ctx, &local_query_fragments_plan_packet)?; // Get local pipeline of local task let build_res = self.get_root_pipeline(ctx, enable_profiling, root_actions)?; @@ -590,6 +613,45 @@ impl QueryCoordinator { Ok(()) } + pub fn prepare_pipeline_local( + &mut self, + ctx: &Arc, + enable_profiling: bool, + packet: &QueryFragmentsPlanPacket, + ) -> Result<()> { + self.info = Some(QueryInfo { + query_ctx: ctx.clone(), + query_id: packet.query_id.clone(), + current_executor: packet.executor.clone(), + query_executor: None, + }); + + for fragment in &packet.fragments { + self.fragments_coordinator.insert( + fragment.fragment_id.to_owned(), + FragmentCoordinator::create(fragment), + ); + } + + let mut input_builder_data = PipelineBuilderData { + input_join_state: None, + input_probe_schema: None, + }; + + for fragment in &packet.fragments { + let fragment_id = fragment.fragment_id; + if let Some(coordinator) = self.fragments_coordinator.get_mut(&fragment_id) { + input_builder_data = coordinator.prepare_pipeline_local( + ctx.clone(), + enable_profiling, + input_builder_data, + )?; + } + } + + Ok(()) + } + pub fn subscribe_fragment( &mut self, ctx: &Arc, @@ -821,6 +883,7 @@ impl FragmentCoordinator { self.initialized = true; let pipeline_ctx = QueryContext::create_from(ctx); + let pipeline_builder = PipelineBuilder::create( pipeline_ctx.get_function_context()?, pipeline_ctx.get_settings(), @@ -828,9 +891,49 @@ impl FragmentCoordinator { enable_profiling, SharedProcessorProfiles::default(), ); - self.pipeline_build_res = Some(pipeline_builder.finalize(&self.physical_plan)?); + let res = pipeline_builder.finalize(&self.physical_plan)?; + + self.pipeline_build_res = Some(res); } Ok(()) } + + pub fn prepare_pipeline_local( + &mut self, + ctx: Arc, + enable_profiling: bool, + input_builder_data: PipelineBuilderData, + ) -> Result { + if !self.initialized { + self.initialized = true; + + let pipeline_ctx = QueryContext::create_from(ctx); + + let mut pipeline_builder = PipelineBuilder::create( + pipeline_ctx.get_function_context()?, + pipeline_ctx.get_settings(), + pipeline_ctx, + enable_profiling, + SharedProcessorProfiles::default(), + ); + + pipeline_builder.join_state = input_builder_data.input_join_state; + pipeline_builder.probe_data_fields = input_builder_data.input_probe_schema; + + let (res, join_state, probe_schema) = + pipeline_builder.finalize_local(&self.physical_plan)?; + + self.pipeline_build_res = Some(res); + return Ok(PipelineBuilderData { + input_join_state: join_state, + input_probe_schema: probe_schema, + }); + } + + Ok(PipelineBuilderData { + input_join_state: None, + input_probe_schema: None, + }) + } } diff --git a/src/query/service/src/interpreters/interpreter_merge_into.rs b/src/query/service/src/interpreters/interpreter_merge_into.rs index 030477858583..893a025641fd 100644 --- a/src/query/service/src/interpreters/interpreter_merge_into.rs +++ b/src/query/service/src/interpreters/interpreter_merge_into.rs @@ -210,11 +210,13 @@ impl MergeIntoInterpreter { if *data_field.name() == row_id_idx.to_string() { row_id_idx = idx; found_row_id = true; - break; } if exchange.is_some() && data_field.name() == ROW_NUMBER_COL_NAME { row_number_idx = idx as i32; + if found_row_id { + break; + } } } @@ -388,11 +390,12 @@ impl MergeIntoInterpreter { output_schema: DataSchemaRef::default(), })) } else { + // let (tx, rx) = mpsc::sync_channel::(0); let merge_append = PhysicalPlan::MergeInto(Box::new(MergeInto { - input: Box::new(merge_into_source), + input: Box::new(merge_into_source.clone()), table_info: table_info.clone(), catalog_info: catalog_.info(), - unmatched, + unmatched: unmatched.clone(), matched, field_index_of_input_schema, row_id_idx, @@ -413,6 +416,8 @@ impl MergeIntoInterpreter { })), table_info: table_info.clone(), catalog_info: catalog_.info(), + unmatched: unmatched.clone(), + input_schema: merge_into_source.output_schema()?, })) }; @@ -446,6 +451,7 @@ impl MergeIntoInterpreter { // EvalScalar(source_join_side_expr) // \ // SourcePlan + let source_plan = join.child(1)?; let join_op = match join.plan() { RelOperator::Join(j) => j, @@ -479,10 +485,19 @@ impl MergeIntoInterpreter { index: source_side_join_expr_index, }], }; - let eval_target_side_condition_sexpr = SExpr::create_unary( - Arc::new(eval_source_side_join_expr_op.into()), - Arc::new(source_plan.clone()), - ); + let eval_target_side_condition_sexpr = if let RelOperator::Exchange(_) = source_plan.plan() + { + // there is another row_number operator here + SExpr::create_unary( + Arc::new(eval_source_side_join_expr_op.into()), + Arc::new(source_plan.child(0)?.child(0)?.clone()), + ) + } else { + SExpr::create_unary( + Arc::new(eval_source_side_join_expr_op.into()), + Arc::new(source_plan.clone()), + ) + }; // eval min/max of source side join expr let min_display_name = format!("min({:?})", source_side_expr); diff --git a/src/query/service/src/pipelines/executor/executor_graph.rs b/src/query/service/src/pipelines/executor/executor_graph.rs index a858d1e3239d..12e2845c6f83 100644 --- a/src/query/service/src/pipelines/executor/executor_graph.rs +++ b/src/query/service/src/pipelines/executor/executor_graph.rs @@ -136,6 +136,29 @@ impl ExecutingGraph { let mut pipes_edges: Vec> = Vec::new(); for pipe in &pipeline.pipes { + println!( + "pipe:\n{:?},input len:{:?}, output_len:{:?}", + pipe, pipe.input_length, pipe.output_length + ); + } + println!("=========================================================================="); + for pipe in &pipeline.pipes { + println!( + "pipe:\n{:?},input len:{:?}, output_len:{:?}", + pipe, pipe.input_length, pipe.output_length + ); + if pipe.input_length != pipes_edges.last().map(|x| x.len()).unwrap_or_default() { + println!( + "|==========================================================================|" + ); + println!( + "pipe:\n{:?},input len:{:?}, output_len:{:?}", + pipe, pipe.input_length, pipe.output_length + ); + println!( + "|==========================================================================|" + ); + } assert_eq!( pipe.input_length, pipes_edges.last().map(|x| x.len()).unwrap_or_default() diff --git a/src/query/service/src/pipelines/mod.rs b/src/query/service/src/pipelines/mod.rs index 266e1c6e482f..732c4a5c54be 100644 --- a/src/query/service/src/pipelines/mod.rs +++ b/src/query/service/src/pipelines/mod.rs @@ -27,5 +27,6 @@ pub use pipe::SourcePipeBuilder; pub use pipe::TransformPipeBuilder; pub use pipeline::Pipeline; pub use pipeline_build_res::PipelineBuildResult; +pub use pipeline_build_res::PipelineBuilderData; pub use pipeline_builder::PipelineBuilder; pub use pipeline_builder::ValueSource; diff --git a/src/query/service/src/pipelines/pipeline_build_res.rs b/src/query/service/src/pipelines/pipeline_build_res.rs index 2acee48a867e..575ae54ba2c3 100644 --- a/src/query/service/src/pipelines/pipeline_build_res.rs +++ b/src/query/service/src/pipelines/pipeline_build_res.rs @@ -16,15 +16,22 @@ use std::sync::Arc; use common_exception::Result; use common_expression::DataBlock; +use common_expression::DataField; use common_pipeline_core::processors::port::OutputPort; use common_pipeline_core::Pipeline; use common_pipeline_core::SourcePipeBuilder; use common_pipeline_sources::OneBlockSource; use common_profile::SharedProcessorProfiles; +use super::processors::transforms::hash_join::HashJoinBuildState; use crate::api::DefaultExchangeInjector; use crate::api::ExchangeInjector; +pub struct PipelineBuilderData { + pub input_join_state: Option>, + pub input_probe_schema: Option>, +} + pub struct PipelineBuildResult { pub main_pipeline: Pipeline, // Containing some sub queries pipelines, must be complete pipeline diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs index 47acd50d1d35..0d9fce7b4f4b 100644 --- a/src/query/service/src/pipelines/pipeline_builder.rs +++ b/src/query/service/src/pipelines/pipeline_builder.rs @@ -257,6 +257,36 @@ impl PipelineBuilder { }) } + pub fn finalize_local( + mut self, + plan: &PhysicalPlan, + ) -> Result<( + PipelineBuildResult, + Option>, + Option>, + )> { + self.build_pipeline(plan)?; + + for source_pipeline in &self.pipelines { + if !source_pipeline.is_complete_pipeline()? { + return Err(ErrorCode::Internal( + "Source pipeline must be complete pipeline.", + )); + } + } + + Ok(( + PipelineBuildResult { + main_pipeline: self.main_pipeline, + sources_pipelines: self.pipelines, + prof_span_set: self.proc_profs, + exchange_injector: self.exchange_injector, + }, + self.join_state, + self.probe_data_fields, + )) + } + fn build_pipeline(&mut self, plan: &PhysicalPlan) -> Result<()> { match plan { PhysicalPlan::TableScan(scan) => self.build_table_scan(scan), @@ -328,6 +358,7 @@ impl PipelineBuilder { fn build_add_row_number(&mut self, add_row_number: &AddRowNumber) -> Result<()> { // it must be distributed merge into execution + self.build_pipeline(&add_row_number.input)?; let node_index = add_row_number .cluster_index .get(&self.ctx.get_cluster().local_id); @@ -353,7 +384,7 @@ impl PipelineBuilder { fn build_merge_into_append_not_matched( &mut self, - merge_into_row_id_apply: &MergeIntoAppendNotMatched, + merge_into_append_not_macted: &MergeIntoAppendNotMatched, ) -> Result<()> { // self.main_pipeline // .add_pipe(TransformDistributedMergeIntoBlockDeserialize::into_pipe()); @@ -362,7 +393,9 @@ impl PipelineBuilder { input, table_info, catalog_info, - } = merge_into_row_id_apply; + unmatched, + input_schema, + } = merge_into_append_not_macted; // receive row numbers and MutationLogs self.build_pipeline(input)?; self.main_pipeline.try_resize(1)?; @@ -394,6 +427,17 @@ impl PipelineBuilder { pipe_items.push(create_dummy_item()); self.main_pipeline.add_pipe(Pipe::create(2, 2, pipe_items)); + // not macthed operation + let merge_into_not_matched_processor = MergeIntoNotMatchedProcessor::create( + unmatched.clone(), + input_schema.clone(), + self.func_ctx.clone(), + )?; + let mut pipe_items = Vec::with_capacity(2); + pipe_items.push(merge_into_not_matched_processor.into_pipe_item()); + pipe_items.push(create_dummy_item()); + self.main_pipeline.add_pipe(Pipe::create(2, 2, pipe_items)); + // split row_number and log // output_port_not_matched_data // output_port_log @@ -711,6 +755,7 @@ impl PipelineBuilder { // receive matched data and not matched data parallelly. let mut pipe_items = Vec::with_capacity(self.main_pipeline.output_len()); for _ in (0..self.main_pipeline.output_len()).step_by(2) { + // Todo(JackTan25): We should optimize pipeline. when only not matched, we should ignore this let matched_split_processor = MatchedSplitProcessor::create( self.ctx.clone(), *row_id_idx, @@ -720,6 +765,7 @@ impl PipelineBuilder { Arc::new(DataSchema::from(tbl.schema())), )?; + // Todo(JackTan25): We should optimize pipeline. when only matched,we should ignore this let merge_into_not_matched_processor = MergeIntoNotMatchedProcessor::create( unmatched.clone(), input.output_schema()?, @@ -815,6 +861,7 @@ impl PipelineBuilder { for idx in 0..(self.main_pipeline.output_len() / 3) { vec.push(idx + self.main_pipeline.output_len() / 3 * 2); } + ranges.push(vec); self.main_pipeline.resize_partial_one(ranges.clone())?; } diff --git a/src/query/service/src/schedulers/fragments/fragmenter.rs b/src/query/service/src/schedulers/fragments/fragmenter.rs index c1676b54ae5e..ee667b4e3bd7 100644 --- a/src/query/service/src/schedulers/fragments/fragmenter.rs +++ b/src/query/service/src/schedulers/fragments/fragmenter.rs @@ -19,6 +19,7 @@ use common_exception::Result; use common_sql::executor::CopyIntoTablePhysicalPlan; use common_sql::executor::CopyIntoTableSource; use common_sql::executor::FragmentKind; +use common_sql::executor::MergeInto; use common_sql::executor::QuerySource; use common_sql::executor::ReplaceInto; @@ -148,6 +149,15 @@ impl PhysicalPlanReplacer for Fragmenter { Ok(PhysicalPlan::TableScan(plan.clone())) } + fn replace_merge_into(&mut self, plan: &MergeInto) -> Result { + let input = self.replace(&plan.input)?; + self.state = State::SelectLeaf; + Ok(PhysicalPlan::MergeInto(Box::new(MergeInto { + input: Box::new(input), + ..plan.clone() + }))) + } + fn replace_replace_into( &mut self, plan: &common_sql::executor::ReplaceInto, diff --git a/src/query/sql/src/executor/physical_plans/physical_add_row_number.rs b/src/query/sql/src/executor/physical_plans/physical_add_row_number.rs index b0cd38d913ff..25f21121379c 100644 --- a/src/query/sql/src/executor/physical_plans/physical_add_row_number.rs +++ b/src/query/sql/src/executor/physical_plans/physical_add_row_number.rs @@ -39,7 +39,7 @@ pub struct AddRowNumber { impl AddRowNumber { pub fn output_schema(&self) -> Result { - todo!() + Ok(self.output_schema.clone()) } } diff --git a/src/query/sql/src/executor/physical_plans/physical_hash_join.rs b/src/query/sql/src/executor/physical_plans/physical_hash_join.rs index 12ea19b4a7b5..dce159b93905 100644 --- a/src/query/sql/src/executor/physical_plans/physical_hash_join.rs +++ b/src/query/sql/src/executor/physical_plans/physical_hash_join.rs @@ -22,6 +22,7 @@ use common_expression::DataField; use common_expression::DataSchemaRef; use common_expression::DataSchemaRefExt; use common_expression::RemoteExpr; +use common_expression::ROW_NUMBER_COL_NAME; use common_functions::BUILTIN_FUNCTIONS; use crate::executor::explain::PlanStatsInfo; @@ -259,6 +260,12 @@ impl PhysicalPlanBuilder { } } + // for distributed merge into, there is a field called "_row_number", but + // it's not an internal row_number, we need to add it here + if let Ok(index) = build_schema.index_of(ROW_NUMBER_COL_NAME) { + build_projections.insert(index); + } + let mut merged_fields = Vec::with_capacity(probe_projections.len() + build_projections.len()); let mut probe_fields = Vec::with_capacity(probe_projections.len()); @@ -350,6 +357,12 @@ impl PhysicalPlanBuilder { } } + // for distributed merge into, there is a field called "_row_number", but + // it's not an internal row_number, we need to add it here + if let Ok(index) = projected_schema.index_of(ROW_NUMBER_COL_NAME) { + projections.insert(index); + } + let mut output_fields = Vec::with_capacity(column_projections.len()); for (i, field) in merged_fields.iter().enumerate() { if projections.contains(&i) { diff --git a/src/query/sql/src/executor/physical_plans/physical_merge_into.rs b/src/query/sql/src/executor/physical_plans/physical_merge_into.rs index 8c6dc864dc2b..8bddc77e8895 100644 --- a/src/query/sql/src/executor/physical_plans/physical_merge_into.rs +++ b/src/query/sql/src/executor/physical_plans/physical_merge_into.rs @@ -55,4 +55,7 @@ pub struct MergeIntoAppendNotMatched { pub input: Box, pub table_info: TableInfo, pub catalog_info: CatalogInfo, + // (DataSchemaRef, Option, Vec,Vec) => (source_schema, condition, value_exprs) + pub unmatched: Vec<(DataSchemaRef, Option, Vec)>, + pub input_schema: DataSchemaRef, } diff --git a/src/query/sql/src/planner/optimizer/distributed/distributed_merge.rs b/src/query/sql/src/planner/optimizer/distributed/distributed_merge.rs index 6b680904e47c..10930c05a17b 100644 --- a/src/query/sql/src/planner/optimizer/distributed/distributed_merge.rs +++ b/src/query/sql/src/planner/optimizer/distributed/distributed_merge.rs @@ -18,6 +18,7 @@ use common_exception::ErrorCode; use common_exception::Result; use crate::optimizer::SExpr; +use crate::plans::AddRowNumber; use crate::plans::Exchange::Broadcast; use crate::plans::Join; use crate::plans::PatternPlan; @@ -54,20 +55,24 @@ impl MergeSourceOptimizer { let right_exchange = join_s_expr.child(1)?; assert!(right_exchange.children.len() == 1); - let right_exchange_input = join_s_expr.child(0)?; + let right_exchange_input = right_exchange.child(0)?; let new_join_children = vec![ Arc::new(left_exchange_input.clone()), Arc::new(SExpr::create_unary( Arc::new(RelOperator::Exchange(Broadcast)), - Arc::new(right_exchange_input.clone()), + Arc::new(SExpr::create_unary( + Arc::new(RelOperator::AddRowNumber(AddRowNumber)), + Arc::new(right_exchange_input.clone()), + )), )), ]; let mut join: Join = join_s_expr.plan().clone().try_into()?; join.need_hold_hash_table = true; - let join_s_expr = join_s_expr.replace_plan(Arc::new(RelOperator::Join(join))); - Ok(join_s_expr.replace_children(new_join_children)) + let mut join_s_expr = join_s_expr.replace_plan(Arc::new(RelOperator::Join(join))); + join_s_expr = join_s_expr.replace_children(new_join_children); + Ok(s_expr.replace_children(vec![Arc::new(join_s_expr)])) } } @@ -125,20 +130,12 @@ impl MergeSourceOptimizer { } .into(), ), - Arc::new(SExpr::create_unary( - Arc::new( - PatternPlan { - plan_type: RelOp::AddRowNumber, - } - .into(), - ), - Arc::new(SExpr::create_leaf(Arc::new( - PatternPlan { - plan_type: RelOp::Pattern, - } - .into(), - ))), - )), + Arc::new(SExpr::create_leaf(Arc::new( + PatternPlan { + plan_type: RelOp::Pattern, + } + .into(), + ))), )), )), ) diff --git a/src/query/sql/src/planner/optimizer/optimizer.rs b/src/query/sql/src/planner/optimizer/optimizer.rs index 65aa3942eb4d..74bdc90fdaea 100644 --- a/src/query/sql/src/planner/optimizer/optimizer.rs +++ b/src/query/sql/src/planner/optimizer/optimizer.rs @@ -135,6 +135,7 @@ pub fn optimize( // left join and right join. // input is a Join_SExpr let merge_into_join_sexpr = optimize_distributed_query(ctx.clone(), &plan.input)?; + let merge_source_optimizer = MergeSourceOptimizer::create(); let optimized_distributed_merge_into_join_sexpr = merge_source_optimizer.optimize(&merge_into_join_sexpr)?; diff --git a/src/query/sql/src/planner/plans/add_row_number.rs b/src/query/sql/src/planner/plans/add_row_number.rs index d9a7715b5aea..21d0851365ef 100644 --- a/src/query/sql/src/planner/plans/add_row_number.rs +++ b/src/query/sql/src/planner/plans/add_row_number.rs @@ -15,7 +15,6 @@ use std::sync::Arc; use common_catalog::table_context::TableContext; -use common_exception::ErrorCode; use super::Operator; use super::RelOp; @@ -35,26 +34,20 @@ impl Operator for AddRowNumber { fn derive_relational_prop( &self, - _rel_expr: &RelExpr, + rel_expr: &RelExpr, ) -> common_exception::Result> { - Err(ErrorCode::Internal( - "Cannot derive relational property for add_row_number plan", - )) + rel_expr.derive_relational_prop_child(0) } fn derive_physical_prop( &self, - _rel_expr: &RelExpr, + rel_expr: &RelExpr, ) -> common_exception::Result { - Err(ErrorCode::Internal( - "Cannot derive physical property for add_row_number plan", - )) + rel_expr.derive_physical_prop_child(0) } - fn derive_cardinality(&self, _rel_expr: &RelExpr) -> common_exception::Result> { - Err(ErrorCode::Internal( - "Cannot derive cardinality for add_row_number plan", - )) + fn derive_cardinality(&self, rel_expr: &RelExpr) -> common_exception::Result> { + rel_expr.derive_cardinality_child(0) } fn compute_required_prop_child( @@ -62,8 +55,8 @@ impl Operator for AddRowNumber { _ctx: Arc, _rel_expr: &RelExpr, _child_index: usize, - _required: &RequiredProperty, + required: &RequiredProperty, ) -> common_exception::Result { - unreachable!() + Ok(required.clone()) } } diff --git a/src/query/sql/src/planner/plans/mod.rs b/src/query/sql/src/planner/plans/mod.rs index a7788918456c..1620d6056ce9 100644 --- a/src/query/sql/src/planner/plans/mod.rs +++ b/src/query/sql/src/planner/plans/mod.rs @@ -52,6 +52,7 @@ mod union_all; mod update; mod window; +pub use add_row_number::AddRowNumber; pub use aggregate::*; pub use call::CallPlan; pub use constant_table_scan::ConstantTableScan; diff --git a/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs b/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs index 9646859d3f4c..ae95e71f4575 100644 --- a/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs +++ b/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs @@ -136,6 +136,7 @@ impl MatchedAggregator { // data_block is from matched_split, so there is only one column. // that's row_id let row_ids = get_row_id(&data_block, 0)?; + println!("row_ids: {:?}", row_ids); let row_id_kind = RowIdKind::downcast_ref_from(data_block.get_meta().unwrap()).unwrap(); match row_id_kind { RowIdKind::Update => { diff --git a/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_split_row_number_and_log.rs b/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_split_row_number_and_log.rs index 7cd88ae1059d..6911b1eba591 100644 --- a/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_split_row_number_and_log.rs +++ b/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_split_row_number_and_log.rs @@ -16,6 +16,7 @@ use std::any::Any; use std::sync::Arc; use common_exception::Result; +use common_expression::BlockMetaInfoDowncast; use common_expression::DataBlock; use common_pipeline_core::pipe::Pipe; use common_pipeline_core::pipe::PipeItem; @@ -25,6 +26,8 @@ use common_pipeline_core::processors::processor::Event; use common_pipeline_core::processors::processor::ProcessorPtr; use common_pipeline_core::processors::Processor; +use crate::operations::common::MutationLogs; + pub struct RowNumberAndLogSplitProcessor { input_port: Arc, output_port_row_number: Arc, @@ -130,6 +133,10 @@ impl Processor for RowNumberAndLogSplitProcessor { if let Some(data_block) = self.input_data.take() { // mutation logs if data_block.is_empty() { + let mix_kind = + MutationLogs::downcast_ref_from(data_block.get_meta().unwrap()).unwrap(); + + println!("logs:\n{:?}", mix_kind); self.output_data_log = Some(data_block); } else { self.output_data_row_number = Some(data_block) diff --git a/src/query/storages/fuse/src/operations/merge_into/processors/transform_add_rownumber_column.rs b/src/query/storages/fuse/src/operations/merge_into/processors/transform_add_rownumber_column.rs index 5ce8d9f781f6..92aba214f510 100644 --- a/src/query/storages/fuse/src/operations/merge_into/processors/transform_add_rownumber_column.rs +++ b/src/query/storages/fuse/src/operations/merge_into/processors/transform_add_rownumber_column.rs @@ -71,9 +71,9 @@ impl Transform for TransformAddRowNumberColumnProcessor { const NAME: &'static str = "TransformAddRowNumberColumnProcessor"; fn transform(&mut self, data: DataBlock) -> Result { let num_rows = data.num_rows() as u64; - let last_row_number = self.generate_row_number(num_rows); + let row_number = self.generate_row_number(num_rows); let mut row_ids = Vec::with_capacity(data.num_rows()); - for number in (last_row_number - num_rows + 1)..=last_row_number { + for number in row_number..row_number + num_rows { row_ids.push(number); } let mut data_block = data; From 4e168dbfb8ec0bcc8b5721c223da0bd621d31364 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Sun, 29 Oct 2023 14:28:36 +0800 Subject: [PATCH 29/39] fix bugs --- .../interpreters/interpreter_merge_into.rs | 6 +- .../src/pipelines/executor/executor_graph.rs | 23 ------- .../service/src/pipelines/pipeline_builder.rs | 61 +++++++++++-------- .../processor_deduplicate_row_number.rs | 6 +- ...cessor_extract_hash_table_by_row_number.rs | 2 +- .../merge_into/mutator/matched_mutator.rs | 1 - ...sor_merge_into_split_row_number_and_log.rs | 2 - .../transform_add_rownumber_column.rs | 8 +-- .../base/09_fuse_engine/09_0026_merge_into | 7 +++ 9 files changed, 51 insertions(+), 65 deletions(-) diff --git a/src/query/service/src/interpreters/interpreter_merge_into.rs b/src/query/service/src/interpreters/interpreter_merge_into.rs index 893a025641fd..5db89a445a7e 100644 --- a/src/query/service/src/interpreters/interpreter_merge_into.rs +++ b/src/query/service/src/interpreters/interpreter_merge_into.rs @@ -210,13 +210,13 @@ impl MergeIntoInterpreter { if *data_field.name() == row_id_idx.to_string() { row_id_idx = idx; found_row_id = true; + break; } + } + for (idx, data_field) in join_output_schema.fields().iter().enumerate() { if exchange.is_some() && data_field.name() == ROW_NUMBER_COL_NAME { row_number_idx = idx as i32; - if found_row_id { - break; - } } } diff --git a/src/query/service/src/pipelines/executor/executor_graph.rs b/src/query/service/src/pipelines/executor/executor_graph.rs index 12e2845c6f83..a858d1e3239d 100644 --- a/src/query/service/src/pipelines/executor/executor_graph.rs +++ b/src/query/service/src/pipelines/executor/executor_graph.rs @@ -136,29 +136,6 @@ impl ExecutingGraph { let mut pipes_edges: Vec> = Vec::new(); for pipe in &pipeline.pipes { - println!( - "pipe:\n{:?},input len:{:?}, output_len:{:?}", - pipe, pipe.input_length, pipe.output_length - ); - } - println!("=========================================================================="); - for pipe in &pipeline.pipes { - println!( - "pipe:\n{:?},input len:{:?}, output_len:{:?}", - pipe, pipe.input_length, pipe.output_length - ); - if pipe.input_length != pipes_edges.last().map(|x| x.len()).unwrap_or_default() { - println!( - "|==========================================================================|" - ); - println!( - "pipe:\n{:?},input len:{:?}, output_len:{:?}", - pipe, pipe.input_length, pipe.output_length - ); - println!( - "|==========================================================================|" - ); - } assert_eq!( pipe.input_length, pipes_edges.last().map(|x| x.len()).unwrap_or_default() diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs index 0d9fce7b4f4b..6fe4baa2c534 100644 --- a/src/query/service/src/pipelines/pipeline_builder.rs +++ b/src/query/service/src/pipelines/pipeline_builder.rs @@ -765,15 +765,40 @@ impl PipelineBuilder { Arc::new(DataSchema::from(tbl.schema())), )?; - // Todo(JackTan25): We should optimize pipeline. when only matched,we should ignore this - let merge_into_not_matched_processor = MergeIntoNotMatchedProcessor::create( - unmatched.clone(), - input.output_schema()?, - self.func_ctx.clone(), - )?; - pipe_items.push(matched_split_processor.into_pipe_item()); - pipe_items.push(merge_into_not_matched_processor.into_pipe_item()); + if !*distributed { + // Todo(JackTan25): We should optimize pipeline. when only matched,we should ignore this + let merge_into_not_matched_processor = MergeIntoNotMatchedProcessor::create( + unmatched.clone(), + input.output_schema()?, + self.func_ctx.clone(), + )?; + + pipe_items.push(merge_into_not_matched_processor.into_pipe_item()); + } else { + let input_num_columns = input.output_schema()?.num_fields(); + assert_eq!( + input.output_schema()?.field(input_num_columns - 1).name(), + ROW_NUMBER_COL_NAME + ); + let input_port = InputPort::create(); + let output_port = OutputPort::create(); + // project row number column + let proc = ProcessorPtr::create(CompoundBlockOperator::create( + input_port.clone(), + output_port.clone(), + input_num_columns, + self.func_ctx.clone(), + vec![BlockOperator::Project { + projection: vec![input_num_columns - 1], + }], + )); + pipe_items.push(PipeItem { + processor: proc, + inputs_port: vec![input_port], + outputs_port: vec![output_port], + }) + }; } self.main_pipeline.add_pipe(Pipe::create( self.main_pipeline.output_len(), @@ -891,25 +916,9 @@ impl PipelineBuilder { builder.add_items_prepend(vec![create_dummy_item()]); self.main_pipeline.add_pipe(builder.finalize()); } else { - let input_num_columns = input.output_schema()?.num_fields(); - assert_eq!( - input.output_schema()?.field(input_num_columns - 1).name(), - ROW_NUMBER_COL_NAME - ); - let input_port = InputPort::create(); - let output_port = OutputPort::create(); - // project row number column - let proc = ProcessorPtr::create(CompoundBlockOperator::create( - input_port.clone(), - output_port.clone(), - input_num_columns, - self.func_ctx.clone(), - vec![BlockOperator::Project { - projection: vec![input_num_columns - 1], - }], - )); builder.add_items_prepend(vec![create_dummy_item()]); - builder.add_transform(input_port, output_port, proc); + // receive row_number + builder.add_items(vec![create_dummy_item()]); self.main_pipeline.add_pipe(builder.finalize()); } diff --git a/src/query/service/src/pipelines/processors/transforms/processor_deduplicate_row_number.rs b/src/query/service/src/pipelines/processors/transforms/processor_deduplicate_row_number.rs index c2db03fd52a8..7f74d73455d5 100644 --- a/src/query/service/src/pipelines/processors/transforms/processor_deduplicate_row_number.rs +++ b/src/query/service/src/pipelines/processors/transforms/processor_deduplicate_row_number.rs @@ -15,7 +15,6 @@ use std::collections::HashSet; use common_exception::Result; -use common_expression::types::NumberType; use common_expression::types::UInt64Type; use common_expression::DataBlock; use common_expression::FromData; @@ -90,10 +89,7 @@ impl DeduplicateRowNumber { pub(crate) fn get_row_number(data_block: &DataBlock, row_number_idx: usize) -> Result> { let row_number_col = data_block.get_by_offset(row_number_idx); - let value = row_number_col - .value - .try_downcast::>() - .unwrap(); + let value = row_number_col.value.try_downcast::().unwrap(); match value { common_expression::Value::Scalar(scalar) => Ok(vec![scalar]), common_expression::Value::Column(column) => Ok(column.into_iter().collect_vec()), diff --git a/src/query/service/src/pipelines/processors/transforms/processor_extract_hash_table_by_row_number.rs b/src/query/service/src/pipelines/processors/transforms/processor_extract_hash_table_by_row_number.rs index ad3c1c60f9f4..386cc7439d1b 100644 --- a/src/query/service/src/pipelines/processors/transforms/processor_extract_hash_table_by_row_number.rs +++ b/src/query/service/src/pipelines/processors/transforms/processor_extract_hash_table_by_row_number.rs @@ -127,7 +127,7 @@ impl Processor for ExtractHashTableByRowNumber { DataType::Number(NumberDataType::UInt64) ); let mut bitmap = MutableBitmap::new(); - let row_numbers = get_row_number(&block, 0)?; + let row_numbers = get_row_number(&block, block.num_columns() - 1)?; for row_number in row_numbers.iter() { if row_number_set.contains(row_number) { bitmap.push(true); diff --git a/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs b/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs index ae95e71f4575..9646859d3f4c 100644 --- a/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs +++ b/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs @@ -136,7 +136,6 @@ impl MatchedAggregator { // data_block is from matched_split, so there is only one column. // that's row_id let row_ids = get_row_id(&data_block, 0)?; - println!("row_ids: {:?}", row_ids); let row_id_kind = RowIdKind::downcast_ref_from(data_block.get_meta().unwrap()).unwrap(); match row_id_kind { RowIdKind::Update => { diff --git a/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_split_row_number_and_log.rs b/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_split_row_number_and_log.rs index 6911b1eba591..b8dd3ba549de 100644 --- a/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_split_row_number_and_log.rs +++ b/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_split_row_number_and_log.rs @@ -135,8 +135,6 @@ impl Processor for RowNumberAndLogSplitProcessor { if data_block.is_empty() { let mix_kind = MutationLogs::downcast_ref_from(data_block.get_meta().unwrap()).unwrap(); - - println!("logs:\n{:?}", mix_kind); self.output_data_log = Some(data_block); } else { self.output_data_row_number = Some(data_block) diff --git a/src/query/storages/fuse/src/operations/merge_into/processors/transform_add_rownumber_column.rs b/src/query/storages/fuse/src/operations/merge_into/processors/transform_add_rownumber_column.rs index 92aba214f510..cf3202148833 100644 --- a/src/query/storages/fuse/src/operations/merge_into/processors/transform_add_rownumber_column.rs +++ b/src/query/storages/fuse/src/operations/merge_into/processors/transform_add_rownumber_column.rs @@ -62,7 +62,7 @@ impl TransformAddRowNumberColumnProcessor { let row_number = self.row_number.fetch_add(num_rows, Ordering::SeqCst); let mut prefix_u64 = self.prefix as u64; prefix_u64 = prefix_u64 << PREFIX_OFFSET; - prefix_u64 & row_number + prefix_u64 | row_number } } @@ -72,14 +72,14 @@ impl Transform for TransformAddRowNumberColumnProcessor { fn transform(&mut self, data: DataBlock) -> Result { let num_rows = data.num_rows() as u64; let row_number = self.generate_row_number(num_rows); - let mut row_ids = Vec::with_capacity(data.num_rows()); + let mut row_numbers = Vec::with_capacity(data.num_rows()); for number in row_number..row_number + num_rows { - row_ids.push(number); + row_numbers.push(number); } let mut data_block = data; let row_number_entry = BlockEntry::new( DataType::Number(NumberDataType::UInt64), - Value::Column(UInt64Type::from_data(row_ids)), + Value::Column(UInt64Type::from_data(row_numbers)), ); data_block.add_column(row_number_entry); Ok(data_block) diff --git a/tests/sqllogictests/suites/base/09_fuse_engine/09_0026_merge_into b/tests/sqllogictests/suites/base/09_fuse_engine/09_0026_merge_into index 1df4e928ef76..69cd69cca08d 100644 --- a/tests/sqllogictests/suites/base/09_fuse_engine/09_0026_merge_into +++ b/tests/sqllogictests/suites/base/09_fuse_engine/09_0026_merge_into @@ -1,6 +1,9 @@ statement ok set enable_experimental_merge_into = 1; +statement ok +set enable_distributed_merge_into = 1; + statement ok drop table if exists t1; @@ -619,5 +622,9 @@ select a, b, c from tt2; 0 20 {'k2':'v2'} 1 30 {'k1':'v1'} + +statement ok +set enable_distributed_merge_into = 0; + statement ok set enable_experimental_merge_into = 0; From 73e621049fc7f7e5ce6c5d463f5d5afef0d34dc5 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Sun, 29 Oct 2023 17:51:14 +0800 Subject: [PATCH 30/39] fix check and clean codes --- src/binaries/query/entry.rs | 6 +- .../interpreters/interpreter_merge_into.rs | 2 +- .../service/src/pipelines/pipeline_builder.rs | 82 +++++++++---------- .../processor_deduplicate_row_number.rs | 6 ++ ...cessor_extract_hash_table_by_row_number.rs | 2 +- .../sql/src/executor/physical_plan_visitor.rs | 2 +- .../distributed/distributed_merge.rs | 6 +- ...istributed_merge_into_block_deserialize.rs | 1 + ..._distributed_merge_into_block_serialize.rs | 1 + .../processor_merge_into_matched_and_split.rs | 14 ++++ .../processors/processor_merge_into_split.rs | 14 ++++ ...sor_merge_into_split_row_number_and_log.rs | 15 ++-- .../transform_add_rownumber_column.rs | 2 +- .../09_0028_distributed_merge_into | 66 ++++++++++++++- 14 files changed, 157 insertions(+), 62 deletions(-) diff --git a/src/binaries/query/entry.rs b/src/binaries/query/entry.rs index 2d664f732122..e715a4ac1f92 100644 --- a/src/binaries/query/entry.rs +++ b/src/binaries/query/entry.rs @@ -83,9 +83,9 @@ pub async fn init_services(conf: &InnerConfig) -> Result<()> { async fn precheck_services(conf: &InnerConfig) -> Result<()> { if conf.query.max_memory_limit_enabled { - // let size = conf.query.max_server_memory_usage as i64; - // info!("Set memory limit: {}", size); - // GLOBAL_MEM_STAT.set_limit(size); + let size = conf.query.max_server_memory_usage as i64; + info!("Set memory limit: {}", size); + GLOBAL_MEM_STAT.set_limit(size); } let tenant = conf.query.tenant_id.clone(); diff --git a/src/query/service/src/interpreters/interpreter_merge_into.rs b/src/query/service/src/interpreters/interpreter_merge_into.rs index 5db89a445a7e..73f7466405d5 100644 --- a/src/query/service/src/interpreters/interpreter_merge_into.rs +++ b/src/query/service/src/interpreters/interpreter_merge_into.rs @@ -385,7 +385,7 @@ impl MergeIntoInterpreter { matched, field_index_of_input_schema, row_id_idx, - segments: segments.clone(), + segments, distributed: false, output_schema: DataSchemaRef::default(), })) diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs index 6fe4baa2c534..af954f08a3fe 100644 --- a/src/query/service/src/pipelines/pipeline_builder.rs +++ b/src/query/service/src/pipelines/pipeline_builder.rs @@ -148,6 +148,7 @@ use super::processors::transforms::TransformAddComputedColumns; use super::processors::transforms::WindowFunctionInfo; use super::processors::TransformExpandGroupingSets; use super::processors::TransformResortAddOnWithoutSourceSchema; +use super::PipelineBuilderData; use crate::api::DefaultExchangeInjector; use crate::api::ExchangeInjector; use crate::pipelines::builders::build_append_data_pipeline; @@ -260,11 +261,7 @@ impl PipelineBuilder { pub fn finalize_local( mut self, plan: &PhysicalPlan, - ) -> Result<( - PipelineBuildResult, - Option>, - Option>, - )> { + ) -> Result<(PipelineBuildResult, PipelineBuilderData)> { self.build_pipeline(plan)?; for source_pipeline in &self.pipelines { @@ -282,8 +279,10 @@ impl PipelineBuilder { prof_span_set: self.proc_profs, exchange_injector: self.exchange_injector, }, - self.join_state, - self.probe_data_fields, + PipelineBuilderData { + input_join_state: self.join_state, + input_probe_schema: self.probe_data_fields, + }, )) } @@ -335,9 +334,7 @@ impl PipelineBuilder { PhysicalPlan::MergeIntoAppendNotMatched(merge_into_append_not_matched) => { self.build_merge_into_append_not_matched(merge_into_append_not_matched) } - PhysicalPlan::AddRowNumber(add_row_number) => { - self.build_add_row_number(&add_row_number) - } + PhysicalPlan::AddRowNumber(add_row_number) => self.build_add_row_number(add_row_number), PhysicalPlan::ReclusterSource(recluster_source) => { self.build_recluster_source(recluster_source) } @@ -386,9 +383,6 @@ impl PipelineBuilder { &mut self, merge_into_append_not_macted: &MergeIntoAppendNotMatched, ) -> Result<()> { - // self.main_pipeline - // .add_pipe(TransformDistributedMergeIntoBlockDeserialize::into_pipe()); - let MergeIntoAppendNotMatched { input, table_info, @@ -411,20 +405,20 @@ impl PipelineBuilder { .add_pipe(RowNumberAndLogSplitProcessor::create()?.into_pipe()); // accumulate source data which is not matched from hashstate - let mut pipe_items = Vec::with_capacity(2); - pipe_items.push(DeduplicateRowNumber::create()?.into_pipe_item()); - pipe_items.push(create_dummy_item()); + let pipe_items = vec![ + DeduplicateRowNumber::create()?.into_pipe_item(), + create_dummy_item(), + ]; self.main_pipeline.add_pipe(Pipe::create(2, 2, pipe_items)); - let mut pipe_items = Vec::with_capacity(2); - pipe_items.push( + let pipe_items = vec![ ExtractHashTableByRowNumber::create( - join_state.clone(), + join_state, self.probe_data_fields.clone().unwrap(), )? .into_pipe_item(), - ); - pipe_items.push(create_dummy_item()); + create_dummy_item(), + ]; self.main_pipeline.add_pipe(Pipe::create(2, 2, pipe_items)); // not macthed operation @@ -433,9 +427,10 @@ impl PipelineBuilder { input_schema.clone(), self.func_ctx.clone(), )?; - let mut pipe_items = Vec::with_capacity(2); - pipe_items.push(merge_into_not_matched_processor.into_pipe_item()); - pipe_items.push(create_dummy_item()); + let pipe_items = vec![ + merge_into_not_matched_processor.into_pipe_item(), + create_dummy_item(), + ]; self.main_pipeline.add_pipe(Pipe::create(2, 2, pipe_items)); // split row_number and log @@ -503,12 +498,13 @@ impl PipelineBuilder { InputPort::create(), OutputPort::create(), table, - cluster_stats_gen.clone(), + cluster_stats_gen, )?; - let mut pipe_items = Vec::with_capacity(2); - pipe_items.push(serialize_block_transform.into_pipe_item()); - pipe_items.push(create_dummy_item()); + let pipe_items = vec![ + serialize_block_transform.into_pipe_item(), + create_dummy_item(), + ]; self.main_pipeline.add_pipe(Pipe::create(2, 2, pipe_items)); // 5. serialize segment @@ -519,9 +515,10 @@ impl PipelineBuilder { table, block_thresholds, ); - let mut pipe_items = Vec::with_capacity(2); - pipe_items.push(serialize_segment_transform.into_pipe_item()); - pipe_items.push(create_dummy_item()); + let pipe_items = vec![ + serialize_segment_transform.into_pipe_item(), + create_dummy_item(), + ]; self.main_pipeline.add_pipe(Pipe::create(2, 2, pipe_items)); // resize to one, because they are all mutation logs now. @@ -855,12 +852,12 @@ impl PipelineBuilder { // ---------------------------------------------------------------------- // row_id port0_1 row_id port0_1 row_id port // matched data port0_2 row_id port1_1 matched data port0_2 - // unmatched data port0_3 matched data port0_2 matched data port1_2 + // row_number port0_3 matched data port0_2 matched data port1_2 // row_id port1_1 matched data port1_2 ...... // matched data port1_2 ===> ..... ====> ...... - // unmatched data port1_2 ..... ...... - // unmatched data port0_3 unmatched data port0_3 unmatched data port - // ...... unmatched data port1_3 + // row_number port1_2 ..... ...... + // row_number port0_3 row_number port0_3 row_number port + // ...... row_number port1_3 // ...... ..... // ---------------------------------------------------------------------- // do shuffle @@ -894,7 +891,7 @@ impl PipelineBuilder { // remove first row_id port self.main_pipeline.output_len() - 1 } else { - // remove first row_id port and last unmatched_port + // remove first row_id port and last row_number_port self.main_pipeline.output_len() - 2 }; // fill default columns @@ -1068,12 +1065,13 @@ impl PipelineBuilder { pipe_items, )); - // distributed execution - // if *distributed { - // self.main_pipeline.try_resize(1)?; - // self.main_pipeline - // .add_pipe(TransformDistributedMergeIntoBlockSerialize::into_pipe()) - // } + for pipe in &self.main_pipeline.pipes { + println!( + "pipe:\n{:?},input_len:{:?},output_len:{:?}", + pipe, pipe.input_length, pipe.output_length + ); + } + Ok(()) } diff --git a/src/query/service/src/pipelines/processors/transforms/processor_deduplicate_row_number.rs b/src/query/service/src/pipelines/processors/transforms/processor_deduplicate_row_number.rs index 7f74d73455d5..5fbd7ac15ca3 100644 --- a/src/query/service/src/pipelines/processors/transforms/processor_deduplicate_row_number.rs +++ b/src/query/service/src/pipelines/processors/transforms/processor_deduplicate_row_number.rs @@ -60,6 +60,12 @@ impl DeduplicateRowNumber { // we receive a empty block as expected? the answer is yes. // but if there is still also some data unmatched, we won't receive // an empty block. + if data_block.is_empty() { + self.unique_row_number.clear(); + self.accepted_data = true; + return Ok(()); + } + let row_number_vec = get_row_number(&data_block, 0)?; let row_number_set: HashSet = row_number_vec.iter().cloned().collect(); assert_eq!(row_number_set.len(), row_number_vec.len()); diff --git a/src/query/service/src/pipelines/processors/transforms/processor_extract_hash_table_by_row_number.rs b/src/query/service/src/pipelines/processors/transforms/processor_extract_hash_table_by_row_number.rs index 386cc7439d1b..604ea7fad745 100644 --- a/src/query/service/src/pipelines/processors/transforms/processor_extract_hash_table_by_row_number.rs +++ b/src/query/service/src/pipelines/processors/transforms/processor_extract_hash_table_by_row_number.rs @@ -127,7 +127,7 @@ impl Processor for ExtractHashTableByRowNumber { DataType::Number(NumberDataType::UInt64) ); let mut bitmap = MutableBitmap::new(); - let row_numbers = get_row_number(&block, block.num_columns() - 1)?; + let row_numbers = get_row_number(block, block.num_columns() - 1)?; for row_number in row_numbers.iter() { if row_number_set.contains(row_number) { bitmap.push(true); diff --git a/src/query/sql/src/executor/physical_plan_visitor.rs b/src/query/sql/src/executor/physical_plan_visitor.rs index 70cba5c1aa6b..738ed29a607d 100644 --- a/src/query/sql/src/executor/physical_plan_visitor.rs +++ b/src/query/sql/src/executor/physical_plan_visitor.rs @@ -88,7 +88,7 @@ pub trait PhysicalPlanReplacer { PhysicalPlan::Deduplicate(plan) => self.replace_deduplicate(plan), PhysicalPlan::ReplaceInto(plan) => self.replace_replace_into(plan), PhysicalPlan::MergeInto(plan) => self.replace_merge_into(plan), - PhysicalPlan::AddRowNumber(plan) => self.replace_add_row_number(&plan), + PhysicalPlan::AddRowNumber(plan) => self.replace_add_row_number(plan), PhysicalPlan::MergeIntoSource(plan) => self.replace_merge_into_source(plan), PhysicalPlan::MergeIntoAppendNotMatched(plan) => { self.replace_merge_into_row_id_apply(plan) diff --git a/src/query/sql/src/planner/optimizer/distributed/distributed_merge.rs b/src/query/sql/src/planner/optimizer/distributed/distributed_merge.rs index 10930c05a17b..467f7052a13e 100644 --- a/src/query/sql/src/planner/optimizer/distributed/distributed_merge.rs +++ b/src/query/sql/src/planner/optimizer/distributed/distributed_merge.rs @@ -43,9 +43,9 @@ impl MergeSourceOptimizer { // and build hashtable. It means all nodes hold the same hashtable. pub fn optimize(&self, s_expr: &SExpr) -> Result { if !s_expr.match_pattern(&self.merge_source_pattern) { - return Err(ErrorCode::BadArguments(format!( - "pattern not match for dirstributed merge source" - ))); + Err(ErrorCode::BadArguments( + "pattern not match for dirstributed merge source".to_string(), + )) } else { let join_s_expr = s_expr.child(0)?; diff --git a/src/query/storages/fuse/src/operations/merge_into/processors/processor_distributed_merge_into_block_deserialize.rs b/src/query/storages/fuse/src/operations/merge_into/processors/processor_distributed_merge_into_block_deserialize.rs index d7cd7851ff70..8ab6a906fb11 100644 --- a/src/query/storages/fuse/src/operations/merge_into/processors/processor_distributed_merge_into_block_deserialize.rs +++ b/src/query/storages/fuse/src/operations/merge_into/processors/processor_distributed_merge_into_block_deserialize.rs @@ -35,6 +35,7 @@ use super::RowIdKind; // RowIds, because arrow-flight requires this. pub struct TransformDistributedMergeIntoBlockDeserialize; +/// this processor will be used in the future for merge into based on shuffle hash join. impl TransformDistributedMergeIntoBlockDeserialize { #[allow(dead_code)] fn create(input: Arc, output: Arc) -> ProcessorPtr { diff --git a/src/query/storages/fuse/src/operations/merge_into/processors/processor_distributed_merge_into_block_serialize.rs b/src/query/storages/fuse/src/operations/merge_into/processors/processor_distributed_merge_into_block_serialize.rs index 0a4097bf19b4..1b6e358adcad 100644 --- a/src/query/storages/fuse/src/operations/merge_into/processors/processor_distributed_merge_into_block_serialize.rs +++ b/src/query/storages/fuse/src/operations/merge_into/processors/processor_distributed_merge_into_block_serialize.rs @@ -40,6 +40,7 @@ use crate::operations::common::MutationLogs; // RowIds, because arrow-flight requires this. pub struct TransformDistributedMergeIntoBlockSerialize; +/// this processor will be used in the future for merge into based on shuffle hash join. impl TransformDistributedMergeIntoBlockSerialize { #[allow(dead_code)] fn create(input: Arc, output: Arc) -> ProcessorPtr { diff --git a/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_matched_and_split.rs b/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_matched_and_split.rs index 72d85a089a7e..be012aabc70b 100644 --- a/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_matched_and_split.rs +++ b/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_matched_and_split.rs @@ -43,6 +43,20 @@ use crate::operations::common::MutationLogs; use crate::operations::merge_into::mutator::DeleteByExprMutator; use crate::operations::merge_into::mutator::UpdateByExprMutator; +#[derive(serde::Serialize, serde::Deserialize, Clone, Debug, PartialEq)] +pub struct SourceFullMatched; + +#[typetag::serde(name = "source_full_macthed")] +impl BlockMetaInfo for SourceFullMatched { + fn equals(&self, info: &Box) -> bool { + SourceFullMatched::downcast_ref_from(info).is_some_and(|other| self == other) + } + + fn clone_self(&self) -> Box { + Box::new(self.clone()) + } +} + enum MutationKind { Update(UpdateDataBlockMutation), Delete(DeleteDataBlockMutation), diff --git a/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_split.rs b/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_split.rs index fe1d6762eaa6..3df7949f3310 100644 --- a/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_split.rs +++ b/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_split.rs @@ -29,6 +29,7 @@ use common_storage::metrics::merge_into::metrics_inc_merge_into_matched_rows; use common_storage::metrics::merge_into::metrics_inc_merge_into_split_milliseconds; use common_storage::metrics::merge_into::metrics_inc_merge_into_unmatched_rows; +use super::processor_merge_into_matched_and_split::SourceFullMatched; use crate::operations::merge_into::mutator::MergeIntoSplitMutator; pub struct MergeIntoSplitProcessor { @@ -147,6 +148,19 @@ impl Processor for MergeIntoSplitProcessor { // Todo:(JackTan25) accutally, we should do insert-only optimization in the future. fn process(&mut self) -> Result<()> { if let Some(data_block) = self.input_data.take() { + // for distributed execution, if one node matched all source data. + // if we use right join, we will receive a empty block, but we must + // give it to downstream. + if data_block.is_empty() { + self.output_data_matched_data = Some(data_block.clone()); + // if a probe block can't match any data of hashtable, it will + // give an empty block here? The answer is no. so for right join, + // when we get an empty block, it says all source data has been matched + let block = data_block.add_meta(Some(Box::new(SourceFullMatched)))?; + self.output_data_not_matched_data = Some(block); + return Ok(()); + } + if self.target_table_empty { self.output_data_not_matched_data = Some(data_block) } else { diff --git a/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_split_row_number_and_log.rs b/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_split_row_number_and_log.rs index b8dd3ba549de..57e29ab0b80d 100644 --- a/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_split_row_number_and_log.rs +++ b/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_split_row_number_and_log.rs @@ -26,7 +26,7 @@ use common_pipeline_core::processors::processor::Event; use common_pipeline_core::processors::processor::ProcessorPtr; use common_pipeline_core::processors::Processor; -use crate::operations::common::MutationLogs; +use super::processor_merge_into_matched_and_split::SourceFullMatched; pub struct RowNumberAndLogSplitProcessor { input_port: Arc, @@ -131,11 +131,14 @@ impl Processor for RowNumberAndLogSplitProcessor { fn process(&mut self) -> Result<()> { if let Some(data_block) = self.input_data.take() { - // mutation logs - if data_block.is_empty() { - let mix_kind = - MutationLogs::downcast_ref_from(data_block.get_meta().unwrap()).unwrap(); - self.output_data_log = Some(data_block); + // all matched or logs + if data_block.get_meta().is_some() { + if SourceFullMatched::downcast_ref_from(data_block.get_meta().unwrap()).is_some() { + self.output_data_row_number = Some(data_block) + } else { + // mutation logs + self.output_data_log = Some(data_block); + } } else { self.output_data_row_number = Some(data_block) } diff --git a/src/query/storages/fuse/src/operations/merge_into/processors/transform_add_rownumber_column.rs b/src/query/storages/fuse/src/operations/merge_into/processors/transform_add_rownumber_column.rs index cf3202148833..c47a8ec03937 100644 --- a/src/query/storages/fuse/src/operations/merge_into/processors/transform_add_rownumber_column.rs +++ b/src/query/storages/fuse/src/operations/merge_into/processors/transform_add_rownumber_column.rs @@ -61,7 +61,7 @@ impl TransformAddRowNumberColumnProcessor { fn generate_row_number(&mut self, num_rows: u64) -> u64 { let row_number = self.row_number.fetch_add(num_rows, Ordering::SeqCst); let mut prefix_u64 = self.prefix as u64; - prefix_u64 = prefix_u64 << PREFIX_OFFSET; + prefix_u64 <<= PREFIX_OFFSET; prefix_u64 | row_number } } diff --git a/tests/sqllogictests/suites/base/09_fuse_engine/09_0028_distributed_merge_into b/tests/sqllogictests/suites/base/09_fuse_engine/09_0028_distributed_merge_into index 656e7029275d..710fb1aa647e 100644 --- a/tests/sqllogictests/suites/base/09_fuse_engine/09_0028_distributed_merge_into +++ b/tests/sqllogictests/suites/base/09_fuse_engine/09_0028_distributed_merge_into @@ -40,16 +40,16 @@ statement ok insert into distributed_source_test values(6,'h',true),(7,'i',true),(8,'j',false),(9,'k',true),(10,'l',false); statement ok -create stage source_parquet file_format = (type = parquet); +create stage distributed_source_parquet file_format = (type = parquet); statement ok -remove @source_parquet; +remove @distributed_source_parquet; statement ok -copy into @source_parquet from (select * from distributed_source_test); +copy into @distributed_source_parquet from (select * from distributed_source_test); statement ok -merge into `distributed_target_test` as tt using (select `a`,`b`,`is_databend_deleted` from @source_parquet (pattern => '.*[.]parquet')) as ss on (ss.`a` = tt.`a`) +merge into `distributed_target_test` as tt using (select `a`,`b`,`is_databend_deleted` from @distributed_source_parquet (pattern => '.*[.]parquet')) as ss on (ss.`a` = tt.`a`) when matched and ss.`is_databend_deleted` = true then delete when matched then update * when not matched and ss.`is_databend_deleted` = false then insert *; query TT @@ -60,6 +60,64 @@ select * from distributed_target_test order by a; 8 j 10 l +## cornor case test +## when a node matched all source, it will give a empty block, +## so in this case, we should not insert anything. +statement ok +drop table if exists cornor_target_table; + +statement ok +create table cornor_target_table(a int,b string,c string); + +statement ok +drop table if exists cornor_source_table; + +statement ok +create table cornor_source_table(a int,b string,c string); + +## add block1 +statement ok +insert into cornor_target_table values(1,'a1','b1'),(2,'a2','b2'); + +## add block2 +statement ok +insert into cornor_target_table values(1,'a3','b3'),(2,'a4','b4'),(3,'a5','b5'); + +## add two blocks for cornor_source_table +statement ok +insert into cornor_source_table values(1,'a6','b6'),(2,'a7','b7'); + +statement ok +insert into cornor_source_table values(3,'a8','b8'); + +statement ok +select * from cornor_target_table order by a,b; +---- +1 a1 b1 +1 a3 b3 +2 a2 b2 +2 a4 b4 +3 a5 b5 + +statement ok +select * from cornor_source_table order by a,b; +---- +1 a6 b6 +2 a7 b7 +3 a8 b8 + +statement ok +merge into cornor_target_table as t1 using (select * from cornor_source_table) as t2 on t1.a = t2.a when matched then update set t1.a = t2.a when not matched then insert *; + +statement ok +select * from cornor_target_table order by a; +---- +1 a6 b6 +1 a6 b6 +2 a7 b7 +2 a7 b7 +3 a8 b8 + statement ok set enable_experimental_merge_into = 0; From 19305898f16cfe240e968ba64efe398668622a58 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Sun, 29 Oct 2023 19:13:53 +0800 Subject: [PATCH 31/39] fix check --- .../service/src/api/rpc/exchange/exchange_manager.rs | 8 ++------ src/query/service/src/pipelines/pipeline_builder.rs | 4 ++-- 2 files changed, 4 insertions(+), 8 deletions(-) diff --git a/src/query/service/src/api/rpc/exchange/exchange_manager.rs b/src/query/service/src/api/rpc/exchange/exchange_manager.rs index 8da8c4b6a12b..4283d096e339 100644 --- a/src/query/service/src/api/rpc/exchange/exchange_manager.rs +++ b/src/query/service/src/api/rpc/exchange/exchange_manager.rs @@ -921,14 +921,10 @@ impl FragmentCoordinator { pipeline_builder.join_state = input_builder_data.input_join_state; pipeline_builder.probe_data_fields = input_builder_data.input_probe_schema; - let (res, join_state, probe_schema) = - pipeline_builder.finalize_local(&self.physical_plan)?; + let (res, builder_data) = pipeline_builder.finalize_local(&self.physical_plan)?; self.pipeline_build_res = Some(res); - return Ok(PipelineBuilderData { - input_join_state: join_state, - input_probe_schema: probe_schema, - }); + return Ok(builder_data); } Ok(PipelineBuilderData { diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs index af954f08a3fe..64b8716f0a1f 100644 --- a/src/query/service/src/pipelines/pipeline_builder.rs +++ b/src/query/service/src/pipelines/pipeline_builder.rs @@ -949,7 +949,7 @@ impl PipelineBuilder { self.main_pipeline.add_pipe(builder.finalize()); } else { builder.add_items_prepend(vec![create_dummy_item()]); - // recieve row_number + // receive row_number builder.add_items(vec![create_dummy_item()]); self.main_pipeline.add_pipe(builder.finalize()); } @@ -1004,7 +1004,7 @@ impl PipelineBuilder { pipe_items.push(serialize_block_transform.into_pipe_item()); } - // recieve row_number + // receive row_number if *distributed { pipe_items.push(create_dummy_item()); } From 223617848d60d52e627946e332e11e900e41aaec Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Sun, 29 Oct 2023 21:36:05 +0800 Subject: [PATCH 32/39] add more tests --- .../service/src/pipelines/pipeline_builder.rs | 7 -- .../sql/src/planner/binder/merge_into.rs | 14 ++-- .../distributed/distributed_merge.rs | 5 +- src/query/sql/src/planner/plans/update.rs | 2 +- .../09_0028_distributed_merge_into | 64 +++++++++++++------ 5 files changed, 56 insertions(+), 36 deletions(-) diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs index 64b8716f0a1f..7df1bcfb728e 100644 --- a/src/query/service/src/pipelines/pipeline_builder.rs +++ b/src/query/service/src/pipelines/pipeline_builder.rs @@ -1065,13 +1065,6 @@ impl PipelineBuilder { pipe_items, )); - for pipe in &self.main_pipeline.pipes { - println!( - "pipe:\n{:?},input_len:{:?},output_len:{:?}", - pipe, pipe.input_length, pipe.output_length - ); - } - Ok(()) } diff --git a/src/query/sql/src/planner/binder/merge_into.rs b/src/query/sql/src/planner/binder/merge_into.rs index e3b62ff56a2b..cfe78d0b32b5 100644 --- a/src/query/sql/src/planner/binder/merge_into.rs +++ b/src/query/sql/src/planner/binder/merge_into.rs @@ -46,6 +46,7 @@ use crate::plans::MergeInto; use crate::plans::Plan; use crate::plans::UnmatchedEvaluator; use crate::BindContext; +use crate::ColumnBinding; use crate::ColumnBindingBuilder; use crate::ColumnEntry; use crate::IndexType; @@ -141,16 +142,16 @@ impl Binder { ); let source_output_columns = &source_context.columns; // we use Vec as the value, because if there could be duplicate names - let mut name_map = HashMap::>::new(); + let mut name_map = HashMap::>::new(); for column in source_output_columns { name_map .entry(column.column_name.clone()) .or_insert_with(|| vec![]) - .push(column.index); + .push(column.clone()); } for (field_idx, field) in default_target_table_schema.fields.iter().enumerate() { - let index = match name_map.get(field.name()) { + let column = match name_map.get(field.name()) { None => { return Err(ErrorCode::SemanticError( format!("can't find {} in source output", field.name).to_string(), @@ -167,18 +168,19 @@ impl Binder { .to_string(), )); } else { - indices[0] + indices[0].clone() } } }; let column = ColumnBindingBuilder::new( field.name.to_string(), - index, - Box::new(field.data_type().into()), + column.index, + column.data_type.clone(), Visibility::Visible, ) .build(); let col = ScalarExpr::BoundColumnRef(BoundColumnRef { span: None, column }); + update_columns.insert(field_idx, col); } Some(update_columns) diff --git a/src/query/sql/src/planner/optimizer/distributed/distributed_merge.rs b/src/query/sql/src/planner/optimizer/distributed/distributed_merge.rs index 467f7052a13e..b2c9680468c3 100644 --- a/src/query/sql/src/planner/optimizer/distributed/distributed_merge.rs +++ b/src/query/sql/src/planner/optimizer/distributed/distributed_merge.rs @@ -14,7 +14,6 @@ use std::sync::Arc; -use common_exception::ErrorCode; use common_exception::Result; use crate::optimizer::SExpr; @@ -43,9 +42,7 @@ impl MergeSourceOptimizer { // and build hashtable. It means all nodes hold the same hashtable. pub fn optimize(&self, s_expr: &SExpr) -> Result { if !s_expr.match_pattern(&self.merge_source_pattern) { - Err(ErrorCode::BadArguments( - "pattern not match for dirstributed merge source".to_string(), - )) + Ok(s_expr.clone()) } else { let join_s_expr = s_expr.child(0)?; diff --git a/src/query/sql/src/planner/plans/update.rs b/src/query/sql/src/planner/plans/update.rs index a9e475005f0c..470db62a7155 100644 --- a/src/query/sql/src/planner/plans/update.rs +++ b/src/query/sql/src/planner/plans/update.rs @@ -115,7 +115,7 @@ impl UpdatePlan { let mut right = right.ok_or_else(|| ErrorCode::Internal("It's a bug"))?; let right_data_type = right.data_type()?; - // cornor case: for merge into, if target_table's fields are not null, when after bind_join, it will + // corner case: for merge into, if target_table's fields are not null, when after bind_join, it will // change into nullable, so we need to cast this. right = wrap_cast_scalar(&right, &right_data_type, target_type)?; diff --git a/tests/sqllogictests/suites/base/09_fuse_engine/09_0028_distributed_merge_into b/tests/sqllogictests/suites/base/09_fuse_engine/09_0028_distributed_merge_into index 710fb1aa647e..404217f0da9e 100644 --- a/tests/sqllogictests/suites/base/09_fuse_engine/09_0028_distributed_merge_into +++ b/tests/sqllogictests/suites/base/09_fuse_engine/09_0028_distributed_merge_into @@ -60,38 +60,38 @@ select * from distributed_target_test order by a; 8 j 10 l -## cornor case test +## corner case test ## when a node matched all source, it will give a empty block, ## so in this case, we should not insert anything. statement ok -drop table if exists cornor_target_table; +drop table if exists corner_target_table; statement ok -create table cornor_target_table(a int,b string,c string); +create table corner_target_table(a int,b string,c string); statement ok -drop table if exists cornor_source_table; +drop table if exists corner_source_table; statement ok -create table cornor_source_table(a int,b string,c string); +create table corner_source_table(a int,b string,c string); ## add block1 statement ok -insert into cornor_target_table values(1,'a1','b1'),(2,'a2','b2'); +insert into corner_target_table values(1,'a1','b1'),(2,'a2','b2'); ## add block2 statement ok -insert into cornor_target_table values(1,'a3','b3'),(2,'a4','b4'),(3,'a5','b5'); +insert into corner_target_table values(1,'a3','b3'),(2,'a4','b4'),(3,'a5','b5'); -## add two blocks for cornor_source_table +## add two blocks for corner_source_table statement ok -insert into cornor_source_table values(1,'a6','b6'),(2,'a7','b7'); +insert into corner_source_table values(1,'a6','b6'),(2,'a7','b7'); statement ok -insert into cornor_source_table values(3,'a8','b8'); +insert into corner_source_table values(3,'a8','b8'); -statement ok -select * from cornor_target_table order by a,b; +query TTT +select * from corner_target_table order by a,b; ---- 1 a1 b1 1 a3 b3 @@ -99,18 +99,18 @@ select * from cornor_target_table order by a,b; 2 a4 b4 3 a5 b5 -statement ok -select * from cornor_source_table order by a,b; +query TTT +select * from corner_source_table order by a,b; ---- 1 a6 b6 2 a7 b7 3 a8 b8 -statement ok -merge into cornor_target_table as t1 using (select * from cornor_source_table) as t2 on t1.a = t2.a when matched then update set t1.a = t2.a when not matched then insert *; +query TTT +merge into corner_target_table as t1 using (select * from corner_source_table) as t2 on t1.a = t2.a when matched then update set t1.a = t2.a when not matched then insert *; -statement ok -select * from cornor_target_table order by a; +query TTT +select * from corner_target_table order by a; ---- 1 a6 b6 1 a6 b6 @@ -118,6 +118,34 @@ select * from cornor_target_table order by a; 2 a7 b7 3 a8 b8 +## test wide-table +statement ok +drop table if exists distributed_test_order; + +statement ok +drop table if exists distributed_random_source; + +statement ok +create table distributed_test_order(id bigint, id1 bigint, id2 bigint, id3 bigint, id4 bigint, id5 bigint, id6 bigint, id7 bigint, s1 varchar, s2 varchar, s3 varchar, s4 varchar, s5 varchar, s6 varchar, s7 varchar, s8 varchar, s9 varchar, s10 varchar, s11 varchar, s12 varchar, s13 varchar, d1 DECIMAL(20, 8), d2 DECIMAL(20, 8), d3 DECIMAL(20, 8), d4 DECIMAL(20, 8), d5 DECIMAL(20, 8), d6 DECIMAL(30, 8), d7 DECIMAL(30, 8), d8 DECIMAL(30, 8), d9 DECIMAL(30, 8), d10 DECIMAL(30, 8),insert_time datetime, insert_time1 datetime, insert_time2 datetime, insert_time3 datetime,i int) CLUSTER BY(to_yyyymmdd(insert_time), id) bloom_index_columns='insert_time,id'; + +statement ok +create table distributed_random_source(id bigint not null, id1 bigint, id2 bigint, id3 bigint, id4 bigint, id5 bigint, id6 bigint, id7 bigint,s1 varchar, s2 varchar, s3 varchar, s4 varchar, s5 varchar, s6 varchar, s7 varchar, s8 varchar, s9 varchar, s10 varchar, s11 varchar, s12 varchar, s13 varchar,d1 DECIMAL(20, 8), d2 DECIMAL(20, 8), d3 DECIMAL(20, 8), d4 DECIMAL(20, 8), d5 DECIMAL(20, 8), d6 DECIMAL(30, 8), d7 DECIMAL(30, 8), d8 DECIMAL(30, 8), d9 DECIMAL(30, 8), d10 DECIMAL(30, 8),insert_time datetime not null, insert_time1 datetime, insert_time2 datetime, insert_time3 datetime,i int) Engine = Random; + +statement ok +create table distributed_random_store(id bigint not null, id1 bigint, id2 bigint, id3 bigint, id4 bigint, id5 bigint, id6 bigint, id7 bigint,s1 varchar, s2 varchar, s3 varchar, s4 varchar, s5 varchar, s6 varchar, s7 varchar, s8 varchar, s9 varchar, s10 varchar, s11 varchar, s12 varchar, s13 varchar,d1 DECIMAL(20, 8), d2 DECIMAL(20, 8), d3 DECIMAL(20, 8), d4 DECIMAL(20, 8), d5 DECIMAL(20, 8), d6 DECIMAL(30, 8), d7 DECIMAL(30, 8), d8 DECIMAL(30, 8), d9 DECIMAL(30, 8), d10 DECIMAL(30, 8),insert_time datetime not null, insert_time1 datetime, insert_time2 datetime, insert_time3 datetime,i int); + +statement ok +insert into distributed_random_store (select id,34 as id1,238 as id2, id3, id4, id5, id6, id7,s1, s2, s3, s4, s5, s6, s7, s8, s9, s10, s11, s12, s13,d1, d2, d3, d4, d5, d6, d7, d8, d9, d10,insert_time,insert_time1,insert_time2,insert_time3,i from distributed_random_source limit 10); + +statement ok +insert into distributed_random_store (select id,34 as id1,238 as id2, id3, id4, id5, id6, id7,s1, s2, s3, s4, s5, s6, s7, s8, s9, s10, s11, s12, s13,d1, d2, d3, d4, d5, d6, d7, d8, d9, d10,insert_time,insert_time1,insert_time2,insert_time3,i from distributed_random_source limit 10); + +statement ok +merge into distributed_test_order as t using (select id,34 as id1,238 as id2, id3, id4, id5, id6, id7,s1, s2, s3, s4, s5, s6, s7, s8, s9, s10, s11, s12, s13,d1, d2, d3, d4, d5, d6, d7, d8, d9, d10,insert_time,insert_time1,insert_time2,insert_time3,i from distributed_random_store) as s on t.id = s.id and t.insert_time = s.insert_time when matched then update * when not matched then insert *; + +statement ok +merge into distributed_test_order as t using (select id,34 as id1,238 as id2, id3, id4, id5, id6, id7,s1, s2, s3, s4, s5, s6, s7, s8, s9, s10, s11, s12, s13,d1, d2, d3, d4, d5, d6, d7, d8, d9, d10,insert_time,insert_time1,insert_time2,insert_time3,i from distributed_random_store) as s on t.id = s.id and t.insert_time = s.insert_time when matched then update * when not matched then insert *; + statement ok set enable_experimental_merge_into = 0; From 8f1739d265934cab7613f26d18ca2ec582d7c71b Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Sun, 29 Oct 2023 22:26:42 +0800 Subject: [PATCH 33/39] fix flaky --- .../suites/base/09_fuse_engine/09_0028_distributed_merge_into | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/sqllogictests/suites/base/09_fuse_engine/09_0028_distributed_merge_into b/tests/sqllogictests/suites/base/09_fuse_engine/09_0028_distributed_merge_into index 404217f0da9e..4ece583b09a9 100644 --- a/tests/sqllogictests/suites/base/09_fuse_engine/09_0028_distributed_merge_into +++ b/tests/sqllogictests/suites/base/09_fuse_engine/09_0028_distributed_merge_into @@ -110,7 +110,7 @@ query TTT merge into corner_target_table as t1 using (select * from corner_source_table) as t2 on t1.a = t2.a when matched then update set t1.a = t2.a when not matched then insert *; query TTT -select * from corner_target_table order by a; +select * from corner_target_table order by a,b; ---- 1 a6 b6 1 a6 b6 From 68aa9bedd0504f82541dbff508619493b9002dca Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Sun, 29 Oct 2023 22:49:46 +0800 Subject: [PATCH 34/39] fix test result --- .../suites/base/09_fuse_engine/09_0028_distributed_merge_into | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/sqllogictests/suites/base/09_fuse_engine/09_0028_distributed_merge_into b/tests/sqllogictests/suites/base/09_fuse_engine/09_0028_distributed_merge_into index 4ece583b09a9..db07f345df97 100644 --- a/tests/sqllogictests/suites/base/09_fuse_engine/09_0028_distributed_merge_into +++ b/tests/sqllogictests/suites/base/09_fuse_engine/09_0028_distributed_merge_into @@ -107,7 +107,7 @@ select * from corner_source_table order by a,b; 3 a8 b8 query TTT -merge into corner_target_table as t1 using (select * from corner_source_table) as t2 on t1.a = t2.a when matched then update set t1.a = t2.a when not matched then insert *; +merge into corner_target_table as t1 using (select * from corner_source_table) as t2 on t1.a = t2.a when matched then update * when not matched then insert *; query TTT select * from corner_target_table order by a,b; From d7fc27b37cca89b910df6d7daf15ecd8b409a63f Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Mon, 30 Oct 2023 16:21:45 +0800 Subject: [PATCH 35/39] fix order --- src/query/service/src/pipelines/pipeline_builder.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs index 7df1bcfb728e..bb617bd4417a 100644 --- a/src/query/service/src/pipelines/pipeline_builder.rs +++ b/src/query/service/src/pipelines/pipeline_builder.rs @@ -1032,10 +1032,11 @@ impl PipelineBuilder { for idx in 0..output_lens { vec.push(idx + 1); } + ranges.push(vec); if *distributed { ranges.push(vec![self.main_pipeline.output_len() - 1]); } - ranges.push(vec); + self.main_pipeline.resize_partial_one(ranges)?; let pipe_items = if !distributed { From edffda8f464be55d19eff47e241543764cb1c8cc Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Mon, 30 Oct 2023 19:32:40 +0800 Subject: [PATCH 36/39] clean codes --- src/query/service/src/pipelines/pipeline_build_res.rs | 1 + .../processors/transforms/processor_deduplicate_row_number.rs | 3 +-- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/query/service/src/pipelines/pipeline_build_res.rs b/src/query/service/src/pipelines/pipeline_build_res.rs index 575ae54ba2c3..693907f6f2f7 100644 --- a/src/query/service/src/pipelines/pipeline_build_res.rs +++ b/src/query/service/src/pipelines/pipeline_build_res.rs @@ -42,6 +42,7 @@ pub struct PipelineBuildResult { pub prof_span_set: SharedProcessorProfiles, pub exchange_injector: Arc, + } impl PipelineBuildResult { diff --git a/src/query/service/src/pipelines/processors/transforms/processor_deduplicate_row_number.rs b/src/query/service/src/pipelines/processors/transforms/processor_deduplicate_row_number.rs index 5fbd7ac15ca3..a09ca7cab62c 100644 --- a/src/query/service/src/pipelines/processors/transforms/processor_deduplicate_row_number.rs +++ b/src/query/service/src/pipelines/processors/transforms/processor_deduplicate_row_number.rs @@ -47,8 +47,7 @@ impl AsyncAccumulatingTransform for DeduplicateRowNumber { if self.unique_row_number.is_empty() { Ok(Some(DataBlock::empty())) } else { - let block = self.apply().await?; - Ok(block) + self.apply().await } } } From deb4d1fa9866ff2b09add43831128ed7c9c91dfd Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Mon, 30 Oct 2023 21:40:06 +0800 Subject: [PATCH 37/39] remove local builder branch --- .../src/api/rpc/exchange/exchange_manager.rs | 119 ++++-------------- .../src/pipelines/pipeline_build_res.rs | 22 +++- .../service/src/pipelines/pipeline_builder.rs | 28 +---- 3 files changed, 49 insertions(+), 120 deletions(-) diff --git a/src/query/service/src/api/rpc/exchange/exchange_manager.rs b/src/query/service/src/api/rpc/exchange/exchange_manager.rs index 4283d096e339..a45de7db3f70 100644 --- a/src/query/service/src/api/rpc/exchange/exchange_manager.rs +++ b/src/query/service/src/api/rpc/exchange/exchange_manager.rs @@ -211,28 +211,6 @@ impl DataExchangeManager { } } - // Create a pipeline based on query plan - #[minitrace::trace] - pub fn init_query_fragments_plan_local( - &self, - ctx: &Arc, - packet: &QueryFragmentsPlanPacket, - ) -> Result<()> { - let queries_coordinator_guard = self.queries_coordinator.lock(); - let queries_coordinator = unsafe { &mut *queries_coordinator_guard.deref().get() }; - - // TODO: When the query is not executed for a long time after submission, we need to remove it - match queries_coordinator.get_mut(&packet.query_id) { - None => Err(ErrorCode::Internal(format!( - "Query {} not found in cluster.", - packet.query_id - ))), - Some(query_coordinator) => { - query_coordinator.prepare_pipeline_local(ctx, packet.enable_profiling, packet) - } - } - } - #[minitrace::trace] pub fn handle_statistics_exchange( &self, @@ -320,7 +298,7 @@ impl DataExchangeManager { .await?; // Submit tasks to localhost - self.init_query_fragments_plan_local(&ctx, &local_query_fragments_plan_packet)?; + self.init_query_fragments_plan(&ctx, &local_query_fragments_plan_packet)?; // Get local pipeline of local task let build_res = self.get_root_pipeline(ctx, enable_profiling, root_actions)?; @@ -603,37 +581,7 @@ impl QueryCoordinator { ); } - for fragment in &packet.fragments { - let fragment_id = fragment.fragment_id; - if let Some(coordinator) = self.fragments_coordinator.get_mut(&fragment_id) { - coordinator.prepare_pipeline(ctx.clone(), enable_profiling)?; - } - } - - Ok(()) - } - - pub fn prepare_pipeline_local( - &mut self, - ctx: &Arc, - enable_profiling: bool, - packet: &QueryFragmentsPlanPacket, - ) -> Result<()> { - self.info = Some(QueryInfo { - query_ctx: ctx.clone(), - query_id: packet.query_id.clone(), - current_executor: packet.executor.clone(), - query_executor: None, - }); - - for fragment in &packet.fragments { - self.fragments_coordinator.insert( - fragment.fragment_id.to_owned(), - FragmentCoordinator::create(fragment), - ); - } - - let mut input_builder_data = PipelineBuilderData { + let mut last_builder_data = PipelineBuilderData { input_join_state: None, input_probe_schema: None, }; @@ -641,11 +589,22 @@ impl QueryCoordinator { for fragment in &packet.fragments { let fragment_id = fragment.fragment_id; if let Some(coordinator) = self.fragments_coordinator.get_mut(&fragment_id) { - input_builder_data = coordinator.prepare_pipeline_local( - ctx.clone(), - enable_profiling, - input_builder_data, - )?; + // before prepare pipeline, it must be none + assert!(coordinator.pipeline_build_res.is_none()); + // reuse last builder data + coordinator.pipeline_build_res = Some( + PipelineBuildResult::create_with_builder_data(last_builder_data.clone()), + ); + + coordinator.prepare_pipeline(ctx.clone(), enable_profiling)?; + + // after prepare pipeline, it must be some + assert!(coordinator.pipeline_build_res.is_some()); + // update builder_data + if coordinator.pipeline_build_res.is_some() { + let new_res = coordinator.pipeline_build_res.as_ref().unwrap(); + last_builder_data = new_res.builder_data.clone(); + } } } @@ -879,32 +838,6 @@ impl FragmentCoordinator { ctx: Arc, enable_profiling: bool, ) -> Result<()> { - if !self.initialized { - self.initialized = true; - - let pipeline_ctx = QueryContext::create_from(ctx); - - let pipeline_builder = PipelineBuilder::create( - pipeline_ctx.get_function_context()?, - pipeline_ctx.get_settings(), - pipeline_ctx, - enable_profiling, - SharedProcessorProfiles::default(), - ); - let res = pipeline_builder.finalize(&self.physical_plan)?; - - self.pipeline_build_res = Some(res); - } - - Ok(()) - } - - pub fn prepare_pipeline_local( - &mut self, - ctx: Arc, - enable_profiling: bool, - input_builder_data: PipelineBuilderData, - ) -> Result { if !self.initialized { self.initialized = true; @@ -918,18 +851,18 @@ impl FragmentCoordinator { SharedProcessorProfiles::default(), ); - pipeline_builder.join_state = input_builder_data.input_join_state; - pipeline_builder.probe_data_fields = input_builder_data.input_probe_schema; + // has last pipeline_res builder_data, we reuse it + if self.pipeline_build_res.is_some() { + let res = self.pipeline_build_res.take().unwrap(); + pipeline_builder.join_state = res.builder_data.input_join_state; + pipeline_builder.probe_data_fields = res.builder_data.input_probe_schema; + } - let (res, builder_data) = pipeline_builder.finalize_local(&self.physical_plan)?; + let res = pipeline_builder.finalize(&self.physical_plan)?; self.pipeline_build_res = Some(res); - return Ok(builder_data); } - Ok(PipelineBuilderData { - input_join_state: None, - input_probe_schema: None, - }) + Ok(()) } } diff --git a/src/query/service/src/pipelines/pipeline_build_res.rs b/src/query/service/src/pipelines/pipeline_build_res.rs index 693907f6f2f7..726c5c28195e 100644 --- a/src/query/service/src/pipelines/pipeline_build_res.rs +++ b/src/query/service/src/pipelines/pipeline_build_res.rs @@ -27,6 +27,7 @@ use super::processors::transforms::hash_join::HashJoinBuildState; use crate::api::DefaultExchangeInjector; use crate::api::ExchangeInjector; +#[derive(Clone)] pub struct PipelineBuilderData { pub input_join_state: Option>, pub input_probe_schema: Option>, @@ -42,7 +43,8 @@ pub struct PipelineBuildResult { pub prof_span_set: SharedProcessorProfiles, pub exchange_injector: Arc, - + /// for local fragment data sharing + pub builder_data: PipelineBuilderData, } impl PipelineBuildResult { @@ -52,6 +54,20 @@ impl PipelineBuildResult { sources_pipelines: vec![], prof_span_set: SharedProcessorProfiles::default(), exchange_injector: DefaultExchangeInjector::create(), + builder_data: PipelineBuilderData { + input_join_state: None, + input_probe_schema: None, + }, + } + } + + pub fn create_with_builder_data(builder_data: PipelineBuilderData) -> PipelineBuildResult { + PipelineBuildResult { + main_pipeline: Pipeline::create(), + sources_pipelines: vec![], + prof_span_set: SharedProcessorProfiles::default(), + exchange_injector: DefaultExchangeInjector::create(), + builder_data, } } @@ -71,6 +87,10 @@ impl PipelineBuildResult { sources_pipelines: vec![], prof_span_set: SharedProcessorProfiles::default(), exchange_injector: DefaultExchangeInjector::create(), + builder_data: PipelineBuilderData { + input_join_state: None, + input_probe_schema: None, + }, }) } diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs index bb617bd4417a..59ca4a68b950 100644 --- a/src/query/service/src/pipelines/pipeline_builder.rs +++ b/src/query/service/src/pipelines/pipeline_builder.rs @@ -255,35 +255,11 @@ impl PipelineBuilder { sources_pipelines: self.pipelines, prof_span_set: self.proc_profs, exchange_injector: self.exchange_injector, - }) - } - - pub fn finalize_local( - mut self, - plan: &PhysicalPlan, - ) -> Result<(PipelineBuildResult, PipelineBuilderData)> { - self.build_pipeline(plan)?; - - for source_pipeline in &self.pipelines { - if !source_pipeline.is_complete_pipeline()? { - return Err(ErrorCode::Internal( - "Source pipeline must be complete pipeline.", - )); - } - } - - Ok(( - PipelineBuildResult { - main_pipeline: self.main_pipeline, - sources_pipelines: self.pipelines, - prof_span_set: self.proc_profs, - exchange_injector: self.exchange_injector, - }, - PipelineBuilderData { + builder_data: PipelineBuilderData { input_join_state: self.join_state, input_probe_schema: self.probe_data_fields, }, - )) + }) } fn build_pipeline(&mut self, plan: &PhysicalPlan) -> Result<()> { From 9fcf4b48662a37b8c079f5b60ae7ab8ba2ea9960 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Mon, 30 Oct 2023 22:05:12 +0800 Subject: [PATCH 38/39] refactor logic --- .../src/api/rpc/exchange/exchange_manager.rs | 30 +------------------ .../src/pipelines/pipeline_build_res.rs | 10 ------- .../service/src/pipelines/pipeline_builder.rs | 4 +++ 3 files changed, 5 insertions(+), 39 deletions(-) diff --git a/src/query/service/src/api/rpc/exchange/exchange_manager.rs b/src/query/service/src/api/rpc/exchange/exchange_manager.rs index a45de7db3f70..b30ed89770a4 100644 --- a/src/query/service/src/api/rpc/exchange/exchange_manager.rs +++ b/src/query/service/src/api/rpc/exchange/exchange_manager.rs @@ -58,7 +58,6 @@ use crate::pipelines::executor::ExecutorSettings; use crate::pipelines::executor::PipelineCompleteExecutor; use crate::pipelines::PipelineBuildResult; use crate::pipelines::PipelineBuilder; -use crate::pipelines::PipelineBuilderData; use crate::schedulers::QueryFragmentActions; use crate::schedulers::QueryFragmentsActions; use crate::sessions::QueryContext; @@ -581,30 +580,10 @@ impl QueryCoordinator { ); } - let mut last_builder_data = PipelineBuilderData { - input_join_state: None, - input_probe_schema: None, - }; - for fragment in &packet.fragments { let fragment_id = fragment.fragment_id; if let Some(coordinator) = self.fragments_coordinator.get_mut(&fragment_id) { - // before prepare pipeline, it must be none - assert!(coordinator.pipeline_build_res.is_none()); - // reuse last builder data - coordinator.pipeline_build_res = Some( - PipelineBuildResult::create_with_builder_data(last_builder_data.clone()), - ); - coordinator.prepare_pipeline(ctx.clone(), enable_profiling)?; - - // after prepare pipeline, it must be some - assert!(coordinator.pipeline_build_res.is_some()); - // update builder_data - if coordinator.pipeline_build_res.is_some() { - let new_res = coordinator.pipeline_build_res.as_ref().unwrap(); - last_builder_data = new_res.builder_data.clone(); - } } } @@ -843,7 +822,7 @@ impl FragmentCoordinator { let pipeline_ctx = QueryContext::create_from(ctx); - let mut pipeline_builder = PipelineBuilder::create( + let pipeline_builder = PipelineBuilder::create( pipeline_ctx.get_function_context()?, pipeline_ctx.get_settings(), pipeline_ctx, @@ -851,13 +830,6 @@ impl FragmentCoordinator { SharedProcessorProfiles::default(), ); - // has last pipeline_res builder_data, we reuse it - if self.pipeline_build_res.is_some() { - let res = self.pipeline_build_res.take().unwrap(); - pipeline_builder.join_state = res.builder_data.input_join_state; - pipeline_builder.probe_data_fields = res.builder_data.input_probe_schema; - } - let res = pipeline_builder.finalize(&self.physical_plan)?; self.pipeline_build_res = Some(res); diff --git a/src/query/service/src/pipelines/pipeline_build_res.rs b/src/query/service/src/pipelines/pipeline_build_res.rs index 726c5c28195e..a180683bfe0e 100644 --- a/src/query/service/src/pipelines/pipeline_build_res.rs +++ b/src/query/service/src/pipelines/pipeline_build_res.rs @@ -61,16 +61,6 @@ impl PipelineBuildResult { } } - pub fn create_with_builder_data(builder_data: PipelineBuilderData) -> PipelineBuildResult { - PipelineBuildResult { - main_pipeline: Pipeline::create(), - sources_pipelines: vec![], - prof_span_set: SharedProcessorProfiles::default(), - exchange_injector: DefaultExchangeInjector::create(), - builder_data, - } - } - pub fn from_blocks(blocks: Vec) -> Result { let mut source_builder = SourcePipeBuilder::create(); diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs index 59ca4a68b950..76b44b1e8308 100644 --- a/src/query/service/src/pipelines/pipeline_builder.rs +++ b/src/query/service/src/pipelines/pipeline_builder.rs @@ -2549,6 +2549,10 @@ impl PipelineBuilder { self.enable_profiling, self.exchange_injector.clone(), )?; + // add sharing data + self.join_state = build_res.builder_data.input_join_state; + self.probe_data_fields = build_res.builder_data.input_probe_schema; + self.main_pipeline = build_res.main_pipeline; self.pipelines.extend(build_res.sources_pipelines); Ok(()) From cb08da0553260e0b41db034815f0388c21aaa98b Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Tue, 31 Oct 2023 12:27:54 +0800 Subject: [PATCH 39/39] clean codes --- src/query/ast/src/ast/statements/merge_into.rs | 2 +- .../src/interpreters/interpreter_merge_into.rs | 13 +++++-------- src/query/service/src/pipelines/pipeline_builder.rs | 1 - .../processor_merge_into_matched_and_split.rs | 2 ++ 4 files changed, 8 insertions(+), 10 deletions(-) diff --git a/src/query/ast/src/ast/statements/merge_into.rs b/src/query/ast/src/ast/statements/merge_into.rs index 3dd1b06b467a..e153b1dc90ed 100644 --- a/src/query/ast/src/ast/statements/merge_into.rs +++ b/src/query/ast/src/ast/statements/merge_into.rs @@ -98,7 +98,7 @@ impl Display for MergeIntoStmt { write!(f, "MERGE INTO ")?; write_dot_separated_list( f, - self.database + self.catalog .iter() .chain(&self.database) .chain(Some(&self.table_ident)), diff --git a/src/query/service/src/interpreters/interpreter_merge_into.rs b/src/query/service/src/interpreters/interpreter_merge_into.rs index 73f7466405d5..318d57e7ba91 100644 --- a/src/query/service/src/interpreters/interpreter_merge_into.rs +++ b/src/query/service/src/interpreters/interpreter_merge_into.rs @@ -205,7 +205,7 @@ impl MergeIntoInterpreter { }; let mut found_row_id = false; - let mut row_number_idx = -1; + let mut row_number_idx = None; for (idx, data_field) in join_output_schema.fields().iter().enumerate() { if *data_field.name() == row_id_idx.to_string() { row_id_idx = idx; @@ -214,10 +214,8 @@ impl MergeIntoInterpreter { } } - for (idx, data_field) in join_output_schema.fields().iter().enumerate() { - if exchange.is_some() && data_field.name() == ROW_NUMBER_COL_NAME { - row_number_idx = idx as i32; - } + if exchange.is_some() { + row_number_idx = Some(join_output_schema.index_of(ROW_NUMBER_COL_NAME)?); } // we can't get row_id_idx, throw an exception @@ -227,7 +225,7 @@ impl MergeIntoInterpreter { )); } - if exchange.is_some() && row_number_idx == -1 { + if exchange.is_some() && row_number_idx.is_none() { return Err(ErrorCode::InvalidRowIdIndex( "can't get internal row_number_idx when running merge into", )); @@ -390,7 +388,6 @@ impl MergeIntoInterpreter { output_schema: DataSchemaRef::default(), })) } else { - // let (tx, rx) = mpsc::sync_channel::(0); let merge_append = PhysicalPlan::MergeInto(Box::new(MergeInto { input: Box::new(merge_into_source.clone()), table_info: table_info.clone(), @@ -402,7 +399,7 @@ impl MergeIntoInterpreter { segments, distributed: true, output_schema: DataSchemaRef::new(DataSchema::new(vec![ - join_output_schema.fields[row_number_idx as usize].clone(), + join_output_schema.fields[row_number_idx.unwrap()].clone(), ])), })); diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs index 76b44b1e8308..d478e9c041ef 100644 --- a/src/query/service/src/pipelines/pipeline_builder.rs +++ b/src/query/service/src/pipelines/pipeline_builder.rs @@ -434,7 +434,6 @@ impl PipelineBuilder { self.main_pipeline.add_pipe(builder.finalize()); // 2.fill computed columns - // fill computed columns let table_computed_schema = &tbl.schema().remove_virtual_computed_fields(); let default_schema: DataSchemaRef = Arc::new(table_default_schema.into()); let computed_schema: DataSchemaRef = Arc::new(table_computed_schema.into()); diff --git a/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_matched_and_split.rs b/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_matched_and_split.rs index be012aabc70b..68a168708bdb 100644 --- a/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_matched_and_split.rs +++ b/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_matched_and_split.rs @@ -62,6 +62,8 @@ enum MutationKind { Delete(DeleteDataBlockMutation), } +// if we use hash shuffle join strategy, the enum +// type can't be parser when transform data between nodes. #[derive(serde::Serialize, serde::Deserialize, Clone, Debug, PartialEq)] pub struct MixRowNumberKindAndLog { pub log: Option,