diff --git a/Cargo.lock b/Cargo.lock index 8f190d2e7627..f6b73a79484b 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4029,6 +4029,7 @@ dependencies = [ "databend-common-settings", "databend-common-storage", "databend-common-storages-result-cache", + "databend-common-storages-stage", "databend-common-storages-view", "databend-common-users", "databend-enterprise-data-mask-feature", diff --git a/src/query/catalog/src/table.rs b/src/query/catalog/src/table.rs index 58b7f281f1c3..2251af702864 100644 --- a/src/query/catalog/src/table.rs +++ b/src/query/catalog/src/table.rs @@ -37,7 +37,6 @@ use databend_common_meta_types::MetaId; use databend_common_pipeline_core::Pipeline; use databend_common_storage::Histogram; use databend_common_storage::StorageMetrics; -use databend_storages_common_table_meta::meta::SnapshotId; use databend_storages_common_table_meta::meta::TableSnapshot; use databend_storages_common_table_meta::table::ChangeType; use databend_storages_common_table_meta::table::OPT_KEY_TEMP_PREFIX; @@ -240,7 +239,7 @@ pub trait Table: Sync + Send { copied_files: Option, update_stream_meta: Vec, overwrite: bool, - prev_snapshot_id: Option, + forbid_occ_retry: bool, _deduplicated_label: Option, ) -> Result<()> { let (_, _, _, _, _, _) = ( @@ -249,7 +248,7 @@ pub trait Table: Sync + Send { update_stream_meta, pipeline, overwrite, - prev_snapshot_id, + forbid_occ_retry, ); Ok(()) diff --git a/src/query/service/src/interpreters/access/privilege_access.rs b/src/query/service/src/interpreters/access/privilege_access.rs index c912c5fcc785..90a7032dba64 100644 --- a/src/query/service/src/interpreters/access/privilege_access.rs +++ b/src/query/service/src/interpreters/access/privilege_access.rs @@ -34,12 +34,14 @@ use databend_common_meta_app::tenant::Tenant; use databend_common_meta_types::seq_value::SeqV; use databend_common_sql::binder::MutationType; use databend_common_sql::optimizer::get_udf_names; +use databend_common_sql::plans::Append; use databend_common_sql::plans::InsertInputSource; use databend_common_sql::plans::Mutation; use databend_common_sql::plans::OptimizeCompactBlock; use databend_common_sql::plans::PresignAction; use databend_common_sql::plans::Recluster; use databend_common_sql::plans::RewriteKind; +use databend_common_sql::BindContext; use databend_common_sql::Planner; use databend_common_users::RoleCacheManager; use databend_common_users::UserApiProvider; @@ -1002,17 +1004,18 @@ impl AccessChecker for PrivilegeAccess { self.validate_access(&GrantObject::Global, UserPrivilegeType::Super, false, false) .await?; } - // Others. - Plan::Insert(plan) => { - let target_table_privileges = if plan.overwrite { + Plan::Append { s_expr, target_table_index,metadata,overwrite,.. } => { + let target_table_privileges = if *overwrite { vec![UserPrivilegeType::Insert, UserPrivilegeType::Delete] } else { vec![UserPrivilegeType::Insert] }; + let (_, catalog, database, table) = + Append::target_table(metadata, *target_table_index); for privilege in target_table_privileges { - self.validate_table_access(&plan.catalog, &plan.database, &plan.table, privilege, false, false).await?; + self.validate_table_access(&catalog, &database, &table, privilege, false, false).await?; } - self.validate_insert_source(ctx, &plan.source).await?; + self.check(ctx, &Plan::Query { s_expr:s_expr.clone(), metadata: metadata.clone(), bind_context: Box::new(BindContext::new()), rewrite_kind: None, formatted_ast: None, ignore_result: false }).await?; } Plan::InsertMultiTable(plan) => { let target_table_privileges = if plan.overwrite { @@ -1164,13 +1167,6 @@ impl AccessChecker for PrivilegeAccess { self.validate_access(&GrantObject::Global, UserPrivilegeType::Alter, false, false) .await?; } - Plan::CopyIntoTable(plan) => { - self.validate_stage_access(&plan.stage_table_info.stage_info, UserPrivilegeType::Read).await?; - self.validate_table_access(plan.catalog_info.catalog_name(), &plan.database_name, &plan.table_name, UserPrivilegeType::Insert, false, false).await?; - if let Some(query) = &plan.query { - self.check(ctx, query).await?; - } - } Plan::CopyIntoLocation(plan) => { self.validate_stage_access(&plan.stage, UserPrivilegeType::Write).await?; let from = plan.from.clone(); diff --git a/src/query/service/src/interpreters/interpreter_append.rs b/src/query/service/src/interpreters/interpreter_append.rs new file mode 100644 index 000000000000..daac037dd749 --- /dev/null +++ b/src/query/service/src/interpreters/interpreter_append.rs @@ -0,0 +1,232 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::sync::Arc; + +use databend_common_catalog::lock::LockTableOption; +use databend_common_catalog::plan::StageTableInfo; +use databend_common_catalog::table::TableExt; +use databend_common_exception::Result; +use databend_common_expression::types::Int32Type; +use databend_common_expression::types::StringType; +use databend_common_expression::DataBlock; +use databend_common_expression::FromData; +use databend_common_expression::SendableDataBlockStream; +use databend_common_sql::executor::physical_plans::MutationKind; +use databend_common_sql::executor::PhysicalPlanBuilder; +use databend_common_sql::optimizer::SExpr; +use databend_common_sql::plans::Append; +use databend_common_sql::plans::AppendType; +use databend_common_sql::IndexType; +use log::info; + +use crate::interpreters::common::check_deduplicate_label; +use crate::interpreters::common::dml_build_update_stream_req; +use crate::interpreters::HookOperator; +use crate::interpreters::Interpreter; +use crate::pipelines::PipelineBuildResult; +use crate::pipelines::PipelineBuilder; +use crate::schedulers::build_query_pipeline_without_render_result_set; +use crate::sessions::QueryContext; +use crate::sessions::TableContext; +use crate::sql::MetadataRef; +use crate::stream::DataBlockStream; + +pub struct AppendInterpreter { + ctx: Arc, + s_expr: SExpr, + metadata: MetadataRef, + target_table_index: IndexType, + stage_table_info: Option>, + overwrite: bool, + forbid_occ_retry: bool, + append_type: AppendType, +} + +#[async_trait::async_trait] +impl Interpreter for AppendInterpreter { + fn name(&self) -> &str { + "AppendInterpreter" + } + + fn is_ddl(&self) -> bool { + false + } + + #[fastrace::trace] + #[async_backtrace::framed] + async fn execute2(&self) -> Result { + if check_deduplicate_label(self.ctx.clone()).await? { + return Ok(PipelineBuildResult::create()); + } + let (target_table, catalog, database, table) = + Append::target_table(&self.metadata, self.target_table_index); + target_table.check_mutable()?; + + // 1. build source and append pipeline + let mut build_res = { + let mut physical_plan_builder = + PhysicalPlanBuilder::new(self.metadata.clone(), self.ctx.clone(), false); + let physical_plan = physical_plan_builder + .build(&self.s_expr, Default::default()) + .await?; + build_query_pipeline_without_render_result_set(&self.ctx, &physical_plan).await? + }; + + // 2. build commit pipeline + let copied_files_meta_req = match &self.stage_table_info { + Some(stage_table_info) => PipelineBuilder::build_upsert_copied_files_to_meta_req( + self.ctx.clone(), + target_table.as_ref(), + stage_table_info + .files_to_copy + .as_deref() + .unwrap_or_default(), + &stage_table_info.copy_into_table_options, + )?, + None => None, + }; + let update_stream_meta = + dml_build_update_stream_req(self.ctx.clone(), &self.metadata).await?; + target_table.commit_insertion( + self.ctx.clone(), + &mut build_res.main_pipeline, + copied_files_meta_req, + update_stream_meta, + self.overwrite, + self.forbid_occ_retry, + unsafe { self.ctx.get_settings().get_deduplicate_label()? }, + )?; + + // 3. Purge files on pipeline finished. + if let Some(stage_table_info) = &self.stage_table_info { + let files_to_copy = stage_table_info + .files_to_copy + .as_deref() + .unwrap_or_default(); + info!( + "set files to be purged, # of copied files: {}, # of duplicated files: {}", + files_to_copy.len(), + stage_table_info.duplicated_files_detected.len() + ); + + let files_to_be_deleted = files_to_copy + .iter() + .map(|f| f.path.clone()) + .chain(stage_table_info.duplicated_files_detected.clone()) + .collect::>(); + PipelineBuilder::set_purge_files_on_finished( + self.ctx.clone(), + files_to_be_deleted, + &stage_table_info.copy_into_table_options, + stage_table_info.stage_info.clone(), + &mut build_res.main_pipeline, + )?; + } + + // 4. Execute hook. + { + let hook_operator = HookOperator::create( + self.ctx.clone(), + catalog, + database, + table, + MutationKind::Insert, + LockTableOption::LockNoRetry, + ); + hook_operator.execute(&mut build_res.main_pipeline).await; + } + + Ok(build_res) + } + + fn inject_result(&self) -> Result { + match &self.append_type { + AppendType::CopyInto => { + let blocks = self.get_copy_into_table_result()?; + Ok(Box::pin(DataBlockStream::create(None, blocks))) + } + AppendType::Insert => Ok(Box::pin(DataBlockStream::create(None, vec![]))), + } + } +} + +impl AppendInterpreter { + pub fn try_create( + ctx: Arc, + s_expr: SExpr, + metadata: MetadataRef, + stage_table_info: Option>, + overwrite: bool, + forbid_occ_retry: bool, + append_type: AppendType, + table_index: IndexType, + ) -> Result { + Ok(AppendInterpreter { + ctx, + s_expr, + metadata, + stage_table_info, + overwrite, + forbid_occ_retry, + append_type, + target_table_index: table_index, + }) + } + + fn get_copy_into_table_result(&self) -> Result> { + let return_all = !self + .stage_table_info + .as_ref() + .unwrap() + .copy_into_table_options + .return_failed_only; + let cs = self.ctx.get_copy_status(); + + let mut results = cs.files.iter().collect::>(); + results.sort_by(|a, b| a.key().cmp(b.key())); + + let n = cs.files.len(); + let mut files = Vec::with_capacity(n); + let mut rows_loaded = Vec::with_capacity(n); + let mut errors_seen = Vec::with_capacity(n); + let mut first_error = Vec::with_capacity(n); + let mut first_error_line = Vec::with_capacity(n); + + for entry in results { + let status = entry.value(); + if let Some(err) = &status.error { + files.push(entry.key().clone()); + rows_loaded.push(status.num_rows_loaded as i32); + errors_seen.push(err.num_errors as i32); + first_error.push(Some(err.first_error.error.to_string().clone())); + first_error_line.push(Some(err.first_error.line as i32 + 1)); + } else if return_all { + files.push(entry.key().clone()); + rows_loaded.push(status.num_rows_loaded as i32); + errors_seen.push(0); + first_error.push(None); + first_error_line.push(None); + } + } + let blocks = vec![DataBlock::new_from_columns(vec![ + StringType::from_data(files), + Int32Type::from_data(rows_loaded), + Int32Type::from_data(errors_seen), + StringType::from_opt_data(first_error), + Int32Type::from_opt_data(first_error_line), + ])]; + Ok(blocks) + } +} diff --git a/src/query/service/src/interpreters/interpreter_copy_into_table.rs b/src/query/service/src/interpreters/interpreter_copy_into_table.rs deleted file mode 100644 index 1b2ecd128ebf..000000000000 --- a/src/query/service/src/interpreters/interpreter_copy_into_table.rs +++ /dev/null @@ -1,403 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::collections::BTreeMap; -use std::sync::Arc; - -use databend_common_catalog::lock::LockTableOption; -use databend_common_exception::Result; -use databend_common_expression::types::Int32Type; -use databend_common_expression::types::StringType; -use databend_common_expression::DataBlock; -use databend_common_expression::FromData; -use databend_common_expression::SendableDataBlockStream; -use databend_common_meta_app::schema::UpdateStreamMetaReq; -use databend_common_pipeline_core::Pipeline; -use databend_common_sql::executor::physical_plans::CopyIntoTable; -use databend_common_sql::executor::physical_plans::CopyIntoTableSource; -use databend_common_sql::executor::physical_plans::Exchange; -use databend_common_sql::executor::physical_plans::FragmentKind; -use databend_common_sql::executor::physical_plans::MutationKind; -use databend_common_sql::executor::physical_plans::TableScan; -use databend_common_sql::executor::table_read_plan::ToReadDataSourcePlan; -use databend_common_sql::executor::PhysicalPlan; -use databend_common_storage::StageFileInfo; -use databend_common_storages_stage::StageTable; -use log::debug; -use log::info; - -use crate::interpreters::common::check_deduplicate_label; -use crate::interpreters::common::dml_build_update_stream_req; -use crate::interpreters::HookOperator; -use crate::interpreters::Interpreter; -use crate::interpreters::SelectInterpreter; -use crate::pipelines::PipelineBuildResult; -use crate::pipelines::PipelineBuilder; -use crate::schedulers::build_query_pipeline_without_render_result_set; -use crate::sessions::QueryContext; -use crate::sessions::TableContext; -use crate::sql::plans::CopyIntoTablePlan; -use crate::sql::plans::Plan; -use crate::stream::DataBlockStream; - -pub struct CopyIntoTableInterpreter { - ctx: Arc, - plan: CopyIntoTablePlan, -} - -impl CopyIntoTableInterpreter { - /// Create a CopyInterpreter with context and [`CopyIntoTablePlan`]. - pub fn try_create(ctx: Arc, plan: CopyIntoTablePlan) -> Result { - Ok(CopyIntoTableInterpreter { ctx, plan }) - } - - #[async_backtrace::framed] - async fn build_query( - &self, - query: &Plan, - ) -> Result<(SelectInterpreter, Vec)> { - let (s_expr, metadata, bind_context, formatted_ast) = match query { - Plan::Query { - s_expr, - metadata, - bind_context, - formatted_ast, - .. - } => (s_expr, metadata, bind_context, formatted_ast), - v => unreachable!("Input plan must be Query, but it's {}", v), - }; - - let update_stream_meta = dml_build_update_stream_req(self.ctx.clone(), metadata).await?; - - let select_interpreter = SelectInterpreter::try_create( - self.ctx.clone(), - *(bind_context.clone()), - *s_expr.clone(), - metadata.clone(), - formatted_ast.clone(), - false, - )?; - - Ok((select_interpreter, update_stream_meta)) - } - - #[async_backtrace::framed] - pub async fn build_physical_plan( - &self, - plan: &CopyIntoTablePlan, - ) -> Result<(PhysicalPlan, Vec)> { - let to_table = self - .ctx - .get_table( - plan.catalog_info.catalog_name(), - &plan.database_name, - &plan.table_name, - ) - .await?; - let mut update_stream_meta_reqs = vec![]; - let (source, project_columns) = if let Some(ref query) = plan.query { - let query = if plan.enable_distributed { - query.remove_exchange_for_select() - } else { - *query.clone() - }; - - let (query_interpreter, update_stream_meta) = self.build_query(&query).await?; - update_stream_meta_reqs = update_stream_meta; - let query_physical_plan = Box::new(query_interpreter.build_physical_plan().await?); - - let result_columns = query_interpreter.get_result_columns(); - ( - CopyIntoTableSource::Query(query_physical_plan), - Some(result_columns), - ) - } else { - let stage_table = StageTable::try_create(plan.stage_table_info.clone())?; - - let data_source_plan = stage_table - .read_plan(self.ctx.clone(), None, None, false, false) - .await?; - - let mut name_mapping = BTreeMap::new(); - for (idx, field) in data_source_plan.schema().fields.iter().enumerate() { - name_mapping.insert(field.name.clone(), idx); - } - - ( - CopyIntoTableSource::Stage(Box::new(PhysicalPlan::TableScan(TableScan { - plan_id: 0, - scan_id: 0, - name_mapping, - stat_info: None, - table_index: None, - internal_column: None, - source: Box::new(data_source_plan), - }))), - None, - ) - }; - - let mut root = PhysicalPlan::CopyIntoTable(Box::new(CopyIntoTable { - plan_id: 0, - 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(), - write_mode: plan.write_mode, - validation_mode: plan.validation_mode.clone(), - project_columns, - source, - is_transform: plan.is_transform, - })); - - if plan.enable_distributed { - root = PhysicalPlan::Exchange(Exchange { - plan_id: 0, - input: Box::new(root), - kind: FragmentKind::Merge, - keys: Vec::new(), - allow_adjust_parallelism: true, - ignore_exchange: false, - }); - } - - let mut next_plan_id = 0; - root.adjust_plan_id(&mut next_plan_id); - - Ok((root, update_stream_meta_reqs)) - } - - fn get_copy_into_table_result(&self) -> Result> { - let return_all = !self - .plan - .stage_table_info - .copy_into_table_options - .return_failed_only; - let cs = self.ctx.get_copy_status(); - - let mut results = cs.files.iter().collect::>(); - results.sort_by(|a, b| a.key().cmp(b.key())); - - let n = cs.files.len(); - let mut files = Vec::with_capacity(n); - let mut rows_loaded = Vec::with_capacity(n); - let mut errors_seen = Vec::with_capacity(n); - let mut first_error = Vec::with_capacity(n); - let mut first_error_line = Vec::with_capacity(n); - - for entry in results { - let status = entry.value(); - if let Some(err) = &status.error { - files.push(entry.key().clone()); - rows_loaded.push(status.num_rows_loaded as i32); - errors_seen.push(err.num_errors as i32); - first_error.push(Some(err.first_error.error.to_string().clone())); - first_error_line.push(Some(err.first_error.line as i32 + 1)); - } else if return_all { - files.push(entry.key().clone()); - rows_loaded.push(status.num_rows_loaded as i32); - errors_seen.push(0); - first_error.push(None); - first_error_line.push(None); - } - } - let blocks = vec![DataBlock::new_from_columns(vec![ - StringType::from_data(files), - Int32Type::from_data(rows_loaded), - Int32Type::from_data(errors_seen), - StringType::from_opt_data(first_error), - Int32Type::from_opt_data(first_error_line), - ])]; - Ok(blocks) - } - - /// Build commit insertion pipeline. - async fn commit_insertion( - &self, - main_pipeline: &mut Pipeline, - plan: &CopyIntoTablePlan, - files_to_copy: Vec, - duplicated_files_detected: Vec, - update_stream_meta: Vec, - deduplicated_label: Option, - ) -> Result<()> { - let ctx = self.ctx.clone(); - let to_table = ctx - .get_table( - plan.catalog_info.catalog_name(), - &plan.database_name, - &plan.table_name, - ) - .await?; - - // Commit. - { - let copied_files_meta_req = PipelineBuilder::build_upsert_copied_files_to_meta_req( - ctx.clone(), - to_table.as_ref(), - &files_to_copy, - &plan.stage_table_info.copy_into_table_options, - )?; - - to_table.commit_insertion( - ctx.clone(), - main_pipeline, - copied_files_meta_req, - update_stream_meta, - plan.write_mode.is_overwrite(), - None, - deduplicated_label, - )?; - } - - // Purge files. - { - info!( - "set files to be purged, # of copied files: {}, # of duplicated files: {}", - files_to_copy.len(), - duplicated_files_detected.len() - ); - - let files_to_be_deleted = files_to_copy - .into_iter() - .map(|v| v.path) - .chain(duplicated_files_detected) - .collect::>(); - // set on_finished callback. - PipelineBuilder::set_purge_files_on_finished( - ctx.clone(), - files_to_be_deleted, - &plan.stage_table_info.copy_into_table_options, - plan.stage_table_info.stage_info.clone(), - main_pipeline, - )?; - } - Ok(()) - } - - async fn on_no_files_to_copy(&self) -> Result { - // currently, there is only one thing that we care about: - // - // if `purge_duplicated_files_in_copy` and `purge` are all enabled, - // and there are duplicated files detected, we should clean them up immediately. - - // it might be better to reuse the PipelineBuilder::set_purge_files_on_finished, - // unfortunately, hooking the on_finished callback of a "blank" pipeline, - // e.g. `PipelineBuildResult::create` leads to runtime error (during pipeline execution). - - if self.plan.stage_table_info.copy_into_table_options.purge - && !self - .plan - .stage_table_info - .duplicated_files_detected - .is_empty() - && self - .ctx - .get_settings() - .get_enable_purge_duplicated_files_in_copy()? - { - info!( - "purge_duplicated_files_in_copy enabled, number of duplicated files: {}", - self.plan.stage_table_info.duplicated_files_detected.len() - ); - - PipelineBuilder::purge_files_immediately( - self.ctx.clone(), - self.plan.stage_table_info.duplicated_files_detected.clone(), - self.plan.stage_table_info.stage_info.clone(), - ) - .await?; - } - Ok(PipelineBuildResult::create()) - } -} - -#[async_trait::async_trait] -impl Interpreter for CopyIntoTableInterpreter { - fn name(&self) -> &str { - "CopyIntoTableInterpreterV2" - } - - fn is_ddl(&self) -> bool { - false - } - - #[fastrace::trace] - #[async_backtrace::framed] - async fn execute2(&self) -> Result { - debug!("ctx.id" = self.ctx.get_id().as_str(); "copy_into_table_interpreter_execute_v2"); - - if check_deduplicate_label(self.ctx.clone()).await? { - return Ok(PipelineBuildResult::create()); - } - - if self.plan.no_file_to_copy { - info!("no file to copy"); - return self.on_no_files_to_copy().await; - } - - let (physical_plan, update_stream_meta) = self.build_physical_plan(&self.plan).await?; - let mut build_res = - build_query_pipeline_without_render_result_set(&self.ctx, &physical_plan).await?; - - // Build commit insertion pipeline. - { - let files_to_copy = self - .plan - .stage_table_info - .files_to_copy - .clone() - .unwrap_or_default(); - - let duplicated_files_detected = - self.plan.stage_table_info.duplicated_files_detected.clone(); - - self.commit_insertion( - &mut build_res.main_pipeline, - &self.plan, - files_to_copy, - duplicated_files_detected, - update_stream_meta, - unsafe { self.ctx.get_settings().get_deduplicate_label()? }, - ) - .await?; - } - - // Execute hook. - { - let hook_operator = HookOperator::create( - self.ctx.clone(), - self.plan.catalog_info.catalog_name().to_string(), - self.plan.database_name.to_string(), - self.plan.table_name.to_string(), - MutationKind::Insert, - LockTableOption::LockNoRetry, - ); - hook_operator.execute(&mut build_res.main_pipeline).await; - } - - Ok(build_res) - } - - fn inject_result(&self) -> Result { - let blocks = if self.plan.no_file_to_copy { - vec![DataBlock::empty_with_schema(self.plan.schema())] - } else { - self.get_copy_into_table_result()? - }; - - Ok(Box::pin(DataBlockStream::create(None, blocks))) - } -} diff --git a/src/query/service/src/interpreters/interpreter_explain.rs b/src/query/service/src/interpreters/interpreter_explain.rs index 9191a3a10373..59e83833f895 100644 --- a/src/query/service/src/interpreters/interpreter_explain.rs +++ b/src/query/service/src/interpreters/interpreter_explain.rs @@ -105,7 +105,6 @@ impl Interpreter for ExplainInterpreter { self.explain_query(s_expr, metadata, bind_context, formatted_ast) .await? } - Plan::Insert(insert_plan) => insert_plan.explain(self.config.verbose).await?, Plan::Replace(replace_plan) => replace_plan.explain(self.config.verbose).await?, Plan::CreateTable(plan) => match &plan.as_select { Some(box Plan::Query { @@ -151,6 +150,17 @@ impl Interpreter for ExplainInterpreter { let plan = interpreter.build_physical_plan(&mutation, None).await?; self.explain_physical_plan(&plan, metadata, &None).await? } + Plan::Append { + s_expr, metadata, .. + } => { + let mut physical_plan_builder = + PhysicalPlanBuilder::new(metadata.clone(), self.ctx.clone(), false); + let physical_plan = physical_plan_builder + .build(s_expr, Default::default()) + .await?; + self.explain_physical_plan(&physical_plan, metadata, &None) + .await? + } _ => self.explain_plan(&self.plan)?, }, diff --git a/src/query/service/src/interpreters/interpreter_factory.rs b/src/query/service/src/interpreters/interpreter_factory.rs index 4520200e047a..47abd2a2bcbb 100644 --- a/src/query/service/src/interpreters/interpreter_factory.rs +++ b/src/query/service/src/interpreters/interpreter_factory.rs @@ -36,13 +36,13 @@ use super::interpreter_table_set_options::SetOptionsInterpreter; use super::interpreter_user_stage_drop::DropUserStageInterpreter; use super::*; use crate::interpreters::access::Accessor; +use crate::interpreters::interpreter_append::AppendInterpreter; use crate::interpreters::interpreter_catalog_drop::DropCatalogInterpreter; use crate::interpreters::interpreter_connection_create::CreateConnectionInterpreter; use crate::interpreters::interpreter_connection_desc::DescConnectionInterpreter; use crate::interpreters::interpreter_connection_drop::DropConnectionInterpreter; use crate::interpreters::interpreter_connection_show::ShowConnectionsInterpreter; use crate::interpreters::interpreter_copy_into_location::CopyIntoLocationInterpreter; -use crate::interpreters::interpreter_copy_into_table::CopyIntoTableInterpreter; use crate::interpreters::interpreter_file_format_create::CreateFileFormatInterpreter; use crate::interpreters::interpreter_file_format_drop::DropFileFormatInterpreter; use crate::interpreters::interpreter_file_format_show::ShowFileFormatsInterpreter; @@ -156,9 +156,23 @@ impl InterpreterFactory { *graphical, )?)), - Plan::CopyIntoTable(copy_plan) => Ok(Arc::new(CopyIntoTableInterpreter::try_create( + Plan::Append { + s_expr, + metadata, + stage_table_info, + overwrite, + forbid_occ_retry, + append_type, + target_table_index, + } => Ok(Arc::new(AppendInterpreter::try_create( ctx, - *copy_plan.clone(), + *s_expr.clone(), + metadata.clone(), + stage_table_info.clone(), + *overwrite, + *forbid_occ_retry, + append_type.clone(), + *target_table_index, )?)), Plan::CopyIntoLocation(copy_plan) => Ok(Arc::new( CopyIntoLocationInterpreter::try_create(ctx, copy_plan.clone())?, @@ -381,8 +395,6 @@ impl InterpreterFactory { *desc_user.clone(), )?)), - Plan::Insert(insert) => InsertInterpreter::try_create(ctx, *insert.clone()), - Plan::Replace(replace) => ReplaceInterpreter::try_create(ctx, *replace.clone()), Plan::DataMutation { s_expr, schema, .. } => Ok(Arc::new( MutationInterpreter::try_create(ctx, *s_expr.clone(), schema.clone())?, diff --git a/src/query/service/src/interpreters/interpreter_insert.rs b/src/query/service/src/interpreters/interpreter_insert.rs deleted file mode 100644 index f9d951033bd9..000000000000 --- a/src/query/service/src/interpreters/interpreter_insert.rs +++ /dev/null @@ -1,262 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::sync::Arc; - -use databend_common_catalog::lock::LockTableOption; -use databend_common_catalog::table::TableExt; -use databend_common_exception::ErrorCode; -use databend_common_exception::Result; -use databend_common_expression::DataSchema; -use databend_common_pipeline_sources::AsyncSourcer; -use databend_common_sql::executor::physical_plans::DistributedInsertSelect; -use databend_common_sql::executor::physical_plans::MutationKind; -use databend_common_sql::executor::PhysicalPlan; -use databend_common_sql::executor::PhysicalPlanBuilder; -use databend_common_sql::plans::Insert; -use databend_common_sql::plans::InsertInputSource; -use databend_common_sql::plans::InsertValue; -use databend_common_sql::plans::Plan; -use databend_common_sql::NameResolutionContext; -use log::info; - -use crate::interpreters::common::check_deduplicate_label; -use crate::interpreters::common::dml_build_update_stream_req; -use crate::interpreters::HookOperator; -use crate::interpreters::Interpreter; -use crate::interpreters::InterpreterPtr; -use crate::pipelines::PipelineBuildResult; -use crate::pipelines::PipelineBuilder; -use crate::pipelines::RawValueSource; -use crate::pipelines::ValueSource; -use crate::schedulers::build_query_pipeline_without_render_result_set; -use crate::sessions::QueryContext; -use crate::sessions::TableContext; - -pub struct InsertInterpreter { - ctx: Arc, - plan: Insert, -} - -impl InsertInterpreter { - pub fn try_create(ctx: Arc, plan: Insert) -> Result { - Ok(Arc::new(InsertInterpreter { ctx, plan })) - } - - 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(format!( - "Fields in select statement is not equal with expected, select fields: {}, insert fields: {}", - select_schema.fields().len(), - output_schema.fields().len(), - ))); - } - - // check if cast needed - let cast_needed = select_schema.as_ref() != &DataSchema::from(output_schema.as_ref()); - Ok(cast_needed) - } -} - -#[async_trait::async_trait] -impl Interpreter for InsertInterpreter { - fn name(&self) -> &str { - "InsertIntoInterpreter" - } - - fn is_ddl(&self) -> bool { - false - } - - #[async_backtrace::framed] - async fn execute2(&self) -> Result { - if check_deduplicate_label(self.ctx.clone()).await? { - return Ok(PipelineBuildResult::create()); - } - let table = if let Some(table_info) = &self.plan.table_info { - // if table_info is provided, we should instantiated table with it. - self.ctx - .get_catalog(&self.plan.catalog) - .await? - .get_table_by_info(table_info)? - } else { - self.ctx - .get_table(&self.plan.catalog, &self.plan.database, &self.plan.table) - .await? - }; - - // check mutability - table.check_mutable()?; - - let mut build_res = PipelineBuildResult::create(); - - match &self.plan.source { - InsertInputSource::Stage(_) => { - unreachable!() - } - InsertInputSource::Values(InsertValue::Values { rows }) => { - build_res.main_pipeline.add_source( - |output| { - let inner = ValueSource::new(rows.clone(), self.plan.dest_schema()); - AsyncSourcer::create(self.ctx.clone(), output, inner) - }, - 1, - )?; - } - InsertInputSource::Values(InsertValue::RawValues { data, start }) => { - build_res.main_pipeline.add_source( - |output| { - let name_resolution_ctx = NameResolutionContext { - deny_column_reference: true, - ..Default::default() - }; - let inner = RawValueSource::new( - data.to_string(), - self.ctx.clone(), - name_resolution_ctx, - self.plan.dest_schema(), - *start, - ); - AsyncSourcer::create(self.ctx.clone(), output, inner) - }, - 1, - )?; - } - InsertInputSource::SelectPlan(plan) => { - let table1 = table.clone(); - let (select_plan, select_column_bindings, metadata) = match plan.as_ref() { - Plan::Query { - s_expr, - metadata, - bind_context, - .. - } => { - let mut builder1 = - PhysicalPlanBuilder::new(metadata.clone(), self.ctx.clone(), false); - ( - builder1.build(s_expr, bind_context.column_set()).await?, - bind_context.columns.clone(), - metadata, - ) - } - _ => unreachable!(), - }; - - let explain_plan = select_plan - .format(metadata.clone(), Default::default())? - .format_pretty()?; - info!("Insert select plan: \n{}", explain_plan); - - let update_stream_meta = - dml_build_update_stream_req(self.ctx.clone(), metadata).await?; - - // here we remove the last exchange merge plan to trigger distribute insert - let insert_select_plan = match (select_plan, table.support_distributed_insert()) { - (PhysicalPlan::Exchange(ref mut exchange), true) => { - // insert can be dispatched to different nodes if table support_distributed_insert - let input = exchange.input.clone(); - - exchange.input = Box::new(PhysicalPlan::DistributedInsertSelect(Box::new( - DistributedInsertSelect { - // TODO(leiysky): we reuse the id of exchange here, - // which is not correct. We should generate a new id for insert. - plan_id: exchange.plan_id, - input, - table_info: table1.get_table_info().clone(), - select_schema: plan.schema(), - select_column_bindings, - insert_schema: self.plan.dest_schema(), - cast_needed: self.check_schema_cast(plan)?, - }, - ))); - PhysicalPlan::Exchange(exchange.clone()) - } - (other_plan, _) => { - // insert should wait until all nodes finished - PhysicalPlan::DistributedInsertSelect(Box::new(DistributedInsertSelect { - // TODO: we reuse the id of other plan here, - // which is not correct. We should generate a new id for insert. - plan_id: other_plan.get_id(), - input: Box::new(other_plan), - table_info: table1.get_table_info().clone(), - select_schema: plan.schema(), - select_column_bindings, - insert_schema: self.plan.dest_schema(), - cast_needed: self.check_schema_cast(plan)?, - })) - } - }; - - let mut build_res = - build_query_pipeline_without_render_result_set(&self.ctx, &insert_select_plan) - .await?; - - table.commit_insertion( - self.ctx.clone(), - &mut build_res.main_pipeline, - None, - update_stream_meta, - self.plan.overwrite, - None, - unsafe { self.ctx.get_settings().get_deduplicate_label()? }, - )?; - - // Execute the hook operator. - { - let hook_operator = HookOperator::create( - self.ctx.clone(), - self.plan.catalog.clone(), - self.plan.database.clone(), - self.plan.table.clone(), - MutationKind::Insert, - LockTableOption::LockNoRetry, - ); - hook_operator.execute(&mut build_res.main_pipeline).await; - } - - return Ok(build_res); - } - }; - - PipelineBuilder::build_append2table_with_commit_pipeline( - self.ctx.clone(), - &mut build_res.main_pipeline, - table.clone(), - self.plan.dest_schema(), - None, - vec![], - self.plan.overwrite, - unsafe { self.ctx.get_settings().get_deduplicate_label()? }, - )?; - - // Execute the hook operator. - { - let hook_operator = HookOperator::create( - self.ctx.clone(), - self.plan.catalog.clone(), - self.plan.database.clone(), - self.plan.table.clone(), - MutationKind::Insert, - LockTableOption::LockNoRetry, - ); - hook_operator.execute(&mut build_res.main_pipeline).await; - } - - Ok(build_res) - } -} diff --git a/src/query/service/src/interpreters/interpreter_replace.rs b/src/query/service/src/interpreters/interpreter_replace.rs index 6018939ae07a..34db039b5af0 100644 --- a/src/query/service/src/interpreters/interpreter_replace.rs +++ b/src/query/service/src/interpreters/interpreter_replace.rs @@ -30,10 +30,11 @@ use databend_common_sql::executor::physical_plans::Exchange; use databend_common_sql::executor::physical_plans::FragmentKind; use databend_common_sql::executor::physical_plans::MutationKind; use databend_common_sql::executor::physical_plans::OnConflictField; -use databend_common_sql::executor::physical_plans::ReplaceAsyncSourcer; +use databend_common_sql::executor::physical_plans::PhysicalValueScan; use databend_common_sql::executor::physical_plans::ReplaceDeduplicate; use databend_common_sql::executor::physical_plans::ReplaceInto; use databend_common_sql::executor::physical_plans::ReplaceSelectCtx; +use databend_common_sql::executor::physical_plans::Values; use databend_common_sql::executor::PhysicalPlan; use databend_common_sql::plans::InsertInputSource; use databend_common_sql::plans::InsertValue; @@ -51,7 +52,6 @@ use parking_lot::RwLock; use crate::interpreters::common::check_deduplicate_label; use crate::interpreters::common::dml_build_update_stream_req; -use crate::interpreters::interpreter_copy_into_table::CopyIntoTableInterpreter; use crate::interpreters::HookOperator; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; @@ -377,7 +377,7 @@ impl ReplaceInterpreter { ctx: Arc, source: &'a InsertInputSource, schema: DataSchemaRef, - purge_info: &mut Option<(Vec, StageInfo, CopyIntoTableOptions)>, + _purge_info: &mut Option<(Vec, StageInfo, CopyIntoTableOptions)>, ) -> Result { match source { InsertInputSource::Values(source) => self @@ -392,28 +392,9 @@ impl ReplaceInterpreter { InsertInputSource::SelectPlan(plan) => { self.connect_query_plan_source(ctx.clone(), plan).await } - InsertInputSource::Stage(plan) => match *plan.clone() { - Plan::CopyIntoTable(copy_plan) => { - let interpreter = - CopyIntoTableInterpreter::try_create(ctx.clone(), *copy_plan.clone())?; - let (physical_plan, _) = interpreter.build_physical_plan(©_plan).await?; - - // TODO optimization: if copy_plan.stage_table_info.files_to_copy is None, there should be a short-cut plan - - *purge_info = Some(( - copy_plan.stage_table_info.files_to_copy.unwrap_or_default(), - copy_plan.stage_table_info.stage_info.clone(), - copy_plan.stage_table_info.copy_into_table_options.clone(), - )); - Ok(ReplaceSourceCtx { - root: Box::new(physical_plan), - select_ctx: None, - update_stream_meta: vec![], - bind_context: None, - }) - } - _ => unreachable!("plan in InsertInputSource::Stag must be CopyIntoTable"), - }, + InsertInputSource::Stage(_) => Err(ErrorCode::StorageUnsupported( + "stage attachment is deprecated in replace into statement", + )), } } @@ -422,13 +403,20 @@ impl ReplaceInterpreter { schema: DataSchemaRef, source: &InsertValue, ) -> Result> { - Ok(Box::new(PhysicalPlan::ReplaceAsyncSourcer( - ReplaceAsyncSourcer { - schema, + let values = match source { + InsertValue::Values { rows } => Values::Values(Arc::new(rows.clone())), + InsertValue::RawValues { data, start } => Values::RawValues { + rest_str: Arc::new(data.clone()), + start: *start, + }, + }; + Ok(Box::new(PhysicalPlan::ValueScan(Box::new( + PhysicalValueScan { plan_id: u32::MAX, - source: source.clone(), + values, + output_schema: schema.clone(), }, - ))) + )))) } #[async_backtrace::framed] diff --git a/src/query/service/src/interpreters/interpreter_table_create.rs b/src/query/service/src/interpreters/interpreter_table_create.rs index f15937f892cc..14dd5e25bb21 100644 --- a/src/query/service/src/interpreters/interpreter_table_create.rs +++ b/src/query/service/src/interpreters/interpreter_table_create.rs @@ -39,6 +39,7 @@ use databend_common_meta_app::schema::TableStatistics; use databend_common_meta_types::MatchSeq; use databend_common_pipeline_core::ExecutionInfo; use databend_common_sql::field_default_value; +use databend_common_sql::plans::create_append_plan_from_subquery; use databend_common_sql::plans::CreateTablePlan; use databend_common_storages_fuse::io::MetaReaders; use databend_common_storages_fuse::FuseStorageFormat; @@ -63,13 +64,11 @@ use crate::interpreters::common::table_option_validation::is_valid_create_opt; use crate::interpreters::common::table_option_validation::is_valid_data_retention_period; use crate::interpreters::common::table_option_validation::is_valid_random_seed; use crate::interpreters::common::table_option_validation::is_valid_row_per_block; -use crate::interpreters::InsertInterpreter; use crate::interpreters::Interpreter; +use crate::interpreters::InterpreterFactory; use crate::pipelines::PipelineBuildResult; use crate::sessions::QueryContext; use crate::sessions::TableContext; -use crate::sql::plans::Insert; -use crate::sql::plans::InsertInputSource; use crate::sql::plans::Plan; use crate::storages::StorageDescription; @@ -224,18 +223,21 @@ impl CreateTableInterpreter { TableIdent::new(table_id, table_id_seq), table_meta, ); - - let insert_plan = Insert { - catalog: self.plan.catalog.clone(), - database: self.plan.database.clone(), - table: self.plan.table.clone(), - schema: self.plan.schema.clone(), - overwrite: false, - source: InsertInputSource::SelectPlan(select_plan), - table_info: Some(table_info), - }; - - let mut pipeline = InsertInterpreter::try_create(self.ctx.clone(), insert_plan)? + let table = self.ctx.build_table_by_table_info(&table_info, None)?; + + let append_plan = create_append_plan_from_subquery( + &select_plan, + self.plan.catalog.clone(), + self.plan.database.clone(), + table, + Arc::new(self.plan.schema.clone().into()), + false, + self.ctx.clone(), + false, + ) + .await?; + let mut pipeline = InterpreterFactory::get(self.ctx.clone(), &append_plan) + .await? .execute2() .await?; diff --git a/src/query/service/src/interpreters/interpreter_table_modify_column.rs b/src/query/service/src/interpreters/interpreter_table_modify_column.rs index d749c81bcc95..62a8d87b2825 100644 --- a/src/query/service/src/interpreters/interpreter_table_modify_column.rs +++ b/src/query/service/src/interpreters/interpreter_table_modify_column.rs @@ -33,16 +33,12 @@ use databend_common_meta_app::schema::SetTableColumnMaskPolicyReq; use databend_common_meta_app::schema::TableMeta; use databend_common_meta_app::schema::UpdateTableMetaReq; use databend_common_meta_types::MatchSeq; -use databend_common_sql::executor::physical_plans::DistributedInsertSelect; -use databend_common_sql::executor::PhysicalPlan; -use databend_common_sql::executor::PhysicalPlanBuilder; use databend_common_sql::field_default_value; +use databend_common_sql::plans::create_append_plan_from_subquery; use databend_common_sql::plans::ModifyColumnAction; use databend_common_sql::plans::ModifyTableColumnPlan; -use databend_common_sql::plans::Plan; use databend_common_sql::BloomIndexColumns; use databend_common_sql::Planner; -use databend_common_storages_fuse::FuseTable; use databend_common_storages_stream::stream_table::STREAM_ENGINE; use databend_common_storages_view::view_table::VIEW_ENGINE; use databend_common_users::UserApiProvider; @@ -50,10 +46,10 @@ use databend_enterprise_data_mask_feature::get_datamask_handler; use databend_storages_common_index::BloomIndex; use databend_storages_common_table_meta::table::OPT_KEY_BLOOM_INDEX_COLUMNS; +use super::InterpreterFactory; use crate::interpreters::common::check_referenced_computed_columns; use crate::interpreters::Interpreter; use crate::pipelines::PipelineBuildResult; -use crate::schedulers::build_query_pipeline_without_render_result_set; use crate::sessions::QueryContext; use crate::sessions::TableContext; @@ -164,12 +160,6 @@ impl ModifyTableColumnInterpreter { let catalog_name = table_info.catalog(); let catalog = self.ctx.get_catalog(catalog_name).await?; - let fuse_table = FuseTable::try_from_table(table.as_ref())?; - let prev_snapshot_id = fuse_table - .read_table_snapshot() - .await - .map_or(None, |v| v.map(|snapshot| snapshot.snapshot_id)); - let mut bloom_index_cols = vec![]; if let Some(v) = table_info.options().get(OPT_KEY_BLOOM_INDEX_COLUMNS) { if let BloomIndexColumns::Specify(cols) = v.parse::()? { @@ -359,54 +349,24 @@ impl ModifyTableColumnInterpreter { let mut planner = Planner::new(self.ctx.clone()); let (plan, _extras) = planner.plan_sql(&sql).await?; - // 3. build physical plan by plan - let (select_plan, select_column_bindings) = match plan { - Plan::Query { - s_expr, - metadata, - bind_context, - .. - } => { - let mut builder1 = - PhysicalPlanBuilder::new(metadata.clone(), self.ctx.clone(), false); - ( - builder1.build(&s_expr, bind_context.column_set()).await?, - bind_context.columns.clone(), - ) - } - _ => unreachable!(), - }; - - // 4. define select schema and insert schema of DistributedInsertSelect plan table_info.meta.schema = new_schema.clone().into(); - let new_table = FuseTable::try_create(table_info)?; - - // 5. build DistributedInsertSelect plan - let insert_plan = - PhysicalPlan::DistributedInsertSelect(Box::new(DistributedInsertSelect { - plan_id: select_plan.get_id(), - input: Box::new(select_plan), - table_info: new_table.get_table_info().clone(), - select_schema: Arc::new(Arc::new(schema).into()), - select_column_bindings, - insert_schema: Arc::new(Arc::new(new_schema).into()), - cast_needed: true, - })); - let mut build_res = - build_query_pipeline_without_render_result_set(&self.ctx, &insert_plan).await?; - - // 6. commit new meta schema and snapshots - new_table.commit_insertion( + let new_table = self.ctx.build_table_by_table_info(&table_info, None)?; + + let append_plan = create_append_plan_from_subquery( + &plan, + self.plan.catalog.clone(), + self.plan.database.clone(), + new_table, + Arc::new(DataSchema::from(&new_schema)), + true, self.ctx.clone(), - &mut build_res.main_pipeline, - None, - vec![], true, - prev_snapshot_id, - None, - )?; - - Ok(build_res) + ) + .await?; + InterpreterFactory::get(self.ctx.clone(), &append_plan) + .await? + .execute2() + .await } // unset data mask policy to a column is a ee feature. diff --git a/src/query/service/src/interpreters/mod.rs b/src/query/service/src/interpreters/mod.rs index 8a31ac0dcafb..c7a555b8d7a4 100644 --- a/src/query/service/src/interpreters/mod.rs +++ b/src/query/service/src/interpreters/mod.rs @@ -16,6 +16,7 @@ mod access; pub(crate) mod common; mod hook; mod interpreter; +mod interpreter_append; mod interpreter_catalog_create; mod interpreter_catalog_drop; mod interpreter_catalog_show_create; @@ -28,7 +29,6 @@ mod interpreter_connection_desc; mod interpreter_connection_drop; mod interpreter_connection_show; mod interpreter_copy_into_location; -mod interpreter_copy_into_table; mod interpreter_data_mask_create; mod interpreter_data_mask_desc; mod interpreter_data_mask_drop; @@ -50,7 +50,6 @@ mod interpreter_file_format_show; mod interpreter_index_create; mod interpreter_index_drop; mod interpreter_index_refresh; -mod interpreter_insert; mod interpreter_insert_multi_table; mod interpreter_kill; mod interpreter_metrics; @@ -171,7 +170,6 @@ pub use interpreter_execute_immediate::ExecuteImmediateInterpreter; pub use interpreter_explain::ExplainInterpreter; pub use interpreter_factory::InterpreterFactory; pub use interpreter_index_refresh::RefreshIndexInterpreter; -pub use interpreter_insert::InsertInterpreter; pub use interpreter_insert_multi_table::InsertMultiTableInterpreter; pub use interpreter_kill::KillInterpreter; pub use interpreter_metrics::InterpreterMetrics; diff --git a/src/query/service/src/pipelines/builders/builder_copy_into_table.rs b/src/query/service/src/pipelines/builders/builder_append.rs similarity index 60% rename from src/query/service/src/pipelines/builders/builder_copy_into_table.rs rename to src/query/service/src/pipelines/builders/builder_append.rs index a843cbba728b..2aa82e115a1c 100644 --- a/src/query/service/src/pipelines/builders/builder_copy_into_table.rs +++ b/src/query/service/src/pipelines/builders/builder_append.rs @@ -25,44 +25,29 @@ use databend_common_expression::DataSchema; use databend_common_expression::DataSchemaRef; use databend_common_expression::DataSchemaRefExt; use databend_common_expression::Scalar; -use databend_common_meta_app::principal::FileFormatParams; -use databend_common_meta_app::principal::ParquetFileFormatParams; use databend_common_meta_app::schema::TableCopiedFileInfo; use databend_common_meta_app::schema::UpsertTableCopiedFileReq; use databend_common_pipeline_core::Pipeline; use databend_common_pipeline_transforms::processors::TransformPipelineHelper; -use databend_common_sql::executor::physical_plans::CopyIntoTable; -use databend_common_sql::executor::physical_plans::CopyIntoTableSource; -use databend_common_sql::plans::CopyIntoTableMode; +use databend_common_sql::executor::physical_plans::PhysicalAppend; use databend_common_storage::StageFileInfo; use log::debug; use log::info; use crate::pipelines::processors::transforms::TransformAddConstColumns; use crate::pipelines::processors::TransformCastSchema; -use crate::pipelines::processors::TransformNullIf; +// use crate::pipelines::processors::TransformNullIf; use crate::pipelines::PipelineBuilder; use crate::sessions::QueryContext; -/// This file implements copy into table pipeline builder. impl PipelineBuilder { - pub(crate) fn build_copy_into_table(&mut self, copy: &CopyIntoTable) -> Result<()> { + pub(crate) fn build_append(&mut self, copy: &PhysicalAppend) -> Result<()> { let to_table = self.ctx.build_table_by_table_info(©.table_info, None)?; - let source_schema = match ©.source { - CopyIntoTableSource::Query(input) => { - self.build_pipeline(input)?; - // Reorder the result for select clause - PipelineBuilder::build_result_projection( - &self.func_ctx, - input.output_schema()?, - copy.project_columns.as_ref().unwrap(), - &mut self.main_pipeline, - false, - )?; - let fields = copy - .project_columns - .as_ref() - .unwrap() + self.ctx + .set_read_block_thresholds(to_table.get_block_thresholds()); + let source_schema = match ©.project_columns { + Some(project_columns) => { + let fields = project_columns .iter() .map(|column_binding| { DataField::new( @@ -73,14 +58,19 @@ impl PipelineBuilder { .collect(); DataSchemaRefExt::create(fields) } - CopyIntoTableSource::Stage(input) => { - self.ctx - .set_read_block_thresholds(to_table.get_block_thresholds()); - - self.build_pipeline(input)?; - copy.required_source_schema.clone() - } + None => copy.input.output_schema()?, }; + + self.build_pipeline(©.input)?; + if let Some(project_columns) = ©.project_columns { + PipelineBuilder::build_result_projection( + &self.func_ctx, + copy.input.output_schema()?, + project_columns, + &mut self.main_pipeline, + false, + )?; + } Self::build_append_data_pipeline( self.ctx.clone(), &mut self.main_pipeline, @@ -91,63 +81,60 @@ impl PipelineBuilder { Ok(()) } - fn need_null_if_processor<'a>( - plan: &'a CopyIntoTable, - source_schema: &Arc, - dest_schema: &Arc, - ) -> Option<&'a [String]> { - if plan.is_transform { - return None; - } - if let FileFormatParams::Parquet(ParquetFileFormatParams { null_if, .. }) = - &plan.stage_table_info.stage_info.file_format_params - { - if !null_if.is_empty() - && source_schema - .fields - .iter() - .zip(dest_schema.fields.iter()) - .any(|(src_field, dest_field)| { - TransformNullIf::column_need_transform( - src_field.data_type(), - dest_field.data_type(), - ) - }) - { - return Some(null_if); - } - } - None - } + // fn need_null_if_processor<'a>( + // plan: &'a CopyIntoTable, + // _source_schema: &Arc, + // _dest_schema: &Arc, + // ) -> Option<&'a [String]> { + // // if let FileFormatParams::Parquet(ParquetFileFormatParams { null_if, .. }) = + // // &plan.stage_table_info.stage_info.file_format_params + // // { + // // if !null_if.is_empty() + // // && source_schema + // // .fields + // // .iter() + // // .zip(dest_schema.fields.iter()) + // // .any(|(src_field, dest_field)| { + // // TransformNullIf::column_need_transform( + // // src_field.data_type(), + // // dest_field.data_type(), + // // ) + // // }) + // // { + // // return Some(null_if); + // // } + // // } + // // None + // todo!() + // } fn build_append_data_pipeline( ctx: Arc, main_pipeline: &mut Pipeline, - plan: &CopyIntoTable, + plan: &PhysicalAppend, source_schema: Arc, to_table: Arc, ) -> Result<()> { 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; - let source_schema = if let Some(null_if) = - Self::need_null_if_processor(plan, &source_schema, plan_required_source_schema) - { - let func_ctx = ctx.get_function_context()?; - main_pipeline.try_add_transformer(|| { - TransformNullIf::try_new( - source_schema.clone(), - plan_required_source_schema.clone(), - func_ctx.clone(), - null_if, - ) - })?; - TransformNullIf::new_schema(&source_schema) - } else { - source_schema - }; + // let source_schema = if let Some(null_if) = + // Self::need_null_if_processor(plan, &source_schema, plan_required_source_schema) + // { + // let func_ctx = ctx.get_function_context()?; + // main_pipeline.try_add_transformer(|| { + // TransformNullIf::try_new( + // source_schema.clone(), + // plan_required_source_schema.clone(), + // func_ctx.clone(), + // null_if, + // ) + // })?; + // TransformNullIf::new_schema(&source_schema) + // } else { + // source_schema + // }; if &source_schema != plan_required_source_schema { // only parquet need cast @@ -171,25 +158,12 @@ impl PipelineBuilder { )?; } - // append data without commit. - match plan_write_mode { - CopyIntoTableMode::Insert { overwrite: _ } => { - Self::build_append2table_without_commit_pipeline( - ctx, - main_pipeline, - to_table.clone(), - plan_required_values_schema.clone(), - )? - } - CopyIntoTableMode::Replace => {} - CopyIntoTableMode::Copy => Self::build_append2table_without_commit_pipeline( - ctx, - main_pipeline, - to_table.clone(), - plan_required_values_schema.clone(), - )?, - } - Ok(()) + Self::build_append2table_without_commit_pipeline( + ctx, + main_pipeline, + to_table.clone(), + plan_required_values_schema.clone(), + ) } pub(crate) fn build_upsert_copied_files_to_meta_req( diff --git a/src/query/service/src/pipelines/builders/builder_append_table.rs b/src/query/service/src/pipelines/builders/builder_append_table.rs index 90e151df2550..379061e6e073 100644 --- a/src/query/service/src/pipelines/builders/builder_append_table.rs +++ b/src/query/service/src/pipelines/builders/builder_append_table.rs @@ -47,7 +47,7 @@ impl PipelineBuilder { copied_files, update_stream_meta, overwrite, - None, + false, deduplicated_label, )?; diff --git a/src/query/service/src/pipelines/builders/builder_commit.rs b/src/query/service/src/pipelines/builders/builder_commit.rs index e4becfd76f55..993f4445a459 100644 --- a/src/query/service/src/pipelines/builders/builder_commit.rs +++ b/src/query/service/src/pipelines/builders/builder_commit.rs @@ -72,7 +72,7 @@ impl PipelineBuilder { snapshot_gen.clone(), input, None, - None, + false, plan.deduplicated_label.clone(), ) }) diff --git a/src/query/service/src/pipelines/builders/builder_distributed_insert_select.rs b/src/query/service/src/pipelines/builders/builder_distributed_insert_select.rs deleted file mode 100644 index ef2bc29e131b..000000000000 --- a/src/query/service/src/pipelines/builders/builder_distributed_insert_select.rs +++ /dev/null @@ -1,67 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_exception::Result; -use databend_common_pipeline_transforms::processors::TransformPipelineHelper; -use databend_common_sql::executor::physical_plans::DistributedInsertSelect; - -use crate::pipelines::processors::TransformCastSchema; -use crate::pipelines::PipelineBuilder; - -impl PipelineBuilder { - pub fn build_distributed_insert_select( - &mut self, - insert_select: &DistributedInsertSelect, - ) -> Result<()> { - let select_schema = &insert_select.select_schema; - let insert_schema = &insert_select.insert_schema; - - self.build_pipeline(&insert_select.input)?; - - // should render result for select - PipelineBuilder::build_result_projection( - &self.func_ctx, - insert_select.input.output_schema()?, - &insert_select.select_column_bindings, - &mut self.main_pipeline, - false, - )?; - - if insert_select.cast_needed { - self.main_pipeline.try_add_transformer(|| { - TransformCastSchema::try_new( - select_schema.clone(), - insert_schema.clone(), - self.func_ctx.clone(), - ) - })?; - } - - let table = self - .ctx - .build_table_by_table_info(&insert_select.table_info, None)?; - - let source_schema = insert_schema; - Self::fill_and_reorder_columns( - self.ctx.clone(), - &mut self.main_pipeline, - table.clone(), - source_schema.clone(), - )?; - - table.append_data(self.ctx.clone(), &mut self.main_pipeline)?; - - Ok(()) - } -} diff --git a/src/query/service/src/pipelines/builders/builder_mutation.rs b/src/query/service/src/pipelines/builders/builder_mutation.rs index 3cd64c367fc5..41a4bc5d7b3e 100644 --- a/src/query/service/src/pipelines/builders/builder_mutation.rs +++ b/src/query/service/src/pipelines/builders/builder_mutation.rs @@ -42,7 +42,6 @@ use databend_common_storages_fuse::FuseTable; use crate::pipelines::processors::transforms::TransformAddComputedColumns; use crate::pipelines::processors::TransformResortAddOnWithoutSourceSchema; use crate::pipelines::PipelineBuilder; - impl PipelineBuilder { // build mutation serialize and mutation pipeline pub(crate) fn build_mutation(&mut self, merge_into: &Mutation) -> Result<()> { diff --git a/src/query/service/src/pipelines/builders/builder_replace_into.rs b/src/query/service/src/pipelines/builders/builder_replace_into.rs index ad3fd8854904..4dc19afd8e6b 100644 --- a/src/query/service/src/pipelines/builders/builder_replace_into.rs +++ b/src/query/service/src/pipelines/builders/builder_replace_into.rs @@ -34,16 +34,15 @@ use databend_common_pipeline_core::processors::InputPort; use databend_common_pipeline_core::processors::OutputPort; use databend_common_pipeline_core::Pipe; use databend_common_pipeline_sources::AsyncSource; -use databend_common_pipeline_sources::AsyncSourcer; +// use databend_common_pipeline_sources::AsyncSourcer; use databend_common_pipeline_transforms::processors::build_compact_block_pipeline; use databend_common_pipeline_transforms::processors::create_dummy_item; use databend_common_pipeline_transforms::processors::TransformPipelineHelper; use databend_common_sql::executor::physical_plans::MutationKind; -use databend_common_sql::executor::physical_plans::ReplaceAsyncSourcer; use databend_common_sql::executor::physical_plans::ReplaceDeduplicate; use databend_common_sql::executor::physical_plans::ReplaceInto; use databend_common_sql::executor::physical_plans::ReplaceSelectCtx; -use databend_common_sql::plans::InsertValue; +// use databend_common_sql::plans::InsertValue; use databend_common_sql::BindContext; use databend_common_sql::Metadata; use databend_common_sql::MetadataRef; @@ -69,36 +68,6 @@ impl PipelineBuilder { Ok(cast_needed) } - // build async sourcer pipeline. - pub(crate) fn build_async_sourcer( - &mut self, - async_sourcer: &ReplaceAsyncSourcer, - ) -> Result<()> { - self.main_pipeline.add_source( - |output| { - let name_resolution_ctx = NameResolutionContext::try_from(self.settings.as_ref())?; - match &async_sourcer.source { - InsertValue::Values { rows } => { - let inner = ValueSource::new(rows.clone(), async_sourcer.schema.clone()); - AsyncSourcer::create(self.ctx.clone(), output, inner) - } - InsertValue::RawValues { data, start } => { - let inner = RawValueSource::new( - data.clone(), - self.ctx.clone(), - name_resolution_ctx, - async_sourcer.schema.clone(), - *start, - ); - AsyncSourcer::create(self.ctx.clone(), output, inner) - } - } - }, - 1, - )?; - Ok(()) - } - // build replace into pipeline. pub(crate) fn build_replace_into(&mut self, replace: &ReplaceInto) -> Result<()> { let ReplaceInto { @@ -398,9 +367,9 @@ pub struct ValueSource { } impl ValueSource { - pub fn new(rows: Vec>, schema: DataSchemaRef) -> Self { + pub fn new(rows: Arc>>, schema: DataSchemaRef) -> Self { Self { - rows: Arc::new(rows), + rows, schema, is_finished: false, } @@ -442,7 +411,7 @@ impl AsyncSource for ValueSource { } pub struct RawValueSource { - data: String, + data: Arc, ctx: Arc, name_resolution_ctx: NameResolutionContext, bind_context: BindContext, @@ -454,7 +423,7 @@ pub struct RawValueSource { impl RawValueSource { pub fn new( - data: String, + data: Arc, ctx: Arc, name_resolution_ctx: NameResolutionContext, schema: DataSchemaRef, diff --git a/src/query/service/src/pipelines/builders/builder_scan.rs b/src/query/service/src/pipelines/builders/builder_scan.rs index 4e85b4e6542e..be5f3d6e1db7 100644 --- a/src/query/service/src/pipelines/builders/builder_scan.rs +++ b/src/query/service/src/pipelines/builders/builder_scan.rs @@ -18,6 +18,7 @@ use databend_common_exception::Result; use databend_common_expression::DataBlock; use databend_common_functions::BUILTIN_FUNCTIONS; use databend_common_pipeline_core::processors::ProcessorPtr; +use databend_common_pipeline_sources::AsyncSourcer; use databend_common_pipeline_sources::OneBlockSource; use databend_common_pipeline_transforms::processors::TransformPipelineHelper; use databend_common_sql::evaluator::BlockOperator; @@ -26,9 +27,14 @@ use databend_common_sql::executor::physical_plans::CacheScan; use databend_common_sql::executor::physical_plans::ConstantTableScan; use databend_common_sql::executor::physical_plans::CteScan; use databend_common_sql::executor::physical_plans::ExpressionScan; +use databend_common_sql::executor::physical_plans::PhysicalValueScan; use databend_common_sql::executor::physical_plans::TableScan; +use databend_common_sql::executor::physical_plans::Values; use databend_common_sql::plans::CacheSource; +use databend_common_sql::NameResolutionContext; +use super::RawValueSource; +use super::ValueSource; use crate::pipelines::processors::transforms::CacheSourceState; use crate::pipelines::processors::transforms::HashJoinCacheState; use crate::pipelines::processors::transforms::MaterializedCteSource; @@ -165,4 +171,33 @@ impl PipelineBuilder { Ok(()) } + + pub(crate) fn build_value_scan(&mut self, scan: &PhysicalValueScan) -> Result<()> { + match &scan.values { + Values::Values(rows) => self.main_pipeline.add_source( + |output| { + let inner = ValueSource::new(rows.clone(), scan.output_schema.clone()); + AsyncSourcer::create(self.ctx.clone(), output, inner) + }, + 1, + ), + Values::RawValues { rest_str, start } => self.main_pipeline.add_source( + |output| { + let name_resolution_ctx = NameResolutionContext { + deny_column_reference: true, + ..Default::default() + }; + let inner = RawValueSource::new( + rest_str.clone(), + self.ctx.clone(), + name_resolution_ctx, + scan.output_schema.clone(), + *start, + ); + AsyncSourcer::create(self.ctx.clone(), output, inner) + }, + 1, + ), + } + } } diff --git a/src/query/service/src/pipelines/builders/mod.rs b/src/query/service/src/pipelines/builders/mod.rs index a35c735954af..ce00718b0d46 100644 --- a/src/query/service/src/pipelines/builders/mod.rs +++ b/src/query/service/src/pipelines/builders/mod.rs @@ -14,14 +14,13 @@ mod builder_add_stream_column; mod builder_aggregate; +mod builder_append; mod builder_append_table; mod builder_async_function; mod builder_column_mutation; mod builder_commit; mod builder_compact; mod builder_copy_into_location; -mod builder_copy_into_table; -mod builder_distributed_insert_select; mod builder_exchange; mod builder_fill_missing_columns; mod builder_filter; diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs index 12e8463ee028..02cd02d26f7b 100644 --- a/src/query/service/src/pipelines/pipeline_builder.rs +++ b/src/query/service/src/pipelines/pipeline_builder.rs @@ -180,9 +180,6 @@ impl PipelineBuilder { PhysicalPlan::ExchangeSink(sink) => self.build_exchange_sink(sink), PhysicalPlan::ExchangeSource(source) => self.build_exchange_source(source), PhysicalPlan::UnionAll(union_all) => self.build_union_all(union_all), - PhysicalPlan::DistributedInsertSelect(insert_select) => { - self.build_distributed_insert_select(insert_select) - } PhysicalPlan::ProjectSet(project_set) => self.build_project_set(project_set), PhysicalPlan::Udf(udf) => self.build_udf(udf), PhysicalPlan::Exchange(_) => Err(ErrorCode::Internal( @@ -198,13 +195,10 @@ impl PipelineBuilder { } // Copy into. - PhysicalPlan::CopyIntoTable(copy) => self.build_copy_into_table(copy), + PhysicalPlan::Append(append) => self.build_append(append), PhysicalPlan::CopyIntoLocation(copy) => self.build_copy_into_location(copy), // Replace. - PhysicalPlan::ReplaceAsyncSourcer(async_sourcer) => { - self.build_async_sourcer(async_sourcer) - } PhysicalPlan::ReplaceDeduplicate(deduplicate) => self.build_deduplicate(deduplicate), PhysicalPlan::ReplaceInto(replace) => self.build_replace_into(replace), @@ -259,6 +253,7 @@ impl PipelineBuilder { PhysicalPlan::ColumnMutation(column_mutation) => { self.build_column_mutation(column_mutation) } + PhysicalPlan::ValueScan(value_scan) => self.build_value_scan(value_scan), }?; self.is_exchange_neighbor = is_exchange_neighbor; diff --git a/src/query/service/src/pipelines/processors/transforms/transform_recursive_cte_source.rs b/src/query/service/src/pipelines/processors/transforms/transform_recursive_cte_source.rs index e86b7b10c3b6..d24f92c25172 100644 --- a/src/query/service/src/pipelines/processors/transforms/transform_recursive_cte_source.rs +++ b/src/query/service/src/pipelines/processors/transforms/transform_recursive_cte_source.rs @@ -323,12 +323,10 @@ async fn create_memory_table_for_cte_scan( | PhysicalPlan::ConstantTableScan(_) | PhysicalPlan::ExpressionScan(_) | PhysicalPlan::CacheScan(_) - | PhysicalPlan::DistributedInsertSelect(_) | PhysicalPlan::ExchangeSource(_) | PhysicalPlan::ExchangeSink(_) - | PhysicalPlan::CopyIntoTable(_) + | PhysicalPlan::Append(_) | PhysicalPlan::CopyIntoLocation(_) - | PhysicalPlan::ReplaceAsyncSourcer(_) | PhysicalPlan::ReplaceDeduplicate(_) | PhysicalPlan::ReplaceInto(_) | PhysicalPlan::ColumnMutation(_) @@ -348,6 +346,7 @@ async fn create_memory_table_for_cte_scan( | PhysicalPlan::ChunkFillAndReorder(_) | PhysicalPlan::ChunkAppendData(_) | PhysicalPlan::ChunkMerge(_) + | PhysicalPlan::ValueScan(_) | PhysicalPlan::ChunkCommitInsert(_) => {} } Ok(()) diff --git a/src/query/service/src/schedulers/fragments/fragmenter.rs b/src/query/service/src/schedulers/fragments/fragmenter.rs index c00a11e7403d..727abbc99b81 100644 --- a/src/query/service/src/schedulers/fragments/fragmenter.rs +++ b/src/query/service/src/schedulers/fragments/fragmenter.rs @@ -18,14 +18,13 @@ use databend_common_catalog::table_context::TableContext; use databend_common_exception::Result; use databend_common_sql::executor::physical_plans::CompactSource; use databend_common_sql::executor::physical_plans::ConstantTableScan; -use databend_common_sql::executor::physical_plans::CopyIntoTable; -use databend_common_sql::executor::physical_plans::CopyIntoTableSource; use databend_common_sql::executor::physical_plans::Exchange; use databend_common_sql::executor::physical_plans::ExchangeSink; use databend_common_sql::executor::physical_plans::ExchangeSource; use databend_common_sql::executor::physical_plans::FragmentKind; use databend_common_sql::executor::physical_plans::HashJoin; use databend_common_sql::executor::physical_plans::MutationSource; +use databend_common_sql::executor::physical_plans::PhysicalAppend; use databend_common_sql::executor::physical_plans::Recluster; use databend_common_sql::executor::physical_plans::ReplaceInto; use databend_common_sql::executor::physical_plans::TableScan; @@ -179,21 +178,13 @@ impl PhysicalPlanReplacer for Fragmenter { }))) } - // TODO(Sky): remove redundant code - fn replace_copy_into_table(&mut self, plan: &CopyIntoTable) -> Result { - match &plan.source { - CopyIntoTableSource::Stage(_) => { - self.state = State::SelectLeaf; - Ok(PhysicalPlan::CopyIntoTable(Box::new(plan.clone()))) - } - CopyIntoTableSource::Query(query_physical_plan) => { - let input = self.replace(query_physical_plan)?; - Ok(PhysicalPlan::CopyIntoTable(Box::new(CopyIntoTable { - source: CopyIntoTableSource::Query(Box::new(input)), - ..plan.clone() - }))) - } - } + fn replace_append(&mut self, plan: &PhysicalAppend) -> Result { + let input = self.replace(&plan.input)?; + Ok(PhysicalPlan::Append(Box::new(PhysicalAppend { + plan_id: plan.plan_id, + input: Box::new(input), + ..plan.clone() + }))) } fn replace_recluster(&mut self, plan: &Recluster) -> Result { diff --git a/src/query/service/src/schedulers/fragments/plan_fragment.rs b/src/query/service/src/schedulers/fragments/plan_fragment.rs index da76a11358df..d7cc63415c0a 100644 --- a/src/query/service/src/schedulers/fragments/plan_fragment.rs +++ b/src/query/service/src/schedulers/fragments/plan_fragment.rs @@ -27,9 +27,8 @@ use databend_common_expression::Value; use databend_common_settings::ReplaceIntoShuffleStrategy; use databend_common_sql::executor::physical_plans::CompactSource; use databend_common_sql::executor::physical_plans::ConstantTableScan; -use databend_common_sql::executor::physical_plans::CopyIntoTable; -use databend_common_sql::executor::physical_plans::CopyIntoTableSource; use databend_common_sql::executor::physical_plans::MutationSource; +use databend_common_sql::executor::physical_plans::PhysicalAppend; use databend_common_sql::executor::physical_plans::Recluster; use databend_common_sql::executor::physical_plans::ReplaceDeduplicate; use databend_common_sql::executor::physical_plans::ReplaceInto; @@ -536,23 +535,13 @@ impl PhysicalPlanReplacer for ReplaceReadSource { })) } - fn replace_copy_into_table(&mut self, plan: &CopyIntoTable) -> Result { - match &plan.source { - CopyIntoTableSource::Query(query_physical_plan) => { - let input = self.replace(query_physical_plan)?; - Ok(PhysicalPlan::CopyIntoTable(Box::new(CopyIntoTable { - source: CopyIntoTableSource::Query(Box::new(input)), - ..plan.clone() - }))) - } - CopyIntoTableSource::Stage(v) => { - let input = self.replace(v)?; - Ok(PhysicalPlan::CopyIntoTable(Box::new(CopyIntoTable { - source: CopyIntoTableSource::Stage(Box::new(input)), - ..plan.clone() - }))) - } - } + fn replace_append(&mut self, plan: &PhysicalAppend) -> Result { + let input = self.replace(&plan.input)?; + Ok(PhysicalPlan::Append(Box::new(PhysicalAppend { + plan_id: plan.plan_id, + input: Box::new(input), + ..plan.clone() + }))) } } diff --git a/src/query/service/src/sessions/queue_mgr.rs b/src/query/service/src/sessions/queue_mgr.rs index 231e05aff5b3..baf4f448cbae 100644 --- a/src/query/service/src/sessions/queue_mgr.rs +++ b/src/query/service/src/sessions/queue_mgr.rs @@ -395,11 +395,10 @@ impl QueryEntry { } // Write: Heavy actions. - Plan::Insert(_) - | Plan::InsertMultiTable(_) + Plan::InsertMultiTable(_) | Plan::Replace(_) | Plan::DataMutation { .. } - | Plan::CopyIntoTable(_) + | Plan::Append { .. } | Plan::CopyIntoLocation(_) => { return true; } diff --git a/src/query/service/src/test_kits/fixture.rs b/src/query/service/src/test_kits/fixture.rs index e46472ab1754..2fb6e373d75d 100644 --- a/src/query/service/src/test_kits/fixture.rs +++ b/src/query/service/src/test_kits/fixture.rs @@ -847,7 +847,7 @@ impl TestFixture { None, vec![], overwrite, - None, + false, None, )?; } else { diff --git a/src/query/sql/Cargo.toml b/src/query/sql/Cargo.toml index 1f017a82dac1..adfb81e625e1 100644 --- a/src/query/sql/Cargo.toml +++ b/src/query/sql/Cargo.toml @@ -32,6 +32,7 @@ databend-common-pipeline-transforms = { workspace = true } databend-common-settings = { workspace = true } databend-common-storage = { workspace = true } databend-common-storages-result-cache = { workspace = true } +databend-common-storages-stage = { workspace = true } databend-common-storages-view = { workspace = true } databend-common-users = { workspace = true } databend-enterprise-data-mask-feature = { workspace = true } diff --git a/src/query/sql/src/executor/format.rs b/src/query/sql/src/executor/format.rs index e0463d9aa8a3..31ab72ee7ebc 100644 --- a/src/query/sql/src/executor/format.rs +++ b/src/query/sql/src/executor/format.rs @@ -26,6 +26,8 @@ use databend_common_pipeline_core::processors::PlanProfile; use itertools::Itertools; use super::physical_plans::AddStreamColumn; +use super::physical_plans::PhysicalValueScan; +use super::physical_plans::Values; use crate::executor::explain::PlanStatsInfo; use crate::executor::physical_plans::AggregateExpand; use crate::executor::physical_plans::AggregateFinal; @@ -37,9 +39,7 @@ use crate::executor::physical_plans::ColumnMutation; use crate::executor::physical_plans::CommitSink; use crate::executor::physical_plans::ConstantTableScan; use crate::executor::physical_plans::CopyIntoLocation; -use crate::executor::physical_plans::CopyIntoTable; use crate::executor::physical_plans::CteScan; -use crate::executor::physical_plans::DistributedInsertSelect; use crate::executor::physical_plans::EvalScalar; use crate::executor::physical_plans::Exchange; use crate::executor::physical_plans::ExchangeSink; @@ -55,6 +55,7 @@ use crate::executor::physical_plans::MutationManipulate; use crate::executor::physical_plans::MutationOrganize; use crate::executor::physical_plans::MutationSource; use crate::executor::physical_plans::MutationSplit; +use crate::executor::physical_plans::PhysicalAppend; use crate::executor::physical_plans::ProjectSet; use crate::executor::physical_plans::RangeJoin; use crate::executor::physical_plans::RangeJoinType; @@ -362,20 +363,14 @@ fn to_format_tree( PhysicalPlan::UnionAll(plan) => union_all_to_format_tree(plan, metadata, profs), PhysicalPlan::ExchangeSource(plan) => exchange_source_to_format_tree(plan, metadata), PhysicalPlan::ExchangeSink(plan) => exchange_sink_to_format_tree(plan, metadata, profs), - PhysicalPlan::DistributedInsertSelect(plan) => { - distributed_insert_to_format_tree(plan.as_ref(), metadata, profs) - } PhysicalPlan::Recluster(_) => Ok(FormatTreeNode::new("Recluster".to_string())), PhysicalPlan::CompactSource(_) => Ok(FormatTreeNode::new("CompactSource".to_string())), PhysicalPlan::CommitSink(plan) => commit_sink_to_format_tree(plan, metadata, profs), PhysicalPlan::ProjectSet(plan) => project_set_to_format_tree(plan, metadata, profs), PhysicalPlan::Udf(plan) => udf_to_format_tree(plan, metadata, profs), PhysicalPlan::RangeJoin(plan) => range_join_to_format_tree(plan, metadata, profs), - PhysicalPlan::CopyIntoTable(plan) => copy_into_table(plan), + PhysicalPlan::Append(plan) => append_to_format_tree(plan, metadata, profs), PhysicalPlan::CopyIntoLocation(plan) => copy_into_location(plan), - PhysicalPlan::ReplaceAsyncSourcer(_) => { - Ok(FormatTreeNode::new("ReplaceAsyncSourcer".to_string())) - } PhysicalPlan::ReplaceDeduplicate(_) => { Ok(FormatTreeNode::new("ReplaceDeduplicate".to_string())) } @@ -485,6 +480,7 @@ fn to_format_tree( )) } PhysicalPlan::AsyncFunction(plan) => async_function_to_format_tree(plan, metadata, profs), + PhysicalPlan::ValueScan(plan) => value_scan_to_format_tree(plan, metadata, profs), } } @@ -690,11 +686,56 @@ fn format_add_stream_column( to_format_tree(&plan.input, metadata, profs) } -fn copy_into_table(plan: &CopyIntoTable) -> Result> { - Ok(FormatTreeNode::new(format!( - "CopyIntoTable: {}", - plan.table_info - ))) +fn append_to_format_tree( + plan: &PhysicalAppend, + metadata: &Metadata, + profs: &HashMap, +) -> Result> { + let mut children = vec![]; + let target_table = FormatTreeNode::new(format!( + "target table: [catalog: {}] [desc: {}]", + plan.table_info.catalog_info.name_ident.catalog_name, plan.table_info.desc + )); + children.push(target_table); + let required_columns = + format_output_columns(plan.required_values_schema.clone(), metadata, false); + children.push(FormatTreeNode::new(format!( + "required columns: [{}]", + required_columns + ))); + + children.push(to_format_tree(&plan.input, metadata, profs)?); + Ok(FormatTreeNode::with_children( + "Append".to_string(), + children, + )) +} + +fn value_scan_to_format_tree( + plan: &PhysicalValueScan, + _metadata: &Metadata, + _profs: &HashMap, +) -> Result> { + let mut children = vec![]; + match &plan.values { + Values::Values(values) => { + children.push(FormatTreeNode::new(format!( + "values: [{}] rows", + values.len() + ))); + } + Values::RawValues { rest_str, start } => { + children.push(FormatTreeNode::new(format!( + "raw values: string length [{}], start [{}]", + rest_str.len(), + start + ))); + } + } + Ok(FormatTreeNode::with_children( + "ValueScan".to_string(), + children, + )) } fn copy_into_location(_: &CopyIntoLocation) -> Result> { @@ -1686,19 +1727,6 @@ fn exchange_sink_to_format_tree( )) } -fn distributed_insert_to_format_tree( - plan: &DistributedInsertSelect, - metadata: &Metadata, - profs: &HashMap, -) -> Result> { - let children = vec![to_format_tree(&plan.input, metadata, profs)?]; - - Ok(FormatTreeNode::with_children( - "DistributedInsertSelect".to_string(), - children, - )) -} - fn commit_sink_to_format_tree( plan: &CommitSink, metadata: &Metadata, diff --git a/src/query/sql/src/executor/physical_plan.rs b/src/query/sql/src/executor/physical_plan.rs index 4bbe18f59e0b..c455fe9c9d37 100644 --- a/src/query/sql/src/executor/physical_plan.rs +++ b/src/query/sql/src/executor/physical_plan.rs @@ -27,6 +27,7 @@ use super::physical_plans::MutationManipulate; use super::physical_plans::MutationOrganize; use super::physical_plans::MutationSource; use super::physical_plans::MutationSplit; +use super::physical_plans::PhysicalValueScan; use crate::executor::physical_plans::AggregateExpand; use crate::executor::physical_plans::AggregateFinal; use crate::executor::physical_plans::AggregatePartial; @@ -44,10 +45,7 @@ use crate::executor::physical_plans::CommitSink; use crate::executor::physical_plans::CompactSource; use crate::executor::physical_plans::ConstantTableScan; use crate::executor::physical_plans::CopyIntoLocation; -use crate::executor::physical_plans::CopyIntoTable; -use crate::executor::physical_plans::CopyIntoTableSource; use crate::executor::physical_plans::CteScan; -use crate::executor::physical_plans::DistributedInsertSelect; use crate::executor::physical_plans::Duplicate; use crate::executor::physical_plans::EvalScalar; use crate::executor::physical_plans::Exchange; @@ -59,11 +57,11 @@ use crate::executor::physical_plans::HashJoin; use crate::executor::physical_plans::Limit; use crate::executor::physical_plans::MaterializedCte; use crate::executor::physical_plans::Mutation; +use crate::executor::physical_plans::PhysicalAppend; use crate::executor::physical_plans::ProjectSet; use crate::executor::physical_plans::RangeJoin; use crate::executor::physical_plans::Recluster; use crate::executor::physical_plans::RecursiveCteScan; -use crate::executor::physical_plans::ReplaceAsyncSourcer; use crate::executor::physical_plans::ReplaceDeduplicate; use crate::executor::physical_plans::ReplaceInto; use crate::executor::physical_plans::RowFetch; @@ -102,19 +100,16 @@ pub enum PhysicalPlan { Udf(Udf), RecursiveCteScan(RecursiveCteScan), - /// For insert into ... select ... in cluster - DistributedInsertSelect(Box), - /// Synthesized by fragmented ExchangeSource(ExchangeSource), ExchangeSink(ExchangeSink), /// Copy into table - CopyIntoTable(Box), + Append(Box), + ValueScan(Box), CopyIntoLocation(Box), /// Replace - ReplaceAsyncSourcer(ReplaceAsyncSourcer), ReplaceDeduplicate(Box), ReplaceInto(Box), @@ -273,26 +268,22 @@ impl PhysicalPlan { *next_id += 1; plan.input.adjust_plan_id(next_id); } - PhysicalPlan::DistributedInsertSelect(plan) => { + PhysicalPlan::ExchangeSource(plan) => { plan.plan_id = *next_id; *next_id += 1; - plan.input.adjust_plan_id(next_id); } - PhysicalPlan::ExchangeSource(plan) => { + PhysicalPlan::ExchangeSink(plan) => { plan.plan_id = *next_id; *next_id += 1; } - PhysicalPlan::ExchangeSink(plan) => { + PhysicalPlan::Append(plan) => { plan.plan_id = *next_id; *next_id += 1; + plan.input.adjust_plan_id(next_id); } - PhysicalPlan::CopyIntoTable(plan) => { + PhysicalPlan::ValueScan(plan) => { plan.plan_id = *next_id; *next_id += 1; - match &mut plan.source { - CopyIntoTableSource::Query(input) => input.adjust_plan_id(next_id), - CopyIntoTableSource::Stage(input) => input.adjust_plan_id(next_id), - }; } PhysicalPlan::CopyIntoLocation(plan) => { plan.plan_id = *next_id; @@ -343,10 +334,6 @@ impl PhysicalPlan { *next_id += 1; plan.input.adjust_plan_id(next_id); } - PhysicalPlan::ReplaceAsyncSourcer(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - } PhysicalPlan::ReplaceDeduplicate(plan) => { plan.plan_id = *next_id; *next_id += 1; @@ -428,7 +415,6 @@ impl PhysicalPlan { PhysicalPlan::RangeJoin(v) => v.plan_id, PhysicalPlan::Exchange(v) => v.plan_id, PhysicalPlan::UnionAll(v) => v.plan_id, - PhysicalPlan::DistributedInsertSelect(v) => v.plan_id, PhysicalPlan::ExchangeSource(v) => v.plan_id, PhysicalPlan::ExchangeSink(v) => v.plan_id, PhysicalPlan::CteScan(v) => v.plan_id, @@ -445,9 +431,8 @@ impl PhysicalPlan { PhysicalPlan::MutationOrganize(v) => v.plan_id, PhysicalPlan::AddStreamColumn(v) => v.plan_id, PhysicalPlan::CommitSink(v) => v.plan_id, - PhysicalPlan::CopyIntoTable(v) => v.plan_id, + PhysicalPlan::Append(v) => v.plan_id, PhysicalPlan::CopyIntoLocation(v) => v.plan_id, - PhysicalPlan::ReplaceAsyncSourcer(v) => v.plan_id, PhysicalPlan::ReplaceDeduplicate(v) => v.plan_id, PhysicalPlan::ReplaceInto(v) => v.plan_id, PhysicalPlan::CompactSource(v) => v.plan_id, @@ -462,6 +447,7 @@ impl PhysicalPlan { PhysicalPlan::ChunkMerge(v) => v.plan_id, PhysicalPlan::ChunkCommitInsert(v) => v.plan_id, PhysicalPlan::RecursiveCteScan(v) => v.plan_id, + PhysicalPlan::ValueScan(v) => v.plan_id, } } @@ -486,7 +472,7 @@ impl PhysicalPlan { PhysicalPlan::UnionAll(plan) => plan.output_schema(), PhysicalPlan::ProjectSet(plan) => plan.output_schema(), PhysicalPlan::RangeJoin(plan) => plan.output_schema(), - PhysicalPlan::CopyIntoTable(plan) => plan.output_schema(), + PhysicalPlan::Append(plan) => plan.output_schema(), PhysicalPlan::CopyIntoLocation(plan) => plan.output_schema(), PhysicalPlan::CteScan(plan) => plan.output_schema(), PhysicalPlan::MaterializedCte(plan) => plan.output_schema(), @@ -502,12 +488,10 @@ impl PhysicalPlan { PhysicalPlan::AddStreamColumn(plan) => plan.output_schema(), PhysicalPlan::Mutation(_) | PhysicalPlan::ColumnMutation(_) - | PhysicalPlan::ReplaceAsyncSourcer(_) | PhysicalPlan::ReplaceDeduplicate(_) | PhysicalPlan::ReplaceInto(_) | PhysicalPlan::CompactSource(_) | PhysicalPlan::CommitSink(_) - | PhysicalPlan::DistributedInsertSelect(_) | PhysicalPlan::Recluster(_) => Ok(DataSchemaRef::default()), PhysicalPlan::Duplicate(plan) => plan.input.output_schema(), PhysicalPlan::Shuffle(plan) => plan.input.output_schema(), @@ -518,6 +502,7 @@ impl PhysicalPlan { PhysicalPlan::ChunkAppendData(_) => todo!(), PhysicalPlan::ChunkMerge(_) => todo!(), PhysicalPlan::ChunkCommitInsert(_) => todo!(), + PhysicalPlan::ValueScan(plan) => plan.output_schema(), } } @@ -544,16 +529,14 @@ impl PhysicalPlan { PhysicalPlan::HashJoin(_) => "HashJoin".to_string(), PhysicalPlan::Exchange(_) => "Exchange".to_string(), PhysicalPlan::UnionAll(_) => "UnionAll".to_string(), - PhysicalPlan::DistributedInsertSelect(_) => "DistributedInsertSelect".to_string(), PhysicalPlan::ExchangeSource(_) => "Exchange Source".to_string(), PhysicalPlan::ExchangeSink(_) => "Exchange Sink".to_string(), PhysicalPlan::ProjectSet(_) => "Unnest".to_string(), PhysicalPlan::CompactSource(_) => "CompactBlock".to_string(), PhysicalPlan::CommitSink(_) => "CommitSink".to_string(), PhysicalPlan::RangeJoin(_) => "RangeJoin".to_string(), - PhysicalPlan::CopyIntoTable(_) => "CopyIntoTable".to_string(), + PhysicalPlan::Append(_) => "Append".to_string(), PhysicalPlan::CopyIntoLocation(_) => "CopyIntoLocation".to_string(), - PhysicalPlan::ReplaceAsyncSourcer(_) => "ReplaceAsyncSourcer".to_string(), PhysicalPlan::ReplaceDeduplicate(_) => "ReplaceDeduplicate".to_string(), PhysicalPlan::ReplaceInto(_) => "Replace".to_string(), PhysicalPlan::MutationSource(_) => "MutationSource".to_string(), @@ -580,6 +563,7 @@ impl PhysicalPlan { PhysicalPlan::ChunkAppendData(_) => "WriteData".to_string(), PhysicalPlan::ChunkMerge(_) => "ChunkMerge".to_string(), PhysicalPlan::ChunkCommitInsert(_) => "Commit".to_string(), + PhysicalPlan::ValueScan(_) => "ValueScan".to_string(), } } @@ -591,8 +575,8 @@ impl PhysicalPlan { | PhysicalPlan::CacheScan(_) | PhysicalPlan::ExchangeSource(_) | PhysicalPlan::CompactSource(_) - | PhysicalPlan::ReplaceAsyncSourcer(_) | PhysicalPlan::Recluster(_) + | PhysicalPlan::ValueScan(_) | PhysicalPlan::RecursiveCteScan(_) => Box::new(std::iter::empty()), PhysicalPlan::Filter(plan) => Box::new(std::iter::once(plan.input.as_ref())), PhysicalPlan::EvalScalar(plan) => Box::new(std::iter::once(plan.input.as_ref())), @@ -613,9 +597,6 @@ impl PhysicalPlan { PhysicalPlan::UnionAll(plan) => Box::new( std::iter::once(plan.left.as_ref()).chain(std::iter::once(plan.right.as_ref())), ), - PhysicalPlan::DistributedInsertSelect(plan) => { - Box::new(std::iter::once(plan.input.as_ref())) - } PhysicalPlan::CommitSink(plan) => Box::new(std::iter::once(plan.input.as_ref())), PhysicalPlan::ProjectSet(plan) => Box::new(std::iter::once(plan.input.as_ref())), PhysicalPlan::RangeJoin(plan) => Box::new( @@ -651,10 +632,7 @@ impl PhysicalPlan { PhysicalPlan::ChunkAppendData(plan) => Box::new(std::iter::once(plan.input.as_ref())), PhysicalPlan::ChunkMerge(plan) => Box::new(std::iter::once(plan.input.as_ref())), PhysicalPlan::ChunkCommitInsert(plan) => Box::new(std::iter::once(plan.input.as_ref())), - PhysicalPlan::CopyIntoTable(v) => match &v.source { - CopyIntoTableSource::Query(v) => Box::new(std::iter::once(v.as_ref())), - CopyIntoTableSource::Stage(v) => Box::new(std::iter::once(v.as_ref())), - }, + PhysicalPlan::Append(v) => Box::new(std::iter::once(v.input.as_ref())), } } @@ -670,7 +648,6 @@ impl PhysicalPlan { PhysicalPlan::Limit(plan) => plan.input.try_find_single_data_source(), PhysicalPlan::Exchange(plan) => plan.input.try_find_single_data_source(), PhysicalPlan::ExchangeSink(plan) => plan.input.try_find_single_data_source(), - 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::Udf(plan) => plan.input.try_find_single_data_source(), @@ -686,8 +663,7 @@ impl PhysicalPlan { | PhysicalPlan::AggregatePartial(_) | PhysicalPlan::CompactSource(_) | PhysicalPlan::CommitSink(_) - | PhysicalPlan::CopyIntoTable(_) - | PhysicalPlan::ReplaceAsyncSourcer(_) + | PhysicalPlan::Append(_) | PhysicalPlan::ReplaceDeduplicate(_) | PhysicalPlan::ReplaceInto(_) | PhysicalPlan::MutationSource(_) @@ -711,6 +687,7 @@ impl PhysicalPlan { | PhysicalPlan::ChunkFillAndReorder(_) | PhysicalPlan::ChunkAppendData(_) | PhysicalPlan::ChunkMerge(_) + | PhysicalPlan::ValueScan(_) | PhysicalPlan::ChunkCommitInsert(_) => None, } } diff --git a/src/query/sql/src/executor/physical_plan_builder.rs b/src/query/sql/src/executor/physical_plan_builder.rs index e386d1d4b5d3..3aab6a6dab06 100644 --- a/src/query/sql/src/executor/physical_plan_builder.rs +++ b/src/query/sql/src/executor/physical_plan_builder.rs @@ -142,6 +142,8 @@ impl PhysicalPlanBuilder { } RelOperator::Recluster(recluster) => self.build_recluster(recluster).await, RelOperator::CompactBlock(compact) => self.build_compact_block(compact).await, + RelOperator::Append(append) => self.build_append(s_expr, append).await, + RelOperator::ValueScan(value_scan) => self.build_value_scan(value_scan).await, } } diff --git a/src/query/sql/src/executor/physical_plan_visitor.rs b/src/query/sql/src/executor/physical_plan_visitor.rs index dd86b4cda858..f08f6a23b53e 100644 --- a/src/query/sql/src/executor/physical_plan_visitor.rs +++ b/src/query/sql/src/executor/physical_plan_visitor.rs @@ -20,6 +20,7 @@ use super::physical_plans::ExpressionScan; use super::physical_plans::MutationManipulate; use super::physical_plans::MutationOrganize; use super::physical_plans::MutationSplit; +use super::physical_plans::PhysicalValueScan; use super::physical_plans::RecursiveCteScan; use crate::executor::physical_plan::PhysicalPlan; use crate::executor::physical_plans::AggregateExpand; @@ -38,10 +39,7 @@ use crate::executor::physical_plans::CommitSink; use crate::executor::physical_plans::CompactSource; use crate::executor::physical_plans::ConstantTableScan; use crate::executor::physical_plans::CopyIntoLocation; -use crate::executor::physical_plans::CopyIntoTable; -use crate::executor::physical_plans::CopyIntoTableSource; use crate::executor::physical_plans::CteScan; -use crate::executor::physical_plans::DistributedInsertSelect; use crate::executor::physical_plans::Duplicate; use crate::executor::physical_plans::EvalScalar; use crate::executor::physical_plans::Exchange; @@ -53,10 +51,10 @@ use crate::executor::physical_plans::Limit; use crate::executor::physical_plans::MaterializedCte; use crate::executor::physical_plans::Mutation; use crate::executor::physical_plans::MutationSource; +use crate::executor::physical_plans::PhysicalAppend; use crate::executor::physical_plans::ProjectSet; use crate::executor::physical_plans::RangeJoin; use crate::executor::physical_plans::Recluster; -use crate::executor::physical_plans::ReplaceAsyncSourcer; use crate::executor::physical_plans::ReplaceDeduplicate; use crate::executor::physical_plans::ReplaceInto; use crate::executor::physical_plans::RowFetch; @@ -89,14 +87,12 @@ pub trait PhysicalPlanReplacer { PhysicalPlan::ExchangeSource(plan) => self.replace_exchange_source(plan), PhysicalPlan::ExchangeSink(plan) => self.replace_exchange_sink(plan), PhysicalPlan::UnionAll(plan) => self.replace_union(plan), - PhysicalPlan::DistributedInsertSelect(plan) => self.replace_insert_select(plan), PhysicalPlan::ProjectSet(plan) => self.replace_project_set(plan), PhysicalPlan::CompactSource(plan) => self.replace_compact_source(plan), PhysicalPlan::CommitSink(plan) => self.replace_commit_sink(plan), PhysicalPlan::RangeJoin(plan) => self.replace_range_join(plan), - PhysicalPlan::CopyIntoTable(plan) => self.replace_copy_into_table(plan), + PhysicalPlan::Append(plan) => self.replace_append(plan), PhysicalPlan::CopyIntoLocation(plan) => self.replace_copy_into_location(plan), - PhysicalPlan::ReplaceAsyncSourcer(plan) => self.replace_async_sourcer(plan), PhysicalPlan::ReplaceDeduplicate(plan) => self.replace_deduplicate(plan), PhysicalPlan::ReplaceInto(plan) => self.replace_replace_into(plan), PhysicalPlan::MutationSource(plan) => self.replace_mutation_source(plan), @@ -122,9 +118,14 @@ pub trait PhysicalPlanReplacer { PhysicalPlan::ChunkAppendData(plan) => self.replace_chunk_append_data(plan), PhysicalPlan::ChunkMerge(plan) => self.replace_chunk_merge(plan), PhysicalPlan::ChunkCommitInsert(plan) => self.replace_chunk_commit_insert(plan), + PhysicalPlan::ValueScan(plan) => self.replace_value_scan(plan), } } + fn replace_value_scan(&mut self, plan: &PhysicalValueScan) -> Result { + Ok(PhysicalPlan::ValueScan(Box::new(plan.clone()))) + } + fn replace_recluster(&mut self, plan: &Recluster) -> Result { Ok(PhysicalPlan::Recluster(Box::new(plan.clone()))) } @@ -399,19 +400,14 @@ pub trait PhysicalPlanReplacer { })) } - fn replace_copy_into_table(&mut self, plan: &CopyIntoTable) -> Result { - match &plan.source { - CopyIntoTableSource::Stage(_) => { - Ok(PhysicalPlan::CopyIntoTable(Box::new(plan.clone()))) - } - CopyIntoTableSource::Query(query_physical_plan) => { - let input = self.replace(query_physical_plan)?; - Ok(PhysicalPlan::CopyIntoTable(Box::new(CopyIntoTable { - source: CopyIntoTableSource::Query(Box::new(input)), - ..plan.clone() - }))) - } - } + fn replace_append(&mut self, plan: &PhysicalAppend) -> Result { + let input = self.replace(&plan.input)?; + + Ok(PhysicalPlan::Append(Box::new(PhysicalAppend { + plan_id: plan.plan_id, + input: Box::new(input), + ..plan.clone() + }))) } fn replace_copy_into_location(&mut self, plan: &CopyIntoLocation) -> Result { @@ -426,22 +422,6 @@ pub trait PhysicalPlanReplacer { }))) } - fn replace_insert_select(&mut self, plan: &DistributedInsertSelect) -> Result { - let input = self.replace(&plan.input)?; - - Ok(PhysicalPlan::DistributedInsertSelect(Box::new( - DistributedInsertSelect { - plan_id: plan.plan_id, - input: Box::new(input), - table_info: plan.table_info.clone(), - select_schema: plan.select_schema.clone(), - insert_schema: plan.insert_schema.clone(), - select_column_bindings: plan.select_column_bindings.clone(), - cast_needed: plan.cast_needed, - }, - ))) - } - fn replace_compact_source(&mut self, plan: &CompactSource) -> Result { Ok(PhysicalPlan::CompactSource(Box::new(plan.clone()))) } @@ -454,10 +434,6 @@ pub trait PhysicalPlanReplacer { }))) } - fn replace_async_sourcer(&mut self, plan: &ReplaceAsyncSourcer) -> Result { - Ok(PhysicalPlan::ReplaceAsyncSourcer(plan.clone())) - } - fn replace_deduplicate(&mut self, plan: &ReplaceDeduplicate) -> Result { let input = self.replace(&plan.input)?; Ok(PhysicalPlan::ReplaceDeduplicate(Box::new( @@ -655,7 +631,6 @@ impl PhysicalPlan { visit(plan); match plan { PhysicalPlan::TableScan(_) - | PhysicalPlan::ReplaceAsyncSourcer(_) | PhysicalPlan::CteScan(_) | PhysicalPlan::RecursiveCteScan(_) | PhysicalPlan::ConstantTableScan(_) @@ -664,6 +639,7 @@ impl PhysicalPlan { | PhysicalPlan::Recluster(_) | PhysicalPlan::ExchangeSource(_) | PhysicalPlan::CompactSource(_) + | PhysicalPlan::ValueScan(_) | PhysicalPlan::MutationSource(_) => {} PhysicalPlan::Filter(plan) => { Self::traverse(&plan.input, pre_visit, visit, post_visit); @@ -709,20 +685,12 @@ impl PhysicalPlan { Self::traverse(&plan.left, pre_visit, visit, post_visit); Self::traverse(&plan.right, pre_visit, visit, post_visit); } - PhysicalPlan::DistributedInsertSelect(plan) => { - Self::traverse(&plan.input, pre_visit, visit, post_visit); - } PhysicalPlan::ProjectSet(plan) => { Self::traverse(&plan.input, pre_visit, visit, post_visit) } - PhysicalPlan::CopyIntoTable(plan) => match &plan.source { - CopyIntoTableSource::Query(input) => { - Self::traverse(input, pre_visit, visit, post_visit); - } - CopyIntoTableSource::Stage(input) => { - Self::traverse(input, pre_visit, visit, post_visit); - } - }, + PhysicalPlan::Append(plan) => { + Self::traverse(&plan.input, pre_visit, visit, post_visit) + } PhysicalPlan::CopyIntoLocation(plan) => { Self::traverse(&plan.input, pre_visit, visit, post_visit) } diff --git a/src/query/sql/src/executor/physical_plans/common.rs b/src/query/sql/src/executor/physical_plans/common.rs index 2670ca5e93ea..a50873a13400 100644 --- a/src/query/sql/src/executor/physical_plans/common.rs +++ b/src/query/sql/src/executor/physical_plans/common.rs @@ -71,7 +71,7 @@ pub enum FragmentKind { Merge, } -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize, Copy)] +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize, Copy, PartialEq, Eq)] pub enum MutationKind { Delete, Update, diff --git a/src/query/sql/src/executor/physical_plans/mod.rs b/src/query/sql/src/executor/physical_plans/mod.rs index 957443396364..9e22ab87e5ac 100644 --- a/src/query/sql/src/executor/physical_plans/mod.rs +++ b/src/query/sql/src/executor/physical_plans/mod.rs @@ -17,6 +17,7 @@ mod physical_add_stream_column; mod physical_aggregate_expand; mod physical_aggregate_final; mod physical_aggregate_partial; +mod physical_append; mod physical_async_func; mod physical_cache_scan; mod physical_column_mutation; @@ -24,9 +25,7 @@ mod physical_commit_sink; mod physical_compact_source; mod physical_constant_table_scan; mod physical_copy_into_location; -mod physical_copy_into_table; mod physical_cte_scan; -mod physical_distributed_insert_select; mod physical_eval_scalar; mod physical_exchange; mod physical_exchange_sink; @@ -56,6 +55,7 @@ mod physical_sort; mod physical_table_scan; mod physical_udf; mod physical_union_all; +mod physical_value_scan; mod physical_window; mod physical_window_partition; @@ -64,6 +64,7 @@ pub use physical_add_stream_column::AddStreamColumn; pub use physical_aggregate_expand::AggregateExpand; pub use physical_aggregate_final::AggregateFinal; pub use physical_aggregate_partial::AggregatePartial; +pub use physical_append::*; pub use physical_async_func::AsyncFunction; pub use physical_async_func::AsyncFunctionDesc; pub use physical_cache_scan::CacheScan; @@ -73,9 +74,7 @@ pub use physical_commit_sink::ReclusterInfoSideCar; pub use physical_compact_source::CompactSource; pub use physical_constant_table_scan::ConstantTableScan; pub use physical_copy_into_location::CopyIntoLocation; -pub use physical_copy_into_table::*; pub use physical_cte_scan::CteScan; -pub use physical_distributed_insert_select::DistributedInsertSelect; pub use physical_eval_scalar::EvalScalar; pub use physical_exchange::Exchange; pub use physical_exchange_sink::ExchangeSink; @@ -106,5 +105,6 @@ pub use physical_table_scan::TableScan; pub use physical_udf::Udf; pub use physical_udf::UdfFunctionDesc; pub use physical_union_all::UnionAll; +pub use physical_value_scan::*; pub use physical_window::*; pub use physical_window_partition::*; diff --git a/src/query/sql/src/executor/physical_plans/physical_copy_into_table.rs b/src/query/sql/src/executor/physical_plans/physical_append.rs similarity index 70% rename from src/query/sql/src/executor/physical_plans/physical_copy_into_table.rs rename to src/query/sql/src/executor/physical_plans/physical_append.rs index 1fd92f5c3ce5..bda5a68c4f20 100644 --- a/src/query/sql/src/executor/physical_plans/physical_copy_into_table.rs +++ b/src/query/sql/src/executor/physical_plans/physical_append.rs @@ -12,43 +12,28 @@ // See the License for the specific language governing permissions and // limitations under the License. -use databend_common_catalog::plan::StageTableInfo; use databend_common_exception::Result; use databend_common_expression::DataSchemaRef; use databend_common_expression::DataSchemaRefExt; use databend_common_expression::Scalar; use databend_common_meta_app::schema::TableInfo; -use enum_as_inner::EnumAsInner; use crate::executor::physical_plan::PhysicalPlan; -use crate::plans::CopyIntoTableMode; -use crate::plans::ValidationMode; use crate::ColumnBinding; #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct CopyIntoTable { +pub struct PhysicalAppend { pub plan_id: u32, + pub input: Box, pub required_values_schema: DataSchemaRef, pub values_consts: Vec, pub required_source_schema: DataSchemaRef, - pub write_mode: CopyIntoTableMode, - pub validation_mode: ValidationMode, - pub stage_table_info: StageTableInfo, pub table_info: TableInfo, - pub project_columns: Option>, - pub source: CopyIntoTableSource, - pub is_transform: bool, -} - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize, EnumAsInner)] -pub enum CopyIntoTableSource { - Query(Box), - Stage(Box), } -impl CopyIntoTable { +impl PhysicalAppend { pub fn output_schema(&self) -> Result { Ok(DataSchemaRefExt::create(vec![])) } diff --git a/src/query/sql/src/executor/physical_plans/physical_distributed_insert_select.rs b/src/query/sql/src/executor/physical_plans/physical_value_scan.rs similarity index 58% rename from src/query/sql/src/executor/physical_plans/physical_distributed_insert_select.rs rename to src/query/sql/src/executor/physical_plans/physical_value_scan.rs index abe0def948db..1eec0606a4ba 100644 --- a/src/query/sql/src/executor/physical_plans/physical_distributed_insert_select.rs +++ b/src/query/sql/src/executor/physical_plans/physical_value_scan.rs @@ -12,21 +12,27 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::sync::Arc; + +use databend_common_exception::Result; use databend_common_expression::DataSchemaRef; -use databend_common_meta_app::schema::TableInfo; +use databend_common_expression::Scalar; -use crate::executor::PhysicalPlan; -use crate::ColumnBinding; +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize, PartialEq, Eq, Hash)] +pub enum Values { + Values(Arc>>), + RawValues { rest_str: Arc, start: usize }, +} #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct DistributedInsertSelect { - /// A unique id of operator in a `PhysicalPlan` tree. +pub struct PhysicalValueScan { pub plan_id: u32, + pub values: Values, + pub output_schema: DataSchemaRef, +} - pub input: Box, - pub table_info: TableInfo, - pub insert_schema: DataSchemaRef, - pub select_schema: DataSchemaRef, - pub select_column_bindings: Vec, - pub cast_needed: bool, +impl PhysicalValueScan { + pub fn output_schema(&self) -> Result { + Ok(self.output_schema.clone()) + } } diff --git a/src/query/sql/src/planner/binder/bind_table_reference/bind_location.rs b/src/query/sql/src/planner/binder/bind_table_reference/bind_location.rs index 72168a47f458..22c8cb0d15d5 100644 --- a/src/query/sql/src/planner/binder/bind_table_reference/bind_location.rs +++ b/src/query/sql/src/planner/binder/bind_table_reference/bind_location.rs @@ -75,6 +75,7 @@ impl Binder { alias, None, options.case_sensitive.unwrap_or(false), + None, ) .await }) diff --git a/src/query/sql/src/planner/binder/copy_into_table.rs b/src/query/sql/src/planner/binder/copy_into_table.rs index aa6582b35078..9245f8b46832 100644 --- a/src/query/sql/src/planner/binder/copy_into_table.rs +++ b/src/query/sql/src/planner/binder/copy_into_table.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::str::FromStr; use std::sync::Arc; use databend_common_ast::ast::ColumnID as AstColumnID; @@ -37,6 +36,7 @@ use databend_common_ast::ast::TypeName; use databend_common_ast::parser::parse_values_with_placeholder; use databend_common_ast::parser::tokenize_sql; use databend_common_ast::Span; +use databend_common_catalog::catalog::CATALOG_DEFAULT; use databend_common_catalog::plan::list_stage_files; use databend_common_catalog::plan::StageTableInfo; use databend_common_catalog::table_context::StageAttachment; @@ -58,6 +58,7 @@ use databend_common_meta_app::principal::NullAs; use databend_common_meta_app::principal::StageInfo; use databend_common_meta_app::principal::COPY_MAX_FILES_PER_COMMIT; use databend_common_storage::StageFilesInfo; +use databend_common_storages_stage::StageTable; use databend_common_users::UserApiProvider; use derive_visitor::Drive; use log::debug; @@ -67,10 +68,10 @@ use parking_lot::RwLock; use crate::binder::bind_query::MaxColumnPosition; use crate::binder::location::parse_uri_location; use crate::binder::Binder; -use crate::plans::CopyIntoTableMode; -use crate::plans::CopyIntoTablePlan; +use crate::optimizer::SExpr; +use crate::plans::Append; +use crate::plans::AppendType; use crate::plans::Plan; -use crate::plans::ValidationMode; use crate::BindContext; use crate::Metadata; use crate::NameResolutionContext; @@ -83,16 +84,32 @@ impl<'a> Binder { bind_context: &mut BindContext, stmt: &CopyIntoTableStmt, ) -> Result { + let (catalog_name, database_name, table_name) = self.normalize_object_identifier_triple( + &stmt.dst.catalog, + &stmt.dst.database, + &stmt.dst.table, + ); match &stmt.src { CopyIntoTableSource::Location(location) => { - let mut plan = self - .bind_copy_into_table_common(bind_context, stmt, location, false) + let (copy_into_table_plan, mut stage_table_info) = self + .bind_copy_into_table_common(bind_context, stmt, location) .await?; - - // for copy from location, collect files explicitly - plan.collect_files(self.ctx.as_ref()).await?; - self.bind_copy_into_table_from_location(bind_context, plan) - .await + copy_into_table_plan + .collect_files( + self.ctx.as_ref(), + &mut stage_table_info, + &catalog_name, + &database_name, + &table_name, + ) + .await?; + self.bind_copy_into_table_from_location( + bind_context, + copy_into_table_plan, + stage_table_info, + AppendType::CopyInto, + ) + .await } CopyIntoTableSource::Query(query) => { self.init_cte(bind_context, &stmt.with)?; @@ -103,12 +120,27 @@ impl<'a> Binder { .write() .set_max_column_position(max_column_position.max_pos); let (select_list, location, alias) = check_transform_query(query)?; - let plan = self - .bind_copy_into_table_common(bind_context, stmt, location, true) + let (copy_into_table_plan, mut stage_table_info) = self + .bind_copy_into_table_common(bind_context, stmt, location) .await?; - self.bind_copy_from_query_into_table(bind_context, plan, select_list, alias) - .await + copy_into_table_plan + .collect_files( + self.ctx.as_ref(), + &mut stage_table_info, + &catalog_name, + &database_name, + &table_name, + ) + .await?; + self.bind_copy_from_query_into_table( + bind_context, + copy_into_table_plan, + stage_table_info, + select_list, + alias, + ) + .await } } } @@ -138,22 +170,27 @@ impl<'a> Binder { bind_context: &mut BindContext, stmt: &CopyIntoTableStmt, location: &FileLocation, - is_transform: bool, - ) -> Result { + ) -> Result<(Append, StageTableInfo)> { let (catalog_name, database_name, table_name) = self.normalize_object_identifier_triple( &stmt.dst.catalog, &stmt.dst.database, &stmt.dst.table, ); - let catalog = self.ctx.get_catalog(&catalog_name).await?; - let catalog_info = catalog.info(); let table = self .ctx .get_table(&catalog_name, &database_name, &table_name) .await?; - let validation_mode = ValidationMode::from_str(stmt.options.validation_mode.as_str()) - .map_err(ErrorCode::SyntaxException)?; + let table_index = self.metadata.write().add_table( + catalog_name, + database_name, + table.clone(), + None, + false, + false, + false, + false, + ); let (mut stage_info, path) = resolve_file_location(self.ctx.as_ref(), location).await?; if !stmt.file_format.is_empty() { @@ -201,32 +238,26 @@ impl<'a> Binder { None }; - Ok(CopyIntoTablePlan { - catalog_info, - database_name, - table_name, - validation_mode, - is_transform, - no_file_to_copy: false, - from_attachment: false, - stage_table_info: StageTableInfo { - schema: stage_schema, - files_info, - stage_info, - files_to_copy: None, - duplicated_files_detected: vec![], - is_select: false, - default_values, - copy_into_location_options: Default::default(), - copy_into_table_options: stmt.options.clone(), - }, + let stage_table_info = StageTableInfo { + schema: stage_schema, + files_info, + stage_info, + files_to_copy: None, + duplicated_files_detected: vec![], + is_select: false, + default_values, + copy_into_location_options: Default::default(), + copy_into_table_options: stmt.options.clone(), + }; + + let copy_into_plan = Append { + table_index, values_consts: vec![], required_source_schema: required_values_schema.clone(), required_values_schema: required_values_schema.clone(), - write_mode: CopyIntoTableMode::Copy, - query: None, - enable_distributed: false, - }) + project_columns: None, + }; + Ok((copy_into_plan, stage_table_info)) } /// Bind COPY INFO FROM @@ -234,19 +265,20 @@ impl<'a> Binder { async fn bind_copy_into_table_from_location( &mut self, bind_ctx: &BindContext, - plan: CopyIntoTablePlan, + mut append_plan: Append, + stage_table_info: StageTableInfo, + append_type: AppendType, ) -> Result { - let use_query = matches!(&plan.stage_table_info.stage_info.file_format_params, + let use_query = matches!(&stage_table_info.stage_info.file_format_params, FileFormatParams::Parquet(fmt) if fmt.missing_field_as == NullAs::Error); + let case_sensitive = stage_table_info.copy_into_table_options.column_match_mode + == Some(ColumnMatchMode::CaseSensitive); if use_query { - let mut select_list = Vec::with_capacity(plan.required_source_schema.num_fields()); - let case_sensitive = plan - .stage_table_info - .copy_into_table_options - .column_match_mode - == Some(ColumnMatchMode::CaseSensitive); - for dest_field in plan.required_source_schema.fields().iter() { + let mut select_list = + Vec::with_capacity(append_plan.required_source_schema.num_fields()); + + for dest_field in append_plan.required_source_schema.fields().iter() { let column = Expr::ColumnRef { span: None, column: ColumnRef { @@ -283,10 +315,42 @@ impl<'a> Binder { }); } - self.bind_copy_from_query_into_table(bind_ctx, plan, &select_list, &None) - .await + self.bind_copy_from_query_into_table( + bind_ctx, + append_plan, + stage_table_info, + &select_list, + &None, + ) + .await } else { - Ok(Plan::CopyIntoTable(Box::new(plan))) + let table = StageTable::try_create(stage_table_info.clone())?; + let table_index = self.metadata.write().add_table( + CATALOG_DEFAULT.to_string(), + "system".to_string(), + table.clone(), + None, + false, + false, + true, + false, + ); + + let (scan, bind_context) = + self.bind_base_table(bind_ctx, "system", table_index, None, &None)?; + append_plan.project_columns = Some(bind_context.columns.clone()); + let target_table_index = append_plan.table_index; + + let s_expr = SExpr::create_unary(Arc::new(append_plan.into()), Arc::new(scan)); + Ok(Plan::Append { + s_expr: Box::new(s_expr), + target_table_index, + metadata: self.metadata.clone(), + stage_table_info: Some(Box::new(stage_table_info)), + overwrite: false, + forbid_occ_retry: false, + append_type, + }) } } @@ -345,7 +409,6 @@ impl<'a> Binder { table_name: String, required_values_schema: DataSchemaRef, values_str: &str, - write_mode: CopyIntoTableMode, ) -> Result { let (data_schema, const_columns) = if values_str.is_empty() { (required_values_schema.clone(), vec![]) @@ -354,9 +417,6 @@ impl<'a> Binder { .await? }; - let catalog = self.ctx.get_catalog(&catalog_name).await?; - let catalog_info = catalog.info(); - let thread_num = self.ctx.get_settings().get_max_threads()? as usize; let (stage_info, files_info, options) = self.bind_attachment(attachment).await?; @@ -377,36 +437,49 @@ impl<'a> Binder { .prepare_default_values(bind_context, &data_schema) .await?; - let plan = CopyIntoTablePlan { - catalog_info, + let table = self + .ctx + .get_table(&catalog_name, &database_name, &table_name) + .await?; + + let table_index = self.metadata.write().add_table( + catalog_name, database_name, - table_name, - no_file_to_copy: false, - from_attachment: true, - required_source_schema: data_schema.clone(), + table, + None, + false, + false, + false, + false, + ); + + let stage_table_info = StageTableInfo { + schema: stage_schema, + files_info, + stage_info, + files_to_copy: Some(files_to_copy), + duplicated_files_detected, + is_select: false, + default_values: Some(default_values), + copy_into_location_options: Default::default(), + copy_into_table_options: options, + }; + + let copy_into_table_plan = Append { + table_index, required_values_schema, values_consts: const_columns, - stage_table_info: StageTableInfo { - schema: stage_schema, - files_info, - stage_info, - files_to_copy: Some(files_to_copy), - duplicated_files_detected, - is_select: false, - default_values: Some(default_values), - copy_into_location_options: Default::default(), - copy_into_table_options: options, - }, - write_mode, - query: None, - validation_mode: ValidationMode::None, - - enable_distributed: false, - is_transform: false, + required_source_schema: data_schema.clone(), + project_columns: None, }; - self.bind_copy_into_table_from_location(bind_context, plan) - .await + self.bind_copy_into_table_from_location( + bind_context, + copy_into_table_plan, + stage_table_info, + AppendType::Insert, + ) + .await } /// Bind COPY INTO
FROM @@ -414,30 +487,24 @@ impl<'a> Binder { async fn bind_copy_from_query_into_table( &mut self, bind_context: &BindContext, - mut plan: CopyIntoTablePlan, + mut copy_into_table_plan: Append, + stage_table_info: StageTableInfo, select_list: &'a [SelectTarget], alias: &Option, ) -> Result { - plan.collect_files(self.ctx.as_ref()).await?; - if plan.no_file_to_copy { - return Ok(Plan::CopyIntoTable(Box::new(plan))); - } - let case_sensitive = plan - .stage_table_info - .copy_into_table_options - .column_match_mode - == Some(ColumnMatchMode::CaseSensitive); - let table_ctx = self.ctx.clone(); + let case_sensitive = stage_table_info.copy_into_table_options.column_match_mode + == Some(ColumnMatchMode::CaseSensitive); let (s_expr, mut from_context) = self .bind_stage_table( table_ctx, bind_context, - plan.stage_table_info.stage_info.clone(), - plan.stage_table_info.files_info.clone(), + stage_table_info.stage_info.clone(), + stage_table_info.files_info.clone(), alias, - plan.stage_table_info.files_to_copy.clone(), + stage_table_info.files_to_copy.clone(), case_sensitive, + Some(stage_table_info.schema.clone()), ) .await?; @@ -459,14 +526,16 @@ impl<'a> Binder { &select_list, )?; - if projections.len() != plan.required_source_schema.num_fields() { + if projections.len() != copy_into_table_plan.required_source_schema.num_fields() { return Err(ErrorCode::BadArguments(format!( "Number of columns in select list ({}) does not match that of the corresponding table ({})", projections.len(), - plan.required_source_schema.num_fields(), + copy_into_table_plan.required_source_schema.num_fields(), ))); } + copy_into_table_plan.project_columns = Some(projections.clone()); + let mut s_expr = self.bind_projection(&mut from_context, &projections, &scalar_items, s_expr)?; @@ -478,8 +547,7 @@ impl<'a> Binder { output_context.columns = from_context.columns; // disable variant check to allow copy invalid JSON into tables - let disable_variant_check = plan - .stage_table_info + let disable_variant_check = stage_table_info .copy_into_table_options .disable_variant_check; if disable_variant_check { @@ -500,16 +568,19 @@ impl<'a> Binder { } } - plan.query = Some(Box::new(Plan::Query { - s_expr: Box::new(s_expr), - metadata: self.metadata.clone(), - bind_context: Box::new(output_context), - rewrite_kind: None, - ignore_result: false, - formatted_ast: None, - })); + let target_table_index = copy_into_table_plan.table_index; + let copy_into = + SExpr::create_unary(Arc::new(copy_into_table_plan.into()), Arc::new(s_expr)); - Ok(Plan::CopyIntoTable(Box::new(plan))) + Ok(Plan::Append { + s_expr: Box::new(copy_into), + target_table_index, + metadata: self.metadata.clone(), + stage_table_info: Some(Box::new(stage_table_info)), + overwrite: false, + forbid_occ_retry: false, + append_type: AppendType::CopyInto, + }) } #[async_backtrace::framed] diff --git a/src/query/sql/src/planner/binder/insert.rs b/src/query/sql/src/planner/binder/insert.rs index c830878ea7e0..c54e9aa03f75 100644 --- a/src/query/sql/src/planner/binder/insert.rs +++ b/src/query/sql/src/planner/binder/insert.rs @@ -17,20 +17,21 @@ use std::sync::Arc; use databend_common_ast::ast::Identifier; use databend_common_ast::ast::InsertSource; use databend_common_ast::ast::InsertStmt; -use databend_common_ast::ast::Statement; use databend_common_exception::ErrorCode; use databend_common_exception::Result; +use databend_common_expression::DataSchemaRef; use databend_common_expression::TableSchema; use databend_common_expression::TableSchemaRefExt; use super::util::TableIdentifier; use crate::binder::Binder; +use crate::executor::physical_plans::Values; use crate::normalize_identifier; -use crate::plans::CopyIntoTableMode; -use crate::plans::Insert; -use crate::plans::InsertInputSource; -use crate::plans::InsertValue; +use crate::optimizer::SExpr; +use crate::plans::Append; +use crate::plans::AppendType; use crate::plans::Plan; +use crate::plans::ValueScan; use crate::BindContext; impl Binder { @@ -99,16 +100,28 @@ impl Binder { .await .map_err(|err| table_identifier.not_found_suggest_error(err))?; + let table_index = self.metadata.write().add_table( + catalog_name.clone(), + database_name.clone(), + table.clone(), + None, + false, + false, + false, + false, + ); + let schema = self.schema_project(&table.schema(), columns)?; + let schema: DataSchemaRef = Arc::new(schema.into()); - let input_source: Result = match source.clone() { + let (source, project_columns) = match source.clone() { InsertSource::Values { rows } => { let mut new_rows = Vec::with_capacity(rows.len()); for row in rows { let new_row = bind_context .exprs_to_scalar( &row, - &Arc::new(schema.clone().into()), + &schema, self.ctx.clone(), &self.name_resolution_ctx, self.metadata.clone(), @@ -116,9 +129,16 @@ impl Binder { .await?; new_rows.push(new_row); } - Ok(InsertInputSource::Values(InsertValue::Values { - rows: new_rows, - })) + ( + SExpr::create_leaf(Arc::new( + ValueScan { + values: Values::Values(Arc::new(new_rows)), + dest_schema: schema.clone(), + } + .into(), + )), + None, + ) } InsertSource::RawValues { rest_str, start } => { let values_str = rest_str.trim_end_matches(';').trim_start().to_owned(); @@ -131,37 +151,51 @@ impl Binder { catalog_name, database_name, table_name, - Arc::new(schema.into()), + schema, &values_str, - CopyIntoTableMode::Insert { - overwrite: *overwrite, - }, ) .await; } - None => Ok(InsertInputSource::Values(InsertValue::RawValues { - data: rest_str, - start, - })), + None => ( + SExpr::create_leaf(Arc::new( + ValueScan { + values: Values::RawValues { + rest_str: Arc::new(values_str), + start, + }, + dest_schema: schema.clone(), + } + .into(), + )), + None, + ), } } InsertSource::Select { query } => { - let statement = Statement::Query(query); - let select_plan = self.bind_statement(bind_context, &statement).await?; - Ok(InsertInputSource::SelectPlan(Box::new(select_plan))) + let (source, bind_context) = self.bind_query(bind_context, &query)?; + (source, Some(bind_context.columns.clone())) } }; - let plan = Insert { - catalog: catalog_name.to_string(), - database: database_name.to_string(), - table: table_name, - schema, - overwrite: *overwrite, - source: input_source?, - table_info: None, + let copy_into = Append { + table_index, + required_values_schema: schema.clone(), + values_consts: vec![], + required_source_schema: schema, + project_columns, }; - Ok(Plan::Insert(Box::new(plan))) + Ok(Plan::Append { + s_expr: Box::new(SExpr::create_unary( + Arc::new(copy_into.into()), + Arc::new(source), + )), + metadata: self.metadata.clone(), + stage_table_info: None, + overwrite: *overwrite, + forbid_occ_retry: false, + append_type: AppendType::Insert, + target_table_index: table_index, + }) } } diff --git a/src/query/sql/src/planner/binder/replace.rs b/src/query/sql/src/planner/binder/replace.rs index c467ea8a6d1c..5aa37952b2a9 100644 --- a/src/query/sql/src/planner/binder/replace.rs +++ b/src/query/sql/src/planner/binder/replace.rs @@ -22,7 +22,6 @@ use databend_common_exception::Result; use crate::binder::Binder; use crate::normalize_identifier; -use crate::plans::CopyIntoTableMode; use crate::plans::InsertInputSource; use crate::plans::InsertValue; use crate::plans::Plan; @@ -121,7 +120,6 @@ impl Binder { table_name.clone(), Arc::new(schema.clone().into()), &values_str, - CopyIntoTableMode::Replace, ) .await?; Ok(InsertInputSource::Stage(Box::new(plan))) diff --git a/src/query/sql/src/planner/binder/table.rs b/src/query/sql/src/planner/binder/table.rs index a7560c519ab5..135d7a320480 100644 --- a/src/query/sql/src/planner/binder/table.rs +++ b/src/query/sql/src/planner/binder/table.rs @@ -31,6 +31,7 @@ use databend_common_ast::ast::TemporalClause; use databend_common_ast::ast::TimeTravelPoint; use databend_common_ast::Span; use databend_common_catalog::catalog_kind::CATALOG_DEFAULT; +use databend_common_catalog::plan::StageTableInfo; use databend_common_catalog::table::NavigationPoint; use databend_common_catalog::table::Table; use databend_common_catalog::table::TimeNavigation; @@ -45,6 +46,7 @@ use databend_common_expression::types::NumberDataType; use databend_common_expression::ConstantFolder; use databend_common_expression::DataField; use databend_common_expression::FunctionContext; +use databend_common_expression::TableSchemaRef; use databend_common_functions::BUILTIN_FUNCTIONS; use databend_common_meta_app::principal::StageInfo; use databend_common_meta_app::schema::IndexMeta; @@ -53,6 +55,7 @@ use databend_common_meta_app::tenant::Tenant; use databend_common_meta_types::MetaId; use databend_common_storage::StageFileInfo; use databend_common_storage::StageFilesInfo; +use databend_common_storages_stage::StageTable; use databend_storages_common_table_meta::table::ChangeType; use log::info; use parking_lot::RwLock; @@ -116,18 +119,34 @@ impl Binder { alias: &Option, files_to_copy: Option>, case_sensitive: bool, + inferred_schema: Option, ) -> Result<(SExpr, BindContext)> { let start = std::time::Instant::now(); let max_column_position = self.metadata.read().get_max_column_position(); - let table = table_ctx - .create_stage_table( - stage_info, + let table = match files_to_copy.as_ref().is_some_and(|files| files.is_empty()) { + true => StageTable::try_create(StageTableInfo { + schema: inferred_schema.unwrap(), + default_values: None, files_info, + stage_info, files_to_copy, - max_column_position, - case_sensitive, - ) - .await?; + duplicated_files_detected: vec![], + is_select: false, + copy_into_location_options: Default::default(), + copy_into_table_options: Default::default(), + })?, + false => { + table_ctx + .create_stage_table( + stage_info, + files_info, + files_to_copy, + max_column_position, + case_sensitive, + ) + .await? + } + }; let table_alias_name = if let Some(table_alias) = alias { Some(normalize_identifier(&table_alias.name, &self.name_resolution_ctx).name) diff --git a/src/query/sql/src/planner/binder/util.rs b/src/query/sql/src/planner/binder/util.rs index f082224649b4..ee86f261b8b7 100644 --- a/src/query/sql/src/planner/binder/util.rs +++ b/src/query/sql/src/planner/binder/util.rs @@ -88,6 +88,8 @@ impl Binder { | RelOperator::Mutation(_) | RelOperator::Recluster(_) | RelOperator::MutationSource(_) + | RelOperator::Append(_) + | RelOperator::ValueScan(_) | RelOperator::CompactBlock(_) => { return Err(ErrorCode::SyntaxException(format!( "{:?} is not allowed in recursive cte", diff --git a/src/query/sql/src/planner/format/display_plan.rs b/src/query/sql/src/planner/format/display_plan.rs index cbb89f162eae..1e37b2757fa5 100644 --- a/src/query/sql/src/planner/format/display_plan.rs +++ b/src/query/sql/src/planner/format/display_plan.rs @@ -40,7 +40,7 @@ impl Plan { Plan::ExplainSyntax { .. } => Ok("ExplainSyntax".to_string()), Plan::ExplainAnalyze { .. } => Ok("ExplainAnalyze".to_string()), - Plan::CopyIntoTable(_) => Ok("CopyIntoTable".to_string()), + Plan::Append { .. } => Ok("Append".to_string()), Plan::CopyIntoLocation(_) => Ok("CopyIntoLocation".to_string()), // catalog @@ -110,8 +110,6 @@ impl Plan { Plan::DropVirtualColumn(_) => Ok("DropVirtualColumn".to_string()), Plan::RefreshVirtualColumn(_) => Ok("RefreshVirtualColumn".to_string()), - // Insert - Plan::Insert(_) => Ok("Insert".to_string()), Plan::InsertMultiTable(_) => Ok("InsertMultiTable".to_string()), Plan::Replace(_) => Ok("Replace".to_string()), Plan::DataMutation { s_expr, .. } => format_merge_into(s_expr), diff --git a/src/query/sql/src/planner/optimizer/decorrelate/subquery_rewriter.rs b/src/query/sql/src/planner/optimizer/decorrelate/subquery_rewriter.rs index 5a2b34ceec35..71ef006caaf9 100644 --- a/src/query/sql/src/planner/optimizer/decorrelate/subquery_rewriter.rs +++ b/src/query/sql/src/planner/optimizer/decorrelate/subquery_rewriter.rs @@ -200,6 +200,8 @@ impl SubqueryRewriter { | RelOperator::Mutation(_) | RelOperator::MutationSource(_) | RelOperator::Recluster(_) + | RelOperator::Append(_) + | RelOperator::ValueScan(_) | RelOperator::CompactBlock(_) => Ok(s_expr.clone()), } } diff --git a/src/query/sql/src/planner/optimizer/dynamic_sample/dynamic_sample.rs b/src/query/sql/src/planner/optimizer/dynamic_sample/dynamic_sample.rs index a2363d69029b..324b9481988d 100644 --- a/src/query/sql/src/planner/optimizer/dynamic_sample/dynamic_sample.rs +++ b/src/query/sql/src/planner/optimizer/dynamic_sample/dynamic_sample.rs @@ -94,6 +94,8 @@ pub async fn dynamic_sample( | RelOperator::Mutation(_) | RelOperator::Recluster(_) | RelOperator::CompactBlock(_) + | RelOperator::Append(_) + | RelOperator::ValueScan(_) | RelOperator::MutationSource(_) => { s_expr.plan().derive_stats(&RelExpr::with_s_expr(s_expr)) } diff --git a/src/query/sql/src/planner/optimizer/format.rs b/src/query/sql/src/planner/optimizer/format.rs index b72f217ae4cb..16304ecedbdc 100644 --- a/src/query/sql/src/planner/optimizer/format.rs +++ b/src/query/sql/src/planner/optimizer/format.rs @@ -77,6 +77,8 @@ pub fn display_rel_op(rel_op: &RelOperator) -> String { RelOperator::MutationSource(_) => "MutationSource".to_string(), RelOperator::Recluster(_) => "Recluster".to_string(), RelOperator::CompactBlock(_) => "CompactBlock".to_string(), + RelOperator::Append(_) => "Append".to_string(), + RelOperator::ValueScan(_) => "ValueScan".to_string(), } } diff --git a/src/query/sql/src/planner/optimizer/hyper_dp/dphyp.rs b/src/query/sql/src/planner/optimizer/hyper_dp/dphyp.rs index bd968b999508..6cfff12823fa 100644 --- a/src/query/sql/src/planner/optimizer/hyper_dp/dphyp.rs +++ b/src/query/sql/src/planner/optimizer/hyper_dp/dphyp.rs @@ -299,7 +299,9 @@ impl DPhpy { | RelOperator::Mutation(_) | RelOperator::MutationSource(_) | RelOperator::Recluster(_) - | RelOperator::CompactBlock(_) => Ok((Arc::new(s_expr.clone()), true)), + | RelOperator::CompactBlock(_) + | RelOperator::Append(_) + | RelOperator::ValueScan(_) => Ok((Arc::new(s_expr.clone()), true)), } } diff --git a/src/query/sql/src/planner/optimizer/mod.rs b/src/query/sql/src/planner/optimizer/mod.rs index d60fc7801671..5d5f3c61ed03 100644 --- a/src/query/sql/src/planner/optimizer/mod.rs +++ b/src/query/sql/src/planner/optimizer/mod.rs @@ -43,6 +43,7 @@ pub use hyper_dp::DPhpy; pub use m_expr::MExpr; pub use memo::Memo; pub use optimizer::optimize; +pub use optimizer::optimize_append; pub use optimizer::optimize_query; pub use optimizer::OptimizerContext; pub use optimizer::RecursiveOptimizer; diff --git a/src/query/sql/src/planner/optimizer/optimizer.rs b/src/query/sql/src/planner/optimizer/optimizer.rs index de1ea299a3fd..2a14cceeb9e8 100644 --- a/src/query/sql/src/planner/optimizer/optimizer.rs +++ b/src/query/sql/src/planner/optimizer/optimizer.rs @@ -46,7 +46,9 @@ use crate::optimizer::RuleID; use crate::optimizer::SExpr; use crate::optimizer::DEFAULT_REWRITE_RULES; use crate::planner::query_executor::QueryExecutor; +use crate::plans::Append; use crate::plans::CopyIntoLocationPlan; +use crate::plans::Exchange; use crate::plans::Join; use crate::plans::JoinType; use crate::plans::MatchedEvaluator; @@ -286,40 +288,35 @@ pub async fn optimize(mut opt_ctx: OptimizerContext, plan: Plan) -> Result from: Box::new(Box::pin(optimize(opt_ctx, *from)).await?), options, })), - Plan::CopyIntoTable(mut plan) if !plan.no_file_to_copy => { - plan.enable_distributed = opt_ctx.enable_distributed_optimization - && opt_ctx - .table_ctx - .get_settings() - .get_enable_distributed_copy()?; - info!( - "after optimization enable_distributed_copy? : {}", - plan.enable_distributed - ); - - if let Some(p) = &plan.query { - let optimized_plan = optimize(opt_ctx.clone(), *p.clone()).await?; - plan.query = Some(Box::new(optimized_plan)); - } - Ok(Plan::CopyIntoTable(plan)) + Plan::Append { + s_expr, + metadata, + stage_table_info, + overwrite, + forbid_occ_retry, + append_type, + target_table_index, + } => { + let append: Append = s_expr.plan().clone().try_into()?; + let source = s_expr.child(0)?.clone(); + let optimized_source = optimize_query(&mut opt_ctx, source).await?; + let optimized_append = optimize_append( + append, + optimized_source, + metadata.clone(), + opt_ctx.table_ctx.as_ref(), + )?; + Ok(Plan::Append { + s_expr: Box::new(optimized_append), + metadata, + stage_table_info, + overwrite, + forbid_occ_retry, + append_type, + target_table_index, + }) } Plan::DataMutation { s_expr, .. } => optimize_mutation(opt_ctx, *s_expr).await, - - // distributed insert will be optimized in `physical_plan_builder` - Plan::Insert(mut plan) => { - match plan.source { - InsertInputSource::SelectPlan(p) => { - let optimized_plan = optimize(opt_ctx.clone(), *p.clone()).await?; - plan.source = InsertInputSource::SelectPlan(Box::new(optimized_plan)); - } - InsertInputSource::Stage(p) => { - let optimized_plan = optimize(opt_ctx.clone(), *p.clone()).await?; - plan.source = InsertInputSource::Stage(Box::new(optimized_plan)); - } - _ => {} - } - Ok(Plan::Insert(plan)) - } Plan::InsertMultiTable(mut plan) => { plan.input_source = optimize(opt_ctx.clone(), plan.input_source.clone()).await?; Ok(Plan::InsertMultiTable(plan)) @@ -590,3 +587,37 @@ async fn optimize_mutation(mut opt_ctx: OptimizerContext, s_expr: SExpr) -> Resu metadata: opt_ctx.metadata.clone(), }) } + +pub fn optimize_append( + append: Append, + source: SExpr, + metadata: MetadataRef, + table_ctx: &dyn TableContext, +) -> Result { + let support_distributed_insert = { + let metadata = metadata.read(); + metadata + .table(append.table_index) + .table() + .support_distributed_insert() + }; + let enable_distributed = table_ctx.get_settings().get_enable_distributed_copy()? + && support_distributed_insert + && matches!(source.plan(), RelOperator::Exchange(Exchange::Merge)); + info!( + "after optimization enable_distributed_copy? : {}", + enable_distributed + ); + match enable_distributed { + true => { + let source = source.child(0).unwrap().clone(); + let copy_into = SExpr::create_unary(Arc::new(append.into()), Arc::new(source)); + let exchange = Arc::new(RelOperator::Exchange(Exchange::Merge)); + Ok(SExpr::create_unary(exchange, Arc::new(copy_into))) + } + false => Ok(SExpr::create_unary( + Arc::new(append.into()), + Arc::new(source), + )), + } +} diff --git a/src/query/sql/src/planner/optimizer/rule/rewrite/rule_semi_to_inner_join.rs b/src/query/sql/src/planner/optimizer/rule/rewrite/rule_semi_to_inner_join.rs index 29dfe4872f60..bc53ff25c6ef 100644 --- a/src/query/sql/src/planner/optimizer/rule/rewrite/rule_semi_to_inner_join.rs +++ b/src/query/sql/src/planner/optimizer/rule/rewrite/rule_semi_to_inner_join.rs @@ -142,7 +142,9 @@ fn find_group_by_keys(child: &SExpr, group_by_keys: &mut HashSet) -> | RelOperator::Mutation(_) | RelOperator::MutationSource(_) | RelOperator::Recluster(_) - | RelOperator::CompactBlock(_) => {} + | RelOperator::CompactBlock(_) + | RelOperator::Append(_) + | RelOperator::ValueScan(_) => {} } Ok(()) } diff --git a/src/query/sql/src/planner/optimizer/s_expr.rs b/src/query/sql/src/planner/optimizer/s_expr.rs index d56f921e5053..0e54a0dfb690 100644 --- a/src/query/sql/src/planner/optimizer/s_expr.rs +++ b/src/query/sql/src/planner/optimizer/s_expr.rs @@ -340,6 +340,8 @@ impl SExpr { | RelOperator::RecursiveCteScan(_) | RelOperator::Mutation(_) | RelOperator::Recluster(_) + | RelOperator::Append(_) + | RelOperator::ValueScan(_) | RelOperator::CompactBlock(_) => {} }; for child in &self.children { @@ -441,6 +443,8 @@ fn find_subquery(rel_op: &RelOperator) -> bool { | RelOperator::RecursiveCteScan(_) | RelOperator::Mutation(_) | RelOperator::Recluster(_) + | RelOperator::Append(_) + | RelOperator::ValueScan(_) | RelOperator::CompactBlock(_) => false, RelOperator::Join(op) => { op.equi_conditions.iter().any(|condition| { diff --git a/src/query/sql/src/planner/optimizer/util.rs b/src/query/sql/src/planner/optimizer/util.rs index 8120893a8cec..657e1c8d562e 100644 --- a/src/query/sql/src/planner/optimizer/util.rs +++ b/src/query/sql/src/planner/optimizer/util.rs @@ -27,4 +27,5 @@ pub fn contains_local_table_scan(s_expr: &SExpr, metadata: &MetadataRef) -> bool false } || matches!(s_expr.plan(), RelOperator::RecursiveCteScan { .. }) + || matches!(s_expr.plan(), RelOperator::ValueScan { .. }) } diff --git a/src/query/sql/src/planner/plans/copy_into_table.rs b/src/query/sql/src/planner/plans/append.rs similarity index 50% rename from src/query/sql/src/planner/plans/copy_into_table.rs rename to src/query/sql/src/planner/plans/append.rs index 75f30f612dce..c99ea9c28578 100644 --- a/src/query/sql/src/planner/plans/copy_into_table.rs +++ b/src/query/sql/src/planner/plans/append.rs @@ -13,13 +13,13 @@ // limitations under the License. use std::fmt::Debug; -use std::fmt::Display; use std::fmt::Formatter; -use std::str::FromStr; +use std::hash::Hash; use std::sync::Arc; use std::time::Instant; use databend_common_catalog::plan::StageTableInfo; +use databend_common_catalog::table::Table; use databend_common_catalog::table_context::FilteredCopyFiles; use databend_common_catalog::table_context::TableContext; use databend_common_exception::ErrorCode; @@ -33,118 +33,113 @@ use databend_common_expression::DataSchemaRefExt; use databend_common_expression::Scalar; use databend_common_meta_app::principal::COPY_MAX_FILES_COMMIT_MSG; use databend_common_meta_app::principal::COPY_MAX_FILES_PER_COMMIT; -use databend_common_meta_app::schema::CatalogInfo; use databend_common_metrics::storage::*; use databend_common_storage::init_stage_operator; use log::info; -use crate::plans::Plan; - -#[derive(PartialEq, Eq, Clone, Debug, serde::Serialize, serde::Deserialize)] -pub enum ValidationMode { - None, - ReturnNRows(u64), - ReturnErrors, - ReturnAllErrors, -} - -impl Display for ValidationMode { - fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { - match self { - ValidationMode::None => write!(f, ""), - ValidationMode::ReturnNRows(v) => write!(f, "RETURN_ROWS={v}"), - ValidationMode::ReturnErrors => write!(f, "RETURN_ERRORS"), - ValidationMode::ReturnAllErrors => write!(f, "RETURN_ALL_ERRORS"), - } - } -} - -impl FromStr for ValidationMode { - type Err = String; - fn from_str(s: &str) -> std::result::Result { - match s.to_uppercase().as_str() { - "" => Ok(ValidationMode::None), - "RETURN_ERRORS" => Ok(ValidationMode::ReturnErrors), - "RETURN_ALL_ERRORS" => Ok(ValidationMode::ReturnAllErrors), - v => { - let rows_str = v.replace("RETURN_", "").replace("_ROWS", ""); - let rows = rows_str.parse::(); - match rows { - Ok(v) => Ok(ValidationMode::ReturnNRows(v)), - Err(_) => Err(format!( - "Unknown validation mode:{v:?}, must one of {{ RETURN__ROWS | RETURN_ERRORS | RETURN_ALL_ERRORS}}" - )), - } - } - } - } +use super::Operator; +use super::Plan; +use super::RelOp; +use crate::executor::physical_plans::PhysicalAppend; +use crate::executor::PhysicalPlan; +use crate::executor::PhysicalPlanBuilder; +use crate::optimizer::optimize_append; +use crate::optimizer::SExpr; +use crate::ColumnBinding; +use crate::IndexType; +use crate::MetadataRef; + +#[derive(Clone, PartialEq, Eq)] +pub struct Append { + // Use table index instead of catalog_name, database_name, table_name here, means once a logic plan is built, + // the target table is determined, and we won't call get_table() again. + pub table_index: IndexType, + pub required_values_schema: DataSchemaRef, + pub values_consts: Vec, + pub required_source_schema: DataSchemaRef, + pub project_columns: Option>, } -#[derive(Clone, Copy, Eq, PartialEq, Debug, serde::Serialize, serde::Deserialize)] -pub enum CopyIntoTableMode { - Insert { overwrite: bool }, - Replace, - Copy, +#[derive(Clone, Debug)] +pub enum AppendType { + Insert, + CopyInto, } -impl Display for CopyIntoTableMode { - fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { - match self { - CopyIntoTableMode::Insert { overwrite } => { - if *overwrite { - write!(f, "INSERT OVERWRITE") - } else { - write!(f, "INSERT") - } - } - CopyIntoTableMode::Replace => write!(f, "REPLACE"), - CopyIntoTableMode::Copy => write!(f, "COPY"), - } - } +pub async fn create_append_plan_from_subquery( + subquery: &Plan, + catalog_name: String, + database_name: String, + table: Arc, + target_schema: DataSchemaRef, + forbid_occ_retry: bool, + ctx: Arc, + overwrite: bool, +) -> Result { + let (project_columns, source, metadata) = match subquery { + Plan::Query { + bind_context, + s_expr, + metadata, + .. + } => ( + Some(bind_context.columns.clone()), + *s_expr.clone(), + metadata.clone(), + ), + _ => unreachable!(), + }; + + let table_index = metadata.write().add_table( + catalog_name, + database_name, + table, + None, + false, + false, + false, + false, + ); + + let insert_plan = Append { + table_index, + required_values_schema: target_schema.clone(), + values_consts: vec![], + required_source_schema: target_schema, + project_columns, + }; + + let optimized_append = optimize_append(insert_plan, source, metadata.clone(), ctx.as_ref())?; + + Ok(Plan::Append { + s_expr: Box::new(optimized_append), + metadata: metadata.clone(), + stage_table_info: None, + overwrite, + forbid_occ_retry, + append_type: AppendType::Insert, + target_table_index: table_index, + }) } -impl CopyIntoTableMode { - pub fn is_overwrite(&self) -> bool { - match self { - CopyIntoTableMode::Insert { overwrite } => *overwrite, - CopyIntoTableMode::Replace => false, - CopyIntoTableMode::Copy => false, - } +impl Hash for Append { + fn hash(&self, _state: &mut H) { + todo!() } } -#[derive(Clone)] -pub struct CopyIntoTablePlan { - pub no_file_to_copy: bool, - - pub catalog_info: Arc, - pub database_name: String, - pub table_name: String, - pub from_attachment: bool, - - 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 stage_table_info: StageTableInfo, - pub query: Option>, - // query may be Some even if is_transform=false - pub is_transform: bool, - - pub enable_distributed: bool, -} - -impl CopyIntoTablePlan { - pub async fn collect_files(&mut self, ctx: &dyn TableContext) -> Result<()> { +impl Append { + pub async fn collect_files( + &self, + ctx: &dyn TableContext, + stage_table_info: &mut StageTableInfo, + catalog_name: &str, + database_name: &str, + table_name: &str, + ) -> Result<()> { ctx.set_status_info("begin to list files"); let start = Instant::now(); - let stage_table_info = &self.stage_table_info; let max_files = stage_table_info.copy_into_table_options.max_files; let max_files = if max_files == 0 { None @@ -192,8 +187,8 @@ impl CopyIntoTablePlan { return Err(ErrorCode::Internal(COPY_MAX_FILES_COMMIT_MSG)); } info!( - "force mode, ignore file filtering. ({}.{})", - &self.database_name, &self.table_name + "force mode, ignore file filtering {}.{}.{}", + catalog_name, database_name, table_name ); (all_source_file_infos, vec![]) } else { @@ -206,9 +201,9 @@ impl CopyIntoTablePlan { duplicated_files, } = ctx .filter_out_copied_files( - self.catalog_info.catalog_name(), - &self.database_name, - &self.table_name, + catalog_name, + database_name, + table_name, &all_source_file_infos, max_files, ) @@ -240,45 +235,39 @@ impl CopyIntoTablePlan { start.elapsed() ); - if need_copy_file_infos.is_empty() { - self.no_file_to_copy = true; - } - - self.stage_table_info.files_to_copy = Some(need_copy_file_infos); - self.stage_table_info.duplicated_files_detected = duplicated; + stage_table_info.files_to_copy = Some(need_copy_file_infos); + stage_table_info.duplicated_files_detected = duplicated; Ok(()) } } -impl Debug for CopyIntoTablePlan { - fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { - let CopyIntoTablePlan { - catalog_info, - database_name, - table_name, - no_file_to_copy, - validation_mode, - stage_table_info, - query, - .. - } = self; - write!( - f, - "Copy into {:}.{database_name:}.{table_name:}", - catalog_info.catalog_name() - )?; - write!(f, ", no_file_to_copy: {no_file_to_copy:?}")?; - write!(f, ", validation_mode: {validation_mode:?}")?; - write!(f, ", from: {stage_table_info:?}")?; - write!(f, " query: {query:?}")?; +impl Debug for Append { + fn fmt(&self, _f: &mut Formatter) -> std::fmt::Result { + // let CopyIntoTablePlan { + // catalog_info, + // database_name, + // table_name, + // no_file_to_copy, + // validation_mode, + // stage_table_info, + // query, + // .. + // } = self; + // write!( + // f, + // "Copy into {:}.{database_name:}.{table_name:}", + // catalog_info.catalog_name() + // )?; + // write!(f, ", no_file_to_copy: {no_file_to_copy:?}")?; + // write!(f, ", validation_mode: {validation_mode:?}")?; + // write!(f, ", from: {stage_table_info:?}")?; + // write!(f, " query: {query:?}")?; Ok(()) } } -/// CopyPlan supports CopyIntoTable & CopyIntoStage - -impl CopyIntoTablePlan { +impl Append { fn copy_into_table_schema() -> DataSchemaRef { DataSchemaRefExt::create(vec![ DataField::new("File", DataType::String), @@ -295,11 +284,68 @@ impl CopyIntoTablePlan { ]) } - pub fn schema(&self) -> DataSchemaRef { - if self.from_attachment { - Arc::new(DataSchema::empty()) - } else { - Self::copy_into_table_schema() + pub fn schema(append_type: &AppendType) -> DataSchemaRef { + match append_type { + AppendType::CopyInto => Self::copy_into_table_schema(), + AppendType::Insert => Arc::new(DataSchema::empty()), + } + } + + pub fn target_table( + metadata: &MetadataRef, + table_index: IndexType, + ) -> (Arc, String, String, String) { + let metadata = metadata.read(); + let t = metadata.table(table_index); + ( + t.table(), + t.catalog().to_string(), + t.database().to_string(), + t.name().to_string(), + ) + } +} + +impl Operator for Append { + fn rel_op(&self) -> RelOp { + RelOp::Append + } +} + +impl PhysicalPlanBuilder { + pub async fn build_append( + &mut self, + s_expr: &SExpr, + plan: &crate::plans::Append, + ) -> Result { + if plan + .project_columns + .as_ref() + .is_some_and(|p| p.len() != plan.required_source_schema.num_fields()) + { + return Err(ErrorCode::BadArguments(format!( + "Fields in select statement is not equal with expected, select fields: {}, insert fields: {}", + plan.project_columns.as_ref().unwrap().len(), + plan.required_source_schema.num_fields(), + ))); } + let target_table = self.metadata.read().table(plan.table_index).table(); + + let column_set = plan + .project_columns + .as_ref() + .map(|project_columns| project_columns.iter().map(|c| c.index).collect()) + .unwrap_or_default(); + let source = self.build(s_expr.child(0)?, column_set).await?; + + Ok(PhysicalPlan::Append(Box::new(PhysicalAppend { + plan_id: 0, + input: Box::new(source), + required_values_schema: plan.required_values_schema.clone(), + values_consts: plan.values_consts.clone(), + required_source_schema: plan.required_source_schema.clone(), + table_info: target_table.get_table_info().clone(), + project_columns: plan.project_columns.clone(), + }))) } } diff --git a/src/query/sql/src/planner/plans/insert.rs b/src/query/sql/src/planner/plans/insert.rs index 7685f4c6c382..a4ff019dc6a2 100644 --- a/src/query/sql/src/planner/plans/insert.rs +++ b/src/query/sql/src/planner/plans/insert.rs @@ -15,6 +15,7 @@ use std::sync::Arc; use databend_common_ast::ast::FormatTreeNode; +use databend_common_exception::ErrorCode; use databend_common_expression::types::StringType; use databend_common_expression::DataBlock; use databend_common_expression::DataSchemaRef; @@ -27,7 +28,6 @@ use serde::Deserialize; use serde::Serialize; use super::Plan; -use crate::plans::CopyIntoTablePlan; #[derive(Clone, Debug, EnumAsInner)] pub enum InsertInputSource { @@ -154,53 +154,9 @@ pub(crate) fn format_insert_source( ) .format_pretty()?), }, - InsertInputSource::Stage(plan) => match *plan.clone() { - Plan::CopyIntoTable(copy_plan) => { - let CopyIntoTablePlan { - no_file_to_copy, - from_attachment, - required_values_schema, - required_source_schema, - write_mode, - validation_mode, - stage_table_info, - enable_distributed, - .. - } = &*copy_plan; - let required_values_schema = required_values_schema - .fields() - .iter() - .map(|field| field.name().to_string()) - .collect::>() - .join(","); - let required_source_schema = required_source_schema - .fields() - .iter() - .map(|field| field.name().to_string()) - .collect::>() - .join(","); - let stage_node = vec![ - FormatTreeNode::new(format!("no_file_to_copy: {no_file_to_copy}")), - FormatTreeNode::new(format!("from_attachment: {from_attachment}")), - FormatTreeNode::new(format!( - "required_values_schema: [{required_values_schema}]" - )), - FormatTreeNode::new(format!( - "required_source_schema: [{required_source_schema}]" - )), - FormatTreeNode::new(format!("write_mode: {write_mode}")), - FormatTreeNode::new(format!("validation_mode: {validation_mode}")), - FormatTreeNode::new(format!("stage_table_info: {stage_table_info}")), - FormatTreeNode::new(format!("enable_distributed: {enable_distributed}")), - ]; - children.extend(stage_node); - Ok( - FormatTreeNode::with_children(format!("{plan_name} (stage):"), children) - .format_pretty()?, - ) - } - _ => unreachable!("plan in InsertInputSource::Stag must be CopyIntoTable"), - }, + InsertInputSource::Stage(_) => Err(ErrorCode::StorageUnsupported( + "stage attachment is deprecated in replace into statement", + )), } } diff --git a/src/query/sql/src/planner/plans/mod.rs b/src/query/sql/src/planner/plans/mod.rs index 2acc83d081d7..d0f2c779a2b2 100644 --- a/src/query/sql/src/planner/plans/mod.rs +++ b/src/query/sql/src/planner/plans/mod.rs @@ -13,12 +13,12 @@ // limitations under the License. mod aggregate; +mod append; mod async_function; mod cache_scan; mod call; mod constant_table_scan; mod copy_into_location; -mod copy_into_table; mod cte_scan; mod data_mask; mod ddl; @@ -52,15 +52,16 @@ mod sort; mod system; mod udf; mod union_all; +mod value_scan; mod window; pub use aggregate::*; +pub use append::*; pub use async_function::AsyncFunction; pub use cache_scan::*; pub use call::CallPlan; pub use constant_table_scan::ConstantTableScan; pub use copy_into_location::*; -pub use copy_into_table::*; pub use cte_scan::CteScan; pub use data_mask::*; pub use ddl::*; @@ -99,4 +100,5 @@ pub use sort::*; pub use system::*; pub use udf::*; pub use union_all::UnionAll; +pub use value_scan::*; pub use window::*; diff --git a/src/query/sql/src/planner/plans/operator.rs b/src/query/sql/src/planner/plans/operator.rs index d7f6b20c5be5..081af0c4e9c9 100644 --- a/src/query/sql/src/planner/plans/operator.rs +++ b/src/query/sql/src/planner/plans/operator.rs @@ -18,7 +18,9 @@ use databend_common_catalog::table_context::TableContext; use databend_common_exception::ErrorCode; use databend_common_exception::Result; +use super::Append; use super::MutationSource; +use super::ValueScan; use crate::optimizer::PhysicalProperty; use crate::optimizer::RelExpr; use crate::optimizer::RelationalProperty; @@ -121,6 +123,8 @@ pub enum RelOp { Recluster, CompactBlock, MutationSource, + Append, + ValueScan, // Pattern Pattern, @@ -153,6 +157,8 @@ pub enum RelOperator { Recluster(Recluster), CompactBlock(OptimizeCompactBlock), MutationSource(MutationSource), + Append(Append), + ValueScan(ValueScan), } impl Operator for RelOperator { @@ -182,6 +188,8 @@ impl Operator for RelOperator { RelOperator::Recluster(rel_op) => rel_op.rel_op(), RelOperator::CompactBlock(rel_op) => rel_op.rel_op(), RelOperator::MutationSource(rel_op) => rel_op.rel_op(), + RelOperator::Append(rel_op) => rel_op.rel_op(), + RelOperator::ValueScan(rel_op) => rel_op.rel_op(), } } @@ -211,6 +219,8 @@ impl Operator for RelOperator { RelOperator::Recluster(rel_op) => rel_op.arity(), RelOperator::CompactBlock(rel_op) => rel_op.arity(), RelOperator::MutationSource(rel_op) => rel_op.arity(), + RelOperator::Append(rel_op) => rel_op.arity(), + RelOperator::ValueScan(rel_op) => rel_op.arity(), } } @@ -240,6 +250,8 @@ impl Operator for RelOperator { RelOperator::Recluster(rel_op) => rel_op.derive_relational_prop(rel_expr), RelOperator::CompactBlock(rel_op) => rel_op.derive_relational_prop(rel_expr), RelOperator::MutationSource(rel_op) => rel_op.derive_relational_prop(rel_expr), + RelOperator::Append(rel_op) => rel_op.derive_relational_prop(rel_expr), + RelOperator::ValueScan(rel_op) => rel_op.derive_relational_prop(rel_expr), } } @@ -269,6 +281,8 @@ impl Operator for RelOperator { RelOperator::Recluster(rel_op) => rel_op.derive_physical_prop(rel_expr), RelOperator::CompactBlock(rel_op) => rel_op.derive_physical_prop(rel_expr), RelOperator::MutationSource(rel_op) => rel_op.derive_physical_prop(rel_expr), + RelOperator::Append(rel_op) => rel_op.derive_physical_prop(rel_expr), + RelOperator::ValueScan(rel_op) => rel_op.derive_physical_prop(rel_expr), } } @@ -298,6 +312,8 @@ impl Operator for RelOperator { RelOperator::Recluster(rel_op) => rel_op.derive_stats(rel_expr), RelOperator::CompactBlock(rel_op) => rel_op.derive_stats(rel_expr), RelOperator::MutationSource(rel_op) => rel_op.derive_stats(rel_expr), + RelOperator::Append(rel_op) => rel_op.derive_stats(rel_expr), + RelOperator::ValueScan(rel_op) => rel_op.derive_stats(rel_expr), } } @@ -381,6 +397,12 @@ impl Operator for RelOperator { RelOperator::MutationSource(rel_op) => { rel_op.compute_required_prop_child(ctx, rel_expr, child_index, required) } + RelOperator::Append(rel_op) => { + rel_op.compute_required_prop_child(ctx, rel_expr, child_index, required) + } + RelOperator::ValueScan(rel_op) => { + rel_op.compute_required_prop_child(ctx, rel_expr, child_index, required) + } } } @@ -463,6 +485,12 @@ impl Operator for RelOperator { RelOperator::MutationSource(rel_op) => { rel_op.compute_required_prop_children(ctx, rel_expr, required) } + RelOperator::Append(rel_op) => { + rel_op.compute_required_prop_children(ctx, rel_expr, required) + } + RelOperator::ValueScan(rel_op) => { + rel_op.compute_required_prop_children(ctx, rel_expr, required) + } } } } @@ -913,3 +941,43 @@ impl TryFrom for MutationSource { } } } + +impl From for RelOperator { + fn from(v: Append) -> Self { + Self::Append(v) + } +} + +impl TryFrom for Append { + type Error = ErrorCode; + fn try_from(value: RelOperator) -> Result { + if let RelOperator::Append(value) = value { + Ok(value) + } else { + Err(ErrorCode::Internal(format!( + "Cannot downcast {:?} to Append", + value.rel_op() + ))) + } + } +} + +impl From for RelOperator { + fn from(v: ValueScan) -> Self { + Self::ValueScan(v) + } +} + +impl TryFrom for ValueScan { + type Error = ErrorCode; + fn try_from(value: RelOperator) -> Result { + if let RelOperator::ValueScan(value) = value { + Ok(value) + } else { + Err(ErrorCode::Internal(format!( + "Cannot downcast {:?} to ValueScan", + value.rel_op() + ))) + } + } +} diff --git a/src/query/sql/src/planner/plans/plan.rs b/src/query/sql/src/planner/plans/plan.rs index e886fefba18f..cb7992ac4244 100644 --- a/src/query/sql/src/planner/plans/plan.rs +++ b/src/query/sql/src/planner/plans/plan.rs @@ -17,6 +17,7 @@ use std::fmt::Formatter; use std::sync::Arc; use databend_common_ast::ast::ExplainKind; +use databend_common_catalog::plan::StageTableInfo; use databend_common_catalog::query_kind::QueryKind; use databend_common_expression::types::DataType; use databend_common_expression::DataField; @@ -24,6 +25,8 @@ use databend_common_expression::DataSchema; use databend_common_expression::DataSchemaRef; use databend_common_expression::DataSchemaRefExt; +use super::Append; +use super::AppendType; use super::CreateDictionaryPlan; use super::DropDictionaryPlan; use super::RenameDictionaryPlan; @@ -43,8 +46,6 @@ use crate::plans::AlterViewPlan; use crate::plans::AlterVirtualColumnPlan; use crate::plans::AnalyzeTablePlan; use crate::plans::CallProcedurePlan; -use crate::plans::CopyIntoTableMode; -use crate::plans::CopyIntoTablePlan; use crate::plans::CreateCatalogPlan; use crate::plans::CreateConnectionPlan; use crate::plans::CreateDatabasePlan; @@ -105,7 +106,6 @@ use crate::plans::ExecuteTaskPlan; use crate::plans::ExistsTablePlan; use crate::plans::GrantPrivilegePlan; use crate::plans::GrantRolePlan; -use crate::plans::Insert; use crate::plans::InsertMultiTable; use crate::plans::KillPlan; use crate::plans::ModifyTableColumnPlan; @@ -150,6 +150,7 @@ use crate::plans::VacuumDropTablePlan; use crate::plans::VacuumTablePlan; use crate::plans::VacuumTemporaryFilesPlan; use crate::BindContext; +use crate::IndexType; use crate::MetadataRef; #[derive(Clone, Debug)] @@ -234,9 +235,6 @@ pub enum Plan { s_expr: Box, need_purge: bool, }, - - // Insert - Insert(Box), InsertMultiTable(Box), Replace(Box), DataMutation { @@ -245,7 +243,15 @@ pub enum Plan { metadata: MetadataRef, }, - CopyIntoTable(Box), + Append { + s_expr: Box, + metadata: MetadataRef, + stage_table_info: Option>, + overwrite: bool, + forbid_occ_retry: bool, + append_type: AppendType, + target_table_index: IndexType, + }, CopyIntoLocation(CopyIntoLocationPlan), // Views @@ -420,15 +426,11 @@ impl Plan { pub fn kind(&self) -> QueryKind { match self { Plan::Query { .. } => QueryKind::Query, - Plan::CopyIntoTable(copy_plan) => match copy_plan.write_mode { - CopyIntoTableMode::Insert { .. } => QueryKind::Insert, - _ => QueryKind::CopyIntoTable, - }, + Plan::Append { .. } => QueryKind::CopyIntoTable, Plan::Explain { .. } | Plan::ExplainAnalyze { .. } | Plan::ExplainAst { .. } | Plan::ExplainSyntax { .. } => QueryKind::Explain, - Plan::Insert(_) => QueryKind::Insert, Plan::Replace(_) | Plan::DataMutation { .. } | Plan::OptimizePurge(_) @@ -481,7 +483,6 @@ impl Plan { Plan::DescNetworkPolicy(plan) => plan.schema(), Plan::ShowNetworkPolicies(plan) => plan.schema(), Plan::DescPasswordPolicy(plan) => plan.schema(), - Plan::CopyIntoTable(plan) => plan.schema(), Plan::CopyIntoLocation(plan) => plan.schema(), Plan::CreateTask(plan) => plan.schema(), Plan::DescribeTask(plan) => plan.schema(), @@ -494,6 +495,7 @@ impl Plan { Plan::CallProcedure(plan) => plan.schema(), Plan::InsertMultiTable(plan) => plan.schema(), Plan::DescUser(plan) => plan.schema(), + Plan::Append { append_type, .. } => Append::schema(append_type), _ => Arc::new(DataSchema::empty()), } diff --git a/src/query/sql/src/planner/plans/value_scan.rs b/src/query/sql/src/planner/plans/value_scan.rs new file mode 100644 index 000000000000..ea16b95b9570 --- /dev/null +++ b/src/query/sql/src/planner/plans/value_scan.rs @@ -0,0 +1,79 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use databend_common_exception::Result; +use databend_common_expression::DataSchemaRef; + +use crate::executor::physical_plans::PhysicalValueScan; +use crate::executor::physical_plans::Values; +use crate::executor::PhysicalPlan; +use crate::executor::PhysicalPlanBuilder; +use crate::optimizer::Distribution; +use crate::optimizer::PhysicalProperty; +use crate::optimizer::RelExpr; +use crate::plans::Operator; +use crate::plans::RelOp; + +#[derive(Clone, Debug, PartialEq, Eq)] +pub struct ValueScan { + pub values: Values, + pub dest_schema: DataSchemaRef, +} + +impl std::hash::Hash for ValueScan { + fn hash(&self, state: &mut H) { + self.values.hash(state); + } +} + +impl Operator for ValueScan { + fn rel_op(&self) -> RelOp { + RelOp::ValueScan + } + + fn arity(&self) -> usize { + 0 + } + + fn derive_physical_prop(&self, _rel_expr: &RelExpr) -> Result { + Ok(PhysicalProperty { + distribution: Distribution::Random, + }) + } +} + +impl PhysicalPlanBuilder { + pub async fn build_value_scan( + &mut self, + plan: &crate::plans::ValueScan, + ) -> Result { + match &plan.values { + Values::Values(values) => Ok(PhysicalPlan::ValueScan(Box::new(PhysicalValueScan { + plan_id: 0, + values: Values::Values(values.clone()), + output_schema: plan.dest_schema.clone(), + }))), + Values::RawValues { rest_str, start } => { + Ok(PhysicalPlan::ValueScan(Box::new(PhysicalValueScan { + plan_id: 0, + values: Values::RawValues { + rest_str: rest_str.clone(), + start: *start, + }, + output_schema: plan.dest_schema.clone(), + }))) + } + } + } +} diff --git a/src/query/storages/fuse/src/fuse_table.rs b/src/query/storages/fuse/src/fuse_table.rs index f90b616b09c8..e0d3ee691919 100644 --- a/src/query/storages/fuse/src/fuse_table.rs +++ b/src/query/storages/fuse/src/fuse_table.rs @@ -70,7 +70,6 @@ use databend_storages_common_cache::LoadParams; use databend_storages_common_table_meta::meta::parse_storage_prefix; use databend_storages_common_table_meta::meta::ClusterKey; use databend_storages_common_table_meta::meta::CompactSegmentInfo; -use databend_storages_common_table_meta::meta::SnapshotId; use databend_storages_common_table_meta::meta::Statistics as FuseStatistics; use databend_storages_common_table_meta::meta::TableSnapshot; use databend_storages_common_table_meta::meta::TableSnapshotStatistics; @@ -786,7 +785,7 @@ impl Table for FuseTable { copied_files: Option, update_stream_meta: Vec, overwrite: bool, - prev_snapshot_id: Option, + forbid_occ_retry: bool, deduplicated_label: Option, ) -> Result<()> { self.do_commit( @@ -795,7 +794,7 @@ impl Table for FuseTable { copied_files, update_stream_meta, overwrite, - prev_snapshot_id, + forbid_occ_retry, deduplicated_label, ) } diff --git a/src/query/storages/fuse/src/operations/commit.rs b/src/query/storages/fuse/src/operations/commit.rs index edc0a5c76124..9cd69f195c13 100644 --- a/src/query/storages/fuse/src/operations/commit.rs +++ b/src/query/storages/fuse/src/operations/commit.rs @@ -42,7 +42,6 @@ use databend_storages_common_cache::CacheAccessor; use databend_storages_common_cache::CachedObject; use databend_storages_common_table_meta::meta::Location; use databend_storages_common_table_meta::meta::SegmentInfo; -use databend_storages_common_table_meta::meta::SnapshotId; use databend_storages_common_table_meta::meta::Statistics; use databend_storages_common_table_meta::meta::TableSnapshot; use databend_storages_common_table_meta::meta::TableSnapshotStatistics; @@ -76,7 +75,7 @@ impl FuseTable { copied_files: Option, update_stream_meta: Vec, overwrite: bool, - prev_snapshot_id: Option, + forbid_occ_retry: bool, deduplicated_label: Option, ) -> Result<()> { let block_thresholds = self.get_block_thresholds(); @@ -110,7 +109,7 @@ impl FuseTable { snapshot_gen.clone(), input, None, - prev_snapshot_id, + forbid_occ_retry, deduplicated_label.clone(), ) })?; 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 551d0aa2c1c3..b0e880aa9774 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 @@ -36,7 +36,6 @@ use databend_common_pipeline_core::processors::Processor; use databend_common_pipeline_core::processors::ProcessorPtr; use databend_storages_common_table_meta::meta::ClusterKey; use databend_storages_common_table_meta::meta::Location; -use databend_storages_common_table_meta::meta::SnapshotId; use databend_storages_common_table_meta::meta::TableSnapshot; use databend_storages_common_table_meta::meta::Versioned; use log::debug; @@ -92,7 +91,7 @@ pub struct CommitSink { new_segment_locs: Vec, start_time: Instant, - prev_snapshot_id: Option, + forbid_retry: bool, change_tracking: bool, update_stream_meta: Vec, @@ -111,7 +110,7 @@ where F: SnapshotGenerator + Send + 'static snapshot_gen: F, input: Arc, max_retry_elapsed: Option, - prev_snapshot_id: Option, + forbid_retry: bool, deduplicated_label: Option, ) -> Result { let purge = Self::do_purge(table, &snapshot_gen); @@ -130,7 +129,7 @@ where F: SnapshotGenerator + Send + 'static input, new_segment_locs: vec![], start_time: Instant::now(), - prev_snapshot_id, + forbid_retry, change_tracking: table.change_tracking_enabled(), update_stream_meta, deduplicated_label, @@ -138,15 +137,12 @@ where F: SnapshotGenerator + Send + 'static } fn is_error_recoverable(&self, e: &ErrorCode) -> bool { - let code = e.code(); - // When prev_snapshot_id is some, means it is an alter table column modification or truncate. - // In this case if commit to meta fail and error is TABLE_VERSION_MISMATCHED operation will be aborted. - if self.prev_snapshot_id.is_some() && code == ErrorCode::TABLE_VERSION_MISMATCHED { + if self.forbid_retry { return false; } - code == ErrorCode::TABLE_VERSION_MISMATCHED - || (self.purge && code == ErrorCode::STORAGE_NOT_FOUND) + e.code() == ErrorCode::TABLE_VERSION_MISMATCHED + || (self.purge && e.code() == ErrorCode::STORAGE_NOT_FOUND) } fn no_side_effects_in_meta_store(e: &ErrorCode) -> bool { @@ -339,28 +335,15 @@ where F: SnapshotGenerator + Send + 'static // save current table info when commit to meta server // if table_id not match, update table meta will fail let table_info = fuse_table.table_info.clone(); - // check if snapshot has been changed - let snapshot_has_changed = self.prev_snapshot_id.is_some_and(|prev_snapshot_id| { - previous - .as_ref() - .map_or(true, |previous| previous.snapshot_id != prev_snapshot_id) - }); - if snapshot_has_changed { - // if snapshot has changed abort operation - self.state = State::Abort(ErrorCode::StorageOther( - "commit failed because the snapshot had changed during the commit process", - )); - } else { - self.snapshot_gen - .fill_default_values(schema, &previous) - .await?; - - self.state = State::GenerateSnapshot { - previous, - cluster_key_meta: fuse_table.cluster_key_meta.clone(), - table_info, - }; - } + self.snapshot_gen + .fill_default_values(schema, &previous) + .await?; + + self.state = State::GenerateSnapshot { + previous, + cluster_key_meta: fuse_table.cluster_key_meta.clone(), + table_info, + }; } State::TryCommit { data, diff --git a/src/query/storages/fuse/src/operations/truncate.rs b/src/query/storages/fuse/src/operations/truncate.rs index 6da256d0f730..4ea432861d7d 100644 --- a/src/query/storages/fuse/src/operations/truncate.rs +++ b/src/query/storages/fuse/src/operations/truncate.rs @@ -20,7 +20,6 @@ use databend_common_exception::Result; use databend_common_expression::DataBlock; use databend_common_pipeline_core::Pipeline; use databend_common_pipeline_sources::OneBlockSource; -use databend_storages_common_table_meta::meta::TableSnapshot; use crate::operations::common::CommitMeta; use crate::operations::common::CommitSink; @@ -38,10 +37,11 @@ impl FuseTable { pipeline: &mut Pipeline, mode: TruncateMode, ) -> Result<()> { - if let Some(prev_snapshot) = self.read_table_snapshot().await? { - self.build_truncate_pipeline(ctx, pipeline, mode, prev_snapshot)?; + if self.read_table_snapshot().await?.is_some() { + self.build_truncate_pipeline(ctx, pipeline, mode) + } else { + Ok(()) } - Ok(()) } #[inline] @@ -51,14 +51,9 @@ impl FuseTable { ctx: Arc, pipeline: &mut Pipeline, mode: TruncateMode, - prev_snapshot: Arc, ) -> Result<()> { // Delete operation commit can retry multi-times if table version mismatched. - let prev_snapshot_id = if !matches!(mode, TruncateMode::Delete) { - Some(prev_snapshot.snapshot_id) - } else { - None - }; + let forbid_occ_retry = !matches!(mode, TruncateMode::Delete); pipeline.add_source( |output| { let meta = CommitMeta { @@ -82,7 +77,7 @@ impl FuseTable { snapshot_gen.clone(), input, None, - prev_snapshot_id, + forbid_occ_retry, None, ) }) diff --git a/src/query/storages/hive/hive/src/hive_table.rs b/src/query/storages/hive/hive/src/hive_table.rs index fa728e47c226..3577e5943927 100644 --- a/src/query/storages/hive/hive/src/hive_table.rs +++ b/src/query/storages/hive/hive/src/hive_table.rs @@ -53,7 +53,6 @@ use databend_common_storage::DataOperator; use databend_common_storages_parquet::ParquetRSPruner; use databend_common_storages_parquet::ParquetRSReaderBuilder; use databend_storages_common_pruner::partition_prunner::PartitionPruner; -use databend_storages_common_table_meta::meta::SnapshotId; use databend_storages_common_table_meta::table::ChangeType; use futures::TryStreamExt; use log::info; @@ -447,7 +446,7 @@ impl Table for HiveTable { _copied_files: Option, _update_stream_meta: Vec, _overwrite: bool, - _prev_snapshot_id: Option, + _forbid_occ_retry: bool, _deduplicated_label: Option, ) -> Result<()> { Err(ErrorCode::Unimplemented(format!( diff --git a/src/query/storages/memory/src/memory_table.rs b/src/query/storages/memory/src/memory_table.rs index b1830ac3c4f2..b31c8aa3b35f 100644 --- a/src/query/storages/memory/src/memory_table.rs +++ b/src/query/storages/memory/src/memory_table.rs @@ -49,7 +49,6 @@ use databend_common_pipeline_sources::SyncSourcer; use databend_common_storage::StorageMetrics; use databend_storages_common_blocks::memory::InMemoryDataKey; use databend_storages_common_blocks::memory::IN_MEMORY_DATA; -use databend_storages_common_table_meta::meta::SnapshotId; use databend_storages_common_table_meta::table::OPT_KEY_TEMP_PREFIX; use parking_lot::Mutex; use parking_lot::RwLock; @@ -244,7 +243,7 @@ impl Table for MemoryTable { _copied_files: Option, _update_stream_meta: Vec, overwrite: bool, - _prev_snapshot_id: Option, + _forbid_occ_retry: bool, _deduplicated_label: Option, ) -> Result<()> { pipeline.try_resize(1)?; diff --git a/tests/sqllogictests/suites/mode/standalone/explain/explain.test b/tests/sqllogictests/suites/mode/standalone/explain/explain.test index 940f72d817bf..fef40733306b 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/explain.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/explain.test @@ -1615,15 +1615,18 @@ HashJoin query T explain insert into t2 select * from t1; ---- -InsertPlan (subquery): -├── table: default.default.t2 -├── inserted columns: [t2.a (#0),t2.b (#1),t2.c (#2)] -├── overwrite: false -└── Scan - ├── table: default.t1 - ├── filters: [] - ├── order by: [] - └── limit: NONE +Append +├── target table: [catalog: default] [desc: 'default'.'t2'] +├── required columns: [#a, #b, #c] +└── TableScan + ├── table: default.default.t1 + ├── output columns: [a (#3), b (#4), c (#5)] + ├── read rows: 0 + ├── read size: 0 + ├── partitions total: 0 + ├── partitions scanned: 0 + ├── push downs: [filters: [], limit: NONE] + └── estimated rows: 0.00 statement ok diff --git a/tests/sqllogictests/suites/mode/standalone/explain/explain_sequence.test b/tests/sqllogictests/suites/mode/standalone/explain/explain_sequence.test index 839f63dfbb3d..826aad5b1087 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/explain_sequence.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/explain_sequence.test @@ -13,16 +13,22 @@ CREATE TABLE tmp(a int, b uint64, c int); query T explain INSERT INTO tmp select 10,nextval(seq),20 from numbers(3); ---- -InsertPlan (subquery): -├── table: default.default.tmp -├── inserted columns: [tmp.a (#0),tmp.b (#1),tmp.c (#2)] -├── overwrite: false +Append +├── target table: [catalog: default] [desc: 'default'.'tmp'] +├── required columns: [#a, #b, #c] └── EvalScalar - ├── scalars: [10 AS (#1), nextval(seq) (#2) AS (#2), 20 AS (#3)] + ├── output columns: [nextval(seq) (#5), 10 (#4), 20 (#6)] + ├── expressions: [10, 20] + ├── estimated rows: 3.00 └── AsyncFunction - ├── scalars: [nextval(seq) AS (#2)] - └── Scan - ├── table: system.numbers - ├── filters: [] - ├── order by: [] - └── limit: NONE + ├── output columns: [nextval(seq) (#5)] + ├── estimated rows: 3.00 + └── TableScan + ├── table: default.system.numbers + ├── output columns: [] + ├── read rows: 3 + ├── read size: < 1 KiB + ├── partitions total: 1 + ├── partitions scanned: 1 + ├── push downs: [filters: [], limit: NONE] + └── estimated rows: 3.00 diff --git a/tests/sqllogictests/suites/mode/standalone/explain/insert.test b/tests/sqllogictests/suites/mode/standalone/explain/insert.test index 58ebfcaa4b56..b5a9c39f052b 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/insert.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/insert.test @@ -13,25 +13,31 @@ create table t2(a int, b int); query T explain insert overwrite t1 values(1, 2), (2, 3), (3, 4), (8, 9); ---- -InsertPlan (values): -├── table: default.default.t1 -├── inserted columns: [t1.a (#0),t1.b (#1)] -└── overwrite: true +Append +├── target table: [catalog: default] [desc: 'default'.'t1'] +├── required columns: [#a, #b] +└── ValueScan + └── values: [4] rows query T explain insert into t2 select * from t1 where a=1; ---- -InsertPlan (subquery): -├── table: default.default.t2 -├── inserted columns: [t2.a (#0),t2.b (#1)] -├── overwrite: false +Append +├── target table: [catalog: default] [desc: 'default'.'t2'] +├── required columns: [#a, #b] └── Filter - ├── filters: [eq(t1.a (#0), 1)] - └── Scan - ├── table: default.t1 - ├── filters: [eq(t1.a (#0), 1)] - ├── order by: [] - └── limit: NONE + ├── output columns: [t1.a (#2), t1.b (#3)] + ├── filters: [is_true(t1.a (#2) = 1)] + ├── estimated rows: 0.00 + └── TableScan + ├── table: default.default.t1 + ├── output columns: [a (#2), b (#3)] + ├── read rows: 0 + ├── read size: 0 + ├── partitions total: 0 + ├── partitions scanned: 0 + ├── push downs: [filters: [is_true(t1.a (#2) = 1)], limit: NONE] + └── estimated rows: 0.00 diff --git a/tests/suites/1_stateful/07_stage_attachment/07_0001_replace_with_stage.result b/tests/suites/1_stateful/07_stage_attachment/07_0001_replace_with_stage.result deleted file mode 100644 index 98cb421d64e0..000000000000 --- a/tests/suites/1_stateful/07_stage_attachment/07_0001_replace_with_stage.result +++ /dev/null @@ -1,48 +0,0 @@ -sample.csv -96 -125 -null -1 'Beijing' 100 China -2 'Shanghai' 80 China -3 'Guangzhou' 60 China -4 'Shenzhen' 70 China -5 'Shenzhen' 55 China -6 'Beijing' 99 China -96 -null -1 'Beijing' 100 China -2 'Shanghai' 80 China -3 'Guangzhou' 60 China -4 'Shenzhen' 70 China -5 'Shenzhen' 55 China -6 'Beijing' 99 China -77 -null -1 'beijing' 2 China -2 'shanghai' 2 China -3 'guangzhou' 2 China -4 'shenzhen' 2 China -5 'shenzhen' 2 China -6 'beijing' 2 China -83 -null -1 'Chengdu' 80 China -2 'shanghai' 2 China -3 'Chongqing' 90 China -4 'shenzhen' 2 China -5 'shenzhen' 2 China -6 'HangZhou' 92 China -9 'Changsha' 91 China -10 'Hong Kong‘ 88 China -{ - "code": 4000, - "message": "duplicated data detected in the values being replaced into (only the first one will be described): at row 7, [\"id\":10]" -} -1 'Chengdu' 80 China -2 'shanghai' 2 China -3 'Chongqing' 90 China -4 'shenzhen' 2 China -5 'shenzhen' 2 China -6 'HangZhou' 92 China -9 'Changsha' 91 China -10 'Hong Kong‘ 88 China diff --git a/tests/suites/1_stateful/07_stage_attachment/07_0001_replace_with_stage.sh b/tests/suites/1_stateful/07_stage_attachment/07_0001_replace_with_stage.sh deleted file mode 100755 index c40c690fff76..000000000000 --- a/tests/suites/1_stateful/07_stage_attachment/07_0001_replace_with_stage.sh +++ /dev/null @@ -1,57 +0,0 @@ -#!/usr/bin/env bash - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -. "$CURDIR"/../../../shell_env.sh - -echo "drop table if exists sample_table" | $BENDSQL_CLIENT_CONNECT -echo "drop stage if exists s1" | $BENDSQL_CLIENT_CONNECT - -## Create table -cat </dev/null - -## Copy from internal stage -echo "CREATE STAGE s1 FILE_FORMAT = (TYPE = CSV)" | $BENDSQL_CLIENT_CONNECT -echo "list @s1" | $BENDSQL_CLIENT_CONNECT | awk '{print $1}' - -## Insert with stage use http API -curl -s -u root: -XPOST "http://localhost:${QUERY_HTTP_HANDLER_PORT}/v1/query" --header 'Content-Type: application/json' -d '{"sql": "replace into sample_table (Id, City, Score) ON(Id) VALUES", "stage_attachment": {"location": "@s1/sample.csv", "copy_options": {"purge": "true"}}, "pagination": { "wait_time_secs": 3}}' | jq -r '.stats.scan_progress.bytes, .stats.write_progress.bytes, .error' - -## list stage has metacache, so we just we aws client to ensure the data are purged -aws --endpoint-url ${STORAGE_S3_ENDPOINT_URL} s3 ls s3://testbucket/admin/stage/internal/s1/sample.csv - -echo "select * from sample_table order by id" | $BENDSQL_CLIENT_CONNECT - - -# use placeholder (?, ?, ?) -aws --endpoint-url ${STORAGE_S3_ENDPOINT_URL} s3 cp s3://testbucket/admin/data/csv/sample.csv s3://testbucket/admin/stage/internal/s1/sample1.csv >/dev/null -curl -s -u root: -XPOST "http://localhost:${QUERY_HTTP_HANDLER_PORT}/v1/query" --header 'Content-Type: application/json' -d '{"sql": "replace into sample_table (Id, City, Score) ON(Id) values (?,?,?)", "stage_attachment": {"location": "@s1/sample1.csv", "copy_options": {"purge": "true"}}, "pagination": { "wait_time_secs": 3}}' | jq -r '.stats.scan_progress.bytes, .error' -echo "select * from sample_table order by id" | $BENDSQL_CLIENT_CONNECT - -# use placeholder (?, ?, 1+1) -aws --endpoint-url ${STORAGE_S3_ENDPOINT_URL} s3 cp s3://testbucket/admin/data/csv/sample_2_columns.csv s3://testbucket/admin/stage/internal/s1/sample2.csv >/dev/null - -curl -s -u root: -XPOST "http://localhost:${QUERY_HTTP_HANDLER_PORT}/v1/query" --header 'Content-Type: application/json' -d '{"sql": "replace into sample_table (Id, City, Score) ON(Id) values (?,?,1+1)", "stage_attachment": {"location": "@s1/sample2.csv", "copy_options": {"purge": "true"}}, "pagination": { "wait_time_secs": 3}}' | jq -r '.stats.scan_progress.bytes, .error' -echo "select * from sample_table order by id" | $BENDSQL_CLIENT_CONNECT - -aws --endpoint-url ${STORAGE_S3_ENDPOINT_URL} s3 cp s3://testbucket/admin/data/csv/sample_3_replace.csv s3://testbucket/admin/stage/internal/s1/sample3.csv >/dev/null -curl -s -u root: -XPOST "http://localhost:${QUERY_HTTP_HANDLER_PORT}/v1/query" --header 'Content-Type: application/json' -d '{"sql": "replace into sample_table (Id, City, Score) ON(Id) values (?,?,?)", "stage_attachment": {"location": "@s1/sample3.csv", "copy_options": {"purge": "true"}}, "pagination": { "wait_time_secs": 3}}' | jq -r '.stats.scan_progress.bytes, .error' -echo "select * from sample_table order by id" | $BENDSQL_CLIENT_CONNECT - -# duplicate value would show error and would not take effect -aws --endpoint-url ${STORAGE_S3_ENDPOINT_URL} s3 cp s3://testbucket/admin/data/csv/sample_3_duplicate.csv s3://testbucket/admin/stage/internal/s1/sample4.csv >/dev/null -curl -s -u root: -XPOST "http://localhost:${QUERY_HTTP_HANDLER_PORT}/v1/query" --header 'Content-Type: application/json' -d '{"sql": "replace into sample_table (Id, City, Score) ON(Id) values (?,?,?)", "stage_attachment": {"location": "@s1/sample4.csv", "copy_options": {"purge": "true"}}, "pagination": { "wait_time_secs": 3}}' | jq -r '.error' -echo "select * from sample_table order by id" | $BENDSQL_CLIENT_CONNECT - -### Drop table. -echo "drop table sample_table" | $BENDSQL_CLIENT_CONNECT -echo "drop stage if exists s1" | $BENDSQL_CLIENT_CONNECT