diff --git a/Cargo.lock b/Cargo.lock index 6e8fb5735e302..af3b52b54c9b6 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4211,7 +4211,6 @@ dependencies = [ "parking_lot 0.12.3", "pretty_assertions", "prqlc", - "rand 0.8.5", "recursive", "regex", "roaring", diff --git a/src/query/service/src/interpreters/interpreter_copy_into_location.rs b/src/query/service/src/interpreters/interpreter_copy_into_location.rs index 5026095e51a08..6d7b5e1fc87a6 100644 --- a/src/query/service/src/interpreters/interpreter_copy_into_location.rs +++ b/src/query/service/src/interpreters/interpreter_copy_into_location.rs @@ -19,8 +19,6 @@ use databend_common_exception::Result; use databend_common_expression::infer_table_schema; use databend_common_meta_app::schema::UpdateStreamMetaReq; use databend_common_pipeline_core::ExecutionInfo; -use databend_common_sql::executor::physical_plans::CopyIntoLocation; -use databend_common_sql::executor::PhysicalPlan; use databend_storages_common_stage::CopyIntoLocationInfo; use log::debug; use log::info; @@ -29,6 +27,9 @@ use crate::interpreters::common::check_deduplicate_label; use crate::interpreters::common::dml_build_update_stream_req; use crate::interpreters::Interpreter; use crate::interpreters::SelectInterpreter; +use crate::physical_plans::CopyIntoLocation; +use crate::physical_plans::PhysicalPlan; +use crate::physical_plans::PhysicalPlanMeta; use crate::pipelines::PipelineBuildResult; use crate::schedulers::build_query_pipeline_without_render_result_set; use crate::sessions::QueryContext; @@ -89,14 +90,14 @@ impl CopyIntoLocationInterpreter { let query_result_schema = query_interpreter.get_result_schema(); let table_schema = infer_table_schema(&query_result_schema)?; - let mut physical_plan = PhysicalPlan::CopyIntoLocation(Box::new(CopyIntoLocation { - plan_id: 0, - input: Box::new(query_physical_plan), + let mut physical_plan: PhysicalPlan = Box::new(CopyIntoLocation { + input: query_physical_plan, project_columns: query_interpreter.get_result_columns(), input_data_schema: query_result_schema, input_table_schema: table_schema, info: info.clone(), - })); + meta: PhysicalPlanMeta::new("CopyIntoLocation"), + }); let mut next_plan_id = 0; physical_plan.adjust_plan_id(&mut next_plan_id); diff --git a/src/query/service/src/interpreters/interpreter_copy_into_table.rs b/src/query/service/src/interpreters/interpreter_copy_into_table.rs index 91b9533f346f2..e95f2d75a7899 100644 --- a/src/query/service/src/interpreters/interpreter_copy_into_table.rs +++ b/src/query/service/src/interpreters/interpreter_copy_into_table.rs @@ -26,14 +26,9 @@ use databend_common_expression::SendableDataBlockStream; use databend_common_meta_app::schema::TableInfo; 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_fuse::FuseTable; use databend_common_storages_stage::StageTable; @@ -46,6 +41,12 @@ use crate::interpreters::common::dml_build_update_stream_req; use crate::interpreters::HookOperator; use crate::interpreters::Interpreter; use crate::interpreters::SelectInterpreter; +use crate::physical_plans::CopyIntoTable; +use crate::physical_plans::CopyIntoTableSource; +use crate::physical_plans::Exchange; +use crate::physical_plans::PhysicalPlan; +use crate::physical_plans::PhysicalPlanMeta; +use crate::physical_plans::TableScan; use crate::pipelines::PipelineBuildResult; use crate::pipelines::PipelineBuilder; use crate::schedulers::build_query_pipeline_without_render_result_set; @@ -113,7 +114,7 @@ impl CopyIntoTableInterpreter { 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 query_physical_plan = query_interpreter.build_physical_plan().await?; let result_columns = query_interpreter.get_result_columns(); ( @@ -133,21 +134,20 @@ impl CopyIntoTableInterpreter { } ( - CopyIntoTableSource::Stage(Box::new(PhysicalPlan::TableScan(TableScan { - plan_id: 0, + CopyIntoTableSource::Stage(Box::new(TableScan { scan_id: 0, name_mapping, stat_info: None, table_index: None, internal_column: None, source: Box::new(data_source_plan), - }))), + meta: PhysicalPlanMeta::new("TableScan"), + })), None, ) }; - let mut root = PhysicalPlan::CopyIntoTable(Box::new(CopyIntoTable { - plan_id: 0, + let mut root: PhysicalPlan = Box::new(CopyIntoTable { required_values_schema: plan.required_values_schema.clone(), values_consts: plan.values_consts.clone(), required_source_schema: plan.required_source_schema.clone(), @@ -159,16 +159,17 @@ impl CopyIntoTableInterpreter { source, is_transform: plan.is_transform, table_meta_timestamps, - })); + meta: PhysicalPlanMeta::new("CopyIntoTable"), + }); if plan.enable_distributed { - root = PhysicalPlan::Exchange(Exchange { - plan_id: 0, - input: Box::new(root), + root = Box::new(Exchange { + input: root, kind: FragmentKind::Merge, keys: Vec::new(), allow_adjust_parallelism: true, ignore_exchange: false, + meta: PhysicalPlanMeta::new("Exchange"), }); } diff --git a/src/query/service/src/interpreters/interpreter_explain.rs b/src/query/service/src/interpreters/interpreter_explain.rs index 88da04bed02bf..3726db02259f1 100644 --- a/src/query/service/src/interpreters/interpreter_explain.rs +++ b/src/query/service/src/interpreters/interpreter_explain.rs @@ -34,8 +34,6 @@ use databend_common_pipeline_core::processors::PlanProfile; use databend_common_pipeline_core::ExecutionInfo; use databend_common_pipeline_core::Pipeline; use databend_common_sql::binder::ExplainConfig; -use databend_common_sql::executor::format_partial_tree; -use databend_common_sql::executor::MutationBuildInfo; use databend_common_sql::plans::Mutation; use databend_common_sql::BindContext; use databend_common_sql::ColumnSet; @@ -55,6 +53,10 @@ use crate::interpreters::interpreter::on_execution_finished; use crate::interpreters::interpreter_mutation::build_mutation_info; use crate::interpreters::interpreter_mutation::MutationInterpreter; use crate::interpreters::Interpreter; +use crate::physical_plans::MutationBuildInfo; +use crate::physical_plans::PhysicalPlan; +use crate::physical_plans::PhysicalPlanBuilder; +use crate::physical_plans::PhysicalPlanDynExt; use crate::pipelines::executor::ExecutorSettings; use crate::pipelines::executor::PipelineCompleteExecutor; use crate::pipelines::executor::PipelinePullingExecutor; @@ -64,8 +66,6 @@ use crate::schedulers::build_query_pipeline; use crate::schedulers::Fragmenter; use crate::schedulers::QueryFragmentsActions; use crate::sessions::QueryContext; -use crate::sql::executor::PhysicalPlan; -use crate::sql::executor::PhysicalPlanBuilder; use crate::sql::optimizer::ir::SExpr; use crate::sql::plans::Plan; @@ -73,7 +73,6 @@ pub struct ExplainInterpreter { ctx: Arc, config: ExplainConfig, kind: ExplainKind, - partial: bool, graphical: bool, plan: Plan, } @@ -164,22 +163,11 @@ impl Interpreter for ExplainInterpreter { _ => self.explain_plan(&self.plan)?, }, - ExplainKind::Join => match &self.plan { - Plan::Query { - s_expr, - metadata, - bind_context, - .. - } => { - let ctx = self.ctx.clone(); - let mut builder = PhysicalPlanBuilder::new(metadata.clone(), ctx, true); - let plan = builder.build(s_expr, bind_context.column_set()).await?; - self.explain_join_order(&plan, metadata)? - } - _ => Err(ErrorCode::Unimplemented( + ExplainKind::Join => { + return Err(ErrorCode::Unimplemented( "Unsupported EXPLAIN JOIN statement", - ))?, - }, + )); + } ExplainKind::AnalyzePlan | ExplainKind::Graphical => match &self.plan { Plan::Query { @@ -299,7 +287,6 @@ impl ExplainInterpreter { plan: Plan, kind: ExplainKind, config: ExplainConfig, - partial: bool, graphical: bool, ) -> Result { Ok(ExplainInterpreter { @@ -307,7 +294,6 @@ impl ExplainInterpreter { plan, kind, config, - partial, graphical, }) } @@ -360,25 +346,15 @@ impl ExplainInterpreter { } } + let metadata = metadata.read(); let result = plan - .format(metadata.clone(), Default::default())? + .format(&metadata, Default::default())? .format_pretty()?; let line_split_result: Vec<&str> = result.lines().collect(); let formatted_plan = StringType::from_data(line_split_result); Ok(vec![DataBlock::new_from_columns(vec![formatted_plan])]) } - pub fn explain_join_order( - &self, - plan: &PhysicalPlan, - metadata: &MetadataRef, - ) -> Result> { - let result = plan.format_join(metadata)?.format_pretty()?; - let line_split_result: Vec<&str> = result.lines().collect(); - let formatted_plan = StringType::from_data(line_split_result); - Ok(vec![DataBlock::new_from_columns(vec![formatted_plan])]) - } - fn format_pipeline(build_res: &PipelineBuildResult) -> Vec { let mut blocks = Vec::with_capacity(1 + build_res.sources_pipelines.len()); // Format root pipeline @@ -412,10 +388,13 @@ impl ExplainInterpreter { .build(&s_expr, required) .await?; - let root_fragment = Fragmenter::try_create(ctx.clone())?.build_fragment(&plan)?; + let fragments = Fragmenter::try_create(ctx.clone())?.build_fragment(&plan)?; let mut fragments_actions = QueryFragmentsActions::create(ctx.clone()); - root_fragment.get_actions(ctx, &mut fragments_actions)?; + + for fragment in fragments { + fragment.get_actions(ctx.clone(), &mut fragments_actions)?; + } let display_string = fragments_actions.display_indent(&metadata).to_string(); let line_split_result = display_string.lines().collect::>(); @@ -479,10 +458,9 @@ impl ExplainInterpreter { if !pruned_partitions_stats.is_empty() { plan.set_pruning_stats(&mut pruned_partitions_stats); } - let result = if self.partial { - format_partial_tree(&plan, metadata, &query_profiles)?.format_pretty()? - } else { - plan.format(metadata.clone(), query_profiles.clone())? + let result = { + let metadata = metadata.read(); + plan.format(&metadata, query_profiles.clone())? .format_pretty()? }; let line_split_result: Vec<&str> = result.lines().collect(); @@ -570,10 +548,13 @@ impl ExplainInterpreter { mutation.metadata.clone(), )?; let plan = interpreter.build_physical_plan(&mutation, true).await?; - let root_fragment = Fragmenter::try_create(self.ctx.clone())?.build_fragment(&plan)?; + let fragments = Fragmenter::try_create(self.ctx.clone())?.build_fragment(&plan)?; let mut fragments_actions = QueryFragmentsActions::create(self.ctx.clone()); - root_fragment.get_actions(self.ctx.clone(), &mut fragments_actions)?; + + for fragment in fragments { + fragment.get_actions(self.ctx.clone(), &mut fragments_actions)?; + } let display_string = fragments_actions .display_indent(&mutation.metadata) diff --git a/src/query/service/src/interpreters/interpreter_factory.rs b/src/query/service/src/interpreters/interpreter_factory.rs index cbf191892f200..f3f4ea0670cea 100644 --- a/src/query/service/src/interpreters/interpreter_factory.rs +++ b/src/query/service/src/interpreters/interpreter_factory.rs @@ -227,7 +227,6 @@ impl InterpreterFactory { kind.clone(), config.clone(), false, - false, )?)), Plan::ExplainAst { formatted_string } => Ok(Arc::new(ExplainInterpreter::try_create( ctx, @@ -235,7 +234,6 @@ impl InterpreterFactory { ExplainKind::Ast(formatted_string.clone()), ExplainConfig::default(), false, - false, )?)), Plan::ExplainSyntax { formatted_sql } => Ok(Arc::new(ExplainInterpreter::try_create( ctx, @@ -243,18 +241,14 @@ impl InterpreterFactory { ExplainKind::Syntax(formatted_sql.clone()), ExplainConfig::default(), false, - false, )?)), Plan::ExplainAnalyze { - graphical, - partial, - plan, + graphical, plan, .. } => Ok(Arc::new(ExplainInterpreter::try_create( ctx, *plan.clone(), ExplainKind::AnalyzePlan, ExplainConfig::default(), - *partial, *graphical, )?)), Plan::ExplainPerf { sql } => Ok(Arc::new(ExplainPerfInterpreter::try_create( diff --git a/src/query/service/src/interpreters/interpreter_index_refresh.rs b/src/query/service/src/interpreters/interpreter_index_refresh.rs index ee4147d52ee1d..961e029f2095e 100644 --- a/src/query/service/src/interpreters/interpreter_index_refresh.rs +++ b/src/query/service/src/interpreters/interpreter_index_refresh.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::any::Any; use std::sync::Arc; use databend_common_base::runtime::GlobalIORuntime; @@ -34,10 +35,6 @@ use databend_common_pipeline_core::ExecutionInfo; use databend_common_pipeline_transforms::processors::TransformPipelineHelper; use databend_common_sql::evaluator::BlockOperator; use databend_common_sql::evaluator::CompoundBlockOperator; -use databend_common_sql::executor::physical_plans::TableScan; -use databend_common_sql::executor::PhysicalPlan; -use databend_common_sql::executor::PhysicalPlanBuilder; -use databend_common_sql::executor::PhysicalPlanReplacer; use databend_common_sql::plans::Plan; use databend_common_sql::plans::RefreshIndexPlan; use databend_common_sql::plans::RelOperator; @@ -51,6 +48,11 @@ use databend_enterprise_aggregating_index::get_agg_index_handler; use databend_storages_common_table_meta::meta::Location; use crate::interpreters::Interpreter; +use crate::physical_plans::DeriveHandle; +use crate::physical_plans::PhysicalPlan; +use crate::physical_plans::PhysicalPlanBuilder; +use crate::physical_plans::PhysicalPlanDynExt; +use crate::physical_plans::TableScan; use crate::pipelines::PipelineBuildResult; use crate::schedulers::build_query_pipeline_without_render_result_set; use crate::sessions::QueryContext; @@ -124,28 +126,16 @@ impl RefreshIndexInterpreter { fuse_table: Arc, segments: Option>, ) -> Result> { - let mut source = vec![]; + let mut sources = vec![]; + query_plan.get_all_data_source(&mut sources); - let mut collect_read_source = |plan: &PhysicalPlan| { - if let PhysicalPlan::TableScan(scan) = plan { - source.push(*scan.source.clone()) - } - }; - - PhysicalPlan::traverse( - query_plan, - &mut |_| true, - &mut collect_read_source, - &mut |_| {}, - ); - - if source.len() != 1 { + if sources.len() != 1 { Err(ErrorCode::Internal( "Invalid source with multiple table scan when do refresh aggregating index" .to_string(), )) } else { - let mut source = source.remove(0); + let (_, mut source) = sources.remove(0); let partitions = match segments { Some(segment_locs) if !segment_locs.is_empty() => { let segment_locations = create_segment_location_vector(segment_locs, None); @@ -187,7 +177,7 @@ impl RefreshIndexInterpreter { }; if !source.parts.is_empty() { - Ok(Some(source)) + Ok(Some(Box::into_inner(source))) } else { Ok(None) } @@ -278,10 +268,8 @@ impl Interpreter for RefreshIndexInterpreter { let new_index_meta = self.update_index_meta(&new_read_source)?; - let mut replace_read_source = ReadSourceReplacer { - source: new_read_source, - }; - query_plan = replace_read_source.replace(&query_plan)?; + let mut handle = ReadSourceDeriveHandle::create(new_read_source); + query_plan = query_plan.derive_with(&mut handle); let mut build_res = build_query_pipeline_without_render_result_set(&self.ctx, &query_plan).await?; @@ -377,14 +365,33 @@ async fn modify_last_update(ctx: Arc, req: UpdateIndexReq) -> Resu Ok(()) } -struct ReadSourceReplacer { +struct ReadSourceDeriveHandle { source: DataSourcePlan, } -impl PhysicalPlanReplacer for ReadSourceReplacer { - fn replace_table_scan(&mut self, plan: &TableScan) -> Result { - let mut plan = plan.clone(); - plan.source = Box::new(self.source.clone()); - Ok(PhysicalPlan::TableScan(plan)) +impl ReadSourceDeriveHandle { + pub fn create(source: DataSourcePlan) -> Box { + Box::new(ReadSourceDeriveHandle { source }) + } +} + +impl DeriveHandle for ReadSourceDeriveHandle { + fn as_any(&mut self) -> &mut dyn Any { + self + } + + fn derive( + &mut self, + v: &PhysicalPlan, + children: Vec, + ) -> std::result::Result> { + let Some(table_scan) = v.downcast_ref::() else { + return Err(children); + }; + + Ok(Box::new(TableScan { + source: Box::new(self.source.clone()), + ..table_scan.clone() + })) } } diff --git a/src/query/service/src/interpreters/interpreter_insert.rs b/src/query/service/src/interpreters/interpreter_insert.rs index 9e0f432e1a6ba..274adf1fbbc1b 100644 --- a/src/query/service/src/interpreters/interpreter_insert.rs +++ b/src/query/service/src/interpreters/interpreter_insert.rs @@ -26,10 +26,7 @@ use databend_common_expression::FromData; use databend_common_expression::SendableDataBlockStream; use databend_common_pipeline_sources::AsyncSourcer; use databend_common_pipeline_transforms::TransformPipelineHelper; -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; @@ -44,6 +41,12 @@ use crate::interpreters::common::dml_build_update_stream_req; use crate::interpreters::HookOperator; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; +use crate::physical_plans::DistributedInsertSelect; +use crate::physical_plans::Exchange; +use crate::physical_plans::IPhysicalPlan; +use crate::physical_plans::PhysicalPlanBuilder; +use crate::physical_plans::PhysicalPlanDynExt; +use crate::physical_plans::PhysicalPlanMeta; use crate::pipelines::processors::transforms::TransformAddConstColumns; use crate::pipelines::PipelineBuildResult; use crate::pipelines::PipelineBuilder; @@ -178,52 +181,52 @@ impl Interpreter for InsertInterpreter { _ => unreachable!(), }; - let explain_plan = select_plan - .format(metadata.clone(), Default::default())? - .format_pretty()?; + let explain_plan = { + let metadata = metadata.read(); + select_plan + .format(&metadata, Default::default())? + .format_pretty()? + }; + info!("Insert select plan: \n{}", explain_plan); let update_stream_meta = dml_build_update_stream_req(self.ctx.clone()).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) => { + let mut insert_select_plan = { + if table.support_distributed_insert() + && let Some(exchange) = select_plan.downcast_ref::() + { // 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)?, - table_meta_timestamps, - }, - ))); - PhysicalPlan::Exchange(exchange.clone()) - } - (other_plan, _) => { + exchange.derive(vec![Box::new(DistributedInsertSelect { + 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)?, + table_meta_timestamps, + meta: PhysicalPlanMeta::new("DistributedInsertSelect"), + })]) + } else { // insert should wait until all nodes finished - PhysicalPlan::DistributedInsertSelect(Box::new(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), + input: select_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)?, table_meta_timestamps, - })) + meta: PhysicalPlanMeta::new("DistributedInsertSelect"), + }) } }; + insert_select_plan.adjust_plan_id(&mut 0); let mut build_res = build_query_pipeline_without_render_result_set(&self.ctx, &insert_select_plan) .await?; diff --git a/src/query/service/src/interpreters/interpreter_insert_multi_table.rs b/src/query/service/src/interpreters/interpreter_insert_multi_table.rs index c3bd900836bc1..ade71f2e0d51e 100644 --- a/src/query/service/src/interpreters/interpreter_insert_multi_table.rs +++ b/src/query/service/src/interpreters/interpreter_insert_multi_table.rs @@ -27,20 +27,7 @@ use databend_common_expression::FromData; use databend_common_expression::RemoteExpr; use databend_common_expression::SendableDataBlockStream; use databend_common_functions::BUILTIN_FUNCTIONS; -use databend_common_sql::executor::physical_plans::CastSchema; -use databend_common_sql::executor::physical_plans::ChunkAppendData; -use databend_common_sql::executor::physical_plans::ChunkCastSchema; -use databend_common_sql::executor::physical_plans::ChunkCommitInsert; -use databend_common_sql::executor::physical_plans::ChunkEvalScalar; -use databend_common_sql::executor::physical_plans::ChunkFillAndReorder; -use databend_common_sql::executor::physical_plans::ChunkMerge; -use databend_common_sql::executor::physical_plans::FillAndReorder; -use databend_common_sql::executor::physical_plans::MultiInsertEvalScalar; use databend_common_sql::executor::physical_plans::MutationKind; -use databend_common_sql::executor::physical_plans::SerializableTable; -use databend_common_sql::executor::physical_plans::ShuffleStrategy; -use databend_common_sql::executor::PhysicalPlan; -use databend_common_sql::executor::PhysicalPlanBuilder; use databend_common_sql::plans::Else; use databend_common_sql::plans::FunctionCall; use databend_common_sql::plans::InsertMultiTable; @@ -54,14 +41,28 @@ use super::HookOperator; use crate::interpreters::common::dml_build_update_stream_req; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; +use crate::physical_plans::CastSchema; +use crate::physical_plans::ChunkAppendData; +use crate::physical_plans::ChunkCastSchema; +use crate::physical_plans::ChunkCommitInsert; +use crate::physical_plans::ChunkEvalScalar; +use crate::physical_plans::ChunkFillAndReorder; +use crate::physical_plans::ChunkFilter; +use crate::physical_plans::ChunkMerge; +use crate::physical_plans::Duplicate; +use crate::physical_plans::FillAndReorder; +use crate::physical_plans::MultiInsertEvalScalar; +use crate::physical_plans::PhysicalPlan; +use crate::physical_plans::PhysicalPlanBuilder; +use crate::physical_plans::PhysicalPlanMeta; +use crate::physical_plans::SerializableTable; +use crate::physical_plans::Shuffle; +use crate::physical_plans::ShuffleStrategy; use crate::pipelines::PipelineBuildResult; use crate::schedulers::build_query_pipeline_without_render_result_set; use crate::sessions::QueryContext; use crate::sessions::TableContext; use crate::sql::executor::cast_expr_to_non_null_boolean; -use crate::sql::executor::physical_plans::ChunkFilter; -use crate::sql::executor::physical_plans::Duplicate; -use crate::sql::executor::physical_plans::Shuffle; use crate::storages::Table; use crate::stream::DataBlockStream; pub struct InsertMultiTableInterpreter { @@ -174,63 +175,65 @@ impl InsertMultiTableInterpreter { let fill_and_reorders = branches.build_fill_and_reorder(self.ctx.clone()).await?; let group_ids = branches.build_group_ids(); - root = PhysicalPlan::Duplicate(Box::new(Duplicate { - plan_id: 0, - input: Box::new(root), + root = Box::new(Duplicate { + input: root, n: branches.len(), - })); + meta: PhysicalPlanMeta::new("Duplicate"), + }); let shuffle_strategy = ShuffleStrategy::Transpose(branches.len()); - root = PhysicalPlan::Shuffle(Box::new(Shuffle { - plan_id: 0, - input: Box::new(root), + root = Box::new(Shuffle { + input: root, strategy: shuffle_strategy, - })); + meta: PhysicalPlanMeta::new("Shuffle"), + }); - root = PhysicalPlan::ChunkFilter(Box::new(ChunkFilter { - plan_id: 0, - input: Box::new(root), + root = Box::new(ChunkFilter { predicates, - })); + input: root, + meta: PhysicalPlanMeta::new("ChunkFilter"), + }); - root = PhysicalPlan::ChunkEvalScalar(Box::new(ChunkEvalScalar { - plan_id: 0, - input: Box::new(root), + root = Box::new(ChunkEvalScalar { eval_scalars, - })); + input: root, + meta: PhysicalPlanMeta::new("ChunkEvalScalar"), + }); - root = PhysicalPlan::ChunkCastSchema(Box::new(ChunkCastSchema { - plan_id: 0, - input: Box::new(root), + root = Box::new(ChunkCastSchema { cast_schemas, - })); + input: root, + meta: PhysicalPlanMeta::new("ChunkCastSchema"), + }); - root = PhysicalPlan::ChunkFillAndReorder(Box::new(ChunkFillAndReorder { - plan_id: 0, - input: Box::new(root), + root = Box::new(ChunkFillAndReorder { fill_and_reorders, - })); + input: root, + meta: PhysicalPlanMeta::new("ChunkFillAndReorder"), + }); - root = PhysicalPlan::ChunkAppendData(Box::new(ChunkAppendData { - plan_id: 0, - input: Box::new(root), + root = Box::new(ChunkAppendData { + input: root, target_tables: serializable_tables.clone(), - })); + meta: PhysicalPlanMeta::new("ChunkAppendData"), + }); - root = PhysicalPlan::ChunkMerge(Box::new(ChunkMerge { - plan_id: 0, - input: Box::new(root), + root = Box::new(ChunkMerge { group_ids, - })); + input: root, + meta: PhysicalPlanMeta::new("ChunkMerge"), + }); - root = PhysicalPlan::ChunkCommitInsert(Box::new(ChunkCommitInsert { - plan_id: 0, - input: Box::new(root), + root = Box::new(ChunkCommitInsert { update_stream_meta, + + input: root, overwrite: self.plan.overwrite, deduplicated_label: None, targets: deduplicated_serializable_tables, - })); + meta: PhysicalPlanMeta::new("ChunkCommitInsert"), + }); + let mut next_plan_id = 0; root.adjust_plan_id(&mut next_plan_id); Ok(root) diff --git a/src/query/service/src/interpreters/interpreter_mutation.rs b/src/query/service/src/interpreters/interpreter_mutation.rs index 2e41ed14faaec..98798e086c529 100644 --- a/src/query/service/src/interpreters/interpreter_mutation.rs +++ b/src/query/service/src/interpreters/interpreter_mutation.rs @@ -30,11 +30,7 @@ use databend_common_pipeline_core::processors::ProcessorPtr; use databend_common_pipeline_sinks::EmptySink; use databend_common_sql::binder::MutationStrategy; use databend_common_sql::binder::MutationType; -use databend_common_sql::executor::physical_plans::create_push_down_filters; use databend_common_sql::executor::physical_plans::MutationKind; -use databend_common_sql::executor::MutationBuildInfo; -use databend_common_sql::executor::PhysicalPlan; -use databend_common_sql::executor::PhysicalPlanBuilder; use databend_common_sql::optimizer::ir::SExpr; use databend_common_sql::planner::MetadataRef; use databend_common_sql::plans; @@ -49,6 +45,11 @@ 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::physical_plans::create_push_down_filters; +use crate::physical_plans::MutationBuildInfo; +use crate::physical_plans::PhysicalPlan; +use crate::physical_plans::PhysicalPlanBuilder; +use crate::physical_plans::PhysicalPlanDynExt; use crate::pipelines::PipelineBuildResult; use crate::schedulers::build_query_pipeline_without_render_result_set; use crate::sessions::QueryContext; @@ -98,9 +99,12 @@ impl Interpreter for MutationInterpreter { // Build physical plan. let physical_plan = self.build_physical_plan(&mutation, false).await?; - let query_plan = physical_plan - .format(self.metadata.clone(), Default::default())? - .format_pretty()?; + let query_plan = { + let metadata = self.metadata.read(); + physical_plan + .format(&metadata, Default::default())? + .format_pretty()? + }; info!("Query physical plan: \n{}", query_plan); diff --git a/src/query/service/src/interpreters/interpreter_optimize_compact_block.rs b/src/query/service/src/interpreters/interpreter_optimize_compact_block.rs index 70560eb5021b5..9154769840d09 100644 --- a/src/query/service/src/interpreters/interpreter_optimize_compact_block.rs +++ b/src/query/service/src/interpreters/interpreter_optimize_compact_block.rs @@ -19,7 +19,6 @@ use databend_common_catalog::lock::LockTableOption; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_pipeline_core::ExecutionInfo; -use databend_common_sql::executor::PhysicalPlanBuilder; use databend_common_sql::optimizer::ir::SExpr; use databend_common_sql::plans::OptimizeCompactBlock; use databend_common_sql::ColumnSet; @@ -27,6 +26,7 @@ use databend_common_sql::MetadataRef; use crate::interpreters::interpreter_optimize_purge::purge; use crate::interpreters::Interpreter; +use crate::physical_plans::PhysicalPlanBuilder; use crate::pipelines::PipelineBuildResult; use crate::schedulers::build_query_pipeline_without_render_result_set; use crate::sessions::QueryContext; diff --git a/src/query/service/src/interpreters/interpreter_replace.rs b/src/query/service/src/interpreters/interpreter_replace.rs index 2a368bdd85f03..4aa19cf9a6496 100644 --- a/src/query/service/src/interpreters/interpreter_replace.rs +++ b/src/query/service/src/interpreters/interpreter_replace.rs @@ -26,17 +26,9 @@ use databend_common_meta_app::principal::StageInfo; use databend_common_meta_app::schema::TableInfo; use databend_common_meta_app::schema::UpdateStreamMetaReq; use databend_common_sql::executor::cast_expr_to_non_null_boolean; -use databend_common_sql::executor::physical_plans::CommitSink; -use databend_common_sql::executor::physical_plans::CommitType; -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::ReplaceDeduplicate; -use databend_common_sql::executor::physical_plans::ReplaceInto; -use databend_common_sql::executor::physical_plans::ReplaceSelectCtx; -use databend_common_sql::executor::PhysicalPlan; use databend_common_sql::plans::InsertInputSource; use databend_common_sql::plans::InsertValue; use databend_common_sql::plans::Plan; @@ -60,6 +52,16 @@ use crate::interpreters::HookOperator; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; use crate::interpreters::SelectInterpreter; +use crate::physical_plans::CommitSink; +use crate::physical_plans::CommitType; +use crate::physical_plans::Exchange; +use crate::physical_plans::PhysicalPlan; +use crate::physical_plans::PhysicalPlanDynExt; +use crate::physical_plans::PhysicalPlanMeta; +use crate::physical_plans::ReplaceAsyncSourcer; +use crate::physical_plans::ReplaceDeduplicate; +use crate::physical_plans::ReplaceInto; +use crate::physical_plans::ReplaceSelectCtx; use crate::pipelines::PipelineBuildResult; use crate::pipelines::PipelineBuilder; use crate::schedulers::build_query_pipeline_without_render_result_set; @@ -134,7 +136,7 @@ impl ReplaceInterpreter { async fn build_physical_plan( &self, ) -> Result<( - Box, + PhysicalPlan, Option<(Vec, StageInfo, CopyIntoTableOptions)>, )> { let plan = &self.plan; @@ -266,34 +268,31 @@ impl ReplaceInterpreter { let mut is_exchange = false; let is_stage_source = matches!(self.plan.source, InsertInputSource::Stage(_)); - if let PhysicalPlan::Exchange(Exchange { - input, - kind: FragmentKind::Merge, - .. - }) = root.as_ref() + if let Some(exchange) = root.downcast_ref::() + && exchange.kind == FragmentKind::Merge { is_exchange = true; - root = input.clone(); + root = exchange.input.clone(); } if is_distributed { - root = Box::new(PhysicalPlan::Exchange(Exchange { - plan_id: 0, + root = Box::new(Exchange { input: root, kind: FragmentKind::Expansive, keys: vec![], allow_adjust_parallelism: true, ignore_exchange: false, - })); + meta: PhysicalPlanMeta::new("Exchange"), + }); } else if is_exchange && !is_stage_source { - root = Box::new(PhysicalPlan::Exchange(Exchange { - plan_id: 0, + root = Box::new(Exchange { input: root, kind: FragmentKind::Merge, keys: vec![], allow_adjust_parallelism: true, ignore_exchange: false, - })); + meta: PhysicalPlanMeta::new("Exchange"), + }); } let max_num_pruning_columns = self @@ -315,23 +314,21 @@ impl ReplaceInterpreter { vec![] }; - root = Box::new(PhysicalPlan::ReplaceDeduplicate(Box::new( - ReplaceDeduplicate { - input: root, - on_conflicts: on_conflicts.clone(), - bloom_filter_column_indexes: bloom_filter_column_indexes.clone(), - table_is_empty, - table_info: table_info.clone(), - select_ctx, - target_schema: plan.schema.clone(), - table_level_range_index, - need_insert: true, - delete_when, - plan_id: u32::MAX, - }, - ))); + root = Box::new(ReplaceDeduplicate { + input: root, + on_conflicts: on_conflicts.clone(), + bloom_filter_column_indexes: bloom_filter_column_indexes.clone(), + table_is_empty, + table_info: table_info.clone(), + select_ctx, + target_schema: plan.schema.clone(), + table_level_range_index, + need_insert: true, + delete_when, + meta: PhysicalPlanMeta::new("ReplaceDeduplicate"), + }); - root = Box::new(PhysicalPlan::ReplaceInto(Box::new(ReplaceInto { + root = Box::new(ReplaceInto { input: root, block_thresholds: fuse_table.get_block_thresholds(), table_info: table_info.clone(), @@ -345,22 +342,22 @@ impl ReplaceInterpreter { .collect(), block_slots: None, need_insert: true, - plan_id: u32::MAX, table_meta_timestamps, - }))); + meta: PhysicalPlanMeta::new("ReplaceInto"), + }); if is_distributed { - root = Box::new(PhysicalPlan::Exchange(Exchange { - plan_id: 0, + root = Box::new(Exchange { input: root, kind: FragmentKind::Merge, keys: vec![], allow_adjust_parallelism: true, ignore_exchange: false, - })); + meta: PhysicalPlanMeta::new("Exchange"), + }); } - root = Box::new(PhysicalPlan::CommitSink(Box::new(CommitSink { + root = Box::new(CommitSink { input: root, snapshot: base_snapshot, table_info: table_info.clone(), @@ -370,10 +367,11 @@ impl ReplaceInterpreter { }, update_stream_meta: update_stream_meta.clone(), deduplicated_label: unsafe { self.ctx.get_settings().get_deduplicate_label()? }, - plan_id: u32::MAX, table_meta_timestamps, recluster_info: None, - }))); + meta: PhysicalPlanMeta::new("CommitSink"), + }); + root.adjust_plan_id(&mut 0); Ok((root, purge_info)) } @@ -427,7 +425,7 @@ impl ReplaceInterpreter { copy_plan.stage_table_info.copy_into_table_options.clone(), )); Ok(ReplaceSourceCtx { - root: Box::new(physical_plan), + root: physical_plan, select_ctx: None, update_stream_meta: vec![], bind_context: None, @@ -445,14 +443,12 @@ impl ReplaceInterpreter { &self, schema: DataSchemaRef, source: &InsertValue, - ) -> Result> { - Ok(Box::new(PhysicalPlan::ReplaceAsyncSourcer( - ReplaceAsyncSourcer { - schema, - plan_id: u32::MAX, - source: source.clone(), - }, - ))) + ) -> Result { + Ok(Box::new(ReplaceAsyncSourcer { + schema, + source: source.clone(), + meta: PhysicalPlanMeta::new("ReplaceAsyncSourcer"), + })) } #[async_backtrace::framed] @@ -483,10 +479,7 @@ impl ReplaceInterpreter { false, )?; - let physical_plan = select_interpreter - .build_physical_plan() - .await - .map(Box::new)?; + let physical_plan = select_interpreter.build_physical_plan().await?; let select_ctx = ReplaceSelectCtx { select_column_bindings: bind_context.columns.clone(), select_schema: query_plan.schema(), @@ -501,7 +494,7 @@ impl ReplaceInterpreter { } struct ReplaceSourceCtx { - root: Box, + root: PhysicalPlan, select_ctx: Option, update_stream_meta: Vec, bind_context: Option, diff --git a/src/query/service/src/interpreters/interpreter_select.rs b/src/query/service/src/interpreters/interpreter_select.rs index e7e505f4ab162..b70082a80fdf6 100644 --- a/src/query/service/src/interpreters/interpreter_select.rs +++ b/src/query/service/src/interpreters/interpreter_select.rs @@ -33,7 +33,6 @@ use databend_common_pipeline_core::PipeItem; use databend_common_pipeline_core::Pipeline; use databend_common_pipeline_transforms::processors::TransformDummy; use databend_common_sql::executor::physical_plans::FragmentKind; -use databend_common_sql::executor::PhysicalPlan; use databend_common_sql::parse_result_scan_args; use databend_common_sql::ColumnBinding; use databend_common_sql::MetadataRef; @@ -46,11 +45,14 @@ use log::info; use crate::interpreters::common::query_build_update_stream_req; use crate::interpreters::Interpreter; +use crate::physical_plans::Exchange; +use crate::physical_plans::PhysicalPlan; +use crate::physical_plans::PhysicalPlanBuilder; +use crate::physical_plans::PhysicalPlanDynExt; use crate::pipelines::PipelineBuildResult; use crate::schedulers::build_query_pipeline; use crate::sessions::QueryContext; use crate::sessions::TableContext; -use crate::sql::executor::PhysicalPlanBuilder; use crate::sql::optimizer::ir::SExpr; use crate::sql::BindContext; @@ -125,7 +127,7 @@ impl SelectInterpreter { &self, mut physical_plan: PhysicalPlan, ) -> Result { - if let PhysicalPlan::Exchange(exchange) = &mut physical_plan { + if let Some(exchange) = physical_plan.downcast_mut_ref::() { if exchange.kind == FragmentKind::Merge && self.ignore_result { exchange.ignore_exchange = self.ignore_result; } @@ -286,9 +288,13 @@ impl Interpreter for SelectInterpreter { // 0. Need to build physical plan first to get the partitions. let physical_plan = self.build_physical_plan().await?; - let query_plan = physical_plan - .format(self.metadata.clone(), Default::default())? - .format_pretty()?; + + let query_plan = { + let metadata = self.metadata.read(); + physical_plan + .format(&metadata, Default::default())? + .format_pretty()? + }; info!("[SELECT-INTERP] Query physical plan:\n{}", query_plan); diff --git a/src/query/service/src/interpreters/interpreter_table_analyze.rs b/src/query/service/src/interpreters/interpreter_table_analyze.rs index 3d784a9ad7cb0..8d620fb67ae86 100644 --- a/src/query/service/src/interpreters/interpreter_table_analyze.rs +++ b/src/query/service/src/interpreters/interpreter_table_analyze.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::any::Any; use std::collections::HashMap; use std::sync::Arc; @@ -19,15 +20,6 @@ use chrono::Utc; use databend_common_catalog::table::TableExt; use databend_common_exception::Result; use databend_common_pipeline_core::processors::ProcessorPtr; -use databend_common_sql::executor::physical_plans::AggregateExpand; -use databend_common_sql::executor::physical_plans::AggregateFinal; -use databend_common_sql::executor::physical_plans::AggregatePartial; -use databend_common_sql::executor::physical_plans::EvalScalar; -use databend_common_sql::executor::physical_plans::Filter; -use databend_common_sql::executor::physical_plans::Sort; -use databend_common_sql::executor::physical_plans::Window; -use databend_common_sql::executor::PhysicalPlan; -use databend_common_sql::executor::PhysicalPlanBuilder; use databend_common_sql::plans::AnalyzeTablePlan; use databend_common_sql::plans::Plan; use databend_common_sql::BindContext; @@ -44,6 +36,11 @@ use itertools::Itertools; use log::info; use crate::interpreters::Interpreter; +use crate::physical_plans::DeriveHandle; +use crate::physical_plans::Exchange; +use crate::physical_plans::PhysicalPlan; +use crate::physical_plans::PhysicalPlanBuilder; +use crate::physical_plans::PhysicalPlanDynExt; use crate::pipelines::PipelineBuildResult; use crate::schedulers::build_query_pipeline; use crate::schedulers::build_query_pipeline_without_render_result_set; @@ -279,75 +276,32 @@ impl Interpreter for AnalyzeTableInterpreter { } fn remove_exchange(plan: PhysicalPlan) -> PhysicalPlan { - #[recursive::recursive] - fn traverse(plan: PhysicalPlan) -> PhysicalPlan { - match plan { - PhysicalPlan::Filter(plan) => PhysicalPlan::Filter(Filter { - plan_id: plan.plan_id, - projections: plan.projections, - input: Box::new(traverse(*plan.input)), - predicates: plan.predicates, - stat_info: plan.stat_info, - }), - PhysicalPlan::EvalScalar(plan) => PhysicalPlan::EvalScalar(EvalScalar { - plan_id: plan.plan_id, - projections: plan.projections, - input: Box::new(traverse(*plan.input)), - exprs: plan.exprs, - stat_info: plan.stat_info, - }), - PhysicalPlan::AggregateExpand(plan) => PhysicalPlan::AggregateExpand(AggregateExpand { - plan_id: plan.plan_id, - input: Box::new(traverse(*plan.input)), - group_bys: plan.group_bys, - grouping_sets: plan.grouping_sets, - stat_info: plan.stat_info, - }), - PhysicalPlan::AggregatePartial(plan) => { - PhysicalPlan::AggregatePartial(AggregatePartial { - plan_id: plan.plan_id, - input: Box::new(traverse(*plan.input)), - group_by: plan.group_by, - agg_funcs: plan.agg_funcs, - rank_limit: plan.rank_limit, - enable_experimental_aggregate_hashtable: plan - .enable_experimental_aggregate_hashtable, - group_by_display: plan.group_by_display, - stat_info: plan.stat_info, - }) - } - PhysicalPlan::AggregateFinal(plan) => PhysicalPlan::AggregateFinal(AggregateFinal { - plan_id: plan.plan_id, - input: Box::new(traverse(*plan.input)), - group_by: plan.group_by, - agg_funcs: plan.agg_funcs, - before_group_by_schema: plan.before_group_by_schema, - group_by_display: plan.group_by_display, - stat_info: plan.stat_info, - }), - PhysicalPlan::Window(plan) => PhysicalPlan::Window(Window { - plan_id: plan.plan_id, - index: plan.index, - input: Box::new(traverse(*plan.input)), - func: plan.func, - partition_by: plan.partition_by, - order_by: plan.order_by, - window_frame: plan.window_frame, - limit: plan.limit, - }), - PhysicalPlan::Sort(plan) => PhysicalPlan::Sort(Sort { - plan_id: plan.plan_id, - input: Box::new(traverse(*plan.input)), - order_by: plan.order_by, - limit: plan.limit, - after_exchange: plan.after_exchange, - pre_projection: plan.pre_projection, - stat_info: plan.stat_info, - }), - PhysicalPlan::Exchange(plan) => traverse(*plan.input), - _ => plan, + struct RemoveExchangeHandle; + + impl RemoveExchangeHandle { + pub fn create() -> Box { + Box::new(RemoveExchangeHandle) + } + } + + impl DeriveHandle for RemoveExchangeHandle { + fn as_any(&mut self) -> &mut dyn Any { + self + } + + fn derive( + &mut self, + v: &PhysicalPlan, + mut children: Vec, + ) -> std::result::Result> { + let Some(_) = v.downcast_ref::() else { + return Err(children); + }; + + assert_eq!(children.len(), 1); + Ok(children.remove(0)) } } - traverse(plan) + plan.derive_with(&mut RemoveExchangeHandle::create()) } 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 6e4af8c2e69a9..415737a3c7e2b 100644 --- a/src/query/service/src/interpreters/interpreter_table_modify_column.rs +++ b/src/query/service/src/interpreters/interpreter_table_modify_column.rs @@ -38,9 +38,6 @@ use databend_common_meta_app::schema::TableInfo; 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::plans::ModifyColumnAction; use databend_common_sql::plans::ModifyTableColumnPlan; use databend_common_sql::plans::Plan; @@ -61,6 +58,10 @@ use databend_storages_common_table_meta::table::OPT_KEY_BLOOM_INDEX_COLUMNS; use crate::interpreters::common::check_referenced_computed_columns; use crate::interpreters::interpreter_table_add_column::commit_table_meta; use crate::interpreters::Interpreter; +use crate::physical_plans::DistributedInsertSelect; +use crate::physical_plans::PhysicalPlan; +use crate::physical_plans::PhysicalPlanBuilder; +use crate::physical_plans::PhysicalPlanMeta; use crate::pipelines::PipelineBuildResult; use crate::schedulers::build_query_pipeline_without_render_result_set; use crate::sessions::QueryContext; @@ -727,16 +728,19 @@ pub(crate) async fn build_select_insert_plan( let new_table = FuseTable::try_create(table_info)?; // 4. build DistributedInsertSelect plan - let insert_plan = PhysicalPlan::DistributedInsertSelect(Box::new(DistributedInsertSelect { - plan_id: select_plan.get_id(), - input: Box::new(select_plan), + let mut insert_plan: PhysicalPlan = Box::new(DistributedInsertSelect { + input: select_plan, table_info: new_table.get_table_info().clone(), select_schema, select_column_bindings, insert_schema: Arc::new(new_schema.into()), cast_needed: true, table_meta_timestamps, - })); + meta: PhysicalPlanMeta::new("DistributedInsertSelect"), + }); + + let mut index = 0; + insert_plan.adjust_plan_id(&mut index); let mut build_res = build_query_pipeline_without_render_result_set(&ctx, &insert_plan).await?; // 5. commit new meta schema and snapshots diff --git a/src/query/service/src/interpreters/interpreter_table_recluster.rs b/src/query/service/src/interpreters/interpreter_table_recluster.rs index f3c53597b06d7..cd499bd26df4d 100644 --- a/src/query/service/src/interpreters/interpreter_table_recluster.rs +++ b/src/query/service/src/interpreters/interpreter_table_recluster.rs @@ -38,16 +38,8 @@ use databend_common_pipeline_core::always_callback; use databend_common_pipeline_core::ExecutionInfo; use databend_common_sql::bind_table; use databend_common_sql::executor::cast_expr_to_non_null_boolean; -use databend_common_sql::executor::physical_plans::CommitSink; -use databend_common_sql::executor::physical_plans::CommitType; -use databend_common_sql::executor::physical_plans::CompactSource; -use databend_common_sql::executor::physical_plans::Exchange; use databend_common_sql::executor::physical_plans::FragmentKind; -use databend_common_sql::executor::physical_plans::HilbertPartition; use databend_common_sql::executor::physical_plans::MutationKind; -use databend_common_sql::executor::physical_plans::Recluster; -use databend_common_sql::executor::PhysicalPlan; -use databend_common_sql::executor::PhysicalPlanBuilder; use databend_common_sql::plans::plan_hilbert_sql; use databend_common_sql::plans::replace_with_constant; use databend_common_sql::plans::set_update_stream_columns; @@ -57,7 +49,6 @@ use databend_common_sql::plans::ReclusterPlan; use databend_common_sql::IdentifierNormalizer; use databend_common_sql::MetadataRef; use databend_common_sql::NameResolutionContext; -use databend_common_sql::QueryExecutor; use databend_common_sql::ScalarExpr; use databend_common_sql::TypeChecker; use databend_enterprise_hilbert_clustering::get_hilbert_clustering_handler; @@ -74,6 +65,16 @@ use crate::interpreters::hook::vacuum_hook::hook_vacuum_temp_files; use crate::interpreters::interpreter_insert_multi_table::scalar_expr_to_remote_expr; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterClusteringHistory; +use crate::physical_plans::physical_plan_builder::PhysicalPlanBuilder; +use crate::physical_plans::CommitSink; +use crate::physical_plans::CommitType; +use crate::physical_plans::CompactSource; +use crate::physical_plans::Exchange; +use crate::physical_plans::HilbertPartition; +use crate::physical_plans::PhysicalPlan; +use crate::physical_plans::PhysicalPlanDynExt; +use crate::physical_plans::PhysicalPlanMeta; +use crate::physical_plans::Recluster; use crate::pipelines::executor::ExecutorSettings; use crate::pipelines::executor::PipelineCompleteExecutor; use crate::pipelines::PipelineBuildResult; @@ -330,11 +331,11 @@ impl ReclusterTableInterpreter { // Adjust number of partitions according to the block size thresholds if total_partitions < block_thresholds.block_per_segment && block_thresholds.check_perfect_segment( - block_thresholds.block_per_segment, // this effectively by-pass the total_blocks criteria - total_rows, - total_bytes, - total_compressed, - ) + block_thresholds.block_per_segment, // this effectively by-pass the total_blocks criteria + total_rows, + total_bytes, + total_compressed, + ) { total_partitions = block_thresholds.block_per_segment; } @@ -419,18 +420,15 @@ impl ReclusterTableInterpreter { metadata.write().replace_all_tables(tbl.clone()); let mut builder = PhysicalPlanBuilder::new(metadata.clone(), self.ctx.clone(), false); - let mut plan = Box::new(builder.build(&s_expr, bind_context.column_set()).await?); + let mut plan = builder.build(&s_expr, bind_context.column_set()).await?; // Check if the plan already has an exchange operator let mut is_exchange = false; - if let PhysicalPlan::Exchange(Exchange { - input, - kind: FragmentKind::Merge, - .. - }) = plan.as_ref() + if let Some(exchange) = plan.downcast_ref::() + && exchange.kind == FragmentKind::Merge { is_exchange = true; - plan = input.clone(); + plan = exchange.input.clone(); } // Determine if we need distributed execution @@ -451,14 +449,14 @@ impl ReclusterTableInterpreter { // Add exchange operator for data distribution, // shuffling data based on the hash of range partition IDs derived from the Hilbert index. - plan = Box::new(PhysicalPlan::Exchange(Exchange { - plan_id: 0, + plan = Box::new(Exchange { input: plan, kind: FragmentKind::Normal, keys: vec![expr], allow_adjust_parallelism: true, ignore_exchange: false, - })); + meta: PhysicalPlanMeta::new("Exchange"), + }); } let table_meta_timestamps = self @@ -467,14 +465,15 @@ impl ReclusterTableInterpreter { // Create the Hilbert partition physical plan, // collecting data into partitions and persist them - let plan = PhysicalPlan::HilbertPartition(Box::new(HilbertPartition { - plan_id: 0, + plan = Box::new(HilbertPartition { + rows_per_block, + table_meta_timestamps, + input: plan, table_info: table_info.clone(), num_partitions: total_partitions, - table_meta_timestamps, - rows_per_block, - })); + meta: PhysicalPlanMeta::new("HilbertPartition"), + }); // Finally, commit the newly clustered table Ok(Some(Self::add_commit_sink( @@ -516,12 +515,13 @@ impl ReclusterTableInterpreter { removed_segment_indexes, removed_segment_summary, } => { - let root = PhysicalPlan::Recluster(Box::new(Recluster { + let root = Box::new(Recluster { tasks, - table_info: table_info.clone(), - plan_id: u32::MAX, table_meta_timestamps, - })); + + table_info: table_info.clone(), + meta: PhysicalPlanMeta::new("Recluster"), + }); Self::add_commit_sink( root, @@ -539,13 +539,13 @@ impl ReclusterTableInterpreter { } ReclusterParts::Compact(parts) => { let merge_meta = parts.partitions_type() == PartInfoType::LazyLevel; - let root = PhysicalPlan::CompactSource(Box::new(CompactSource { + let root = Box::new(CompactSource { parts, table_info: table_info.clone(), column_ids: snapshot.schema.to_leaf_column_id_set(), - plan_id: u32::MAX, table_meta_timestamps, - })); + meta: PhysicalPlanMeta::new("CompactSource"), + }); Self::add_commit_sink( root, @@ -704,7 +704,7 @@ impl ReclusterTableInterpreter { } fn add_commit_sink( - input: PhysicalPlan, + mut input: PhysicalPlan, is_distributed: bool, table_info: TableInfo, snapshot: Arc, @@ -712,35 +712,34 @@ impl ReclusterTableInterpreter { recluster_info: Option, table_meta_timestamps: TableMetaTimestamps, ) -> PhysicalPlan { - let plan = if is_distributed { - PhysicalPlan::Exchange(Exchange { - plan_id: 0, - input: Box::new(input), + if is_distributed { + input = Box::new(Exchange { + input, kind: FragmentKind::Merge, keys: vec![], allow_adjust_parallelism: true, ignore_exchange: false, - }) - } else { - input - }; + meta: PhysicalPlanMeta::new("Exchange"), + }); + } - let kind = if recluster_info.is_some() { - MutationKind::Recluster - } else { - MutationKind::Compact - }; - PhysicalPlan::CommitSink(Box::new(CommitSink { - input: Box::new(plan), + let mut kind = MutationKind::Compact; + + if recluster_info.is_some() { + kind = MutationKind::Recluster + } + + Box::new(CommitSink { + input, table_info, snapshot: Some(snapshot), commit_type: CommitType::Mutation { kind, merge_meta }, update_stream_meta: vec![], deduplicated_label: None, table_meta_timestamps, - plan_id: u32::MAX, recluster_info, - })) + meta: PhysicalPlanMeta::new("CommitSink"), + }) } } diff --git a/src/query/service/src/lib.rs b/src/query/service/src/lib.rs index 69a9ba3b420e5..bcf14ab576fc5 100644 --- a/src/query/service/src/lib.rs +++ b/src/query/service/src/lib.rs @@ -37,6 +37,7 @@ #![feature(variant_count)] #![feature(duration_constructors)] #![feature(get_mut_unchecked)] +#![feature(box_into_inner)] #![allow(clippy::diverging_sub_expression)] #![allow(clippy::arc_with_non_send_sync)] @@ -63,6 +64,8 @@ pub mod test_kits; mod global_services; pub mod task; +pub mod physical_plans; + pub use databend_common_sql as sql; pub use databend_common_storages_factory as storages; pub use global_services::GlobalServices; diff --git a/src/query/sql/src/executor/explain.rs b/src/query/service/src/physical_plans/explain.rs similarity index 100% rename from src/query/sql/src/executor/explain.rs rename to src/query/service/src/physical_plans/explain.rs diff --git a/src/query/service/src/physical_plans/format.rs b/src/query/service/src/physical_plans/format.rs new file mode 100644 index 0000000000000..50ef083ee3eff --- /dev/null +++ b/src/query/service/src/physical_plans/format.rs @@ -0,0 +1,175 @@ +// 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::HashMap; + +use databend_common_ast::ast::FormatTreeNode; +use databend_common_base::base::format_byte_size; +use databend_common_catalog::plan::PartStatistics; +use databend_common_expression::DataSchemaRef; +use databend_common_sql::executor::physical_plans::AggregateFunctionDesc; +use databend_common_sql::IndexType; +use databend_common_sql::Metadata; + +use crate::physical_plans::explain::PlanStatsInfo; +use crate::physical_plans::PhysicalRuntimeFilter; + +pub struct FormatContext<'a> { + pub metadata: &'a Metadata, + pub scan_id_to_runtime_filters: HashMap>, +} + +pub fn pretty_display_agg_desc(desc: &AggregateFunctionDesc, metadata: &Metadata) -> String { + format!( + "{}({})", + desc.sig.name, + desc.arg_indices + .iter() + .map(|&index| { metadata.column(index).name() }) + .collect::>() + .join(", ") + ) +} + +pub fn part_stats_info_to_format_tree(info: &PartStatistics) -> Vec> { + let read_size = format_byte_size(info.read_bytes); + let mut items = vec![ + FormatTreeNode::new(format!("read rows: {}", info.read_rows)), + FormatTreeNode::new(format!("read size: {}", read_size)), + FormatTreeNode::new(format!("partitions total: {}", info.partitions_total)), + FormatTreeNode::new(format!("partitions scanned: {}", info.partitions_scanned)), + ]; + + // format is like "pruning stats: [segments: , blocks: ]" + let mut blocks_pruning_description = String::new(); + + // range pruning status. + if info.pruning_stats.blocks_range_pruning_before > 0 { + blocks_pruning_description += &format!( + "range pruning: {} to {}", + info.pruning_stats.blocks_range_pruning_before, + info.pruning_stats.blocks_range_pruning_after + ); + } + + // bloom pruning status. + if info.pruning_stats.blocks_bloom_pruning_before > 0 { + if !blocks_pruning_description.is_empty() { + blocks_pruning_description += ", "; + } + blocks_pruning_description += &format!( + "bloom pruning: {} to {}", + info.pruning_stats.blocks_bloom_pruning_before, + info.pruning_stats.blocks_bloom_pruning_after + ); + } + + // inverted index pruning status. + if info.pruning_stats.blocks_inverted_index_pruning_before > 0 { + if !blocks_pruning_description.is_empty() { + blocks_pruning_description += ", "; + } + blocks_pruning_description += &format!( + "inverted pruning: {} to {}", + info.pruning_stats.blocks_inverted_index_pruning_before, + info.pruning_stats.blocks_inverted_index_pruning_after + ); + } + + // vector index pruning status. + if info.pruning_stats.blocks_vector_index_pruning_before > 0 { + if !blocks_pruning_description.is_empty() { + blocks_pruning_description += ", "; + } + blocks_pruning_description += &format!( + "vector pruning: {} to {}", + info.pruning_stats.blocks_vector_index_pruning_before, + info.pruning_stats.blocks_vector_index_pruning_after + ); + } + + // Combine segment pruning and blocks pruning descriptions if any + if info.pruning_stats.segments_range_pruning_before > 0 + || !blocks_pruning_description.is_empty() + { + let mut pruning_description = String::new(); + + if info.pruning_stats.segments_range_pruning_before > 0 { + pruning_description += &format!( + "segments: ", + info.pruning_stats.segments_range_pruning_before, + info.pruning_stats.segments_range_pruning_after + ); + } + + if !blocks_pruning_description.is_empty() { + if !pruning_description.is_empty() { + pruning_description += ", "; + } + pruning_description += &format!("blocks: <{}>", blocks_pruning_description); + } + + items.push(FormatTreeNode::new(format!( + "pruning stats: [{}]", + pruning_description + ))); + } + + items +} + +pub fn plan_stats_info_to_format_tree(info: &PlanStatsInfo) -> Vec> { + vec![FormatTreeNode::new(format!( + "estimated rows: {0:.2}", + info.estimated_rows + ))] +} + +pub fn format_output_columns( + output_schema: DataSchemaRef, + metadata: &Metadata, + format_table: bool, +) -> String { + output_schema + .fields() + .iter() + .map(|field| match field.name().parse::() { + Ok(column_index) => { + if column_index == usize::MAX { + return String::from("dummy value"); + } + let column_entry = metadata.column(column_index); + match column_entry.table_index() { + Some(table_index) if format_table => match metadata + .table(table_index) + .alias_name() + { + Some(alias_name) => { + format!("{}.{} (#{})", alias_name, column_entry.name(), column_index) + } + None => format!( + "{}.{} (#{})", + metadata.table(table_index).name(), + column_entry.name(), + column_index, + ), + }, + _ => format!("{} (#{})", column_entry.name(), column_index), + } + } + _ => format!("#{}", field.name()), + }) + .collect::>() + .join(", ") +} diff --git a/src/query/service/src/physical_plans/mod.rs b/src/query/service/src/physical_plans/mod.rs new file mode 100644 index 0000000000000..f4bee09135d2b --- /dev/null +++ b/src/query/service/src/physical_plans/mod.rs @@ -0,0 +1,106 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +mod physical_add_stream_column; +mod physical_aggregate_expand; +mod physical_aggregate_final; +mod physical_aggregate_partial; +mod physical_async_func; +#[allow(dead_code)] +mod physical_broadcast; +mod physical_cache_scan; +mod physical_column_mutation; +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_distributed_insert_select; +mod physical_eval_scalar; +mod physical_exchange; +mod physical_exchange_sink; +mod physical_exchange_source; +mod physical_expression_scan; +mod physical_filter; +mod physical_hash_join; +mod physical_join; +mod physical_join_filter; +mod physical_limit; +mod physical_multi_table_insert; +mod physical_mutation; +mod physical_mutation_into_organize; +mod physical_mutation_into_split; +mod physical_mutation_manipulate; +mod physical_mutation_source; +mod physical_project_set; +mod physical_r_cte_scan; +mod physical_range_join; +mod physical_recluster; +mod physical_refresh_index; +mod physical_replace_async_source; +mod physical_replace_deduplicate; +mod physical_replace_into; +mod physical_row_fetch; +mod physical_sort; +mod physical_table_scan; +mod physical_udf; +mod physical_union_all; +mod physical_window; +mod physical_window_partition; + +pub use physical_add_stream_column::AddStreamColumn; +pub use physical_aggregate_expand::AggregateExpand; +pub use physical_aggregate_partial::AggregatePartial; +pub use physical_broadcast::BroadcastSink; +pub use physical_broadcast::*; +pub use physical_column_mutation::ColumnMutation; +pub use physical_commit_sink::*; +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_distributed_insert_select::DistributedInsertSelect; +pub use physical_exchange::Exchange; +pub use physical_exchange_sink::ExchangeSink; +pub use physical_exchange_source::ExchangeSource; +pub use physical_hash_join::HashJoin; +pub use physical_join_filter::JoinRuntimeFilter; +pub use physical_join_filter::PhysicalRuntimeFilter; +pub use physical_join_filter::PhysicalRuntimeFilters; +pub use physical_multi_table_insert::*; +pub use physical_mutation_into_organize::MutationOrganize; +pub use physical_mutation_into_split::MutationSplit; +pub use physical_mutation_manipulate::MutationManipulate; +pub use physical_mutation_source::*; +pub use physical_r_cte_scan::RecursiveCteScan; +pub use physical_range_join::*; +pub use physical_recluster::HilbertPartition; +pub use physical_recluster::Recluster; +pub use physical_replace_async_source::ReplaceAsyncSourcer; +pub use physical_replace_deduplicate::*; +pub use physical_replace_into::ReplaceInto; +pub use physical_row_fetch::RowFetch; +pub use physical_table_scan::TableScan; +pub use physical_udf::UdfFunctionDesc; +pub use physical_union_all::UnionAll; +pub use physical_window::*; +pub use physical_window_partition::*; +pub mod explain; +mod format; +mod physical_asof_join; +mod physical_plan; +pub mod physical_plan_builder; +mod physical_plan_visitor; + +pub use physical_plan::*; +pub use physical_plan_builder::*; diff --git a/src/query/sql/src/executor/physical_plans/physical_add_stream_column.rs b/src/query/service/src/physical_plans/physical_add_stream_column.rs similarity index 63% rename from src/query/sql/src/executor/physical_plans/physical_add_stream_column.rs rename to src/query/service/src/physical_plans/physical_add_stream_column.rs index cc2ef6a85c35e..34d7fe60591ed 100644 --- a/src/query/sql/src/executor/physical_plans/physical_add_stream_column.rs +++ b/src/query/service/src/physical_plans/physical_add_stream_column.rs @@ -12,45 +12,127 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::any::Any; use std::collections::HashMap; +use databend_common_ast::ast::FormatTreeNode; use databend_common_catalog::plan::StreamColumn; use databend_common_catalog::plan::StreamColumnType; +use databend_common_catalog::table_context::TableContext; use databend_common_exception::Result; -use databend_common_expression::DataSchemaRef; use databend_common_expression::RemoteExpr; use databend_common_expression::ORIGIN_BLOCK_ID_COL_NAME; use databend_common_expression::ORIGIN_BLOCK_ROW_NUM_COL_NAME; use databend_common_expression::ORIGIN_VERSION_COL_NAME; +use databend_common_functions::BUILTIN_FUNCTIONS; +use databend_common_pipeline_transforms::TransformPipelineHelper; +use databend_common_sql::evaluator::BlockOperator; +use databend_common_sql::plans::BoundColumnRef; +use databend_common_sql::plans::ConstantExpr; +use databend_common_sql::plans::FunctionCall; +use databend_common_sql::Binder; +use databend_common_sql::ColumnBindingBuilder; +use databend_common_sql::MetadataRef; +use databend_common_sql::ScalarExpr; +use databend_common_sql::StreamContext; +use databend_common_sql::Visibility; +use databend_common_sql::CURRENT_BLOCK_ID_COL_NAME; +use databend_common_sql::CURRENT_BLOCK_ROW_NUM_COL_NAME; -use crate::executor::PhysicalPlan; -use crate::planner::CURRENT_BLOCK_ID_COL_NAME; -use crate::planner::CURRENT_BLOCK_ROW_NUM_COL_NAME; -use crate::plans::BoundColumnRef; -use crate::plans::ConstantExpr; -use crate::plans::FunctionCall; -use crate::Binder; -use crate::ColumnBindingBuilder; -use crate::MetadataRef; -use crate::ScalarExpr; -use crate::Visibility; +use crate::physical_plans::format::FormatContext; +use crate::physical_plans::physical_plan::IPhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlanMeta; +use crate::pipelines::processors::transforms::TransformAddStreamColumns; +use crate::pipelines::PipelineBuilder; #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct AddStreamColumn { - pub plan_id: u32, - pub input: Box, + pub meta: PhysicalPlanMeta, + pub input: PhysicalPlan, pub exprs: Vec, pub projections: Vec, pub stream_columns: Vec, } +#[typetag::serde] +impl IPhysicalPlan for AddStreamColumn { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn children<'a>(&'a self) -> Box + 'a> { + Box::new(std::iter::once(&self.input)) + } + + fn children_mut<'a>(&'a mut self) -> Box + 'a> { + Box::new(std::iter::once(&mut self.input)) + } + + fn to_format_node( + &self, + _: &mut FormatContext<'_>, + mut children: Vec>, + ) -> Result> { + // ignore self + assert_eq!(children.len(), 1); + Ok(children.pop().unwrap()) + } + + fn derive(&self, mut children: Vec) -> PhysicalPlan { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + self.input.build_pipeline(builder)?; + + let exprs = self + .exprs + .iter() + .map(|expr| expr.as_expr(&BUILTIN_FUNCTIONS)) + .collect::>(); + + let operators = vec![ + BlockOperator::Map { + exprs, + projections: None, + }, + BlockOperator::Project { + projection: self.projections.clone(), + }, + ]; + + let stream_context = StreamContext { + stream_columns: self.stream_columns.clone(), + operators, + func_ctx: builder.ctx.get_function_context()?, + }; + + builder + .main_pipeline + .add_transformer(|| TransformAddStreamColumns::new(stream_context.clone())); + + Ok(()) + } +} + impl AddStreamColumn { - pub fn new( + pub fn create( metadata: &MetadataRef, input: PhysicalPlan, table_index: usize, table_version: u64, - ) -> Result { + ) -> Result { let input_schema = input.output_schema()?; let num_fields = input_schema.fields().len(); let column_entries = metadata.read().columns_by_table_index(table_index); @@ -155,16 +237,12 @@ impl AddStreamColumn { // ORIGIN_BLOCK_ROW_NUM, ORIGIN_BLOCK_ID. let stream_columns = vec![stream_columns[2].clone(), stream_columns[1].clone()]; - Ok(Self { - plan_id: 0, - input: Box::new(input), + Ok(Box::new(AddStreamColumn { + input, exprs, projections, stream_columns, - }) - } - - pub fn output_schema(&self) -> Result { - self.input.output_schema() + meta: PhysicalPlanMeta::new("AddStreamColumn"), + })) } } diff --git a/src/query/service/src/physical_plans/physical_aggregate_expand.rs b/src/query/service/src/physical_plans/physical_aggregate_expand.rs new file mode 100644 index 0000000000000..33dd14d8cd9a6 --- /dev/null +++ b/src/query/service/src/physical_plans/physical_aggregate_expand.rs @@ -0,0 +1,205 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::any::Any; + +use databend_common_ast::ast::FormatTreeNode; +use databend_common_exception::Result; +use databend_common_expression::types::DataType; +use databend_common_expression::types::NumberDataType; +use databend_common_expression::DataField; +use databend_common_expression::DataSchemaRef; +use databend_common_expression::DataSchemaRefExt; +use databend_common_pipeline_transforms::TransformPipelineHelper; +use databend_common_sql::plans::GroupingSets; +use databend_common_sql::IndexType; + +use crate::physical_plans::explain::PlanStatsInfo; +use crate::physical_plans::format::format_output_columns; +use crate::physical_plans::format::plan_stats_info_to_format_tree; +use crate::physical_plans::format::FormatContext; +use crate::physical_plans::physical_plan::IPhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlanMeta; +use crate::pipelines::processors::transforms::aggregator::TransformExpandGroupingSets; +use crate::pipelines::PipelineBuilder; + +/// Add dummy data before `GROUPING SETS`. +#[derive(serde::Serialize, serde::Deserialize, Clone, Debug)] +pub struct AggregateExpand { + pub meta: PhysicalPlanMeta, + pub input: PhysicalPlan, + pub group_bys: Vec, + pub grouping_sets: GroupingSets, + + // Only used for explain + pub stat_info: Option, +} + +#[typetag::serde] +impl IPhysicalPlan for AggregateExpand { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn output_schema(&self) -> Result { + let input_schema = self.input.output_schema()?; + let mut output_fields = input_schema.fields().clone(); + // Add virtual columns to group by. + output_fields.reserve(self.group_bys.len() + 1); + + for (group_by, (actual, ty)) in self + .group_bys + .iter() + .zip(self.grouping_sets.dup_group_items.iter()) + { + // All group by columns will wrap nullable. + let i = input_schema.index_of(&group_by.to_string())?; + let f = &mut output_fields[i]; + debug_assert!(f.data_type() == ty || f.data_type().wrap_nullable() == *ty); + *f = DataField::new(f.name(), f.data_type().wrap_nullable()); + let new_field = DataField::new(&actual.to_string(), ty.clone()); + output_fields.push(new_field); + } + + output_fields.push(DataField::new( + &self.grouping_sets.grouping_id_index.to_string(), + DataType::Number(NumberDataType::UInt32), + )); + Ok(DataSchemaRefExt::create(output_fields)) + } + + fn children<'a>(&'a self) -> Box + 'a> { + Box::new(std::iter::once(&self.input)) + } + + fn children_mut<'a>(&'a mut self) -> Box + 'a> { + Box::new(std::iter::once(&mut self.input)) + } + + fn to_format_node( + &self, + ctx: &mut FormatContext<'_>, + children: Vec>, + ) -> Result> { + let sets = self + .grouping_sets + .sets + .iter() + .map(|set| { + set.iter() + .map(|&index| ctx.metadata.column(index).name()) + .collect::>() + .join(", ") + }) + .map(|s| format!("({})", s)) + .collect::>() + .join(", "); + + let mut node_children = vec![ + FormatTreeNode::new(format!( + "output columns: [{}]", + format_output_columns(self.output_schema()?, ctx.metadata, true) + )), + FormatTreeNode::new(format!("grouping sets: [{sets}]")), + ]; + + if let Some(info) = &self.stat_info { + let items = plan_stats_info_to_format_tree(info); + node_children.extend(items); + } + + node_children.extend(children); + Ok(FormatTreeNode::with_children( + "AggregateExpand".to_string(), + node_children, + )) + } + + fn get_desc(&self) -> Result { + Ok(self + .grouping_sets + .sets + .iter() + .map(|set| { + set.iter() + .map(|x| x.to_string()) + .collect::>() + .join(", ") + }) + .map(|s| format!("({})", s)) + .collect::>() + .join(", ")) + } + + fn derive(&self, mut children: Vec) -> PhysicalPlan { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + self.input.build_pipeline(builder)?; + + let input_schema = self.input.output_schema()?; + let group_bys = self + .group_bys + .iter() + .take(self.group_bys.len() - 1) // The last group-by will be virtual column `_grouping_id` + .map(|i| input_schema.index_of(&i.to_string())) + .collect::>>()?; + let grouping_sets = self + .grouping_sets + .sets + .iter() + .map(|sets| { + sets.iter() + .map(|i| { + let i = input_schema.index_of(&i.to_string())?; + let offset = group_bys.iter().position(|j| *j == i).unwrap(); + Ok(offset) + }) + .collect::>>() + }) + .collect::>>()?; + let mut grouping_ids = Vec::with_capacity(grouping_sets.len()); + let mask = (1 << group_bys.len()) - 1; + for set in grouping_sets { + let mut id = 0; + for i in set { + id |= 1 << i; + } + // For element in `group_bys`, + // if it is in current grouping set: set 0, else: set 1. (1 represents it will be NULL in grouping) + // Example: GROUP BY GROUPING SETS ((a, b), (a), (b), ()) + // group_bys: [a, b] + // grouping_sets: [[0, 1], [0], [1], []] + // grouping_ids: 00, 01, 10, 11 + grouping_ids.push(!id & mask); + } + + builder.main_pipeline.add_accumulating_transformer(|| { + TransformExpandGroupingSets::new(group_bys.clone(), grouping_ids.clone()) + }); + Ok(()) + } +} diff --git a/src/query/sql/src/executor/physical_plans/physical_aggregate_final.rs b/src/query/service/src/physical_plans/physical_aggregate_final.rs similarity index 60% rename from src/query/sql/src/executor/physical_plans/physical_aggregate_final.rs rename to src/query/service/src/physical_plans/physical_aggregate_final.rs index d3511fbb8cdaa..304be654adc78 100644 --- a/src/query/sql/src/executor/physical_plans/physical_aggregate_final.rs +++ b/src/query/service/src/physical_plans/physical_aggregate_final.rs @@ -12,36 +12,53 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::any::Any; +use std::collections::HashMap; use std::sync::Arc; +use databend_common_ast::ast::FormatTreeNode; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::DataField; use databend_common_expression::DataSchemaRef; use databend_common_expression::DataSchemaRefExt; use databend_common_expression::RemoteExpr; - -use super::SortDesc; -use crate::executor::explain::PlanStatsInfo; -use crate::executor::physical_plans::AggregateExpand; -use crate::executor::physical_plans::AggregateFunctionDesc; -use crate::executor::physical_plans::AggregateFunctionSignature; -use crate::executor::physical_plans::AggregatePartial; -use crate::executor::physical_plans::Exchange; -use crate::executor::PhysicalPlan; -use crate::executor::PhysicalPlanBuilder; -use crate::optimizer::ir::SExpr; -use crate::plans::AggregateMode; -use crate::plans::DummyTableScan; -use crate::ColumnSet; -use crate::IndexType; -use crate::ScalarExpr; +use databend_common_pipeline_core::processors::ProcessorPtr; +use databend_common_sql::executor::physical_plans::AggregateFunctionDesc; +use databend_common_sql::executor::physical_plans::AggregateFunctionSignature; +use databend_common_sql::executor::physical_plans::SortDesc; +use databend_common_sql::optimizer::ir::SExpr; +use databend_common_sql::plans::Aggregate; +use databend_common_sql::plans::AggregateMode; +use databend_common_sql::plans::DummyTableScan; +use databend_common_sql::ColumnSet; +use databend_common_sql::IndexType; +use databend_common_sql::ScalarExpr; +use itertools::Itertools; + +use super::AggregateExpand; +use super::AggregatePartial; +use super::Exchange; +use super::ExchangeSource; +use super::PhysicalPlanDynExt; +use crate::physical_plans::explain::PlanStatsInfo; +use crate::physical_plans::format::format_output_columns; +use crate::physical_plans::format::plan_stats_info_to_format_tree; +use crate::physical_plans::format::pretty_display_agg_desc; +use crate::physical_plans::format::FormatContext; +use crate::physical_plans::physical_plan::IPhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlanMeta; +use crate::physical_plans::physical_plan_builder::PhysicalPlanBuilder; +use crate::pipelines::processors::transforms::aggregator::build_partition_bucket; +use crate::pipelines::processors::transforms::aggregator::AggregateInjector; +use crate::pipelines::processors::transforms::aggregator::FinalSingleStateAggregator; +use crate::pipelines::PipelineBuilder; #[derive(serde::Serialize, serde::Deserialize, Clone, Debug)] pub struct AggregateFinal { - // A unique id of operator in a `PhysicalPlan` tree, only used for display. - pub plan_id: u32, - pub input: Box, + meta: PhysicalPlanMeta, + pub input: PhysicalPlan, pub group_by: Vec, pub agg_funcs: Vec, pub before_group_by_schema: DataSchemaRef, @@ -51,8 +68,20 @@ pub struct AggregateFinal { pub stat_info: Option, } -impl AggregateFinal { - pub fn output_schema(&self) -> Result { +#[typetag::serde] +impl IPhysicalPlan for AggregateFinal { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn output_schema(&self) -> Result { let mut fields = Vec::with_capacity(self.agg_funcs.len() + self.group_by.len()); for agg in self.agg_funcs.iter() { let data_type = agg.sig.return_type.clone(); @@ -68,13 +97,144 @@ impl AggregateFinal { } Ok(DataSchemaRefExt::create(fields)) } + + fn children<'a>(&'a self) -> Box + 'a> { + Box::new(std::iter::once(&self.input)) + } + + fn children_mut<'a>(&'a mut self) -> Box + 'a> { + Box::new(std::iter::once(&mut self.input)) + } + + fn to_format_node( + &self, + ctx: &mut FormatContext<'_>, + children: Vec>, + ) -> Result> { + let group_by = self + .group_by + .iter() + .map(|&index| { + let name = ctx.metadata.column(index).name(); + Ok(name) + }) + .collect::>>()? + .join(", "); + + let agg_funcs = self + .agg_funcs + .iter() + .map(|agg| pretty_display_agg_desc(agg, ctx.metadata)) + .collect::>() + .join(", "); + + let mut node_children = vec![ + FormatTreeNode::new(format!( + "output columns: [{}]", + format_output_columns(self.output_schema()?, ctx.metadata, true) + )), + FormatTreeNode::new(format!("group by: [{group_by}]")), + FormatTreeNode::new(format!("aggregate functions: [{agg_funcs}]")), + ]; + + if let Some(info) = &self.stat_info { + let items = plan_stats_info_to_format_tree(info); + node_children.extend(items); + } + + node_children.extend(children); + Ok(FormatTreeNode::with_children( + "AggregateFinal".to_string(), + node_children, + )) + } + + fn get_desc(&self) -> Result { + Ok(self.agg_funcs.iter().map(|x| x.display.clone()).join(", ")) + } + + fn get_labels(&self) -> Result>> { + let mut labels = HashMap::with_capacity(2); + if !self.group_by_display.is_empty() { + labels.insert(String::from("Grouping keys"), self.group_by_display.clone()); + } + + if !self.agg_funcs.is_empty() { + labels.insert( + String::from("Aggregate Functions"), + self.agg_funcs.iter().map(|x| x.display.clone()).collect(), + ); + } + + Ok(labels) + } + + fn derive(&self, mut children: Vec) -> PhysicalPlan { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + let max_block_size = builder.settings.get_max_block_size()?; + let enable_experimental_aggregate_hashtable = builder + .settings + .get_enable_experimental_aggregate_hashtable()?; + let max_spill_io_requests = builder.settings.get_max_spill_io_requests()?; + let max_restore_worker = builder.settings.get_max_aggregate_restore_worker()?; + + let mut is_cluster_aggregate = false; + if self.input.downcast_ref::().is_some() { + is_cluster_aggregate = true; + } + + let params = PipelineBuilder::build_aggregator_params( + self.before_group_by_schema.clone(), + &self.group_by, + &self.agg_funcs, + enable_experimental_aggregate_hashtable, + is_cluster_aggregate, + max_block_size as usize, + max_spill_io_requests as usize, + )?; + + if params.group_columns.is_empty() { + self.input.build_pipeline(builder)?; + + builder.main_pipeline.try_resize(1)?; + builder.main_pipeline.add_transform(|input, output| { + Ok(ProcessorPtr::create( + FinalSingleStateAggregator::try_create(input, output, ¶ms)?, + )) + })?; + + return Ok(()); + } + + let old_inject = builder.exchange_injector.clone(); + + if self.input.downcast_ref::().is_some() { + builder.exchange_injector = + AggregateInjector::create(builder.ctx.clone(), params.clone()); + } + + self.input.build_pipeline(builder)?; + + builder.exchange_injector = old_inject; + build_partition_bucket( + &mut builder.main_pipeline, + params.clone(), + max_restore_worker, + ) + } } impl PhysicalPlanBuilder { - pub(crate) async fn build_aggregate( + pub async fn build_aggregate( &mut self, s_expr: &SExpr, - agg: &crate::plans::Aggregate, + agg: &Aggregate, mut required: ColumnSet, stat_info: PlanStatsInfo, ) -> Result { @@ -98,7 +258,7 @@ impl PhysicalPlanBuilder { return self.build(&expr, required).await; } - let agg = crate::plans::Aggregate { + let agg = Aggregate { group_items: agg.group_items.clone(), aggregate_functions: used, from_distinct: agg.from_distinct, @@ -112,7 +272,7 @@ impl PhysicalPlanBuilder { let input_schema = input.output_schema()?; let group_items = agg.group_items.iter().map(|v| v.index).collect::>(); - let result = match &agg.mode { + let result: PhysicalPlan = match &agg.mode { AggregateMode::Partial => { let group_by_display = agg .group_items @@ -261,115 +421,114 @@ impl PhysicalPlanBuilder { (desc, limit) }); - match input { - PhysicalPlan::Exchange(Exchange { input, kind, .. }) - if group_by_shuffle_mode == "before_merge" => - { - let aggregate_partial = if let Some(grouping_sets) = agg.grouping_sets { - let expand = AggregateExpand { - plan_id: 0, - input, - group_bys: group_items.clone(), - grouping_sets, - stat_info: Some(stat_info.clone()), - }; - AggregatePartial { - plan_id: 0, - input: Box::new(PhysicalPlan::AggregateExpand(expand)), - agg_funcs, - enable_experimental_aggregate_hashtable, - group_by_display, - group_by: group_items, - stat_info: Some(stat_info), - rank_limit: None, - } - } else { - AggregatePartial { - plan_id: 0, - input, - agg_funcs, - enable_experimental_aggregate_hashtable, - group_by_display, - group_by: group_items, - stat_info: Some(stat_info), - rank_limit, - } - }; - - let keys = { - let schema = aggregate_partial.output_schema()?; - let end = schema.num_fields(); - let start = end - aggregate_partial.group_by.len(); - (start..end) - .map(|id| RemoteExpr::ColumnRef { - span: None, - id, - data_type: schema.field(id).data_type().clone(), - display_name: (id - start).to_string(), - }) - .collect() - }; - - PhysicalPlan::Exchange(Exchange { - plan_id: 0, - kind, - allow_adjust_parallelism: true, - ignore_exchange: false, - input: Box::new(PhysicalPlan::AggregatePartial(aggregate_partial)), - keys, + if group_by_shuffle_mode == "before_merge" + && let Some(exchange) = input.downcast_ref::() + { + let kind = exchange.kind.clone(); + let aggregate_partial = if let Some(grouping_sets) = agg.grouping_sets { + let expand = Box::new(AggregateExpand { + grouping_sets, + input: exchange.input.clone(), + group_bys: group_items.clone(), + stat_info: Some(stat_info.clone()), + meta: PhysicalPlanMeta::new("AggregateExpand"), + }); + + Box::new(AggregatePartial { + input: expand, + agg_funcs, + enable_experimental_aggregate_hashtable, + group_by_display, + group_by: group_items, + stat_info: Some(stat_info), + rank_limit: None, + meta: PhysicalPlanMeta::new("AggregatePartial"), }) - } - _ => { - if let Some(grouping_sets) = agg.grouping_sets { - let expand = AggregateExpand { - plan_id: 0, - input: Box::new(input), - group_bys: group_items.clone(), - grouping_sets, - stat_info: Some(stat_info.clone()), - }; - PhysicalPlan::AggregatePartial(AggregatePartial { - plan_id: 0, - agg_funcs, - enable_experimental_aggregate_hashtable, - group_by_display, - group_by: group_items, - input: Box::new(PhysicalPlan::AggregateExpand(expand)), - stat_info: Some(stat_info), - rank_limit: None, - }) - } else { - PhysicalPlan::AggregatePartial(AggregatePartial { - plan_id: 0, - agg_funcs, - enable_experimental_aggregate_hashtable, - group_by_display, - group_by: group_items, - input: Box::new(input), - stat_info: Some(stat_info), - rank_limit, + } else { + Box::new(AggregatePartial { + input, + agg_funcs, + rank_limit, + group_by_display, + enable_experimental_aggregate_hashtable, + group_by: group_items, + stat_info: Some(stat_info), + meta: PhysicalPlanMeta::new("AggregatePartial"), + }) + }; + + let keys = { + let schema = aggregate_partial.output_schema()?; + let end = schema.num_fields(); + let start = end - aggregate_partial.group_by.len(); + (start..end) + .map(|id| RemoteExpr::ColumnRef { + span: None, + id, + data_type: schema.field(id).data_type().clone(), + display_name: (id - start).to_string(), }) - } - } + .collect() + }; + + Box::new(Exchange { + keys, + kind, + ignore_exchange: false, + allow_adjust_parallelism: true, + meta: PhysicalPlanMeta::new("Exchange"), + input: aggregate_partial, + }) + } else if let Some(grouping_sets) = agg.grouping_sets { + let expand = AggregateExpand { + input, + grouping_sets, + group_bys: group_items.clone(), + stat_info: Some(stat_info.clone()), + meta: PhysicalPlanMeta::new("AggregateExpand"), + }; + + Box::new(AggregatePartial { + agg_funcs, + group_by_display, + enable_experimental_aggregate_hashtable, + rank_limit: None, + group_by: group_items, + input: Box::new(expand), + stat_info: Some(stat_info), + meta: PhysicalPlanMeta::new("AggregatePartial"), + }) + } else { + Box::new(AggregatePartial { + input, + agg_funcs, + enable_experimental_aggregate_hashtable, + group_by_display, + group_by: group_items, + stat_info: Some(stat_info), + rank_limit, + meta: PhysicalPlanMeta::new("AggregatePartial"), + }) } } // Hack to get before group by schema, we should refactor this AggregateMode::Final => { - let input_schema = match input { - PhysicalPlan::AggregatePartial(ref agg) => agg.input.output_schema()?, + let input_schema = { + let mut plan = &input; - PhysicalPlan::Exchange(Exchange { - input: box PhysicalPlan::AggregatePartial(ref agg), - .. - }) => agg.input.output_schema()?, + if let Some(exchange) = plan.downcast_ref::() { + plan = &exchange.input; + } - _ => { + let Some(aggregate) = plan.downcast_ref::() else { return Err(ErrorCode::Internal(format!( "invalid input physical plan: {}", - input.name(), + input.get_name(), ))); - } + }; + + aggregate.input.output_schema()? }; let mut agg_funcs: Vec = agg @@ -491,46 +650,46 @@ impl PhysicalPlanBuilder { } } - match input { - PhysicalPlan::AggregatePartial(ref partial) => { - let before_group_by_schema = partial.input.output_schema()?; - - PhysicalPlan::AggregateFinal(AggregateFinal { - plan_id: 0, - group_by_display: partial.group_by_display.clone(), - input: Box::new(input), - group_by: group_items, - agg_funcs, - before_group_by_schema, - - stat_info: Some(stat_info), - }) - } - - PhysicalPlan::Exchange(Exchange { - input: box PhysicalPlan::AggregatePartial(ref partial), - .. - }) => { - let before_group_by_schema = partial.input.output_schema()?; - - PhysicalPlan::AggregateFinal(AggregateFinal { - plan_id: 0, - group_by_display: partial.group_by_display.clone(), - input: Box::new(input), - group_by: group_items, - agg_funcs, - before_group_by_schema, - - stat_info: Some(stat_info), - }) - } + if let Some(partial) = input.downcast_ref::() { + let group_by_display = partial.group_by_display.clone(); + let before_group_by_schema = partial.input.output_schema()?; + + Box::new(AggregateFinal { + input, + agg_funcs, + group_by_display, + before_group_by_schema, + group_by: group_items, + stat_info: Some(stat_info), + meta: PhysicalPlanMeta::new("AggregateFinal"), + }) + } else { + let Some(exchange) = input.downcast_ref::() else { + return Err(ErrorCode::Internal(format!( + "invalid input physical plan: {}", + input.get_name(), + ))); + }; - _ => { + let Some(partial) = exchange.input.downcast_ref::() else { return Err(ErrorCode::Internal(format!( "invalid input physical plan: {}", - input.name(), + input.get_name(), ))); - } + }; + + let group_by_display = partial.group_by_display.clone(); + let before_group_by_schema = partial.input.output_schema()?; + + Box::new(AggregateFinal { + input, + agg_funcs, + group_by_display, + before_group_by_schema, + group_by: group_items, + stat_info: Some(stat_info), + meta: PhysicalPlanMeta::new("AggregateFinal"), + }) } } AggregateMode::Initial => { diff --git a/src/query/service/src/physical_plans/physical_aggregate_partial.rs b/src/query/service/src/physical_plans/physical_aggregate_partial.rs new file mode 100644 index 0000000000000..1a6b47edd19a8 --- /dev/null +++ b/src/query/service/src/physical_plans/physical_aggregate_partial.rs @@ -0,0 +1,268 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::any::Any; +use std::collections::HashMap; +use std::sync::Arc; + +use databend_common_ast::ast::FormatTreeNode; +use databend_common_catalog::table_context::TableContext; +use databend_common_exception::Result; +use databend_common_expression::types::DataType; +#[allow(unused_imports)] +use databend_common_expression::DataBlock; +use databend_common_expression::DataField; +use databend_common_expression::DataSchemaRef; +use databend_common_expression::DataSchemaRefExt; +use databend_common_expression::HashTableConfig; +use databend_common_expression::LimitType; +use databend_common_expression::SortColumnDescription; +use databend_common_pipeline_core::processors::ProcessorPtr; +use databend_common_pipeline_transforms::TransformPipelineHelper; +use databend_common_pipeline_transforms::TransformSortPartial; +use databend_common_sql::executor::physical_plans::AggregateFunctionDesc; +use databend_common_sql::executor::physical_plans::SortDesc; +use databend_common_sql::IndexType; +use databend_common_storage::DataOperator; +use itertools::Itertools; + +use crate::physical_plans::explain::PlanStatsInfo; +use crate::physical_plans::format::plan_stats_info_to_format_tree; +use crate::physical_plans::format::pretty_display_agg_desc; +use crate::physical_plans::format::FormatContext; +use crate::physical_plans::physical_plan::IPhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlanMeta; +use crate::pipelines::processors::transforms::aggregator::AggregateInjector; +use crate::pipelines::processors::transforms::aggregator::PartialSingleStateAggregator; +use crate::pipelines::processors::transforms::aggregator::TransformAggregateSpillWriter; +use crate::pipelines::processors::transforms::aggregator::TransformPartialAggregate; +use crate::pipelines::PipelineBuilder; + +#[derive(serde::Serialize, serde::Deserialize, Clone, Debug)] +pub struct AggregatePartial { + pub meta: PhysicalPlanMeta, + pub input: PhysicalPlan, + pub group_by: Vec, + pub agg_funcs: Vec, + pub enable_experimental_aggregate_hashtable: bool, + pub group_by_display: Vec, + + // Order by keys if keys are subset of group by key, then we can use rank to filter data in previous + pub rank_limit: Option<(Vec, usize)>, + // Only used for explain + pub stat_info: Option, +} + +#[typetag::serde] +impl IPhysicalPlan for AggregatePartial { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn output_schema(&self) -> Result { + let input_schema = self.input.output_schema()?; + + let mut fields = Vec::with_capacity(self.agg_funcs.len() + self.group_by.len()); + + fields.extend(self.agg_funcs.iter().map(|func| { + let name = func.output_column.to_string(); + DataField::new(&name, DataType::Binary) + })); + + for (idx, field) in self.group_by.iter().zip( + self.group_by + .iter() + .map(|index| input_schema.field_with_name(&index.to_string())), + ) { + fields.push(DataField::new(&idx.to_string(), field?.data_type().clone())); + } + + Ok(DataSchemaRefExt::create(fields)) + } + + fn children<'a>(&'a self) -> Box + 'a> { + Box::new(std::iter::once(&self.input)) + } + + fn children_mut<'a>(&'a mut self) -> Box + 'a> { + Box::new(std::iter::once(&mut self.input)) + } + + fn to_format_node( + &self, + ctx: &mut FormatContext<'_>, + children: Vec>, + ) -> Result> { + let group_by = self + .group_by + .iter() + .map(|&index| ctx.metadata.column(index).name()) + .join(", "); + + let agg_funcs = self + .agg_funcs + .iter() + .map(|agg| pretty_display_agg_desc(agg, ctx.metadata)) + .collect::>() + .join(", "); + + let mut node_children = vec![ + FormatTreeNode::new(format!("group by: [{group_by}]")), + FormatTreeNode::new(format!("aggregate functions: [{agg_funcs}]")), + ]; + + if let Some(info) = &self.stat_info { + node_children.extend(plan_stats_info_to_format_tree(info)); + } + + if let Some((_, r)) = &self.rank_limit { + node_children.push(FormatTreeNode::new(format!("rank limit: {r}"))); + } + + node_children.extend(children); + + Ok(FormatTreeNode::with_children( + "AggregatePartial".to_string(), + node_children, + )) + } + + fn get_desc(&self) -> Result { + Ok(self.agg_funcs.iter().map(|x| x.display.clone()).join(", ")) + } + + fn get_labels(&self) -> Result>> { + let mut labels = HashMap::with_capacity(2); + + if !self.group_by_display.is_empty() { + labels.insert(String::from("Grouping keys"), self.group_by_display.clone()); + } + + if !self.agg_funcs.is_empty() { + labels.insert( + String::from("Aggregate Functions"), + self.agg_funcs.iter().map(|x| x.display.clone()).collect(), + ); + } + + Ok(labels) + } + + fn derive(&self, mut children: Vec) -> PhysicalPlan { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + builder.contain_sink_processor = true; + self.input.build_pipeline(builder)?; + + let max_block_size = builder.settings.get_max_block_size()?; + let max_threads = builder.settings.get_max_threads()?; + let max_spill_io_requests = builder.settings.get_max_spill_io_requests()?; + + let enable_experimental_aggregate_hashtable = builder + .settings + .get_enable_experimental_aggregate_hashtable()?; + + let params = PipelineBuilder::build_aggregator_params( + self.input.output_schema()?, + &self.group_by, + &self.agg_funcs, + enable_experimental_aggregate_hashtable, + builder.is_exchange_parent(), + max_block_size as usize, + max_spill_io_requests as usize, + )?; + + if params.group_columns.is_empty() { + return builder.main_pipeline.try_add_accumulating_transformer(|| { + PartialSingleStateAggregator::try_new(¶ms) + }); + } + + let schema_before_group_by = params.input_schema.clone(); + + // Need a global atomic to read the max current radix bits hint + let partial_agg_config = if !builder.is_exchange_parent() { + HashTableConfig::default().with_partial(true, max_threads as usize) + } else { + HashTableConfig::default() + .cluster_with_partial(true, builder.ctx.get_cluster().nodes.len()) + }; + + // For rank limit, we can filter data using sort with rank before partial + if let Some(rank_limit) = &self.rank_limit { + let sort_desc = rank_limit + .0 + .iter() + .map(|desc| { + let offset = schema_before_group_by.index_of(&desc.order_by.to_string())?; + Ok(SortColumnDescription { + offset, + asc: desc.asc, + nulls_first: desc.nulls_first, + }) + }) + .collect::>>()?; + let sort_desc: Arc<[_]> = sort_desc.into(); + + builder.main_pipeline.add_transformer(|| { + TransformSortPartial::new(LimitType::LimitRank(rank_limit.1), sort_desc.clone()) + }); + } + + builder.main_pipeline.add_transform(|input, output| { + Ok(ProcessorPtr::create(TransformPartialAggregate::try_create( + builder.ctx.clone(), + input, + output, + params.clone(), + partial_agg_config.clone(), + )?)) + })?; + + // If cluster mode, spill write will be completed in exchange serialize, because we need scatter the block data first + if !builder.is_exchange_parent() { + let operator = DataOperator::instance().spill_operator(); + let location_prefix = builder.ctx.query_id_spill_prefix(); + + builder.main_pipeline.add_transform(|input, output| { + Ok(ProcessorPtr::create( + TransformAggregateSpillWriter::try_create( + builder.ctx.clone(), + input, + output, + operator.clone(), + params.clone(), + location_prefix.clone(), + )?, + )) + })?; + } + + builder.exchange_injector = AggregateInjector::create(builder.ctx.clone(), params.clone()); + Ok(()) + } +} diff --git a/src/query/sql/src/executor/physical_plans/physical_asof_join.rs b/src/query/service/src/physical_plans/physical_asof_join.rs similarity index 92% rename from src/query/sql/src/executor/physical_plans/physical_asof_join.rs rename to src/query/service/src/physical_plans/physical_asof_join.rs index 7cc90d45a2b11..15bc4a1c56434 100644 --- a/src/query/sql/src/executor/physical_plans/physical_asof_join.rs +++ b/src/query/service/src/physical_plans/physical_asof_join.rs @@ -19,33 +19,33 @@ use databend_common_exception::Result; use databend_common_expression::types::DataType; use databend_common_expression::types::NumberScalar; use databend_common_expression::Scalar; +use databend_common_sql::binder::bind_window_function_info; +use databend_common_sql::binder::WindowFunctionInfo; +use databend_common_sql::binder::WindowOrderByInfo; +use databend_common_sql::optimizer::ir::RelExpr; +use databend_common_sql::optimizer::ir::SExpr; +use databend_common_sql::plans::BoundColumnRef; +use databend_common_sql::plans::ComparisonOp; +use databend_common_sql::plans::ConstantExpr; +use databend_common_sql::plans::FunctionCall; +use databend_common_sql::plans::Join; +use databend_common_sql::plans::LagLeadFunction; +use databend_common_sql::plans::ScalarItem; +use databend_common_sql::plans::WindowFunc; +use databend_common_sql::plans::WindowFuncFrame; +use databend_common_sql::plans::WindowFuncFrameBound; +use databend_common_sql::plans::WindowFuncFrameUnits; +use databend_common_sql::plans::WindowFuncType; +use databend_common_sql::plans::WindowOrderBy; +use databend_common_sql::ColumnBindingBuilder; +use databend_common_sql::ColumnEntry; +use databend_common_sql::ColumnSet; +use databend_common_sql::DerivedColumn; +use databend_common_sql::ScalarExpr; +use databend_common_sql::Visibility; -use crate::binder::bind_window_function_info; -use crate::binder::ColumnBindingBuilder; -use crate::binder::WindowFunctionInfo; -use crate::binder::WindowOrderByInfo; -use crate::executor::PhysicalPlan; -use crate::executor::PhysicalPlanBuilder; -use crate::optimizer::ir::RelExpr; -use crate::optimizer::ir::SExpr; -use crate::plans::BoundColumnRef; -use crate::plans::ComparisonOp; -use crate::plans::ConstantExpr; -use crate::plans::FunctionCall; -use crate::plans::Join; -use crate::plans::LagLeadFunction; -use crate::plans::ScalarExpr; -use crate::plans::ScalarItem; -use crate::plans::WindowFunc; -use crate::plans::WindowFuncFrame; -use crate::plans::WindowFuncFrameBound; -use crate::plans::WindowFuncFrameUnits; -use crate::plans::WindowFuncType; -use crate::plans::WindowOrderBy; -use crate::ColumnEntry; -use crate::ColumnSet; -use crate::DerivedColumn; -use crate::Visibility; +use crate::physical_plans::physical_plan::PhysicalPlan; +use crate::physical_plans::physical_plan_builder::PhysicalPlanBuilder; impl PhysicalPlanBuilder { pub async fn build_asof_join( diff --git a/src/query/sql/src/executor/physical_plans/physical_async_func.rs b/src/query/service/src/physical_plans/physical_async_func.rs similarity index 53% rename from src/query/sql/src/executor/physical_plans/physical_async_func.rs rename to src/query/service/src/physical_plans/physical_async_func.rs index 503a90dfac72f..311b6309ff2ea 100644 --- a/src/query/sql/src/executor/physical_plans/physical_async_func.rs +++ b/src/query/service/src/physical_plans/physical_async_func.rs @@ -12,34 +12,56 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::any::Any; + +use databend_common_ast::ast::FormatTreeNode; +use databend_common_catalog::plan::DataSourcePlan; use databend_common_exception::ErrorCode; use databend_common_exception::Result; -use databend_common_expression::types::DataType; use databend_common_expression::DataField; use databend_common_expression::DataSchemaRef; use databend_common_expression::DataSchemaRefExt; +use databend_common_pipeline_transforms::TransformPipelineHelper; +use databend_common_sql::binder::AsyncFunctionDesc; +use databend_common_sql::optimizer::ir::SExpr; +use databend_common_sql::ColumnSet; +use databend_common_sql::ScalarExpr; +use itertools::Itertools; -use crate::executor::explain::PlanStatsInfo; -use crate::executor::PhysicalPlan; -use crate::executor::PhysicalPlanBuilder; -use crate::optimizer::ir::SExpr; -use crate::plans::AsyncFunctionArgument; -use crate::ColumnSet; -use crate::IndexType; -use crate::ScalarExpr; +use crate::physical_plans::explain::PlanStatsInfo; +use crate::physical_plans::format::format_output_columns; +use crate::physical_plans::format::plan_stats_info_to_format_tree; +use crate::physical_plans::format::FormatContext; +use crate::physical_plans::physical_plan::IPhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlanMeta; +use crate::physical_plans::physical_plan_builder::PhysicalPlanBuilder; +use crate::pipelines::processors::transforms::TransformAsyncFunction; +use crate::pipelines::PipelineBuilder; #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct AsyncFunction { - // A unique id of operator in a `PhysicalPlan` tree, only used for display. - pub plan_id: u32, - pub input: Box, + pub meta: PhysicalPlanMeta, + pub input: PhysicalPlan, pub async_func_descs: Vec, // Only used for explain pub stat_info: Option, } -impl AsyncFunction { - pub fn output_schema(&self) -> Result { +#[typetag::serde] +impl IPhysicalPlan for AsyncFunction { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn output_schema(&self) -> Result { let input_schema = self.input.output_schema()?; let mut fields = input_schema.fields().clone(); for async_func_desc in self.async_func_descs.iter() { @@ -49,24 +71,82 @@ impl AsyncFunction { } Ok(DataSchemaRefExt::create(fields)) } -} -#[derive(Clone, Debug, Eq, PartialEq, serde::Serialize, serde::Deserialize)] -pub struct AsyncFunctionDesc { - pub func_name: String, - pub display_name: String, - pub output_column: IndexType, - pub arg_indices: Vec, - pub data_type: Box, + fn children<'a>(&'a self) -> Box + 'a> { + Box::new(std::iter::once(&self.input)) + } + + fn children_mut<'a>(&'a mut self) -> Box + 'a> { + Box::new(std::iter::once(&mut self.input)) + } + + fn to_format_node( + &self, + ctx: &mut FormatContext<'_>, + children: Vec>, + ) -> Result> { + let mut node_children = vec![FormatTreeNode::new(format!( + "output columns: [{}]", + format_output_columns(self.output_schema()?, ctx.metadata, true) + ))]; + + if let Some(info) = &self.stat_info { + node_children.extend(plan_stats_info_to_format_tree(info)); + } + + node_children.extend(children); - pub func_arg: AsyncFunctionArgument, + Ok(FormatTreeNode::with_children( + "AsyncFunction".to_string(), + node_children, + )) + } + + #[recursive::recursive] + fn try_find_single_data_source(&self) -> Option<&DataSourcePlan> { + self.input.try_find_single_data_source() + } + + fn get_desc(&self) -> Result { + Ok(self + .async_func_descs + .iter() + .map(|x| x.display_name.clone()) + .join(", ")) + } + + fn derive(&self, mut children: Vec) -> PhysicalPlan { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + self.input.build_pipeline(builder)?; + + let operators = TransformAsyncFunction::init_operators(&self.async_func_descs)?; + let sequence_counters = + TransformAsyncFunction::create_sequence_counters(self.async_func_descs.len()); + + builder.main_pipeline.add_async_transformer(|| { + TransformAsyncFunction::new( + builder.ctx.clone(), + self.async_func_descs.clone(), + operators.clone(), + sequence_counters.clone(), + ) + }); + + Ok(()) + } } impl PhysicalPlanBuilder { - pub(crate) async fn build_async_func( + pub async fn build_async_func( &mut self, s_expr: &SExpr, - async_func_plan: &crate::plans::AsyncFunction, + async_func_plan: &databend_common_sql::plans::AsyncFunction, mut required: ColumnSet, stat_info: PlanStatsInfo, ) -> Result { @@ -128,9 +208,9 @@ impl PhysicalPlanBuilder { }) .collect::>>()?; - Ok(PhysicalPlan::AsyncFunction(AsyncFunction { - plan_id: 0, - input: Box::new(input), + Ok(Box::new(AsyncFunction { + input, + meta: PhysicalPlanMeta::new("AsyncFunction"), async_func_descs, stat_info: Some(stat_info), })) diff --git a/src/query/service/src/physical_plans/physical_broadcast.rs b/src/query/service/src/physical_plans/physical_broadcast.rs new file mode 100644 index 0000000000000..505387056f9b2 --- /dev/null +++ b/src/query/service/src/physical_plans/physical_broadcast.rs @@ -0,0 +1,159 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::any::Any; + +use databend_common_ast::ast::FormatTreeNode; +use databend_common_catalog::table_context::TableContext; +use databend_common_exception::Result; +use databend_common_expression::DataSchemaRef; +use databend_common_sql::executor::physical_plans::FragmentKind; + +use super::Exchange; +use crate::physical_plans::format::FormatContext; +use crate::physical_plans::physical_plan::IPhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlanMeta; +use crate::pipelines::processors::transforms::BroadcastSinkProcessor; +use crate::pipelines::processors::transforms::BroadcastSourceProcessor; +use crate::pipelines::PipelineBuilder; + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct BroadcastSource { + pub meta: PhysicalPlanMeta, + pub broadcast_id: u32, +} + +#[typetag::serde] +impl IPhysicalPlan for BroadcastSource { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn derive(&self, children: Vec) -> PhysicalPlan { + assert!(children.is_empty()); + Box::new(self.clone()) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + let receiver = builder.ctx.broadcast_source_receiver(self.broadcast_id); + + builder.main_pipeline.add_source( + |output| { + BroadcastSourceProcessor::create(builder.ctx.clone(), receiver.clone(), output) + }, + 1, + ) + } +} + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct BroadcastSink { + pub meta: PhysicalPlanMeta, + pub broadcast_id: u32, + pub input: PhysicalPlan, +} + +#[typetag::serde] +impl IPhysicalPlan for BroadcastSink { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn output_schema(&self) -> Result { + Ok(DataSchemaRef::default()) + } + + fn children<'a>(&'a self) -> Box + 'a> { + Box::new(std::iter::once(&self.input)) + } + + fn children_mut<'a>(&'a mut self) -> Box + 'a> { + Box::new(std::iter::once(&mut self.input)) + } + + fn to_format_node( + &self, + _ctx: &mut FormatContext<'_>, + _: Vec>, + ) -> Result> { + // ignore children + Ok(FormatTreeNode::new("RuntimeFilterSink".to_string())) + } + + fn derive(&self, mut children: Vec) -> PhysicalPlan { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + self.input.build_pipeline(builder)?; + + builder.main_pipeline.resize(1, true)?; + builder.main_pipeline.add_sink(|input| { + BroadcastSinkProcessor::create( + input, + builder.ctx.broadcast_sink_sender(self.broadcast_id), + ) + }) + } +} + +pub fn build_broadcast_plan(broadcast_id: u32) -> Result { + let broadcast_source: PhysicalPlan = Box::new(BroadcastSource { + meta: PhysicalPlanMeta::new("BroadcastSource"), + broadcast_id, + }); + + let exchange = Box::new(Exchange { + input: broadcast_source, + kind: FragmentKind::Expansive, + keys: vec![], + allow_adjust_parallelism: true, + ignore_exchange: false, + meta: PhysicalPlanMeta::new("Exchange"), + }); + + Ok(Box::new(BroadcastSink { + broadcast_id, + input: exchange, + meta: PhysicalPlanMeta::new("BroadcastSink"), + })) +} + +pub fn build_broadcast_plans(ctx: &dyn TableContext) -> Result> { + let mut plans = vec![]; + let next_broadcast_id = ctx.get_next_broadcast_id(); + ctx.reset_broadcast_id(); + for broadcast_id in 0..next_broadcast_id { + plans.push(build_broadcast_plan(broadcast_id)?); + } + Ok(plans) +} diff --git a/src/query/service/src/physical_plans/physical_cache_scan.rs b/src/query/service/src/physical_plans/physical_cache_scan.rs new file mode 100644 index 0000000000000..ef029e3281a7b --- /dev/null +++ b/src/query/service/src/physical_plans/physical_cache_scan.rs @@ -0,0 +1,148 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::any::Any; + +use databend_common_ast::ast::FormatTreeNode; +use databend_common_exception::ErrorCode; +use databend_common_exception::Result; +use databend_common_expression::DataSchemaRef; +use databend_common_expression::DataSchemaRefExt; +use databend_common_sql::plans::CacheSource; +use databend_common_sql::ColumnSet; + +use crate::physical_plans::format::format_output_columns; +use crate::physical_plans::format::FormatContext; +use crate::physical_plans::physical_plan::IPhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlanMeta; +use crate::physical_plans::physical_plan_builder::PhysicalPlanBuilder; +use crate::pipelines::processors::transforms::CacheSourceState; +use crate::pipelines::processors::transforms::HashJoinCacheState; +use crate::pipelines::processors::transforms::TransformCacheScan; +use crate::pipelines::PipelineBuilder; + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct CacheScan { + pub meta: PhysicalPlanMeta, + pub cache_source: CacheSource, + pub output_schema: DataSchemaRef, +} + +#[typetag::serde] +impl IPhysicalPlan for CacheScan { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn output_schema(&self) -> Result { + Ok(self.output_schema.clone()) + } + + fn to_format_node( + &self, + ctx: &mut FormatContext<'_>, + _: Vec>, + ) -> Result> { + let mut children = Vec::with_capacity(2); + children.push(FormatTreeNode::new(format!( + "output columns: [{}]", + format_output_columns(self.output_schema()?, ctx.metadata, true) + ))); + + match &self.cache_source { + CacheSource::HashJoinBuild((cache_index, column_indexes)) => { + let mut column_indexes = column_indexes.clone(); + column_indexes.sort(); + children.push(FormatTreeNode::new(format!("cache index: {}", cache_index))); + children.push(FormatTreeNode::new(format!( + "column indexes: {:?}", + column_indexes + ))); + } + } + + Ok(FormatTreeNode::with_children( + "CacheScan".to_string(), + children, + )) + } + + fn derive(&self, children: Vec) -> PhysicalPlan { + assert!(children.is_empty()); + Box::new(self.clone()) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + let max_threads = builder.settings.get_max_threads()?; + let max_block_size = builder.settings.get_max_block_size()? as usize; + let cache_source_state = match &self.cache_source { + CacheSource::HashJoinBuild((cache_index, column_indexes)) => { + let hash_join_state = match builder.hash_join_states.get(cache_index) { + Some(hash_join_state) => hash_join_state.clone(), + None => { + return Err(ErrorCode::Internal( + "Hash join state not found during building cache scan".to_string(), + )); + } + }; + CacheSourceState::HashJoinCacheState(HashJoinCacheState::new( + column_indexes.clone(), + hash_join_state, + max_block_size, + )) + } + }; + + builder.main_pipeline.add_source( + |output| { + TransformCacheScan::create(builder.ctx.clone(), output, cache_source_state.clone()) + }, + max_threads as usize, + ) + } +} + +impl PhysicalPlanBuilder { + pub async fn build_cache_scan( + &mut self, + scan: &databend_common_sql::plans::CacheScan, + required: ColumnSet, + ) -> Result { + // 1. Prune unused Columns. + let used: ColumnSet = required.intersection(&scan.columns).cloned().collect(); + let (cache_source, fields) = if used == scan.columns { + (scan.cache_source.clone(), scan.schema.fields().clone()) + } else { + let new_scan = scan.prune_columns(used); + ( + new_scan.cache_source.clone(), + new_scan.schema.fields().clone(), + ) + }; + // 2. Build physical plan. + Ok(Box::new(CacheScan { + cache_source, + meta: PhysicalPlanMeta::new("CacheScan"), + output_schema: DataSchemaRefExt::create(fields), + })) + } +} diff --git a/src/query/service/src/physical_plans/physical_column_mutation.rs b/src/query/service/src/physical_plans/physical_column_mutation.rs new file mode 100644 index 0000000000000..f47a2453bdf31 --- /dev/null +++ b/src/query/service/src/physical_plans/physical_column_mutation.rs @@ -0,0 +1,203 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::any::Any; +use std::collections::HashMap; + +use databend_common_ast::ast::FormatTreeNode; +use databend_common_catalog::table::Table; +use databend_common_exception::Result; +use databend_common_expression::DataSchemaRef; +use databend_common_expression::RemoteExpr; +use databend_common_functions::BUILTIN_FUNCTIONS; +use databend_common_meta_app::schema::TableInfo; +use databend_common_pipeline_transforms::TransformPipelineHelper; +use databend_common_sql::evaluator::BlockOperator; +use databend_common_sql::evaluator::CompoundBlockOperator; +use databend_common_sql::executor::physical_plans::MutationKind; +use databend_common_storages_fuse::operations::TransformSerializeBlock; +use databend_common_storages_fuse::FuseTable; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; + +use crate::physical_plans::format::FormatContext; +use crate::physical_plans::physical_plan::IPhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlanMeta; +use crate::pipelines::PipelineBuilder; + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct ColumnMutation { + pub meta: PhysicalPlanMeta, + pub input: PhysicalPlan, + pub table_info: TableInfo, + pub mutation_expr: Option>, + pub computed_expr: Option>, + pub mutation_kind: MutationKind, + pub field_id_to_schema_index: HashMap, + pub input_num_columns: usize, + pub has_filter_column: bool, + pub table_meta_timestamps: TableMetaTimestamps, + pub udf_col_num: usize, +} + +#[typetag::serde] +impl IPhysicalPlan for ColumnMutation { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn output_schema(&self) -> Result { + Ok(DataSchemaRef::default()) + } + + fn children<'a>(&'a self) -> Box + 'a> { + Box::new(std::iter::once(&self.input)) + } + + fn children_mut<'a>(&'a mut self) -> Box + 'a> { + Box::new(std::iter::once(&mut self.input)) + } + + fn to_format_node( + &self, + _: &mut FormatContext<'_>, + mut children: Vec>, + ) -> Result> { + // ignore self + assert_eq!(children.len(), 1); + Ok(children.pop().unwrap()) + } + + fn derive(&self, mut children: Vec) -> PhysicalPlan { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + self.input.build_pipeline(builder)?; + + let mut field_id_to_schema_index = self.field_id_to_schema_index.clone(); + if let Some(mutation_expr) = &self.mutation_expr { + let mut block_operators = Vec::new(); + let mut next_column_offset = self.input_num_columns; + let mut schema_offset_to_new_offset = HashMap::new(); + + // Build update expression BlockOperator. + let mut exprs = Vec::with_capacity(mutation_expr.len()); + for (id, remote_expr) in mutation_expr { + let expr = remote_expr.as_expr(&BUILTIN_FUNCTIONS); + let schema_index = field_id_to_schema_index.get(id).unwrap(); + schema_offset_to_new_offset.insert(*schema_index, next_column_offset); + field_id_to_schema_index + .entry(*id) + .and_modify(|e| *e = next_column_offset); + next_column_offset += 1; + exprs.push(expr); + } + if !exprs.is_empty() { + block_operators.push(BlockOperator::Map { + exprs, + projections: None, + }); + } + + // Build computed expression BlockOperator. + if let Some(computed_expr) = &self.computed_expr + && !computed_expr.is_empty() + { + let mut exprs = Vec::with_capacity(computed_expr.len()); + for (id, remote_expr) in computed_expr.iter() { + let expr = + remote_expr + .as_expr(&BUILTIN_FUNCTIONS) + .project_column_ref(|index| { + *schema_offset_to_new_offset.get(index).unwrap_or(index) + }); + let schema_index = field_id_to_schema_index.get(id).unwrap(); + schema_offset_to_new_offset.insert(*schema_index, next_column_offset); + field_id_to_schema_index + .entry(*id) + .and_modify(|e| *e = next_column_offset); + next_column_offset += 1; + exprs.push(expr); + } + block_operators.push(BlockOperator::Map { + exprs, + projections: None, + }); + } + + // Keep the original order of the columns. + let num_output_columns = self.input_num_columns - self.has_filter_column as usize - self.udf_col_num; + let mut projection = Vec::with_capacity(num_output_columns); + for idx in 0..num_output_columns { + if let Some(index) = schema_offset_to_new_offset.get(&idx) { + projection.push(*index); + } else { + projection.push(idx); + } + } + block_operators.push(BlockOperator::Project { projection }); + + builder.main_pipeline.add_transformer(|| { + CompoundBlockOperator::new( + block_operators.clone(), + builder.func_ctx.clone(), + self.input_num_columns, + ) + }); + + return Ok(()); + } + + let table = builder + .ctx + .build_table_by_table_info(&self.table_info, None)?; + let table = FuseTable::try_from_table(table.as_ref())?; + + let block_thresholds = table.get_block_thresholds(); + let cluster_stats_gen = if matches!(self.mutation_kind, MutationKind::Delete) { + table.get_cluster_stats_gen(builder.ctx.clone(), 0, block_thresholds, None)? + } else { + table.cluster_gen_for_append( + builder.ctx.clone(), + &mut builder.main_pipeline, + block_thresholds, + None, + )? + }; + + builder.main_pipeline.add_transform(|input, output| { + let proc = TransformSerializeBlock::try_create( + builder.ctx.clone(), + input, + output, + table, + cluster_stats_gen.clone(), + self.mutation_kind, + self.table_meta_timestamps, + )?; + proc.into_processor() + }) + } +} diff --git a/src/query/service/src/physical_plans/physical_commit_sink.rs b/src/query/service/src/physical_plans/physical_commit_sink.rs new file mode 100644 index 0000000000000..0dec7f3e463fa --- /dev/null +++ b/src/query/service/src/physical_plans/physical_commit_sink.rs @@ -0,0 +1,216 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::any::Any; +use std::sync::Arc; + +use databend_common_catalog::plan::ReclusterInfoSideCar; +use databend_common_catalog::table_context::TableContext; +use databend_common_exception::Result; +use databend_common_expression::DataSchemaRef; +use databend_common_meta_app::schema::TableInfo; +use databend_common_meta_app::schema::UpdateStreamMetaReq; +use databend_common_pipeline_core::ExecutionInfo; +use databend_common_pipeline_transforms::TransformPipelineHelper; +use databend_common_sql::executor::physical_plans::MutationKind; +use databend_common_sql::plans::TruncateMode; +use databend_common_storages_fuse::operations::MutationGenerator; +use databend_common_storages_fuse::operations::TableMutationAggregator; +use databend_common_storages_fuse::operations::TransformMergeCommitMeta; +use databend_common_storages_fuse::operations::TruncateGenerator; +use databend_common_storages_fuse::FuseTable; +use databend_storages_common_table_meta::meta::ExtendedBlockMeta; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; +use databend_storages_common_table_meta::meta::TableSnapshot; +use databend_storages_common_table_meta::readers::snapshot_reader::TableSnapshotAccessor; + +use crate::physical_plans::physical_plan::IPhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlanMeta; +use crate::pipelines::PipelineBuilder; + +// serde is required by `PhysicalPlan` +/// The commit sink is used to commit the data to the table. +#[derive(serde::Serialize, serde::Deserialize, Clone, Debug)] +pub struct CommitSink { + pub meta: PhysicalPlanMeta, + pub input: PhysicalPlan, + pub snapshot: Option>, + pub table_info: TableInfo, + pub commit_type: CommitType, + pub update_stream_meta: Vec, + pub deduplicated_label: Option, + pub table_meta_timestamps: TableMetaTimestamps, + + // Used for recluster. + pub recluster_info: Option, +} + +#[typetag::serde] +impl IPhysicalPlan for CommitSink { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn output_schema(&self) -> Result { + Ok(DataSchemaRef::default()) + } + + fn children<'a>(&'a self) -> Box + 'a> { + Box::new(std::iter::once(&self.input)) + } + + fn children_mut<'a>(&'a mut self) -> Box + 'a> { + Box::new(std::iter::once(&mut self.input)) + } + + fn derive(&self, mut children: Vec) -> PhysicalPlan { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + self.input.build_pipeline(builder)?; + + let table = builder + .ctx + .build_table_by_table_info(&self.table_info, None)?; + let table = FuseTable::try_from_table(table.as_ref())?; + + builder.main_pipeline.try_resize(1)?; + match &self.commit_type { + CommitType::Truncate { mode } => { + let prev_snapshot_id = match mode { + TruncateMode::Delete => None, + _ => self.snapshot.as_ref().map(|snapshot| snapshot.snapshot_id), + }; + let snapshot_gen = TruncateGenerator::new(mode.clone()); + if matches!(mode, TruncateMode::Delete) { + let mutation_status = builder.ctx.get_mutation_status(); + let deleted_rows = self + .snapshot + .as_ref() + .map_or(0, |snapshot| snapshot.summary.row_count); + builder + .main_pipeline + .set_on_finished(move |info: &ExecutionInfo| match &info.res { + Ok(_) => { + mutation_status.write().deleted_rows = deleted_rows; + Ok(()) + } + Err(error_code) => Err(error_code.clone()), + }); + } + builder.main_pipeline.add_sink(|input| { + databend_common_storages_fuse::operations::CommitSink::try_create( + table, + builder.ctx.clone(), + None, + self.update_stream_meta.clone(), + snapshot_gen.clone(), + input, + None, + prev_snapshot_id, + self.deduplicated_label.clone(), + self.table_meta_timestamps, + ) + }) + } + CommitType::Mutation { kind, merge_meta } => { + if *merge_meta { + let cluster_key_id = table.cluster_key_id(); + builder.main_pipeline.add_accumulating_transformer(|| { + TransformMergeCommitMeta::create(cluster_key_id) + }); + } else { + builder + .main_pipeline + .add_async_accumulating_transformer(|| { + let base_segments = if matches!( + kind, + MutationKind::Compact + | MutationKind::Insert + | MutationKind::Recluster + ) { + vec![] + } else { + self.snapshot.segments().to_vec() + }; + + // extract re-cluster related mutations from physical plan + let recluster_info = self.recluster_info.clone().unwrap_or_default(); + + let extended_merged_blocks = recluster_info + .merged_blocks + .iter() + .map(|block_meta| { + Arc::new(ExtendedBlockMeta { + block_meta: Arc::unwrap_or_clone(block_meta.clone()), + draft_virtual_block_meta: None, + }) + }) + .collect::>>(); + + TableMutationAggregator::create( + table, + builder.ctx.clone(), + base_segments, + extended_merged_blocks, + recluster_info.removed_segment_indexes, + recluster_info.removed_statistics, + *kind, + self.table_meta_timestamps, + ) + }); + } + + let snapshot_gen = MutationGenerator::new(self.snapshot.clone(), *kind); + builder.main_pipeline.add_sink(|input| { + databend_common_storages_fuse::operations::CommitSink::try_create( + table, + builder.ctx.clone(), + None, + self.update_stream_meta.clone(), + snapshot_gen.clone(), + input, + None, + None, + self.deduplicated_label.clone(), + self.table_meta_timestamps, + ) + }) + } + } + } +} + +#[derive(serde::Serialize, serde::Deserialize, Clone, Debug)] +pub enum CommitType { + Truncate { + mode: TruncateMode, + }, + Mutation { + kind: MutationKind, + merge_meta: bool, + }, +} diff --git a/src/query/service/src/physical_plans/physical_compact_source.rs b/src/query/service/src/physical_plans/physical_compact_source.rs new file mode 100644 index 0000000000000..293c2d271e11d --- /dev/null +++ b/src/query/service/src/physical_plans/physical_compact_source.rs @@ -0,0 +1,289 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::any::Any; +use std::collections::HashSet; + +use databend_common_base::runtime::Runtime; +use databend_common_catalog::plan::PartInfoType; +use databend_common_catalog::plan::Partitions; +use databend_common_catalog::plan::PartitionsShuffleKind; +use databend_common_catalog::plan::Projection; +use databend_common_catalog::table::Table; +use databend_common_catalog::table::TableExt; +use databend_common_catalog::table_context::TableContext; +use databend_common_exception::ErrorCode; +use databend_common_exception::Result; +use databend_common_expression::ColumnId; +use databend_common_meta_app::schema::TableInfo; +use databend_common_pipeline_sources::EmptySource; +use databend_common_pipeline_sources::PrefetchAsyncSourcer; +use databend_common_pipeline_transforms::TransformPipelineHelper; +use databend_common_sql::executor::physical_plans::FragmentKind; +use databend_common_sql::executor::physical_plans::MutationKind; +use databend_common_sql::StreamContext; +use databend_common_storages_fuse::operations::BlockCompactMutator; +use databend_common_storages_fuse::operations::CompactLazyPartInfo; +use databend_common_storages_fuse::operations::CompactTransform; +use databend_common_storages_fuse::operations::TableMutationAggregator; +use databend_common_storages_fuse::operations::TransformSerializeBlock; +use databend_common_storages_fuse::FuseTable; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; + +use crate::physical_plans::physical_plan::IPhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlanMeta; +use crate::physical_plans::CommitSink; +use crate::physical_plans::CommitType; +use crate::physical_plans::Exchange; +use crate::physical_plans::PhysicalPlanBuilder; +use crate::pipelines::PipelineBuilder; + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct CompactSource { + pub meta: PhysicalPlanMeta, + pub parts: Partitions, + pub table_info: TableInfo, + pub column_ids: HashSet, + pub table_meta_timestamps: TableMetaTimestamps, +} + +#[typetag::serde] +impl IPhysicalPlan for CompactSource { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn derive(&self, children: Vec) -> PhysicalPlan { + assert!(children.is_empty()); + Box::new(self.clone()) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + let table = builder + .ctx + .build_table_by_table_info(&self.table_info, None)?; + let table = FuseTable::try_from_table(table.as_ref())?; + + if self.parts.is_empty() { + return builder.main_pipeline.add_source(EmptySource::create, 1); + } + + let is_lazy = self.parts.partitions_type() == PartInfoType::LazyLevel; + let thresholds = table.get_block_thresholds(); + let cluster_key_id = table.cluster_key_id(); + let mut max_threads = builder.settings.get_max_threads()? as usize; + + if is_lazy { + let query_ctx = builder.ctx.clone(); + + let lazy_parts = self + .parts + .partitions + .iter() + .map(|v| { + v.as_any() + .downcast_ref::() + .unwrap() + .clone() + }) + .collect::>(); + + let column_ids = self.column_ids.clone(); + builder.main_pipeline.set_on_init(move || { + let ctx = query_ctx.clone(); + let partitions = + Runtime::with_worker_threads(2, Some("build_compact_tasks".to_string()))? + .block_on(async move { + let partitions = BlockCompactMutator::build_compact_tasks( + ctx.clone(), + column_ids.clone(), + cluster_key_id, + thresholds, + lazy_parts, + ) + .await?; + + Result::<_>::Ok(partitions) + })?; + + let partitions = Partitions::create(PartitionsShuffleKind::Mod, partitions); + query_ctx.set_partitions(partitions)?; + Ok(()) + }); + } else { + max_threads = max_threads.min(self.parts.len()).max(1); + builder.ctx.set_partitions(self.parts.clone())?; + } + + let block_reader = table.create_block_reader( + builder.ctx.clone(), + Projection::Columns(table.all_column_indices()), + false, + table.change_tracking_enabled(), + false, + )?; + let stream_ctx = if table.change_tracking_enabled() { + Some(StreamContext::try_create( + builder.ctx.get_function_context()?, + table.schema_with_stream(), + table.get_table_info().ident.seq, + false, + false, + )?) + } else { + None + }; + // Add source pipe. + builder.main_pipeline.add_source( + |output| { + let source = databend_common_storages_fuse::operations::CompactSource::create( + builder.ctx.clone(), + block_reader.clone(), + 1, + ); + PrefetchAsyncSourcer::create(builder.ctx.clone(), output, source) + }, + max_threads, + )?; + let storage_format = table.get_storage_format(); + builder.main_pipeline.add_block_meta_transformer(|| { + CompactTransform::create( + builder.ctx.clone(), + block_reader.clone(), + storage_format, + stream_ctx.clone(), + ) + }); + + // sort + let cluster_stats_gen = table.cluster_gen_for_append( + builder.ctx.clone(), + &mut builder.main_pipeline, + thresholds, + None, + )?; + builder.main_pipeline.add_transform(|input, output| { + let proc = TransformSerializeBlock::try_create( + builder.ctx.clone(), + input, + output, + table, + cluster_stats_gen.clone(), + MutationKind::Compact, + self.table_meta_timestamps, + )?; + proc.into_processor() + })?; + + if is_lazy { + builder.main_pipeline.try_resize(1)?; + builder + .main_pipeline + .add_async_accumulating_transformer(|| { + TableMutationAggregator::create( + table, + builder.ctx.clone(), + vec![], + vec![], + vec![], + Default::default(), + MutationKind::Compact, + self.table_meta_timestamps, + ) + }); + } + Ok(()) + } +} + +impl PhysicalPlanBuilder { + pub async fn build_compact_block( + &mut self, + compact_block: &databend_common_sql::plans::OptimizeCompactBlock, + ) -> Result { + let databend_common_sql::plans::OptimizeCompactBlock { + catalog, + database, + table, + limit, + } = compact_block; + + let tenant = self.ctx.get_tenant(); + let catalog = self.ctx.get_catalog(catalog).await?; + let tbl = catalog.get_table(&tenant, database, table).await?; + // check mutability + tbl.check_mutable()?; + + let table_info = tbl.get_table_info().clone(); + + let Some((parts, snapshot)) = tbl.compact_blocks(self.ctx.clone(), limit.clone()).await? + else { + return Err(ErrorCode::NoNeedToCompact(format!( + "No need to do compact for '{database}'.'{table}'" + ))); + }; + + let table_meta_timestamps = self + .ctx + .get_table_meta_timestamps(tbl.as_ref(), Some(snapshot.clone()))?; + + let merge_meta = parts.partitions_type() == PartInfoType::LazyLevel; + let mut root: PhysicalPlan = Box::new(CompactSource { + parts, + table_info: table_info.clone(), + column_ids: snapshot.schema.to_leaf_column_id_set(), + table_meta_timestamps, + meta: PhysicalPlanMeta::new("ConstantTableScan"), + }); + + let is_distributed = (!self.ctx.get_cluster().is_empty()) + && self.ctx.get_settings().get_enable_distributed_compact()?; + if is_distributed { + root = Box::new(Exchange { + input: root, + kind: FragmentKind::Merge, + keys: vec![], + allow_adjust_parallelism: true, + ignore_exchange: false, + meta: PhysicalPlanMeta::new("ConstantTableScan"), + }); + } + + root = Box::new(CommitSink { + input: root, + table_info, + snapshot: Some(snapshot), + commit_type: CommitType::Mutation { + kind: MutationKind::Compact, + merge_meta, + }, + update_stream_meta: vec![], + deduplicated_label: None, + recluster_info: None, + table_meta_timestamps, + meta: PhysicalPlanMeta::new("CommitSink"), + }); + + root.adjust_plan_id(&mut 0); + Ok(root) + } +} diff --git a/src/query/service/src/physical_plans/physical_constant_table_scan.rs b/src/query/service/src/physical_plans/physical_constant_table_scan.rs new file mode 100644 index 0000000000000..62194ec8ead1a --- /dev/null +++ b/src/query/service/src/physical_plans/physical_constant_table_scan.rs @@ -0,0 +1,152 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::any::Any; + +use databend_common_ast::ast::FormatTreeNode; +use databend_common_exception::Result; +use databend_common_expression::Column; +use databend_common_expression::DataBlock; +use databend_common_expression::DataSchemaRef; +use databend_common_pipeline_sources::OneBlockSource; +use databend_common_sql::ColumnSet; +use databend_common_sql::IndexType; +use itertools::Itertools; + +use crate::physical_plans::format::format_output_columns; +use crate::physical_plans::format::FormatContext; +use crate::physical_plans::physical_plan::IPhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlanMeta; +use crate::physical_plans::PhysicalPlanBuilder; +use crate::pipelines::PipelineBuilder; + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct ConstantTableScan { + pub meta: PhysicalPlanMeta, + pub values: Vec, + pub num_rows: usize, + pub output_schema: DataSchemaRef, +} + +#[typetag::serde] +impl IPhysicalPlan for ConstantTableScan { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn output_schema(&self) -> Result { + Ok(self.output_schema.clone()) + } + + fn to_format_node( + &self, + ctx: &mut FormatContext<'_>, + _: Vec>, + ) -> Result> { + if self.num_rows == 0 { + return Ok(FormatTreeNode::new(self.name().to_string())); + } + + let mut children = Vec::with_capacity(self.values.len() + 1); + children.push(FormatTreeNode::new(format!( + "output columns: [{}]", + format_output_columns(self.output_schema()?, ctx.metadata, true) + ))); + for (i, value) in self.values.iter().enumerate() { + let column = value.iter().map(|val| format!("{val}")).join(", "); + children.push(FormatTreeNode::new(format!("column {}: [{}]", i, column))); + } + + Ok(FormatTreeNode::with_children( + self.name().to_string(), + children, + )) + } + + fn derive(&self, children: Vec) -> PhysicalPlan { + assert!(children.is_empty()); + Box::new(self.clone()) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + builder.main_pipeline.add_source( + |output| { + let block = if !self.values.is_empty() { + DataBlock::new_from_columns(self.values.clone()) + } else { + DataBlock::new(vec![], self.num_rows) + }; + OneBlockSource::create(output, block) + }, + 1, + ) + } +} + +impl ConstantTableScan { + pub fn name(&self) -> &str { + if self.num_rows == 0 { + "EmptyResultScan" + } else { + "ConstantTableScan" + } + } +} + +impl PhysicalPlanBuilder { + pub async fn build_constant_table_scan( + &mut self, + scan: &databend_common_sql::plans::ConstantTableScan, + required: ColumnSet, + ) -> Result { + debug_assert!(scan + .schema + .fields + .iter() + .map(|field| field.name().parse::().unwrap()) + .collect::() + .is_superset(&scan.columns)); + + let used: ColumnSet = required.intersection(&scan.columns).copied().collect(); + if used.len() < scan.columns.len() { + let databend_common_sql::plans::ConstantTableScan { + values, + num_rows, + schema, + .. + } = scan.prune_columns(used); + return Ok(Box::new(ConstantTableScan { + values, + num_rows, + output_schema: schema, + meta: PhysicalPlanMeta::new("ConstantTableScan"), + })); + } + + Ok(Box::new(ConstantTableScan { + values: scan.values.clone(), + num_rows: scan.num_rows, + output_schema: scan.schema.clone(), + meta: PhysicalPlanMeta::new("ConstantTableScan"), + })) + } +} diff --git a/src/query/service/src/physical_plans/physical_copy_into_location.rs b/src/query/service/src/physical_plans/physical_copy_into_location.rs new file mode 100644 index 0000000000000..8564a45001685 --- /dev/null +++ b/src/query/service/src/physical_plans/physical_copy_into_location.rs @@ -0,0 +1,129 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::any::Any; + +use chrono::Duration; +use databend_common_ast::ast::FormatTreeNode; +use databend_common_catalog::plan::DataSourcePlan; +use databend_common_exception::Result; +use databend_common_expression::types::DataType; +use databend_common_expression::types::NumberDataType; +use databend_common_expression::DataField; +use databend_common_expression::DataSchemaRef; +use databend_common_expression::DataSchemaRefExt; +use databend_common_expression::TableSchemaRef; +use databend_common_sql::ColumnBinding; +use databend_common_storages_stage::StageSinkTable; +use databend_storages_common_stage::CopyIntoLocationInfo; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; + +use crate::physical_plans::format::FormatContext; +use crate::physical_plans::physical_plan::IPhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlanMeta; +use crate::pipelines::PipelineBuilder; + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct CopyIntoLocation { + pub meta: PhysicalPlanMeta, + pub input: PhysicalPlan, + pub project_columns: Vec, + pub input_data_schema: DataSchemaRef, + pub input_table_schema: TableSchemaRef, + pub info: CopyIntoLocationInfo, +} + +#[typetag::serde] +impl IPhysicalPlan for CopyIntoLocation { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn output_schema(&self) -> Result { + Ok(DataSchemaRefExt::create(vec![ + DataField::new("rows_unloaded", DataType::Number(NumberDataType::UInt64)), + DataField::new("input_bytes", DataType::Number(NumberDataType::UInt64)), + DataField::new("output_bytes", DataType::Number(NumberDataType::UInt64)), + ])) + } + + fn children<'a>(&'a self) -> Box + 'a> { + Box::new(std::iter::once(&self.input)) + } + + fn children_mut<'a>(&'a mut self) -> Box + 'a> { + Box::new(std::iter::once(&mut self.input)) + } + + fn to_format_node( + &self, + _ctx: &mut FormatContext<'_>, + children: Vec>, + ) -> Result> { + Ok(FormatTreeNode::with_children( + "CopyIntoLocation".to_string(), + children, + )) + } + + #[recursive::recursive] + fn try_find_single_data_source(&self) -> Option<&DataSourcePlan> { + self.input.try_find_single_data_source() + } + + fn derive(&self, mut children: Vec) -> PhysicalPlan { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + self.input.build_pipeline(builder)?; + + // Reorder the result for select clause + PipelineBuilder::build_result_projection( + &builder.func_ctx, + self.input.output_schema()?, + &self.project_columns, + &mut builder.main_pipeline, + false, + )?; + + // The stage table that copying into + let to_table = StageSinkTable::create(self.info.clone(), self.input_table_schema.clone())?; + + // StageSinkTable needs not to hold the table meta timestamps invariants, just pass a dummy one + let dummy_table_meta_timestamps = TableMetaTimestamps::new(None, Duration::hours(1)); + PipelineBuilder::build_append2table_with_commit_pipeline( + builder.ctx.clone(), + &mut builder.main_pipeline, + to_table, + self.input_data_schema.clone(), + None, + vec![], + false, + unsafe { builder.settings.get_deduplicate_label()? }, + dummy_table_meta_timestamps, + ) + } +} diff --git a/src/query/service/src/physical_plans/physical_copy_into_table.rs b/src/query/service/src/physical_plans/physical_copy_into_table.rs new file mode 100644 index 0000000000000..622e9f7e47245 --- /dev/null +++ b/src/query/service/src/physical_plans/physical_copy_into_table.rs @@ -0,0 +1,173 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::any::Any; + +use databend_common_ast::ast::FormatTreeNode; +use databend_common_catalog::plan::StageTableInfo; +use databend_common_catalog::table_context::TableContext; +use databend_common_exception::Result; +use databend_common_expression::DataField; +use databend_common_expression::DataSchemaRef; +use databend_common_expression::DataSchemaRefExt; +use databend_common_expression::Scalar; +use databend_common_meta_app::schema::TableInfo; +use databend_common_sql::plans::CopyIntoTableMode; +use databend_common_sql::plans::ValidationMode; +use databend_common_sql::ColumnBinding; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; + +use crate::physical_plans::format::FormatContext; +use crate::physical_plans::physical_plan::IPhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlanMeta; +use crate::pipelines::PipelineBuilder; + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct CopyIntoTable { + pub meta: PhysicalPlanMeta, + 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, + pub table_meta_timestamps: TableMetaTimestamps, +} + +#[typetag::serde] +impl IPhysicalPlan for CopyIntoTable { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn output_schema(&self) -> Result { + Ok(DataSchemaRefExt::create(vec![])) + } + + fn children<'a>(&'a self) -> Box + 'a> { + match &self.source { + CopyIntoTableSource::Query(v) => Box::new(std::iter::once(v)), + CopyIntoTableSource::Stage(v) => Box::new(std::iter::once(v)), + } + } + + fn children_mut<'a>(&'a mut self) -> Box + 'a> { + match &mut self.source { + CopyIntoTableSource::Query(v) => Box::new(std::iter::once(v)), + CopyIntoTableSource::Stage(v) => Box::new(std::iter::once(v)), + } + } + + fn to_format_node( + &self, + _ctx: &mut FormatContext<'_>, + children: Vec>, + ) -> Result> { + Ok(FormatTreeNode::with_children( + format!("CopyIntoTable: {}", self.table_info), + children, + )) + } + + fn derive(&self, mut children: Vec) -> PhysicalPlan { + match &self.source { + CopyIntoTableSource::Query(_) => { + let mut new_copy_into_table = self.clone(); + assert_eq!(children.len(), 1); + let input = children.pop().unwrap(); + new_copy_into_table.source = CopyIntoTableSource::Query(input); + Box::new(new_copy_into_table) + } + CopyIntoTableSource::Stage(_) => { + let mut new_copy_into_table = self.clone(); + assert_eq!(children.len(), 1); + let input = children.pop().unwrap(); + new_copy_into_table.source = CopyIntoTableSource::Stage(input); + Box::new(new_copy_into_table) + } + } + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + let to_table = builder + .ctx + .build_table_by_table_info(&self.table_info, None)?; + + // build_copy_into_table_input + let source_schema = match &self.source { + CopyIntoTableSource::Query(input) => { + input.build_pipeline(builder)?; + + // Reorder the result for select clause + PipelineBuilder::build_result_projection( + &builder.func_ctx, + input.output_schema()?, + self.project_columns.as_ref().unwrap(), + &mut builder.main_pipeline, + false, + )?; + let fields = self + .project_columns + .as_ref() + .unwrap() + .iter() + .map(|column_binding| { + DataField::new( + &column_binding.column_name, + *column_binding.data_type.clone(), + ) + }) + .collect(); + + DataSchemaRefExt::create(fields) + } + CopyIntoTableSource::Stage(input) => { + builder + .ctx + .set_read_block_thresholds(to_table.get_block_thresholds()); + + builder.build_pipeline(input)?; + self.required_source_schema.clone() + } + }; + + PipelineBuilder::build_copy_into_table_append( + builder.ctx.clone(), + &mut builder.main_pipeline, + self, + source_schema, + to_table, + )?; + Ok(()) + } +} + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub enum CopyIntoTableSource { + Query(PhysicalPlan), + Stage(PhysicalPlan), +} diff --git a/src/query/service/src/physical_plans/physical_distributed_insert_select.rs b/src/query/service/src/physical_plans/physical_distributed_insert_select.rs new file mode 100644 index 0000000000000..a90043dfed384 --- /dev/null +++ b/src/query/service/src/physical_plans/physical_distributed_insert_select.rs @@ -0,0 +1,124 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::any::Any; + +use databend_common_catalog::plan::DataSourcePlan; +use databend_common_exception::Result; +use databend_common_expression::DataSchemaRef; +use databend_common_meta_app::schema::TableInfo; +use databend_common_pipeline_transforms::TransformPipelineHelper; +use databend_common_sql::ColumnBinding; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; + +use crate::physical_plans::physical_plan::IPhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlanMeta; +use crate::pipelines::processors::transforms::TransformCastSchema; +use crate::pipelines::PipelineBuilder; + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct DistributedInsertSelect { + pub meta: PhysicalPlanMeta, + pub input: PhysicalPlan, + pub table_info: TableInfo, + pub insert_schema: DataSchemaRef, + pub select_schema: DataSchemaRef, + pub select_column_bindings: Vec, + pub cast_needed: bool, + pub table_meta_timestamps: TableMetaTimestamps, +} + +#[typetag::serde] +impl IPhysicalPlan for DistributedInsertSelect { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn output_schema(&self) -> Result { + Ok(DataSchemaRef::default()) + } + + fn children<'a>(&'a self) -> Box + 'a> { + Box::new(std::iter::once(&self.input)) + } + + fn children_mut<'a>(&'a mut self) -> Box + 'a> { + Box::new(std::iter::once(&mut self.input)) + } + + #[recursive::recursive] + fn try_find_single_data_source(&self) -> Option<&DataSourcePlan> { + self.input.try_find_single_data_source() + } + + fn derive(&self, mut children: Vec) -> PhysicalPlan { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + self.input.build_pipeline(builder)?; + + let select_schema = &self.select_schema; + let insert_schema = &self.insert_schema; + // should render result for select + PipelineBuilder::build_result_projection( + &builder.func_ctx, + self.input.output_schema()?, + &self.select_column_bindings, + &mut builder.main_pipeline, + false, + )?; + + if self.cast_needed { + builder.main_pipeline.try_add_transformer(|| { + TransformCastSchema::try_new( + select_schema.clone(), + insert_schema.clone(), + builder.func_ctx.clone(), + ) + })?; + } + + let table = builder + .ctx + .build_table_by_table_info(&self.table_info, None)?; + + let source_schema = insert_schema; + PipelineBuilder::fill_and_reorder_columns( + builder.ctx.clone(), + &mut builder.main_pipeline, + table.clone(), + source_schema.clone(), + )?; + + table.append_data( + builder.ctx.clone(), + &mut builder.main_pipeline, + self.table_meta_timestamps, + )?; + + Ok(()) + } +} diff --git a/src/query/sql/src/executor/physical_plans/physical_eval_scalar.rs b/src/query/service/src/physical_plans/physical_eval_scalar.rs similarity index 64% rename from src/query/sql/src/executor/physical_plans/physical_eval_scalar.rs rename to src/query/service/src/physical_plans/physical_eval_scalar.rs index 89c30b3bc8e92..f24b15b878e63 100644 --- a/src/query/sql/src/executor/physical_plans/physical_eval_scalar.rs +++ b/src/query/service/src/physical_plans/physical_eval_scalar.rs @@ -12,10 +12,14 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::any::Any; use std::collections::BTreeSet; +use std::collections::HashMap; use std::collections::HashSet; use std::sync::Arc; +use databend_common_ast::ast::FormatTreeNode; +use databend_common_catalog::plan::DataSourcePlan; use databend_common_exception::Result; use databend_common_expression::ConstantFolder; use databend_common_expression::DataField; @@ -24,38 +28,59 @@ use databend_common_expression::DataSchemaRefExt; use databend_common_expression::RemoteExpr; use databend_common_expression::Scalar; use databend_common_functions::BUILTIN_FUNCTIONS; +use databend_common_pipeline_transforms::TransformPipelineHelper; +use databend_common_sql::evaluator::BlockOperator; +use databend_common_sql::evaluator::CompoundBlockOperator; +use databend_common_sql::optimizer::ir::Matcher; +use databend_common_sql::optimizer::ir::SExpr; +use databend_common_sql::plans::Filter; +use databend_common_sql::plans::FunctionCall; +use databend_common_sql::plans::ProjectSet; +use databend_common_sql::plans::RelOp; +use databend_common_sql::plans::RelOperator; +use databend_common_sql::plans::ScalarExpr; +use databend_common_sql::plans::ScalarItem; +use databend_common_sql::plans::Visitor; +use databend_common_sql::ColumnSet; +use databend_common_sql::IndexType; +use databend_common_sql::TypeCheck; +use itertools::Itertools; -use crate::executor::explain::PlanStatsInfo; -use crate::executor::physical_plan::PhysicalPlan; -use crate::executor::physical_plan_builder::PhysicalPlanBuilder; -use crate::optimizer::ir::Matcher; -use crate::optimizer::ir::SExpr; -use crate::plans::Filter; -use crate::plans::FunctionCall; -use crate::plans::ProjectSet; -use crate::plans::RelOp; -use crate::plans::RelOperator; -use crate::plans::ScalarExpr; -use crate::plans::ScalarItem; -use crate::plans::Visitor; -use crate::ColumnSet; -use crate::IndexType; -use crate::TypeCheck; +use crate::physical_plans::explain::PlanStatsInfo; +use crate::physical_plans::format::format_output_columns; +use crate::physical_plans::format::plan_stats_info_to_format_tree; +use crate::physical_plans::format::FormatContext; +use crate::physical_plans::physical_plan::IPhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlanMeta; +use crate::physical_plans::physical_plan_builder::PhysicalPlanBuilder; +use crate::pipelines::PipelineBuilder; #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct EvalScalar { - // A unique id of operator in a `PhysicalPlan` tree, only used for display. - pub plan_id: u32, + meta: PhysicalPlanMeta, pub projections: ColumnSet, - pub input: Box, + pub input: PhysicalPlan, pub exprs: Vec<(RemoteExpr, IndexType)>, /// Only used for explain pub stat_info: Option, } -impl EvalScalar { - pub fn output_schema(&self) -> Result { +#[typetag::serde] +impl IPhysicalPlan for EvalScalar { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn output_schema(&self) -> Result { if self.exprs.is_empty() { return self.input.output_schema(); } @@ -78,13 +103,124 @@ impl EvalScalar { } Ok(DataSchemaRefExt::create(fields)) } + + fn children<'a>(&'a self) -> Box + 'a> { + Box::new(std::iter::once(&self.input)) + } + + fn children_mut<'a>(&'a mut self) -> Box + 'a> { + Box::new(std::iter::once(&mut self.input)) + } + + fn to_format_node( + &self, + ctx: &mut FormatContext<'_>, + mut children: Vec>, + ) -> Result> { + if self.exprs.is_empty() { + assert_eq!(children.len(), 1); + return Ok(children.pop().unwrap()); + } + + let scalars = self + .exprs + .iter() + .map(|(expr, _)| expr.as_expr(&BUILTIN_FUNCTIONS).sql_display()) + .collect::>() + .join(", "); + + let mut node_children = vec![ + FormatTreeNode::new(format!( + "output columns: [{}]", + format_output_columns(self.output_schema()?, ctx.metadata, true) + )), + FormatTreeNode::new(format!("expressions: [{scalars}]")), + ]; + + if let Some(info) = &self.stat_info { + node_children.extend(plan_stats_info_to_format_tree(info)); + } + + node_children.extend(children); + + Ok(FormatTreeNode::with_children( + "EvalScalar".to_string(), + node_children, + )) + } + + #[recursive::recursive] + fn try_find_single_data_source(&self) -> Option<&DataSourcePlan> { + self.input.try_find_single_data_source() + } + + fn display_in_profile(&self) -> bool { + !self.exprs.is_empty() + } + + fn get_desc(&self) -> Result { + Ok(self + .exprs + .iter() + .map(|(x, _)| x.as_expr(&BUILTIN_FUNCTIONS).sql_display()) + .join(", ")) + } + + fn get_labels(&self) -> Result>> { + Ok(HashMap::from([( + String::from("List of Expressions"), + self.exprs + .iter() + .map(|(x, _)| x.as_expr(&BUILTIN_FUNCTIONS).sql_display()) + .collect(), + )])) + } + + fn derive(&self, mut children: Vec) -> PhysicalPlan { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + self.input.build_pipeline(builder)?; + + let input_schema = self.input.output_schema()?; + let exprs = self + .exprs + .iter() + .map(|(scalar, _)| scalar.as_expr(&BUILTIN_FUNCTIONS)) + .collect::>(); + + if exprs.is_empty() { + return Ok(()); + } + + let op = BlockOperator::Map { + exprs, + projections: Some(self.projections.clone()), + }; + + let num_input_columns = input_schema.num_fields(); + + builder.main_pipeline.add_transformer(|| { + CompoundBlockOperator::new( + vec![op.clone()], + builder.func_ctx.clone(), + num_input_columns, + ) + }); + + Ok(()) + } } impl PhysicalPlanBuilder { - pub(crate) async fn build_eval_scalar( + pub async fn build_eval_scalar( &mut self, s_expr: &SExpr, - eval_scalar: &crate::plans::EvalScalar, + eval_scalar: &databend_common_sql::plans::EvalScalar, mut required: ColumnSet, stat_info: PlanStatsInfo, ) -> Result { @@ -117,14 +253,14 @@ impl PhysicalPlanBuilder { .cloned() .collect(); let column_projections = column_projections.clone().into_iter().collect::>(); - let eval_scalar = crate::plans::EvalScalar { items: used }; + let eval_scalar = databend_common_sql::plans::EvalScalar { items: used }; self.create_eval_scalar(&eval_scalar, column_projections, input, stat_info) } } - pub(crate) fn create_eval_scalar( + pub fn create_eval_scalar( &mut self, - eval_scalar: &crate::plans::EvalScalar, + eval_scalar: &databend_common_sql::plans::EvalScalar, column_projections: Vec, input: PhysicalPlan, stat_info: PlanStatsInfo, @@ -165,12 +301,12 @@ impl PhysicalPlanBuilder { projections.insert(index + input_column_nums); } } - Ok(PhysicalPlan::EvalScalar(EvalScalar { - plan_id: 0, - projections, - input: Box::new(input), + Ok(Box::new(EvalScalar { + input, exprs, + projections, stat_info: Some(stat_info), + meta: PhysicalPlanMeta::new("EvalScalar"), })) } diff --git a/src/query/service/src/physical_plans/physical_exchange.rs b/src/query/service/src/physical_plans/physical_exchange.rs new file mode 100644 index 0000000000000..2f4ea2b514c31 --- /dev/null +++ b/src/query/service/src/physical_plans/physical_exchange.rs @@ -0,0 +1,167 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::any::Any; + +use databend_common_ast::ast::FormatTreeNode; +use databend_common_catalog::plan::DataSourcePlan; +use databend_common_exception::Result; +use databend_common_expression::ConstantFolder; +use databend_common_expression::RemoteExpr; +use databend_common_functions::BUILTIN_FUNCTIONS; +use databend_common_sql::executor::physical_plans::FragmentKind; +use databend_common_sql::optimizer::ir::SExpr; +use databend_common_sql::ColumnSet; +use databend_common_sql::TypeCheck; + +use crate::physical_plans::format::format_output_columns; +use crate::physical_plans::format::FormatContext; +use crate::physical_plans::physical_plan::IPhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlanMeta; +use crate::physical_plans::PhysicalPlanBuilder; + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct Exchange { + pub meta: PhysicalPlanMeta, + pub input: PhysicalPlan, + pub kind: FragmentKind, + pub keys: Vec, + pub ignore_exchange: bool, + pub allow_adjust_parallelism: bool, +} + +#[typetag::serde] +impl IPhysicalPlan for Exchange { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn to_format_node( + &self, + ctx: &mut FormatContext<'_>, + children: Vec>, + ) -> Result> { + let mut node_children = vec![ + FormatTreeNode::new(format!( + "output columns: [{}]", + format_output_columns(self.output_schema()?, ctx.metadata, true) + )), + FormatTreeNode::new(format!("exchange type: {}", match self.kind { + FragmentKind::Init => "Init-Partition".to_string(), + FragmentKind::Normal => format!( + "Hash({})", + self.keys + .iter() + .map(|key| { key.as_expr(&BUILTIN_FUNCTIONS).sql_display() }) + .collect::>() + .join(", ") + ), + FragmentKind::Expansive => "Broadcast".to_string(), + FragmentKind::Merge => "Merge".to_string(), + })), + ]; + + node_children.extend(children); + Ok(FormatTreeNode::with_children( + "Exchange".to_string(), + node_children, + )) + } + + fn children(&self) -> Box + '_> { + Box::new(std::iter::once(&self.input)) + } + + fn children_mut(&mut self) -> Box + '_> { + Box::new(std::iter::once(&mut self.input)) + } + + #[recursive::recursive] + fn try_find_single_data_source(&self) -> Option<&DataSourcePlan> { + self.input.try_find_single_data_source() + } + + fn is_distributed_plan(&self) -> bool { + true + } + + fn display_in_profile(&self) -> bool { + false + } + + fn derive(&self, mut children: Vec) -> PhysicalPlan { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } +} + +impl PhysicalPlanBuilder { + pub async fn build_exchange( + &mut self, + s_expr: &SExpr, + exchange: &databend_common_sql::plans::Exchange, + mut required: ColumnSet, + ) -> Result { + // 1. Prune unused Columns. + if let databend_common_sql::plans::Exchange::Hash(exprs) = exchange { + for expr in exprs { + required.extend(expr.used_columns()); + } + } + + // 2. Build physical plan. + let input = self.build(s_expr.child(0)?, required).await?; + let input_schema = input.output_schema()?; + let mut keys = vec![]; + let mut allow_adjust_parallelism = true; + let kind = match exchange { + databend_common_sql::plans::Exchange::Hash(scalars) => { + for scalar in scalars { + let expr = scalar + .type_check(input_schema.as_ref())? + .project_column_ref(|index| { + input_schema.index_of(&index.to_string()).unwrap() + }); + let (expr, _) = ConstantFolder::fold(&expr, &self.func_ctx, &BUILTIN_FUNCTIONS); + keys.push(expr.as_remote_expr()); + } + FragmentKind::Normal + } + databend_common_sql::plans::Exchange::Broadcast => FragmentKind::Expansive, + databend_common_sql::plans::Exchange::Merge => FragmentKind::Merge, + databend_common_sql::plans::Exchange::MergeSort => { + allow_adjust_parallelism = false; + FragmentKind::Merge + } + }; + Ok(Box::new(Exchange { + input, + kind, + keys, + allow_adjust_parallelism, + ignore_exchange: false, + meta: PhysicalPlanMeta::new("Exchange"), + })) + } +} diff --git a/src/query/service/src/physical_plans/physical_exchange_sink.rs b/src/query/service/src/physical_plans/physical_exchange_sink.rs new file mode 100644 index 0000000000000..38236e543825a --- /dev/null +++ b/src/query/service/src/physical_plans/physical_exchange_sink.rs @@ -0,0 +1,120 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::any::Any; + +use databend_common_ast::ast::FormatTreeNode; +use databend_common_catalog::plan::DataSourcePlan; +use databend_common_exception::Result; +use databend_common_expression::DataSchemaRef; +use databend_common_expression::RemoteExpr; +use databend_common_sql::executor::physical_plans::FragmentKind; + +use crate::physical_plans::format::format_output_columns; +use crate::physical_plans::format::FormatContext; +use crate::physical_plans::physical_plan::IPhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlanMeta; +use crate::pipelines::PipelineBuilder; + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct ExchangeSink { + pub meta: PhysicalPlanMeta, + pub input: PhysicalPlan, + // Input schema of exchanged data + pub schema: DataSchemaRef, + pub kind: FragmentKind, + pub keys: Vec, + + // Fragment ID of sink fragment + pub destination_fragment_id: usize, + + // Addresses of destination nodes + pub query_id: String, + pub ignore_exchange: bool, + pub allow_adjust_parallelism: bool, +} + +#[typetag::serde] +impl IPhysicalPlan for ExchangeSink { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn output_schema(&self) -> Result { + Ok(self.schema.clone()) + } + + fn children<'a>(&'a self) -> Box + 'a> { + Box::new(std::iter::once(&self.input)) + } + + fn children_mut<'a>(&'a mut self) -> Box + 'a> { + Box::new(std::iter::once(&mut self.input)) + } + + fn to_format_node( + &self, + ctx: &mut FormatContext<'_>, + children: Vec>, + ) -> Result> { + let mut node_children = vec![FormatTreeNode::new(format!( + "output columns: [{}]", + format_output_columns(self.output_schema()?, ctx.metadata, true) + ))]; + + node_children.push(FormatTreeNode::new(format!( + "destination fragment: [{}]", + self.destination_fragment_id + ))); + + node_children.extend(children); + Ok(FormatTreeNode::with_children( + "ExchangeSink".to_string(), + node_children, + )) + } + + #[recursive::recursive] + fn try_find_single_data_source(&self) -> Option<&DataSourcePlan> { + self.input.try_find_single_data_source() + } + + fn is_distributed_plan(&self) -> bool { + true + } + + fn display_in_profile(&self) -> bool { + false + } + + fn derive(&self, mut children: Vec) -> PhysicalPlan { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + // ExchangeSink will be appended by `ExchangeManager::execute_pipeline` + self.input.build_pipeline(builder) + } +} diff --git a/src/query/service/src/physical_plans/physical_exchange_source.rs b/src/query/service/src/physical_plans/physical_exchange_source.rs new file mode 100644 index 0000000000000..1c495bdf2ceee --- /dev/null +++ b/src/query/service/src/physical_plans/physical_exchange_source.rs @@ -0,0 +1,115 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::any::Any; + +use databend_common_ast::ast::FormatTreeNode; +use databend_common_exception::Result; +use databend_common_expression::DataSchemaRef; +use databend_common_pipeline_core::PlanScope; + +use crate::physical_plans::format::format_output_columns; +use crate::physical_plans::format::FormatContext; +use crate::physical_plans::physical_plan::IPhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlanMeta; +use crate::pipelines::PipelineBuilder; + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct ExchangeSource { + pub meta: PhysicalPlanMeta, + + // Output schema of exchanged data + pub schema: DataSchemaRef, + + // Fragment ID of source fragment + pub source_fragment_id: usize, + pub query_id: String, +} + +#[typetag::serde] +impl IPhysicalPlan for ExchangeSource { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn output_schema(&self) -> Result { + Ok(self.schema.clone()) + } + + fn to_format_node( + &self, + ctx: &mut FormatContext<'_>, + children: Vec>, + ) -> Result> { + let mut node_children = vec![FormatTreeNode::new(format!( + "output columns: [{}]", + format_output_columns(self.output_schema()?, ctx.metadata, true) + ))]; + + node_children.push(FormatTreeNode::new(format!( + "source fragment: [{}]", + self.source_fragment_id + ))); + + node_children.extend(children); + Ok(FormatTreeNode::with_children( + "ExchangeSource".to_string(), + node_children, + )) + } + + fn is_distributed_plan(&self) -> bool { + true + } + + fn display_in_profile(&self) -> bool { + false + } + + fn derive(&self, children: Vec) -> PhysicalPlan { + assert!(children.is_empty()); + Box::new(self.clone()) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + let exchange_manager = builder.ctx.get_exchange_manager(); + let build_res = exchange_manager.get_fragment_source( + &self.query_id, + self.source_fragment_id, + builder.exchange_injector.clone(), + )?; + + let plan_scope = PlanScope::get_plan_scope(); + let build_pipeline = build_res.main_pipeline.finalize(plan_scope); + + // add sharing data + builder.join_state = build_res.builder_data.input_join_state; + builder.merge_into_probe_data_fields = build_res.builder_data.input_probe_schema; + + // Merge pipeline + assert_eq!(builder.main_pipeline.output_len(), 0); + let sinks = builder.main_pipeline.merge(build_pipeline)?; + builder.main_pipeline.extend_sinks(sinks); + builder.pipelines.extend(build_res.sources_pipelines); + Ok(()) + } +} diff --git a/src/query/service/src/physical_plans/physical_expression_scan.rs b/src/query/service/src/physical_plans/physical_expression_scan.rs new file mode 100644 index 0000000000000..ca5c8e21fce5d --- /dev/null +++ b/src/query/service/src/physical_plans/physical_expression_scan.rs @@ -0,0 +1,169 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::any::Any; + +use databend_common_ast::ast::FormatTreeNode; +use databend_common_exception::Result; +use databend_common_expression::ConstantFolder; +use databend_common_expression::DataSchemaRef; +use databend_common_expression::RemoteExpr; +use databend_common_functions::BUILTIN_FUNCTIONS; +use databend_common_pipeline_core::processors::ProcessorPtr; +use databend_common_sql::optimizer::ir::SExpr; +use databend_common_sql::ColumnSet; +use databend_common_sql::TypeCheck; +use itertools::Itertools; + +use crate::physical_plans::format::format_output_columns; +use crate::physical_plans::format::FormatContext; +use crate::physical_plans::physical_plan::IPhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlanMeta; +use crate::physical_plans::PhysicalPlanBuilder; +use crate::pipelines::processors::transforms::TransformExpressionScan; +use crate::pipelines::PipelineBuilder; + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct ExpressionScan { + pub meta: PhysicalPlanMeta, + pub values: Vec>, + pub input: PhysicalPlan, + pub output_schema: DataSchemaRef, +} + +#[typetag::serde] +impl IPhysicalPlan for ExpressionScan { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn output_schema(&self) -> Result { + Ok(self.output_schema.clone()) + } + + fn children<'a>(&'a self) -> Box + 'a> { + Box::new(std::iter::once(&self.input)) + } + + fn children_mut<'a>(&'a mut self) -> Box + 'a> { + Box::new(std::iter::once(&mut self.input)) + } + + fn to_format_node( + &self, + ctx: &mut FormatContext<'_>, + children: Vec>, + ) -> Result> { + let mut node_children = Vec::with_capacity(self.values.len() + 1); + node_children.push(FormatTreeNode::new(format!( + "output columns: [{}]", + format_output_columns(self.output_schema()?, ctx.metadata, true) + ))); + + for (i, value) in self.values.iter().enumerate() { + let column = value + .iter() + .map(|val| val.as_expr(&BUILTIN_FUNCTIONS).sql_display()) + .join(", "); + node_children.push(FormatTreeNode::new(format!("column {}: [{}]", i, column))); + } + + node_children.extend(children); + + Ok(FormatTreeNode::with_children( + "ExpressionScan".to_string(), + node_children, + )) + } + + fn derive(&self, mut children: Vec) -> PhysicalPlan { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + self.input.build_pipeline(builder)?; + + let values = self + .values + .iter() + .map(|row| { + row.iter() + .map(|scalar| scalar.as_expr(&BUILTIN_FUNCTIONS)) + .collect::>() + }) + .collect::>(); + + let fun_ctx = builder.func_ctx.clone(); + + builder.main_pipeline.add_transform(|input, output| { + Ok(ProcessorPtr::create(TransformExpressionScan::create( + input, + output, + values.clone(), + fun_ctx.clone(), + ))) + })?; + + Ok(()) + } +} + +impl PhysicalPlanBuilder { + pub async fn build_expression_scan( + &mut self, + s_expr: &SExpr, + scan: &databend_common_sql::plans::ExpressionScan, + required: ColumnSet, + ) -> Result { + let input = self.build(s_expr.child(0)?, required).await?; + let input_schema = input.output_schema()?; + + let values = scan + .values + .iter() + .map(|row| { + row.iter() + .map(|scalar| { + let expr = scalar + .type_check(input_schema.as_ref())? + .project_column_ref(|index| { + input_schema.index_of(&index.to_string()).unwrap() + }); + let (expr, _) = + ConstantFolder::fold(&expr, &self.func_ctx, &BUILTIN_FUNCTIONS); + Ok(expr.as_remote_expr()) + }) + .collect::>>() + }) + .collect::>>()?; + + Ok(Box::new(ExpressionScan { + values, + input, + output_schema: scan.schema.clone(), + meta: PhysicalPlanMeta::new("ExpressionScan"), + })) + } +} diff --git a/src/query/service/src/physical_plans/physical_filter.rs b/src/query/service/src/physical_plans/physical_filter.rs new file mode 100644 index 0000000000000..81cceb360bee1 --- /dev/null +++ b/src/query/service/src/physical_plans/physical_filter.rs @@ -0,0 +1,204 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::any::Any; +use std::collections::HashMap; + +use databend_common_ast::ast::FormatTreeNode; +use databend_common_catalog::plan::DataSourcePlan; +use databend_common_exception::Result; +use databend_common_expression::ConstantFolder; +use databend_common_expression::DataSchemaRef; +use databend_common_expression::DataSchemaRefExt; +use databend_common_expression::RemoteExpr; +use databend_common_functions::BUILTIN_FUNCTIONS; +use databend_common_sql::executor::cast_expr_to_non_null_boolean; +use databend_common_sql::optimizer::ir::SExpr; +use databend_common_sql::ColumnSet; +use databend_common_sql::TypeCheck; +use itertools::Itertools; + +use crate::physical_plans::explain::PlanStatsInfo; +use crate::physical_plans::format::format_output_columns; +use crate::physical_plans::format::plan_stats_info_to_format_tree; +use crate::physical_plans::format::FormatContext; +use crate::physical_plans::physical_plan::IPhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlanMeta; +use crate::physical_plans::PhysicalPlanBuilder; +use crate::pipelines::PipelineBuilder; + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct Filter { + meta: PhysicalPlanMeta, + pub projections: ColumnSet, + pub input: PhysicalPlan, + // Assumption: expression's data type must be `DataType::Boolean`. + pub predicates: Vec, + + // Only used for explain + pub stat_info: Option, +} + +#[typetag::serde] +impl IPhysicalPlan for Filter { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn output_schema(&self) -> Result { + let input_schema = self.input.output_schema()?; + let mut fields = Vec::with_capacity(self.projections.len()); + for (i, field) in input_schema.fields().iter().enumerate() { + if self.projections.contains(&i) { + fields.push(field.clone()); + } + } + Ok(DataSchemaRefExt::create(fields)) + } + + fn children<'a>(&'a self) -> Box + 'a> { + Box::new(std::iter::once(&self.input)) + } + + fn children_mut<'a>(&'a mut self) -> Box + 'a> { + Box::new(std::iter::once(&mut self.input)) + } + + fn to_format_node( + &self, + ctx: &mut FormatContext<'_>, + children: Vec>, + ) -> Result> { + let filter = self + .predicates + .iter() + .map(|pred| pred.as_expr(&BUILTIN_FUNCTIONS).sql_display()) + .join(", "); + + let mut node_children = vec![ + FormatTreeNode::new(format!( + "output columns: [{}]", + format_output_columns(self.output_schema()?, ctx.metadata, true) + )), + FormatTreeNode::new(format!("filters: [{filter}]")), + ]; + + if let Some(info) = &self.stat_info { + node_children.extend(plan_stats_info_to_format_tree(info)); + } + + node_children.extend(children); + Ok(FormatTreeNode::with_children( + "Filter".to_string(), + node_children, + )) + } + + #[recursive::recursive] + fn try_find_single_data_source(&self) -> Option<&DataSourcePlan> { + self.input.try_find_single_data_source() + } + + fn get_desc(&self) -> Result { + Ok(match self.predicates.is_empty() { + true => String::new(), + false => self.predicates[0].as_expr(&BUILTIN_FUNCTIONS).sql_display(), + }) + } + + fn get_labels(&self) -> Result>> { + Ok(HashMap::from([( + String::from("Filter condition"), + self.predicates + .iter() + .map(|x| x.as_expr(&BUILTIN_FUNCTIONS).sql_display()) + .collect(), + )])) + } + + fn derive(&self, mut children: Vec) -> PhysicalPlan { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + self.input.build_pipeline(builder)?; + + builder.main_pipeline.add_transform( + builder.filter_transform_builder(&self.predicates, self.projections.clone())?, + ) + } +} + +impl PhysicalPlanBuilder { + pub async fn build_filter( + &mut self, + s_expr: &SExpr, + filter: &databend_common_sql::plans::Filter, + mut required: ColumnSet, + stat_info: PlanStatsInfo, + ) -> Result { + // 1. Prune unused Columns. + let used = filter.predicates.iter().fold(required.clone(), |acc, v| { + acc.union(&v.used_columns()).cloned().collect() + }); + + // 2. Build physical plan. + let input = self.build(s_expr.child(0)?, used).await?; + required = required + .union(self.metadata.read().get_retained_column()) + .cloned() + .collect(); + let column_projections = required.clone().into_iter().collect::>(); + let input_schema = input.output_schema()?; + let mut projections = ColumnSet::new(); + for column in column_projections.iter() { + if let Some((index, _)) = input_schema.column_with_name(&column.to_string()) { + projections.insert(index); + } + } + + Ok(Box::new(Filter { + meta: PhysicalPlanMeta::new("Filter"), + projections, + input, + predicates: filter + .predicates + .iter() + .map(|scalar| { + let expr = scalar + .type_check(input_schema.as_ref())? + .project_column_ref(|index| { + input_schema.index_of(&index.to_string()).unwrap() + }); + let expr = cast_expr_to_non_null_boolean(expr)?; + let (expr, _) = ConstantFolder::fold(&expr, &self.func_ctx, &BUILTIN_FUNCTIONS); + Ok(expr.as_remote_expr()) + }) + .collect::>()?, + + stat_info: Some(stat_info), + })) + } +} diff --git a/src/query/sql/src/executor/physical_plans/physical_hash_join.rs b/src/query/service/src/physical_plans/physical_hash_join.rs similarity index 69% rename from src/query/sql/src/executor/physical_plans/physical_hash_join.rs rename to src/query/service/src/physical_plans/physical_hash_join.rs index 72b233e170f32..a7438aa89808a 100644 --- a/src/query/sql/src/executor/physical_plans/physical_hash_join.rs +++ b/src/query/service/src/physical_plans/physical_hash_join.rs @@ -12,9 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::any::Any; use std::collections::HashMap; use std::collections::HashSet; +use std::sync::Arc; +use databend_common_ast::ast::FormatTreeNode; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::type_check::check_cast; @@ -26,21 +29,37 @@ use databend_common_expression::DataSchemaRef; use databend_common_expression::DataSchemaRefExt; use databend_common_expression::RemoteExpr; use databend_common_functions::BUILTIN_FUNCTIONS; +use databend_common_pipeline_core::processors::ProcessorPtr; +use databend_common_sql::optimizer::ir::SExpr; +use databend_common_sql::plans::Join; +use databend_common_sql::plans::JoinType; +use databend_common_sql::ColumnEntry; +use databend_common_sql::ColumnSet; +use databend_common_sql::IndexType; +use databend_common_sql::ScalarExpr; +use databend_common_sql::TypeCheck; +use itertools::Itertools; +use tokio::sync::Barrier; use super::physical_join_filter::PhysicalRuntimeFilters; use super::JoinRuntimeFilter; -use crate::executor::explain::PlanStatsInfo; -use crate::executor::physical_plans::Exchange; -use crate::executor::PhysicalPlan; -use crate::executor::PhysicalPlanBuilder; -use crate::optimizer::ir::SExpr; -use crate::plans::Join; -use crate::plans::JoinType; -use crate::ColumnEntry; -use crate::ColumnSet; -use crate::IndexType; -use crate::ScalarExpr; -use crate::TypeCheck; +use crate::physical_plans::explain::PlanStatsInfo; +use crate::physical_plans::format::format_output_columns; +use crate::physical_plans::format::plan_stats_info_to_format_tree; +use crate::physical_plans::format::FormatContext; +use crate::physical_plans::physical_plan::IPhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlanDynExt; +use crate::physical_plans::physical_plan::PhysicalPlanMeta; +use crate::physical_plans::Exchange; +use crate::physical_plans::PhysicalPlanBuilder; +use crate::pipelines::processors::transforms::HashJoinProbeState; +use crate::pipelines::processors::transforms::TransformHashJoinBuild; +use crate::pipelines::processors::transforms::TransformHashJoinProbe; +use crate::pipelines::processors::HashJoinBuildState; +use crate::pipelines::processors::HashJoinDesc; +use crate::pipelines::processors::HashJoinState; +use crate::pipelines::PipelineBuilder; // Type aliases to simplify complex return types type JoinConditionsResult = ( @@ -66,8 +85,7 @@ type MergedFieldsResult = ( #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct HashJoin { - // A unique id of operator in a `PhysicalPlan` tree, only used for display. - pub plan_id: u32, + pub meta: PhysicalPlanMeta, // After building the probe key and build key, we apply probe_projections to probe_datablock // and build_projections to build_datablock, which can help us reduce memory usage and calls // of expensive functions (take_compacted_indices and gather), after processing other_conditions, @@ -76,8 +94,8 @@ pub struct HashJoin { pub probe_projections: ColumnSet, pub build_projections: ColumnSet, - pub build: Box, - pub probe: Box, + pub build: PhysicalPlan, + pub probe: PhysicalPlan, pub build_keys: Vec, pub probe_keys: Vec, pub is_null_equal: Vec, @@ -108,22 +126,338 @@ pub struct HashJoin { pub broadcast_id: Option, } -impl HashJoin { - pub fn output_schema(&self) -> Result { +#[typetag::serde] +impl IPhysicalPlan for HashJoin { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn output_schema(&self) -> Result { Ok(self.output_schema.clone()) } + + fn children<'a>(&'a self) -> Box + 'a> { + Box::new(std::iter::once(&self.probe).chain(std::iter::once(&self.build))) + } + + fn children_mut<'a>(&'a mut self) -> Box + 'a> { + Box::new(std::iter::once(&mut self.probe).chain(std::iter::once(&mut self.build))) + } + + fn to_format_node( + &self, + ctx: &mut FormatContext<'_>, + mut children: Vec>, + ) -> Result> { + for rf in self.runtime_filter.filters.iter() { + ctx.scan_id_to_runtime_filters + .entry(rf.scan_id) + .or_default() + .push(rf.clone()); + } + + let build_keys = self + .build_keys + .iter() + .map(|scalar| scalar.as_expr(&BUILTIN_FUNCTIONS).sql_display()) + .collect::>() + .join(", "); + + let probe_keys = self + .probe_keys + .iter() + .map(|scalar| scalar.as_expr(&BUILTIN_FUNCTIONS).sql_display()) + .collect::>() + .join(", "); + + let is_null_equal = self.is_null_equal.iter().map(|b| format!("{b}")).join(", "); + + let filters = self + .non_equi_conditions + .iter() + .map(|filter| filter.as_expr(&BUILTIN_FUNCTIONS).sql_display()) + .collect::>() + .join(", "); + + assert_eq!(children.len(), 2); + children[0].payload = format!("{}(Build)", children[0].payload); + children[1].payload = format!("{}(Probe)", children[1].payload); + + let mut build_runtime_filters = vec![]; + for rf in self.runtime_filter.filters.iter() { + let mut s = format!( + "filter id:{}, build key:{}, probe key:{}, filter type:", + rf.id, + rf.build_key.as_expr(&BUILTIN_FUNCTIONS).sql_display(), + rf.probe_key.as_expr(&BUILTIN_FUNCTIONS).sql_display(), + ); + if rf.enable_bloom_runtime_filter { + s += "bloom,"; + } + if rf.enable_inlist_runtime_filter { + s += "inlist,"; + } + if rf.enable_min_max_runtime_filter { + s += "min_max,"; + } + s = s.trim_end_matches(',').to_string(); + build_runtime_filters.push(FormatTreeNode::new(s)); + } + + let mut node_children = vec![ + FormatTreeNode::new(format!( + "output columns: [{}]", + format_output_columns(self.output_schema()?, ctx.metadata, true) + )), + FormatTreeNode::new(format!("join type: {}", self.join_type)), + FormatTreeNode::new(format!("build keys: [{build_keys}]")), + FormatTreeNode::new(format!("probe keys: [{probe_keys}]")), + FormatTreeNode::new(format!("keys is null equal: [{is_null_equal}]")), + FormatTreeNode::new(format!("filters: [{filters}]")), + ]; + + if !build_runtime_filters.is_empty() { + if self.broadcast_id.is_some() { + node_children.push(FormatTreeNode::with_children( + format!("build join filters(distributed):"), + build_runtime_filters, + )); + } else { + node_children.push(FormatTreeNode::with_children( + format!("build join filters:"), + build_runtime_filters, + )); + } + } + + if let Some((cache_index, column_map)) = &self.build_side_cache_info { + let mut column_indexes = column_map.keys().collect::>(); + column_indexes.sort(); + node_children.push(FormatTreeNode::new(format!("cache index: {}", cache_index))); + node_children.push(FormatTreeNode::new(format!( + "cache columns: {:?}", + column_indexes + ))); + } + + if let Some(info) = &self.stat_info { + let items = plan_stats_info_to_format_tree(info); + node_children.extend(items); + } + + node_children.extend(children); + + Ok(FormatTreeNode::with_children( + "HashJoin".to_string(), + node_children, + )) + } + + fn get_desc(&self) -> Result { + let mut conditions = self + .build_keys + .iter() + .zip(self.probe_keys.iter()) + .map(|(l, r)| { + format!( + "({} = {})", + l.as_expr(&BUILTIN_FUNCTIONS).sql_display(), + r.as_expr(&BUILTIN_FUNCTIONS).sql_display() + ) + }) + .collect::>(); + + conditions.extend( + self.non_equi_conditions + .iter() + .map(|x| x.as_expr(&BUILTIN_FUNCTIONS).sql_display()), + ); + + Ok(conditions.join(" AND ")) + } + + fn get_labels(&self) -> Result>> { + let mut labels = HashMap::with_capacity(4); + labels.insert(String::from("Join Type"), vec![self.join_type.to_string()]); + + if !self.build_keys.is_empty() { + labels.insert( + String::from("Join Build Side Keys"), + self.build_keys + .iter() + .map(|x| x.as_expr(&BUILTIN_FUNCTIONS).sql_display()) + .collect(), + ); + } + + if !self.probe_keys.is_empty() { + labels.insert( + String::from("Join Probe Side Keys"), + self.probe_keys + .iter() + .map(|x| x.as_expr(&BUILTIN_FUNCTIONS).sql_display()) + .collect(), + ); + } + + if !self.non_equi_conditions.is_empty() { + labels.insert( + String::from("Join Conditions"), + self.non_equi_conditions + .iter() + .map(|x| x.as_expr(&BUILTIN_FUNCTIONS).sql_display()) + .collect(), + ); + } + + Ok(labels) + } + + fn derive(&self, mut children: Vec) -> PhysicalPlan { + let mut new_hash_join = self.clone(); + assert_eq!(children.len(), 2); + new_hash_join.build = children.pop().unwrap(); + new_hash_join.probe = children.pop().unwrap(); + Box::new(new_hash_join) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + // Create the join state with optimization flags + let state = self.build_state(builder)?; + + if let Some((build_cache_index, _)) = self.build_side_cache_info { + builder + .hash_join_states + .insert(build_cache_index, state.clone()); + } + + // Build both phases of the Hash Join + self.build_right(builder, state.clone())?; + self.build_left(builder, state.clone())?; + + // In the case of spilling, we need to share state among multiple threads + // Quickly fetch all data from this round to quickly start the next round + builder + .main_pipeline + .resize(builder.main_pipeline.output_len(), true) + } +} + +impl HashJoin { + fn build_state(&self, builder: &mut PipelineBuilder) -> Result> { + let (enable_optimization, is_distributed) = builder.merge_into_get_optimization_flag(self); + HashJoinState::try_create( + builder.ctx.clone(), + self.build.output_schema()?, + &self.build_projections, + HashJoinDesc::create(self)?, + &self.probe_to_build, + is_distributed, + enable_optimization, + self.build_side_cache_info.clone(), + ) + } + + fn build_left(&self, builder: &mut PipelineBuilder, state: Arc) -> Result<()> { + self.probe.build_pipeline(builder)?; + + let max_block_size = builder.settings.get_max_block_size()? as usize; + let barrier = Barrier::new(builder.main_pipeline.output_len()); + let probe_state = Arc::new(HashJoinProbeState::create( + builder.ctx.clone(), + builder.func_ctx.clone(), + state.clone(), + &self.probe_projections, + &self.build_projections, + &self.probe_keys, + self.probe.output_schema()?, + &self.join_type, + builder.main_pipeline.output_len(), + barrier, + )?); + + builder.main_pipeline.add_transform(|input, output| { + Ok(ProcessorPtr::create(TransformHashJoinProbe::create( + input, + output, + self.projections.clone(), + probe_state.clone(), + max_block_size, + builder.func_ctx.clone(), + &self.join_type, + !self.non_equi_conditions.is_empty(), + )?)) + })?; + + // For merge-into operations that need to hold the hash table + if self.need_hold_hash_table { + // Extract projected fields from probe schema + let mut projected_fields = vec![]; + for (i, field) in probe_state.probe_schema.fields().iter().enumerate() { + if probe_state.probe_projections.contains(&i) { + projected_fields.push(field.clone()); + } + } + builder.merge_into_probe_data_fields = Some(projected_fields); + } + + Ok(()) + } + + fn build_right(&self, builder: &mut PipelineBuilder, state: Arc) -> Result<()> { + let right_builder = builder.create_sub_pipeline_builder(); + let mut build_res = right_builder.finalize(&self.build)?; + + assert!(build_res.main_pipeline.is_pulling_pipeline()?); + let output_len = build_res.main_pipeline.output_len(); + let build_state = HashJoinBuildState::try_create( + builder.ctx.clone(), + builder.func_ctx.clone(), + &self.build_keys, + &self.build_projections, + state.clone(), + output_len, + self.broadcast_id, + )?; + build_state.add_runtime_filter_ready(); + + let create_sink_processor = |input| { + Ok(ProcessorPtr::create(TransformHashJoinBuild::try_create( + input, + build_state.clone(), + )?)) + }; + // For distributed merge-into when source as build side + if self.need_hold_hash_table { + builder.join_state = Some(build_state.clone()) + } + build_res.main_pipeline.add_sink(create_sink_processor)?; + + builder + .pipelines + .push(build_res.main_pipeline.finalize(None)); + builder.pipelines.extend(build_res.sources_pipelines); + Ok(()) + } } impl PhysicalPlanBuilder { /// Builds the physical plans for both sides of the join - pub(crate) async fn build_join_sides( + pub async fn build_join_sides( &mut self, s_expr: &SExpr, left_required: ColumnSet, right_required: ColumnSet, - ) -> Result<(Box, Box)> { - let probe_side = Box::new(self.build(s_expr.child(0)?, left_required).await?); - let build_side = Box::new(self.build(s_expr.child(1)?, right_required).await?); + ) -> Result<(PhysicalPlan, PhysicalPlan)> { + let probe_side = self.build(s_expr.child(0)?, left_required).await?; + let build_side = self.build(s_expr.child(1)?, right_required).await?; Ok((probe_side, build_side)) } @@ -152,7 +486,7 @@ impl PhysicalPlanBuilder { /// /// # Returns /// * `Result` - The prepared schema for the build side - pub(crate) fn prepare_build_schema( + pub fn prepare_build_schema( &self, join_type: &JoinType, build_side: &PhysicalPlan, @@ -188,7 +522,7 @@ impl PhysicalPlanBuilder { /// /// # Returns /// * `Result` - The prepared schema for the probe side - pub(crate) fn prepare_probe_schema( + pub fn prepare_probe_schema( &self, join_type: &JoinType, probe_side: &PhysicalPlan, @@ -219,51 +553,53 @@ impl PhysicalPlanBuilder { /// * `build_side` - The build side physical plan fn unify_keys( &self, - probe_side: &mut Box, - build_side: &mut Box, + probe_side: &mut PhysicalPlan, + build_side: &mut PhysicalPlan, ) -> Result<()> { // Unify the data types of the left and right exchange keys - if let ( - PhysicalPlan::Exchange(Exchange { - keys: probe_keys, .. - }), - PhysicalPlan::Exchange(Exchange { - keys: build_keys, .. - }), - ) = (probe_side.as_mut(), build_side.as_mut()) + let Some(probe_exchange) = probe_side.downcast_mut_ref::() else { + return Ok(()); + }; + + let Some(build_exchange) = build_side.downcast_mut_ref::() else { + return Ok(()); + }; + + let cast_rules = &BUILTIN_FUNCTIONS.get_auto_cast_rules("eq"); + for (probe_key, build_key) in probe_exchange + .keys + .iter_mut() + .zip(build_exchange.keys.iter_mut()) { - let cast_rules = &BUILTIN_FUNCTIONS.get_auto_cast_rules("eq"); - for (probe_key, build_key) in probe_keys.iter_mut().zip(build_keys.iter_mut()) { - let probe_expr = probe_key.as_expr(&BUILTIN_FUNCTIONS); - let build_expr = build_key.as_expr(&BUILTIN_FUNCTIONS); - let common_ty = common_super_type( - probe_expr.data_type().clone(), - build_expr.data_type().clone(), - cast_rules, - ) - .ok_or_else(|| { - ErrorCode::IllegalDataType(format!( - "Cannot find common type for probe key {:?} and build key {:?}", - &probe_expr, &build_expr - )) - })?; - *probe_key = check_cast( - probe_expr.span(), - false, - probe_expr, - &common_ty, - &BUILTIN_FUNCTIONS, - )? - .as_remote_expr(); - *build_key = check_cast( - build_expr.span(), - false, - build_expr, - &common_ty, - &BUILTIN_FUNCTIONS, - )? - .as_remote_expr(); - } + let probe_expr = probe_key.as_expr(&BUILTIN_FUNCTIONS); + let build_expr = build_key.as_expr(&BUILTIN_FUNCTIONS); + let common_ty = common_super_type( + probe_expr.data_type().clone(), + build_expr.data_type().clone(), + cast_rules, + ) + .ok_or_else(|| { + ErrorCode::IllegalDataType(format!( + "Cannot find common type for probe key {:?} and build key {:?}", + &probe_expr, &build_expr + )) + })?; + *probe_key = check_cast( + probe_expr.span(), + false, + probe_expr, + &common_ty, + &BUILTIN_FUNCTIONS, + )? + .as_remote_expr(); + *build_key = check_cast( + build_expr.span(), + false, + build_expr, + &common_ty, + &BUILTIN_FUNCTIONS, + )? + .as_remote_expr(); } Ok(()) @@ -790,8 +1126,8 @@ impl PhysicalPlanBuilder { &self, s_expr: &SExpr, join: &Join, - probe_side: Box, - build_side: Box, + probe_side: PhysicalPlan, + build_side: PhysicalPlan, projections: ColumnSet, probe_projections: ColumnSet, build_projections: ColumnSet, @@ -808,14 +1144,13 @@ impl PhysicalPlanBuilder { let build_side_data_distribution = s_expr.build_side_child().get_data_distribution()?; let broadcast_id = if build_side_data_distribution .as_ref() - .is_some_and(|e| matches!(e, crate::plans::Exchange::Hash(_))) + .is_some_and(|e| matches!(e, databend_common_sql::plans::Exchange::Hash(_))) { Some(self.ctx.get_next_broadcast_id()) } else { None }; - Ok(PhysicalPlan::HashJoin(HashJoin { - plan_id: 0, + Ok(Box::new(HashJoin { projections, build_projections, probe_projections, @@ -827,6 +1162,7 @@ impl PhysicalPlanBuilder { is_null_equal, non_equi_conditions, marker_index: join.marker_index, + meta: PhysicalPlanMeta::new("HashJoin"), from_correlated_subquery: join.from_correlated_subquery, probe_to_build, output_schema, diff --git a/src/query/sql/src/executor/physical_plans/physical_join.rs b/src/query/service/src/physical_plans/physical_join.rs similarity index 91% rename from src/query/sql/src/executor/physical_plans/physical_join.rs rename to src/query/service/src/physical_plans/physical_join.rs index 1b09ecc6a2eed..85e0830565602 100644 --- a/src/query/sql/src/executor/physical_plans/physical_join.rs +++ b/src/query/service/src/physical_plans/physical_join.rs @@ -13,18 +13,18 @@ // limitations under the License. use databend_common_exception::Result; +use databend_common_sql::binder::JoinPredicate; +use databend_common_sql::optimizer::ir::RelExpr; +use databend_common_sql::optimizer::ir::RelationalProperty; +use databend_common_sql::optimizer::ir::SExpr; +use databend_common_sql::plans::Join; +use databend_common_sql::plans::JoinType; +use databend_common_sql::ColumnSet; +use databend_common_sql::ScalarExpr; -use crate::binder::JoinPredicate; -use crate::executor::explain::PlanStatsInfo; -use crate::executor::PhysicalPlan; -use crate::executor::PhysicalPlanBuilder; -use crate::optimizer::ir::RelExpr; -use crate::optimizer::ir::RelationalProperty; -use crate::optimizer::ir::SExpr; -use crate::plans::Join; -use crate::plans::JoinType; -use crate::ColumnSet; -use crate::ScalarExpr; +use crate::physical_plans::explain::PlanStatsInfo; +use crate::physical_plans::physical_plan::PhysicalPlan; +use crate::physical_plans::PhysicalPlanBuilder; pub enum PhysicalJoinType { Hash, @@ -125,10 +125,10 @@ fn check_condition( } impl PhysicalPlanBuilder { - pub(crate) async fn build_join( + pub async fn build_join( &mut self, s_expr: &SExpr, - join: &crate::plans::Join, + join: &databend_common_sql::plans::Join, required: ColumnSet, stat_info: PlanStatsInfo, ) -> Result { diff --git a/src/query/sql/src/executor/physical_plans/physical_join_filter.rs b/src/query/service/src/physical_plans/physical_join_filter.rs similarity index 96% rename from src/query/sql/src/executor/physical_plans/physical_join_filter.rs rename to src/query/service/src/physical_plans/physical_join_filter.rs index 99fa05d95c102..ec1ed306e0c3b 100644 --- a/src/query/sql/src/executor/physical_plans/physical_join_filter.rs +++ b/src/query/service/src/physical_plans/physical_join_filter.rs @@ -20,16 +20,15 @@ use databend_common_exception::Result; use databend_common_expression::types::DataType; use databend_common_expression::RemoteExpr; use databend_common_functions::BUILTIN_FUNCTIONS; +use databend_common_sql::optimizer::ir::RelExpr; +use databend_common_sql::optimizer::ir::SExpr; +use databend_common_sql::plans::Exchange; +use databend_common_sql::plans::Join; +use databend_common_sql::plans::JoinType; +use databend_common_sql::IndexType; +use databend_common_sql::MetadataRef; use databend_storages_common_table_meta::table::get_change_type; -use crate::optimizer::ir::RelExpr; -use crate::optimizer::ir::SExpr; -use crate::plans::Exchange; -use crate::plans::Join; -use crate::plans::JoinType; -use crate::IndexType; -use crate::MetadataRef; - #[derive(Clone, Debug, serde::Serialize, serde::Deserialize, Default)] pub struct PhysicalRuntimeFilters { pub filters: Vec, diff --git a/src/query/sql/src/executor/physical_plans/physical_limit.rs b/src/query/service/src/physical_plans/physical_limit.rs similarity index 50% rename from src/query/sql/src/executor/physical_plans/physical_limit.rs rename to src/query/service/src/physical_plans/physical_limit.rs index b0c3e64ecd713..60fc00a221ca5 100644 --- a/src/query/sql/src/executor/physical_plans/physical_limit.rs +++ b/src/query/service/src/physical_plans/physical_limit.rs @@ -12,25 +12,36 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::any::Any; +use std::collections::HashMap; + +use databend_common_ast::ast::FormatTreeNode; +use databend_common_catalog::plan::DataSourcePlan; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::DataField; -use databend_common_expression::DataSchemaRef; use databend_common_expression::ROW_ID_COL_NAME; +use databend_common_pipeline_core::processors::ProcessorPtr; +use databend_common_sql::optimizer::ir::SExpr; +use databend_common_sql::ColumnEntry; +use databend_common_sql::ColumnSet; -use crate::executor::explain::PlanStatsInfo; -use crate::executor::physical_plans::physical_row_fetch::RowFetch; -use crate::executor::PhysicalPlan; -use crate::executor::PhysicalPlanBuilder; -use crate::optimizer::ir::SExpr; -use crate::ColumnEntry; -use crate::ColumnSet; +use crate::physical_plans::explain::PlanStatsInfo; +use crate::physical_plans::format::format_output_columns; +use crate::physical_plans::format::plan_stats_info_to_format_tree; +use crate::physical_plans::format::FormatContext; +use crate::physical_plans::physical_plan::IPhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlanMeta; +use crate::physical_plans::physical_row_fetch::RowFetch; +use crate::physical_plans::PhysicalPlanBuilder; +use crate::pipelines::processors::transforms::TransformLimit; +use crate::pipelines::PipelineBuilder; #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct Limit { - // A unique id of operator in a `PhysicalPlan` tree, only used for display. - pub plan_id: u32, - pub input: Box, + meta: PhysicalPlanMeta, + pub input: PhysicalPlan, pub limit: Option, pub offset: usize, @@ -38,17 +49,110 @@ pub struct Limit { pub stat_info: Option, } -impl Limit { - pub fn output_schema(&self) -> Result { - self.input.output_schema() +#[typetag::serde] +impl IPhysicalPlan for Limit { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn children<'a>(&'a self) -> Box + 'a> { + Box::new(std::iter::once(&self.input)) + } + + fn children_mut<'a>(&'a mut self) -> Box + 'a> { + Box::new(std::iter::once(&mut self.input)) + } + + fn to_format_node( + &self, + ctx: &mut FormatContext<'_>, + children: Vec>, + ) -> Result> { + let mut node_children = vec![ + FormatTreeNode::new(format!( + "output columns: [{}]", + format_output_columns(self.output_schema()?, ctx.metadata, true) + )), + FormatTreeNode::new(format!( + "limit: {}", + self.limit + .map_or("NONE".to_string(), |limit| limit.to_string()) + )), + FormatTreeNode::new(format!("offset: {}", self.offset)), + ]; + + if let Some(info) = &self.stat_info { + node_children.extend(plan_stats_info_to_format_tree(info)); + } + + node_children.extend(children); + Ok(FormatTreeNode::with_children( + "Limit".to_string(), + node_children, + )) + } + + #[recursive::recursive] + fn try_find_single_data_source(&self) -> Option<&DataSourcePlan> { + self.input.try_find_single_data_source() + } + + fn get_desc(&self) -> Result { + Ok(match self.limit { + Some(limit) => format!("LIMIT {} OFFSET {}", limit, self.offset), + None => format!("OFFSET {}", self.offset), + }) + } + + fn get_labels(&self) -> Result>> { + let mut labels = HashMap::with_capacity(2); + labels.insert(String::from("Offset"), vec![self.offset.to_string()]); + + if let Some(limit) = self.limit { + labels.insert(String::from("Number of rows"), vec![limit.to_string()]); + } + + Ok(labels) + } + + fn derive(&self, mut children: Vec) -> PhysicalPlan { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + self.input.build_pipeline(builder)?; + + if self.limit.is_some() || self.offset != 0 { + builder.main_pipeline.try_resize(1)?; + return builder.main_pipeline.add_transform(|input, output| { + Ok(ProcessorPtr::create(TransformLimit::try_create( + self.limit, + self.offset, + input, + output, + )?)) + }); + } + + Ok(()) } } impl PhysicalPlanBuilder { - pub(crate) async fn build_limit( + pub async fn build_limit( &mut self, s_expr: &SExpr, - limit: &crate::plans::Limit, + limit: &databend_common_sql::plans::Limit, mut required: ColumnSet, stat_info: PlanStatsInfo, ) -> Result { @@ -66,12 +170,12 @@ impl PhysicalPlanBuilder { let input_plan = self.build(s_expr.child(0)?, required).await?; let metadata = self.metadata.read().clone(); if limit.before_exchange || metadata.lazy_columns().is_empty() { - return Ok(PhysicalPlan::Limit(Limit { - plan_id: 0, - input: Box::new(input_plan), + return Ok(Box::new(Limit { + input: input_plan, limit: limit.limit, offset: limit.offset, stat_info: Some(stat_info), + meta: PhysicalPlanMeta::new("Limit"), })); } @@ -86,12 +190,12 @@ impl PhysicalPlanBuilder { .ok_or_else(|| ErrorCode::Internal("Internal column _row_id is not found"))?; if !input_schema.has_field(&row_id_col_index.to_string()) { - return Ok(PhysicalPlan::Limit(Limit { - plan_id: 0, - input: Box::new(input_plan), + return Ok(Box::new(Limit { + input: input_plan, limit: limit.limit, offset: limit.offset, stat_info: Some(stat_info), + meta: PhysicalPlanMeta::new("Limit"), })); } @@ -109,12 +213,12 @@ impl PhysicalPlanBuilder { if limit.before_exchange || lazy_columns.is_empty() { // If there is no lazy column, we don't need to build a `RowFetch` plan. - return Ok(PhysicalPlan::Limit(Limit { - plan_id: 0, - input: Box::new(input_plan), + return Ok(Box::new(Limit { + input: input_plan, limit: limit.limit, offset: limit.offset, stat_info: Some(stat_info), + meta: PhysicalPlanMeta::new("Limit"), })); } @@ -146,15 +250,15 @@ impl PhysicalPlanBuilder { false, ); - Ok(PhysicalPlan::RowFetch(RowFetch { - plan_id: 0, - input: Box::new(PhysicalPlan::Limit(Limit { - plan_id: 0, - input: Box::new(input_plan), + Ok(Box::new(RowFetch { + meta: PhysicalPlanMeta::new("RowFetch"), + input: Box::new(Limit { + meta: PhysicalPlanMeta::new("RowFetch"), + input: input_plan, limit: limit.limit, offset: limit.offset, stat_info: Some(stat_info.clone()), - })), + }), source: Box::new(source_info), row_id_col_offset, cols_to_fetch, diff --git a/src/query/service/src/physical_plans/physical_multi_table_insert.rs b/src/query/service/src/physical_plans/physical_multi_table_insert.rs new file mode 100644 index 0000000000000..e1f9c2def38b0 --- /dev/null +++ b/src/query/service/src/physical_plans/physical_multi_table_insert.rs @@ -0,0 +1,871 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::any::Any; +use std::collections::HashMap; +use std::sync::Arc; + +use databend_common_ast::ast::FormatTreeNode; +use databend_common_catalog::catalog::CatalogManager; +use databend_common_catalog::table_context::TableContext; +use databend_common_exception::ErrorCode; +use databend_common_exception::Result; +use databend_common_expression::DataSchema; +use databend_common_expression::DataSchemaRef; +use databend_common_expression::LimitType; +use databend_common_expression::RemoteExpr; +use databend_common_expression::SortColumnDescription; +use databend_common_functions::BUILTIN_FUNCTIONS; +use databend_common_meta_app::schema::CatalogInfo; +use databend_common_meta_app::schema::TableInfo; +use databend_common_meta_app::schema::UpdateStreamMetaReq; +use databend_common_pipeline_core::processors::ProcessorPtr; +use databend_common_pipeline_core::DynTransformBuilder; +use databend_common_pipeline_sinks::AsyncSinker; +use databend_common_pipeline_transforms::TransformSortPartial; +use databend_common_sql::evaluator::CompoundBlockOperator; +use databend_common_sql::ColumnSet; +use databend_common_storages_fuse::operations::CommitMultiTableInsert; +use databend_common_storages_fuse::FuseTable; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; +use itertools::Itertools; + +use crate::physical_plans::format::FormatContext; +use crate::physical_plans::physical_plan::IPhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlanMeta; +use crate::pipelines::PipelineBuilder; + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct Duplicate { + pub meta: PhysicalPlanMeta, + pub input: PhysicalPlan, + pub n: usize, +} + +#[typetag::serde] +impl IPhysicalPlan for Duplicate { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn children<'a>(&'a self) -> Box + 'a> { + Box::new(std::iter::once(&self.input)) + } + + fn children_mut<'a>(&'a mut self) -> Box + 'a> { + Box::new(std::iter::once(&mut self.input)) + } + + fn to_format_node( + &self, + _: &mut FormatContext<'_>, + children: Vec>, + ) -> Result> { + let mut node_children = vec![FormatTreeNode::new(format!( + "Duplicate data to {} branch", + self.n + ))]; + + node_children.extend(children); + + Ok(FormatTreeNode::with_children( + "Duplicate".to_string(), + node_children, + )) + } + + fn derive(&self, mut children: Vec) -> PhysicalPlan { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + self.input.build_pipeline(builder)?; + + builder.main_pipeline.duplicate(true, self.n) + } +} + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct Shuffle { + pub meta: PhysicalPlanMeta, + pub input: PhysicalPlan, + pub strategy: ShuffleStrategy, +} + +#[typetag::serde] +impl IPhysicalPlan for Shuffle { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn children<'a>(&'a self) -> Box + 'a> { + Box::new(std::iter::once(&self.input)) + } + + fn children_mut<'a>(&'a mut self) -> Box + 'a> { + Box::new(std::iter::once(&mut self.input)) + } + + fn to_format_node( + &self, + _: &mut FormatContext<'_>, + mut children: Vec>, + ) -> Result> { + // ignore self + assert_eq!(children.len(), 1); + Ok(children.pop().unwrap()) + } + + fn display_in_profile(&self) -> bool { + false + } + + fn derive(&self, mut children: Vec) -> PhysicalPlan { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + self.input.build_pipeline(builder)?; + + builder + .main_pipeline + .reorder_inputs(self.strategy.shuffle(builder.main_pipeline.output_len())?); + Ok(()) + } +} + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub enum ShuffleStrategy { + Transpose(usize), +} + +impl ShuffleStrategy { + pub fn shuffle(&self, total: usize) -> Result> { + match self { + ShuffleStrategy::Transpose(n) => { + if total % n != 0 { + return Err(ErrorCode::Internal(format!( + "total rows {} is not divisible by n {}", + total, n + ))); + } + let mut result = vec![0; total]; + for i in 0..*n { + for j in 0..total / n { + result[i + j * n] = i * (total / n) + j; + } + } + Ok(result) + } + } + } +} + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct ChunkFilter { + pub meta: PhysicalPlanMeta, + pub input: PhysicalPlan, + pub predicates: Vec>, +} + +#[typetag::serde] +impl IPhysicalPlan for ChunkFilter { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn children<'a>(&'a self) -> Box + 'a> { + Box::new(std::iter::once(&self.input)) + } + + fn children_mut<'a>(&'a mut self) -> Box + 'a> { + Box::new(std::iter::once(&mut self.input)) + } + + fn to_format_node( + &self, + _: &mut FormatContext<'_>, + mut children: Vec>, + ) -> Result> { + if self.predicates.iter().all(|x| x.is_none()) { + assert_eq!(children.len(), 1); + return Ok(children.pop().unwrap()); + } + let mut node_children = Vec::new(); + for (i, predicate) in self.predicates.iter().enumerate() { + if let Some(predicate) = predicate { + node_children.push(FormatTreeNode::new(format!( + "branch {}: {}", + i, + predicate.as_expr(&BUILTIN_FUNCTIONS).sql_display() + ))); + } else { + node_children.push(FormatTreeNode::new(format!("branch {}: None", i))); + } + } + + node_children.extend(children); + Ok(FormatTreeNode::with_children( + "Filter".to_string(), + node_children, + )) + } + + fn derive(&self, mut children: Vec) -> PhysicalPlan { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + self.input.build_pipeline(builder)?; + + if self.predicates.iter().all(|x| x.is_none()) { + return Ok(()); + } + let mut f: Vec = Vec::with_capacity(self.predicates.len()); + let projection: ColumnSet = (0..self.input.output_schema()?.fields.len()).collect(); + for predicate in self.predicates.iter() { + if let Some(predicate) = predicate { + f.push(Box::new(builder.filter_transform_builder( + &[predicate.clone()], + projection.clone(), + )?)); + } else { + f.push(Box::new(builder.dummy_transform_builder()?)); + } + } + + builder.main_pipeline.add_transforms_by_chunk(f) + } +} + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct ChunkEvalScalar { + pub meta: PhysicalPlanMeta, + pub input: PhysicalPlan, + pub eval_scalars: Vec>, +} + +#[typetag::serde] +impl IPhysicalPlan for ChunkEvalScalar { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn children<'a>(&'a self) -> Box + 'a> { + Box::new(std::iter::once(&self.input)) + } + + fn children_mut<'a>(&'a mut self) -> Box + 'a> { + Box::new(std::iter::once(&mut self.input)) + } + + fn to_format_node( + &self, + _: &mut FormatContext<'_>, + mut children: Vec>, + ) -> Result> { + if self.eval_scalars.iter().all(|x| x.is_none()) { + assert_eq!(children.len(), 1); + return Ok(children.pop().unwrap()); + } + + let mut node_children = Vec::new(); + for (i, eval_scalar) in self.eval_scalars.iter().enumerate() { + if let Some(eval_scalar) = eval_scalar { + node_children.push(FormatTreeNode::new(format!( + "branch {}: {}", + i, + eval_scalar + .remote_exprs + .iter() + .map(|x| x.as_expr(&BUILTIN_FUNCTIONS).sql_display()) + .join(", ") + ))); + } else { + node_children.push(FormatTreeNode::new(format!("branch {}: None", i))); + } + } + + node_children.extend(children); + Ok(FormatTreeNode::with_children( + "EvalScalar".to_string(), + node_children, + )) + } + + fn derive(&self, mut children: Vec) -> PhysicalPlan { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + self.input.build_pipeline(builder)?; + + if self.eval_scalars.iter().all(|x| x.is_none()) { + return Ok(()); + } + let num_input_columns = self.input.output_schema()?.num_fields(); + let mut f: Vec = Vec::with_capacity(self.eval_scalars.len()); + for eval_scalar in self.eval_scalars.iter() { + if let Some(eval_scalar) = eval_scalar { + f.push(Box::new(builder.map_transform_builder( + num_input_columns, + eval_scalar.remote_exprs.clone(), + Some(eval_scalar.projection.clone()), + )?)); + } else { + f.push(Box::new(builder.dummy_transform_builder()?)); + } + } + + builder.main_pipeline.add_transforms_by_chunk(f) + } +} + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct MultiInsertEvalScalar { + pub remote_exprs: Vec, + pub projection: ColumnSet, +} + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct ChunkCastSchema { + pub meta: PhysicalPlanMeta, + pub input: PhysicalPlan, + pub cast_schemas: Vec>, +} + +#[typetag::serde] +impl IPhysicalPlan for ChunkCastSchema { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn children<'a>(&'a self) -> Box + 'a> { + Box::new(std::iter::once(&self.input)) + } + + fn children_mut<'a>(&'a mut self) -> Box + 'a> { + Box::new(std::iter::once(&mut self.input)) + } + + fn display_in_profile(&self) -> bool { + false + } + + fn to_format_node( + &self, + _: &mut FormatContext<'_>, + mut children: Vec>, + ) -> Result> { + assert_eq!(children.len(), 1); + Ok(children.pop().unwrap()) + } + + fn derive(&self, mut children: Vec) -> PhysicalPlan { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + self.input.build_pipeline(builder)?; + + if self.cast_schemas.iter().all(|x| x.is_none()) { + return Ok(()); + } + let mut f: Vec = Vec::with_capacity(self.cast_schemas.len()); + for cast_schema in self.cast_schemas.iter() { + if let Some(cast_schema) = cast_schema { + f.push(Box::new(builder.cast_schema_transform_builder( + cast_schema.source_schema.clone(), + cast_schema.target_schema.clone(), + )?)); + } else { + f.push(Box::new(builder.dummy_transform_builder()?)); + } + } + builder.main_pipeline.add_transforms_by_chunk(f) + } +} + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct CastSchema { + pub source_schema: DataSchemaRef, + pub target_schema: DataSchemaRef, +} + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct ChunkFillAndReorder { + pub meta: PhysicalPlanMeta, + pub input: PhysicalPlan, + pub fill_and_reorders: Vec>, +} + +#[typetag::serde] +impl IPhysicalPlan for ChunkFillAndReorder { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn children<'a>(&'a self) -> Box + 'a> { + Box::new(std::iter::once(&self.input)) + } + + fn children_mut<'a>(&'a mut self) -> Box + 'a> { + Box::new(std::iter::once(&mut self.input)) + } + + fn to_format_node( + &self, + _: &mut FormatContext<'_>, + mut children: Vec>, + ) -> Result> { + // ignore self + assert_eq!(children.len(), 1); + Ok(children.pop().unwrap()) + } + + fn display_in_profile(&self) -> bool { + false + } + + fn derive(&self, mut children: Vec) -> PhysicalPlan { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + self.input.build_pipeline(builder)?; + + if self.fill_and_reorders.iter().all(|x| x.is_none()) { + return Ok(()); + } + let mut f: Vec = Vec::with_capacity(self.fill_and_reorders.len()); + for fill_and_reorder in self.fill_and_reorders.iter() { + if let Some(fill_and_reorder) = fill_and_reorder { + let table = builder + .ctx + .build_table_by_table_info(&fill_and_reorder.target_table_info, None)?; + f.push(Box::new(builder.fill_and_reorder_transform_builder( + table, + fill_and_reorder.source_schema.clone(), + )?)); + } else { + f.push(Box::new(builder.dummy_transform_builder()?)); + } + } + + builder.main_pipeline.add_transforms_by_chunk(f) + } +} + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct FillAndReorder { + pub source_schema: DataSchemaRef, + pub target_table_info: TableInfo, +} + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct ChunkAppendData { + pub meta: PhysicalPlanMeta, + pub input: PhysicalPlan, + pub target_tables: Vec, +} + +#[typetag::serde] +impl IPhysicalPlan for ChunkAppendData { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn children<'a>(&'a self) -> Box + 'a> { + Box::new(std::iter::once(&self.input)) + } + + fn children_mut<'a>(&'a mut self) -> Box + 'a> { + Box::new(std::iter::once(&mut self.input)) + } + + fn to_format_node( + &self, + _ctx: &mut FormatContext<'_>, + children: Vec>, + ) -> Result> { + Ok(FormatTreeNode::with_children( + "WriteData".to_string(), + children, + )) + } + + fn derive(&self, mut children: Vec) -> PhysicalPlan { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + self.input.build_pipeline(builder)?; + + let mut compact_task_builders: Vec = + Vec::with_capacity(self.target_tables.len()); + let mut compact_transform_builders: Vec = + Vec::with_capacity(self.target_tables.len()); + let mut serialize_block_builders: Vec = + Vec::with_capacity(self.target_tables.len()); + let mut eval_cluster_key_builders: Vec = + Vec::with_capacity(self.target_tables.len()); + let mut eval_cluster_key_num = 0; + let mut sort_builders: Vec = + Vec::with_capacity(self.target_tables.len()); + let mut sort_num = 0; + + for append_data in self.target_tables.iter() { + let table = builder + .ctx + .build_table_by_table_info(&append_data.target_table_info, None)?; + let block_thresholds = table.get_block_thresholds(); + compact_task_builders.push(Box::new( + builder.block_compact_task_builder(block_thresholds)?, + )); + compact_transform_builders.push(Box::new(builder.block_compact_transform_builder()?)); + let schema: Arc = DataSchema::from(table.schema()).into(); + let num_input_columns = schema.num_fields(); + let fuse_table = FuseTable::try_from_table(table.as_ref())?; + let cluster_stats_gen = fuse_table.get_cluster_stats_gen( + builder.ctx.clone(), + 0, + block_thresholds, + Some(schema), + )?; + let operators = cluster_stats_gen.operators.clone(); + if !operators.is_empty() { + let func_ctx2 = cluster_stats_gen.func_ctx.clone(); + + eval_cluster_key_builders.push(Box::new(move |input, output| { + Ok(ProcessorPtr::create(CompoundBlockOperator::create( + input, + output, + num_input_columns, + func_ctx2.clone(), + operators.clone(), + ))) + })); + eval_cluster_key_num += 1; + } else { + eval_cluster_key_builders.push(Box::new(builder.dummy_transform_builder()?)); + } + let cluster_keys = &cluster_stats_gen.cluster_key_index; + if !cluster_keys.is_empty() { + let sort_desc: Vec = cluster_keys + .iter() + .map(|index| SortColumnDescription { + offset: *index, + asc: true, + nulls_first: false, + }) + .collect(); + let sort_desc: Arc<[_]> = sort_desc.into(); + sort_builders.push(Box::new( + move |transform_input_port, transform_output_port| { + Ok(ProcessorPtr::create(TransformSortPartial::try_create( + transform_input_port, + transform_output_port, + LimitType::None, + sort_desc.clone(), + )?)) + }, + )); + sort_num += 1; + } else { + sort_builders.push(Box::new(builder.dummy_transform_builder()?)); + } + serialize_block_builders.push(Box::new( + builder.with_tid_serialize_block_transform_builder( + table, + cluster_stats_gen, + append_data.table_meta_timestamps, + )?, + )); + } + builder + .main_pipeline + .add_transforms_by_chunk(compact_task_builders)?; + + builder + .main_pipeline + .add_transforms_by_chunk(compact_transform_builders)?; + + if eval_cluster_key_num > 0 { + builder + .main_pipeline + .add_transforms_by_chunk(eval_cluster_key_builders)?; + } + + if sort_num > 0 { + builder + .main_pipeline + .add_transforms_by_chunk(sort_builders)?; + } + + builder + .main_pipeline + .add_transforms_by_chunk(serialize_block_builders) + } +} + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct SerializableTable { + pub target_catalog_info: Arc, + pub target_table_info: TableInfo, + pub table_meta_timestamps: TableMetaTimestamps, +} + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct ChunkMerge { + pub meta: PhysicalPlanMeta, + pub input: PhysicalPlan, + pub group_ids: Vec, +} + +#[typetag::serde] +impl IPhysicalPlan for ChunkMerge { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn children<'a>(&'a self) -> Box + 'a> { + Box::new(std::iter::once(&self.input)) + } + + fn children_mut<'a>(&'a mut self) -> Box + 'a> { + Box::new(std::iter::once(&mut self.input)) + } + + fn to_format_node( + &self, + _: &mut FormatContext<'_>, + mut children: Vec>, + ) -> Result> { + // ignore self + assert_eq!(children.len(), 1); + Ok(children.pop().unwrap()) + } + + fn display_in_profile(&self) -> bool { + false + } + + fn derive(&self, mut children: Vec) -> PhysicalPlan { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + self.input.build_pipeline(builder)?; + + let group_ids = &self.group_ids; + assert_eq!(builder.main_pipeline.output_len() % group_ids.len(), 0); + let chunk_size = builder.main_pipeline.output_len() / group_ids.len(); + let mut widths = Vec::with_capacity(group_ids.len()); + let mut last_group_id = group_ids[0]; + let mut width = 1; + for group_id in group_ids.iter().skip(1) { + if *group_id == last_group_id { + width += 1; + } else { + widths.push(width * chunk_size); + last_group_id = *group_id; + width = 1; + } + } + widths.push(width * chunk_size); + builder.main_pipeline.resize_partial_one_with_width(widths) + } +} + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct ChunkCommitInsert { + pub meta: PhysicalPlanMeta, + pub input: PhysicalPlan, + pub update_stream_meta: Vec, + pub overwrite: bool, + pub deduplicated_label: Option, + pub targets: Vec, +} + +#[typetag::serde] +impl IPhysicalPlan for ChunkCommitInsert { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn children<'a>(&'a self) -> Box + 'a> { + Box::new(std::iter::once(&self.input)) + } + + fn children_mut<'a>(&'a mut self) -> Box + 'a> { + Box::new(std::iter::once(&mut self.input)) + } + + fn derive(&self, mut children: Vec) -> PhysicalPlan { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + self.input.build_pipeline(builder)?; + + let mut table_meta_timestampss = HashMap::new(); + + let mut serialize_segment_builders: Vec = + Vec::with_capacity(self.targets.len()); + let mut mutation_aggregator_builders: Vec = + Vec::with_capacity(self.targets.len()); + let mut tables = HashMap::new(); + + for target in &self.targets { + let table = builder + .ctx + .build_table_by_table_info(&target.target_table_info, None)?; + let block_thresholds = table.get_block_thresholds(); + serialize_segment_builders.push(Box::new( + builder.serialize_segment_transform_builder( + table.clone(), + block_thresholds, + target.table_meta_timestamps, + )?, + )); + mutation_aggregator_builders.push(Box::new( + builder.mutation_aggregator_transform_builder( + table.clone(), + target.table_meta_timestamps, + )?, + )); + table_meta_timestampss.insert(table.get_id(), target.table_meta_timestamps); + tables.insert(table.get_id(), table); + } + + builder + .main_pipeline + .add_transforms_by_chunk(serialize_segment_builders)?; + builder + .main_pipeline + .add_transforms_by_chunk(mutation_aggregator_builders)?; + builder.main_pipeline.try_resize(1)?; + + let catalog = CatalogManager::instance().build_catalog( + self.targets[0].target_catalog_info.clone(), + builder.ctx.session_state(), + )?; + + builder.main_pipeline.add_sink(|input| { + Ok(ProcessorPtr::create(AsyncSinker::create( + input, + CommitMultiTableInsert::create( + tables.clone(), + builder.ctx.clone(), + self.overwrite, + self.update_stream_meta.clone(), + self.deduplicated_label.clone(), + catalog.clone(), + table_meta_timestampss.clone(), + ), + ))) + }) + } +} diff --git a/src/query/sql/src/executor/physical_plans/physical_mutation.rs b/src/query/service/src/physical_plans/physical_mutation.rs similarity index 78% rename from src/query/sql/src/executor/physical_plans/physical_mutation.rs rename to src/query/service/src/physical_plans/physical_mutation.rs index 8179ad56c2998..c9a547b6249b6 100644 --- a/src/query/sql/src/executor/physical_plans/physical_mutation.rs +++ b/src/query/service/src/physical_plans/physical_mutation.rs @@ -12,11 +12,14 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::any::Any; use std::collections::BTreeSet; use std::collections::HashMap; use std::sync::Arc; +use databend_common_ast::ast::FormatTreeNode; use databend_common_catalog::plan::NUM_ROW_ID_PREFIX_BITS; +use databend_common_catalog::table::Table; use databend_common_catalog::table_context::TableContext; use databend_common_exception::ErrorCode; use databend_common_exception::Result; @@ -38,52 +41,61 @@ use databend_common_expression::PREDICATE_COLUMN_NAME; use databend_common_expression::ROW_ID_COL_NAME; use databend_common_functions::BUILTIN_FUNCTIONS; use databend_common_meta_app::schema::TableInfo; +use databend_common_pipeline_core::processors::InputPort; +use databend_common_pipeline_core::processors::OutputPort; +use databend_common_pipeline_core::Pipe; +use databend_common_sql::binder::wrap_cast; +use databend_common_sql::binder::MutationStrategy; +use databend_common_sql::binder::MutationType; +use databend_common_sql::executor::physical_plans::FragmentKind; +use databend_common_sql::executor::physical_plans::MutationKind; +use databend_common_sql::optimizer::ir::SExpr; +use databend_common_sql::parse_computed_expr; +use databend_common_sql::plans::BoundColumnRef; +use databend_common_sql::plans::ConstantExpr; +use databend_common_sql::plans::FunctionCall; +use databend_common_sql::plans::TruncateMode; +use databend_common_sql::BindContext; +use databend_common_sql::ColumnBindingBuilder; +use databend_common_sql::ColumnEntry; +use databend_common_sql::ColumnSet; +use databend_common_sql::IndexType; +use databend_common_sql::MetadataRef; +use databend_common_sql::ScalarExpr; +use databend_common_sql::TypeCheck; +use databend_common_sql::Visibility; +use databend_common_sql::DUMMY_COLUMN_INDEX; +use databend_common_storages_fuse::operations::TransformSerializeBlock; +use databend_common_storages_fuse::FuseTable; use databend_storages_common_table_meta::meta::Location; use databend_storages_common_table_meta::meta::TableMetaTimestamps; use databend_storages_common_table_meta::meta::NUM_BLOCK_ID_BITS; use databend_storages_common_table_meta::readers::snapshot_reader::TableSnapshotAccessor; use itertools::Itertools; +use tokio::sync::Semaphore; use super::ColumnMutation; use super::CommitType; -use crate::binder::wrap_cast; -use crate::binder::MutationStrategy; -use crate::binder::MutationType; -use crate::executor::physical_plan::PhysicalPlan; -use crate::executor::physical_plans::CommitSink; -use crate::executor::physical_plans::Exchange; -use crate::executor::physical_plans::FragmentKind; -use crate::executor::physical_plans::MutationKind; -use crate::executor::physical_plans::MutationManipulate; -use crate::executor::physical_plans::MutationOrganize; -use crate::executor::physical_plans::MutationSplit; -use crate::executor::physical_plans::RowFetch; -use crate::executor::PhysicalPlanBuilder; -use crate::optimizer::ir::SExpr; -use crate::parse_computed_expr; -use crate::plans::BoundColumnRef; -use crate::plans::ConstantExpr; -use crate::plans::FunctionCall; -use crate::plans::TruncateMode; -use crate::BindContext; -use crate::ColumnBindingBuilder; -use crate::ColumnEntry; -use crate::ColumnSet; -use crate::IndexType; -use crate::MetadataRef; -use crate::ScalarExpr; -use crate::TypeCheck; -use crate::Visibility; -use crate::DUMMY_COLUMN_INDEX; +use crate::physical_plans::format::FormatContext; +use crate::physical_plans::physical_plan::IPhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlanMeta; +use crate::physical_plans::CommitSink; +use crate::physical_plans::Exchange; +use crate::physical_plans::MutationManipulate; +use crate::physical_plans::MutationOrganize; +use crate::physical_plans::MutationSplit; +use crate::physical_plans::PhysicalPlanBuilder; +use crate::physical_plans::RowFetch; +use crate::pipelines::PipelineBuilder; // The predicate_column_index should not be conflict with update expr's column_binding's index. pub const PREDICATE_COLUMN_INDEX: IndexType = u64::MAX as usize; -pub type MatchExpr = Vec<(Option, Option>)>; #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct Mutation { - pub plan_id: u32, - pub input: Box, + pub meta: PhysicalPlanMeta, + pub input: PhysicalPlan, pub table_info: TableInfo, // (DataSchemaRef, Option, Vec,Vec) => (source_schema, condition, value_exprs) pub unmatched: Vec<(DataSchemaRef, Option, Vec)>, @@ -96,14 +108,148 @@ pub struct Mutation { pub table_meta_timestamps: TableMetaTimestamps, } +#[typetag::serde] +impl IPhysicalPlan for Mutation { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn output_schema(&self) -> Result { + Ok(DataSchemaRef::default()) + } + + fn children<'a>(&'a self) -> Box + 'a> { + Box::new(std::iter::once(&self.input)) + } + + fn children_mut<'a>(&'a mut self) -> Box + 'a> { + Box::new(std::iter::once(&mut self.input)) + } + + fn to_format_node( + &self, + ctx: &mut FormatContext<'_>, + children: Vec>, + ) -> Result> { + let table_entry = ctx.metadata.table(self.target_table_index).clone(); + let mut node_children = vec![FormatTreeNode::new(format!( + "target table: [catalog: {}] [database: {}] [table: {}]", + table_entry.catalog(), + table_entry.database(), + table_entry.name() + ))]; + + node_children.extend(children); + Ok(FormatTreeNode::with_children( + "DataMutation".to_string(), + node_children, + )) + } + + fn derive(&self, mut children: Vec) -> PhysicalPlan { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + self.input.build_pipeline(builder)?; + + let tbl = builder + .ctx + .build_table_by_table_info(&self.table_info, None)?; + + let table = FuseTable::try_from_table(tbl.as_ref())?; + let block_thresholds = table.get_block_thresholds(); + + let cluster_stats_gen = + table.get_cluster_stats_gen(builder.ctx.clone(), 0, block_thresholds, None)?; + + let max_threads = builder.settings.get_max_threads()? as usize; + let io_request_semaphore = Arc::new(Semaphore::new(max_threads)); + + // For row_id port, create rowid_aggregate_mutator + // For matched data port and unmatched port, do serialize + let serialize_len = match self.strategy { + MutationStrategy::NotMatchedOnly => builder.main_pipeline.output_len(), + MutationStrategy::MixedMatched | MutationStrategy::MatchedOnly => { + // remove row id port + builder.main_pipeline.output_len() - 1 + } + MutationStrategy::Direct => unreachable!(), + }; + + // 1. Fill default and computed columns + builder.build_fill_columns_in_merge_into( + tbl.clone(), + serialize_len, + self.need_match, + self.unmatched.clone(), + )?; + + // 2. Add cluster‘s blocksort if it's a cluster table + builder.build_compact_and_cluster_sort_in_merge_into( + table, + self.need_match, + serialize_len, + block_thresholds, + )?; + + let mut pipe_items = Vec::with_capacity(builder.main_pipeline.output_len()); + + // 3.1 Add rowid_aggregate_mutator for row_id port + if self.need_match { + pipe_items.push(table.rowid_aggregate_mutator( + builder.ctx.clone(), + cluster_stats_gen.clone(), + io_request_semaphore, + self.segments.clone(), + false, + self.table_meta_timestamps, + )?); + } + + // 3.2 Add serialize_block_transform for data port + for _ in 0..serialize_len { + let serialize_block_transform = TransformSerializeBlock::try_create( + builder.ctx.clone(), + InputPort::create(), + OutputPort::create(), + table, + cluster_stats_gen.clone(), + MutationKind::MergeInto, + self.table_meta_timestamps, + )?; + pipe_items.push(serialize_block_transform.into_pipe_item()); + } + + let output_len = pipe_items.iter().map(|item| item.outputs_port.len()).sum(); + builder.main_pipeline.add_pipe(Pipe::create( + builder.main_pipeline.output_len(), + output_len, + pipe_items, + )); + + Ok(()) + } +} + impl PhysicalPlanBuilder { pub async fn build_mutation( &mut self, s_expr: &SExpr, - mutation: &crate::plans::Mutation, + mutation: &databend_common_sql::plans::Mutation, mut required: ColumnSet, ) -> Result { - let crate::plans::Mutation { + let databend_common_sql::plans::Mutation { bind_context, metadata, catalog_name, @@ -172,8 +318,8 @@ impl PhysicalPlanBuilder { if *truncate_table { // Do truncate. - plan = PhysicalPlan::CommitSink(Box::new(CommitSink { - input: Box::new(plan), + plan = Box::new(CommitSink { + input: plan, snapshot: mutation_build_info.table_snapshot, table_info: table_info.clone(), // let's use update first, we will do some optimizations and select exact strategy @@ -182,10 +328,10 @@ impl PhysicalPlanBuilder { }, update_stream_meta: vec![], deduplicated_label: unsafe { self.ctx.get_settings().get_deduplicate_label()? }, - plan_id: u32::MAX, recluster_info: None, + meta: PhysicalPlanMeta::new("CommitSink"), table_meta_timestamps: mutation_build_info.table_meta_timestamps, - })); + }); plan.adjust_plan_id(&mut 0); return Ok(plan); } @@ -239,9 +385,9 @@ impl PhysicalPlanBuilder { (None, None, MutationKind::Delete) }; - plan = PhysicalPlan::ColumnMutation(ColumnMutation { - plan_id: 0, - input: Box::new(plan), + plan = Box::new(ColumnMutation { + input: plan, + meta: PhysicalPlanMeta::new("ColumnMutation"), table_info: mutation_build_info.table_info.clone(), mutation_expr, computed_expr, @@ -254,18 +400,18 @@ impl PhysicalPlanBuilder { }); if *distributed { - plan = PhysicalPlan::Exchange(Exchange { - plan_id: 0, - input: Box::new(plan), + plan = Box::new(Exchange { + input: plan, kind: FragmentKind::Merge, keys: vec![], allow_adjust_parallelism: true, ignore_exchange: false, + meta: PhysicalPlanMeta::new("Exchange"), }); } - plan = PhysicalPlan::CommitSink(Box::new(CommitSink { - input: Box::new(plan), + plan = Box::new(CommitSink { + input: plan, snapshot: mutation_build_info.table_snapshot, table_info: table_info.clone(), // let's use update first, we will do some optimizations and select exact strategy @@ -275,10 +421,10 @@ impl PhysicalPlanBuilder { }, update_stream_meta: vec![], deduplicated_label: unsafe { self.ctx.get_settings().get_deduplicate_label()? }, - plan_id: u32::MAX, + meta: PhysicalPlanMeta::new("CommitSink"), recluster_info: None, table_meta_timestamps: mutation_build_info.table_meta_timestamps, - })); + }); plan.adjust_plan_id(&mut 0); return Ok(plan); @@ -295,23 +441,23 @@ impl PhysicalPlanBuilder { // different nodes update the same physical block simultaneously, data blocks that are needed // to insert just keep in local node. if *distributed && *row_id_shuffle && !is_not_matched_only { - plan = PhysicalPlan::Exchange(build_block_id_shuffle_exchange( + plan = build_block_id_shuffle_exchange( plan, bind_context, mutation_input_schema.clone(), database_name, &table_name, - )?); + )?; } // If the mutation type is FullOperation, we use row_id column to split a block // into matched and not matched parts. if matches!(strategy, MutationStrategy::MixedMatched) { - plan = PhysicalPlan::MutationSplit(Box::new(MutationSplit { - plan_id: 0, - input: Box::new(plan), + plan = Box::new(MutationSplit { + input: plan, split_index: row_id_offset, - })); + meta: PhysicalPlanMeta::new("MutationSplit"), + }); } // Construct row fetch plan for lazy columns. @@ -321,7 +467,7 @@ impl PhysicalPlanBuilder { .get_table_lazy_columns(target_table_index) && !lazy_columns.is_empty() { - plan = PhysicalPlan::RowFetch(build_mutation_row_fetch( + plan = build_mutation_row_fetch( plan, metadata.clone(), mutation_input_schema.clone(), @@ -329,7 +475,7 @@ impl PhysicalPlanBuilder { lazy_columns.clone(), *target_table_index, row_id_offset, - )); + ); } let output_schema = plan.output_schema()?; @@ -440,9 +586,8 @@ impl PhysicalPlanBuilder { } } - plan = PhysicalPlan::MutationManipulate(Box::new(MutationManipulate { - plan_id: 0, - input: Box::new(plan.clone()), + plan = Box::new(MutationManipulate { + input: plan, table_info: table_info.clone(), unmatched: unmatched.clone(), matched: matched.clone(), @@ -451,13 +596,15 @@ impl PhysicalPlanBuilder { row_id_idx: row_id_offset, can_try_update_column_only: *can_try_update_column_only, unmatched_schema: mutation_input_schema.clone(), - })); + meta: PhysicalPlanMeta::new("MutationManipulate"), + target_table_index: *target_table_index, + }); - plan = PhysicalPlan::MutationOrganize(Box::new(MutationOrganize { - plan_id: 0, - input: Box::new(plan.clone()), + plan = Box::new(MutationOrganize { + input: plan, strategy: strategy.clone(), - })); + meta: PhysicalPlanMeta::new("MutationOrganize"), + }); let segments: Vec<_> = mutation_build_info .table_snapshot @@ -467,8 +614,8 @@ impl PhysicalPlanBuilder { .enumerate() .collect(); - let mutation = PhysicalPlan::Mutation(Box::new(Mutation { - input: Box::new(plan.clone()), + plan = Box::new(Mutation { + input: plan, table_info: table_info.clone(), unmatched, segments: segments.clone(), @@ -477,22 +624,20 @@ impl PhysicalPlanBuilder { target_table_index: *target_table_index, need_match: !is_not_matched_only, target_build_optimization: false, - plan_id: u32::MAX, + meta: PhysicalPlanMeta::new("Mutation"), table_meta_timestamps: mutation_build_info.table_meta_timestamps, - })); + }); - let commit_input = if !distributed { - mutation - } else { - PhysicalPlan::Exchange(Exchange { - plan_id: 0, - input: Box::new(mutation), + if *distributed { + plan = Box::new(Exchange { + input: plan, kind: FragmentKind::Merge, keys: vec![], allow_adjust_parallelism: true, ignore_exchange: false, - }) - }; + meta: PhysicalPlanMeta::new("Exchange"), + }); + } let mutation_kind = match mutation_type { MutationType::Update | MutationType::Merge => MutationKind::Update, @@ -500,8 +645,8 @@ impl PhysicalPlanBuilder { }; // build mutation_aggregate - let mut physical_plan = PhysicalPlan::CommitSink(Box::new(CommitSink { - input: Box::new(commit_input), + let mut physical_plan = Box::new(CommitSink { + input: plan, snapshot: mutation_build_info.table_snapshot, table_info: table_info.clone(), // let's use update first, we will do some optimizations and select exact strategy @@ -511,10 +656,11 @@ impl PhysicalPlanBuilder { }, update_stream_meta: mutation_build_info.update_stream_meta, deduplicated_label: unsafe { self.ctx.get_settings().get_deduplicate_label()? }, - plan_id: u32::MAX, recluster_info: None, + meta: PhysicalPlanMeta::new("CommitSink"), table_meta_timestamps: mutation_build_info.table_meta_timestamps, - })); + }); + physical_plan.adjust_plan_id(&mut 0); Ok(physical_plan) } @@ -542,7 +688,7 @@ pub fn build_block_id_shuffle_exchange( mutation_input_schema: Arc, database_name: &str, table_name: &str, -) -> Result { +) -> Result { let mut row_id_column = None; for column_binding in bind_context.columns.iter() { if BindContext::match_column_binding( @@ -592,14 +738,14 @@ pub fn build_block_id_shuffle_exchange( &BUILTIN_FUNCTIONS, )?; - Ok(Exchange { - plan_id: 0, - input: Box::new(plan), + Ok(Box::new(Exchange { + input: plan, kind: FragmentKind::Normal, + meta: PhysicalPlanMeta::new("Exchange"), keys: vec![block_id_shuffle_key.as_remote_expr()], allow_adjust_parallelism: true, ignore_exchange: false, - }) + })) } fn build_mutation_row_fetch( @@ -610,7 +756,7 @@ fn build_mutation_row_fetch( lazy_columns: ColumnSet, target_table_index: usize, row_id_offset: usize, -) -> RowFetch { +) -> PhysicalPlan { let metadata = metadata.read(); let lazy_columns = lazy_columns @@ -652,16 +798,16 @@ fn build_mutation_row_fetch( false, ); - RowFetch { - plan_id: 0, - input: Box::new(plan), + Box::new(RowFetch { + input: plan, source: Box::new(source), row_id_col_offset: row_id_offset, cols_to_fetch, fetched_fields, need_wrap_nullable, stat_info: None, - } + meta: PhysicalPlanMeta::new("RowFetch"), + }) } #[allow(clippy::too_many_arguments)] diff --git a/src/query/service/src/physical_plans/physical_mutation_into_organize.rs b/src/query/service/src/physical_plans/physical_mutation_into_organize.rs new file mode 100644 index 0000000000000..92d622703a8ba --- /dev/null +++ b/src/query/service/src/physical_plans/physical_mutation_into_organize.rs @@ -0,0 +1,143 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::any::Any; + +use databend_common_ast::ast::FormatTreeNode; +use databend_common_exception::Result; +use databend_common_sql::binder::MutationStrategy; + +use crate::physical_plans::format::FormatContext; +use crate::physical_plans::physical_plan::IPhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlanMeta; +use crate::pipelines::PipelineBuilder; + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct MutationOrganize { + pub meta: PhysicalPlanMeta, + pub input: PhysicalPlan, + pub strategy: MutationStrategy, +} + +#[typetag::serde] +impl IPhysicalPlan for MutationOrganize { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn children<'a>(&'a self) -> Box + 'a> { + Box::new(std::iter::once(&self.input)) + } + + fn children_mut<'a>(&'a mut self) -> Box + 'a> { + Box::new(std::iter::once(&mut self.input)) + } + + fn to_format_node( + &self, + _: &mut FormatContext<'_>, + mut children: Vec>, + ) -> Result> { + // ignore self + assert_eq!(children.len(), 1); + Ok(children.pop().unwrap()) + } + + fn derive(&self, mut children: Vec) -> PhysicalPlan { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + self.input.build_pipeline(builder)?; + + // The complete pipeline: + // ----------------------------------------------------------------------------------------- + // row_id port0_1 row_id port0_1 row_id port0_1 + // matched data port0_2 ..... row_id port1_1 row_id port + // unmatched port0_3 data port0_2 ...... + // row_id port1_1 ====> row_id port1_1 ====> data port0_2 ====> data port0 + // matched data port1_2 ..... data port1_2 data port1 + // unmatched port1_3 data port1_2 ...... + // ...... ..... + // ----------------------------------------------------------------------------------------- + // 1. matched only or complete pipeline are same with above + // 2. for unmatched only, there are no row_id port + + let mut ranges = Vec::with_capacity(builder.main_pipeline.output_len()); + let mut rules = Vec::with_capacity(builder.main_pipeline.output_len()); + match self.strategy { + MutationStrategy::MixedMatched => { + assert_eq!(builder.main_pipeline.output_len() % 3, 0); + // merge matched update ports and not matched ports ===> data ports + for idx in (0..builder.main_pipeline.output_len()).step_by(3) { + ranges.push(vec![idx]); + ranges.push(vec![idx + 1, idx + 2]); + } + builder.main_pipeline.resize_partial_one(ranges.clone())?; + assert_eq!(builder.main_pipeline.output_len() % 2, 0); + let row_id_len = builder.main_pipeline.output_len() / 2; + for idx in 0..row_id_len { + rules.push(idx); + rules.push(idx + row_id_len); + } + builder.main_pipeline.reorder_inputs(rules); + self.resize_row_id(2, builder)?; + } + MutationStrategy::MatchedOnly => { + assert_eq!(builder.main_pipeline.output_len() % 2, 0); + let row_id_len = builder.main_pipeline.output_len() / 2; + for idx in 0..row_id_len { + rules.push(idx); + rules.push(idx + row_id_len); + } + builder.main_pipeline.reorder_inputs(rules); + self.resize_row_id(2, builder)?; + } + MutationStrategy::NotMatchedOnly => {} + MutationStrategy::Direct => unreachable!(), + } + Ok(()) + } +} + +impl MutationOrganize { + fn resize_row_id(&self, step: usize, builder: &mut PipelineBuilder) -> Result<()> { + // resize row_id + let row_id_len = builder.main_pipeline.output_len() / step; + let mut ranges = Vec::with_capacity(builder.main_pipeline.output_len()); + let mut vec = Vec::with_capacity(row_id_len); + for idx in 0..row_id_len { + vec.push(idx); + } + ranges.push(vec.clone()); + + // data ports + for idx in 0..row_id_len { + ranges.push(vec![idx + row_id_len]); + } + + builder.main_pipeline.resize_partial_one(ranges.clone()) + } +} diff --git a/src/query/service/src/physical_plans/physical_mutation_into_split.rs b/src/query/service/src/physical_plans/physical_mutation_into_split.rs new file mode 100644 index 0000000000000..53536614e93b9 --- /dev/null +++ b/src/query/service/src/physical_plans/physical_mutation_into_split.rs @@ -0,0 +1,95 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::any::Any; + +use databend_common_ast::ast::FormatTreeNode; +use databend_common_exception::Result; +use databend_common_pipeline_core::Pipe; +use databend_common_sql::IndexType; +use databend_common_storages_fuse::operations::MutationSplitProcessor; + +use crate::physical_plans::format::FormatContext; +use crate::physical_plans::physical_plan::IPhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlanMeta; +use crate::pipelines::PipelineBuilder; + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct MutationSplit { + pub meta: PhysicalPlanMeta, + pub input: PhysicalPlan, + pub split_index: IndexType, +} + +#[typetag::serde] +impl IPhysicalPlan for MutationSplit { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn children<'a>(&'a self) -> Box + 'a> { + Box::new(std::iter::once(&self.input)) + } + + fn children_mut<'a>(&'a mut self) -> Box + 'a> { + Box::new(std::iter::once(&mut self.input)) + } + + fn to_format_node( + &self, + _: &mut FormatContext<'_>, + mut children: Vec>, + ) -> Result> { + // ignore self + assert_eq!(children.len(), 1); + Ok(children.pop().unwrap()) + } + + fn derive(&self, mut children: Vec) -> PhysicalPlan { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + self.input.build_pipeline(builder)?; + + builder + .main_pipeline + .try_resize(builder.settings.get_max_threads()? as usize)?; + + // The MutationStrategy is FullOperation, use row_id_idx to split + let mut items = Vec::with_capacity(builder.main_pipeline.output_len()); + let output_len = builder.main_pipeline.output_len(); + for _ in 0..output_len { + let merge_into_split_processor = + MutationSplitProcessor::create(self.split_index as u32)?; + items.push(merge_into_split_processor.into_pipe_item()); + } + + builder + .main_pipeline + .add_pipe(Pipe::create(output_len, output_len * 2, items)); + Ok(()) + } +} diff --git a/src/query/service/src/physical_plans/physical_mutation_manipulate.rs b/src/query/service/src/physical_plans/physical_mutation_manipulate.rs new file mode 100644 index 0000000000000..278b6d5de111a --- /dev/null +++ b/src/query/service/src/physical_plans/physical_mutation_manipulate.rs @@ -0,0 +1,252 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::any::Any; +use std::collections::HashMap; +use std::sync::Arc; + +use databend_common_ast::ast::FormatTreeNode; +use databend_common_exception::Result; +use databend_common_expression::DataSchema; +use databend_common_expression::DataSchemaRef; +use databend_common_expression::FieldIndex; +use databend_common_expression::RemoteExpr; +use databend_common_functions::BUILTIN_FUNCTIONS; +use databend_common_meta_app::schema::TableInfo; +use databend_common_pipeline_core::Pipe; +use databend_common_sql::binder::MutationStrategy; +use databend_common_sql::executor::physical_plans::MatchExpr; +use databend_common_storages_fuse::operations::MatchedSplitProcessor; +use databend_common_storages_fuse::operations::MergeIntoNotMatchedProcessor; +use itertools::Itertools; + +use crate::physical_plans::format::FormatContext; +use crate::physical_plans::physical_plan::IPhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlanMeta; +use crate::pipelines::PipelineBuilder; + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct MutationManipulate { + pub meta: PhysicalPlanMeta, + pub input: PhysicalPlan, + pub table_info: TableInfo, + // (DataSchemaRef, Option, Vec,Vec) => (source_schema, condition, value_exprs) + pub unmatched: Vec<(DataSchemaRef, Option, Vec)>, + // the first option stands for the condition + // the second option stands for update/delete + pub matched: MatchExpr, + // used to record the index of target table's field in merge_source_schema + pub field_index_of_input_schema: HashMap, + pub strategy: MutationStrategy, + pub row_id_idx: usize, + pub can_try_update_column_only: bool, + pub unmatched_schema: DataSchemaRef, + pub target_table_index: usize, +} + +#[typetag::serde] +impl IPhysicalPlan for MutationManipulate { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn children<'a>(&'a self) -> Box + 'a> { + Box::new(std::iter::once(&self.input)) + } + + fn children_mut<'a>(&'a mut self) -> Box + 'a> { + Box::new(std::iter::once(&mut self.input)) + } + + fn to_format_node( + &self, + ctx: &mut FormatContext<'_>, + children: Vec>, + ) -> Result> { + let table_entry = ctx.metadata.table(self.target_table_index).clone(); + let target_schema = table_entry.table().schema_with_stream(); + + // Matched clauses. + let mut matched_children = Vec::with_capacity(self.matched.len()); + for evaluator in &self.matched { + let condition_format = evaluator.0.as_ref().map_or_else( + || "condition: None".to_string(), + |predicate| { + format!( + "condition: {}", + predicate.as_expr(&BUILTIN_FUNCTIONS).sql_display() + ) + }, + ); + + if evaluator.1.is_none() { + matched_children.push(FormatTreeNode::new(format!( + "matched delete: [{}]", + condition_format + ))); + } else { + let mut update_list = evaluator.1.as_ref().unwrap().clone(); + update_list.sort_by(|a, b| a.0.cmp(&b.0)); + let update_format = update_list + .iter() + .map(|(field_idx, expr)| { + format!( + "{} = {}", + target_schema.field(*field_idx).name(), + expr.as_expr(&BUILTIN_FUNCTIONS).sql_display() + ) + }) + .join(","); + matched_children.push(FormatTreeNode::new(format!( + "matched update: [{}, update set {}]", + condition_format, update_format + ))); + } + } + + // UnMatched clauses. + let mut unmatched_children = Vec::with_capacity(self.unmatched.len()); + for evaluator in &self.unmatched { + let condition_format = evaluator.1.as_ref().map_or_else( + || "condition: None".to_string(), + |predicate| { + format!( + "condition: {}", + predicate.as_expr(&BUILTIN_FUNCTIONS).sql_display() + ) + }, + ); + let insert_schema_format = evaluator + .0 + .fields + .iter() + .map(|field| field.name()) + .join(","); + + let values_format = evaluator + .2 + .iter() + .map(|expr| expr.as_expr(&BUILTIN_FUNCTIONS).sql_display()) + .join(","); + + let unmatched_format = format!( + "insert into ({}) values({})", + insert_schema_format, values_format + ); + + unmatched_children.push(FormatTreeNode::new(format!( + "unmatched insert: [{}, {}]", + condition_format, unmatched_format + ))); + } + + let mut node_children = vec![]; + + node_children.extend(matched_children); + node_children.extend(unmatched_children); + node_children.extend(children); + + Ok(FormatTreeNode::with_children( + "MutationManipulate".to_string(), + node_children, + )) + } + + fn derive(&self, mut children: Vec) -> PhysicalPlan { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } + + // Handle matched and unmatched data separately. + // This is a complete pipeline with matched and not matched clauses, for matched only or unmatched only + // we will delicate useless pipeline and processor + // +-----------------------------+-+ + // +-----------------------+ Matched | +-+ + // | +---+--------------->| MatchedSplitProcessor | + // | | | | +-+ + // +----------------------+ | +---+ +-----------------------------+-+ + // | MergeInto +---------->|MutationSplitProcessor | + // +----------------------+ | +---+ +-----------------------------+ + // | | | NotMatched | +-+ + // | +---+--------------->| MergeIntoNotMatchedProcessor| | + // +-----------------------+ | +-+ + // +-----------------------------+ + // Note: here the output_port of MatchedSplitProcessor are arranged in the following order + // (0) -> output_port_row_id + // (1) -> output_port_updated + + // Outputs from MatchedSplitProcessor's output_port_updated and MergeIntoNotMatchedProcessor's output_port are merged and processed uniformly by the subsequent ResizeProcessor + // receive matched data and not matched data parallelly. + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + self.input.build_pipeline(builder)?; + + let (step, need_match, need_unmatch) = match self.strategy { + MutationStrategy::MatchedOnly => (1, true, false), + MutationStrategy::NotMatchedOnly => (1, false, true), + MutationStrategy::MixedMatched => (2, true, true), + MutationStrategy::Direct => unreachable!(), + }; + + let tbl = builder + .ctx + .build_table_by_table_info(&self.table_info, None)?; + + let input_schema = self.input.output_schema()?; + let mut pipe_items = Vec::with_capacity(builder.main_pipeline.output_len()); + for _ in (0..builder.main_pipeline.output_len()).step_by(step) { + if need_match { + let matched_split_processor = MatchedSplitProcessor::create( + builder.ctx.clone(), + self.row_id_idx, + self.matched.clone(), + self.field_index_of_input_schema.clone(), + input_schema.clone(), + Arc::new(DataSchema::from(tbl.schema_with_stream())), + false, + self.can_try_update_column_only, + )?; + pipe_items.push(matched_split_processor.into_pipe_item()); + } + + if need_unmatch { + let merge_into_not_matched_processor = MergeIntoNotMatchedProcessor::create( + self.unmatched.clone(), + self.unmatched_schema.clone(), + builder.func_ctx.clone(), + builder.ctx.clone(), + )?; + pipe_items.push(merge_into_not_matched_processor.into_pipe_item()); + } + } + + let output_len = pipe_items.iter().map(|item| item.outputs_port.len()).sum(); + builder.main_pipeline.add_pipe(Pipe::create( + builder.main_pipeline.output_len(), + output_len, + pipe_items.clone(), + )); + + Ok(()) + } +} diff --git a/src/query/service/src/physical_plans/physical_mutation_source.rs b/src/query/service/src/physical_plans/physical_mutation_source.rs new file mode 100644 index 0000000000000..3d68b0d499788 --- /dev/null +++ b/src/query/service/src/physical_plans/physical_mutation_source.rs @@ -0,0 +1,335 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::any::Any; + +use databend_common_ast::ast::FormatTreeNode; +use databend_common_base::runtime::Runtime; +use databend_common_catalog::plan::Filters; +use databend_common_catalog::plan::PartInfoType; +use databend_common_catalog::plan::PartStatistics; +use databend_common_catalog::plan::Partitions; +use databend_common_catalog::plan::Projection; +use databend_common_catalog::table::Table; +use databend_common_catalog::table_context::TableContext; +use databend_common_exception::Result; +use databend_common_expression::type_check::check_function; +use databend_common_expression::types::DataType; +use databend_common_expression::ConstantFolder; +use databend_common_expression::DataBlock; +use databend_common_expression::DataField; +use databend_common_expression::DataSchemaRef; +use databend_common_expression::DataSchemaRefExt; +use databend_common_expression::FunctionContext; +use databend_common_functions::BUILTIN_FUNCTIONS; +use databend_common_meta_app::schema::TableInfo; +use databend_common_pipeline_sources::OneBlockSource; +use databend_common_pipeline_transforms::TransformPipelineHelper; +use databend_common_sql::binder::MutationType; +use databend_common_sql::executor::cast_expr_to_non_null_boolean; +use databend_common_sql::ColumnSet; +use databend_common_sql::IndexType; +use databend_common_sql::ScalarExpr; +use databend_common_sql::StreamContext; +use databend_common_storages_fuse::operations::CommitMeta; +use databend_common_storages_fuse::operations::ConflictResolveContext; +use databend_common_storages_fuse::operations::MutationAction; +use databend_common_storages_fuse::operations::MutationBlockPruningContext; +use databend_common_storages_fuse::FuseLazyPartInfo; +use databend_common_storages_fuse::FuseTable; +use databend_common_storages_fuse::SegmentLocation; + +use crate::physical_plans::format::format_output_columns; +use crate::physical_plans::format::part_stats_info_to_format_tree; +use crate::physical_plans::format::FormatContext; +use crate::physical_plans::physical_plan::IPhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlanMeta; +use crate::physical_plans::PhysicalPlanBuilder; +use crate::pipelines::processors::transforms::TransformAddStreamColumns; +use crate::pipelines::PipelineBuilder; + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct MutationSource { + pub meta: PhysicalPlanMeta, + pub table_index: IndexType, + pub table_info: TableInfo, + pub filters: Option, + pub output_schema: DataSchemaRef, + pub input_type: MutationType, + pub read_partition_columns: ColumnSet, + pub truncate_table: bool, + + pub partitions: Partitions, + pub statistics: PartStatistics, +} + +#[typetag::serde] +impl IPhysicalPlan for MutationSource { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn output_schema(&self) -> Result { + Ok(self.output_schema.clone()) + } + + fn to_format_node( + &self, + ctx: &mut FormatContext<'_>, + _: Vec>, + ) -> Result> { + let table = ctx.metadata.table(self.table_index); + let table_name = format!("{}.{}.{}", table.catalog(), table.database(), table.name()); + + let filters = self + .filters + .as_ref() + .map(|filters| filters.filter.as_expr(&BUILTIN_FUNCTIONS).sql_display()) + .unwrap_or_default(); + + let mut node_children = vec![ + FormatTreeNode::new(format!("table: {table_name}")), + FormatTreeNode::new(format!( + "output columns: [{}]", + format_output_columns(self.output_schema()?, ctx.metadata, false) + )), + FormatTreeNode::new(format!("filters: [{filters}]")), + ]; + + let payload = match self.input_type { + MutationType::Update => "Update", + MutationType::Delete if self.truncate_table => "DeleteAll", + MutationType::Delete => "Delete", + MutationType::Merge => "Merge", + }; + + // Part stats. + node_children.extend(part_stats_info_to_format_tree(&self.statistics)); + Ok(FormatTreeNode::with_children( + format!("MutationSource({})", payload), + node_children, + )) + } + + fn try_find_mutation_source(&self) -> Option { + Some(self.clone()) + } + + fn derive(&self, children: Vec) -> PhysicalPlan { + assert!(children.is_empty()); + Box::new(self.clone()) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + let table = builder + .ctx + .build_table_by_table_info(&self.table_info, None)?; + + let table = FuseTable::try_from_table(table.as_ref())?.clone(); + let is_delete = self.input_type == MutationType::Delete; + if self.truncate_table { + // There is no filter and the mutation type is delete, + // we can truncate the table directly. + debug_assert!(self.partitions.is_empty() && is_delete); + return builder.main_pipeline.add_source( + |output| { + let meta = CommitMeta { + conflict_resolve_context: ConflictResolveContext::None, + new_segment_locs: vec![], + table_id: table.get_id(), + virtual_schema: None, + }; + let block = DataBlock::empty_with_meta(Box::new(meta)); + OneBlockSource::create(output, block) + }, + 1, + ); + } + + let read_partition_columns: Vec = + self.read_partition_columns.clone().into_iter().collect(); + + let is_lazy = self.partitions.partitions_type() == PartInfoType::LazyLevel && is_delete; + if is_lazy { + let ctx = builder.ctx.clone(); + let table_clone = table.clone(); + let ctx_clone = builder.ctx.clone(); + let filters_clone = self.filters.clone(); + let projection = Projection::Columns(read_partition_columns.clone()); + let mut segment_locations = Vec::with_capacity(self.partitions.partitions.len()); + for part in &self.partitions.partitions { + // Safe to downcast because we know the partition is lazy + let part: &FuseLazyPartInfo = FuseLazyPartInfo::from_part(part)?; + segment_locations.push(SegmentLocation { + segment_idx: part.segment_index, + location: part.segment_location.clone(), + snapshot_loc: None, + }); + } + let prune_ctx = MutationBlockPruningContext { + segment_locations, + block_count: None, + }; + Runtime::with_worker_threads(2, Some("do_mutation_block_pruning".to_string()))? + .block_on(async move { + let (_, partitions) = table_clone + .do_mutation_block_pruning( + ctx_clone, + filters_clone, + projection, + prune_ctx, + true, + ) + .await?; + ctx.set_partitions(partitions)?; + Ok(()) + })?; + } else { + builder.ctx.set_partitions(self.partitions.clone())?; + } + + let filter = self.filters.clone().map(|v| v.filter); + let mutation_action = if is_delete { + MutationAction::Deletion + } else { + MutationAction::Update + }; + let col_indices = self.read_partition_columns.clone().into_iter().collect(); + let update_mutation_with_filter = + self.input_type == MutationType::Update && filter.is_some(); + table.add_mutation_source( + builder.ctx.clone(), + filter, + col_indices, + &mut builder.main_pipeline, + mutation_action, + )?; + + if table.change_tracking_enabled() { + let stream_ctx = StreamContext::try_create( + builder.ctx.get_function_context()?, + table.schema_with_stream(), + table.get_table_info().ident.seq, + is_delete, + update_mutation_with_filter, + )?; + builder + .main_pipeline + .add_transformer(|| TransformAddStreamColumns::new(stream_ctx.clone())); + } + + Ok(()) + } +} + +impl PhysicalPlanBuilder { + pub async fn build_mutation_source( + &mut self, + mutation_source: &databend_common_sql::plans::MutationSource, + ) -> Result { + let filters = if !mutation_source.predicates.is_empty() { + Some(create_push_down_filters( + &self.ctx.get_function_context()?, + &mutation_source.predicates, + )?) + } else { + None + }; + let mutation_info = self.mutation_build_info.as_ref().unwrap(); + + let metadata = self.metadata.read(); + let mut fields = Vec::with_capacity(mutation_source.columns.len()); + for column_index in mutation_source.columns.iter() { + let column = metadata.column(*column_index); + // Ignore virtual computed columns. + if let Ok(column_id) = mutation_source.schema.index_of(&column.name()) { + fields.push((column.name(), *column_index, column_id)); + } + } + fields.sort_by_key(|(_, _, id)| *id); + + let mut fields = fields + .into_iter() + .map(|(name, index, _)| { + let table_field = mutation_source.schema.field_with_name(&name)?; + let data_type = DataType::from(table_field.data_type()); + Ok(DataField::new(&index.to_string(), data_type)) + }) + .collect::>>()?; + + if let Some(predicate_index) = mutation_source.predicate_column_index { + fields.push(DataField::new( + &predicate_index.to_string(), + DataType::Boolean, + )); + } + let output_schema = DataSchemaRefExt::create(fields); + + let truncate_table = + mutation_source.mutation_type == MutationType::Delete && filters.is_none(); + Ok(Box::new(MutationSource { + table_index: mutation_source.table_index, + output_schema, + table_info: mutation_info.table_info.clone(), + filters, + input_type: mutation_source.mutation_type.clone(), + read_partition_columns: mutation_source.read_partition_columns.clone(), + truncate_table, + meta: PhysicalPlanMeta::new("MutationSource"), + partitions: mutation_info.partitions.clone(), + statistics: mutation_info.statistics.clone(), + })) + } +} + +/// create push down filters +pub fn create_push_down_filters( + func_ctx: &FunctionContext, + predicates: &[ScalarExpr], +) -> Result { + let predicates = predicates + .iter() + .map(|p| { + Ok(p.as_expr()? + .project_column_ref(|col| col.column_name.clone())) + }) + .collect::>>()?; + + let expr = predicates + .into_iter() + .try_reduce(|lhs, rhs| { + check_function(None, "and_filters", &[], &[lhs, rhs], &BUILTIN_FUNCTIONS) + })? + .unwrap(); + let expr = cast_expr_to_non_null_boolean(expr)?; + let (filter, _) = ConstantFolder::fold(&expr, func_ctx, &BUILTIN_FUNCTIONS); + let remote_filter = filter.as_remote_expr(); + + // prepare the inverse filter expression + let remote_inverted_filter = + check_function(None, "not", &[], &[filter], &BUILTIN_FUNCTIONS)?.as_remote_expr(); + + Ok(Filters { + filter: remote_filter, + inverted_filter: remote_inverted_filter, + }) +} diff --git a/src/query/service/src/physical_plans/physical_plan.rs b/src/query/service/src/physical_plans/physical_plan.rs new file mode 100644 index 0000000000000..9240bf4a6dc9f --- /dev/null +++ b/src/query/service/src/physical_plans/physical_plan.rs @@ -0,0 +1,296 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::any::Any; +use std::collections::HashMap; +use std::fmt::Debug; + +use databend_common_ast::ast::FormatTreeNode; +use databend_common_base::runtime::profile::get_statistics_desc; +use databend_common_catalog::plan::DataSourcePlan; +use databend_common_catalog::plan::PartStatistics; +use databend_common_exception::ErrorCode; +use databend_common_exception::Result; +use databend_common_expression::DataSchemaRef; +use databend_common_pipeline_core::PlanProfile; +use databend_common_sql::Metadata; + +use crate::physical_plans::format::FormatContext; +use crate::physical_plans::MutationSource; +use crate::pipelines::PipelineBuilder; + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct PhysicalPlanMeta { + plan_id: u32, + name: String, +} + +impl PhysicalPlanMeta { + pub fn new(name: impl Into) -> PhysicalPlanMeta { + PhysicalPlanMeta::with_plan_id(name, 0) + } + + pub fn with_plan_id(name: impl Into, plan_id: u32) -> PhysicalPlanMeta { + PhysicalPlanMeta { + plan_id, + name: name.into(), + } + } +} + +pub trait DeriveHandle: Send + Sync + 'static { + fn as_any(&mut self) -> &mut dyn Any; + + fn derive( + &mut self, + v: &PhysicalPlan, + children: Vec, + ) -> std::result::Result>; +} + +#[typetag::serde] +pub trait IPhysicalPlan: Debug + Send + Sync + 'static { + fn as_any(&self) -> &dyn Any; + + fn get_meta(&self) -> &PhysicalPlanMeta; + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta; + + // For methods with default implementations, the default implementation is usually sufficient. + fn get_id(&self) -> u32 { + self.get_meta().plan_id + } + + fn get_name(&self) -> String { + self.get_meta().name.clone() + } + + /// Adjust the plan_id of the physical plan. + /// This function will assign a unique plan_id to each physical plan node in a top-down manner. + /// Which means the plan_id of a node is always greater than the plan_id of its parent node. + // #[recursive::recursive] + fn adjust_plan_id(&mut self, next_id: &mut u32) { + self.get_meta_mut().plan_id = *next_id; + *next_id += 1; + + for child in self.children_mut() { + child.adjust_plan_id(next_id); + } + } + + fn output_schema(&self) -> Result { + match self.children().next() { + None => Ok(DataSchemaRef::default()), + Some(child) => child.output_schema(), + } + } + + fn children(&self) -> Box + '_> { + Box::new(std::iter::empty()) + } + + fn children_mut(&mut self) -> Box + '_> { + Box::new(std::iter::empty()) + } + + fn to_format_node( + &self, + _ctx: &mut FormatContext<'_>, + children: Vec>, + ) -> Result> { + Ok(FormatTreeNode::with_children(self.get_name(), children)) + } + + /// Used to find data source info in a non-aggregation and single-table query plan. + fn try_find_single_data_source(&self) -> Option<&DataSourcePlan> { + None + } + + // #[recursive::recursive] + fn try_find_mutation_source(&self) -> Option { + for child in self.children() { + if let Some(plan) = child.try_find_mutation_source() { + return Some(plan); + } + } + + None + } + + // #[recursive::recursive] + fn get_all_data_source(&self, sources: &mut Vec<(u32, Box)>) { + for child in self.children() { + child.get_all_data_source(sources); + } + } + + // #[recursive::recursive] + fn set_pruning_stats(&mut self, stats: &mut HashMap) { + for child in self.children_mut() { + child.set_pruning_stats(stats) + } + } + + // #[recursive::recursive] + fn is_distributed_plan(&self) -> bool { + self.children().any(|child| child.is_distributed_plan()) + } + + // #[recursive::recursive] + fn is_warehouse_distributed_plan(&self) -> bool { + self.children() + .any(|child| child.is_warehouse_distributed_plan()) + } + + fn display_in_profile(&self) -> bool { + true + } + + fn get_desc(&self) -> Result { + Ok(String::new()) + } + + fn get_labels(&self) -> Result>> { + Ok(HashMap::new()) + } + + fn derive(&self, children: Vec) -> PhysicalPlan; + + fn build_pipeline(&self, builder: &mut PipelineBuilder) -> Result<()> { + self.build_pipeline2(builder) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + let _ = builder; + Err(ErrorCode::Unimplemented(format!( + "UnImplement build_pipeline method for {:?}", + self.get_name() + ))) + } +} + +pub trait PhysicalPlanVisitor: Send + Sync + 'static { + fn as_any(&mut self) -> &mut dyn Any; + + fn visit(&mut self, plan: &PhysicalPlan) -> Result<()>; +} + +pub trait PhysicalPlanDynExt { + fn format( + &self, + metadata: &Metadata, + profs: HashMap, + ) -> Result> { + let mut context = FormatContext { + metadata, + scan_id_to_runtime_filters: HashMap::new(), + }; + + self.to_format_tree(&profs, &mut context) + } + + fn to_format_tree( + &self, + profs: &HashMap, + ctx: &mut FormatContext<'_>, + ) -> Result>; + + fn downcast_ref(&self) -> Option<&To>; + + fn downcast_mut_ref(&mut self) -> Option<&mut To>; + + fn derive_with(&self, handle: &mut Box) -> PhysicalPlan; + + fn visit(&self, visitor: &mut Box) -> Result<()>; +} + +impl PhysicalPlanDynExt for Box { + fn to_format_tree( + &self, + profs: &HashMap, + ctx: &mut FormatContext<'_>, + ) -> Result> { + let mut children = Vec::with_capacity(4); + for child in self.children() { + children.push(child.to_format_tree(profs, ctx)?); + } + + let mut format_tree_node = self.to_format_node(ctx, children)?; + + if let Some(prof) = profs.get(&self.get_id()) { + let mut children = Vec::with_capacity(format_tree_node.children.len() + 10); + for (_, desc) in get_statistics_desc().iter() { + if prof.statistics[desc.index] != 0 { + children.push(FormatTreeNode::new(format!( + "{}: {}", + desc.display_name.to_lowercase(), + desc.human_format(prof.statistics[desc.index]) + ))); + } + } + + children.append(&mut format_tree_node.children); + format_tree_node.children = children; + } + + Ok(format_tree_node) + } + + fn downcast_ref(&self) -> Option<&To> { + self.as_any().downcast_ref() + } + + fn downcast_mut_ref(&mut self) -> Option<&mut To> { + unsafe { + match self.downcast_ref::() { + None => None, + #[allow(invalid_reference_casting)] + Some(v) => Some(&mut *(v as *const To as *mut To)), + } + } + } + + fn derive_with(&self, handle: &mut Box) -> PhysicalPlan { + let mut children = vec![]; + for child in self.children() { + children.push(child.derive_with(handle)); + } + + match handle.derive(self, children) { + Ok(v) => v, + Err(children) => self.derive(children), + } + } + + fn visit(&self, visitor: &mut Box) -> Result<()> { + for child in self.children() { + child.visit(visitor)?; + } + + visitor.visit(self) + } +} + +impl Clone for PhysicalPlan { + fn clone(&self) -> Self { + let mut children = vec![]; + for child in self.children() { + children.push(child.clone()); + } + + self.derive(children) + } +} + +pub type PhysicalPlan = Box; diff --git a/src/query/sql/src/executor/physical_plan_builder.rs b/src/query/service/src/physical_plans/physical_plan_builder.rs similarity index 90% rename from src/query/sql/src/executor/physical_plan_builder.rs rename to src/query/service/src/physical_plans/physical_plan_builder.rs index c50adc9f30db4..6ac0fd20de962 100644 --- a/src/query/sql/src/executor/physical_plan_builder.rs +++ b/src/query/service/src/physical_plans/physical_plan_builder.rs @@ -21,24 +21,24 @@ use databend_common_exception::Result; use databend_common_expression::FunctionContext; use databend_common_meta_app::schema::TableInfo; use databend_common_meta_app::schema::UpdateStreamMetaReq; +use databend_common_sql::optimizer::ir::RelExpr; +use databend_common_sql::optimizer::ir::SExpr; +use databend_common_sql::plans::RelOperator; +use databend_common_sql::ColumnSet; +use databend_common_sql::MetadataRef; use databend_storages_common_table_meta::meta::TableMetaTimestamps; use databend_storages_common_table_meta::meta::TableSnapshot; -use crate::executor::explain::PlanStatsInfo; -use crate::executor::PhysicalPlan; -use crate::optimizer::ir::RelExpr; -use crate::optimizer::ir::SExpr; -use crate::plans::RelOperator; -use crate::ColumnSet; -use crate::MetadataRef; +use crate::physical_plans::explain::PlanStatsInfo; +use crate::physical_plans::physical_plan::PhysicalPlan; pub struct PhysicalPlanBuilder { - pub(crate) metadata: MetadataRef, - pub(crate) ctx: Arc, - pub(crate) func_ctx: FunctionContext, - pub(crate) dry_run: bool, + pub metadata: MetadataRef, + pub ctx: Arc, + pub func_ctx: FunctionContext, + pub dry_run: bool, // DataMutation info, used to build MergeInto physical plan - pub(crate) mutation_build_info: Option, + pub mutation_build_info: Option, } impl PhysicalPlanBuilder { @@ -53,7 +53,7 @@ impl PhysicalPlanBuilder { } } - pub(crate) fn build_plan_stat_info(&self, s_expr: &SExpr) -> Result { + pub fn build_plan_stat_info(&self, s_expr: &SExpr) -> Result { let rel_expr = RelExpr::with_s_expr(s_expr); let stat_info = rel_expr.derive_cardinality()?; diff --git a/src/query/service/src/physical_plans/physical_plan_visitor.rs b/src/query/service/src/physical_plans/physical_plan_visitor.rs new file mode 100644 index 0000000000000..e8350298f45d1 --- /dev/null +++ b/src/query/service/src/physical_plans/physical_plan_visitor.rs @@ -0,0 +1,608 @@ +// 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. + +// pub trait PhysicalPlanReplacer { +// fn replace(&mut self, plan: &PhysicalPlan) -> Result { +// match plan { +// PhysicalPlan::TableScan(plan) => self.replace_table_scan(plan), +// PhysicalPlan::RecursiveCteScan(plan) => self.replace_recursive_cte_scan(plan), +// PhysicalPlan::Filter(plan) => self.replace_filter(plan), +// PhysicalPlan::EvalScalar(plan) => self.replace_eval_scalar(plan), +// PhysicalPlan::AggregateExpand(plan) => self.replace_aggregate_expand(plan), +// PhysicalPlan::AggregatePartial(plan) => self.replace_aggregate_partial(plan), +// PhysicalPlan::AggregateFinal(plan) => self.replace_aggregate_final(plan), +// PhysicalPlan::Window(plan) => self.replace_window(plan), +// PhysicalPlan::WindowPartition(plan) => self.replace_window_partition(plan), +// PhysicalPlan::Sort(plan) => self.replace_sort(plan), +// PhysicalPlan::Limit(plan) => self.replace_limit(plan), +// PhysicalPlan::RowFetch(plan) => self.replace_row_fetch(plan), +// PhysicalPlan::HashJoin(plan) => self.replace_hash_join(plan), +// PhysicalPlan::Exchange(plan) => self.replace_exchange(plan), +// 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::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), +// PhysicalPlan::ColumnMutation(plan) => self.replace_column_mutation(plan), +// PhysicalPlan::Mutation(plan) => self.replace_mutation(plan), +// PhysicalPlan::MutationSplit(plan) => self.replace_mutation_split(plan), +// PhysicalPlan::MutationManipulate(plan) => self.replace_mutation_manipulate(plan), +// PhysicalPlan::MutationOrganize(plan) => self.replace_mutation_organize(plan), +// PhysicalPlan::AddStreamColumn(plan) => self.replace_add_stream_column(plan), +// PhysicalPlan::ConstantTableScan(plan) => self.replace_constant_table_scan(plan), +// PhysicalPlan::ExpressionScan(plan) => self.replace_expression_scan(plan), +// PhysicalPlan::CacheScan(plan) => self.replace_cache_scan(plan), +// PhysicalPlan::Recluster(plan) => self.replace_recluster(plan), +// PhysicalPlan::HilbertPartition(plan) => self.replace_hilbert_serialize(plan), +// PhysicalPlan::Udf(plan) => self.replace_udf(plan), +// PhysicalPlan::AsyncFunction(plan) => self.replace_async_function(plan), +// PhysicalPlan::Duplicate(plan) => self.replace_duplicate(plan), +// PhysicalPlan::Shuffle(plan) => self.replace_shuffle(plan), +// PhysicalPlan::ChunkFilter(plan) => self.replace_chunk_filter(plan), +// PhysicalPlan::ChunkEvalScalar(plan) => self.replace_chunk_eval_scalar(plan), +// PhysicalPlan::ChunkCastSchema(plan) => self.replace_chunk_cast_schema(plan), +// PhysicalPlan::ChunkFillAndReorder(plan) => self.replace_chunk_fill_and_reorder(plan), +// 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::BroadcastSource(plan) => self.replace_runtime_filter_source(plan), +// PhysicalPlan::BroadcastSink(plan) => self.replace_runtime_filter_sink(plan), +// } +// } +// +// fn replace_runtime_filter_source(&mut self, plan: &BroadcastSource) -> Result { +// Ok(PhysicalPlan::BroadcastSource(plan.clone())) +// } +// +// fn replace_runtime_filter_sink(&mut self, plan: &BroadcastSink) -> Result { +// let input = self.replace(&plan.input)?; +// Ok(Box::new(BroadcastSink { +// broadcast_id: plan.broadcast_id, +// input: Box::new(input), +// meta: PhysicalPlanMeta::new("BroadcastSink"), +// })) +// } +// +// fn replace_recluster(&mut self, plan: &Recluster) -> Result { +// Ok(PhysicalPlan::Recluster(Box::new(plan.clone()))) +// } +// +// fn replace_hilbert_serialize(&mut self, plan: &HilbertPartition) -> Result { +// let input = self.replace(&plan.input)?; +// Ok(PhysicalPlan::HilbertPartition(Box::new(HilbertPartition { +// input: Box::new(input), +// ..plan.clone() +// }))) +// } +// +// fn replace_table_scan(&mut self, plan: &TableScan) -> Result { +// Ok(PhysicalPlan::TableScan(plan.clone())) +// } +// +// fn replace_recursive_cte_scan(&mut self, plan: &RecursiveCteScan) -> Result { +// Ok(PhysicalPlan::RecursiveCteScan(plan.clone())) +// } +// +// fn replace_constant_table_scan(&mut self, plan: &ConstantTableScan) -> Result { +// Ok(PhysicalPlan::ConstantTableScan(plan.clone())) +// } +// +// fn replace_expression_scan(&mut self, plan: &ExpressionScan) -> Result { +// Ok(PhysicalPlan::ExpressionScan(plan.clone())) +// } +// +// fn replace_cache_scan(&mut self, plan: &CacheScan) -> Result { +// Ok(PhysicalPlan::CacheScan(plan.clone())) +// } +// +// fn replace_filter(&mut self, plan: &Filter) -> Result { +// let input = self.replace(&plan.input)?; +// +// Ok(Box::new(Filter { +// projections: plan.projections.clone(), +// input: Box::new(input), +// predicates: plan.predicates.clone(), +// stat_info: plan.stat_info.clone(), +// meta: PhysicalPlanMeta::new("Filter"), +// })) +// } +// +// fn replace_eval_scalar(&mut self, plan: &EvalScalar) -> Result { +// let input = self.replace(&plan.input)?; +// +// Ok(PhysicalPlan::EvalScalar(EvalScalar { +// plan_id: plan.plan_id, +// projections: plan.projections.clone(), +// input: Box::new(input), +// exprs: plan.exprs.clone(), +// stat_info: plan.stat_info.clone(), +// })) +// } +// +// fn replace_aggregate_expand(&mut self, plan: &AggregateExpand) -> Result { +// let input = self.replace(&plan.input)?; +// +// Ok(PhysicalPlan::AggregateExpand(AggregateExpand { +// plan_id: plan.plan_id, +// input: Box::new(input), +// group_bys: plan.group_bys.clone(), +// grouping_sets: plan.grouping_sets.clone(), +// stat_info: plan.stat_info.clone(), +// })) +// } +// +// fn replace_aggregate_partial(&mut self, plan: &AggregatePartial) -> Result { +// let input = self.replace(&plan.input)?; +// +// Ok(PhysicalPlan::AggregatePartial(AggregatePartial { +// plan_id: plan.plan_id, +// input: Box::new(input), +// enable_experimental_aggregate_hashtable: plan.enable_experimental_aggregate_hashtable, +// group_by: plan.group_by.clone(), +// group_by_display: plan.group_by_display.clone(), +// agg_funcs: plan.agg_funcs.clone(), +// stat_info: plan.stat_info.clone(), +// rank_limit: plan.rank_limit.clone(), +// })) +// } +// +// fn replace_aggregate_final(&mut self, plan: &AggregateFinal) -> Result { +// let input = self.replace(&plan.input)?; +// +// Ok(PhysicalPlan::AggregateFinal(AggregateFinal { +// plan_id: plan.plan_id, +// input: Box::new(input), +// before_group_by_schema: plan.before_group_by_schema.clone(), +// group_by: plan.group_by.clone(), +// agg_funcs: plan.agg_funcs.clone(), +// group_by_display: plan.group_by_display.clone(), +// stat_info: plan.stat_info.clone(), +// })) +// } +// +// fn replace_window(&mut self, plan: &Window) -> Result { +// let input = self.replace(&plan.input)?; +// +// Ok(PhysicalPlan::Window(Window { +// plan_id: plan.plan_id, +// index: plan.index, +// input: Box::new(input), +// func: plan.func.clone(), +// partition_by: plan.partition_by.clone(), +// order_by: plan.order_by.clone(), +// window_frame: plan.window_frame.clone(), +// limit: plan.limit, +// })) +// } +// +// fn replace_window_partition(&mut self, plan: &WindowPartition) -> Result { +// let input = self.replace(&plan.input)?; +// +// Ok(PhysicalPlan::WindowPartition(WindowPartition { +// plan_id: plan.plan_id, +// input: Box::new(input), +// partition_by: plan.partition_by.clone(), +// order_by: plan.order_by.clone(), +// after_exchange: plan.after_exchange, +// top_n: plan.top_n.clone(), +// stat_info: plan.stat_info.clone(), +// })) +// } +// +// fn replace_hash_join(&mut self, plan: &HashJoin) -> Result { +// let build = self.replace(&plan.build)?; +// let probe = self.replace(&plan.probe)?; +// +// Ok(PhysicalPlan::HashJoin(HashJoin { +// plan_id: plan.plan_id, +// projections: plan.projections.clone(), +// probe_projections: plan.probe_projections.clone(), +// build_projections: plan.build_projections.clone(), +// build: Box::new(build), +// probe: Box::new(probe), +// build_keys: plan.build_keys.clone(), +// probe_keys: plan.probe_keys.clone(), +// is_null_equal: plan.is_null_equal.clone(), +// non_equi_conditions: plan.non_equi_conditions.clone(), +// join_type: plan.join_type.clone(), +// marker_index: plan.marker_index, +// from_correlated_subquery: plan.from_correlated_subquery, +// probe_to_build: plan.probe_to_build.clone(), +// output_schema: plan.output_schema.clone(), +// need_hold_hash_table: plan.need_hold_hash_table, +// stat_info: plan.stat_info.clone(), +// runtime_filter: plan.runtime_filter.clone(), +// single_to_inner: plan.single_to_inner.clone(), +// build_side_cache_info: plan.build_side_cache_info.clone(), +// broadcast_id: plan.broadcast_id, +// })) +// } +// +// fn replace_range_join(&mut self, plan: &RangeJoin) -> Result { +// let left = self.replace(&plan.left)?; +// let right = self.replace(&plan.right)?; +// +// Ok(PhysicalPlan::RangeJoin(RangeJoin { +// plan_id: plan.plan_id, +// left: Box::new(left), +// right: Box::new(right), +// conditions: plan.conditions.clone(), +// other_conditions: plan.other_conditions.clone(), +// join_type: plan.join_type.clone(), +// range_join_type: plan.range_join_type.clone(), +// output_schema: plan.output_schema.clone(), +// stat_info: plan.stat_info.clone(), +// })) +// } +// +// fn replace_sort(&mut self, plan: &Sort) -> Result { +// let input = self.replace(&plan.input)?; +// +// Ok(PhysicalPlan::Sort(Sort { +// plan_id: plan.plan_id, +// input: Box::new(input), +// order_by: plan.order_by.clone(), +// limit: plan.limit, +// after_exchange: plan.after_exchange, +// pre_projection: plan.pre_projection.clone(), +// stat_info: plan.stat_info.clone(), +// })) +// } +// +// fn replace_limit(&mut self, plan: &Limit) -> Result { +// let input = self.replace(&plan.input)?; +// +// Ok(PhysicalPlan::Limit(Limit { +// plan_id: plan.plan_id, +// input: Box::new(input), +// limit: plan.limit, +// offset: plan.offset, +// stat_info: plan.stat_info.clone(), +// })) +// } +// +// fn replace_row_fetch(&mut self, plan: &RowFetch) -> Result { +// let input = self.replace(&plan.input)?; +// +// Ok(PhysicalPlan::RowFetch(RowFetch { +// plan_id: plan.plan_id, +// input: Box::new(input), +// source: plan.source.clone(), +// row_id_col_offset: plan.row_id_col_offset, +// cols_to_fetch: plan.cols_to_fetch.clone(), +// fetched_fields: plan.fetched_fields.clone(), +// stat_info: plan.stat_info.clone(), +// need_wrap_nullable: plan.need_wrap_nullable, +// })) +// } +// +// fn replace_exchange(&mut self, plan: &Exchange) -> Result { +// let input = self.replace(&plan.input)?; +// +// Ok(PhysicalPlan::Exchange(Exchange { +// plan_id: plan.plan_id, +// input: Box::new(input), +// kind: plan.kind.clone(), +// keys: plan.keys.clone(), +// ignore_exchange: plan.ignore_exchange, +// allow_adjust_parallelism: plan.allow_adjust_parallelism, +// })) +// } +// +// fn replace_exchange_source(&mut self, plan: &ExchangeSource) -> Result { +// Ok(PhysicalPlan::ExchangeSource(plan.clone())) +// } +// +// fn replace_exchange_sink(&mut self, plan: &ExchangeSink) -> Result { +// let input = self.replace(&plan.input)?; +// +// Ok(PhysicalPlan::ExchangeSink(ExchangeSink { +// // TODO(leiysky): we reuse the plan id of the Exchange node here, +// // should generate a new one. +// plan_id: plan.plan_id, +// +// input: Box::new(input), +// schema: plan.schema.clone(), +// kind: plan.kind.clone(), +// keys: plan.keys.clone(), +// destination_fragment_id: plan.destination_fragment_id, +// query_id: plan.query_id.clone(), +// ignore_exchange: plan.ignore_exchange, +// allow_adjust_parallelism: plan.allow_adjust_parallelism, +// })) +// } +// +// fn replace_union(&mut self, plan: &UnionAll) -> Result { +// let left = self.replace(&plan.left)?; +// let right = self.replace(&plan.right)?; +// Ok(PhysicalPlan::UnionAll(UnionAll { +// plan_id: plan.plan_id, +// left: Box::new(left), +// right: Box::new(right), +// left_outputs: plan.left_outputs.clone(), +// right_outputs: plan.right_outputs.clone(), +// schema: plan.schema.clone(), +// stat_info: plan.stat_info.clone(), +// cte_scan_names: plan.cte_scan_names.clone(), +// })) +// } +// +// fn replace_copy_into_table(&mut self, plan: &CopyIntoTable) -> Result { +// match &plan.source { +// CopyIntoTableSource::Stage(_) => { +// Ok(PhysicalPlan::CopyIntoTable(Box::new(plan.clone()))) +// } +// CopyIntoTableSource::Query(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_copy_into_location(&mut self, plan: &CopyIntoLocation) -> Result { +// let input = self.replace(&plan.input)?; +// +// Ok(PhysicalPlan::CopyIntoLocation(Box::new(CopyIntoLocation { +// plan_id: plan.plan_id, +// input: Box::new(input), +// project_columns: plan.project_columns.clone(), +// input_data_schema: plan.input_data_schema.clone(), +// input_table_schema: plan.input_table_schema.clone(), +// info: plan.info.clone(), +// }))) +// } +// +// fn replace_insert_select(&mut self, plan: &DistributedInsertSelect) -> Result { +// let input = self.replace(&plan.input)?; +// +// Ok(PhysicalPlan::DistributedInsertSelect(Box::new( +// DistributedInsertSelect { +// input: Box::new(input), +// ..plan.clone() +// }, +// ))) +// } +// +// fn replace_compact_source(&mut self, plan: &CompactSource) -> Result { +// Ok(PhysicalPlan::CompactSource(Box::new(plan.clone()))) +// } +// +// fn replace_commit_sink(&mut self, plan: &CommitSink) -> Result { +// let input = self.replace(&plan.input)?; +// Ok(PhysicalPlan::CommitSink(Box::new(CommitSink { +// input: Box::new(input), +// ..plan.clone() +// }))) +// } +// +// 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( +// ReplaceDeduplicate { +// input: Box::new(input), +// ..plan.clone() +// }, +// ))) +// } +// +// fn replace_replace_into(&mut self, plan: &ReplaceInto) -> Result { +// let input = self.replace(&plan.input)?; +// Ok(PhysicalPlan::ReplaceInto(Box::new(ReplaceInto { +// input: Box::new(input), +// ..plan.clone() +// }))) +// } +// +// fn replace_mutation_source(&mut self, plan: &MutationSource) -> Result { +// Ok(PhysicalPlan::MutationSource(plan.clone())) +// } +// +// fn replace_column_mutation(&mut self, plan: &ColumnMutation) -> Result { +// let input = self.replace(&plan.input)?; +// Ok(PhysicalPlan::ColumnMutation(ColumnMutation { +// input: Box::new(input), +// ..plan.clone() +// })) +// } +// +// fn replace_mutation(&mut self, plan: &Mutation) -> Result { +// let input = self.replace(&plan.input)?; +// Ok(PhysicalPlan::Mutation(Box::new(Mutation { +// input: Box::new(input), +// ..plan.clone() +// }))) +// } +// +// fn replace_mutation_split(&mut self, plan: &MutationSplit) -> Result { +// let input = self.replace(&plan.input)?; +// Ok(PhysicalPlan::MutationSplit(Box::new(MutationSplit { +// input: Box::new(input), +// ..plan.clone() +// }))) +// } +// +// fn replace_mutation_manipulate(&mut self, plan: &MutationManipulate) -> Result { +// let input = self.replace(&plan.input)?; +// Ok(PhysicalPlan::MutationManipulate(Box::new( +// MutationManipulate { +// input: Box::new(input), +// ..plan.clone() +// }, +// ))) +// } +// +// fn replace_mutation_organize(&mut self, plan: &MutationOrganize) -> Result { +// let input = self.replace(&plan.input)?; +// Ok(PhysicalPlan::MutationOrganize(Box::new(MutationOrganize { +// input: Box::new(input), +// ..plan.clone() +// }))) +// } +// +// fn replace_add_stream_column(&mut self, plan: &AddStreamColumn) -> Result { +// let input = self.replace(&plan.input)?; +// Ok(PhysicalPlan::AddStreamColumn(Box::new(AddStreamColumn { +// input: Box::new(input), +// ..plan.clone() +// }))) +// } +// +// fn replace_project_set(&mut self, plan: &ProjectSet) -> Result { +// let input = self.replace(&plan.input)?; +// Ok(PhysicalPlan::ProjectSet(ProjectSet { +// plan_id: plan.plan_id, +// input: Box::new(input), +// srf_exprs: plan.srf_exprs.clone(), +// projections: plan.projections.clone(), +// stat_info: plan.stat_info.clone(), +// })) +// } +// +// fn replace_udf(&mut self, plan: &Udf) -> Result { +// let input = self.replace(&plan.input)?; +// Ok(PhysicalPlan::Udf(Udf { +// plan_id: plan.plan_id, +// input: Box::new(input), +// udf_funcs: plan.udf_funcs.clone(), +// stat_info: plan.stat_info.clone(), +// script_udf: plan.script_udf, +// })) +// } +// +// fn replace_async_function(&mut self, plan: &AsyncFunction) -> Result { +// let input = self.replace(&plan.input)?; +// Ok(PhysicalPlan::AsyncFunction(AsyncFunction { +// plan_id: plan.plan_id, +// input: Box::new(input), +// async_func_descs: plan.async_func_descs.clone(), +// stat_info: plan.stat_info.clone(), +// })) +// } +// +// fn replace_duplicate(&mut self, plan: &Duplicate) -> Result { +// let input = self.replace(&plan.input)?; +// Ok(PhysicalPlan::Duplicate(Box::new(Duplicate { +// input: Box::new(input), +// ..plan.clone() +// }))) +// } +// +// fn replace_shuffle(&mut self, plan: &Shuffle) -> Result { +// let input = self.replace(&plan.input)?; +// Ok(PhysicalPlan::Shuffle(Box::new(Shuffle { +// input: Box::new(input), +// ..plan.clone() +// }))) +// } +// +// fn replace_chunk_filter(&mut self, plan: &ChunkFilter) -> Result { +// let input = self.replace(&plan.input)?; +// Ok(PhysicalPlan::ChunkFilter(Box::new(ChunkFilter { +// input: Box::new(input), +// predicates: plan.predicates.clone(), +// ..plan.clone() +// }))) +// } +// +// fn replace_chunk_eval_scalar(&mut self, plan: &ChunkEvalScalar) -> Result { +// let input = self.replace(&plan.input)?; +// Ok(PhysicalPlan::ChunkEvalScalar(Box::new(ChunkEvalScalar { +// input: Box::new(input), +// ..plan.clone() +// }))) +// } +// +// fn replace_chunk_cast_schema(&mut self, plan: &ChunkCastSchema) -> Result { +// let input = self.replace(&plan.input)?; +// Ok(PhysicalPlan::ChunkCastSchema(Box::new(ChunkCastSchema { +// input: Box::new(input), +// cast_schemas: plan.cast_schemas.clone(), +// ..plan.clone() +// }))) +// } +// +// fn replace_chunk_fill_and_reorder( +// &mut self, +// plan: &ChunkFillAndReorder, +// ) -> Result { +// let input = self.replace(&plan.input)?; +// Ok(PhysicalPlan::ChunkFillAndReorder(Box::new( +// ChunkFillAndReorder { +// input: Box::new(input), +// ..plan.clone() +// }, +// ))) +// } +// +// fn replace_chunk_append_data(&mut self, plan: &ChunkAppendData) -> Result { +// let input = self.replace(&plan.input)?; +// Ok(PhysicalPlan::ChunkAppendData(Box::new(ChunkAppendData { +// input: Box::new(input), +// ..plan.clone() +// }))) +// } +// +// fn replace_chunk_merge(&mut self, plan: &ChunkMerge) -> Result { +// let input = self.replace(&plan.input)?; +// Ok(PhysicalPlan::ChunkMerge(Box::new(ChunkMerge { +// input: Box::new(input), +// ..plan.clone() +// }))) +// } +// +// fn replace_chunk_commit_insert(&mut self, plan: &ChunkCommitInsert) -> Result { +// let input = self.replace(&plan.input)?; +// Ok(PhysicalPlan::ChunkCommitInsert(Box::new( +// ChunkCommitInsert { +// input: Box::new(input), +// ..plan.clone() +// }, +// ))) +// } +// } + +// impl PhysicalPlan { +// pub fn traverse<'a, 'b>( +// plan: &'a PhysicalPlan, +// pre_visit: &'b mut dyn FnMut(&'a PhysicalPlan) -> bool, +// visit: &'b mut dyn FnMut(&'a PhysicalPlan), +// post_visit: &'b mut dyn FnMut(&'a PhysicalPlan), +// ) { +// if pre_visit(plan) { +// visit(plan); +// +// for child in plan.children() { +// Self::traverse(&child, pre_visit, visit, post_visit); +// } +// +// post_visit(plan); +// } +// } +// } diff --git a/src/query/service/src/physical_plans/physical_project_set.rs b/src/query/service/src/physical_plans/physical_project_set.rs new file mode 100644 index 0000000000000..bfcfe9a63e9ee --- /dev/null +++ b/src/query/service/src/physical_plans/physical_project_set.rs @@ -0,0 +1,212 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::any::Any; + +use databend_common_ast::ast::FormatTreeNode; +use databend_common_catalog::plan::DataSourcePlan; +use databend_common_exception::Result; +use databend_common_expression::ConstantFolder; +use databend_common_expression::DataField; +use databend_common_expression::DataSchemaRef; +use databend_common_expression::DataSchemaRefExt; +use databend_common_expression::RemoteExpr; +use databend_common_functions::BUILTIN_FUNCTIONS; +use databend_common_pipeline_core::processors::ProcessorPtr; +use databend_common_sql::optimizer::ir::SExpr; +use databend_common_sql::ColumnSet; +use databend_common_sql::IndexType; +use databend_common_sql::TypeCheck; +use itertools::Itertools; + +use crate::physical_plans::explain::PlanStatsInfo; +use crate::physical_plans::format::format_output_columns; +use crate::physical_plans::format::plan_stats_info_to_format_tree; +use crate::physical_plans::format::FormatContext; +use crate::physical_plans::physical_plan::IPhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlanMeta; +use crate::physical_plans::PhysicalPlanBuilder; +use crate::pipelines::processors::transforms::TransformSRF; +use crate::pipelines::PipelineBuilder; + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct ProjectSet { + meta: PhysicalPlanMeta, + pub projections: ColumnSet, + pub input: PhysicalPlan, + pub srf_exprs: Vec<(RemoteExpr, IndexType)>, + + // Only used for explain + pub stat_info: Option, +} + +#[typetag::serde] +impl IPhysicalPlan for ProjectSet { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn output_schema(&self) -> Result { + let input_schema = self.input.output_schema()?; + let mut fields = Vec::with_capacity(input_schema.num_fields() + self.srf_exprs.len()); + for (i, field) in input_schema.fields().iter().enumerate() { + if self.projections.contains(&i) { + fields.push(field.clone()); + } + } + fields.extend(self.srf_exprs.iter().map(|(srf, index)| { + DataField::new( + &index.to_string(), + srf.as_expr(&BUILTIN_FUNCTIONS).data_type().clone(), + ) + })); + Ok(DataSchemaRefExt::create(fields)) + } + + fn children<'a>(&'a self) -> Box + 'a> { + Box::new(std::iter::once(&self.input)) + } + + fn children_mut<'a>(&'a mut self) -> Box + 'a> { + Box::new(std::iter::once(&mut self.input)) + } + + fn to_format_node( + &self, + ctx: &mut FormatContext<'_>, + children: Vec>, + ) -> Result> { + let mut node_children = vec![FormatTreeNode::new(format!( + "output columns: [{}]", + format_output_columns(self.output_schema()?, ctx.metadata, true) + ))]; + + if let Some(info) = &self.stat_info { + let items = plan_stats_info_to_format_tree(info); + node_children.extend(items); + } + + node_children.extend(vec![FormatTreeNode::new(format!( + "set returning functions: {}", + self.srf_exprs + .iter() + .map(|(expr, _)| expr.clone().as_expr(&BUILTIN_FUNCTIONS).sql_display()) + .collect::>() + .join(", ") + ))]); + + node_children.extend(children); + Ok(FormatTreeNode::with_children( + "ProjectSet".to_string(), + node_children, + )) + } + + #[recursive::recursive] + fn try_find_single_data_source(&self) -> Option<&DataSourcePlan> { + self.input.try_find_single_data_source() + } + + fn get_desc(&self) -> Result { + Ok(self + .srf_exprs + .iter() + .map(|(x, _)| x.as_expr(&BUILTIN_FUNCTIONS).sql_display()) + .join(", ")) + } + + fn derive(&self, mut children: Vec) -> PhysicalPlan { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + self.input.build_pipeline(builder)?; + + let srf_exprs = self + .srf_exprs + .iter() + .map(|(expr, _)| expr.as_expr(&BUILTIN_FUNCTIONS)) + .collect::>(); + let max_block_size = builder.settings.get_max_block_size()? as usize; + + builder.main_pipeline.add_transform(|input, output| { + Ok(ProcessorPtr::create(TransformSRF::try_create( + input, + output, + builder.func_ctx.clone(), + self.projections.clone(), + srf_exprs.clone(), + max_block_size, + ))) + }) + } +} + +impl PhysicalPlanBuilder { + pub async fn build_project_set( + &mut self, + s_expr: &SExpr, + project_set: &databend_common_sql::plans::ProjectSet, + mut required: ColumnSet, + stat_info: PlanStatsInfo, + ) -> Result { + // 1. Prune unused Columns. + let column_projections = required.clone().into_iter().collect::>(); + for s in project_set.srfs.iter() { + required.extend(s.scalar.used_columns().iter().copied()); + } + + // 2. Build physical plan. + let input = self.build(s_expr.child(0)?, required).await?; + let input_schema = input.output_schema()?; + let srf_exprs = project_set + .srfs + .iter() + .map(|item| { + let expr = item + .scalar + .type_check(input_schema.as_ref())? + .project_column_ref(|index| input_schema.index_of(&index.to_string()).unwrap()); + let (expr, _) = ConstantFolder::fold(&expr, &self.func_ctx, &BUILTIN_FUNCTIONS); + Ok((expr.as_remote_expr(), item.index)) + }) + .collect::>>()?; + + let mut projections = ColumnSet::new(); + for column in column_projections.iter() { + if let Some((index, _)) = input_schema.column_with_name(&column.to_string()) { + projections.insert(index); + } + } + + Ok(Box::new(ProjectSet { + input, + meta: PhysicalPlanMeta::new("Unnest"), + srf_exprs, + projections, + stat_info: Some(stat_info), + })) + } +} diff --git a/src/query/service/src/physical_plans/physical_r_cte_scan.rs b/src/query/service/src/physical_plans/physical_r_cte_scan.rs new file mode 100644 index 0000000000000..0d5c7a729ff6c --- /dev/null +++ b/src/query/service/src/physical_plans/physical_r_cte_scan.rs @@ -0,0 +1,95 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::any::Any; +use std::fmt::Display; + +use databend_common_exception::Result; +use databend_common_expression::DataSchemaRef; +use databend_common_expression::DataSchemaRefExt; + +use crate::physical_plans::explain::PlanStatsInfo; +use crate::physical_plans::physical_plan::IPhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlanMeta; +use crate::physical_plans::PhysicalPlanBuilder; +use crate::pipelines::processors::transforms::TransformRecursiveCteScan; +use crate::pipelines::PipelineBuilder; + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct RecursiveCteScan { + meta: PhysicalPlanMeta, + pub output_schema: DataSchemaRef, + pub table_name: String, + pub stat: PlanStatsInfo, +} + +#[typetag::serde] +impl IPhysicalPlan for RecursiveCteScan { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn output_schema(&self) -> Result { + Ok(self.output_schema.clone()) + } + + fn derive(&self, children: Vec) -> PhysicalPlan { + assert!(children.is_empty()); + Box::new(self.clone()) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + let max_threads = builder.settings.get_max_threads()?; + builder.main_pipeline.add_source( + |output_port| { + TransformRecursiveCteScan::create( + builder.ctx.clone(), + output_port.clone(), + self.table_name.clone(), + ) + }, + 1, + )?; + builder.main_pipeline.resize(max_threads as usize, true) + } +} + +impl PhysicalPlanBuilder { + pub async fn build_recursive_cte_scan( + &mut self, + recursive_cte_scan: &databend_common_sql::plans::RecursiveCteScan, + stat_info: PlanStatsInfo, + ) -> Result { + Ok(Box::new(RecursiveCteScan { + meta: PhysicalPlanMeta::new("RecursiveCteScan"), + output_schema: DataSchemaRefExt::create(recursive_cte_scan.fields.clone()), + table_name: recursive_cte_scan.table_name.clone(), + stat: stat_info, + })) + } +} + +impl Display for RecursiveCteScan { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + write!(f, "RecursiveCTEScan") + } +} diff --git a/src/query/sql/src/executor/physical_plans/physical_range_join.rs b/src/query/service/src/physical_plans/physical_range_join.rs similarity index 52% rename from src/query/sql/src/executor/physical_plans/physical_range_join.rs rename to src/query/service/src/physical_plans/physical_range_join.rs index 94317540f2f99..5834f652f4eb5 100644 --- a/src/query/sql/src/executor/physical_plans/physical_range_join.rs +++ b/src/query/service/src/physical_plans/physical_range_join.rs @@ -12,8 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::any::Any; use std::sync::Arc; +use databend_common_ast::ast::FormatTreeNode; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::type_check::common_super_type; @@ -22,27 +24,37 @@ use databend_common_expression::DataSchemaRef; use databend_common_expression::DataSchemaRefExt; use databend_common_expression::RemoteExpr; use databend_common_functions::BUILTIN_FUNCTIONS; +use databend_common_pipeline_core::processors::ProcessorPtr; +use databend_common_pipeline_sinks::Sinker; +use databend_common_sql::binder::wrap_cast; +use databend_common_sql::binder::JoinPredicate; +use databend_common_sql::optimizer::ir::RelExpr; +use databend_common_sql::optimizer::ir::RelationalProperty; +use databend_common_sql::optimizer::ir::SExpr; +use databend_common_sql::plans::Join; +use databend_common_sql::plans::JoinType; +use databend_common_sql::ColumnSet; +use databend_common_sql::ScalarExpr; +use databend_common_sql::TypeCheck; -use crate::binder::wrap_cast; -use crate::binder::JoinPredicate; -use crate::executor::explain::PlanStatsInfo; -use crate::executor::PhysicalPlan; -use crate::executor::PhysicalPlanBuilder; -use crate::optimizer::ir::RelExpr; -use crate::optimizer::ir::RelationalProperty; -use crate::optimizer::ir::SExpr; -use crate::plans::Join; -use crate::plans::JoinType; -use crate::ColumnSet; -use crate::ScalarExpr; -use crate::TypeCheck; +use crate::physical_plans::explain::PlanStatsInfo; +use crate::physical_plans::format::format_output_columns; +use crate::physical_plans::format::plan_stats_info_to_format_tree; +use crate::physical_plans::format::FormatContext; +use crate::physical_plans::physical_plan::IPhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlanMeta; +use crate::physical_plans::PhysicalPlanBuilder; +use crate::pipelines::processors::transforms::range_join::RangeJoinState; +use crate::pipelines::processors::transforms::range_join::TransformRangeJoinLeft; +use crate::pipelines::processors::transforms::range_join::TransformRangeJoinRight; +use crate::pipelines::PipelineBuilder; #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct RangeJoin { - // A unique id of operator in a `PhysicalPlan` tree, only used for display. - pub plan_id: u32, - pub left: Box, - pub right: Box, + pub meta: PhysicalPlanMeta, + pub left: PhysicalPlan, + pub right: PhysicalPlan, // The first two conditions: (>, >=, <, <=) // Condition's left/right side only contains one table's column pub conditions: Vec, @@ -57,10 +69,166 @@ pub struct RangeJoin { pub stat_info: Option, } -impl RangeJoin { - pub fn output_schema(&self) -> Result { +#[typetag::serde] +impl IPhysicalPlan for RangeJoin { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn output_schema(&self) -> Result { Ok(self.output_schema.clone()) } + + fn children<'a>(&'a self) -> Box + 'a> { + Box::new(std::iter::once(&self.left).chain(std::iter::once(&self.right))) + } + + fn children_mut<'a>(&'a mut self) -> Box + 'a> { + Box::new(std::iter::once(&mut self.left).chain(std::iter::once(&mut self.right))) + } + + fn to_format_node( + &self, + ctx: &mut FormatContext<'_>, + mut children: Vec>, + ) -> Result> { + let range_join_conditions = self + .conditions + .iter() + .map(|condition| { + let left = condition + .left_expr + .as_expr(&BUILTIN_FUNCTIONS) + .sql_display(); + let right = condition + .right_expr + .as_expr(&BUILTIN_FUNCTIONS) + .sql_display(); + format!("{left} {:?} {right}", condition.operator) + }) + .collect::>() + .join(", "); + + let other_conditions = self + .other_conditions + .iter() + .map(|filter| filter.as_expr(&BUILTIN_FUNCTIONS).sql_display()) + .collect::>() + .join(", "); + + assert_eq!(children.len(), 2); + children[0].payload = format!("{}(Left)", children[0].payload); + children[1].payload = format!("{}(Right)", children[1].payload); + + let mut node_children = vec![ + FormatTreeNode::new(format!( + "output columns: [{}]", + format_output_columns(self.output_schema()?, ctx.metadata, true) + )), + FormatTreeNode::new(format!("join type: {}", self.join_type)), + FormatTreeNode::new(format!("range join conditions: [{range_join_conditions}]")), + FormatTreeNode::new(format!("other conditions: [{other_conditions}]")), + ]; + + if let Some(info) = &self.stat_info { + let items = plan_stats_info_to_format_tree(info); + node_children.extend(items); + } + + node_children.extend(children); + Ok(FormatTreeNode::with_children( + match self.range_join_type { + RangeJoinType::IEJoin => "IEJoin".to_string(), + RangeJoinType::Merge => "MergeJoin".to_string(), + }, + node_children, + )) + } + + fn get_desc(&self) -> Result { + let mut condition = self + .conditions + .iter() + .map(|condition| { + let left = condition + .left_expr + .as_expr(&BUILTIN_FUNCTIONS) + .sql_display(); + let right = condition + .right_expr + .as_expr(&BUILTIN_FUNCTIONS) + .sql_display(); + format!("{left} {:?} {right}", condition.operator) + }) + .collect::>(); + + condition.extend( + self.other_conditions + .iter() + .map(|x| x.as_expr(&BUILTIN_FUNCTIONS).sql_display()), + ); + + Ok(condition.join(" AND ")) + } + + fn derive(&self, mut children: Vec) -> PhysicalPlan { + let mut new_range_join = self.clone(); + assert_eq!(children.len(), 2); + new_range_join.right = children.pop().unwrap(); + new_range_join.left = children.pop().unwrap(); + Box::new(new_range_join) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + let state = Arc::new(RangeJoinState::new(builder.ctx.clone(), self)); + self.build_right(state.clone(), builder)?; + self.build_left(state, builder) + } +} + +impl RangeJoin { + // Build the left-side pipeline for Range Join + fn build_left(&self, state: Arc, builder: &mut PipelineBuilder) -> Result<()> { + self.left.build_pipeline(builder)?; + + let max_threads = builder.settings.get_max_threads()? as usize; + builder.main_pipeline.try_resize(max_threads)?; + builder.main_pipeline.add_transform(|input, output| { + Ok(ProcessorPtr::create(TransformRangeJoinLeft::create( + input, + output, + state.clone(), + ))) + }) + } + + // Build the right-side pipeline for Range Join + fn build_right(&self, state: Arc, builder: &mut PipelineBuilder) -> Result<()> { + let right_side_builder = builder.create_sub_pipeline_builder(); + + let mut right_res = right_side_builder.finalize(&self.right)?; + right_res.main_pipeline.add_sink(|input| { + Ok(ProcessorPtr::create( + Sinker::::create( + input, + TransformRangeJoinRight::create(state.clone()), + ), + )) + })?; + + builder + .pipelines + .push(right_res.main_pipeline.finalize(None)); + builder.pipelines.extend(right_res.sources_pipelines); + Ok(()) + } } #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] @@ -122,10 +290,10 @@ impl PhysicalPlanBuilder { .collect::>(), ); - Ok(PhysicalPlan::RangeJoin(RangeJoin { - plan_id: 0, + Ok(Box::new(RangeJoin { left: left_side, right: right_side, + meta: PhysicalPlanMeta::new("RangeJoin"), conditions: range_conditions .iter() .map(|scalar| { diff --git a/src/query/service/src/pipelines/builders/builder_recluster.rs b/src/query/service/src/physical_plans/physical_recluster.rs similarity index 56% rename from src/query/service/src/pipelines/builders/builder_recluster.rs rename to src/query/service/src/physical_plans/physical_recluster.rs index 05d2d63dd55d7..c6651b3f6e3e3 100644 --- a/src/query/service/src/pipelines/builders/builder_recluster.rs +++ b/src/query/service/src/physical_plans/physical_recluster.rs @@ -12,32 +12,78 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::any::Any; +use std::sync::atomic; +use std::sync::atomic::AtomicUsize; + use databend_common_catalog::plan::DataSourceInfo; use databend_common_catalog::plan::DataSourcePlan; +use databend_common_catalog::plan::ReclusterTask; +use databend_common_catalog::table::Table; +use databend_common_catalog::table_context::TableContext; use databend_common_exception::ErrorCode; use databend_common_exception::Result; +use databend_common_expression::DataSchemaRef; use databend_common_expression::DataSchemaRefExt; use databend_common_expression::SortColumnDescription; +use databend_common_io::constants::DEFAULT_BLOCK_BUFFER_SIZE; +use databend_common_meta_app::schema::TableInfo; use databend_common_metrics::storage::metrics_inc_recluster_block_bytes_to_read; use databend_common_metrics::storage::metrics_inc_recluster_block_nums_to_read; use databend_common_metrics::storage::metrics_inc_recluster_row_nums_to_read; +use databend_common_pipeline_core::processors::ProcessorPtr; use databend_common_pipeline_sources::EmptySource; -use databend_common_pipeline_transforms::processors::build_compact_block_no_split_pipeline; -use databend_common_pipeline_transforms::processors::TransformPipelineHelper; +use databend_common_pipeline_transforms::build_compact_block_no_split_pipeline; +use databend_common_pipeline_transforms::MemorySettings; +use databend_common_pipeline_transforms::TransformPipelineHelper; use databend_common_sql::evaluator::CompoundBlockOperator; use databend_common_sql::executor::physical_plans::MutationKind; -use databend_common_sql::executor::physical_plans::Recluster; use databend_common_sql::StreamContext; -use databend_common_storages_factory::Table; use databend_common_storages_fuse::operations::TransformSerializeBlock; +use databend_common_storages_fuse::statistics::ClusterStatsGenerator; use databend_common_storages_fuse::FuseTable; -use databend_common_storages_fuse::TableContext; +use databend_common_storages_fuse::FUSE_OPT_KEY_BLOCK_IN_MEM_SIZE_THRESHOLD; +use databend_storages_common_cache::TempDirManager; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; +use crate::physical_plans::physical_plan::IPhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlanMeta; use crate::pipelines::builders::SortPipelineBuilder; +use crate::pipelines::memory_settings::MemorySettingsExt; +use crate::pipelines::processors::transforms::CompactStrategy; +use crate::pipelines::processors::transforms::HilbertPartitionExchange; use crate::pipelines::processors::transforms::TransformAddStreamColumns; +use crate::pipelines::processors::transforms::TransformWindowPartitionCollect; use crate::pipelines::PipelineBuilder; +use crate::spillers::SpillerDiskConfig; + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct Recluster { + pub meta: PhysicalPlanMeta, + pub tasks: Vec, + pub table_info: TableInfo, + pub table_meta_timestamps: TableMetaTimestamps, +} + +#[typetag::serde] +impl IPhysicalPlan for Recluster { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn derive(&self, children: Vec) -> PhysicalPlan { + assert!(children.is_empty()); + Box::new(self.clone()) + } -impl PipelineBuilder { /// The flow of Pipeline is as follows: // ┌──────────┐ ┌───────────────┐ ┌─────────┐ // │FuseSource├────►│CompoundBlockOp├────►│SortMerge├────┐ @@ -58,16 +104,16 @@ impl PipelineBuilder { // │ ┌──────────────┐ │ // └───►│SerializeBlock├───┘ // └──────────────┘ - pub(crate) fn build_recluster(&mut self, recluster: &Recluster) -> Result<()> { - match recluster.tasks.len() { - 0 => self.main_pipeline.add_source(EmptySource::create, 1), + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + match self.tasks.len() { + 0 => builder.main_pipeline.add_source(EmptySource::create, 1), 1 => { - let table = self + let table = builder .ctx - .build_table_by_table_info(&recluster.table_info, None)?; + .build_table_by_table_info(&self.table_info, None)?; let table = FuseTable::try_from_table(table.as_ref())?; - let task = &recluster.tasks[0]; + let task = &self.tasks[0]; let recluster_block_nums = task.parts.len(); let block_thresholds = table.get_block_thresholds(); let table_info = table.get_table_info(); @@ -100,25 +146,32 @@ impl PipelineBuilder { ); } - self.ctx.set_partitions(plan.parts.clone())?; + builder.ctx.set_partitions(plan.parts.clone())?; - table.read_data(self.ctx.clone(), &plan, &mut self.main_pipeline, false)?; + table.read_data( + builder.ctx.clone(), + &plan, + &mut builder.main_pipeline, + false, + )?; let num_input_columns = schema.fields().len(); if table.change_tracking_enabled() { let stream_ctx = StreamContext::try_create( - self.ctx.get_function_context()?, + builder.ctx.get_function_context()?, schema, table_info.ident.seq, false, false, )?; - self.main_pipeline + + builder + .main_pipeline .add_transformer(|| TransformAddStreamColumns::new(stream_ctx.clone())); } let cluster_stats_gen = table.get_cluster_stats_gen( - self.ctx.clone(), + builder.ctx.clone(), task.level + 1, block_thresholds, None, @@ -126,7 +179,7 @@ impl PipelineBuilder { let operators = cluster_stats_gen.operators.clone(); if !operators.is_empty() { let func_ctx2 = cluster_stats_gen.func_ctx.clone(); - self.main_pipeline.add_transformer(move || { + builder.main_pipeline.add_transformer(move || { CompoundBlockOperator::new( operators.clone(), func_ctx2.clone(), @@ -156,33 +209,34 @@ impl PipelineBuilder { ); let sort_pipeline_builder = - SortPipelineBuilder::create(self.ctx.clone(), schema, sort_descs.into())? + SortPipelineBuilder::create(builder.ctx.clone(), schema, sort_descs.into())? .with_block_size_hit(sort_block_size) .remove_order_col_at_last(); // Todo(zhyass): Recluster will no longer perform sort in the near future. - sort_pipeline_builder.build_full_sort_pipeline(&mut self.main_pipeline)?; + sort_pipeline_builder.build_full_sort_pipeline(&mut builder.main_pipeline)?; // Compact after merge sort. - let max_threads = self.ctx.get_settings().get_max_threads()? as usize; + let max_threads = builder.ctx.get_settings().get_max_threads()? as usize; build_compact_block_no_split_pipeline( - &mut self.main_pipeline, + &mut builder.main_pipeline, block_thresholds, max_threads, )?; - self.main_pipeline - .add_transform(|transform_input_port, transform_output_port| { + builder.main_pipeline.add_transform( + |transform_input_port, transform_output_port| { let proc = TransformSerializeBlock::try_create( - self.ctx.clone(), + builder.ctx.clone(), transform_input_port, transform_output_port, table, cluster_stats_gen.clone(), MutationKind::Recluster, - recluster.table_meta_timestamps, + self.table_meta_timestamps, )?; proc.into_processor() - }) + }, + ) } _ => Err(ErrorCode::Internal( "A node can only execute one recluster task".to_string(), @@ -190,3 +244,112 @@ impl PipelineBuilder { } } } + +#[derive(serde::Serialize, serde::Deserialize, Clone, Debug)] +pub struct HilbertPartition { + pub meta: PhysicalPlanMeta, + pub input: PhysicalPlan, + pub table_info: TableInfo, + pub num_partitions: usize, + pub table_meta_timestamps: TableMetaTimestamps, + pub rows_per_block: usize, +} + +#[typetag::serde] +impl IPhysicalPlan for HilbertPartition { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn output_schema(&self) -> Result { + Ok(DataSchemaRef::default()) + } + + fn children<'a>(&'a self) -> Box + 'a> { + Box::new(std::iter::once(&self.input)) + } + + fn children_mut<'a>(&'a mut self) -> Box + 'a> { + Box::new(std::iter::once(&mut self.input)) + } + + fn derive(&self, mut children: Vec) -> PhysicalPlan { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + self.input.build_pipeline(builder)?; + + let num_processors = builder.main_pipeline.output_len(); + let table = builder + .ctx + .build_table_by_table_info(&self.table_info, None)?; + let table = FuseTable::try_from_table(table.as_ref())?; + + builder.main_pipeline.exchange( + num_processors, + HilbertPartitionExchange::create(self.num_partitions), + ); + + let settings = builder.settings.clone(); + let disk_bytes_limit = settings.get_window_partition_spilling_to_disk_bytes_limit()?; + let temp_dir_manager = TempDirManager::instance(); + + let enable_dio = settings.get_enable_dio()?; + let disk_spill = temp_dir_manager + .get_disk_spill_dir(disk_bytes_limit, &builder.ctx.get_id()) + .map(|temp_dir| SpillerDiskConfig::new(temp_dir, enable_dio)) + .transpose()?; + + let window_spill_settings = MemorySettings::from_window_settings(&builder.ctx)?; + let processor_id = AtomicUsize::new(0); + let max_bytes_per_block = std::cmp::min( + 4 * table.get_option( + FUSE_OPT_KEY_BLOCK_IN_MEM_SIZE_THRESHOLD, + DEFAULT_BLOCK_BUFFER_SIZE, + ), + 400 * 1024 * 1024, + ); + builder.main_pipeline.add_transform(|input, output| { + Ok(ProcessorPtr::create(Box::new( + TransformWindowPartitionCollect::new( + builder.ctx.clone(), + input, + output, + &settings, + processor_id.fetch_add(1, atomic::Ordering::AcqRel), + num_processors, + self.num_partitions, + window_spill_settings.clone(), + disk_spill.clone(), + CompactStrategy::new(self.rows_per_block, max_bytes_per_block), + )?, + ))) + })?; + + builder + .main_pipeline + .add_transform(|transform_input_port, transform_output_port| { + let proc = TransformSerializeBlock::try_create( + builder.ctx.clone(), + transform_input_port, + transform_output_port, + table, + ClusterStatsGenerator::default(), + MutationKind::Recluster, + self.table_meta_timestamps, + )?; + proc.into_processor() + }) + } +} diff --git a/src/query/sql/src/executor/physical_plans/physical_refresh_index.rs b/src/query/service/src/physical_plans/physical_refresh_index.rs similarity index 93% rename from src/query/sql/src/executor/physical_plans/physical_refresh_index.rs rename to src/query/service/src/physical_plans/physical_refresh_index.rs index 590978c962f6b..d855c92c76038 100644 --- a/src/query/sql/src/executor/physical_plans/physical_refresh_index.rs +++ b/src/query/service/src/physical_plans/physical_refresh_index.rs @@ -21,13 +21,13 @@ use databend_common_expression::DataField; use databend_common_expression::DataSchemaRef; use databend_common_expression::DataSchemaRefExt; use databend_common_meta_app::schema::TableInfo; +use databend_common_sql::ColumnBinding; -use crate::executor::PhysicalPlan; -use crate::ColumnBinding; +use crate::physical_plans::PhysicalPlan; #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct RefreshIndex { - pub input: Box, + pub input: PhysicalPlan, pub index_id: u64, pub table_info: TableInfo, pub select_schema: DataSchemaRef, diff --git a/src/query/service/src/physical_plans/physical_replace_async_source.rs b/src/query/service/src/physical_plans/physical_replace_async_source.rs new file mode 100644 index 0000000000000..f2dd8ca6ab742 --- /dev/null +++ b/src/query/service/src/physical_plans/physical_replace_async_source.rs @@ -0,0 +1,80 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::any::Any; + +use databend_common_exception::Result; +use databend_common_expression::DataSchemaRef; +use databend_common_pipeline_sources::AsyncSourcer; +use databend_common_sql::plans::InsertValue; +use databend_common_sql::NameResolutionContext; + +use crate::physical_plans::physical_plan::IPhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlanMeta; +use crate::pipelines::PipelineBuilder; +use crate::pipelines::RawValueSource; +use crate::pipelines::ValueSource; + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct ReplaceAsyncSourcer { + pub meta: PhysicalPlanMeta, + pub schema: DataSchemaRef, + pub source: InsertValue, +} + +#[typetag::serde] +impl IPhysicalPlan for ReplaceAsyncSourcer { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn derive(&self, children: Vec) -> PhysicalPlan { + assert!(children.is_empty()); + Box::new(self.clone()) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + builder.main_pipeline.add_source( + |output| { + let name_resolution_ctx = + NameResolutionContext::try_from(builder.settings.as_ref())?; + match &self.source { + InsertValue::Values { rows } => { + let inner = ValueSource::new(rows.clone(), self.schema.clone()); + AsyncSourcer::create(builder.ctx.clone(), output, inner) + } + InsertValue::RawValues { data, start } => { + let inner = RawValueSource::new( + data.clone(), + builder.ctx.clone(), + name_resolution_ctx, + self.schema.clone(), + *start, + ); + AsyncSourcer::create(builder.ctx.clone(), output, inner) + } + } + }, + 1, + ) + } +} diff --git a/src/query/service/src/physical_plans/physical_replace_deduplicate.rs b/src/query/service/src/physical_plans/physical_replace_deduplicate.rs new file mode 100644 index 0000000000000..13acc738148ad --- /dev/null +++ b/src/query/service/src/physical_plans/physical_replace_deduplicate.rs @@ -0,0 +1,220 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::any::Any; +use std::collections::HashMap; +use std::sync::Arc; + +use databend_common_catalog::table::Table; +use databend_common_exception::ErrorCode; +use databend_common_exception::Result; +use databend_common_expression::ColumnId; +use databend_common_expression::DataSchema; +use databend_common_expression::DataSchemaRef; +use databend_common_expression::FieldIndex; +use databend_common_expression::RemoteExpr; +use databend_common_expression::TableSchemaRef; +use databend_common_functions::BUILTIN_FUNCTIONS; +use databend_common_meta_app::schema::TableInfo; +use databend_common_pipeline_transforms::build_compact_block_pipeline; +use databend_common_pipeline_transforms::TransformPipelineHelper; +use databend_common_sql::executor::physical_plans::OnConflictField; +use databend_common_sql::ColumnBinding; +use databend_common_storages_fuse::operations::ReplaceIntoProcessor; +use databend_common_storages_fuse::operations::UnbranchedReplaceIntoProcessor; +use databend_common_storages_fuse::FuseTable; +use databend_storages_common_table_meta::meta::ColumnStatistics; + +use crate::physical_plans::physical_plan::IPhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlanMeta; +use crate::pipelines::processors::transforms::TransformCastSchema; +use crate::pipelines::PipelineBuilder; + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct ReplaceDeduplicate { + pub meta: PhysicalPlanMeta, + pub input: PhysicalPlan, + pub on_conflicts: Vec, + pub bloom_filter_column_indexes: Vec, + pub table_is_empty: bool, + pub table_info: TableInfo, + pub target_schema: TableSchemaRef, + pub select_ctx: Option, + pub table_level_range_index: HashMap, + pub need_insert: bool, + pub delete_when: Option<(RemoteExpr, String)>, +} + +#[typetag::serde] +impl IPhysicalPlan for ReplaceDeduplicate { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn output_schema(&self) -> Result { + Ok(DataSchemaRef::default()) + } + + fn children<'a>(&'a self) -> Box + 'a> { + Box::new(std::iter::once(&self.input)) + } + + fn children_mut<'a>(&'a mut self) -> Box + 'a> { + Box::new(std::iter::once(&mut self.input)) + } + + fn derive(&self, mut children: Vec) -> PhysicalPlan { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + self.input.build_pipeline(builder)?; + + let tbl = builder + .ctx + .build_table_by_table_info(&self.table_info, None)?; + let table = FuseTable::try_from_table(tbl.as_ref())?; + + let mut delete_column_idx = 0; + let mut modified_schema = DataSchema::from(self.target_schema.clone()).into(); + if let Some(ReplaceSelectCtx { + select_column_bindings, + select_schema, + }) = &self.select_ctx + { + PipelineBuilder::build_result_projection( + &builder.func_ctx, + self.input.output_schema()?, + select_column_bindings, + &mut builder.main_pipeline, + false, + )?; + + let mut target_schema: DataSchema = self.target_schema.clone().into(); + if let Some((_, delete_column)) = &self.delete_when { + delete_column_idx = select_schema.index_of(delete_column.as_str())?; + let delete_column = select_schema.field(delete_column_idx).clone(); + target_schema + .fields + .insert(delete_column_idx, delete_column); + modified_schema = Arc::new(target_schema.clone()); + } + let target_schema = Arc::new(target_schema.clone()); + if target_schema.fields().len() != select_schema.fields().len() { + return Err(ErrorCode::BadArguments( + "The number of columns in the target table is different from the number of columns in the SELECT clause", + )); + } + if PipelineBuilder::check_schema_cast(select_schema.clone(), target_schema.clone())? { + builder.main_pipeline.try_add_transformer(|| { + TransformCastSchema::try_new( + select_schema.clone(), + target_schema.clone(), + builder.func_ctx.clone(), + ) + })?; + } + } + + PipelineBuilder::fill_and_reorder_columns( + builder.ctx.clone(), + &mut builder.main_pipeline, + tbl.clone(), + Arc::new(self.target_schema.clone().into()), + )?; + + let block_thresholds = table.get_block_thresholds(); + build_compact_block_pipeline(&mut builder.main_pipeline, block_thresholds)?; + + let _ = table.cluster_gen_for_append( + builder.ctx.clone(), + &mut builder.main_pipeline, + block_thresholds, + Some(modified_schema), + )?; + // 1. resize input to 1, since the UpsertTransform need to de-duplicate inputs "globally" + builder.main_pipeline.try_resize(1)?; + + // 2. connect with ReplaceIntoProcessor + + // ┌──────────────────────┐ + // │ ├──┐ + // ┌─────────────┐ │ ├──┘ + // │ UpsertSource├─────►│ ReplaceIntoProcessor │ + // └─────────────┘ │ ├──┐ + // │ ├──┘ + // └──────────────────────┘ + // NOTE: here the pipe items of last pipe are arranged in the following order + // (0) -> output_port_append_data + // (1) -> output_port_merge_into_action + // the "downstream" is supposed to be connected with a processor which can process MergeIntoOperations + // in our case, it is the broadcast processor + let delete_when = if let Some((remote_expr, delete_column)) = &self.delete_when { + Some(( + remote_expr.as_expr(&BUILTIN_FUNCTIONS), + delete_column.clone(), + )) + } else { + None + }; + let cluster_keys = table.linear_cluster_keys(builder.ctx.clone()); + if self.need_insert { + let replace_into_processor = ReplaceIntoProcessor::create( + builder.ctx.clone(), + self.on_conflicts.clone(), + cluster_keys, + self.bloom_filter_column_indexes.clone(), + &table.schema(), + self.table_is_empty, + self.table_level_range_index.clone(), + delete_when.map(|(expr, _)| (expr, delete_column_idx)), + )?; + builder + .main_pipeline + .add_pipe(replace_into_processor.into_pipe()); + } else { + let replace_into_processor = UnbranchedReplaceIntoProcessor::create( + builder.ctx.as_ref(), + self.on_conflicts.clone(), + cluster_keys, + self.bloom_filter_column_indexes.clone(), + &table.schema(), + self.table_is_empty, + self.table_level_range_index.clone(), + delete_when.map(|_| delete_column_idx), + )?; + builder + .main_pipeline + .add_pipe(replace_into_processor.into_pipe()); + } + Ok(()) + } +} + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct ReplaceSelectCtx { + pub select_column_bindings: Vec, + pub select_schema: DataSchemaRef, +} diff --git a/src/query/service/src/physical_plans/physical_replace_into.rs b/src/query/service/src/physical_plans/physical_replace_into.rs new file mode 100644 index 0000000000000..ba13cbb890387 --- /dev/null +++ b/src/query/service/src/physical_plans/physical_replace_into.rs @@ -0,0 +1,236 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::any::Any; +use std::sync::Arc; + +use databend_common_catalog::table::Table; +use databend_common_exception::Result; +use databend_common_expression::BlockThresholds; +use databend_common_expression::DataSchema; +use databend_common_expression::DataSchemaRef; +use databend_common_expression::FieldIndex; +use databend_common_meta_app::schema::TableInfo; +use databend_common_pipeline_core::processors::InputPort; +use databend_common_pipeline_core::processors::OutputPort; +use databend_common_pipeline_core::Pipe; +use databend_common_pipeline_transforms::create_dummy_item; +use databend_common_sql::executor::physical_plans::MutationKind; +use databend_common_sql::executor::physical_plans::OnConflictField; +use databend_common_storages_fuse::operations::BroadcastProcessor; +use databend_common_storages_fuse::operations::TransformSerializeBlock; +use databend_common_storages_fuse::FuseTable; +use databend_storages_common_table_meta::meta::BlockSlotDescription; +use databend_storages_common_table_meta::meta::Location; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; +use tokio::sync::Semaphore; + +use crate::physical_plans::physical_plan::IPhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlanMeta; +use crate::pipelines::PipelineBuilder; + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct ReplaceInto { + pub meta: PhysicalPlanMeta, + pub input: PhysicalPlan, + pub block_thresholds: BlockThresholds, + pub table_info: TableInfo, + pub on_conflicts: Vec, + pub bloom_filter_column_indexes: Vec, + pub segments: Vec<(usize, Location)>, + pub block_slots: Option, + pub need_insert: bool, + pub table_meta_timestamps: TableMetaTimestamps, +} + +#[typetag::serde] +impl IPhysicalPlan for ReplaceInto { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn output_schema(&self) -> Result { + Ok(DataSchemaRef::default()) + } + + fn children<'a>(&'a self) -> Box + 'a> { + Box::new(std::iter::once(&self.input)) + } + + fn children_mut<'a>(&'a mut self) -> Box + 'a> { + Box::new(std::iter::once(&mut self.input)) + } + + fn derive(&self, mut children: Vec) -> PhysicalPlan { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + self.input.build_pipeline(builder)?; + + let max_threads = builder.settings.get_max_threads()?; + let segment_partition_num = std::cmp::min(self.segments.len(), max_threads as usize); + let table = builder + .ctx + .build_table_by_table_info(&self.table_info, None)?; + let table = FuseTable::try_from_table(table.as_ref())?; + let schema = DataSchema::from(table.schema()).into(); + let cluster_stats_gen = table.get_cluster_stats_gen( + builder.ctx.clone(), + 0, + self.block_thresholds, + Some(schema), + )?; + + // connect to broadcast processor and append transform + let serialize_block_transform = TransformSerializeBlock::try_create( + builder.ctx.clone(), + InputPort::create(), + OutputPort::create(), + table, + cluster_stats_gen, + MutationKind::Replace, + self.table_meta_timestamps, + )?; + let mut block_builder = serialize_block_transform.get_block_builder(); + block_builder.source_schema = table.schema_with_stream(); + + if !self.need_insert { + if segment_partition_num == 0 { + return Ok(()); + } + let broadcast_processor = BroadcastProcessor::new(segment_partition_num); + builder + .main_pipeline + .add_pipe(Pipe::create(1, segment_partition_num, vec![ + broadcast_processor.into_pipe_item(), + ])); + let max_threads = builder.settings.get_max_threads()?; + let io_request_semaphore = Arc::new(Semaphore::new(max_threads as usize)); + + let merge_into_operation_aggregators = table.merge_into_mutators( + builder.ctx.clone(), + segment_partition_num, + block_builder, + self.on_conflicts.clone(), + self.bloom_filter_column_indexes.clone(), + &self.segments, + self.block_slots.clone(), + io_request_semaphore, + )?; + builder.main_pipeline.add_pipe(Pipe::create( + segment_partition_num, + segment_partition_num, + merge_into_operation_aggregators, + )); + return Ok(()); + } + + // The Block Size and Rows is promised by DataSource by user. + if segment_partition_num == 0 { + let dummy_item = create_dummy_item(); + // ┌──────────────────────┐ ┌──────────────────┐ + // │ ├──┬────────►│ SerializeBlock │ + // ┌─────────────┐ │ ├──┘ └──────────────────┘ + // │ UpsertSource├─────►│ ReplaceIntoProcessor │ + // └─────────────┘ │ ├──┐ ┌──────────────────┐ + // │ ├──┴────────►│ DummyTransform │ + // └──────────────────────┘ └──────────────────┘ + // wrap them into pipeline, order matters! + builder.main_pipeline.add_pipe(Pipe::create(2, 2, vec![ + serialize_block_transform.into_pipe_item(), + dummy_item, + ])); + } else { + // ┌──────────────────────┐ ┌──────────────────┐ + // │ ├──┬────────►│ SerializeBlock │ + // ┌─────────────┐ │ ├──┘ └──────────────────┘ + // │ UpsertSource├─────►│ ReplaceIntoProcessor │ + // └─────────────┘ │ ├──┐ ┌──────────────────┐ + // │ ├──┴────────►│BroadcastProcessor│ + // └──────────────────────┘ └──────────────────┘ + let broadcast_processor = BroadcastProcessor::new(segment_partition_num); + // wrap them into pipeline, order matters! + builder + .main_pipeline + .add_pipe(Pipe::create(2, segment_partition_num + 1, vec![ + serialize_block_transform.into_pipe_item(), + broadcast_processor.into_pipe_item(), + ])); + }; + + // 4. connect with MergeIntoOperationAggregators + if segment_partition_num != 0 { + // ┌──────────────────┐ ┌────────────────┐ + // ────►│ SerializeBlock ├──────────────►│ DummyTransform │ + // └──────────────────┘ └────────────────┘ + // + // ┌───────────────────┐ ┌──────────────────────┐ + // ────►│ ├──┬──────────►│MergeIntoOperationAggr│ + // │ ├──┘ └──────────────────────┘ + // │ BroadcastProcessor│ + // │ ├──┐ ┌──────────────────────┐ + // │ ├──┴──────────►│MergeIntoOperationAggr│ + // │ │ └──────────────────────┘ + // │ ├──┐ + // │ ├──┴──────────►┌──────────────────────┐ + // └───────────────────┘ │MergeIntoOperationAggr│ + // └──────────────────────┘ + + let item_size = segment_partition_num + 1; + let mut pipe_items = Vec::with_capacity(item_size); + // setup the dummy transform + pipe_items.push(create_dummy_item()); + + let max_threads = builder.settings.get_max_threads()?; + let io_request_semaphore = Arc::new(Semaphore::new(max_threads as usize)); + + // setup the merge into operation aggregators + let mut merge_into_operation_aggregators = table.merge_into_mutators( + builder.ctx.clone(), + segment_partition_num, + block_builder, + self.on_conflicts.clone(), + self.bloom_filter_column_indexes.clone(), + &self.segments, + self.block_slots.clone(), + io_request_semaphore, + )?; + assert_eq!( + segment_partition_num, + merge_into_operation_aggregators.len() + ); + pipe_items.append(&mut merge_into_operation_aggregators); + + // extend the pipeline + assert_eq!(builder.main_pipeline.output_len(), item_size); + assert_eq!(pipe_items.len(), item_size); + builder + .main_pipeline + .add_pipe(Pipe::create(item_size, item_size, pipe_items)); + } + Ok(()) + } +} diff --git a/src/query/service/src/physical_plans/physical_row_fetch.rs b/src/query/service/src/physical_plans/physical_row_fetch.rs new file mode 100644 index 0000000000000..be01b65d36caf --- /dev/null +++ b/src/query/service/src/physical_plans/physical_row_fetch.rs @@ -0,0 +1,177 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::any::Any; +use std::sync::Arc; + +use databend_common_ast::ast::FormatTreeNode; +use databend_common_base::runtime::GlobalIORuntime; +use databend_common_catalog::plan::DataSourcePlan; +use databend_common_catalog::plan::Projection; +use databend_common_exception::Result; +use databend_common_expression::DataField; +use databend_common_expression::DataSchemaRef; +use databend_common_expression::DataSchemaRefExt; +use databend_common_pipeline_core::processors::InputPort; +use databend_common_pipeline_core::processors::OutputPort; +use databend_common_pipeline_core::Pipe; +use databend_common_pipeline_core::PipeItem; +use databend_common_pipeline_transforms::create_dummy_item; +use databend_common_storages_fuse::operations::row_fetch_processor; +use itertools::Itertools; +use tokio::sync::Semaphore; + +use crate::physical_plans::explain::PlanStatsInfo; +use crate::physical_plans::format::format_output_columns; +use crate::physical_plans::format::plan_stats_info_to_format_tree; +use crate::physical_plans::format::FormatContext; +use crate::physical_plans::physical_plan::IPhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlanMeta; +use crate::physical_plans::MutationSplit; +use crate::physical_plans::PhysicalPlanDynExt; +use crate::pipelines::PipelineBuilder; + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct RowFetch { + pub meta: PhysicalPlanMeta, + pub input: PhysicalPlan, + // cloned from `input`. + pub source: Box, + // projection on the source table schema. + pub cols_to_fetch: Projection, + pub row_id_col_offset: usize, + pub fetched_fields: Vec, + pub need_wrap_nullable: bool, + + /// Only used for explain + pub stat_info: Option, +} + +#[typetag::serde] +impl IPhysicalPlan for RowFetch { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn output_schema(&self) -> Result { + let mut fields = self.input.output_schema()?.fields().clone(); + fields.extend_from_slice(&self.fetched_fields); + Ok(DataSchemaRefExt::create(fields)) + } + + fn children<'a>(&'a self) -> Box + 'a> { + Box::new(std::iter::once(&self.input)) + } + + fn children_mut<'a>(&'a mut self) -> Box + 'a> { + Box::new(std::iter::once(&mut self.input)) + } + + fn to_format_node( + &self, + ctx: &mut FormatContext<'_>, + children: Vec>, + ) -> Result> { + let table_schema = self.source.source_info.schema(); + let projected_schema = self.cols_to_fetch.project_schema(&table_schema); + let fields_to_fetch = projected_schema.fields(); + + let mut node_children = vec![ + FormatTreeNode::new(format!( + "output columns: [{}]", + format_output_columns(self.output_schema()?, ctx.metadata, true) + )), + FormatTreeNode::new(format!( + "columns to fetch: [{}]", + fields_to_fetch.iter().map(|f| f.name()).join(", ") + )), + ]; + + if let Some(info) = &self.stat_info { + node_children.extend(plan_stats_info_to_format_tree(info)); + } + + node_children.extend(children); + + Ok(FormatTreeNode::with_children( + "RowFetch".to_string(), + node_children, + )) + } + + #[recursive::recursive] + fn try_find_single_data_source(&self) -> Option<&DataSourcePlan> { + self.input.try_find_single_data_source() + } + + fn get_desc(&self) -> Result { + let table_schema = self.source.source_info.schema(); + let projected_schema = self.cols_to_fetch.project_schema(&table_schema); + Ok(projected_schema.fields.iter().map(|f| f.name()).join(", ")) + } + + fn derive(&self, mut children: Vec) -> PhysicalPlan { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + self.input.build_pipeline(builder)?; + + let max_io_requests = builder.settings.get_max_storage_io_requests()? as usize; + let row_fetch_runtime = GlobalIORuntime::instance(); + let row_fetch_semaphore = Arc::new(Semaphore::new(max_io_requests)); + let processor = row_fetch_processor( + builder.ctx.clone(), + self.row_id_col_offset, + &self.source, + self.cols_to_fetch.clone(), + self.need_wrap_nullable, + row_fetch_semaphore, + row_fetch_runtime, + )?; + + if self.input.downcast_ref::().is_none() { + builder.main_pipeline.add_transform(processor)?; + } else { + let output_len = builder.main_pipeline.output_len(); + let mut pipe_items = Vec::with_capacity(output_len); + for i in 0..output_len { + if i % 2 == 0 { + let input = InputPort::create(); + let output = OutputPort::create(); + let processor_ptr = processor(input.clone(), output.clone())?; + pipe_items.push(PipeItem::create(processor_ptr, vec![input], vec![output])); + } else { + pipe_items.push(create_dummy_item()); + } + } + builder + .main_pipeline + .add_pipe(Pipe::create(output_len, output_len, pipe_items)); + } + + Ok(()) + } +} diff --git a/src/query/service/src/physical_plans/physical_sort.rs b/src/query/service/src/physical_plans/physical_sort.rs new file mode 100644 index 0000000000000..bbd405ccfc27b --- /dev/null +++ b/src/query/service/src/physical_plans/physical_sort.rs @@ -0,0 +1,370 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::any::Any; + +use databend_common_ast::ast::FormatTreeNode; +use databend_common_catalog::plan::DataSourcePlan; +use databend_common_exception::Result; +use databend_common_expression::types::DataType; +use databend_common_expression::DataField; +use databend_common_expression::DataSchema; +use databend_common_expression::DataSchemaRef; +use databend_common_expression::DataSchemaRefExt; +use databend_common_expression::SortColumnDescription; +use databend_common_pipeline_transforms::processors::sort::utils::ORDER_COL_NAME; +use databend_common_pipeline_transforms::TransformPipelineHelper; +use databend_common_sql::evaluator::BlockOperator; +use databend_common_sql::evaluator::CompoundBlockOperator; +use databend_common_sql::executor::physical_plans::SortDesc; +use databend_common_sql::optimizer::ir::SExpr; +use databend_common_sql::plans::WindowFuncType; +use databend_common_sql::ColumnSet; +use databend_common_sql::IndexType; +use itertools::Itertools; + +use crate::physical_plans::explain::PlanStatsInfo; +use crate::physical_plans::format::format_output_columns; +use crate::physical_plans::format::plan_stats_info_to_format_tree; +use crate::physical_plans::format::FormatContext; +use crate::physical_plans::physical_plan::IPhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlanMeta; +use crate::physical_plans::PhysicalPlanBuilder; +use crate::physical_plans::WindowPartition; +use crate::physical_plans::WindowPartitionTopN; +use crate::physical_plans::WindowPartitionTopNFunc; +use crate::pipelines::builders::SortPipelineBuilder; +use crate::pipelines::PipelineBuilder; + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct Sort { + pub meta: PhysicalPlanMeta, + pub input: PhysicalPlan, + pub order_by: Vec, + /// limit = Limit.limit + Limit.offset + pub limit: Option, + /// If the sort plan is after the exchange plan. + /// It's [None] if the sorting plan is in single node mode. + pub after_exchange: Option, + pub pre_projection: Option>, + + // Only used for explain + pub stat_info: Option, +} + +#[typetag::serde] +impl IPhysicalPlan for Sort { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn output_schema(&self) -> Result { + let input_schema = self.input.output_schema()?; + let mut fields = input_schema.fields().clone(); + if matches!(self.after_exchange, Some(true)) { + // If the plan is after exchange plan in cluster mode, + // the order column is at the last of the input schema. + debug_assert_eq!(fields.last().unwrap().name(), ORDER_COL_NAME); + debug_assert_eq!( + fields.last().unwrap().data_type(), + &self.order_col_type(&input_schema)? + ); + fields.pop(); + } else { + if let Some(proj) = &self.pre_projection { + let fileted_fields = proj + .iter() + .filter_map(|index| input_schema.field_with_name(&index.to_string()).ok()) + .cloned() + .collect::>(); + if fileted_fields.len() < fields.len() { + // Only if the projection is not a full projection, we need to add a projection transform. + fields = fileted_fields + } + } + + if matches!(self.after_exchange, Some(false)) { + // If the plan is before exchange plan in cluster mode, + // the order column should be added to the output schema. + fields.push(DataField::new( + ORDER_COL_NAME, + self.order_col_type(&input_schema)?, + )); + } + } + + Ok(DataSchemaRefExt::create(fields)) + } + + fn children<'a>(&'a self) -> Box + 'a> { + Box::new(std::iter::once(&self.input)) + } + + fn children_mut<'a>(&'a mut self) -> Box + 'a> { + Box::new(std::iter::once(&mut self.input)) + } + + fn to_format_node( + &self, + ctx: &mut FormatContext<'_>, + children: Vec>, + ) -> Result> { + let sort_keys = self + .order_by + .iter() + .map(|sort_key| { + Ok(format!( + "{} {} {}", + sort_key.display_name, + if sort_key.asc { "ASC" } else { "DESC" }, + if sort_key.nulls_first { + "NULLS FIRST" + } else { + "NULLS LAST" + } + )) + }) + .collect::>>()? + .join(", "); + + let mut node_children = vec![ + FormatTreeNode::new(format!( + "output columns: [{}]", + format_output_columns(self.output_schema()?, ctx.metadata, true) + )), + FormatTreeNode::new(format!("sort keys: [{sort_keys}]")), + ]; + + if let Some(info) = &self.stat_info { + node_children.extend(plan_stats_info_to_format_tree(info)); + } + + node_children.extend(children); + Ok(FormatTreeNode::with_children( + "Sort".to_string(), + node_children, + )) + } + + #[recursive::recursive] + fn try_find_single_data_source(&self) -> Option<&DataSourcePlan> { + self.input.try_find_single_data_source() + } + + fn get_desc(&self) -> Result { + Ok(self + .order_by + .iter() + .map(|x| { + format!( + "{}{}{}", + x.display_name, + if x.asc { "" } else { " DESC" }, + if x.nulls_first { " NULLS FIRST" } else { "" }, + ) + }) + .join(", ")) + } + + fn derive(&self, mut children: Vec) -> PhysicalPlan { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + self.input.build_pipeline(builder)?; + + let input_schema = self.input.output_schema()?; + + if !matches!(self.after_exchange, Some(true)) { + // If the Sort plan is after exchange, we don't need to do a projection, + // because the data is already projected in each cluster node. + if let Some(proj) = &self.pre_projection { + // Do projection to reduce useless data copying during sorting. + let projection = proj + .iter() + .filter_map(|i| input_schema.index_of(&i.to_string()).ok()) + .collect::>(); + + if projection.len() < input_schema.fields().len() { + // Only if the projection is not a full projection, we need to add a projection transform. + builder.main_pipeline.add_transformer(|| { + CompoundBlockOperator::new( + vec![BlockOperator::Project { + projection: projection.clone(), + }], + builder.func_ctx.clone(), + input_schema.num_fields(), + ) + }); + } + } + } + + let plan_schema = self.output_schema()?; + + let sort_desc = self + .order_by + .iter() + .map(|desc| { + let offset = plan_schema.index_of(&desc.order_by.to_string())?; + Ok(SortColumnDescription { + offset, + asc: desc.asc, + nulls_first: desc.nulls_first, + }) + }) + .collect::>>()?; + + let max_threads = builder.settings.get_max_threads()? as usize; + let sort_desc = sort_desc.into(); + + // TODO(Winter): the query will hang in MultiSortMergeProcessor when max_threads == 1 and output_len != 1 + if builder.main_pipeline.output_len() == 1 || max_threads == 1 { + builder.main_pipeline.try_resize(max_threads)?; + } + + let sort_builder = + SortPipelineBuilder::create(builder.ctx.clone(), plan_schema, sort_desc)? + .with_limit(self.limit); + + match self.after_exchange { + Some(true) => { + // Build for the coordinator node. + // We only build a `MultiSortMergeTransform`, + // as the data is already sorted in each cluster node. + // The input number of the transform is equal to the number of cluster nodes. + if builder.main_pipeline.output_len() > 1 { + sort_builder + .remove_order_col_at_last() + .build_multi_merge(&mut builder.main_pipeline) + } else { + sort_builder + .remove_order_col_at_last() + .build_merge_sort_pipeline(&mut builder.main_pipeline, true) + } + } + Some(false) => { + // Build for each cluster node. + // We build the full sort pipeline for it. + // Don't remove the order column at last. + sort_builder.build_full_sort_pipeline(&mut builder.main_pipeline) + } + None => { + // Build for single node mode. + // We build the full sort pipeline for it. + sort_builder + .remove_order_col_at_last() + .build_full_sort_pipeline(&mut builder.main_pipeline) + } + } + } +} + +impl Sort { + fn order_col_type(&self, schema: &DataSchema) -> Result { + if self.order_by.len() == 1 { + let order_by_field = schema.field_with_name(&self.order_by[0].order_by.to_string())?; + if matches!( + order_by_field.data_type(), + DataType::Number(_) | DataType::Date | DataType::Timestamp | DataType::String + ) { + return Ok(order_by_field.data_type().clone()); + } + } + Ok(DataType::Binary) + } +} + +impl PhysicalPlanBuilder { + pub async fn build_sort( + &mut self, + s_expr: &SExpr, + sort: &databend_common_sql::plans::Sort, + mut required: ColumnSet, + stat_info: PlanStatsInfo, + ) -> Result { + // 1. Prune unused Columns. + sort.items.iter().for_each(|s| { + required.insert(s.index); + }); + + // If the query will be optimized by lazy reading, we don't need to do pre-projection. + let pre_projection: Option> = if self.metadata.read().lazy_columns().is_empty() { + sort.pre_projection.clone() + } else { + None + }; + let input_plan = self.build(s_expr.child(0)?, required).await?; + + let order_by = sort + .items + .iter() + .map(|v| SortDesc { + asc: v.asc, + nulls_first: v.nulls_first, + order_by: v.index, + display_name: self.metadata.read().column(v.index).name(), + }) + .collect::>(); + + // Add WindowPartition for parallel sort in window. + if let Some(window) = &sort.window_partition { + let window_partition = window + .partition_by + .iter() + .map(|v| v.index) + .collect::>(); + + return Ok(Box::new(WindowPartition { + meta: PhysicalPlanMeta::new("WindowPartition"), + input: input_plan.clone(), + partition_by: window_partition.clone(), + order_by: order_by.clone(), + after_exchange: sort.after_exchange, + top_n: window.top.map(|top| WindowPartitionTopN { + func: match window.func { + WindowFuncType::RowNumber => WindowPartitionTopNFunc::RowNumber, + WindowFuncType::Rank => WindowPartitionTopNFunc::Rank, + WindowFuncType::DenseRank => WindowPartitionTopNFunc::DenseRank, + _ => unreachable!(), + }, + top, + }), + stat_info: Some(stat_info.clone()), + })); + }; + + // 2. Build physical plan. + Ok(Box::new(Sort { + order_by, + pre_projection, + input: input_plan, + limit: sort.limit, + after_exchange: sort.after_exchange, + stat_info: Some(stat_info), + meta: PhysicalPlanMeta::new("Sort"), + })) + } +} diff --git a/src/query/sql/src/executor/physical_plans/physical_table_scan.rs b/src/query/service/src/physical_plans/physical_table_scan.rs similarity index 69% rename from src/query/sql/src/executor/physical_plans/physical_table_scan.rs rename to src/query/service/src/physical_plans/physical_table_scan.rs index ea303927eda2b..ab4614383230c 100644 --- a/src/query/sql/src/executor/physical_plans/physical_table_scan.rs +++ b/src/query/service/src/physical_plans/physical_table_scan.rs @@ -12,22 +12,29 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::any::Any; use std::collections::btree_map::Entry; use std::collections::BTreeMap; +use std::collections::HashMap; use std::collections::HashSet; use std::sync::Arc; +use databend_common_ast::ast::FormatTreeNode; use databend_common_ast::parser::token::TokenKind; use databend_common_ast::parser::tokenize_sql; use databend_common_catalog::catalog::CatalogManager; +use databend_common_catalog::plan::DataSourceInfo; use databend_common_catalog::plan::DataSourcePlan; use databend_common_catalog::plan::Filters; use databend_common_catalog::plan::InternalColumn; +use databend_common_catalog::plan::PartStatistics; +use databend_common_catalog::plan::PartitionsShuffleKind; use databend_common_catalog::plan::PrewhereInfo; use databend_common_catalog::plan::Projection; use databend_common_catalog::plan::PushDownInfo; use databend_common_catalog::plan::VirtualColumnField; use databend_common_catalog::plan::VirtualColumnInfo; +use databend_common_catalog::table_context::TableContext; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::type_check::check_function; @@ -44,37 +51,48 @@ use databend_common_expression::TableSchema; use databend_common_expression::TableSchemaRef; use databend_common_expression::ROW_ID_COL_NAME; use databend_common_functions::BUILTIN_FUNCTIONS; +use databend_common_pipeline_transforms::TransformPipelineHelper; +use databend_common_sql::binder::INTERNAL_COLUMN_FACTORY; +use databend_common_sql::evaluator::BlockOperator; +use databend_common_sql::evaluator::CompoundBlockOperator; +use databend_common_sql::executor::cast_expr_to_non_null_boolean; +use databend_common_sql::executor::table_read_plan::ToReadDataSourcePlan; +use databend_common_sql::plans::FunctionCall; +use databend_common_sql::BaseTableColumn; +use databend_common_sql::ColumnEntry; +use databend_common_sql::ColumnSet; +use databend_common_sql::DerivedColumn; +use databend_common_sql::IndexType; +use databend_common_sql::Metadata; +use databend_common_sql::ScalarExpr; +use databend_common_sql::TableInternalColumn; +use databend_common_sql::TypeCheck; +use databend_common_sql::VirtualColumn; +use databend_common_sql::DUMMY_COLUMN_INDEX; +use databend_common_sql::DUMMY_TABLE_INDEX; +use itertools::Itertools; use jsonb::keypath::KeyPath; use jsonb::keypath::KeyPaths; use rand::distributions::Bernoulli; use rand::distributions::Distribution; use rand::thread_rng; -use crate::binder::INTERNAL_COLUMN_FACTORY; -use crate::executor::cast_expr_to_non_null_boolean; -use crate::executor::explain::PlanStatsInfo; -use crate::executor::physical_plans::AddStreamColumn; -use crate::executor::table_read_plan::ToReadDataSourcePlan; -use crate::executor::PhysicalPlan; -use crate::executor::PhysicalPlanBuilder; -use crate::plans::FunctionCall; -use crate::BaseTableColumn; -use crate::ColumnEntry; -use crate::ColumnSet; -use crate::DerivedColumn; -use crate::IndexType; -use crate::Metadata; -use crate::ScalarExpr; -use crate::TableInternalColumn; -use crate::TypeCheck; -use crate::VirtualColumn; -use crate::DUMMY_COLUMN_INDEX; -use crate::DUMMY_TABLE_INDEX; +use crate::physical_plans::explain::PlanStatsInfo; +use crate::physical_plans::format::format_output_columns; +use crate::physical_plans::format::part_stats_info_to_format_tree; +use crate::physical_plans::format::plan_stats_info_to_format_tree; +use crate::physical_plans::format::FormatContext; +use crate::physical_plans::physical_plan::IPhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlanMeta; +use crate::physical_plans::AddStreamColumn; +use crate::physical_plans::PhysicalPlanBuilder; +use crate::pipelines::processors::transforms::TransformAddInternalColumns; +use crate::pipelines::PipelineBuilder; #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct TableScan { - // A unique id of operator in a `PhysicalPlan` tree, only used for display. - pub plan_id: u32, + pub meta: PhysicalPlanMeta, pub scan_id: usize, pub name_mapping: BTreeMap, pub source: Box, @@ -84,7 +102,310 @@ pub struct TableScan { pub stat_info: Option, } +#[typetag::serde] +impl IPhysicalPlan for TableScan { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn output_schema(&self) -> Result { + let schema = self.source.schema(); + let mut fields = Vec::with_capacity(self.name_mapping.len()); + let mut name_and_ids = self + .name_mapping + .iter() + .map(|(name, id)| { + let index = schema.index_of(name)?; + Ok((name, id, index)) + }) + .collect::>>()?; + // Make the order of output fields the same as their indexes in te table schema. + name_and_ids.sort_by_key(|(_, _, index)| *index); + + for (name, id, _) in name_and_ids { + let orig_field = schema.field_with_name(name)?; + let data_type = DataType::from(orig_field.data_type()); + fields.push(DataField::new(&id.to_string(), data_type)); + } + + Ok(DataSchemaRefExt::create(fields)) + } + + fn to_format_node( + &self, + ctx: &mut FormatContext<'_>, + _: Vec>, + ) -> Result> { + if self.table_index == Some(DUMMY_TABLE_INDEX) { + return Ok(FormatTreeNode::new("DummyTableScan".to_string())); + } + + let table_name = match self.table_index { + None => format!( + "{}.{}", + self.source.source_info.catalog_name(), + self.source.source_info.desc() + ), + Some(table_index) => { + let table = ctx.metadata.table(table_index).clone(); + format!("{}.{}.{}", table.catalog(), table.database(), table.name()) + } + }; + let filters = self + .source + .push_downs + .as_ref() + .and_then(|extras| { + extras + .filters + .as_ref() + .map(|filters| filters.filter.as_expr(&BUILTIN_FUNCTIONS).sql_display()) + }) + .unwrap_or_default(); + + let limit = self + .source + .push_downs + .as_ref() + .map_or("NONE".to_string(), |extras| { + extras + .limit + .map_or("NONE".to_string(), |limit| limit.to_string()) + }); + + let virtual_columns = self.source.push_downs.as_ref().and_then(|extras| { + extras.virtual_column.as_ref().map(|virtual_column| { + let mut names = virtual_column + .virtual_column_fields + .iter() + .map(|c| c.name.clone()) + .collect::>(); + names.sort(); + names.iter().join(", ") + }) + }); + + let agg_index = self + .source + .push_downs + .as_ref() + .and_then(|extras| extras.agg_index.as_ref()); + + let mut children = vec![ + FormatTreeNode::new(format!("table: {table_name}")), + FormatTreeNode::new(format!( + "output columns: [{}]", + format_output_columns(self.output_schema()?, ctx.metadata, false) + )), + ]; + + // Part stats. + children.extend(part_stats_info_to_format_tree(&self.source.statistics)); + // Push downs. + let push_downs = format!("push downs: [filters: [{filters}], limit: {limit}]"); + children.push(FormatTreeNode::new(push_downs)); + + // runtime filters + let rf = ctx.scan_id_to_runtime_filters.get(&self.scan_id); + if let Some(rf) = rf { + let rf = rf.iter().map(|rf| format!("#{:?}", rf.id)).join(", "); + children.push(FormatTreeNode::new(format!("apply join filters: [{rf}]"))); + } + + // Virtual columns. + if let Some(virtual_columns) = virtual_columns { + if !virtual_columns.is_empty() { + let virtual_columns = format!("virtual columns: [{virtual_columns}]"); + children.push(FormatTreeNode::new(virtual_columns)); + } + } + + // Aggregating index + if let Some(agg_index) = agg_index { + let (_, agg_index_sql, _) = ctx + .metadata + .get_agg_indexes(&table_name) + .unwrap() + .iter() + .find(|(index, _, _)| *index == agg_index.index_id) + .unwrap(); + + children.push(FormatTreeNode::new(format!( + "aggregating index: [{agg_index_sql}]" + ))); + + let agg_sel = agg_index + .selection + .iter() + .map(|(expr, _)| expr.as_expr(&BUILTIN_FUNCTIONS).sql_display()) + .join(", "); + let agg_filter = agg_index + .filter + .as_ref() + .map(|f| f.as_expr(&BUILTIN_FUNCTIONS).sql_display()); + let text = if let Some(f) = agg_filter { + format!("rewritten query: [selection: [{agg_sel}], filter: {f}]") + } else { + format!("rewritten query: [selection: [{agg_sel}]]") + }; + children.push(FormatTreeNode::new(text)); + } + + if let Some(info) = &self.stat_info { + children.extend(plan_stats_info_to_format_tree(info)); + } + + Ok(FormatTreeNode::with_children( + "TableScan".to_string(), + children, + )) + } + + fn try_find_single_data_source(&self) -> Option<&DataSourcePlan> { + Some(&self.source) + } + + fn get_all_data_source(&self, sources: &mut Vec<(u32, Box)>) { + sources.push((self.get_id(), self.source.clone())); + } + + fn set_pruning_stats(&mut self, stats: &mut HashMap) { + if let Some(stat) = stats.remove(&self.get_id()) { + self.source.statistics = stat; + } + } + + fn is_warehouse_distributed_plan(&self) -> bool { + self.source.parts.kind == PartitionsShuffleKind::BroadcastWarehouse + } + + fn get_desc(&self) -> Result { + Ok(format!( + "{}.{}", + self.source.source_info.catalog_name(), + self.source.source_info.desc() + )) + } + + fn get_labels(&self) -> Result>> { + Ok(HashMap::from([ + (String::from("Full table name"), vec![format!( + "{}.{}", + self.source.source_info.catalog_name(), + self.source.source_info.desc() + )]), + ( + format!( + "Columns ({} / {})", + self.output_schema()?.num_fields(), + std::cmp::max( + self.output_schema()?.num_fields(), + self.source.source_info.schema().num_fields(), + ) + ), + self.name_mapping.keys().cloned().collect(), + ), + (String::from("Total partitions"), vec![self + .source + .statistics + .partitions_total + .to_string()]), + ])) + } + + fn derive(&self, children: Vec) -> PhysicalPlan { + assert!(children.is_empty()); + Box::new(self.clone()) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + let table = builder.ctx.build_table_from_source_plan(&self.source)?; + builder.ctx.set_partitions(self.source.parts.clone())?; + builder + .ctx + .set_wait_runtime_filter(self.scan_id, builder.contain_sink_processor); + + if builder.ctx.get_settings().get_enable_prune_pipeline()? { + if let Some(prune_pipeline) = table.build_prune_pipeline( + builder.ctx.clone(), + &self.source, + &mut builder.main_pipeline, + self.get_id(), + )? { + builder.pipelines.push(prune_pipeline); + } + } + + table.read_data( + builder.ctx.clone(), + &self.source, + &mut builder.main_pipeline, + true, + )?; + + // Fill internal columns if needed. + if let Some(internal_columns) = &self.internal_column { + builder + .main_pipeline + .add_transformer(|| TransformAddInternalColumns::new(internal_columns.clone())); + } + + let schema = self.source.schema(); + let mut projection = self + .name_mapping + .keys() + .map(|name| schema.index_of(name.as_str())) + .collect::>>()?; + projection.sort(); + + // if projection is sequential, no need to add projection + if projection != (0..schema.fields().len()).collect::>() { + let ops = vec![BlockOperator::Project { projection }]; + let num_input_columns = schema.num_fields(); + builder.main_pipeline.add_transformer(|| { + CompoundBlockOperator::new(ops.clone(), builder.func_ctx.clone(), num_input_columns) + }); + } + + Ok(()) + } +} + impl TableScan { + pub fn create( + scan_id: usize, + name_mapping: BTreeMap, + source: Box, + table_index: Option, + stat_info: Option, + internal_column: Option>, + ) -> PhysicalPlan { + let name = match &source.source_info { + DataSourceInfo::TableSource(_) => "TableScan".to_string(), + DataSourceInfo::StageSource(_) => "StageScan".to_string(), + DataSourceInfo::ParquetSource(_) => "ParquetScan".to_string(), + DataSourceInfo::ResultScanSource(_) => "ResultScan".to_string(), + DataSourceInfo::ORCSource(_) => "OrcScan".to_string(), + }; + + Box::new(TableScan { + meta: PhysicalPlanMeta::new(name), + source, + scan_id, + name_mapping, + table_index, + stat_info, + internal_column, + }) + } + pub fn output_fields( schema: TableSchemaRef, name_mapping: &BTreeMap, @@ -107,17 +428,12 @@ impl TableScan { } Ok(fields) } - - pub fn output_schema(&self) -> Result { - let fields = TableScan::output_fields(self.source.schema(), &self.name_mapping)?; - Ok(DataSchemaRefExt::create(fields)) - } } impl PhysicalPlanBuilder { - pub(crate) async fn build_table_scan( + pub async fn build_table_scan( &mut self, - scan: &crate::plans::Scan, + scan: &databend_common_sql::plans::Scan, required: ColumnSet, stat_info: PlanStatsInfo, ) -> Result { @@ -308,30 +624,29 @@ impl PhysicalPlanBuilder { metadata.set_table_source(scan.table_index, source.clone()); } - let mut plan = PhysicalPlan::TableScan(TableScan { - plan_id: 0, - scan_id: scan.scan_id, + let mut plan = TableScan::create( + scan.scan_id, name_mapping, - source: Box::new(source), - table_index: Some(scan.table_index), - stat_info: Some(stat_info), + Box::new(source), + Some(scan.table_index), + Some(stat_info), internal_column, - }); + ); // Update stream columns if needed. if scan.update_stream_columns { - plan = PhysicalPlan::AddStreamColumn(Box::new(AddStreamColumn::new( + plan = AddStreamColumn::create( &self.metadata, plan, scan.table_index, table.get_table_info().ident.seq, - )?)); + )?; } Ok(plan) } - pub(crate) async fn build_dummy_table_scan(&mut self) -> Result { + pub async fn build_dummy_table_scan(&mut self) -> Result { let catalogs = CatalogManager::instance(); let table = catalogs .get_default_catalog(self.ctx.session_state())? @@ -345,22 +660,22 @@ impl PhysicalPlanBuilder { let source = table .read_plan(self.ctx.clone(), None, None, false, self.dry_run) .await?; - Ok(PhysicalPlan::TableScan(TableScan { - plan_id: 0, - scan_id: DUMMY_TABLE_INDEX, - name_mapping: BTreeMap::from([("dummy".to_string(), DUMMY_COLUMN_INDEX)]), - source: Box::new(source), - table_index: Some(DUMMY_TABLE_INDEX), - stat_info: Some(PlanStatsInfo { + + Ok(TableScan::create( + DUMMY_TABLE_INDEX, + BTreeMap::from([("dummy".to_string(), DUMMY_COLUMN_INDEX)]), + Box::new(source), + Some(DUMMY_TABLE_INDEX), + Some(PlanStatsInfo { estimated_rows: 1.0, }), - internal_column: None, - })) + None, + )) } fn push_downs( &self, - scan: &crate::plans::Scan, + scan: &databend_common_sql::plans::Scan, table_schema: &TableSchema, virtual_columns: BTreeMap, has_inner_column: bool, @@ -663,8 +978,8 @@ impl PhysicalPlanBuilder { Ok(Some(virtual_column_info)) } - pub(crate) fn build_agg_index( - agg: &crate::plans::AggIndexInfo, + pub fn build_agg_index( + agg: &databend_common_sql::plans::AggIndexInfo, source_fields: &[DataField], ) -> Result { // Build projection diff --git a/src/query/sql/src/executor/physical_plans/physical_udf.rs b/src/query/service/src/physical_plans/physical_udf.rs similarity index 53% rename from src/query/sql/src/executor/physical_plans/physical_udf.rs rename to src/query/service/src/physical_plans/physical_udf.rs index acfd628b36af3..24130af2f12b1 100644 --- a/src/query/sql/src/executor/physical_plans/physical_udf.rs +++ b/src/query/service/src/physical_plans/physical_udf.rs @@ -12,8 +12,11 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::any::Any; use std::collections::BTreeMap; +use databend_common_ast::ast::FormatTreeNode; +use databend_common_catalog::plan::DataSourcePlan; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::types::DataType; @@ -21,29 +24,50 @@ use databend_common_expression::DataField; use databend_common_expression::DataSchemaRef; use databend_common_expression::DataSchemaRefExt; use databend_common_functions::BUILTIN_FUNCTIONS; +use databend_common_pipeline_transforms::TransformPipelineHelper; +use databend_common_sql::optimizer::ir::SExpr; +use databend_common_sql::plans::UDFType; +use databend_common_sql::ColumnSet; +use databend_common_sql::IndexType; +use databend_common_sql::ScalarExpr; +use itertools::Itertools; -use crate::executor::explain::PlanStatsInfo; -use crate::executor::PhysicalPlan; -use crate::executor::PhysicalPlanBuilder; -use crate::optimizer::ir::SExpr; -use crate::plans::UDFType; -use crate::ColumnSet; -use crate::IndexType; -use crate::ScalarExpr; +use crate::physical_plans::explain::PlanStatsInfo; +use crate::physical_plans::format::format_output_columns; +use crate::physical_plans::format::plan_stats_info_to_format_tree; +use crate::physical_plans::format::FormatContext; +use crate::physical_plans::physical_plan::IPhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlanMeta; +use crate::physical_plans::PhysicalPlanBuilder; +use crate::pipelines::processors::transforms::TransformUdfScript; +use crate::pipelines::processors::transforms::TransformUdfServer; +use crate::pipelines::PipelineBuilder; #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct Udf { - // A unique id of operator in a `PhysicalPlan` tree, only used for display. - pub plan_id: u32, - pub input: Box, + pub meta: PhysicalPlanMeta, + pub input: PhysicalPlan, pub udf_funcs: Vec, pub script_udf: bool, // Only used for explain pub stat_info: Option, } -impl Udf { - pub fn output_schema(&self) -> Result { +#[typetag::serde] +impl IPhysicalPlan for Udf { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn output_schema(&self) -> Result { let input_schema = self.input.output_schema()?; let mut fields = input_schema.fields().clone(); for udf_func in self.udf_funcs.iter() { @@ -53,6 +77,95 @@ impl Udf { } Ok(DataSchemaRefExt::create(fields)) } + + fn children<'a>(&'a self) -> Box + 'a> { + Box::new(std::iter::once(&self.input)) + } + + fn children_mut<'a>(&'a mut self) -> Box + 'a> { + Box::new(std::iter::once(&mut self.input)) + } + + fn to_format_node( + &self, + ctx: &mut FormatContext<'_>, + children: Vec>, + ) -> Result> { + let mut node_children = vec![FormatTreeNode::new(format!( + "output columns: [{}]", + format_output_columns(self.output_schema()?, ctx.metadata, true) + ))]; + + if let Some(info) = &self.stat_info { + let items = plan_stats_info_to_format_tree(info); + node_children.extend(items); + } + + node_children.extend(vec![FormatTreeNode::new(format!( + "udf functions: {}", + self.udf_funcs + .iter() + .map(|func| { + let arg_exprs = func.arg_exprs.join(", "); + format!("{}({})", func.func_name, arg_exprs) + }) + .collect::>() + .join(", ") + ))]); + + node_children.extend(children); + Ok(FormatTreeNode::with_children( + "Udf".to_string(), + node_children, + )) + } + + #[recursive::recursive] + fn try_find_single_data_source(&self) -> Option<&DataSourcePlan> { + self.input.try_find_single_data_source() + } + + fn get_desc(&self) -> Result { + Ok(self + .udf_funcs + .iter() + .map(|x| format!("{}({})", x.func_name, x.arg_exprs.join(", "))) + .join(", ")) + } + + fn derive(&self, mut children: Vec) -> PhysicalPlan { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + self.input.build_pipeline(builder)?; + + if self.script_udf { + let runtimes = TransformUdfScript::init_runtime(&self.udf_funcs)?; + builder.main_pipeline.try_add_transformer(|| { + Ok(TransformUdfScript::new( + builder.func_ctx.clone(), + self.udf_funcs.clone(), + runtimes.clone(), + )) + }) + } else { + let semaphore = TransformUdfServer::init_semaphore(builder.ctx.clone())?; + let endpoints = + TransformUdfServer::init_endpoints(builder.ctx.clone(), &self.udf_funcs)?; + builder.main_pipeline.try_add_async_transformer(|| { + TransformUdfServer::new( + builder.ctx.clone(), + self.udf_funcs.clone(), + semaphore.clone(), + endpoints.clone(), + ) + }) + } + } } #[derive(Clone, Debug, Eq, Hash, PartialEq, serde::Serialize, serde::Deserialize)] @@ -69,10 +182,10 @@ pub struct UdfFunctionDesc { } impl PhysicalPlanBuilder { - pub(crate) async fn build_udf( + pub async fn build_udf( &mut self, s_expr: &SExpr, - udf_plan: &crate::plans::Udf, + udf_plan: &databend_common_sql::plans::Udf, mut required: ColumnSet, stat_info: PlanStatsInfo, ) -> Result { @@ -146,12 +259,12 @@ impl PhysicalPlanBuilder { }) .collect::>>()?; - Ok(PhysicalPlan::Udf(Udf { - plan_id: 0, - input: Box::new(input), + Ok(Box::new(Udf { + input, udf_funcs, script_udf: udf_plan.script_udf, stat_info: Some(stat_info), + meta: PhysicalPlanMeta::new("Udf"), })) } } diff --git a/src/query/service/src/physical_plans/physical_union_all.rs b/src/query/service/src/physical_plans/physical_union_all.rs new file mode 100644 index 0000000000000..3a679ac8c552e --- /dev/null +++ b/src/query/service/src/physical_plans/physical_union_all.rs @@ -0,0 +1,333 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::any::Any; + +use databend_common_ast::ast::FormatTreeNode; +use databend_common_exception::Result; +use databend_common_expression::DataField; +use databend_common_expression::DataSchema; +use databend_common_expression::DataSchemaRef; +use databend_common_expression::DataSchemaRefExt; +use databend_common_expression::RemoteExpr; +use databend_common_functions::BUILTIN_FUNCTIONS; +use databend_common_pipeline_core::processors::ProcessorPtr; +use databend_common_sql::evaluator::BlockOperator; +use databend_common_sql::evaluator::CompoundBlockOperator; +use databend_common_sql::optimizer::ir::SExpr; +use databend_common_sql::ColumnSet; +use databend_common_sql::IndexType; +use databend_common_sql::ScalarExpr; +use databend_common_sql::TypeCheck; +use itertools::Itertools; + +use crate::physical_plans::explain::PlanStatsInfo; +use crate::physical_plans::format::format_output_columns; +use crate::physical_plans::format::plan_stats_info_to_format_tree; +use crate::physical_plans::format::FormatContext; +use crate::physical_plans::physical_plan::IPhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlanMeta; +use crate::physical_plans::PhysicalPlanBuilder; +use crate::pipelines::processors::transforms::TransformRecursiveCteSource; +use crate::pipelines::PipelineBuilder; + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct UnionAll { + meta: PhysicalPlanMeta, + pub left: PhysicalPlan, + pub right: PhysicalPlan, + pub left_outputs: Vec<(IndexType, Option)>, + pub right_outputs: Vec<(IndexType, Option)>, + pub schema: DataSchemaRef, + pub cte_scan_names: Vec, + + // Only used for explain + pub stat_info: Option, +} + +#[typetag::serde] +impl IPhysicalPlan for UnionAll { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn output_schema(&self) -> Result { + Ok(self.schema.clone()) + } + + fn children<'a>(&'a self) -> Box + 'a> { + Box::new(std::iter::once(&self.left).chain(std::iter::once(&self.right))) + } + + fn children_mut<'a>(&'a mut self) -> Box + 'a> { + Box::new(std::iter::once(&mut self.left).chain(std::iter::once(&mut self.right))) + } + + fn to_format_node( + &self, + ctx: &mut FormatContext<'_>, + children: Vec>, + ) -> Result> { + let mut node_children = vec![FormatTreeNode::new(format!( + "output columns: [{}]", + format_output_columns(self.output_schema()?, ctx.metadata, true) + ))]; + + if let Some(info) = &self.stat_info { + let items = plan_stats_info_to_format_tree(info); + node_children.extend(items); + } + + let root = if !self.cte_scan_names.is_empty() { + "UnionAll(recursive cte)".to_string() + } else { + "UnionAll".to_string() + }; + + node_children.extend(children); + Ok(FormatTreeNode::with_children(root, node_children)) + } + + fn get_desc(&self) -> Result { + Ok(self + .left_outputs + .iter() + .zip(self.right_outputs.iter()) + .map(|(l, r)| format!("#{} <- #{}", l.0, r.0)) + .join(", ")) + } + + fn derive(&self, mut children: Vec) -> PhysicalPlan { + let mut new_union_all = self.clone(); + assert_eq!(children.len(), 2); + new_union_all.right = children.pop().unwrap(); + new_union_all.left = children.pop().unwrap(); + Box::new(new_union_all) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + if !self.cte_scan_names.is_empty() { + return self.build_recursive_cte_source(builder); + } + + self.left.build_pipeline(builder)?; + self.project_input(self.left.output_schema()?, &self.left_outputs, builder)?; + let left_sinks = builder.main_pipeline.take_sinks(); + + self.right.build_pipeline(builder)?; + self.project_input(self.right.output_schema()?, &self.right_outputs, builder)?; + let right_sinks = builder.main_pipeline.take_sinks(); + + let outputs = std::cmp::max(left_sinks.len(), right_sinks.len()); + let sequence_groups = vec![(left_sinks.len(), false), (right_sinks.len(), false)]; + + builder.main_pipeline.extend_sinks(left_sinks); + builder.main_pipeline.extend_sinks(right_sinks); + + match builder.settings.get_enable_parallel_union_all()? { + true => builder.main_pipeline.resize(outputs, false), + false => builder + .main_pipeline + .sequence_group(sequence_groups, outputs), + } + } +} + +impl UnionAll { + fn project_input( + &self, + schema: DataSchemaRef, + projection: &[(IndexType, Option)], + builder: &mut PipelineBuilder, + ) -> Result<()> { + let mut expr_offset = schema.num_fields(); + let mut new_projection = Vec::with_capacity(projection.len()); + let mut exprs = Vec::with_capacity(projection.len()); + for (idx, expr) in projection { + let Some(expr) = expr else { + new_projection.push(schema.index_of(&idx.to_string())?); + continue; + }; + + exprs.push(expr.as_expr(&BUILTIN_FUNCTIONS)); + new_projection.push(expr_offset); + expr_offset += 1; + } + + let mut operators = Vec::with_capacity(2); + if !exprs.is_empty() { + operators.push(BlockOperator::Map { + exprs, + projections: None, + }); + } + + operators.push(BlockOperator::Project { + projection: new_projection, + }); + + builder.main_pipeline.add_transform(|input, output| { + Ok(ProcessorPtr::create(CompoundBlockOperator::create( + input, + output, + schema.num_fields(), + builder.func_ctx.clone(), + operators.clone(), + ))) + }) + } + + fn build_recursive_cte_source(&self, builder: &mut PipelineBuilder) -> Result<()> { + let max_threads = builder.settings.get_max_threads()?; + builder.main_pipeline.add_source( + |output_port| { + TransformRecursiveCteSource::try_create( + builder.ctx.clone(), + output_port.clone(), + self.clone(), + ) + }, + 1, + )?; + + builder.main_pipeline.resize(max_threads as usize, true) + } +} + +impl PhysicalPlanBuilder { + pub async fn build_union_all( + &mut self, + s_expr: &SExpr, + union_all: &databend_common_sql::plans::UnionAll, + mut required: ColumnSet, + stat_info: PlanStatsInfo, + ) -> Result { + // 1. Prune unused Columns. + let metadata = self.metadata.read().clone(); + let lazy_columns = metadata.lazy_columns(); + required.extend(lazy_columns); + + // Use left's output columns as the offset indices + // if the union has a CTE, the output columns are not filtered + // otherwise, if the output columns of the union do not contain the columns used by the plan in the union, the expression will fail to obtain data. + let (offset_indices, left_required, right_required) = + if !union_all.cte_scan_names.is_empty() { + let left: ColumnSet = union_all + .left_outputs + .iter() + .map(|(index, _)| *index) + .collect(); + let right: ColumnSet = union_all + .right_outputs + .iter() + .map(|(index, _)| *index) + .collect(); + + let offset_indices: Vec = (0..union_all.left_outputs.len()).collect(); + (offset_indices, left, right) + } else { + let offset_indices: Vec = (0..union_all.left_outputs.len()) + .filter(|index| required.contains(&union_all.output_indexes[*index])) + .collect(); + + if offset_indices.is_empty() { + ( + vec![0], + ColumnSet::from([union_all.left_outputs[0].0]), + ColumnSet::from([union_all.right_outputs[0].0]), + ) + } else { + offset_indices.iter().fold( + (vec![], ColumnSet::default(), ColumnSet::default()), + |(mut offset_indices, mut left, mut right), &index| { + left.insert(union_all.left_outputs[index].0); + right.insert(union_all.right_outputs[index].0); + offset_indices.push(index); + (offset_indices, left, right) + }, + ) + } + }; + + // 2. Build physical plan. + let left_plan = self.build(s_expr.child(0)?, left_required.clone()).await?; + let right_plan = self.build(s_expr.child(1)?, right_required.clone()).await?; + + let left_schema = left_plan.output_schema()?; + let right_schema = right_plan.output_schema()?; + + let left_outputs = process_outputs(&union_all.left_outputs, &offset_indices, &left_schema)?; + let right_outputs = + process_outputs(&union_all.right_outputs, &offset_indices, &right_schema)?; + + let mut fields = Vec::with_capacity(offset_indices.len()); + for offset in offset_indices { + let index = union_all.output_indexes[offset]; + let data_type = if let Some(scalar_expr) = &union_all.left_outputs[offset].1 { + let expr = scalar_expr + .type_check(left_schema.as_ref())? + .project_column_ref(|idx| left_schema.index_of(&idx.to_string()).unwrap()); + expr.data_type().clone() + } else { + let col_index = union_all.left_outputs[offset].0; + left_schema + .field_with_name(&col_index.to_string())? + .data_type() + .clone() + }; + + fields.push(DataField::new(&index.to_string(), data_type)); + } + + Ok(Box::new(UnionAll { + meta: PhysicalPlanMeta::new("UnionAll"), + left: left_plan, + right: right_plan, + left_outputs, + right_outputs, + schema: DataSchemaRefExt::create(fields), + + cte_scan_names: union_all.cte_scan_names.clone(), + stat_info: Some(stat_info), + })) + } +} + +fn process_outputs( + outputs: &[(IndexType, Option)], + offset_indices: &[usize], + schema: &DataSchema, +) -> Result)>> { + let mut results = Vec::with_capacity(offset_indices.len()); + for index in offset_indices { + let output = &outputs[*index]; + if let Some(scalar_expr) = &output.1 { + let expr = scalar_expr + .type_check(schema)? + .project_column_ref(|idx| schema.index_of(&idx.to_string()).unwrap()); + results.push((output.0, Some(expr.as_remote_expr()))); + } else { + results.push((output.0, None)); + } + } + Ok(results) +} diff --git a/src/query/sql/src/executor/physical_plans/physical_window.rs b/src/query/service/src/physical_plans/physical_window.rs similarity index 59% rename from src/query/sql/src/executor/physical_plans/physical_window.rs rename to src/query/service/src/physical_plans/physical_window.rs index f2cac19cb0fa1..ba2ad1d7b6da7 100644 --- a/src/query/sql/src/executor/physical_plans/physical_window.rs +++ b/src/query/service/src/physical_plans/physical_window.rs @@ -12,14 +12,18 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::any::Any; use std::fmt::Display; +use databend_common_ast::ast::FormatTreeNode; +use databend_common_catalog::plan::DataSourcePlan; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::type_check; use databend_common_expression::type_check::common_super_type; use databend_common_expression::types::DataType; use databend_common_expression::types::NumberDataType; +use databend_common_expression::with_number_mapped_type; use databend_common_expression::Constant; use databend_common_expression::ConstantFolder; use databend_common_expression::DataField; @@ -29,29 +33,40 @@ use databend_common_expression::Expr; use databend_common_expression::FunctionContext; use databend_common_expression::RawExpr; use databend_common_functions::BUILTIN_FUNCTIONS; +use databend_common_pipeline_core::processors::Processor; +use databend_common_pipeline_core::processors::ProcessorPtr; +use databend_common_sql::binder::wrap_cast; +use databend_common_sql::executor::physical_plans::AggregateFunctionDesc; +use databend_common_sql::executor::physical_plans::AggregateFunctionSignature; +use databend_common_sql::executor::physical_plans::SortDesc; +use databend_common_sql::optimizer::ir::SExpr; +use databend_common_sql::plans::WindowFuncFrame; +use databend_common_sql::plans::WindowFuncFrameBound; +use databend_common_sql::plans::WindowFuncType; +use databend_common_sql::ColumnSet; +use databend_common_sql::IndexType; +use databend_common_sql::ScalarExpr; +use databend_common_sql::TypeCheck; -use crate::binder::wrap_cast; -use crate::executor::explain::PlanStatsInfo; -use crate::executor::physical_plans::common::AggregateFunctionDesc; -use crate::executor::physical_plans::common::AggregateFunctionSignature; -use crate::executor::physical_plans::common::SortDesc; -use crate::executor::PhysicalPlan; -use crate::executor::PhysicalPlanBuilder; -use crate::optimizer::ir::SExpr; -use crate::plans::WindowFuncFrame; -use crate::plans::WindowFuncFrameBound; -use crate::plans::WindowFuncType; -use crate::ColumnSet; -use crate::IndexType; -use crate::ScalarExpr; -use crate::TypeCheck; +use crate::physical_plans::explain::PlanStatsInfo; +use crate::physical_plans::format::format_output_columns; +use crate::physical_plans::format::pretty_display_agg_desc; +use crate::physical_plans::format::FormatContext; +use crate::physical_plans::physical_plan::IPhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlanMeta; +use crate::physical_plans::PhysicalPlanBuilder; +use crate::pipelines::processors::transforms::FrameBound; +use crate::pipelines::processors::transforms::TransformWindow; +use crate::pipelines::processors::transforms::WindowFunctionInfo; +use crate::pipelines::processors::transforms::WindowSortDesc; +use crate::pipelines::PipelineBuilder; #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct Window { - // A unique id of operator in a `PhysicalPlan` tree, only used for display. - pub plan_id: u32, + pub meta: PhysicalPlanMeta, pub index: IndexType, - pub input: Box, + pub input: PhysicalPlan, pub func: WindowFunction, pub partition_by: Vec, pub order_by: Vec, @@ -59,8 +74,20 @@ pub struct Window { pub limit: Option, } -impl Window { - pub fn output_schema(&self) -> Result { +#[typetag::serde] +impl IPhysicalPlan for Window { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn output_schema(&self) -> Result { let input_schema = self.input.output_schema()?; let mut fields = Vec::with_capacity(input_schema.fields().len() + 1); fields.extend_from_slice(input_schema.fields()); @@ -70,6 +97,197 @@ impl Window { )); Ok(DataSchemaRefExt::create(fields)) } + + fn children<'a>(&'a self) -> Box + 'a> { + Box::new(std::iter::once(&self.input)) + } + + fn children_mut<'a>(&'a mut self) -> Box + 'a> { + Box::new(std::iter::once(&mut self.input)) + } + + fn to_format_node( + &self, + ctx: &mut FormatContext<'_>, + children: Vec>, + ) -> Result> { + let partition_by = self + .partition_by + .iter() + .map(|&index| Ok(ctx.metadata.column(index).name())) + .collect::>>()? + .join(", "); + + let order_by = self + .order_by + .iter() + .map(|v| v.display_name.clone()) + .collect::>() + .join(", "); + + let frame = self.window_frame.to_string(); + + let func = match &self.func { + WindowFunction::Aggregate(agg) => pretty_display_agg_desc(agg, ctx.metadata), + func => format!("{}", func), + }; + + let mut node_children = vec![ + FormatTreeNode::new(format!( + "output columns: [{}]", + format_output_columns(self.output_schema()?, ctx.metadata, true) + )), + FormatTreeNode::new(format!("aggregate function: [{func}]")), + FormatTreeNode::new(format!("partition by: [{partition_by}]")), + FormatTreeNode::new(format!("order by: [{order_by}]")), + FormatTreeNode::new(format!("frame: [{frame}]")), + ]; + + if let Some(limit) = self.limit { + node_children.push(FormatTreeNode::new(format!("limit: [{limit}]"))) + } + + node_children.extend(children); + Ok(FormatTreeNode::with_children( + "Window".to_string(), + node_children, + )) + } + + #[recursive::recursive] + fn try_find_single_data_source(&self) -> Option<&DataSourcePlan> { + self.input.try_find_single_data_source() + } + + fn get_desc(&self) -> Result { + let partition_by = self + .partition_by + .iter() + .map(|x| x.to_string()) + .collect::>() + .join(", "); + + let order_by = self + .order_by + .iter() + .map(|x| { + format!( + "{}{}{}", + x.display_name, + if x.asc { "" } else { " DESC" }, + if x.nulls_first { " NULLS FIRST" } else { "" }, + ) + }) + .collect::>() + .join(", "); + + Ok(format!( + "partition by {}, order by {}", + partition_by, order_by + )) + } + + fn derive(&self, mut children: Vec) -> PhysicalPlan { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + self.input.build_pipeline(builder)?; + + let input_schema = self.input.output_schema()?; + let partition_by = self + .partition_by + .iter() + .map(|p| { + let offset = input_schema.index_of(&p.to_string())?; + Ok(offset) + }) + .collect::>>()?; + let order_by = self + .order_by + .iter() + .map(|o| { + let offset = input_schema.index_of(&o.order_by.to_string())?; + Ok(WindowSortDesc { + offset, + asc: o.asc, + nulls_first: o.nulls_first, + is_nullable: input_schema.field(offset).is_nullable(), + }) + }) + .collect::>>()?; + + let old_output_len = builder.main_pipeline.output_len(); + // `TransformWindow` is a pipeline breaker. + if partition_by.is_empty() { + builder.main_pipeline.try_resize(1)?; + } + let func = WindowFunctionInfo::try_create(&self.func, &input_schema)?; + // Window + builder.main_pipeline.add_transform(|input, output| { + // The transform can only be created here, because it cannot be cloned. + + let transform = if self.window_frame.units.is_rows() { + let start_bound = FrameBound::try_from(&self.window_frame.start_bound)?; + let end_bound = FrameBound::try_from(&self.window_frame.end_bound)?; + Box::new(TransformWindow::::try_create_rows( + input, + output, + func.clone(), + partition_by.clone(), + order_by.clone(), + (start_bound, end_bound), + )?) as Box + } else { + if order_by.len() == 1 { + // If the length of order_by is 1, there may be a RANGE frame. + let data_type = input_schema + .field(order_by[0].offset) + .data_type() + .remove_nullable(); + with_number_mapped_type!(|NUM_TYPE| match data_type { + DataType::Number(NumberDataType::NUM_TYPE) => { + let start_bound = FrameBound::try_from(&self.window_frame.start_bound)?; + let end_bound = FrameBound::try_from(&self.window_frame.end_bound)?; + return Ok(ProcessorPtr::create(Box::new( + TransformWindow::::try_create_range( + input, + output, + func.clone(), + partition_by.clone(), + order_by.clone(), + (start_bound, end_bound), + )?, + ) + as Box)); + } + _ => {} + }) + } + + // There is no offset in the RANGE frame. (just CURRENT ROW or UNBOUNDED) + // So we can use any number type to create the transform. + let start_bound = FrameBound::try_from(&self.window_frame.start_bound)?; + let end_bound = FrameBound::try_from(&self.window_frame.end_bound)?; + Box::new(TransformWindow::::try_create_range( + input, + output, + func.clone(), + partition_by.clone(), + order_by.clone(), + (start_bound, end_bound), + )?) as Box + }; + Ok(ProcessorPtr::create(transform)) + })?; + if partition_by.is_empty() { + builder.main_pipeline.try_resize(old_output_len)?; + } + Ok(()) + } } #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] @@ -150,10 +368,10 @@ pub struct NtileFunctionDesc { } impl PhysicalPlanBuilder { - pub(crate) async fn build_window( + pub async fn build_window( &mut self, s_expr: &SExpr, - window: &crate::plans::Window, + window: &databend_common_sql::plans::Window, mut required: ColumnSet, _stat_info: PlanStatsInfo, ) -> Result { @@ -360,15 +578,15 @@ impl PhysicalPlanBuilder { WindowFuncType::CumeDist => WindowFunction::CumeDist, }; - Ok(PhysicalPlan::Window(Window { - plan_id: 0, + Ok(Box::new(Window { + input, index: w.index, - input: Box::new(input), func, partition_by: partition_items, order_by: order_by_items, window_frame: w.frame.clone(), limit: w.limit, + meta: PhysicalPlanMeta::new("Window"), })) } } diff --git a/src/query/service/src/physical_plans/physical_window_partition.rs b/src/query/service/src/physical_plans/physical_window_partition.rs new file mode 100644 index 0000000000000..2ff1e722fc63d --- /dev/null +++ b/src/query/service/src/physical_plans/physical_window_partition.rs @@ -0,0 +1,223 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::any::Any; +use std::sync::atomic; +use std::sync::atomic::AtomicUsize; + +use databend_common_ast::ast::FormatTreeNode; +use databend_common_catalog::plan::DataSourcePlan; +use databend_common_catalog::table_context::TableContext; +use databend_common_exception::Result; +use databend_common_expression::SortColumnDescription; +use databend_common_pipeline_core::processors::ProcessorPtr; +use databend_common_pipeline_transforms::MemorySettings; +use databend_common_sql::executor::physical_plans::SortDesc; +use databend_common_sql::IndexType; +use databend_storages_common_cache::TempDirManager; + +use crate::physical_plans::explain::PlanStatsInfo; +use crate::physical_plans::format::format_output_columns; +use crate::physical_plans::format::plan_stats_info_to_format_tree; +use crate::physical_plans::format::FormatContext; +use crate::physical_plans::physical_plan::IPhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlan; +use crate::physical_plans::physical_plan::PhysicalPlanMeta; +use crate::pipelines::memory_settings::MemorySettingsExt; +use crate::pipelines::processors::transforms::SortStrategy; +use crate::pipelines::processors::transforms::TransformWindowPartitionCollect; +use crate::pipelines::processors::transforms::WindowPartitionExchange; +use crate::pipelines::processors::transforms::WindowPartitionTopNExchange; +use crate::pipelines::PipelineBuilder; +use crate::spillers::SpillerDiskConfig; + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct WindowPartition { + pub meta: PhysicalPlanMeta, + pub input: PhysicalPlan, + pub partition_by: Vec, + pub order_by: Vec, + pub after_exchange: Option, + pub top_n: Option, + + pub stat_info: Option, +} + +#[typetag::serde] +impl IPhysicalPlan for WindowPartition { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta + } + + fn get_meta_mut(&mut self) -> &mut PhysicalPlanMeta { + &mut self.meta + } + + fn children<'a>(&'a self) -> Box + 'a> { + Box::new(std::iter::once(&self.input)) + } + + fn children_mut<'a>(&'a mut self) -> Box + 'a> { + Box::new(std::iter::once(&mut self.input)) + } + + fn to_format_node( + &self, + ctx: &mut FormatContext<'_>, + children: Vec>, + ) -> Result> { + let partition_by = self + .partition_by + .iter() + .map(|&index| Ok(ctx.metadata.column(index).name())) + .collect::>>()? + .join(", "); + + let mut node_children = vec![ + FormatTreeNode::new(format!( + "output columns: [{}]", + format_output_columns(self.output_schema()?, ctx.metadata, true) + )), + FormatTreeNode::new(format!("hash keys: [{partition_by}]")), + ]; + + if let Some(top_n) = &self.top_n { + node_children.push(FormatTreeNode::new(format!("top: {}", top_n.top))); + } + + if let Some(info) = &self.stat_info { + node_children.extend(plan_stats_info_to_format_tree(info)); + } + + node_children.extend(children); + Ok(FormatTreeNode::with_children( + "WindowPartition".to_string(), + node_children, + )) + } + + #[recursive::recursive] + fn try_find_single_data_source(&self) -> Option<&DataSourcePlan> { + self.input.try_find_single_data_source() + } + + fn derive(&self, mut children: Vec) -> PhysicalPlan { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } + + fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { + self.input.build_pipeline(builder)?; + + let num_processors = builder.main_pipeline.output_len(); + + // Settings. + let settings = builder.settings.clone(); + let num_partitions = builder.settings.get_window_num_partitions()?; + + let plan_schema = self.output_schema()?; + + let partition_by = self + .partition_by + .iter() + .map(|index| plan_schema.index_of(&index.to_string())) + .collect::>>()?; + + let sort_desc = self + .order_by + .iter() + .map(|desc| { + let offset = plan_schema.index_of(&desc.order_by.to_string())?; + Ok(SortColumnDescription { + offset, + asc: desc.asc, + nulls_first: desc.nulls_first, + }) + }) + .collect::>>()?; + + if let Some(top_n) = &self.top_n + && top_n.top < 10000 + { + builder.main_pipeline.exchange( + num_processors, + WindowPartitionTopNExchange::create( + partition_by.clone(), + sort_desc.clone(), + top_n.top, + top_n.func, + num_partitions as u64, + ), + ) + } else { + builder.main_pipeline.exchange( + num_processors, + WindowPartitionExchange::create(partition_by.clone(), num_partitions), + ); + } + + let temp_dir_manager = TempDirManager::instance(); + let disk_bytes_limit = settings.get_window_partition_spilling_to_disk_bytes_limit()?; + let enable_dio = settings.get_enable_dio()?; + let disk_spill = temp_dir_manager + .get_disk_spill_dir(disk_bytes_limit, &builder.ctx.get_id()) + .map(|temp_dir| SpillerDiskConfig::new(temp_dir, enable_dio)) + .transpose()?; + + let have_order_col = self.after_exchange.unwrap_or(false); + let window_spill_settings = MemorySettings::from_window_settings(&builder.ctx)?; + + let processor_id = AtomicUsize::new(0); + builder.main_pipeline.add_transform(|input, output| { + let strategy = SortStrategy::try_create( + &settings, + sort_desc.clone(), + plan_schema.clone(), + have_order_col, + )?; + Ok(ProcessorPtr::create(Box::new( + TransformWindowPartitionCollect::new( + builder.ctx.clone(), + input, + output, + &settings, + processor_id.fetch_add(1, atomic::Ordering::AcqRel), + num_processors, + num_partitions, + window_spill_settings.clone(), + disk_spill.clone(), + strategy, + )?, + ))) + }) + } +} + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct WindowPartitionTopN { + pub func: WindowPartitionTopNFunc, + pub top: usize, +} + +#[derive(Clone, Copy, Debug, serde::Serialize, serde::Deserialize)] +pub enum WindowPartitionTopNFunc { + RowNumber, + Rank, + DenseRank, +} diff --git a/src/query/service/src/pipelines/builders/builder_add_stream_column.rs b/src/query/service/src/pipelines/builders/builder_add_stream_column.rs deleted file mode 100644 index fd9df6bc7cbd1..0000000000000 --- a/src/query/service/src/pipelines/builders/builder_add_stream_column.rs +++ /dev/null @@ -1,60 +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_functions::BUILTIN_FUNCTIONS; -use databend_common_pipeline_transforms::processors::TransformPipelineHelper; -use databend_common_sql::evaluator::BlockOperator; -use databend_common_sql::executor::physical_plans::AddStreamColumn; -use databend_common_sql::StreamContext; -use databend_common_storages_fuse::TableContext; - -use crate::pipelines::processors::transforms::TransformAddStreamColumns; -use crate::pipelines::PipelineBuilder; - -impl PipelineBuilder { - pub(crate) fn build_add_stream_column( - &mut self, - add_stream_column: &AddStreamColumn, - ) -> Result<()> { - self.build_pipeline(&add_stream_column.input)?; - - let exprs = add_stream_column - .exprs - .iter() - .map(|expr| expr.as_expr(&BUILTIN_FUNCTIONS)) - .collect::>(); - - let operators = vec![ - BlockOperator::Map { - exprs, - projections: None, - }, - BlockOperator::Project { - projection: add_stream_column.projections.clone(), - }, - ]; - - let stream_context = StreamContext { - stream_columns: add_stream_column.stream_columns.clone(), - operators, - func_ctx: self.ctx.get_function_context()?, - }; - - self.main_pipeline - .add_transformer(|| TransformAddStreamColumns::new(stream_context.clone())); - - Ok(()) - } -} diff --git a/src/query/service/src/pipelines/builders/builder_aggregate.rs b/src/query/service/src/pipelines/builders/builder_aggregate.rs index 3af3a4fdb0186..45081cb977338 100644 --- a/src/query/service/src/pipelines/builders/builder_aggregate.rs +++ b/src/query/service/src/pipelines/builders/builder_aggregate.rs @@ -14,225 +14,23 @@ use std::sync::Arc; -use databend_common_catalog::table_context::TableContext; use databend_common_exception::Result; use databend_common_expression::AggregateFunctionRef; use databend_common_expression::DataField; use databend_common_expression::DataSchemaRef; -use databend_common_expression::HashTableConfig; -use databend_common_expression::LimitType; -use databend_common_expression::SortColumnDescription; use databend_common_functions::aggregates::AggregateFunctionFactory; use databend_common_functions::aggregates::AggregateFunctionSortDesc; -use databend_common_pipeline_core::processors::ProcessorPtr; -use databend_common_pipeline_transforms::processors::TransformPipelineHelper; -use databend_common_pipeline_transforms::processors::TransformSortPartial; -use databend_common_sql::executor::physical_plans::AggregateExpand; -use databend_common_sql::executor::physical_plans::AggregateFinal; use databend_common_sql::executor::physical_plans::AggregateFunctionDesc; -use databend_common_sql::executor::physical_plans::AggregatePartial; -use databend_common_sql::executor::PhysicalPlan; use databend_common_sql::plans::UDFType; use databend_common_sql::IndexType; -use databend_common_storage::DataOperator; use itertools::Itertools; -use crate::pipelines::processors::transforms::aggregator::build_partition_bucket; use crate::pipelines::processors::transforms::aggregator::create_udaf_script_function; -use crate::pipelines::processors::transforms::aggregator::AggregateInjector; use crate::pipelines::processors::transforms::aggregator::AggregatorParams; -use crate::pipelines::processors::transforms::aggregator::FinalSingleStateAggregator; -use crate::pipelines::processors::transforms::aggregator::PartialSingleStateAggregator; -use crate::pipelines::processors::transforms::aggregator::TransformAggregateSpillWriter; -use crate::pipelines::processors::transforms::aggregator::TransformExpandGroupingSets; -use crate::pipelines::processors::transforms::aggregator::TransformPartialAggregate; use crate::pipelines::PipelineBuilder; impl PipelineBuilder { - pub(crate) fn build_aggregate_expand(&mut self, expand: &AggregateExpand) -> Result<()> { - self.build_pipeline(&expand.input)?; - let input_schema = expand.input.output_schema()?; - let group_bys = expand - .group_bys - .iter() - .take(expand.group_bys.len() - 1) // The last group-by will be virtual column `_grouping_id` - .map(|i| input_schema.index_of(&i.to_string())) - .collect::>>()?; - let grouping_sets = expand - .grouping_sets - .sets - .iter() - .map(|sets| { - sets.iter() - .map(|i| { - let i = input_schema.index_of(&i.to_string())?; - let offset = group_bys.iter().position(|j| *j == i).unwrap(); - Ok(offset) - }) - .collect::>>() - }) - .collect::>>()?; - let mut grouping_ids = Vec::with_capacity(grouping_sets.len()); - let mask = (1 << group_bys.len()) - 1; - for set in grouping_sets { - let mut id = 0; - for i in set { - id |= 1 << i; - } - // For element in `group_bys`, - // if it is in current grouping set: set 0, else: set 1. (1 represents it will be NULL in grouping) - // Example: GROUP BY GROUPING SETS ((a, b), (a), (b), ()) - // group_bys: [a, b] - // grouping_sets: [[0, 1], [0], [1], []] - // grouping_ids: 00, 01, 10, 11 - grouping_ids.push(!id & mask); - } - - self.main_pipeline.add_accumulating_transformer(|| { - TransformExpandGroupingSets::new(group_bys.clone(), grouping_ids.clone()) - }); - Ok(()) - } - - pub(crate) fn build_aggregate_partial(&mut self, aggregate: &AggregatePartial) -> Result<()> { - self.contain_sink_processor = true; - self.build_pipeline(&aggregate.input)?; - - let max_block_size = self.settings.get_max_block_size()?; - let max_threads = self.settings.get_max_threads()?; - let max_spill_io_requests = self.settings.get_max_spill_io_requests()?; - - let enable_experimental_aggregate_hashtable = self - .settings - .get_enable_experimental_aggregate_hashtable()?; - - let params = Self::build_aggregator_params( - aggregate.input.output_schema()?, - &aggregate.group_by, - &aggregate.agg_funcs, - enable_experimental_aggregate_hashtable, - self.is_exchange_parent(), - max_block_size as usize, - max_spill_io_requests as usize, - )?; - - if params.group_columns.is_empty() { - return self.main_pipeline.try_add_accumulating_transformer(|| { - PartialSingleStateAggregator::try_new(¶ms) - }); - } - - let schema_before_group_by = params.input_schema.clone(); - - // Need a global atomic to read the max current radix bits hint - let partial_agg_config = if !self.is_exchange_parent() { - HashTableConfig::default().with_partial(true, max_threads as usize) - } else { - HashTableConfig::default() - .cluster_with_partial(true, self.ctx.get_cluster().nodes.len()) - }; - - // For rank limit, we can filter data using sort with rank before partial - if let Some(rank_limit) = &aggregate.rank_limit { - let sort_desc = rank_limit - .0 - .iter() - .map(|desc| { - let offset = schema_before_group_by.index_of(&desc.order_by.to_string())?; - Ok(SortColumnDescription { - offset, - asc: desc.asc, - nulls_first: desc.nulls_first, - }) - }) - .collect::>>()?; - let sort_desc: Arc<[_]> = sort_desc.into(); - - self.main_pipeline.add_transformer(|| { - TransformSortPartial::new(LimitType::LimitRank(rank_limit.1), sort_desc.clone()) - }); - } - - self.main_pipeline.add_transform(|input, output| { - Ok(ProcessorPtr::create(TransformPartialAggregate::try_create( - self.ctx.clone(), - input, - output, - params.clone(), - partial_agg_config.clone(), - )?)) - })?; - - // If cluster mode, spill write will be completed in exchange serialize, because we need scatter the block data first - if !self.is_exchange_parent() { - let operator = DataOperator::instance().spill_operator(); - let location_prefix = self.ctx.query_id_spill_prefix(); - - self.main_pipeline.add_transform(|input, output| { - Ok(ProcessorPtr::create( - TransformAggregateSpillWriter::try_create( - self.ctx.clone(), - input, - output, - operator.clone(), - params.clone(), - location_prefix.clone(), - )?, - )) - })?; - } - - self.exchange_injector = AggregateInjector::create(self.ctx.clone(), params.clone()); - Ok(()) - } - - pub(crate) fn build_aggregate_final(&mut self, aggregate: &AggregateFinal) -> Result<()> { - let max_block_size = self.settings.get_max_block_size()?; - let enable_experimental_aggregate_hashtable = self - .settings - .get_enable_experimental_aggregate_hashtable()?; - let max_spill_io_requests = self.settings.get_max_spill_io_requests()?; - let max_restore_worker = self.settings.get_max_aggregate_restore_worker()?; - - let mut is_cluster_aggregate = false; - if matches!(aggregate.input.as_ref(), PhysicalPlan::ExchangeSource(_)) { - is_cluster_aggregate = true; - } - - let params = Self::build_aggregator_params( - aggregate.before_group_by_schema.clone(), - &aggregate.group_by, - &aggregate.agg_funcs, - enable_experimental_aggregate_hashtable, - is_cluster_aggregate, - max_block_size as usize, - max_spill_io_requests as usize, - )?; - - if params.group_columns.is_empty() { - self.build_pipeline(&aggregate.input)?; - self.main_pipeline.try_resize(1)?; - self.main_pipeline.add_transform(|input, output| { - Ok(ProcessorPtr::create( - FinalSingleStateAggregator::try_create(input, output, ¶ms)?, - )) - })?; - - return Ok(()); - } - - let old_inject = self.exchange_injector.clone(); - - let input: &PhysicalPlan = &aggregate.input; - if matches!(input, PhysicalPlan::ExchangeSource(_)) { - self.exchange_injector = AggregateInjector::create(self.ctx.clone(), params.clone()); - } - self.build_pipeline(&aggregate.input)?; - self.exchange_injector = old_inject; - build_partition_bucket(&mut self.main_pipeline, params.clone(), max_restore_worker) - } - - fn build_aggregator_params( + pub fn build_aggregator_params( input_schema: DataSchemaRef, group_by: &[IndexType], agg_funcs: &[AggregateFunctionDesc], diff --git a/src/query/service/src/pipelines/builders/builder_async_function.rs b/src/query/service/src/pipelines/builders/builder_async_function.rs deleted file mode 100644 index efc805f7f118c..0000000000000 --- a/src/query/service/src/pipelines/builders/builder_async_function.rs +++ /dev/null @@ -1,41 +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::AsyncFunction; - -use crate::pipelines::processors::transforms::TransformAsyncFunction; -use crate::pipelines::PipelineBuilder; - -impl PipelineBuilder { - pub(crate) fn build_async_function(&mut self, async_function: &AsyncFunction) -> Result<()> { - self.build_pipeline(&async_function.input)?; - - let operators = TransformAsyncFunction::init_operators(&async_function.async_func_descs)?; - let sequence_counters = - TransformAsyncFunction::create_sequence_counters(async_function.async_func_descs.len()); - - self.main_pipeline.add_async_transformer(|| { - TransformAsyncFunction::new( - self.ctx.clone(), - async_function.async_func_descs.clone(), - operators.clone(), - sequence_counters.clone(), - ) - }); - - Ok(()) - } -} diff --git a/src/query/service/src/pipelines/builders/builder_broadcast.rs b/src/query/service/src/pipelines/builders/builder_broadcast.rs deleted file mode 100644 index fc2b8ab28a9ad..0000000000000 --- a/src/query/service/src/pipelines/builders/builder_broadcast.rs +++ /dev/null @@ -1,39 +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_sql::executor::physical_plans::BroadcastSink; -use databend_common_sql::executor::physical_plans::BroadcastSource; - -use crate::pipelines::processors::transforms::BroadcastSinkProcessor; -use crate::pipelines::processors::transforms::BroadcastSourceProcessor; -use crate::pipelines::PipelineBuilder; - -impl PipelineBuilder { - pub(crate) fn build_broadcast_source(&mut self, source: &BroadcastSource) -> Result<()> { - let receiver = self.ctx.broadcast_source_receiver(source.broadcast_id); - self.main_pipeline.add_source( - |output| BroadcastSourceProcessor::create(self.ctx.clone(), receiver.clone(), output), - 1, - ) - } - - pub(crate) fn build_broadcast_sink(&mut self, sink: &BroadcastSink) -> Result<()> { - self.build_pipeline(&sink.input)?; - self.main_pipeline.resize(1, true)?; - self.main_pipeline.add_sink(|input| { - BroadcastSinkProcessor::create(input, self.ctx.broadcast_sink_sender(sink.broadcast_id)) - }) - } -} diff --git a/src/query/service/src/pipelines/builders/builder_commit.rs b/src/query/service/src/pipelines/builders/builder_commit.rs deleted file mode 100644 index 23ff8730e7347..0000000000000 --- a/src/query/service/src/pipelines/builders/builder_commit.rs +++ /dev/null @@ -1,142 +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::table_context::TableContext; -use databend_common_exception::Result; -use databend_common_pipeline_core::ExecutionInfo; -use databend_common_pipeline_transforms::processors::TransformPipelineHelper; -use databend_common_sql::executor::physical_plans::CommitSink as PhysicalCommitSink; -use databend_common_sql::executor::physical_plans::CommitType; -use databend_common_sql::executor::physical_plans::MutationKind; -use databend_common_sql::plans::TruncateMode; -use databend_common_storages_fuse::operations::CommitSink; -use databend_common_storages_fuse::operations::MutationGenerator; -use databend_common_storages_fuse::operations::TableMutationAggregator; -use databend_common_storages_fuse::operations::TransformMergeCommitMeta; -use databend_common_storages_fuse::operations::TruncateGenerator; -use databend_common_storages_fuse::FuseTable; -use databend_storages_common_table_meta::meta::ExtendedBlockMeta; -use databend_storages_common_table_meta::readers::snapshot_reader::TableSnapshotAccessor; - -use crate::pipelines::PipelineBuilder; - -impl PipelineBuilder { - pub(crate) fn build_commit_sink(&mut self, plan: &PhysicalCommitSink) -> Result<()> { - self.build_pipeline(&plan.input)?; - let table = self.ctx.build_table_by_table_info(&plan.table_info, None)?; - let table = FuseTable::try_from_table(table.as_ref())?; - - self.main_pipeline.try_resize(1)?; - match &plan.commit_type { - CommitType::Truncate { mode } => { - let prev_snapshot_id = match mode { - TruncateMode::Delete => None, - _ => plan.snapshot.as_ref().map(|snapshot| snapshot.snapshot_id), - }; - let snapshot_gen = TruncateGenerator::new(mode.clone()); - if matches!(mode, TruncateMode::Delete) { - let mutation_status = self.ctx.get_mutation_status(); - let deleted_rows = plan - .snapshot - .as_ref() - .map_or(0, |snapshot| snapshot.summary.row_count); - self.main_pipeline - .set_on_finished(move |info: &ExecutionInfo| match &info.res { - Ok(_) => { - mutation_status.write().deleted_rows = deleted_rows; - Ok(()) - } - Err(error_code) => Err(error_code.clone()), - }); - } - self.main_pipeline.add_sink(|input| { - CommitSink::try_create( - table, - self.ctx.clone(), - None, - plan.update_stream_meta.clone(), - snapshot_gen.clone(), - input, - None, - prev_snapshot_id, - plan.deduplicated_label.clone(), - plan.table_meta_timestamps, - ) - }) - } - CommitType::Mutation { kind, merge_meta } => { - if *merge_meta { - let cluster_key_id = table.cluster_key_id(); - self.main_pipeline.add_accumulating_transformer(|| { - TransformMergeCommitMeta::create(cluster_key_id) - }); - } else { - self.main_pipeline.add_async_accumulating_transformer(|| { - let base_segments = if matches!( - kind, - MutationKind::Compact | MutationKind::Insert | MutationKind::Recluster - ) { - vec![] - } else { - plan.snapshot.segments().to_vec() - }; - - // extract re-cluster related mutations from physical plan - let recluster_info = plan.recluster_info.clone().unwrap_or_default(); - - let extended_merged_blocks = recluster_info - .merged_blocks - .iter() - .map(|block_meta| { - Arc::new(ExtendedBlockMeta { - block_meta: Arc::unwrap_or_clone(block_meta.clone()), - draft_virtual_block_meta: None, - }) - }) - .collect::>>(); - - TableMutationAggregator::create( - table, - self.ctx.clone(), - base_segments, - extended_merged_blocks, - recluster_info.removed_segment_indexes, - recluster_info.removed_statistics, - *kind, - plan.table_meta_timestamps, - ) - }); - } - - let snapshot_gen = MutationGenerator::new(plan.snapshot.clone(), *kind); - self.main_pipeline.add_sink(|input| { - CommitSink::try_create( - table, - self.ctx.clone(), - None, - plan.update_stream_meta.clone(), - snapshot_gen.clone(), - input, - None, - None, - plan.deduplicated_label.clone(), - plan.table_meta_timestamps, - ) - }) - } - } - } -} diff --git a/src/query/service/src/pipelines/builders/builder_compact.rs b/src/query/service/src/pipelines/builders/builder_compact.rs deleted file mode 100644 index 143e20c245220..0000000000000 --- a/src/query/service/src/pipelines/builders/builder_compact.rs +++ /dev/null @@ -1,173 +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_base::runtime::Runtime; -use databend_common_catalog::plan::PartInfoType; -use databend_common_catalog::plan::Partitions; -use databend_common_catalog::plan::PartitionsShuffleKind; -use databend_common_catalog::plan::Projection; -use databend_common_catalog::table::Table; -use databend_common_catalog::table_context::TableContext; -use databend_common_exception::Result; -use databend_common_pipeline_sources::EmptySource; -use databend_common_pipeline_sources::PrefetchAsyncSourcer; -use databend_common_pipeline_transforms::processors::TransformPipelineHelper; -use databend_common_sql::executor::physical_plans::CompactSource as PhysicalCompactSource; -use databend_common_sql::executor::physical_plans::MutationKind; -use databend_common_sql::StreamContext; -use databend_common_storages_fuse::operations::BlockCompactMutator; -use databend_common_storages_fuse::operations::CompactLazyPartInfo; -use databend_common_storages_fuse::operations::CompactSource; -use databend_common_storages_fuse::operations::CompactTransform; -use databend_common_storages_fuse::operations::TableMutationAggregator; -use databend_common_storages_fuse::operations::TransformSerializeBlock; -use databend_common_storages_fuse::FuseTable; - -use crate::pipelines::PipelineBuilder; - -impl PipelineBuilder { - pub(crate) fn build_compact_source( - &mut self, - compact_block: &PhysicalCompactSource, - ) -> Result<()> { - let table = self - .ctx - .build_table_by_table_info(&compact_block.table_info, None)?; - let table = FuseTable::try_from_table(table.as_ref())?; - - if compact_block.parts.is_empty() { - return self.main_pipeline.add_source(EmptySource::create, 1); - } - - let is_lazy = compact_block.parts.partitions_type() == PartInfoType::LazyLevel; - let thresholds = table.get_block_thresholds(); - let cluster_key_id = table.cluster_key_id(); - let mut max_threads = self.ctx.get_settings().get_max_threads()? as usize; - - if is_lazy { - let query_ctx = self.ctx.clone(); - - let lazy_parts = compact_block - .parts - .partitions - .iter() - .map(|v| { - v.as_any() - .downcast_ref::() - .unwrap() - .clone() - }) - .collect::>(); - - let column_ids = compact_block.column_ids.clone(); - self.main_pipeline.set_on_init(move || { - let ctx = query_ctx.clone(); - let partitions = - Runtime::with_worker_threads(2, Some("build_compact_tasks".to_string()))? - .block_on(async move { - let partitions = BlockCompactMutator::build_compact_tasks( - ctx.clone(), - column_ids.clone(), - cluster_key_id, - thresholds, - lazy_parts, - ) - .await?; - - Result::<_>::Ok(partitions) - })?; - - let partitions = Partitions::create(PartitionsShuffleKind::Mod, partitions); - query_ctx.set_partitions(partitions)?; - Ok(()) - }); - } else { - max_threads = max_threads.min(compact_block.parts.len()).max(1); - self.ctx.set_partitions(compact_block.parts.clone())?; - } - - let block_reader = table.create_block_reader( - self.ctx.clone(), - Projection::Columns(table.all_column_indices()), - false, - table.change_tracking_enabled(), - false, - )?; - let stream_ctx = if table.change_tracking_enabled() { - Some(StreamContext::try_create( - self.ctx.get_function_context()?, - table.schema_with_stream(), - table.get_table_info().ident.seq, - false, - false, - )?) - } else { - None - }; - // Add source pipe. - self.main_pipeline.add_source( - |output| { - let source = CompactSource::create(self.ctx.clone(), block_reader.clone(), 1); - PrefetchAsyncSourcer::create(self.ctx.clone(), output, source) - }, - max_threads, - )?; - let storage_format = table.get_storage_format(); - self.main_pipeline.add_block_meta_transformer(|| { - CompactTransform::create( - self.ctx.clone(), - block_reader.clone(), - storage_format, - stream_ctx.clone(), - ) - }); - - // sort - let cluster_stats_gen = table.cluster_gen_for_append( - self.ctx.clone(), - &mut self.main_pipeline, - thresholds, - None, - )?; - self.main_pipeline.add_transform(|input, output| { - let proc = TransformSerializeBlock::try_create( - self.ctx.clone(), - input, - output, - table, - cluster_stats_gen.clone(), - MutationKind::Compact, - compact_block.table_meta_timestamps, - )?; - proc.into_processor() - })?; - - if is_lazy { - self.main_pipeline.try_resize(1)?; - self.main_pipeline.add_async_accumulating_transformer(|| { - TableMutationAggregator::create( - table, - self.ctx.clone(), - vec![], - vec![], - vec![], - Default::default(), - MutationKind::Compact, - compact_block.table_meta_timestamps, - ) - }); - } - Ok(()) - } -} diff --git a/src/query/service/src/pipelines/builders/builder_copy_into_location.rs b/src/query/service/src/pipelines/builders/builder_copy_into_location.rs deleted file mode 100644 index 7f96489066399..0000000000000 --- a/src/query/service/src/pipelines/builders/builder_copy_into_location.rs +++ /dev/null @@ -1,54 +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 chrono::Duration; -use databend_common_catalog::table_context::TableContext; -use databend_common_exception::Result; -use databend_common_sql::executor::physical_plans::CopyIntoLocation; -use databend_common_storages_stage::StageSinkTable; -use databend_storages_common_table_meta::meta::TableMetaTimestamps; - -use crate::pipelines::PipelineBuilder; - -impl PipelineBuilder { - pub(crate) fn build_copy_into_location(&mut self, copy: &CopyIntoLocation) -> Result<()> { - self.build_pipeline(©.input)?; - - // Reorder the result for select clause - PipelineBuilder::build_result_projection( - &self.func_ctx, - copy.input.output_schema()?, - ©.project_columns, - &mut self.main_pipeline, - false, - )?; - - // The stage table that copying into - let to_table = StageSinkTable::create(copy.info.clone(), copy.input_table_schema.clone())?; - - // StageSinkTable needs not to hold the table meta timestamps invariants, just pass a dummy one - let dummy_table_meta_timestamps = TableMetaTimestamps::new(None, Duration::hours(1)); - PipelineBuilder::build_append2table_with_commit_pipeline( - self.ctx.clone(), - &mut self.main_pipeline, - to_table, - copy.input_data_schema.clone(), - None, - vec![], - false, - unsafe { self.ctx.get_settings().get_deduplicate_label()? }, - dummy_table_meta_timestamps, - ) - } -} diff --git a/src/query/service/src/pipelines/builders/builder_copy_into_table.rs b/src/query/service/src/pipelines/builders/builder_copy_into_table.rs index 31b4fe335f3b0..b02cfb2743e54 100644 --- a/src/query/service/src/pipelines/builders/builder_copy_into_table.rs +++ b/src/query/service/src/pipelines/builders/builder_copy_into_table.rs @@ -20,10 +20,8 @@ use databend_common_ast::ast::CopyIntoTableOptions; use databend_common_catalog::table::Table; use databend_common_catalog::table_context::TableContext; use databend_common_exception::Result; -use databend_common_expression::DataField; 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; @@ -31,14 +29,13 @@ 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_storage::StageFileInfo; use databend_common_storages_stage::TransformNullIf; use log::debug; use log::info; +use crate::physical_plans::CopyIntoTable; use crate::pipelines::processors::transforms::TransformAddConstColumns; use crate::pipelines::processors::transforms::TransformCastSchema; use crate::pipelines::PipelineBuilder; @@ -46,58 +43,6 @@ 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<()> { - let to_table = self.ctx.build_table_by_table_info(©.table_info, None)?; - let source_schema = self.build_copy_into_table_input(copy, &to_table)?; - Self::build_copy_into_table_append( - self.ctx.clone(), - &mut self.main_pipeline, - copy, - source_schema, - to_table, - )?; - Ok(()) - } - pub(crate) fn build_copy_into_table_input( - &mut self, - copy: &CopyIntoTable, - to_table: &Arc, - ) -> Result { - 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() - .iter() - .map(|column_binding| { - DataField::new( - &column_binding.column_name, - *column_binding.data_type.clone(), - ) - }) - .collect(); - Ok(DataSchemaRefExt::create(fields)) - } - CopyIntoTableSource::Stage(input) => { - self.ctx - .set_read_block_thresholds(to_table.get_block_thresholds()); - - self.build_pipeline(input)?; - Ok(copy.required_source_schema.clone()) - } - } - } - fn need_null_if_processor<'a>( plan: &'a CopyIntoTable, source_schema: &Arc, @@ -127,7 +72,7 @@ impl PipelineBuilder { None } - fn build_copy_into_table_append( + pub fn build_copy_into_table_append( ctx: Arc, main_pipeline: &mut Pipeline, plan: &CopyIntoTable, 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 c45d68be9a889..0000000000000 --- a/src/query/service/src/pipelines/builders/builder_distributed_insert_select.rs +++ /dev/null @@ -1,71 +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::transforms::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, - insert_select.table_meta_timestamps, - )?; - - Ok(()) - } -} diff --git a/src/query/service/src/pipelines/builders/builder_exchange.rs b/src/query/service/src/pipelines/builders/builder_exchange.rs deleted file mode 100644 index 3271a1aea8174..0000000000000 --- a/src/query/service/src/pipelines/builders/builder_exchange.rs +++ /dev/null @@ -1,51 +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_core::PlanScope; -use databend_common_sql::executor::physical_plans::ExchangeSink; -use databend_common_sql::executor::physical_plans::ExchangeSource; - -use crate::pipelines::PipelineBuilder; - -impl PipelineBuilder { - pub fn build_exchange_source(&mut self, exchange_source: &ExchangeSource) -> Result<()> { - let exchange_manager = self.ctx.get_exchange_manager(); - let build_res = exchange_manager.get_fragment_source( - &exchange_source.query_id, - exchange_source.source_fragment_id, - self.exchange_injector.clone(), - )?; - - let plan_scope = PlanScope::get_plan_scope(); - let build_pipeline = build_res.main_pipeline.finalize(plan_scope); - - // add sharing data - self.join_state = build_res.builder_data.input_join_state; - self.merge_into_probe_data_fields = build_res.builder_data.input_probe_schema; - - // Merge pipeline - assert_eq!(self.main_pipeline.output_len(), 0); - let sinks = self.main_pipeline.merge(build_pipeline)?; - self.main_pipeline.extend_sinks(sinks); - self.pipelines.extend(build_res.sources_pipelines); - Ok(()) - } - - pub fn build_exchange_sink(&mut self, exchange_sink: &ExchangeSink) -> Result<()> { - // ExchangeSink will be appended by `ExchangeManager::execute_pipeline` - self.build_pipeline(&exchange_sink.input)?; - Ok(()) - } -} diff --git a/src/query/service/src/pipelines/builders/builder_filter.rs b/src/query/service/src/pipelines/builders/builder_filter.rs deleted file mode 100644 index 29d221ec63b6b..0000000000000 --- a/src/query/service/src/pipelines/builders/builder_filter.rs +++ /dev/null @@ -1,28 +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_sql::executor::physical_plans::Filter; - -use crate::pipelines::PipelineBuilder; -impl PipelineBuilder { - pub(crate) fn build_filter(&mut self, filter: &Filter) -> Result<()> { - self.build_pipeline(&filter.input)?; - self.main_pipeline.add_transform( - self.filter_transform_builder(&filter.predicates, filter.projections.clone())?, - )?; - - Ok(()) - } -} diff --git a/src/query/service/src/pipelines/builders/builder_hilbert_partition.rs b/src/query/service/src/pipelines/builders/builder_hilbert_partition.rs deleted file mode 100644 index 86b23bfca9238..0000000000000 --- a/src/query/service/src/pipelines/builders/builder_hilbert_partition.rs +++ /dev/null @@ -1,102 +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::atomic; -use std::sync::atomic::AtomicUsize; - -use databend_common_catalog::table_context::TableContext; -use databend_common_exception::Result; -use databend_common_io::constants::DEFAULT_BLOCK_BUFFER_SIZE; -use databend_common_pipeline_core::processors::ProcessorPtr; -use databend_common_pipeline_transforms::MemorySettings; -use databend_common_sql::executor::physical_plans::HilbertPartition; -use databend_common_sql::executor::physical_plans::MutationKind; -use databend_common_storages_fuse::operations::TransformSerializeBlock; -use databend_common_storages_fuse::statistics::ClusterStatsGenerator; -use databend_common_storages_fuse::FuseTable; -use databend_common_storages_fuse::FUSE_OPT_KEY_BLOCK_IN_MEM_SIZE_THRESHOLD; -use databend_storages_common_cache::TempDirManager; - -use crate::pipelines::memory_settings::MemorySettingsExt; -use crate::pipelines::processors::transforms::CompactStrategy; -use crate::pipelines::processors::transforms::HilbertPartitionExchange; -use crate::pipelines::processors::transforms::TransformWindowPartitionCollect; -use crate::pipelines::PipelineBuilder; -use crate::spillers::SpillerDiskConfig; - -impl PipelineBuilder { - pub(crate) fn build_hilbert_partition(&mut self, partition: &HilbertPartition) -> Result<()> { - self.build_pipeline(&partition.input)?; - let num_processors = self.main_pipeline.output_len(); - let table = self - .ctx - .build_table_by_table_info(&partition.table_info, None)?; - let table = FuseTable::try_from_table(table.as_ref())?; - - self.main_pipeline.exchange( - num_processors, - HilbertPartitionExchange::create(partition.num_partitions), - ); - - let settings = self.ctx.get_settings(); - let disk_bytes_limit = settings.get_window_partition_spilling_to_disk_bytes_limit()?; - let temp_dir_manager = TempDirManager::instance(); - - let enable_dio = settings.get_enable_dio()?; - let disk_spill = temp_dir_manager - .get_disk_spill_dir(disk_bytes_limit, &self.ctx.get_id()) - .map(|temp_dir| SpillerDiskConfig::new(temp_dir, enable_dio)) - .transpose()?; - - let window_spill_settings = MemorySettings::from_window_settings(&self.ctx)?; - let processor_id = AtomicUsize::new(0); - let max_bytes_per_block = std::cmp::min( - 4 * table.get_option( - FUSE_OPT_KEY_BLOCK_IN_MEM_SIZE_THRESHOLD, - DEFAULT_BLOCK_BUFFER_SIZE, - ), - 400 * 1024 * 1024, - ); - self.main_pipeline.add_transform(|input, output| { - Ok(ProcessorPtr::create(Box::new( - TransformWindowPartitionCollect::new( - self.ctx.clone(), - input, - output, - &settings, - processor_id.fetch_add(1, atomic::Ordering::AcqRel), - num_processors, - partition.num_partitions, - window_spill_settings.clone(), - disk_spill.clone(), - CompactStrategy::new(partition.rows_per_block, max_bytes_per_block), - )?, - ))) - })?; - - self.main_pipeline - .add_transform(|transform_input_port, transform_output_port| { - let proc = TransformSerializeBlock::try_create( - self.ctx.clone(), - transform_input_port, - transform_output_port, - table, - ClusterStatsGenerator::default(), - MutationKind::Recluster, - partition.table_meta_timestamps, - )?; - proc.into_processor() - }) - } -} diff --git a/src/query/service/src/pipelines/builders/builder_insert_multi_table.rs b/src/query/service/src/pipelines/builders/builder_insert_multi_table.rs deleted file mode 100644 index 28b6cb434a0f4..0000000000000 --- a/src/query/service/src/pipelines/builders/builder_insert_multi_table.rs +++ /dev/null @@ -1,326 +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::HashMap; -use std::sync::Arc; - -use databend_common_catalog::catalog::CatalogManager; -use databend_common_exception::Result; -use databend_common_expression::DataSchema; -use databend_common_expression::LimitType; -use databend_common_expression::SortColumnDescription; -use databend_common_pipeline_core::processors::ProcessorPtr; -use databend_common_pipeline_core::DynTransformBuilder; -use databend_common_pipeline_sinks::AsyncSinker; -use databend_common_pipeline_transforms::processors::TransformSortPartial; -use databend_common_sql::executor::physical_plans::ChunkAppendData; -use databend_common_sql::executor::physical_plans::ChunkCastSchema; -use databend_common_sql::executor::physical_plans::ChunkCommitInsert; -use databend_common_sql::executor::physical_plans::ChunkEvalScalar; -use databend_common_sql::executor::physical_plans::ChunkFillAndReorder; -use databend_common_sql::executor::physical_plans::ChunkFilter; -use databend_common_sql::executor::physical_plans::ChunkMerge; -use databend_common_sql::executor::physical_plans::Duplicate; -use databend_common_sql::executor::physical_plans::Shuffle; -use databend_common_sql::ColumnSet; -use databend_common_storages_fuse::operations::CommitMultiTableInsert; -use databend_common_storages_fuse::FuseTable; -use databend_common_storages_fuse::TableContext; - -use crate::pipelines::PipelineBuilder; -use crate::sql::evaluator::CompoundBlockOperator; -impl PipelineBuilder { - pub(crate) fn build_duplicate(&mut self, plan: &Duplicate) -> Result<()> { - self.build_pipeline(&plan.input)?; - self.main_pipeline.duplicate(true, plan.n)?; - Ok(()) - } - - pub(crate) fn build_shuffle(&mut self, plan: &Shuffle) -> Result<()> { - self.build_pipeline(&plan.input)?; - self.main_pipeline - .reorder_inputs(plan.strategy.shuffle(self.main_pipeline.output_len())?); - Ok(()) - } - - pub(crate) fn build_chunk_filter(&mut self, plan: &ChunkFilter) -> Result<()> { - self.build_pipeline(&plan.input)?; - if plan.predicates.iter().all(|x| x.is_none()) { - return Ok(()); - } - let mut f: Vec = Vec::with_capacity(plan.predicates.len()); - let projection: ColumnSet = (0..plan.input.output_schema()?.fields.len()).collect(); - for predicate in plan.predicates.iter() { - if let Some(predicate) = predicate { - f.push(Box::new(self.filter_transform_builder( - &[predicate.clone()], - projection.clone(), - )?)); - } else { - f.push(Box::new(self.dummy_transform_builder()?)); - } - } - self.main_pipeline.add_transforms_by_chunk(f)?; - Ok(()) - } - - pub(crate) fn build_chunk_eval_scalar(&mut self, plan: &ChunkEvalScalar) -> Result<()> { - self.build_pipeline(&plan.input)?; - if plan.eval_scalars.iter().all(|x| x.is_none()) { - return Ok(()); - } - let num_input_columns = plan.input.output_schema()?.num_fields(); - let mut f: Vec = Vec::with_capacity(plan.eval_scalars.len()); - for eval_scalar in plan.eval_scalars.iter() { - if let Some(eval_scalar) = eval_scalar { - f.push(Box::new(self.map_transform_builder( - num_input_columns, - eval_scalar.remote_exprs.clone(), - Some(eval_scalar.projection.clone()), - )?)); - } else { - f.push(Box::new(self.dummy_transform_builder()?)); - } - } - self.main_pipeline.add_transforms_by_chunk(f)?; - Ok(()) - } - - pub(crate) fn build_chunk_cast_schema(&mut self, plan: &ChunkCastSchema) -> Result<()> { - self.build_pipeline(&plan.input)?; - if plan.cast_schemas.iter().all(|x| x.is_none()) { - return Ok(()); - } - let mut f: Vec = Vec::with_capacity(plan.cast_schemas.len()); - for cast_schema in plan.cast_schemas.iter() { - if let Some(cast_schema) = cast_schema { - f.push(Box::new(self.cast_schema_transform_builder( - cast_schema.source_schema.clone(), - cast_schema.target_schema.clone(), - )?)); - } else { - f.push(Box::new(self.dummy_transform_builder()?)); - } - } - self.main_pipeline.add_transforms_by_chunk(f)?; - Ok(()) - } - - pub(crate) fn build_chunk_fill_and_reorder( - &mut self, - plan: &ChunkFillAndReorder, - ) -> Result<()> { - self.build_pipeline(&plan.input)?; - if plan.fill_and_reorders.iter().all(|x| x.is_none()) { - return Ok(()); - } - let mut f: Vec = Vec::with_capacity(plan.fill_and_reorders.len()); - for fill_and_reorder in plan.fill_and_reorders.iter() { - if let Some(fill_and_reorder) = fill_and_reorder { - let table = self - .ctx - .build_table_by_table_info(&fill_and_reorder.target_table_info, None)?; - f.push(Box::new(self.fill_and_reorder_transform_builder( - table, - fill_and_reorder.source_schema.clone(), - )?)); - } else { - f.push(Box::new(self.dummy_transform_builder()?)); - } - } - self.main_pipeline.add_transforms_by_chunk(f)?; - Ok(()) - } - - pub(crate) fn build_chunk_merge(&mut self, plan: &ChunkMerge) -> Result<()> { - self.build_pipeline(&plan.input)?; - let group_ids = &plan.group_ids; - assert_eq!(self.main_pipeline.output_len() % group_ids.len(), 0); - let chunk_size = self.main_pipeline.output_len() / group_ids.len(); - let mut widths = Vec::with_capacity(group_ids.len()); - let mut last_group_id = group_ids[0]; - let mut width = 1; - for group_id in group_ids.iter().skip(1) { - if *group_id == last_group_id { - width += 1; - } else { - widths.push(width * chunk_size); - last_group_id = *group_id; - width = 1; - } - } - widths.push(width * chunk_size); - self.main_pipeline.resize_partial_one_with_width(widths)?; - Ok(()) - } - - pub(crate) fn build_chunk_append_data(&mut self, plan: &ChunkAppendData) -> Result<()> { - self.build_pipeline(&plan.input)?; - let mut compact_task_builders: Vec = - Vec::with_capacity(plan.target_tables.len()); - let mut compact_transform_builders: Vec = - Vec::with_capacity(plan.target_tables.len()); - let mut serialize_block_builders: Vec = - Vec::with_capacity(plan.target_tables.len()); - let mut eval_cluster_key_builders: Vec = - Vec::with_capacity(plan.target_tables.len()); - let mut eval_cluster_key_num = 0; - let mut sort_builders: Vec = - Vec::with_capacity(plan.target_tables.len()); - let mut sort_num = 0; - - for append_data in plan.target_tables.iter() { - let table = self - .ctx - .build_table_by_table_info(&append_data.target_table_info, None)?; - let block_thresholds = table.get_block_thresholds(); - compact_task_builders - .push(Box::new(self.block_compact_task_builder(block_thresholds)?)); - compact_transform_builders.push(Box::new(self.block_compact_transform_builder()?)); - let schema: Arc = DataSchema::from(table.schema()).into(); - let num_input_columns = schema.num_fields(); - let fuse_table = FuseTable::try_from_table(table.as_ref())?; - let cluster_stats_gen = fuse_table.get_cluster_stats_gen( - self.ctx.clone(), - 0, - block_thresholds, - Some(schema), - )?; - let operators = cluster_stats_gen.operators.clone(); - if !operators.is_empty() { - let func_ctx2 = cluster_stats_gen.func_ctx.clone(); - - eval_cluster_key_builders.push(Box::new(move |input, output| { - Ok(ProcessorPtr::create(CompoundBlockOperator::create( - input, - output, - num_input_columns, - func_ctx2.clone(), - operators.clone(), - ))) - })); - eval_cluster_key_num += 1; - } else { - eval_cluster_key_builders.push(Box::new(self.dummy_transform_builder()?)); - } - let cluster_keys = &cluster_stats_gen.cluster_key_index; - if !cluster_keys.is_empty() { - let sort_desc: Vec = cluster_keys - .iter() - .map(|index| SortColumnDescription { - offset: *index, - asc: true, - nulls_first: false, - }) - .collect(); - let sort_desc: Arc<[_]> = sort_desc.into(); - sort_builders.push(Box::new( - move |transform_input_port, transform_output_port| { - Ok(ProcessorPtr::create(TransformSortPartial::try_create( - transform_input_port, - transform_output_port, - LimitType::None, - sort_desc.clone(), - )?)) - }, - )); - sort_num += 1; - } else { - sort_builders.push(Box::new(self.dummy_transform_builder()?)); - } - serialize_block_builders.push(Box::new( - self.with_tid_serialize_block_transform_builder( - table, - cluster_stats_gen, - append_data.table_meta_timestamps, - )?, - )); - } - self.main_pipeline - .add_transforms_by_chunk(compact_task_builders)?; - self.main_pipeline - .add_transforms_by_chunk(compact_transform_builders)?; - if eval_cluster_key_num > 0 { - self.main_pipeline - .add_transforms_by_chunk(eval_cluster_key_builders)?; - } - if sort_num > 0 { - self.main_pipeline.add_transforms_by_chunk(sort_builders)?; - } - self.main_pipeline - .add_transforms_by_chunk(serialize_block_builders)?; - Ok(()) - } - - pub(crate) fn build_chunk_commit_insert(&mut self, plan: &ChunkCommitInsert) -> Result<()> { - let ChunkCommitInsert { - plan_id: _, - input, - update_stream_meta, - overwrite, - deduplicated_label, - targets, - } = plan; - let mut table_meta_timestampss = HashMap::new(); - self.build_pipeline(input)?; - let mut serialize_segment_builders: Vec = - Vec::with_capacity(targets.len()); - let mut mutation_aggregator_builders: Vec = - Vec::with_capacity(targets.len()); - let mut tables = HashMap::new(); - for target in targets { - let table = self - .ctx - .build_table_by_table_info(&target.target_table_info, None)?; - let block_thresholds = table.get_block_thresholds(); - serialize_segment_builders.push(Box::new(self.serialize_segment_transform_builder( - table.clone(), - block_thresholds, - target.table_meta_timestamps, - )?)); - mutation_aggregator_builders.push(Box::new( - self.mutation_aggregator_transform_builder( - table.clone(), - target.table_meta_timestamps, - )?, - )); - table_meta_timestampss.insert(table.get_id(), target.table_meta_timestamps); - tables.insert(table.get_id(), table); - } - self.main_pipeline - .add_transforms_by_chunk(serialize_segment_builders)?; - self.main_pipeline - .add_transforms_by_chunk(mutation_aggregator_builders)?; - self.main_pipeline.try_resize(1)?; - let catalog = CatalogManager::instance().build_catalog( - targets[0].target_catalog_info.clone(), - self.ctx.session_state(), - )?; - self.main_pipeline.add_sink(|input| { - Ok(ProcessorPtr::create(AsyncSinker::create( - input, - CommitMultiTableInsert::create( - tables.clone(), - self.ctx.clone(), - *overwrite, - update_stream_meta.clone(), - deduplicated_label.clone(), - catalog.clone(), - table_meta_timestampss.clone(), - ), - ))) - })?; - Ok(()) - } -} diff --git a/src/query/service/src/pipelines/builders/builder_join.rs b/src/query/service/src/pipelines/builders/builder_join.rs index f3eaede41d135..ab58c105d85d4 100644 --- a/src/query/service/src/pipelines/builders/builder_join.rs +++ b/src/query/service/src/pipelines/builders/builder_join.rs @@ -12,214 +12,16 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::Arc; - -use databend_common_base::base::tokio::sync::Barrier; -use databend_common_exception::Result; -use databend_common_pipeline_core::processors::ProcessorPtr; -use databend_common_pipeline_sinks::Sinker; -use databend_common_sql::executor::physical_plans::HashJoin; -use databend_common_sql::executor::physical_plans::RangeJoin; -use databend_common_sql::executor::PhysicalPlan; - -use crate::pipelines::processors::transforms::range_join::RangeJoinState; -use crate::pipelines::processors::transforms::range_join::TransformRangeJoinLeft; -use crate::pipelines::processors::transforms::range_join::TransformRangeJoinRight; -use crate::pipelines::processors::transforms::HashJoinBuildState; -use crate::pipelines::processors::transforms::HashJoinProbeState; -use crate::pipelines::processors::transforms::TransformHashJoinBuild; -use crate::pipelines::processors::transforms::TransformHashJoinProbe; -use crate::pipelines::processors::HashJoinDesc; -use crate::pipelines::processors::HashJoinState; use crate::pipelines::PipelineBuilder; use crate::sessions::QueryContext; impl PipelineBuilder { // Create a new pipeline builder with the same context as the current builder - fn create_sub_pipeline_builder(&self) -> PipelineBuilder { + pub fn create_sub_pipeline_builder(&self) -> PipelineBuilder { let sub_context = QueryContext::create_from(self.ctx.as_ref()); let mut sub_builder = PipelineBuilder::create(self.func_ctx.clone(), self.settings.clone(), sub_context); sub_builder.hash_join_states = self.hash_join_states.clone(); sub_builder } - - pub(crate) fn build_hash_join(&mut self, join: &HashJoin) -> Result<()> { - // Get optimization flags for merge-into operations - let (enable_optimization, is_distributed) = self.merge_into_get_optimization_flag(join); - - // Create the join state with optimization flags - let state = self.build_hash_join_state(join, is_distributed, enable_optimization)?; - if let Some((build_cache_index, _)) = join.build_side_cache_info { - self.hash_join_states - .insert(build_cache_index, state.clone()); - } - - // Build both phases of the Hash Join - self.build_hash_join_build_side(&join.build, join, state.clone())?; - self.build_hash_join_probe_side(join, state)?; - - // In the case of spilling, we need to share state among multiple threads - // Quickly fetch all data from this round to quickly start the next round - self.main_pipeline - .resize(self.main_pipeline.output_len(), true) - } - - // Create the Hash Join state - fn build_hash_join_state( - &mut self, - join: &HashJoin, - merge_into_is_distributed: bool, - enable_merge_into_optimization: bool, - ) -> Result> { - HashJoinState::try_create( - self.ctx.clone(), - join.build.output_schema()?, - &join.build_projections, - HashJoinDesc::create(join)?, - &join.probe_to_build, - merge_into_is_distributed, - enable_merge_into_optimization, - join.build_side_cache_info.clone(), - ) - } - - // Build the build-side pipeline for Hash Join - fn build_hash_join_build_side( - &mut self, - build: &PhysicalPlan, - hash_join_plan: &HashJoin, - join_state: Arc, - ) -> Result<()> { - let build_side_builder = self.create_sub_pipeline_builder(); - let mut build_res = build_side_builder.finalize(build)?; - - assert!(build_res.main_pipeline.is_pulling_pipeline()?); - let output_len = build_res.main_pipeline.output_len(); - let build_state = HashJoinBuildState::try_create( - self.ctx.clone(), - self.func_ctx.clone(), - &hash_join_plan.build_keys, - &hash_join_plan.build_projections, - join_state.clone(), - output_len, - hash_join_plan.broadcast_id, - )?; - build_state.add_runtime_filter_ready(); - - let create_sink_processor = |input| { - Ok(ProcessorPtr::create(TransformHashJoinBuild::try_create( - input, - build_state.clone(), - )?)) - }; - // For distributed merge-into when source as build side - if hash_join_plan.need_hold_hash_table { - self.join_state = Some(build_state.clone()) - } - build_res.main_pipeline.add_sink(create_sink_processor)?; - - self.pipelines.push(build_res.main_pipeline.finalize(None)); - self.pipelines.extend(build_res.sources_pipelines); - Ok(()) - } - - // Build the probe-side pipeline for Hash Join - fn build_hash_join_probe_side( - &mut self, - join: &HashJoin, - state: Arc, - ) -> Result<()> { - self.build_pipeline(&join.probe)?; - - let max_block_size = self.settings.get_max_block_size()? as usize; - let barrier = Barrier::new(self.main_pipeline.output_len()); - let probe_state = Arc::new(HashJoinProbeState::create( - self.ctx.clone(), - self.func_ctx.clone(), - state.clone(), - &join.probe_projections, - &join.build_projections, - &join.probe_keys, - join.probe.output_schema()?, - &join.join_type, - self.main_pipeline.output_len(), - barrier, - )?); - - self.main_pipeline.add_transform(|input, output| { - Ok(ProcessorPtr::create(TransformHashJoinProbe::create( - input, - output, - join.projections.clone(), - probe_state.clone(), - max_block_size, - self.func_ctx.clone(), - &join.join_type, - !join.non_equi_conditions.is_empty(), - )?)) - })?; - - // For merge-into operations that need to hold the hash table - if join.need_hold_hash_table { - // Extract projected fields from probe schema - let mut projected_fields = vec![]; - for (i, field) in probe_state.probe_schema.fields().iter().enumerate() { - if probe_state.probe_projections.contains(&i) { - projected_fields.push(field.clone()); - } - } - self.merge_into_probe_data_fields = Some(projected_fields); - } - - Ok(()) - } - - pub(crate) fn build_range_join(&mut self, range_join: &RangeJoin) -> Result<()> { - let state = Arc::new(RangeJoinState::new(self.ctx.clone(), range_join)); - self.build_range_join_right_side(range_join, state.clone())?; - self.build_range_join_left_side(range_join, state)?; - Ok(()) - } - - // Build the left-side pipeline for Range Join - fn build_range_join_left_side( - &mut self, - range_join: &RangeJoin, - state: Arc, - ) -> Result<()> { - self.build_pipeline(&range_join.left)?; - let max_threads = self.settings.get_max_threads()? as usize; - self.main_pipeline.try_resize(max_threads)?; - self.main_pipeline.add_transform(|input, output| { - Ok(ProcessorPtr::create(TransformRangeJoinLeft::create( - input, - output, - state.clone(), - ))) - })?; - Ok(()) - } - - // Build the right-side pipeline for Range Join - fn build_range_join_right_side( - &mut self, - range_join: &RangeJoin, - state: Arc, - ) -> Result<()> { - let right_side_builder = self.create_sub_pipeline_builder(); - - let mut right_res = right_side_builder.finalize(&range_join.right)?; - right_res.main_pipeline.add_sink(|input| { - Ok(ProcessorPtr::create( - Sinker::::create( - input, - TransformRangeJoinRight::create(state.clone()), - ), - )) - })?; - self.pipelines.push(right_res.main_pipeline.finalize(None)); - self.pipelines.extend(right_res.sources_pipelines); - Ok(()) - } } diff --git a/src/query/service/src/pipelines/builders/builder_limit.rs b/src/query/service/src/pipelines/builders/builder_limit.rs deleted file mode 100644 index 950545854b1f8..0000000000000 --- a/src/query/service/src/pipelines/builders/builder_limit.rs +++ /dev/null @@ -1,39 +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_core::processors::ProcessorPtr; -use databend_common_sql::executor::physical_plans::Limit; - -use crate::pipelines::processors::transforms::TransformLimit; -use crate::pipelines::PipelineBuilder; - -impl PipelineBuilder { - pub(crate) fn build_limit(&mut self, limit: &Limit) -> Result<()> { - self.build_pipeline(&limit.input)?; - - if limit.limit.is_some() || limit.offset != 0 { - self.main_pipeline.try_resize(1)?; - return self.main_pipeline.add_transform(|input, output| { - Ok(ProcessorPtr::create(TransformLimit::try_create( - limit.limit, - limit.offset, - input, - output, - )?)) - }); - } - Ok(()) - } -} diff --git a/src/query/service/src/pipelines/builders/builder_mutation.rs b/src/query/service/src/pipelines/builders/builder_mutation.rs index 4b7a3d2b67628..70963874daf6a 100644 --- a/src/query/service/src/pipelines/builders/builder_mutation.rs +++ b/src/query/service/src/pipelines/builders/builder_mutation.rs @@ -15,7 +15,6 @@ use std::collections::HashMap; use std::sync::Arc; -use databend_common_base::base::tokio::sync::Semaphore; use databend_common_catalog::table::Table; use databend_common_exception::ErrorCode; use databend_common_exception::Result; @@ -23,8 +22,6 @@ use databend_common_expression::BlockThresholds; use databend_common_expression::DataSchema; use databend_common_expression::DataSchemaRef; use databend_common_pipeline_core::processors::create_resize_item; -use databend_common_pipeline_core::processors::InputPort; -use databend_common_pipeline_core::processors::OutputPort; use databend_common_pipeline_core::processors::ProcessorPtr; use databend_common_pipeline_core::Pipe; use databend_common_pipeline_transforms::processors::create_dummy_item; @@ -33,11 +30,7 @@ use databend_common_pipeline_transforms::processors::BlockCompactBuilder; use databend_common_pipeline_transforms::processors::BlockMetaTransformer; use databend_common_pipeline_transforms::processors::TransformCompactBlock; use databend_common_pipeline_transforms::processors::TransformPipelineHelper; -use databend_common_sql::binder::MutationStrategy; -use databend_common_sql::executor::physical_plans::Mutation; -use databend_common_sql::executor::physical_plans::MutationKind; use databend_common_sql::DefaultExprBinder; -use databend_common_storages_fuse::operations::TransformSerializeBlock; use databend_common_storages_fuse::operations::UnMatchedExprs; use databend_common_storages_fuse::FuseTable; @@ -53,88 +46,6 @@ use crate::pipelines::processors::transforms::TransformResortAddOnWithoutSourceS use crate::pipelines::PipelineBuilder; impl PipelineBuilder { - // build mutation serialize and mutation pipeline - pub(crate) fn build_mutation(&mut self, merge_into: &Mutation) -> Result<()> { - self.build_pipeline(&merge_into.input)?; - - let tbl = self - .ctx - .build_table_by_table_info(&merge_into.table_info, None)?; - - let table = FuseTable::try_from_table(tbl.as_ref())?; - let block_thresholds = table.get_block_thresholds(); - - let cluster_stats_gen = - table.get_cluster_stats_gen(self.ctx.clone(), 0, block_thresholds, None)?; - - let io_request_semaphore = - Arc::new(Semaphore::new(self.settings.get_max_threads()? as usize)); - - // For row_id port, create rowid_aggregate_mutator - // For matched data port and unmatched port, do serialize - let serialize_len = match merge_into.strategy { - MutationStrategy::NotMatchedOnly => self.main_pipeline.output_len(), - MutationStrategy::MixedMatched | MutationStrategy::MatchedOnly => { - // remove row id port - self.main_pipeline.output_len() - 1 - } - MutationStrategy::Direct => unreachable!(), - }; - - // 1. Fill default and computed columns - self.build_fill_columns_in_merge_into( - tbl.clone(), - serialize_len, - merge_into.need_match, - merge_into.unmatched.clone(), - )?; - - // 2. Add cluster‘s blocksort if it's a cluster table - self.build_compact_and_cluster_sort_in_merge_into( - table, - merge_into.need_match, - serialize_len, - block_thresholds, - )?; - - let mut pipe_items = Vec::with_capacity(self.main_pipeline.output_len()); - - // 3.1 Add rowid_aggregate_mutator for row_id port - if merge_into.need_match { - pipe_items.push(table.rowid_aggregate_mutator( - self.ctx.clone(), - cluster_stats_gen.clone(), - io_request_semaphore, - merge_into.segments.clone(), - false, - merge_into.table_meta_timestamps, - )?); - } - - // 3.2 Add serialize_block_transform for data port - for _ in 0..serialize_len { - let serialize_block_transform = TransformSerializeBlock::try_create( - self.ctx.clone(), - InputPort::create(), - OutputPort::create(), - table, - cluster_stats_gen.clone(), - MutationKind::MergeInto, - merge_into.table_meta_timestamps, - )?; - pipe_items.push(serialize_block_transform.into_pipe_item()); - } - - let output_len = pipe_items.iter().map(|item| item.outputs_port.len()).sum(); - self.main_pipeline.add_pipe(Pipe::create( - self.main_pipeline.output_len(), - output_len, - pipe_items, - )); - - Ok(()) - } - pub fn build_fill_columns_in_merge_into( &mut self, tbl: Arc, diff --git a/src/query/service/src/pipelines/builders/builder_mutation_manipulate.rs b/src/query/service/src/pipelines/builders/builder_mutation_manipulate.rs deleted file mode 100644 index beacaa78dd5b8..0000000000000 --- a/src/query/service/src/pipelines/builders/builder_mutation_manipulate.rs +++ /dev/null @@ -1,102 +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_exception::Result; -use databend_common_expression::DataSchema; -use databend_common_pipeline_core::Pipe; -use databend_common_sql::binder::MutationStrategy; -use databend_common_sql::executor::physical_plans::MutationManipulate; -use databend_common_storages_fuse::operations::MatchedSplitProcessor; -use databend_common_storages_fuse::operations::MergeIntoNotMatchedProcessor; - -use crate::pipelines::PipelineBuilder; - -impl PipelineBuilder { - // Handle matched and unmatched data separately. - // This is a complete pipeline with matched and not matched clauses, for matched only or unmatched only - // we will delicate useless pipeline and processor - // +-----------------------------+-+ - // +-----------------------+ Matched | +-+ - // | +---+--------------->| MatchedSplitProcessor | - // | | | | +-+ - // +----------------------+ | +---+ +-----------------------------+-+ - // | MergeInto +---------->|MutationSplitProcessor | - // +----------------------+ | +---+ +-----------------------------+ - // | | | NotMatched | +-+ - // | +---+--------------->| MergeIntoNotMatchedProcessor| | - // +-----------------------+ | +-+ - // +-----------------------------+ - // Note: here the output_port of MatchedSplitProcessor are arranged in the following order - // (0) -> output_port_row_id - // (1) -> output_port_updated - - // Outputs from MatchedSplitProcessor's output_port_updated and MergeIntoNotMatchedProcessor's output_port are merged and processed uniformly by the subsequent ResizeProcessor - // receive matched data and not matched data parallelly. - pub(crate) fn build_mutation_manipulate( - &mut self, - merge_into_manipulate: &MutationManipulate, - ) -> Result<()> { - self.build_pipeline(&merge_into_manipulate.input)?; - - let (step, need_match, need_unmatch) = match merge_into_manipulate.strategy { - MutationStrategy::MatchedOnly => (1, true, false), - MutationStrategy::NotMatchedOnly => (1, false, true), - MutationStrategy::MixedMatched => (2, true, true), - MutationStrategy::Direct => unreachable!(), - }; - - let tbl = self - .ctx - .build_table_by_table_info(&merge_into_manipulate.table_info, None)?; - - let input_schema = merge_into_manipulate.input.output_schema()?; - let mut pipe_items = Vec::with_capacity(self.main_pipeline.output_len()); - for _ in (0..self.main_pipeline.output_len()).step_by(step) { - if need_match { - let matched_split_processor = MatchedSplitProcessor::create( - self.ctx.clone(), - merge_into_manipulate.row_id_idx, - merge_into_manipulate.matched.clone(), - merge_into_manipulate.field_index_of_input_schema.clone(), - input_schema.clone(), - Arc::new(DataSchema::from(tbl.schema_with_stream())), - false, - merge_into_manipulate.can_try_update_column_only, - )?; - pipe_items.push(matched_split_processor.into_pipe_item()); - } - - if need_unmatch { - let merge_into_not_matched_processor = MergeIntoNotMatchedProcessor::create( - merge_into_manipulate.unmatched.clone(), - merge_into_manipulate.unmatched_schema.clone(), - self.func_ctx.clone(), - self.ctx.clone(), - )?; - pipe_items.push(merge_into_not_matched_processor.into_pipe_item()); - } - } - - let output_len = pipe_items.iter().map(|item| item.outputs_port.len()).sum(); - self.main_pipeline.add_pipe(Pipe::create( - self.main_pipeline.output_len(), - output_len, - pipe_items.clone(), - )); - - Ok(()) - } -} diff --git a/src/query/service/src/pipelines/builders/builder_mutation_organize.rs b/src/query/service/src/pipelines/builders/builder_mutation_organize.rs deleted file mode 100644 index 52aa3e124655d..0000000000000 --- a/src/query/service/src/pipelines/builders/builder_mutation_organize.rs +++ /dev/null @@ -1,95 +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_sql::binder::MutationStrategy; -use databend_common_sql::executor::physical_plans::MutationOrganize; - -use crate::pipelines::PipelineBuilder; - -impl PipelineBuilder { - // Organize outputs and resize row_id - pub(crate) fn build_mutation_organize( - &mut self, - merge_into_organize: &MutationOrganize, - ) -> Result<()> { - self.build_pipeline(&merge_into_organize.input)?; - - // The complete pipeline: - // ----------------------------------------------------------------------------------------- - // row_id port0_1 row_id port0_1 row_id port0_1 - // matched data port0_2 ..... row_id port1_1 row_id port - // unmatched port0_3 data port0_2 ...... - // row_id port1_1 ====> row_id port1_1 ====> data port0_2 ====> data port0 - // matched data port1_2 ..... data port1_2 data port1 - // unmatched port1_3 data port1_2 ...... - // ...... ..... - // ----------------------------------------------------------------------------------------- - // 1. matched only or complete pipeline are same with above - // 2. for unmatched only, there are no row_id port - - let mut ranges = Vec::with_capacity(self.main_pipeline.output_len()); - let mut rules = Vec::with_capacity(self.main_pipeline.output_len()); - match merge_into_organize.strategy { - MutationStrategy::MixedMatched => { - assert_eq!(self.main_pipeline.output_len() % 3, 0); - // merge matched update ports and not matched ports ===> data ports - for idx in (0..self.main_pipeline.output_len()).step_by(3) { - ranges.push(vec![idx]); - ranges.push(vec![idx + 1, idx + 2]); - } - self.main_pipeline.resize_partial_one(ranges.clone())?; - assert_eq!(self.main_pipeline.output_len() % 2, 0); - let row_id_len = self.main_pipeline.output_len() / 2; - for idx in 0..row_id_len { - rules.push(idx); - rules.push(idx + row_id_len); - } - self.main_pipeline.reorder_inputs(rules); - self.resize_row_id(2)?; - } - MutationStrategy::MatchedOnly => { - assert_eq!(self.main_pipeline.output_len() % 2, 0); - let row_id_len = self.main_pipeline.output_len() / 2; - for idx in 0..row_id_len { - rules.push(idx); - rules.push(idx + row_id_len); - } - self.main_pipeline.reorder_inputs(rules); - self.resize_row_id(2)?; - } - MutationStrategy::NotMatchedOnly => {} - MutationStrategy::Direct => unreachable!(), - } - Ok(()) - } - - fn resize_row_id(&mut self, step: usize) -> Result<()> { - // resize row_id - let row_id_len = self.main_pipeline.output_len() / step; - let mut ranges = Vec::with_capacity(self.main_pipeline.output_len()); - let mut vec = Vec::with_capacity(row_id_len); - for idx in 0..row_id_len { - vec.push(idx); - } - ranges.push(vec.clone()); - - // data ports - for idx in 0..row_id_len { - ranges.push(vec![idx + row_id_len]); - } - - self.main_pipeline.resize_partial_one(ranges.clone()) - } -} diff --git a/src/query/service/src/pipelines/builders/builder_mutation_source.rs b/src/query/service/src/pipelines/builders/builder_mutation_source.rs deleted file mode 100644 index 69ab3ef6fa316..0000000000000 --- a/src/query/service/src/pipelines/builders/builder_mutation_source.rs +++ /dev/null @@ -1,148 +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_base::runtime::Runtime; -use databend_common_catalog::plan::PartInfoType; -use databend_common_catalog::plan::Projection; -use databend_common_catalog::table::Table; -use databend_common_catalog::table_context::TableContext; -use databend_common_exception::Result; -use databend_common_expression::DataBlock; -use databend_common_pipeline_sources::OneBlockSource; -use databend_common_pipeline_transforms::processors::TransformPipelineHelper; -use databend_common_sql::binder::MutationType; -use databend_common_sql::executor::physical_plans::MutationSource; -use databend_common_sql::StreamContext; -use databend_common_storages_fuse::operations::CommitMeta; -use databend_common_storages_fuse::operations::ConflictResolveContext; -use databend_common_storages_fuse::operations::MutationAction; -use databend_common_storages_fuse::operations::MutationBlockPruningContext; -use databend_common_storages_fuse::FuseLazyPartInfo; -use databend_common_storages_fuse::FuseTable; -use databend_common_storages_fuse::SegmentLocation; - -use crate::pipelines::processors::transforms::TransformAddStreamColumns; -use crate::pipelines::PipelineBuilder; - -impl PipelineBuilder { - pub(crate) fn build_mutation_source(&mut self, mutation_source: &MutationSource) -> Result<()> { - let table = self - .ctx - .build_table_by_table_info(&mutation_source.table_info, None)?; - - let table = FuseTable::try_from_table(table.as_ref())?.clone(); - let is_delete = mutation_source.input_type == MutationType::Delete; - if mutation_source.truncate_table { - // There is no filter and the mutation type is delete, - // we can truncate the table directly. - debug_assert!(mutation_source.partitions.is_empty() && is_delete); - return self.main_pipeline.add_source( - |output| { - let meta = CommitMeta { - conflict_resolve_context: ConflictResolveContext::None, - new_segment_locs: vec![], - table_id: table.get_id(), - virtual_schema: None, - }; - let block = DataBlock::empty_with_meta(Box::new(meta)); - OneBlockSource::create(output, block) - }, - 1, - ); - } - - let read_partition_columns: Vec = mutation_source - .read_partition_columns - .clone() - .into_iter() - .collect(); - - let is_lazy = - mutation_source.partitions.partitions_type() == PartInfoType::LazyLevel && is_delete; - if is_lazy { - let ctx = self.ctx.clone(); - let table_clone = table.clone(); - let ctx_clone = self.ctx.clone(); - let filters_clone = mutation_source.filters.clone(); - let projection = Projection::Columns(read_partition_columns.clone()); - let mut segment_locations = - Vec::with_capacity(mutation_source.partitions.partitions.len()); - for part in &mutation_source.partitions.partitions { - // Safe to downcast because we know the partition is lazy - let part: &FuseLazyPartInfo = FuseLazyPartInfo::from_part(part)?; - segment_locations.push(SegmentLocation { - segment_idx: part.segment_index, - location: part.segment_location.clone(), - snapshot_loc: None, - }); - } - let prune_ctx = MutationBlockPruningContext { - segment_locations, - block_count: None, - }; - Runtime::with_worker_threads(2, Some("do_mutation_block_pruning".to_string()))? - .block_on(async move { - let (_, partitions) = table_clone - .do_mutation_block_pruning( - ctx_clone, - filters_clone, - projection, - prune_ctx, - true, - ) - .await?; - ctx.set_partitions(partitions)?; - Ok(()) - })?; - } else { - self.ctx - .set_partitions(mutation_source.partitions.clone())?; - } - - let filter = mutation_source.filters.clone().map(|v| v.filter); - let mutation_action = if is_delete { - MutationAction::Deletion - } else { - MutationAction::Update - }; - let col_indices = mutation_source - .read_partition_columns - .clone() - .into_iter() - .collect(); - let update_mutation_with_filter = - mutation_source.input_type == MutationType::Update && filter.is_some(); - table.add_mutation_source( - self.ctx.clone(), - filter, - col_indices, - &mut self.main_pipeline, - mutation_action, - )?; - - if table.change_tracking_enabled() { - let stream_ctx = StreamContext::try_create( - self.ctx.get_function_context()?, - table.schema_with_stream(), - table.get_table_info().ident.seq, - is_delete, - update_mutation_with_filter, - )?; - self.main_pipeline - .add_transformer(|| TransformAddStreamColumns::new(stream_ctx.clone())); - } - - Ok(()) - } -} diff --git a/src/query/service/src/pipelines/builders/builder_mutation_split.rs b/src/query/service/src/pipelines/builders/builder_mutation_split.rs deleted file mode 100644 index 6e6d982c5ea74..0000000000000 --- a/src/query/service/src/pipelines/builders/builder_mutation_split.rs +++ /dev/null @@ -1,42 +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_core::Pipe; -use databend_common_sql::executor::physical_plans::MutationSplit; -use databend_common_storages_fuse::operations::MutationSplitProcessor; -use databend_common_storages_fuse::TableContext; - -use crate::pipelines::PipelineBuilder; - -impl PipelineBuilder { - pub(crate) fn build_mutation_split(&mut self, merge_into_split: &MutationSplit) -> Result<()> { - self.build_pipeline(&merge_into_split.input)?; - self.main_pipeline - .try_resize(self.ctx.get_settings().get_max_threads()? as usize)?; - - // The MutationStrategy is FullOperation, use row_id_idx to split - let mut items = Vec::with_capacity(self.main_pipeline.output_len()); - let output_len = self.main_pipeline.output_len(); - for _ in 0..output_len { - let merge_into_split_processor = - MutationSplitProcessor::create(merge_into_split.split_index as u32)?; - items.push(merge_into_split_processor.into_pipe_item()); - } - self.main_pipeline - .add_pipe(Pipe::create(output_len, output_len * 2, items)); - - Ok(()) - } -} diff --git a/src/query/service/src/pipelines/builders/builder_project.rs b/src/query/service/src/pipelines/builders/builder_project.rs index 660aa90805f14..fb31eb2e54ccb 100644 --- a/src/query/service/src/pipelines/builders/builder_project.rs +++ b/src/query/service/src/pipelines/builders/builder_project.rs @@ -15,17 +15,14 @@ use databend_common_exception::Result; use databend_common_expression::DataSchemaRef; use databend_common_expression::FunctionContext; -use databend_common_functions::BUILTIN_FUNCTIONS; use databend_common_pipeline_core::processors::ProcessorPtr; use databend_common_pipeline_core::Pipeline; use databend_common_pipeline_sinks::EmptySink; use databend_common_pipeline_transforms::processors::TransformPipelineHelper; use databend_common_sql::evaluator::BlockOperator; use databend_common_sql::evaluator::CompoundBlockOperator; -use databend_common_sql::executor::physical_plans::ProjectSet; use databend_common_sql::ColumnBinding; -use crate::pipelines::processors::transforms::TransformSRF; use crate::pipelines::PipelineBuilder; impl PipelineBuilder { @@ -68,26 +65,4 @@ impl PipelineBuilder { Ok(()) } - - pub(crate) fn build_project_set(&mut self, project_set: &ProjectSet) -> Result<()> { - self.build_pipeline(&project_set.input)?; - - let srf_exprs = project_set - .srf_exprs - .iter() - .map(|(expr, _)| expr.as_expr(&BUILTIN_FUNCTIONS)) - .collect::>(); - let max_block_size = self.settings.get_max_block_size()? as usize; - - self.main_pipeline.add_transform(|input, output| { - Ok(ProcessorPtr::create(TransformSRF::try_create( - input, - output, - self.func_ctx.clone(), - project_set.projections.clone(), - srf_exprs.clone(), - max_block_size, - ))) - }) - } } diff --git a/src/query/service/src/pipelines/builders/builder_recursive_cte.rs b/src/query/service/src/pipelines/builders/builder_recursive_cte.rs deleted file mode 100644 index 2784daecfe5a3..0000000000000 --- a/src/query/service/src/pipelines/builders/builder_recursive_cte.rs +++ /dev/null @@ -1,54 +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_catalog::table_context::TableContext; -use databend_common_exception::Result; -use databend_common_sql::executor::physical_plans::RecursiveCteScan; -use databend_common_sql::executor::physical_plans::UnionAll; - -use crate::pipelines::processors::transforms::TransformRecursiveCteScan; -use crate::pipelines::processors::transforms::TransformRecursiveCteSource; -use crate::pipelines::PipelineBuilder; - -impl PipelineBuilder { - pub fn build_recursive_cte_source(&mut self, union_all: &UnionAll) -> Result<()> { - let max_threads = self.ctx.get_settings().get_max_threads()?; - self.main_pipeline.add_source( - |output_port| { - TransformRecursiveCteSource::try_create( - self.ctx.clone(), - output_port.clone(), - union_all.clone(), - ) - }, - 1, - )?; - self.main_pipeline.resize(max_threads as usize, true) - } - - pub fn build_recursive_cte_scan(&mut self, r_cte_scan: &RecursiveCteScan) -> Result<()> { - let max_threads = self.ctx.get_settings().get_max_threads()?; - self.main_pipeline.add_source( - |output_port| { - TransformRecursiveCteScan::create( - self.ctx.clone(), - output_port.clone(), - r_cte_scan.table_name.clone(), - ) - }, - 1, - )?; - self.main_pipeline.resize(max_threads as usize, true) - } -} 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 be41d1a2fe301..009376f5fe621 100644 --- a/src/query/service/src/pipelines/builders/builder_replace_into.rs +++ b/src/query/service/src/pipelines/builders/builder_replace_into.rs @@ -17,8 +17,6 @@ use std::sync::Arc; use databend_common_ast::ast::Expr; use databend_common_ast::parser::parse_values; use databend_common_ast::parser::tokenize_sql; -use databend_common_base::base::tokio::sync::Semaphore; -use databend_common_catalog::table::Table; use databend_common_catalog::table_context::TableContext; use databend_common_exception::ErrorCode; use databend_common_exception::Result; @@ -30,354 +28,24 @@ use databend_common_expression::Scalar; use databend_common_formats::FastFieldDecoderValues; use databend_common_formats::FastValuesDecodeFallback; use databend_common_formats::FastValuesDecoder; -use databend_common_functions::BUILTIN_FUNCTIONS; -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_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::BindContext; use databend_common_sql::Metadata; use databend_common_sql::MetadataRef; use databend_common_sql::NameResolutionContext; -use databend_common_storages_fuse::operations::BroadcastProcessor; -use databend_common_storages_fuse::operations::ReplaceIntoProcessor; -use databend_common_storages_fuse::operations::TransformSerializeBlock; -use databend_common_storages_fuse::operations::UnbranchedReplaceIntoProcessor; -use databend_common_storages_fuse::FuseTable; use parking_lot::RwLock; -use crate::pipelines::processors::transforms::TransformCastSchema; use crate::pipelines::PipelineBuilder; impl PipelineBuilder { // check if cast needed - fn check_schema_cast( + pub fn check_schema_cast( select_schema: Arc, output_schema: Arc, ) -> Result { let cast_needed = select_schema != output_schema; 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 { - input, - block_thresholds, - table_info, - on_conflicts, - bloom_filter_column_indexes, - segments, - block_slots, - need_insert, - .. - } = replace; - let max_threads = self.settings.get_max_threads()?; - let segment_partition_num = std::cmp::min(segments.len(), max_threads as usize); - let table = self.ctx.build_table_by_table_info(table_info, None)?; - let table = FuseTable::try_from_table(table.as_ref())?; - let schema = DataSchema::from(table.schema()).into(); - let cluster_stats_gen = - table.get_cluster_stats_gen(self.ctx.clone(), 0, *block_thresholds, Some(schema))?; - self.build_pipeline(input)?; - // connect to broadcast processor and append transform - let serialize_block_transform = TransformSerializeBlock::try_create( - self.ctx.clone(), - InputPort::create(), - OutputPort::create(), - table, - cluster_stats_gen, - MutationKind::Replace, - replace.table_meta_timestamps, - )?; - let mut block_builder = serialize_block_transform.get_block_builder(); - block_builder.source_schema = table.schema_with_stream(); - - if !*need_insert { - if segment_partition_num == 0 { - return Ok(()); - } - let broadcast_processor = BroadcastProcessor::new(segment_partition_num); - self.main_pipeline - .add_pipe(Pipe::create(1, segment_partition_num, vec![ - broadcast_processor.into_pipe_item(), - ])); - let max_threads = self.settings.get_max_threads()?; - let io_request_semaphore = Arc::new(Semaphore::new(max_threads as usize)); - - let merge_into_operation_aggregators = table.merge_into_mutators( - self.ctx.clone(), - segment_partition_num, - block_builder, - on_conflicts.clone(), - bloom_filter_column_indexes.clone(), - segments, - block_slots.clone(), - io_request_semaphore, - )?; - self.main_pipeline.add_pipe(Pipe::create( - segment_partition_num, - segment_partition_num, - merge_into_operation_aggregators, - )); - return Ok(()); - } - - // The Block Size and Rows is promised by DataSource by user. - if segment_partition_num == 0 { - let dummy_item = create_dummy_item(); - // ┌──────────────────────┐ ┌──────────────────┐ - // │ ├──┬────────►│ SerializeBlock │ - // ┌─────────────┐ │ ├──┘ └──────────────────┘ - // │ UpsertSource├─────►│ ReplaceIntoProcessor │ - // └─────────────┘ │ ├──┐ ┌──────────────────┐ - // │ ├──┴────────►│ DummyTransform │ - // └──────────────────────┘ └──────────────────┘ - // wrap them into pipeline, order matters! - self.main_pipeline.add_pipe(Pipe::create(2, 2, vec![ - serialize_block_transform.into_pipe_item(), - dummy_item, - ])); - } else { - // ┌──────────────────────┐ ┌──────────────────┐ - // │ ├──┬────────►│ SerializeBlock │ - // ┌─────────────┐ │ ├──┘ └──────────────────┘ - // │ UpsertSource├─────►│ ReplaceIntoProcessor │ - // └─────────────┘ │ ├──┐ ┌──────────────────┐ - // │ ├──┴────────►│BroadcastProcessor│ - // └──────────────────────┘ └──────────────────┘ - let broadcast_processor = BroadcastProcessor::new(segment_partition_num); - // wrap them into pipeline, order matters! - self.main_pipeline - .add_pipe(Pipe::create(2, segment_partition_num + 1, vec![ - serialize_block_transform.into_pipe_item(), - broadcast_processor.into_pipe_item(), - ])); - }; - - // 4. connect with MergeIntoOperationAggregators - if segment_partition_num != 0 { - // ┌──────────────────┐ ┌────────────────┐ - // ────►│ SerializeBlock ├──────────────►│ DummyTransform │ - // └──────────────────┘ └────────────────┘ - // - // ┌───────────────────┐ ┌──────────────────────┐ - // ────►│ ├──┬──────────►│MergeIntoOperationAggr│ - // │ ├──┘ └──────────────────────┘ - // │ BroadcastProcessor│ - // │ ├──┐ ┌──────────────────────┐ - // │ ├──┴──────────►│MergeIntoOperationAggr│ - // │ │ └──────────────────────┘ - // │ ├──┐ - // │ ├──┴──────────►┌──────────────────────┐ - // └───────────────────┘ │MergeIntoOperationAggr│ - // └──────────────────────┘ - - let item_size = segment_partition_num + 1; - let mut pipe_items = Vec::with_capacity(item_size); - // setup the dummy transform - pipe_items.push(create_dummy_item()); - - let max_threads = self.settings.get_max_threads()?; - let io_request_semaphore = Arc::new(Semaphore::new(max_threads as usize)); - - // setup the merge into operation aggregators - let mut merge_into_operation_aggregators = table.merge_into_mutators( - self.ctx.clone(), - segment_partition_num, - block_builder, - on_conflicts.clone(), - bloom_filter_column_indexes.clone(), - segments, - block_slots.clone(), - io_request_semaphore, - )?; - assert_eq!( - segment_partition_num, - merge_into_operation_aggregators.len() - ); - pipe_items.append(&mut merge_into_operation_aggregators); - - // extend the pipeline - assert_eq!(self.main_pipeline.output_len(), item_size); - assert_eq!(pipe_items.len(), item_size); - self.main_pipeline - .add_pipe(Pipe::create(item_size, item_size, pipe_items)); - } - Ok(()) - } - - // build deduplicate pipeline. - pub(crate) fn build_deduplicate(&mut self, deduplicate: &ReplaceDeduplicate) -> Result<()> { - let ReplaceDeduplicate { - input, - on_conflicts, - bloom_filter_column_indexes, - table_is_empty, - table_info, - select_ctx, - table_level_range_index, - target_schema, - need_insert, - delete_when, - .. - } = deduplicate; - - let tbl = self.ctx.build_table_by_table_info(table_info, None)?; - let table = FuseTable::try_from_table(tbl.as_ref())?; - self.build_pipeline(input)?; - let mut delete_column_idx = 0; - let mut modified_schema = DataSchema::from(target_schema.clone()).into(); - if let Some(ReplaceSelectCtx { - select_column_bindings, - select_schema, - }) = select_ctx - { - PipelineBuilder::build_result_projection( - &self.func_ctx, - input.output_schema()?, - select_column_bindings, - &mut self.main_pipeline, - false, - )?; - - let mut target_schema: DataSchema = target_schema.clone().into(); - if let Some((_, delete_column)) = delete_when { - delete_column_idx = select_schema.index_of(delete_column.as_str())?; - let delete_column = select_schema.field(delete_column_idx).clone(); - target_schema - .fields - .insert(delete_column_idx, delete_column); - modified_schema = Arc::new(target_schema.clone()); - } - let target_schema = Arc::new(target_schema.clone()); - if target_schema.fields().len() != select_schema.fields().len() { - return Err(ErrorCode::BadArguments( - "The number of columns in the target table is different from the number of columns in the SELECT clause", - )); - } - if Self::check_schema_cast(select_schema.clone(), target_schema.clone())? { - self.main_pipeline.try_add_transformer(|| { - TransformCastSchema::try_new( - select_schema.clone(), - target_schema.clone(), - self.func_ctx.clone(), - ) - })?; - } - } - - Self::fill_and_reorder_columns( - self.ctx.clone(), - &mut self.main_pipeline, - tbl.clone(), - Arc::new(target_schema.clone().into()), - )?; - - let block_thresholds = table.get_block_thresholds(); - build_compact_block_pipeline(&mut self.main_pipeline, block_thresholds)?; - - let _ = table.cluster_gen_for_append( - self.ctx.clone(), - &mut self.main_pipeline, - block_thresholds, - Some(modified_schema), - )?; - // 1. resize input to 1, since the UpsertTransform need to de-duplicate inputs "globally" - self.main_pipeline.try_resize(1)?; - - // 2. connect with ReplaceIntoProcessor - - // ┌──────────────────────┐ - // │ ├──┐ - // ┌─────────────┐ │ ├──┘ - // │ UpsertSource├─────►│ ReplaceIntoProcessor │ - // └─────────────┘ │ ├──┐ - // │ ├──┘ - // └──────────────────────┘ - // NOTE: here the pipe items of last pipe are arranged in the following order - // (0) -> output_port_append_data - // (1) -> output_port_merge_into_action - // the "downstream" is supposed to be connected with a processor which can process MergeIntoOperations - // in our case, it is the broadcast processor - let delete_when = if let Some((remote_expr, delete_column)) = delete_when { - Some(( - remote_expr.as_expr(&BUILTIN_FUNCTIONS), - delete_column.clone(), - )) - } else { - None - }; - let cluster_keys = table.linear_cluster_keys(self.ctx.clone()); - if *need_insert { - let replace_into_processor = ReplaceIntoProcessor::create( - self.ctx.clone(), - on_conflicts.clone(), - cluster_keys, - bloom_filter_column_indexes.clone(), - &table.schema(), - *table_is_empty, - table_level_range_index.clone(), - delete_when.map(|(expr, _)| (expr, delete_column_idx)), - )?; - self.main_pipeline - .add_pipe(replace_into_processor.into_pipe()); - } else { - let replace_into_processor = UnbranchedReplaceIntoProcessor::create( - self.ctx.as_ref(), - on_conflicts.clone(), - cluster_keys, - bloom_filter_column_indexes.clone(), - &table.schema(), - *table_is_empty, - table_level_range_index.clone(), - delete_when.map(|_| delete_column_idx), - )?; - self.main_pipeline - .add_pipe(replace_into_processor.into_pipe()); - } - Ok(()) - } } pub struct ValueSource { diff --git a/src/query/service/src/pipelines/builders/builder_row_fetch.rs b/src/query/service/src/pipelines/builders/builder_row_fetch.rs deleted file mode 100644 index 5b90720d5c9e6..0000000000000 --- a/src/query/service/src/pipelines/builders/builder_row_fetch.rs +++ /dev/null @@ -1,66 +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_base::runtime::GlobalIORuntime; -use databend_common_exception::Result; -use databend_common_pipeline_core::processors::InputPort; -use databend_common_pipeline_core::processors::OutputPort; -use databend_common_pipeline_core::Pipe; -use databend_common_pipeline_core::PipeItem; -use databend_common_pipeline_transforms::processors::create_dummy_item; -use databend_common_sql::executor::physical_plans::RowFetch; -use databend_common_sql::executor::PhysicalPlan; -use databend_common_storages_fuse::operations::row_fetch_processor; -use databend_common_storages_fuse::TableContext; -use tokio::sync::Semaphore; - -use crate::pipelines::PipelineBuilder; -impl PipelineBuilder { - pub(crate) fn build_row_fetch(&mut self, row_fetch: &RowFetch) -> Result<()> { - self.build_pipeline(&row_fetch.input)?; - let max_io_requests = self.ctx.get_settings().get_max_storage_io_requests()? as usize; - let row_fetch_runtime = GlobalIORuntime::instance(); - let row_fetch_semaphore = Arc::new(Semaphore::new(max_io_requests)); - let processor = row_fetch_processor( - self.ctx.clone(), - row_fetch.row_id_col_offset, - &row_fetch.source, - row_fetch.cols_to_fetch.clone(), - row_fetch.need_wrap_nullable, - row_fetch_semaphore, - row_fetch_runtime, - )?; - if !matches!(&*row_fetch.input, PhysicalPlan::MutationSplit(_)) { - self.main_pipeline.add_transform(processor)?; - } else { - let output_len = self.main_pipeline.output_len(); - let mut pipe_items = Vec::with_capacity(output_len); - for i in 0..output_len { - if i % 2 == 0 { - let input = InputPort::create(); - let output = OutputPort::create(); - let processor_ptr = processor(input.clone(), output.clone())?; - pipe_items.push(PipeItem::create(processor_ptr, vec![input], vec![output])); - } else { - pipe_items.push(create_dummy_item()); - } - } - self.main_pipeline - .add_pipe(Pipe::create(output_len, output_len, pipe_items)); - } - Ok(()) - } -} diff --git a/src/query/service/src/pipelines/builders/builder_scalar.rs b/src/query/service/src/pipelines/builders/builder_scalar.rs deleted file mode 100644 index 1b89b905dab00..0000000000000 --- a/src/query/service/src/pipelines/builders/builder_scalar.rs +++ /dev/null @@ -1,52 +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_functions::BUILTIN_FUNCTIONS; -use databend_common_pipeline_transforms::processors::TransformPipelineHelper; -use databend_common_sql::evaluator::BlockOperator; -use databend_common_sql::evaluator::CompoundBlockOperator; -use databend_common_sql::executor::physical_plans::EvalScalar; - -use crate::pipelines::PipelineBuilder; - -impl PipelineBuilder { - pub(crate) fn build_eval_scalar(&mut self, eval_scalar: &EvalScalar) -> Result<()> { - self.build_pipeline(&eval_scalar.input)?; - - let input_schema = eval_scalar.input.output_schema()?; - let exprs = eval_scalar - .exprs - .iter() - .map(|(scalar, _)| scalar.as_expr(&BUILTIN_FUNCTIONS)) - .collect::>(); - - if exprs.is_empty() { - return Ok(()); - } - - let op = BlockOperator::Map { - exprs, - projections: Some(eval_scalar.projections.clone()), - }; - - let num_input_columns = input_schema.num_fields(); - - self.main_pipeline.add_transformer(|| { - CompoundBlockOperator::new(vec![op.clone()], self.func_ctx.clone(), num_input_columns) - }); - - Ok(()) - } -} diff --git a/src/query/service/src/pipelines/builders/builder_scan.rs b/src/query/service/src/pipelines/builders/builder_scan.rs deleted file mode 100644 index 8a3270b4421bd..0000000000000 --- a/src/query/service/src/pipelines/builders/builder_scan.rs +++ /dev/null @@ -1,156 +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_catalog::table_context::TableContext; -use databend_common_exception::ErrorCode; -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::OneBlockSource; -use databend_common_pipeline_transforms::processors::TransformPipelineHelper; -use databend_common_sql::evaluator::BlockOperator; -use databend_common_sql::evaluator::CompoundBlockOperator; -use databend_common_sql::executor::physical_plans::CacheScan; -use databend_common_sql::executor::physical_plans::ConstantTableScan; -use databend_common_sql::executor::physical_plans::ExpressionScan; -use databend_common_sql::executor::physical_plans::TableScan; -use databend_common_sql::plans::CacheSource; - -use crate::pipelines::processors::transforms::CacheSourceState; -use crate::pipelines::processors::transforms::HashJoinCacheState; -use crate::pipelines::processors::transforms::TransformAddInternalColumns; -use crate::pipelines::processors::transforms::TransformCacheScan; -use crate::pipelines::processors::transforms::TransformExpressionScan; -use crate::pipelines::PipelineBuilder; - -impl PipelineBuilder { - pub(crate) fn build_table_scan(&mut self, scan: &TableScan) -> Result<()> { - let table = self.ctx.build_table_from_source_plan(&scan.source)?; - self.ctx.set_partitions(scan.source.parts.clone())?; - self.ctx - .set_wait_runtime_filter(scan.scan_id, self.contain_sink_processor); - if self.ctx.get_settings().get_enable_prune_pipeline()? { - if let Some(prune_pipeline) = table.build_prune_pipeline( - self.ctx.clone(), - &scan.source, - &mut self.main_pipeline, - scan.plan_id, - )? { - self.pipelines.push(prune_pipeline); - } - } - table.read_data( - self.ctx.clone(), - &scan.source, - &mut self.main_pipeline, - true, - )?; - - // Fill internal columns if needed. - if let Some(internal_columns) = &scan.internal_column { - self.main_pipeline - .add_transformer(|| TransformAddInternalColumns::new(internal_columns.clone())); - } - - let schema = scan.source.schema(); - let mut projection = scan - .name_mapping - .keys() - .map(|name| schema.index_of(name.as_str())) - .collect::>>()?; - projection.sort(); - - // if projection is sequential, no need to add projection - if projection != (0..schema.fields().len()).collect::>() { - let ops = vec![BlockOperator::Project { projection }]; - let num_input_columns = schema.num_fields(); - self.main_pipeline.add_transformer(|| { - CompoundBlockOperator::new(ops.clone(), self.func_ctx.clone(), num_input_columns) - }); - } - - Ok(()) - } - - pub(crate) fn build_constant_table_scan(&mut self, scan: &ConstantTableScan) -> Result<()> { - self.main_pipeline.add_source( - |output| { - let block = if !scan.values.is_empty() { - DataBlock::new_from_columns(scan.values.clone()) - } else { - DataBlock::new(vec![], scan.num_rows) - }; - OneBlockSource::create(output, block) - }, - 1, - ) - } - - pub(crate) fn build_cache_scan(&mut self, scan: &CacheScan) -> Result<()> { - let max_threads = self.settings.get_max_threads()?; - let max_block_size = self.settings.get_max_block_size()? as usize; - let cache_source_state = match &scan.cache_source { - CacheSource::HashJoinBuild((cache_index, column_indexes)) => { - let hash_join_state = match self.hash_join_states.get(cache_index) { - Some(hash_join_state) => hash_join_state.clone(), - None => { - return Err(ErrorCode::Internal( - "Hash join state not found during building cache scan".to_string(), - )); - } - }; - CacheSourceState::HashJoinCacheState(HashJoinCacheState::new( - column_indexes.clone(), - hash_join_state, - max_block_size, - )) - } - }; - - self.main_pipeline.add_source( - |output| { - TransformCacheScan::create(self.ctx.clone(), output, cache_source_state.clone()) - }, - max_threads as usize, - ) - } - - pub(crate) fn build_expression_scan(&mut self, scan: &ExpressionScan) -> Result<()> { - self.build_pipeline(&scan.input)?; - - let values = scan - .values - .iter() - .map(|row| { - row.iter() - .map(|scalar| scalar.as_expr(&BUILTIN_FUNCTIONS)) - .collect::>() - }) - .collect::>(); - - let fun_ctx = self.func_ctx.clone(); - - self.main_pipeline.add_transform(|input, output| { - Ok(ProcessorPtr::create(TransformExpressionScan::create( - input, - output, - values.clone(), - fun_ctx.clone(), - ))) - })?; - - Ok(()) - } -} diff --git a/src/query/service/src/pipelines/builders/builder_sort.rs b/src/query/service/src/pipelines/builders/builder_sort.rs index 5665787f45543..7ebba24b21dd2 100644 --- a/src/query/service/src/pipelines/builders/builder_sort.rs +++ b/src/query/service/src/pipelines/builders/builder_sort.rs @@ -26,123 +26,18 @@ use databend_common_pipeline_transforms::processors::try_add_multi_sort_merge; use databend_common_pipeline_transforms::processors::TransformPipelineHelper; use databend_common_pipeline_transforms::processors::TransformSortPartial; use databend_common_pipeline_transforms::MemorySettings; -use databend_common_sql::evaluator::BlockOperator; -use databend_common_sql::evaluator::CompoundBlockOperator; -use databend_common_sql::executor::physical_plans::Sort; use databend_common_storage::DataOperator; use databend_common_storages_fuse::TableContext; use databend_storages_common_cache::TempDirManager; use crate::pipelines::memory_settings::MemorySettingsExt; use crate::pipelines::processors::transforms::TransformSortBuilder; -use crate::pipelines::PipelineBuilder; use crate::sessions::QueryContext; use crate::spillers::Spiller; use crate::spillers::SpillerConfig; use crate::spillers::SpillerDiskConfig; use crate::spillers::SpillerType; -impl PipelineBuilder { - // The pipeline graph of distributed sort can be found in https://github.com/datafuselabs/databend/pull/13881 - pub(crate) fn build_sort(&mut self, sort: &Sort) -> Result<()> { - self.build_pipeline(&sort.input)?; - - let input_schema = sort.input.output_schema()?; - - if !matches!(sort.after_exchange, Some(true)) { - // If the Sort plan is after exchange, we don't need to do a projection, - // because the data is already projected in each cluster node. - if let Some(proj) = &sort.pre_projection { - // Do projection to reduce useless data copying during sorting. - let projection = proj - .iter() - .filter_map(|i| input_schema.index_of(&i.to_string()).ok()) - .collect::>(); - - if projection.len() < input_schema.fields().len() { - // Only if the projection is not a full projection, we need to add a projection transform. - self.main_pipeline.add_transformer(|| { - CompoundBlockOperator::new( - vec![BlockOperator::Project { - projection: projection.clone(), - }], - self.func_ctx.clone(), - input_schema.num_fields(), - ) - }); - } - } - } - - let plan_schema = sort.output_schema()?; - - let sort_desc = sort - .order_by - .iter() - .map(|desc| { - let offset = plan_schema.index_of(&desc.order_by.to_string())?; - Ok(SortColumnDescription { - offset, - asc: desc.asc, - nulls_first: desc.nulls_first, - }) - }) - .collect::>>()?; - - self.build_sort_pipeline(plan_schema, sort_desc, sort.limit, sort.after_exchange) - } - - fn build_sort_pipeline( - &mut self, - plan_schema: DataSchemaRef, - sort_desc: Vec, - limit: Option, - after_exchange: Option, - ) -> Result<()> { - let max_threads = self.settings.get_max_threads()? as usize; - let sort_desc = sort_desc.into(); - - // TODO(Winter): the query will hang in MultiSortMergeProcessor when max_threads == 1 and output_len != 1 - if self.main_pipeline.output_len() == 1 || max_threads == 1 { - self.main_pipeline.try_resize(max_threads)?; - } - - let builder = SortPipelineBuilder::create(self.ctx.clone(), plan_schema, sort_desc)? - .with_limit(limit); - - match after_exchange { - Some(true) => { - // Build for the coordinator node. - // We only build a `MultiSortMergeTransform`, - // as the data is already sorted in each cluster node. - // The input number of the transform is equal to the number of cluster nodes. - if self.main_pipeline.output_len() > 1 { - builder - .remove_order_col_at_last() - .build_multi_merge(&mut self.main_pipeline) - } else { - builder - .remove_order_col_at_last() - .build_merge_sort_pipeline(&mut self.main_pipeline, true) - } - } - Some(false) => { - // Build for each cluster node. - // We build the full sort pipeline for it. - // Don't remove the order column at last. - builder.build_full_sort_pipeline(&mut self.main_pipeline) - } - None => { - // Build for single node mode. - // We build the full sort pipeline for it. - builder - .remove_order_col_at_last() - .build_full_sort_pipeline(&mut self.main_pipeline) - } - } - } -} - pub struct SortPipelineBuilder { ctx: Arc, schema: DataSchemaRef, diff --git a/src/query/service/src/pipelines/builders/builder_udf.rs b/src/query/service/src/pipelines/builders/builder_udf.rs deleted file mode 100644 index da9b35d5559bf..0000000000000 --- a/src/query/service/src/pipelines/builders/builder_udf.rs +++ /dev/null @@ -1,49 +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::Udf; - -use crate::pipelines::processors::transforms::TransformUdfScript; -use crate::pipelines::processors::transforms::TransformUdfServer; -use crate::pipelines::PipelineBuilder; - -impl PipelineBuilder { - pub(crate) fn build_udf(&mut self, udf: &Udf) -> Result<()> { - self.build_pipeline(&udf.input)?; - - if udf.script_udf { - let runtimes = TransformUdfScript::init_runtime(&udf.udf_funcs)?; - self.main_pipeline.try_add_transformer(|| { - Ok(TransformUdfScript::new( - self.func_ctx.clone(), - udf.udf_funcs.clone(), - runtimes.clone(), - )) - }) - } else { - let semaphore = TransformUdfServer::init_semaphore(self.ctx.clone())?; - let endpoints = TransformUdfServer::init_endpoints(self.ctx.clone(), &udf.udf_funcs)?; - self.main_pipeline.try_add_async_transformer(|| { - TransformUdfServer::new( - self.ctx.clone(), - udf.udf_funcs.clone(), - semaphore.clone(), - endpoints.clone(), - ) - }) - } - } -} diff --git a/src/query/service/src/pipelines/builders/builder_union_all.rs b/src/query/service/src/pipelines/builders/builder_union_all.rs deleted file mode 100644 index 757347d1f7c70..0000000000000 --- a/src/query/service/src/pipelines/builders/builder_union_all.rs +++ /dev/null @@ -1,95 +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_catalog::table_context::TableContext; -use databend_common_exception::Result; -use databend_common_expression::RemoteExpr; -use databend_common_functions::BUILTIN_FUNCTIONS; -use databend_common_pipeline_core::processors::ProcessorPtr; -use databend_common_sql::evaluator::BlockOperator; -use databend_common_sql::evaluator::CompoundBlockOperator; -use databend_common_sql::executor::physical_plans::UnionAll; -use databend_common_sql::executor::PhysicalPlan; -use databend_common_sql::IndexType; - -use crate::pipelines::PipelineBuilder; - -impl PipelineBuilder { - pub fn build_union_all(&mut self, union_all: &UnionAll) -> Result<()> { - if !union_all.cte_scan_names.is_empty() { - return self.build_recursive_cte_source(union_all); - } - - self.build_union_all_children(&union_all.left, &union_all.left_outputs)?; - let left_sinks = self.main_pipeline.take_sinks(); - self.build_union_all_children(&union_all.right, &union_all.right_outputs)?; - let right_sinks = self.main_pipeline.take_sinks(); - - let outputs = std::cmp::max(left_sinks.len(), right_sinks.len()); - let sequence_groups = vec![(left_sinks.len(), false), (right_sinks.len(), false)]; - - self.main_pipeline.extend_sinks(left_sinks); - self.main_pipeline.extend_sinks(right_sinks); - - match self.ctx.get_settings().get_enable_parallel_union_all()? { - true => self.main_pipeline.resize(outputs, false), - false => self.main_pipeline.sequence_group(sequence_groups, outputs), - } - } - - fn build_union_all_children( - &mut self, - input: &PhysicalPlan, - projection: &[(IndexType, Option)], - ) -> Result<()> { - self.build_pipeline(input)?; - let output_schema = input.output_schema()?; - - let mut expr_offset = output_schema.num_fields(); - let mut new_projection = Vec::with_capacity(projection.len()); - let mut exprs = Vec::with_capacity(projection.len()); - for (idx, expr) in projection { - let Some(expr) = expr else { - new_projection.push(output_schema.index_of(&idx.to_string())?); - continue; - }; - - exprs.push(expr.as_expr(&BUILTIN_FUNCTIONS)); - new_projection.push(expr_offset); - expr_offset += 1; - } - - let mut operators = Vec::with_capacity(2); - if !exprs.is_empty() { - operators.push(BlockOperator::Map { - exprs, - projections: None, - }); - } - - operators.push(BlockOperator::Project { - projection: new_projection, - }); - - self.main_pipeline.add_transform(|input, output| { - Ok(ProcessorPtr::create(CompoundBlockOperator::create( - input, - output, - output_schema.num_fields(), - self.func_ctx.clone(), - operators.clone(), - ))) - }) - } -} diff --git a/src/query/service/src/pipelines/builders/builder_window.rs b/src/query/service/src/pipelines/builders/builder_window.rs deleted file mode 100644 index 187bb25d7dd77..0000000000000 --- a/src/query/service/src/pipelines/builders/builder_window.rs +++ /dev/null @@ -1,228 +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::atomic; -use std::sync::atomic::AtomicUsize; - -use databend_common_catalog::table_context::TableContext; -use databend_common_exception::Result; -use databend_common_expression::types::DataType; -use databend_common_expression::types::NumberDataType; -use databend_common_expression::with_number_mapped_type; -use databend_common_expression::SortColumnDescription; -use databend_common_pipeline_core::processors::Processor; -use databend_common_pipeline_core::processors::ProcessorPtr; -use databend_common_pipeline_transforms::MemorySettings; -use databend_common_sql::executor::physical_plans::Window; -use databend_common_sql::executor::physical_plans::WindowPartition; -use databend_storages_common_cache::TempDirManager; - -use crate::pipelines::memory_settings::MemorySettingsExt; -use crate::pipelines::processors::transforms::FrameBound; -use crate::pipelines::processors::transforms::SortStrategy; -use crate::pipelines::processors::transforms::TransformWindow; -use crate::pipelines::processors::transforms::TransformWindowPartitionCollect; -use crate::pipelines::processors::transforms::WindowFunctionInfo; -use crate::pipelines::processors::transforms::WindowPartitionExchange; -use crate::pipelines::processors::transforms::WindowPartitionTopNExchange; -use crate::pipelines::processors::transforms::WindowSortDesc; -use crate::pipelines::PipelineBuilder; -use crate::spillers::SpillerDiskConfig; - -impl PipelineBuilder { - pub(crate) fn build_window(&mut self, window: &Window) -> Result<()> { - self.build_pipeline(&window.input)?; - - let input_schema = window.input.output_schema()?; - let partition_by = window - .partition_by - .iter() - .map(|p| { - let offset = input_schema.index_of(&p.to_string())?; - Ok(offset) - }) - .collect::>>()?; - let order_by = window - .order_by - .iter() - .map(|o| { - let offset = input_schema.index_of(&o.order_by.to_string())?; - Ok(WindowSortDesc { - offset, - asc: o.asc, - nulls_first: o.nulls_first, - is_nullable: input_schema.field(offset).is_nullable(), - }) - }) - .collect::>>()?; - - let old_output_len = self.main_pipeline.output_len(); - // `TransformWindow` is a pipeline breaker. - if partition_by.is_empty() { - self.main_pipeline.try_resize(1)?; - } - let func = WindowFunctionInfo::try_create(&window.func, &input_schema)?; - // Window - self.main_pipeline.add_transform(|input, output| { - // The transform can only be created here, because it cannot be cloned. - - let transform = if window.window_frame.units.is_rows() { - let start_bound = FrameBound::try_from(&window.window_frame.start_bound)?; - let end_bound = FrameBound::try_from(&window.window_frame.end_bound)?; - Box::new(TransformWindow::::try_create_rows( - input, - output, - func.clone(), - partition_by.clone(), - order_by.clone(), - (start_bound, end_bound), - )?) as Box - } else { - if order_by.len() == 1 { - // If the length of order_by is 1, there may be a RANGE frame. - let data_type = input_schema - .field(order_by[0].offset) - .data_type() - .remove_nullable(); - with_number_mapped_type!(|NUM_TYPE| match data_type { - DataType::Number(NumberDataType::NUM_TYPE) => { - let start_bound = - FrameBound::try_from(&window.window_frame.start_bound)?; - let end_bound = FrameBound::try_from(&window.window_frame.end_bound)?; - return Ok(ProcessorPtr::create(Box::new( - TransformWindow::::try_create_range( - input, - output, - func.clone(), - partition_by.clone(), - order_by.clone(), - (start_bound, end_bound), - )?, - ) - as Box)); - } - _ => {} - }) - } - - // There is no offset in the RANGE frame. (just CURRENT ROW or UNBOUNDED) - // So we can use any number type to create the transform. - let start_bound = FrameBound::try_from(&window.window_frame.start_bound)?; - let end_bound = FrameBound::try_from(&window.window_frame.end_bound)?; - Box::new(TransformWindow::::try_create_range( - input, - output, - func.clone(), - partition_by.clone(), - order_by.clone(), - (start_bound, end_bound), - )?) as Box - }; - Ok(ProcessorPtr::create(transform)) - })?; - if partition_by.is_empty() { - self.main_pipeline.try_resize(old_output_len)?; - } - Ok(()) - } - - pub(crate) fn build_window_partition( - &mut self, - window_partition: &WindowPartition, - ) -> Result<()> { - self.build_pipeline(&window_partition.input)?; - - let num_processors = self.main_pipeline.output_len(); - - // Settings. - let settings = self.ctx.get_settings(); - let num_partitions = settings.get_window_num_partitions()?; - - let plan_schema = window_partition.output_schema()?; - - let partition_by = window_partition - .partition_by - .iter() - .map(|index| plan_schema.index_of(&index.to_string())) - .collect::>>()?; - - let sort_desc = window_partition - .order_by - .iter() - .map(|desc| { - let offset = plan_schema.index_of(&desc.order_by.to_string())?; - Ok(SortColumnDescription { - offset, - asc: desc.asc, - nulls_first: desc.nulls_first, - }) - }) - .collect::>>()?; - - if let Some(top_n) = &window_partition.top_n - && top_n.top < 10000 - { - self.main_pipeline.exchange( - num_processors, - WindowPartitionTopNExchange::create( - partition_by.clone(), - sort_desc.clone(), - top_n.top, - top_n.func, - num_partitions as u64, - ), - ) - } else { - self.main_pipeline.exchange( - num_processors, - WindowPartitionExchange::create(partition_by.clone(), num_partitions), - ); - } - - let temp_dir_manager = TempDirManager::instance(); - let disk_bytes_limit = settings.get_window_partition_spilling_to_disk_bytes_limit()?; - let enable_dio = settings.get_enable_dio()?; - let disk_spill = temp_dir_manager - .get_disk_spill_dir(disk_bytes_limit, &self.ctx.get_id()) - .map(|temp_dir| SpillerDiskConfig::new(temp_dir, enable_dio)) - .transpose()?; - - let have_order_col = window_partition.after_exchange.unwrap_or(false); - let window_spill_settings = MemorySettings::from_window_settings(&self.ctx)?; - - let processor_id = AtomicUsize::new(0); - self.main_pipeline.add_transform(|input, output| { - let strategy = SortStrategy::try_create( - &settings, - sort_desc.clone(), - plan_schema.clone(), - have_order_col, - )?; - Ok(ProcessorPtr::create(Box::new( - TransformWindowPartitionCollect::new( - self.ctx.clone(), - input, - output, - &settings, - processor_id.fetch_add(1, atomic::Ordering::AcqRel), - num_processors, - num_partitions, - window_spill_settings.clone(), - disk_spill.clone(), - strategy, - )?, - ))) - }) - } -} diff --git a/src/query/service/src/pipelines/builders/merge_into_join_optimizations.rs b/src/query/service/src/pipelines/builders/merge_into_join_optimizations.rs index f92646e61479c..a46cd22d07ddd 100644 --- a/src/query/service/src/pipelines/builders/merge_into_join_optimizations.rs +++ b/src/query/service/src/pipelines/builders/merge_into_join_optimizations.rs @@ -12,17 +12,18 @@ // See the License for the specific language governing permissions and // limitations under the License. -use databend_common_sql::executor::physical_plans::HashJoin; -use databend_common_sql::executor::PhysicalPlan; use databend_common_storages_fuse::operations::need_reserve_block_info; +use crate::physical_plans::HashJoin; +use crate::physical_plans::PhysicalPlanDynExt; +use crate::physical_plans::TableScan; use crate::pipelines::PipelineBuilder; impl PipelineBuilder { pub(crate) fn merge_into_get_optimization_flag(&self, join: &HashJoin) -> (bool, bool) { // for merge into target table as build side. - match &*join.build { - PhysicalPlan::TableScan(scan) => match scan.table_index { + if let Some(scan) = join.build.downcast_ref::() { + return match scan.table_index { None | Some(databend_common_sql::DUMMY_TABLE_INDEX) => (false, false), Some(table_index) => match need_reserve_block_info(self.ctx.clone(), table_index) { // due to issue https://github.com/datafuselabs/databend/issues/15643, @@ -32,8 +33,9 @@ impl PipelineBuilder { (true, is_distributed) => (false, is_distributed), _ => (false, false), }, - }, - _ => (false, false), + }; } + + (false, false) } } diff --git a/src/query/service/src/pipelines/builders/mod.rs b/src/query/service/src/pipelines/builders/mod.rs index 552580bc7c467..89b8b4372d57f 100644 --- a/src/query/service/src/pipelines/builders/mod.rs +++ b/src/query/service/src/pipelines/builders/mod.rs @@ -12,41 +12,16 @@ // See the License for the specific language governing permissions and // limitations under the License. -mod builder_add_stream_column; mod builder_aggregate; mod builder_append_table; -mod builder_async_function; -mod builder_broadcast; -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; -mod builder_hilbert_partition; -mod builder_insert_multi_table; mod builder_join; -mod builder_limit; mod builder_mutation; -mod builder_mutation_manipulate; -mod builder_mutation_organize; -mod builder_mutation_source; -mod builder_mutation_split; mod builder_on_finished; mod builder_project; -mod builder_recluster; -mod builder_recursive_cte; mod builder_replace_into; -mod builder_row_fetch; -mod builder_scalar; -mod builder_scan; mod builder_sort; -mod builder_udf; -mod builder_union_all; -mod builder_window; mod merge_into_join_optimizations; mod transform_builder; diff --git a/src/query/service/src/pipelines/builders/transform_builder.rs b/src/query/service/src/pipelines/builders/transform_builder.rs index 3a77f8f15b6f2..d8c6653903804 100644 --- a/src/query/service/src/pipelines/builders/transform_builder.rs +++ b/src/query/service/src/pipelines/builders/transform_builder.rs @@ -31,6 +31,7 @@ use databend_common_pipeline_transforms::processors::TransformCompactBlock; use databend_common_pipeline_transforms::processors::TransformDummy; use databend_common_sql::evaluator::BlockOperator; use databend_common_sql::evaluator::CompoundBlockOperator; +use databend_common_sql::executor::physical_plans::MutationKind; use databend_common_sql::ColumnSet; use databend_common_storages_factory::Table; use databend_common_storages_fuse::operations::new_serialize_segment_processor; @@ -48,7 +49,6 @@ use crate::pipelines::processors::transforms::TransformResortAddOn; use crate::pipelines::processors::InputPort; use crate::pipelines::processors::OutputPort; use crate::pipelines::PipelineBuilder; -use crate::sql::executor::physical_plans::MutationKind; impl PipelineBuilder { pub(crate) fn filter_transform_builder( diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs index e2e2b1c30a001..f73e6fd4ecd19 100644 --- a/src/query/service/src/pipelines/pipeline_builder.rs +++ b/src/query/service/src/pipelines/pipeline_builder.rs @@ -26,10 +26,12 @@ use databend_common_pipeline_core::processors::PlanScopeGuard; use databend_common_pipeline_core::ExecutionInfo; use databend_common_pipeline_core::Pipeline; use databend_common_settings::Settings; -use databend_common_sql::executor::PhysicalPlan; use super::PipelineBuilderData; use crate::interpreters::CreateTableInterpreter; +use crate::physical_plans::ExchangeSink; +use crate::physical_plans::PhysicalPlan; +use crate::physical_plans::PhysicalPlanDynExt; use crate::pipelines::processors::HashJoinBuildState; use crate::pipelines::processors::HashJoinState; use crate::pipelines::PipelineBuildResult; @@ -114,36 +116,25 @@ impl PipelineBuilder { } pub(crate) fn add_plan_scope(&mut self, plan: &PhysicalPlan) -> Result> { - match plan { - PhysicalPlan::EvalScalar(v) if v.exprs.is_empty() => Ok(None), - - // hided plans in profile - PhysicalPlan::Shuffle(_) => Ok(None), - PhysicalPlan::Exchange(_) => Ok(None), - PhysicalPlan::ExchangeSink(_) => Ok(None), - PhysicalPlan::ExchangeSource(_) => Ok(None), - PhysicalPlan::ChunkCastSchema(_) => Ok(None), - PhysicalPlan::ChunkFillAndReorder(_) => Ok(None), - PhysicalPlan::ChunkMerge(_) => Ok(None), + if !plan.display_in_profile() { + return Ok(None); + } - _ => { - let desc = plan.get_desc()?; - let plan_labels = plan.get_labels()?; - let mut profile_labels = Vec::with_capacity(plan_labels.len()); - for (name, value) in plan_labels { - profile_labels.push(ProfileLabel::create(name, value)); - } + let desc = plan.get_desc()?; + let plan_labels = plan.get_labels()?; + let mut profile_labels = Vec::with_capacity(plan_labels.len()); + for (name, value) in plan_labels { + profile_labels.push(ProfileLabel::create(name, value)); + } - let scope = PlanScope::create( - plan.get_id(), - plan.name(), - Arc::new(desc), - Arc::new(profile_labels), - ); + let scope = PlanScope::create( + plan.get_id(), + plan.get_name(), + Arc::new(desc), + Arc::new(profile_labels), + ); - Ok(Some(scope.enter_scope_guard())) - } - } + Ok(Some(scope.enter_scope_guard())) } pub(crate) fn is_exchange_parent(&self) -> bool { @@ -158,145 +149,9 @@ impl PipelineBuilder { pub(crate) fn build_pipeline(&mut self, plan: &PhysicalPlan) -> Result<()> { let _guard = self.add_plan_scope(plan)?; self.is_exchange_stack - .push(matches!(plan, PhysicalPlan::ExchangeSink(_))); - - match plan { - // ============================== - // 1. Data Source Plans - // ============================== - // Basic table scans - retrieve data from tables - PhysicalPlan::TableScan(scan) => self.build_table_scan(scan), - PhysicalPlan::ConstantTableScan(scan) => self.build_constant_table_scan(scan), - PhysicalPlan::CacheScan(cache_scan) => self.build_cache_scan(cache_scan), - PhysicalPlan::ExpressionScan(expression_scan) => { - self.build_expression_scan(expression_scan) - } - PhysicalPlan::RecursiveCteScan(scan) => self.build_recursive_cte_scan(scan), - - // Special source operations - PhysicalPlan::MutationSource(mutation_source) => { - self.build_mutation_source(mutation_source) - } - - // ============================== - // 2. Relational Operators - // ============================== - // Filtering and projection - PhysicalPlan::Filter(filter) => self.build_filter(filter), - PhysicalPlan::EvalScalar(eval_scalar) => self.build_eval_scalar(eval_scalar), - PhysicalPlan::ProjectSet(project_set) => self.build_project_set(project_set), - - // Sorting and limiting - PhysicalPlan::Sort(sort) => self.build_sort(sort), - PhysicalPlan::Limit(limit) => self.build_limit(limit), - PhysicalPlan::RowFetch(row_fetch) => self.build_row_fetch(row_fetch), - - // Join operations - PhysicalPlan::HashJoin(join) => self.build_hash_join(join), - PhysicalPlan::RangeJoin(range_join) => self.build_range_join(range_join), - - // Aggregation operations - PhysicalPlan::AggregateExpand(aggregate) => self.build_aggregate_expand(aggregate), - PhysicalPlan::AggregatePartial(aggregate) => self.build_aggregate_partial(aggregate), - PhysicalPlan::AggregateFinal(aggregate) => self.build_aggregate_final(aggregate), - - // Window functions - PhysicalPlan::Window(window) => self.build_window(window), - PhysicalPlan::WindowPartition(window_partition) => { - self.build_window_partition(window_partition) - } - - PhysicalPlan::UnionAll(union_all) => self.build_union_all(union_all), - - // ============================== - // 3. Data Distribution - // ============================== - PhysicalPlan::ExchangeSink(sink) => self.build_exchange_sink(sink), - PhysicalPlan::ExchangeSource(source) => self.build_exchange_source(source), - PhysicalPlan::DistributedInsertSelect(insert_select) => { - self.build_distributed_insert_select(insert_select) - } - PhysicalPlan::Shuffle(shuffle) => self.build_shuffle(shuffle), - PhysicalPlan::Duplicate(duplicate) => self.build_duplicate(duplicate), - PhysicalPlan::BroadcastSource(source) => self.build_broadcast_source(source), - PhysicalPlan::BroadcastSink(sink) => self.build_broadcast_sink(sink), - - // ============================== - // 4. Data Modification Operations - // ============================== - // Copy operations - PhysicalPlan::CopyIntoTable(copy) => self.build_copy_into_table(copy), - PhysicalPlan::CopyIntoLocation(copy) => self.build_copy_into_location(copy), - - // Replace operations - 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), - - // Mutation operations (DELETE/UPDATE) - PhysicalPlan::Mutation(mutation) => self.build_mutation(mutation), - PhysicalPlan::MutationSplit(mutation_split) => { - self.build_mutation_split(mutation_split) - } - PhysicalPlan::MutationManipulate(mutation_manipulate) => { - self.build_mutation_manipulate(mutation_manipulate) - } - PhysicalPlan::MutationOrganize(mutation_organize) => { - self.build_mutation_organize(mutation_organize) - } - PhysicalPlan::AddStreamColumn(add_stream_column) => { - self.build_add_stream_column(add_stream_column) - } - PhysicalPlan::ColumnMutation(column_mutation) => { - self.build_column_mutation(column_mutation) - } - - // Commit operations - PhysicalPlan::CommitSink(plan) => self.build_commit_sink(plan), - - // MERGE INTO chunk processing operations - PhysicalPlan::ChunkFilter(chunk_filter) => self.build_chunk_filter(chunk_filter), - PhysicalPlan::ChunkEvalScalar(chunk_project) => { - self.build_chunk_eval_scalar(chunk_project) - } - PhysicalPlan::ChunkCastSchema(chunk_cast_schema) => { - self.build_chunk_cast_schema(chunk_cast_schema) - } - PhysicalPlan::ChunkFillAndReorder(chunk_fill_and_reorder) => { - self.build_chunk_fill_and_reorder(chunk_fill_and_reorder) - } - PhysicalPlan::ChunkAppendData(chunk_append_data) => { - self.build_chunk_append_data(chunk_append_data) - } - PhysicalPlan::ChunkMerge(chunk_merge) => self.build_chunk_merge(chunk_merge), - PhysicalPlan::ChunkCommitInsert(chunk_commit_insert) => { - self.build_chunk_commit_insert(chunk_commit_insert) - } - - // ============================== - // 5. Data Maintenance Operations - // ============================== - PhysicalPlan::CompactSource(compact) => self.build_compact_source(compact), - PhysicalPlan::Recluster(recluster) => self.build_recluster(recluster), - PhysicalPlan::HilbertPartition(partition) => self.build_hilbert_partition(partition), - - // ============================== - // 6. Special Processing Operations - // ============================== - // User-defined functions and async operations - PhysicalPlan::Udf(udf) => self.build_udf(udf), - PhysicalPlan::AsyncFunction(async_func) => self.build_async_function(async_func), - - // ============================== - // 7. Invalid Plans - // ============================== - PhysicalPlan::Exchange(_) => Err(ErrorCode::Internal( - "Invalid physical plan with PhysicalPlan::Exchange", - )), - }?; + .push(plan.downcast_ref::().is_some()); + plan.build_pipeline(self)?; self.is_exchange_stack.pop(); Ok(()) diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/desc.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/desc.rs index db3d5c2dc5e47..e055cc61fd18b 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/desc.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/desc.rs @@ -19,11 +19,11 @@ use databend_common_expression::Expr; use databend_common_expression::RemoteExpr; use databend_common_functions::BUILTIN_FUNCTIONS; use databend_common_sql::executor::cast_expr_to_non_null_boolean; -use databend_common_sql::executor::physical_plans::HashJoin; -use databend_common_sql::executor::PhysicalRuntimeFilter; -use databend_common_sql::executor::PhysicalRuntimeFilters; use parking_lot::RwLock; +use crate::physical_plans::HashJoin; +use crate::physical_plans::PhysicalRuntimeFilter; +use crate::physical_plans::PhysicalRuntimeFilters; use crate::sql::plans::JoinType; pub const MARKER_KIND_TRUE: u8 = 0; diff --git a/src/query/service/src/pipelines/processors/transforms/range_join/ie_join_state.rs b/src/query/service/src/pipelines/processors/transforms/range_join/ie_join_state.rs index e0cfec1cd01e1..7ee62f83d2772 100644 --- a/src/query/service/src/pipelines/processors/transforms/range_join/ie_join_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/range_join/ie_join_state.rs @@ -39,8 +39,8 @@ use databend_common_expression::SortColumnDescription; use databend_common_expression::Value; use databend_common_functions::BUILTIN_FUNCTIONS; use databend_common_pipeline_transforms::processors::sort_merge; -use databend_common_sql::executor::physical_plans::RangeJoin; +use crate::physical_plans::RangeJoin; use crate::pipelines::processors::transforms::range_join::filter_block; use crate::pipelines::processors::transforms::range_join::order_match; use crate::pipelines::processors::transforms::range_join::probe_l1; diff --git a/src/query/service/src/pipelines/processors/transforms/range_join/range_join_state.rs b/src/query/service/src/pipelines/processors/transforms/range_join/range_join_state.rs index 1ea06c6a37431..e270c1709154a 100644 --- a/src/query/service/src/pipelines/processors/transforms/range_join/range_join_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/range_join/range_join_state.rs @@ -30,13 +30,13 @@ use databend_common_expression::FunctionContext; use databend_common_expression::RemoteExpr; use databend_common_expression::ScalarRef; use databend_common_functions::BUILTIN_FUNCTIONS; -use databend_common_sql::executor::physical_plans::RangeJoin; -use databend_common_sql::executor::physical_plans::RangeJoinCondition; -use databend_common_sql::executor::physical_plans::RangeJoinType; use databend_common_sql::plans::JoinType; use parking_lot::Mutex; use parking_lot::RwLock; +use crate::physical_plans::RangeJoin; +use crate::physical_plans::RangeJoinCondition; +use crate::physical_plans::RangeJoinType; use crate::pipelines::executor::WatchNotify; use crate::pipelines::processors::transforms::range_join::IEJoinState; use crate::pipelines::processors::transforms::wrap_true_validity; diff --git a/src/query/service/src/pipelines/processors/transforms/transform_async_function.rs b/src/query/service/src/pipelines/processors/transforms/transform_async_function.rs index 29ddb80dd7f3c..0bf2e5f7758db 100644 --- a/src/query/service/src/pipelines/processors/transforms/transform_async_function.rs +++ b/src/query/service/src/pipelines/processors/transforms/transform_async_function.rs @@ -25,11 +25,11 @@ use databend_common_expression::FromData; use databend_common_meta_app::schema::GetSequenceNextValueReq; use databend_common_meta_app::schema::SequenceIdent; use databend_common_pipeline_transforms::processors::AsyncTransform; +use databend_common_sql::binder::AsyncFunctionDesc; use databend_common_storages_fuse::TableContext; use crate::pipelines::processors::transforms::transform_dictionary::DictionaryOperator; use crate::sessions::QueryContext; -use crate::sql::executor::physical_plans::AsyncFunctionDesc; use crate::sql::plans::AsyncFunctionArgument; // Structure to manage sequence numbers in batches diff --git a/src/query/service/src/pipelines/processors/transforms/transform_branched_async_function.rs b/src/query/service/src/pipelines/processors/transforms/transform_branched_async_function.rs index 659a742cca603..e6ee4bffac891 100644 --- a/src/query/service/src/pipelines/processors/transforms/transform_branched_async_function.rs +++ b/src/query/service/src/pipelines/processors/transforms/transform_branched_async_function.rs @@ -20,11 +20,11 @@ use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::DataBlock; use databend_common_expression::SourceSchemaIndex; use databend_common_pipeline_transforms::processors::AsyncTransform; +use databend_common_sql::binder::AsyncFunctionDesc; use crate::pipelines::processors::transforms::transform_async_function::SequenceCounters; use crate::pipelines::processors::transforms::TransformAsyncFunction; use crate::sessions::QueryContext; -use crate::sql::executor::physical_plans::AsyncFunctionDesc; use crate::sql::plans::AsyncFunctionArgument; /// The key of branches is `SourceSchemaIndex`, see `TransformResortAddOnWithoutSourceSchema`. diff --git a/src/query/service/src/pipelines/processors/transforms/transform_dictionary.rs b/src/query/service/src/pipelines/processors/transforms/transform_dictionary.rs index 3afa8d1deb408..78d911201cbce 100644 --- a/src/query/service/src/pipelines/processors/transforms/transform_dictionary.rs +++ b/src/query/service/src/pipelines/processors/transforms/transform_dictionary.rs @@ -39,6 +39,7 @@ use databend_common_expression::DataBlock; use databend_common_expression::Scalar; use databend_common_expression::ScalarRef; use databend_common_expression::Value; +use databend_common_sql::binder::AsyncFunctionDesc; use jiff::tz::TimeZone; use redis::aio::ConnectionManager; use redis::AsyncCommands; @@ -49,7 +50,6 @@ use redis::RedisConnectionInfo; use sqlx::MySqlPool; use crate::pipelines::processors::transforms::TransformAsyncFunction; -use crate::sql::executor::physical_plans::AsyncFunctionDesc; use crate::sql::plans::AsyncFunctionArgument; use crate::sql::plans::DictGetFunctionArgument; use crate::sql::plans::DictionarySource; 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 c7ae2cd8c73fa..5f3b58ca5efc5 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 @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::any::Any; use std::sync::Arc; use databend_common_ast::ast::Engine; @@ -38,8 +39,6 @@ use databend_common_pipeline_core::processors::OutputPort; use databend_common_pipeline_core::processors::ProcessorPtr; use databend_common_pipeline_sources::AsyncSource; use databend_common_pipeline_sources::AsyncSourcer; -use databend_common_sql::executor::physical_plans::UnionAll; -use databend_common_sql::executor::PhysicalPlan; use databend_common_sql::plans::CreateTablePlan; use databend_common_sql::plans::DropTablePlan; use databend_common_sql::IndexType; @@ -49,6 +48,11 @@ use futures_util::TryStreamExt; use crate::interpreters::CreateTableInterpreter; use crate::interpreters::DropTableInterpreter; use crate::interpreters::Interpreter; +use crate::physical_plans::PhysicalPlan; +use crate::physical_plans::PhysicalPlanDynExt; +use crate::physical_plans::PhysicalPlanVisitor; +use crate::physical_plans::RecursiveCteScan; +use crate::physical_plans::UnionAll; use crate::pipelines::executor::ExecutorSettings; use crate::pipelines::executor::PipelinePullingExecutor; use crate::schedulers::build_query_pipeline_without_render_result_set; @@ -150,7 +154,6 @@ impl TransformRecursiveCteSource { impl AsyncSource for TransformRecursiveCteSource { const NAME: &'static str = "TransformRecursiveCteSource"; - #[async_backtrace::framed] async fn generate(&mut self) -> Result> { let mut res = None; let mut data = DataBlock::empty(); @@ -225,144 +228,89 @@ async fn drop_tables(ctx: Arc, table_names: Vec) -> Result Ok(()) } -#[async_recursion::async_recursion(#[recursive::recursive])] async fn create_memory_table_for_cte_scan( ctx: &Arc, plan: &PhysicalPlan, ) -> Result<()> { - match plan { - PhysicalPlan::Filter(plan) => { - create_memory_table_for_cte_scan(ctx, plan.input.as_ref()).await?; - } - PhysicalPlan::EvalScalar(plan) => { - create_memory_table_for_cte_scan(ctx, plan.input.as_ref()).await?; - } - PhysicalPlan::ProjectSet(plan) => { - create_memory_table_for_cte_scan(ctx, plan.input.as_ref()).await?; - } - PhysicalPlan::AggregateExpand(plan) => { - create_memory_table_for_cte_scan(ctx, plan.input.as_ref()).await?; - } - PhysicalPlan::AggregatePartial(plan) => { - create_memory_table_for_cte_scan(ctx, plan.input.as_ref()).await?; - } - PhysicalPlan::AggregateFinal(plan) => { - create_memory_table_for_cte_scan(ctx, plan.input.as_ref()).await?; - } - PhysicalPlan::Window(plan) => { - create_memory_table_for_cte_scan(ctx, plan.input.as_ref()).await?; - } - PhysicalPlan::WindowPartition(plan) => { - create_memory_table_for_cte_scan(ctx, plan.input.as_ref()).await?; - } - PhysicalPlan::Sort(plan) => { - create_memory_table_for_cte_scan(ctx, plan.input.as_ref()).await?; - } - PhysicalPlan::Limit(plan) => { - create_memory_table_for_cte_scan(ctx, plan.input.as_ref()).await?; - } - PhysicalPlan::RowFetch(plan) => { - create_memory_table_for_cte_scan(ctx, plan.input.as_ref()).await?; - } - PhysicalPlan::HashJoin(plan) => { - create_memory_table_for_cte_scan(ctx, plan.build.as_ref()).await?; - create_memory_table_for_cte_scan(ctx, plan.probe.as_ref()).await?; - } - PhysicalPlan::RangeJoin(plan) => { - create_memory_table_for_cte_scan(ctx, plan.left.as_ref()).await?; - create_memory_table_for_cte_scan(ctx, plan.right.as_ref()).await?; - } - PhysicalPlan::Exchange(plan) => { - create_memory_table_for_cte_scan(ctx, plan.input.as_ref()).await?; - } - PhysicalPlan::UnionAll(plan) => { - create_memory_table_for_cte_scan(ctx, plan.left.as_ref()).await?; - create_memory_table_for_cte_scan(ctx, plan.right.as_ref()).await?; - } + struct CollectMemoryTable { + ctx: Arc, + plans: Vec, + } - PhysicalPlan::Udf(plan) => { - create_memory_table_for_cte_scan(ctx, plan.input.as_ref()).await?; + impl CollectMemoryTable { + pub fn create(ctx: Arc) -> Box { + Box::new(CollectMemoryTable { ctx, plans: vec![] }) } - PhysicalPlan::RecursiveCteScan(plan) => { - // Create memory table for cte scan - let table_fields = plan - .output_schema - .fields() - .iter() - .map(|field| { - Ok(TableField::new( - field.name(), - infer_schema_type(field.data_type())?, - )) - }) - .collect::>>()?; - let schema = TableSchemaRefExt::create(table_fields); - let create_table_plan = CreateTablePlan { - create_option: CreateOption::CreateIfNotExists, - tenant: Tenant { - tenant: ctx.get_tenant().tenant, - }, - catalog: ctx.get_current_catalog(), - database: ctx.get_current_database(), - table: plan.table_name.clone(), - schema, - engine: Engine::Memory, - engine_options: Default::default(), - table_properties: Default::default(), - table_partition: None, - storage_params: None, - options: Default::default(), - field_comments: vec![], - cluster_key: None, - as_select: None, - table_indexes: None, - attached_columns: None, - }; - let create_table_interpreter = - CreateTableInterpreter::try_create(ctx.clone(), create_table_plan)?; - let _ = create_table_interpreter.execute(ctx.clone()).await?; + pub fn take(&mut self) -> Vec { + std::mem::take(&mut self.plans) } - PhysicalPlan::Shuffle(plan) => { - create_memory_table_for_cte_scan(ctx, plan.input.as_ref()).await?; + } + + impl PhysicalPlanVisitor for CollectMemoryTable { + fn as_any(&mut self) -> &mut dyn Any { + self } - PhysicalPlan::AsyncFunction(plan) => { - create_memory_table_for_cte_scan(ctx, plan.input.as_ref()).await?; + + fn visit(&mut self, plan: &PhysicalPlan) -> Result<()> { + if let Some(recursive_cte_scan) = plan.downcast_ref::() { + let table_fields = recursive_cte_scan + .output_schema + .fields() + .iter() + .map(|field| { + Ok(TableField::new( + field.name(), + infer_schema_type(field.data_type())?, + )) + }) + .collect::>>()?; + let schema = TableSchemaRefExt::create(table_fields); + + self.plans.push(CreateTablePlan { + schema, + create_option: CreateOption::CreateIfNotExists, + tenant: Tenant { + tenant: self.ctx.get_tenant().tenant, + }, + catalog: self.ctx.get_current_catalog(), + database: self.ctx.get_current_database(), + table: recursive_cte_scan.table_name.clone(), + engine: Engine::Memory, + engine_options: Default::default(), + table_properties: Default::default(), + table_partition: None, + storage_params: None, + options: Default::default(), + field_comments: vec![], + cluster_key: None, + as_select: None, + table_indexes: None, + attached_columns: None, + }); + } + + Ok(()) } - PhysicalPlan::TableScan(_) - | PhysicalPlan::ConstantTableScan(_) - | PhysicalPlan::ExpressionScan(_) - | PhysicalPlan::CacheScan(_) - | PhysicalPlan::DistributedInsertSelect(_) - | PhysicalPlan::ExchangeSource(_) - | PhysicalPlan::ExchangeSink(_) - | PhysicalPlan::CopyIntoTable(_) - | PhysicalPlan::CopyIntoLocation(_) - | PhysicalPlan::ReplaceAsyncSourcer(_) - | PhysicalPlan::ReplaceDeduplicate(_) - | PhysicalPlan::ReplaceInto(_) - | PhysicalPlan::ColumnMutation(_) - | PhysicalPlan::MutationSource(_) - | PhysicalPlan::Mutation(_) - | PhysicalPlan::MutationSplit(_) - | PhysicalPlan::MutationManipulate(_) - | PhysicalPlan::MutationOrganize(_) - | PhysicalPlan::AddStreamColumn(_) - | PhysicalPlan::CompactSource(_) - | PhysicalPlan::CommitSink(_) - | PhysicalPlan::Recluster(_) - | PhysicalPlan::HilbertPartition(_) - | PhysicalPlan::Duplicate(_) - | PhysicalPlan::ChunkFilter(_) - | PhysicalPlan::ChunkEvalScalar(_) - | PhysicalPlan::ChunkCastSchema(_) - | PhysicalPlan::ChunkFillAndReorder(_) - | PhysicalPlan::ChunkAppendData(_) - | PhysicalPlan::ChunkMerge(_) - | PhysicalPlan::ChunkCommitInsert(_) - | PhysicalPlan::BroadcastSource(_) - | PhysicalPlan::BroadcastSink(_) => {} } + + let mut visitor = CollectMemoryTable::create(ctx.clone()); + plan.visit(&mut visitor)?; + + let create_table_plans = { + let visitor = visitor + .as_any() + .downcast_mut::() + .unwrap(); + visitor.take() + }; + + for create_table_plan in create_table_plans { + let create_table_interpreter = + CreateTableInterpreter::try_create(ctx.clone(), create_table_plan)?; + let _ = create_table_interpreter.execute(ctx.clone()).await?; + } + Ok(()) } diff --git a/src/query/service/src/pipelines/processors/transforms/transform_udf_script.rs b/src/query/service/src/pipelines/processors/transforms/transform_udf_script.rs index ce1b08a2bd3a5..7d8780879b951 100644 --- a/src/query/service/src/pipelines/processors/transforms/transform_udf_script.rs +++ b/src/query/service/src/pipelines/processors/transforms/transform_udf_script.rs @@ -36,7 +36,6 @@ use databend_common_expression::DataSchema; use databend_common_expression::FunctionContext; use databend_common_expression::Value; use databend_common_pipeline_transforms::processors::Transform; -use databend_common_sql::executor::physical_plans::UdfFunctionDesc; use databend_common_sql::plans::UDFLanguage; use databend_common_sql::plans::UDFScriptCode; use databend_common_sql::plans::UDFType; @@ -45,6 +44,7 @@ use tempfile::TempDir; use super::runtime_pool::Pool; use super::runtime_pool::RuntimeBuilder; +use crate::physical_plans::UdfFunctionDesc; pub enum ScriptRuntime { JavaScript(JsRuntimePool), @@ -604,10 +604,11 @@ mod venv { use databend_common_cache::LruCache; use databend_common_cache::MemSized; - use databend_common_sql::executor::physical_plans::UdfFunctionDesc; use parking_lot::RwLock; use tempfile::TempDir; + use crate::physical_plans::UdfFunctionDesc; + pub fn install_deps(temp_dir_path: &Path, deps: &[String]) -> Result<(), String> { if deps.is_empty() { return Ok(()); diff --git a/src/query/service/src/pipelines/processors/transforms/transform_udf_server.rs b/src/query/service/src/pipelines/processors/transforms/transform_udf_server.rs index 838942dbc5e6e..23c2b7d3e1391 100644 --- a/src/query/service/src/pipelines/processors/transforms/transform_udf_server.rs +++ b/src/query/service/src/pipelines/processors/transforms/transform_udf_server.rs @@ -29,11 +29,11 @@ use databend_common_expression::DataBlock; use databend_common_metrics::external_server::record_error_external; use databend_common_metrics::external_server::record_retry_external; use databend_common_pipeline_transforms::processors::AsyncTransform; -use databend_common_sql::executor::physical_plans::UdfFunctionDesc; use databend_common_version::UDF_CLIENT_USER_AGENT; use tokio::sync::Semaphore; use tonic::transport::Endpoint; +use crate::physical_plans::UdfFunctionDesc; use crate::sessions::QueryContext; pub struct TransformUdfServer { diff --git a/src/query/service/src/pipelines/processors/transforms/window/partition/window_partition_partial_top_n_exchange.rs b/src/query/service/src/pipelines/processors/transforms/window/partition/window_partition_partial_top_n_exchange.rs index 2a95fd4561035..dc0a482689c92 100644 --- a/src/query/service/src/pipelines/processors/transforms/window/partition/window_partition_partial_top_n_exchange.rs +++ b/src/query/service/src/pipelines/processors/transforms/window/partition/window_partition_partial_top_n_exchange.rs @@ -23,7 +23,7 @@ use databend_common_expression::SortCompare; use databend_common_pipeline_core::processors::Exchange; use super::WindowPartitionMeta; -use crate::sql::executor::physical_plans::WindowPartitionTopNFunc; +use crate::physical_plans::WindowPartitionTopNFunc; pub struct WindowPartitionTopNExchange { partition_indices: Box<[usize]>, diff --git a/src/query/service/src/pipelines/processors/transforms/window/transform_window.rs b/src/query/service/src/pipelines/processors/transforms/window/transform_window.rs index 77b5641f7bdec..f851913b5d7f8 100644 --- a/src/query/service/src/pipelines/processors/transforms/window/transform_window.rs +++ b/src/query/service/src/pipelines/processors/transforms/window/transform_window.rs @@ -38,13 +38,13 @@ use databend_common_pipeline_core::processors::Event; use databend_common_pipeline_core::processors::InputPort; use databend_common_pipeline_core::processors::OutputPort; use databend_common_pipeline_core::processors::Processor; -use databend_common_sql::executor::physical_plans::LagLeadDefault; use databend_common_sql::plans::WindowFuncFrameUnits; use super::frame_bound::FrameBound; use super::window_function::WindowFuncAggImpl; use super::window_function::WindowFunctionImpl; use super::WindowFunctionInfo; +use crate::physical_plans::LagLeadDefault; #[derive(Debug, Clone)] pub struct WindowSortDesc { diff --git a/src/query/service/src/pipelines/processors/transforms/window/window_function.rs b/src/query/service/src/pipelines/processors/transforms/window/window_function.rs index ce0e6013c8fae..b6a898976b2b8 100644 --- a/src/query/service/src/pipelines/processors/transforms/window/window_function.rs +++ b/src/query/service/src/pipelines/processors/transforms/window/window_function.rs @@ -29,10 +29,11 @@ use databend_common_expression::StateAddr; use databend_common_functions::aggregates::AggregateFunction; use databend_common_functions::aggregates::AggregateFunctionFactory; use databend_common_functions::aggregates::AggregateFunctionSortDesc; -use databend_common_sql::executor::physical_plans::LagLeadDefault; -use databend_common_sql::executor::physical_plans::WindowFunction; use itertools::Itertools; +use crate::physical_plans::LagLeadDefault; +use crate::physical_plans::WindowFunction; + #[derive(Clone)] pub enum WindowFunctionInfo { // (func instance, argument offsets) diff --git a/src/query/service/src/schedulers/fragments/fragmenter.rs b/src/query/service/src/schedulers/fragments/fragmenter.rs index 2e7a6e878b819..f104ea02ab636 100644 --- a/src/query/service/src/schedulers/fragments/fragmenter.rs +++ b/src/query/service/src/schedulers/fragments/fragmenter.rs @@ -12,28 +12,32 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::any::Any; +use std::collections::HashMap; use std::sync::Arc; +use databend_common_catalog::cluster_info::Cluster; use databend_common_catalog::table_context::TableContext; use databend_common_exception::Result; use databend_common_meta_types::NodeInfo; -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::Recluster; -use databend_common_sql::executor::physical_plans::ReplaceInto; -use databend_common_sql::executor::physical_plans::TableScan; -use databend_common_sql::executor::physical_plans::UnionAll; -use databend_common_sql::executor::PhysicalPlanReplacer; use crate::clusters::ClusterHelper; +use crate::physical_plans::BroadcastSink; +use crate::physical_plans::CompactSource; +use crate::physical_plans::ConstantTableScan; +use crate::physical_plans::DeriveHandle; +use crate::physical_plans::Exchange; +use crate::physical_plans::ExchangeSink; +use crate::physical_plans::ExchangeSource; +use crate::physical_plans::MutationSource; +use crate::physical_plans::PhysicalPlan; +use crate::physical_plans::PhysicalPlanDynExt; +use crate::physical_plans::PhysicalPlanMeta; +use crate::physical_plans::PhysicalPlanVisitor; +use crate::physical_plans::Recluster; +use crate::physical_plans::ReplaceInto; +use crate::physical_plans::TableScan; use crate::schedulers::fragments::plan_fragment::FragmentType; use crate::schedulers::PlanFragment; use crate::servers::flight::v1::exchange::BroadcastExchange; @@ -41,15 +45,12 @@ use crate::servers::flight::v1::exchange::DataExchange; use crate::servers::flight::v1::exchange::MergeExchange; use crate::servers::flight::v1::exchange::ShuffleDataExchange; use crate::sessions::QueryContext; -use crate::sql::executor::physical_plans::Mutation; -use crate::sql::executor::PhysicalPlan; /// Visitor to split a `PhysicalPlan` into fragments. pub struct Fragmenter { ctx: Arc, - fragments: Vec, query_id: String, - state: State, + fragments: Vec, } /// A state to track if is visiting a source fragment, useful when building fragments. @@ -76,7 +77,6 @@ impl Fragmenter { Ok(Self { ctx, fragments: vec![], - state: State::Other, query_id, }) } @@ -97,248 +97,234 @@ impl Fragmenter { ctx.get_cluster().local_id() } - pub fn get_exchange( - ctx: Arc, - plan: &PhysicalPlan, - ) -> Result> { - match plan { - PhysicalPlan::ExchangeSink(plan) => match plan.kind { - FragmentKind::Normal => Ok(Some(ShuffleDataExchange::create( - Self::get_executors(ctx), - plan.keys.clone(), - ))), - FragmentKind::Merge => Ok(Some(MergeExchange::create( - Self::get_local_executor(ctx), - plan.ignore_exchange, - plan.allow_adjust_parallelism, - ))), - FragmentKind::Expansive => { - Ok(Some(BroadcastExchange::create(Self::get_executors(ctx)))) - } - _ => Ok(None), - }, - _ => Ok(None), + pub fn build_fragment(self, plan: &PhysicalPlan) -> Result> { + let mut handle = FragmentDeriveHandle::create(self.query_id.clone(), self.ctx.clone()); + let root = plan.derive_with(&mut handle); + let mut fragments = { + let handle = handle + .as_any() + .downcast_mut::() + .unwrap(); + handle.take_fragments() + }; + + let mut fragment_type = FragmentType::Root; + if let Some(_broadcast_sink) = plan.downcast_ref::() { + fragment_type = FragmentType::Intermediate; } - } - pub fn build_fragment(mut self, plan: &PhysicalPlan) -> Result { - let root = self.replace(plan)?; - let fragment_type = match plan { - PhysicalPlan::BroadcastSink(_) => FragmentType::Intermediate, - _ => FragmentType::Root, - }; - let mut root_fragment = PlanFragment { + fragments.insert(self.ctx.get_fragment_id(), PlanFragment { plan: root, fragment_type, fragment_id: self.ctx.get_fragment_id(), exchange: None, query_id: self.query_id.clone(), source_fragments: self.fragments, - }; - Self::resolve_fragment_connection(&mut root_fragment); + }); - Ok(root_fragment) - } + let edges = Self::collect_fragments_edge(fragments.values()); - fn resolve_fragment_connection(fragment: &mut PlanFragment) { - for source_fragment in fragment.source_fragments.iter_mut() { - if let PhysicalPlan::ExchangeSink(ExchangeSink { - destination_fragment_id, - .. - }) = &mut source_fragment.plan - { - // Fill the destination_fragment_id with parent fragment id. - *destination_fragment_id = fragment.fragment_id; + for (source, target) in edges { + let Some(fragment) = fragments.get_mut(&source) else { + continue; + }; + + if let Some(exchange_sink) = fragment.plan.downcast_mut_ref::() { + exchange_sink.destination_fragment_id = target; } } - } -} -impl PhysicalPlanReplacer for Fragmenter { - fn replace_table_scan(&mut self, plan: &TableScan) -> Result { - self.state = State::SelectLeaf; - Ok(PhysicalPlan::TableScan(plan.clone())) + Ok(fragments.into_values().collect::>()) } - fn replace_constant_table_scan(&mut self, plan: &ConstantTableScan) -> Result { - self.state = State::SelectLeaf; - Ok(PhysicalPlan::ConstantTableScan(plan.clone())) - } - - fn replace_mutation_source(&mut self, plan: &MutationSource) -> Result { - self.state = State::MutationSource; - Ok(PhysicalPlan::MutationSource(plan.clone())) - } - - fn replace_mutation(&mut self, plan: &Mutation) -> Result { - let input = self.replace(&plan.input)?; - Ok(PhysicalPlan::Mutation(Box::new(Mutation { - input: Box::new(input), - ..plan.clone() - }))) - } - - fn replace_replace_into(&mut self, plan: &ReplaceInto) -> Result { - let input = self.replace(&plan.input)?; - self.state = State::ReplaceInto; - Ok(PhysicalPlan::ReplaceInto(Box::new(ReplaceInto { - input: Box::new(input), - ..plan.clone() - }))) - } + fn collect_fragments_edge<'a>( + iter: impl Iterator, + ) -> HashMap { + struct EdgeVisitor { + target_fragment_id: usize, + map: HashMap, + } - // 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()))) + impl EdgeVisitor { + pub fn create(target_fragment_id: usize) -> Box { + Box::new(EdgeVisitor { + target_fragment_id, + map: Default::default(), + }) } - 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() - }))) + + pub fn take(&mut self) -> HashMap { + std::mem::take(&mut self.map) } } - } - fn replace_recluster(&mut self, plan: &Recluster) -> Result { - self.state = State::Recluster; - Ok(PhysicalPlan::Recluster(Box::new(plan.clone()))) - } - - fn replace_compact_source(&mut self, plan: &CompactSource) -> Result { - self.state = State::Compact; - Ok(PhysicalPlan::CompactSource(Box::new(plan.clone()))) - } - - fn replace_hash_join(&mut self, plan: &HashJoin) -> Result { - let mut fragments = vec![]; - let build_input = self.replace(plan.build.as_ref())?; - - // Consume current fragments to prevent them being consumed by `probe_input`. - fragments.append(&mut self.fragments); - let probe_input = self.replace(plan.probe.as_ref())?; - fragments.append(&mut self.fragments); - - self.fragments = fragments; - - Ok(PhysicalPlan::HashJoin(HashJoin { - plan_id: plan.plan_id, - projections: plan.projections.clone(), - probe_projections: plan.probe_projections.clone(), - build_projections: plan.build_projections.clone(), - build: Box::new(build_input), - probe: Box::new(probe_input), - build_keys: plan.build_keys.clone(), - probe_keys: plan.probe_keys.clone(), - is_null_equal: plan.is_null_equal.clone(), - non_equi_conditions: plan.non_equi_conditions.clone(), - join_type: plan.join_type.clone(), - marker_index: plan.marker_index, - from_correlated_subquery: plan.from_correlated_subquery, - probe_to_build: plan.probe_to_build.clone(), - output_schema: plan.output_schema.clone(), - need_hold_hash_table: plan.need_hold_hash_table, - stat_info: plan.stat_info.clone(), - single_to_inner: plan.single_to_inner.clone(), - build_side_cache_info: plan.build_side_cache_info.clone(), - runtime_filter: plan.runtime_filter.clone(), - broadcast_id: plan.broadcast_id, - })) - } - - fn replace_union(&mut self, plan: &UnionAll) -> Result { - let mut fragments = vec![]; - let left_input = self.replace(plan.left.as_ref())?; - let left_state = self.state.clone(); + impl PhysicalPlanVisitor for EdgeVisitor { + fn as_any(&mut self) -> &mut dyn Any { + self + } - // Consume current fragments to prevent them being consumed by `right_input`. - fragments.append(&mut self.fragments); - let right_input = self.replace(plan.right.as_ref())?; - let right_state = self.state.clone(); + fn visit(&mut self, plan: &PhysicalPlan) -> Result<()> { + if let Some(v) = plan.downcast_ref::() { + if let Some(v) = self + .map + .insert(v.source_fragment_id, self.target_fragment_id) + { + assert_eq!(v, self.target_fragment_id); + } + } - fragments.append(&mut self.fragments); - self.fragments = fragments; + Ok(()) + } + } - // If any of the input is a source fragment, the union all is a source fragment. - if left_state == State::SelectLeaf || right_state == State::SelectLeaf { - self.state = State::SelectLeaf; - } else { - self.state = State::Other; + let mut edges = HashMap::new(); + for fragment in iter { + let mut visitor = EdgeVisitor::create(fragment.fragment_id); + fragment.plan.visit(&mut visitor).unwrap(); + if let Some(v) = visitor.as_any().downcast_mut::() { + edges.extend(v.take().into_iter()) + } } - Ok(PhysicalPlan::UnionAll(UnionAll { - left: Box::new(left_input), - right: Box::new(right_input), - ..plan.clone() - })) + edges } +} - fn replace_exchange(&mut self, plan: &Exchange) -> Result { - // Recursively rewrite input - let input = self.replace(plan.input.as_ref())?; - let input_schema = input.output_schema()?; - - let plan_id = plan.plan_id; - - let source_fragment_id = self.ctx.get_fragment_id(); - let plan = PhysicalPlan::ExchangeSink(ExchangeSink { - // TODO(leiysky): we reuse the plan id here, - // should generate a new one for the sink. - plan_id, +struct FragmentDeriveHandle { + state: State, + query_id: String, + ctx: Arc, + fragments: HashMap, +} - input: Box::new(input), - schema: input_schema.clone(), - kind: plan.kind.clone(), - keys: plan.keys.clone(), +impl FragmentDeriveHandle { + pub fn create(query_id: String, ctx: Arc) -> Box { + Box::new(FragmentDeriveHandle { + ctx, + query_id, + state: State::Other, + fragments: HashMap::new(), + }) + } - query_id: self.query_id.clone(), + pub fn take_fragments(&mut self) -> HashMap { + std::mem::take(&mut self.fragments) + } - // We will connect the fragments later, so we just - // set the fragment id to a invalid value here. - destination_fragment_id: usize::MAX, - ignore_exchange: plan.ignore_exchange, - allow_adjust_parallelism: plan.allow_adjust_parallelism, - }); - let fragment_type = match self.state { - State::SelectLeaf => FragmentType::Source, - State::MutationSource => FragmentType::MutationSource, - State::Other => FragmentType::Intermediate, - State::ReplaceInto => FragmentType::ReplaceInto, - State::Compact => FragmentType::Compact, - State::Recluster => FragmentType::Recluster, + pub fn get_exchange( + cluster: Arc, + plan: &PhysicalPlan, + ) -> Result> { + let Some(exchange_sink) = plan.downcast_ref::() else { + return Ok(None); }; - self.state = State::Other; - let exchange = Self::get_exchange(self.ctx.clone(), &plan)?; - let mut source_fragment = PlanFragment { - plan, - fragment_type, + let get_executors = |cluster: Arc| { + let cluster_nodes = cluster.get_nodes(); - fragment_id: source_fragment_id, - exchange, - query_id: self.query_id.clone(), - - source_fragments: self.fragments.drain(..).collect(), + cluster_nodes + .iter() + .map(|node| &node.id) + .cloned() + .collect::>() }; - // Fill the destination_fragment_id for source fragments of `source_fragment`. - Self::resolve_fragment_connection(&mut source_fragment); + Ok(match exchange_sink.kind { + FragmentKind::Normal => Some(ShuffleDataExchange::create( + get_executors(cluster), + exchange_sink.keys.clone(), + )), + FragmentKind::Merge => Some(MergeExchange::create( + cluster.local_id(), + exchange_sink.ignore_exchange, + exchange_sink.allow_adjust_parallelism, + )), + FragmentKind::Expansive => Some(BroadcastExchange::create(get_executors(cluster))), + FragmentKind::Init => None, + }) + } +} + +impl DeriveHandle for FragmentDeriveHandle { + fn as_any(&mut self) -> &mut dyn Any { + self + } - self.fragments.push(source_fragment); + fn derive( + &mut self, + v: &PhysicalPlan, + mut children: Vec, + ) -> std::result::Result> { + if let Some(_recluster) = v.downcast_ref::() { + self.state = State::Recluster; + } else if let Some(_table_scan) = v.downcast_ref::() { + self.state = State::SelectLeaf; + } else if let Some(_const_table_scan) = v.downcast_ref::() { + self.state = State::SelectLeaf; + } else if let Some(_compact_source) = v.downcast_ref::() { + self.state = State::Compact; + } else if let Some(_replace_into) = v.downcast_ref::() { + self.state = State::ReplaceInto; + } else if let Some(_mutation_source) = v.downcast_ref::() { + self.state = State::MutationSource; + } - Ok(PhysicalPlan::ExchangeSource(ExchangeSource { - // TODO(leiysky): we reuse the plan id here, - // should generate a new one for the source. - plan_id, + if let Some(exchange) = v.downcast_ref::() { + let input = children.remove(0); + let input_schema = input.output_schema().unwrap(); + + let plan_id = v.get_id(); + let source_fragment_id = self.ctx.get_fragment_id(); + + let plan: PhysicalPlan = Box::new(ExchangeSink { + input, + schema: input_schema.clone(), + kind: exchange.kind.clone(), + keys: exchange.keys.clone(), + + query_id: self.query_id.clone(), + + // We will connect the fragments later, so we just + // set the fragment id to a invalid value here. + destination_fragment_id: usize::MAX, + ignore_exchange: exchange.ignore_exchange, + allow_adjust_parallelism: exchange.allow_adjust_parallelism, + meta: PhysicalPlanMeta::with_plan_id("ExchangeSink", plan_id), + }); + + let fragment_type = match self.state { + State::SelectLeaf => FragmentType::Source, + State::MutationSource => FragmentType::MutationSource, + State::Other => FragmentType::Intermediate, + State::ReplaceInto => FragmentType::ReplaceInto, + State::Compact => FragmentType::Compact, + State::Recluster => FragmentType::Recluster, + }; - schema: input_schema, - query_id: self.query_id.clone(), + self.state = State::Other; + let cluster = self.ctx.get_cluster(); + let exchange = Self::get_exchange(cluster, &plan).unwrap(); + + let source_fragment = PlanFragment { + plan, + exchange, + fragment_type, + source_fragments: vec![], + fragment_id: source_fragment_id, + query_id: self.query_id.clone(), + }; + + self.fragments.insert(source_fragment_id, source_fragment); + + return Ok(Box::new(ExchangeSource { + schema: input_schema, + query_id: self.query_id.clone(), + + source_fragment_id, + meta: PhysicalPlanMeta::with_plan_id("ExchangeSource", plan_id), + })); + } - source_fragment_id, - })) + Err(children) } } diff --git a/src/query/service/src/schedulers/fragments/plan_fragment.rs b/src/query/service/src/schedulers/fragments/plan_fragment.rs index 18f2b35267eb4..9f3e4032a6c5f 100644 --- a/src/query/service/src/schedulers/fragments/plan_fragment.rs +++ b/src/query/service/src/schedulers/fragments/plan_fragment.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::any::Any; use std::collections::HashMap; use std::sync::Arc; @@ -24,27 +25,29 @@ use databend_common_expression::BlockEntry; use databend_common_expression::Column; use databend_common_expression::DataBlock; 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::Recluster; -use databend_common_sql::executor::physical_plans::ReplaceDeduplicate; -use databend_common_sql::executor::physical_plans::ReplaceInto; -use databend_common_sql::executor::physical_plans::TableScan; use databend_common_storages_fuse::TableContext; use databend_storages_common_table_meta::meta::BlockSlotDescription; use databend_storages_common_table_meta::meta::Location; +use crate::physical_plans::CompactSource; +use crate::physical_plans::ConstantTableScan; +use crate::physical_plans::DeriveHandle; +use crate::physical_plans::ExchangeSink; +use crate::physical_plans::IPhysicalPlan; +use crate::physical_plans::MutationSource; +use crate::physical_plans::PhysicalPlan; +use crate::physical_plans::PhysicalPlanDynExt; +use crate::physical_plans::PhysicalPlanVisitor; +use crate::physical_plans::Recluster; +use crate::physical_plans::ReplaceDeduplicate; +use crate::physical_plans::ReplaceInto; +use crate::physical_plans::TableScan; use crate::schedulers::Fragmenter; use crate::schedulers::QueryFragmentAction; use crate::schedulers::QueryFragmentActions; use crate::schedulers::QueryFragmentsActions; use crate::servers::flight::v1::exchange::DataExchange; use crate::sessions::QueryContext; -use crate::sql::executor::PhysicalPlan; -use crate::sql::executor::PhysicalPlanReplacer; /// Type of plan fragment #[derive(Clone, Debug, PartialEq, Eq)] @@ -211,13 +214,11 @@ impl PlanFragment { } for (executor, sources) in executor_partitions { - let mut plan = self.plan.clone(); // Replace `ReadDataSourcePlan` with rewritten one and generate new fragment for it. - let mut replace_read_source = ReplaceReadSource { sources }; - plan = replace_read_source.replace(&plan)?; + let mut handle = ReadSourceDeriveHandle::create(sources); + let plan = self.plan.derive_with(&mut handle); - fragment_actions - .add_action(QueryFragmentAction::create(executor.clone(), plan.clone())); + fragment_actions.add_action(QueryFragmentAction::create(executor.clone(), plan)); } Ok(()) @@ -228,12 +229,11 @@ impl PlanFragment { ctx: Arc, fragment_actions: &mut QueryFragmentActions, ) -> Result<()> { - let plan = match &self.plan { - PhysicalPlan::ExchangeSink(plan) => plan, - _ => unreachable!("logic error"), + let Some(plan) = self.plan.downcast_ref::() else { + unreachable!("logic error"); }; - let plan = PhysicalPlan::ExchangeSink(plan.clone()); + let plan: PhysicalPlan = Box::new(plan.clone()); let mutation_source = plan.try_find_mutation_source().unwrap(); let partitions: &Partitions = &mutation_source.partitions; @@ -242,11 +242,8 @@ impl PlanFragment { let partition_reshuffle = partitions.reshuffle(executors)?; for (executor, parts) in partition_reshuffle.into_iter() { - let mut plan = self.plan.clone(); - - let mut replace_mutation_source = ReplaceMutationSource { partitions: parts }; - plan = replace_mutation_source.replace(&plan)?; - + let mut handle = MutationSourceDeriveHandle::create(parts); + let plan = self.plan.derive_with(&mut handle); fragment_actions.add_action(QueryFragmentAction::create(executor, plan)); } @@ -258,15 +255,44 @@ impl PlanFragment { ctx: Arc, fragment_actions: &mut QueryFragmentActions, ) -> Result<()> { - let plan = match &self.plan { - PhysicalPlan::ExchangeSink(plan) => plan, - _ => unreachable!("logic error"), - }; - let plan = match plan.input.as_ref() { - PhysicalPlan::ReplaceInto(plan) => plan, - _ => unreachable!("logic error"), - }; - let partitions = &plan.segments; + struct PartitionsCollector { + partitions: Vec<(usize, Location)>, + } + + impl PartitionsCollector { + pub fn create() -> Box { + Box::new(PartitionsCollector { partitions: vec![] }) + } + + pub fn take(&mut self) -> Vec<(usize, Location)> { + std::mem::take(&mut self.partitions) + } + } + + impl PhysicalPlanVisitor for PartitionsCollector { + fn as_any(&mut self) -> &mut dyn Any { + self + } + + fn visit(&mut self, plan: &PhysicalPlan) -> Result<()> { + if let Some(v) = plan.downcast_ref::() { + assert!(self.partitions.is_empty()); + self.partitions = v.segments.clone(); + } + + Ok(()) + } + } + + let mut visitor = PartitionsCollector::create(); + self.plan.visit(&mut visitor)?; + + let mut partitions = vec![]; + + if let Some(v) = visitor.as_any().downcast_mut::() { + partitions = v.take(); + } + let executors = Fragmenter::get_executors(ctx.clone()); let local_id = ctx.get_cluster().local_id.clone(); match ctx.get_settings().get_replace_into_shuffle_strategy()? { @@ -276,13 +302,8 @@ impl PlanFragment { let mut plan = self.plan.clone(); let need_insert = executor == local_id; - let mut replace_replace_into = ReplaceReplaceInto { - partitions: parts, - slot: None, - need_insert, - }; - plan = replace_replace_into.replace(&plan)?; - + let mut handle = ReplaceDeriveHandle::create(parts, None, need_insert); + plan = plan.derive_with(&mut handle); fragment_actions.add_action(QueryFragmentAction::create(executor, plan)); } } @@ -291,17 +312,17 @@ impl PlanFragment { // assign all the segment locations to each one of the executors, // but for each segment, one executor only need to take part of the blocks for (executor_idx, executor) in executors.into_iter().enumerate() { - let mut plan = self.plan.clone(); let need_insert = executor == local_id; - let mut replace_replace_into = ReplaceReplaceInto { - partitions: partitions.clone(), - slot: Some(BlockSlotDescription { + let mut handle = ReplaceDeriveHandle::create( + partitions.clone(), + Some(BlockSlotDescription { num_slots, slot: executor_idx as u32, }), need_insert, - }; - plan = replace_replace_into.replace(&plan)?; + ); + + let plan = self.plan.derive_with(&mut handle); fragment_actions.add_action(QueryFragmentAction::create(executor, plan)); } @@ -315,26 +336,50 @@ impl PlanFragment { ctx: Arc, fragment_actions: &mut QueryFragmentActions, ) -> Result<()> { - let exchange_sink = match &self.plan { - PhysicalPlan::ExchangeSink(plan) => plan, - _ => unreachable!("logic error"), - }; - let compact_block = match exchange_sink.input.as_ref() { - PhysicalPlan::CompactSource(plan) => plan, - _ => unreachable!("logic error"), - }; + struct SourceCollector { + partitions: Option, + } + + impl SourceCollector { + pub fn create() -> Box { + Box::new(SourceCollector { partitions: None }) + } - let partitions: &Partitions = &compact_block.parts; + pub fn take(&mut self) -> Partitions { + self.partitions.take().unwrap() + } + } + + impl PhysicalPlanVisitor for SourceCollector { + fn as_any(&mut self) -> &mut dyn Any { + self + } + + fn visit(&mut self, plan: &PhysicalPlan) -> Result<()> { + if let Some(v) = plan.downcast_ref::() { + assert!(self.partitions.is_none()); + self.partitions = Some(v.parts.clone()); + } + + Ok(()) + } + } + + let mut visitor = SourceCollector::create(); + self.plan.visit(&mut visitor)?; + + let partitions = visitor + .as_any() + .downcast_mut::() + .unwrap() + .take(); let executors = Fragmenter::get_executors_nodes(ctx); let partition_reshuffle = partitions.reshuffle(executors)?; for (executor, parts) in partition_reshuffle.into_iter() { - let mut plan = self.plan.clone(); - - let mut replace_compact_source = ReplaceCompactBlock { partitions: parts }; - plan = replace_compact_source.replace(&plan)?; - + let mut handle = CompactSourceDeriveHandle::create(parts); + let plan = self.plan.derive_with(&mut handle); fragment_actions.add_action(QueryFragmentAction::create(executor, plan)); } @@ -346,16 +391,46 @@ impl PlanFragment { ctx: Arc, fragment_actions: &mut QueryFragmentActions, ) -> Result<()> { - let exchange_sink = match &self.plan { - PhysicalPlan::ExchangeSink(plan) => plan, - _ => unreachable!("logic error"), - }; - let recluster = match exchange_sink.input.as_ref() { - PhysicalPlan::Recluster(plan) => plan, - _ => unreachable!("logic error"), - }; + struct TasksCollector { + tasks: Vec, + } + + impl TasksCollector { + pub fn create() -> Box { + Box::new(TasksCollector { tasks: Vec::new() }) + } + + pub fn take(&mut self) -> Vec { + std::mem::take(&mut self.tasks) + } + } + + impl PhysicalPlanVisitor for TasksCollector { + fn as_any(&mut self) -> &mut dyn Any { + self + } + + fn visit(&mut self, plan: &PhysicalPlan) -> Result<()> { + if let Some(recluster) = plan.downcast_ref::() { + if !self.tasks.is_empty() { + unreachable!("logic error, expect only one recluster"); + } + + self.tasks = recluster.tasks.clone() + } + + Ok(()) + } + } + + let mut visitor = TasksCollector::create(); + self.plan.visit(&mut visitor)?; + + let mut tasks = vec![]; + if let Some(visitor) = visitor.as_any().downcast_mut::() { + tasks = visitor.take(); + } - let tasks = recluster.tasks.clone(); let executors = Fragmenter::get_executors(ctx); if tasks.len() > executors.len() { return Err(ErrorCode::Internal(format!( @@ -367,9 +442,8 @@ impl PlanFragment { let task_reshuffle = Self::reshuffle(executors, tasks)?; for (executor, tasks) in task_reshuffle.into_iter() { - let mut plan = self.plan.clone(); - let mut replace_recluster = ReplaceRecluster { tasks }; - plan = replace_recluster.replace(&plan)?; + let mut handle = ReclusterDeriveHandle::create(tasks); + let plan = self.plan.derive_with(&mut handle); fragment_actions.add_action(QueryFragmentAction::create(executor, plan)); } @@ -423,30 +497,53 @@ impl PlanFragment { )); } - let mut data_sources = HashMap::new(); - - let mut collect_data_source = |plan: &PhysicalPlan| { - if let PhysicalPlan::TableScan(scan) = plan { - data_sources.insert(scan.plan_id, DataSource::Table(*scan.source.clone())); - } else if let PhysicalPlan::ConstantTableScan(scan) = plan { - data_sources.insert( - scan.plan_id, - DataSource::ConstTable(ConstTableColumn { - columns: scan.values.clone(), - num_rows: scan.num_rows, - }), - ); + struct DataSourceVisitor { + data_sources: HashMap, + } + + impl DataSourceVisitor { + pub fn create() -> Box { + Box::new(DataSourceVisitor { + data_sources: HashMap::new(), + }) + } + + pub fn take(&mut self) -> HashMap { + std::mem::take(&mut self.data_sources) + } + } + + impl PhysicalPlanVisitor for DataSourceVisitor { + fn as_any(&mut self) -> &mut dyn Any { + self } - }; - PhysicalPlan::traverse( - &self.plan, - &mut |_| true, - &mut collect_data_source, - &mut |_| {}, - ); + fn visit(&mut self, plan: &PhysicalPlan) -> Result<()> { + if let Some(scan) = plan.downcast_ref::() { + self.data_sources + .insert(plan.get_id(), DataSource::Table(*scan.source.clone())); + } else if let Some(scan) = plan.downcast_ref::() { + self.data_sources.insert( + plan.get_id(), + DataSource::ConstTable(ConstTableColumn { + columns: scan.values.clone(), + num_rows: scan.num_rows, + }), + ); + } - Ok(data_sources) + Ok(()) + } + } + + let mut visitor = DataSourceVisitor::create(); + self.plan.visit(&mut visitor)?; + + let data_sources = visitor + .as_any() + .downcast_mut::() + .unwrap(); + Ok(data_sources.take()) } } @@ -487,137 +584,208 @@ impl TryFrom for ConstTableColumn { } } -struct ReplaceReadSource { +struct ReadSourceDeriveHandle { sources: HashMap, } -impl PhysicalPlanReplacer for ReplaceReadSource { - fn replace_table_scan(&mut self, plan: &TableScan) -> Result { - let source = self.sources.remove(&plan.plan_id).ok_or_else(|| { - ErrorCode::Internal(format!( - "Cannot find data source for table scan plan {}", - plan.plan_id - )) - })?; - - let source = DataSourcePlan::try_from(source)?; - - Ok(PhysicalPlan::TableScan(TableScan { - plan_id: plan.plan_id, - scan_id: plan.scan_id, - source: Box::new(source), - name_mapping: plan.name_mapping.clone(), - table_index: plan.table_index, - stat_info: plan.stat_info.clone(), - internal_column: plan.internal_column.clone(), - })) +impl ReadSourceDeriveHandle { + pub fn create(sources: HashMap) -> Box { + Box::new(ReadSourceDeriveHandle { sources }) } +} - fn replace_constant_table_scan(&mut self, plan: &ConstantTableScan) -> Result { - let source = self.sources.remove(&plan.plan_id).ok_or_else(|| { - ErrorCode::Internal(format!( - "Cannot find data source for constant table scan plan {}", - plan.plan_id - )) - })?; - - let const_table_columns = ConstTableColumn::try_from(source)?; - - Ok(PhysicalPlan::ConstantTableScan(ConstantTableScan { - plan_id: plan.plan_id, - values: const_table_columns.columns, - num_rows: const_table_columns.num_rows, - output_schema: plan.output_schema.clone(), - })) +impl DeriveHandle for ReadSourceDeriveHandle { + fn as_any(&mut self) -> &mut dyn Any { + self } - 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 derive( + &mut self, + v: &PhysicalPlan, + children: Vec, + ) -> std::result::Result> { + if let Some(table_scan) = v.downcast_ref::() { + let Some(source) = self.sources.remove(&table_scan.get_id()) else { + unreachable!( + "Cannot find data source for table scan plan {}", + table_scan.get_id() + ) + }; + + let Ok(source) = DataSourcePlan::try_from(source) else { + unreachable!("Cannot create data source plan"); + }; + + return Ok(Box::new(TableScan { + source: Box::new(source), + ..table_scan.clone() + })); + } else if let Some(table_scan) = v.downcast_ref::() { + let Some(source) = self.sources.remove(&table_scan.get_id()) else { + unreachable!( + "Cannot find data source for constant table scan plan {}", + table_scan.get_id() + ) + }; + + let Ok(const_table_columns) = ConstTableColumn::try_from(source) else { + unreachable!("Cannot convert Table to Vec") + }; + + return Ok(Box::new(ConstantTableScan { + values: const_table_columns.columns, + num_rows: const_table_columns.num_rows, + ..table_scan.clone() + })); } + + Err(children) } } -struct ReplaceRecluster { - pub tasks: Vec, +struct ReclusterDeriveHandle { + tasks: Vec, +} + +impl ReclusterDeriveHandle { + pub fn create(tasks: Vec) -> Box { + Box::new(ReclusterDeriveHandle { tasks }) + } } -impl PhysicalPlanReplacer for ReplaceRecluster { - fn replace_recluster(&mut self, plan: &Recluster) -> Result { - Ok(PhysicalPlan::Recluster(Box::new(Recluster { +impl DeriveHandle for ReclusterDeriveHandle { + fn as_any(&mut self) -> &mut dyn Any { + self + } + + fn derive( + &mut self, + v: &PhysicalPlan, + children: Vec, + ) -> std::result::Result> { + let Some(recluster) = v.downcast_ref::() else { + return Err(children); + }; + + Ok(Box::new(Recluster { tasks: self.tasks.clone(), - ..plan.clone() - }))) + ..recluster.clone() + })) } } -struct ReplaceMutationSource { - pub partitions: Partitions, +struct MutationSourceDeriveHandle { + partitions: Partitions, +} + +impl MutationSourceDeriveHandle { + pub fn create(partitions: Partitions) -> Box { + Box::new(MutationSourceDeriveHandle { partitions }) + } } -impl PhysicalPlanReplacer for ReplaceMutationSource { - fn replace_mutation_source(&mut self, plan: &MutationSource) -> Result { - Ok(PhysicalPlan::MutationSource(MutationSource { +impl DeriveHandle for MutationSourceDeriveHandle { + fn as_any(&mut self) -> &mut dyn Any { + self + } + + fn derive( + &mut self, + v: &PhysicalPlan, + children: Vec, + ) -> std::result::Result> { + let Some(mutation_source) = v.downcast_ref::() else { + return Err(children); + }; + + Ok(Box::new(MutationSource { partitions: self.partitions.clone(), - ..plan.clone() + ..mutation_source.clone() })) } } -struct ReplaceCompactBlock { +struct CompactSourceDeriveHandle { pub partitions: Partitions, } -impl PhysicalPlanReplacer for ReplaceCompactBlock { - fn replace_compact_source(&mut self, plan: &CompactSource) -> Result { - Ok(PhysicalPlan::CompactSource(Box::new(CompactSource { +impl CompactSourceDeriveHandle { + pub fn create(partitions: Partitions) -> Box { + Box::new(CompactSourceDeriveHandle { partitions }) + } +} + +impl DeriveHandle for CompactSourceDeriveHandle { + fn as_any(&mut self) -> &mut dyn Any { + self + } + + fn derive( + &mut self, + v: &PhysicalPlan, + children: Vec, + ) -> std::result::Result> { + let Some(compact_source) = v.downcast_ref::() else { + return Err(children); + }; + + Ok(Box::new(CompactSource { parts: self.partitions.clone(), - ..plan.clone() - }))) + ..compact_source.clone() + })) } } -struct ReplaceReplaceInto { +struct ReplaceDeriveHandle { pub partitions: Vec<(usize, Location)>, // for standalone mode, slot is None pub slot: Option, pub need_insert: bool, } -impl PhysicalPlanReplacer for ReplaceReplaceInto { - fn replace_replace_into(&mut self, plan: &ReplaceInto) -> Result { - let input = self.replace(&plan.input)?; - Ok(PhysicalPlan::ReplaceInto(Box::new(ReplaceInto { - input: Box::new(input), - need_insert: self.need_insert, - segments: self.partitions.clone(), - block_slots: self.slot.clone(), - ..plan.clone() - }))) +impl ReplaceDeriveHandle { + pub fn create( + partitions: Vec<(usize, Location)>, + slot: Option, + need_insert: bool, + ) -> Box { + Box::new(ReplaceDeriveHandle { + partitions, + slot, + need_insert, + }) + } +} + +impl DeriveHandle for ReplaceDeriveHandle { + fn as_any(&mut self) -> &mut dyn Any { + self } - fn replace_deduplicate(&mut self, plan: &ReplaceDeduplicate) -> Result { - let input = self.replace(&plan.input)?; - Ok(PhysicalPlan::ReplaceDeduplicate(Box::new( - ReplaceDeduplicate { - input: Box::new(input), + fn derive( + &mut self, + v: &PhysicalPlan, + mut children: Vec, + ) -> std::result::Result> { + if let Some(replace_into) = v.downcast_ref::() { + assert_eq!(children.len(), 1); + return Ok(Box::new(ReplaceInto { + input: children.remove(0), + need_insert: self.need_insert, + segments: self.partitions.clone(), + block_slots: self.slot.clone(), + ..replace_into.clone() + })); + } else if let Some(replace_deduplicate) = v.downcast_ref::() { + assert_eq!(children.len(), 1); + return Ok(Box::new(ReplaceDeduplicate { + input: children.remove(0), need_insert: self.need_insert, table_is_empty: self.partitions.is_empty(), - ..plan.clone() - }, - ))) + ..replace_deduplicate.clone() + })); + } + + Err(children) } } diff --git a/src/query/service/src/schedulers/fragments/query_fragment_actions.rs b/src/query/service/src/schedulers/fragments/query_fragment_actions.rs index ea2dccf3d2379..06e3cad91fa40 100644 --- a/src/query/service/src/schedulers/fragments/query_fragment_actions.rs +++ b/src/query/service/src/schedulers/fragments/query_fragment_actions.rs @@ -26,6 +26,9 @@ use databend_common_expression::DataSchemaRef; use databend_common_meta_types::NodeInfo; use crate::clusters::ClusterHelper; +use crate::physical_plans::ExchangeSink; +use crate::physical_plans::PhysicalPlan; +use crate::physical_plans::PhysicalPlanDynExt; use crate::servers::flight::v1::exchange::DataExchange; use crate::servers::flight::v1::packets::DataflowDiagramBuilder; use crate::servers::flight::v1::packets::QueryEnv; @@ -33,20 +36,19 @@ use crate::servers::flight::v1::packets::QueryFragment; use crate::servers::flight::v1::packets::QueryFragments; use crate::sessions::QueryContext; use crate::sessions::TableContext; -use crate::sql::executor::PhysicalPlan; // Query plan fragment with executor name #[derive(Debug)] pub struct QueryFragmentAction { - pub physical_plan: PhysicalPlan, pub executor: String, + pub physical_plan: PhysicalPlan, } impl QueryFragmentAction { pub fn create(executor: String, physical_plan: PhysicalPlan) -> QueryFragmentAction { QueryFragmentAction { - physical_plan, executor, + physical_plan, } } } @@ -131,7 +133,7 @@ impl QueryFragmentsActions { let mut fragment_ids = Vec::new(); for fragment_actions in &self.fragments_actions { let plan = &fragment_actions.fragment_actions[0].physical_plan; - if !matches!(plan, PhysicalPlan::ExchangeSink(_)) { + if plan.downcast_ref::().is_none() { fragment_ids.push(fragment_actions.fragment_id); } } @@ -263,8 +265,8 @@ impl QueryFragmentsActions { for fragment_action in &fragment_actions.fragment_actions { let query_fragment = QueryFragment::create( fragment_actions.fragment_id, - fragment_action.physical_plan.clone(), fragment_actions.data_exchange.clone(), + fragment_action.physical_plan.clone(), ); match fragments_packets.entry(fragment_action.executor.clone()) { diff --git a/src/query/service/src/schedulers/fragments/query_fragment_actions_display.rs b/src/query/service/src/schedulers/fragments/query_fragment_actions_display.rs index adb0b6c3bcd18..0b6b59d95b30a 100644 --- a/src/query/service/src/schedulers/fragments/query_fragment_actions_display.rs +++ b/src/query/service/src/schedulers/fragments/query_fragment_actions_display.rs @@ -17,6 +17,7 @@ use std::fmt::Formatter; use databend_common_sql::MetadataRef; +use crate::physical_plans::PhysicalPlanDynExt; use crate::schedulers::QueryFragmentActions; use crate::schedulers::QueryFragmentsActions; use crate::servers::flight::v1::exchange::DataExchange; @@ -77,11 +78,14 @@ impl Display for QueryFragmentActionsWrap<'_> { if !self.inner.fragment_actions.is_empty() { let fragment_action = &self.inner.fragment_actions[0]; - let plan_display_string = fragment_action - .physical_plan - .format(self.metadata.clone(), Default::default()) - .and_then(|node| Ok(node.format_pretty_with_prefix(" ")?)) - .unwrap(); + let plan_display_string = { + let metadata = self.metadata.read(); + fragment_action + .physical_plan + .format(&metadata, Default::default()) + .and_then(|node| Ok(node.format_pretty_with_prefix(" ")?)) + .unwrap() + }; write!(f, "{}", plan_display_string)?; } diff --git a/src/query/service/src/schedulers/scheduler.rs b/src/query/service/src/schedulers/scheduler.rs index d5ae57bc7776f..e5811b8232258 100644 --- a/src/query/service/src/schedulers/scheduler.rs +++ b/src/query/service/src/schedulers/scheduler.rs @@ -17,12 +17,13 @@ use std::sync::Arc; use async_trait::async_trait; use databend_common_exception::Result; use databend_common_expression::DataBlock; -use databend_common_sql::executor::build_broadcast_plans; use databend_common_sql::planner::QueryExecutor; use databend_common_sql::Planner; use futures_util::TryStreamExt; use crate::interpreters::InterpreterFactory; +use crate::physical_plans::build_broadcast_plans; +use crate::physical_plans::PhysicalPlan; use crate::pipelines::executor::ExecutorSettings; use crate::pipelines::executor::PipelinePullingExecutor; use crate::pipelines::PipelineBuildResult; @@ -31,7 +32,6 @@ use crate::schedulers::Fragmenter; use crate::schedulers::QueryFragmentsActions; use crate::sessions::QueryContext; use crate::sessions::TableContext; -use crate::sql::executor::PhysicalPlan; use crate::sql::ColumnBinding; use crate::stream::PullingExecutorStream; @@ -102,8 +102,11 @@ pub async fn build_distributed_pipeline( .chain(std::iter::once(plan)) { let fragmenter = Fragmenter::try_create(ctx.clone())?; - let root_fragment = fragmenter.build_fragment(plan)?; - root_fragment.get_actions(ctx.clone(), &mut fragments_actions)?; + let fragments = fragmenter.build_fragment(plan)?; + + for fragment in fragments { + fragment.get_actions(ctx.clone(), &mut fragments_actions)?; + } } let exchange_manager = ctx.get_exchange_manager(); @@ -134,9 +137,8 @@ impl ServiceQueryExecutor { } } -#[async_trait] -impl QueryExecutor for ServiceQueryExecutor { - async fn execute_query_with_physical_plan( +impl ServiceQueryExecutor { + pub async fn execute_query_with_physical_plan( &self, plan: &PhysicalPlan, ) -> Result> { @@ -149,7 +151,10 @@ impl QueryExecutor for ServiceQueryExecutor { .try_collect::>() .await } +} +#[async_trait] +impl QueryExecutor for ServiceQueryExecutor { async fn execute_query_with_sql_string(&self, query_sql: &str) -> Result> { let mut planner = Planner::new(self.ctx.clone()); let (plan, _) = planner.plan_sql(query_sql).await?; diff --git a/src/query/service/src/servers/flight/v1/exchange/exchange_manager.rs b/src/query/service/src/servers/flight/v1/exchange/exchange_manager.rs index d2f9051a39d15..8fb914292042f 100644 --- a/src/query/service/src/servers/flight/v1/exchange/exchange_manager.rs +++ b/src/query/service/src/servers/flight/v1/exchange/exchange_manager.rs @@ -37,7 +37,6 @@ use databend_common_exception::Result; use databend_common_grpc::ConnectionFactory; use databend_common_pipeline_core::basic_callback; use databend_common_pipeline_core::ExecutionInfo; -use databend_common_sql::executor::PhysicalPlan; use fastrace::prelude::*; use log::warn; use parking_lot::Mutex; @@ -55,6 +54,7 @@ use super::statistics_receiver::StatisticsReceiver; use super::statistics_sender::StatisticsSender; use crate::clusters::ClusterHelper; use crate::clusters::FlightParams; +use crate::physical_plans::PhysicalPlan; use crate::pipelines::executor::ExecutorSettings; use crate::pipelines::executor::PipelineCompleteExecutor; use crate::pipelines::PipelineBuildResult; diff --git a/src/query/service/src/servers/flight/v1/packets/packet_fragment.rs b/src/query/service/src/servers/flight/v1/packets/packet_fragment.rs index c47877363d0a5..d4a2c1e3f2114 100644 --- a/src/query/service/src/servers/flight/v1/packets/packet_fragment.rs +++ b/src/query/service/src/servers/flight/v1/packets/packet_fragment.rs @@ -15,21 +15,21 @@ use std::fmt::Debug; use std::fmt::Formatter; +use crate::physical_plans::PhysicalPlan; use crate::servers::flight::v1::exchange::DataExchange; -use crate::sql::executor::PhysicalPlan; #[derive(Clone, serde::Serialize, serde::Deserialize)] pub struct QueryFragment { - pub physical_plan: PhysicalPlan, pub fragment_id: usize, pub data_exchange: Option, + pub physical_plan: PhysicalPlan, } impl QueryFragment { pub fn create( fragment_id: usize, - physical_plan: PhysicalPlan, data_exchange: Option, + physical_plan: PhysicalPlan, ) -> QueryFragment { QueryFragment { physical_plan, diff --git a/src/query/service/tests/it/parquet_rs/utils.rs b/src/query/service/tests/it/parquet_rs/utils.rs index d168734347eb8..636e9baeeb517 100644 --- a/src/query/service/tests/it/parquet_rs/utils.rs +++ b/src/query/service/tests/it/parquet_rs/utils.rs @@ -17,9 +17,9 @@ use std::sync::Arc; use databend_common_catalog::plan::DataSourcePlan; use databend_common_catalog::table_context::TableContext; use databend_common_exception::Result; -use databend_common_sql::executor::PhysicalPlanBuilder; use databend_common_sql::plans::Plan; use databend_common_sql::Planner; +use databend_query::physical_plans::PhysicalPlanBuilder; use databend_query::test_kits::ConfigBuilder; use databend_query::test_kits::TestFixture; diff --git a/src/query/service/tests/it/pipelines/builders/mod.rs b/src/query/service/tests/it/pipelines/builders/mod.rs deleted file mode 100644 index 2661dc6476e8f..0000000000000 --- a/src/query/service/tests/it/pipelines/builders/mod.rs +++ /dev/null @@ -1,15 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -mod runtime_filter; diff --git a/src/query/service/tests/it/pipelines/builders/runtime_filter.rs b/src/query/service/tests/it/pipelines/builders/runtime_filter.rs deleted file mode 100644 index b739612ea5a44..0000000000000 --- a/src/query/service/tests/it/pipelines/builders/runtime_filter.rs +++ /dev/null @@ -1,134 +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_base::base::tokio; -use databend_common_exception::Result; -use databend_common_expression::SendableDataBlockStream; -use databend_common_sql::executor::physical_plans::HashJoin; -use databend_common_sql::executor::PhysicalPlan; -use databend_common_sql::executor::PhysicalPlanBuilder; -use databend_common_sql::plans::Plan; -use databend_common_sql::Planner; -use databend_query::interpreters::InterpreterFactory; -use databend_query::pipelines::processors::HashJoinBuildState; -use databend_query::pipelines::processors::HashJoinDesc; -use databend_query::pipelines::processors::HashJoinState; -use databend_query::sessions::QueryContext; -use databend_query::sessions::TableContext; -use databend_query::test_kits::TestFixture; - -async fn plan_sql(ctx: Arc, sql: &str) -> Result { - let mut planner = Planner::new(ctx.clone()); - let (plan, _) = planner.plan_sql(sql).await?; - Ok(plan) -} - -async fn execute_sql(ctx: Arc, sql: &str) -> Result { - let plan = plan_sql(ctx.clone(), sql).await?; - let it = InterpreterFactory::get(ctx.clone(), &plan).await?; - it.execute(ctx).await -} - -async fn physical_plan(ctx: Arc, sql: &str) -> Result { - let plan = plan_sql(ctx.clone(), sql).await?; - match plan { - Plan::Query { - s_expr, - metadata, - bind_context, - .. - } => { - let mut builder = PhysicalPlanBuilder::new(metadata.clone(), ctx, false); - builder.build(&s_expr, bind_context.column_set()).await - } - _ => unreachable!("Query plan expected"), - } -} - -// The method is used to find the join in the physical plan. -// The physical plan should be a simple tree which only contains one binary operator and the binary operator is join. -fn find_join(plan: &PhysicalPlan) -> Result { - match plan { - PhysicalPlan::HashJoin(join) => Ok(join.clone()), - PhysicalPlan::Filter(plan) => find_join(plan.input.as_ref()), - PhysicalPlan::EvalScalar(plan) => find_join(plan.input.as_ref()), - PhysicalPlan::ProjectSet(plan) => find_join(plan.input.as_ref()), - PhysicalPlan::AggregateExpand(plan) => find_join(plan.input.as_ref()), - PhysicalPlan::AggregatePartial(plan) => find_join(plan.input.as_ref()), - PhysicalPlan::AggregateFinal(plan) => find_join(plan.input.as_ref()), - PhysicalPlan::Window(plan) => find_join(plan.input.as_ref()), - PhysicalPlan::Sort(plan) => find_join(plan.input.as_ref()), - PhysicalPlan::Limit(plan) => find_join(plan.input.as_ref()), - PhysicalPlan::RowFetch(plan) => find_join(plan.input.as_ref()), - _ => unreachable!("unexpected plan: {:?}", plan.name()), - } -} - -async fn join_build_state( - ctx: &Arc, - join: &HashJoin, -) -> Result> { - let func_ctx = ctx.get_function_context()?; - - let join_state = HashJoinState::try_create( - ctx.clone(), - join.build.output_schema()?, - &join.build_projections, - HashJoinDesc::create(join)?, - &join.probe_to_build, - false, - true, - None, - )?; - let build_state = HashJoinBuildState::try_create( - ctx.clone(), - func_ctx, - &join.build_keys, - &join.build_projections, - join_state.clone(), - 1, - None, - )?; - Ok(build_state) -} - -#[tokio::test(flavor = "multi_thread", worker_threads = 1)] -async fn test_generate_runtime_filter() -> Result<()> { - let fixture = TestFixture::setup().await?; - // Create table - let _ = execute_sql( - fixture.new_query_ctx().await?, - "CREATE TABLE aa (number int) as select number from numbers(10000000)", - ) - .await?; - - let _ = execute_sql( - fixture.new_query_ctx().await?, - "CREATE TABLE bb (number int) as select number from numbers(10)", - ) - .await?; - - let plan = physical_plan( - fixture.new_query_ctx().await?, - "SELECT * FROM aa JOIN bb ON aa.number = bb.number", - ) - .await?; - let join = find_join(&plan)?; - let join_build_state = join_build_state(&fixture.new_query_ctx().await?, &join).await?; - assert!(join_build_state.get_enable_bloom_runtime_filter()); - assert!(join_build_state.get_enable_min_max_runtime_filter()); - Ok(()) -} diff --git a/src/query/service/tests/it/pipelines/mod.rs b/src/query/service/tests/it/pipelines/mod.rs index 4fa71f2eb15a4..9048ca928cbc9 100644 --- a/src/query/service/tests/it/pipelines/mod.rs +++ b/src/query/service/tests/it/pipelines/mod.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -mod builders; mod executor; mod filter; mod transforms; diff --git a/src/query/service/tests/it/sql/planner/optimizer/optimizer_test.rs b/src/query/service/tests/it/sql/planner/optimizer/optimizer_test.rs index 4d6d5672b19b0..c0c5105cf2748 100644 --- a/src/query/service/tests/it/sql/planner/optimizer/optimizer_test.rs +++ b/src/query/service/tests/it/sql/planner/optimizer/optimizer_test.rs @@ -32,7 +32,6 @@ use databend_common_expression::types::NumberScalar; use databend_common_expression::types::F64; use databend_common_expression::Scalar; use databend_common_meta_types::NodeInfo; -use databend_common_sql::executor::PhysicalPlanBuilder; use databend_common_sql::optimize; use databend_common_sql::optimizer::ir::SExpr; use databend_common_sql::optimizer::ir::SExprVisitor; @@ -49,6 +48,8 @@ use databend_common_sql::Metadata; use databend_common_sql::MetadataRef; use databend_common_storage::Datum; use databend_query::clusters::ClusterHelper; +use databend_query::physical_plans::PhysicalPlanBuilder; +use databend_query::physical_plans::PhysicalPlanDynExt; use databend_query::sessions::QueryContext; use databend_query::test_kits::TestFixture; use goldenfile::Mint; @@ -593,11 +594,12 @@ async fn run_test_case( let physical = builder.build(&s_expr, bind_context.column_set()).await?; write_result(mint, &format!("{}_physical.txt", case.stem), |f| { + let metadata = metadata.read(); writeln!( f, "{}", physical - .format(metadata, Default::default())? + .format(&metadata, Default::default())? .format_pretty()? ) .map_err(|e| ErrorCode::Internal(format!("Failed to write: {}", e))) diff --git a/src/query/service/tests/it/storages/fuse/operations/mutation/block_compact_mutator.rs b/src/query/service/tests/it/storages/fuse/operations/mutation/block_compact_mutator.rs index f57ac1b95d9da..015ca5934c6aa 100644 --- a/src/query/service/tests/it/storages/fuse/operations/mutation/block_compact_mutator.rs +++ b/src/query/service/tests/it/storages/fuse/operations/mutation/block_compact_mutator.rs @@ -22,16 +22,17 @@ use databend_common_catalog::table::CompactionLimits; use databend_common_catalog::table::Table; use databend_common_exception::Result; use databend_common_expression::BlockThresholds; -use databend_common_sql::executor::physical_plans::CommitSink; -use databend_common_sql::executor::physical_plans::CommitType; -use databend_common_sql::executor::physical_plans::CompactSource; use databend_common_sql::executor::physical_plans::MutationKind; -use databend_common_sql::executor::PhysicalPlan; use databend_common_storages_fuse::io::SegmentsIO; use databend_common_storages_fuse::operations::BlockCompactMutator; use databend_common_storages_fuse::operations::CompactBlockPartInfo; use databend_common_storages_fuse::operations::CompactOptions; use databend_common_storages_fuse::statistics::reducers::merge_statistics_mut; +use databend_query::physical_plans::CommitSink; +use databend_query::physical_plans::CommitType; +use databend_query::physical_plans::CompactSource; +use databend_query::physical_plans::PhysicalPlan; +use databend_query::physical_plans::PhysicalPlanMeta; use databend_query::pipelines::executor::ExecutorSettings; use databend_query::pipelines::executor::PipelineCompleteExecutor; use databend_query::schedulers::build_query_pipeline_without_render_result_set; @@ -124,17 +125,18 @@ async fn do_compact(ctx: Arc, table: Arc) -> Result, table: Arc) -> Result, - ) -> Result> { - let metadata = metadata.read().clone(); - let mut context = FormatContext { - scan_id_to_runtime_filters: HashMap::new(), - }; - to_format_tree(self, &metadata, &profs, &mut context) - } - - #[recursive::recursive] - pub fn format_join(&self, metadata: &MetadataRef) -> Result> { - match self { - PhysicalPlan::TableScan(plan) => { - if plan.table_index == Some(DUMMY_TABLE_INDEX) { - return Ok(FormatTreeNode::with_children( - format!("Scan: dummy, rows: {}", plan.source.statistics.read_rows), - vec![], - )); - } - - match plan.table_index { - None => Ok(FormatTreeNode::with_children( - format!( - "Scan: {}.{} (read rows: {})", - plan.source.source_info.catalog_name(), - plan.source.source_info.desc(), - plan.source.statistics.read_rows - ), - vec![], - )), - Some(table_index) => { - let table = metadata.read().table(table_index).clone(); - let table_name = - format!("{}.{}.{}", table.catalog(), table.database(), table.name()); - - Ok(FormatTreeNode::with_children( - format!( - "Scan: {} (#{}) (read rows: {})", - table_name, table_index, plan.source.statistics.read_rows - ), - vec![], - )) - } - } - } - PhysicalPlan::HashJoin(plan) => { - let build_child = plan.build.format_join(metadata)?; - let probe_child = plan.probe.format_join(metadata)?; - - let children = vec![ - FormatTreeNode::with_children("Build".to_string(), vec![build_child]), - FormatTreeNode::with_children("Probe".to_string(), vec![probe_child]), - ]; - - let _estimated_rows = if let Some(info) = &plan.stat_info { - format!("{0:.2}", info.estimated_rows) - } else { - String::from("None") - }; - - Ok(FormatTreeNode::with_children( - format!("HashJoin: {}", plan.join_type), - children, - )) - } - PhysicalPlan::RangeJoin(plan) => { - let left_child = plan.left.format_join(metadata)?; - let right_child = plan.right.format_join(metadata)?; - - let children = vec![ - FormatTreeNode::with_children("Left".to_string(), vec![left_child]), - FormatTreeNode::with_children("Right".to_string(), vec![right_child]), - ]; - - let _estimated_rows = if let Some(info) = &plan.stat_info { - format!("{0:.2}", info.estimated_rows) - } else { - String::from("none") - }; - - Ok(FormatTreeNode::with_children( - format!("RangeJoin: {}", plan.join_type), - children, - )) - } - PhysicalPlan::UnionAll(union_all) => { - let left_child = union_all.left.format_join(metadata)?; - let right_child = union_all.right.format_join(metadata)?; - - let children = vec![ - FormatTreeNode::with_children("Left".to_string(), vec![left_child]), - FormatTreeNode::with_children("Right".to_string(), vec![right_child]), - ]; - - Ok(FormatTreeNode::with_children( - "UnionAll".to_string(), - children, - )) - } - other => { - let children = other - .children() - .map(|child| child.format_join(metadata)) - .collect::>>>()?; - - if children.len() == 1 { - Ok(children[0].clone()) - } else { - Ok(FormatTreeNode::with_children( - format!("{:?}", other), - children, - )) - } - } - } - } -} - -// The method will only collect scan,filter and join nodes -// It's only used to debug cardinality estimator. -#[recursive::recursive] -pub fn format_partial_tree( - plan: &PhysicalPlan, - metadata: &MetadataRef, - profs: &HashMap, -) -> Result> { - match plan { - PhysicalPlan::TableScan(plan) => { - if plan.table_index == Some(DUMMY_TABLE_INDEX) { - return Ok(FormatTreeNode::new("DummyTableScan".to_string())); - } - let table_name = match plan.table_index { - None => format!( - "{}.{}", - plan.source.source_info.catalog_name(), - plan.source.source_info.desc() - ), - Some(table_index) => { - let metadata = metadata.read().clone(); - let table = metadata.table(table_index).clone(); - format!("{}.{}.{}", table.catalog(), table.database(), table.name()) - } - }; - let mut children = vec![FormatTreeNode::new(format!("table: {table_name}"))]; - if let Some(info) = &plan.stat_info { - let items = plan_stats_info_to_format_tree(info); - children.extend(items); - } - append_output_rows_info(&mut children, profs, plan.plan_id); - - Ok(FormatTreeNode::with_children( - "TableScan".to_string(), - children, - )) - } - PhysicalPlan::Filter(plan) => { - let filter = plan - .predicates - .iter() - .map(|pred| pred.as_expr(&BUILTIN_FUNCTIONS).sql_display()) - .join(", "); - let mut children = vec![FormatTreeNode::new(format!("filters: [{filter}]"))]; - if let Some(info) = &plan.stat_info { - let items = plan_stats_info_to_format_tree(info); - children.extend(items); - } - append_output_rows_info(&mut children, profs, plan.plan_id); - children.push(format_partial_tree(&plan.input, metadata, profs)?); - Ok(FormatTreeNode::with_children( - "Filter".to_string(), - children, - )) - } - PhysicalPlan::HashJoin(plan) => { - let build_child = format_partial_tree(&plan.build, metadata, profs)?; - let probe_child = format_partial_tree(&plan.probe, metadata, profs)?; - let mut children = vec![]; - if let Some(info) = &plan.stat_info { - let items = plan_stats_info_to_format_tree(info); - children.extend(items); - } - append_output_rows_info(&mut children, profs, plan.plan_id); - children.push(build_child); - children.push(probe_child); - - Ok(FormatTreeNode::with_children( - format!("HashJoin: {}", plan.join_type), - children, - )) - } - PhysicalPlan::RangeJoin(plan) => { - let left_child = format_partial_tree(&plan.left, metadata, profs)?; - let right_child = format_partial_tree(&plan.right, metadata, profs)?; - let mut children = vec![]; - if let Some(info) = &plan.stat_info { - let items = plan_stats_info_to_format_tree(info); - children.extend(items); - } - append_output_rows_info(&mut children, profs, plan.plan_id); - - let children = vec![ - FormatTreeNode::with_children("Left".to_string(), vec![left_child]), - FormatTreeNode::with_children("Right".to_string(), vec![right_child]), - ]; - - Ok(FormatTreeNode::with_children( - format!("RangeJoin: {}", plan.join_type), - children, - )) - } - PhysicalPlan::UnionAll(union_all) => { - let left_child = format_partial_tree(&union_all.left, metadata, profs)?; - let right_child = format_partial_tree(&union_all.right, metadata, profs)?; - let mut children = vec![]; - if let Some(info) = &union_all.stat_info { - let items = plan_stats_info_to_format_tree(info); - children.extend(items); - } - append_output_rows_info(&mut children, profs, union_all.plan_id); - let children = vec![ - FormatTreeNode::with_children("Left".to_string(), vec![left_child]), - FormatTreeNode::with_children("Right".to_string(), vec![right_child]), - ]; - - Ok(FormatTreeNode::with_children( - "UnionAll".to_string(), - children, - )) - } - PhysicalPlan::MutationSource(plan) => { - let metadata = metadata.read().clone(); - let table = metadata.table(plan.table_index).clone(); - let table_name = format!("{}.{}.{}", table.catalog(), table.database(), table.name()); - let mut children = vec![FormatTreeNode::new(format!("table: {table_name}"))]; - if let Some(filters) = &plan.filters { - let filter = filters.filter.as_expr(&BUILTIN_FUNCTIONS).sql_display(); - children.push(FormatTreeNode::new(format!("filters: [{filter}]"))); - } - append_output_rows_info(&mut children, profs, plan.plan_id); - Ok(FormatTreeNode::with_children( - "MutationSource".to_string(), - children, - )) - } - other => { - let children = other - .children() - .map(|child| format_partial_tree(child, metadata, profs)) - .collect::>>>()?; - - if children.len() == 1 { - Ok(children[0].clone()) - } else { - Ok(FormatTreeNode::with_children( - format!("{:?}", other), - children, - )) - } - } - } -} - -struct FormatContext { - scan_id_to_runtime_filters: HashMap>, -} - -#[recursive::recursive] -fn to_format_tree( - plan: &PhysicalPlan, - metadata: &Metadata, - profs: &HashMap, - context: &mut FormatContext, -) -> Result> { - match plan { - PhysicalPlan::TableScan(plan) => table_scan_to_format_tree(plan, metadata, profs, context), - PhysicalPlan::Filter(plan) => filter_to_format_tree(plan, metadata, profs, context), - PhysicalPlan::EvalScalar(plan) => { - eval_scalar_to_format_tree(plan, metadata, profs, context) - } - PhysicalPlan::AggregateExpand(plan) => { - aggregate_expand_to_format_tree(plan, metadata, profs, context) - } - PhysicalPlan::AggregatePartial(plan) => { - aggregate_partial_to_format_tree(plan, metadata, profs, context) - } - PhysicalPlan::AggregateFinal(plan) => { - aggregate_final_to_format_tree(plan, metadata, profs, context) - } - PhysicalPlan::Window(plan) => window_to_format_tree(plan, metadata, profs, context), - PhysicalPlan::WindowPartition(plan) => { - window_partition_to_format_tree(plan, metadata, profs, context) - } - PhysicalPlan::Sort(plan) => sort_to_format_tree(plan, metadata, profs, context), - PhysicalPlan::Limit(plan) => limit_to_format_tree(plan, metadata, profs, context), - PhysicalPlan::RowFetch(plan) => row_fetch_to_format_tree(plan, metadata, profs, context), - PhysicalPlan::HashJoin(plan) => hash_join_to_format_tree(plan, metadata, profs, context), - PhysicalPlan::Exchange(plan) => exchange_to_format_tree(plan, metadata, profs, context), - PhysicalPlan::UnionAll(plan) => union_all_to_format_tree(plan, metadata, profs, context), - PhysicalPlan::ExchangeSource(plan) => exchange_source_to_format_tree(plan, metadata), - PhysicalPlan::ExchangeSink(plan) => { - exchange_sink_to_format_tree(plan, metadata, profs, context) - } - PhysicalPlan::DistributedInsertSelect(plan) => { - distributed_insert_to_format_tree(plan.as_ref(), metadata, profs, context) - } - PhysicalPlan::Recluster(_) => Ok(FormatTreeNode::new("Recluster".to_string())), - PhysicalPlan::HilbertPartition(_) => { - Ok(FormatTreeNode::new("HilbertPartition".to_string())) - } - PhysicalPlan::CompactSource(_) => Ok(FormatTreeNode::new("CompactSource".to_string())), - PhysicalPlan::CommitSink(plan) => { - commit_sink_to_format_tree(plan, metadata, profs, context) - } - PhysicalPlan::ProjectSet(plan) => { - project_set_to_format_tree(plan, metadata, profs, context) - } - PhysicalPlan::Udf(plan) => udf_to_format_tree(plan, metadata, profs, context), - PhysicalPlan::RangeJoin(plan) => range_join_to_format_tree(plan, metadata, profs, context), - PhysicalPlan::CopyIntoTable(plan) => copy_into_table(plan), - PhysicalPlan::CopyIntoLocation(plan) => copy_into_location(plan), - PhysicalPlan::ReplaceAsyncSourcer(_) => { - Ok(FormatTreeNode::new("ReplaceAsyncSourcer".to_string())) - } - PhysicalPlan::ReplaceDeduplicate(_) => { - Ok(FormatTreeNode::new("ReplaceDeduplicate".to_string())) - } - PhysicalPlan::ReplaceInto(_) => Ok(FormatTreeNode::new("Replace".to_string())), - PhysicalPlan::MutationSource(plan) => format_mutation_source(plan, metadata, profs), - PhysicalPlan::ColumnMutation(plan) => { - format_column_mutation(plan, metadata, profs, context) - } - PhysicalPlan::Mutation(plan) => format_merge_into(plan, metadata, profs, context), - PhysicalPlan::MutationSplit(plan) => { - format_merge_into_split(plan, metadata, profs, context) - } - PhysicalPlan::MutationManipulate(plan) => { - format_merge_into_manipulate(plan, metadata, profs, context) - } - PhysicalPlan::MutationOrganize(plan) => { - format_merge_into_organize(plan, metadata, profs, context) - } - PhysicalPlan::AddStreamColumn(plan) => { - format_add_stream_column(plan, metadata, profs, context) - } - PhysicalPlan::RecursiveCteScan(_) => { - Ok(FormatTreeNode::new("RecursiveCTEScan".to_string())) - } - PhysicalPlan::ConstantTableScan(plan) => constant_table_scan_to_format_tree(plan, metadata), - PhysicalPlan::ExpressionScan(plan) => { - expression_scan_to_format_tree(plan, metadata, profs, context) - } - PhysicalPlan::CacheScan(plan) => cache_scan_to_format_tree(plan, metadata), - PhysicalPlan::Duplicate(plan) => { - let mut children = Vec::new(); - children.push(FormatTreeNode::new(format!( - "Duplicate data to {} branch", - plan.n - ))); - append_profile_info(&mut children, profs, plan.plan_id); - children.push(to_format_tree(&plan.input, metadata, profs, context)?); - Ok(FormatTreeNode::with_children( - "Duplicate".to_string(), - children, - )) - } - PhysicalPlan::Shuffle(plan) => to_format_tree(&plan.input, metadata, profs, context), /* will be hided in explain */ - PhysicalPlan::ChunkFilter(plan) => { - if plan.predicates.iter().all(|x| x.is_none()) { - return to_format_tree(&plan.input, metadata, profs, context); - } - let mut children = Vec::new(); - for (i, predicate) in plan.predicates.iter().enumerate() { - if let Some(predicate) = predicate { - children.push(FormatTreeNode::new(format!( - "branch {}: {}", - i, - predicate.as_expr(&BUILTIN_FUNCTIONS).sql_display() - ))); - } else { - children.push(FormatTreeNode::new(format!("branch {}: None", i))); - } - } - append_profile_info(&mut children, profs, plan.plan_id); - children.push(to_format_tree(&plan.input, metadata, profs, context)?); - Ok(FormatTreeNode::with_children( - "Filter".to_string(), - children, - )) - } - PhysicalPlan::ChunkEvalScalar(plan) => { - let mut children = Vec::new(); - if plan.eval_scalars.iter().all(|x| x.is_none()) { - return to_format_tree(&plan.input, metadata, profs, context); - } - for (i, eval_scalar) in plan.eval_scalars.iter().enumerate() { - if let Some(eval_scalar) = eval_scalar { - children.push(FormatTreeNode::new(format!( - "branch {}: {}", - i, - eval_scalar - .remote_exprs - .iter() - .map(|x| x.as_expr(&BUILTIN_FUNCTIONS).sql_display()) - .join(", ") - ))); - } else { - children.push(FormatTreeNode::new(format!("branch {}: None", i))); - } - } - append_profile_info(&mut children, profs, plan.plan_id); - children.push(to_format_tree(&plan.input, metadata, profs, context)?); - Ok(FormatTreeNode::with_children( - "EvalScalar".to_string(), - children, - )) - } - PhysicalPlan::ChunkCastSchema(plan) => { - to_format_tree(&plan.input, metadata, profs, context) - } // will be hided in explain - PhysicalPlan::ChunkFillAndReorder(plan) => { - to_format_tree(&plan.input, metadata, profs, context) - } // will be hided in explain - PhysicalPlan::ChunkAppendData(plan) => { - let mut children = Vec::new(); - append_profile_info(&mut children, profs, plan.plan_id); - children.push(to_format_tree(&plan.input, metadata, profs, context)?); - Ok(FormatTreeNode::with_children( - "WriteData".to_string(), - children, - )) - } - PhysicalPlan::ChunkMerge(plan) => to_format_tree(&plan.input, metadata, profs, context), /* will be hided in explain */ - PhysicalPlan::ChunkCommitInsert(plan) => { - let mut children = Vec::new(); - append_profile_info(&mut children, profs, plan.plan_id); - children.push(to_format_tree(&plan.input, metadata, profs, context)?); - Ok(FormatTreeNode::with_children( - "Commit".to_string(), - children, - )) - } - PhysicalPlan::AsyncFunction(plan) => { - async_function_to_format_tree(plan, metadata, profs, context) - } - PhysicalPlan::BroadcastSource(_plan) => { - Ok(FormatTreeNode::new("RuntimeFilterSource".to_string())) - } - PhysicalPlan::BroadcastSink(_plan) => { - Ok(FormatTreeNode::new("RuntimeFilterSink".to_string())) - } - } -} - -/// Helper function to add profile info to the format tree. -fn append_profile_info( - children: &mut Vec>, - profs: &HashMap, - plan_id: u32, -) { - if let Some(prof) = profs.get(&plan_id) { - for (_, desc) in get_statistics_desc().iter() { - if prof.statistics[desc.index] != 0 { - children.push(FormatTreeNode::new(format!( - "{}: {}", - desc.display_name.to_lowercase(), - desc.human_format(prof.statistics[desc.index]) - ))); - } - } - } -} - -fn append_output_rows_info( - children: &mut Vec>, - profs: &HashMap, - plan_id: u32, -) { - if let Some(prof) = profs.get(&plan_id) { - for (_, desc) in get_statistics_desc().iter() { - if desc.display_name != "output rows" { - continue; - } - if prof.statistics[desc.index] != 0 { - children.push(FormatTreeNode::new(format!( - "{}: {}", - desc.display_name.to_lowercase(), - desc.human_format(prof.statistics[desc.index]) - ))); - } - break; - } - } -} - -fn format_mutation_source( - plan: &MutationSource, - metadata: &Metadata, - profs: &HashMap, -) -> Result> { - let table = metadata.table(plan.table_index); - let table_name = format!("{}.{}.{}", table.catalog(), table.database(), table.name()); - let filters = plan - .filters - .as_ref() - .map(|filters| filters.filter.as_expr(&BUILTIN_FUNCTIONS).sql_display()) - .unwrap_or_default(); - let mut children = vec![ - FormatTreeNode::new(format!("table: {table_name}")), - FormatTreeNode::new(format!( - "output columns: [{}]", - format_output_columns(plan.output_schema()?, metadata, false) - )), - FormatTreeNode::new(format!("filters: [{filters}]")), - ]; - - let payload = match plan.input_type { - MutationType::Update => "Update", - MutationType::Delete => { - if plan.truncate_table { - "DeleteAll" - } else { - "Delete" - } - } - MutationType::Merge => "Merge", - }; - - // Part stats. - children.extend(part_stats_info_to_format_tree(&plan.statistics)); - append_profile_info(&mut children, profs, plan.plan_id); - - Ok(FormatTreeNode::with_children( - format!("MutationSource({})", payload), - children, - )) -} - -fn format_column_mutation( - plan: &ColumnMutation, - metadata: &Metadata, - profs: &HashMap, - context: &mut FormatContext, -) -> Result> { - to_format_tree(&plan.input, metadata, profs, context) -} - -fn format_merge_into( - merge_into: &Mutation, - metadata: &Metadata, - profs: &HashMap, - context: &mut FormatContext, -) -> Result> { - let table_entry = metadata.table(merge_into.target_table_index).clone(); - let target_table = vec![FormatTreeNode::new(format!( - "target table: [catalog: {}] [database: {}] [table: {}]", - table_entry.catalog(), - table_entry.database(), - table_entry.name() - ))]; - let target_schema = table_entry.table().schema_with_stream(); - - let merge_into_organize: &PhysicalPlan = &merge_into.input; - let merge_into_manipulate: &PhysicalPlan = - if let PhysicalPlan::MutationOrganize(plan) = merge_into_organize { - &plan.input - } else { - return Err(ErrorCode::Internal( - "Expect MutationOrganize after MergeIntoSerialize ".to_string(), - )); - }; - - let children = if let PhysicalPlan::MutationManipulate(plan) = merge_into_manipulate { - // Matched clauses. - let mut matched_children = Vec::with_capacity(plan.matched.len()); - for evaluator in &plan.matched { - let condition_format = evaluator.0.as_ref().map_or_else( - || "condition: None".to_string(), - |predicate| { - format!( - "condition: {}", - predicate.as_expr(&BUILTIN_FUNCTIONS).sql_display() - ) - }, - ); - if evaluator.1.is_none() { - matched_children.push(FormatTreeNode::new(format!( - "matched delete: [{}]", - condition_format - ))); - } else { - let mut update_list = evaluator.1.as_ref().unwrap().clone(); - update_list.sort_by(|a, b| a.0.cmp(&b.0)); - let update_format = update_list - .iter() - .map(|(field_idx, expr)| { - format!( - "{} = {}", - target_schema.field(*field_idx).name(), - expr.as_expr(&BUILTIN_FUNCTIONS).sql_display() - ) - }) - .join(","); - matched_children.push(FormatTreeNode::new(format!( - "matched update: [{}, update set {}]", - condition_format, update_format - ))); - } - } - - // UnMatched clauses. - let mut unmatched_children = Vec::with_capacity(plan.unmatched.len()); - for evaluator in &plan.unmatched { - let condition_format = evaluator.1.as_ref().map_or_else( - || "condition: None".to_string(), - |predicate| { - format!( - "condition: {}", - predicate.as_expr(&BUILTIN_FUNCTIONS).sql_display() - ) - }, - ); - let insert_schema_format = evaluator - .0 - .fields - .iter() - .map(|field| field.name()) - .join(","); - let values_format = evaluator - .2 - .iter() - .map(|expr| expr.as_expr(&BUILTIN_FUNCTIONS).sql_display()) - .join(","); - let unmatched_format = format!( - "insert into ({}) values({})", - insert_schema_format, values_format - ); - unmatched_children.push(FormatTreeNode::new(format!( - "unmatched insert: [{}, {}]", - condition_format, unmatched_format - ))); - } - - [target_table, matched_children, unmatched_children, vec![ - to_format_tree(&plan.input, metadata, profs, context)?, - ]] - .concat() - } else { - return Err(ErrorCode::Internal( - "Expect MutationManipulate after MutationOrganize ".to_string(), - )); - }; - Ok(FormatTreeNode::with_children( - "DataMutation".to_string(), - children, - )) -} - -fn format_merge_into_split( - plan: &MutationSplit, - metadata: &Metadata, - profs: &HashMap, - context: &mut FormatContext, -) -> Result> { - to_format_tree(&plan.input, metadata, profs, context) -} - -fn format_merge_into_manipulate( - plan: &MutationManipulate, - metadata: &Metadata, - profs: &HashMap, - context: &mut FormatContext, -) -> Result> { - to_format_tree(&plan.input, metadata, profs, context) -} - -fn format_merge_into_organize( - plan: &MutationOrganize, - metadata: &Metadata, - profs: &HashMap, - context: &mut FormatContext, -) -> Result> { - to_format_tree(&plan.input, metadata, profs, context) -} - -fn format_add_stream_column( - plan: &AddStreamColumn, - metadata: &Metadata, - profs: &HashMap, - context: &mut FormatContext, -) -> Result> { - to_format_tree(&plan.input, metadata, profs, context) -} - -fn copy_into_table(plan: &CopyIntoTable) -> Result> { - Ok(FormatTreeNode::new(format!( - "CopyIntoTable: {}", - plan.table_info - ))) -} - -fn copy_into_location(_: &CopyIntoLocation) -> Result> { - Ok(FormatTreeNode::new("CopyIntoLocation".to_string())) -} - -fn table_scan_to_format_tree( - plan: &TableScan, - metadata: &Metadata, - profs: &HashMap, - context: &mut FormatContext, -) -> Result> { - if plan.table_index == Some(DUMMY_TABLE_INDEX) { - return Ok(FormatTreeNode::new("DummyTableScan".to_string())); - } - - let table_name = match plan.table_index { - None => format!( - "{}.{}", - plan.source.source_info.catalog_name(), - plan.source.source_info.desc() - ), - Some(table_index) => { - let table = metadata.table(table_index).clone(); - format!("{}.{}.{}", table.catalog(), table.database(), table.name()) - } - }; - let filters = plan - .source - .push_downs - .as_ref() - .and_then(|extras| { - extras - .filters - .as_ref() - .map(|filters| filters.filter.as_expr(&BUILTIN_FUNCTIONS).sql_display()) - }) - .unwrap_or_default(); - - let limit = plan - .source - .push_downs - .as_ref() - .map_or("NONE".to_string(), |extras| { - extras - .limit - .map_or("NONE".to_string(), |limit| limit.to_string()) - }); - - let virtual_columns = plan.source.push_downs.as_ref().and_then(|extras| { - extras.virtual_column.as_ref().map(|virtual_column| { - let mut names = virtual_column - .virtual_column_fields - .iter() - .map(|c| c.name.clone()) - .collect::>(); - names.sort(); - names.iter().join(", ") - }) - }); - - let agg_index = plan - .source - .push_downs - .as_ref() - .and_then(|extras| extras.agg_index.as_ref()); - - let mut children = vec![ - FormatTreeNode::new(format!("table: {table_name}")), - FormatTreeNode::new(format!( - "output columns: [{}]", - format_output_columns(plan.output_schema()?, metadata, false) - )), - ]; - - // Part stats. - children.extend(part_stats_info_to_format_tree(&plan.source.statistics)); - // Push downs. - let push_downs = format!("push downs: [filters: [{filters}], limit: {limit}]"); - children.push(FormatTreeNode::new(push_downs)); - - // runtime filters - let rf = context.scan_id_to_runtime_filters.get(&plan.scan_id); - if let Some(rf) = rf { - let rf = rf.iter().map(|rf| format!("#{:?}", rf.id)).join(", "); - children.push(FormatTreeNode::new(format!("apply join filters: [{rf}]"))); - } - - // Virtual columns. - if let Some(virtual_columns) = virtual_columns { - if !virtual_columns.is_empty() { - let virtual_columns = format!("virtual columns: [{virtual_columns}]"); - children.push(FormatTreeNode::new(virtual_columns)); - } - } - - // Aggregating index - if let Some(agg_index) = agg_index { - let (_, agg_index_sql, _) = metadata - .get_agg_indexes(&table_name) - .unwrap() - .iter() - .find(|(index, _, _)| *index == agg_index.index_id) - .unwrap(); - - children.push(FormatTreeNode::new(format!( - "aggregating index: [{agg_index_sql}]" - ))); - - let agg_sel = agg_index - .selection - .iter() - .map(|(expr, _)| expr.as_expr(&BUILTIN_FUNCTIONS).sql_display()) - .join(", "); - let agg_filter = agg_index - .filter - .as_ref() - .map(|f| f.as_expr(&BUILTIN_FUNCTIONS).sql_display()); - let text = if let Some(f) = agg_filter { - format!("rewritten query: [selection: [{agg_sel}], filter: {f}]") - } else { - format!("rewritten query: [selection: [{agg_sel}]]") - }; - children.push(FormatTreeNode::new(text)); - } - - if let Some(info) = &plan.stat_info { - let items = plan_stats_info_to_format_tree(info); - children.extend(items); - } - - append_profile_info(&mut children, profs, plan.plan_id); - - Ok(FormatTreeNode::with_children( - "TableScan".to_string(), - children, - )) -} - -fn constant_table_scan_to_format_tree( - plan: &ConstantTableScan, - metadata: &Metadata, -) -> Result> { - if plan.num_rows == 0 { - return Ok(FormatTreeNode::new(plan.name().to_string())); - } - - let mut children = Vec::with_capacity(plan.values.len() + 1); - children.push(FormatTreeNode::new(format!( - "output columns: [{}]", - format_output_columns(plan.output_schema()?, metadata, true) - ))); - for (i, value) in plan.values.iter().enumerate() { - let column = value.iter().map(|val| format!("{val}")).join(", "); - children.push(FormatTreeNode::new(format!("column {}: [{}]", i, column))); - } - Ok(FormatTreeNode::with_children( - plan.name().to_string(), - children, - )) -} - -fn expression_scan_to_format_tree( - plan: &ExpressionScan, - metadata: &Metadata, - profs: &HashMap, - context: &mut FormatContext, -) -> Result> { - let mut children = Vec::with_capacity(plan.values.len() + 1); - children.push(FormatTreeNode::new(format!( - "output columns: [{}]", - format_output_columns(plan.output_schema()?, metadata, true) - ))); - for (i, value) in plan.values.iter().enumerate() { - let column = value - .iter() - .map(|val| val.as_expr(&BUILTIN_FUNCTIONS).sql_display()) - .join(", "); - children.push(FormatTreeNode::new(format!("column {}: [{}]", i, column))); - } - - children.push(to_format_tree(&plan.input, metadata, profs, context)?); - - Ok(FormatTreeNode::with_children( - "ExpressionScan".to_string(), - children, - )) -} - -fn cache_scan_to_format_tree( - plan: &CacheScan, - metadata: &Metadata, -) -> Result> { - let mut children = Vec::with_capacity(2); - children.push(FormatTreeNode::new(format!( - "output columns: [{}]", - format_output_columns(plan.output_schema()?, metadata, true) - ))); - - match &plan.cache_source { - CacheSource::HashJoinBuild((cache_index, column_indexes)) => { - let mut column_indexes = column_indexes.clone(); - column_indexes.sort(); - children.push(FormatTreeNode::new(format!("cache index: {}", cache_index))); - children.push(FormatTreeNode::new(format!( - "column indexes: {:?}", - column_indexes - ))); - } - } - - Ok(FormatTreeNode::with_children( - "CacheScan".to_string(), - children, - )) -} - -fn filter_to_format_tree( - plan: &Filter, - metadata: &Metadata, - profs: &HashMap, - context: &mut FormatContext, -) -> Result> { - let filter = plan - .predicates - .iter() - .map(|pred| pred.as_expr(&BUILTIN_FUNCTIONS).sql_display()) - .join(", "); - let mut children = vec![ - FormatTreeNode::new(format!( - "output columns: [{}]", - format_output_columns(plan.output_schema()?, metadata, true) - )), - FormatTreeNode::new(format!("filters: [{filter}]")), - ]; - - if let Some(info) = &plan.stat_info { - let items = plan_stats_info_to_format_tree(info); - children.extend(items); - } - - append_profile_info(&mut children, profs, plan.plan_id); - - children.push(to_format_tree(&plan.input, metadata, profs, context)?); - - Ok(FormatTreeNode::with_children( - "Filter".to_string(), - children, - )) -} - -fn eval_scalar_to_format_tree( - plan: &EvalScalar, - metadata: &Metadata, - profs: &HashMap, - context: &mut FormatContext, -) -> Result> { - if plan.exprs.is_empty() { - return to_format_tree(&plan.input, metadata, profs, context); - } - let scalars = plan - .exprs - .iter() - .map(|(expr, _)| expr.as_expr(&BUILTIN_FUNCTIONS).sql_display()) - .collect::>() - .join(", "); - let mut children = vec![ - FormatTreeNode::new(format!( - "output columns: [{}]", - format_output_columns(plan.output_schema()?, metadata, true) - )), - FormatTreeNode::new(format!("expressions: [{scalars}]")), - ]; - - if let Some(info) = &plan.stat_info { - let items = plan_stats_info_to_format_tree(info); - children.extend(items); - } - - append_profile_info(&mut children, profs, plan.plan_id); - - children.push(to_format_tree(&plan.input, metadata, profs, context)?); - - Ok(FormatTreeNode::with_children( - "EvalScalar".to_string(), - children, - )) -} - -fn async_function_to_format_tree( - plan: &AsyncFunction, - metadata: &Metadata, - profs: &HashMap, - context: &mut FormatContext, -) -> Result> { - let mut children = vec![FormatTreeNode::new(format!( - "output columns: [{}]", - format_output_columns(plan.output_schema()?, metadata, true) - ))]; - - if let Some(info) = &plan.stat_info { - let items = plan_stats_info_to_format_tree(info); - children.extend(items); - } - - append_profile_info(&mut children, profs, plan.plan_id); - - children.push(to_format_tree(&plan.input, metadata, profs, context)?); - - Ok(FormatTreeNode::with_children( - "AsyncFunction".to_string(), - children, - )) -} - -pub fn pretty_display_agg_desc(desc: &AggregateFunctionDesc, metadata: &Metadata) -> String { - format!( - "{}({})", - desc.sig.name, - desc.arg_indices - .iter() - .map(|&index| { metadata.column(index).name() }) - .collect::>() - .join(", ") - ) -} - -fn aggregate_expand_to_format_tree( - plan: &AggregateExpand, - metadata: &Metadata, - profs: &HashMap, - context: &mut FormatContext, -) -> Result> { - let sets = plan - .grouping_sets - .sets - .iter() - .map(|set| { - set.iter() - .map(|&index| metadata.column(index).name()) - .collect::>() - .join(", ") - }) - .map(|s| format!("({})", s)) - .collect::>() - .join(", "); - - let mut children = vec![ - FormatTreeNode::new(format!( - "output columns: [{}]", - format_output_columns(plan.output_schema()?, metadata, true) - )), - FormatTreeNode::new(format!("grouping sets: [{sets}]")), - ]; - - if let Some(info) = &plan.stat_info { - let items = plan_stats_info_to_format_tree(info); - children.extend(items); - } - - append_profile_info(&mut children, profs, plan.plan_id); - - children.push(to_format_tree(&plan.input, metadata, profs, context)?); - - Ok(FormatTreeNode::with_children( - "AggregateExpand".to_string(), - children, - )) -} - -fn aggregate_partial_to_format_tree( - plan: &AggregatePartial, - metadata: &Metadata, - profs: &HashMap, - context: &mut FormatContext, -) -> Result> { - let group_by = plan - .group_by - .iter() - .map(|&index| metadata.column(index).name()) - .join(", "); - let agg_funcs = plan - .agg_funcs - .iter() - .map(|agg| pretty_display_agg_desc(agg, metadata)) - .collect::>() - .join(", "); - - let mut children = vec![ - FormatTreeNode::new(format!("group by: [{group_by}]")), - FormatTreeNode::new(format!("aggregate functions: [{agg_funcs}]")), - ]; - - if let Some(info) = &plan.stat_info { - let items = plan_stats_info_to_format_tree(info); - children.extend(items); - } - - if let Some((_, r)) = &plan.rank_limit { - children.push(FormatTreeNode::new(format!("rank limit: {r}"))); - } - - append_profile_info(&mut children, profs, plan.plan_id); - - children.push(to_format_tree(&plan.input, metadata, profs, context)?); - - Ok(FormatTreeNode::with_children( - "AggregatePartial".to_string(), - children, - )) -} - -fn aggregate_final_to_format_tree( - plan: &AggregateFinal, - metadata: &Metadata, - profs: &HashMap, - context: &mut FormatContext, -) -> Result> { - let group_by = plan - .group_by - .iter() - .map(|&index| { - let name = metadata.column(index).name(); - Ok(name) - }) - .collect::>>()? - .join(", "); - - let agg_funcs = plan - .agg_funcs - .iter() - .map(|agg| pretty_display_agg_desc(agg, metadata)) - .collect::>() - .join(", "); - - let mut children = vec![ - FormatTreeNode::new(format!( - "output columns: [{}]", - format_output_columns(plan.output_schema()?, metadata, true) - )), - FormatTreeNode::new(format!("group by: [{group_by}]")), - FormatTreeNode::new(format!("aggregate functions: [{agg_funcs}]")), - ]; - - if let Some(info) = &plan.stat_info { - let items = plan_stats_info_to_format_tree(info); - children.extend(items); - } - - append_profile_info(&mut children, profs, plan.plan_id); - - children.push(to_format_tree(&plan.input, metadata, profs, context)?); - - Ok(FormatTreeNode::with_children( - "AggregateFinal".to_string(), - children, - )) -} - -fn window_to_format_tree( - plan: &Window, - metadata: &Metadata, - profs: &HashMap, - context: &mut FormatContext, -) -> Result> { - let partition_by = plan - .partition_by - .iter() - .map(|&index| { - let name = metadata.column(index).name(); - Ok(name) - }) - .collect::>>()? - .join(", "); - - let order_by = plan - .order_by - .iter() - .map(|v| v.display_name.clone()) - .collect::>() - .join(", "); - - let frame = plan.window_frame.to_string(); - - let func = match &plan.func { - WindowFunction::Aggregate(agg) => pretty_display_agg_desc(agg, metadata), - func => format!("{}", func), - }; - - let mut children = vec![ - FormatTreeNode::new(format!( - "output columns: [{}]", - format_output_columns(plan.output_schema()?, metadata, true) - )), - FormatTreeNode::new(format!("aggregate function: [{func}]")), - FormatTreeNode::new(format!("partition by: [{partition_by}]")), - FormatTreeNode::new(format!("order by: [{order_by}]")), - FormatTreeNode::new(format!("frame: [{frame}]")), - ]; - - if let Some(limit) = plan.limit { - children.push(FormatTreeNode::new(format!("limit: [{limit}]"))) - } - - append_profile_info(&mut children, profs, plan.plan_id); - - children.push(to_format_tree(&plan.input, metadata, profs, context)?); - - Ok(FormatTreeNode::with_children( - "Window".to_string(), - children, - )) -} - -fn sort_to_format_tree( - plan: &Sort, - metadata: &Metadata, - prof_span_set: &HashMap, - context: &mut FormatContext, -) -> Result> { - let sort_keys = plan - .order_by - .iter() - .map(|sort_key| { - Ok(format!( - "{} {} {}", - sort_key.display_name, - if sort_key.asc { "ASC" } else { "DESC" }, - if sort_key.nulls_first { - "NULLS FIRST" - } else { - "NULLS LAST" - } - )) - }) - .collect::>>()? - .join(", "); - - let mut children = vec![ - FormatTreeNode::new(format!( - "output columns: [{}]", - format_output_columns(plan.output_schema()?, metadata, true) - )), - FormatTreeNode::new(format!("sort keys: [{sort_keys}]")), - ]; - - if let Some(info) = &plan.stat_info { - let items = plan_stats_info_to_format_tree(info); - children.extend(items); - } - - append_profile_info(&mut children, prof_span_set, plan.plan_id); - - children.push(to_format_tree( - &plan.input, - metadata, - prof_span_set, - context, - )?); - - Ok(FormatTreeNode::with_children("Sort".to_string(), children)) -} - -fn window_partition_to_format_tree( - plan: &WindowPartition, - metadata: &Metadata, - prof_span_set: &HashMap, - context: &mut FormatContext, -) -> Result> { - let partition_by = plan - .partition_by - .iter() - .map(|&index| { - let name = metadata.column(index).name(); - Ok(name) - }) - .collect::>>()? - .join(", "); - - let mut children = vec![ - FormatTreeNode::new(format!( - "output columns: [{}]", - format_output_columns(plan.output_schema()?, metadata, true) - )), - FormatTreeNode::new(format!("hash keys: [{partition_by}]")), - ]; - - if let Some(top_n) = &plan.top_n { - children.push(FormatTreeNode::new(format!("top: {}", top_n.top))); - } - - if let Some(info) = &plan.stat_info { - let items = plan_stats_info_to_format_tree(info); - children.extend(items); - } - - append_profile_info(&mut children, prof_span_set, plan.plan_id); - - children.push(to_format_tree( - &plan.input, - metadata, - prof_span_set, - context, - )?); - - Ok(FormatTreeNode::with_children( - "WindowPartition".to_string(), - children, - )) -} - -fn limit_to_format_tree( - plan: &Limit, - metadata: &Metadata, - profs: &HashMap, - context: &mut FormatContext, -) -> Result> { - let mut children = vec![ - FormatTreeNode::new(format!( - "output columns: [{}]", - format_output_columns(plan.output_schema()?, metadata, true) - )), - FormatTreeNode::new(format!( - "limit: {}", - plan.limit - .map_or("NONE".to_string(), |limit| limit.to_string()) - )), - FormatTreeNode::new(format!("offset: {}", plan.offset)), - ]; - - if let Some(info) = &plan.stat_info { - let items = plan_stats_info_to_format_tree(info); - children.extend(items); - } - - append_profile_info(&mut children, profs, plan.plan_id); - - children.push(to_format_tree(&plan.input, metadata, profs, context)?); - - Ok(FormatTreeNode::with_children("Limit".to_string(), children)) -} - -fn row_fetch_to_format_tree( - plan: &RowFetch, - metadata: &Metadata, - profs: &HashMap, - context: &mut FormatContext, -) -> Result> { - let table_schema = plan.source.source_info.schema(); - let projected_schema = plan.cols_to_fetch.project_schema(&table_schema); - let fields_to_fetch = projected_schema.fields(); - - let mut children = vec![ - FormatTreeNode::new(format!( - "output columns: [{}]", - format_output_columns(plan.output_schema()?, metadata, true) - )), - FormatTreeNode::new(format!( - "columns to fetch: [{}]", - fields_to_fetch.iter().map(|f| f.name()).join(", ") - )), - ]; - - if let Some(info) = &plan.stat_info { - let items = plan_stats_info_to_format_tree(info); - children.extend(items); - } - - append_profile_info(&mut children, profs, plan.plan_id); - - children.push(to_format_tree(&plan.input, metadata, profs, context)?); - - Ok(FormatTreeNode::with_children( - "RowFetch".to_string(), - children, - )) -} - -fn range_join_to_format_tree( - plan: &RangeJoin, - metadata: &Metadata, - profs: &HashMap, - context: &mut FormatContext, -) -> Result> { - let range_join_conditions = plan - .conditions - .iter() - .map(|condition| { - let left = condition - .left_expr - .as_expr(&BUILTIN_FUNCTIONS) - .sql_display(); - let right = condition - .right_expr - .as_expr(&BUILTIN_FUNCTIONS) - .sql_display(); - format!("{left} {:?} {right}", condition.operator) - }) - .collect::>() - .join(", "); - let other_conditions = plan - .other_conditions - .iter() - .map(|filter| filter.as_expr(&BUILTIN_FUNCTIONS).sql_display()) - .collect::>() - .join(", "); - - let mut left_child = to_format_tree(&plan.left, metadata, profs, context)?; - let mut right_child = to_format_tree(&plan.right, metadata, profs, context)?; - - left_child.payload = format!("{}(Left)", left_child.payload); - right_child.payload = format!("{}(Right)", right_child.payload); - - let mut children = vec![ - FormatTreeNode::new(format!( - "output columns: [{}]", - format_output_columns(plan.output_schema()?, metadata, true) - )), - FormatTreeNode::new(format!("join type: {}", plan.join_type)), - FormatTreeNode::new(format!("range join conditions: [{range_join_conditions}]")), - FormatTreeNode::new(format!("other conditions: [{other_conditions}]")), - ]; - - if let Some(info) = &plan.stat_info { - let items = plan_stats_info_to_format_tree(info); - children.extend(items); - } - - append_profile_info(&mut children, profs, plan.plan_id); - - children.push(left_child); - children.push(right_child); - - Ok(FormatTreeNode::with_children( - match plan.range_join_type { - RangeJoinType::IEJoin => "IEJoin".to_string(), - RangeJoinType::Merge => "MergeJoin".to_string(), - }, - children, - )) -} - -fn hash_join_to_format_tree( - plan: &HashJoin, - metadata: &Metadata, - profs: &HashMap, - context: &mut FormatContext, -) -> Result> { - for rf in plan.runtime_filter.filters.iter() { - context - .scan_id_to_runtime_filters - .entry(rf.scan_id) - .or_default() - .push(rf.clone()); - } - let build_keys = plan - .build_keys - .iter() - .map(|scalar| scalar.as_expr(&BUILTIN_FUNCTIONS).sql_display()) - .collect::>() - .join(", "); - let probe_keys = plan - .probe_keys - .iter() - .map(|scalar| scalar.as_expr(&BUILTIN_FUNCTIONS).sql_display()) - .collect::>() - .join(", "); - let is_null_equal = plan.is_null_equal.iter().map(|b| format!("{b}")).join(", "); - let filters = plan - .non_equi_conditions - .iter() - .map(|filter| filter.as_expr(&BUILTIN_FUNCTIONS).sql_display()) - .collect::>() - .join(", "); - - let mut build_child = to_format_tree(&plan.build, metadata, profs, context)?; - let mut probe_child = to_format_tree(&plan.probe, metadata, profs, context)?; - - build_child.payload = format!("{}(Build)", build_child.payload); - probe_child.payload = format!("{}(Probe)", probe_child.payload); - - let mut build_runtime_filters = vec![]; - for rf in plan.runtime_filter.filters.iter() { - let mut s = format!( - "filter id:{}, build key:{}, probe key:{}, filter type:", - rf.id, - rf.build_key.as_expr(&BUILTIN_FUNCTIONS).sql_display(), - rf.probe_key.as_expr(&BUILTIN_FUNCTIONS).sql_display(), - ); - if rf.enable_bloom_runtime_filter { - s += "bloom,"; - } - if rf.enable_inlist_runtime_filter { - s += "inlist,"; - } - if rf.enable_min_max_runtime_filter { - s += "min_max,"; - } - s = s.trim_end_matches(',').to_string(); - build_runtime_filters.push(FormatTreeNode::new(s)); - } - - let mut children = vec![ - FormatTreeNode::new(format!( - "output columns: [{}]", - format_output_columns(plan.output_schema()?, metadata, true) - )), - FormatTreeNode::new(format!("join type: {}", plan.join_type)), - FormatTreeNode::new(format!("build keys: [{build_keys}]")), - FormatTreeNode::new(format!("probe keys: [{probe_keys}]")), - FormatTreeNode::new(format!("keys is null equal: [{is_null_equal}]")), - FormatTreeNode::new(format!("filters: [{filters}]")), - ]; - - if !build_runtime_filters.is_empty() { - if plan.broadcast_id.is_some() { - children.push(FormatTreeNode::with_children( - format!("build join filters(distributed):"), - build_runtime_filters, - )); - } else { - children.push(FormatTreeNode::with_children( - format!("build join filters:"), - build_runtime_filters, - )); - } - } - - if let Some((cache_index, column_map)) = &plan.build_side_cache_info { - let mut column_indexes = column_map.keys().collect::>(); - column_indexes.sort(); - children.push(FormatTreeNode::new(format!("cache index: {}", cache_index))); - children.push(FormatTreeNode::new(format!( - "cache columns: {:?}", - column_indexes - ))); - } - - if let Some(info) = &plan.stat_info { - let items = plan_stats_info_to_format_tree(info); - children.extend(items); - } - - append_profile_info(&mut children, profs, plan.plan_id); - - children.push(build_child); - children.push(probe_child); - - Ok(FormatTreeNode::with_children( - "HashJoin".to_string(), - children, - )) -} - -fn exchange_to_format_tree( - plan: &Exchange, - metadata: &Metadata, - profs: &HashMap, - context: &mut FormatContext, -) -> Result> { - Ok(FormatTreeNode::with_children("Exchange".to_string(), vec![ - FormatTreeNode::new(format!( - "output columns: [{}]", - format_output_columns(plan.output_schema()?, metadata, true) - )), - FormatTreeNode::new(format!("exchange type: {}", match plan.kind { - FragmentKind::Init => "Init-Partition".to_string(), - FragmentKind::Normal => format!( - "Hash({})", - plan.keys - .iter() - .map(|key| { key.as_expr(&BUILTIN_FUNCTIONS).sql_display() }) - .collect::>() - .join(", ") - ), - FragmentKind::Expansive => "Broadcast".to_string(), - FragmentKind::Merge => "Merge".to_string(), - })), - to_format_tree(&plan.input, metadata, profs, context)?, - ])) -} - -fn union_all_to_format_tree( - plan: &UnionAll, - metadata: &Metadata, - profs: &HashMap, - context: &mut FormatContext, -) -> Result> { - let mut children = vec![FormatTreeNode::new(format!( - "output columns: [{}]", - format_output_columns(plan.output_schema()?, metadata, true) - ))]; - - if let Some(info) = &plan.stat_info { - let items = plan_stats_info_to_format_tree(info); - children.extend(items); - } - - append_profile_info(&mut children, profs, plan.plan_id); - - children.extend(vec![ - to_format_tree(&plan.left, metadata, profs, context)?, - to_format_tree(&plan.right, metadata, profs, context)?, - ]); - - let root = if !plan.cte_scan_names.is_empty() { - "UnionAll(recursive cte)".to_string() - } else { - "UnionAll".to_string() - }; - - Ok(FormatTreeNode::with_children(root, children)) -} - -fn part_stats_info_to_format_tree(info: &PartStatistics) -> Vec> { - let read_size = format_byte_size(info.read_bytes); - let mut items = vec![ - FormatTreeNode::new(format!("read rows: {}", info.read_rows)), - FormatTreeNode::new(format!("read size: {}", read_size)), - FormatTreeNode::new(format!("partitions total: {}", info.partitions_total)), - FormatTreeNode::new(format!("partitions scanned: {}", info.partitions_scanned)), - ]; - - // format is like "pruning stats: [segments: , blocks: ]" - let mut blocks_pruning_description = String::new(); - - // range pruning status. - if info.pruning_stats.blocks_range_pruning_before > 0 { - blocks_pruning_description += &format!( - "range pruning: {} to {}", - info.pruning_stats.blocks_range_pruning_before, - info.pruning_stats.blocks_range_pruning_after - ); - } - - // bloom pruning status. - if info.pruning_stats.blocks_bloom_pruning_before > 0 { - if !blocks_pruning_description.is_empty() { - blocks_pruning_description += ", "; - } - blocks_pruning_description += &format!( - "bloom pruning: {} to {}", - info.pruning_stats.blocks_bloom_pruning_before, - info.pruning_stats.blocks_bloom_pruning_after - ); - } - - // inverted index pruning status. - if info.pruning_stats.blocks_inverted_index_pruning_before > 0 { - if !blocks_pruning_description.is_empty() { - blocks_pruning_description += ", "; - } - blocks_pruning_description += &format!( - "inverted pruning: {} to {}", - info.pruning_stats.blocks_inverted_index_pruning_before, - info.pruning_stats.blocks_inverted_index_pruning_after - ); - } - - // vector index pruning status. - if info.pruning_stats.blocks_vector_index_pruning_before > 0 { - if !blocks_pruning_description.is_empty() { - blocks_pruning_description += ", "; - } - blocks_pruning_description += &format!( - "vector pruning: {} to {}", - info.pruning_stats.blocks_vector_index_pruning_before, - info.pruning_stats.blocks_vector_index_pruning_after - ); - } - - // Combine segment pruning and blocks pruning descriptions if any - if info.pruning_stats.segments_range_pruning_before > 0 - || !blocks_pruning_description.is_empty() - { - let mut pruning_description = String::new(); - - if info.pruning_stats.segments_range_pruning_before > 0 { - pruning_description += &format!( - "segments: ", - info.pruning_stats.segments_range_pruning_before, - info.pruning_stats.segments_range_pruning_after - ); - } - - if !blocks_pruning_description.is_empty() { - if !pruning_description.is_empty() { - pruning_description += ", "; - } - pruning_description += &format!("blocks: <{}>", blocks_pruning_description); - } - - items.push(FormatTreeNode::new(format!( - "pruning stats: [{}]", - pruning_description - ))); - } - - items -} - -fn plan_stats_info_to_format_tree(info: &PlanStatsInfo) -> Vec> { - vec![FormatTreeNode::new(format!( - "estimated rows: {0:.2}", - info.estimated_rows - ))] -} - -fn exchange_source_to_format_tree( - plan: &ExchangeSource, - metadata: &Metadata, -) -> Result> { - let mut children = vec![FormatTreeNode::new(format!( - "output columns: [{}]", - format_output_columns(plan.output_schema()?, metadata, true) - ))]; - - children.push(FormatTreeNode::new(format!( - "source fragment: [{}]", - plan.source_fragment_id - ))); - - Ok(FormatTreeNode::with_children( - "ExchangeSource".to_string(), - children, - )) -} - -fn exchange_sink_to_format_tree( - plan: &ExchangeSink, - metadata: &Metadata, - profs: &HashMap, - context: &mut FormatContext, -) -> Result> { - let mut children = vec![FormatTreeNode::new(format!( - "output columns: [{}]", - format_output_columns(plan.output_schema()?, metadata, true) - ))]; - - children.push(FormatTreeNode::new(format!( - "destination fragment: [{}]", - plan.destination_fragment_id - ))); - - children.push(to_format_tree(&plan.input, metadata, profs, context)?); - - Ok(FormatTreeNode::with_children( - "ExchangeSink".to_string(), - children, - )) -} - -fn distributed_insert_to_format_tree( - plan: &DistributedInsertSelect, - metadata: &Metadata, - profs: &HashMap, - context: &mut FormatContext, -) -> Result> { - let children = vec![to_format_tree(&plan.input, metadata, profs, context)?]; - - Ok(FormatTreeNode::with_children( - "DistributedInsertSelect".to_string(), - children, - )) -} - -fn commit_sink_to_format_tree( - plan: &CommitSink, - metadata: &Metadata, - profs: &HashMap, - context: &mut FormatContext, -) -> Result> { - let children = vec![to_format_tree(&plan.input, metadata, profs, context)?]; - Ok(FormatTreeNode::with_children( - "CommitSink".to_string(), - children, - )) -} - -fn project_set_to_format_tree( - plan: &ProjectSet, - metadata: &Metadata, - profs: &HashMap, - context: &mut FormatContext, -) -> Result> { - let mut children = vec![FormatTreeNode::new(format!( - "output columns: [{}]", - format_output_columns(plan.output_schema()?, metadata, true) - ))]; - - if let Some(info) = &plan.stat_info { - let items = plan_stats_info_to_format_tree(info); - children.extend(items); - } - - append_profile_info(&mut children, profs, plan.plan_id); - - children.extend(vec![FormatTreeNode::new(format!( - "set returning functions: {}", - plan.srf_exprs - .iter() - .map(|(expr, _)| expr.clone().as_expr(&BUILTIN_FUNCTIONS).sql_display()) - .collect::>() - .join(", ") - ))]); - - children.extend(vec![to_format_tree(&plan.input, metadata, profs, context)?]); - - Ok(FormatTreeNode::with_children( - "ProjectSet".to_string(), - children, - )) -} - -fn udf_to_format_tree( - plan: &Udf, - metadata: &Metadata, - profs: &HashMap, - context: &mut FormatContext, -) -> Result> { - let mut children = vec![FormatTreeNode::new(format!( - "output columns: [{}]", - format_output_columns(plan.output_schema()?, metadata, true) - ))]; - - if let Some(info) = &plan.stat_info { - let items = plan_stats_info_to_format_tree(info); - children.extend(items); - } - - append_profile_info(&mut children, profs, plan.plan_id); - - children.extend(vec![FormatTreeNode::new(format!( - "udf functions: {}", - plan.udf_funcs - .iter() - .map(|func| { - let arg_exprs = func.arg_exprs.join(", "); - format!("{}({})", func.func_name, arg_exprs) - }) - .collect::>() - .join(", ") - ))]); - - children.extend(vec![to_format_tree(&plan.input, metadata, profs, context)?]); - - Ok(FormatTreeNode::with_children("Udf".to_string(), children)) -} - -fn format_output_columns( - output_schema: DataSchemaRef, - metadata: &Metadata, - format_table: bool, -) -> String { - output_schema - .fields() - .iter() - .map(|field| match field.name().parse::() { - Ok(column_index) => { - if column_index == usize::MAX { - return String::from("dummy value"); - } - let column_entry = metadata.column(column_index); - match column_entry.table_index() { - Some(table_index) if format_table => match metadata - .table(table_index) - .alias_name() - { - Some(alias_name) => { - format!("{}.{} (#{})", alias_name, column_entry.name(), column_index) - } - None => format!( - "{}.{} (#{})", - metadata.table(table_index).name(), - column_entry.name(), - column_index, - ), - }, - _ => format!("{} (#{})", column_entry.name(), column_index), - } - } - _ => format!("#{}", field.name()), - }) - .collect::>() - .join(", ") -} diff --git a/src/query/sql/src/executor/mod.rs b/src/query/sql/src/executor/mod.rs index 3e40852552cac..5935ece1cf452 100644 --- a/src/query/sql/src/executor/mod.rs +++ b/src/query/sql/src/executor/mod.rs @@ -12,22 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. -mod explain; -mod format; -mod physical_plan; -mod physical_plan_builder; -mod physical_plan_visitor; pub mod physical_plans; mod util; pub mod table_read_plan; -pub use format::format_partial_tree; -pub use physical_plan::PhysicalPlan; -pub use physical_plan_builder::MutationBuildInfo; -pub use physical_plan_builder::PhysicalPlanBuilder; -pub use physical_plan_visitor::PhysicalPlanReplacer; -pub use physical_plans::build_broadcast_plans; -pub use physical_plans::PhysicalRuntimeFilter; -pub use physical_plans::PhysicalRuntimeFilters; pub use util::*; diff --git a/src/query/sql/src/executor/physical_plan.rs b/src/query/sql/src/executor/physical_plan.rs deleted file mode 100644 index 33f29fabf5cd1..0000000000000 --- a/src/query/sql/src/executor/physical_plan.rs +++ /dev/null @@ -1,1139 +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::HashMap; - -use databend_common_catalog::plan::DataSourceInfo; -use databend_common_catalog::plan::DataSourcePlan; -use databend_common_catalog::plan::PartStatistics; -use databend_common_catalog::plan::PartitionsShuffleKind; -use databend_common_exception::Result; -use databend_common_expression::DataSchemaRef; -use databend_common_functions::BUILTIN_FUNCTIONS; -use educe::Educe; -use enum_as_inner::EnumAsInner; -use itertools::Itertools; - -use super::physical_plans::AddStreamColumn; -use super::physical_plans::BroadcastSink; -use super::physical_plans::BroadcastSource; -use super::physical_plans::HilbertPartition; -use super::physical_plans::MutationManipulate; -use super::physical_plans::MutationOrganize; -use super::physical_plans::MutationSource; -use super::physical_plans::MutationSplit; -use crate::executor::physical_plans::AggregateExpand; -use crate::executor::physical_plans::AggregateFinal; -use crate::executor::physical_plans::AggregatePartial; -use crate::executor::physical_plans::AsyncFunction; -use crate::executor::physical_plans::CacheScan; -use crate::executor::physical_plans::ChunkAppendData; -use crate::executor::physical_plans::ChunkCastSchema; -use crate::executor::physical_plans::ChunkCommitInsert; -use crate::executor::physical_plans::ChunkEvalScalar; -use crate::executor::physical_plans::ChunkFillAndReorder; -use crate::executor::physical_plans::ChunkFilter; -use crate::executor::physical_plans::ChunkMerge; -use crate::executor::physical_plans::ColumnMutation; -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::DistributedInsertSelect; -use crate::executor::physical_plans::Duplicate; -use crate::executor::physical_plans::EvalScalar; -use crate::executor::physical_plans::Exchange; -use crate::executor::physical_plans::ExchangeSink; -use crate::executor::physical_plans::ExchangeSource; -use crate::executor::physical_plans::ExpressionScan; -use crate::executor::physical_plans::Filter; -use crate::executor::physical_plans::HashJoin; -use crate::executor::physical_plans::Limit; -use crate::executor::physical_plans::Mutation; -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; -use crate::executor::physical_plans::Shuffle; -use crate::executor::physical_plans::Sort; -use crate::executor::physical_plans::TableScan; -use crate::executor::physical_plans::Udf; -use crate::executor::physical_plans::UnionAll; -use crate::executor::physical_plans::Window; -use crate::executor::physical_plans::WindowPartition; - -#[derive(serde::Serialize, serde::Deserialize, Educe, EnumAsInner)] -#[educe( - Clone(bound = false, attrs = "#[recursive::recursive]"), - Debug(bound = false, attrs = "#[recursive::recursive]") -)] -#[allow(clippy::large_enum_variant)] -pub enum PhysicalPlan { - /// Query - TableScan(TableScan), - Filter(Filter), - EvalScalar(EvalScalar), - ProjectSet(ProjectSet), - AggregateExpand(AggregateExpand), - AggregatePartial(AggregatePartial), - AggregateFinal(AggregateFinal), - Window(Window), - Sort(Sort), - WindowPartition(WindowPartition), - Limit(Limit), - RowFetch(RowFetch), - HashJoin(HashJoin), - RangeJoin(RangeJoin), - Exchange(Exchange), - UnionAll(UnionAll), - ConstantTableScan(ConstantTableScan), - ExpressionScan(ExpressionScan), - CacheScan(CacheScan), - Udf(Udf), - RecursiveCteScan(RecursiveCteScan), - - /// For insert into ... select ... in cluster - DistributedInsertSelect(Box), - - /// Synthesized by fragmented - ExchangeSource(ExchangeSource), - ExchangeSink(ExchangeSink), - - /// Copy into table - CopyIntoTable(Box), - CopyIntoLocation(Box), - - /// Replace - ReplaceAsyncSourcer(ReplaceAsyncSourcer), - ReplaceDeduplicate(Box), - ReplaceInto(Box), - - /// Mutation - Mutation(Box), - MutationSplit(Box), - MutationManipulate(Box), - MutationOrganize(Box), - AddStreamColumn(Box), - ColumnMutation(ColumnMutation), - MutationSource(MutationSource), - - /// Compact - CompactSource(Box), - - /// Commit - CommitSink(Box), - - /// Recluster - Recluster(Box), - HilbertPartition(Box), - - /// Multi table insert - Duplicate(Box), - Shuffle(Box), - ChunkFilter(Box), - ChunkEvalScalar(Box), - ChunkCastSchema(Box), - ChunkFillAndReorder(Box), - ChunkAppendData(Box), - ChunkMerge(Box), - ChunkCommitInsert(Box), - - // async function call - AsyncFunction(AsyncFunction), - - // broadcast - BroadcastSource(BroadcastSource), - BroadcastSink(BroadcastSink), -} - -impl PhysicalPlan { - /// Adjust the plan_id of the physical plan. - /// This function will assign a unique plan_id to each physical plan node in a top-down manner. - /// Which means the plan_id of a node is always greater than the plan_id of its parent node. - #[recursive::recursive] - pub fn adjust_plan_id(&mut self, next_id: &mut u32) { - match self { - PhysicalPlan::AsyncFunction(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - } - PhysicalPlan::TableScan(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - } - PhysicalPlan::Filter(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - plan.input.adjust_plan_id(next_id); - } - PhysicalPlan::EvalScalar(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - plan.input.adjust_plan_id(next_id); - } - PhysicalPlan::ProjectSet(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - plan.input.adjust_plan_id(next_id); - } - PhysicalPlan::AggregateExpand(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - plan.input.adjust_plan_id(next_id); - } - PhysicalPlan::AggregatePartial(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - plan.input.adjust_plan_id(next_id); - } - PhysicalPlan::AggregateFinal(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - plan.input.adjust_plan_id(next_id); - } - PhysicalPlan::Window(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - plan.input.adjust_plan_id(next_id); - } - PhysicalPlan::WindowPartition(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - plan.input.adjust_plan_id(next_id); - } - PhysicalPlan::Sort(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - plan.input.adjust_plan_id(next_id); - } - PhysicalPlan::Limit(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - plan.input.adjust_plan_id(next_id); - } - PhysicalPlan::RowFetch(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - plan.input.adjust_plan_id(next_id); - } - PhysicalPlan::HashJoin(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - plan.probe.adjust_plan_id(next_id); - plan.build.adjust_plan_id(next_id); - } - PhysicalPlan::RangeJoin(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - plan.left.adjust_plan_id(next_id); - plan.right.adjust_plan_id(next_id); - } - PhysicalPlan::Exchange(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - plan.input.adjust_plan_id(next_id); - } - PhysicalPlan::UnionAll(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - plan.left.adjust_plan_id(next_id); - plan.right.adjust_plan_id(next_id); - } - PhysicalPlan::RecursiveCteScan(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - } - PhysicalPlan::ConstantTableScan(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - } - PhysicalPlan::ExpressionScan(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - } - PhysicalPlan::CacheScan(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - } - PhysicalPlan::Udf(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - plan.input.adjust_plan_id(next_id); - } - PhysicalPlan::DistributedInsertSelect(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - plan.input.adjust_plan_id(next_id); - } - PhysicalPlan::ExchangeSource(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - } - PhysicalPlan::ExchangeSink(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - } - PhysicalPlan::CopyIntoTable(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; - *next_id += 1; - plan.input.adjust_plan_id(next_id); - } - PhysicalPlan::ReplaceInto(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - plan.input.adjust_plan_id(next_id); - } - PhysicalPlan::MutationSource(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - } - PhysicalPlan::ColumnMutation(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - plan.input.adjust_plan_id(next_id); - } - PhysicalPlan::Mutation(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - plan.input.adjust_plan_id(next_id); - } - PhysicalPlan::MutationSplit(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - plan.input.adjust_plan_id(next_id); - } - PhysicalPlan::MutationManipulate(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - plan.input.adjust_plan_id(next_id); - } - PhysicalPlan::MutationOrganize(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - plan.input.adjust_plan_id(next_id); - } - PhysicalPlan::AddStreamColumn(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - plan.input.adjust_plan_id(next_id); - } - PhysicalPlan::CommitSink(plan) => { - plan.plan_id = *next_id; - *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; - plan.input.adjust_plan_id(next_id); - } - PhysicalPlan::CompactSource(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - } - PhysicalPlan::Recluster(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - } - PhysicalPlan::HilbertPartition(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - } - PhysicalPlan::Duplicate(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - plan.input.adjust_plan_id(next_id); - } - PhysicalPlan::Shuffle(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - plan.input.adjust_plan_id(next_id); - } - PhysicalPlan::ChunkFilter(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - plan.input.adjust_plan_id(next_id); - } - PhysicalPlan::ChunkEvalScalar(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - plan.input.adjust_plan_id(next_id); - } - PhysicalPlan::ChunkCastSchema(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - plan.input.adjust_plan_id(next_id); - } - PhysicalPlan::ChunkFillAndReorder(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - plan.input.adjust_plan_id(next_id); - } - PhysicalPlan::ChunkAppendData(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - plan.input.adjust_plan_id(next_id); - } - PhysicalPlan::ChunkMerge(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - plan.input.adjust_plan_id(next_id); - } - PhysicalPlan::ChunkCommitInsert(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - plan.input.adjust_plan_id(next_id); - } - PhysicalPlan::BroadcastSource(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - } - PhysicalPlan::BroadcastSink(plan) => { - plan.plan_id = *next_id; - *next_id += 1; - plan.input.adjust_plan_id(next_id); - } - } - } - - /// Get the id of the plan node - pub fn get_id(&self) -> u32 { - match self { - PhysicalPlan::AsyncFunction(v) => v.plan_id, - PhysicalPlan::TableScan(v) => v.plan_id, - PhysicalPlan::Filter(v) => v.plan_id, - PhysicalPlan::EvalScalar(v) => v.plan_id, - PhysicalPlan::ProjectSet(v) => v.plan_id, - PhysicalPlan::AggregateExpand(v) => v.plan_id, - PhysicalPlan::AggregatePartial(v) => v.plan_id, - PhysicalPlan::AggregateFinal(v) => v.plan_id, - PhysicalPlan::Window(v) => v.plan_id, - PhysicalPlan::WindowPartition(v) => v.plan_id, - PhysicalPlan::Sort(v) => v.plan_id, - PhysicalPlan::Limit(v) => v.plan_id, - PhysicalPlan::RowFetch(v) => v.plan_id, - PhysicalPlan::HashJoin(v) => v.plan_id, - 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::ConstantTableScan(v) => v.plan_id, - PhysicalPlan::ExpressionScan(v) => v.plan_id, - PhysicalPlan::CacheScan(v) => v.plan_id, - PhysicalPlan::Udf(v) => v.plan_id, - PhysicalPlan::MutationSource(v) => v.plan_id, - PhysicalPlan::ColumnMutation(v) => v.plan_id, - PhysicalPlan::Mutation(v) => v.plan_id, - PhysicalPlan::MutationSplit(v) => v.plan_id, - PhysicalPlan::MutationManipulate(v) => v.plan_id, - 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::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, - PhysicalPlan::Recluster(v) => v.plan_id, - PhysicalPlan::HilbertPartition(v) => v.plan_id, - PhysicalPlan::Duplicate(v) => v.plan_id, - PhysicalPlan::Shuffle(v) => v.plan_id, - PhysicalPlan::ChunkFilter(v) => v.plan_id, - PhysicalPlan::ChunkEvalScalar(v) => v.plan_id, - PhysicalPlan::ChunkCastSchema(v) => v.plan_id, - PhysicalPlan::ChunkFillAndReorder(v) => v.plan_id, - PhysicalPlan::ChunkAppendData(v) => v.plan_id, - PhysicalPlan::ChunkMerge(v) => v.plan_id, - PhysicalPlan::ChunkCommitInsert(v) => v.plan_id, - PhysicalPlan::RecursiveCteScan(v) => v.plan_id, - PhysicalPlan::BroadcastSource(v) => v.plan_id, - PhysicalPlan::BroadcastSink(v) => v.plan_id, - } - } - - pub fn output_schema(&self) -> Result { - match self { - PhysicalPlan::AsyncFunction(plan) => plan.output_schema(), - PhysicalPlan::TableScan(plan) => plan.output_schema(), - PhysicalPlan::Filter(plan) => plan.output_schema(), - PhysicalPlan::EvalScalar(plan) => plan.output_schema(), - PhysicalPlan::AggregateExpand(plan) => plan.output_schema(), - PhysicalPlan::AggregatePartial(plan) => plan.output_schema(), - PhysicalPlan::AggregateFinal(plan) => plan.output_schema(), - PhysicalPlan::Window(plan) => plan.output_schema(), - PhysicalPlan::WindowPartition(plan) => plan.output_schema(), - PhysicalPlan::Sort(plan) => plan.output_schema(), - PhysicalPlan::Limit(plan) => plan.output_schema(), - PhysicalPlan::RowFetch(plan) => plan.output_schema(), - PhysicalPlan::HashJoin(plan) => plan.output_schema(), - PhysicalPlan::Exchange(plan) => plan.output_schema(), - PhysicalPlan::ExchangeSource(plan) => plan.output_schema(), - PhysicalPlan::ExchangeSink(plan) => plan.output_schema(), - 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::CopyIntoLocation(plan) => plan.output_schema(), - PhysicalPlan::ConstantTableScan(plan) => plan.output_schema(), - PhysicalPlan::ExpressionScan(plan) => plan.output_schema(), - PhysicalPlan::CacheScan(plan) => plan.output_schema(), - PhysicalPlan::RecursiveCteScan(plan) => plan.output_schema(), - PhysicalPlan::Udf(plan) => plan.output_schema(), - PhysicalPlan::MutationSource(plan) => plan.output_schema(), - PhysicalPlan::MutationSplit(plan) => plan.output_schema(), - PhysicalPlan::MutationManipulate(plan) => plan.output_schema(), - PhysicalPlan::MutationOrganize(plan) => plan.output_schema(), - PhysicalPlan::AddStreamColumn(plan) => plan.output_schema(), - PhysicalPlan::Mutation(_) - | PhysicalPlan::ColumnMutation(_) - | PhysicalPlan::ReplaceAsyncSourcer(_) - | PhysicalPlan::ReplaceDeduplicate(_) - | PhysicalPlan::ReplaceInto(_) - | PhysicalPlan::CompactSource(_) - | PhysicalPlan::CommitSink(_) - | PhysicalPlan::DistributedInsertSelect(_) - | PhysicalPlan::Recluster(_) - | PhysicalPlan::BroadcastSource(_) - | PhysicalPlan::BroadcastSink(_) - | PhysicalPlan::HilbertPartition(_) => Ok(DataSchemaRef::default()), - PhysicalPlan::Duplicate(plan) => plan.input.output_schema(), - PhysicalPlan::Shuffle(plan) => plan.input.output_schema(), - PhysicalPlan::ChunkFilter(plan) => plan.input.output_schema(), - PhysicalPlan::ChunkEvalScalar(_) => todo!(), - PhysicalPlan::ChunkCastSchema(_) => todo!(), - PhysicalPlan::ChunkFillAndReorder(_) => todo!(), - PhysicalPlan::ChunkAppendData(_) => todo!(), - PhysicalPlan::ChunkMerge(_) => todo!(), - PhysicalPlan::ChunkCommitInsert(_) => todo!(), - } - } - - pub fn name(&self) -> String { - match self { - PhysicalPlan::TableScan(v) => match &v.source.source_info { - DataSourceInfo::TableSource(_) => "TableScan".to_string(), - DataSourceInfo::StageSource(_) => "StageScan".to_string(), - DataSourceInfo::ParquetSource(_) => "ParquetScan".to_string(), - DataSourceInfo::ResultScanSource(_) => "ResultScan".to_string(), - DataSourceInfo::ORCSource(_) => "OrcScan".to_string(), - }, - PhysicalPlan::AsyncFunction(_) => "AsyncFunction".to_string(), - PhysicalPlan::Filter(_) => "Filter".to_string(), - PhysicalPlan::EvalScalar(_) => "EvalScalar".to_string(), - PhysicalPlan::AggregateExpand(_) => "AggregateExpand".to_string(), - PhysicalPlan::AggregatePartial(_) => "AggregatePartial".to_string(), - PhysicalPlan::AggregateFinal(_) => "AggregateFinal".to_string(), - PhysicalPlan::Window(_) => "Window".to_string(), - PhysicalPlan::WindowPartition(_) => "WindowPartition".to_string(), - PhysicalPlan::Sort(_) => "Sort".to_string(), - PhysicalPlan::Limit(_) => "Limit".to_string(), - PhysicalPlan::RowFetch(_) => "RowFetch".to_string(), - 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::CopyIntoLocation(_) => "CopyIntoLocation".to_string(), - PhysicalPlan::ReplaceAsyncSourcer(_) => "ReplaceAsyncSourcer".to_string(), - PhysicalPlan::ReplaceDeduplicate(_) => "ReplaceDeduplicate".to_string(), - PhysicalPlan::ReplaceInto(_) => "Replace".to_string(), - PhysicalPlan::MutationSource(_) => "MutationSource".to_string(), - PhysicalPlan::ColumnMutation(_) => "ColumnMutation".to_string(), - PhysicalPlan::Mutation(_) => "MergeInto".to_string(), - PhysicalPlan::MutationSplit(_) => "MutationSplit".to_string(), - PhysicalPlan::MutationManipulate(_) => "MutationManipulate".to_string(), - PhysicalPlan::MutationOrganize(_) => "MutationOrganize".to_string(), - PhysicalPlan::AddStreamColumn(_) => "AddStreamColumn".to_string(), - PhysicalPlan::RecursiveCteScan(_) => "RecursiveCteScan".to_string(), - PhysicalPlan::ConstantTableScan(_) => "PhysicalConstantTableScan".to_string(), - PhysicalPlan::ExpressionScan(_) => "ExpressionScan".to_string(), - PhysicalPlan::CacheScan(_) => "CacheScan".to_string(), - PhysicalPlan::Recluster(_) => "Recluster".to_string(), - PhysicalPlan::HilbertPartition(_) => "HilbertPartition".to_string(), - PhysicalPlan::Udf(_) => "Udf".to_string(), - PhysicalPlan::Duplicate(_) => "Duplicate".to_string(), - PhysicalPlan::Shuffle(_) => "Shuffle".to_string(), - PhysicalPlan::ChunkFilter(_) => "Filter".to_string(), - PhysicalPlan::ChunkEvalScalar(_) => "EvalScalar".to_string(), - PhysicalPlan::ChunkCastSchema(_) => "CastSchema".to_string(), - PhysicalPlan::ChunkFillAndReorder(_) => "FillAndReorder".to_string(), - PhysicalPlan::ChunkAppendData(_) => "WriteData".to_string(), - PhysicalPlan::ChunkMerge(_) => "ChunkMerge".to_string(), - PhysicalPlan::ChunkCommitInsert(_) => "Commit".to_string(), - PhysicalPlan::BroadcastSource(_) => "RuntimeFilterSource".to_string(), - PhysicalPlan::BroadcastSink(_) => "RuntimeFilterSink".to_string(), - } - } - - pub fn children<'a>(&'a self) -> Box + 'a> { - match self { - PhysicalPlan::TableScan(_) - | PhysicalPlan::ConstantTableScan(_) - | PhysicalPlan::CacheScan(_) - | PhysicalPlan::ExchangeSource(_) - | PhysicalPlan::CompactSource(_) - | PhysicalPlan::ReplaceAsyncSourcer(_) - | PhysicalPlan::Recluster(_) - | PhysicalPlan::RecursiveCteScan(_) - | PhysicalPlan::BroadcastSource(_) => Box::new(std::iter::empty()), - PhysicalPlan::HilbertPartition(plan) => Box::new(std::iter::once(plan.input.as_ref())), - PhysicalPlan::Filter(plan) => Box::new(std::iter::once(plan.input.as_ref())), - PhysicalPlan::EvalScalar(plan) => Box::new(std::iter::once(plan.input.as_ref())), - PhysicalPlan::AggregateExpand(plan) => Box::new(std::iter::once(plan.input.as_ref())), - PhysicalPlan::AggregatePartial(plan) => Box::new(std::iter::once(plan.input.as_ref())), - PhysicalPlan::AggregateFinal(plan) => Box::new(std::iter::once(plan.input.as_ref())), - PhysicalPlan::Window(plan) => Box::new(std::iter::once(plan.input.as_ref())), - PhysicalPlan::WindowPartition(plan) => Box::new(std::iter::once(plan.input.as_ref())), - PhysicalPlan::Sort(plan) => Box::new(std::iter::once(plan.input.as_ref())), - PhysicalPlan::Limit(plan) => Box::new(std::iter::once(plan.input.as_ref())), - PhysicalPlan::RowFetch(plan) => Box::new(std::iter::once(plan.input.as_ref())), - PhysicalPlan::HashJoin(plan) => Box::new( - std::iter::once(plan.probe.as_ref()).chain(std::iter::once(plan.build.as_ref())), - ), - PhysicalPlan::ExpressionScan(plan) => Box::new(std::iter::once(plan.input.as_ref())), - PhysicalPlan::Exchange(plan) => Box::new(std::iter::once(plan.input.as_ref())), - PhysicalPlan::ExchangeSink(plan) => Box::new(std::iter::once(plan.input.as_ref())), - PhysicalPlan::BroadcastSink(plan) => Box::new(std::iter::once(plan.input.as_ref())), - 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( - std::iter::once(plan.left.as_ref()).chain(std::iter::once(plan.right.as_ref())), - ), - PhysicalPlan::ReplaceDeduplicate(plan) => { - Box::new(std::iter::once(plan.input.as_ref())) - } - PhysicalPlan::ReplaceInto(plan) => Box::new(std::iter::once(plan.input.as_ref())), - PhysicalPlan::MutationSource(_) => Box::new(std::iter::empty()), - PhysicalPlan::ColumnMutation(plan) => Box::new(std::iter::once(plan.input.as_ref())), - PhysicalPlan::Mutation(plan) => Box::new(std::iter::once(plan.input.as_ref())), - PhysicalPlan::MutationSplit(plan) => Box::new(std::iter::once(plan.input.as_ref())), - PhysicalPlan::MutationManipulate(plan) => { - Box::new(std::iter::once(plan.input.as_ref())) - } - PhysicalPlan::MutationOrganize(plan) => Box::new(std::iter::once(plan.input.as_ref())), - PhysicalPlan::AddStreamColumn(plan) => Box::new(std::iter::once(plan.input.as_ref())), - PhysicalPlan::Udf(plan) => Box::new(std::iter::once(plan.input.as_ref())), - PhysicalPlan::AsyncFunction(plan) => Box::new(std::iter::once(plan.input.as_ref())), - PhysicalPlan::CopyIntoLocation(plan) => Box::new(std::iter::once(plan.input.as_ref())), - PhysicalPlan::Duplicate(plan) => Box::new(std::iter::once(plan.input.as_ref())), - PhysicalPlan::Shuffle(plan) => Box::new(std::iter::once(plan.input.as_ref())), - PhysicalPlan::ChunkFilter(plan) => Box::new(std::iter::once(plan.input.as_ref())), - PhysicalPlan::ChunkEvalScalar(plan) => Box::new(std::iter::once(plan.input.as_ref())), - PhysicalPlan::ChunkCastSchema(plan) => Box::new(std::iter::once(plan.input.as_ref())), - PhysicalPlan::ChunkFillAndReorder(plan) => { - Box::new(std::iter::once(plan.input.as_ref())) - } - 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())), - }, - } - } - - pub fn children_mut<'a>(&'a mut self) -> Box + 'a> { - match self { - PhysicalPlan::TableScan(_) - | PhysicalPlan::ConstantTableScan(_) - | PhysicalPlan::CacheScan(_) - | PhysicalPlan::ExchangeSource(_) - | PhysicalPlan::CompactSource(_) - | PhysicalPlan::ReplaceAsyncSourcer(_) - | PhysicalPlan::Recluster(_) - | PhysicalPlan::BroadcastSource(_) - | PhysicalPlan::RecursiveCteScan(_) => Box::new(std::iter::empty()), - PhysicalPlan::HilbertPartition(plan) => Box::new(std::iter::once(plan.input.as_mut())), - PhysicalPlan::Filter(plan) => Box::new(std::iter::once(plan.input.as_mut())), - PhysicalPlan::EvalScalar(plan) => Box::new(std::iter::once(plan.input.as_mut())), - PhysicalPlan::AggregateExpand(plan) => Box::new(std::iter::once(plan.input.as_mut())), - PhysicalPlan::AggregatePartial(plan) => Box::new(std::iter::once(plan.input.as_mut())), - PhysicalPlan::AggregateFinal(plan) => Box::new(std::iter::once(plan.input.as_mut())), - PhysicalPlan::Window(plan) => Box::new(std::iter::once(plan.input.as_mut())), - PhysicalPlan::WindowPartition(plan) => Box::new(std::iter::once(plan.input.as_mut())), - PhysicalPlan::Sort(plan) => Box::new(std::iter::once(plan.input.as_mut())), - PhysicalPlan::Limit(plan) => Box::new(std::iter::once(plan.input.as_mut())), - PhysicalPlan::RowFetch(plan) => Box::new(std::iter::once(plan.input.as_mut())), - PhysicalPlan::HashJoin(plan) => Box::new( - std::iter::once(plan.probe.as_mut()).chain(std::iter::once(plan.build.as_mut())), - ), - PhysicalPlan::ExpressionScan(plan) => Box::new(std::iter::once(plan.input.as_mut())), - PhysicalPlan::Exchange(plan) => Box::new(std::iter::once(plan.input.as_mut())), - PhysicalPlan::ExchangeSink(plan) => Box::new(std::iter::once(plan.input.as_mut())), - PhysicalPlan::UnionAll(plan) => Box::new( - std::iter::once(plan.left.as_mut()).chain(std::iter::once(plan.right.as_mut())), - ), - PhysicalPlan::DistributedInsertSelect(plan) => { - Box::new(std::iter::once(plan.input.as_mut())) - } - PhysicalPlan::CommitSink(plan) => Box::new(std::iter::once(plan.input.as_mut())), - PhysicalPlan::ProjectSet(plan) => Box::new(std::iter::once(plan.input.as_mut())), - PhysicalPlan::RangeJoin(plan) => Box::new( - std::iter::once(plan.left.as_mut()).chain(std::iter::once(plan.right.as_mut())), - ), - PhysicalPlan::ReplaceDeduplicate(plan) => { - Box::new(std::iter::once(plan.input.as_mut())) - } - PhysicalPlan::ReplaceInto(plan) => Box::new(std::iter::once(plan.input.as_mut())), - PhysicalPlan::MutationSource(_) => Box::new(std::iter::empty()), - PhysicalPlan::ColumnMutation(plan) => Box::new(std::iter::once(plan.input.as_mut())), - PhysicalPlan::Mutation(plan) => Box::new(std::iter::once(plan.input.as_mut())), - PhysicalPlan::MutationSplit(plan) => Box::new(std::iter::once(plan.input.as_mut())), - PhysicalPlan::MutationManipulate(plan) => { - Box::new(std::iter::once(plan.input.as_mut())) - } - PhysicalPlan::MutationOrganize(plan) => Box::new(std::iter::once(plan.input.as_mut())), - PhysicalPlan::AddStreamColumn(plan) => Box::new(std::iter::once(plan.input.as_mut())), - PhysicalPlan::Udf(plan) => Box::new(std::iter::once(plan.input.as_mut())), - PhysicalPlan::AsyncFunction(plan) => Box::new(std::iter::once(plan.input.as_mut())), - PhysicalPlan::CopyIntoLocation(plan) => Box::new(std::iter::once(plan.input.as_mut())), - PhysicalPlan::Duplicate(plan) => Box::new(std::iter::once(plan.input.as_mut())), - PhysicalPlan::Shuffle(plan) => Box::new(std::iter::once(plan.input.as_mut())), - PhysicalPlan::ChunkFilter(plan) => Box::new(std::iter::once(plan.input.as_mut())), - PhysicalPlan::ChunkEvalScalar(plan) => Box::new(std::iter::once(plan.input.as_mut())), - PhysicalPlan::ChunkCastSchema(plan) => Box::new(std::iter::once(plan.input.as_mut())), - PhysicalPlan::ChunkFillAndReorder(plan) => { - Box::new(std::iter::once(plan.input.as_mut())) - } - PhysicalPlan::ChunkAppendData(plan) => Box::new(std::iter::once(plan.input.as_mut())), - PhysicalPlan::ChunkMerge(plan) => Box::new(std::iter::once(plan.input.as_mut())), - PhysicalPlan::ChunkCommitInsert(plan) => Box::new(std::iter::once(plan.input.as_mut())), - PhysicalPlan::CopyIntoTable(v) => match &mut v.source { - CopyIntoTableSource::Query(v) => Box::new(std::iter::once(v.as_mut())), - CopyIntoTableSource::Stage(v) => Box::new(std::iter::once(v.as_mut())), - }, - PhysicalPlan::BroadcastSink(plan) => Box::new(std::iter::once(plan.input.as_mut())), - } - } - - /// Used to find data source info in a non-aggregation and single-table query plan. - pub fn try_find_single_data_source(&self) -> Option<&DataSourcePlan> { - match self { - PhysicalPlan::TableScan(scan) => Some(&scan.source), - PhysicalPlan::Filter(plan) => plan.input.try_find_single_data_source(), - PhysicalPlan::EvalScalar(plan) => plan.input.try_find_single_data_source(), - PhysicalPlan::Window(plan) => plan.input.try_find_single_data_source(), - PhysicalPlan::WindowPartition(plan) => plan.input.try_find_single_data_source(), - PhysicalPlan::Sort(plan) => plan.input.try_find_single_data_source(), - 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(), - PhysicalPlan::AsyncFunction(plan) => plan.input.try_find_single_data_source(), - PhysicalPlan::CopyIntoLocation(plan) => plan.input.try_find_single_data_source(), - PhysicalPlan::UnionAll(_) - | PhysicalPlan::ExchangeSource(_) - | PhysicalPlan::HashJoin(_) - | PhysicalPlan::RangeJoin(_) - | PhysicalPlan::AggregateExpand(_) - | PhysicalPlan::AggregateFinal(_) - | PhysicalPlan::AggregatePartial(_) - | PhysicalPlan::CompactSource(_) - | PhysicalPlan::CommitSink(_) - | PhysicalPlan::CopyIntoTable(_) - | PhysicalPlan::ReplaceAsyncSourcer(_) - | PhysicalPlan::ReplaceDeduplicate(_) - | PhysicalPlan::ReplaceInto(_) - | PhysicalPlan::MutationSource(_) - | PhysicalPlan::ColumnMutation(_) - | PhysicalPlan::Mutation(_) - | PhysicalPlan::MutationSplit(_) - | PhysicalPlan::MutationManipulate(_) - | PhysicalPlan::MutationOrganize(_) - | PhysicalPlan::AddStreamColumn(_) - | PhysicalPlan::ConstantTableScan(_) - | PhysicalPlan::ExpressionScan(_) - | PhysicalPlan::CacheScan(_) - | PhysicalPlan::RecursiveCteScan(_) - | PhysicalPlan::Recluster(_) - | PhysicalPlan::HilbertPartition(_) - | PhysicalPlan::Duplicate(_) - | PhysicalPlan::Shuffle(_) - | PhysicalPlan::ChunkFilter(_) - | PhysicalPlan::ChunkEvalScalar(_) - | PhysicalPlan::ChunkCastSchema(_) - | PhysicalPlan::ChunkFillAndReorder(_) - | PhysicalPlan::ChunkAppendData(_) - | PhysicalPlan::ChunkMerge(_) - | PhysicalPlan::ChunkCommitInsert(_) - | PhysicalPlan::BroadcastSource(_) - | PhysicalPlan::BroadcastSink(_) => None, - } - } - - #[recursive::recursive] - pub fn is_distributed_plan(&self) -> bool { - self.children().any(|child| child.is_distributed_plan()) - || matches!( - self, - Self::ExchangeSource(_) | Self::ExchangeSink(_) | Self::Exchange(_) - ) - } - - #[recursive::recursive] - pub fn is_warehouse_distributed_plan(&self) -> bool { - self.children() - .any(|child| child.is_warehouse_distributed_plan()) - || matches!(self, Self::TableScan(v) if v.source.parts.kind == PartitionsShuffleKind::BroadcastWarehouse) - } - - pub fn get_desc(&self) -> Result { - Ok(match self { - PhysicalPlan::TableScan(v) => format!( - "{}.{}", - v.source.source_info.catalog_name(), - v.source.source_info.desc() - ), - PhysicalPlan::Filter(v) => match v.predicates.is_empty() { - true => String::new(), - false => v.predicates[0].as_expr(&BUILTIN_FUNCTIONS).sql_display(), - }, - PhysicalPlan::AggregatePartial(v) => { - v.agg_funcs.iter().map(|x| x.display.clone()).join(", ") - } - PhysicalPlan::AggregateFinal(v) => { - v.agg_funcs.iter().map(|x| x.display.clone()).join(", ") - } - PhysicalPlan::Sort(v) => v - .order_by - .iter() - .map(|x| { - format!( - "{}{}{}", - x.display_name, - if x.asc { "" } else { " DESC" }, - if x.nulls_first { " NULLS FIRST" } else { "" }, - ) - }) - .join(", "), - PhysicalPlan::Limit(v) => match v.limit { - Some(limit) => format!("LIMIT {} OFFSET {}", limit, v.offset), - None => format!("OFFSET {}", v.offset), - }, - PhysicalPlan::EvalScalar(v) => v - .exprs - .iter() - .map(|(x, _)| x.as_expr(&BUILTIN_FUNCTIONS).sql_display()) - .join(", "), - PhysicalPlan::HashJoin(v) => { - let mut conditions = v - .build_keys - .iter() - .zip(v.probe_keys.iter()) - .map(|(l, r)| { - format!( - "({} = {})", - l.as_expr(&BUILTIN_FUNCTIONS).sql_display(), - r.as_expr(&BUILTIN_FUNCTIONS).sql_display() - ) - }) - .collect::>(); - - conditions.extend( - v.non_equi_conditions - .iter() - .map(|x| x.as_expr(&BUILTIN_FUNCTIONS).sql_display()), - ); - - conditions.join(" AND ") - } - PhysicalPlan::ProjectSet(v) => v - .srf_exprs - .iter() - .map(|(x, _)| x.as_expr(&BUILTIN_FUNCTIONS).sql_display()) - .join(", "), - PhysicalPlan::AggregateExpand(v) => v - .grouping_sets - .sets - .iter() - .map(|set| { - set.iter() - .map(|x| x.to_string()) - .collect::>() - .join(", ") - }) - .map(|s| format!("({})", s)) - .collect::>() - .join(", "), - PhysicalPlan::Window(v) => { - let partition_by = v - .partition_by - .iter() - .map(|x| x.to_string()) - .collect::>() - .join(", "); - - let order_by = v - .order_by - .iter() - .map(|x| { - format!( - "{}{}{}", - x.display_name, - if x.asc { "" } else { " DESC" }, - if x.nulls_first { " NULLS FIRST" } else { "" }, - ) - }) - .collect::>() - .join(", "); - - format!("partition by {}, order by {}", partition_by, order_by) - } - PhysicalPlan::RowFetch(v) => { - let table_schema = v.source.source_info.schema(); - let projected_schema = v.cols_to_fetch.project_schema(&table_schema); - projected_schema.fields.iter().map(|f| f.name()).join(", ") - } - PhysicalPlan::RangeJoin(v) => { - let mut condition = v - .conditions - .iter() - .map(|condition| { - let left = condition - .left_expr - .as_expr(&BUILTIN_FUNCTIONS) - .sql_display(); - let right = condition - .right_expr - .as_expr(&BUILTIN_FUNCTIONS) - .sql_display(); - format!("{left} {:?} {right}", condition.operator) - }) - .collect::>(); - - condition.extend( - v.other_conditions - .iter() - .map(|x| x.as_expr(&BUILTIN_FUNCTIONS).sql_display()), - ); - - condition.join(" AND ") - } - PhysicalPlan::Udf(v) => v - .udf_funcs - .iter() - .map(|x| format!("{}({})", x.func_name, x.arg_exprs.join(", "))) - .join(", "), - PhysicalPlan::UnionAll(v) => v - .left_outputs - .iter() - .zip(v.right_outputs.iter()) - .map(|(l, r)| format!("#{} <- #{}", l.0, r.0)) - .join(", "), - PhysicalPlan::AsyncFunction(v) => v - .async_func_descs - .iter() - .map(|x| x.display_name.clone()) - .join(", "), - _ => String::new(), - }) - } - - pub fn get_labels(&self) -> Result>> { - let mut labels = HashMap::with_capacity(16); - - match self { - PhysicalPlan::TableScan(v) => { - labels.insert(String::from("Full table name"), vec![format!( - "{}.{}", - v.source.source_info.catalog_name(), - v.source.source_info.desc() - )]); - - labels.insert( - format!( - "Columns ({} / {})", - v.output_schema()?.num_fields(), - std::cmp::max( - v.output_schema()?.num_fields(), - v.source.source_info.schema().num_fields(), - ) - ), - v.name_mapping.keys().cloned().collect(), - ); - labels.insert(String::from("Total partitions"), vec![v - .source - .statistics - .partitions_total - .to_string()]); - } - PhysicalPlan::Filter(v) => { - labels.insert( - String::from("Filter condition"), - v.predicates - .iter() - .map(|x| x.as_expr(&BUILTIN_FUNCTIONS).sql_display()) - .collect(), - ); - } - PhysicalPlan::Limit(v) => { - labels.insert(String::from("Offset"), vec![v.offset.to_string()]); - - if let Some(limit) = v.limit { - labels.insert(String::from("Number of rows"), vec![limit.to_string()]); - } - } - PhysicalPlan::EvalScalar(v) => { - labels.insert( - String::from("List of Expressions"), - v.exprs - .iter() - .map(|(x, _)| x.as_expr(&BUILTIN_FUNCTIONS).sql_display()) - .collect(), - ); - } - PhysicalPlan::AggregatePartial(v) => { - if !v.group_by_display.is_empty() { - labels.insert(String::from("Grouping keys"), v.group_by_display.clone()); - } - - if !v.agg_funcs.is_empty() { - labels.insert( - String::from("Aggregate Functions"), - v.agg_funcs.iter().map(|x| x.display.clone()).collect(), - ); - } - } - PhysicalPlan::AggregateFinal(v) => { - if !v.group_by_display.is_empty() { - labels.insert(String::from("Grouping keys"), v.group_by_display.clone()); - } - - if !v.agg_funcs.is_empty() { - labels.insert( - String::from("Aggregate Functions"), - v.agg_funcs.iter().map(|x| x.display.clone()).collect(), - ); - } - } - PhysicalPlan::HashJoin(v) => { - labels.insert(String::from("Join Type"), vec![v.join_type.to_string()]); - - if !v.build_keys.is_empty() { - labels.insert( - String::from("Join Build Side Keys"), - v.build_keys - .iter() - .map(|x| x.as_expr(&BUILTIN_FUNCTIONS).sql_display()) - .collect(), - ); - } - - if !v.probe_keys.is_empty() { - labels.insert( - String::from("Join Probe Side Keys"), - v.probe_keys - .iter() - .map(|x| x.as_expr(&BUILTIN_FUNCTIONS).sql_display()) - .collect(), - ); - } - - if !v.non_equi_conditions.is_empty() { - labels.insert( - String::from("Join Conditions"), - v.non_equi_conditions - .iter() - .map(|x| x.as_expr(&BUILTIN_FUNCTIONS).sql_display()) - .collect(), - ); - } - } - _ => {} - }; - - Ok(labels) - } - - #[recursive::recursive] - pub fn try_find_mutation_source(&self) -> Option { - match self { - PhysicalPlan::MutationSource(mutation_source) => Some(mutation_source.clone()), - _ => { - for child in self.children() { - if let Some(plan) = child.try_find_mutation_source() { - return Some(plan); - } - } - None - } - } - } - - #[recursive::recursive] - pub fn get_all_data_source(&self, sources: &mut Vec<(u32, Box)>) { - match self { - PhysicalPlan::TableScan(table_scan) => { - sources.push((table_scan.plan_id, table_scan.source.clone())); - } - _ => { - for child in self.children() { - child.get_all_data_source(sources); - } - } - } - } - - pub fn set_pruning_stats(&mut self, stats: &mut HashMap) { - match self { - PhysicalPlan::TableScan(table_scan) => { - if let Some(stat) = stats.remove(&table_scan.plan_id) { - table_scan.source.statistics = stat; - } - } - _ => { - for child in self.children_mut() { - child.set_pruning_stats(stats) - } - } - } - } -} diff --git a/src/query/sql/src/executor/physical_plan_visitor.rs b/src/query/sql/src/executor/physical_plan_visitor.rs deleted file mode 100644 index 7267b7665e370..0000000000000 --- a/src/query/sql/src/executor/physical_plan_visitor.rs +++ /dev/null @@ -1,802 +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 super::physical_plans::AddStreamColumn; -use super::physical_plans::BroadcastSink; -use super::physical_plans::BroadcastSource; -use super::physical_plans::CacheScan; -use super::physical_plans::ExpressionScan; -use super::physical_plans::HilbertPartition; -use super::physical_plans::MutationManipulate; -use super::physical_plans::MutationOrganize; -use super::physical_plans::MutationSplit; -use super::physical_plans::RecursiveCteScan; -use crate::executor::physical_plan::PhysicalPlan; -use crate::executor::physical_plans::AggregateExpand; -use crate::executor::physical_plans::AggregateFinal; -use crate::executor::physical_plans::AggregatePartial; -use crate::executor::physical_plans::AsyncFunction; -use crate::executor::physical_plans::ChunkAppendData; -use crate::executor::physical_plans::ChunkCastSchema; -use crate::executor::physical_plans::ChunkCommitInsert; -use crate::executor::physical_plans::ChunkEvalScalar; -use crate::executor::physical_plans::ChunkFillAndReorder; -use crate::executor::physical_plans::ChunkFilter; -use crate::executor::physical_plans::ChunkMerge; -use crate::executor::physical_plans::ColumnMutation; -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::DistributedInsertSelect; -use crate::executor::physical_plans::Duplicate; -use crate::executor::physical_plans::EvalScalar; -use crate::executor::physical_plans::Exchange; -use crate::executor::physical_plans::ExchangeSink; -use crate::executor::physical_plans::ExchangeSource; -use crate::executor::physical_plans::Filter; -use crate::executor::physical_plans::HashJoin; -use crate::executor::physical_plans::Limit; -use crate::executor::physical_plans::Mutation; -use crate::executor::physical_plans::MutationSource; -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; -use crate::executor::physical_plans::Shuffle; -use crate::executor::physical_plans::Sort; -use crate::executor::physical_plans::TableScan; -use crate::executor::physical_plans::Udf; -use crate::executor::physical_plans::UnionAll; -use crate::executor::physical_plans::Window; -use crate::executor::physical_plans::WindowPartition; - -pub trait PhysicalPlanReplacer { - fn replace(&mut self, plan: &PhysicalPlan) -> Result { - match plan { - PhysicalPlan::TableScan(plan) => self.replace_table_scan(plan), - PhysicalPlan::RecursiveCteScan(plan) => self.replace_recursive_cte_scan(plan), - PhysicalPlan::Filter(plan) => self.replace_filter(plan), - PhysicalPlan::EvalScalar(plan) => self.replace_eval_scalar(plan), - PhysicalPlan::AggregateExpand(plan) => self.replace_aggregate_expand(plan), - PhysicalPlan::AggregatePartial(plan) => self.replace_aggregate_partial(plan), - PhysicalPlan::AggregateFinal(plan) => self.replace_aggregate_final(plan), - PhysicalPlan::Window(plan) => self.replace_window(plan), - PhysicalPlan::WindowPartition(plan) => self.replace_window_partition(plan), - PhysicalPlan::Sort(plan) => self.replace_sort(plan), - PhysicalPlan::Limit(plan) => self.replace_limit(plan), - PhysicalPlan::RowFetch(plan) => self.replace_row_fetch(plan), - PhysicalPlan::HashJoin(plan) => self.replace_hash_join(plan), - PhysicalPlan::Exchange(plan) => self.replace_exchange(plan), - 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::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), - PhysicalPlan::ColumnMutation(plan) => self.replace_column_mutation(plan), - PhysicalPlan::Mutation(plan) => self.replace_mutation(plan), - PhysicalPlan::MutationSplit(plan) => self.replace_mutation_split(plan), - PhysicalPlan::MutationManipulate(plan) => self.replace_mutation_manipulate(plan), - PhysicalPlan::MutationOrganize(plan) => self.replace_mutation_organize(plan), - PhysicalPlan::AddStreamColumn(plan) => self.replace_add_stream_column(plan), - PhysicalPlan::ConstantTableScan(plan) => self.replace_constant_table_scan(plan), - PhysicalPlan::ExpressionScan(plan) => self.replace_expression_scan(plan), - PhysicalPlan::CacheScan(plan) => self.replace_cache_scan(plan), - PhysicalPlan::Recluster(plan) => self.replace_recluster(plan), - PhysicalPlan::HilbertPartition(plan) => self.replace_hilbert_serialize(plan), - PhysicalPlan::Udf(plan) => self.replace_udf(plan), - PhysicalPlan::AsyncFunction(plan) => self.replace_async_function(plan), - PhysicalPlan::Duplicate(plan) => self.replace_duplicate(plan), - PhysicalPlan::Shuffle(plan) => self.replace_shuffle(plan), - PhysicalPlan::ChunkFilter(plan) => self.replace_chunk_filter(plan), - PhysicalPlan::ChunkEvalScalar(plan) => self.replace_chunk_eval_scalar(plan), - PhysicalPlan::ChunkCastSchema(plan) => self.replace_chunk_cast_schema(plan), - PhysicalPlan::ChunkFillAndReorder(plan) => self.replace_chunk_fill_and_reorder(plan), - 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::BroadcastSource(plan) => self.replace_runtime_filter_source(plan), - PhysicalPlan::BroadcastSink(plan) => self.replace_runtime_filter_sink(plan), - } - } - - fn replace_runtime_filter_source(&mut self, plan: &BroadcastSource) -> Result { - Ok(PhysicalPlan::BroadcastSource(plan.clone())) - } - - fn replace_runtime_filter_sink(&mut self, plan: &BroadcastSink) -> Result { - let input = self.replace(&plan.input)?; - Ok(PhysicalPlan::BroadcastSink(BroadcastSink { - plan_id: plan.plan_id, - broadcast_id: plan.broadcast_id, - input: Box::new(input), - })) - } - - fn replace_recluster(&mut self, plan: &Recluster) -> Result { - Ok(PhysicalPlan::Recluster(Box::new(plan.clone()))) - } - - fn replace_hilbert_serialize(&mut self, plan: &HilbertPartition) -> Result { - let input = self.replace(&plan.input)?; - Ok(PhysicalPlan::HilbertPartition(Box::new(HilbertPartition { - input: Box::new(input), - ..plan.clone() - }))) - } - - fn replace_table_scan(&mut self, plan: &TableScan) -> Result { - Ok(PhysicalPlan::TableScan(plan.clone())) - } - - fn replace_recursive_cte_scan(&mut self, plan: &RecursiveCteScan) -> Result { - Ok(PhysicalPlan::RecursiveCteScan(plan.clone())) - } - - fn replace_constant_table_scan(&mut self, plan: &ConstantTableScan) -> Result { - Ok(PhysicalPlan::ConstantTableScan(plan.clone())) - } - - fn replace_expression_scan(&mut self, plan: &ExpressionScan) -> Result { - Ok(PhysicalPlan::ExpressionScan(plan.clone())) - } - - fn replace_cache_scan(&mut self, plan: &CacheScan) -> Result { - Ok(PhysicalPlan::CacheScan(plan.clone())) - } - - fn replace_filter(&mut self, plan: &Filter) -> Result { - let input = self.replace(&plan.input)?; - - Ok(PhysicalPlan::Filter(Filter { - plan_id: plan.plan_id, - projections: plan.projections.clone(), - input: Box::new(input), - predicates: plan.predicates.clone(), - stat_info: plan.stat_info.clone(), - })) - } - - fn replace_eval_scalar(&mut self, plan: &EvalScalar) -> Result { - let input = self.replace(&plan.input)?; - - Ok(PhysicalPlan::EvalScalar(EvalScalar { - plan_id: plan.plan_id, - projections: plan.projections.clone(), - input: Box::new(input), - exprs: plan.exprs.clone(), - stat_info: plan.stat_info.clone(), - })) - } - - fn replace_aggregate_expand(&mut self, plan: &AggregateExpand) -> Result { - let input = self.replace(&plan.input)?; - - Ok(PhysicalPlan::AggregateExpand(AggregateExpand { - plan_id: plan.plan_id, - input: Box::new(input), - group_bys: plan.group_bys.clone(), - grouping_sets: plan.grouping_sets.clone(), - stat_info: plan.stat_info.clone(), - })) - } - - fn replace_aggregate_partial(&mut self, plan: &AggregatePartial) -> Result { - let input = self.replace(&plan.input)?; - - Ok(PhysicalPlan::AggregatePartial(AggregatePartial { - plan_id: plan.plan_id, - input: Box::new(input), - enable_experimental_aggregate_hashtable: plan.enable_experimental_aggregate_hashtable, - group_by: plan.group_by.clone(), - group_by_display: plan.group_by_display.clone(), - agg_funcs: plan.agg_funcs.clone(), - stat_info: plan.stat_info.clone(), - rank_limit: plan.rank_limit.clone(), - })) - } - - fn replace_aggregate_final(&mut self, plan: &AggregateFinal) -> Result { - let input = self.replace(&plan.input)?; - - Ok(PhysicalPlan::AggregateFinal(AggregateFinal { - plan_id: plan.plan_id, - input: Box::new(input), - before_group_by_schema: plan.before_group_by_schema.clone(), - group_by: plan.group_by.clone(), - agg_funcs: plan.agg_funcs.clone(), - group_by_display: plan.group_by_display.clone(), - stat_info: plan.stat_info.clone(), - })) - } - - fn replace_window(&mut self, plan: &Window) -> Result { - let input = self.replace(&plan.input)?; - - Ok(PhysicalPlan::Window(Window { - plan_id: plan.plan_id, - index: plan.index, - input: Box::new(input), - func: plan.func.clone(), - partition_by: plan.partition_by.clone(), - order_by: plan.order_by.clone(), - window_frame: plan.window_frame.clone(), - limit: plan.limit, - })) - } - - fn replace_window_partition(&mut self, plan: &WindowPartition) -> Result { - let input = self.replace(&plan.input)?; - - Ok(PhysicalPlan::WindowPartition(WindowPartition { - plan_id: plan.plan_id, - input: Box::new(input), - partition_by: plan.partition_by.clone(), - order_by: plan.order_by.clone(), - after_exchange: plan.after_exchange, - top_n: plan.top_n.clone(), - stat_info: plan.stat_info.clone(), - })) - } - - fn replace_hash_join(&mut self, plan: &HashJoin) -> Result { - let build = self.replace(&plan.build)?; - let probe = self.replace(&plan.probe)?; - - Ok(PhysicalPlan::HashJoin(HashJoin { - plan_id: plan.plan_id, - projections: plan.projections.clone(), - probe_projections: plan.probe_projections.clone(), - build_projections: plan.build_projections.clone(), - build: Box::new(build), - probe: Box::new(probe), - build_keys: plan.build_keys.clone(), - probe_keys: plan.probe_keys.clone(), - is_null_equal: plan.is_null_equal.clone(), - non_equi_conditions: plan.non_equi_conditions.clone(), - join_type: plan.join_type.clone(), - marker_index: plan.marker_index, - from_correlated_subquery: plan.from_correlated_subquery, - probe_to_build: plan.probe_to_build.clone(), - output_schema: plan.output_schema.clone(), - need_hold_hash_table: plan.need_hold_hash_table, - stat_info: plan.stat_info.clone(), - runtime_filter: plan.runtime_filter.clone(), - single_to_inner: plan.single_to_inner.clone(), - build_side_cache_info: plan.build_side_cache_info.clone(), - broadcast_id: plan.broadcast_id, - })) - } - - fn replace_range_join(&mut self, plan: &RangeJoin) -> Result { - let left = self.replace(&plan.left)?; - let right = self.replace(&plan.right)?; - - Ok(PhysicalPlan::RangeJoin(RangeJoin { - plan_id: plan.plan_id, - left: Box::new(left), - right: Box::new(right), - conditions: plan.conditions.clone(), - other_conditions: plan.other_conditions.clone(), - join_type: plan.join_type.clone(), - range_join_type: plan.range_join_type.clone(), - output_schema: plan.output_schema.clone(), - stat_info: plan.stat_info.clone(), - })) - } - - fn replace_sort(&mut self, plan: &Sort) -> Result { - let input = self.replace(&plan.input)?; - - Ok(PhysicalPlan::Sort(Sort { - plan_id: plan.plan_id, - input: Box::new(input), - order_by: plan.order_by.clone(), - limit: plan.limit, - after_exchange: plan.after_exchange, - pre_projection: plan.pre_projection.clone(), - stat_info: plan.stat_info.clone(), - })) - } - - fn replace_limit(&mut self, plan: &Limit) -> Result { - let input = self.replace(&plan.input)?; - - Ok(PhysicalPlan::Limit(Limit { - plan_id: plan.plan_id, - input: Box::new(input), - limit: plan.limit, - offset: plan.offset, - stat_info: plan.stat_info.clone(), - })) - } - - fn replace_row_fetch(&mut self, plan: &RowFetch) -> Result { - let input = self.replace(&plan.input)?; - - Ok(PhysicalPlan::RowFetch(RowFetch { - plan_id: plan.plan_id, - input: Box::new(input), - source: plan.source.clone(), - row_id_col_offset: plan.row_id_col_offset, - cols_to_fetch: plan.cols_to_fetch.clone(), - fetched_fields: plan.fetched_fields.clone(), - stat_info: plan.stat_info.clone(), - need_wrap_nullable: plan.need_wrap_nullable, - })) - } - - fn replace_exchange(&mut self, plan: &Exchange) -> Result { - let input = self.replace(&plan.input)?; - - Ok(PhysicalPlan::Exchange(Exchange { - plan_id: plan.plan_id, - input: Box::new(input), - kind: plan.kind.clone(), - keys: plan.keys.clone(), - ignore_exchange: plan.ignore_exchange, - allow_adjust_parallelism: plan.allow_adjust_parallelism, - })) - } - - fn replace_exchange_source(&mut self, plan: &ExchangeSource) -> Result { - Ok(PhysicalPlan::ExchangeSource(plan.clone())) - } - - fn replace_exchange_sink(&mut self, plan: &ExchangeSink) -> Result { - let input = self.replace(&plan.input)?; - - Ok(PhysicalPlan::ExchangeSink(ExchangeSink { - // TODO(leiysky): we reuse the plan id of the Exchange node here, - // should generate a new one. - plan_id: plan.plan_id, - - input: Box::new(input), - schema: plan.schema.clone(), - kind: plan.kind.clone(), - keys: plan.keys.clone(), - destination_fragment_id: plan.destination_fragment_id, - query_id: plan.query_id.clone(), - ignore_exchange: plan.ignore_exchange, - allow_adjust_parallelism: plan.allow_adjust_parallelism, - })) - } - - fn replace_union(&mut self, plan: &UnionAll) -> Result { - let left = self.replace(&plan.left)?; - let right = self.replace(&plan.right)?; - Ok(PhysicalPlan::UnionAll(UnionAll { - plan_id: plan.plan_id, - left: Box::new(left), - right: Box::new(right), - left_outputs: plan.left_outputs.clone(), - right_outputs: plan.right_outputs.clone(), - schema: plan.schema.clone(), - stat_info: plan.stat_info.clone(), - cte_scan_names: plan.cte_scan_names.clone(), - })) - } - - fn replace_copy_into_table(&mut self, plan: &CopyIntoTable) -> Result { - match &plan.source { - CopyIntoTableSource::Stage(_) => { - Ok(PhysicalPlan::CopyIntoTable(Box::new(plan.clone()))) - } - CopyIntoTableSource::Query(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_copy_into_location(&mut self, plan: &CopyIntoLocation) -> Result { - let input = self.replace(&plan.input)?; - - Ok(PhysicalPlan::CopyIntoLocation(Box::new(CopyIntoLocation { - plan_id: plan.plan_id, - input: Box::new(input), - project_columns: plan.project_columns.clone(), - input_data_schema: plan.input_data_schema.clone(), - input_table_schema: plan.input_table_schema.clone(), - info: plan.info.clone(), - }))) - } - - fn replace_insert_select(&mut self, plan: &DistributedInsertSelect) -> Result { - let input = self.replace(&plan.input)?; - - Ok(PhysicalPlan::DistributedInsertSelect(Box::new( - DistributedInsertSelect { - input: Box::new(input), - ..plan.clone() - }, - ))) - } - - fn replace_compact_source(&mut self, plan: &CompactSource) -> Result { - Ok(PhysicalPlan::CompactSource(Box::new(plan.clone()))) - } - - fn replace_commit_sink(&mut self, plan: &CommitSink) -> Result { - let input = self.replace(&plan.input)?; - Ok(PhysicalPlan::CommitSink(Box::new(CommitSink { - input: Box::new(input), - ..plan.clone() - }))) - } - - 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( - ReplaceDeduplicate { - input: Box::new(input), - ..plan.clone() - }, - ))) - } - - fn replace_replace_into(&mut self, plan: &ReplaceInto) -> Result { - let input = self.replace(&plan.input)?; - Ok(PhysicalPlan::ReplaceInto(Box::new(ReplaceInto { - input: Box::new(input), - ..plan.clone() - }))) - } - - fn replace_mutation_source(&mut self, plan: &MutationSource) -> Result { - Ok(PhysicalPlan::MutationSource(plan.clone())) - } - - fn replace_column_mutation(&mut self, plan: &ColumnMutation) -> Result { - let input = self.replace(&plan.input)?; - Ok(PhysicalPlan::ColumnMutation(ColumnMutation { - input: Box::new(input), - ..plan.clone() - })) - } - - fn replace_mutation(&mut self, plan: &Mutation) -> Result { - let input = self.replace(&plan.input)?; - Ok(PhysicalPlan::Mutation(Box::new(Mutation { - input: Box::new(input), - ..plan.clone() - }))) - } - - fn replace_mutation_split(&mut self, plan: &MutationSplit) -> Result { - let input = self.replace(&plan.input)?; - Ok(PhysicalPlan::MutationSplit(Box::new(MutationSplit { - input: Box::new(input), - ..plan.clone() - }))) - } - - fn replace_mutation_manipulate(&mut self, plan: &MutationManipulate) -> Result { - let input = self.replace(&plan.input)?; - Ok(PhysicalPlan::MutationManipulate(Box::new( - MutationManipulate { - input: Box::new(input), - ..plan.clone() - }, - ))) - } - - fn replace_mutation_organize(&mut self, plan: &MutationOrganize) -> Result { - let input = self.replace(&plan.input)?; - Ok(PhysicalPlan::MutationOrganize(Box::new(MutationOrganize { - input: Box::new(input), - ..plan.clone() - }))) - } - - fn replace_add_stream_column(&mut self, plan: &AddStreamColumn) -> Result { - let input = self.replace(&plan.input)?; - Ok(PhysicalPlan::AddStreamColumn(Box::new(AddStreamColumn { - input: Box::new(input), - ..plan.clone() - }))) - } - - fn replace_project_set(&mut self, plan: &ProjectSet) -> Result { - let input = self.replace(&plan.input)?; - Ok(PhysicalPlan::ProjectSet(ProjectSet { - plan_id: plan.plan_id, - input: Box::new(input), - srf_exprs: plan.srf_exprs.clone(), - projections: plan.projections.clone(), - stat_info: plan.stat_info.clone(), - })) - } - - fn replace_udf(&mut self, plan: &Udf) -> Result { - let input = self.replace(&plan.input)?; - Ok(PhysicalPlan::Udf(Udf { - plan_id: plan.plan_id, - input: Box::new(input), - udf_funcs: plan.udf_funcs.clone(), - stat_info: plan.stat_info.clone(), - script_udf: plan.script_udf, - })) - } - - fn replace_async_function(&mut self, plan: &AsyncFunction) -> Result { - let input = self.replace(&plan.input)?; - Ok(PhysicalPlan::AsyncFunction(AsyncFunction { - plan_id: plan.plan_id, - input: Box::new(input), - async_func_descs: plan.async_func_descs.clone(), - stat_info: plan.stat_info.clone(), - })) - } - - fn replace_duplicate(&mut self, plan: &Duplicate) -> Result { - let input = self.replace(&plan.input)?; - Ok(PhysicalPlan::Duplicate(Box::new(Duplicate { - input: Box::new(input), - ..plan.clone() - }))) - } - - fn replace_shuffle(&mut self, plan: &Shuffle) -> Result { - let input = self.replace(&plan.input)?; - Ok(PhysicalPlan::Shuffle(Box::new(Shuffle { - input: Box::new(input), - ..plan.clone() - }))) - } - - fn replace_chunk_filter(&mut self, plan: &ChunkFilter) -> Result { - let input = self.replace(&plan.input)?; - Ok(PhysicalPlan::ChunkFilter(Box::new(ChunkFilter { - input: Box::new(input), - predicates: plan.predicates.clone(), - ..plan.clone() - }))) - } - - fn replace_chunk_eval_scalar(&mut self, plan: &ChunkEvalScalar) -> Result { - let input = self.replace(&plan.input)?; - Ok(PhysicalPlan::ChunkEvalScalar(Box::new(ChunkEvalScalar { - input: Box::new(input), - ..plan.clone() - }))) - } - - fn replace_chunk_cast_schema(&mut self, plan: &ChunkCastSchema) -> Result { - let input = self.replace(&plan.input)?; - Ok(PhysicalPlan::ChunkCastSchema(Box::new(ChunkCastSchema { - input: Box::new(input), - cast_schemas: plan.cast_schemas.clone(), - ..plan.clone() - }))) - } - - fn replace_chunk_fill_and_reorder( - &mut self, - plan: &ChunkFillAndReorder, - ) -> Result { - let input = self.replace(&plan.input)?; - Ok(PhysicalPlan::ChunkFillAndReorder(Box::new( - ChunkFillAndReorder { - input: Box::new(input), - ..plan.clone() - }, - ))) - } - - fn replace_chunk_append_data(&mut self, plan: &ChunkAppendData) -> Result { - let input = self.replace(&plan.input)?; - Ok(PhysicalPlan::ChunkAppendData(Box::new(ChunkAppendData { - input: Box::new(input), - ..plan.clone() - }))) - } - - fn replace_chunk_merge(&mut self, plan: &ChunkMerge) -> Result { - let input = self.replace(&plan.input)?; - Ok(PhysicalPlan::ChunkMerge(Box::new(ChunkMerge { - input: Box::new(input), - ..plan.clone() - }))) - } - - fn replace_chunk_commit_insert(&mut self, plan: &ChunkCommitInsert) -> Result { - let input = self.replace(&plan.input)?; - Ok(PhysicalPlan::ChunkCommitInsert(Box::new( - ChunkCommitInsert { - input: Box::new(input), - ..plan.clone() - }, - ))) - } -} - -impl PhysicalPlan { - pub fn traverse<'a, 'b>( - plan: &'a PhysicalPlan, - pre_visit: &'b mut dyn FnMut(&'a PhysicalPlan) -> bool, - visit: &'b mut dyn FnMut(&'a PhysicalPlan), - post_visit: &'b mut dyn FnMut(&'a PhysicalPlan), - ) { - if pre_visit(plan) { - visit(plan); - match plan { - PhysicalPlan::TableScan(_) - | PhysicalPlan::ReplaceAsyncSourcer(_) - | PhysicalPlan::RecursiveCteScan(_) - | PhysicalPlan::ConstantTableScan(_) - | PhysicalPlan::ExpressionScan(_) - | PhysicalPlan::CacheScan(_) - | PhysicalPlan::Recluster(_) - | PhysicalPlan::HilbertPartition(_) - | PhysicalPlan::ExchangeSource(_) - | PhysicalPlan::CompactSource(_) - | PhysicalPlan::MutationSource(_) - | PhysicalPlan::BroadcastSource(_) => {} - PhysicalPlan::Filter(plan) => { - Self::traverse(&plan.input, pre_visit, visit, post_visit); - } - PhysicalPlan::EvalScalar(plan) => { - Self::traverse(&plan.input, pre_visit, visit, post_visit); - } - PhysicalPlan::AggregateExpand(plan) => { - Self::traverse(&plan.input, pre_visit, visit, post_visit); - } - PhysicalPlan::AggregatePartial(plan) => { - Self::traverse(&plan.input, pre_visit, visit, post_visit); - } - PhysicalPlan::AggregateFinal(plan) => { - Self::traverse(&plan.input, pre_visit, visit, post_visit); - } - PhysicalPlan::Window(plan) => { - Self::traverse(&plan.input, pre_visit, visit, post_visit); - } - PhysicalPlan::WindowPartition(plan) => { - Self::traverse(&plan.input, pre_visit, visit, post_visit); - } - PhysicalPlan::Sort(plan) => { - Self::traverse(&plan.input, pre_visit, visit, post_visit); - } - PhysicalPlan::Limit(plan) => { - Self::traverse(&plan.input, pre_visit, visit, post_visit); - } - PhysicalPlan::RowFetch(plan) => { - Self::traverse(&plan.input, pre_visit, visit, post_visit); - } - PhysicalPlan::HashJoin(plan) => { - Self::traverse(&plan.build, pre_visit, visit, post_visit); - Self::traverse(&plan.probe, pre_visit, visit, post_visit); - } - PhysicalPlan::Exchange(plan) => { - Self::traverse(&plan.input, pre_visit, visit, post_visit); - } - PhysicalPlan::ExchangeSink(plan) => { - Self::traverse(&plan.input, pre_visit, visit, post_visit); - } - PhysicalPlan::UnionAll(plan) => { - 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::CopyIntoLocation(plan) => { - Self::traverse(&plan.input, pre_visit, visit, post_visit) - } - PhysicalPlan::RangeJoin(plan) => { - Self::traverse(&plan.left, pre_visit, visit, post_visit); - Self::traverse(&plan.right, pre_visit, visit, post_visit); - } - PhysicalPlan::CommitSink(plan) => { - Self::traverse(&plan.input, pre_visit, visit, post_visit); - } - PhysicalPlan::ReplaceDeduplicate(plan) => { - Self::traverse(&plan.input, pre_visit, visit, post_visit); - } - PhysicalPlan::ReplaceInto(plan) => { - Self::traverse(&plan.input, pre_visit, visit, post_visit); - } - PhysicalPlan::ColumnMutation(plan) => { - Self::traverse(&plan.input, pre_visit, visit, post_visit); - } - PhysicalPlan::Mutation(plan) => { - Self::traverse(&plan.input, pre_visit, visit, post_visit); - } - PhysicalPlan::MutationSplit(plan) => { - Self::traverse(&plan.input, pre_visit, visit, post_visit); - } - PhysicalPlan::MutationManipulate(plan) => { - Self::traverse(&plan.input, pre_visit, visit, post_visit); - } - PhysicalPlan::MutationOrganize(plan) => { - Self::traverse(&plan.input, pre_visit, visit, post_visit); - } - PhysicalPlan::AddStreamColumn(plan) => { - Self::traverse(&plan.input, pre_visit, visit, post_visit); - } - PhysicalPlan::Udf(plan) => { - Self::traverse(&plan.input, pre_visit, visit, post_visit); - } - PhysicalPlan::AsyncFunction(plan) => { - Self::traverse(&plan.input, pre_visit, visit, post_visit); - } - PhysicalPlan::Duplicate(plan) => { - Self::traverse(&plan.input, pre_visit, visit, post_visit); - } - PhysicalPlan::Shuffle(plan) => { - Self::traverse(&plan.input, pre_visit, visit, post_visit); - } - PhysicalPlan::ChunkFilter(plan) => { - Self::traverse(&plan.input, pre_visit, visit, post_visit); - } - PhysicalPlan::ChunkEvalScalar(plan) => { - Self::traverse(&plan.input, pre_visit, visit, post_visit); - } - PhysicalPlan::ChunkCastSchema(plan) => { - Self::traverse(&plan.input, pre_visit, visit, post_visit); - } - PhysicalPlan::ChunkFillAndReorder(plan) => { - Self::traverse(&plan.input, pre_visit, visit, post_visit); - } - PhysicalPlan::ChunkAppendData(plan) => { - Self::traverse(&plan.input, pre_visit, visit, post_visit); - } - PhysicalPlan::ChunkMerge(plan) => { - Self::traverse(&plan.input, pre_visit, visit, post_visit); - } - PhysicalPlan::ChunkCommitInsert(plan) => { - Self::traverse(&plan.input, pre_visit, visit, post_visit); - } - PhysicalPlan::BroadcastSink(plan) => { - Self::traverse(&plan.input, pre_visit, visit, post_visit); - } - } - post_visit(plan); - } - } -} diff --git a/src/query/sql/src/executor/physical_plans/common.rs b/src/query/sql/src/executor/physical_plans/common.rs index 545179b4af4d6..94e4201f18a67 100644 --- a/src/query/sql/src/executor/physical_plans/common.rs +++ b/src/query/sql/src/executor/physical_plans/common.rs @@ -16,6 +16,8 @@ use std::fmt::Display; use std::fmt::Formatter; use databend_common_expression::types::DataType; +use databend_common_expression::FieldIndex; +use databend_common_expression::RemoteExpr; use databend_common_expression::Scalar; use databend_common_functions::aggregates::AggregateFunctionSortDesc; @@ -23,6 +25,8 @@ use crate::plans::UDFField; use crate::plans::UDFType; use crate::IndexType; +pub type MatchExpr = Vec<(Option, Option>)>; + #[derive(Clone, Debug, Eq, PartialEq, serde::Serialize, serde::Deserialize)] pub struct AggregateFunctionSignature { pub name: String, diff --git a/src/query/sql/src/executor/physical_plans/mod.rs b/src/query/sql/src/executor/physical_plans/mod.rs index 2f2afb69d3368..318a0a4a8e2a9 100644 --- a/src/query/sql/src/executor/physical_plans/mod.rs +++ b/src/query/sql/src/executor/physical_plans/mod.rs @@ -13,104 +13,5 @@ // limitations under the License. mod common; -mod physical_add_stream_column; -mod physical_aggregate_expand; -mod physical_aggregate_final; -mod physical_aggregate_partial; -mod physical_async_func; -#[allow(dead_code)] -mod physical_broadcast; -mod physical_cache_scan; -mod physical_column_mutation; -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_distributed_insert_select; -mod physical_eval_scalar; -mod physical_exchange; -mod physical_exchange_sink; -mod physical_exchange_source; -mod physical_expression_scan; -mod physical_filter; -mod physical_hash_join; -mod physical_join; -mod physical_join_filter; -mod physical_limit; -mod physical_multi_table_insert; -mod physical_mutation; -mod physical_mutation_into_organize; -mod physical_mutation_into_split; -mod physical_mutation_manipulate; -mod physical_mutation_source; -mod physical_project_set; -mod physical_r_cte_scan; -mod physical_range_join; -mod physical_recluster; -mod physical_refresh_index; -mod physical_replace_async_source; -mod physical_replace_deduplicate; -mod physical_replace_into; -mod physical_row_fetch; -mod physical_sort; -mod physical_table_scan; -mod physical_udf; -mod physical_union_all; -mod physical_window; -mod physical_window_partition; pub use common::*; -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_async_func::AsyncFunction; -pub use physical_async_func::AsyncFunctionDesc; -pub use physical_broadcast::BroadcastSink; -pub use physical_broadcast::BroadcastSource; -pub use physical_broadcast::*; -pub use physical_cache_scan::CacheScan; -pub use physical_column_mutation::ColumnMutation; -pub use physical_commit_sink::*; -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_distributed_insert_select::DistributedInsertSelect; -pub use physical_eval_scalar::EvalScalar; -pub use physical_exchange::Exchange; -pub use physical_exchange_sink::ExchangeSink; -pub use physical_exchange_source::ExchangeSource; -pub use physical_expression_scan::ExpressionScan; -pub use physical_filter::Filter; -pub use physical_hash_join::HashJoin; -pub use physical_join::PhysicalJoinType; -pub use physical_join_filter::JoinRuntimeFilter; -pub use physical_join_filter::PhysicalRuntimeFilter; -pub use physical_join_filter::PhysicalRuntimeFilters; -pub use physical_limit::Limit; -pub use physical_multi_table_insert::*; -pub use physical_mutation::*; -pub use physical_mutation_into_organize::MutationOrganize; -pub use physical_mutation_into_split::MutationSplit; -pub use physical_mutation_manipulate::MutationManipulate; -pub use physical_mutation_source::*; -pub use physical_project_set::ProjectSet; -pub use physical_r_cte_scan::RecursiveCteScan; -pub use physical_range_join::*; -pub use physical_recluster::HilbertPartition; -pub use physical_recluster::Recluster; -pub use physical_refresh_index::RefreshIndex; -pub use physical_replace_async_source::ReplaceAsyncSourcer; -pub use physical_replace_deduplicate::*; -pub use physical_replace_into::ReplaceInto; -pub use physical_row_fetch::RowFetch; -pub use physical_sort::Sort; -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_window::*; -pub use physical_window_partition::*; -mod physical_asof_join; diff --git a/src/query/sql/src/executor/physical_plans/physical_aggregate_expand.rs b/src/query/sql/src/executor/physical_plans/physical_aggregate_expand.rs deleted file mode 100644 index a9ea7055912cd..0000000000000 --- a/src/query/sql/src/executor/physical_plans/physical_aggregate_expand.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_expression::types::DataType; -use databend_common_expression::types::NumberDataType; -use databend_common_expression::DataField; -use databend_common_expression::DataSchemaRef; -use databend_common_expression::DataSchemaRefExt; - -use crate::executor::explain::PlanStatsInfo; -use crate::executor::PhysicalPlan; -use crate::plans::GroupingSets; -use crate::IndexType; - -/// Add dummy data before `GROUPING SETS`. -#[derive(serde::Serialize, serde::Deserialize, Clone, Debug)] -pub struct AggregateExpand { - // A unique id of operator in a `PhysicalPlan` tree, only used for display. - pub plan_id: u32, - pub input: Box, - pub group_bys: Vec, - pub grouping_sets: GroupingSets, - - // Only used for explain - pub stat_info: Option, -} - -impl AggregateExpand { - pub fn output_schema(&self) -> Result { - let input_schema = self.input.output_schema()?; - let mut output_fields = input_schema.fields().clone(); - // Add virtual columns to group by. - output_fields.reserve(self.group_bys.len() + 1); - - for (group_by, (actual, ty)) in self - .group_bys - .iter() - .zip(self.grouping_sets.dup_group_items.iter()) - { - // All group by columns will wrap nullable. - let i = input_schema.index_of(&group_by.to_string())?; - let f = &mut output_fields[i]; - debug_assert!(f.data_type() == ty || f.data_type().wrap_nullable() == *ty); - *f = DataField::new(f.name(), f.data_type().wrap_nullable()); - let new_field = DataField::new(&actual.to_string(), ty.clone()); - output_fields.push(new_field); - } - - output_fields.push(DataField::new( - &self.grouping_sets.grouping_id_index.to_string(), - DataType::Number(NumberDataType::UInt32), - )); - Ok(DataSchemaRefExt::create(output_fields)) - } -} diff --git a/src/query/sql/src/executor/physical_plans/physical_aggregate_partial.rs b/src/query/sql/src/executor/physical_plans/physical_aggregate_partial.rs deleted file mode 100644 index a8a73071aaa57..0000000000000 --- a/src/query/sql/src/executor/physical_plans/physical_aggregate_partial.rs +++ /dev/null @@ -1,66 +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_expression::types::DataType; -#[allow(unused_imports)] -use databend_common_expression::DataBlock; -use databend_common_expression::DataField; -use databend_common_expression::DataSchemaRef; -use databend_common_expression::DataSchemaRefExt; - -use super::SortDesc; -use crate::executor::explain::PlanStatsInfo; -use crate::executor::physical_plans::common::AggregateFunctionDesc; -use crate::executor::PhysicalPlan; -use crate::IndexType; - -#[derive(serde::Serialize, serde::Deserialize, Clone, Debug)] -pub struct AggregatePartial { - // A unique id of operator in a `PhysicalPlan` tree, only used for display. - pub plan_id: u32, - pub input: Box, - pub group_by: Vec, - pub agg_funcs: Vec, - pub enable_experimental_aggregate_hashtable: bool, - pub group_by_display: Vec, - - // Order by keys if keys are subset of group by key, then we can use rank to filter data in previous - pub rank_limit: Option<(Vec, usize)>, - // Only used for explain - pub stat_info: Option, -} - -impl AggregatePartial { - pub fn output_schema(&self) -> Result { - let input_schema = self.input.output_schema()?; - - let mut fields = Vec::with_capacity(self.agg_funcs.len() + self.group_by.len()); - - fields.extend(self.agg_funcs.iter().map(|func| { - let name = func.output_column.to_string(); - DataField::new(&name, DataType::Binary) - })); - - for (idx, field) in self.group_by.iter().zip( - self.group_by - .iter() - .map(|index| input_schema.field_with_name(&index.to_string())), - ) { - fields.push(DataField::new(&idx.to_string(), field?.data_type().clone())); - } - - Ok(DataSchemaRefExt::create(fields)) - } -} diff --git a/src/query/sql/src/executor/physical_plans/physical_broadcast.rs b/src/query/sql/src/executor/physical_plans/physical_broadcast.rs deleted file mode 100644 index 36e42243e4ff0..0000000000000 --- a/src/query/sql/src/executor/physical_plans/physical_broadcast.rs +++ /dev/null @@ -1,64 +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_catalog::table_context::TableContext; -use databend_common_exception::Result; - -use super::Exchange; -use super::FragmentKind; -use crate::executor::PhysicalPlan; - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct BroadcastSource { - pub plan_id: u32, - pub broadcast_id: u32, -} - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct BroadcastSink { - pub plan_id: u32, - pub broadcast_id: u32, - pub input: Box, -} - -pub fn build_broadcast_plan(broadcast_id: u32) -> Result { - let broadcast_source = Box::new(PhysicalPlan::BroadcastSource(BroadcastSource { - plan_id: 0, - broadcast_id, - })); - let exchange = Box::new(PhysicalPlan::Exchange(Exchange { - plan_id: 0, - input: broadcast_source, - kind: FragmentKind::Expansive, - keys: vec![], - allow_adjust_parallelism: true, - ignore_exchange: false, - })); - let broadcast_sink = PhysicalPlan::BroadcastSink(BroadcastSink { - plan_id: 0, - broadcast_id, - input: exchange, - }); - Ok(broadcast_sink) -} - -pub fn build_broadcast_plans(ctx: &dyn TableContext) -> Result> { - let mut plans = vec![]; - let next_broadcast_id = ctx.get_next_broadcast_id(); - ctx.reset_broadcast_id(); - for broadcast_id in 0..next_broadcast_id { - plans.push(build_broadcast_plan(broadcast_id)?); - } - Ok(plans) -} diff --git a/src/query/sql/src/executor/physical_plans/physical_cache_scan.rs b/src/query/sql/src/executor/physical_plans/physical_cache_scan.rs deleted file mode 100644 index 98575dca5b3d5..0000000000000 --- a/src/query/sql/src/executor/physical_plans/physical_cache_scan.rs +++ /dev/null @@ -1,62 +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_expression::DataSchemaRef; -use databend_common_expression::DataSchemaRefExt; - -use crate::executor::PhysicalPlan; -use crate::executor::PhysicalPlanBuilder; -use crate::plans::CacheSource; -use crate::ColumnSet; - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct CacheScan { - // A unique id of operator in a `PhysicalPlan` tree, only used for display. - pub plan_id: u32, - pub cache_source: CacheSource, - pub output_schema: DataSchemaRef, -} - -impl CacheScan { - pub fn output_schema(&self) -> Result { - Ok(self.output_schema.clone()) - } -} - -impl PhysicalPlanBuilder { - pub(crate) async fn build_cache_scan( - &mut self, - scan: &crate::plans::CacheScan, - required: ColumnSet, - ) -> Result { - // 1. Prune unused Columns. - let used: ColumnSet = required.intersection(&scan.columns).cloned().collect(); - let (cache_source, fields) = if used == scan.columns { - (scan.cache_source.clone(), scan.schema.fields().clone()) - } else { - let new_scan = scan.prune_columns(used); - ( - new_scan.cache_source.clone(), - new_scan.schema.fields().clone(), - ) - }; - // 2. Build physical plan. - Ok(PhysicalPlan::CacheScan(CacheScan { - plan_id: 0, - cache_source, - output_schema: DataSchemaRefExt::create(fields), - })) - } -} diff --git a/src/query/sql/src/executor/physical_plans/physical_commit_sink.rs b/src/query/sql/src/executor/physical_plans/physical_commit_sink.rs deleted file mode 100644 index 81f24be9f22b1..0000000000000 --- a/src/query/sql/src/executor/physical_plans/physical_commit_sink.rs +++ /dev/null @@ -1,53 +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::plan::ReclusterInfoSideCar; -use databend_common_meta_app::schema::TableInfo; -use databend_common_meta_app::schema::UpdateStreamMetaReq; -use databend_storages_common_table_meta::meta::TableMetaTimestamps; -use databend_storages_common_table_meta::meta::TableSnapshot; - -use crate::executor::physical_plans::common::MutationKind; -use crate::executor::PhysicalPlan; -use crate::plans::TruncateMode; - -// serde is required by `PhysicalPlan` -/// The commit sink is used to commit the data to the table. -#[derive(serde::Serialize, serde::Deserialize, Clone, Debug)] -pub struct CommitSink { - pub plan_id: u32, - pub input: Box, - pub snapshot: Option>, - pub table_info: TableInfo, - pub commit_type: CommitType, - pub update_stream_meta: Vec, - pub deduplicated_label: Option, - pub table_meta_timestamps: TableMetaTimestamps, - - // Used for recluster. - pub recluster_info: Option, -} - -#[derive(serde::Serialize, serde::Deserialize, Clone, Debug)] -pub enum CommitType { - Truncate { - mode: TruncateMode, - }, - Mutation { - kind: MutationKind, - merge_meta: bool, - }, -} diff --git a/src/query/sql/src/executor/physical_plans/physical_compact_source.rs b/src/query/sql/src/executor/physical_plans/physical_compact_source.rs deleted file mode 100644 index 70bb52b44da9a..0000000000000 --- a/src/query/sql/src/executor/physical_plans/physical_compact_source.rs +++ /dev/null @@ -1,114 +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::HashSet; - -use databend_common_catalog::plan::PartInfoType; -use databend_common_catalog::plan::Partitions; -use databend_common_catalog::table::TableExt; -use databend_common_exception::ErrorCode; -use databend_common_exception::Result; -use databend_common_expression::ColumnId; -use databend_common_meta_app::schema::TableInfo; -use databend_storages_common_table_meta::meta::TableMetaTimestamps; - -use crate::executor::physical_plans::CommitSink; -use crate::executor::physical_plans::CommitType; -use crate::executor::physical_plans::Exchange; -use crate::executor::physical_plans::FragmentKind; -use crate::executor::physical_plans::MutationKind; -use crate::executor::PhysicalPlan; -use crate::executor::PhysicalPlanBuilder; - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct CompactSource { - pub plan_id: u32, - pub parts: Partitions, - pub table_info: TableInfo, - pub column_ids: HashSet, - pub table_meta_timestamps: TableMetaTimestamps, -} - -impl PhysicalPlanBuilder { - pub async fn build_compact_block( - &mut self, - compact_block: &crate::plans::OptimizeCompactBlock, - ) -> Result { - let crate::plans::OptimizeCompactBlock { - catalog, - database, - table, - limit, - } = compact_block; - - let tenant = self.ctx.get_tenant(); - let catalog = self.ctx.get_catalog(catalog).await?; - let tbl = catalog.get_table(&tenant, database, table).await?; - // check mutability - tbl.check_mutable()?; - - let table_info = tbl.get_table_info().clone(); - - let Some((parts, snapshot)) = tbl.compact_blocks(self.ctx.clone(), limit.clone()).await? - else { - return Err(ErrorCode::NoNeedToCompact(format!( - "No need to do compact for '{database}'.'{table}'" - ))); - }; - - let table_meta_timestamps = self - .ctx - .get_table_meta_timestamps(tbl.as_ref(), Some(snapshot.clone()))?; - - let merge_meta = parts.partitions_type() == PartInfoType::LazyLevel; - let mut root = PhysicalPlan::CompactSource(Box::new(CompactSource { - parts, - table_info: table_info.clone(), - column_ids: snapshot.schema.to_leaf_column_id_set(), - plan_id: u32::MAX, - table_meta_timestamps, - })); - - let is_distributed = (!self.ctx.get_cluster().is_empty()) - && self.ctx.get_settings().get_enable_distributed_compact()?; - if is_distributed { - root = PhysicalPlan::Exchange(Exchange { - plan_id: 0, - input: Box::new(root), - kind: FragmentKind::Merge, - keys: vec![], - allow_adjust_parallelism: true, - ignore_exchange: false, - }); - } - - root = PhysicalPlan::CommitSink(Box::new(CommitSink { - input: Box::new(root), - table_info, - snapshot: Some(snapshot), - commit_type: CommitType::Mutation { - kind: MutationKind::Compact, - merge_meta, - }, - update_stream_meta: vec![], - deduplicated_label: None, - plan_id: u32::MAX, - recluster_info: None, - table_meta_timestamps, - })); - - root.adjust_plan_id(&mut 0); - Ok(root) - } -} diff --git a/src/query/sql/src/executor/physical_plans/physical_constant_table_scan.rs b/src/query/sql/src/executor/physical_plans/physical_constant_table_scan.rs deleted file mode 100644 index a67a9859348b7..0000000000000 --- a/src/query/sql/src/executor/physical_plans/physical_constant_table_scan.rs +++ /dev/null @@ -1,84 +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_expression::Column; -use databend_common_expression::DataSchemaRef; - -use crate::executor::PhysicalPlan; -use crate::executor::PhysicalPlanBuilder; -use crate::ColumnSet; -use crate::IndexType; - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct ConstantTableScan { - // A unique id of operator in a `PhysicalPlan` tree, only used for display. - pub plan_id: u32, - pub values: Vec, - pub num_rows: usize, - pub output_schema: DataSchemaRef, -} - -impl ConstantTableScan { - pub fn output_schema(&self) -> Result { - Ok(self.output_schema.clone()) - } - - pub fn name(&self) -> &str { - if self.num_rows == 0 { - "EmptyResultScan" - } else { - "ConstantTableScan" - } - } -} - -impl PhysicalPlanBuilder { - pub(crate) async fn build_constant_table_scan( - &mut self, - scan: &crate::plans::ConstantTableScan, - required: ColumnSet, - ) -> Result { - debug_assert!(scan - .schema - .fields - .iter() - .map(|field| field.name().parse::().unwrap()) - .collect::() - .is_superset(&scan.columns)); - - let used: ColumnSet = required.intersection(&scan.columns).copied().collect(); - if used.len() < scan.columns.len() { - let crate::plans::ConstantTableScan { - values, - num_rows, - schema, - .. - } = scan.prune_columns(used); - return Ok(PhysicalPlan::ConstantTableScan(ConstantTableScan { - plan_id: 0, - values, - num_rows, - output_schema: schema, - })); - } - - Ok(PhysicalPlan::ConstantTableScan(ConstantTableScan { - plan_id: 0, - values: scan.values.clone(), - num_rows: scan.num_rows, - output_schema: scan.schema.clone(), - })) - } -} diff --git a/src/query/sql/src/executor/physical_plans/physical_copy_into_location.rs b/src/query/sql/src/executor/physical_plans/physical_copy_into_location.rs deleted file mode 100644 index 5b069b8587664..0000000000000 --- a/src/query/sql/src/executor/physical_plans/physical_copy_into_location.rs +++ /dev/null @@ -1,45 +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_expression::types::DataType; -use databend_common_expression::types::NumberDataType; -use databend_common_expression::DataField; -use databend_common_expression::DataSchemaRef; -use databend_common_expression::DataSchemaRefExt; -use databend_common_expression::TableSchemaRef; -use databend_storages_common_stage::CopyIntoLocationInfo; - -use crate::executor::PhysicalPlan; -use crate::ColumnBinding; - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct CopyIntoLocation { - pub plan_id: u32, - pub input: Box, - pub project_columns: Vec, - pub input_data_schema: DataSchemaRef, - pub input_table_schema: TableSchemaRef, - pub info: CopyIntoLocationInfo, -} - -impl CopyIntoLocation { - pub fn output_schema(&self) -> Result { - Ok(DataSchemaRefExt::create(vec![ - DataField::new("rows_unloaded", DataType::Number(NumberDataType::UInt64)), - DataField::new("input_bytes", DataType::Number(NumberDataType::UInt64)), - DataField::new("output_bytes", DataType::Number(NumberDataType::UInt64)), - ])) - } -} diff --git a/src/query/sql/src/executor/physical_plans/physical_copy_into_table.rs b/src/query/sql/src/executor/physical_plans/physical_copy_into_table.rs deleted file mode 100644 index 6d678ad12efce..0000000000000 --- a/src/query/sql/src/executor/physical_plans/physical_copy_into_table.rs +++ /dev/null @@ -1,57 +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_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 databend_storages_common_table_meta::meta::TableMetaTimestamps; -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 plan_id: u32, - - 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, - pub table_meta_timestamps: TableMetaTimestamps, -} - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize, EnumAsInner)] -pub enum CopyIntoTableSource { - Query(Box), - Stage(Box), -} - -impl CopyIntoTable { - 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_distributed_insert_select.rs deleted file mode 100644 index 678cf7b210637..0000000000000 --- a/src/query/sql/src/executor/physical_plans/physical_distributed_insert_select.rs +++ /dev/null @@ -1,34 +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_expression::DataSchemaRef; -use databend_common_meta_app::schema::TableInfo; -use databend_storages_common_table_meta::meta::TableMetaTimestamps; - -use crate::executor::PhysicalPlan; -use crate::ColumnBinding; - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct DistributedInsertSelect { - /// A unique id of operator in a `PhysicalPlan` tree. - pub plan_id: u32, - - pub input: Box, - pub table_info: TableInfo, - pub insert_schema: DataSchemaRef, - pub select_schema: DataSchemaRef, - pub select_column_bindings: Vec, - pub cast_needed: bool, - pub table_meta_timestamps: TableMetaTimestamps, -} diff --git a/src/query/sql/src/executor/physical_plans/physical_exchange.rs b/src/query/sql/src/executor/physical_plans/physical_exchange.rs deleted file mode 100644 index 1e831519c415b..0000000000000 --- a/src/query/sql/src/executor/physical_plans/physical_exchange.rs +++ /dev/null @@ -1,94 +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_expression::ConstantFolder; -use databend_common_expression::DataSchemaRef; -use databend_common_expression::RemoteExpr; -use databend_common_functions::BUILTIN_FUNCTIONS; - -use crate::executor::physical_plans::common::FragmentKind; -use crate::executor::PhysicalPlan; -use crate::executor::PhysicalPlanBuilder; -use crate::optimizer::ir::SExpr; -use crate::ColumnSet; -use crate::TypeCheck; - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct Exchange { - // A unique id of operator in a `PhysicalPlan` tree, only used for display. - pub plan_id: u32, - - pub input: Box, - pub kind: FragmentKind, - pub keys: Vec, - pub ignore_exchange: bool, - pub allow_adjust_parallelism: bool, -} - -impl Exchange { - pub fn output_schema(&self) -> Result { - self.input.output_schema() - } -} - -impl PhysicalPlanBuilder { - pub(crate) async fn build_exchange( - &mut self, - s_expr: &SExpr, - exchange: &crate::plans::Exchange, - mut required: ColumnSet, - ) -> Result { - // 1. Prune unused Columns. - if let crate::plans::Exchange::Hash(exprs) = exchange { - for expr in exprs { - required.extend(expr.used_columns()); - } - } - - // 2. Build physical plan. - let input = Box::new(self.build(s_expr.child(0)?, required).await?); - let input_schema = input.output_schema()?; - let mut keys = vec![]; - let mut allow_adjust_parallelism = true; - let kind = match exchange { - crate::plans::Exchange::Hash(scalars) => { - for scalar in scalars { - let expr = scalar - .type_check(input_schema.as_ref())? - .project_column_ref(|index| { - input_schema.index_of(&index.to_string()).unwrap() - }); - let (expr, _) = ConstantFolder::fold(&expr, &self.func_ctx, &BUILTIN_FUNCTIONS); - keys.push(expr.as_remote_expr()); - } - FragmentKind::Normal - } - crate::plans::Exchange::Broadcast => FragmentKind::Expansive, - crate::plans::Exchange::Merge => FragmentKind::Merge, - crate::plans::Exchange::MergeSort => { - allow_adjust_parallelism = false; - FragmentKind::Merge - } - }; - Ok(PhysicalPlan::Exchange(Exchange { - plan_id: 0, - input, - kind, - keys, - allow_adjust_parallelism, - ignore_exchange: false, - })) - } -} diff --git a/src/query/sql/src/executor/physical_plans/physical_exchange_sink.rs b/src/query/sql/src/executor/physical_plans/physical_exchange_sink.rs deleted file mode 100644 index cb5ad775bf2bb..0000000000000 --- a/src/query/sql/src/executor/physical_plans/physical_exchange_sink.rs +++ /dev/null @@ -1,45 +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_expression::DataSchemaRef; -use databend_common_expression::RemoteExpr; - -use crate::executor::physical_plans::common::FragmentKind; -use crate::executor::PhysicalPlan; - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct ExchangeSink { - // A unique id of operator in a `PhysicalPlan` tree, only used for display. - pub plan_id: u32, - pub input: Box, - // Input schema of exchanged data - pub schema: DataSchemaRef, - pub kind: FragmentKind, - pub keys: Vec, - - // Fragment ID of sink fragment - pub destination_fragment_id: usize, - - // Addresses of destination nodes - pub query_id: String, - pub ignore_exchange: bool, - pub allow_adjust_parallelism: bool, -} - -impl ExchangeSink { - pub fn output_schema(&self) -> Result { - Ok(self.schema.clone()) - } -} diff --git a/src/query/sql/src/executor/physical_plans/physical_exchange_source.rs b/src/query/sql/src/executor/physical_plans/physical_exchange_source.rs deleted file mode 100644 index 08ec4b44b4f86..0000000000000 --- a/src/query/sql/src/executor/physical_plans/physical_exchange_source.rs +++ /dev/null @@ -1,35 +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_expression::DataSchemaRef; - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct ExchangeSource { - // A unique id of operator in a `PhysicalPlan` tree, only used for display. - pub plan_id: u32, - - // Output schema of exchanged data - pub schema: DataSchemaRef, - - // Fragment ID of source fragment - pub source_fragment_id: usize, - pub query_id: String, -} - -impl ExchangeSource { - pub fn output_schema(&self) -> Result { - Ok(self.schema.clone()) - } -} diff --git a/src/query/sql/src/executor/physical_plans/physical_expression_scan.rs b/src/query/sql/src/executor/physical_plans/physical_expression_scan.rs deleted file mode 100644 index 5721f80368321..0000000000000 --- a/src/query/sql/src/executor/physical_plans/physical_expression_scan.rs +++ /dev/null @@ -1,78 +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_expression::ConstantFolder; -use databend_common_expression::DataSchemaRef; -use databend_common_expression::RemoteExpr; -use databend_common_functions::BUILTIN_FUNCTIONS; - -use crate::executor::PhysicalPlan; -use crate::executor::PhysicalPlanBuilder; -use crate::optimizer::ir::SExpr; -use crate::ColumnSet; -use crate::TypeCheck; - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct ExpressionScan { - // A unique id of operator in a `PhysicalPlan` tree, only used for display. - pub plan_id: u32, - pub values: Vec>, - pub input: Box, - pub output_schema: DataSchemaRef, -} - -impl ExpressionScan { - pub fn output_schema(&self) -> Result { - Ok(self.output_schema.clone()) - } -} - -impl PhysicalPlanBuilder { - pub(crate) async fn build_expression_scan( - &mut self, - s_expr: &SExpr, - scan: &crate::plans::ExpressionScan, - required: ColumnSet, - ) -> Result { - let input = self.build(s_expr.child(0)?, required).await?; - let input_schema = input.output_schema()?; - - let values = scan - .values - .iter() - .map(|row| { - row.iter() - .map(|scalar| { - let expr = scalar - .type_check(input_schema.as_ref())? - .project_column_ref(|index| { - input_schema.index_of(&index.to_string()).unwrap() - }); - let (expr, _) = - ConstantFolder::fold(&expr, &self.func_ctx, &BUILTIN_FUNCTIONS); - Ok(expr.as_remote_expr()) - }) - .collect::>>() - }) - .collect::>>()?; - - Ok(PhysicalPlan::ExpressionScan(ExpressionScan { - plan_id: 0, - values, - input: Box::new(input), - output_schema: scan.schema.clone(), - })) - } -} diff --git a/src/query/sql/src/executor/physical_plans/physical_filter.rs b/src/query/sql/src/executor/physical_plans/physical_filter.rs deleted file mode 100644 index 3051e5ec9307b..0000000000000 --- a/src/query/sql/src/executor/physical_plans/physical_filter.rs +++ /dev/null @@ -1,106 +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_expression::ConstantFolder; -use databend_common_expression::DataSchemaRef; -use databend_common_expression::DataSchemaRefExt; -use databend_common_expression::RemoteExpr; -use databend_common_functions::BUILTIN_FUNCTIONS; - -use crate::executor::cast_expr_to_non_null_boolean; -use crate::executor::explain::PlanStatsInfo; -use crate::executor::PhysicalPlan; -use crate::executor::PhysicalPlanBuilder; -use crate::optimizer::ir::SExpr; -use crate::ColumnSet; -use crate::TypeCheck; - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct Filter { - // A unique id of operator in a `PhysicalPlan` tree, only used for display. - pub plan_id: u32, - pub projections: ColumnSet, - pub input: Box, - // Assumption: expression's data type must be `DataType::Boolean`. - pub predicates: Vec, - - // Only used for explain - pub stat_info: Option, -} - -impl Filter { - pub fn output_schema(&self) -> Result { - let input_schema = self.input.output_schema()?; - let mut fields = Vec::with_capacity(self.projections.len()); - for (i, field) in input_schema.fields().iter().enumerate() { - if self.projections.contains(&i) { - fields.push(field.clone()); - } - } - Ok(DataSchemaRefExt::create(fields)) - } -} - -impl PhysicalPlanBuilder { - pub(crate) async fn build_filter( - &mut self, - s_expr: &SExpr, - filter: &crate::plans::Filter, - mut required: ColumnSet, - stat_info: PlanStatsInfo, - ) -> Result { - // 1. Prune unused Columns. - let used = filter.predicates.iter().fold(required.clone(), |acc, v| { - acc.union(&v.used_columns()).cloned().collect() - }); - - // 2. Build physical plan. - let input = Box::new(self.build(s_expr.child(0)?, used).await?); - required = required - .union(self.metadata.read().get_retained_column()) - .cloned() - .collect(); - let column_projections = required.clone().into_iter().collect::>(); - let input_schema = input.output_schema()?; - let mut projections = ColumnSet::new(); - for column in column_projections.iter() { - if let Some((index, _)) = input_schema.column_with_name(&column.to_string()) { - projections.insert(index); - } - } - - Ok(PhysicalPlan::Filter(Filter { - plan_id: 0, - projections, - input, - predicates: filter - .predicates - .iter() - .map(|scalar| { - let expr = scalar - .type_check(input_schema.as_ref())? - .project_column_ref(|index| { - input_schema.index_of(&index.to_string()).unwrap() - }); - let expr = cast_expr_to_non_null_boolean(expr)?; - let (expr, _) = ConstantFolder::fold(&expr, &self.func_ctx, &BUILTIN_FUNCTIONS); - Ok(expr.as_remote_expr()) - }) - .collect::>()?, - - stat_info: Some(stat_info), - })) - } -} diff --git a/src/query/sql/src/executor/physical_plans/physical_multi_table_insert.rs b/src/query/sql/src/executor/physical_plans/physical_multi_table_insert.rs deleted file mode 100644 index 41c1f88954dd6..0000000000000 --- a/src/query/sql/src/executor/physical_plans/physical_multi_table_insert.rs +++ /dev/null @@ -1,144 +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_exception::ErrorCode; -use databend_common_exception::Result; -use databend_common_expression::DataSchemaRef; -use databend_common_expression::RemoteExpr; -use databend_common_meta_app::schema::CatalogInfo; -use databend_common_meta_app::schema::TableInfo; -use databend_common_meta_app::schema::UpdateStreamMetaReq; -use databend_storages_common_table_meta::meta::TableMetaTimestamps; - -use crate::executor::PhysicalPlan; -use crate::ColumnSet; -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct Duplicate { - pub plan_id: u32, - pub input: Box, - pub n: usize, -} - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct Shuffle { - pub plan_id: u32, - pub input: Box, - pub strategy: ShuffleStrategy, -} - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub enum ShuffleStrategy { - Transpose(usize), -} - -impl ShuffleStrategy { - pub fn shuffle(&self, total: usize) -> Result> { - match self { - ShuffleStrategy::Transpose(n) => { - if total % n != 0 { - return Err(ErrorCode::Internal(format!( - "total rows {} is not divisible by n {}", - total, n - ))); - } - let mut result = vec![0; total]; - for i in 0..*n { - for j in 0..total / n { - result[i + j * n] = i * (total / n) + j; - } - } - Ok(result) - } - } - } -} - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct ChunkFilter { - pub plan_id: u32, - pub input: Box, - pub predicates: Vec>, -} - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct ChunkEvalScalar { - pub plan_id: u32, - pub input: Box, - pub eval_scalars: Vec>, -} - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct MultiInsertEvalScalar { - pub remote_exprs: Vec, - pub projection: ColumnSet, -} - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct ChunkCastSchema { - pub plan_id: u32, - pub input: Box, - pub cast_schemas: Vec>, -} - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct CastSchema { - pub source_schema: DataSchemaRef, - pub target_schema: DataSchemaRef, -} - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct ChunkFillAndReorder { - pub plan_id: u32, - pub input: Box, - pub fill_and_reorders: Vec>, -} - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct FillAndReorder { - pub source_schema: DataSchemaRef, - pub target_table_info: TableInfo, -} - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct ChunkAppendData { - pub plan_id: u32, - pub input: Box, - pub target_tables: Vec, -} - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct SerializableTable { - pub target_catalog_info: Arc, - pub target_table_info: TableInfo, - pub table_meta_timestamps: TableMetaTimestamps, -} - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct ChunkMerge { - pub plan_id: u32, - pub input: Box, - pub group_ids: Vec, -} - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct ChunkCommitInsert { - pub plan_id: u32, - pub input: Box, - pub update_stream_meta: Vec, - pub overwrite: bool, - pub deduplicated_label: Option, - pub targets: Vec, -} diff --git a/src/query/sql/src/executor/physical_plans/physical_mutation_into_organize.rs b/src/query/sql/src/executor/physical_plans/physical_mutation_into_organize.rs deleted file mode 100644 index f2d6df77da87c..0000000000000 --- a/src/query/sql/src/executor/physical_plans/physical_mutation_into_organize.rs +++ /dev/null @@ -1,32 +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_expression::DataSchemaRef; - -use crate::binder::MutationStrategy; -use crate::executor::physical_plan::PhysicalPlan; - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct MutationOrganize { - pub plan_id: u32, - pub input: Box, - pub strategy: MutationStrategy, -} - -impl MutationOrganize { - pub fn output_schema(&self) -> Result { - self.input.output_schema() - } -} diff --git a/src/query/sql/src/executor/physical_plans/physical_mutation_manipulate.rs b/src/query/sql/src/executor/physical_plans/physical_mutation_manipulate.rs deleted file mode 100644 index 50976f11327be..0000000000000 --- a/src/query/sql/src/executor/physical_plans/physical_mutation_manipulate.rs +++ /dev/null @@ -1,50 +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::HashMap; - -use databend_common_exception::Result; -use databend_common_expression::DataSchemaRef; -use databend_common_expression::FieldIndex; -use databend_common_expression::RemoteExpr; -use databend_common_meta_app::schema::TableInfo; - -use crate::binder::MutationStrategy; -use crate::executor::physical_plan::PhysicalPlan; - -pub type MatchExpr = Vec<(Option, Option>)>; - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct MutationManipulate { - pub plan_id: u32, - pub input: Box, - pub table_info: TableInfo, - // (DataSchemaRef, Option, Vec,Vec) => (source_schema, condition, value_exprs) - pub unmatched: Vec<(DataSchemaRef, Option, Vec)>, - // the first option stands for the condition - // the second option stands for update/delete - pub matched: MatchExpr, - // used to record the index of target table's field in merge_source_schema - pub field_index_of_input_schema: HashMap, - pub strategy: MutationStrategy, - pub row_id_idx: usize, - pub can_try_update_column_only: bool, - pub unmatched_schema: DataSchemaRef, -} - -impl MutationManipulate { - pub fn output_schema(&self) -> Result { - self.input.output_schema() - } -} diff --git a/src/query/sql/src/executor/physical_plans/physical_mutation_source.rs b/src/query/sql/src/executor/physical_plans/physical_mutation_source.rs deleted file mode 100644 index b04c294220ab5..0000000000000 --- a/src/query/sql/src/executor/physical_plans/physical_mutation_source.rs +++ /dev/null @@ -1,150 +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_catalog::plan::Filters; -use databend_common_catalog::plan::PartStatistics; -use databend_common_catalog::plan::Partitions; -use databend_common_exception::Result; -use databend_common_expression::type_check::check_function; -use databend_common_expression::types::DataType; -use databend_common_expression::ConstantFolder; -use databend_common_expression::DataField; -use databend_common_expression::DataSchemaRef; -use databend_common_expression::DataSchemaRefExt; -use databend_common_expression::FunctionContext; -use databend_common_functions::BUILTIN_FUNCTIONS; -use databend_common_meta_app::schema::TableInfo; - -use crate::binder::MutationType; -use crate::executor::cast_expr_to_non_null_boolean; -use crate::executor::PhysicalPlan; -use crate::executor::PhysicalPlanBuilder; -use crate::ColumnSet; -use crate::IndexType; -use crate::ScalarExpr; - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct MutationSource { - // A unique id of operator in a `PhysicalPlan` tree, only used for display. - pub plan_id: u32, - pub table_index: IndexType, - pub table_info: TableInfo, - pub filters: Option, - pub output_schema: DataSchemaRef, - pub input_type: MutationType, - pub read_partition_columns: ColumnSet, - pub truncate_table: bool, - - pub partitions: Partitions, - pub statistics: PartStatistics, -} - -impl MutationSource { - pub fn output_schema(&self) -> Result { - Ok(self.output_schema.clone()) - } -} - -impl PhysicalPlanBuilder { - pub(crate) async fn build_mutation_source( - &mut self, - mutation_source: &crate::plans::MutationSource, - ) -> Result { - let filters = if !mutation_source.predicates.is_empty() { - Some(create_push_down_filters( - &self.ctx.get_function_context()?, - &mutation_source.predicates, - )?) - } else { - None - }; - let mutation_info = self.mutation_build_info.as_ref().unwrap(); - - let metadata = self.metadata.read(); - let mut fields = Vec::with_capacity(mutation_source.columns.len()); - for column_index in mutation_source.columns.iter() { - let column = metadata.column(*column_index); - // Ignore virtual computed columns. - if let Ok(column_id) = mutation_source.schema.index_of(&column.name()) { - fields.push((column.name(), *column_index, column_id)); - } - } - fields.sort_by_key(|(_, _, id)| *id); - - let mut fields = fields - .into_iter() - .map(|(name, index, _)| { - let table_field = mutation_source.schema.field_with_name(&name)?; - let data_type = DataType::from(table_field.data_type()); - Ok(DataField::new(&index.to_string(), data_type)) - }) - .collect::>>()?; - - if let Some(predicate_index) = mutation_source.predicate_column_index { - fields.push(DataField::new( - &predicate_index.to_string(), - DataType::Boolean, - )); - } - let output_schema = DataSchemaRefExt::create(fields); - - let truncate_table = - mutation_source.mutation_type == MutationType::Delete && filters.is_none(); - Ok(PhysicalPlan::MutationSource(MutationSource { - plan_id: 0, - table_index: mutation_source.table_index, - output_schema, - table_info: mutation_info.table_info.clone(), - filters, - input_type: mutation_source.mutation_type.clone(), - read_partition_columns: mutation_source.read_partition_columns.clone(), - truncate_table, - partitions: mutation_info.partitions.clone(), - statistics: mutation_info.statistics.clone(), - })) - } -} - -/// create push down filters -pub fn create_push_down_filters( - func_ctx: &FunctionContext, - predicates: &[ScalarExpr], -) -> Result { - let predicates = predicates - .iter() - .map(|p| { - Ok(p.as_expr()? - .project_column_ref(|col| col.column_name.clone())) - }) - .collect::>>()?; - - let expr = predicates - .into_iter() - .try_reduce(|lhs, rhs| { - check_function(None, "and_filters", &[], &[lhs, rhs], &BUILTIN_FUNCTIONS) - })? - .unwrap(); - let expr = cast_expr_to_non_null_boolean(expr)?; - let (filter, _) = ConstantFolder::fold(&expr, func_ctx, &BUILTIN_FUNCTIONS); - let remote_filter = filter.as_remote_expr(); - - // prepare the inverse filter expression - let remote_inverted_filter = - check_function(None, "not", &[], &[filter], &BUILTIN_FUNCTIONS)?.as_remote_expr(); - - Ok(Filters { - filter: remote_filter, - inverted_filter: remote_inverted_filter, - }) -} diff --git a/src/query/sql/src/executor/physical_plans/physical_project_set.rs b/src/query/sql/src/executor/physical_plans/physical_project_set.rs deleted file mode 100644 index 99b85519223b8..0000000000000 --- a/src/query/sql/src/executor/physical_plans/physical_project_set.rs +++ /dev/null @@ -1,107 +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_expression::ConstantFolder; -use databend_common_expression::DataField; -use databend_common_expression::DataSchemaRef; -use databend_common_expression::DataSchemaRefExt; -use databend_common_expression::RemoteExpr; -use databend_common_functions::BUILTIN_FUNCTIONS; - -use crate::executor::explain::PlanStatsInfo; -use crate::executor::PhysicalPlan; -use crate::executor::PhysicalPlanBuilder; -use crate::optimizer::ir::SExpr; -use crate::ColumnSet; -use crate::IndexType; -use crate::TypeCheck; - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct ProjectSet { - // A unique id of operator in a `PhysicalPlan` tree, only used for display. - pub plan_id: u32, - pub projections: ColumnSet, - pub input: Box, - pub srf_exprs: Vec<(RemoteExpr, IndexType)>, - - // Only used for explain - pub stat_info: Option, -} - -impl ProjectSet { - pub fn output_schema(&self) -> Result { - let input_schema = self.input.output_schema()?; - let mut fields = Vec::with_capacity(input_schema.num_fields() + self.srf_exprs.len()); - for (i, field) in input_schema.fields().iter().enumerate() { - if self.projections.contains(&i) { - fields.push(field.clone()); - } - } - fields.extend(self.srf_exprs.iter().map(|(srf, index)| { - DataField::new( - &index.to_string(), - srf.as_expr(&BUILTIN_FUNCTIONS).data_type().clone(), - ) - })); - Ok(DataSchemaRefExt::create(fields)) - } -} - -impl PhysicalPlanBuilder { - pub(crate) async fn build_project_set( - &mut self, - s_expr: &SExpr, - project_set: &crate::plans::ProjectSet, - mut required: ColumnSet, - stat_info: PlanStatsInfo, - ) -> Result { - // 1. Prune unused Columns. - let column_projections = required.clone().into_iter().collect::>(); - for s in project_set.srfs.iter() { - required.extend(s.scalar.used_columns().iter().copied()); - } - - // 2. Build physical plan. - let input = self.build(s_expr.child(0)?, required).await?; - let input_schema = input.output_schema()?; - let srf_exprs = project_set - .srfs - .iter() - .map(|item| { - let expr = item - .scalar - .type_check(input_schema.as_ref())? - .project_column_ref(|index| input_schema.index_of(&index.to_string()).unwrap()); - let (expr, _) = ConstantFolder::fold(&expr, &self.func_ctx, &BUILTIN_FUNCTIONS); - Ok((expr.as_remote_expr(), item.index)) - }) - .collect::>>()?; - - let mut projections = ColumnSet::new(); - for column in column_projections.iter() { - if let Some((index, _)) = input_schema.column_with_name(&column.to_string()) { - projections.insert(index); - } - } - - Ok(PhysicalPlan::ProjectSet(ProjectSet { - plan_id: 0, - input: Box::new(input), - srf_exprs, - projections, - stat_info: Some(stat_info), - })) - } -} diff --git a/src/query/sql/src/executor/physical_plans/physical_r_cte_scan.rs b/src/query/sql/src/executor/physical_plans/physical_r_cte_scan.rs deleted file mode 100644 index 459a6c3c91bd2..0000000000000 --- a/src/query/sql/src/executor/physical_plans/physical_r_cte_scan.rs +++ /dev/null @@ -1,59 +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::fmt::Display; - -use databend_common_exception::Result; -use databend_common_expression::DataSchemaRef; -use databend_common_expression::DataSchemaRefExt; - -use crate::executor::explain::PlanStatsInfo; -use crate::executor::PhysicalPlan; -use crate::executor::PhysicalPlanBuilder; - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct RecursiveCteScan { - // A unique id of operator in a `PhysicalPlan` tree, only used for display. - pub plan_id: u32, - pub output_schema: DataSchemaRef, - pub table_name: String, - pub stat: PlanStatsInfo, -} - -impl RecursiveCteScan { - pub fn output_schema(&self) -> Result { - Ok(self.output_schema.clone()) - } -} - -impl PhysicalPlanBuilder { - pub(crate) async fn build_recursive_cte_scan( - &mut self, - recursive_cte_scan: &crate::plans::RecursiveCteScan, - stat_info: PlanStatsInfo, - ) -> Result { - Ok(PhysicalPlan::RecursiveCteScan(RecursiveCteScan { - plan_id: 0, - output_schema: DataSchemaRefExt::create(recursive_cte_scan.fields.clone()), - table_name: recursive_cte_scan.table_name.clone(), - stat: stat_info, - })) - } -} - -impl Display for RecursiveCteScan { - fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { - write!(f, "RecursiveCTEScan") - } -} diff --git a/src/query/sql/src/executor/physical_plans/physical_recluster.rs b/src/query/sql/src/executor/physical_plans/physical_recluster.rs deleted file mode 100644 index 9227c86b64199..0000000000000 --- a/src/query/sql/src/executor/physical_plans/physical_recluster.rs +++ /dev/null @@ -1,37 +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_catalog::plan::ReclusterTask; -use databend_common_meta_app::schema::TableInfo; -use databend_storages_common_table_meta::meta::TableMetaTimestamps; - -use crate::executor::PhysicalPlan; - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct Recluster { - pub plan_id: u32, - pub tasks: Vec, - pub table_info: TableInfo, - pub table_meta_timestamps: TableMetaTimestamps, -} - -#[derive(serde::Serialize, serde::Deserialize, Clone, Debug)] -pub struct HilbertPartition { - pub plan_id: u32, - pub input: Box, - pub table_info: TableInfo, - pub num_partitions: usize, - pub table_meta_timestamps: TableMetaTimestamps, - pub rows_per_block: usize, -} diff --git a/src/query/sql/src/executor/physical_plans/physical_replace_async_source.rs b/src/query/sql/src/executor/physical_plans/physical_replace_async_source.rs deleted file mode 100644 index f7a640fd9cc00..0000000000000 --- a/src/query/sql/src/executor/physical_plans/physical_replace_async_source.rs +++ /dev/null @@ -1,24 +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_expression::DataSchemaRef; - -use crate::plans::InsertValue; - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct ReplaceAsyncSourcer { - pub plan_id: u32, - pub schema: DataSchemaRef, - pub source: InsertValue, -} diff --git a/src/query/sql/src/executor/physical_plans/physical_replace_deduplicate.rs b/src/query/sql/src/executor/physical_plans/physical_replace_deduplicate.rs deleted file mode 100644 index 24224053d03b4..0000000000000 --- a/src/query/sql/src/executor/physical_plans/physical_replace_deduplicate.rs +++ /dev/null @@ -1,48 +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::HashMap; - -use databend_common_expression::ColumnId; -use databend_common_expression::DataSchemaRef; -use databend_common_expression::FieldIndex; -use databend_common_expression::RemoteExpr; -use databend_common_expression::TableSchemaRef; -use databend_common_meta_app::schema::TableInfo; -use databend_storages_common_table_meta::meta::ColumnStatistics; - -use crate::executor::physical_plans::common::OnConflictField; -use crate::executor::PhysicalPlan; -use crate::ColumnBinding; - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct ReplaceDeduplicate { - pub plan_id: u32, - pub input: Box, - pub on_conflicts: Vec, - pub bloom_filter_column_indexes: Vec, - pub table_is_empty: bool, - pub table_info: TableInfo, - pub target_schema: TableSchemaRef, - pub select_ctx: Option, - pub table_level_range_index: HashMap, - pub need_insert: bool, - pub delete_when: Option<(RemoteExpr, String)>, -} - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct ReplaceSelectCtx { - pub select_column_bindings: Vec, - pub select_schema: DataSchemaRef, -} diff --git a/src/query/sql/src/executor/physical_plans/physical_replace_into.rs b/src/query/sql/src/executor/physical_plans/physical_replace_into.rs deleted file mode 100644 index 75d0af7ac2cd3..0000000000000 --- a/src/query/sql/src/executor/physical_plans/physical_replace_into.rs +++ /dev/null @@ -1,39 +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_expression::BlockThresholds; -use databend_common_expression::FieldIndex; -use databend_common_meta_app::schema::TableInfo; -use databend_storages_common_table_meta::meta::BlockSlotDescription; -use databend_storages_common_table_meta::meta::Location; -use databend_storages_common_table_meta::meta::TableMetaTimestamps; - -use crate::executor::physical_plans::common::OnConflictField; -use crate::executor::PhysicalPlan; - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct ReplaceInto { - /// A unique id of operator in a `PhysicalPlan` tree. - pub plan_id: u32, - - pub input: Box, - pub block_thresholds: BlockThresholds, - pub table_info: TableInfo, - pub on_conflicts: Vec, - pub bloom_filter_column_indexes: Vec, - pub segments: Vec<(usize, Location)>, - pub block_slots: Option, - pub need_insert: bool, - pub table_meta_timestamps: TableMetaTimestamps, -} diff --git a/src/query/sql/src/executor/physical_plans/physical_row_fetch.rs b/src/query/sql/src/executor/physical_plans/physical_row_fetch.rs deleted file mode 100644 index 3a5aca4f7a309..0000000000000 --- a/src/query/sql/src/executor/physical_plans/physical_row_fetch.rs +++ /dev/null @@ -1,48 +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_catalog::plan::DataSourcePlan; -use databend_common_catalog::plan::Projection; -use databend_common_exception::Result; -use databend_common_expression::DataField; -use databend_common_expression::DataSchemaRef; -use databend_common_expression::DataSchemaRefExt; - -use crate::executor::explain::PlanStatsInfo; -use crate::executor::PhysicalPlan; - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct RowFetch { - // A unique id of operator in a `PhysicalPlan` tree, only used for display. - pub plan_id: u32, - pub input: Box, - // cloned from `input`. - pub source: Box, - // projection on the source table schema. - pub cols_to_fetch: Projection, - pub row_id_col_offset: usize, - pub fetched_fields: Vec, - pub need_wrap_nullable: bool, - - /// Only used for explain - pub stat_info: Option, -} - -impl RowFetch { - pub fn output_schema(&self) -> Result { - let mut fields = self.input.output_schema()?.fields().clone(); - fields.extend_from_slice(&self.fetched_fields); - Ok(DataSchemaRefExt::create(fields)) - } -} diff --git a/src/query/sql/src/executor/physical_plans/physical_sort.rs b/src/query/sql/src/executor/physical_plans/physical_sort.rs deleted file mode 100644 index 94b765107f310..0000000000000 --- a/src/query/sql/src/executor/physical_plans/physical_sort.rs +++ /dev/null @@ -1,175 +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_expression::types::DataType; -use databend_common_expression::DataField; -use databend_common_expression::DataSchema; -use databend_common_expression::DataSchemaRef; -use databend_common_expression::DataSchemaRefExt; -use databend_common_pipeline_transforms::processors::sort::utils::ORDER_COL_NAME; - -use crate::executor::explain::PlanStatsInfo; -use crate::executor::physical_plans::common::SortDesc; -use crate::executor::physical_plans::WindowPartition; -use crate::executor::physical_plans::WindowPartitionTopN; -use crate::executor::physical_plans::WindowPartitionTopNFunc; -use crate::executor::PhysicalPlan; -use crate::executor::PhysicalPlanBuilder; -use crate::optimizer::ir::SExpr; -use crate::plans::WindowFuncType; -use crate::ColumnSet; -use crate::IndexType; - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct Sort { - /// A unique id of operator in a `PhysicalPlan` tree, only used for display. - pub plan_id: u32, - pub input: Box, - pub order_by: Vec, - /// limit = Limit.limit + Limit.offset - pub limit: Option, - /// If the sort plan is after the exchange plan. - /// It's [None] if the sorting plan is in single node mode. - pub after_exchange: Option, - pub pre_projection: Option>, - - // Only used for explain - pub stat_info: Option, -} - -impl Sort { - fn order_col_type(&self, schema: &DataSchema) -> Result { - if self.order_by.len() == 1 { - let order_by_field = schema.field_with_name(&self.order_by[0].order_by.to_string())?; - if matches!( - order_by_field.data_type(), - DataType::Number(_) | DataType::Date | DataType::Timestamp | DataType::String - ) { - return Ok(order_by_field.data_type().clone()); - } - } - Ok(DataType::Binary) - } - - pub fn output_schema(&self) -> Result { - let input_schema = self.input.output_schema()?; - let mut fields = input_schema.fields().clone(); - if matches!(self.after_exchange, Some(true)) { - // If the plan is after exchange plan in cluster mode, - // the order column is at the last of the input schema. - debug_assert_eq!(fields.last().unwrap().name(), ORDER_COL_NAME); - debug_assert_eq!( - fields.last().unwrap().data_type(), - &self.order_col_type(&input_schema)? - ); - fields.pop(); - } else { - if let Some(proj) = &self.pre_projection { - let fileted_fields = proj - .iter() - .filter_map(|index| input_schema.field_with_name(&index.to_string()).ok()) - .cloned() - .collect::>(); - if fileted_fields.len() < fields.len() { - // Only if the projection is not a full projection, we need to add a projection transform. - fields = fileted_fields - } - } - - if matches!(self.after_exchange, Some(false)) { - // If the plan is before exchange plan in cluster mode, - // the order column should be added to the output schema. - fields.push(DataField::new( - ORDER_COL_NAME, - self.order_col_type(&input_schema)?, - )); - } - } - - Ok(DataSchemaRefExt::create(fields)) - } -} - -impl PhysicalPlanBuilder { - pub(crate) async fn build_sort( - &mut self, - s_expr: &SExpr, - sort: &crate::plans::Sort, - mut required: ColumnSet, - stat_info: PlanStatsInfo, - ) -> Result { - // 1. Prune unused Columns. - sort.items.iter().for_each(|s| { - required.insert(s.index); - }); - - // If the query will be optimized by lazy reading, we don't need to do pre-projection. - let pre_projection: Option> = if self.metadata.read().lazy_columns().is_empty() { - sort.pre_projection.clone() - } else { - None - }; - let input_plan = self.build(s_expr.child(0)?, required).await?; - - let order_by = sort - .items - .iter() - .map(|v| SortDesc { - asc: v.asc, - nulls_first: v.nulls_first, - order_by: v.index, - display_name: self.metadata.read().column(v.index).name(), - }) - .collect::>(); - - // Add WindowPartition for parallel sort in window. - if let Some(window) = &sort.window_partition { - let window_partition = window - .partition_by - .iter() - .map(|v| v.index) - .collect::>(); - - return Ok(PhysicalPlan::WindowPartition(WindowPartition { - plan_id: 0, - input: Box::new(input_plan.clone()), - partition_by: window_partition.clone(), - order_by: order_by.clone(), - after_exchange: sort.after_exchange, - top_n: window.top.map(|top| WindowPartitionTopN { - func: match window.func { - WindowFuncType::RowNumber => WindowPartitionTopNFunc::RowNumber, - WindowFuncType::Rank => WindowPartitionTopNFunc::Rank, - WindowFuncType::DenseRank => WindowPartitionTopNFunc::DenseRank, - _ => unreachable!(), - }, - top, - }), - stat_info: Some(stat_info.clone()), - })); - }; - - // 2. Build physical plan. - Ok(PhysicalPlan::Sort(Sort { - plan_id: 0, - input: Box::new(input_plan), - order_by, - limit: sort.limit, - after_exchange: sort.after_exchange, - pre_projection, - stat_info: Some(stat_info), - })) - } -} diff --git a/src/query/sql/src/executor/physical_plans/physical_union_all.rs b/src/query/sql/src/executor/physical_plans/physical_union_all.rs deleted file mode 100644 index 3605c0b037403..0000000000000 --- a/src/query/sql/src/executor/physical_plans/physical_union_all.rs +++ /dev/null @@ -1,169 +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_expression::DataField; -use databend_common_expression::DataSchema; -use databend_common_expression::DataSchemaRef; -use databend_common_expression::DataSchemaRefExt; -use databend_common_expression::RemoteExpr; - -use crate::executor::explain::PlanStatsInfo; -use crate::executor::PhysicalPlan; -use crate::executor::PhysicalPlanBuilder; -use crate::optimizer::ir::SExpr; -use crate::ColumnSet; -use crate::IndexType; -use crate::ScalarExpr; -use crate::TypeCheck; - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct UnionAll { - // A unique id of operator in a `PhysicalPlan` tree, only used for display. - pub plan_id: u32, - pub left: Box, - pub right: Box, - pub left_outputs: Vec<(IndexType, Option)>, - pub right_outputs: Vec<(IndexType, Option)>, - pub schema: DataSchemaRef, - pub cte_scan_names: Vec, - - // Only used for explain - pub stat_info: Option, -} - -impl UnionAll { - pub fn output_schema(&self) -> Result { - Ok(self.schema.clone()) - } -} - -impl PhysicalPlanBuilder { - pub(crate) async fn build_union_all( - &mut self, - s_expr: &SExpr, - union_all: &crate::plans::UnionAll, - mut required: ColumnSet, - stat_info: PlanStatsInfo, - ) -> Result { - // 1. Prune unused Columns. - let metadata = self.metadata.read().clone(); - let lazy_columns = metadata.lazy_columns(); - required.extend(lazy_columns); - - // Use left's output columns as the offset indices - // if the union has a CTE, the output columns are not filtered - // otherwise, if the output columns of the union do not contain the columns used by the plan in the union, the expression will fail to obtain data. - let (offset_indices, left_required, right_required) = - if !union_all.cte_scan_names.is_empty() { - let left: ColumnSet = union_all - .left_outputs - .iter() - .map(|(index, _)| *index) - .collect(); - let right: ColumnSet = union_all - .right_outputs - .iter() - .map(|(index, _)| *index) - .collect(); - - let offset_indices: Vec = (0..union_all.left_outputs.len()).collect(); - (offset_indices, left, right) - } else { - let offset_indices: Vec = (0..union_all.left_outputs.len()) - .filter(|index| required.contains(&union_all.output_indexes[*index])) - .collect(); - - if offset_indices.is_empty() { - ( - vec![0], - ColumnSet::from([union_all.left_outputs[0].0]), - ColumnSet::from([union_all.right_outputs[0].0]), - ) - } else { - offset_indices.iter().fold( - (vec![], ColumnSet::default(), ColumnSet::default()), - |(mut offset_indices, mut left, mut right), &index| { - left.insert(union_all.left_outputs[index].0); - right.insert(union_all.right_outputs[index].0); - offset_indices.push(index); - (offset_indices, left, right) - }, - ) - } - }; - - // 2. Build physical plan. - let left_plan = self.build(s_expr.child(0)?, left_required.clone()).await?; - let right_plan = self.build(s_expr.child(1)?, right_required.clone()).await?; - - let left_schema = left_plan.output_schema()?; - let right_schema = right_plan.output_schema()?; - - let left_outputs = process_outputs(&union_all.left_outputs, &offset_indices, &left_schema)?; - let right_outputs = - process_outputs(&union_all.right_outputs, &offset_indices, &right_schema)?; - - let mut fields = Vec::with_capacity(offset_indices.len()); - for offset in offset_indices { - let index = union_all.output_indexes[offset]; - let data_type = if let Some(scalar_expr) = &union_all.left_outputs[offset].1 { - let expr = scalar_expr - .type_check(left_schema.as_ref())? - .project_column_ref(|idx| left_schema.index_of(&idx.to_string()).unwrap()); - expr.data_type().clone() - } else { - let col_index = union_all.left_outputs[offset].0; - left_schema - .field_with_name(&col_index.to_string())? - .data_type() - .clone() - }; - - fields.push(DataField::new(&index.to_string(), data_type)); - } - - Ok(PhysicalPlan::UnionAll(UnionAll { - plan_id: 0, - left: Box::new(left_plan), - right: Box::new(right_plan), - left_outputs, - right_outputs, - schema: DataSchemaRefExt::create(fields), - - cte_scan_names: union_all.cte_scan_names.clone(), - stat_info: Some(stat_info), - })) - } -} - -fn process_outputs( - outputs: &[(IndexType, Option)], - offset_indices: &[usize], - schema: &DataSchema, -) -> Result)>> { - let mut results = Vec::with_capacity(offset_indices.len()); - for index in offset_indices { - let output = &outputs[*index]; - if let Some(scalar_expr) = &output.1 { - let expr = scalar_expr - .type_check(schema)? - .project_column_ref(|idx| schema.index_of(&idx.to_string()).unwrap()); - results.push((output.0, Some(expr.as_remote_expr()))); - } else { - results.push((output.0, None)); - } - } - Ok(results) -} diff --git a/src/query/sql/src/executor/physical_plans/physical_window_partition.rs b/src/query/sql/src/executor/physical_plans/physical_window_partition.rs deleted file mode 100644 index b0ff12d3f8685..0000000000000 --- a/src/query/sql/src/executor/physical_plans/physical_window_partition.rs +++ /dev/null @@ -1,52 +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_expression::DataSchemaRef; - -use crate::executor::explain::PlanStatsInfo; -use crate::executor::physical_plans::SortDesc; -use crate::executor::PhysicalPlan; -use crate::IndexType; - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct WindowPartition { - pub plan_id: u32, - pub input: Box, - pub partition_by: Vec, - pub order_by: Vec, - pub after_exchange: Option, - pub top_n: Option, - - pub stat_info: Option, -} - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct WindowPartitionTopN { - pub func: WindowPartitionTopNFunc, - pub top: usize, -} - -#[derive(Clone, Copy, Debug, serde::Serialize, serde::Deserialize)] -pub enum WindowPartitionTopNFunc { - RowNumber, - Rank, - DenseRank, -} - -impl WindowPartition { - pub fn output_schema(&self) -> Result { - self.input.output_schema() - } -} diff --git a/src/query/sql/src/executor/physical_plans/physical_mutation_into_split.rs b/src/query/sql/src/planner/binder/async_function_desc.rs similarity index 58% rename from src/query/sql/src/executor/physical_plans/physical_mutation_into_split.rs rename to src/query/sql/src/planner/binder/async_function_desc.rs index f94b1955afddb..44e1e575b3917 100644 --- a/src/query/sql/src/executor/physical_plans/physical_mutation_into_split.rs +++ b/src/query/sql/src/planner/binder/async_function_desc.rs @@ -12,21 +12,18 @@ // 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 databend_common_expression::types::DataType; -use crate::executor::physical_plan::PhysicalPlan; +use crate::plans::AsyncFunctionArgument; use crate::IndexType; -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct MutationSplit { - pub plan_id: u32, - pub input: Box, - pub split_index: IndexType, -} +#[derive(Clone, Debug, Eq, PartialEq, serde::Serialize, serde::Deserialize)] +pub struct AsyncFunctionDesc { + pub func_name: String, + pub display_name: String, + pub output_column: IndexType, + pub arg_indices: Vec, + pub data_type: Box, -impl MutationSplit { - pub fn output_schema(&self) -> Result { - self.input.output_schema() - } + pub func_arg: AsyncFunctionArgument, } diff --git a/src/query/sql/src/planner/binder/default_expr.rs b/src/query/sql/src/planner/binder/default_expr.rs index 272ed22d80d5e..872fb275b5287 100644 --- a/src/query/sql/src/planner/binder/default_expr.rs +++ b/src/query/sql/src/planner/binder/default_expr.rs @@ -38,7 +38,7 @@ use parking_lot::RwLock; use crate::binder::expr_values::ExprValuesRewriter; use crate::binder::wrap_cast; -use crate::executor::physical_plans::AsyncFunctionDesc; +use crate::binder::AsyncFunctionDesc; use crate::planner::binder::BindContext; use crate::planner::semantic::NameResolutionContext; use crate::planner::semantic::TypeChecker; diff --git a/src/query/sql/src/planner/binder/mod.rs b/src/query/sql/src/planner/binder/mod.rs index 25f54a71e84ef..1a5fd1d563f3a 100644 --- a/src/query/sql/src/planner/binder/mod.rs +++ b/src/query/sql/src/planner/binder/mod.rs @@ -13,6 +13,7 @@ // limitations under the License. mod aggregate; +mod async_function_desc; mod bind_context; mod bind_mutation; mod bind_query; @@ -60,6 +61,7 @@ mod virtual_column; mod window; pub use aggregate::AggregateInfo; +pub use async_function_desc::AsyncFunctionDesc; pub use bind_context::*; pub use bind_mutation::target_probe; pub use bind_mutation::MutationStrategy; diff --git a/src/query/sql/src/planner/execution/query_executor.rs b/src/query/sql/src/planner/execution/query_executor.rs index b6b6e430f5d35..2b448ac04f11b 100644 --- a/src/query/sql/src/planner/execution/query_executor.rs +++ b/src/query/sql/src/planner/execution/query_executor.rs @@ -16,12 +16,7 @@ use async_trait::async_trait; use databend_common_exception::Result; use databend_common_expression::DataBlock; -use crate::executor::PhysicalPlan; - #[async_trait] pub trait QueryExecutor: Send + Sync { - async fn execute_query_with_physical_plan(&self, plan: &PhysicalPlan) - -> Result>; - async fn execute_query_with_sql_string(&self, sql: &str) -> Result>; } diff --git a/src/query/sql/src/planner/optimizer/ir/expr/s_expr.rs b/src/query/sql/src/planner/optimizer/ir/expr/s_expr.rs index 118372f96546c..2de469b9ce282 100644 --- a/src/query/sql/src/planner/optimizer/ir/expr/s_expr.rs +++ b/src/query/sql/src/planner/optimizer/ir/expr/s_expr.rs @@ -46,7 +46,7 @@ use crate::IndexType; )] pub struct SExpr { pub plan: Arc, - pub(crate) children: Vec>, + pub children: Vec>, pub(crate) original_group: Option, diff --git a/src/query/sql/src/planner/optimizer/optimizers/hyper_dp/dynamic_sample/dynamic_sample.rs b/src/query/sql/src/planner/optimizer/optimizers/hyper_dp/dynamic_sample/dynamic_sample.rs deleted file mode 100644 index eb645151c4230..0000000000000 --- a/src/query/sql/src/planner/optimizer/optimizers/hyper_dp/dynamic_sample/dynamic_sample.rs +++ /dev/null @@ -1,146 +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::ops::Deref; -use std::sync::Arc; -use std::time::Duration; - -use databend_common_base::base::tokio::time::Instant; -use databend_common_catalog::table_context::TableContext; -use databend_common_exception::Result; - -use crate::optimizer::ir::RelExpr; -use crate::optimizer::ir::SExpr; -use crate::optimizer::ir::StatInfo; -use crate::optimizer::optimizers::hyper_dp::dynamic_sample::filter_selectivity_sample; -use crate::optimizer::optimizers::hyper_dp::dynamic_sample::join_selectivity_sample; -use crate::planner::QueryExecutor; -use crate::plans::Aggregate; -use crate::plans::AggregateMode; -use crate::plans::Limit; -use crate::plans::Operator; -use crate::plans::ProjectSet; -use crate::plans::RelOperator; -use crate::plans::UnionAll; -use crate::MetadataRef; - -#[async_recursion::async_recursion(#[recursive::recursive])] -pub async fn dynamic_sample( - ctx: Arc, - metadata: MetadataRef, - s_expr: &SExpr, - sample_executor: Arc, -) -> Result> { - let time_budget = - Duration::from_millis(ctx.get_settings().get_dynamic_sample_time_budget_ms()?); - let start_time = Instant::now(); - - async fn sample_with_budget( - start_time: Instant, - time_budget: Duration, - fallback: F, - sample_fn: impl FnOnce() -> Fut, - ) -> Result> - where - F: FnOnce() -> Result>, - Fut: std::future::Future>>, - { - if time_budget.as_millis() == 0 || start_time.elapsed() > time_budget { - fallback() - } else { - let remaining_time = time_budget - start_time.elapsed(); - match tokio::time::timeout(remaining_time, sample_fn()).await { - Ok(Ok(result)) => Ok(result), - // The error contains the timeout error or the error from the sample_fn - Ok(Err(_)) | Err(_) => fallback(), - } - } - } - - match s_expr.plan() { - RelOperator::Filter(_) => { - sample_with_budget( - start_time, - time_budget, - || { - let rel_expr = RelExpr::with_s_expr(s_expr); - rel_expr.derive_cardinality() - }, - || filter_selectivity_sample(ctx, metadata, s_expr, sample_executor), - ) - .await - } - RelOperator::Join(_) => { - join_selectivity_sample(ctx, metadata, s_expr, sample_executor).await - } - RelOperator::Scan(_) - | RelOperator::DummyTableScan(_) - | RelOperator::ConstantTableScan(_) - | RelOperator::CacheScan(_) - | RelOperator::ExpressionScan(_) - | RelOperator::RecursiveCteScan(_) - | RelOperator::Mutation(_) - | RelOperator::CompactBlock(_) - | RelOperator::MutationSource(_) => { - s_expr.plan().derive_stats(&RelExpr::with_s_expr(s_expr)) - } - - RelOperator::Aggregate(agg) => { - let child_stat_info = - dynamic_sample(ctx, metadata, s_expr.child(0)?, sample_executor).await?; - if agg.mode == AggregateMode::Final { - return Ok(child_stat_info); - } - let agg = Aggregate::try_from(s_expr.plan().clone())?; - agg.derive_agg_stats(child_stat_info) - } - RelOperator::Limit(_) => { - let child_stat_info = - dynamic_sample(ctx, metadata, s_expr.child(0)?, sample_executor).await?; - let limit = Limit::try_from(s_expr.plan().clone())?; - limit.derive_limit_stats(child_stat_info) - } - RelOperator::UnionAll(_) => { - let left_stat_info = dynamic_sample( - ctx.clone(), - metadata.clone(), - s_expr.child(0)?, - sample_executor.clone(), - ) - .await?; - let right_stat_info = - dynamic_sample(ctx, metadata, s_expr.child(1)?, sample_executor).await?; - let union = UnionAll::try_from(s_expr.plan().clone())?; - union.derive_union_stats(left_stat_info, right_stat_info) - } - RelOperator::ProjectSet(_) => { - let mut child_stat_info = - dynamic_sample(ctx, metadata, s_expr.child(0)?, sample_executor) - .await? - .deref() - .clone(); - let project_set = ProjectSet::try_from(s_expr.plan().clone())?; - project_set.derive_project_set_stats(&mut child_stat_info) - } - - RelOperator::EvalScalar(_) - | RelOperator::Sort(_) - | RelOperator::Exchange(_) - | RelOperator::Window(_) - | RelOperator::Udf(_) - | RelOperator::AsyncFunction(_) => { - dynamic_sample(ctx, metadata, s_expr.child(0)?, sample_executor).await - } - } -} diff --git a/src/query/sql/src/planner/optimizer/optimizers/hyper_dp/dynamic_sample/filter_selectivity_sample.rs b/src/query/sql/src/planner/optimizer/optimizers/hyper_dp/dynamic_sample/filter_selectivity_sample.rs deleted file mode 100644 index 4f4b1244d4e49..0000000000000 --- a/src/query/sql/src/planner/optimizer/optimizers/hyper_dp/dynamic_sample/filter_selectivity_sample.rs +++ /dev/null @@ -1,145 +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::HashSet; -use std::sync::Arc; - -use databend_common_ast::ast::SampleConfig; -use databend_common_ast::ast::SampleRowLevel; -use databend_common_catalog::table_context::TableContext; -use databend_common_exception::ErrorCode; -use databend_common_exception::Result; -use databend_common_expression::types::DataType; -use databend_common_expression::types::NumberDataType; -use num_traits::ToPrimitive; - -use crate::executor::PhysicalPlanBuilder; -use crate::optimizer::ir::RelExpr; -use crate::optimizer::ir::SExpr; -use crate::optimizer::ir::SelectivityEstimator; -use crate::optimizer::ir::StatInfo; -use crate::optimizer::statistics::CollectStatisticsOptimizer; -use crate::optimizer::OptimizerContext; -use crate::planner::QueryExecutor; -use crate::plans::Aggregate; -use crate::plans::AggregateFunction; -use crate::plans::AggregateMode; -use crate::plans::RelOperator; -use crate::plans::ScalarItem; -use crate::ColumnSet; -use crate::MetadataRef; -use crate::ScalarExpr; - -pub async fn filter_selectivity_sample( - ctx: Arc, - metadata: MetadataRef, - s_expr: &SExpr, - sample_executor: Arc, -) -> Result> { - // filter cardinality by sample will be called in `dphyp`, so we can ensure the filter is in complex query(contains not only one table) - // Because it's meaningless for filter cardinality by sample in single table query. - let child = s_expr.child(0)?; - let child_rel_expr = RelExpr::with_s_expr(child); - if let RelOperator::Scan(mut scan) = child.plan().clone() { - let num_rows = scan - .statistics - .table_stats - .as_ref() - .and_then(|s| s.num_rows) - .unwrap_or(0); - // Calculate sample size (0.2% of total data) - let sample_size = (num_rows as f64 * 0.002).ceil(); - let mut new_s_expr = s_expr.clone(); - // If the table is too small, we don't need to sample. - if sample_size >= 10.0 { - let sample_conf = SampleConfig { - row_level: Some(SampleRowLevel::RowsNum(sample_size)), - block_level: Some(50.0), - }; - scan.sample = Some(sample_conf); - let new_child = SExpr::create_leaf(Arc::new(RelOperator::Scan(scan))); - new_s_expr = s_expr.replace_children(vec![Arc::new(new_child)]); - - let opt_ctx = OptimizerContext::new(ctx.clone(), metadata.clone()); - let mut collect_statistics_optimizer = CollectStatisticsOptimizer::new(opt_ctx); - new_s_expr = collect_statistics_optimizer - .optimize_async(&new_s_expr) - .await?; - } - - new_s_expr = SExpr::create_unary( - Arc::new(create_count_aggregate(AggregateMode::Partial).into()), - Arc::new(new_s_expr), - ); - new_s_expr = SExpr::create_unary( - Arc::new(create_count_aggregate(AggregateMode::Final).into()), - Arc::new(new_s_expr), - ); - - let mut builder = PhysicalPlanBuilder::new(metadata.clone(), ctx.clone(), false); - let mut required = ColumnSet::new(); - required.insert(0); - let plan = builder.build(&new_s_expr, required).await?; - - let result = sample_executor - .execute_query_with_physical_plan(&plan) - .await?; - if let Some(block) = result.first() { - if let Some(count) = block.get_last_column().as_number() { - if let Some(number_scalar) = count.index(0) { - // Compute and return selectivity - let selectivity = number_scalar.to_f64().to_f64().unwrap() / sample_size; - let stat_info = child_rel_expr.derive_cardinality()?; - let mut statistics = stat_info.statistics.clone(); - let mut sb = SelectivityEstimator::new( - &mut statistics, - stat_info.cardinality, - HashSet::new(), - ); - sb.update_other_statistic_by_selectivity(selectivity); - let stat_info = Arc::new(StatInfo { - cardinality: (selectivity * num_rows as f64).ceil(), - statistics, - }); - *s_expr.stat_info.lock().unwrap() = Some(stat_info.clone()); - return Ok(stat_info); - } - } - } - } - Err(ErrorCode::Internal( - "Failed to calculate filter selectivity by sample".to_string(), - )) -} - -fn create_count_aggregate(mode: AggregateMode) -> Aggregate { - Aggregate { - mode, - group_items: vec![], - aggregate_functions: vec![ScalarItem { - scalar: ScalarExpr::AggregateFunction(AggregateFunction { - span: None, - func_name: "count".to_string(), - distinct: false, - params: vec![], - args: vec![], - return_type: Box::new(DataType::Number(NumberDataType::UInt64)), - sort_descs: vec![], - display_name: "".to_string(), - }), - index: 0, - }], - ..Default::default() - } -} diff --git a/src/query/sql/src/planner/optimizer/optimizers/hyper_dp/dynamic_sample/join_selectivity_sample.rs b/src/query/sql/src/planner/optimizer/optimizers/hyper_dp/dynamic_sample/join_selectivity_sample.rs deleted file mode 100644 index 8c65a3a02a57a..0000000000000 --- a/src/query/sql/src/planner/optimizer/optimizers/hyper_dp/dynamic_sample/join_selectivity_sample.rs +++ /dev/null @@ -1,49 +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::table_context::TableContext; -use databend_common_exception::Result; - -use crate::optimizer::ir::SExpr; -use crate::optimizer::ir::StatInfo; -use crate::optimizer::optimizers::hyper_dp::dynamic_sample::dynamic_sample; -use crate::planner::QueryExecutor; -use crate::plans::Join; -use crate::MetadataRef; - -pub async fn join_selectivity_sample( - ctx: Arc, - metadata: MetadataRef, - s_expr: &SExpr, - sample_executor: Arc, -) -> Result> { - let left_stat_info = dynamic_sample( - ctx.clone(), - metadata.clone(), - s_expr.child(0)?, - sample_executor.clone(), - ) - .await?; - let right_stat_info = dynamic_sample( - ctx.clone(), - metadata.clone(), - s_expr.child(1)?, - sample_executor.clone(), - ) - .await?; - let join = Join::try_from(s_expr.plan().clone())?; - join.derive_join_stats(left_stat_info, right_stat_info) -} diff --git a/src/query/sql/src/planner/optimizer/optimizers/hyper_dp/dynamic_sample/mod.rs b/src/query/sql/src/planner/optimizer/optimizers/hyper_dp/dynamic_sample/mod.rs deleted file mode 100644 index ef397234e5cf0..0000000000000 --- a/src/query/sql/src/planner/optimizer/optimizers/hyper_dp/dynamic_sample/mod.rs +++ /dev/null @@ -1,22 +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. - -#[allow(clippy::module_inception)] -mod dynamic_sample; -mod filter_selectivity_sample; -mod join_selectivity_sample; - -pub use dynamic_sample::dynamic_sample; -pub use filter_selectivity_sample::filter_selectivity_sample; -pub use join_selectivity_sample::join_selectivity_sample; diff --git a/src/query/sql/src/planner/optimizer/optimizers/hyper_dp/join_relation.rs b/src/query/sql/src/planner/optimizer/optimizers/hyper_dp/join_relation.rs index a022b558a5d14..52e6033e46269 100644 --- a/src/query/sql/src/planner/optimizer/optimizers/hyper_dp/join_relation.rs +++ b/src/query/sql/src/planner/optimizer/optimizers/hyper_dp/join_relation.rs @@ -21,21 +21,18 @@ use databend_common_exception::Result; use crate::optimizer::ir::RelExpr; use crate::optimizer::ir::SExpr; -use crate::optimizer::optimizers::hyper_dp::dynamic_sample::dynamic_sample; use crate::planner::QueryExecutor; use crate::IndexType; use crate::MetadataRef; pub struct JoinRelation { s_expr: SExpr, - sample_executor: Option>, } impl JoinRelation { - pub fn new(s_expr: &SExpr, sample_executor: Option>) -> Self { + pub fn new(s_expr: &SExpr, _sample_executor: Option>) -> Self { Self { s_expr: s_expr.clone(), - sample_executor, } } @@ -45,22 +42,11 @@ impl JoinRelation { pub async fn cardinality( &self, - ctx: Arc, - metadata: MetadataRef, + _ctx: Arc, + _metadata: MetadataRef, ) -> Result { - let card = if let Some(sample_executor) = &self.sample_executor { - dynamic_sample( - ctx.clone(), - metadata.clone(), - &self.s_expr, - sample_executor.clone(), - ) - .await? - .cardinality - } else { - let rel_expr = RelExpr::with_s_expr(&self.s_expr); - rel_expr.derive_cardinality()?.cardinality - }; + let rel_expr = RelExpr::with_s_expr(&self.s_expr); + let card = rel_expr.derive_cardinality()?.cardinality; Ok(card) } } diff --git a/src/query/sql/src/planner/optimizer/optimizers/hyper_dp/mod.rs b/src/query/sql/src/planner/optimizer/optimizers/hyper_dp/mod.rs index c4e59477e1b54..411cbd7d40235 100644 --- a/src/query/sql/src/planner/optimizer/optimizers/hyper_dp/mod.rs +++ b/src/query/sql/src/planner/optimizer/optimizers/hyper_dp/mod.rs @@ -13,7 +13,6 @@ // limitations under the License. mod dphyp; -mod dynamic_sample; mod join_node; mod join_relation; mod query_graph; diff --git a/src/query/sql/src/planner/plans/r_cte_scan.rs b/src/query/sql/src/planner/plans/r_cte_scan.rs index b00aff2993c7e..a39bb2fc9e497 100644 --- a/src/query/sql/src/planner/plans/r_cte_scan.rs +++ b/src/query/sql/src/planner/plans/r_cte_scan.rs @@ -32,8 +32,8 @@ use crate::ColumnSet; #[derive(Clone, Debug, PartialEq, Eq)] pub struct RecursiveCteScan { - pub(crate) fields: Vec, - pub(crate) table_name: String, + pub fields: Vec, + pub table_name: String, } impl RecursiveCteScan {