From b564f78055cd682b594d82f032cac0c8b7b284d6 Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Mon, 17 Jul 2023 21:25:50 +0800 Subject: [PATCH 01/40] refactor copy into --- .../src/interpreters/interpreter_copy.rs | 348 +++--------------- .../service/src/pipelines/builders/copy.rs | 96 ++--- .../service/src/pipelines/builders/mod.rs | 1 - .../service/src/pipelines/pipeline_builder.rs | 80 ++-- .../src/schedulers/fragments/fragmenter.rs | 16 +- .../src/schedulers/fragments/plan_fragment.rs | 29 +- src/query/sql/src/executor/format.rs | 23 +- src/query/sql/src/executor/physical_plan.rs | 81 +--- .../sql/src/executor/physical_plan_display.rs | 20 +- .../sql/src/executor/physical_plan_visitor.rs | 49 +-- src/query/sql/src/executor/profile.rs | 6 +- src/query/sql/src/planner/binder/copy.rs | 2 +- src/query/sql/src/planner/plans/copy.rs | 3 +- 13 files changed, 202 insertions(+), 552 deletions(-) diff --git a/src/query/service/src/interpreters/interpreter_copy.rs b/src/query/service/src/interpreters/interpreter_copy.rs index fdebffd14d880..3d0c7cba4e066 100644 --- a/src/query/service/src/interpreters/interpreter_copy.rs +++ b/src/query/service/src/interpreters/interpreter_copy.rs @@ -13,11 +13,9 @@ // limitations under the License. use std::sync::Arc; -use std::time::Instant; use common_catalog::plan::StageTableInfo; use common_catalog::table::AppendMode; -use common_exception::ErrorCode; use common_exception::Result; use common_expression::infer_table_schema; use common_expression::BlockThresholds; @@ -27,12 +25,11 @@ use common_expression::DataSchemaRefExt; use common_meta_app::principal::StageInfo; use common_pipeline_core::Pipeline; use common_sql::executor::table_read_plan::ToReadDataSourcePlan; -use common_sql::executor::CopyIntoTableFromQuery; -use common_sql::executor::DistributedCopyIntoTableFromStage; +use common_sql::executor::CopyIntoTable; +use common_sql::executor::CopyIntoTableSource; use common_sql::executor::Exchange; use common_sql::executor::FragmentKind; use common_sql::executor::PhysicalPlan; -use common_sql::plans::CopyIntoTableMode; use common_sql::plans::CopyIntoTablePlan; use common_storage::StageFileInfo; use common_storage::StageFilesInfo; @@ -43,12 +40,9 @@ use crate::interpreters::common::check_deduplicate_label; use crate::interpreters::Interpreter; use crate::interpreters::SelectInterpreter; use crate::pipelines::builders::build_append2table_with_commit_pipeline; -use crate::pipelines::builders::build_append_data_pipeline; use crate::pipelines::builders::build_commit_data_pipeline; -use crate::pipelines::builders::set_copy_on_finished; -use crate::pipelines::builders::CopyPlanType; use crate::pipelines::PipelineBuildResult; -use crate::schedulers::build_distributed_pipeline; +use crate::schedulers::build_query_pipeline_without_render_result_set; use crate::sessions::QueryContext; use crate::sessions::TableContext; use crate::sql::plans::CopyPlan; @@ -146,86 +140,61 @@ impl CopyInterpreter { } #[async_backtrace::framed] - async fn try_transform_copy_plan_from_local_to_distributed( + async fn build_physical_plan( &self, plan: &CopyIntoTablePlan, - ) -> Result> { - let ctx = self.ctx.clone(); - let to_table = ctx + ) -> Result<(PhysicalPlan, Vec)> { + let to_table = self + .ctx .get_table(&plan.catalog_name, &plan.database_name, &plan.table_name) .await?; - let table_ctx: Arc = self.ctx.clone(); - let files = plan.collect_files(&table_ctx).await?; - if files.is_empty() { - return Ok(None); - } - let mut stage_table_info = plan.stage_table_info.clone(); - stage_table_info.files_to_copy = Some(files.clone()); - let stage_table = StageTable::try_create(stage_table_info.clone())?; - let read_source_plan = { - stage_table - .read_plan_with_catalog( - self.ctx.clone(), - plan.catalog_name.to_string(), - None, - None, - false, - ) - .await? + let files = plan.collect_files(self.ctx.as_ref()).await?; + let source = if let Some(ref query) = plan.query { + let (select_interpreter, _) = self.build_query(query).await?; + CopyIntoTableSource::Query(Box::new(select_interpreter.build_physical_plan().await?)) + } else { + let stage_table_info = StageTableInfo { + files_to_copy: Some(files.clone()), + ..plan.stage_table_info.clone() + }; + let stage_table = StageTable::try_create(stage_table_info)?; + let read_source_plan = Box::new( + stage_table + .read_plan_with_catalog( + self.ctx.clone(), + plan.catalog_name.to_string(), + None, + None, + false, + ) + .await?, + ); + CopyIntoTableSource::Stage(read_source_plan) }; - if read_source_plan.parts.len() <= 1 { - return Ok(None); - } - if plan.query.is_none() { - Ok(Some(CopyPlanType::DistributedCopyIntoTableFromStage( - DistributedCopyIntoTableFromStage { - // TODO(leiysky): we reuse the id of exchange here, - // which is not correct. We should generate a new id for insert. - plan_id: 0, - catalog_name: plan.catalog_name.clone(), - database_name: plan.database_name.clone(), - table_name: plan.table_name.clone(), - required_values_schema: plan.required_values_schema.clone(), - values_consts: plan.values_consts.clone(), - required_source_schema: plan.required_source_schema.clone(), - stage_table_info: plan.stage_table_info.clone(), - source: Box::new(read_source_plan), - files, - table_info: to_table.get_table_info().clone(), - force: plan.force, - write_mode: plan.write_mode, - thresholds: to_table.get_block_thresholds(), - validation_mode: plan.validation_mode.clone(), - }, - ))) - } else { - // plan query must exist, we can use unwarp directly. - let (select_interpreter, _) = self.build_query(plan.query.as_ref().unwrap()).await?; - let plan_query = select_interpreter.build_physical_plan().await?; - Ok(Some(CopyPlanType::CopyIntoTableFromQuery( - CopyIntoTableFromQuery { - // add exchange plan node to enable distributed - // TODO(leiysky): we reuse the id of exchange here, - // which is not correct. We should generate a new id - plan_id: 0, - catalog_name: plan.catalog_name.clone(), - database_name: plan.database_name.clone(), - table_name: plan.table_name.clone(), - required_source_schema: plan.required_source_schema.clone(), - values_consts: plan.values_consts.clone(), - required_values_schema: plan.required_values_schema.clone(), - write_mode: plan.write_mode, - validation_mode: plan.validation_mode.clone(), - force: plan.force, - stage_table_info: plan.stage_table_info.clone(), - local_node_id: self.ctx.get_cluster().local_id.clone(), - input: Box::new(plan_query), - files: plan.collect_files(&table_ctx).await?, - table_info: to_table.get_table_info().clone(), - }, - ))) + let mut root = PhysicalPlan::CopyIntoTable(Box::new(CopyIntoTable { + catalog_name: plan.catalog_name.clone(), + required_values_schema: plan.required_values_schema.clone(), + values_consts: plan.values_consts.clone(), + required_source_schema: plan.required_source_schema.clone(), + stage_table_info: plan.stage_table_info.clone(), + table_info: to_table.get_table_info().clone(), + force: plan.force, + write_mode: plan.write_mode, + validation_mode: plan.validation_mode.clone(), + + files: files.clone(), + source, + })); + if plan.enable_distributed { + root = PhysicalPlan::Exchange(Exchange { + plan_id: 0, + input: Box::new(root), + kind: FragmentKind::Merge, + keys: Vec::new(), + }); } + Ok((root, files)) } #[async_backtrace::framed] @@ -259,195 +228,6 @@ impl CopyInterpreter { Ok(()) } - - /// Build a COPY pipeline in standalone mode. - #[async_backtrace::framed] - async fn build_local_copy_into_table_pipeline( - &self, - plan: &CopyIntoTablePlan, - ) -> Result { - let catalog = plan.catalog_name.as_str(); - let database = plan.database_name.as_str(); - let table = plan.table_name.as_str(); - - let ctx = self.ctx.clone(); - let to_table = ctx.get_table(catalog, database, table).await?; - - let mut build_res; - let source_schema; - let files; - - match &plan.query { - None => { - let table_ctx: Arc = self.ctx.clone(); - - files = plan.collect_files(&table_ctx).await?; - source_schema = plan.required_source_schema.clone(); - build_res = PipelineBuildResult::create(); - if !files.is_empty() { - self.build_read_stage_table_data_pipeline( - &mut build_res.main_pipeline, - plan, - to_table.get_block_thresholds(), - files.clone(), - ) - .await?; - } else { - return Ok(build_res); - } - } - Some(query) => { - files = plan - .stage_table_info - .files_to_copy - .clone() - .ok_or(ErrorCode::Internal("files_to_copy should not be None"))?; - - let (select_interpreter, query_source_schema) = self.build_query(query).await?; - let plan = select_interpreter.build_physical_plan().await?; - build_res = select_interpreter.build_pipeline(plan).await?; - source_schema = query_source_schema; - } - } - - let file_sizes: u64 = files.iter().map(|f| f.size).sum(); - - // Append data. - { - self.set_status(&format!( - "Copy begin to append data: {} files, size_in_bytes:{} into table", - files.len(), - file_sizes - )); - - let start = Instant::now(); - build_append_data_pipeline( - ctx.clone(), - &mut build_res.main_pipeline, - CopyPlanType::CopyIntoTablePlanOption(plan.clone()), - source_schema, - to_table.clone(), - )?; - - // Perf - { - self.set_status(&format!( - "Copy append data finished, cost:{} secs", - start.elapsed().as_secs() - )); - } - } - - // Commit data. - { - // if it's replace mode, don't commit, because COPY is the source of replace. - match plan.write_mode { - CopyIntoTableMode::Replace => set_copy_on_finished( - ctx, - files, - plan.stage_table_info.stage_info.copy_options.purge, - plan.stage_table_info.stage_info.clone(), - &mut build_res.main_pipeline, - )?, - _ => { - // commit. - build_commit_data_pipeline( - ctx.clone(), - &mut build_res.main_pipeline, - plan.stage_table_info.stage_info.clone(), - to_table, - files, - plan.force, - plan.stage_table_info.stage_info.copy_options.purge, - plan.write_mode.is_overwrite(), - )? - } - } - } - - Ok(build_res) - } - - /// Build distributed pipeline from source node id. - #[async_backtrace::framed] - async fn build_cluster_copy_into_table_pipeline( - &self, - distributed_plan: &CopyPlanType, - ) -> Result { - let ( - catalog_name, - database_name, - table_name, - stage_info, - files, - force, - purge, - is_overwrite, - ); - let mut build_res = match distributed_plan { - CopyPlanType::DistributedCopyIntoTableFromStage(plan) => { - catalog_name = plan.catalog_name.clone(); - database_name = plan.database_name.clone(); - table_name = plan.table_name.clone(); - stage_info = plan.stage_table_info.stage_info.clone(); - files = plan.files.clone(); - force = plan.force; - purge = plan.stage_table_info.stage_info.copy_options.purge; - is_overwrite = plan.write_mode.is_overwrite(); - // add exchange plan node to enable distributed - // TODO(leiysky): we reuse the id of exchange here, - // which is not correct. We should generate a new id for insert. - let exchange_plan = PhysicalPlan::Exchange(Exchange { - plan_id: 0, - input: Box::new(PhysicalPlan::DistributedCopyIntoTableFromStage(Box::new( - plan.clone(), - ))), - kind: FragmentKind::Merge, - keys: Vec::new(), - }); - - build_distributed_pipeline(&self.ctx, &exchange_plan, false).await? - } - CopyPlanType::CopyIntoTableFromQuery(plan) => { - catalog_name = plan.catalog_name.clone(); - database_name = plan.database_name.clone(); - table_name = plan.table_name.clone(); - stage_info = plan.stage_table_info.stage_info.clone(); - files = plan.files.clone(); - force = plan.force; - purge = plan.stage_table_info.stage_info.copy_options.purge; - is_overwrite = plan.write_mode.is_overwrite(); - // add exchange plan node to enable distributed - // TODO(leiysky): we reuse the id of exchange here, - // which is not correct. We should generate a new id - let exchange_plan = PhysicalPlan::Exchange(Exchange { - plan_id: 0, - input: Box::new(PhysicalPlan::CopyIntoTableFromQuery(Box::new(plan.clone()))), - kind: FragmentKind::Merge, - keys: Vec::new(), - }); - build_distributed_pipeline(&self.ctx, &exchange_plan, false).await? - } - _ => unreachable!(), - }; - let to_table = self - .ctx - .get_table(&catalog_name, &database_name, &table_name) - .await?; - - // commit. - build_commit_data_pipeline( - self.ctx.clone(), - &mut build_res.main_pipeline, - stage_info, - to_table, - files, - force, - purge, - is_overwrite, - )?; - Ok(build_res) - } } #[async_trait::async_trait] @@ -465,22 +245,16 @@ impl Interpreter for CopyInterpreter { match &self.plan { CopyPlan::IntoTable(plan) => { - if plan.enable_distributed { - let distributed_plan_op = self - .try_transform_copy_plan_from_local_to_distributed(plan) - .await?; - if let Some(distributed_plan) = distributed_plan_op { - let build_res = self - .build_cluster_copy_into_table_pipeline(&distributed_plan) - .await?; - - Ok(build_res) - } else { - self.build_local_copy_into_table_pipeline(plan).await - } - } else { - self.build_local_copy_into_table_pipeline(plan).await - } + let (physical_plan, files) = self.build_physical_plan(plan).await?; + let mut build_res = build_query_pipeline_without_render_result_set( + &self.ctx, + &physical_plan, + false, + ) + .await?; + build_commit_data_pipeline(&self.ctx, &mut build_res.main_pipeline, plan, &files) + .await?; + Ok(build_res) } CopyPlan::IntoStage { stage, from, path, .. diff --git a/src/query/service/src/pipelines/builders/copy.rs b/src/query/service/src/pipelines/builders/copy.rs index d4968bc89b2ee..4bc24dc9e19af 100644 --- a/src/query/service/src/pipelines/builders/copy.rs +++ b/src/query/service/src/pipelines/builders/copy.rs @@ -29,8 +29,7 @@ use common_meta_app::principal::StageInfo; use common_meta_app::schema::TableCopiedFileInfo; use common_meta_app::schema::UpsertTableCopiedFileReq; use common_pipeline_core::Pipeline; -use common_sql::executor::CopyIntoTableFromQuery; -use common_sql::executor::DistributedCopyIntoTableFromStage; +use common_sql::executor::CopyIntoTable; use common_sql::plans::CopyIntoTableMode; use common_sql::plans::CopyIntoTablePlan; use common_storage::common_metrics::copy::metrics_inc_copy_purge_files_cost_milliseconds; @@ -47,48 +46,18 @@ use crate::pipelines::processors::transforms::TransformAddConstColumns; use crate::pipelines::processors::TransformCastSchema; use crate::sessions::QueryContext; -pub enum CopyPlanType { - CopyIntoTablePlanOption(CopyIntoTablePlan), - DistributedCopyIntoTableFromStage(DistributedCopyIntoTableFromStage), - // also distributed plan, but we think the real distributed part is the query - // so no "distributed" prefix here. - CopyIntoTableFromQuery(CopyIntoTableFromQuery), -} - pub fn build_append_data_pipeline( ctx: Arc, main_pipeline: &mut Pipeline, - plan: CopyPlanType, + plan: &CopyIntoTable, source_schema: Arc, to_table: Arc, ) -> Result<()> { - let plan_required_source_schema: DataSchemaRef; - let plan_required_values_schema: DataSchemaRef; - let plan_values_consts: Vec; - let plan_write_mode: CopyIntoTableMode; - - match plan { - CopyPlanType::CopyIntoTablePlanOption(plan) => { - plan_required_source_schema = plan.required_source_schema; - plan_required_values_schema = plan.required_values_schema; - plan_values_consts = plan.values_consts; - plan_write_mode = plan.write_mode; - } - CopyPlanType::DistributedCopyIntoTableFromStage(plan) => { - plan_required_source_schema = plan.required_source_schema; - plan_required_values_schema = plan.required_values_schema; - plan_values_consts = plan.values_consts; - plan_write_mode = plan.write_mode; - } - CopyPlanType::CopyIntoTableFromQuery(plan) => { - plan_required_source_schema = plan.required_source_schema; - plan_required_values_schema = plan.required_values_schema; - plan_values_consts = plan.values_consts; - plan_write_mode = plan.write_mode; - } - } - - if source_schema != plan_required_source_schema { + let plan_required_source_schema = &plan.required_source_schema; + let plan_values_consts = &plan.values_consts; + let plan_required_values_schema = &plan.required_values_schema; + let plan_write_mode = &plan.write_mode; + if &source_schema != plan_required_source_schema { // only parquet need cast let func_ctx = ctx.get_function_context()?; main_pipeline.add_transform(|transform_input_port, transform_output_port| { @@ -118,7 +87,7 @@ pub fn build_append_data_pipeline( ctx, main_pipeline, to_table.clone(), - plan_required_values_schema, + plan_required_values_schema.clone(), AppendMode::Copy, )?, CopyIntoTableMode::Replace => {} @@ -126,48 +95,51 @@ pub fn build_append_data_pipeline( ctx, main_pipeline, to_table.clone(), - plan_required_values_schema, + plan_required_values_schema.clone(), AppendMode::Copy, )?, } Ok(()) } -#[allow(clippy::too_many_arguments)] -pub fn build_commit_data_pipeline( - ctx: Arc, +pub async fn build_commit_data_pipeline( + ctx: &Arc, main_pipeline: &mut Pipeline, - stage_info: StageInfo, - to_table: Arc, - files: Vec, - copy_force_option: bool, - copy_purge_option: bool, - insert_overwrite_option: bool, + plan: &CopyIntoTablePlan, + files: &[StageFileInfo], ) -> Result<()> { + let to_table = ctx + .get_table(&plan.catalog_name, &plan.database_name, &plan.table_name) + .await?; // Source node will do: // 1. commit // 2. purge // commit let copied_files_meta_req = build_upsert_copied_files_to_meta_req( ctx.clone(), - to_table.clone(), - stage_info.clone(), - files.clone(), - copy_force_option, + to_table.as_ref(), + &plan.stage_table_info.stage_info, + files, + plan.force, )?; to_table.commit_insertion( ctx.clone(), main_pipeline, copied_files_meta_req, - insert_overwrite_option, + plan.write_mode.is_overwrite(), )?; // set on_finished callback. - set_copy_on_finished(ctx, files, copy_purge_option, stage_info, main_pipeline)?; + set_copy_on_finished( + ctx.clone(), + files.to_vec(), + plan.stage_table_info.stage_info.copy_options.purge, + plan.stage_table_info.stage_info.clone(), + main_pipeline, + )?; Ok(()) } - pub fn set_copy_on_finished( ctx: Arc, files: Vec, @@ -222,13 +194,13 @@ pub fn set_copy_on_finished( pub fn build_upsert_copied_files_to_meta_req( ctx: Arc, - to_table: Arc, - stage_info: StageInfo, - copied_files: Vec, + to_table: &dyn Table, + stage_info: &StageInfo, + copied_files: &[StageFileInfo], force: bool, ) -> Result> { let mut copied_file_tree = BTreeMap::new(); - for file in &copied_files { + for file in copied_files { // Short the etag to 7 bytes for less space in metasrv. let short_etag = file.etag.clone().map(|mut v| { v.truncate(7); @@ -274,7 +246,7 @@ fn fill_const_columns( pipeline: &mut Pipeline, input_schema: DataSchemaRef, output_schema: DataSchemaRef, - const_values: Vec, + const_values: &[Scalar], ) -> Result<()> { pipeline.add_transform(|transform_input_port, transform_output_port| { TransformAddConstColumns::try_create( @@ -283,7 +255,7 @@ fn fill_const_columns( transform_output_port, input_schema.clone(), output_schema.clone(), - const_values.clone(), + const_values.to_vec(), ) })?; Ok(()) diff --git a/src/query/service/src/pipelines/builders/mod.rs b/src/query/service/src/pipelines/builders/mod.rs index 0829d266f1507..36bc0cb58a1b7 100644 --- a/src/query/service/src/pipelines/builders/mod.rs +++ b/src/query/service/src/pipelines/builders/mod.rs @@ -19,7 +19,6 @@ pub use copy::build_append_data_pipeline; pub use copy::build_commit_data_pipeline; pub use copy::build_upsert_copied_files_to_meta_req; pub use copy::set_copy_on_finished; -pub use copy::CopyPlanType; pub use table::build_append2table_with_commit_pipeline; pub use table::build_append2table_without_commit_pipeline; pub use table::build_fill_missing_columns_pipeline; diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs index 1d555b747d4ff..d37293ec2d591 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::convert::TryFrom; use std::sync::Arc; +use std::time::Instant; use async_channel::Receiver; use common_catalog::table::AppendMode; @@ -54,10 +55,10 @@ use common_sql::executor::AggregateExpand; use common_sql::executor::AggregateFinal; use common_sql::executor::AggregateFunctionDesc; use common_sql::executor::AggregatePartial; -use common_sql::executor::CopyIntoTableFromQuery; +use common_sql::executor::CopyIntoTable; +use common_sql::executor::CopyIntoTableSource; use common_sql::executor::DeleteFinal; use common_sql::executor::DeletePartial; -use common_sql::executor::DistributedCopyIntoTableFromStage; use common_sql::executor::DistributedInsertSelect; use common_sql::executor::EvalScalar; use common_sql::executor::ExchangeSink; @@ -86,6 +87,7 @@ use common_storages_fuse::operations::TransformSerializeBlock; use common_storages_fuse::FuseTable; use common_storages_stage::StageTable; use petgraph::matrix_graph::Zero; +use tracing::info; use super::processors::transforms::FrameBound; use super::processors::transforms::WindowFunctionInfo; @@ -94,7 +96,6 @@ use crate::api::DefaultExchangeInjector; use crate::api::ExchangeInjector; use crate::pipelines::builders::build_append_data_pipeline; use crate::pipelines::builders::build_fill_missing_columns_pipeline; -use crate::pipelines::builders::CopyPlanType; use crate::pipelines::processors::transforms::build_partition_bucket; use crate::pipelines::processors::transforms::AggregateInjector; use crate::pipelines::processors::transforms::FinalSingleStateAggregator; @@ -176,6 +177,11 @@ impl PipelineBuilder { }) } + fn set_status(&self, status: &str) { + self.ctx.set_status_info(status); + info!(status); + } + fn build_pipeline(&mut self, plan: &PhysicalPlan) -> Result<()> { match plan { PhysicalPlan::TableScan(scan) => self.build_table_scan(scan), @@ -206,55 +212,41 @@ impl PipelineBuilder { PhysicalPlan::DeletePartial(delete) => self.build_delete_partial(delete), PhysicalPlan::DeleteFinal(delete) => self.build_delete_final(delete), PhysicalPlan::RangeJoin(range_join) => self.build_range_join(range_join), - PhysicalPlan::DistributedCopyIntoTableFromStage(distributed_plan) => { - self.build_distributed_copy_into_table_from_stage(distributed_plan) - } - PhysicalPlan::CopyIntoTableFromQuery(copy_plan) => { - self.build_copy_into_table_from_query(copy_plan) - } + PhysicalPlan::CopyIntoTable(copy) => self.build_copy_into_table(copy), } } - fn build_copy_into_table_from_query( - &mut self, - copy_plan: &CopyIntoTableFromQuery, - ) -> Result<()> { - self.build_pipeline(©_plan.input)?; - let catalog = self.ctx.get_catalog(©_plan.catalog_name)?; - let to_table = catalog.get_table_by_info(©_plan.table_info)?; + fn build_copy_into_table(&mut self, copy: &CopyIntoTable) -> Result<()> { + let catalog = self.ctx.get_catalog(©.catalog_name)?; + let to_table = catalog.get_table_by_info(©.table_info)?; + let file_sizes: u64 = copy.files.iter().map(|f| f.size).sum(); + match ©.source { + CopyIntoTableSource::Query(input) => { + self.build_pipeline(input)?; + } + CopyIntoTableSource::Stage(source) => { + let stage_table = StageTable::try_create(copy.stage_table_info.clone())?; + stage_table.set_block_thresholds(to_table.get_block_thresholds()); + stage_table.read_data(self.ctx.clone(), source, &mut self.main_pipeline)?; + } + } + self.set_status(&format!( + "Copy begin to append data: {} files, size_in_bytes:{} into table", + copy.files.len(), + file_sizes + )); + let start = Instant::now(); build_append_data_pipeline( self.ctx.clone(), &mut self.main_pipeline, - CopyPlanType::CopyIntoTableFromQuery(copy_plan.clone()), - copy_plan.required_source_schema.clone(), - to_table, - )?; - Ok(()) - } - - fn build_distributed_copy_into_table_from_stage( - &mut self, - distributed_plan: &DistributedCopyIntoTableFromStage, - ) -> Result<()> { - let catalog = self.ctx.get_catalog(&distributed_plan.catalog_name)?; - let to_table = catalog.get_table_by_info(&distributed_plan.table_info)?; - let stage_table_info = distributed_plan.stage_table_info.clone(); - let stage_table = StageTable::try_create(stage_table_info)?; - stage_table.set_block_thresholds(distributed_plan.thresholds); - let ctx = self.ctx.clone(); - let table_ctx: Arc = ctx.clone(); - - stage_table.read_data(table_ctx, &distributed_plan.source, &mut self.main_pipeline)?; - - // append data - build_append_data_pipeline( - ctx, - &mut self.main_pipeline, - CopyPlanType::DistributedCopyIntoTableFromStage(distributed_plan.clone()), - distributed_plan.required_source_schema.clone(), + copy, + copy.required_source_schema.clone(), to_table, )?; - + self.set_status(&format!( + "Copy append data finished, cost:{} secs", + start.elapsed().as_secs() + )); Ok(()) } diff --git a/src/query/service/src/schedulers/fragments/fragmenter.rs b/src/query/service/src/schedulers/fragments/fragmenter.rs index 09de1b6ce6b8b..97e38f0feb575 100644 --- a/src/query/service/src/schedulers/fragments/fragmenter.rs +++ b/src/query/service/src/schedulers/fragments/fragmenter.rs @@ -16,7 +16,8 @@ use std::sync::Arc; use common_catalog::table_context::TableContext; use common_exception::Result; -use common_sql::executor::DistributedCopyIntoTableFromStage; +use common_sql::executor::CopyIntoTable; +use common_sql::executor::CopyIntoTableSource; use common_sql::executor::FragmentKind; use crate::api::BroadcastExchange; @@ -139,14 +140,11 @@ impl PhysicalPlanReplacer for Fragmenter { Ok(PhysicalPlan::TableScan(plan.clone())) } - fn replace_copy_into_table( - &mut self, - plan: &DistributedCopyIntoTableFromStage, - ) -> Result { - self.state = State::SelectLeaf; - Ok(PhysicalPlan::DistributedCopyIntoTableFromStage(Box::new( - plan.clone(), - ))) + fn replace_copy_into_table(&mut self, plan: &CopyIntoTable) -> Result { + if let CopyIntoTableSource::Stage(_) = plan.source { + self.state = State::SelectLeaf; + } + Ok(PhysicalPlan::CopyIntoTable(Box::new(plan.clone()))) } fn replace_delete_partial( diff --git a/src/query/service/src/schedulers/fragments/plan_fragment.rs b/src/query/service/src/schedulers/fragments/plan_fragment.rs index 5094c26197cab..ad004acaaa22d 100644 --- a/src/query/service/src/schedulers/fragments/plan_fragment.rs +++ b/src/query/service/src/schedulers/fragments/plan_fragment.rs @@ -18,8 +18,9 @@ use common_catalog::plan::DataSourcePlan; use common_catalog::plan::Partitions; use common_exception::ErrorCode; use common_exception::Result; +use common_sql::executor::CopyIntoTable; +use common_sql::executor::CopyIntoTableSource; use common_sql::executor::DeletePartial; -use common_sql::executor::DistributedCopyIntoTableFromStage; use crate::api::DataExchange; use crate::schedulers::Fragmenter; @@ -203,12 +204,13 @@ impl PlanFragment { let mut source = vec![]; - let mut collect_read_source = |plan: &PhysicalPlan| { - if let PhysicalPlan::TableScan(scan) = plan { - source.push(*scan.source.clone()) - } else if let PhysicalPlan::DistributedCopyIntoTableFromStage(distributed_plan) = plan { - source.push(*distributed_plan.source.clone()) + let mut collect_read_source = |plan: &PhysicalPlan| match plan { + PhysicalPlan::TableScan(scan) => source.push(*scan.source.clone()), + PhysicalPlan::CopyIntoTable(copy) => { + // Safe to unwrap because we have checked the fragment type. + source.push(*copy.source.as_stage().cloned().unwrap()) } + _ => unreachable!("possibly you add new source fragment but forget to handle it here"), }; PhysicalPlan::traverse( @@ -244,16 +246,11 @@ impl PhysicalPlanReplacer for ReplaceReadSource { })) } - fn replace_copy_into_table( - &mut self, - plan: &DistributedCopyIntoTableFromStage, - ) -> Result { - Ok(PhysicalPlan::DistributedCopyIntoTableFromStage(Box::new( - DistributedCopyIntoTableFromStage { - source: Box::new(self.source.clone()), - ..plan.clone() - }, - ))) + fn replace_copy_into_table(&mut self, plan: &CopyIntoTable) -> Result { + Ok(PhysicalPlan::CopyIntoTable(Box::new(CopyIntoTable { + source: CopyIntoTableSource::Stage(Box::new(self.source.clone())), + ..plan.clone() + }))) } } diff --git a/src/query/sql/src/executor/format.rs b/src/query/sql/src/executor/format.rs index 74b2dd554c2d0..38ba9864aae16 100644 --- a/src/query/sql/src/executor/format.rs +++ b/src/query/sql/src/executor/format.rs @@ -23,10 +23,9 @@ use super::AggregateExpand; use super::AggregateFinal; use super::AggregateFunctionDesc; use super::AggregatePartial; -use super::CopyIntoTableFromQuery; +use super::CopyIntoTable; use super::DeleteFinal; use super::DeletePartial; -use super::DistributedCopyIntoTableFromStage; use super::EvalScalar; use super::Exchange; use super::Filter; @@ -173,10 +172,7 @@ fn to_format_tree( runtime_filter_source_to_format_tree(plan, metadata, prof_span_set) } PhysicalPlan::RangeJoin(plan) => range_join_to_format_tree(plan, metadata, prof_span_set), - PhysicalPlan::DistributedCopyIntoTableFromStage(plan) => { - distributed_copy_into_table_from_stage(plan) - } - PhysicalPlan::CopyIntoTableFromQuery(plan) => copy_into_table_from_query(plan), + PhysicalPlan::CopyIntoTable(plan) => copy_into_table(plan), } } @@ -198,19 +194,10 @@ fn append_profile_info( } } -fn distributed_copy_into_table_from_stage( - plan: &DistributedCopyIntoTableFromStage, -) -> Result> { - Ok(FormatTreeNode::new(format!( - "copy into table {}.{}.{} from {:?}", - plan.catalog_name, plan.database_name, plan.table_name, plan.source - ))) -} - -fn copy_into_table_from_query(plan: &CopyIntoTableFromQuery) -> Result> { +fn copy_into_table(plan: &CopyIntoTable) -> Result> { Ok(FormatTreeNode::new(format!( - "copy into table {}.{}.{} from {:?}", - plan.catalog_name, plan.database_name, plan.table_name, plan.input + "CopyIntoTable: {}", + plan.table_info ))) } diff --git a/src/query/sql/src/executor/physical_plan.rs b/src/query/sql/src/executor/physical_plan.rs index 2191a352c9738..417b9ab563531 100644 --- a/src/query/sql/src/executor/physical_plan.rs +++ b/src/query/sql/src/executor/physical_plan.rs @@ -25,7 +25,6 @@ use common_catalog::plan::StageTableInfo; use common_exception::Result; use common_expression::types::DataType; use common_expression::types::NumberDataType; -use common_expression::BlockThresholds; use common_expression::DataBlock; use common_expression::DataField; use common_expression::DataSchemaRef; @@ -38,6 +37,7 @@ use common_functions::aggregates::AggregateFunctionFactory; use common_functions::BUILTIN_FUNCTIONS; use common_meta_app::schema::TableInfo; use common_storage::StageFileInfo; +use enum_as_inner::EnumAsInner; use storages_common_table_meta::meta::TableSnapshot; use crate::executor::explain::PlanStatsInfo; @@ -702,60 +702,28 @@ impl UnionAll { } #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct DistributedCopyIntoTableFromStage { - pub plan_id: u32, +pub struct CopyIntoTable { pub catalog_name: String, - pub database_name: String, - pub table_name: String, - // ... into table() .. -> pub required_values_schema: DataSchemaRef, - // (1, ?, 'a', ?) -> (1, 'a') pub values_consts: Vec, - // (1, ?, 'a', ?) -> (?, ?) pub required_source_schema: DataSchemaRef, - pub write_mode: CopyIntoTableMode, pub validation_mode: ValidationMode, pub force: bool, - pub stage_table_info: StageTableInfo, - pub source: Box, - - pub thresholds: BlockThresholds, pub files: Vec, pub table_info: TableInfo, -} -impl DistributedCopyIntoTableFromStage { - pub fn output_schema(&self) -> Result { - Ok(DataSchemaRef::default()) - } + pub source: CopyIntoTableSource, } -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct CopyIntoTableFromQuery { - pub plan_id: u32, - pub catalog_name: String, - pub database_name: String, - pub table_name: String, - - pub required_values_schema: DataSchemaRef, // ... into table() .. -> - pub values_consts: Vec, // (1, ?, 'a', ?) -> (1, 'a') - pub required_source_schema: DataSchemaRef, // (1, ?, 'a', ?) -> (?, ?) - - pub write_mode: CopyIntoTableMode, - pub validation_mode: ValidationMode, - pub force: bool, - - pub stage_table_info: StageTableInfo, - pub local_node_id: String, - // after build_query, we will make it as the input - pub input: Box, - pub files: Vec, - pub table_info: TableInfo, +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize, EnumAsInner)] +pub enum CopyIntoTableSource { + Query(Box), + Stage(Box), } -impl CopyIntoTableFromQuery { +impl CopyIntoTable { pub fn output_schema(&self) -> Result { Ok(DataSchemaRef::default()) } @@ -856,7 +824,7 @@ impl Display for RefreshIndex { } } -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize, EnumAsInner)] pub enum PhysicalPlan { TableScan(TableScan), Filter(Filter), @@ -878,17 +846,13 @@ pub enum PhysicalPlan { /// For insert into ... select ... in cluster DistributedInsertSelect(Box), - /// add distributed copy into table from @stage - DistributedCopyIntoTableFromStage(Box), - /// /// add distributed copy into table from query - CopyIntoTableFromQuery(Box), /// Synthesized by fragmenter ExchangeSource(ExchangeSource), ExchangeSink(ExchangeSink), - /// For distributed delete DeletePartial(Box), DeleteFinal(Box), + CopyIntoTable(Box), } impl PhysicalPlan { @@ -923,10 +887,11 @@ impl PhysicalPlan { PhysicalPlan::DistributedInsertSelect(v) => v.plan_id, PhysicalPlan::ExchangeSource(v) => v.plan_id, PhysicalPlan::ExchangeSink(v) => v.plan_id, - PhysicalPlan::DeletePartial(_) | PhysicalPlan::DeleteFinal(_) => unreachable!(), - // for distributed_copy_into_table, planId is useless - PhysicalPlan::DistributedCopyIntoTableFromStage(v) => v.plan_id, - PhysicalPlan::CopyIntoTableFromQuery(v) => v.plan_id, + PhysicalPlan::DeletePartial(_) + | PhysicalPlan::DeleteFinal(_) + | PhysicalPlan::CopyIntoTable(_) => { + unreachable!() + } } } @@ -954,8 +919,7 @@ impl PhysicalPlan { PhysicalPlan::DeletePartial(plan) => plan.output_schema(), PhysicalPlan::DeleteFinal(plan) => plan.output_schema(), PhysicalPlan::RangeJoin(plan) => plan.output_schema(), - PhysicalPlan::DistributedCopyIntoTableFromStage(plan) => plan.output_schema(), - PhysicalPlan::CopyIntoTableFromQuery(plan) => plan.output_schema(), + PhysicalPlan::CopyIntoTable(plan) => plan.output_schema(), } } @@ -983,10 +947,7 @@ impl PhysicalPlan { PhysicalPlan::DeletePartial(_) => "DeletePartial".to_string(), PhysicalPlan::DeleteFinal(_) => "DeleteFinal".to_string(), PhysicalPlan::RangeJoin(_) => "RangeJoin".to_string(), - PhysicalPlan::DistributedCopyIntoTableFromStage(_) => { - "DistributedCopyIntoTableFromStage".to_string() - } - PhysicalPlan::CopyIntoTableFromQuery(_) => "CopyIntoTableFromQuery".to_string(), + PhysicalPlan::CopyIntoTable(_) => "DistributedCopyIntoTable".to_string(), } } @@ -1025,8 +986,7 @@ impl PhysicalPlan { PhysicalPlan::RangeJoin(plan) => Box::new( std::iter::once(plan.left.as_ref()).chain(std::iter::once(plan.right.as_ref())), ), - PhysicalPlan::DistributedCopyIntoTableFromStage(_) => Box::new(std::iter::empty()), - PhysicalPlan::CopyIntoTableFromQuery(_) => Box::new(std::iter::empty()), + PhysicalPlan::CopyIntoTable(_) => Box::new(std::iter::empty()), } } @@ -1045,8 +1005,6 @@ impl PhysicalPlan { PhysicalPlan::DistributedInsertSelect(plan) => plan.input.try_find_single_data_source(), PhysicalPlan::ProjectSet(plan) => plan.input.try_find_single_data_source(), PhysicalPlan::RowFetch(plan) => plan.input.try_find_single_data_source(), - PhysicalPlan::CopyIntoTableFromQuery(plan) => plan.input.try_find_single_data_source(), - PhysicalPlan::DistributedCopyIntoTableFromStage(plan) => Some(&plan.source), PhysicalPlan::RuntimeFilterSource(_) | PhysicalPlan::UnionAll(_) | PhysicalPlan::ExchangeSource(_) @@ -1056,7 +1014,8 @@ impl PhysicalPlan { | PhysicalPlan::AggregateFinal(_) | PhysicalPlan::AggregatePartial(_) | PhysicalPlan::DeletePartial(_) - | PhysicalPlan::DeleteFinal(_) => None, + | PhysicalPlan::DeleteFinal(_) + | PhysicalPlan::CopyIntoTable(_) => None, } } } diff --git a/src/query/sql/src/executor/physical_plan_display.rs b/src/query/sql/src/executor/physical_plan_display.rs index cd89d2b81a73e..988b7ad81422b 100644 --- a/src/query/sql/src/executor/physical_plan_display.rs +++ b/src/query/sql/src/executor/physical_plan_display.rs @@ -19,10 +19,9 @@ use common_functions::BUILTIN_FUNCTIONS; use itertools::Itertools; use super::AggregateExpand; -use super::CopyIntoTableFromQuery; +use super::CopyIntoTable; use super::DeleteFinal; use super::DeletePartial; -use super::DistributedCopyIntoTableFromStage; use super::DistributedInsertSelect; use super::ProjectSet; use super::RowFetch; @@ -83,12 +82,7 @@ impl<'a> Display for PhysicalPlanIndentFormatDisplay<'a> { PhysicalPlan::ProjectSet(unnest) => write!(f, "{}", unnest)?, PhysicalPlan::RuntimeFilterSource(plan) => write!(f, "{}", plan)?, PhysicalPlan::RangeJoin(plan) => write!(f, "{}", plan)?, - PhysicalPlan::DistributedCopyIntoTableFromStage(copy_into_table_from_stage) => { - write!(f, "{}", copy_into_table_from_stage)? - } - PhysicalPlan::CopyIntoTableFromQuery(copy_into_table_from_query) => { - write!(f, "{}", copy_into_table_from_query)? - } + PhysicalPlan::CopyIntoTable(copy_into_table) => write!(f, "{}", copy_into_table)?, } for node in self.node.children() { @@ -369,15 +363,9 @@ impl Display for DeleteFinal { write!(f, "DeleteFinal") } } -impl Display for DistributedCopyIntoTableFromStage { - fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { - write!(f, "DistributedCopyIntoTableFromStage") - } -} - -impl Display for CopyIntoTableFromQuery { +impl Display for CopyIntoTable { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { - write!(f, "CopyIntoTableFromQuery") + write!(f, "CopyIntoTable") } } diff --git a/src/query/sql/src/executor/physical_plan_visitor.rs b/src/query/sql/src/executor/physical_plan_visitor.rs index 26c1145cb41ab..ad0a7a980c079 100644 --- a/src/query/sql/src/executor/physical_plan_visitor.rs +++ b/src/query/sql/src/executor/physical_plan_visitor.rs @@ -17,10 +17,10 @@ use common_exception::Result; use super::AggregateExpand; use super::AggregateFinal; use super::AggregatePartial; -use super::CopyIntoTableFromQuery; +use super::CopyIntoTable; +use super::CopyIntoTableSource; use super::DeleteFinal; use super::DeletePartial; -use super::DistributedCopyIntoTableFromStage; use super::DistributedInsertSelect; use super::EvalScalar; use super::Exchange; @@ -65,12 +65,7 @@ pub trait PhysicalPlanReplacer { PhysicalPlan::DeletePartial(plan) => self.replace_delete_partial(plan), PhysicalPlan::DeleteFinal(plan) => self.replace_delete_final(plan), PhysicalPlan::RangeJoin(plan) => self.replace_range_join(plan), - PhysicalPlan::DistributedCopyIntoTableFromStage(plan) => { - self.replace_copy_into_table(plan) - } - PhysicalPlan::CopyIntoTableFromQuery(plan) => { - self.replace_copy_into_table_from_query(plan) - } + PhysicalPlan::CopyIntoTable(plan) => self.replace_copy_into_table(plan), } } @@ -285,26 +280,19 @@ pub trait PhysicalPlanReplacer { })) } - fn replace_copy_into_table( - &mut self, - plan: &DistributedCopyIntoTableFromStage, - ) -> Result { - Ok(PhysicalPlan::DistributedCopyIntoTableFromStage(Box::new( - plan.clone(), - ))) - } - - fn replace_copy_into_table_from_query( - &mut self, - plan: &CopyIntoTableFromQuery, - ) -> Result { - let input = self.replace(&plan.input)?; - Ok(PhysicalPlan::CopyIntoTableFromQuery(Box::new( - CopyIntoTableFromQuery { - input: Box::new(input), - ..plan.clone() - }, - ))) + fn replace_copy_into_table(&mut self, plan: &CopyIntoTable) -> Result { + match &plan.source { + CopyIntoTableSource::Stage(_) => { + Ok(PhysicalPlan::CopyIntoTable(Box::new(plan.clone()))) + } + CopyIntoTableSource::Query(input) => { + let input = self.replace(input)?; + Ok(PhysicalPlan::CopyIntoTable(Box::new(CopyIntoTable { + source: CopyIntoTableSource::Query(Box::new(input)), + ..plan.clone() + }))) + } + } } fn replace_insert_select(&mut self, plan: &DistributedInsertSelect) -> Result { @@ -425,10 +413,7 @@ impl PhysicalPlan { PhysicalPlan::ProjectSet(plan) => { Self::traverse(&plan.input, pre_visit, visit, post_visit) } - PhysicalPlan::DistributedCopyIntoTableFromStage(_) => {} - PhysicalPlan::CopyIntoTableFromQuery(plan) => { - Self::traverse(&plan.input, pre_visit, visit, post_visit); - } + PhysicalPlan::CopyIntoTable(_) => {} PhysicalPlan::RuntimeFilterSource(plan) => { Self::traverse(&plan.left_side, pre_visit, visit, post_visit); Self::traverse(&plan.right_side, pre_visit, visit, post_visit); diff --git a/src/query/sql/src/executor/profile.rs b/src/query/sql/src/executor/profile.rs index 905676db1b8a9..fe2df7dd16c9c 100644 --- a/src/query/sql/src/executor/profile.rs +++ b/src/query/sql/src/executor/profile.rs @@ -450,9 +450,9 @@ fn flatten_plan_node_profile( }; plan_node_profs.push(prof); } - PhysicalPlan::DeletePartial(_) | PhysicalPlan::DeleteFinal(_) => unreachable!(), - PhysicalPlan::DistributedCopyIntoTableFromStage(_) => unreachable!(), - PhysicalPlan::CopyIntoTableFromQuery(_) => unreachable!(), + PhysicalPlan::DeletePartial(_) + | PhysicalPlan::DeleteFinal(_) + | PhysicalPlan::CopyIntoTable(_) => unreachable!(), } Ok(()) diff --git a/src/query/sql/src/planner/binder/copy.rs b/src/query/sql/src/planner/binder/copy.rs index 45ba474235bb3..7cfbdf5fbace4 100644 --- a/src/query/sql/src/planner/binder/copy.rs +++ b/src/query/sql/src/planner/binder/copy.rs @@ -660,7 +660,7 @@ impl<'a> Binder { select_list: &'a [SelectTarget], alias: &Option, ) -> Result { - let need_copy_file_infos = plan.collect_files(&self.ctx).await?; + let need_copy_file_infos = plan.collect_files(self.ctx.as_ref()).await?; if need_copy_file_infos.is_empty() { return Ok(Plan::Copy(Box::new(CopyPlan::NoFileToCopy))); diff --git a/src/query/sql/src/planner/plans/copy.rs b/src/query/sql/src/planner/plans/copy.rs index 012fae2c5b52a..0ebc2e066eaf2 100644 --- a/src/query/sql/src/planner/plans/copy.rs +++ b/src/query/sql/src/planner/plans/copy.rs @@ -15,7 +15,6 @@ use std::fmt::Debug; use std::fmt::Formatter; use std::str::FromStr; -use std::sync::Arc; use std::time::Instant; use common_catalog::plan::StageTableInfo; @@ -97,7 +96,7 @@ pub struct CopyIntoTablePlan { } impl CopyIntoTablePlan { - pub async fn collect_files(&self, ctx: &Arc) -> Result> { + pub async fn collect_files(&self, ctx: &dyn TableContext) -> Result> { ctx.set_status_info("begin to list files"); let start = Instant::now(); From 38b1843ef9012c0968023b7cd4956035f0747f96 Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Mon, 17 Jul 2023 21:50:33 +0800 Subject: [PATCH 02/40] fix panic --- .../service/src/schedulers/fragments/plan_fragment.rs | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/query/service/src/schedulers/fragments/plan_fragment.rs b/src/query/service/src/schedulers/fragments/plan_fragment.rs index ad004acaaa22d..3b1abed923fce 100644 --- a/src/query/service/src/schedulers/fragments/plan_fragment.rs +++ b/src/query/service/src/schedulers/fragments/plan_fragment.rs @@ -207,10 +207,11 @@ impl PlanFragment { let mut collect_read_source = |plan: &PhysicalPlan| match plan { PhysicalPlan::TableScan(scan) => source.push(*scan.source.clone()), PhysicalPlan::CopyIntoTable(copy) => { - // Safe to unwrap because we have checked the fragment type. - source.push(*copy.source.as_stage().cloned().unwrap()) + if let Some(stage) = copy.source.as_stage().cloned() { + source.push(stage); + } } - _ => unreachable!("possibly you add new source fragment but forget to handle it here"), + _ => {} }; PhysicalPlan::traverse( From ef1c6e3ac3f925f7da37162529af9411a1e7b62e Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Mon, 17 Jul 2023 22:17:08 +0800 Subject: [PATCH 03/40] fix --- src/query/service/src/schedulers/fragments/plan_fragment.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/query/service/src/schedulers/fragments/plan_fragment.rs b/src/query/service/src/schedulers/fragments/plan_fragment.rs index 3b1abed923fce..8b3aa30d8762a 100644 --- a/src/query/service/src/schedulers/fragments/plan_fragment.rs +++ b/src/query/service/src/schedulers/fragments/plan_fragment.rs @@ -208,7 +208,7 @@ impl PlanFragment { PhysicalPlan::TableScan(scan) => source.push(*scan.source.clone()), PhysicalPlan::CopyIntoTable(copy) => { if let Some(stage) = copy.source.as_stage().cloned() { - source.push(stage); + source.push(*stage); } } _ => {} From 6e478224bad99b013b7d4dfa3f487708b8320942 Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Mon, 17 Jul 2023 23:11:01 +0800 Subject: [PATCH 04/40] fix --- .../src/schedulers/fragments/plan_fragment.rs | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/src/query/service/src/schedulers/fragments/plan_fragment.rs b/src/query/service/src/schedulers/fragments/plan_fragment.rs index 8b3aa30d8762a..a95d4fec9aa82 100644 --- a/src/query/service/src/schedulers/fragments/plan_fragment.rs +++ b/src/query/service/src/schedulers/fragments/plan_fragment.rs @@ -248,10 +248,17 @@ impl PhysicalPlanReplacer for ReplaceReadSource { } fn replace_copy_into_table(&mut self, plan: &CopyIntoTable) -> Result { - Ok(PhysicalPlan::CopyIntoTable(Box::new(CopyIntoTable { - source: CopyIntoTableSource::Stage(Box::new(self.source.clone())), - ..plan.clone() - }))) + match &plan.source { + CopyIntoTableSource::Query(_) => { + Ok(PhysicalPlan::CopyIntoTable(Box::new(plan.clone()))) + } + CopyIntoTableSource::Stage(_) => { + Ok(PhysicalPlan::CopyIntoTable(Box::new(CopyIntoTable { + source: CopyIntoTableSource::Stage(Box::new(self.source.clone())), + ..plan.clone() + }))) + } + } } } From aad10694b3ae0f63774c7b4bb8e43eba3b68344b Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Mon, 17 Jul 2023 23:31:31 +0800 Subject: [PATCH 05/40] fix --- src/query/sql/src/executor/physical_plan.rs | 2 +- src/query/sql/src/executor/physical_plan_visitor.rs | 7 ++++++- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/src/query/sql/src/executor/physical_plan.rs b/src/query/sql/src/executor/physical_plan.rs index 417b9ab563531..f7ba5b01f60b2 100644 --- a/src/query/sql/src/executor/physical_plan.rs +++ b/src/query/sql/src/executor/physical_plan.rs @@ -947,7 +947,7 @@ impl PhysicalPlan { PhysicalPlan::DeletePartial(_) => "DeletePartial".to_string(), PhysicalPlan::DeleteFinal(_) => "DeleteFinal".to_string(), PhysicalPlan::RangeJoin(_) => "RangeJoin".to_string(), - PhysicalPlan::CopyIntoTable(_) => "DistributedCopyIntoTable".to_string(), + PhysicalPlan::CopyIntoTable(_) => "CopyIntoTable".to_string(), } } diff --git a/src/query/sql/src/executor/physical_plan_visitor.rs b/src/query/sql/src/executor/physical_plan_visitor.rs index ad0a7a980c079..8f1d5725e6697 100644 --- a/src/query/sql/src/executor/physical_plan_visitor.rs +++ b/src/query/sql/src/executor/physical_plan_visitor.rs @@ -413,7 +413,12 @@ impl PhysicalPlan { PhysicalPlan::ProjectSet(plan) => { Self::traverse(&plan.input, pre_visit, visit, post_visit) } - PhysicalPlan::CopyIntoTable(_) => {} + PhysicalPlan::CopyIntoTable(plan) => match &plan.source { + CopyIntoTableSource::Query(input) => { + Self::traverse(&input, pre_visit, visit, post_visit); + } + CopyIntoTableSource::Stage(_) => {} + }, PhysicalPlan::RuntimeFilterSource(plan) => { Self::traverse(&plan.left_side, pre_visit, visit, post_visit); Self::traverse(&plan.right_side, pre_visit, visit, post_visit); From 89a74fad362afe551d046f92d88923fec4aaece9 Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Mon, 17 Jul 2023 23:45:19 +0800 Subject: [PATCH 06/40] make lint --- src/query/sql/src/executor/physical_plan_visitor.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/query/sql/src/executor/physical_plan_visitor.rs b/src/query/sql/src/executor/physical_plan_visitor.rs index 8f1d5725e6697..ca3cf8e8c4297 100644 --- a/src/query/sql/src/executor/physical_plan_visitor.rs +++ b/src/query/sql/src/executor/physical_plan_visitor.rs @@ -415,7 +415,7 @@ impl PhysicalPlan { } PhysicalPlan::CopyIntoTable(plan) => match &plan.source { CopyIntoTableSource::Query(input) => { - Self::traverse(&input, pre_visit, visit, post_visit); + Self::traverse(input, pre_visit, visit, post_visit); } CopyIntoTableSource::Stage(_) => {} }, From a5782f5566178c93107a80884568dfbb1c7b5a0a Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Tue, 18 Jul 2023 16:25:17 +0800 Subject: [PATCH 07/40] fix logic error --- .../src/interpreters/interpreter_copy.rs | 2 +- .../service/src/pipelines/pipeline_builder.rs | 18 ------------------ 2 files changed, 1 insertion(+), 19 deletions(-) diff --git a/src/query/service/src/interpreters/interpreter_copy.rs b/src/query/service/src/interpreters/interpreter_copy.rs index 3d0c7cba4e066..2d4446ae7cb0b 100644 --- a/src/query/service/src/interpreters/interpreter_copy.rs +++ b/src/query/service/src/interpreters/interpreter_copy.rs @@ -186,7 +186,7 @@ impl CopyInterpreter { files: files.clone(), source, })); - if plan.enable_distributed { + if plan.enable_distributed && plan.query.is_none() { root = PhysicalPlan::Exchange(Exchange { plan_id: 0, input: Box::new(root), diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs index d37293ec2d591..f28395e6f2449 100644 --- a/src/query/service/src/pipelines/pipeline_builder.rs +++ b/src/query/service/src/pipelines/pipeline_builder.rs @@ -14,7 +14,6 @@ use std::convert::TryFrom; use std::sync::Arc; -use std::time::Instant; use async_channel::Receiver; use common_catalog::table::AppendMode; @@ -87,7 +86,6 @@ use common_storages_fuse::operations::TransformSerializeBlock; use common_storages_fuse::FuseTable; use common_storages_stage::StageTable; use petgraph::matrix_graph::Zero; -use tracing::info; use super::processors::transforms::FrameBound; use super::processors::transforms::WindowFunctionInfo; @@ -177,11 +175,6 @@ impl PipelineBuilder { }) } - fn set_status(&self, status: &str) { - self.ctx.set_status_info(status); - info!(status); - } - fn build_pipeline(&mut self, plan: &PhysicalPlan) -> Result<()> { match plan { PhysicalPlan::TableScan(scan) => self.build_table_scan(scan), @@ -219,7 +212,6 @@ impl PipelineBuilder { fn build_copy_into_table(&mut self, copy: &CopyIntoTable) -> Result<()> { let catalog = self.ctx.get_catalog(©.catalog_name)?; let to_table = catalog.get_table_by_info(©.table_info)?; - let file_sizes: u64 = copy.files.iter().map(|f| f.size).sum(); match ©.source { CopyIntoTableSource::Query(input) => { self.build_pipeline(input)?; @@ -230,12 +222,6 @@ impl PipelineBuilder { stage_table.read_data(self.ctx.clone(), source, &mut self.main_pipeline)?; } } - self.set_status(&format!( - "Copy begin to append data: {} files, size_in_bytes:{} into table", - copy.files.len(), - file_sizes - )); - let start = Instant::now(); build_append_data_pipeline( self.ctx.clone(), &mut self.main_pipeline, @@ -243,10 +229,6 @@ impl PipelineBuilder { copy.required_source_schema.clone(), to_table, )?; - self.set_status(&format!( - "Copy append data finished, cost:{} secs", - start.elapsed().as_secs() - )); Ok(()) } From 02ef7ff9a879e886d804ec56390f9629f3da099e Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Sun, 23 Jul 2023 06:28:32 +0800 Subject: [PATCH 08/40] replace into values --- .../src/interpreters/interpreter_copy.rs | 2 +- .../src/interpreters/interpreter_insert.rs | 283 +--------- .../src/interpreters/interpreter_replace.rs | 202 +++++--- src/query/service/src/pipelines/mod.rs | 1 + .../service/src/pipelines/pipeline_builder.rs | 482 ++++++++++++++++++ src/query/sql/src/executor/format.rs | 3 + src/query/sql/src/executor/physical_plan.rs | 60 ++- .../sql/src/executor/physical_plan_display.rs | 24 + .../sql/src/executor/physical_plan_visitor.rs | 33 ++ src/query/sql/src/executor/profile.rs | 5 +- src/query/storages/fuse/src/fuse_table.rs | 10 +- src/query/storages/fuse/src/operations/mod.rs | 4 +- .../storages/fuse/src/operations/replace.rs | 194 +------ .../fuse/src/operations/replace_into/mod.rs | 6 +- .../mutator/merge_into_mutator.rs | 2 +- .../mutator/mutator_replace_into.rs | 2 +- .../processors/processor_replace_into.rs | 2 +- 17 files changed, 744 insertions(+), 571 deletions(-) diff --git a/src/query/service/src/interpreters/interpreter_copy.rs b/src/query/service/src/interpreters/interpreter_copy.rs index 2d4446ae7cb0b..45689d1c44633 100644 --- a/src/query/service/src/interpreters/interpreter_copy.rs +++ b/src/query/service/src/interpreters/interpreter_copy.rs @@ -140,7 +140,7 @@ impl CopyInterpreter { } #[async_backtrace::framed] - async fn build_physical_plan( + pub async fn build_physical_plan( &self, plan: &CopyIntoTablePlan, ) -> Result<(PhysicalPlan, Vec)> { diff --git a/src/query/service/src/interpreters/interpreter_insert.rs b/src/query/service/src/interpreters/interpreter_insert.rs index e46707701da53..cafabb77f3e46 100644 --- a/src/query/service/src/interpreters/interpreter_insert.rs +++ b/src/query/service/src/interpreters/interpreter_insert.rs @@ -12,28 +12,14 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::VecDeque; -use std::io::BufRead; -use std::io::Cursor; -use std::ops::Not; use std::str::FromStr; use std::sync::Arc; -use aho_corasick::AhoCorasick; -use common_ast::parser::parse_comma_separated_exprs; -use common_ast::parser::tokenize_sql; use common_catalog::table::AppendMode; use common_exception::ErrorCode; use common_exception::Result; -use common_expression::ColumnBuilder; -use common_expression::DataBlock; use common_expression::DataSchema; -use common_expression::DataSchemaRef; -use common_formats::FastFieldDecoderValues; -use common_io::cursor_ext::ReadBytesExt; -use common_io::cursor_ext::ReadCheckPointExt; use common_meta_app::principal::StageFileFormatType; -use common_pipeline_sources::AsyncSource; use common_pipeline_sources::AsyncSourcer; use common_sql::executor::DistributedInsertSelect; use common_sql::executor::PhysicalPlan; @@ -41,13 +27,8 @@ use common_sql::executor::PhysicalPlanBuilder; use common_sql::plans::Insert; use common_sql::plans::InsertInputSource; use common_sql::plans::Plan; -use common_sql::BindContext; -use common_sql::Metadata; -use common_sql::MetadataRef; use common_sql::NameResolutionContext; -use once_cell::sync::Lazy; use parking_lot::Mutex; -use parking_lot::RwLock; use crate::interpreters::common::check_deduplicate_label; use crate::interpreters::Interpreter; @@ -56,15 +37,11 @@ use crate::pipelines::builders::build_append2table_with_commit_pipeline; use crate::pipelines::processors::transforms::TransformRuntimeCastSchema; use crate::pipelines::PipelineBuildResult; use crate::pipelines::SourcePipeBuilder; +use crate::pipelines::ValueSource; use crate::schedulers::build_query_pipeline_without_render_result_set; use crate::sessions::QueryContext; use crate::sessions::TableContext; -// Pre-generate the positions of `(`, `'` and `\` -static PATTERNS: &[&str] = &["(", "'", "\\"]; - -static INSERT_TOKEN_FINDER: Lazy = Lazy::new(|| AhoCorasick::new(PATTERNS).unwrap()); - pub struct InsertInterpreter { ctx: Arc, plan: Insert, @@ -289,261 +266,3 @@ impl Interpreter for InsertInterpreter { Ok(()) } } - -pub struct ValueSource { - data: String, - ctx: Arc, - name_resolution_ctx: NameResolutionContext, - bind_context: BindContext, - schema: DataSchemaRef, - metadata: MetadataRef, - is_finished: bool, -} - -#[async_trait::async_trait] -impl AsyncSource for ValueSource { - const NAME: &'static str = "ValueSource"; - const SKIP_EMPTY_DATA_BLOCK: bool = true; - - #[async_trait::unboxed_simple] - #[async_backtrace::framed] - async fn generate(&mut self) -> Result> { - if self.is_finished { - return Ok(None); - } - - // Use the number of '(' to estimate the number of rows - let mut estimated_rows = 0; - let mut positions = VecDeque::new(); - for mat in INSERT_TOKEN_FINDER.find_iter(&self.data) { - if mat.pattern() == 0.into() { - estimated_rows += 1; - continue; - } - positions.push_back(mat.start()); - } - - let mut reader = Cursor::new(self.data.as_bytes()); - let block = self - .read(estimated_rows, &mut reader, &mut positions) - .await?; - self.is_finished = true; - Ok(Some(block)) - } -} - -impl ValueSource { - pub fn new( - data: String, - ctx: Arc, - name_resolution_ctx: NameResolutionContext, - schema: DataSchemaRef, - ) -> Self { - let bind_context = BindContext::new(); - let metadata = Arc::new(RwLock::new(Metadata::default())); - - Self { - data, - ctx, - name_resolution_ctx, - schema, - bind_context, - metadata, - is_finished: false, - } - } - - #[async_backtrace::framed] - pub async fn read>( - &self, - estimated_rows: usize, - reader: &mut Cursor, - positions: &mut VecDeque, - ) -> Result { - let mut columns = self - .schema - .fields() - .iter() - .map(|f| ColumnBuilder::with_capacity(f.data_type(), estimated_rows)) - .collect::>(); - - let mut bind_context = self.bind_context.clone(); - - let format = self.ctx.get_format_settings()?; - let field_decoder = FastFieldDecoderValues::create_for_insert(format); - - for row in 0.. { - let _ = reader.ignore_white_spaces(); - if reader.eof() { - break; - } - // Not the first row - if row != 0 { - reader.must_ignore_byte(b',')?; - } - - self.parse_next_row( - &field_decoder, - reader, - &mut columns, - positions, - &mut bind_context, - self.metadata.clone(), - ) - .await?; - } - - let columns = columns - .into_iter() - .map(|col| col.build()) - .collect::>(); - Ok(DataBlock::new_from_columns(columns)) - } - - /// Parse single row value, like ('111', 222, 1 + 1) - #[async_backtrace::framed] - async fn parse_next_row>( - &self, - field_decoder: &FastFieldDecoderValues, - reader: &mut Cursor, - columns: &mut [ColumnBuilder], - positions: &mut VecDeque, - bind_context: &mut BindContext, - metadata: MetadataRef, - ) -> Result<()> { - let _ = reader.ignore_white_spaces(); - let col_size = columns.len(); - let start_pos_of_row = reader.checkpoint(); - - // Start of the row --- '(' - if !reader.ignore_byte(b'(') { - return Err(ErrorCode::BadDataValueType( - "Must start with parentheses".to_string(), - )); - } - // Ignore the positions in the previous row. - while let Some(pos) = positions.front() { - if *pos < start_pos_of_row as usize { - positions.pop_front(); - } else { - break; - } - } - - for col_idx in 0..col_size { - let _ = reader.ignore_white_spaces(); - let col_end = if col_idx + 1 == col_size { b')' } else { b',' }; - - let col = columns - .get_mut(col_idx) - .ok_or_else(|| ErrorCode::Internal("ColumnBuilder is None"))?; - - let (need_fallback, pop_count) = field_decoder - .read_field(col, reader, positions) - .map(|_| { - let _ = reader.ignore_white_spaces(); - let need_fallback = reader.ignore_byte(col_end).not(); - (need_fallback, col_idx + 1) - }) - .unwrap_or((true, col_idx)); - - // ColumnBuilder and expr-parser both will eat the end ')' of the row. - if need_fallback { - for col in columns.iter_mut().take(pop_count) { - col.pop(); - } - // rollback to start position of the row - reader.rollback(start_pos_of_row + 1); - skip_to_next_row(reader, 1)?; - let end_pos_of_row = reader.position(); - - // Parse from expression and append all columns. - reader.set_position(start_pos_of_row); - let row_len = end_pos_of_row - start_pos_of_row; - let buf = &reader.remaining_slice()[..row_len as usize]; - - let sql = std::str::from_utf8(buf).unwrap(); - let settings = self.ctx.get_settings(); - let sql_dialect = settings.get_sql_dialect()?; - let tokens = tokenize_sql(sql)?; - let exprs = parse_comma_separated_exprs(&tokens[1..tokens.len()], sql_dialect)?; - - let values = bind_context - .exprs_to_scalar( - exprs, - &self.schema, - self.ctx.clone(), - &self.name_resolution_ctx, - metadata, - ) - .await?; - - for (col, scalar) in columns.iter_mut().zip(values) { - col.push(scalar.as_ref()); - } - reader.set_position(end_pos_of_row); - return Ok(()); - } - } - - Ok(()) - } -} - -// Values |(xxx), (yyy), (zzz) -pub fn skip_to_next_row>(reader: &mut Cursor, mut balance: i32) -> Result<()> { - let _ = reader.ignore_white_spaces(); - - let mut quoted = false; - let mut escaped = false; - - while balance > 0 { - let buffer = reader.remaining_slice(); - if buffer.is_empty() { - break; - } - - let size = buffer.len(); - - let it = buffer - .iter() - .position(|&c| c == b'(' || c == b')' || c == b'\\' || c == b'\''); - - if let Some(it) = it { - let c = buffer[it]; - reader.consume(it + 1); - - if it == 0 && escaped { - escaped = false; - continue; - } - escaped = false; - - match c { - b'\\' => { - escaped = true; - continue; - } - b'\'' => { - quoted ^= true; - continue; - } - b')' => { - if !quoted { - balance -= 1; - } - } - b'(' => { - if !quoted { - balance += 1; - } - } - _ => {} - } - } else { - escaped = false; - reader.consume(size); - } - } - Ok(()) -} diff --git a/src/query/service/src/interpreters/interpreter_replace.rs b/src/query/service/src/interpreters/interpreter_replace.rs index 9bd250e5aa811..8118b53ee5bbd 100644 --- a/src/query/service/src/interpreters/interpreter_replace.rs +++ b/src/query/service/src/interpreters/interpreter_replace.rs @@ -19,21 +19,27 @@ use common_exception::ErrorCode; use common_exception::Result; use common_expression::DataSchema; use common_expression::DataSchemaRef; -use common_pipeline_sources::AsyncSourcer; +use common_sql::executor::AsyncSourcerPlan; +use common_sql::executor::Deduplicate; +use common_sql::executor::DeleteFinal; +use common_sql::executor::OnConflictField; +use common_sql::executor::PhysicalPlan; +use common_sql::executor::ReplaceInto; +use common_sql::plans::CopyPlan; use common_sql::plans::InsertInputSource; use common_sql::plans::Plan; use common_sql::plans::Replace; -use common_sql::NameResolutionContext; +use common_storages_factory::Table; +use common_storages_fuse::FuseTable; +use storages_common_table_meta::meta::TableSnapshot; use crate::interpreters::common::check_deduplicate_label; use crate::interpreters::interpreter_copy::CopyInterpreter; -use crate::interpreters::interpreter_insert::ValueSource; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; use crate::interpreters::SelectInterpreter; -use crate::pipelines::builders::build_fill_missing_columns_pipeline; -use crate::pipelines::processors::TransformCastSchema; use crate::pipelines::PipelineBuildResult; +use crate::schedulers::build_query_pipeline_without_render_result_set; use crate::sessions::QueryContext; #[allow(dead_code)] @@ -62,40 +68,84 @@ impl Interpreter for ReplaceInterpreter { self.check_on_conflicts()?; + let physical_plan = self.build_physical_plan().await?; + let build_res = + build_query_pipeline_without_render_result_set(&self.ctx, &physical_plan, false) + .await?; + Ok(build_res) + } +} + +impl ReplaceInterpreter { + async fn build_physical_plan(&self) -> Result> { let plan = &self.plan; let table = self .ctx .get_table(&plan.catalog, &plan.database, &plan.table) .await?; - - let mut pipeline = self - .connect_input_source(self.ctx.clone(), &self.plan.source, self.plan.schema()) - .await?; - - if pipeline.main_pipeline.is_empty() { - return Ok(pipeline); + let schema = table.schema(); + let mut on_conflicts = Vec::with_capacity(plan.on_conflict_fields.len()); + for f in &plan.on_conflict_fields { + let field_name = f.name(); + let (field_index, _) = match schema.column_with_name(field_name) { + Some(idx) => idx, + None => { + return Err(ErrorCode::Internal( + "not expected, on conflict field not found (after binding)", + )); + } + }; + on_conflicts.push(OnConflictField { + table_field: f.clone(), + field_index, + }) } + let fuse_table = + table + .as_any() + .downcast_ref::() + .ok_or(ErrorCode::Unimplemented(format!( + "table {}, engine type {}, does not support REPLACE INTO", + table.name(), + table.get_table_info().engine(), + )))?; + let table_info = fuse_table.get_table_info(); + let base_snapshot = fuse_table.read_table_snapshot().await?.unwrap_or_else(|| { + Arc::new(TableSnapshot::new_empty_snapshot(schema.as_ref().clone())) + }); - build_fill_missing_columns_pipeline( - self.ctx.clone(), - &mut pipeline.main_pipeline, - table.clone(), - self.plan.schema(), - )?; - - let on_conflict_fields = plan.on_conflict_fields.clone(); - table - .replace_into( - self.ctx.clone(), - &mut pipeline.main_pipeline, - on_conflict_fields, - ) + let empty_table = base_snapshot.segments.is_empty(); + let max_threads = self.ctx.get_settings().get_max_threads()?; + let segment_partition_num = + std::cmp::min(base_snapshot.segments.len(), max_threads as usize); + let mut root = self + .connect_input_source(self.ctx.clone(), &self.plan.source, self.plan.schema()) .await?; - Ok(pipeline) + root = Box::new(PhysicalPlan::Deduplicate(Deduplicate { + input: root, + on_conflicts: on_conflicts.clone(), + empty_table, + table_info: table_info.clone(), + catalog_name: plan.catalog.clone(), + schema: self.plan.schema(), + })); + root = Box::new(PhysicalPlan::ReplaceInto(ReplaceInto { + input: root, + segment_partition_num, + block_thresholds: fuse_table.get_block_thresholds(), + table_info: table_info.clone(), + catalog_name: plan.catalog.clone(), + on_conflicts, + snapshot: (*base_snapshot).clone(), + })); + root = Box::new(PhysicalPlan::DeleteFinal(Box::new(DeleteFinal { + input: root, + snapshot: (*base_snapshot).clone(), + table_info: table_info.clone(), + catalog_name: plan.catalog.clone(), + }))); + Ok(root) } -} - -impl ReplaceInterpreter { fn check_on_conflicts(&self) -> Result<()> { if self.plan.on_conflict_fields.is_empty() { Err(ErrorCode::BadArguments( @@ -111,21 +161,25 @@ impl ReplaceInterpreter { ctx: Arc, source: &'a InsertInputSource, schema: DataSchemaRef, - ) -> Result { + ) -> Result> { match source { - InsertInputSource::Values(data) => { - self.connect_value_source(ctx.clone(), schema.clone(), data) - } + InsertInputSource::Values(data) => self.connect_value_source(schema.clone(), data), InsertInputSource::SelectPlan(plan) => { - self.connect_query_plan_source(ctx.clone(), schema.clone(), plan) - .await + self.connect_query_plan_source(ctx.clone(), plan).await } InsertInputSource::Stage(plan) => match *plan.clone() { - Plan::Copy(copy_plan) => { - let interpreter = CopyInterpreter::try_create(ctx.clone(), *copy_plan.clone())?; - interpreter.execute2().await - } + Plan::Copy(copy_plan) => match copy_plan.as_ref() { + CopyPlan::IntoTable(copy_into_table_plan) => { + let interpreter = + CopyInterpreter::try_create(ctx.clone(), *copy_plan.clone())?; + interpreter + .build_physical_plan(copy_into_table_plan) + .await + .map(|x| Box::new(x.0)) + } + _ => unreachable!("plan in InsertInputSource::Stag must be CopyIntoTable"), + }, _ => unreachable!("plan in InsertInputSource::Stag must be Copy"), }, _ => Err(ErrorCode::Unimplemented( @@ -136,35 +190,22 @@ impl ReplaceInterpreter { fn connect_value_source( &self, - ctx: Arc, schema: DataSchemaRef, value_data: &str, - ) -> Result { - let mut build_res = PipelineBuildResult::create(); - let settings = ctx.get_settings(); - build_res.main_pipeline.add_source( - |output| { - let name_resolution_ctx = NameResolutionContext::try_from(settings.as_ref())?; - let inner = ValueSource::new( - value_data.to_string(), - ctx.clone(), - name_resolution_ctx, - schema.clone(), - ); - AsyncSourcer::create(ctx.clone(), output, inner) - }, - 1, - )?; - Ok(build_res) + ) -> Result> { + Ok(Box::new(PhysicalPlan::AsyncSourcer(AsyncSourcerPlan { + value_data: value_data.to_string(), + schema, + }))) } #[async_backtrace::framed] async fn connect_query_plan_source<'a>( &'a self, ctx: Arc, - self_schema: DataSchemaRef, + // self_schema: DataSchemaRef, query_plan: &Plan, - ) -> Result { + ) -> Result> { let (s_expr, metadata, bind_context, formatted_ast) = match query_plan { Plan::Query { s_expr, @@ -185,26 +226,29 @@ impl ReplaceInterpreter { false, )?; - let mut build_res = select_interpreter.execute2().await?; - - let select_schema = query_plan.schema(); - let target_schema = self_schema; - if self.check_schema_cast(query_plan)? { - let func_ctx = ctx.get_function_context()?; - build_res.main_pipeline.add_transform( - |transform_input_port, transform_output_port| { - TransformCastSchema::try_create( - transform_input_port, - transform_output_port, - select_schema.clone(), - target_schema.clone(), - func_ctx.clone(), - ) - }, - )?; - } + // let mut build_res = select_interpreter.execute2().await?; - Ok(build_res) + // let select_schema = query_plan.schema(); + // let target_schema = self_schema; + // if self.check_schema_cast(query_plan)? { + // let func_ctx = ctx.get_function_context()?; + // build_res.main_pipeline.add_transform( + // |transform_input_port, transform_output_port| { + // TransformCastSchema::try_create( + // transform_input_port, + // transform_output_port, + // select_schema.clone(), + // target_schema.clone(), + // func_ctx.clone(), + // ) + // }, + // )?; + // } + + select_interpreter + .build_physical_plan() + .await + .map(|x| Box::new(x)) } // TODO duplicated diff --git a/src/query/service/src/pipelines/mod.rs b/src/query/service/src/pipelines/mod.rs index 249c98d4e631e..266e1c6e482f0 100644 --- a/src/query/service/src/pipelines/mod.rs +++ b/src/query/service/src/pipelines/mod.rs @@ -28,3 +28,4 @@ pub use pipe::TransformPipeBuilder; pub use pipeline::Pipeline; pub use pipeline_build_res::PipelineBuildResult; pub use pipeline_builder::PipelineBuilder; +pub use pipeline_builder::ValueSource; diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs index f28395e6f2449..0a3a3767774d5 100644 --- a/src/query/service/src/pipelines/pipeline_builder.rs +++ b/src/query/service/src/pipelines/pipeline_builder.rs @@ -12,10 +12,18 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::VecDeque; use std::convert::TryFrom; +use std::io::BufRead; +use std::io::Cursor; +use std::ops::Not; use std::sync::Arc; +use aho_corasick::AhoCorasick; use async_channel::Receiver; +use common_ast::parser::parse_comma_separated_exprs; +use common_ast::parser::tokenize_sql; +use common_base::base::tokio::sync::Semaphore; use common_catalog::table::AppendMode; use common_exception::ErrorCode; use common_exception::Result; @@ -25,27 +33,35 @@ use common_expression::types::NumberDataType; use common_expression::with_hash_method; use common_expression::with_mappedhash_method; use common_expression::with_number_mapped_type; +use common_expression::ColumnBuilder; use common_expression::DataBlock; use common_expression::DataSchemaRef; use common_expression::FunctionContext; use common_expression::HashMethodKind; use common_expression::RemoteExpr; use common_expression::SortColumnDescription; +use common_formats::FastFieldDecoderValues; use common_functions::aggregates::AggregateFunctionFactory; use common_functions::aggregates::AggregateFunctionRef; use common_functions::BUILTIN_FUNCTIONS; +use common_io::cursor_ext::ReadBytesExt; +use common_io::cursor_ext::ReadCheckPointExt; 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_core::processors::Processor; use common_pipeline_sinks::EmptySink; use common_pipeline_sinks::Sinker; use common_pipeline_sinks::UnionReceiveSink; +use common_pipeline_sources::AsyncSource; +use common_pipeline_sources::AsyncSourcer; use common_pipeline_transforms::processors::profile_wrapper::ProcessorProfileWrapper; use common_pipeline_transforms::processors::profile_wrapper::ProfileStub; use common_pipeline_transforms::processors::profile_wrapper::TransformProfileWrapper; use common_pipeline_transforms::processors::transforms::build_full_sort_pipeline; +use common_pipeline_transforms::processors::transforms::create_dummy_item; use common_pipeline_transforms::processors::transforms::Transformer; use common_profile::SharedProcessorProfiles; use common_sql::evaluator::BlockOperator; @@ -54,8 +70,10 @@ use common_sql::executor::AggregateExpand; use common_sql::executor::AggregateFinal; use common_sql::executor::AggregateFunctionDesc; use common_sql::executor::AggregatePartial; +use common_sql::executor::AsyncSourcerPlan; use common_sql::executor::CopyIntoTable; use common_sql::executor::CopyIntoTableSource; +use common_sql::executor::Deduplicate; use common_sql::executor::DeleteFinal; use common_sql::executor::DeletePartial; use common_sql::executor::DistributedInsertSelect; @@ -69,22 +87,32 @@ use common_sql::executor::PhysicalPlan; use common_sql::executor::Project; use common_sql::executor::ProjectSet; use common_sql::executor::RangeJoin; +use common_sql::executor::ReplaceInto; use common_sql::executor::RowFetch; use common_sql::executor::RuntimeFilterSource; use common_sql::executor::Sort; use common_sql::executor::TableScan; use common_sql::executor::UnionAll; use common_sql::executor::Window; +use common_sql::BindContext; use common_sql::ColumnBinding; use common_sql::IndexType; +use common_sql::Metadata; +use common_sql::MetadataRef; +use common_sql::NameResolutionContext; use common_storage::DataOperator; use common_storages_factory::Table; use common_storages_fuse::operations::build_row_fetcher_pipeline; +use common_storages_fuse::operations::common::TransformSerializeSegment; +use common_storages_fuse::operations::replace_into::BroadcastProcessor; +use common_storages_fuse::operations::replace_into::ReplaceIntoProcessor; use common_storages_fuse::operations::FillInternalColumnProcessor; use common_storages_fuse::operations::MutationKind; use common_storages_fuse::operations::TransformSerializeBlock; use common_storages_fuse::FuseTable; use common_storages_stage::StageTable; +use once_cell::sync::Lazy; +use parking_lot::RwLock; use petgraph::matrix_graph::Zero; use super::processors::transforms::FrameBound; @@ -206,9 +234,200 @@ impl PipelineBuilder { PhysicalPlan::DeleteFinal(delete) => self.build_delete_final(delete), PhysicalPlan::RangeJoin(range_join) => self.build_range_join(range_join), PhysicalPlan::CopyIntoTable(copy) => self.build_copy_into_table(copy), + PhysicalPlan::AsyncSourcer(async_sourcer) => self.build_async_sourcer(async_sourcer), + PhysicalPlan::Deduplicate(deduplicate) => self.build_deduplicate(deduplicate), + PhysicalPlan::ReplaceInto(replace) => self.build_replace_into(replace), } } + fn build_deduplicate(&mut self, deduplicate: &Deduplicate) -> Result<()> { + let Deduplicate { + input, + on_conflicts, + empty_table, + table_info, + catalog_name, + schema, + } = deduplicate; + let tbl = self + .ctx + .build_table_by_table_info(catalog_name, table_info, None)?; + let table = FuseTable::try_from_table(tbl.as_ref())?; + self.build_pipeline(input)?; + build_fill_missing_columns_pipeline( + self.ctx.clone(), + &mut self.main_pipeline, + tbl.clone(), + schema.clone(), + )?; + let _ = table.cluster_gen_for_append( + self.ctx.clone(), + &mut self.main_pipeline, + table.get_block_thresholds(), + )?; + // 1. resize input to 1, since the UpsertTransform need to de-duplicate inputs "globally" + self.main_pipeline.try_resize(1)?; + + // 2. connect with ReplaceIntoProcessor + + // ┌──────────────────────┐ + // │ ├──┐ + // ┌─────────────┐ │ ├──┘ + // │ UpsertSource├─────►│ ReplaceIntoProcessor │ + // └─────────────┘ │ ├──┐ + // │ ├──┘ + // └──────────────────────┘ + // NOTE: here the pipe items of last pipe are arranged in the following order + // (0) -> output_port_append_data + // (1) -> output_port_merge_into_action + // the "downstream" is supposed to be connected with a processor which can process MergeIntoOperations + // in our case, it is the broadcast processor + + let replace_into_processor = + ReplaceIntoProcessor::create(on_conflicts.clone(), *empty_table); + self.main_pipeline + .add_pipe(replace_into_processor.into_pipe()); + Ok(()) + } + + fn build_replace_into(&mut self, replace: &ReplaceInto) -> Result<()> { + let ReplaceInto { + input, + segment_partition_num, + block_thresholds, + table_info, + on_conflicts, + catalog_name, + snapshot, + } = replace; + let table = self + .ctx + .build_table_by_table_info(catalog_name, table_info, None)?; + let table = FuseTable::try_from_table(table.as_ref())?; + let cluster_stats_gen = + table.get_cluster_stats_gen(self.ctx.clone(), 0, *block_thresholds)?; + self.build_pipeline(input)?; + // connect to broadcast processor and append transform + let serialize_block_transform = TransformSerializeBlock::try_create( + self.ctx.clone(), + InputPort::create(), + OutputPort::create(), + table, + cluster_stats_gen, + )?; + let block_builder = serialize_block_transform.get_block_builder(); + + let serialize_segment_transform = TransformSerializeSegment::new( + InputPort::create(), + OutputPort::create(), + table, + *block_thresholds, + ); + if *segment_partition_num == 0 { + let dummy_item = create_dummy_item(); + // ┌──────────────────────┐ ┌──────────────────┐ + // │ ├──┬────────►│ SerializeBlock │ + // ┌─────────────┐ │ ├──┘ └──────────────────┘ + // │ UpsertSource├─────►│ ReplaceIntoProcessor │ + // └─────────────┘ │ ├──┐ ┌──────────────────┐ + // │ ├──┴────────►│ DummyTransform │ + // └──────────────────────┘ └──────────────────┘ + // wrap them into pipeline, order matters! + self.main_pipeline.add_pipe(Pipe::create(2, 2, vec![ + serialize_block_transform.into_pipe_item(), + dummy_item, + ])); + } else { + // ┌──────────────────────┐ ┌──────────────────┐ + // │ ├──┬────────►│ SerializeBlock │ + // ┌─────────────┐ │ ├──┘ └──────────────────┘ + // │ UpsertSource├─────►│ ReplaceIntoProcessor │ + // └─────────────┘ │ ├──┐ ┌──────────────────┐ + // │ ├──┴────────►│BroadcastProcessor│ + // └──────────────────────┘ └──────────────────┘ + let broadcast_processor = BroadcastProcessor::new(*segment_partition_num); + // wrap them into pipeline, order matters! + self.main_pipeline + .add_pipe(Pipe::create(2, segment_partition_num + 1, vec![ + serialize_block_transform.into_pipe_item(), + broadcast_processor.into_pipe_item(), + ])); + }; + + // 4. connect with MergeIntoOperationAggregators + if *segment_partition_num == 0 { + let dummy_item = create_dummy_item(); + self.main_pipeline.add_pipe(Pipe::create(2, 2, vec![ + serialize_segment_transform.into_pipe_item(), + dummy_item, + ])); + } else { + // ┌──────────────────┐ ┌────────────────┐ + // ────►│ SerializeBlock ├──────────────►│SerializeSegment│ + // └──────────────────┘ └────────────────┘ + // + // ┌───────────────────┐ ┌──────────────────────┐ + // ────►│ ├──┬──────────►│MergeIntoOperationAggr│ + // │ ├──┘ └──────────────────────┘ + // │ BroadcastProcessor│ + // │ ├──┐ ┌──────────────────────┐ + // │ ├──┴──────────►│MergeIntoOperationAggr│ + // │ │ └──────────────────────┘ + // │ ├──┐ + // │ ├──┴──────────►┌──────────────────────┐ + // └───────────────────┘ │MergeIntoOperationAggr│ + // └──────────────────────┘ + + let item_size = segment_partition_num + 1; + let mut pipe_items = Vec::with_capacity(item_size); + // setup the dummy transform + pipe_items.push(serialize_segment_transform.into_pipe_item()); + + let max_io_request = self.ctx.get_settings().get_max_storage_io_requests()?; + let io_request_semaphore = Arc::new(Semaphore::new(max_io_request as usize)); + + // setup the merge into operation aggregators + let mut merge_into_operation_aggregators = table.merge_into_mutators( + self.ctx.clone(), + *segment_partition_num, + block_builder, + on_conflicts.clone(), + snapshot, + io_request_semaphore, + )?; + assert_eq!( + *segment_partition_num, + merge_into_operation_aggregators.len() + ); + pipe_items.append(&mut merge_into_operation_aggregators); + + // extend the pipeline + assert_eq!(self.main_pipeline.output_len(), item_size); + assert_eq!(pipe_items.len(), item_size); + self.main_pipeline + .add_pipe(Pipe::create(item_size, item_size, pipe_items)); + } + Ok(()) + } + + fn build_async_sourcer(&mut self, async_sourcer: &AsyncSourcerPlan) -> Result<()> { + let settings = self.ctx.get_settings(); + self.main_pipeline.add_source( + |output| { + let name_resolution_ctx = NameResolutionContext::try_from(settings.as_ref())?; + let inner = ValueSource::new( + async_sourcer.value_data.clone(), + self.ctx.clone(), + name_resolution_ctx, + async_sourcer.schema.clone(), + ); + AsyncSourcer::create(self.ctx.clone(), output, inner) + }, + 1, + )?; + Ok(()) + } + fn build_copy_into_table(&mut self, copy: &CopyIntoTable) -> Result<()> { let catalog = self.ctx.get_catalog(©.catalog_name)?; let to_table = catalog.get_table_by_info(©.table_info)?; @@ -1470,3 +1689,266 @@ impl PipelineBuilder { ))) } } + +// Pre-generate the positions of `(`, `'` and `\` +static PATTERNS: &[&str] = &["(", "'", "\\"]; + +static INSERT_TOKEN_FINDER: Lazy = Lazy::new(|| AhoCorasick::new(PATTERNS).unwrap()); + +pub struct ValueSource { + data: String, + ctx: Arc, + name_resolution_ctx: NameResolutionContext, + bind_context: BindContext, + schema: DataSchemaRef, + metadata: MetadataRef, + is_finished: bool, +} + +#[async_trait::async_trait] +impl AsyncSource for ValueSource { + const NAME: &'static str = "ValueSource"; + const SKIP_EMPTY_DATA_BLOCK: bool = true; + + #[async_trait::unboxed_simple] + #[async_backtrace::framed] + async fn generate(&mut self) -> Result> { + if self.is_finished { + return Ok(None); + } + + // Use the number of '(' to estimate the number of rows + let mut estimated_rows = 0; + let mut positions = VecDeque::new(); + for mat in INSERT_TOKEN_FINDER.find_iter(&self.data) { + if mat.pattern() == 0.into() { + estimated_rows += 1; + continue; + } + positions.push_back(mat.start()); + } + + let mut reader = Cursor::new(self.data.as_bytes()); + let block = self + .read(estimated_rows, &mut reader, &mut positions) + .await?; + self.is_finished = true; + Ok(Some(block)) + } +} + +impl ValueSource { + pub fn new( + data: String, + ctx: Arc, + name_resolution_ctx: NameResolutionContext, + schema: DataSchemaRef, + ) -> Self { + let bind_context = BindContext::new(); + let metadata = Arc::new(RwLock::new(Metadata::default())); + + Self { + data, + ctx, + name_resolution_ctx, + schema, + bind_context, + metadata, + is_finished: false, + } + } + + #[async_backtrace::framed] + pub async fn read>( + &self, + estimated_rows: usize, + reader: &mut Cursor, + positions: &mut VecDeque, + ) -> Result { + let mut columns = self + .schema + .fields() + .iter() + .map(|f| ColumnBuilder::with_capacity(f.data_type(), estimated_rows)) + .collect::>(); + + let mut bind_context = self.bind_context.clone(); + + let format = self.ctx.get_format_settings()?; + let field_decoder = FastFieldDecoderValues::create_for_insert(format); + + for row in 0.. { + let _ = reader.ignore_white_spaces(); + if reader.eof() { + break; + } + // Not the first row + if row != 0 { + reader.must_ignore_byte(b',')?; + } + + self.parse_next_row( + &field_decoder, + reader, + &mut columns, + positions, + &mut bind_context, + self.metadata.clone(), + ) + .await?; + } + + let columns = columns + .into_iter() + .map(|col| col.build()) + .collect::>(); + Ok(DataBlock::new_from_columns(columns)) + } + + /// Parse single row value, like ('111', 222, 1 + 1) + #[async_backtrace::framed] + async fn parse_next_row>( + &self, + field_decoder: &FastFieldDecoderValues, + reader: &mut Cursor, + columns: &mut [ColumnBuilder], + positions: &mut VecDeque, + bind_context: &mut BindContext, + metadata: MetadataRef, + ) -> Result<()> { + let _ = reader.ignore_white_spaces(); + let col_size = columns.len(); + let start_pos_of_row = reader.checkpoint(); + + // Start of the row --- '(' + if !reader.ignore_byte(b'(') { + return Err(ErrorCode::BadDataValueType( + "Must start with parentheses".to_string(), + )); + } + // Ignore the positions in the previous row. + while let Some(pos) = positions.front() { + if *pos < start_pos_of_row as usize { + positions.pop_front(); + } else { + break; + } + } + + for col_idx in 0..col_size { + let _ = reader.ignore_white_spaces(); + let col_end = if col_idx + 1 == col_size { b')' } else { b',' }; + + let col = columns + .get_mut(col_idx) + .ok_or_else(|| ErrorCode::Internal("ColumnBuilder is None"))?; + + let (need_fallback, pop_count) = field_decoder + .read_field(col, reader, positions) + .map(|_| { + let _ = reader.ignore_white_spaces(); + let need_fallback = reader.ignore_byte(col_end).not(); + (need_fallback, col_idx + 1) + }) + .unwrap_or((true, col_idx)); + + // ColumnBuilder and expr-parser both will eat the end ')' of the row. + if need_fallback { + for col in columns.iter_mut().take(pop_count) { + col.pop(); + } + // rollback to start position of the row + reader.rollback(start_pos_of_row + 1); + skip_to_next_row(reader, 1)?; + let end_pos_of_row = reader.position(); + + // Parse from expression and append all columns. + reader.set_position(start_pos_of_row); + let row_len = end_pos_of_row - start_pos_of_row; + let buf = &reader.remaining_slice()[..row_len as usize]; + + let sql = std::str::from_utf8(buf).unwrap(); + let settings = self.ctx.get_settings(); + let sql_dialect = settings.get_sql_dialect()?; + let tokens = tokenize_sql(sql)?; + let exprs = parse_comma_separated_exprs(&tokens[1..tokens.len()], sql_dialect)?; + + let values = bind_context + .exprs_to_scalar( + exprs, + &self.schema, + self.ctx.clone(), + &self.name_resolution_ctx, + metadata, + ) + .await?; + + for (col, scalar) in columns.iter_mut().zip(values) { + col.push(scalar.as_ref()); + } + reader.set_position(end_pos_of_row); + return Ok(()); + } + } + + Ok(()) + } +} + +// Values |(xxx), (yyy), (zzz) +pub fn skip_to_next_row>(reader: &mut Cursor, mut balance: i32) -> Result<()> { + let _ = reader.ignore_white_spaces(); + + let mut quoted = false; + let mut escaped = false; + + while balance > 0 { + let buffer = reader.remaining_slice(); + if buffer.is_empty() { + break; + } + + let size = buffer.len(); + + let it = buffer + .iter() + .position(|&c| c == b'(' || c == b')' || c == b'\\' || c == b'\''); + + if let Some(it) = it { + let c = buffer[it]; + reader.consume(it + 1); + + if it == 0 && escaped { + escaped = false; + continue; + } + escaped = false; + + match c { + b'\\' => { + escaped = true; + continue; + } + b'\'' => { + quoted ^= true; + continue; + } + b')' => { + if !quoted { + balance -= 1; + } + } + b'(' => { + if !quoted { + balance += 1; + } + } + _ => {} + } + } else { + escaped = false; + reader.consume(size); + } + } + Ok(()) +} diff --git a/src/query/sql/src/executor/format.rs b/src/query/sql/src/executor/format.rs index 38ba9864aae16..c44c9b4a084eb 100644 --- a/src/query/sql/src/executor/format.rs +++ b/src/query/sql/src/executor/format.rs @@ -173,6 +173,9 @@ fn to_format_tree( } PhysicalPlan::RangeJoin(plan) => range_join_to_format_tree(plan, metadata, prof_span_set), PhysicalPlan::CopyIntoTable(plan) => copy_into_table(plan), + PhysicalPlan::AsyncSourcer(_) => Ok(FormatTreeNode::new("AsyncSourcer".to_string())), + PhysicalPlan::Deduplicate(_) => Ok(FormatTreeNode::new("Deduplicate".to_string())), + PhysicalPlan::ReplaceInto(_) => Ok(FormatTreeNode::new("Replace".to_string())), } } diff --git a/src/query/sql/src/executor/physical_plan.rs b/src/query/sql/src/executor/physical_plan.rs index f7ba5b01f60b2..e668a0fc525c2 100644 --- a/src/query/sql/src/executor/physical_plan.rs +++ b/src/query/sql/src/executor/physical_plan.rs @@ -25,6 +25,7 @@ use common_catalog::plan::StageTableInfo; use common_exception::Result; use common_expression::types::DataType; use common_expression::types::NumberDataType; +use common_expression::BlockThresholds; use common_expression::DataBlock; use common_expression::DataField; use common_expression::DataSchemaRef; @@ -792,6 +793,7 @@ impl DeleteFinal { } } +// TODO(sky): make TableMutationAggregator distributed #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct DeleteFinal { pub input: Box, @@ -800,6 +802,39 @@ pub struct DeleteFinal { pub catalog_name: String, } +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct AsyncSourcerPlan { + pub value_data: String, + pub schema: DataSchemaRef, +} + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct Deduplicate { + pub input: Box, + pub on_conflicts: Vec, + pub empty_table: bool, + pub table_info: TableInfo, + pub catalog_name: String, + pub schema: DataSchemaRef, +} + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct OnConflictField { + pub table_field: common_expression::TableField, + pub field_index: common_expression::FieldIndex, +} + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct ReplaceInto { + pub input: Box, + pub segment_partition_num: usize, + pub block_thresholds: BlockThresholds, + pub table_info: TableInfo, + pub on_conflicts: Vec, + pub catalog_name: String, + pub snapshot: TableSnapshot, +} + #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct RefreshIndex { pub input: Box, @@ -850,9 +885,15 @@ pub enum PhysicalPlan { ExchangeSource(ExchangeSource), ExchangeSink(ExchangeSink), + /// Delete DeletePartial(Box), DeleteFinal(Box), + /// Copy into table CopyIntoTable(Box), + /// Replace + AsyncSourcer(AsyncSourcerPlan), + Deduplicate(Deduplicate), + ReplaceInto(ReplaceInto), } impl PhysicalPlan { @@ -889,7 +930,10 @@ impl PhysicalPlan { PhysicalPlan::ExchangeSink(v) => v.plan_id, PhysicalPlan::DeletePartial(_) | PhysicalPlan::DeleteFinal(_) - | PhysicalPlan::CopyIntoTable(_) => { + | PhysicalPlan::CopyIntoTable(_) + | PhysicalPlan::AsyncSourcer(_) + | PhysicalPlan::Deduplicate(_) + | PhysicalPlan::ReplaceInto(_) => { unreachable!() } } @@ -920,6 +964,9 @@ impl PhysicalPlan { PhysicalPlan::DeleteFinal(plan) => plan.output_schema(), PhysicalPlan::RangeJoin(plan) => plan.output_schema(), PhysicalPlan::CopyIntoTable(plan) => plan.output_schema(), + PhysicalPlan::AsyncSourcer(_) + | PhysicalPlan::Deduplicate(_) + | PhysicalPlan::ReplaceInto(_) => Ok(DataSchemaRef::default()), } } @@ -948,6 +995,9 @@ impl PhysicalPlan { PhysicalPlan::DeleteFinal(_) => "DeleteFinal".to_string(), PhysicalPlan::RangeJoin(_) => "RangeJoin".to_string(), PhysicalPlan::CopyIntoTable(_) => "CopyIntoTable".to_string(), + PhysicalPlan::AsyncSourcer(_) => "AsyncSourcer".to_string(), + PhysicalPlan::Deduplicate(_) => "Deduplicate".to_string(), + PhysicalPlan::ReplaceInto(_) => "Replace".to_string(), } } @@ -987,6 +1037,9 @@ impl PhysicalPlan { std::iter::once(plan.left.as_ref()).chain(std::iter::once(plan.right.as_ref())), ), PhysicalPlan::CopyIntoTable(_) => Box::new(std::iter::empty()), + PhysicalPlan::AsyncSourcer(_) => Box::new(std::iter::empty()), + PhysicalPlan::Deduplicate(plan) => Box::new(std::iter::once(plan.input.as_ref())), + PhysicalPlan::ReplaceInto(plan) => Box::new(std::iter::once(plan.input.as_ref())), } } @@ -1015,7 +1068,10 @@ impl PhysicalPlan { | PhysicalPlan::AggregatePartial(_) | PhysicalPlan::DeletePartial(_) | PhysicalPlan::DeleteFinal(_) - | PhysicalPlan::CopyIntoTable(_) => None, + | PhysicalPlan::CopyIntoTable(_) + | PhysicalPlan::AsyncSourcer(_) + | PhysicalPlan::Deduplicate(_) + | PhysicalPlan::ReplaceInto(_) => None, } } } diff --git a/src/query/sql/src/executor/physical_plan_display.rs b/src/query/sql/src/executor/physical_plan_display.rs index 988b7ad81422b..7af613682c4f6 100644 --- a/src/query/sql/src/executor/physical_plan_display.rs +++ b/src/query/sql/src/executor/physical_plan_display.rs @@ -19,11 +19,14 @@ use common_functions::BUILTIN_FUNCTIONS; use itertools::Itertools; use super::AggregateExpand; +use super::AsyncSourcerPlan; use super::CopyIntoTable; +use super::Deduplicate; use super::DeleteFinal; use super::DeletePartial; use super::DistributedInsertSelect; use super::ProjectSet; +use super::ReplaceInto; use super::RowFetch; use crate::executor::AggregateFinal; use crate::executor::AggregatePartial; @@ -83,6 +86,9 @@ impl<'a> Display for PhysicalPlanIndentFormatDisplay<'a> { PhysicalPlan::RuntimeFilterSource(plan) => write!(f, "{}", plan)?, PhysicalPlan::RangeJoin(plan) => write!(f, "{}", plan)?, PhysicalPlan::CopyIntoTable(copy_into_table) => write!(f, "{}", copy_into_table)?, + PhysicalPlan::AsyncSourcer(async_sourcer) => write!(f, "{}", async_sourcer)?, + PhysicalPlan::Deduplicate(deduplicate) => write!(f, "{}", deduplicate)?, + PhysicalPlan::ReplaceInto(replace) => write!(f, "{}", replace)?, } for node in self.node.children() { @@ -390,3 +396,21 @@ impl Display for ProjectSet { ) } } + +impl Display for AsyncSourcerPlan { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!(f, "AsyncSourcer") + } +} + +impl Display for Deduplicate { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!(f, "Deduplicate") + } +} + +impl Display for ReplaceInto { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!(f, "Replace") + } +} diff --git a/src/query/sql/src/executor/physical_plan_visitor.rs b/src/query/sql/src/executor/physical_plan_visitor.rs index ca3cf8e8c4297..e5d0eea6633ce 100644 --- a/src/query/sql/src/executor/physical_plan_visitor.rs +++ b/src/query/sql/src/executor/physical_plan_visitor.rs @@ -17,8 +17,10 @@ use common_exception::Result; use super::AggregateExpand; use super::AggregateFinal; use super::AggregatePartial; +use super::AsyncSourcerPlan; use super::CopyIntoTable; use super::CopyIntoTableSource; +use super::Deduplicate; use super::DeleteFinal; use super::DeletePartial; use super::DistributedInsertSelect; @@ -32,6 +34,7 @@ use super::Limit; use super::PhysicalPlan; use super::Project; use super::ProjectSet; +use super::ReplaceInto; use super::RowFetch; use super::Sort; use super::TableScan; @@ -66,6 +69,9 @@ pub trait PhysicalPlanReplacer { PhysicalPlan::DeleteFinal(plan) => self.replace_delete_final(plan), PhysicalPlan::RangeJoin(plan) => self.replace_range_join(plan), PhysicalPlan::CopyIntoTable(plan) => self.replace_copy_into_table(plan), + PhysicalPlan::AsyncSourcer(plan) => self.replace_async_sourcer(plan), + PhysicalPlan::Deduplicate(plan) => self.replace_deduplicate(plan), + PhysicalPlan::ReplaceInto(plan) => self.replace_replace_into(plan), } } @@ -324,6 +330,26 @@ pub trait PhysicalPlanReplacer { }))) } + fn replace_async_sourcer(&mut self, plan: &AsyncSourcerPlan) -> Result { + Ok(PhysicalPlan::AsyncSourcer(plan.clone())) + } + + fn replace_deduplicate(&mut self, plan: &Deduplicate) -> Result { + let input = self.replace(&plan.input)?; + Ok(PhysicalPlan::Deduplicate(Deduplicate { + input: Box::new(input), + ..plan.clone() + })) + } + + fn replace_replace_into(&mut self, plan: &ReplaceInto) -> Result { + let input = self.replace(&plan.input)?; + Ok(PhysicalPlan::ReplaceInto(ReplaceInto { + 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 { @@ -362,6 +388,7 @@ impl PhysicalPlan { visit(plan); match plan { PhysicalPlan::TableScan(_) => {} + PhysicalPlan::AsyncSourcer(_) => {} PhysicalPlan::Filter(plan) => { Self::traverse(&plan.input, pre_visit, visit, post_visit); } @@ -431,6 +458,12 @@ impl PhysicalPlan { PhysicalPlan::DeleteFinal(plan) => { Self::traverse(&plan.input, pre_visit, visit, post_visit); } + PhysicalPlan::Deduplicate(plan) => { + Self::traverse(&plan.input, pre_visit, visit, post_visit); + } + PhysicalPlan::ReplaceInto(plan) => { + Self::traverse(&plan.input, pre_visit, visit, post_visit); + } } post_visit(plan); } diff --git a/src/query/sql/src/executor/profile.rs b/src/query/sql/src/executor/profile.rs index fe2df7dd16c9c..093f00513ea40 100644 --- a/src/query/sql/src/executor/profile.rs +++ b/src/query/sql/src/executor/profile.rs @@ -452,7 +452,10 @@ fn flatten_plan_node_profile( } PhysicalPlan::DeletePartial(_) | PhysicalPlan::DeleteFinal(_) - | PhysicalPlan::CopyIntoTable(_) => unreachable!(), + | PhysicalPlan::CopyIntoTable(_) + | PhysicalPlan::AsyncSourcer(_) + | PhysicalPlan::Deduplicate(_) + | PhysicalPlan::ReplaceInto(_) => unreachable!(), } Ok(()) diff --git a/src/query/storages/fuse/src/fuse_table.rs b/src/query/storages/fuse/src/fuse_table.rs index 681dcea273887..de76d474165d6 100644 --- a/src/query/storages/fuse/src/fuse_table.rs +++ b/src/query/storages/fuse/src/fuse_table.rs @@ -15,6 +15,7 @@ use std::any::Any; use std::collections::BTreeMap; use std::collections::HashMap; +use std::panic; use std::str; use std::str::FromStr; use std::sync::Arc; @@ -542,12 +543,11 @@ impl Table for FuseTable { #[async_backtrace::framed] async fn replace_into( &self, - ctx: Arc, - pipeline: &mut Pipeline, - on_conflict_fields: Vec, + _ctx: Arc, + _pipeline: &mut Pipeline, + _on_conflict_fields: Vec, ) -> Result<()> { - self.build_replace_pipeline(ctx, on_conflict_fields, pipeline) - .await + panic!("deprecated") } fn commit_insertion( diff --git a/src/query/storages/fuse/src/operations/mod.rs b/src/query/storages/fuse/src/operations/mod.rs index 7c26c688769f6..5e355ff108152 100644 --- a/src/query/storages/fuse/src/operations/mod.rs +++ b/src/query/storages/fuse/src/operations/mod.rs @@ -26,8 +26,8 @@ mod read; mod read_data; mod read_partitions; mod recluster; -mod replace; -mod replace_into; +pub mod replace; +pub mod replace_into; mod revert; mod truncate; mod update; diff --git a/src/query/storages/fuse/src/operations/replace.rs b/src/query/storages/fuse/src/operations/replace.rs index 0eced5f8ca87f..a2ce8d80329b4 100644 --- a/src/query/storages/fuse/src/operations/replace.rs +++ b/src/query/storages/fuse/src/operations/replace.rs @@ -17,16 +17,11 @@ use std::sync::Arc; use common_base::base::tokio::sync::Semaphore; use common_catalog::table::Table; use common_catalog::table_context::TableContext; -use common_exception::ErrorCode; use common_exception::Result; -use common_expression::TableField; -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::create_dummy_item; use common_pipeline_transforms::processors::transforms::AsyncAccumulatingTransformer; +use common_sql::executor::OnConflictField; use rand::prelude::SliceRandom; use storages_common_table_meta::meta::Location; use storages_common_table_meta::meta::TableSnapshot; @@ -37,18 +32,13 @@ use crate::io::ReadSettings; use crate::operations::common::CommitSink; use crate::operations::common::MutationGenerator; use crate::operations::common::TableMutationAggregator; -use crate::operations::common::TransformSerializeBlock; -use crate::operations::common::TransformSerializeSegment; use crate::operations::mutation::SegmentIndex; -use crate::operations::replace_into::BroadcastProcessor; use crate::operations::replace_into::MergeIntoOperationAggregator; -use crate::operations::replace_into::OnConflictField; -use crate::operations::replace_into::ReplaceIntoProcessor; use crate::pipelines::Pipeline; use crate::FuseTable; impl FuseTable { - // The pipeline going to be constructed + // The big picture of the replace into pipeline: // // - If table is not empty: // @@ -101,185 +91,7 @@ impl FuseTable { // └─────►│ResizeProcessor(1) ├──────►│TableMutationAggregator├────────►│ CommitSink │ // └───────────────────┘ └───────────────────────┘ └───────────────────┘ - #[async_backtrace::framed] - pub async fn build_replace_pipeline<'a>( - &'a self, - ctx: Arc, - on_conflict_field_identifiers: Vec, - pipeline: &'a mut Pipeline, - ) -> Result<()> { - let schema = self.table_info.schema(); - - let mut on_conflicts = Vec::with_capacity(on_conflict_field_identifiers.len()); - for f in on_conflict_field_identifiers { - let field_name = f.name(); - let (field_index, _) = match schema.column_with_name(field_name) { - Some(idx) => idx, - None => { - return Err(ErrorCode::Internal( - "not expected, on conflict field not found (after binding)", - )); - } - }; - on_conflicts.push(OnConflictField { - table_field: f.clone(), - field_index, - }) - } - - let cluster_stats_gen = - self.cluster_gen_for_append(ctx.clone(), pipeline, self.get_block_thresholds())?; - - // 1. resize input to 1, since the UpsertTransform need to de-duplicate inputs "globally" - pipeline.try_resize(1)?; - - // 2. connect with ReplaceIntoProcessor - - // ┌──────────────────────┐ - // │ ├──┐ - // ┌─────────────┐ │ ├──┘ - // │ UpsertSource├─────►│ ReplaceIntoProcessor │ - // └─────────────┘ │ ├──┐ - // │ ├──┘ - // └──────────────────────┘ - // NOTE: here the pipe items of last pipe are arranged in the following order - // (0) -> output_port_append_data - // (1) -> output_port_merge_into_action - // the "downstream" is supposed to be connected with a processor which can process MergeIntoOperations - // in our case, it is the broadcast processor - - let base_snapshot = self.read_table_snapshot().await?.unwrap_or_else(|| { - Arc::new(TableSnapshot::new_empty_snapshot(schema.as_ref().clone())) - }); - - let empty_table = base_snapshot.segments.is_empty(); - let replace_into_processor = - ReplaceIntoProcessor::create(on_conflicts.clone(), empty_table); - pipeline.add_pipe(replace_into_processor.into_pipe()); - - // 3. connect to broadcast processor and append transform - - let base_snapshot = self.read_table_snapshot().await?.unwrap_or_else(|| { - Arc::new(TableSnapshot::new_empty_snapshot(schema.as_ref().clone())) - }); - - let max_threads = ctx.get_settings().get_max_threads()?; - let segment_partition_num = - std::cmp::min(base_snapshot.segments.len(), max_threads as usize); - - let serialize_block_transform = TransformSerializeBlock::try_create( - ctx.clone(), - InputPort::create(), - OutputPort::create(), - self, - cluster_stats_gen, - )?; - let block_builder = serialize_block_transform.get_block_builder(); - - let serialize_segment_transform = TransformSerializeSegment::new( - InputPort::create(), - OutputPort::create(), - self, - self.get_block_thresholds(), - ); - - if segment_partition_num == 0 { - let dummy_item = create_dummy_item(); - // ┌──────────────────────┐ ┌──────────────────┐ - // │ ├──┬────────►│ SerializeBlock │ - // ┌─────────────┐ │ ├──┘ └──────────────────┘ - // │ UpsertSource├─────►│ ReplaceIntoProcessor │ - // └─────────────┘ │ ├──┐ ┌──────────────────┐ - // │ ├──┴────────►│ DummyTransform │ - // └──────────────────────┘ └──────────────────┘ - // wrap them into pipeline, order matters! - pipeline.add_pipe(Pipe::create(2, 2, vec![ - serialize_block_transform.into_pipe_item(), - dummy_item, - ])); - } else { - // ┌──────────────────────┐ ┌──────────────────┐ - // │ ├──┬────────►│ SerializeBlock │ - // ┌─────────────┐ │ ├──┘ └──────────────────┘ - // │ UpsertSource├─────►│ ReplaceIntoProcessor │ - // └─────────────┘ │ ├──┐ ┌──────────────────┐ - // │ ├──┴────────►│BroadcastProcessor│ - // └──────────────────────┘ └──────────────────┘ - let broadcast_processor = BroadcastProcessor::new(segment_partition_num); - // wrap them into pipeline, order matters! - pipeline.add_pipe(Pipe::create(2, segment_partition_num + 1, vec![ - serialize_block_transform.into_pipe_item(), - broadcast_processor.into_pipe_item(), - ])); - }; - - // 4. connect with MergeIntoOperationAggregators - if segment_partition_num == 0 { - let dummy_item = create_dummy_item(); - pipeline.add_pipe(Pipe::create(2, 2, vec![ - serialize_segment_transform.into_pipe_item(), - dummy_item, - ])); - } else { - // ┌──────────────────┐ ┌────────────────┐ - // ────►│ SerializeBlock ├──────────────►│SerializeSegment│ - // └──────────────────┘ └────────────────┘ - // - // ┌───────────────────┐ ┌──────────────────────┐ - // ────►│ ├──┬──────────►│MergeIntoOperationAggr│ - // │ ├──┘ └──────────────────────┘ - // │ BroadcastProcessor│ - // │ ├──┐ ┌──────────────────────┐ - // │ ├──┴──────────►│MergeIntoOperationAggr│ - // │ │ └──────────────────────┘ - // │ ├──┐ - // │ ├──┴──────────►┌──────────────────────┐ - // └───────────────────┘ │MergeIntoOperationAggr│ - // └──────────────────────┘ - - let item_size = segment_partition_num + 1; - let mut pipe_items = Vec::with_capacity(item_size); - // setup the dummy transform - pipe_items.push(serialize_segment_transform.into_pipe_item()); - - let max_io_request = ctx.get_settings().get_max_storage_io_requests()?; - let io_request_semaphore = Arc::new(Semaphore::new(max_io_request as usize)); - - // setup the merge into operation aggregators - let mut merge_into_operation_aggregators = self - .merge_into_mutators( - ctx.clone(), - segment_partition_num, - block_builder, - on_conflicts.clone(), - &base_snapshot, - io_request_semaphore, - ) - .await?; - assert_eq!( - segment_partition_num, - merge_into_operation_aggregators.len() - ); - pipe_items.append(&mut merge_into_operation_aggregators); - - // extend the pipeline - assert_eq!(pipeline.output_len(), item_size); - assert_eq!(pipe_items.len(), item_size); - pipeline.add_pipe(Pipe::create(item_size, item_size, pipe_items)); - } - - // 5. connect with mutation pipes, the TableMutationAggregator, then CommitSink - // - // ┌───────────────────┐ ┌───────────────────────┐ ┌───────────────────┐ - // │ResizeProcessor(1) ├──────►│TableMutationAggregator├────────►│ CommitSink │ - // └───────────────────┘ └───────────────────────┘ └───────────────────┘ - self.chain_mutation_pipes(&ctx, pipeline, base_snapshot, MutationKind::Replace)?; - - Ok(()) - } - - #[async_backtrace::framed] - async fn merge_into_mutators( + pub fn merge_into_mutators( &self, ctx: Arc, num_partition: usize, diff --git a/src/query/storages/fuse/src/operations/replace_into/mod.rs b/src/query/storages/fuse/src/operations/replace_into/mod.rs index 8a173111eb4f2..0411fd1acfd16 100644 --- a/src/query/storages/fuse/src/operations/replace_into/mod.rs +++ b/src/query/storages/fuse/src/operations/replace_into/mod.rs @@ -20,8 +20,4 @@ pub use processors::BroadcastProcessor; pub use processors::MergeIntoOperationAggregator; pub use processors::ReplaceIntoProcessor; -#[derive(Clone, Debug)] -pub struct OnConflictField { - pub table_field: common_expression::TableField, - pub field_index: common_expression::FieldIndex, -} + diff --git a/src/query/storages/fuse/src/operations/replace_into/mutator/merge_into_mutator.rs b/src/query/storages/fuse/src/operations/replace_into/mutator/merge_into_mutator.rs index 69e7d014ee733..dbcbb5dff0a7b 100644 --- a/src/query/storages/fuse/src/operations/replace_into/mutator/merge_into_mutator.rs +++ b/src/query/storages/fuse/src/operations/replace_into/mutator/merge_into_mutator.rs @@ -33,6 +33,7 @@ use common_expression::FieldIndex; use common_expression::Scalar; use common_expression::TableSchema; use common_sql::evaluator::BlockOperator; +use common_sql::executor::OnConflictField; use opendal::Operator; use storages_common_cache::LoadParams; use storages_common_table_meta::meta::BlockMeta; @@ -58,7 +59,6 @@ use crate::operations::replace_into::meta::merge_into_operation_meta::MergeIntoO use crate::operations::replace_into::meta::merge_into_operation_meta::UniqueKeyDigest; use crate::operations::replace_into::mutator::column_hash::row_hash_of_columns; use crate::operations::replace_into::mutator::deletion_accumulator::DeletionAccumulator; -use crate::operations::replace_into::OnConflictField; struct AggregationContext { segment_locations: AHashMap, // the fields specified in ON CONFLICT clause diff --git a/src/query/storages/fuse/src/operations/replace_into/mutator/mutator_replace_into.rs b/src/query/storages/fuse/src/operations/replace_into/mutator/mutator_replace_into.rs index 9d07fb05271fb..d6d3cd231683e 100644 --- a/src/query/storages/fuse/src/operations/replace_into/mutator/mutator_replace_into.rs +++ b/src/query/storages/fuse/src/operations/replace_into/mutator/mutator_replace_into.rs @@ -20,12 +20,12 @@ use common_expression::Column; use common_expression::DataBlock; use common_expression::Scalar; use common_functions::aggregates::eval_aggr; +use common_sql::executor::OnConflictField; use crate::operations::replace_into::meta::merge_into_operation_meta::DeletionByColumn; use crate::operations::replace_into::meta::merge_into_operation_meta::MergeIntoOperation; use crate::operations::replace_into::meta::merge_into_operation_meta::UniqueKeyDigest; use crate::operations::replace_into::mutator::column_hash::row_hash_of_columns; -use crate::operations::replace_into::OnConflictField; // Replace is somehow a simplified merge_into, which // - do insertion for "matched" branch diff --git a/src/query/storages/fuse/src/operations/replace_into/processors/processor_replace_into.rs b/src/query/storages/fuse/src/operations/replace_into/processors/processor_replace_into.rs index f08f47ce6ddf1..0d38ef34676cc 100644 --- a/src/query/storages/fuse/src/operations/replace_into/processors/processor_replace_into.rs +++ b/src/query/storages/fuse/src/operations/replace_into/processors/processor_replace_into.rs @@ -24,9 +24,9 @@ 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 common_sql::executor::OnConflictField; use crate::operations::replace_into::mutator::mutator_replace_into::ReplaceIntoMutator; -use crate::operations::replace_into::OnConflictField; pub struct ReplaceIntoProcessor { replace_into_mutator: ReplaceIntoMutator, From 0805f3cf4934df6dad13417f624be5c1a12bdaf2 Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Mon, 24 Jul 2023 07:16:16 +0800 Subject: [PATCH 09/40] fix --- .../pipeline/sources/src/async_source.rs | 5 ++++ .../transform_accumulating_async.rs | 3 ++- .../src/interpreters/interpreter_delete.rs | 18 ++++++++----- .../src/interpreters/interpreter_replace.rs | 21 ++++++++++----- .../service/src/pipelines/pipeline_builder.rs | 16 +++++------ src/query/sql/src/executor/format.rs | 6 ++--- src/query/sql/src/executor/physical_plan.rs | 27 +++++++++++++------ .../sql/src/executor/physical_plan_display.rs | 6 ++--- .../sql/src/executor/physical_plan_visitor.rs | 18 +++++++------ src/query/sql/src/executor/profile.rs | 2 +- .../storages/fuse/src/operations/commit.rs | 2 +- .../fuse/src/operations/common/mod.rs | 1 - .../operations/common/mutation_accumulator.rs | 14 +++------- .../common/processors/sink_commit.rs | 6 +++++ .../transform_mutation_aggregator.rs | 2 +- src/query/storages/fuse/src/operations/mod.rs | 1 - .../storages/fuse/src/operations/recluster.rs | 2 +- .../storages/fuse/src/operations/replace.rs | 2 +- .../processors/processor_replace_into.rs | 3 +++ .../storages/fuse/src/operations/update.rs | 2 +- 20 files changed, 94 insertions(+), 63 deletions(-) diff --git a/src/query/pipeline/sources/src/async_source.rs b/src/query/pipeline/sources/src/async_source.rs index a8e0dfc1b2ca8..e1597a2ff8ea8 100644 --- a/src/query/pipeline/sources/src/async_source.rs +++ b/src/query/pipeline/sources/src/async_source.rs @@ -24,6 +24,7 @@ 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 tracing::error; #[async_trait::async_trait] pub trait AsyncSource: Send { @@ -90,6 +91,10 @@ impl Processor for AsyncSourcer { match self.generated_data.take() { None => Ok(Event::Async), Some(data_block) => { + error!( + "Async source: Pushed data block to output port,{:?}", + data_block.columns() + ); self.output.push_data(Ok(data_block)); Ok(Event::NeedConsume) } diff --git a/src/query/pipeline/transforms/src/processors/transforms/transform_accumulating_async.rs b/src/query/pipeline/transforms/src/processors/transforms/transform_accumulating_async.rs index 951633a4471cc..d8a7ef5959ede 100644 --- a/src/query/pipeline/transforms/src/processors/transforms/transform_accumulating_async.rs +++ b/src/query/pipeline/transforms/src/processors/transforms/transform_accumulating_async.rs @@ -21,7 +21,6 @@ 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; - #[async_trait::async_trait] pub trait AsyncAccumulatingTransform: Send { const NAME: &'static str; @@ -82,6 +81,7 @@ impl Processor for AsyncAccumulatingTra } if let Some(data_block) = self.output_data.take() { + eprintln!("{} - send data block: {:?}", self.name(), data_block.get_meta()); self.output.push_data(Ok(data_block)); return Ok(Event::NeedConsume); } @@ -92,6 +92,7 @@ impl Processor for AsyncAccumulatingTra if self.input.has_data() { self.input_data = Some(self.input.pull_data().unwrap()?); + eprintln!("{} - recv data block: {:?}", self.name(), self.input_data.as_ref().unwrap().get_meta()); return Ok(Event::Async); } diff --git a/src/query/service/src/interpreters/interpreter_delete.rs b/src/query/service/src/interpreters/interpreter_delete.rs index 9f48cf2740de4..0b9408e1df7b3 100644 --- a/src/query/service/src/interpreters/interpreter_delete.rs +++ b/src/query/service/src/interpreters/interpreter_delete.rs @@ -28,11 +28,12 @@ use common_expression::RemoteExpr; use common_expression::ROW_ID_COL_NAME; use common_functions::BUILTIN_FUNCTIONS; use common_meta_app::schema::TableInfo; +use common_sql::executor::MutationKind; use common_sql::executor::cast_expr_to_non_null_boolean; -use common_sql::executor::DeleteFinal; use common_sql::executor::DeletePartial; use common_sql::executor::Exchange; use common_sql::executor::FragmentKind; +use common_sql::executor::MutationAggregate; use common_sql::executor::PhysicalPlan; use common_sql::optimizer::CascadesOptimizer; use common_sql::optimizer::HeuristicOptimizer; @@ -291,12 +292,15 @@ impl DeleteInterpreter { }); } - Ok(PhysicalPlan::DeleteFinal(Box::new(DeleteFinal { - input: Box::new(root), - snapshot, - table_info, - catalog_name, - }))) + Ok(PhysicalPlan::MutationAggregate(Box::new( + MutationAggregate { + input: Box::new(root), + snapshot, + table_info, + catalog_name, + mutation_kind: MutationKind::Delete, + }, + ))) } } diff --git a/src/query/service/src/interpreters/interpreter_replace.rs b/src/query/service/src/interpreters/interpreter_replace.rs index 8118b53ee5bbd..6de083f802f6b 100644 --- a/src/query/service/src/interpreters/interpreter_replace.rs +++ b/src/query/service/src/interpreters/interpreter_replace.rs @@ -21,7 +21,8 @@ use common_expression::DataSchema; use common_expression::DataSchemaRef; use common_sql::executor::AsyncSourcerPlan; use common_sql::executor::Deduplicate; -use common_sql::executor::DeleteFinal; +use common_sql::executor::MutationAggregate; +use common_sql::executor::MutationKind; use common_sql::executor::OnConflictField; use common_sql::executor::PhysicalPlan; use common_sql::executor::ReplaceInto; @@ -32,6 +33,7 @@ use common_sql::plans::Replace; use common_storages_factory::Table; use common_storages_fuse::FuseTable; use storages_common_table_meta::meta::TableSnapshot; +use tracing::error; use crate::interpreters::common::check_deduplicate_label; use crate::interpreters::interpreter_copy::CopyInterpreter; @@ -69,9 +71,11 @@ impl Interpreter for ReplaceInterpreter { self.check_on_conflicts()?; let physical_plan = self.build_physical_plan().await?; + error!("physical_plan: {:?}", physical_plan); let build_res = build_query_pipeline_without_render_result_set(&self.ctx, &physical_plan, false) .await?; + error!("build_res: {:?}", build_res.main_pipeline); Ok(build_res) } } @@ -138,12 +142,15 @@ impl ReplaceInterpreter { on_conflicts, snapshot: (*base_snapshot).clone(), })); - root = Box::new(PhysicalPlan::DeleteFinal(Box::new(DeleteFinal { - input: root, - snapshot: (*base_snapshot).clone(), - table_info: table_info.clone(), - catalog_name: plan.catalog.clone(), - }))); + root = Box::new(PhysicalPlan::MutationAggregate(Box::new( + MutationAggregate { + input: root, + snapshot: (*base_snapshot).clone(), + table_info: table_info.clone(), + catalog_name: plan.catalog.clone(), + mutation_kind: MutationKind::Replace, + }, + ))); Ok(root) } fn check_on_conflicts(&self) -> Result<()> { diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs index 0a3a3767774d5..88022e3ae65c4 100644 --- a/src/query/service/src/pipelines/pipeline_builder.rs +++ b/src/query/service/src/pipelines/pipeline_builder.rs @@ -74,7 +74,6 @@ use common_sql::executor::AsyncSourcerPlan; use common_sql::executor::CopyIntoTable; use common_sql::executor::CopyIntoTableSource; use common_sql::executor::Deduplicate; -use common_sql::executor::DeleteFinal; use common_sql::executor::DeletePartial; use common_sql::executor::DistributedInsertSelect; use common_sql::executor::EvalScalar; @@ -83,6 +82,8 @@ use common_sql::executor::ExchangeSource; use common_sql::executor::Filter; use common_sql::executor::HashJoin; use common_sql::executor::Limit; +use common_sql::executor::MutationAggregate; +use common_sql::executor::MutationKind; use common_sql::executor::PhysicalPlan; use common_sql::executor::Project; use common_sql::executor::ProjectSet; @@ -107,7 +108,6 @@ use common_storages_fuse::operations::common::TransformSerializeSegment; use common_storages_fuse::operations::replace_into::BroadcastProcessor; use common_storages_fuse::operations::replace_into::ReplaceIntoProcessor; use common_storages_fuse::operations::FillInternalColumnProcessor; -use common_storages_fuse::operations::MutationKind; use common_storages_fuse::operations::TransformSerializeBlock; use common_storages_fuse::FuseTable; use common_storages_stage::StageTable; @@ -231,7 +231,7 @@ impl PipelineBuilder { self.build_runtime_filter_source(runtime_filter_source) } PhysicalPlan::DeletePartial(delete) => self.build_delete_partial(delete), - PhysicalPlan::DeleteFinal(delete) => self.build_delete_final(delete), + PhysicalPlan::MutationAggregate(plan) => self.build_mutation_aggregate(plan), PhysicalPlan::RangeJoin(range_join) => self.build_range_join(range_join), PhysicalPlan::CopyIntoTable(copy) => self.build_copy_into_table(copy), PhysicalPlan::AsyncSourcer(async_sourcer) => self.build_async_sourcer(async_sourcer), @@ -493,18 +493,18 @@ impl PipelineBuilder { /// +-----------------------+ +----------+ /// |TableMutationAggregator| ---> |CommitSink| /// +-----------------------+ +----------+ - fn build_delete_final(&mut self, delete: &DeleteFinal) -> Result<()> { - self.build_pipeline(&delete.input)?; + fn build_mutation_aggregate(&mut self, plan: &MutationAggregate) -> Result<()> { + self.build_pipeline(&plan.input)?; let table = self.ctx - .build_table_by_table_info(&delete.catalog_name, &delete.table_info, None)?; + .build_table_by_table_info(&plan.catalog_name, &plan.table_info, None)?; let table = FuseTable::try_from_table(table.as_ref())?; let ctx: Arc = self.ctx.clone(); table.chain_mutation_pipes( &ctx, &mut self.main_pipeline, - Arc::new(delete.snapshot.clone()), - MutationKind::Delete, + Arc::new(plan.snapshot.clone()), + plan.mutation_kind, )?; Ok(()) } diff --git a/src/query/sql/src/executor/format.rs b/src/query/sql/src/executor/format.rs index c44c9b4a084eb..0b5832a679d57 100644 --- a/src/query/sql/src/executor/format.rs +++ b/src/query/sql/src/executor/format.rs @@ -24,13 +24,13 @@ use super::AggregateFinal; use super::AggregateFunctionDesc; use super::AggregatePartial; use super::CopyIntoTable; -use super::DeleteFinal; use super::DeletePartial; use super::EvalScalar; use super::Exchange; use super::Filter; use super::HashJoin; use super::Limit; +use super::MutationAggregate; use super::PhysicalPlan; use super::Project; use super::ProjectSet; @@ -164,7 +164,7 @@ fn to_format_tree( PhysicalPlan::DeletePartial(plan) => { delete_partial_to_format_tree(plan.as_ref(), metadata, prof_span_set) } - PhysicalPlan::DeleteFinal(plan) => { + PhysicalPlan::MutationAggregate(plan) => { delete_final_to_format_tree(plan.as_ref(), metadata, prof_span_set) } PhysicalPlan::ProjectSet(plan) => project_set_to_format_tree(plan, metadata, prof_span_set), @@ -921,7 +921,7 @@ fn delete_partial_to_format_tree( } fn delete_final_to_format_tree( - plan: &DeleteFinal, + plan: &MutationAggregate, metadata: &MetadataRef, prof_span_set: &SharedProcessorProfiles, ) -> Result> { diff --git a/src/query/sql/src/executor/physical_plan.rs b/src/query/sql/src/executor/physical_plan.rs index e668a0fc525c2..0dc4425d7f1ab 100644 --- a/src/query/sql/src/executor/physical_plan.rs +++ b/src/query/sql/src/executor/physical_plan.rs @@ -787,7 +787,7 @@ impl DeletePartial { } } -impl DeleteFinal { +impl MutationAggregate { pub fn output_schema(&self) -> Result { Ok(DataSchemaRef::default()) } @@ -795,11 +795,22 @@ impl DeleteFinal { // TODO(sky): make TableMutationAggregator distributed #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct DeleteFinal { +pub struct MutationAggregate { pub input: Box, pub snapshot: TableSnapshot, pub table_info: TableInfo, pub catalog_name: String, + pub mutation_kind: MutationKind, +} + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize, Copy)] +/// This is used by MutationAccumulator, so no compact here. +pub enum MutationKind { + Delete, + Update, + Replace, + Recluster, + Insert, } #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] @@ -887,7 +898,7 @@ pub enum PhysicalPlan { /// Delete DeletePartial(Box), - DeleteFinal(Box), + MutationAggregate(Box), /// Copy into table CopyIntoTable(Box), /// Replace @@ -929,7 +940,7 @@ impl PhysicalPlan { PhysicalPlan::ExchangeSource(v) => v.plan_id, PhysicalPlan::ExchangeSink(v) => v.plan_id, PhysicalPlan::DeletePartial(_) - | PhysicalPlan::DeleteFinal(_) + | PhysicalPlan::MutationAggregate(_) | PhysicalPlan::CopyIntoTable(_) | PhysicalPlan::AsyncSourcer(_) | PhysicalPlan::Deduplicate(_) @@ -961,7 +972,7 @@ impl PhysicalPlan { PhysicalPlan::ProjectSet(plan) => plan.output_schema(), PhysicalPlan::RuntimeFilterSource(plan) => plan.output_schema(), PhysicalPlan::DeletePartial(plan) => plan.output_schema(), - PhysicalPlan::DeleteFinal(plan) => plan.output_schema(), + PhysicalPlan::MutationAggregate(plan) => plan.output_schema(), PhysicalPlan::RangeJoin(plan) => plan.output_schema(), PhysicalPlan::CopyIntoTable(plan) => plan.output_schema(), PhysicalPlan::AsyncSourcer(_) @@ -992,7 +1003,7 @@ impl PhysicalPlan { PhysicalPlan::ProjectSet(_) => "Unnest".to_string(), PhysicalPlan::RuntimeFilterSource(_) => "RuntimeFilterSource".to_string(), PhysicalPlan::DeletePartial(_) => "DeletePartial".to_string(), - PhysicalPlan::DeleteFinal(_) => "DeleteFinal".to_string(), + PhysicalPlan::MutationAggregate(_) => "DeleteFinal".to_string(), PhysicalPlan::RangeJoin(_) => "RangeJoin".to_string(), PhysicalPlan::CopyIntoTable(_) => "CopyIntoTable".to_string(), PhysicalPlan::AsyncSourcer(_) => "AsyncSourcer".to_string(), @@ -1027,7 +1038,7 @@ impl PhysicalPlan { Box::new(std::iter::once(plan.input.as_ref())) } PhysicalPlan::DeletePartial(_plan) => Box::new(std::iter::empty()), - PhysicalPlan::DeleteFinal(plan) => Box::new(std::iter::once(plan.input.as_ref())), + PhysicalPlan::MutationAggregate(plan) => Box::new(std::iter::once(plan.input.as_ref())), PhysicalPlan::ProjectSet(plan) => Box::new(std::iter::once(plan.input.as_ref())), PhysicalPlan::RuntimeFilterSource(plan) => Box::new( std::iter::once(plan.left_side.as_ref()) @@ -1067,7 +1078,7 @@ impl PhysicalPlan { | PhysicalPlan::AggregateFinal(_) | PhysicalPlan::AggregatePartial(_) | PhysicalPlan::DeletePartial(_) - | PhysicalPlan::DeleteFinal(_) + | PhysicalPlan::MutationAggregate(_) | PhysicalPlan::CopyIntoTable(_) | PhysicalPlan::AsyncSourcer(_) | PhysicalPlan::Deduplicate(_) diff --git a/src/query/sql/src/executor/physical_plan_display.rs b/src/query/sql/src/executor/physical_plan_display.rs index 7af613682c4f6..3c0d6cc829372 100644 --- a/src/query/sql/src/executor/physical_plan_display.rs +++ b/src/query/sql/src/executor/physical_plan_display.rs @@ -22,9 +22,9 @@ use super::AggregateExpand; use super::AsyncSourcerPlan; use super::CopyIntoTable; use super::Deduplicate; -use super::DeleteFinal; use super::DeletePartial; use super::DistributedInsertSelect; +use super::MutationAggregate; use super::ProjectSet; use super::ReplaceInto; use super::RowFetch; @@ -81,7 +81,7 @@ impl<'a> Display for PhysicalPlanIndentFormatDisplay<'a> { PhysicalPlan::UnionAll(union_all) => write!(f, "{}", union_all)?, PhysicalPlan::DistributedInsertSelect(insert_select) => write!(f, "{}", insert_select)?, PhysicalPlan::DeletePartial(delete) => write!(f, "{}", delete)?, - PhysicalPlan::DeleteFinal(delete) => write!(f, "{}", delete)?, + PhysicalPlan::MutationAggregate(delete) => write!(f, "{}", delete)?, PhysicalPlan::ProjectSet(unnest) => write!(f, "{}", unnest)?, PhysicalPlan::RuntimeFilterSource(plan) => write!(f, "{}", plan)?, PhysicalPlan::RangeJoin(plan) => write!(f, "{}", plan)?, @@ -364,7 +364,7 @@ impl Display for DeletePartial { } } -impl Display for DeleteFinal { +impl Display for MutationAggregate { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { write!(f, "DeleteFinal") } diff --git a/src/query/sql/src/executor/physical_plan_visitor.rs b/src/query/sql/src/executor/physical_plan_visitor.rs index e5d0eea6633ce..6090cbb582b9a 100644 --- a/src/query/sql/src/executor/physical_plan_visitor.rs +++ b/src/query/sql/src/executor/physical_plan_visitor.rs @@ -21,7 +21,6 @@ use super::AsyncSourcerPlan; use super::CopyIntoTable; use super::CopyIntoTableSource; use super::Deduplicate; -use super::DeleteFinal; use super::DeletePartial; use super::DistributedInsertSelect; use super::EvalScalar; @@ -31,6 +30,7 @@ use super::ExchangeSource; use super::Filter; use super::HashJoin; use super::Limit; +use super::MutationAggregate; use super::PhysicalPlan; use super::Project; use super::ProjectSet; @@ -66,7 +66,7 @@ pub trait PhysicalPlanReplacer { PhysicalPlan::ProjectSet(plan) => self.replace_project_set(plan), PhysicalPlan::RuntimeFilterSource(plan) => self.replace_runtime_filter_source(plan), PhysicalPlan::DeletePartial(plan) => self.replace_delete_partial(plan), - PhysicalPlan::DeleteFinal(plan) => self.replace_delete_final(plan), + PhysicalPlan::MutationAggregate(plan) => self.replace_delete_final(plan), PhysicalPlan::RangeJoin(plan) => self.replace_range_join(plan), PhysicalPlan::CopyIntoTable(plan) => self.replace_copy_into_table(plan), PhysicalPlan::AsyncSourcer(plan) => self.replace_async_sourcer(plan), @@ -322,12 +322,14 @@ pub trait PhysicalPlanReplacer { Ok(PhysicalPlan::DeletePartial(Box::new(plan.clone()))) } - fn replace_delete_final(&mut self, plan: &DeleteFinal) -> Result { + fn replace_delete_final(&mut self, plan: &MutationAggregate) -> Result { let input = self.replace(&plan.input)?; - Ok(PhysicalPlan::DeleteFinal(Box::new(DeleteFinal { - input: Box::new(input), - ..plan.clone() - }))) + Ok(PhysicalPlan::MutationAggregate(Box::new( + MutationAggregate { + input: Box::new(input), + ..plan.clone() + }, + ))) } fn replace_async_sourcer(&mut self, plan: &AsyncSourcerPlan) -> Result { @@ -455,7 +457,7 @@ impl PhysicalPlan { Self::traverse(&plan.right, pre_visit, visit, post_visit); } PhysicalPlan::DeletePartial(_) => {} - PhysicalPlan::DeleteFinal(plan) => { + PhysicalPlan::MutationAggregate(plan) => { Self::traverse(&plan.input, pre_visit, visit, post_visit); } PhysicalPlan::Deduplicate(plan) => { diff --git a/src/query/sql/src/executor/profile.rs b/src/query/sql/src/executor/profile.rs index 093f00513ea40..dcf91e7f14943 100644 --- a/src/query/sql/src/executor/profile.rs +++ b/src/query/sql/src/executor/profile.rs @@ -451,7 +451,7 @@ fn flatten_plan_node_profile( plan_node_profs.push(prof); } PhysicalPlan::DeletePartial(_) - | PhysicalPlan::DeleteFinal(_) + | PhysicalPlan::MutationAggregate(_) | PhysicalPlan::CopyIntoTable(_) | PhysicalPlan::AsyncSourcer(_) | PhysicalPlan::Deduplicate(_) diff --git a/src/query/storages/fuse/src/operations/commit.rs b/src/query/storages/fuse/src/operations/commit.rs index 72ec0041725d7..76dfdd214eacc 100644 --- a/src/query/storages/fuse/src/operations/commit.rs +++ b/src/query/storages/fuse/src/operations/commit.rs @@ -33,6 +33,7 @@ use common_meta_types::MatchSeq; use common_pipeline_core::processors::processor::ProcessorPtr; use common_pipeline_core::Pipeline; use common_pipeline_transforms::processors::transforms::AsyncAccumulatingTransformer; +use common_sql::executor::MutationKind; use opendal::Operator; use storages_common_cache::CacheAccessor; use storages_common_cache_manager::CachedObject; @@ -47,7 +48,6 @@ use storages_common_table_meta::table::OPT_KEY_SNAPSHOT_LOCATION; use tracing::info; use tracing::warn; -use super::common::MutationKind; use crate::io::MetaWriter; use crate::io::SegmentsIO; use crate::io::TableMetaLocationGenerator; diff --git a/src/query/storages/fuse/src/operations/common/mod.rs b/src/query/storages/fuse/src/operations/common/mod.rs index 145c1cd41ffb2..9c986e143f57a 100644 --- a/src/query/storages/fuse/src/operations/common/mod.rs +++ b/src/query/storages/fuse/src/operations/common/mod.rs @@ -20,7 +20,6 @@ mod snapshot_generator; pub use abort_operation::AbortOperation; pub use mutation_accumulator::MutationAccumulator; -pub use mutation_accumulator::MutationKind; pub use mutation_log::*; pub use processors::*; pub use snapshot_generator::*; diff --git a/src/query/storages/fuse/src/operations/common/mutation_accumulator.rs b/src/query/storages/fuse/src/operations/common/mutation_accumulator.rs index edccd47ee42a9..ffe2e4267ba88 100644 --- a/src/query/storages/fuse/src/operations/common/mutation_accumulator.rs +++ b/src/query/storages/fuse/src/operations/common/mutation_accumulator.rs @@ -22,6 +22,7 @@ use common_catalog::table_context::TableContext; use common_exception::Result; use common_expression::BlockThresholds; use common_expression::TableSchemaRef; +use common_sql::executor::MutationKind; use opendal::Operator; use storages_common_table_meta::meta::BlockMeta; use storages_common_table_meta::meta::FormatVersion; @@ -30,6 +31,7 @@ use storages_common_table_meta::meta::SegmentInfo; use storages_common_table_meta::meta::Statistics; use storages_common_table_meta::meta::Versioned; use tracing::error; +use tracing::info; use super::ConflictResolveContext; use super::SnapshotChanges; @@ -77,16 +79,6 @@ impl BlockMutations { } } -#[derive(Clone, Copy)] -/// This is used by MutationAccumulator, so no compact here. -pub enum MutationKind { - Delete, - Update, - Replace, - Recluster, - Insert, -} - pub struct MutationAccumulator { ctx: Arc, schema: TableSchemaRef, @@ -245,6 +237,8 @@ impl MutationAccumulator { .map(|(path, _segment, format_version)| (path.clone(), *format_version)) .chain(merged_segments) .collect(); + info!("merged_segments:{:?}", merged_segments); + info!("append_segments:{:?}", self.appended_segments); match self.kind { MutationKind::Insert => ConflictResolveContext::AppendOnly(SnapshotMerged { merged_segments, diff --git a/src/query/storages/fuse/src/operations/common/processors/sink_commit.rs b/src/query/storages/fuse/src/operations/common/processors/sink_commit.rs index 1a97aa027318f..0912d13c09460 100644 --- a/src/query/storages/fuse/src/operations/common/processors/sink_commit.rs +++ b/src/query/storages/fuse/src/operations/common/processors/sink_commit.rs @@ -32,6 +32,7 @@ use storages_common_table_meta::meta::TableSnapshot; use storages_common_table_meta::meta::Versioned; use table_lock::TableLockHandlerWrapper; use table_lock::TableLockHeartbeat; +use tracing::info; use crate::io::TableMetaLocationGenerator; use crate::metrics::metrics_inc_commit_aborts; @@ -142,6 +143,10 @@ where F: SnapshotGenerator + Send + 'static self.abort_operation = meta.abort_operation; self.backoff = FuseTable::set_backoff(self.max_retry_elapsed); + info!( + "conflict resolve context: {:?}", + meta.conflict_resolve_context + ); self.snapshot_gen .set_conflict_resolve_context(meta.conflict_resolve_context); @@ -210,6 +215,7 @@ where F: SnapshotGenerator + Send + 'static .generate_new_snapshot(schema, cluster_key_meta, previous) { Ok(snapshot) => { + info!("generated snapshot: {:?}", snapshot); self.state = State::TryCommit { data: snapshot.to_bytes()?, snapshot, diff --git a/src/query/storages/fuse/src/operations/common/processors/transform_mutation_aggregator.rs b/src/query/storages/fuse/src/operations/common/processors/transform_mutation_aggregator.rs index 32e474ee5d1c3..d2e98bceaad85 100644 --- a/src/query/storages/fuse/src/operations/common/processors/transform_mutation_aggregator.rs +++ b/src/query/storages/fuse/src/operations/common/processors/transform_mutation_aggregator.rs @@ -22,13 +22,13 @@ use common_expression::BlockThresholds; use common_expression::DataBlock; use common_expression::TableSchemaRef; use common_pipeline_transforms::processors::transforms::transform_accumulating_async::AsyncAccumulatingTransform; +use common_sql::executor::MutationKind; use opendal::Operator; use storages_common_table_meta::meta::Location; use storages_common_table_meta::meta::Statistics; use tracing::debug; use crate::io::TableMetaLocationGenerator; -use crate::operations::common::mutation_accumulator::MutationKind; use crate::operations::common::CommitMeta; use crate::operations::common::MutationAccumulator; use crate::operations::common::MutationLogs; diff --git a/src/query/storages/fuse/src/operations/mod.rs b/src/query/storages/fuse/src/operations/mod.rs index 5e355ff108152..77a50bda99d55 100644 --- a/src/query/storages/fuse/src/operations/mod.rs +++ b/src/query/storages/fuse/src/operations/mod.rs @@ -36,7 +36,6 @@ pub mod util; pub use agg_index_sink::AggIndexSink; pub use common::BlockMetaIndex; pub use common::FillInternalColumnProcessor; -pub use common::MutationKind; pub use common::TransformSerializeBlock; pub use compact::CompactOptions; pub use mutation::BlockCompactMutator; diff --git a/src/query/storages/fuse/src/operations/recluster.rs b/src/query/storages/fuse/src/operations/recluster.rs index 94ee92b1a4ca4..4214fbcc22902 100644 --- a/src/query/storages/fuse/src/operations/recluster.rs +++ b/src/query/storages/fuse/src/operations/recluster.rs @@ -30,12 +30,12 @@ use common_pipeline_core::processors::processor::ProcessorPtr; use common_pipeline_transforms::processors::transforms::build_full_sort_pipeline; use common_pipeline_transforms::processors::transforms::AsyncAccumulatingTransformer; use common_sql::evaluator::CompoundBlockOperator; +use common_sql::executor::MutationKind; use storages_common_table_meta::meta::BlockMeta; use crate::operations::common::BlockMetaIndex; use crate::operations::common::CommitSink; use crate::operations::common::MutationGenerator; -use crate::operations::common::MutationKind; use crate::operations::common::TableMutationAggregator; use crate::operations::common::TransformSerializeBlock; use crate::operations::common::TransformSerializeSegment; diff --git a/src/query/storages/fuse/src/operations/replace.rs b/src/query/storages/fuse/src/operations/replace.rs index a2ce8d80329b4..08f1a04a48722 100644 --- a/src/query/storages/fuse/src/operations/replace.rs +++ b/src/query/storages/fuse/src/operations/replace.rs @@ -21,12 +21,12 @@ use common_exception::Result; use common_pipeline_core::pipe::PipeItem; use common_pipeline_core::processors::processor::ProcessorPtr; use common_pipeline_transforms::processors::transforms::AsyncAccumulatingTransformer; +use common_sql::executor::MutationKind; use common_sql::executor::OnConflictField; use rand::prelude::SliceRandom; use storages_common_table_meta::meta::Location; use storages_common_table_meta::meta::TableSnapshot; -use super::common::MutationKind; use crate::io::BlockBuilder; use crate::io::ReadSettings; use crate::operations::common::CommitSink; diff --git a/src/query/storages/fuse/src/operations/replace_into/processors/processor_replace_into.rs b/src/query/storages/fuse/src/operations/replace_into/processors/processor_replace_into.rs index 0d38ef34676cc..6f4d7f659040b 100644 --- a/src/query/storages/fuse/src/operations/replace_into/processors/processor_replace_into.rs +++ b/src/query/storages/fuse/src/operations/replace_into/processors/processor_replace_into.rs @@ -25,6 +25,7 @@ use common_pipeline_core::processors::processor::Event; use common_pipeline_core::processors::processor::ProcessorPtr; use common_pipeline_core::processors::Processor; use common_sql::executor::OnConflictField; +use tracing::error; use crate::operations::replace_into::mutator::mutator_replace_into::ReplaceIntoMutator; @@ -102,6 +103,7 @@ impl Processor for ReplaceIntoProcessor { let mut pushed_something = false; if self.output_port_append_data.can_push() { if let Some(data) = self.output_data_append.take() { + error!("replace into processor: output data: {:?}", data); self.output_port_append_data.push_data(Ok(data)); pushed_something = true; } @@ -126,6 +128,7 @@ impl Processor for ReplaceIntoProcessor { { // no pending data (being sent to down streams) self.input_data = Some(self.input_port.pull_data().unwrap()?); + error!("replace into processor: input data: {:?}", self.input_data); Ok(Event::Sync) } else { // data pending diff --git a/src/query/storages/fuse/src/operations/update.rs b/src/query/storages/fuse/src/operations/update.rs index cbaa68587ed2e..24d6d068fdb7e 100644 --- a/src/query/storages/fuse/src/operations/update.rs +++ b/src/query/storages/fuse/src/operations/update.rs @@ -28,11 +28,11 @@ use common_expression::TableSchema; use common_expression::ROW_ID_COL_NAME; use common_functions::BUILTIN_FUNCTIONS; use common_sql::evaluator::BlockOperator; +use common_sql::executor::MutationKind; use common_sql::plans::PREDICATE_COLUMN_NAME; use storages_common_table_meta::meta::TableSnapshot; use tracing::info; -use crate::operations::common::MutationKind; use crate::operations::common::TransformSerializeBlock; use crate::operations::mutation::MutationAction; use crate::operations::mutation::MutationSource; From c010430fed33a4fd0ae7d154d9a90dc4cf839671 Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Tue, 25 Jul 2023 10:31:09 +0800 Subject: [PATCH 10/40] fix --- .../src/interpreters/interpreter_replace.rs | 36 ---------------- .../service/src/pipelines/pipeline_builder.rs | 41 +++++++++++++++++++ 2 files changed, 41 insertions(+), 36 deletions(-) diff --git a/src/query/service/src/interpreters/interpreter_replace.rs b/src/query/service/src/interpreters/interpreter_replace.rs index 6de083f802f6b..fd80334ded8d9 100644 --- a/src/query/service/src/interpreters/interpreter_replace.rs +++ b/src/query/service/src/interpreters/interpreter_replace.rs @@ -233,45 +233,9 @@ impl ReplaceInterpreter { false, )?; - // let mut build_res = select_interpreter.execute2().await?; - - // let select_schema = query_plan.schema(); - // let target_schema = self_schema; - // if self.check_schema_cast(query_plan)? { - // let func_ctx = ctx.get_function_context()?; - // build_res.main_pipeline.add_transform( - // |transform_input_port, transform_output_port| { - // TransformCastSchema::try_create( - // transform_input_port, - // transform_output_port, - // select_schema.clone(), - // target_schema.clone(), - // func_ctx.clone(), - // ) - // }, - // )?; - // } - select_interpreter .build_physical_plan() .await .map(|x| Box::new(x)) } - - // TODO duplicated - fn check_schema_cast(&self, plan: &Plan) -> Result { - let output_schema = &self.plan.schema; - let select_schema = plan.schema(); - - // validate schema - if select_schema.fields().len() < output_schema.fields().len() { - return Err(ErrorCode::BadArguments( - "Fields in select statement is less than expected", - )); - } - - // check if cast needed - let cast_needed = select_schema != DataSchema::from(output_schema.as_ref()).into(); - Ok(cast_needed) - } } diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs index 88022e3ae65c4..1f0dedd4780ab 100644 --- a/src/query/service/src/pipelines/pipeline_builder.rs +++ b/src/query/service/src/pipelines/pipeline_builder.rs @@ -16,6 +16,7 @@ use std::collections::VecDeque; use std::convert::TryFrom; use std::io::BufRead; use std::io::Cursor; +use std::ops::Deref; use std::ops::Not; use std::sync::Arc; @@ -35,11 +36,13 @@ use common_expression::with_mappedhash_method; use common_expression::with_number_mapped_type; use common_expression::ColumnBuilder; use common_expression::DataBlock; +use common_expression::DataSchema; use common_expression::DataSchemaRef; use common_expression::FunctionContext; use common_expression::HashMethodKind; use common_expression::RemoteExpr; use common_expression::SortColumnDescription; +use common_expression::TableSchema; use common_formats::FastFieldDecoderValues; use common_functions::aggregates::AggregateFunctionFactory; use common_functions::aggregates::AggregateFunctionRef; @@ -95,6 +98,7 @@ use common_sql::executor::Sort; use common_sql::executor::TableScan; use common_sql::executor::UnionAll; use common_sql::executor::Window; +use common_sql::plans::InsertInputSource; use common_sql::BindContext; use common_sql::ColumnBinding; use common_sql::IndexType; @@ -240,6 +244,22 @@ impl PipelineBuilder { } } + fn check_schema_cast( + select_schema: Arc, + output_schema: Arc, + ) -> Result { + // validate schema + if select_schema.fields().len() < output_schema.fields().len() { + return Err(ErrorCode::BadArguments( + "Fields in select statement is less than expected", + )); + } + + // check if cast needed + let cast_needed = select_schema != output_schema; + Ok(cast_needed) + } + fn build_deduplicate(&mut self, deduplicate: &Deduplicate) -> Result<()> { let Deduplicate { input, @@ -260,6 +280,27 @@ impl PipelineBuilder { tbl.clone(), schema.clone(), )?; + + if !matches!(input.deref(), PhysicalPlan::AsyncSourcer(_)) + && !matches!(input.deref(), PhysicalPlan::CopyIntoTable(_)) + { + let select_schema = input.output_schema()?; + let target_schema = schema.clone(); + if Self::check_schema_cast(select_schema.clone(), target_schema.clone())? { + let func_ctx = self.ctx.get_function_context()?; + self.main_pipeline.add_transform( + |transform_input_port, transform_output_port| { + TransformCastSchema::try_create( + transform_input_port, + transform_output_port, + select_schema.clone(), + target_schema.clone(), + func_ctx.clone(), + ) + }, + )?; + } + } let _ = table.cluster_gen_for_append( self.ctx.clone(), &mut self.main_pipeline, From ca6b203e2edfd920eb1025679e1b681a8ff2be29 Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Wed, 26 Jul 2023 06:01:26 +0800 Subject: [PATCH 11/40] fix render result --- .../src/interpreters/interpreter_replace.rs | 22 ++++---- .../service/src/pipelines/pipeline_builder.rs | 51 +++++++++++-------- src/query/sql/src/executor/physical_plan.rs | 3 +- 3 files changed, 44 insertions(+), 32 deletions(-) diff --git a/src/query/service/src/interpreters/interpreter_replace.rs b/src/query/service/src/interpreters/interpreter_replace.rs index fd80334ded8d9..9996c99035eeb 100644 --- a/src/query/service/src/interpreters/interpreter_replace.rs +++ b/src/query/service/src/interpreters/interpreter_replace.rs @@ -30,6 +30,7 @@ use common_sql::plans::CopyPlan; use common_sql::plans::InsertInputSource; use common_sql::plans::Plan; use common_sql::plans::Replace; +use common_sql::ColumnBinding; use common_storages_factory::Table; use common_storages_fuse::FuseTable; use storages_common_table_meta::meta::TableSnapshot; @@ -122,7 +123,7 @@ impl ReplaceInterpreter { let max_threads = self.ctx.get_settings().get_max_threads()?; let segment_partition_num = std::cmp::min(base_snapshot.segments.len(), max_threads as usize); - let mut root = self + let (mut root, select_column_bindings) = self .connect_input_source(self.ctx.clone(), &self.plan.source, self.plan.schema()) .await?; root = Box::new(PhysicalPlan::Deduplicate(Deduplicate { @@ -131,7 +132,8 @@ impl ReplaceInterpreter { empty_table, table_info: table_info.clone(), catalog_name: plan.catalog.clone(), - schema: self.plan.schema(), + target_schema: plan.schema(), + select_column_bindings, })); root = Box::new(PhysicalPlan::ReplaceInto(ReplaceInto { input: root, @@ -168,9 +170,11 @@ impl ReplaceInterpreter { ctx: Arc, source: &'a InsertInputSource, schema: DataSchemaRef, - ) -> Result> { + ) -> Result<(Box, Option>)> { match source { - InsertInputSource::Values(data) => self.connect_value_source(schema.clone(), data), + InsertInputSource::Values(data) => self + .connect_value_source(schema.clone(), data) + .map(|x| (x, None)), InsertInputSource::SelectPlan(plan) => { self.connect_query_plan_source(ctx.clone(), plan).await @@ -183,7 +187,7 @@ impl ReplaceInterpreter { interpreter .build_physical_plan(copy_into_table_plan) .await - .map(|x| Box::new(x.0)) + .map(|x| (Box::new(x.0), None)) } _ => unreachable!("plan in InsertInputSource::Stag must be CopyIntoTable"), }, @@ -210,9 +214,8 @@ impl ReplaceInterpreter { async fn connect_query_plan_source<'a>( &'a self, ctx: Arc, - // self_schema: DataSchemaRef, query_plan: &Plan, - ) -> Result> { + ) -> Result<(Box, Option>)> { let (s_expr, metadata, bind_context, formatted_ast) = match query_plan { Plan::Query { s_expr, @@ -233,9 +236,10 @@ impl ReplaceInterpreter { false, )?; - select_interpreter + let physical_plan = select_interpreter .build_physical_plan() .await - .map(|x| Box::new(x)) + .map(|x| Box::new(x))?; + Ok((physical_plan, Some(bind_context.columns.clone()))) } } diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs index 1f0dedd4780ab..5b250a0adcbcc 100644 --- a/src/query/service/src/pipelines/pipeline_builder.rs +++ b/src/query/service/src/pipelines/pipeline_builder.rs @@ -151,6 +151,7 @@ use crate::pipelines::processors::TransformLimit; use crate::pipelines::processors::TransformRuntimeFilter; use crate::pipelines::Pipeline; use crate::pipelines::PipelineBuildResult; +use crate::schedulers::build_query_pipeline; use crate::sessions::QueryContext; use crate::sessions::TableContext; @@ -267,40 +268,46 @@ impl PipelineBuilder { empty_table, table_info, catalog_name, - schema, + target_schema, + select_column_bindings, } = deduplicate; let tbl = self .ctx .build_table_by_table_info(catalog_name, table_info, None)?; let table = FuseTable::try_from_table(tbl.as_ref())?; self.build_pipeline(input)?; + if let Some(select_column_bindings) = select_column_bindings { + let select_schema = input.output_schema()?; + PipelineBuilder::render_result_set( + &self.ctx.get_function_context()?, + select_schema.clone(), + select_column_bindings, + &mut self.main_pipeline, + false, + )?; + // if Self::check_schema_cast(select_schema.clone(), target_schema.clone())? { + // let func_ctx = self.ctx.get_function_context()?; + // self.main_pipeline.add_transform( + // |transform_input_port, transform_output_port| { + // TransformCastSchema::try_create( + // transform_input_port, + // transform_output_port, + // select_schema.clone(), + // target_schema.clone(), + // func_ctx.clone(), + // ) + // }, + // )?; + // } + } + build_fill_missing_columns_pipeline( self.ctx.clone(), &mut self.main_pipeline, tbl.clone(), - schema.clone(), + target_schema.clone(), )?; - if !matches!(input.deref(), PhysicalPlan::AsyncSourcer(_)) - && !matches!(input.deref(), PhysicalPlan::CopyIntoTable(_)) - { - let select_schema = input.output_schema()?; - let target_schema = schema.clone(); - if Self::check_schema_cast(select_schema.clone(), target_schema.clone())? { - let func_ctx = self.ctx.get_function_context()?; - self.main_pipeline.add_transform( - |transform_input_port, transform_output_port| { - TransformCastSchema::try_create( - transform_input_port, - transform_output_port, - select_schema.clone(), - target_schema.clone(), - func_ctx.clone(), - ) - }, - )?; - } - } let _ = table.cluster_gen_for_append( self.ctx.clone(), &mut self.main_pipeline, diff --git a/src/query/sql/src/executor/physical_plan.rs b/src/query/sql/src/executor/physical_plan.rs index 0dc4425d7f1ab..d697c41c74420 100644 --- a/src/query/sql/src/executor/physical_plan.rs +++ b/src/query/sql/src/executor/physical_plan.rs @@ -826,7 +826,8 @@ pub struct Deduplicate { pub empty_table: bool, pub table_info: TableInfo, pub catalog_name: String, - pub schema: DataSchemaRef, + pub target_schema: DataSchemaRef, + pub select_column_bindings: Option>, } #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] From 2bc78b9dd323b1c95821b7930ad215c21f9a88ed Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Wed, 26 Jul 2023 06:40:03 +0800 Subject: [PATCH 12/40] fix schema cast --- .../src/interpreters/interpreter_replace.rs | 15 ++++--- .../service/src/pipelines/pipeline_builder.rs | 40 ++++++++++--------- src/query/sql/src/executor/physical_plan.rs | 8 +++- 3 files changed, 39 insertions(+), 24 deletions(-) diff --git a/src/query/service/src/interpreters/interpreter_replace.rs b/src/query/service/src/interpreters/interpreter_replace.rs index 9996c99035eeb..53c834dfa8415 100644 --- a/src/query/service/src/interpreters/interpreter_replace.rs +++ b/src/query/service/src/interpreters/interpreter_replace.rs @@ -26,6 +26,7 @@ use common_sql::executor::MutationKind; use common_sql::executor::OnConflictField; use common_sql::executor::PhysicalPlan; use common_sql::executor::ReplaceInto; +use common_sql::executor::SelectCtx; use common_sql::plans::CopyPlan; use common_sql::plans::InsertInputSource; use common_sql::plans::Plan; @@ -123,7 +124,7 @@ impl ReplaceInterpreter { let max_threads = self.ctx.get_settings().get_max_threads()?; let segment_partition_num = std::cmp::min(base_snapshot.segments.len(), max_threads as usize); - let (mut root, select_column_bindings) = self + let (mut root, select_ctx) = self .connect_input_source(self.ctx.clone(), &self.plan.source, self.plan.schema()) .await?; root = Box::new(PhysicalPlan::Deduplicate(Deduplicate { @@ -132,8 +133,8 @@ impl ReplaceInterpreter { empty_table, table_info: table_info.clone(), catalog_name: plan.catalog.clone(), + select_ctx, target_schema: plan.schema(), - select_column_bindings, })); root = Box::new(PhysicalPlan::ReplaceInto(ReplaceInto { input: root, @@ -170,7 +171,7 @@ impl ReplaceInterpreter { ctx: Arc, source: &'a InsertInputSource, schema: DataSchemaRef, - ) -> Result<(Box, Option>)> { + ) -> Result<(Box, Option)> { match source { InsertInputSource::Values(data) => self .connect_value_source(schema.clone(), data) @@ -215,7 +216,7 @@ impl ReplaceInterpreter { &'a self, ctx: Arc, query_plan: &Plan, - ) -> Result<(Box, Option>)> { + ) -> Result<(Box, Option)> { let (s_expr, metadata, bind_context, formatted_ast) = match query_plan { Plan::Query { s_expr, @@ -240,6 +241,10 @@ impl ReplaceInterpreter { .build_physical_plan() .await .map(|x| Box::new(x))?; - Ok((physical_plan, Some(bind_context.columns.clone()))) + let select_ctx = SelectCtx { + select_column_bindings: bind_context.columns.clone(), + select_schema: query_plan.schema(), + }; + Ok((physical_plan, Some(select_ctx))) } } diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs index 5b250a0adcbcc..d7a84f203c974 100644 --- a/src/query/service/src/pipelines/pipeline_builder.rs +++ b/src/query/service/src/pipelines/pipeline_builder.rs @@ -94,6 +94,7 @@ use common_sql::executor::RangeJoin; use common_sql::executor::ReplaceInto; use common_sql::executor::RowFetch; use common_sql::executor::RuntimeFilterSource; +use common_sql::executor::SelectCtx; use common_sql::executor::Sort; use common_sql::executor::TableScan; use common_sql::executor::UnionAll; @@ -269,36 +270,39 @@ impl PipelineBuilder { table_info, catalog_name, target_schema, - select_column_bindings, + select_ctx, } = deduplicate; let tbl = self .ctx .build_table_by_table_info(catalog_name, table_info, None)?; let table = FuseTable::try_from_table(tbl.as_ref())?; self.build_pipeline(input)?; - if let Some(select_column_bindings) = select_column_bindings { - let select_schema = input.output_schema()?; + if let Some(SelectCtx { + select_column_bindings, + select_schema, + }) = select_ctx + { PipelineBuilder::render_result_set( &self.ctx.get_function_context()?, - select_schema.clone(), + input.output_schema()?, select_column_bindings, &mut self.main_pipeline, false, )?; - // if Self::check_schema_cast(select_schema.clone(), target_schema.clone())? { - // let func_ctx = self.ctx.get_function_context()?; - // self.main_pipeline.add_transform( - // |transform_input_port, transform_output_port| { - // TransformCastSchema::try_create( - // transform_input_port, - // transform_output_port, - // select_schema.clone(), - // target_schema.clone(), - // func_ctx.clone(), - // ) - // }, - // )?; - // } + if Self::check_schema_cast(select_schema.clone(), target_schema.clone())? { + let func_ctx = self.ctx.get_function_context()?; + self.main_pipeline.add_transform( + |transform_input_port, transform_output_port| { + TransformCastSchema::try_create( + transform_input_port, + transform_output_port, + select_schema.clone(), + target_schema.clone(), + func_ctx.clone(), + ) + }, + )?; + } } build_fill_missing_columns_pipeline( diff --git a/src/query/sql/src/executor/physical_plan.rs b/src/query/sql/src/executor/physical_plan.rs index d697c41c74420..6344b6bf070f8 100644 --- a/src/query/sql/src/executor/physical_plan.rs +++ b/src/query/sql/src/executor/physical_plan.rs @@ -827,7 +827,13 @@ pub struct Deduplicate { pub table_info: TableInfo, pub catalog_name: String, pub target_schema: DataSchemaRef, - pub select_column_bindings: Option>, + pub select_ctx: Option, +} + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct SelectCtx { + pub select_column_bindings: Vec, + pub select_schema: DataSchemaRef, } #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] From 48fb10e7a2180ddddd27a4499bec2baf93926bf7 Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Wed, 26 Jul 2023 07:44:06 +0800 Subject: [PATCH 13/40] temp --- src/query/service/src/interpreters/interpreter_replace.rs | 2 -- .../storages/fuse/src/operations/common/mutation_accumulator.rs | 1 - .../common/processors/transform_mutation_aggregator.rs | 2 -- src/query/storages/fuse/src/operations/replace.rs | 1 - .../src/operations/replace_into/mutator/mutator_replace_into.rs | 1 - 5 files changed, 7 deletions(-) diff --git a/src/query/service/src/interpreters/interpreter_replace.rs b/src/query/service/src/interpreters/interpreter_replace.rs index 870977a0c147c..9281e5af41bb1 100644 --- a/src/query/service/src/interpreters/interpreter_replace.rs +++ b/src/query/service/src/interpreters/interpreter_replace.rs @@ -14,11 +14,9 @@ use std::sync::Arc; -use common_base::runtime::GlobalIORuntime; use common_catalog::table_context::TableContext; use common_exception::ErrorCode; use common_exception::Result; -use common_expression::DataSchema; use common_expression::DataSchemaRef; use common_sql::executor::AsyncSourcerPlan; use common_sql::executor::Deduplicate; diff --git a/src/query/storages/fuse/src/operations/common/mutation_accumulator.rs b/src/query/storages/fuse/src/operations/common/mutation_accumulator.rs index e2b9545213f1c..fc3071e3bc90c 100644 --- a/src/query/storages/fuse/src/operations/common/mutation_accumulator.rs +++ b/src/query/storages/fuse/src/operations/common/mutation_accumulator.rs @@ -31,7 +31,6 @@ use storages_common_table_meta::meta::Location; use storages_common_table_meta::meta::SegmentInfo; use storages_common_table_meta::meta::Statistics; use storages_common_table_meta::meta::Versioned; -use tracing::error; use tracing::info; use super::ConflictResolveContext; diff --git a/src/query/storages/fuse/src/operations/common/processors/transform_mutation_aggregator.rs b/src/query/storages/fuse/src/operations/common/processors/transform_mutation_aggregator.rs index 9a5258b92ea66..6e16b8a1b225a 100644 --- a/src/query/storages/fuse/src/operations/common/processors/transform_mutation_aggregator.rs +++ b/src/query/storages/fuse/src/operations/common/processors/transform_mutation_aggregator.rs @@ -25,8 +25,6 @@ use storages_common_table_meta::meta::Location; use storages_common_table_meta::meta::Statistics; use tracing::debug; -use crate::io::TableMetaLocationGenerator; -use crate::operations::common::mutation_accumulator::MutationKind; use crate::operations::common::CommitMeta; use crate::operations::common::MutationAccumulator; use crate::operations::common::MutationLogs; diff --git a/src/query/storages/fuse/src/operations/replace.rs b/src/query/storages/fuse/src/operations/replace.rs index 404674287e8fb..72890201f9552 100644 --- a/src/query/storages/fuse/src/operations/replace.rs +++ b/src/query/storages/fuse/src/operations/replace.rs @@ -15,7 +15,6 @@ use std::sync::Arc; use common_base::base::tokio::sync::Semaphore; -use common_catalog::table::Table; use common_catalog::table_context::TableContext; use common_exception::Result; use common_pipeline_core::pipe::PipeItem; diff --git a/src/query/storages/fuse/src/operations/replace_into/mutator/mutator_replace_into.rs b/src/query/storages/fuse/src/operations/replace_into/mutator/mutator_replace_into.rs index 7badc66d1b4e3..3e22b18465053 100644 --- a/src/query/storages/fuse/src/operations/replace_into/mutator/mutator_replace_into.rs +++ b/src/query/storages/fuse/src/operations/replace_into/mutator/mutator_replace_into.rs @@ -48,7 +48,6 @@ use crate::operations::replace_into::meta::merge_into_operation_meta::MergeIntoO use crate::operations::replace_into::meta::merge_into_operation_meta::UniqueKeyDigest; use crate::operations::replace_into::mutator::column_hash::row_hash_of_columns; use crate::operations::replace_into::mutator::column_hash::RowScalarValue; -use crate::operations::replace_into::OnConflictField; // Replace is somehow a simplified merge_into, which // - do insertion for "matched" branch From 8406189fc29b556142ae238aec7c509af9e95a13 Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Wed, 26 Jul 2023 20:08:45 +0800 Subject: [PATCH 14/40] respect #12147 --- src/query/catalog/src/table.rs | 2 ++ .../src/interpreters/interpreter_replace.rs | 9 ++++++--- .../service/src/pipelines/pipeline_builder.rs | 18 +++++++++++++----- .../servers/mysql/mysql_interactive_worker.rs | 3 ++- src/query/sql/src/executor/physical_plan.rs | 8 ++++++-- 5 files changed, 29 insertions(+), 11 deletions(-) diff --git a/src/query/catalog/src/table.rs b/src/query/catalog/src/table.rs index e285f0650ebc9..d0f9135cfb62d 100644 --- a/src/query/catalog/src/table.rs +++ b/src/query/catalog/src/table.rs @@ -37,6 +37,8 @@ use common_meta_app::schema::UpsertTableCopiedFileReq; use common_meta_types::MetaId; use common_pipeline_core::Pipeline; use common_storage::StorageMetrics; +use serde::Deserialize; +use serde::Serialize; use crate::plan::DataSourceInfo; use crate::plan::DataSourcePlan; diff --git a/src/query/service/src/interpreters/interpreter_replace.rs b/src/query/service/src/interpreters/interpreter_replace.rs index 9281e5af41bb1..71da3acc61b57 100644 --- a/src/query/service/src/interpreters/interpreter_replace.rs +++ b/src/query/service/src/interpreters/interpreter_replace.rs @@ -14,6 +14,7 @@ use std::sync::Arc; +use common_catalog::table::ColumnStatistics; use common_catalog::table_context::TableContext; use common_exception::ErrorCode; use common_exception::Result; @@ -122,7 +123,8 @@ impl ReplaceInterpreter { Arc::new(TableSnapshot::new_empty_snapshot(schema.as_ref().clone())) }); - let empty_table = base_snapshot.segments.is_empty(); + let table_is_empty = base_snapshot.segments.is_empty(); + let table_level_range_index = base_snapshot.summary.col_stats.clone(); let max_threads = self.ctx.get_settings().get_max_threads()?; let segment_partition_num = std::cmp::min(base_snapshot.segments.len(), max_threads as usize); @@ -132,11 +134,12 @@ impl ReplaceInterpreter { root = Box::new(PhysicalPlan::Deduplicate(Deduplicate { input: root, on_conflicts: on_conflicts.clone(), - empty_table, + table_is_empty, table_info: table_info.clone(), catalog_name: plan.catalog.clone(), select_ctx, - target_schema: plan.schema(), + table_schema: plan.schema.clone(), + table_level_range_index, })); root = Box::new(PhysicalPlan::ReplaceInto(ReplaceInto { input: root, diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs index d0f44beec6719..10467dcc53bb0 100644 --- a/src/query/service/src/pipelines/pipeline_builder.rs +++ b/src/query/service/src/pipelines/pipeline_builder.rs @@ -267,16 +267,18 @@ impl PipelineBuilder { let Deduplicate { input, on_conflicts, - empty_table, + table_is_empty, table_info, catalog_name, - target_schema, select_ctx, + table_level_range_index, + table_schema, } = deduplicate; let tbl = self .ctx .build_table_by_table_info(catalog_name, table_info, None)?; let table = FuseTable::try_from_table(tbl.as_ref())?; + let target_schema: Arc = Arc::new(table_schema.clone().into()); self.build_pipeline(input)?; if let Some(SelectCtx { select_column_bindings, @@ -335,9 +337,15 @@ impl PipelineBuilder { // (1) -> output_port_merge_into_action // the "downstream" is supposed to be connected with a processor which can process MergeIntoOperations // in our case, it is the broadcast processor - - let replace_into_processor = - ReplaceIntoProcessor::create(on_conflicts.clone(), *empty_table); + let cluster_keys = table.cluster_keys(self.ctx.clone()); + let replace_into_processor = ReplaceIntoProcessor::create( + self.ctx.as_ref(), + on_conflicts.clone(), + cluster_keys, + table_schema.as_ref(), + *table_is_empty, + table_level_range_index.clone(), + )?; self.main_pipeline .add_pipe(replace_into_processor.into_pipe()); Ok(()) diff --git a/src/query/service/src/servers/mysql/mysql_interactive_worker.rs b/src/query/service/src/servers/mysql/mysql_interactive_worker.rs index 7562a356083c5..1264b053b9a8c 100644 --- a/src/query/service/src/servers/mysql/mysql_interactive_worker.rs +++ b/src/query/service/src/servers/mysql/mysql_interactive_worker.rs @@ -395,10 +395,11 @@ impl InteractiveWorkerBase { // Wrap the data stream, log finish event at the end of stream let intercepted_stream = async_stream::stream! { - + println!("1"); while let Some(item) = data_stream.next().await { yield item }; + println!("2"); }; Ok::<_, ErrorCode>(intercepted_stream.boxed()) diff --git a/src/query/sql/src/executor/physical_plan.rs b/src/query/sql/src/executor/physical_plan.rs index 55e25b790565c..0877466cbebd7 100644 --- a/src/query/sql/src/executor/physical_plan.rs +++ b/src/query/sql/src/executor/physical_plan.rs @@ -13,6 +13,7 @@ // limitations under the License. use std::collections::BTreeMap; +use std::collections::HashMap; use std::collections::HashSet; use std::fmt::Display; use std::fmt::Formatter; @@ -26,6 +27,7 @@ use common_exception::Result; use common_expression::types::DataType; use common_expression::types::NumberDataType; use common_expression::BlockThresholds; +use common_expression::ColumnId; use common_expression::DataBlock; use common_expression::DataField; use common_expression::DataSchemaRef; @@ -39,6 +41,7 @@ use common_functions::BUILTIN_FUNCTIONS; use common_meta_app::schema::TableInfo; use common_storage::StageFileInfo; use enum_as_inner::EnumAsInner; +use storages_common_table_meta::meta::ColumnStatistics; use storages_common_table_meta::meta::TableSnapshot; use crate::executor::explain::PlanStatsInfo; @@ -832,11 +835,12 @@ pub struct AsyncSourcerPlan { pub struct Deduplicate { pub input: Box, pub on_conflicts: Vec, - pub empty_table: bool, + pub table_is_empty: bool, pub table_info: TableInfo, pub catalog_name: String, - pub target_schema: DataSchemaRef, + pub table_schema: TableSchemaRef, pub select_ctx: Option, + pub table_level_range_index: HashMap, } #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] From c8120795651151adaf2aeae118683827139e4145 Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Wed, 26 Jul 2023 20:22:40 +0800 Subject: [PATCH 15/40] respect #12100 --- .../src/interpreters/interpreter_replace.rs | 79 +++++++++++++++++-- 1 file changed, 71 insertions(+), 8 deletions(-) diff --git a/src/query/service/src/interpreters/interpreter_replace.rs b/src/query/service/src/interpreters/interpreter_replace.rs index 71da3acc61b57..9625916d8e6c3 100644 --- a/src/query/service/src/interpreters/interpreter_replace.rs +++ b/src/query/service/src/interpreters/interpreter_replace.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use common_catalog::table::ColumnStatistics; +use common_base::runtime::GlobalIORuntime; use common_catalog::table_context::TableContext; use common_exception::ErrorCode; use common_exception::Result; @@ -33,11 +33,10 @@ use common_sql::plans::OptimizeTableAction; use common_sql::plans::OptimizeTablePlan; use common_sql::plans::Plan; use common_sql::plans::Replace; -use common_sql::ColumnBinding; use common_storages_factory::Table; use common_storages_fuse::FuseTable; use storages_common_table_meta::meta::TableSnapshot; -use tracing::error; +use tracing::info; use crate::interpreters::common::check_deduplicate_label; use crate::interpreters::interpreter_copy::CopyInterpreter; @@ -45,6 +44,8 @@ use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; use crate::interpreters::OptimizeTableInterpreter; use crate::interpreters::SelectInterpreter; +use crate::pipelines::executor::ExecutorSettings; +use crate::pipelines::executor::PipelineCompleteExecutor; use crate::pipelines::PipelineBuildResult; use crate::schedulers::build_query_pipeline_without_render_result_set; use crate::sessions::QueryContext; @@ -74,14 +75,76 @@ impl Interpreter for ReplaceInterpreter { } self.check_on_conflicts()?; - let physical_plan = self.build_physical_plan().await?; - error!("physical_plan: {:?}", physical_plan); - let build_res = + let mut pipeline = build_query_pipeline_without_render_result_set(&self.ctx, &physical_plan, false) .await?; - error!("build_res: {:?}", build_res.main_pipeline); - Ok(build_res) + + let plan = &self.plan; + let table = self + .ctx + .get_table(&plan.catalog, &plan.database, &plan.table) + .await?; + let has_cluster_key = !table.cluster_keys(self.ctx.clone()).is_empty(); + if !pipeline.main_pipeline.is_empty() + && has_cluster_key + && self.ctx.get_settings().get_enable_auto_reclustering()? + { + let ctx = self.ctx.clone(); + let catalog = self.plan.catalog.clone(); + let database = self.plan.database.to_string(); + let table = self.plan.table.to_string(); + pipeline.main_pipeline.set_on_finished(|err| { + if err.is_none() { + info!("execute replace into finished successfully. running table optimization job."); + match GlobalIORuntime::instance().block_on({ + async move { + ctx.evict_table_from_cache(&catalog, &database, &table)?; + let optimize_interpreter = OptimizeTableInterpreter::try_create(ctx.clone(), + OptimizeTablePlan { + catalog, + database, + table, + action: OptimizeTableAction::CompactBlocks, + limit: None, + } + )?; + + let mut build_res = optimize_interpreter.execute2().await?; + + if build_res.main_pipeline.is_empty() { + return Ok(()); + } + + let settings = ctx.get_settings(); + let query_id = ctx.get_id(); + build_res.set_max_threads(settings.get_max_threads()? as usize); + let settings = ExecutorSettings::try_create(&settings, query_id)?; + + if build_res.main_pipeline.is_complete_pipeline()? { + let mut pipelines = build_res.sources_pipelines; + pipelines.push(build_res.main_pipeline); + + let complete_executor = PipelineCompleteExecutor::from_pipelines(pipelines, settings)?; + + ctx.set_executor(complete_executor.get_inner())?; + complete_executor.execute()?; + } + Ok(()) + } + }) { + Ok(_) => { + info!("execute replace into finished successfully. table optimization job finished."); + } + Err(e) => { info!("execute replace into finished successfully. table optimization job failed. {:?}", e)} + } + + return Ok(()); + } + Ok(()) + }); + } + Ok(pipeline) } } From 8c7440260b773bbdf3904c040fa6645ba2596c59 Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Wed, 26 Jul 2023 20:29:46 +0800 Subject: [PATCH 16/40] make lint --- src/query/catalog/src/table.rs | 2 -- .../transforms/transform_accumulating_async.rs | 12 ++++++++++-- .../service/src/interpreters/interpreter_delete.rs | 2 +- .../service/src/interpreters/interpreter_replace.rs | 2 +- src/query/service/src/pipelines/pipeline_builder.rs | 5 ----- .../storages/fuse/src/operations/replace_into/mod.rs | 2 -- .../replace_into/mutator/mutator_replace_into.rs | 2 +- .../processors/processor_replace_into.rs | 2 +- 8 files changed, 14 insertions(+), 15 deletions(-) diff --git a/src/query/catalog/src/table.rs b/src/query/catalog/src/table.rs index d0f9135cfb62d..e285f0650ebc9 100644 --- a/src/query/catalog/src/table.rs +++ b/src/query/catalog/src/table.rs @@ -37,8 +37,6 @@ use common_meta_app::schema::UpsertTableCopiedFileReq; use common_meta_types::MetaId; use common_pipeline_core::Pipeline; use common_storage::StorageMetrics; -use serde::Deserialize; -use serde::Serialize; use crate::plan::DataSourceInfo; use crate::plan::DataSourcePlan; diff --git a/src/query/pipeline/transforms/src/processors/transforms/transform_accumulating_async.rs b/src/query/pipeline/transforms/src/processors/transforms/transform_accumulating_async.rs index d8a7ef5959ede..1f5ddc3703a83 100644 --- a/src/query/pipeline/transforms/src/processors/transforms/transform_accumulating_async.rs +++ b/src/query/pipeline/transforms/src/processors/transforms/transform_accumulating_async.rs @@ -81,7 +81,11 @@ impl Processor for AsyncAccumulatingTra } if let Some(data_block) = self.output_data.take() { - eprintln!("{} - send data block: {:?}", self.name(), data_block.get_meta()); + eprintln!( + "{} - send data block: {:?}", + self.name(), + data_block.get_meta() + ); self.output.push_data(Ok(data_block)); return Ok(Event::NeedConsume); } @@ -92,7 +96,11 @@ impl Processor for AsyncAccumulatingTra if self.input.has_data() { self.input_data = Some(self.input.pull_data().unwrap()?); - eprintln!("{} - recv data block: {:?}", self.name(), self.input_data.as_ref().unwrap().get_meta()); + eprintln!( + "{} - recv data block: {:?}", + self.name(), + self.input_data.as_ref().unwrap().get_meta() + ); return Ok(Event::Async); } diff --git a/src/query/service/src/interpreters/interpreter_delete.rs b/src/query/service/src/interpreters/interpreter_delete.rs index 0b9408e1df7b3..b4a729a796765 100644 --- a/src/query/service/src/interpreters/interpreter_delete.rs +++ b/src/query/service/src/interpreters/interpreter_delete.rs @@ -28,12 +28,12 @@ use common_expression::RemoteExpr; use common_expression::ROW_ID_COL_NAME; use common_functions::BUILTIN_FUNCTIONS; use common_meta_app::schema::TableInfo; -use common_sql::executor::MutationKind; use common_sql::executor::cast_expr_to_non_null_boolean; use common_sql::executor::DeletePartial; use common_sql::executor::Exchange; use common_sql::executor::FragmentKind; use common_sql::executor::MutationAggregate; +use common_sql::executor::MutationKind; use common_sql::executor::PhysicalPlan; use common_sql::optimizer::CascadesOptimizer; use common_sql::optimizer::HeuristicOptimizer; diff --git a/src/query/service/src/interpreters/interpreter_replace.rs b/src/query/service/src/interpreters/interpreter_replace.rs index 9625916d8e6c3..91d181ca8401d 100644 --- a/src/query/service/src/interpreters/interpreter_replace.rs +++ b/src/query/service/src/interpreters/interpreter_replace.rs @@ -308,7 +308,7 @@ impl ReplaceInterpreter { let physical_plan = select_interpreter .build_physical_plan() .await - .map(|x| Box::new(x))?; + .map(Box::new)?; let select_ctx = SelectCtx { select_column_bindings: bind_context.columns.clone(), select_schema: query_plan.schema(), diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs index 10467dcc53bb0..4d07823ca7d40 100644 --- a/src/query/service/src/pipelines/pipeline_builder.rs +++ b/src/query/service/src/pipelines/pipeline_builder.rs @@ -16,7 +16,6 @@ use std::collections::VecDeque; use std::convert::TryFrom; use std::io::BufRead; use std::io::Cursor; -use std::ops::Deref; use std::ops::Not; use std::sync::Arc; use std::sync::Mutex; @@ -44,7 +43,6 @@ use common_expression::FunctionContext; use common_expression::HashMethodKind; use common_expression::RemoteExpr; use common_expression::SortColumnDescription; -use common_expression::TableSchema; use common_formats::FastFieldDecoderValues; use common_functions::aggregates::AggregateFunctionFactory; use common_functions::aggregates::AggregateFunctionRef; @@ -88,7 +86,6 @@ use common_sql::executor::Filter; use common_sql::executor::HashJoin; use common_sql::executor::Limit; use common_sql::executor::MutationAggregate; -use common_sql::executor::MutationKind; use common_sql::executor::PhysicalPlan; use common_sql::executor::Project; use common_sql::executor::ProjectSet; @@ -101,7 +98,6 @@ use common_sql::executor::Sort; use common_sql::executor::TableScan; use common_sql::executor::UnionAll; use common_sql::executor::Window; -use common_sql::plans::InsertInputSource; use common_sql::BindContext; use common_sql::ColumnBinding; use common_sql::IndexType; @@ -153,7 +149,6 @@ use crate::pipelines::processors::TransformLimit; use crate::pipelines::processors::TransformRuntimeFilter; use crate::pipelines::Pipeline; use crate::pipelines::PipelineBuildResult; -use crate::schedulers::build_query_pipeline; use crate::sessions::QueryContext; use crate::sessions::TableContext; diff --git a/src/query/storages/fuse/src/operations/replace_into/mod.rs b/src/query/storages/fuse/src/operations/replace_into/mod.rs index 0411fd1acfd16..dbec97c8aaa02 100644 --- a/src/query/storages/fuse/src/operations/replace_into/mod.rs +++ b/src/query/storages/fuse/src/operations/replace_into/mod.rs @@ -19,5 +19,3 @@ mod processors; pub use processors::BroadcastProcessor; pub use processors::MergeIntoOperationAggregator; pub use processors::ReplaceIntoProcessor; - - diff --git a/src/query/storages/fuse/src/operations/replace_into/mutator/mutator_replace_into.rs b/src/query/storages/fuse/src/operations/replace_into/mutator/mutator_replace_into.rs index 3e22b18465053..e9c439342fc48 100644 --- a/src/query/storages/fuse/src/operations/replace_into/mutator/mutator_replace_into.rs +++ b/src/query/storages/fuse/src/operations/replace_into/mutator/mutator_replace_into.rs @@ -35,8 +35,8 @@ use common_expression::ScalarRef; use common_expression::TableSchema; use common_expression::Value; use common_functions::aggregates::eval_aggr; -use common_sql::executor::OnConflictField; use common_functions::BUILTIN_FUNCTIONS; +use common_sql::executor::OnConflictField; use storages_common_table_meta::meta::ColumnStatistics; use tracing::info; diff --git a/src/query/storages/fuse/src/operations/replace_into/processors/processor_replace_into.rs b/src/query/storages/fuse/src/operations/replace_into/processors/processor_replace_into.rs index 9775c073ceace..608f66196f4e9 100644 --- a/src/query/storages/fuse/src/operations/replace_into/processors/processor_replace_into.rs +++ b/src/query/storages/fuse/src/operations/replace_into/processors/processor_replace_into.rs @@ -30,8 +30,8 @@ use common_pipeline_core::processors::processor::Event; use common_pipeline_core::processors::processor::ProcessorPtr; use common_pipeline_core::processors::Processor; use common_sql::executor::OnConflictField; -use tracing::error; use storages_common_table_meta::meta::ColumnStatistics; +use tracing::error; use crate::operations::replace_into::mutator::mutator_replace_into::ReplaceIntoMutator; From 477fdfdd54fa058568e8af3c7203deafe59dd450 Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Wed, 26 Jul 2023 21:32:35 +0800 Subject: [PATCH 17/40] respect #12130 --- .../service/src/interpreters/interpreter_copy.rs | 11 +++++++++-- src/query/service/src/pipelines/pipeline_builder.rs | 9 ++++++++- src/query/sql/src/executor/physical_plan.rs | 10 +++++++++- src/query/sql/src/executor/physical_plan_visitor.rs | 12 ++++++++---- 4 files changed, 34 insertions(+), 8 deletions(-) diff --git a/src/query/service/src/interpreters/interpreter_copy.rs b/src/query/service/src/interpreters/interpreter_copy.rs index 45689d1c44633..3179a10a53cef 100644 --- a/src/query/service/src/interpreters/interpreter_copy.rs +++ b/src/query/service/src/interpreters/interpreter_copy.rs @@ -150,8 +150,15 @@ impl CopyInterpreter { .await?; let files = plan.collect_files(self.ctx.as_ref()).await?; let source = if let Some(ref query) = plan.query { - let (select_interpreter, _) = self.build_query(query).await?; - CopyIntoTableSource::Query(Box::new(select_interpreter.build_physical_plan().await?)) + let (select_interpreter, query_source_schema) = self.build_query(query).await?; + let plan_query = select_interpreter.build_physical_plan().await?; + let result_columns = select_interpreter.get_result_columns(); + CopyIntoTableSource::Query(Box::new(common_sql::executor::QueryCtx { + plan: plan_query, + ignore_result: select_interpreter.get_ignore_result(), + result_columns, + query_source_schema, + })) } else { let stage_table_info = StageTableInfo { files_to_copy: Some(files.clone()), diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs index 4d07823ca7d40..6c3ac8ff131e1 100644 --- a/src/query/service/src/pipelines/pipeline_builder.rs +++ b/src/query/service/src/pipelines/pipeline_builder.rs @@ -489,7 +489,14 @@ impl PipelineBuilder { let to_table = catalog.get_table_by_info(©.table_info)?; match ©.source { CopyIntoTableSource::Query(input) => { - self.build_pipeline(input)?; + self.build_pipeline(&input.plan)?; + Self::render_result_set( + &self.ctx.get_function_context()?, + input.plan.output_schema()?, + &input.result_columns, + &mut self.main_pipeline, + input.ignore_result, + )?; } CopyIntoTableSource::Stage(source) => { let stage_table = StageTable::try_create(copy.stage_table_info.clone())?; diff --git a/src/query/sql/src/executor/physical_plan.rs b/src/query/sql/src/executor/physical_plan.rs index 0877466cbebd7..ebfcadad2afc8 100644 --- a/src/query/sql/src/executor/physical_plan.rs +++ b/src/query/sql/src/executor/physical_plan.rs @@ -732,10 +732,18 @@ pub struct CopyIntoTable { #[derive(Clone, Debug, serde::Serialize, serde::Deserialize, EnumAsInner)] pub enum CopyIntoTableSource { - Query(Box), + Query(Box), Stage(Box), } +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct QueryCtx { + pub plan: PhysicalPlan, + pub query_source_schema: DataSchemaRef, + pub ignore_result: bool, + pub result_columns: Vec, +} + impl CopyIntoTable { pub fn output_schema(&self) -> Result { Ok(DataSchemaRef::default()) diff --git a/src/query/sql/src/executor/physical_plan_visitor.rs b/src/query/sql/src/executor/physical_plan_visitor.rs index 6090cbb582b9a..c5ceeb027e0f6 100644 --- a/src/query/sql/src/executor/physical_plan_visitor.rs +++ b/src/query/sql/src/executor/physical_plan_visitor.rs @@ -34,6 +34,7 @@ use super::MutationAggregate; use super::PhysicalPlan; use super::Project; use super::ProjectSet; +use super::QueryCtx; use super::ReplaceInto; use super::RowFetch; use super::Sort; @@ -291,10 +292,13 @@ pub trait PhysicalPlanReplacer { CopyIntoTableSource::Stage(_) => { Ok(PhysicalPlan::CopyIntoTable(Box::new(plan.clone()))) } - CopyIntoTableSource::Query(input) => { - let input = self.replace(input)?; + CopyIntoTableSource::Query(query_ctx) => { + let input = self.replace(&query_ctx.plan)?; Ok(PhysicalPlan::CopyIntoTable(Box::new(CopyIntoTable { - source: CopyIntoTableSource::Query(Box::new(input)), + source: CopyIntoTableSource::Query(Box::new(QueryCtx { + plan: input, + ..*query_ctx.clone() + })), ..plan.clone() }))) } @@ -444,7 +448,7 @@ impl PhysicalPlan { } PhysicalPlan::CopyIntoTable(plan) => match &plan.source { CopyIntoTableSource::Query(input) => { - Self::traverse(input, pre_visit, visit, post_visit); + Self::traverse(&input.plan, pre_visit, visit, post_visit); } CopyIntoTableSource::Stage(_) => {} }, From 41f6361428b434150633cdd135d7bfbe22a992f3 Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Fri, 28 Jul 2023 10:29:18 +0800 Subject: [PATCH 18/40] fix merge --- src/query/service/src/pipelines/pipeline_builder.rs | 12 ++---------- src/query/sql/src/executor/physical_plan.rs | 1 - src/query/sql/src/executor/physical_plan_visitor.rs | 6 +++--- src/query/sql/src/executor/profile.rs | 3 --- 4 files changed, 5 insertions(+), 17 deletions(-) diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs index 1c7e71febea67..7e5241c221d74 100644 --- a/src/query/service/src/pipelines/pipeline_builder.rs +++ b/src/query/service/src/pipelines/pipeline_builder.rs @@ -76,10 +76,8 @@ use common_sql::executor::AggregatePartial; use common_sql::executor::AsyncSourcerPlan; use common_sql::executor::CopyIntoTable; use common_sql::executor::CopyIntoTableSource; -use common_sql::executor::Deduplicate; -use common_sql::executor::CopyIntoTableFromQuery; use common_sql::executor::CteScan; -use common_sql::executor::DeleteFinal; +use common_sql::executor::Deduplicate; use common_sql::executor::DeletePartial; use common_sql::executor::DistributedInsertSelect; use common_sql::executor::EvalScalar; @@ -88,8 +86,8 @@ use common_sql::executor::ExchangeSource; use common_sql::executor::Filter; use common_sql::executor::HashJoin; use common_sql::executor::Limit; -use common_sql::executor::MutationAggregate; use common_sql::executor::MaterializedCte; +use common_sql::executor::MutationAggregate; use common_sql::executor::PhysicalPlan; use common_sql::executor::Project; use common_sql::executor::ProjectSet; @@ -247,12 +245,6 @@ impl PipelineBuilder { PhysicalPlan::DeletePartial(delete) => self.build_delete_partial(delete), PhysicalPlan::MutationAggregate(plan) => self.build_mutation_aggregate(plan), PhysicalPlan::RangeJoin(range_join) => self.build_range_join(range_join), - PhysicalPlan::DistributedCopyIntoTableFromStage(distributed_plan) => { - self.build_distributed_copy_into_table_from_stage(distributed_plan) - } - PhysicalPlan::CopyIntoTableFromQuery(copy_plan) => { - self.build_copy_into_table_from_query(copy_plan) - } PhysicalPlan::MaterializedCte(materialized_cte) => { self.build_materialized_cte(materialized_cte) } diff --git a/src/query/sql/src/executor/physical_plan.rs b/src/query/sql/src/executor/physical_plan.rs index e9ee81f73cf18..c1b599f810499 100644 --- a/src/query/sql/src/executor/physical_plan.rs +++ b/src/query/sql/src/executor/physical_plan.rs @@ -1157,7 +1157,6 @@ impl PhysicalPlan { | PhysicalPlan::AsyncSourcer(_) | PhysicalPlan::Deduplicate(_) | PhysicalPlan::ReplaceInto(_) - | PhysicalPlan::DeleteFinal(_) | PhysicalPlan::CteScan(_) => None, } } diff --git a/src/query/sql/src/executor/physical_plan_visitor.rs b/src/query/sql/src/executor/physical_plan_visitor.rs index 323d8b2121e61..c648bc91c2cf9 100644 --- a/src/query/sql/src/executor/physical_plan_visitor.rs +++ b/src/query/sql/src/executor/physical_plan_visitor.rs @@ -414,9 +414,9 @@ impl PhysicalPlan { if pre_visit(plan) { visit(plan); match plan { - PhysicalPlan::TableScan(_) => {} - PhysicalPlan::AsyncSourcer(_) => {} - PhysicalPlan::TableScan(_) | PhysicalPlan::CteScan(_) => {} + PhysicalPlan::TableScan(_) + | PhysicalPlan::AsyncSourcer(_) + | PhysicalPlan::CteScan(_) => {} PhysicalPlan::Filter(plan) => { Self::traverse(&plan.input, pre_visit, visit, post_visit); } diff --git a/src/query/sql/src/executor/profile.rs b/src/query/sql/src/executor/profile.rs index 8ad21623df0e7..a216ed56df143 100644 --- a/src/query/sql/src/executor/profile.rs +++ b/src/query/sql/src/executor/profile.rs @@ -463,9 +463,6 @@ fn flatten_plan_node_profile( plan_node_profs.push(prof); } PhysicalPlan::MaterializedCte(_) => todo!(), - PhysicalPlan::DeletePartial(_) | PhysicalPlan::DeleteFinal(_) => unreachable!(), - PhysicalPlan::DistributedCopyIntoTableFromStage(_) => unreachable!(), - PhysicalPlan::CopyIntoTableFromQuery(_) => unreachable!(), PhysicalPlan::DeletePartial(_) | PhysicalPlan::MutationAggregate(_) | PhysicalPlan::CopyIntoTable(_) From 0d3ada35ae78ab362413fa2404faaedf9a9a2e39 Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Mon, 31 Jul 2023 08:56:43 +0800 Subject: [PATCH 19/40] add exchange --- .../src/interpreters/interpreter_replace.rs | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/src/query/service/src/interpreters/interpreter_replace.rs b/src/query/service/src/interpreters/interpreter_replace.rs index 91d181ca8401d..f8d0851938f8c 100644 --- a/src/query/service/src/interpreters/interpreter_replace.rs +++ b/src/query/service/src/interpreters/interpreter_replace.rs @@ -21,6 +21,7 @@ use common_exception::Result; use common_expression::DataSchemaRef; use common_sql::executor::AsyncSourcerPlan; use common_sql::executor::Deduplicate; +use common_sql::executor::Exchange; use common_sql::executor::MutationAggregate; use common_sql::executor::MutationKind; use common_sql::executor::OnConflictField; @@ -186,6 +187,7 @@ impl ReplaceInterpreter { Arc::new(TableSnapshot::new_empty_snapshot(schema.as_ref().clone())) }); + let is_distributed = !self.ctx.get_cluster().is_empty(); let table_is_empty = base_snapshot.segments.is_empty(); let table_level_range_index = base_snapshot.summary.col_stats.clone(); let max_threads = self.ctx.get_settings().get_max_threads()?; @@ -204,6 +206,14 @@ impl ReplaceInterpreter { table_schema: plan.schema.clone(), table_level_range_index, })); + if is_distributed { + root = Box::new(PhysicalPlan::Exchange(Exchange { + plan_id: 0, + input: root, + kind: common_sql::executor::FragmentKind::Expansive, + keys: vec![], + })); + } root = Box::new(PhysicalPlan::ReplaceInto(ReplaceInto { input: root, segment_partition_num, @@ -213,6 +223,14 @@ impl ReplaceInterpreter { on_conflicts, snapshot: (*base_snapshot).clone(), })); + if is_distributed { + root = Box::new(PhysicalPlan::Exchange(Exchange { + plan_id: 0, + input: root, + kind: common_sql::executor::FragmentKind::Merge, + keys: vec![], + })); + } root = Box::new(PhysicalPlan::MutationAggregate(Box::new( MutationAggregate { input: root, From 5b65dbee17ed8bb4144039645933af6603364ff6 Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Mon, 31 Jul 2023 09:31:43 +0800 Subject: [PATCH 20/40] fix conflict --- src/query/pipeline/sources/src/async_source.rs | 5 ----- .../service/src/interpreters/interpreter_replace.rs | 11 +++++++++-- src/query/storages/fuse/src/operations/commit.rs | 2 +- .../src/operations/common/processors/sink_commit.rs | 1 - .../replace_into/processors/processor_replace_into.rs | 3 --- 5 files changed, 10 insertions(+), 12 deletions(-) diff --git a/src/query/pipeline/sources/src/async_source.rs b/src/query/pipeline/sources/src/async_source.rs index e1597a2ff8ea8..a8e0dfc1b2ca8 100644 --- a/src/query/pipeline/sources/src/async_source.rs +++ b/src/query/pipeline/sources/src/async_source.rs @@ -24,7 +24,6 @@ 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 tracing::error; #[async_trait::async_trait] pub trait AsyncSource: Send { @@ -91,10 +90,6 @@ impl Processor for AsyncSourcer { match self.generated_data.take() { None => Ok(Event::Async), Some(data_block) => { - error!( - "Async source: Pushed data block to output port,{:?}", - data_block.columns() - ); self.output.push_data(Ok(data_block)); Ok(Event::NeedConsume) } diff --git a/src/query/service/src/interpreters/interpreter_replace.rs b/src/query/service/src/interpreters/interpreter_replace.rs index f8d0851938f8c..eeed3309464d2 100644 --- a/src/query/service/src/interpreters/interpreter_replace.rs +++ b/src/query/service/src/interpreters/interpreter_replace.rs @@ -13,6 +13,7 @@ // limitations under the License. use std::sync::Arc; +use std::time::Instant; use common_base::runtime::GlobalIORuntime; use common_catalog::table_context::TableContext; @@ -36,10 +37,12 @@ use common_sql::plans::Plan; use common_sql::plans::Replace; use common_storages_factory::Table; use common_storages_fuse::FuseTable; +use log::info; use storages_common_table_meta::meta::TableSnapshot; -use tracing::info; use crate::interpreters::common::check_deduplicate_label; +use crate::interpreters::common::metrics_inc_replace_execution_time_ms; +use crate::interpreters::common::metrics_inc_replace_mutation_time_ms; use crate::interpreters::interpreter_copy::CopyInterpreter; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; @@ -76,6 +79,8 @@ impl Interpreter for ReplaceInterpreter { } self.check_on_conflicts()?; + let start = Instant::now(); + let physical_plan = self.build_physical_plan().await?; let mut pipeline = build_query_pipeline_without_render_result_set(&self.ctx, &physical_plan, false) @@ -95,7 +100,8 @@ impl Interpreter for ReplaceInterpreter { let catalog = self.plan.catalog.clone(); let database = self.plan.database.to_string(); let table = self.plan.table.to_string(); - pipeline.main_pipeline.set_on_finished(|err| { + pipeline.main_pipeline.set_on_finished(move |err| { + metrics_inc_replace_mutation_time_ms(start.elapsed().as_millis() as u64); if err.is_none() { info!("execute replace into finished successfully. running table optimization job."); match GlobalIORuntime::instance().block_on({ @@ -142,6 +148,7 @@ impl Interpreter for ReplaceInterpreter { return Ok(()); } + metrics_inc_replace_execution_time_ms(start.elapsed().as_millis() as u64); Ok(()) }); } diff --git a/src/query/storages/fuse/src/operations/commit.rs b/src/query/storages/fuse/src/operations/commit.rs index f71c6f9f49c59..936d620cd55fa 100644 --- a/src/query/storages/fuse/src/operations/commit.rs +++ b/src/query/storages/fuse/src/operations/commit.rs @@ -34,10 +34,10 @@ use common_meta_types::MatchSeq; use common_pipeline_core::processors::processor::ProcessorPtr; use common_pipeline_core::Pipeline; use common_pipeline_transforms::processors::transforms::AsyncAccumulatingTransformer; +use common_sql::executor::MutationKind; use log::debug; use log::info; use log::warn; -use common_sql::executor::MutationKind; use opendal::Operator; use storages_common_cache::CacheAccessor; use storages_common_cache_manager::CachedObject; diff --git a/src/query/storages/fuse/src/operations/common/processors/sink_commit.rs b/src/query/storages/fuse/src/operations/common/processors/sink_commit.rs index 883b9181fef49..714574b0d7bd4 100644 --- a/src/query/storages/fuse/src/operations/common/processors/sink_commit.rs +++ b/src/query/storages/fuse/src/operations/common/processors/sink_commit.rs @@ -38,7 +38,6 @@ use storages_common_table_meta::meta::TableSnapshot; use storages_common_table_meta::meta::Versioned; use table_lock::TableLockHandlerWrapper; use table_lock::TableLockHeartbeat; -use tracing::info; use crate::io::TableMetaLocationGenerator; use crate::metrics::metrics_inc_commit_aborts; diff --git a/src/query/storages/fuse/src/operations/replace_into/processors/processor_replace_into.rs b/src/query/storages/fuse/src/operations/replace_into/processors/processor_replace_into.rs index b6e73df3ded90..2aaf8e3fbc791 100644 --- a/src/query/storages/fuse/src/operations/replace_into/processors/processor_replace_into.rs +++ b/src/query/storages/fuse/src/operations/replace_into/processors/processor_replace_into.rs @@ -32,7 +32,6 @@ use common_pipeline_core::processors::processor::ProcessorPtr; use common_pipeline_core::processors::Processor; use common_sql::executor::OnConflictField; use storages_common_table_meta::meta::ColumnStatistics; -use tracing::error; use crate::metrics::metrics_inc_replace_process_input_block_time_ms; use crate::operations::replace_into::mutator::mutator_replace_into::ReplaceIntoMutator; @@ -124,7 +123,6 @@ impl Processor for ReplaceIntoProcessor { let mut pushed_something = false; if self.output_port_append_data.can_push() { if let Some(data) = self.output_data_append.take() { - error!("replace into processor: output data: {:?}", data); self.output_port_append_data.push_data(Ok(data)); pushed_something = true; } @@ -149,7 +147,6 @@ impl Processor for ReplaceIntoProcessor { { // no pending data (being sent to down streams) self.input_data = Some(self.input_port.pull_data().unwrap()?); - error!("replace into processor: input data: {:?}", self.input_data); Ok(Event::Sync) } else { // data pending From 5445e6b3739241c3245980094509955070dade5d Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Mon, 31 Jul 2023 17:59:26 +0800 Subject: [PATCH 21/40] fix schema cast --- .../ci/deploy/databend-query-standalone.sh | 2 +- .../src/interpreters/interpreter_replace.rs | 36 +++++++++---------- .../service/src/pipelines/pipeline_builder.rs | 8 +++-- 3 files changed, 24 insertions(+), 22 deletions(-) diff --git a/scripts/ci/deploy/databend-query-standalone.sh b/scripts/ci/deploy/databend-query-standalone.sh index 69a4fb732773e..20d287ed8b9df 100755 --- a/scripts/ci/deploy/databend-query-standalone.sh +++ b/scripts/ci/deploy/databend-query-standalone.sh @@ -26,7 +26,7 @@ echo "Waiting on databend-meta 10 seconds..." python3 scripts/ci/wait_tcp.py --timeout 10 --port 9191 echo 'Start databend-query...' -nohup target/${BUILD_PROFILE}/databend-query -c scripts/ci/deploy/config/databend-query-node-1.toml --internal-enable-sandbox-tenant & +nohup target/${BUILD_PROFILE}/databend-query -c scripts/ci/deploy/config/databend-query-node-2.toml --internal-enable-sandbox-tenant & echo "Waiting on databend-query 10 seconds..." python3 scripts/ci/wait_tcp.py --timeout 30 --port 3307 diff --git a/src/query/service/src/interpreters/interpreter_replace.rs b/src/query/service/src/interpreters/interpreter_replace.rs index eeed3309464d2..52d75b48f38dd 100644 --- a/src/query/service/src/interpreters/interpreter_replace.rs +++ b/src/query/service/src/interpreters/interpreter_replace.rs @@ -22,7 +22,7 @@ use common_exception::Result; use common_expression::DataSchemaRef; use common_sql::executor::AsyncSourcerPlan; use common_sql::executor::Deduplicate; -use common_sql::executor::Exchange; +// use common_sql::executor::Exchange; use common_sql::executor::MutationAggregate; use common_sql::executor::MutationKind; use common_sql::executor::OnConflictField; @@ -194,7 +194,7 @@ impl ReplaceInterpreter { Arc::new(TableSnapshot::new_empty_snapshot(schema.as_ref().clone())) }); - let is_distributed = !self.ctx.get_cluster().is_empty(); + // let is_distributed = !self.ctx.get_cluster().is_empty(); let table_is_empty = base_snapshot.segments.is_empty(); let table_level_range_index = base_snapshot.summary.col_stats.clone(); let max_threads = self.ctx.get_settings().get_max_threads()?; @@ -213,14 +213,14 @@ impl ReplaceInterpreter { table_schema: plan.schema.clone(), table_level_range_index, })); - if is_distributed { - root = Box::new(PhysicalPlan::Exchange(Exchange { - plan_id: 0, - input: root, - kind: common_sql::executor::FragmentKind::Expansive, - keys: vec![], - })); - } + // if is_distributed { + // root = Box::new(PhysicalPlan::Exchange(Exchange { + // plan_id: 0, + // input: root, + // kind: common_sql::executor::FragmentKind::Expansive, + // keys: vec![], + // })); + // } root = Box::new(PhysicalPlan::ReplaceInto(ReplaceInto { input: root, segment_partition_num, @@ -230,14 +230,14 @@ impl ReplaceInterpreter { on_conflicts, snapshot: (*base_snapshot).clone(), })); - if is_distributed { - root = Box::new(PhysicalPlan::Exchange(Exchange { - plan_id: 0, - input: root, - kind: common_sql::executor::FragmentKind::Merge, - keys: vec![], - })); - } + // if is_distributed { + // root = Box::new(PhysicalPlan::Exchange(Exchange { + // plan_id: 0, + // input: root, + // kind: common_sql::executor::FragmentKind::Merge, + // keys: vec![], + // })); + // } root = Box::new(PhysicalPlan::MutationAggregate(Box::new( MutationAggregate { input: root, diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs index 8c4e66728152d..0dd0934e704af 100644 --- a/src/query/service/src/pipelines/pipeline_builder.rs +++ b/src/query/service/src/pipelines/pipeline_builder.rs @@ -502,7 +502,7 @@ impl PipelineBuilder { fn build_copy_into_table(&mut self, copy: &CopyIntoTable) -> Result<()> { let catalog = self.ctx.get_catalog(©.catalog_name)?; let to_table = catalog.get_table_by_info(©.table_info)?; - match ©.source { + let source_schema = match ©.source { CopyIntoTableSource::Query(input) => { self.build_pipeline(&input.plan)?; Self::render_result_set( @@ -512,18 +512,20 @@ impl PipelineBuilder { &mut self.main_pipeline, input.ignore_result, )?; + input.query_source_schema.clone() } CopyIntoTableSource::Stage(source) => { let stage_table = StageTable::try_create(copy.stage_table_info.clone())?; stage_table.set_block_thresholds(to_table.get_block_thresholds()); stage_table.read_data(self.ctx.clone(), source, &mut self.main_pipeline)?; + copy.required_source_schema.clone() } - } + }; build_append_data_pipeline( self.ctx.clone(), &mut self.main_pipeline, copy, - copy.required_source_schema.clone(), + source_schema, to_table, )?; Ok(()) From 0b340c75035a209036ff26aa377c7bdc9fa70f7b Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Mon, 31 Jul 2023 19:13:06 +0800 Subject: [PATCH 22/40] fix conlfict --- .../service/src/interpreters/interpreter_copy.rs | 8 ++++++-- .../service/src/interpreters/interpreter_replace.rs | 7 ++++--- src/query/service/src/pipelines/builders/copy.rs | 6 +++++- src/query/service/src/pipelines/pipeline_builder.rs | 13 +++++++------ src/query/sql/src/executor/physical_plan.rs | 6 +++--- 5 files changed, 25 insertions(+), 15 deletions(-) diff --git a/src/query/service/src/interpreters/interpreter_copy.rs b/src/query/service/src/interpreters/interpreter_copy.rs index ae12911abb545..7c450b8528d04 100644 --- a/src/query/service/src/interpreters/interpreter_copy.rs +++ b/src/query/service/src/interpreters/interpreter_copy.rs @@ -147,7 +147,11 @@ impl CopyInterpreter { ) -> Result<(PhysicalPlan, Vec)> { let to_table = self .ctx - .get_table(&plan.catalog_info.catalog_name(), &plan.database_name, &plan.table_name) + .get_table( + plan.catalog_info.catalog_name(), + &plan.database_name, + &plan.table_name, + ) .await?; let files = plan.collect_files(self.ctx.as_ref()).await?; let source = if let Some(ref query) = plan.query { @@ -181,7 +185,7 @@ impl CopyInterpreter { }; let mut root = PhysicalPlan::CopyIntoTable(Box::new(CopyIntoTable { - catalog_name: plan.catalog_name.clone(), + catalog_info: plan.catalog_info.clone(), required_values_schema: plan.required_values_schema.clone(), values_consts: plan.values_consts.clone(), required_source_schema: plan.required_source_schema.clone(), diff --git a/src/query/service/src/interpreters/interpreter_replace.rs b/src/query/service/src/interpreters/interpreter_replace.rs index 52d75b48f38dd..8524668a64eec 100644 --- a/src/query/service/src/interpreters/interpreter_replace.rs +++ b/src/query/service/src/interpreters/interpreter_replace.rs @@ -163,6 +163,7 @@ impl ReplaceInterpreter { .ctx .get_table(&plan.catalog, &plan.database, &plan.table) .await?; + let catalog = self.ctx.get_catalog(&plan.catalog).await?; let schema = table.schema(); let mut on_conflicts = Vec::with_capacity(plan.on_conflict_fields.len()); for f in &plan.on_conflict_fields { @@ -208,7 +209,7 @@ impl ReplaceInterpreter { on_conflicts: on_conflicts.clone(), table_is_empty, table_info: table_info.clone(), - catalog_name: plan.catalog.clone(), + catalog_info: catalog.info(), select_ctx, table_schema: plan.schema.clone(), table_level_range_index, @@ -226,7 +227,7 @@ impl ReplaceInterpreter { segment_partition_num, block_thresholds: fuse_table.get_block_thresholds(), table_info: table_info.clone(), - catalog_name: plan.catalog.clone(), + catalog_info: catalog.info(), on_conflicts, snapshot: (*base_snapshot).clone(), })); @@ -243,7 +244,7 @@ impl ReplaceInterpreter { input: root, snapshot: (*base_snapshot).clone(), table_info: table_info.clone(), - catalog_name: plan.catalog.clone(), + catalog_info: catalog.info(), mutation_kind: MutationKind::Replace, }, ))); diff --git a/src/query/service/src/pipelines/builders/copy.rs b/src/query/service/src/pipelines/builders/copy.rs index 849cf89edd596..5a584500913c6 100644 --- a/src/query/service/src/pipelines/builders/copy.rs +++ b/src/query/service/src/pipelines/builders/copy.rs @@ -109,7 +109,11 @@ pub async fn build_commit_data_pipeline( files: &[StageFileInfo], ) -> Result<()> { let to_table = ctx - .get_table(&plan.catalog_name, &plan.database_name, &plan.table_name) + .get_table( + plan.catalog_info.catalog_name(), + &plan.database_name, + &plan.table_name, + ) .await?; // Source node will do: // 1. commit diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs index c8a44ddad7cf8..7a66216765708 100644 --- a/src/query/service/src/pipelines/pipeline_builder.rs +++ b/src/query/service/src/pipelines/pipeline_builder.rs @@ -279,14 +279,14 @@ impl PipelineBuilder { on_conflicts, table_is_empty, table_info, - catalog_name, + catalog_info, select_ctx, table_level_range_index, table_schema, } = deduplicate; let tbl = self .ctx - .build_table_by_table_info(catalog_name, table_info, None)?; + .build_table_by_table_info(catalog_info, table_info, None)?; let table = FuseTable::try_from_table(tbl.as_ref())?; let target_schema: Arc = Arc::new(table_schema.clone().into()); self.build_pipeline(input)?; @@ -368,12 +368,12 @@ impl PipelineBuilder { block_thresholds, table_info, on_conflicts, - catalog_name, + catalog_info, snapshot, } = replace; let table = self .ctx - .build_table_by_table_info(catalog_name, table_info, None)?; + .build_table_by_table_info(catalog_info, table_info, None)?; let table = FuseTable::try_from_table(table.as_ref())?; let cluster_stats_gen = table.get_cluster_stats_gen(self.ctx.clone(), 0, *block_thresholds)?; @@ -500,8 +500,9 @@ impl PipelineBuilder { } fn build_copy_into_table(&mut self, copy: &CopyIntoTable) -> Result<()> { - let catalog = self.ctx.get_catalog(©.catalog_name)?; - let to_table = catalog.get_table_by_info(©.table_info)?; + let to_table = + self.ctx + .build_table_by_table_info(©.catalog_info, ©.table_info, None)?; let source_schema = match ©.source { CopyIntoTableSource::Query(input) => { self.build_pipeline(&input.plan)?; diff --git a/src/query/sql/src/executor/physical_plan.rs b/src/query/sql/src/executor/physical_plan.rs index 294652467b201..71616175af76b 100644 --- a/src/query/sql/src/executor/physical_plan.rs +++ b/src/query/sql/src/executor/physical_plan.rs @@ -789,7 +789,7 @@ impl UnionAll { #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct CopyIntoTable { - pub catalog_info:CatalogInfo, + pub catalog_info: CatalogInfo, pub required_values_schema: DataSchemaRef, pub values_consts: Vec, pub required_source_schema: DataSchemaRef, @@ -918,7 +918,7 @@ pub struct Deduplicate { pub on_conflicts: Vec, pub table_is_empty: bool, pub table_info: TableInfo, - pub catalog_name: String, + pub catalog_info: CatalogInfo, pub table_schema: TableSchemaRef, pub select_ctx: Option, pub table_level_range_index: HashMap, @@ -943,7 +943,7 @@ pub struct ReplaceInto { pub block_thresholds: BlockThresholds, pub table_info: TableInfo, pub on_conflicts: Vec, - pub catalog_name: String, + pub catalog_info: CatalogInfo, pub snapshot: TableSnapshot, } From 88726fd363a5ac216c151631ec1c3ba139894b09 Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Mon, 31 Jul 2023 19:28:19 +0800 Subject: [PATCH 23/40] fix --- scripts/ci/deploy/databend-query-standalone.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/scripts/ci/deploy/databend-query-standalone.sh b/scripts/ci/deploy/databend-query-standalone.sh index 20d287ed8b9df..69a4fb732773e 100755 --- a/scripts/ci/deploy/databend-query-standalone.sh +++ b/scripts/ci/deploy/databend-query-standalone.sh @@ -26,7 +26,7 @@ echo "Waiting on databend-meta 10 seconds..." python3 scripts/ci/wait_tcp.py --timeout 10 --port 9191 echo 'Start databend-query...' -nohup target/${BUILD_PROFILE}/databend-query -c scripts/ci/deploy/config/databend-query-node-2.toml --internal-enable-sandbox-tenant & +nohup target/${BUILD_PROFILE}/databend-query -c scripts/ci/deploy/config/databend-query-node-1.toml --internal-enable-sandbox-tenant & echo "Waiting on databend-query 10 seconds..." python3 scripts/ci/wait_tcp.py --timeout 30 --port 3307 From 5f7f3c7594729bf07c04e16361d8cc83b175cfe3 Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Mon, 31 Jul 2023 20:31:50 +0800 Subject: [PATCH 24/40] fix copy plan --- src/query/service/src/interpreters/interpreter_copy.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/query/service/src/interpreters/interpreter_copy.rs b/src/query/service/src/interpreters/interpreter_copy.rs index 7c450b8528d04..6f8529addaeae 100644 --- a/src/query/service/src/interpreters/interpreter_copy.rs +++ b/src/query/service/src/interpreters/interpreter_copy.rs @@ -198,7 +198,7 @@ impl CopyInterpreter { files: files.clone(), source, })); - if plan.enable_distributed && plan.query.is_none() { + if plan.enable_distributed { root = PhysicalPlan::Exchange(Exchange { plan_id: 0, input: Box::new(root), From fd3587e8c6af18420989761685dd58def566f449 Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Mon, 31 Jul 2023 21:35:58 +0800 Subject: [PATCH 25/40] clear log --- .../transforms/transform_accumulating_async.rs | 10 ---------- .../fuse/src/operations/common/mutation_accumulator.rs | 2 -- .../src/operations/common/processors/sink_commit.rs | 5 ----- 3 files changed, 17 deletions(-) diff --git a/src/query/pipeline/transforms/src/processors/transforms/transform_accumulating_async.rs b/src/query/pipeline/transforms/src/processors/transforms/transform_accumulating_async.rs index 1f5ddc3703a83..209474572cce8 100644 --- a/src/query/pipeline/transforms/src/processors/transforms/transform_accumulating_async.rs +++ b/src/query/pipeline/transforms/src/processors/transforms/transform_accumulating_async.rs @@ -81,11 +81,6 @@ impl Processor for AsyncAccumulatingTra } if let Some(data_block) = self.output_data.take() { - eprintln!( - "{} - send data block: {:?}", - self.name(), - data_block.get_meta() - ); self.output.push_data(Ok(data_block)); return Ok(Event::NeedConsume); } @@ -96,11 +91,6 @@ impl Processor for AsyncAccumulatingTra if self.input.has_data() { self.input_data = Some(self.input.pull_data().unwrap()?); - eprintln!( - "{} - recv data block: {:?}", - self.name(), - self.input_data.as_ref().unwrap().get_meta() - ); return Ok(Event::Async); } diff --git a/src/query/storages/fuse/src/operations/common/mutation_accumulator.rs b/src/query/storages/fuse/src/operations/common/mutation_accumulator.rs index 15e93657d6216..a484657004fcc 100644 --- a/src/query/storages/fuse/src/operations/common/mutation_accumulator.rs +++ b/src/query/storages/fuse/src/operations/common/mutation_accumulator.rs @@ -256,8 +256,6 @@ impl MutationAccumulator { .map(|(path, _segment, format_version)| (path.clone(), *format_version)) .chain(merged_segments) .collect(); - info!("merged_segments:{:?}", merged_segments); - info!("append_segments:{:?}", self.appended_segments); match self.kind { MutationKind::Insert => ConflictResolveContext::AppendOnly(( SnapshotMerged { diff --git a/src/query/storages/fuse/src/operations/common/processors/sink_commit.rs b/src/query/storages/fuse/src/operations/common/processors/sink_commit.rs index 714574b0d7bd4..d506030dc1bc3 100644 --- a/src/query/storages/fuse/src/operations/common/processors/sink_commit.rs +++ b/src/query/storages/fuse/src/operations/common/processors/sink_commit.rs @@ -163,10 +163,6 @@ where F: SnapshotGenerator + Send + 'static self.abort_operation = meta.abort_operation; self.backoff = FuseTable::set_backoff(self.max_retry_elapsed); - info!( - "conflict resolve context: {:?}", - meta.conflict_resolve_context - ); self.snapshot_gen .set_conflict_resolve_context(meta.conflict_resolve_context); @@ -236,7 +232,6 @@ where F: SnapshotGenerator + Send + 'static .generate_new_snapshot(schema, cluster_key_meta, previous) { Ok(snapshot) => { - info!("generated snapshot: {:?}", snapshot); self.state = State::TryCommit { data: snapshot.to_bytes()?, snapshot, From c9db180d28c1273309e0b957d2c030c8d15fcdbb Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Mon, 31 Jul 2023 23:09:53 +0800 Subject: [PATCH 26/40] fix copy --- .../src/schedulers/fragments/plan_fragment.rs | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/src/query/service/src/schedulers/fragments/plan_fragment.rs b/src/query/service/src/schedulers/fragments/plan_fragment.rs index a95d4fec9aa82..f00dae042bfbd 100644 --- a/src/query/service/src/schedulers/fragments/plan_fragment.rs +++ b/src/query/service/src/schedulers/fragments/plan_fragment.rs @@ -21,6 +21,7 @@ use common_exception::Result; use common_sql::executor::CopyIntoTable; use common_sql::executor::CopyIntoTableSource; use common_sql::executor::DeletePartial; +use common_sql::executor::QueryCtx; use crate::api::DataExchange; use crate::schedulers::Fragmenter; @@ -249,8 +250,15 @@ impl PhysicalPlanReplacer for ReplaceReadSource { fn replace_copy_into_table(&mut self, plan: &CopyIntoTable) -> Result { match &plan.source { - CopyIntoTableSource::Query(_) => { - Ok(PhysicalPlan::CopyIntoTable(Box::new(plan.clone()))) + CopyIntoTableSource::Query(query_ctx) => { + let input = self.replace(&query_ctx.plan)?; + Ok(PhysicalPlan::CopyIntoTable(Box::new(CopyIntoTable { + source: CopyIntoTableSource::Query(Box::new(QueryCtx { + plan: input, + ..*query_ctx.clone() + })), + ..plan.clone() + }))) } CopyIntoTableSource::Stage(_) => { Ok(PhysicalPlan::CopyIntoTable(Box::new(CopyIntoTable { From 84dd7c9ec511efd2309685e13dab4021f449f5b9 Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Tue, 1 Aug 2023 08:43:22 +0800 Subject: [PATCH 27/40] fix copy --- .../src/schedulers/fragments/fragmenter.rs | 20 ++++++++++++++++--- 1 file changed, 17 insertions(+), 3 deletions(-) diff --git a/src/query/service/src/schedulers/fragments/fragmenter.rs b/src/query/service/src/schedulers/fragments/fragmenter.rs index 97e38f0feb575..b0d798178e256 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::CopyIntoTable; use common_sql::executor::CopyIntoTableSource; use common_sql::executor::FragmentKind; +use common_sql::executor::QueryCtx; use crate::api::BroadcastExchange; use crate::api::DataExchange; @@ -140,11 +141,24 @@ impl PhysicalPlanReplacer for Fragmenter { Ok(PhysicalPlan::TableScan(plan.clone())) } + // TODO(Sky): remove rebudant code fn replace_copy_into_table(&mut self, plan: &CopyIntoTable) -> Result { - if let CopyIntoTableSource::Stage(_) = plan.source { - self.state = State::SelectLeaf; + match &plan.source { + CopyIntoTableSource::Stage(_) => { + self.state = State::SelectLeaf; + Ok(PhysicalPlan::CopyIntoTable(Box::new(plan.clone()))) + } + CopyIntoTableSource::Query(query_ctx) => { + let input = self.replace(&query_ctx.plan)?; + Ok(PhysicalPlan::CopyIntoTable(Box::new(CopyIntoTable { + source: CopyIntoTableSource::Query(Box::new(QueryCtx { + plan: input, + ..*query_ctx.clone() + })), + ..plan.clone() + }))) + } } - Ok(PhysicalPlan::CopyIntoTable(Box::new(plan.clone()))) } fn replace_delete_partial( From 0f0b0580e4cfdd0e04163347284fc5c075b3a80e Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Tue, 1 Aug 2023 08:57:56 +0800 Subject: [PATCH 28/40] run ci --- src/query/service/src/schedulers/fragments/fragmenter.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/query/service/src/schedulers/fragments/fragmenter.rs b/src/query/service/src/schedulers/fragments/fragmenter.rs index b0d798178e256..7d31a6f9da3be 100644 --- a/src/query/service/src/schedulers/fragments/fragmenter.rs +++ b/src/query/service/src/schedulers/fragments/fragmenter.rs @@ -141,7 +141,7 @@ impl PhysicalPlanReplacer for Fragmenter { Ok(PhysicalPlan::TableScan(plan.clone())) } - // TODO(Sky): remove rebudant code + // TODO(Sky): remove rebudant code fn replace_copy_into_table(&mut self, plan: &CopyIntoTable) -> Result { match &plan.source { CopyIntoTableSource::Stage(_) => { From 43ecd8a50f3821e1998a96a4ea7165e45bde57e9 Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Tue, 1 Aug 2023 16:12:39 +0800 Subject: [PATCH 29/40] fix purge --- .../src/interpreters/interpreter_replace.rs | 50 ++++++++++++++++--- 1 file changed, 42 insertions(+), 8 deletions(-) diff --git a/src/query/service/src/interpreters/interpreter_replace.rs b/src/query/service/src/interpreters/interpreter_replace.rs index 8524668a64eec..dbdadc4d0f2b1 100644 --- a/src/query/service/src/interpreters/interpreter_replace.rs +++ b/src/query/service/src/interpreters/interpreter_replace.rs @@ -20,6 +20,7 @@ use common_catalog::table_context::TableContext; use common_exception::ErrorCode; use common_exception::Result; use common_expression::DataSchemaRef; +use common_meta_app::principal::StageInfo; use common_sql::executor::AsyncSourcerPlan; use common_sql::executor::Deduplicate; // use common_sql::executor::Exchange; @@ -35,6 +36,7 @@ use common_sql::plans::OptimizeTableAction; use common_sql::plans::OptimizeTablePlan; use common_sql::plans::Plan; use common_sql::plans::Replace; +use common_storage::StageFileInfo; use common_storages_factory::Table; use common_storages_fuse::FuseTable; use log::info; @@ -48,6 +50,7 @@ use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; use crate::interpreters::OptimizeTableInterpreter; use crate::interpreters::SelectInterpreter; +use crate::pipelines::builders::set_copy_on_finished; use crate::pipelines::executor::ExecutorSettings; use crate::pipelines::executor::PipelineCompleteExecutor; use crate::pipelines::PipelineBuildResult; @@ -58,11 +61,16 @@ use crate::sessions::QueryContext; pub struct ReplaceInterpreter { ctx: Arc, plan: Replace, + files: Option>, } impl ReplaceInterpreter { pub fn try_create(ctx: Arc, plan: Replace) -> Result { - Ok(Arc::new(ReplaceInterpreter { ctx, plan })) + Ok(Arc::new(ReplaceInterpreter { + ctx, + plan, + files: None, + })) } } @@ -81,11 +89,24 @@ impl Interpreter for ReplaceInterpreter { self.check_on_conflicts()?; let start = Instant::now(); - let physical_plan = self.build_physical_plan().await?; + // replace + let (physical_plan, purge_info) = self.build_physical_plan().await?; let mut pipeline = build_query_pipeline_without_render_result_set(&self.ctx, &physical_plan, false) .await?; + // purge + if let Some((files, stage_info)) = purge_info { + set_copy_on_finished( + self.ctx.clone(), + files, + stage_info.copy_options.purge, + stage_info.clone(), + &mut pipeline.main_pipeline, + )?; + } + + // recluster let plan = &self.plan; let table = self .ctx @@ -157,7 +178,9 @@ impl Interpreter for ReplaceInterpreter { } impl ReplaceInterpreter { - async fn build_physical_plan(&self) -> Result> { + async fn build_physical_plan( + &self, + ) -> Result<(Box, Option<(Vec, StageInfo)>)> { let plan = &self.plan; let table = self .ctx @@ -201,8 +224,14 @@ impl ReplaceInterpreter { let max_threads = self.ctx.get_settings().get_max_threads()?; let segment_partition_num = std::cmp::min(base_snapshot.segments.len(), max_threads as usize); + let mut purge_info = None; let (mut root, select_ctx) = self - .connect_input_source(self.ctx.clone(), &self.plan.source, self.plan.schema()) + .connect_input_source( + self.ctx.clone(), + &self.plan.source, + self.plan.schema(), + &mut purge_info, + ) .await?; root = Box::new(PhysicalPlan::Deduplicate(Deduplicate { input: root, @@ -248,7 +277,7 @@ impl ReplaceInterpreter { mutation_kind: MutationKind::Replace, }, ))); - Ok(root) + Ok((root, purge_info)) } fn check_on_conflicts(&self) -> Result<()> { if self.plan.on_conflict_fields.is_empty() { @@ -265,6 +294,7 @@ impl ReplaceInterpreter { ctx: Arc, source: &'a InsertInputSource, schema: DataSchemaRef, + purge_info: &mut Option<(Vec, StageInfo)>, ) -> Result<(Box, Option)> { match source { InsertInputSource::Values(data) => self @@ -279,10 +309,14 @@ impl ReplaceInterpreter { CopyPlan::IntoTable(copy_into_table_plan) => { let interpreter = CopyInterpreter::try_create(ctx.clone(), *copy_plan.clone())?; - interpreter + let (physical_plan, files) = interpreter .build_physical_plan(copy_into_table_plan) - .await - .map(|x| (Box::new(x.0), None)) + .await?; + *purge_info = Some(( + files, + copy_into_table_plan.stage_table_info.stage_info.clone(), + )); + Ok((Box::new(physical_plan), None)) } _ => unreachable!("plan in InsertInputSource::Stag must be CopyIntoTable"), }, From 56dc5e21354cc0303493b09067e46b1171c21fbe Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Tue, 1 Aug 2023 16:16:44 +0800 Subject: [PATCH 30/40] make lint --- src/query/service/src/interpreters/interpreter_replace.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/query/service/src/interpreters/interpreter_replace.rs b/src/query/service/src/interpreters/interpreter_replace.rs index dbdadc4d0f2b1..1f6e3f72b1a2f 100644 --- a/src/query/service/src/interpreters/interpreter_replace.rs +++ b/src/query/service/src/interpreters/interpreter_replace.rs @@ -101,7 +101,7 @@ impl Interpreter for ReplaceInterpreter { self.ctx.clone(), files, stage_info.copy_options.purge, - stage_info.clone(), + stage_info, &mut pipeline.main_pipeline, )?; } From 1cb294e3ad8ff9a557fd29f7d1de47db8da0bac9 Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Wed, 2 Aug 2023 08:25:20 +0800 Subject: [PATCH 31/40] add exchange --- .../src/interpreters/interpreter_replace.rs | 36 +++++++++---------- 1 file changed, 18 insertions(+), 18 deletions(-) diff --git a/src/query/service/src/interpreters/interpreter_replace.rs b/src/query/service/src/interpreters/interpreter_replace.rs index 1f6e3f72b1a2f..a11ee9bb9572c 100644 --- a/src/query/service/src/interpreters/interpreter_replace.rs +++ b/src/query/service/src/interpreters/interpreter_replace.rs @@ -23,7 +23,7 @@ use common_expression::DataSchemaRef; use common_meta_app::principal::StageInfo; use common_sql::executor::AsyncSourcerPlan; use common_sql::executor::Deduplicate; -// use common_sql::executor::Exchange; +use common_sql::executor::Exchange; use common_sql::executor::MutationAggregate; use common_sql::executor::MutationKind; use common_sql::executor::OnConflictField; @@ -218,7 +218,7 @@ impl ReplaceInterpreter { Arc::new(TableSnapshot::new_empty_snapshot(schema.as_ref().clone())) }); - // let is_distributed = !self.ctx.get_cluster().is_empty(); + let is_distributed = !self.ctx.get_cluster().is_empty(); let table_is_empty = base_snapshot.segments.is_empty(); let table_level_range_index = base_snapshot.summary.col_stats.clone(); let max_threads = self.ctx.get_settings().get_max_threads()?; @@ -243,14 +243,14 @@ impl ReplaceInterpreter { table_schema: plan.schema.clone(), table_level_range_index, })); - // if is_distributed { - // root = Box::new(PhysicalPlan::Exchange(Exchange { - // plan_id: 0, - // input: root, - // kind: common_sql::executor::FragmentKind::Expansive, - // keys: vec![], - // })); - // } + if is_distributed { + root = Box::new(PhysicalPlan::Exchange(Exchange { + plan_id: 0, + input: root, + kind: common_sql::executor::FragmentKind::Expansive, + keys: vec![], + })); + } root = Box::new(PhysicalPlan::ReplaceInto(ReplaceInto { input: root, segment_partition_num, @@ -260,14 +260,14 @@ impl ReplaceInterpreter { on_conflicts, snapshot: (*base_snapshot).clone(), })); - // if is_distributed { - // root = Box::new(PhysicalPlan::Exchange(Exchange { - // plan_id: 0, - // input: root, - // kind: common_sql::executor::FragmentKind::Merge, - // keys: vec![], - // })); - // } + if is_distributed { + root = Box::new(PhysicalPlan::Exchange(Exchange { + plan_id: 0, + input: root, + kind: common_sql::executor::FragmentKind::Merge, + keys: vec![], + })); + } root = Box::new(PhysicalPlan::MutationAggregate(Box::new( MutationAggregate { input: root, From ea6866362470446b0e6a34c440c78b61e8bc3895 Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Fri, 4 Aug 2023 09:53:20 +0800 Subject: [PATCH 32/40] disable dist for value source --- src/query/service/src/interpreters/interpreter_replace.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/query/service/src/interpreters/interpreter_replace.rs b/src/query/service/src/interpreters/interpreter_replace.rs index a11ee9bb9572c..0e7416c82cb61 100644 --- a/src/query/service/src/interpreters/interpreter_replace.rs +++ b/src/query/service/src/interpreters/interpreter_replace.rs @@ -218,7 +218,9 @@ impl ReplaceInterpreter { Arc::new(TableSnapshot::new_empty_snapshot(schema.as_ref().clone())) }); - let is_distributed = !self.ctx.get_cluster().is_empty(); + let is_multi_node = !self.ctx.get_cluster().is_empty(); + let is_value_source = matches!(self.plan.source, InsertInputSource::Values(_)); + let is_distributed = is_multi_node && !is_value_source; let table_is_empty = base_snapshot.segments.is_empty(); let table_level_range_index = base_snapshot.summary.col_stats.clone(); let max_threads = self.ctx.get_settings().get_max_threads()?; From 7c6da18762583316b51dc8f0fb78e46b63d1efef Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Fri, 4 Aug 2023 09:55:13 +0800 Subject: [PATCH 33/40] adjust exchange --- .../src/interpreters/interpreter_replace.rs | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/src/query/service/src/interpreters/interpreter_replace.rs b/src/query/service/src/interpreters/interpreter_replace.rs index 0e7416c82cb61..acf2733709912 100644 --- a/src/query/service/src/interpreters/interpreter_replace.rs +++ b/src/query/service/src/interpreters/interpreter_replace.rs @@ -235,6 +235,14 @@ impl ReplaceInterpreter { &mut purge_info, ) .await?; + if is_distributed { + root = Box::new(PhysicalPlan::Exchange(Exchange { + plan_id: 0, + input: root, + kind: common_sql::executor::FragmentKind::Expansive, + keys: vec![], + })); + } root = Box::new(PhysicalPlan::Deduplicate(Deduplicate { input: root, on_conflicts: on_conflicts.clone(), @@ -245,14 +253,6 @@ impl ReplaceInterpreter { table_schema: plan.schema.clone(), table_level_range_index, })); - if is_distributed { - root = Box::new(PhysicalPlan::Exchange(Exchange { - plan_id: 0, - input: root, - kind: common_sql::executor::FragmentKind::Expansive, - keys: vec![], - })); - } root = Box::new(PhysicalPlan::ReplaceInto(ReplaceInto { input: root, segment_partition_num, From 03a6cc7d77b9f0f1aa3884a02bae7b134f87e18e Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Fri, 4 Aug 2023 10:46:36 +0800 Subject: [PATCH 34/40] remove top exchange --- src/query/service/src/interpreters/interpreter_replace.rs | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/query/service/src/interpreters/interpreter_replace.rs b/src/query/service/src/interpreters/interpreter_replace.rs index acf2733709912..226b374dfddf0 100644 --- a/src/query/service/src/interpreters/interpreter_replace.rs +++ b/src/query/service/src/interpreters/interpreter_replace.rs @@ -235,6 +235,10 @@ impl ReplaceInterpreter { &mut purge_info, ) .await?; + // remove top exchange + if let PhysicalPlan::Exchange(Exchange { input, .. }) = root.as_ref() { + root = input.clone(); + } if is_distributed { root = Box::new(PhysicalPlan::Exchange(Exchange { plan_id: 0, From 154bbb7e7ea5e8fcde97cc32418bf5570b15e82c Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Fri, 4 Aug 2023 11:08:11 +0800 Subject: [PATCH 35/40] adjust replace into --- src/query/service/src/interpreters/interpreter_replace.rs | 2 +- src/query/service/src/pipelines/pipeline_builder.rs | 4 ++-- src/query/sql/src/executor/physical_plan.rs | 3 ++- src/query/storages/fuse/src/operations/replace.rs | 4 ++-- 4 files changed, 7 insertions(+), 6 deletions(-) diff --git a/src/query/service/src/interpreters/interpreter_replace.rs b/src/query/service/src/interpreters/interpreter_replace.rs index 226b374dfddf0..abb1174a17c5d 100644 --- a/src/query/service/src/interpreters/interpreter_replace.rs +++ b/src/query/service/src/interpreters/interpreter_replace.rs @@ -264,7 +264,7 @@ impl ReplaceInterpreter { table_info: table_info.clone(), catalog_info: catalog.info(), on_conflicts, - snapshot: (*base_snapshot).clone(), + segments: base_snapshot.segments.clone(), })); if is_distributed { root = Box::new(PhysicalPlan::Exchange(Exchange { diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs index 7a66216765708..6b456eac864f2 100644 --- a/src/query/service/src/pipelines/pipeline_builder.rs +++ b/src/query/service/src/pipelines/pipeline_builder.rs @@ -369,7 +369,7 @@ impl PipelineBuilder { table_info, on_conflicts, catalog_info, - snapshot, + segments, } = replace; let table = self .ctx @@ -463,7 +463,7 @@ impl PipelineBuilder { *segment_partition_num, block_builder, on_conflicts.clone(), - snapshot, + &segments, io_request_semaphore, )?; assert_eq!( diff --git a/src/query/sql/src/executor/physical_plan.rs b/src/query/sql/src/executor/physical_plan.rs index 71616175af76b..55404b8aa00e1 100644 --- a/src/query/sql/src/executor/physical_plan.rs +++ b/src/query/sql/src/executor/physical_plan.rs @@ -43,6 +43,7 @@ use common_meta_app::schema::TableInfo; use common_storage::StageFileInfo; use enum_as_inner::EnumAsInner; use storages_common_table_meta::meta::ColumnStatistics; +use storages_common_table_meta::meta::Location; use storages_common_table_meta::meta::TableSnapshot; use crate::executor::explain::PlanStatsInfo; @@ -944,7 +945,7 @@ pub struct ReplaceInto { pub table_info: TableInfo, pub on_conflicts: Vec, pub catalog_info: CatalogInfo, - pub snapshot: TableSnapshot, + pub segments: Vec, } #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] diff --git a/src/query/storages/fuse/src/operations/replace.rs b/src/query/storages/fuse/src/operations/replace.rs index 5e6438b2341d5..6221b27e45ffe 100644 --- a/src/query/storages/fuse/src/operations/replace.rs +++ b/src/query/storages/fuse/src/operations/replace.rs @@ -96,10 +96,10 @@ impl FuseTable { num_partition: usize, block_builder: BlockBuilder, on_conflicts: Vec, - table_snapshot: &TableSnapshot, + segments: &[Location], io_request_semaphore: Arc, ) -> Result> { - let chunks = Self::partition_segments(&table_snapshot.segments, num_partition); + let chunks = Self::partition_segments(segments, num_partition); let read_settings = ReadSettings::from_ctx(&ctx)?; let mut items = Vec::with_capacity(num_partition); for chunk_of_segment_locations in chunks { From 29d18ac82a44ac0a30abe8384e0334b30e26313a Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Fri, 4 Aug 2023 11:32:21 +0800 Subject: [PATCH 36/40] reshuffle --- .../src/schedulers/fragments/fragmenter.rs | 18 ++++ .../src/schedulers/fragments/plan_fragment.rs | 91 +++++++++++++++++++ 2 files changed, 109 insertions(+) diff --git a/src/query/service/src/schedulers/fragments/fragmenter.rs b/src/query/service/src/schedulers/fragments/fragmenter.rs index 7d31a6f9da3be..36cf4ecfab5eb 100644 --- a/src/query/service/src/schedulers/fragments/fragmenter.rs +++ b/src/query/service/src/schedulers/fragments/fragmenter.rs @@ -20,6 +20,7 @@ use common_sql::executor::CopyIntoTable; use common_sql::executor::CopyIntoTableSource; use common_sql::executor::FragmentKind; use common_sql::executor::QueryCtx; +use common_sql::executor::ReplaceInto; use crate::api::BroadcastExchange; use crate::api::DataExchange; @@ -50,9 +51,12 @@ pub struct Fragmenter { /// SelectLeaf: visiting a source fragment of select statement. /// /// DeleteLeaf: visiting a source fragment of delete statement. +/// +/// Replace: visiting a fragment that contains a replace into plan. enum State { SelectLeaf, DeleteLeaf, + ReplaceInto, Other, } @@ -141,6 +145,19 @@ impl PhysicalPlanReplacer for Fragmenter { Ok(PhysicalPlan::TableScan(plan.clone())) } + fn replace_replace_into( + &mut self, + plan: &common_sql::executor::ReplaceInto, + ) -> Result { + let input = self.replace(&plan.input)?; + self.state = State::ReplaceInto; + + Ok(PhysicalPlan::ReplaceInto(ReplaceInto { + input: Box::new(input), + ..plan.clone() + })) + } + // TODO(Sky): remove rebudant code fn replace_copy_into_table(&mut self, plan: &CopyIntoTable) -> Result { match &plan.source { @@ -224,6 +241,7 @@ impl PhysicalPlanReplacer for Fragmenter { State::SelectLeaf => FragmentType::Source, State::DeleteLeaf => FragmentType::DeleteLeaf, State::Other => FragmentType::Intermediate, + State::ReplaceInto => FragmentType::ReplaceInto, }; self.state = State::Other; let exchange = Self::get_exchange( diff --git a/src/query/service/src/schedulers/fragments/plan_fragment.rs b/src/query/service/src/schedulers/fragments/plan_fragment.rs index f00dae042bfbd..69ccfa5149e5d 100644 --- a/src/query/service/src/schedulers/fragments/plan_fragment.rs +++ b/src/query/service/src/schedulers/fragments/plan_fragment.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::HashMap; use std::sync::Arc; use common_catalog::plan::DataSourcePlan; @@ -22,6 +23,8 @@ use common_sql::executor::CopyIntoTable; use common_sql::executor::CopyIntoTableSource; use common_sql::executor::DeletePartial; use common_sql::executor::QueryCtx; +use common_sql::executor::ReplaceInto; +use storages_common_table_meta::meta::Location; use crate::api::DataExchange; use crate::schedulers::Fragmenter; @@ -49,6 +52,8 @@ pub enum FragmentType { /// Leaf fragment of a delete plan, which contains /// a `DeletePartial` operator. DeleteLeaf, + /// Intermediate fragment of a replace into plan, which contains a `ReplaceInto` operator. + ReplaceInto, } #[derive(Clone)] @@ -127,6 +132,14 @@ impl PlanFragment { } actions.add_fragment_actions(fragment_actions)?; } + FragmentType::ReplaceInto => { + // Redistribute partitions + let mut fragment_actions = self.redistribute_replace_into(ctx)?; + if let Some(ref exchange) = self.exchange { + fragment_actions.set_exchange(exchange.clone()); + } + actions.add_fragment_actions(fragment_actions)?; + } } Ok(()) @@ -196,6 +209,70 @@ impl PlanFragment { Ok(fragment_actions) } + fn reshuffle_segments( + executors: Vec, + partitions: Vec, + ) -> Result>> { + let num_parts = partitions.len(); + let num_executors = executors.len(); + let mut executors_sorted = executors.clone(); + executors_sorted.sort(); + let mut executor_part = HashMap::default(); + // the first num_parts % num_executors get parts_per_node parts + // the remaining get parts_per_node - 1 parts + let parts_per_node = (num_parts + num_executors - 1) / num_executors; + for (idx, executor) in executors_sorted.iter().enumerate() { + let begin = parts_per_node * idx; + let end = num_parts.min(parts_per_node * (idx + 1)); + let parts = partitions[begin..end].to_vec(); + executor_part.insert(executor.clone(), parts); + if end == num_parts && idx < num_executors - 1 { + // reach here only when num_executors > num_parts + executors_sorted[(idx + 1)..].iter().for_each(|executor| { + executor_part.insert(executor.clone(), vec![]); + }); + break; + } + } + + Ok(executor_part) + } + + fn redistribute_replace_into(&self, ctx: Arc) -> Result { + let plan = match &self.plan { + PhysicalPlan::ExchangeSink(plan) => plan, + _ => unreachable!("logic error"), + }; + let plan = match plan.input.as_ref() { + PhysicalPlan::ReplaceInto(plan) => plan, + _ => unreachable!("logic error"), + }; + let partitions = &plan.segments; + let executors = Fragmenter::get_executors(ctx); + let mut fragment_actions = QueryFragmentActions::create(self.fragment_id); + // 把partitions等分到executors上 + let executor_num = executors.len(); + let mut partition_reshuffle = vec![]; + for (i, part) in partitions.iter().enumerate() { + let executor = executors[i % executor_num].clone(); + partition_reshuffle.push((executor, vec![part.clone()])); + } + + for (executor, parts) in partitions.iter() { + let mut plan = self.plan.clone(); + + let mut replace_replace_partial = ReplaceReplaceInto { + partitions: parts.clone(), + }; + plan = replace_replace_partial.replace(&plan)?; + + fragment_actions + .add_action(QueryFragmentAction::create(executor.clone(), plan.clone())); + } + + Ok(fragment_actions) + } + fn get_read_source(&self) -> Result { if self.fragment_type != FragmentType::Source { return Err(ErrorCode::Internal( @@ -282,3 +359,17 @@ impl PhysicalPlanReplacer for ReplaceDeletePartial { }))) } } + +struct ReplaceReplaceInto { + pub partitions: Vec, +} + +impl PhysicalPlanReplacer for ReplaceReplaceInto { + fn replace_replace_into(&mut self, plan: &ReplaceInto) -> Result { + self.replace(&plan.input); + Ok(PhysicalPlan::ReplaceInto(Box::new(ReplaceInto { + segments: self.partitions.clone(), + ..plan.clone() + }))) + } +} From 5ab41ab8c61b064fbcdc71cd75f835ceddf013f7 Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Fri, 4 Aug 2023 11:33:13 +0800 Subject: [PATCH 37/40] fix --- src/query/service/src/schedulers/fragments/plan_fragment.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/query/service/src/schedulers/fragments/plan_fragment.rs b/src/query/service/src/schedulers/fragments/plan_fragment.rs index 69ccfa5149e5d..1c7566c877e80 100644 --- a/src/query/service/src/schedulers/fragments/plan_fragment.rs +++ b/src/query/service/src/schedulers/fragments/plan_fragment.rs @@ -258,7 +258,7 @@ impl PlanFragment { partition_reshuffle.push((executor, vec![part.clone()])); } - for (executor, parts) in partitions.iter() { + for (executor, parts) in partition_reshuffle.iter() { let mut plan = self.plan.clone(); let mut replace_replace_partial = ReplaceReplaceInto { From 6db338ef41676a3d57978cf305998ade9ffbcb2b Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Fri, 4 Aug 2023 11:52:52 +0800 Subject: [PATCH 38/40] fix reshuffle --- .../service/src/pipelines/pipeline_builder.rs | 2 +- .../src/schedulers/fragments/plan_fragment.rs | 21 +++++++------------ 2 files changed, 9 insertions(+), 14 deletions(-) diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs index 6b456eac864f2..fb68f0d45e653 100644 --- a/src/query/service/src/pipelines/pipeline_builder.rs +++ b/src/query/service/src/pipelines/pipeline_builder.rs @@ -463,7 +463,7 @@ impl PipelineBuilder { *segment_partition_num, block_builder, on_conflicts.clone(), - &segments, + segments, io_request_semaphore, )?; assert_eq!( diff --git a/src/query/service/src/schedulers/fragments/plan_fragment.rs b/src/query/service/src/schedulers/fragments/plan_fragment.rs index 1c7566c877e80..2b113024687f9 100644 --- a/src/query/service/src/schedulers/fragments/plan_fragment.rs +++ b/src/query/service/src/schedulers/fragments/plan_fragment.rs @@ -215,7 +215,7 @@ impl PlanFragment { ) -> Result>> { let num_parts = partitions.len(); let num_executors = executors.len(); - let mut executors_sorted = executors.clone(); + let mut executors_sorted = executors; executors_sorted.sort(); let mut executor_part = HashMap::default(); // the first num_parts % num_executors get parts_per_node parts @@ -250,21 +250,15 @@ impl PlanFragment { let partitions = &plan.segments; let executors = Fragmenter::get_executors(ctx); let mut fragment_actions = QueryFragmentActions::create(self.fragment_id); - // 把partitions等分到executors上 - let executor_num = executors.len(); - let mut partition_reshuffle = vec![]; - for (i, part) in partitions.iter().enumerate() { - let executor = executors[i % executor_num].clone(); - partition_reshuffle.push((executor, vec![part.clone()])); - } + let partition_reshuffle = Self::reshuffle_segments(executors, partitions.clone())?; for (executor, parts) in partition_reshuffle.iter() { let mut plan = self.plan.clone(); - let mut replace_replace_partial = ReplaceReplaceInto { + let mut replace_replace_into = ReplaceReplaceInto { partitions: parts.clone(), }; - plan = replace_replace_partial.replace(&plan)?; + plan = replace_replace_into.replace(&plan)?; fragment_actions .add_action(QueryFragmentAction::create(executor.clone(), plan.clone())); @@ -366,10 +360,11 @@ struct ReplaceReplaceInto { impl PhysicalPlanReplacer for ReplaceReplaceInto { fn replace_replace_into(&mut self, plan: &ReplaceInto) -> Result { - self.replace(&plan.input); - Ok(PhysicalPlan::ReplaceInto(Box::new(ReplaceInto { + let input = self.replace(&plan.input)?; + Ok(PhysicalPlan::ReplaceInto(ReplaceInto { + input: Box::new(input), segments: self.partitions.clone(), ..plan.clone() - }))) + })) } } From 866d47515d12631986a4e9e25791ee10b68dcfa3 Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Fri, 4 Aug 2023 12:02:34 +0800 Subject: [PATCH 39/40] move segment_partition_num --- .../service/src/interpreters/interpreter_replace.rs | 4 ---- src/query/service/src/pipelines/pipeline_builder.rs | 13 +++++++------ src/query/sql/src/executor/physical_plan.rs | 1 - 3 files changed, 7 insertions(+), 11 deletions(-) diff --git a/src/query/service/src/interpreters/interpreter_replace.rs b/src/query/service/src/interpreters/interpreter_replace.rs index abb1174a17c5d..634bb3fee74b1 100644 --- a/src/query/service/src/interpreters/interpreter_replace.rs +++ b/src/query/service/src/interpreters/interpreter_replace.rs @@ -223,9 +223,6 @@ impl ReplaceInterpreter { let is_distributed = is_multi_node && !is_value_source; let table_is_empty = base_snapshot.segments.is_empty(); let table_level_range_index = base_snapshot.summary.col_stats.clone(); - let max_threads = self.ctx.get_settings().get_max_threads()?; - let segment_partition_num = - std::cmp::min(base_snapshot.segments.len(), max_threads as usize); let mut purge_info = None; let (mut root, select_ctx) = self .connect_input_source( @@ -259,7 +256,6 @@ impl ReplaceInterpreter { })); root = Box::new(PhysicalPlan::ReplaceInto(ReplaceInto { input: root, - segment_partition_num, block_thresholds: fuse_table.get_block_thresholds(), 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 fb68f0d45e653..0f92f0d86dfb8 100644 --- a/src/query/service/src/pipelines/pipeline_builder.rs +++ b/src/query/service/src/pipelines/pipeline_builder.rs @@ -364,13 +364,14 @@ impl PipelineBuilder { fn build_replace_into(&mut self, replace: &ReplaceInto) -> Result<()> { let ReplaceInto { input, - segment_partition_num, block_thresholds, table_info, on_conflicts, catalog_info, segments, } = replace; + let max_threads = self.ctx.get_settings().get_max_threads()?; + let segment_partition_num = std::cmp::min(segments.len(), max_threads as usize); let table = self .ctx .build_table_by_table_info(catalog_info, table_info, None)?; @@ -394,7 +395,7 @@ impl PipelineBuilder { table, *block_thresholds, ); - if *segment_partition_num == 0 { + if segment_partition_num == 0 { let dummy_item = create_dummy_item(); // ┌──────────────────────┐ ┌──────────────────┐ // │ ├──┬────────►│ SerializeBlock │ @@ -416,7 +417,7 @@ impl PipelineBuilder { // └─────────────┘ │ ├──┐ ┌──────────────────┐ // │ ├──┴────────►│BroadcastProcessor│ // └──────────────────────┘ └──────────────────┘ - let broadcast_processor = BroadcastProcessor::new(*segment_partition_num); + let broadcast_processor = BroadcastProcessor::new(segment_partition_num); // wrap them into pipeline, order matters! self.main_pipeline .add_pipe(Pipe::create(2, segment_partition_num + 1, vec![ @@ -426,7 +427,7 @@ impl PipelineBuilder { }; // 4. connect with MergeIntoOperationAggregators - if *segment_partition_num == 0 { + if segment_partition_num == 0 { let dummy_item = create_dummy_item(); self.main_pipeline.add_pipe(Pipe::create(2, 2, vec![ serialize_segment_transform.into_pipe_item(), @@ -460,14 +461,14 @@ impl PipelineBuilder { // setup the merge into operation aggregators let mut merge_into_operation_aggregators = table.merge_into_mutators( self.ctx.clone(), - *segment_partition_num, + segment_partition_num, block_builder, on_conflicts.clone(), segments, io_request_semaphore, )?; assert_eq!( - *segment_partition_num, + segment_partition_num, merge_into_operation_aggregators.len() ); pipe_items.append(&mut merge_into_operation_aggregators); diff --git a/src/query/sql/src/executor/physical_plan.rs b/src/query/sql/src/executor/physical_plan.rs index 55404b8aa00e1..0058b374db6a5 100644 --- a/src/query/sql/src/executor/physical_plan.rs +++ b/src/query/sql/src/executor/physical_plan.rs @@ -940,7 +940,6 @@ pub struct OnConflictField { #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct ReplaceInto { pub input: Box, - pub segment_partition_num: usize, pub block_thresholds: BlockThresholds, pub table_info: TableInfo, pub on_conflicts: Vec, From 691ef8c58753ceca8512ac973cfbd817840497de Mon Sep 17 00:00:00 2001 From: dantengsky Date: Fri, 4 Aug 2023 12:30:20 +0800 Subject: [PATCH 40/40] resolve conflicts --- src/query/service/src/interpreters/interpreter_replace.rs | 8 ++++++++ src/query/service/src/pipelines/pipeline_builder.rs | 4 ++++ src/query/sql/src/executor/physical_plan.rs | 2 ++ src/query/storages/fuse/src/operations/commit.rs | 1 - src/query/storages/fuse/src/operations/replace.rs | 2 ++ 5 files changed, 16 insertions(+), 1 deletion(-) diff --git a/src/query/service/src/interpreters/interpreter_replace.rs b/src/query/service/src/interpreters/interpreter_replace.rs index a5ca93c32e9c5..f1fa28ed6bc74 100644 --- a/src/query/service/src/interpreters/interpreter_replace.rs +++ b/src/query/service/src/interpreters/interpreter_replace.rs @@ -242,9 +242,15 @@ impl ReplaceInterpreter { keys: vec![], })); } + + let bloom_filter_column_index = fuse_table + .choose_most_significant_bloom_filter_column(&on_conflicts) + .await?; + root = Box::new(PhysicalPlan::Deduplicate(Deduplicate { input: root, on_conflicts: on_conflicts.clone(), + bloom_filter_column_index, table_is_empty, table_info: table_info.clone(), catalog_info: catalog.info(), @@ -258,6 +264,7 @@ impl ReplaceInterpreter { table_info: table_info.clone(), catalog_info: catalog.info(), on_conflicts, + bloom_filter_column_index, segments: base_snapshot.segments.clone(), })); if is_distributed { @@ -279,6 +286,7 @@ impl ReplaceInterpreter { ))); Ok((root, purge_info)) } + fn check_on_conflicts(&self) -> Result<()> { if self.plan.on_conflict_fields.is_empty() { Err(ErrorCode::BadArguments( diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs index 0f92f0d86dfb8..0d887d633ba6d 100644 --- a/src/query/service/src/pipelines/pipeline_builder.rs +++ b/src/query/service/src/pipelines/pipeline_builder.rs @@ -277,6 +277,7 @@ impl PipelineBuilder { let Deduplicate { input, on_conflicts, + bloom_filter_column_index, table_is_empty, table_info, catalog_info, @@ -352,6 +353,7 @@ impl PipelineBuilder { self.ctx.as_ref(), on_conflicts.clone(), cluster_keys, + *bloom_filter_column_index, table_schema.as_ref(), *table_is_empty, table_level_range_index.clone(), @@ -367,6 +369,7 @@ impl PipelineBuilder { block_thresholds, table_info, on_conflicts, + bloom_filter_column_index, catalog_info, segments, } = replace; @@ -464,6 +467,7 @@ impl PipelineBuilder { segment_partition_num, block_builder, on_conflicts.clone(), + *bloom_filter_column_index, segments, io_request_semaphore, )?; diff --git a/src/query/sql/src/executor/physical_plan.rs b/src/query/sql/src/executor/physical_plan.rs index 0058b374db6a5..1164acac20bbd 100644 --- a/src/query/sql/src/executor/physical_plan.rs +++ b/src/query/sql/src/executor/physical_plan.rs @@ -917,6 +917,7 @@ pub struct AsyncSourcerPlan { pub struct Deduplicate { pub input: Box, pub on_conflicts: Vec, + pub bloom_filter_column_index: Option, pub table_is_empty: bool, pub table_info: TableInfo, pub catalog_info: CatalogInfo, @@ -943,6 +944,7 @@ pub struct ReplaceInto { pub block_thresholds: BlockThresholds, pub table_info: TableInfo, pub on_conflicts: Vec, + pub bloom_filter_column_index: Option, pub catalog_info: CatalogInfo, pub segments: Vec, } diff --git a/src/query/storages/fuse/src/operations/commit.rs b/src/query/storages/fuse/src/operations/commit.rs index 0a460236ae39e..0939a8c5f2279 100644 --- a/src/query/storages/fuse/src/operations/commit.rs +++ b/src/query/storages/fuse/src/operations/commit.rs @@ -62,7 +62,6 @@ use crate::operations::common::AbortOperation; use crate::operations::common::AppendGenerator; use crate::operations::common::CommitSink; use crate::operations::common::ConflictResolveContext; -use crate::operations::common::MutationKind; use crate::operations::common::TableMutationAggregator; use crate::operations::common::TransformSerializeSegment; use crate::statistics::merge_statistics; diff --git a/src/query/storages/fuse/src/operations/replace.rs b/src/query/storages/fuse/src/operations/replace.rs index dd612354ebf0a..2185ca56eb434 100644 --- a/src/query/storages/fuse/src/operations/replace.rs +++ b/src/query/storages/fuse/src/operations/replace.rs @@ -15,6 +15,7 @@ use std::sync::Arc; use common_base::base::tokio::sync::Semaphore; +use common_catalog::table::Table; use common_catalog::table_context::TableContext; use common_exception::Result; use common_expression::FieldIndex; @@ -92,6 +93,7 @@ impl FuseTable { // └─────►│ResizeProcessor(1) ├──────►│TableMutationAggregator├────────►│ CommitSink │ // └───────────────────┘ └───────────────────────┘ └───────────────────┘ + #[allow(clippy::too_many_arguments)] pub fn merge_into_mutators( &self, ctx: Arc,