diff --git a/Cargo.lock b/Cargo.lock index e892d62b79e4e..092cc3df02d6f 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4197,6 +4197,7 @@ dependencies = [ "similar", "simsearch", "tokio", + "typetag", "unicase", "url", ] 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..23877fb5fc6e4 100644 --- a/src/query/service/src/interpreters/interpreter_copy_into_location.rs +++ b/src/query/service/src/interpreters/interpreter_copy_into_location.rs @@ -20,7 +20,7 @@ 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_common_sql::executor::{IPhysicalPlan, PhysicalPlan, PhysicalPlanMeta}; use databend_storages_common_stage::CopyIntoLocationInfo; use log::debug; use log::info; @@ -89,14 +89,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: Box = 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 4c5694151dee6..ad52b4e1e431c 100644 --- a/src/query/service/src/interpreters/interpreter_copy_into_table.rs +++ b/src/query/service/src/interpreters/interpreter_copy_into_table.rs @@ -33,7 +33,7 @@ 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_sql::executor::{IPhysicalPlan, PhysicalPlan, PhysicalPlanMeta}; use databend_common_storage::StageFileInfo; use databend_common_storages_fuse::FuseTable; use databend_common_storages_stage::StageTable; @@ -100,7 +100,7 @@ impl CopyIntoTableInterpreter { &self, table_info: TableInfo, plan: &CopyIntoTablePlan, - ) -> Result<(PhysicalPlan, Vec)> { + ) -> Result<(Box, Vec)> { let to_table = self .ctx .get_table( @@ -125,7 +125,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(); ( @@ -145,21 +145,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: Box = 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(), @@ -171,16 +170,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"), }); } @@ -318,14 +318,14 @@ impl CopyIntoTableInterpreter { if self.plan.stage_table_info.copy_into_table_options.purge && !self - .plan - .stage_table_info - .duplicated_files_detected - .is_empty() + .plan + .stage_table_info + .duplicated_files_detected + .is_empty() && self - .ctx - .get_settings() - .get_enable_purge_duplicated_files_in_copy()? + .ctx + .get_settings() + .get_enable_purge_duplicated_files_in_copy()? { info!( "purge_duplicated_files_in_copy enabled, number of duplicated files: {}", @@ -337,7 +337,7 @@ impl CopyIntoTableInterpreter { self.plan.stage_table_info.duplicated_files_detected.clone(), self.plan.stage_table_info.stage_info.clone(), ) - .await?; + .await?; } Ok(PipelineBuildResult::create()) } @@ -406,7 +406,7 @@ impl Interpreter for CopyIntoTableInterpreter { unsafe { self.ctx.get_settings().get_deduplicate_label()? }, self.plan.path_prefix.clone(), ) - .await?; + .await?; } // Execute hook. diff --git a/src/query/service/src/interpreters/interpreter_explain.rs b/src/query/service/src/interpreters/interpreter_explain.rs index 88da04bed02bf..7cb468961e73d 100644 --- a/src/query/service/src/interpreters/interpreter_explain.rs +++ b/src/query/service/src/interpreters/interpreter_explain.rs @@ -34,7 +34,7 @@ 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::IPhysicalPlan; use databend_common_sql::executor::MutationBuildInfo; use databend_common_sql::plans::Mutation; use databend_common_sql::BindContext; @@ -68,6 +68,7 @@ use crate::sql::executor::PhysicalPlan; use crate::sql::executor::PhysicalPlanBuilder; use crate::sql::optimizer::ir::SExpr; use crate::sql::plans::Plan; +use databend_common_sql::executor::PhysicalPlanDynExt; pub struct ExplainInterpreter { ctx: Arc, @@ -164,22 +165,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 { @@ -324,7 +314,7 @@ impl ExplainInterpreter { pub async fn explain_physical_plan( &self, - plan: &PhysicalPlan, + plan: &Box, metadata: &MetadataRef, formatted_ast: &Option, ) -> Result> { @@ -360,25 +350,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 @@ -479,10 +459,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(); @@ -586,7 +565,7 @@ impl ExplainInterpreter { fn inject_pruned_partitions_stats( &self, - plan: &mut PhysicalPlan, + plan: &mut Box, metadata: &MetadataRef, ) -> Result<()> { let mut sources = vec![]; diff --git a/src/query/service/src/interpreters/interpreter_index_refresh.rs b/src/query/service/src/interpreters/interpreter_index_refresh.rs index ee4147d52ee1d..a5bdb339a853e 100644 --- a/src/query/service/src/interpreters/interpreter_index_refresh.rs +++ b/src/query/service/src/interpreters/interpreter_index_refresh.rs @@ -35,9 +35,10 @@ 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::{DeriveHandle, PhysicalPlanDynExt}; +use databend_common_sql::executor::IPhysicalPlan; 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; @@ -54,6 +55,7 @@ use crate::interpreters::Interpreter; use crate::pipelines::PipelineBuildResult; use crate::schedulers::build_query_pipeline_without_render_result_set; use crate::sessions::QueryContext; +use crate::test_kits::query_count; pub struct RefreshIndexInterpreter { ctx: Arc, @@ -120,32 +122,20 @@ impl RefreshIndexInterpreter { #[async_backtrace::framed] async fn get_read_source( &self, - query_plan: &PhysicalPlan, + query_plan: &Box, 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::new(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,29 @@ 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 new(source:DataSourcePlan) -> Box { + Box::new(ReadSourceDeriveHandle { source }) + } +} + +impl DeriveHandle for ReadSourceDeriveHandle { + fn derive( + &mut self, + v: &Box, + children: Vec>, + ) -> std::result::Result, Vec>> { + 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 e45615a8ae5f3..27bff610409e9 100644 --- a/src/query/service/src/interpreters/interpreter_insert.rs +++ b/src/query/service/src/interpreters/interpreter_insert.rs @@ -25,9 +25,9 @@ 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::{DistributedInsertSelect, Exchange}; use databend_common_sql::executor::physical_plans::MutationKind; -use databend_common_sql::executor::PhysicalPlan; +use databend_common_sql::executor::{IPhysicalPlan, PhysicalPlan, PhysicalPlanDynExt, PhysicalPlanMeta}; use databend_common_sql::executor::PhysicalPlanBuilder; use databend_common_sql::plans::Insert; use databend_common_sql::plans::InsertInputSource; @@ -174,55 +174,49 @@ 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"), + }) } }; - let mut build_res = - build_query_pipeline_without_render_result_set(&self.ctx, &insert_select_plan) - .await?; + 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?; table.commit_insertion( self.ctx.clone(), 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..2f085cf4548b8 100644 --- a/src/query/service/src/interpreters/interpreter_insert_multi_table.rs +++ b/src/query/service/src/interpreters/interpreter_insert_multi_table.rs @@ -39,8 +39,10 @@ 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::IPhysicalPlan; use databend_common_sql::executor::PhysicalPlan; use databend_common_sql::executor::PhysicalPlanBuilder; +use databend_common_sql::executor::PhysicalPlanMeta; use databend_common_sql::plans::Else; use databend_common_sql::plans::FunctionCall; use databend_common_sql::plans::InsertMultiTable; @@ -130,7 +132,7 @@ impl Interpreter for InsertMultiTableInterpreter { } impl InsertMultiTableInterpreter { - pub async fn build_physical_plan(&self) -> Result { + pub async fn build_physical_plan(&self) -> Result> { let (mut root, _) = self.build_source_physical_plan().await?; let update_stream_meta = dml_build_update_stream_req(self.ctx.clone()).await?; let source_schema = root.output_schema()?; @@ -174,69 +176,71 @@ 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) } - async fn build_source_physical_plan(&self) -> Result<(PhysicalPlan, MetadataRef)> { + async fn build_source_physical_plan(&self) -> Result<(Box, MetadataRef)> { match &self.plan.input_source { Plan::Query { s_expr, diff --git a/src/query/service/src/interpreters/interpreter_mutation.rs b/src/query/service/src/interpreters/interpreter_mutation.rs index 2e41ed14faaec..d6f0aeedc5dd3 100644 --- a/src/query/service/src/interpreters/interpreter_mutation.rs +++ b/src/query/service/src/interpreters/interpreter_mutation.rs @@ -32,6 +32,7 @@ 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::{IPhysicalPlan, PhysicalPlanDynExt}; use databend_common_sql::executor::MutationBuildInfo; use databend_common_sql::executor::PhysicalPlan; use databend_common_sql::executor::PhysicalPlanBuilder; @@ -98,9 +99,10 @@ 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); @@ -167,7 +169,7 @@ impl MutationInterpreter { &self, mutation: &Mutation, dry_run: bool, - ) -> Result { + ) -> Result> { // Prepare MutationBuildInfo for PhysicalPlanBuilder to build DataMutation physical plan. let mutation_build_info = build_mutation_info(self.ctx.clone(), mutation, dry_run).await?; // Build physical plan. @@ -230,7 +232,7 @@ pub async fn build_mutation_info( table_snapshot.clone(), dry_run, ) - .await?; + .await?; let table_meta_timestamps = ctx.get_table_meta_timestamps(table.as_ref(), table_snapshot.clone())?; @@ -282,8 +284,8 @@ async fn mutation_source_partitions( let cluster = ctx.get_cluster(); let is_lazy = fuse_table.is_column_oriented() || (!dry_run - && !cluster.is_empty() - && table_snapshot.segments.len() >= cluster.nodes.len()); + && !cluster.is_empty() + && table_snapshot.segments.len() >= cluster.nodes.len()); (is_lazy, true) } else { (false, false) diff --git a/src/query/service/src/interpreters/interpreter_replace.rs b/src/query/service/src/interpreters/interpreter_replace.rs index efbde608318cd..c1cb946e29673 100644 --- a/src/query/service/src/interpreters/interpreter_replace.rs +++ b/src/query/service/src/interpreters/interpreter_replace.rs @@ -1,5 +1,3 @@ -// 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 @@ -25,7 +23,7 @@ use databend_common_functions::BUILTIN_FUNCTIONS; 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::{cast_expr_to_non_null_boolean, IPhysicalPlan, PhysicalPlanDynExt, PhysicalPlanMeta}; use databend_common_sql::executor::physical_plans::CommitSink; use databend_common_sql::executor::physical_plans::CommitType; use databend_common_sql::executor::physical_plans::Exchange; @@ -133,7 +131,7 @@ impl ReplaceInterpreter { async fn build_physical_plan( &self, ) -> Result<( - Box, + Box, Option<(Vec, StageInfo, CopyIntoTableOptions)>, )> { let plan = &self.plan; @@ -264,34 +262,29 @@ 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 @@ -313,23 +306,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(), @@ -343,22 +334,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(), @@ -368,10 +359,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)) } @@ -424,7 +416,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, @@ -442,14 +434,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] @@ -482,8 +472,7 @@ impl ReplaceInterpreter { let physical_plan = select_interpreter .build_physical_plan() - .await - .map(Box::new)?; + .await?; let select_ctx = ReplaceSelectCtx { select_column_bindings: bind_context.columns.clone(), select_schema: query_plan.schema(), @@ -498,7 +487,7 @@ impl ReplaceInterpreter { } struct ReplaceSourceCtx { - root: Box, + root: Box, 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 4d2941f6aea66..5f434eff91720 100644 --- a/src/query/service/src/interpreters/interpreter_select.rs +++ b/src/query/service/src/interpreters/interpreter_select.rs @@ -32,7 +32,8 @@ use databend_common_pipeline_core::Pipe; 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::physical_plans::{Exchange, FragmentKind}; +use databend_common_sql::executor::{IPhysicalPlan, PhysicalPlanDynExt}; use databend_common_sql::executor::PhysicalPlan; use databend_common_sql::parse_result_scan_args; use databend_common_sql::ColumnBinding; @@ -110,7 +111,7 @@ impl SelectInterpreter { #[inline] #[async_backtrace::framed] - pub async fn build_physical_plan(&self) -> Result { + pub async fn build_physical_plan(&self) -> Result> { let mut builder = PhysicalPlanBuilder::new(self.metadata.clone(), self.ctx.clone(), false); self.ctx .set_status_info("[SELECT-INTERP] Building physical plan"); @@ -122,9 +123,9 @@ impl SelectInterpreter { #[async_backtrace::framed] pub async fn build_pipeline( &self, - mut physical_plan: PhysicalPlan, + mut physical_plan: Box, ) -> 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; } @@ -136,7 +137,7 @@ impl SelectInterpreter { &physical_plan, self.ignore_result, ) - .await?; + .await?; // consume stream let update_stream_metas = query_build_update_stream_req(&self.ctx).await?; @@ -285,9 +286,11 @@ 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 d9dc116c15ea5..875b63c72d1a8 100644 --- a/src/query/service/src/interpreters/interpreter_table_analyze.rs +++ b/src/query/service/src/interpreters/interpreter_table_analyze.rs @@ -19,14 +19,14 @@ 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::{AggregateExpand, Exchange}; 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::{DeriveHandle, IPhysicalPlan, PhysicalPlan, PhysicalPlanDynExt}; use databend_common_sql::executor::PhysicalPlanBuilder; use databend_common_sql::plans::AnalyzeTablePlan; use databend_common_sql::plans::Plan; @@ -59,7 +59,7 @@ impl AnalyzeTableInterpreter { Ok(AnalyzeTableInterpreter { ctx, plan }) } - async fn plan_sql(&self, sql: String) -> Result<(PhysicalPlan, BindContext)> { + async fn plan_sql(&self, sql: String) -> Result<(Box, BindContext)> { let mut planner = Planner::new(self.ctx.clone()); let (plan, _) = planner.plan_sql(&sql).await?; let (select_plan, bind_context) = match &plan { @@ -234,7 +234,7 @@ impl Interpreter for AnalyzeTableInterpreter { &histogram_plan, false, ) - .await?; + .await?; let (tx, rx) = async_channel::unbounded(); histogram_build_res.main_pipeline.add_sink(|input_port| { Ok(ProcessorPtr::create(HistogramInfoSink::create( @@ -269,76 +269,24 @@ 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, +fn remove_exchange(plan: Box) -> Box { + struct RemoveExchangeHandle; + + impl DeriveHandle for RemoveExchangeHandle { + fn derive( + &mut self, + v: &Box, + mut children: Vec>, + ) -> std::result::Result, Vec>> { + let Some(_) = v.downcast_ref::() else { + return Err(children); + }; + + assert_eq!(children.len(), 1); + Ok(children.remove(0)) } } - traverse(plan) + let mut handle = Box::new(RemoveExchangeHandle); + plan.derive_with(&mut handle) } 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..9d51d7433ebb9 100644 --- a/src/query/service/src/interpreters/interpreter_table_modify_column.rs +++ b/src/query/service/src/interpreters/interpreter_table_modify_column.rs @@ -39,7 +39,7 @@ 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::{IPhysicalPlan, PhysicalPlan, PhysicalPlanMeta}; use databend_common_sql::executor::PhysicalPlanBuilder; use databend_common_sql::plans::ModifyColumnAction; use databend_common_sql::plans::ModifyTableColumnPlan; @@ -227,7 +227,7 @@ impl ModifyTableColumnInterpreter { for (index_name, index) in &table_info.meta.indexes { if index.column_ids.contains(&old_field.column_id) && old_field.data_type.remove_nullable() - != field.data_type.remove_nullable() + != field.data_type.remove_nullable() { return Err(ErrorCode::ColumnReferencedByInvertedIndex(format!( "column `{}` is referenced by inverted index, drop inverted index `{}` first", @@ -289,7 +289,7 @@ impl ModifyTableColumnInterpreter { table_info.meta.clone(), catalog, ) - .await?; + .await?; return Ok(PipelineBuildResult::create()); } @@ -452,7 +452,7 @@ impl ModifyTableColumnInterpreter { prev_snapshot_id, table_meta_timestamps, ) - .await + .await } // Set column comment. @@ -492,7 +492,7 @@ impl ModifyTableColumnInterpreter { table_info.meta.clone(), catalog, ) - .await?; + .await?; } Ok(PipelineBuildResult::create()) @@ -650,7 +650,7 @@ impl Interpreter for ModifyTableColumnInterpreter { table_meta, column.to_string(), ) - .await? + .await? } }; @@ -685,9 +685,9 @@ fn is_string_to_binary(old_ty: &TableDataType, new_ty: &TableDataType) -> bool { ) => { old_tys.len() == new_tys.len() && old_tys - .iter() - .zip(new_tys) - .all(|(old_ty, new_ty)| is_string_to_binary(old_ty, new_ty)) + .iter() + .zip(new_tys) + .all(|(old_ty, new_ty)| is_string_to_binary(old_ty, new_ty)) } _ => false, } @@ -727,16 +727,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: Box = 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..5f02d1ab21d64 100644 --- a/src/query/service/src/interpreters/interpreter_table_recluster.rs +++ b/src/query/service/src/interpreters/interpreter_table_recluster.rs @@ -37,7 +37,7 @@ use databend_common_meta_app::schema::TableInfo; 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::{cast_expr_to_non_null_boolean, IPhysicalPlan, PhysicalPlanDynExt, PhysicalPlanMeta}; use databend_common_sql::executor::physical_plans::CommitSink; use databend_common_sql::executor::physical_plans::CommitType; use databend_common_sql::executor::physical_plans::CompactSource; @@ -299,7 +299,7 @@ impl ReclusterTableInterpreter { tbl: &Arc, push_downs: &mut Option, hilbert_info: &mut Option, - ) -> Result> { + ) -> Result>> { LicenseManagerSwitch::instance() .check_enterprise_enabled(self.ctx.get_license_key(), Feature::HilbertClustering)?; let handler = get_hilbert_clustering_handler(); @@ -330,11 +330,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 +419,13 @@ 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 +446,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 +462,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( @@ -493,7 +489,7 @@ impl ReclusterTableInterpreter { tbl: &Arc, push_downs: &mut Option, limit: Option, - ) -> Result> { + ) -> Result>> { let Some((parts, snapshot)) = tbl .recluster(self.ctx.clone(), push_downs.clone(), limit) .await? @@ -516,12 +512,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 +536,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,43 +701,42 @@ impl ReclusterTableInterpreter { } fn add_commit_sink( - input: PhysicalPlan, + mut input: Box, is_distributed: bool, table_info: TableInfo, snapshot: Arc, merge_meta: bool, recluster_info: Option, table_meta_timestamps: TableMetaTimestamps, - ) -> PhysicalPlan { - let plan = if is_distributed { - PhysicalPlan::Exchange(Exchange { - plan_id: 0, - input: Box::new(input), + ) -> Box { + 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 a6115a346ab93..8f555b64cb849 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)] diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs index e2e2b1c30a001..e0210b0653eef 100644 --- a/src/query/service/src/pipelines/pipeline_builder.rs +++ b/src/query/service/src/pipelines/pipeline_builder.rs @@ -26,7 +26,7 @@ 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 databend_common_sql::executor::{IPhysicalPlan, PhysicalPlan}; use super::PipelineBuilderData; use crate::interpreters::CreateTableInterpreter; @@ -81,8 +81,8 @@ impl PipelineBuilder { } } - pub fn finalize(mut self, plan: &PhysicalPlan) -> Result { - self.build_pipeline(plan)?; + pub fn finalize(mut self, plan: &Box) -> Result { + // self.build_pipeline(plan)?; for source_pipeline in &self.pipelines { if !source_pipeline.is_complete_pipeline()? { @@ -136,7 +136,7 @@ impl PipelineBuilder { let scope = PlanScope::create( plan.get_id(), - plan.name(), + plan.get_name(), Arc::new(desc), Arc::new(profile_labels), ); 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 41469da10a4f4..4283a4b5f8b58 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 @@ -314,8 +314,7 @@ async fn create_memory_table_for_cte_scan( table_indexes: None, attached_columns: None, }; - let create_table_interpreter = - CreateTableInterpreter::try_create(ctx.clone(), create_table_plan)?; + let create_table_interpreter = CreateTableInterpreter::try_create(ctx.clone(), create_table_plan)?; let _ = create_table_interpreter.execute(ctx.clone()).await?; } PhysicalPlan::Shuffle(plan) => { diff --git a/src/query/service/src/schedulers/fragments/fragmenter.rs b/src/query/service/src/schedulers/fragments/fragmenter.rs index 2e7a6e878b819..bb1fe33df58df 100644 --- a/src/query/service/src/schedulers/fragments/fragmenter.rs +++ b/src/query/service/src/schedulers/fragments/fragmenter.rs @@ -12,12 +12,14 @@ // 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::table_context::TableContext; use databend_common_exception::Result; +use databend_common_functions::aggregates::assert_arguments; use databend_common_meta_types::NodeInfo; -use databend_common_sql::executor::physical_plans::CompactSource; +use databend_common_sql::executor::physical_plans::{BroadcastSink, 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; @@ -31,7 +33,8 @@ 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 databend_common_sql::executor::{DeriveHandle, PhysicalPlanDynExt, PhysicalPlanMeta, PhysicalPlanVisitor}; +use databend_common_sql::executor::IPhysicalPlan; use crate::clusters::ClusterHelper; use crate::schedulers::fragments::plan_fragment::FragmentType; @@ -47,9 +50,9 @@ use crate::sql::executor::PhysicalPlan; /// Visitor to split a `PhysicalPlan` into fragments. pub struct Fragmenter { ctx: Arc, - fragments: Vec, - query_id: String, state: State, + query_id: String, + fragments: Vec, } /// A state to track if is visiting a source fragment, useful when building fragments. @@ -121,224 +124,223 @@ impl Fragmenter { } } - 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 { + pub fn build_fragment(mut self, plan: &Box) -> Result> { + let mut fragments = HashMap::new(); + let mut handle = FragmentDeriveHandle::new(self.query_id.clone(), self.ctx.clone(), &mut fragments); + let root = plan.derive_with(&mut handle); + + let mut fragment_type = FragmentType::Root; + if let Some(_broadcast_sink) = plan.downcast_ref::() { + fragment_type = FragmentType::Intermediate; + } + + 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 { + if let Some(exchange_sink) = fragments[&source].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())) - } - fn replace_constant_table_scan(&mut self, plan: &ConstantTableScan) -> Result { - self.state = State::SelectLeaf; - Ok(PhysicalPlan::ConstantTableScan(plan.clone())) + Ok(fragments.into_values().collect::>()) } - 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 collect_fragments_edge<'a>(iter: impl Iterator) -> HashMap { + struct EdgeVisitor { + target_fragment_id: usize, + map: HashMap, + } - 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() - }))) - } + impl PhysicalPlanVisitor for EdgeVisitor { + fn visit(&mut self, plan: &Box) -> 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); + } + } - // TODO(Sky): remove redundant code - fn replace_copy_into_table(&mut self, plan: &CopyIntoTable) -> Result { - match &plan.source { - CopyIntoTableSource::Stage(_) => { - self.state = State::SelectLeaf; - Ok(PhysicalPlan::CopyIntoTable(Box::new(plan.clone()))) - } - CopyIntoTableSource::Query(query_physical_plan) => { - let input = self.replace(query_physical_plan)?; - Ok(PhysicalPlan::CopyIntoTable(Box::new(CopyIntoTable { - source: CopyIntoTableSource::Query(Box::new(input)), - ..plan.clone() - }))) + Ok(()) } } - } - 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()))) - } + let mut edges = HashMap::new(); + for fragment in iter { + let mut visitor = Box::new(EdgeVisitor { map: HashMap::new(), target_fragment_id: fragment.fragment_id }); + fragment.plan.visit(&mut visitor).unwrap(); + edges.extend(visitor.map.into_iter()) + } - 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, - })) + edges } +} - 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(); - - // 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(); +struct FragmentDeriveHandle<'a> { + state: State, + query_id: String, + ctx: Arc, + fragments: &'a mut HashMap, +} - fragments.append(&mut self.fragments); - self.fragments = fragments; +impl<'a> FragmentDeriveHandle<'a> { + pub fn new(query_id: String, ctx: Arc, fragments: &'a mut HashMap) -> Box { + Box::new(FragmentDeriveHandle { + ctx, + query_id, + fragments, + state: State::Other, + }) + } +} - // 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 { +impl<'a> DeriveHandle for FragmentDeriveHandle<'a> { + fn derive( + &mut self, + v: &Box, + children: Vec>, + ) -> std::result::Result, Vec>> { + 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 { - self.state = State::Other; + } 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::UnionAll(UnionAll { - left: Box::new(left_input), - right: Box::new(right_input), - ..plan.clone() - })) - } - - 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, - - input: Box::new(input), - schema: input_schema.clone(), - kind: plan.kind.clone(), - keys: plan.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: 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, - }; - self.state = State::Other; - let exchange = Self::get_exchange(self.ctx.clone(), &plan)?; - - let mut source_fragment = PlanFragment { - plan, - fragment_type, - - fragment_id: source_fragment_id, - exchange, - query_id: self.query_id.clone(), + if let Some(exchange) = v.downcast_ref::() { + let input = children.remove(0); + let input_schema = input.output_schema()?; + + let plan_id = v.get_id(); + let source_fragment_id = self.ctx.get_fragment_id(); + + let plan = 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, + }; - source_fragments: self.fragments.drain(..).collect(), - }; - - // Fill the destination_fragment_id for source fragments of `source_fragment`. - Self::resolve_fragment_connection(&mut source_fragment); - - self.fragments.push(source_fragment); - - Ok(PhysicalPlan::ExchangeSource(ExchangeSource { - // TODO(leiysky): we reuse the plan id here, - // should generate a new one for the source. - plan_id, - - schema: input_schema, - query_id: self.query_id.clone(), + self.state = State::Other; + let exchange = Self::get_exchange(self.ctx.clone(), &plan)?; + + 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) } } +// impl PhysicalPlanReplacer for Fragmenter { + +// 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(); +// +// // 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(); +// +// fragments.append(&mut self.fragments); +// self.fragments = fragments; +// +// // 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; +// } +// +// Ok(PhysicalPlan::UnionAll(UnionAll { +// left: Box::new(left_input), +// right: Box::new(right_input), +// ..plan.clone() +// })) +// } +// } diff --git a/src/query/service/src/schedulers/fragments/plan_fragment.rs b/src/query/service/src/schedulers/fragments/plan_fragment.rs index 18f2b35267eb4..18466cad61370 100644 --- a/src/query/service/src/schedulers/fragments/plan_fragment.rs +++ b/src/query/service/src/schedulers/fragments/plan_fragment.rs @@ -28,11 +28,15 @@ 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::ExchangeSink; 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_sql::executor::DeriveHandle; +use databend_common_sql::executor::IPhysicalPlan; +use databend_common_sql::executor::PhysicalPlanDynExt; use databend_common_storages_fuse::TableContext; use databend_storages_common_table_meta::meta::BlockSlotDescription; use databend_storages_common_table_meta::meta::Location; @@ -44,7 +48,6 @@ 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)] @@ -68,7 +71,7 @@ pub enum FragmentType { #[derive(Clone)] pub struct PlanFragment { - pub plan: PhysicalPlan, + pub plan: Box, pub fragment_type: FragmentType, pub fragment_id: usize, pub exchange: Option, @@ -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::new(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: Box = 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::new(parts); + let plan = self.plan.derive_with(&mut handle); fragment_actions.add_action(QueryFragmentAction::create(executor, plan)); } @@ -276,13 +273,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::new(parts, None, need_insert); + plan = plan.derive_with(&mut handle); fragment_actions.add_action(QueryFragmentAction::create(executor, plan)); } } @@ -291,17 +283,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::new( + 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)); } @@ -330,11 +322,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_compact_source = ReplaceCompactBlock { partitions: parts }; - plan = replace_compact_source.replace(&plan)?; - + let mut handle = CompactSourceDeriveHandle::new(parts); + let plan = self.plan.derive_with(&mut handle); fragment_actions.add_action(QueryFragmentAction::create(executor, plan)); } @@ -367,9 +356,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::new(tasks); + let plan = self.plan.derive_with(&mut handle); fragment_actions.add_action(QueryFragmentAction::create(executor, plan)); } @@ -487,137 +475,178 @@ 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 new(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 derive( + &mut self, + v: &Box, + children: Vec>, + ) -> std::result::Result, Vec>> { + 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()) + }; - 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() - }))) - } + return Ok(Box::new(TableScan { + source: Box::new(DataSourcePlan::try_from(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 new(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 derive( + &mut self, + v: &Box, + children: Vec>, + ) -> std::result::Result, Vec>> { + 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 new(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 derive( + &mut self, + v: &Box, + children: Vec>, + ) -> std::result::Result, Vec>> { + 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 new(partitions: Partitions) -> Box { + Box::new(CompactSourceDeriveHandle { partitions }) + } +} + +impl DeriveHandle for CompactSourceDeriveHandle { + fn derive( + &mut self, + v: &Box, + children: Vec>, + ) -> std::result::Result, Vec>> { + 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 new( + partitions: Vec<(usize, Location)>, + slot: Option, + need_insert: bool, + ) -> Box { + Box::new(ReplaceDeriveHandle { + partitions, + slot, + need_insert, + }) } +} - fn replace_deduplicate(&mut self, plan: &ReplaceDeduplicate) -> Result { - let input = self.replace(&plan.input)?; - Ok(PhysicalPlan::ReplaceDeduplicate(Box::new( - ReplaceDeduplicate { - input: Box::new(input), +impl DeriveHandle for ReplaceDeriveHandle { + fn derive( + &mut self, + v: &Box, + children: Vec>, + ) -> std::result::Result, Vec>> { + if let Some(replace_into) = v.downcast_ref::() { + assert_eq!(children.len(), 1); + return Ok(Box::new(ReplaceInto { + input: children[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[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..6a5145d251129 100644 --- a/src/query/service/src/schedulers/fragments/query_fragment_actions.rs +++ b/src/query/service/src/schedulers/fragments/query_fragment_actions.rs @@ -24,7 +24,8 @@ use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::DataSchemaRef; use databend_common_meta_types::NodeInfo; - +use databend_common_sql::executor::{IPhysicalPlan, PhysicalPlanDynExt}; +use databend_common_sql::executor::physical_plans::ExchangeSink; use crate::clusters::ClusterHelper; use crate::servers::flight::v1::exchange::DataExchange; use crate::servers::flight::v1::packets::DataflowDiagramBuilder; @@ -38,15 +39,15 @@ 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: Box, } impl QueryFragmentAction { - pub fn create(executor: String, physical_plan: PhysicalPlan) -> QueryFragmentAction { + pub fn create(executor: String, physical_plan: Box) -> QueryFragmentAction { QueryFragmentAction { - physical_plan, executor, + physical_plan, } } } @@ -131,7 +132,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 +264,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..ef7127be3479f 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 @@ -14,7 +14,7 @@ use std::fmt::Display; use std::fmt::Formatter; - +use databend_common_sql::executor::PhysicalPlanDynExt; use databend_common_sql::MetadataRef; use crate::schedulers::QueryFragmentActions; @@ -77,11 +77,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..2d0ddeed40ec6 100644 --- a/src/query/service/src/schedulers/scheduler.rs +++ b/src/query/service/src/schedulers/scheduler.rs @@ -18,6 +18,7 @@ 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::executor::IPhysicalPlan; use databend_common_sql::planner::QueryExecutor; use databend_common_sql::Planner; use futures_util::TryStreamExt; @@ -42,7 +43,7 @@ use crate::stream::PullingExecutorStream; pub async fn build_query_pipeline( ctx: &Arc, result_columns: &[ColumnBinding], - plan: &PhysicalPlan, + plan: &Box, ignore_result: bool, ) -> Result { let mut build_res = build_query_pipeline_without_render_result_set(ctx, plan).await?; @@ -61,7 +62,7 @@ pub async fn build_query_pipeline( #[async_backtrace::framed] pub async fn build_query_pipeline_without_render_result_set( ctx: &Arc, - plan: &PhysicalPlan, + plan: &Box, ) -> Result { let build_res = if !plan.is_distributed_plan() { build_local_pipeline(ctx, plan).await @@ -79,22 +80,23 @@ pub async fn build_query_pipeline_without_render_result_set( #[async_backtrace::framed] pub async fn build_local_pipeline( ctx: &Arc, - plan: &PhysicalPlan, + plan: &Box, ) -> Result { let pipeline = PipelineBuilder::create(ctx.get_function_context()?, ctx.get_settings(), ctx.clone()); - let mut build_res = pipeline.finalize(plan)?; + // let mut build_res = pipeline.finalize(plan)?; - let settings = ctx.get_settings(); - build_res.set_max_threads(settings.get_max_threads()? as usize); - Ok(build_res) + // let settings = ctx.get_settings(); + // build_res.set_max_threads(settings.get_max_threads()? as usize); + // Ok(build_res) + unimplemented!() } /// Build distributed pipeline via fragment and actions. #[async_backtrace::framed] pub async fn build_distributed_pipeline( ctx: &Arc, - plan: &PhysicalPlan, + plan: &Box, ) -> Result { let mut fragments_actions = QueryFragmentsActions::create(ctx.clone()); for plan in build_broadcast_plans(ctx.as_ref())? @@ -138,7 +140,7 @@ impl ServiceQueryExecutor { impl QueryExecutor for ServiceQueryExecutor { async fn execute_query_with_physical_plan( &self, - plan: &PhysicalPlan, + plan: &Box, ) -> Result> { let build_res = build_query_pipeline_without_render_result_set(&self.ctx, plan).await?; let settings = ExecutorSettings::try_create(self.ctx.clone())?; 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 1bb5af570df33..60a4ab64aefb3 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 @@ -36,7 +36,7 @@ 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 databend_common_sql::executor::{IPhysicalPlan, PhysicalPlan}; use fastrace::prelude::*; use log::warn; use parking_lot::Mutex; @@ -964,7 +964,7 @@ impl QueryCoordinator { struct FragmentCoordinator { initialized: bool, fragment_id: usize, - physical_plan: PhysicalPlan, + physical_plan: Box, data_exchange: Option, pipeline_build_res: Option, } 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..4b722bda6e844 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,23 @@ use std::fmt::Debug; use std::fmt::Formatter; +use databend_common_sql::executor::IPhysicalPlan; + 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: Box, } impl QueryFragment { pub fn create( fragment_id: usize, - physical_plan: PhysicalPlan, data_exchange: Option, + physical_plan: Box, ) -> QueryFragment { QueryFragment { physical_plan, diff --git a/src/query/service/tests/it/pipelines/builders/runtime_filter.rs b/src/query/service/tests/it/pipelines/builders/runtime_filter.rs index b739612ea5a44..a7cb04f5c0a69 100644 --- a/src/query/service/tests/it/pipelines/builders/runtime_filter.rs +++ b/src/query/service/tests/it/pipelines/builders/runtime_filter.rs @@ -18,7 +18,7 @@ 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::{IPhysicalPlan, PhysicalPlan}; use databend_common_sql::executor::PhysicalPlanBuilder; use databend_common_sql::plans::Plan; use databend_common_sql::Planner; @@ -42,7 +42,7 @@ async fn execute_sql(ctx: Arc, sql: &str) -> Result, sql: &str) -> Result { +async fn physical_plan(ctx: Arc, sql: &str) -> Result> { let plan = plan_sql(ctx.clone(), sql).await?; match plan { Plan::Query { diff --git a/src/query/sql/Cargo.toml b/src/query/sql/Cargo.toml index 1ced7b5eb052e..2fd6c166f24f4 100644 --- a/src/query/sql/Cargo.toml +++ b/src/query/sql/Cargo.toml @@ -47,6 +47,7 @@ chrono-tz = { workspace = true } cidr = { workspace = true } cron = { workspace = true } ctor = { workspace = true } +typetag = { workspace = true } dashmap = { workspace = true } derive-visitor = { workspace = true } educe = { workspace = true } diff --git a/src/query/sql/src/executor/format.rs b/src/query/sql/src/executor/format.rs index 684607c500e4e..8b69f1c713ea7 100644 --- a/src/query/sql/src/executor/format.rs +++ b/src/query/sql/src/executor/format.rs @@ -73,461 +73,9 @@ use crate::planner::DUMMY_TABLE_INDEX; use crate::plans::CacheSource; use crate::IndexType; -impl PhysicalPlan { - pub fn format( - &self, - metadata: MetadataRef, - profs: HashMap, - ) -> 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())) - } - } +pub struct FormatContext<'a> { + pub metadata: &'a Metadata, + pub scan_id_to_runtime_filters: HashMap>, } /// Helper function to add profile info to the format tree. @@ -571,525 +119,6 @@ fn append_output_rows_info( } } -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!( "{}({})", @@ -1102,594 +131,7 @@ pub fn pretty_display_agg_desc(desc: &AggregateFunctionDesc, metadata: &Metadata ) } -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> { +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)), @@ -1764,155 +206,14 @@ fn part_stats_info_to_format_tree(info: &PartStatistics) -> Vec Vec> { +pub 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( +pub fn format_output_columns( output_schema: DataSchemaRef, metadata: &Metadata, format_table: bool, diff --git a/src/query/sql/src/executor/mod.rs b/src/query/sql/src/executor/mod.rs index 3e40852552cac..f25007281bc7f 100644 --- a/src/query/sql/src/executor/mod.rs +++ b/src/query/sql/src/executor/mod.rs @@ -22,12 +22,15 @@ mod util; pub mod table_read_plan; -pub use format::format_partial_tree; +pub use physical_plan::DeriveHandle; +pub use physical_plan::IPhysicalPlan; pub use physical_plan::PhysicalPlan; +pub use physical_plan::PhysicalPlanDynExt; +pub use physical_plan::PhysicalPlanMeta; 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 physical_plan::PhysicalPlanVisitor; pub use util::*; diff --git a/src/query/sql/src/executor/physical_plan.rs b/src/query/sql/src/executor/physical_plan.rs index 33f29fabf5cd1..bd1ce0d9e11fd 100644 --- a/src/query/sql/src/executor/physical_plan.rs +++ b/src/query/sql/src/executor/physical_plan.rs @@ -12,15 +12,21 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::any::Any; +use std::any::TypeId; 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::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_exception::{ErrorCode, Result}; use databend_common_expression::DataSchemaRef; use databend_common_functions::BUILTIN_FUNCTIONS; +use databend_common_pipeline_core::PlanProfile; use educe::Educe; use enum_as_inner::EnumAsInner; use itertools::Itertools; @@ -33,6 +39,7 @@ use super::physical_plans::MutationManipulate; use super::physical_plans::MutationOrganize; use super::physical_plans::MutationSource; use super::physical_plans::MutationSplit; +use crate::executor::format::FormatContext; use crate::executor::physical_plans::AggregateExpand; use crate::executor::physical_plans::AggregateFinal; use crate::executor::physical_plans::AggregatePartial; @@ -78,6 +85,246 @@ use crate::executor::physical_plans::Udf; use crate::executor::physical_plans::UnionAll; use crate::executor::physical_plans::Window; use crate::executor::physical_plans::WindowPartition; +use crate::Metadata; + +#[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 derive( + &mut self, + v: &Box, + children: Vec>, + ) -> std::result::Result, Vec>>; +} + +#[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 get_desc(&self) -> Result { + Ok(String::new()) + } + + fn get_labels(&self) -> Result>> { + Ok(HashMap::new()) + } + + fn derive(&self, children: Vec>) -> Box; +} + +pub trait PhysicalPlanVisitor { + fn visit(&mut self, plan: &Box) -> 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) -> Box; + + 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 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; + } + + 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> { + // self.as_any().downcast_mut() + unimplemented!() + } + + fn derive_with(&self, handle: &mut Box) -> Box { + let mut children = vec![]; + for child in self.children() { + children.push(child.derive_with(handle)); + } + + self.derive(children) + } + + fn visit(&self, visitor: &mut Box) -> Result<()> { + for child in self.children() { + child.visit(visitor)?; + } + + visitor.visit(self) + } +} + +impl Clone for Box { + fn clone(&self) -> Self { + let mut children = vec![]; + for child in self.children() { + children.push(child.clone()); + } + + self.derive(children) + } +} #[derive(serde::Serialize, serde::Deserialize, Educe, EnumAsInner)] #[educe( @@ -164,382 +411,6 @@ pub enum PhysicalPlan { } 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 { @@ -602,538 +473,4 @@ impl PhysicalPlan { 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_builder.rs b/src/query/sql/src/executor/physical_plan_builder.rs index c50adc9f30db4..f03f51e71b8f5 100644 --- a/src/query/sql/src/executor/physical_plan_builder.rs +++ b/src/query/sql/src/executor/physical_plan_builder.rs @@ -25,6 +25,7 @@ use databend_storages_common_table_meta::meta::TableMetaTimestamps; use databend_storages_common_table_meta::meta::TableSnapshot; use crate::executor::explain::PlanStatsInfo; +use crate::executor::IPhysicalPlan; use crate::executor::PhysicalPlan; use crate::optimizer::ir::RelExpr; use crate::optimizer::ir::SExpr; @@ -62,7 +63,11 @@ impl PhysicalPlanBuilder { }) } - pub async fn build(&mut self, s_expr: &SExpr, required: ColumnSet) -> Result { + pub async fn build( + &mut self, + s_expr: &SExpr, + required: ColumnSet, + ) -> Result> { let mut plan = self.build_physical_plan(s_expr, required).await?; plan.adjust_plan_id(&mut 0); @@ -74,7 +79,7 @@ impl PhysicalPlanBuilder { &mut self, s_expr: &SExpr, required: ColumnSet, - ) -> Result { + ) -> Result> { // Build stat info. let stat_info = self.build_plan_stat_info(s_expr)?; match s_expr.plan() { diff --git a/src/query/sql/src/executor/physical_plan_visitor.rs b/src/query/sql/src/executor/physical_plan_visitor.rs index 7267b7665e370..987f4b4c2e1e6 100644 --- a/src/query/sql/src/executor/physical_plan_visitor.rs +++ b/src/query/sql/src/executor/physical_plan_visitor.rs @@ -12,790 +12,599 @@ // 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() - }, - ))) - } -} +use crate::executor::IPhysicalPlan; + +// 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), + plan: &'a Box, + pre_visit: &'b mut dyn FnMut(&'a Box) -> bool, + visit: &'b mut dyn FnMut(&'a Box), + post_visit: &'b mut dyn FnMut(&'a Box), ) { 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); - } + + for child in plan.children() { + Self::traverse(&child, pre_visit, visit, post_visit); } + post_visit(plan); } } diff --git a/src/query/sql/src/executor/physical_plans/physical_add_stream_column.rs b/src/query/sql/src/executor/physical_plans/physical_add_stream_column.rs index c08bb9660fe99..c0431407871f3 100644 --- a/src/query/sql/src/executor/physical_plans/physical_add_stream_column.rs +++ b/src/query/sql/src/executor/physical_plans/physical_add_stream_column.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::collections::HashMap; +use databend_common_ast::ast::FormatTreeNode; use databend_common_catalog::plan::StreamColumn; use databend_common_catalog::plan::StreamColumnType; use databend_common_exception::Result; @@ -23,7 +25,11 @@ 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 crate::executor::format::FormatContext; +use crate::executor::physical_plan::DeriveHandle; +use crate::executor::IPhysicalPlan; use crate::executor::PhysicalPlan; +use crate::executor::PhysicalPlanMeta; use crate::planner::CURRENT_BLOCK_ID_COL_NAME; use crate::planner::CURRENT_BLOCK_ROW_NUM_COL_NAME; use crate::plans::BoundColumnRef; @@ -37,20 +43,61 @@ use crate::Visibility; #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct AddStreamColumn { - pub plan_id: u32, - pub input: Box, + pub meta: PhysicalPlanMeta, + pub input: Box, 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>) -> Box { + 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 AddStreamColumn { pub fn new( metadata: &MetadataRef, - input: PhysicalPlan, + input: Box, 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); @@ -156,16 +203,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/sql/src/executor/physical_plans/physical_aggregate_expand.rs b/src/query/sql/src/executor/physical_plans/physical_aggregate_expand.rs index a9ea7055912cd..d2a00e9b3ca35 100644 --- a/src/query/sql/src/executor/physical_plans/physical_aggregate_expand.rs +++ b/src/query/sql/src/executor/physical_plans/physical_aggregate_expand.rs @@ -12,6 +12,10 @@ // 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::types::NumberDataType; @@ -20,16 +24,21 @@ use databend_common_expression::DataSchemaRef; use databend_common_expression::DataSchemaRefExt; use crate::executor::explain::PlanStatsInfo; +use crate::executor::format::format_output_columns; +use crate::executor::format::plan_stats_info_to_format_tree; +use crate::executor::format::FormatContext; +use crate::executor::physical_plan::DeriveHandle; +use crate::executor::IPhysicalPlan; use crate::executor::PhysicalPlan; +use crate::executor::PhysicalPlanMeta; 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 meta: PhysicalPlanMeta, + pub input: Box, pub group_bys: Vec, pub grouping_sets: GroupingSets, @@ -37,8 +46,20 @@ pub struct AggregateExpand { pub stat_info: Option, } -impl AggregateExpand { - pub fn output_schema(&self) -> Result { +#[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. @@ -64,4 +85,76 @@ impl AggregateExpand { )); 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>) -> Box { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } } diff --git a/src/query/sql/src/executor/physical_plans/physical_aggregate_final.rs b/src/query/sql/src/executor/physical_plans/physical_aggregate_final.rs index d3511fbb8cdaa..131001c857292 100644 --- a/src/query/sql/src/executor/physical_plans/physical_aggregate_final.rs +++ b/src/query/sql/src/executor/physical_plans/physical_aggregate_final.rs @@ -12,24 +12,37 @@ // 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::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::DataSchemaRefExt; use databend_common_expression::RemoteExpr; +use itertools::Itertools; use super::SortDesc; use crate::executor::explain::PlanStatsInfo; +use crate::executor::format::format_output_columns; +use crate::executor::format::plan_stats_info_to_format_tree; +use crate::executor::format::pretty_display_agg_desc; +use crate::executor::format::FormatContext; +use crate::executor::physical_plan::DeriveHandle; +use crate::executor::physical_plan::PhysicalPlanDynExt; 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::IPhysicalPlan; use crate::executor::PhysicalPlan; use crate::executor::PhysicalPlanBuilder; +use crate::executor::PhysicalPlanMeta; use crate::optimizer::ir::SExpr; use crate::plans::AggregateMode; use crate::plans::DummyTableScan; @@ -39,9 +52,8 @@ use crate::ScalarExpr; #[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: Box, pub group_by: Vec, pub agg_funcs: Vec, pub before_group_by_schema: DataSchemaRef, @@ -51,8 +63,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,6 +92,86 @@ 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>) -> Box { + 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 { @@ -77,7 +181,7 @@ impl PhysicalPlanBuilder { agg: &crate::plans::Aggregate, mut required: ColumnSet, stat_info: PlanStatsInfo, - ) -> Result { + ) -> Result> { // 1. Prune unused Columns. let mut used = vec![]; for item in &agg.aggregate_functions { @@ -112,7 +216,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: Box = match &agg.mode { AggregateMode::Partial => { let group_by_display = agg .group_items @@ -261,115 +365,116 @@ 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() + 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"), + }) + } 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"), }; - PhysicalPlan::Exchange(Exchange { - plan_id: 0, - kind, - allow_adjust_parallelism: true, - ignore_exchange: false, - input: Box::new(PhysicalPlan::AggregatePartial(aggregate_partial)), - keys, + 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"), }) - } - _ => { - 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, - }) - } } } } // 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 +596,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/sql/src/executor/physical_plans/physical_aggregate_partial.rs b/src/query/sql/src/executor/physical_plans/physical_aggregate_partial.rs index a8a73071aaa57..c13d758c1fad5 100644 --- a/src/query/sql/src/executor/physical_plans/physical_aggregate_partial.rs +++ b/src/query/sql/src/executor/physical_plans/physical_aggregate_partial.rs @@ -12,6 +12,11 @@ // 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::types::DataType; #[allow(unused_imports)] @@ -19,18 +24,24 @@ use databend_common_expression::DataBlock; use databend_common_expression::DataField; use databend_common_expression::DataSchemaRef; use databend_common_expression::DataSchemaRefExt; +use itertools::Itertools; use super::SortDesc; use crate::executor::explain::PlanStatsInfo; +use crate::executor::format::plan_stats_info_to_format_tree; +use crate::executor::format::pretty_display_agg_desc; +use crate::executor::format::FormatContext; +use crate::executor::physical_plan::DeriveHandle; use crate::executor::physical_plans::common::AggregateFunctionDesc; +use crate::executor::IPhysicalPlan; use crate::executor::PhysicalPlan; +use crate::executor::PhysicalPlanMeta; 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 meta: PhysicalPlanMeta, + pub input: Box, pub group_by: Vec, pub agg_funcs: Vec, pub enable_experimental_aggregate_hashtable: bool, @@ -42,8 +53,20 @@ pub struct AggregatePartial { pub stat_info: Option, } -impl AggregatePartial { - pub fn output_schema(&self) -> Result { +#[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()); @@ -63,4 +86,81 @@ impl AggregatePartial { 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>) -> Box { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } } diff --git a/src/query/sql/src/executor/physical_plans/physical_asof_join.rs b/src/query/sql/src/executor/physical_plans/physical_asof_join.rs index 7cc90d45a2b11..7dfbdcea85b6c 100644 --- a/src/query/sql/src/executor/physical_plans/physical_asof_join.rs +++ b/src/query/sql/src/executor/physical_plans/physical_asof_join.rs @@ -24,6 +24,7 @@ use crate::binder::bind_window_function_info; use crate::binder::ColumnBindingBuilder; use crate::binder::WindowFunctionInfo; use crate::binder::WindowOrderByInfo; +use crate::executor::IPhysicalPlan; use crate::executor::PhysicalPlan; use crate::executor::PhysicalPlanBuilder; use crate::optimizer::ir::RelExpr; @@ -55,7 +56,7 @@ impl PhysicalPlanBuilder { required: (ColumnSet, ColumnSet), mut range_conditions: Vec, mut other_conditions: Vec, - ) -> Result { + ) -> Result> { let mut window_index: usize = 0; if range_conditions.is_empty() { diff --git a/src/query/sql/src/executor/physical_plans/physical_async_func.rs b/src/query/sql/src/executor/physical_plans/physical_async_func.rs index 503a90dfac72f..a2500291cfb89 100644 --- a/src/query/sql/src/executor/physical_plans/physical_async_func.rs +++ b/src/query/sql/src/executor/physical_plans/physical_async_func.rs @@ -12,16 +12,27 @@ // 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 itertools::Itertools; use crate::executor::explain::PlanStatsInfo; +use crate::executor::format::format_output_columns; +use crate::executor::format::plan_stats_info_to_format_tree; +use crate::executor::format::FormatContext; +use crate::executor::physical_plan::DeriveHandle; +use crate::executor::IPhysicalPlan; use crate::executor::PhysicalPlan; use crate::executor::PhysicalPlanBuilder; +use crate::executor::PhysicalPlanMeta; use crate::optimizer::ir::SExpr; use crate::plans::AsyncFunctionArgument; use crate::ColumnSet; @@ -30,16 +41,27 @@ use crate::ScalarExpr; #[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: Box, 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,6 +71,58 @@ impl AsyncFunction { } 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 { + node_children.extend(plan_stats_info_to_format_tree(info)); + } + + node_children.extend(children); + + 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>) -> Box { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } } #[derive(Clone, Debug, Eq, PartialEq, serde::Serialize, serde::Deserialize)] @@ -69,7 +143,7 @@ impl PhysicalPlanBuilder { async_func_plan: &crate::plans::AsyncFunction, mut required: ColumnSet, stat_info: PlanStatsInfo, - ) -> Result { + ) -> Result> { // 1. Prune unused Columns. let mut used = vec![]; for item in async_func_plan.items.iter() { @@ -128,9 +202,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/sql/src/executor/physical_plans/physical_broadcast.rs b/src/query/sql/src/executor/physical_plans/physical_broadcast.rs index 36e42243e4ff0..29337e830da09 100644 --- a/src/query/sql/src/executor/physical_plans/physical_broadcast.rs +++ b/src/query/sql/src/executor/physical_plans/physical_broadcast.rs @@ -12,48 +12,120 @@ // 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 super::Exchange; use super::FragmentKind; +use crate::executor::format::FormatContext; +use crate::executor::physical_plan::DeriveHandle; +use crate::executor::IPhysicalPlan; use crate::executor::PhysicalPlan; +use crate::executor::PhysicalPlanMeta; #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct BroadcastSource { - pub plan_id: u32, + 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>) -> Box { + assert!(children.is_empty()); + Box::new(self.clone()) + } +} + #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct BroadcastSink { - pub plan_id: u32, + pub meta: PhysicalPlanMeta, pub broadcast_id: u32, - pub input: Box, + pub input: Box, +} + +#[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>) -> Box { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } } -pub fn build_broadcast_plan(broadcast_id: u32) -> Result { - let broadcast_source = Box::new(PhysicalPlan::BroadcastSource(BroadcastSource { - plan_id: 0, +pub fn build_broadcast_plan(broadcast_id: u32) -> Result> { + let broadcast_source: Box = Box::new(BroadcastSource { + meta: PhysicalPlanMeta::new("BroadcastSource"), broadcast_id, - })); - let exchange = Box::new(PhysicalPlan::Exchange(Exchange { - plan_id: 0, + }); + + let exchange = Box::new(Exchange { input: broadcast_source, kind: FragmentKind::Expansive, keys: vec![], allow_adjust_parallelism: true, ignore_exchange: false, - })); - let broadcast_sink = PhysicalPlan::BroadcastSink(BroadcastSink { - plan_id: 0, + meta: PhysicalPlanMeta::new("Exchange"), + }); + + Ok(Box::new(BroadcastSink { broadcast_id, input: exchange, - }); - Ok(broadcast_sink) + meta: PhysicalPlanMeta::new("BroadcastSink"), + })) } -pub fn build_broadcast_plans(ctx: &dyn TableContext) -> Result> { +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(); 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 index 98575dca5b3d5..fee02bc52e175 100644 --- a/src/query/sql/src/executor/physical_plans/physical_cache_scan.rs +++ b/src/query/sql/src/executor/physical_plans/physical_cache_scan.rs @@ -12,27 +12,80 @@ // 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_expression::DataSchemaRefExt; +use crate::executor::format::format_output_columns; +use crate::executor::format::FormatContext; +use crate::executor::physical_plan::DeriveHandle; +use crate::executor::IPhysicalPlan; use crate::executor::PhysicalPlan; use crate::executor::PhysicalPlanBuilder; +use crate::executor::PhysicalPlanMeta; 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 meta: PhysicalPlanMeta, pub cache_source: CacheSource, pub output_schema: DataSchemaRef, } -impl CacheScan { - pub fn output_schema(&self) -> Result { +#[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>) -> Box { + assert!(children.is_empty()); + Box::new(self.clone()) + } } impl PhysicalPlanBuilder { @@ -40,7 +93,7 @@ impl PhysicalPlanBuilder { &mut self, scan: &crate::plans::CacheScan, required: ColumnSet, - ) -> Result { + ) -> Result> { // 1. Prune unused Columns. let used: ColumnSet = required.intersection(&scan.columns).cloned().collect(); let (cache_source, fields) = if used == scan.columns { @@ -53,9 +106,9 @@ impl PhysicalPlanBuilder { ) }; // 2. Build physical plan. - Ok(PhysicalPlan::CacheScan(CacheScan { - plan_id: 0, + Ok(Box::new(CacheScan { cache_source, + meta: PhysicalPlanMeta::new("CacheScan"), output_schema: DataSchemaRefExt::create(fields), })) } diff --git a/src/query/sql/src/executor/physical_plans/physical_column_mutation.rs b/src/query/sql/src/executor/physical_plans/physical_column_mutation.rs index d3475e796ee16..6ebfad86d8e0c 100644 --- a/src/query/sql/src/executor/physical_plans/physical_column_mutation.rs +++ b/src/query/sql/src/executor/physical_plans/physical_column_mutation.rs @@ -12,19 +12,28 @@ // 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::DataSchemaRef; use databend_common_expression::RemoteExpr; use databend_common_meta_app::schema::TableInfo; use databend_storages_common_table_meta::meta::TableMetaTimestamps; +use crate::executor::format::FormatContext; +use crate::executor::physical_plan::DeriveHandle; use crate::executor::physical_plan::PhysicalPlan; use crate::executor::physical_plans::MutationKind; +use crate::executor::IPhysicalPlan; +use crate::executor::PhysicalPlanMeta; #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct ColumnMutation { - pub plan_id: u32, - pub input: Box, + pub meta: PhysicalPlanMeta, + pub input: Box, pub table_info: TableInfo, pub mutation_expr: Option>, pub computed_expr: Option>, @@ -34,3 +43,48 @@ pub struct ColumnMutation { pub has_filter_column: bool, pub table_meta_timestamps: TableMetaTimestamps, } + +#[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>) -> Box { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } +} 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 index 81f24be9f22b1..a06d7fb2d62e4 100644 --- a/src/query/sql/src/executor/physical_plans/physical_commit_sink.rs +++ b/src/query/sql/src/executor/physical_plans/physical_commit_sink.rs @@ -12,24 +12,31 @@ // 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::DataSourcePlan; use databend_common_catalog::plan::ReclusterInfoSideCar; +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_storages_common_table_meta::meta::TableMetaTimestamps; use databend_storages_common_table_meta::meta::TableSnapshot; +use crate::executor::physical_plan::DeriveHandle; use crate::executor::physical_plans::common::MutationKind; +use crate::executor::IPhysicalPlan; use crate::executor::PhysicalPlan; +use crate::executor::PhysicalPlanMeta; 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 meta: PhysicalPlanMeta, + pub input: Box, pub snapshot: Option>, pub table_info: TableInfo, pub commit_type: CommitType, @@ -41,6 +48,41 @@ pub struct CommitSink { 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>) -> Box { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } +} + #[derive(serde::Serialize, serde::Deserialize, Clone, Debug)] pub enum CommitType { Truncate { 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 index 70bb52b44da9a..587fd227d3e64 100644 --- a/src/query/sql/src/executor/physical_plans/physical_compact_source.rs +++ b/src/query/sql/src/executor/physical_plans/physical_compact_source.rs @@ -12,39 +12,64 @@ // 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_catalog::plan::DataSourcePlan; 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_expression::DataSchemaRef; use databend_common_meta_app::schema::TableInfo; use databend_storages_common_table_meta::meta::TableMetaTimestamps; +use crate::executor::physical_plan::DeriveHandle; 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::IPhysicalPlan; use crate::executor::PhysicalPlan; use crate::executor::PhysicalPlanBuilder; +use crate::executor::PhysicalPlanMeta; #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct CompactSource { - pub plan_id: u32, + 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>) -> Box { + assert!(children.is_empty()); + Box::new(self.clone()) + } +} + impl PhysicalPlanBuilder { pub async fn build_compact_block( &mut self, compact_block: &crate::plans::OptimizeCompactBlock, - ) -> Result { + ) -> Result> { let crate::plans::OptimizeCompactBlock { catalog, database, @@ -72,29 +97,29 @@ impl PhysicalPlanBuilder { .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 { + let mut root: Box = 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("ConstantTableScan"), + }); 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), + root = Box::new(Exchange { + input: root, kind: FragmentKind::Merge, keys: vec![], allow_adjust_parallelism: true, ignore_exchange: false, + meta: PhysicalPlanMeta::new("ConstantTableScan"), }); } - root = PhysicalPlan::CommitSink(Box::new(CommitSink { - input: Box::new(root), + root = Box::new(CommitSink { + input: root, table_info, snapshot: Some(snapshot), commit_type: CommitType::Mutation { @@ -103,10 +128,10 @@ impl PhysicalPlanBuilder { }, update_stream_meta: vec![], deduplicated_label: None, - plan_id: u32::MAX, recluster_info: None, table_meta_timestamps, - })); + meta: PhysicalPlanMeta::new("CommitSink"), + }); 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 index a67a9859348b7..ccf4c0182d418 100644 --- 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 @@ -12,29 +12,81 @@ // 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::DataSchemaRef; +use itertools::Itertools; +use crate::executor::format::format_output_columns; +use crate::executor::format::FormatContext; +use crate::executor::physical_plan::DeriveHandle; +use crate::executor::IPhysicalPlan; use crate::executor::PhysicalPlan; use crate::executor::PhysicalPlanBuilder; +use crate::executor::PhysicalPlanMeta; 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, + meta: PhysicalPlanMeta, pub values: Vec, pub num_rows: usize, pub output_schema: DataSchemaRef, } -impl ConstantTableScan { - pub fn output_schema(&self) -> Result { +#[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>) -> Box { + assert!(children.is_empty()); + Box::new(self.clone()) + } +} + +impl ConstantTableScan { pub fn name(&self) -> &str { if self.num_rows == 0 { "EmptyResultScan" @@ -49,7 +101,7 @@ impl PhysicalPlanBuilder { &mut self, scan: &crate::plans::ConstantTableScan, required: ColumnSet, - ) -> Result { + ) -> Result> { debug_assert!(scan .schema .fields @@ -66,19 +118,19 @@ impl PhysicalPlanBuilder { schema, .. } = scan.prune_columns(used); - return Ok(PhysicalPlan::ConstantTableScan(ConstantTableScan { - plan_id: 0, + return Ok(Box::new(ConstantTableScan { values, num_rows, output_schema: schema, + meta: PhysicalPlanMeta::new("ConstantTableScan"), })); } - Ok(PhysicalPlan::ConstantTableScan(ConstantTableScan { - plan_id: 0, + 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/sql/src/executor/physical_plans/physical_copy_into_location.rs b/src/query/sql/src/executor/physical_plans/physical_copy_into_location.rs index 5b069b8587664..0d742d09242fd 100644 --- 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 @@ -12,6 +12,10 @@ // 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::types::NumberDataType; @@ -21,25 +25,74 @@ use databend_common_expression::DataSchemaRefExt; use databend_common_expression::TableSchemaRef; use databend_storages_common_stage::CopyIntoLocationInfo; +use crate::executor::format::FormatContext; +use crate::executor::physical_plan::DeriveHandle; +use crate::executor::IPhysicalPlan; use crate::executor::PhysicalPlan; +use crate::executor::PhysicalPlanMeta; use crate::ColumnBinding; #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct CopyIntoLocation { - pub plan_id: u32, - pub input: Box, + pub meta: PhysicalPlanMeta, + 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 { +#[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>) -> Box { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } } 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 index 6d678ad12efce..4099cf1286435 100644 --- 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 @@ -12,6 +12,10 @@ // 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_catalog::plan::StageTableInfo; use databend_common_exception::Result; use databend_common_expression::DataSchemaRef; @@ -21,15 +25,18 @@ use databend_common_meta_app::schema::TableInfo; use databend_storages_common_table_meta::meta::TableMetaTimestamps; use enum_as_inner::EnumAsInner; +use crate::executor::format::FormatContext; +use crate::executor::physical_plan::DeriveHandle; use crate::executor::physical_plan::PhysicalPlan; +use crate::executor::IPhysicalPlan; +use crate::executor::PhysicalPlanMeta; 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 meta: PhysicalPlanMeta, pub required_values_schema: DataSchemaRef, pub values_consts: Vec, pub required_source_schema: DataSchemaRef, @@ -44,14 +51,72 @@ pub struct CopyIntoTable { pub table_meta_timestamps: TableMetaTimestamps, } -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize, EnumAsInner)] -pub enum CopyIntoTableSource { - Query(Box), - Stage(Box), -} +#[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 + } -impl CopyIntoTable { - pub fn output_schema(&self) -> Result { + 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>) -> Box { + 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) + } + } + } +} + +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize, EnumAsInner)] +pub enum CopyIntoTableSource { + Query(Box), + Stage(Box), } 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 index 678cf7b210637..4c24da61b2262 100644 --- 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 @@ -12,19 +12,24 @@ // 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_storages_common_table_meta::meta::TableMetaTimestamps; +use crate::executor::physical_plan::DeriveHandle; +use crate::executor::IPhysicalPlan; use crate::executor::PhysicalPlan; +use crate::executor::PhysicalPlanMeta; 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 meta: PhysicalPlanMeta, + pub input: Box, pub table_info: TableInfo, pub insert_schema: DataSchemaRef, pub select_schema: DataSchemaRef, @@ -32,3 +37,43 @@ pub struct DistributedInsertSelect { 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>) -> Box { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } +} diff --git a/src/query/sql/src/executor/physical_plans/physical_eval_scalar.rs b/src/query/sql/src/executor/physical_plans/physical_eval_scalar.rs index 89c30b3bc8e92..68412e0ef12bb 100644 --- a/src/query/sql/src/executor/physical_plans/physical_eval_scalar.rs +++ b/src/query/sql/src/executor/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,10 +28,17 @@ use databend_common_expression::DataSchemaRefExt; use databend_common_expression::RemoteExpr; use databend_common_expression::Scalar; use databend_common_functions::BUILTIN_FUNCTIONS; +use itertools::Itertools; use crate::executor::explain::PlanStatsInfo; +use crate::executor::format::format_output_columns; +use crate::executor::format::plan_stats_info_to_format_tree; +use crate::executor::format::FormatContext; +use crate::executor::physical_plan::DeriveHandle; use crate::executor::physical_plan::PhysicalPlan; use crate::executor::physical_plan_builder::PhysicalPlanBuilder; +use crate::executor::IPhysicalPlan; +use crate::executor::PhysicalPlanMeta; use crate::optimizer::ir::Matcher; use crate::optimizer::ir::SExpr; use crate::plans::Filter; @@ -44,18 +55,29 @@ use crate::TypeCheck; #[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: Box, 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,6 +100,83 @@ 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 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>) -> Box { + 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 { @@ -87,7 +186,7 @@ impl PhysicalPlanBuilder { eval_scalar: &crate::plans::EvalScalar, mut required: ColumnSet, stat_info: PlanStatsInfo, - ) -> Result { + ) -> Result> { // 1. Prune unused Columns. let column_projections = required.clone(); let mut used = vec![]; @@ -126,9 +225,9 @@ impl PhysicalPlanBuilder { &mut self, eval_scalar: &crate::plans::EvalScalar, column_projections: Vec, - input: PhysicalPlan, + input: Box, stat_info: PlanStatsInfo, - ) -> Result { + ) -> Result> { let input_schema = input.output_schema()?; let exprs = eval_scalar .items @@ -165,12 +264,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/sql/src/executor/physical_plans/physical_exchange.rs b/src/query/sql/src/executor/physical_plans/physical_exchange.rs index 1e831519c415b..1a6fc65046cb6 100644 --- a/src/query/sql/src/executor/physical_plans/physical_exchange.rs +++ b/src/query/sql/src/executor/physical_plans/physical_exchange.rs @@ -12,34 +12,105 @@ // 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::DataSchemaRef; use databend_common_expression::RemoteExpr; use databend_common_functions::BUILTIN_FUNCTIONS; +use crate::executor::format::format_output_columns; +use crate::executor::format::FormatContext; +use crate::executor::physical_plan::DeriveHandle; use crate::executor::physical_plans::common::FragmentKind; +use crate::executor::IPhysicalPlan; use crate::executor::PhysicalPlan; use crate::executor::PhysicalPlanBuilder; +use crate::executor::PhysicalPlanMeta; 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 meta: PhysicalPlanMeta, + 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() +#[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 derive(&self, mut children: Vec>) -> Box { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) } } @@ -49,7 +120,7 @@ impl PhysicalPlanBuilder { s_expr: &SExpr, exchange: &crate::plans::Exchange, mut required: ColumnSet, - ) -> Result { + ) -> Result> { // 1. Prune unused Columns. if let crate::plans::Exchange::Hash(exprs) = exchange { for expr in exprs { @@ -58,7 +129,7 @@ impl PhysicalPlanBuilder { } // 2. Build physical plan. - let input = Box::new(self.build(s_expr.child(0)?, required).await?); + 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; @@ -82,13 +153,13 @@ impl PhysicalPlanBuilder { FragmentKind::Merge } }; - Ok(PhysicalPlan::Exchange(Exchange { - plan_id: 0, + Ok(Box::new(Exchange { input, kind, keys, allow_adjust_parallelism, ignore_exchange: false, + meta: PhysicalPlanMeta::new("Exchange"), })) } } 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 index cb5ad775bf2bb..61837c2875c07 100644 --- a/src/query/sql/src/executor/physical_plans/physical_exchange_sink.rs +++ b/src/query/sql/src/executor/physical_plans/physical_exchange_sink.rs @@ -12,18 +12,26 @@ // 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 crate::executor::format::format_output_columns; +use crate::executor::format::FormatContext; +use crate::executor::physical_plan::DeriveHandle; use crate::executor::physical_plans::common::FragmentKind; +use crate::executor::IPhysicalPlan; use crate::executor::PhysicalPlan; +use crate::executor::PhysicalPlanMeta; #[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, + pub meta: PhysicalPlanMeta, + pub input: Box, // Input schema of exchanged data pub schema: DataSchemaRef, pub kind: FragmentKind, @@ -38,8 +46,68 @@ pub struct ExchangeSink { pub allow_adjust_parallelism: bool, } -impl ExchangeSink { - pub fn output_schema(&self) -> Result { +#[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 derive(&self, mut children: Vec>) -> Box { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } } 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 index 08ec4b44b4f86..c0d61ffb56d47 100644 --- a/src/query/sql/src/executor/physical_plans/physical_exchange_source.rs +++ b/src/query/sql/src/executor/physical_plans/physical_exchange_source.rs @@ -12,13 +12,22 @@ // 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 crate::executor::format::format_output_columns; +use crate::executor::format::FormatContext; +use crate::executor::physical_plan::DeriveHandle; +use crate::executor::IPhysicalPlan; +use crate::executor::PhysicalPlanMeta; + #[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, + pub meta: PhysicalPlanMeta, // Output schema of exchanged data pub schema: DataSchemaRef, @@ -28,8 +37,51 @@ pub struct ExchangeSource { pub query_id: String, } -impl ExchangeSource { - pub fn output_schema(&self) -> Result { +#[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 derive(&self, children: Vec>) -> Box { + assert!(children.is_empty()); + Box::new(self.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 index 5721f80368321..2a97a750109a2 100644 --- a/src/query/sql/src/executor/physical_plans/physical_expression_scan.rs +++ b/src/query/sql/src/executor/physical_plans/physical_expression_scan.rs @@ -12,31 +12,95 @@ // 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 itertools::Itertools; +use crate::executor::format::format_output_columns; +use crate::executor::format::FormatContext; +use crate::executor::physical_plan::DeriveHandle; +use crate::executor::IPhysicalPlan; use crate::executor::PhysicalPlan; use crate::executor::PhysicalPlanBuilder; +use crate::executor::PhysicalPlanMeta; 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 meta: PhysicalPlanMeta, pub values: Vec>, - pub input: Box, + pub input: Box, pub output_schema: DataSchemaRef, } -impl ExpressionScan { - pub fn output_schema(&self) -> Result { +#[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>) -> Box { + 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 { @@ -45,7 +109,7 @@ impl PhysicalPlanBuilder { s_expr: &SExpr, scan: &crate::plans::ExpressionScan, required: ColumnSet, - ) -> Result { + ) -> Result> { let input = self.build(s_expr.child(0)?, required).await?; let input_schema = input.output_schema()?; @@ -68,11 +132,11 @@ impl PhysicalPlanBuilder { }) .collect::>>()?; - Ok(PhysicalPlan::ExpressionScan(ExpressionScan { - plan_id: 0, + Ok(Box::new(ExpressionScan { values, - input: Box::new(input), + input, output_schema: scan.schema.clone(), + meta: PhysicalPlanMeta::new("ExpressionScan"), })) } } diff --git a/src/query/sql/src/executor/physical_plans/physical_filter.rs b/src/query/sql/src/executor/physical_plans/physical_filter.rs index 3051e5ec9307b..2f0eb2d56444a 100644 --- a/src/query/sql/src/executor/physical_plans/physical_filter.rs +++ b/src/query/sql/src/executor/physical_plans/physical_filter.rs @@ -12,27 +12,38 @@ // 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 itertools::Itertools; use crate::executor::cast_expr_to_non_null_boolean; use crate::executor::explain::PlanStatsInfo; +use crate::executor::format::format_output_columns; +use crate::executor::format::plan_stats_info_to_format_tree; +use crate::executor::format::FormatContext; +use crate::executor::physical_plan::DeriveHandle; +use crate::executor::IPhysicalPlan; use crate::executor::PhysicalPlan; use crate::executor::PhysicalPlanBuilder; +use crate::executor::PhysicalPlanMeta; 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, + meta: PhysicalPlanMeta, pub projections: ColumnSet, - pub input: Box, + pub input: Box, // Assumption: expression's data type must be `DataType::Boolean`. pub predicates: Vec, @@ -40,8 +51,20 @@ pub struct Filter { pub stat_info: Option, } -impl Filter { - pub fn output_schema(&self) -> Result { +#[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() { @@ -51,6 +74,75 @@ impl Filter { } 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>) -> Box { + 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 { @@ -60,14 +152,14 @@ impl PhysicalPlanBuilder { filter: &crate::plans::Filter, mut required: ColumnSet, stat_info: PlanStatsInfo, - ) -> Result { + ) -> 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?); + let input = self.build(s_expr.child(0)?, used).await?; required = required .union(self.metadata.read().get_retained_column()) .cloned() @@ -81,8 +173,8 @@ impl PhysicalPlanBuilder { } } - Ok(PhysicalPlan::Filter(Filter { - plan_id: 0, + Ok(Box::new(Filter { + meta: PhysicalPlanMeta::new("Filter"), projections, input, predicates: filter diff --git a/src/query/sql/src/executor/physical_plans/physical_hash_join.rs b/src/query/sql/src/executor/physical_plans/physical_hash_join.rs index 72b233e170f32..f206c63d87258 100644 --- a/src/query/sql/src/executor/physical_plans/physical_hash_join.rs +++ b/src/query/sql/src/executor/physical_plans/physical_hash_join.rs @@ -12,9 +12,11 @@ // 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 databend_common_ast::ast::FormatTreeNode; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::type_check::check_cast; @@ -26,13 +28,21 @@ use databend_common_expression::DataSchemaRef; use databend_common_expression::DataSchemaRefExt; use databend_common_expression::RemoteExpr; use databend_common_functions::BUILTIN_FUNCTIONS; +use itertools::Itertools; use super::physical_join_filter::PhysicalRuntimeFilters; use super::JoinRuntimeFilter; use crate::executor::explain::PlanStatsInfo; +use crate::executor::format::format_output_columns; +use crate::executor::format::plan_stats_info_to_format_tree; +use crate::executor::format::FormatContext; +use crate::executor::physical_plan::DeriveHandle; +use crate::executor::physical_plan::PhysicalPlanDynExt; use crate::executor::physical_plans::Exchange; +use crate::executor::IPhysicalPlan; use crate::executor::PhysicalPlan; use crate::executor::PhysicalPlanBuilder; +use crate::executor::PhysicalPlanMeta; use crate::optimizer::ir::SExpr; use crate::plans::Join; use crate::plans::JoinType; @@ -66,8 +76,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 +85,8 @@ pub struct HashJoin { pub probe_projections: ColumnSet, pub build_projections: ColumnSet, - pub build: Box, - pub probe: Box, + pub build: Box, + pub probe: Box, pub build_keys: Vec, pub probe_keys: Vec, pub is_null_equal: Vec, @@ -108,10 +117,209 @@ 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>) -> Box { + 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) + } } impl PhysicalPlanBuilder { @@ -121,9 +329,9 @@ impl PhysicalPlanBuilder { 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<(Box, Box)> { + 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)) } @@ -155,7 +363,7 @@ impl PhysicalPlanBuilder { pub(crate) fn prepare_build_schema( &self, join_type: &JoinType, - build_side: &PhysicalPlan, + build_side: &Box, ) -> Result { match join_type { JoinType::Left | JoinType::LeftSingle | JoinType::LeftAsof | JoinType::Full => { @@ -191,7 +399,7 @@ impl PhysicalPlanBuilder { pub(crate) fn prepare_probe_schema( &self, join_type: &JoinType, - probe_side: &PhysicalPlan, + probe_side: &Box, ) -> Result { match join_type { JoinType::Right | JoinType::RightSingle | JoinType::RightAsof | JoinType::Full => { @@ -219,51 +427,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 Box, + build_side: &mut Box, ) -> 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 +1000,8 @@ impl PhysicalPlanBuilder { &self, s_expr: &SExpr, join: &Join, - probe_side: Box, - build_side: Box, + probe_side: Box, + build_side: Box, projections: ColumnSet, probe_projections: ColumnSet, build_projections: ColumnSet, @@ -804,7 +1014,7 @@ impl PhysicalPlanBuilder { build_side_cache_info: Option<(usize, HashMap)>, runtime_filter: PhysicalRuntimeFilters, stat_info: PlanStatsInfo, - ) -> Result { + ) -> Result> { let build_side_data_distribution = s_expr.build_side_child().get_data_distribution()?; let broadcast_id = if build_side_data_distribution .as_ref() @@ -814,8 +1024,7 @@ impl PhysicalPlanBuilder { } else { None }; - Ok(PhysicalPlan::HashJoin(HashJoin { - plan_id: 0, + Ok(Box::new(HashJoin { projections, build_projections, probe_projections, @@ -827,6 +1036,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, @@ -848,7 +1058,7 @@ impl PhysicalPlanBuilder { left_required: ColumnSet, right_required: ColumnSet, stat_info: PlanStatsInfo, - ) -> Result { + ) -> Result> { // Step 1: Build probe and build sides let (mut probe_side, mut build_side) = self .build_join_sides(s_expr, left_required, right_required) diff --git a/src/query/sql/src/executor/physical_plans/physical_join.rs b/src/query/sql/src/executor/physical_plans/physical_join.rs index 1b09ecc6a2eed..e19e0ba868e1b 100644 --- a/src/query/sql/src/executor/physical_plans/physical_join.rs +++ b/src/query/sql/src/executor/physical_plans/physical_join.rs @@ -16,6 +16,7 @@ use databend_common_exception::Result; use crate::binder::JoinPredicate; use crate::executor::explain::PlanStatsInfo; +use crate::executor::IPhysicalPlan; use crate::executor::PhysicalPlan; use crate::executor::PhysicalPlanBuilder; use crate::optimizer::ir::RelExpr; @@ -131,7 +132,7 @@ impl PhysicalPlanBuilder { join: &crate::plans::Join, required: ColumnSet, stat_info: PlanStatsInfo, - ) -> Result { + ) -> Result> { // 1. Prune unused Columns. let mut others_required = join .non_equi_conditions diff --git a/src/query/sql/src/executor/physical_plans/physical_limit.rs b/src/query/sql/src/executor/physical_plans/physical_limit.rs index b0c3e64ecd713..6e576edb457b1 100644 --- a/src/query/sql/src/executor/physical_plans/physical_limit.rs +++ b/src/query/sql/src/executor/physical_plans/physical_limit.rs @@ -12,6 +12,11 @@ // 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; @@ -19,18 +24,23 @@ use databend_common_expression::DataSchemaRef; use databend_common_expression::ROW_ID_COL_NAME; use crate::executor::explain::PlanStatsInfo; +use crate::executor::format::format_output_columns; +use crate::executor::format::plan_stats_info_to_format_tree; +use crate::executor::format::FormatContext; +use crate::executor::physical_plan::DeriveHandle; use crate::executor::physical_plans::physical_row_fetch::RowFetch; +use crate::executor::IPhysicalPlan; use crate::executor::PhysicalPlan; use crate::executor::PhysicalPlanBuilder; +use crate::executor::PhysicalPlanMeta; use crate::optimizer::ir::SExpr; use crate::ColumnEntry; use crate::ColumnSet; #[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: Box, pub limit: Option, pub offset: usize, @@ -38,9 +48,86 @@ 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>) -> Box { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) } } @@ -51,7 +138,7 @@ impl PhysicalPlanBuilder { limit: &crate::plans::Limit, mut required: ColumnSet, stat_info: PlanStatsInfo, - ) -> Result { + ) -> Result> { // 1. Prune unused Columns. // Apply lazy. let metadata = self.metadata.read().clone(); @@ -66,12 +153,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 +173,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 +196,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 +233,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/sql/src/executor/physical_plans/physical_multi_table_insert.rs b/src/query/sql/src/executor/physical_plans/physical_multi_table_insert.rs index 41c1f88954dd6..ce330fa617077 100644 --- 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 @@ -12,33 +12,132 @@ // 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::DataSchemaRef; use databend_common_expression::RemoteExpr; +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_storages_common_table_meta::meta::TableMetaTimestamps; +use itertools::Itertools; +use crate::executor::format::FormatContext; +use crate::executor::physical_plan::DeriveHandle; +use crate::executor::IPhysicalPlan; use crate::executor::PhysicalPlan; +use crate::executor::PhysicalPlanMeta; use crate::ColumnSet; + #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct Duplicate { - pub plan_id: u32, - pub input: Box, + pub meta: PhysicalPlanMeta, + pub input: Box, 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>) -> Box { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } +} + #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct Shuffle { - pub plan_id: u32, - pub input: Box, + pub meta: PhysicalPlanMeta, + pub input: Box, 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 derive(&self, mut children: Vec>) -> Box { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } +} + #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub enum ShuffleStrategy { Transpose(usize), @@ -68,18 +167,143 @@ impl ShuffleStrategy { #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct ChunkFilter { - pub plan_id: u32, - pub input: Box, + pub meta: PhysicalPlanMeta, + pub input: Box, 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>) -> Box { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } +} + #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct ChunkEvalScalar { - pub plan_id: u32, - pub input: Box, + pub meta: PhysicalPlanMeta, + pub input: Box, 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>) -> Box { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } +} + #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct MultiInsertEvalScalar { pub remote_exprs: Vec, @@ -88,11 +312,51 @@ pub struct MultiInsertEvalScalar { #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct ChunkCastSchema { - pub plan_id: u32, - pub input: Box, + pub meta: PhysicalPlanMeta, + pub input: Box, 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 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>) -> Box { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } +} + #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct CastSchema { pub source_schema: DataSchemaRef, @@ -101,11 +365,52 @@ pub struct CastSchema { #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct ChunkFillAndReorder { - pub plan_id: u32, - pub input: Box, + pub meta: PhysicalPlanMeta, + pub input: Box, 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 derive(&self, mut children: Vec>) -> Box { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } +} + #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct FillAndReorder { pub source_schema: DataSchemaRef, @@ -114,11 +419,53 @@ pub struct FillAndReorder { #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct ChunkAppendData { - pub plan_id: u32, - pub input: Box, + pub meta: PhysicalPlanMeta, + pub input: Box, 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>) -> Box { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } +} + #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct SerializableTable { pub target_catalog_info: Arc, @@ -128,17 +475,89 @@ pub struct SerializableTable { #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct ChunkMerge { - pub plan_id: u32, - pub input: Box, + pub meta: PhysicalPlanMeta, + pub input: Box, 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 derive(&self, mut children: Vec>) -> Box { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } +} + #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct ChunkCommitInsert { - pub plan_id: u32, - pub input: Box, + pub meta: PhysicalPlanMeta, + pub input: Box, 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>) -> Box { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } +} diff --git a/src/query/sql/src/executor/physical_plans/physical_mutation.rs b/src/query/sql/src/executor/physical_plans/physical_mutation.rs index 9feb40c125a1d..af7dc666838a0 100644 --- a/src/query/sql/src/executor/physical_plans/physical_mutation.rs +++ b/src/query/sql/src/executor/physical_plans/physical_mutation.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::sync::Arc; +use databend_common_ast::ast::FormatTreeNode; +use databend_common_catalog::plan::DataSourcePlan; use databend_common_catalog::plan::NUM_ROW_ID_PREFIX_BITS; use databend_common_catalog::table_context::TableContext; use databend_common_exception::ErrorCode; @@ -48,6 +51,8 @@ use super::CommitType; use crate::binder::wrap_cast; use crate::binder::MutationStrategy; use crate::binder::MutationType; +use crate::executor::format::FormatContext; +use crate::executor::physical_plan::DeriveHandle; use crate::executor::physical_plan::PhysicalPlan; use crate::executor::physical_plans::CommitSink; use crate::executor::physical_plans::Exchange; @@ -57,7 +62,9 @@ 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::IPhysicalPlan; use crate::executor::PhysicalPlanBuilder; +use crate::executor::PhysicalPlanMeta; use crate::optimizer::ir::SExpr; use crate::parse_computed_expr; use crate::plans::BoundColumnRef; @@ -81,8 +88,8 @@ pub type MatchExpr = Vec<(Option, Option, + pub meta: PhysicalPlanMeta, + pub input: Box, pub table_info: TableInfo, // (DataSchemaRef, Option, Vec,Vec) => (source_schema, condition, value_exprs) pub unmatched: Vec<(DataSchemaRef, Option, Vec)>, @@ -95,13 +102,68 @@ 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>) -> Box { + 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_mutation( &mut self, s_expr: &SExpr, mutation: &crate::plans::Mutation, required: ColumnSet, - ) -> Result { + ) -> Result> { let crate::plans::Mutation { bind_context, metadata, @@ -142,8 +204,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 @@ -152,10 +214,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); } @@ -209,9 +271,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, @@ -223,18 +285,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 @@ -244,10 +306,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); @@ -264,23 +326,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. @@ -290,7 +352,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(), @@ -298,7 +360,7 @@ impl PhysicalPlanBuilder { lazy_columns.clone(), *target_table_index, row_id_offset, - )); + ); } let output_schema = plan.output_schema()?; @@ -409,9 +471,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(), @@ -420,13 +481,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 @@ -436,8 +499,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(), @@ -446,22 +509,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, @@ -469,8 +530,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 @@ -480,10 +541,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) } @@ -506,12 +568,12 @@ impl PhysicalPlanBuilder { } pub fn build_block_id_shuffle_exchange( - plan: PhysicalPlan, + plan: Box, bind_context: &BindContext, 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( @@ -561,25 +623,25 @@ 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( - plan: PhysicalPlan, + plan: Box, metadata: MetadataRef, mutation_input_schema: Arc, strategy: MutationStrategy, lazy_columns: ColumnSet, target_table_index: usize, row_id_offset: usize, -) -> RowFetch { +) -> Box { let metadata = metadata.read(); let lazy_columns = lazy_columns @@ -621,16 +683,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/sql/src/executor/physical_plans/physical_mutation_into_organize.rs b/src/query/sql/src/executor/physical_plans/physical_mutation_into_organize.rs index f2d6df77da87c..7afc6aea326d7 100644 --- 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 @@ -12,21 +12,63 @@ // 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 crate::binder::MutationStrategy; +use crate::executor::format::FormatContext; +use crate::executor::physical_plan::DeriveHandle; use crate::executor::physical_plan::PhysicalPlan; +use crate::executor::IPhysicalPlan; +use crate::executor::PhysicalPlanMeta; #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct MutationOrganize { - pub plan_id: u32, - pub input: Box, + pub meta: PhysicalPlanMeta, + pub input: Box, pub strategy: MutationStrategy, } -impl MutationOrganize { - pub fn output_schema(&self) -> Result { - self.input.output_schema() +#[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>) -> Box { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) } } diff --git a/src/query/sql/src/executor/physical_plans/physical_mutation_into_split.rs b/src/query/sql/src/executor/physical_plans/physical_mutation_into_split.rs index f94b1955afddb..e1ec705e8d5a7 100644 --- a/src/query/sql/src/executor/physical_plans/physical_mutation_into_split.rs +++ b/src/query/sql/src/executor/physical_plans/physical_mutation_into_split.rs @@ -12,21 +12,63 @@ // 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 crate::executor::format::FormatContext; +use crate::executor::physical_plan::DeriveHandle; use crate::executor::physical_plan::PhysicalPlan; +use crate::executor::IPhysicalPlan; +use crate::executor::PhysicalPlanMeta; use crate::IndexType; #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct MutationSplit { - pub plan_id: u32, - pub input: Box, + pub meta: PhysicalPlanMeta, + pub input: Box, pub split_index: IndexType, } -impl MutationSplit { - pub fn output_schema(&self) -> Result { - self.input.output_schema() +#[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>) -> Box { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) } } 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 index 50976f11327be..fafb4ced660a0 100644 --- a/src/query/sql/src/executor/physical_plans/physical_mutation_manipulate.rs +++ b/src/query/sql/src/executor/physical_plans/physical_mutation_manipulate.rs @@ -12,23 +12,31 @@ // 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_exception::Result; 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 itertools::Itertools; use crate::binder::MutationStrategy; +use crate::executor::format::FormatContext; +use crate::executor::physical_plan::DeriveHandle; use crate::executor::physical_plan::PhysicalPlan; +use crate::executor::IPhysicalPlan; +use crate::executor::PhysicalPlanMeta; pub type MatchExpr = Vec<(Option, Option>)>; #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct MutationManipulate { - pub plan_id: u32, - pub input: Box, + pub meta: PhysicalPlanMeta, + pub input: Box, pub table_info: TableInfo, // (DataSchemaRef, Option, Vec,Vec) => (source_schema, condition, value_exprs) pub unmatched: Vec<(DataSchemaRef, Option, Vec)>, @@ -41,10 +49,130 @@ pub struct MutationManipulate { pub row_id_idx: usize, pub can_try_update_column_only: bool, pub unmatched_schema: DataSchemaRef, + pub target_table_index: usize, } -impl MutationManipulate { - pub fn output_schema(&self) -> Result { - self.input.output_schema() +#[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>) -> Box { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) } } 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 index b04c294220ab5..58d0955355673 100644 --- a/src/query/sql/src/executor/physical_plans/physical_mutation_source.rs +++ b/src/query/sql/src/executor/physical_plans/physical_mutation_source.rs @@ -12,6 +12,10 @@ // 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_catalog::plan::Filters; use databend_common_catalog::plan::PartStatistics; use databend_common_catalog::plan::Partitions; @@ -28,16 +32,21 @@ use databend_common_meta_app::schema::TableInfo; use crate::binder::MutationType; use crate::executor::cast_expr_to_non_null_boolean; +use crate::executor::format::format_output_columns; +use crate::executor::format::part_stats_info_to_format_tree; +use crate::executor::format::FormatContext; +use crate::executor::physical_plan::DeriveHandle; +use crate::executor::IPhysicalPlan; use crate::executor::PhysicalPlan; use crate::executor::PhysicalPlanBuilder; +use crate::executor::PhysicalPlanMeta; 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 meta: PhysicalPlanMeta, pub table_index: IndexType, pub table_info: TableInfo, pub filters: Option, @@ -50,17 +59,76 @@ pub struct MutationSource { pub statistics: PartStatistics, } -impl MutationSource { - pub fn output_schema(&self) -> Result { +#[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>) -> Box { + assert!(children.is_empty()); + Box::new(self.clone()) + } } impl PhysicalPlanBuilder { pub(crate) async fn build_mutation_source( &mut self, mutation_source: &crate::plans::MutationSource, - ) -> Result { + ) -> Result> { let filters = if !mutation_source.predicates.is_empty() { Some(create_push_down_filters( &self.ctx.get_function_context()?, @@ -101,8 +169,7 @@ impl PhysicalPlanBuilder { let truncate_table = mutation_source.mutation_type == MutationType::Delete && filters.is_none(); - Ok(PhysicalPlan::MutationSource(MutationSource { - plan_id: 0, + Ok(Box::new(MutationSource { table_index: mutation_source.table_index, output_schema, table_info: mutation_info.table_info.clone(), @@ -110,6 +177,7 @@ impl PhysicalPlanBuilder { 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(), })) 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 index 99b85519223b8..7b4dc84e2ef6b 100644 --- a/src/query/sql/src/executor/physical_plans/physical_project_set.rs +++ b/src/query/sql/src/executor/physical_plans/physical_project_set.rs @@ -12,6 +12,10 @@ // 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; @@ -19,10 +23,17 @@ use databend_common_expression::DataSchemaRef; use databend_common_expression::DataSchemaRefExt; use databend_common_expression::RemoteExpr; use databend_common_functions::BUILTIN_FUNCTIONS; +use itertools::Itertools; use crate::executor::explain::PlanStatsInfo; +use crate::executor::format::format_output_columns; +use crate::executor::format::plan_stats_info_to_format_tree; +use crate::executor::format::FormatContext; +use crate::executor::physical_plan::DeriveHandle; +use crate::executor::IPhysicalPlan; use crate::executor::PhysicalPlan; use crate::executor::PhysicalPlanBuilder; +use crate::executor::PhysicalPlanMeta; use crate::optimizer::ir::SExpr; use crate::ColumnSet; use crate::IndexType; @@ -30,18 +41,29 @@ 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, + meta: PhysicalPlanMeta, pub projections: ColumnSet, - pub input: Box, + 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 { +#[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() { @@ -57,6 +79,67 @@ impl ProjectSet { })); 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>) -> Box { + 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 { @@ -66,7 +149,7 @@ impl PhysicalPlanBuilder { project_set: &crate::plans::ProjectSet, mut required: ColumnSet, stat_info: PlanStatsInfo, - ) -> Result { + ) -> Result> { // 1. Prune unused Columns. let column_projections = required.clone().into_iter().collect::>(); for s in project_set.srfs.iter() { @@ -96,9 +179,9 @@ impl PhysicalPlanBuilder { } } - Ok(PhysicalPlan::ProjectSet(ProjectSet { - plan_id: 0, - input: Box::new(input), + Ok(Box::new(ProjectSet { + input, + meta: PhysicalPlanMeta::new("Unnest"), 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 index 459a6c3c91bd2..3f92f60abc1bd 100644 --- 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 @@ -12,6 +12,7 @@ // 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; @@ -19,22 +20,41 @@ use databend_common_expression::DataSchemaRef; use databend_common_expression::DataSchemaRefExt; use crate::executor::explain::PlanStatsInfo; +use crate::executor::physical_plan::DeriveHandle; +use crate::executor::IPhysicalPlan; use crate::executor::PhysicalPlan; use crate::executor::PhysicalPlanBuilder; +use crate::executor::PhysicalPlanMeta; #[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, + meta: PhysicalPlanMeta, pub output_schema: DataSchemaRef, pub table_name: String, pub stat: PlanStatsInfo, } -impl RecursiveCteScan { - pub fn output_schema(&self) -> Result { +#[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>) -> Box { + assert!(children.is_empty()); + Box::new(self.clone()) + } } impl PhysicalPlanBuilder { @@ -42,9 +62,9 @@ impl PhysicalPlanBuilder { &mut self, recursive_cte_scan: &crate::plans::RecursiveCteScan, stat_info: PlanStatsInfo, - ) -> Result { - Ok(PhysicalPlan::RecursiveCteScan(RecursiveCteScan { - plan_id: 0, + ) -> 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, diff --git a/src/query/sql/src/executor/physical_plans/physical_range_join.rs b/src/query/sql/src/executor/physical_plans/physical_range_join.rs index 94317540f2f99..82d596148f141 100644 --- a/src/query/sql/src/executor/physical_plans/physical_range_join.rs +++ b/src/query/sql/src/executor/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; @@ -26,8 +28,14 @@ use databend_common_functions::BUILTIN_FUNCTIONS; use crate::binder::wrap_cast; use crate::binder::JoinPredicate; use crate::executor::explain::PlanStatsInfo; +use crate::executor::format::format_output_columns; +use crate::executor::format::plan_stats_info_to_format_tree; +use crate::executor::format::FormatContext; +use crate::executor::physical_plan::DeriveHandle; +use crate::executor::IPhysicalPlan; use crate::executor::PhysicalPlan; use crate::executor::PhysicalPlanBuilder; +use crate::executor::PhysicalPlanMeta; use crate::optimizer::ir::RelExpr; use crate::optimizer::ir::RelationalProperty; use crate::optimizer::ir::SExpr; @@ -39,10 +47,9 @@ use crate::TypeCheck; #[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: Box, + pub right: Box, // The first two conditions: (>, >=, <, <=) // Condition's left/right side only contains one table's column pub conditions: Vec, @@ -57,10 +64,124 @@ 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>) -> Box { + 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) + } } #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] @@ -86,7 +207,7 @@ impl PhysicalPlanBuilder { right_required: ColumnSet, mut range_conditions: Vec, mut other_conditions: Vec, - ) -> Result { + ) -> Result> { let left_prop = RelExpr::with_s_expr(s_expr.child(1)?).derive_relational_prop()?; let right_prop = RelExpr::with_s_expr(s_expr.child(0)?).derive_relational_prop()?; @@ -122,10 +243,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/sql/src/executor/physical_plans/physical_recluster.rs b/src/query/sql/src/executor/physical_plans/physical_recluster.rs index 9227c86b64199..2b5c6154db42e 100644 --- a/src/query/sql/src/executor/physical_plans/physical_recluster.rs +++ b/src/query/sql/src/executor/physical_plans/physical_recluster.rs @@ -12,26 +12,88 @@ // 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_catalog::plan::ReclusterTask; +use databend_common_exception::Result; +use databend_common_expression::DataSchemaRef; use databend_common_meta_app::schema::TableInfo; use databend_storages_common_table_meta::meta::TableMetaTimestamps; +use crate::executor::physical_plan::DeriveHandle; +use crate::executor::IPhysicalPlan; use crate::executor::PhysicalPlan; +use crate::executor::PhysicalPlanMeta; #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct Recluster { - pub plan_id: u32, + 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>) -> Box { + assert!(children.is_empty()); + Box::new(self.clone()) + } +} + #[derive(serde::Serialize, serde::Deserialize, Clone, Debug)] pub struct HilbertPartition { - pub plan_id: u32, - pub input: Box, + pub meta: PhysicalPlanMeta, + pub input: Box, 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>) -> Box { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } +} 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 index f7a640fd9cc00..ca8003ca3a5e9 100644 --- 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 @@ -12,13 +12,38 @@ // 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_expression::DataSchemaRef; +use crate::executor::physical_plan::DeriveHandle; +use crate::executor::IPhysicalPlan; +use crate::executor::PhysicalPlanMeta; use crate::plans::InsertValue; #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct ReplaceAsyncSourcer { - pub plan_id: u32, + 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>) -> Box { + assert!(children.is_empty()); + Box::new(self.clone()) + } +} 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 index 24224053d03b4..9e5d519bb7087 100644 --- a/src/query/sql/src/executor/physical_plans/physical_replace_deduplicate.rs +++ b/src/query/sql/src/executor/physical_plans/physical_replace_deduplicate.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::HashMap; +use databend_common_catalog::plan::DataSourcePlan; +use databend_common_exception::Result; use databend_common_expression::ColumnId; use databend_common_expression::DataSchemaRef; use databend_common_expression::FieldIndex; @@ -22,14 +25,17 @@ use databend_common_expression::TableSchemaRef; use databend_common_meta_app::schema::TableInfo; use databend_storages_common_table_meta::meta::ColumnStatistics; +use crate::executor::physical_plan::DeriveHandle; use crate::executor::physical_plans::common::OnConflictField; +use crate::executor::IPhysicalPlan; use crate::executor::PhysicalPlan; +use crate::executor::PhysicalPlanMeta; use crate::ColumnBinding; #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct ReplaceDeduplicate { - pub plan_id: u32, - pub input: Box, + pub meta: PhysicalPlanMeta, + pub input: Box, pub on_conflicts: Vec, pub bloom_filter_column_indexes: Vec, pub table_is_empty: bool, @@ -41,6 +47,41 @@ pub struct ReplaceDeduplicate { 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>) -> Box { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } +} + #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct ReplaceSelectCtx { pub select_column_bindings: Vec, 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 index 75d0af7ac2cd3..22b50a53a10ec 100644 --- a/src/query/sql/src/executor/physical_plans/physical_replace_into.rs +++ b/src/query/sql/src/executor/physical_plans/physical_replace_into.rs @@ -12,22 +12,28 @@ // 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::BlockThresholds; +use databend_common_expression::DataSchemaRef; 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_plan::DeriveHandle; use crate::executor::physical_plans::common::OnConflictField; +use crate::executor::IPhysicalPlan; use crate::executor::PhysicalPlan; +use crate::executor::PhysicalPlanMeta; #[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 meta: PhysicalPlanMeta, + pub input: Box, pub block_thresholds: BlockThresholds, pub table_info: TableInfo, pub on_conflicts: Vec, @@ -37,3 +43,38 @@ pub struct ReplaceInto { 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>) -> Box { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } +} 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 index 3a5aca4f7a309..ea7f25309546d 100644 --- a/src/query/sql/src/executor/physical_plans/physical_row_fetch.rs +++ b/src/query/sql/src/executor/physical_plans/physical_row_fetch.rs @@ -12,21 +12,30 @@ // 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_catalog::plan::Projection; use databend_common_exception::Result; use databend_common_expression::DataField; use databend_common_expression::DataSchemaRef; use databend_common_expression::DataSchemaRefExt; +use itertools::Itertools; use crate::executor::explain::PlanStatsInfo; +use crate::executor::format::format_output_columns; +use crate::executor::format::plan_stats_info_to_format_tree; +use crate::executor::format::FormatContext; +use crate::executor::physical_plan::DeriveHandle; +use crate::executor::IPhysicalPlan; use crate::executor::PhysicalPlan; +use crate::executor::PhysicalPlanMeta; #[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, + pub meta: PhysicalPlanMeta, + pub input: Box, // cloned from `input`. pub source: Box, // projection on the source table schema. @@ -39,10 +48,82 @@ pub struct RowFetch { pub stat_info: Option, } -impl RowFetch { - pub fn output_schema(&self) -> Result { +#[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>) -> Box { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } } diff --git a/src/query/sql/src/executor/physical_plans/physical_sort.rs b/src/query/sql/src/executor/physical_plans/physical_sort.rs index 94b765107f310..aafe56a4907bc 100644 --- a/src/query/sql/src/executor/physical_plans/physical_sort.rs +++ b/src/query/sql/src/executor/physical_plans/physical_sort.rs @@ -12,6 +12,10 @@ // 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; @@ -19,14 +23,21 @@ 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 itertools::Itertools; use crate::executor::explain::PlanStatsInfo; +use crate::executor::format::format_output_columns; +use crate::executor::format::plan_stats_info_to_format_tree; +use crate::executor::format::FormatContext; +use crate::executor::physical_plan::DeriveHandle; 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::IPhysicalPlan; use crate::executor::PhysicalPlan; use crate::executor::PhysicalPlanBuilder; +use crate::executor::PhysicalPlanMeta; use crate::optimizer::ir::SExpr; use crate::plans::WindowFuncType; use crate::ColumnSet; @@ -34,9 +45,8 @@ 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 meta: PhysicalPlanMeta, + pub input: Box, pub order_by: Vec, /// limit = Limit.limit + Limit.offset pub limit: Option, @@ -49,21 +59,20 @@ pub struct Sort { 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) +#[typetag::serde] +impl IPhysicalPlan for Sort { + fn as_any(&self) -> &dyn Any { + self + } + fn get_meta(&self) -> &PhysicalPlanMeta { + &self.meta } - pub fn output_schema(&self) -> Result { + 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)) { @@ -100,6 +109,100 @@ impl Sort { 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>) -> Box { + 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 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 { @@ -109,7 +212,7 @@ impl PhysicalPlanBuilder { sort: &crate::plans::Sort, mut required: ColumnSet, stat_info: PlanStatsInfo, - ) -> Result { + ) -> Result> { // 1. Prune unused Columns. sort.items.iter().for_each(|s| { required.insert(s.index); @@ -142,9 +245,9 @@ impl PhysicalPlanBuilder { .map(|v| v.index) .collect::>(); - return Ok(PhysicalPlan::WindowPartition(WindowPartition { - plan_id: 0, - input: Box::new(input_plan.clone()), + 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, @@ -162,14 +265,14 @@ impl PhysicalPlanBuilder { }; // 2. Build physical plan. - Ok(PhysicalPlan::Sort(Sort { - plan_id: 0, - input: Box::new(input_plan), + Ok(Box::new(Sort { order_by, + pre_projection, + input: input_plan, limit: sort.limit, after_exchange: sort.after_exchange, - pre_projection, 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/sql/src/executor/physical_plans/physical_table_scan.rs index 0541b88c9abd1..72fbf9bc2a1cc 100644 --- a/src/query/sql/src/executor/physical_plans/physical_table_scan.rs +++ b/src/query/sql/src/executor/physical_plans/physical_table_scan.rs @@ -12,17 +12,23 @@ // 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; @@ -44,6 +50,7 @@ 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 itertools::Itertools; use jsonb::keypath::KeyPath; use jsonb::keypath::KeyPaths; use rand::distributions::Bernoulli; @@ -53,10 +60,17 @@ 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::format::format_output_columns; +use crate::executor::format::part_stats_info_to_format_tree; +use crate::executor::format::plan_stats_info_to_format_tree; +use crate::executor::format::FormatContext; +use crate::executor::physical_plan::DeriveHandle; use crate::executor::physical_plans::AddStreamColumn; use crate::executor::table_read_plan::ToReadDataSourcePlan; +use crate::executor::IPhysicalPlan; use crate::executor::PhysicalPlan; use crate::executor::PhysicalPlanBuilder; +use crate::executor::PhysicalPlanMeta; use crate::plans::FunctionCall; use crate::BaseTableColumn; use crate::ColumnEntry; @@ -73,8 +87,7 @@ use crate::DUMMY_TABLE_INDEX; #[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 +97,258 @@ 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>) -> Box { + assert!(children.is_empty()); + Box::new(self.clone()) + } +} + impl TableScan { + pub fn new( + scan_id: usize, + name_mapping: BTreeMap, + source: Box, + table_index: Option, + stat_info: Option, + internal_column: Option>, + ) -> Box { + 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,11 +371,6 @@ 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 { @@ -120,7 +379,7 @@ impl PhysicalPlanBuilder { scan: &crate::plans::Scan, required: ColumnSet, stat_info: PlanStatsInfo, - ) -> Result { + ) -> Result> { // 1. Prune unused Columns. // Some table may not have any column, // e.g. `system.sync_crash_me` @@ -308,30 +567,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::new( + 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::new( &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(crate) async fn build_dummy_table_scan(&mut self) -> Result> { let catalogs = CatalogManager::instance(); let table = catalogs .get_default_catalog(self.ctx.session_state())? @@ -345,17 +603,17 @@ 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::new( + 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( diff --git a/src/query/sql/src/executor/physical_plans/physical_udf.rs b/src/query/sql/src/executor/physical_plans/physical_udf.rs index acfd628b36af3..15ecbcf08607b 100644 --- a/src/query/sql/src/executor/physical_plans/physical_udf.rs +++ b/src/query/sql/src/executor/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,48 @@ use databend_common_expression::DataField; use databend_common_expression::DataSchemaRef; use databend_common_expression::DataSchemaRefExt; use databend_common_functions::BUILTIN_FUNCTIONS; +use itertools::Itertools; use crate::executor::explain::PlanStatsInfo; +use crate::executor::format::format_output_columns; +use crate::executor::format::plan_stats_info_to_format_tree; +use crate::executor::format::FormatContext; +use crate::executor::physical_plan::DeriveHandle; +use crate::executor::IPhysicalPlan; use crate::executor::PhysicalPlan; use crate::executor::PhysicalPlanBuilder; +use crate::executor::PhysicalPlanMeta; use crate::optimizer::ir::SExpr; use crate::plans::UDFType; use crate::ColumnSet; use crate::IndexType; +use crate::Metadata; use crate::ScalarExpr; #[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: Box, 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 +75,70 @@ 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>) -> Box { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } } #[derive(Clone, Debug, Eq, Hash, PartialEq, serde::Serialize, serde::Deserialize)] @@ -75,7 +161,7 @@ impl PhysicalPlanBuilder { udf_plan: &crate::plans::Udf, mut required: ColumnSet, stat_info: PlanStatsInfo, - ) -> Result { + ) -> Result> { // 1. Prune unused Columns. let mut used = vec![]; for item in udf_plan.items.iter() { @@ -146,12 +232,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/sql/src/executor/physical_plans/physical_union_all.rs b/src/query/sql/src/executor/physical_plans/physical_union_all.rs index 26ea91d34f81f..1108f28611135 100644 --- a/src/query/sql/src/executor/physical_plans/physical_union_all.rs +++ b/src/query/sql/src/executor/physical_plans/physical_union_all.rs @@ -12,16 +12,26 @@ // 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 itertools::Itertools; use crate::executor::explain::PlanStatsInfo; +use crate::executor::format::format_output_columns; +use crate::executor::format::plan_stats_info_to_format_tree; +use crate::executor::format::FormatContext; +use crate::executor::physical_plan::DeriveHandle; +use crate::executor::IPhysicalPlan; use crate::executor::PhysicalPlan; use crate::executor::PhysicalPlanBuilder; +use crate::executor::PhysicalPlanMeta; use crate::optimizer::ir::SExpr; use crate::ColumnSet; use crate::IndexType; @@ -30,10 +40,9 @@ 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, + meta: PhysicalPlanMeta, + pub left: Box, + pub right: Box, pub left_outputs: Vec<(IndexType, Option)>, pub right_outputs: Vec<(IndexType, Option)>, pub schema: DataSchemaRef, @@ -43,10 +52,74 @@ pub struct UnionAll { pub stat_info: Option, } -impl UnionAll { - pub fn output_schema(&self) -> Result { +#[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>) -> Box { + 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) + } } impl PhysicalPlanBuilder { @@ -56,7 +129,7 @@ impl PhysicalPlanBuilder { union_all: &crate::plans::UnionAll, mut required: ColumnSet, stat_info: PlanStatsInfo, - ) -> Result { + ) -> Result> { // 1. Prune unused Columns. let metadata = self.metadata.read().clone(); let lazy_columns = metadata.lazy_columns(); @@ -128,14 +201,13 @@ impl PhysicalPlanBuilder { let right_outputs = process_outputs(&union_all.right_outputs, &right_required, &right_schema)?; - Ok(PhysicalPlan::UnionAll(UnionAll { - plan_id: 0, - left: Box::new(left_plan), - right: Box::new(right_plan), + Ok(Box::new(UnionAll { + left: left_plan, + right: right_plan, left_outputs, right_outputs, schema: DataSchemaRefExt::create(fields), - + meta: PhysicalPlanMeta::new("UnionAll"), cte_scan_names: union_all.cte_scan_names.clone(), stat_info: Some(stat_info), })) diff --git a/src/query/sql/src/executor/physical_plans/physical_window.rs b/src/query/sql/src/executor/physical_plans/physical_window.rs index f2cac19cb0fa1..b716f51caf875 100644 --- a/src/query/sql/src/executor/physical_plans/physical_window.rs +++ b/src/query/sql/src/executor/physical_plans/physical_window.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::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; @@ -32,11 +35,17 @@ use databend_common_functions::BUILTIN_FUNCTIONS; use crate::binder::wrap_cast; use crate::executor::explain::PlanStatsInfo; +use crate::executor::format::format_output_columns; +use crate::executor::format::pretty_display_agg_desc; +use crate::executor::format::FormatContext; +use crate::executor::physical_plan::DeriveHandle; use crate::executor::physical_plans::common::AggregateFunctionDesc; use crate::executor::physical_plans::common::AggregateFunctionSignature; use crate::executor::physical_plans::common::SortDesc; +use crate::executor::IPhysicalPlan; use crate::executor::PhysicalPlan; use crate::executor::PhysicalPlanBuilder; +use crate::executor::PhysicalPlanMeta; use crate::optimizer::ir::SExpr; use crate::plans::WindowFuncFrame; use crate::plans::WindowFuncFrameBound; @@ -48,10 +57,9 @@ use crate::TypeCheck; #[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: Box, pub func: WindowFunction, pub partition_by: Vec, pub order_by: Vec, @@ -59,8 +67,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 +90,104 @@ 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>) -> Box { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } } #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] @@ -156,7 +274,7 @@ impl PhysicalPlanBuilder { window: &crate::plans::Window, mut required: ColumnSet, _stat_info: PlanStatsInfo, - ) -> Result { + ) -> Result> { // 1. DO NOT Prune unused Columns cause window may not in required, eg: // select s1.a from ( select t1.a as a, dense_rank() over(order by t1.a desc) as rk // from (select 'a1' as a) t1 ) s1 @@ -360,15 +478,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/sql/src/executor/physical_plans/physical_window_partition.rs b/src/query/sql/src/executor/physical_plans/physical_window_partition.rs index b0ff12d3f8685..75ea095fab111 100644 --- a/src/query/sql/src/executor/physical_plans/physical_window_partition.rs +++ b/src/query/sql/src/executor/physical_plans/physical_window_partition.rs @@ -12,18 +12,28 @@ // 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 crate::executor::explain::PlanStatsInfo; +use crate::executor::format::format_output_columns; +use crate::executor::format::plan_stats_info_to_format_tree; +use crate::executor::format::FormatContext; +use crate::executor::physical_plan::DeriveHandle; use crate::executor::physical_plans::SortDesc; +use crate::executor::IPhysicalPlan; use crate::executor::PhysicalPlan; +use crate::executor::PhysicalPlanMeta; use crate::IndexType; #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct WindowPartition { - pub plan_id: u32, - pub input: Box, + pub meta: PhysicalPlanMeta, + pub input: Box, pub partition_by: Vec, pub order_by: Vec, pub after_exchange: Option, @@ -32,6 +42,77 @@ pub struct WindowPartition { 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>) -> Box { + let mut new_physical_plan = self.clone(); + assert_eq!(children.len(), 1); + new_physical_plan.input = children.pop().unwrap(); + Box::new(new_physical_plan) + } +} + #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] pub struct WindowPartitionTopN { pub func: WindowPartitionTopNFunc, @@ -44,9 +125,3 @@ pub enum WindowPartitionTopNFunc { Rank, DenseRank, } - -impl WindowPartition { - pub fn output_schema(&self) -> Result { - self.input.output_schema() - } -} diff --git a/src/query/sql/src/planner/execution/query_executor.rs b/src/query/sql/src/planner/execution/query_executor.rs index b6b6e430f5d35..f8697c4b4fc2f 100644 --- a/src/query/sql/src/planner/execution/query_executor.rs +++ b/src/query/sql/src/planner/execution/query_executor.rs @@ -16,12 +16,15 @@ use async_trait::async_trait; use databend_common_exception::Result; use databend_common_expression::DataBlock; +use crate::executor::IPhysicalPlan; 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_physical_plan( + &self, + plan: &Box, + ) -> Result>; async fn execute_query_with_sql_string(&self, sql: &str) -> Result>; }