From 5aa48d828459b46b6576a988c390cc4d95de1253 Mon Sep 17 00:00:00 2001 From: Arenatlx <314806019@qq.com> Date: Mon, 15 Apr 2024 18:51:06 +0800 Subject: [PATCH] planner: move base plan related output of core pkg and make it well-pkged (#52529) ref pingcap/tidb#51664 --- pkg/executor/BUILD.bazel | 2 + pkg/executor/adapter.go | 11 +- pkg/executor/benchmark_test.go | 19 +- pkg/executor/builder.go | 27 +-- pkg/executor/compiler.go | 7 +- pkg/executor/distsql.go | 9 +- pkg/executor/executor.go | 3 +- pkg/executor/foreign_key.go | 3 +- pkg/executor/importer/BUILD.bazel | 1 + .../importer/importer_testkit_test.go | 3 +- pkg/executor/index_merge_reader.go | 9 +- pkg/executor/infoschema_reader.go | 3 +- pkg/executor/internal/builder/BUILD.bazel | 2 +- .../internal/builder/builder_utils.go | 2 +- pkg/executor/internal/mpp/BUILD.bazel | 1 + .../internal/mpp/executor_with_retry.go | 2 +- .../internal/mpp/local_mpp_coordinator.go | 9 +- pkg/executor/internal/testutil/BUILD.bazel | 1 + pkg/executor/internal/testutil/testutil.go | 2 +- pkg/executor/mpp_gather.go | 5 +- pkg/executor/prepared.go | 3 +- pkg/executor/resource_tag_test.go | 3 +- pkg/executor/table_reader.go | 5 +- .../table_readers_required_rows_test.go | 3 +- pkg/executor/test/executor/BUILD.bazel | 1 + pkg/executor/test/executor/executor_test.go | 13 +- pkg/planner/BUILD.bazel | 1 + pkg/planner/cascades/BUILD.bazel | 1 + pkg/planner/cascades/optimize.go | 9 +- pkg/planner/core/BUILD.bazel | 3 +- pkg/planner/core/access_object.go | 13 +- pkg/planner/core/base/BUILD.bazel | 22 +++ pkg/planner/core/{ => base}/plan_base.go | 9 +- pkg/planner/core/base/task_base.go | 34 ++++ pkg/planner/core/casetest/BUILD.bazel | 1 + pkg/planner/core/casetest/plan_test.go | 13 +- .../core/casetest/planstats/BUILD.bazel | 1 + .../casetest/planstats/plan_stats_test.go | 29 +-- ...ash_selection_late_materialization_test.go | 3 +- pkg/planner/core/common_plans.go | 57 +++--- pkg/planner/core/debugtrace.go | 7 +- pkg/planner/core/encode.go | 27 +-- pkg/planner/core/exhaust_physical_plans.go | 185 +++++++++--------- .../core/exhaust_physical_plans_test.go | 3 +- pkg/planner/core/expression_rewriter.go | 7 +- pkg/planner/core/find_best_task.go | 95 ++++----- pkg/planner/core/find_best_task_test.go | 21 +- pkg/planner/core/flat_plan.go | 15 +- pkg/planner/core/foreign_key.go | 25 +-- pkg/planner/core/fragment.go | 13 +- pkg/planner/core/hint_utils.go | 13 +- pkg/planner/core/indexmerge_path.go | 5 +- pkg/planner/core/initialize.go | 159 +++++++-------- pkg/planner/core/issuetest/BUILD.bazel | 1 + .../core/issuetest/planner_issue_test.go | 3 +- pkg/planner/core/logical_plan_builder.go | 19 +- pkg/planner/core/logical_plans.go | 13 +- pkg/planner/core/logical_plans_test.go | 9 +- .../core/memtable_predicate_extractor.go | 41 ++-- pkg/planner/core/optimizer.go | 69 +++---- pkg/planner/core/optimizer_test.go | 159 +++++++-------- pkg/planner/core/partition_prune.go | 3 +- pkg/planner/core/pb_to_plan.go | 27 +-- pkg/planner/core/physical_plan_test.go | 13 +- pkg/planner/core/physical_plans.go | 167 ++++++++-------- pkg/planner/core/plan.go | 72 ++++--- pkg/planner/core/plan_cache.go | 31 +-- pkg/planner/core/plan_cache_lru_test.go | 3 +- pkg/planner/core/plan_cache_utils.go | 13 +- pkg/planner/core/plan_cacheable_checker.go | 31 +-- pkg/planner/core/plan_cost_detail.go | 3 +- pkg/planner/core/plan_cost_ver1.go | 15 +- pkg/planner/core/plan_cost_ver2.go | 19 +- pkg/planner/core/plan_cost_ver2_test.go | 3 +- pkg/planner/core/plan_test.go | 17 +- pkg/planner/core/plan_to_pb.go | 37 ++-- pkg/planner/core/planbuilder.go | 111 +++++------ pkg/planner/core/planbuilder_test.go | 15 +- pkg/planner/core/point_get_plan.go | 71 +++---- pkg/planner/core/preprocess.go | 3 +- .../core/rule_aggregation_push_down.go | 7 +- pkg/planner/core/rule_collect_plan_stats.go | 9 +- pkg/planner/core/rule_decorrelate.go | 9 +- pkg/planner/core/rule_eliminate_projection.go | 5 +- .../core/rule_inject_extra_projection.go | 11 +- pkg/planner/core/rule_join_reorder.go | 5 +- pkg/planner/core/rule_join_reorder_dp_test.go | 7 +- pkg/planner/core/rule_partition_processor.go | 55 +++--- pkg/planner/core/rule_predicate_push_down.go | 15 +- .../core/rule_predicate_simplification.go | 5 +- pkg/planner/core/runtime_filter.go | 5 +- pkg/planner/core/runtime_filter_generator.go | 7 +- pkg/planner/core/scalar_subq_expression.go | 3 +- pkg/planner/core/stats.go | 3 +- pkg/planner/core/stringer.go | 11 +- pkg/planner/core/task.go | 109 ++++++----- pkg/planner/core/task_base.go | 60 +++--- .../tiflash_selection_late_materialization.go | 11 +- pkg/planner/core/util.go | 21 +- pkg/planner/implementation/BUILD.bazel | 1 + pkg/planner/implementation/base.go | 2 +- pkg/planner/memo/BUILD.bazel | 2 + pkg/planner/memo/group_test.go | 5 +- pkg/planner/memo/implementation.go | 2 +- pkg/planner/optimize.go | 17 +- pkg/server/BUILD.bazel | 1 + pkg/server/conn.go | 9 +- pkg/session/BUILD.bazel | 1 + pkg/session/session.go | 3 +- pkg/sessiontxn/isolation/BUILD.bazel | 1 + pkg/sessiontxn/isolation/optimistic.go | 3 +- pkg/sessiontxn/isolation/readcommitted.go | 7 +- pkg/sessiontxn/isolation/repeatable_read.go | 7 +- 113 files changed, 1226 insertions(+), 1081 deletions(-) create mode 100644 pkg/planner/core/base/BUILD.bazel rename pkg/planner/core/{ => base}/plan_base.go (95%) create mode 100644 pkg/planner/core/base/task_base.go diff --git a/pkg/executor/BUILD.bazel b/pkg/executor/BUILD.bazel index e47e79efa740e..1387473c8ec06 100644 --- a/pkg/executor/BUILD.bazel +++ b/pkg/executor/BUILD.bazel @@ -153,6 +153,7 @@ go_library( "//pkg/planner/cardinality", "//pkg/planner/context", "//pkg/planner/core", + "//pkg/planner/core/base", "//pkg/planner/util", "//pkg/planner/util/fixcontrol", "//pkg/plugin", @@ -410,6 +411,7 @@ go_test( "//pkg/parser/terror", "//pkg/planner", "//pkg/planner/core", + "//pkg/planner/core/base", "//pkg/planner/property", "//pkg/planner/util", "//pkg/server", diff --git a/pkg/executor/adapter.go b/pkg/executor/adapter.go index 64dcd2859a669..b96c28ee7f057 100644 --- a/pkg/executor/adapter.go +++ b/pkg/executor/adapter.go @@ -47,6 +47,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/planner" plannercore "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/plugin" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/sessionstates" @@ -221,7 +222,7 @@ type ExecStmt struct { // InfoSchema stores a reference to the schema information. InfoSchema infoschema.InfoSchema // Plan stores a reference to the final physical plan. - Plan plannercore.Plan + Plan base.Plan // Text represents the origin query text. Text string @@ -393,7 +394,7 @@ func (a *ExecStmt) RebuildPlan(ctx context.Context) (int64, error) { } // IsFastPlan exports for testing. -func IsFastPlan(p plannercore.Plan) bool { +func IsFastPlan(p base.Plan) bool { if proj, ok := p.(*plannercore.PhysicalProjection); ok { p = proj.Children()[0] } @@ -802,7 +803,7 @@ func (a *ExecStmt) handleNoDelay(ctx context.Context, e exec.Executor, isPessimi return false, nil, nil } -func isNoResultPlan(p plannercore.Plan) bool { +func isNoResultPlan(p base.Plan) bool { if p.Schema().Len() == 0 { return true } @@ -1711,7 +1712,7 @@ func collectWarningsForSlowLog(stmtCtx *stmtctx.StatementContext) []variable.JSO } // GetResultRowsCount gets the count of the statement result rows. -func GetResultRowsCount(stmtCtx *stmtctx.StatementContext, p plannercore.Plan) int64 { +func GetResultRowsCount(stmtCtx *stmtctx.StatementContext, p base.Plan) int64 { runtimeStatsColl := stmtCtx.RuntimeStatsColl if runtimeStatsColl == nil { return 0 @@ -1735,7 +1736,7 @@ func getFlatPlan(stmtCtx *stmtctx.StatementContext) *plannercore.FlatPhysicalPla f := flat.(*plannercore.FlatPhysicalPlan) return f } - p := pp.(plannercore.Plan) + p := pp.(base.Plan) flat := plannercore.FlattenPhysicalPlan(p, false) if flat != nil { stmtCtx.SetFlatPlan(flat) diff --git a/pkg/executor/benchmark_test.go b/pkg/executor/benchmark_test.go index 5eeb1a3a391b6..b9b4cf4e1f17e 100644 --- a/pkg/executor/benchmark_test.go +++ b/pkg/executor/benchmark_test.go @@ -37,6 +37,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/sessionctx" @@ -51,7 +52,7 @@ import ( var ( _ exec.Executor = &testutil.MockDataSource{} - _ core.PhysicalPlan = &testutil.MockDataPhysicalPlan{} + _ base.PhysicalPlan = &testutil.MockDataPhysicalPlan{} wideString = strings.Repeat("x", 5*1024) ) @@ -80,7 +81,7 @@ func buildStreamAggExecutor(ctx sessionctx.Context, srcExec exec.Executor, schem sg.SetSchema(schema) sg.Init(ctx.GetPlanCtx(), nil, 0) - var tail core.PhysicalPlan = sg + var tail base.PhysicalPlan = sg // if data source is not sorted, we have to attach sort, to make the input of stream-agg sorted if !dataSourceSorted { byItems := make([]*util.ByItems, 0, len(sg.GroupByItems)) @@ -96,7 +97,7 @@ func buildStreamAggExecutor(ctx sessionctx.Context, srcExec exec.Executor, schem } var ( - plan core.PhysicalPlan + plan base.PhysicalPlan splitter core.PartitionSplitterType = core.PartitionHashSplitterType ) if concurrency > 1 { @@ -105,8 +106,8 @@ func buildStreamAggExecutor(ctx sessionctx.Context, srcExec exec.Executor, schem } plan = core.PhysicalShuffle{ Concurrency: concurrency, - Tails: []core.PhysicalPlan{tail}, - DataSources: []core.PhysicalPlan{src}, + Tails: []base.PhysicalPlan{tail}, + DataSources: []base.PhysicalPlan{src}, SplitterType: splitter, ByItemArrays: [][]expression.Expression{sg.GroupByItems}, }.Init(ctx.GetPlanCtx(), nil, 0) @@ -315,7 +316,7 @@ func buildWindowExecutor(ctx sessionctx.Context, windowFunc string, funcs int, f win.SetSchema(winSchema) win.Init(ctx.GetPlanCtx(), nil, 0) - var tail core.PhysicalPlan = win + var tail base.PhysicalPlan = win if !dataSourceSorted { byItems := make([]*util.ByItems, 0, len(partitionBy)) for _, col := range partitionBy { @@ -329,7 +330,7 @@ func buildWindowExecutor(ctx sessionctx.Context, windowFunc string, funcs int, f win.SetChildren(src) } - var plan core.PhysicalPlan + var plan base.PhysicalPlan if concurrency > 1 { byItems := make([]expression.Expression, 0, len(win.PartitionBy)) for _, item := range win.PartitionBy { @@ -338,8 +339,8 @@ func buildWindowExecutor(ctx sessionctx.Context, windowFunc string, funcs int, f plan = core.PhysicalShuffle{ Concurrency: concurrency, - Tails: []core.PhysicalPlan{tail}, - DataSources: []core.PhysicalPlan{src}, + Tails: []base.PhysicalPlan{tail}, + DataSources: []base.PhysicalPlan{src}, SplitterType: core.PartitionHashSplitterType, ByItemArrays: [][]expression.Expression{byItems}, }.Init(ctx.GetPlanCtx(), nil, 0) diff --git a/pkg/executor/builder.go b/pkg/executor/builder.go index f6584ec80342b..42448f1189b51 100644 --- a/pkg/executor/builder.go +++ b/pkg/executor/builder.go @@ -59,6 +59,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" plannercore "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/base" plannerutil "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" @@ -150,11 +151,11 @@ func NewMockExecutorBuilderForTest(ctx sessionctx.Context, is infoschema.InfoSch } // Build builds an executor tree according to `p`. -func (b *MockExecutorBuilder) Build(p plannercore.Plan) exec.Executor { +func (b *MockExecutorBuilder) Build(p base.Plan) exec.Executor { return b.build(p) } -func (b *executorBuilder) build(p plannercore.Plan) exec.Executor { +func (b *executorBuilder) build(p base.Plan) exec.Executor { switch v := p.(type) { case nil: return nil @@ -2160,8 +2161,8 @@ func (b *executorBuilder) buildTopN(v *plannercore.PhysicalTopN) exec.Executor { func (b *executorBuilder) buildApply(v *plannercore.PhysicalApply) exec.Executor { var ( - innerPlan plannercore.PhysicalPlan - outerPlan plannercore.PhysicalPlan + innerPlan base.PhysicalPlan + outerPlan base.PhysicalPlan ) if v.InnerChildIdx == 0 { innerPlan = v.Children()[0] @@ -2865,7 +2866,7 @@ func markChildrenUsedCols(outputCols []*expression.Column, childSchemas ...*expr return } -func (*executorBuilder) corColInDistPlan(plans []plannercore.PhysicalPlan) bool { +func (*executorBuilder) corColInDistPlan(plans []base.PhysicalPlan) bool { for _, p := range plans { switch x := p.(type) { case *plannercore.PhysicalSelection: @@ -2892,7 +2893,7 @@ func (*executorBuilder) corColInDistPlan(plans []plannercore.PhysicalPlan) bool } // corColInAccess checks whether there's correlated column in access conditions. -func (*executorBuilder) corColInAccess(p plannercore.PhysicalPlan) bool { +func (*executorBuilder) corColInAccess(p base.PhysicalPlan) bool { var access []expression.Expression switch x := p.(type) { case *plannercore.PhysicalTableScan: @@ -2908,7 +2909,7 @@ func (*executorBuilder) corColInAccess(p plannercore.PhysicalPlan) bool { return false } -func (b *executorBuilder) newDataReaderBuilder(p plannercore.PhysicalPlan) (*dataReaderBuilder, error) { +func (b *executorBuilder) newDataReaderBuilder(p base.PhysicalPlan) (*dataReaderBuilder, error) { ts, err := b.getSnapshotTS() if err != nil { return nil, err @@ -3183,7 +3184,7 @@ func (b *executorBuilder) buildIndexNestedLoopHashJoin(v *plannercore.PhysicalIn func buildNoRangeTableReader(b *executorBuilder, v *plannercore.PhysicalTableReader) (*TableReaderExecutor, error) { tablePlans := v.TablePlans if v.StoreType == kv.TiFlash { - tablePlans = []plannercore.PhysicalPlan{v.GetTablePlan()} + tablePlans = []base.PhysicalPlan{v.GetTablePlan()} } dagReq, err := builder.ConstructDAGReq(b.ctx, tablePlans, v.StoreType) if err != nil { @@ -3644,7 +3645,7 @@ func (b *executorBuilder) buildIndexReader(v *plannercore.PhysicalIndexReader) e return ret } -func buildTableReq(b *executorBuilder, schemaLen int, plans []plannercore.PhysicalPlan) (dagReq *tipb.DAGRequest, val table.Table, err error) { +func buildTableReq(b *executorBuilder, schemaLen int, plans []base.PhysicalPlan) (dagReq *tipb.DAGRequest, val table.Table, err error) { tableReq, err := builder.ConstructDAGReq(b.ctx, plans, kv.TiKV) if err != nil { return nil, nil, err @@ -3665,7 +3666,7 @@ func buildTableReq(b *executorBuilder, schemaLen int, plans []plannercore.Physic // buildIndexReq is designed to create a DAG for index request. // If len(ByItems) != 0 means index request should return related columns // to sort result rows in TiDB side for parition tables. -func buildIndexReq(ctx sessionctx.Context, columns []*model.IndexColumn, handleLen int, plans []plannercore.PhysicalPlan) (dagReq *tipb.DAGRequest, err error) { +func buildIndexReq(ctx sessionctx.Context, columns []*model.IndexColumn, handleLen int, plans []base.PhysicalPlan) (dagReq *tipb.DAGRequest, err error) { indexReq, err := builder.ConstructDAGReq(ctx, plans, kv.TiKV) if err != nil { return nil, err @@ -4001,7 +4002,7 @@ func (b *executorBuilder) buildIndexMergeReader(v *plannercore.PhysicalIndexMerg // 1. dataReaderBuilder calculate data range from argument, rather than plan. // 2. the result executor is already opened. type dataReaderBuilder struct { - plan plannercore.Plan + plan base.Plan *executorBuilder selectResultHook // for testing @@ -4013,7 +4014,7 @@ type dataReaderBuilder struct { } type mockPhysicalIndexReader struct { - plannercore.PhysicalPlan + base.PhysicalPlan e exec.Executor } @@ -4028,7 +4029,7 @@ func (builder *dataReaderBuilder) buildExecutorForIndexJoin(ctx context.Context, return builder.buildExecutorForIndexJoinInternal(ctx, builder.plan, lookUpContents, indexRanges, keyOff2IdxOff, cwc, canReorderHandles, memTracker, interruptSignal) } -func (builder *dataReaderBuilder) buildExecutorForIndexJoinInternal(ctx context.Context, plan plannercore.Plan, lookUpContents []*indexJoinLookUpContent, +func (builder *dataReaderBuilder) buildExecutorForIndexJoinInternal(ctx context.Context, plan base.Plan, lookUpContents []*indexJoinLookUpContent, indexRanges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCmpFuncManager, canReorderHandles bool, memTracker *memory.Tracker, interruptSignal *atomic.Value) (exec.Executor, error) { switch v := plan.(type) { case *plannercore.PhysicalTableReader: diff --git a/pkg/executor/compiler.go b/pkg/executor/compiler.go index 9d8a66cfa974b..b363e05cd1f50 100644 --- a/pkg/executor/compiler.go +++ b/pkg/executor/compiler.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner" plannercore "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/sessiontxn/staleread" @@ -158,9 +159,9 @@ func (c *Compiler) Compile(ctx context.Context, stmtNode ast.StmtNode) (_ *ExecS // If the estimated output row count of any operator in the physical plan tree // is greater than the specific threshold, we'll set it to lowPriority when // sending it to the coprocessor. -func needLowerPriority(p plannercore.Plan) bool { +func needLowerPriority(p base.Plan) bool { switch x := p.(type) { - case plannercore.PhysicalPlan: + case base.PhysicalPlan: return isPhysicalPlanNeedLowerPriority(x) case *plannercore.Execute: return needLowerPriority(x.Plan) @@ -180,7 +181,7 @@ func needLowerPriority(p plannercore.Plan) bool { return false } -func isPhysicalPlanNeedLowerPriority(p plannercore.PhysicalPlan) bool { +func isPhysicalPlanNeedLowerPriority(p base.PhysicalPlan) bool { expensiveThreshold := int64(config.GetGlobalConfig().Log.ExpensiveThreshold) if int64(p.StatsCount()) > expensiveThreshold { return true diff --git a/pkg/executor/distsql.go b/pkg/executor/distsql.go index 09f4ce205fa7a..5bf29a4956fa9 100644 --- a/pkg/executor/distsql.go +++ b/pkg/executor/distsql.go @@ -40,6 +40,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" plannercore "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/base" plannerutil "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/table" @@ -204,7 +205,7 @@ type IndexReaderExecutor struct { corColInAccess bool idxCols []*expression.Column colLens []int - plans []plannercore.PhysicalPlan + plans []base.PhysicalPlan memTracker *memory.Tracker @@ -470,8 +471,8 @@ type IndexLookUpExecutor struct { corColInIdxSide bool corColInTblSide bool corColInAccess bool - idxPlans []plannercore.PhysicalPlan - tblPlans []plannercore.PhysicalPlan + idxPlans []base.PhysicalPlan + tblPlans []base.PhysicalPlan idxCols []*expression.Column colLens []int // PushedLimit is used to skip the preceding and tailing handles when Limit is sunk into IndexLookUpReader. @@ -1594,7 +1595,7 @@ func GetLackHandles(expectedHandles []kv.Handle, obtainedHandlesMap *kv.HandleMa return diffHandles } -func getPhysicalPlanIDs(plans []plannercore.PhysicalPlan) []int { +func getPhysicalPlanIDs(plans []base.PhysicalPlan) []int { planIDs := make([]int, 0, len(plans)) for _, p := range plans { planIDs = append(planIDs, p.ID()) diff --git a/pkg/executor/executor.go b/pkg/executor/executor.go index 2c01a172d3697..e30200b1fdc53 100644 --- a/pkg/executor/executor.go +++ b/pkg/executor/executor.go @@ -54,6 +54,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/terror" planctx "github.com/pingcap/tidb/pkg/planner/context" plannercore "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/util/fixcontrol" "github.com/pingcap/tidb/pkg/privilege" "github.com/pingcap/tidb/pkg/resourcemanager/pool/workerpool" @@ -1452,7 +1453,7 @@ func init() { // While doing optimization in the plan package, we need to execute uncorrelated subquery, // but the plan package cannot import the executor package because of the dependency cycle. // So we assign a function implemented in the executor package to the plan package to avoid the dependency cycle. - plannercore.EvalSubqueryFirstRow = func(ctx context.Context, p plannercore.PhysicalPlan, is infoschema.InfoSchema, pctx planctx.PlanContext) ([]types.Datum, error) { + plannercore.EvalSubqueryFirstRow = func(ctx context.Context, p base.PhysicalPlan, is infoschema.InfoSchema, pctx planctx.PlanContext) ([]types.Datum, error) { defer func(begin time.Time) { s := pctx.GetSessionVars() s.StmtCtx.SetSkipPlanCache(errors.NewNoStackError("query has uncorrelated sub-queries is un-cacheable")) diff --git a/pkg/executor/foreign_key.go b/pkg/executor/foreign_key.go index f664aa1a9f192..d82959533317c 100644 --- a/pkg/executor/foreign_key.go +++ b/pkg/executor/foreign_key.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/planner" plannercore "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" "github.com/pingcap/tidb/pkg/table" @@ -719,7 +720,7 @@ func (fkc *FKCascadeExec) buildExecutor(ctx context.Context) (exec.Executor, err // this is to avoid performance issue, see: https://github.com/pingcap/tidb/issues/38631 var maxHandleFKValueInOneCascade = 1024 -func (fkc *FKCascadeExec) buildFKCascadePlan(ctx context.Context) (plannercore.Plan, error) { +func (fkc *FKCascadeExec) buildFKCascadePlan(ctx context.Context) (base.Plan, error) { if len(fkc.fkValues) == 0 && len(fkc.fkUpdatedValuesMap) == 0 { return nil, nil } diff --git a/pkg/executor/importer/BUILD.bazel b/pkg/executor/importer/BUILD.bazel index 05bb1253cfba1..e10852388322a 100644 --- a/pkg/executor/importer/BUILD.bazel +++ b/pkg/executor/importer/BUILD.bazel @@ -129,6 +129,7 @@ go_test( "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/planner/core", + "//pkg/planner/core/base", "//pkg/planner/util", "//pkg/session", "//pkg/sessionctx/variable", diff --git a/pkg/executor/importer/importer_testkit_test.go b/pkg/executor/importer/importer_testkit_test.go index 2d3a1979ef407..3c88fce1befed 100644 --- a/pkg/executor/importer/importer_testkit_test.go +++ b/pkg/executor/importer/importer_testkit_test.go @@ -39,6 +39,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/model" plannercore "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/session" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/testkit" @@ -261,7 +262,7 @@ func getTableImporter(ctx context.Context, t *testing.T, store kv.Storage, table require.True(t, ok) table, err := do.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr(tableName)) require.NoError(t, err) - var selectPlan plannercore.PhysicalPlan + var selectPlan base.PhysicalPlan if path == "" { selectPlan = &plannercore.PhysicalSelection{} } diff --git a/pkg/executor/index_merge_reader.go b/pkg/executor/index_merge_reader.go index 3f75db213c306..b5013cf60f25f 100644 --- a/pkg/executor/index_merge_reader.go +++ b/pkg/executor/index_merge_reader.go @@ -39,6 +39,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" plannercore "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/base" plannerutil "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/table" @@ -129,8 +130,8 @@ type IndexMergeReaderExecutor struct { // checkIndexValue is used to check the consistency of the index data. *checkIndexValue // nolint:unused - partialPlans [][]plannercore.PhysicalPlan - tblPlans []plannercore.PhysicalPlan + partialPlans [][]base.PhysicalPlan + tblPlans []base.PhysicalPlan partialNetDataSizes []float64 dataAvgRowSize float64 @@ -1649,7 +1650,7 @@ type partialIndexWorker struct { scannedKeys uint64 pushedLimit *plannercore.PushedDownLimit dagPB *tipb.DAGRequest - plan []plannercore.PhysicalPlan + plan []base.PhysicalPlan } func syncErr(ctx context.Context, finished <-chan struct{}, errCh chan<- *indexMergeTableTask, err error) { @@ -1840,7 +1841,7 @@ type indexMergeTableScanWorker struct { workCh <-chan *indexMergeTableTask finished <-chan struct{} indexMergeExec *IndexMergeReaderExecutor - tblPlans []plannercore.PhysicalPlan + tblPlans []base.PhysicalPlan // memTracker is used to track the memory usage of this executor. memTracker *memory.Tracker diff --git a/pkg/executor/infoschema_reader.go b/pkg/executor/infoschema_reader.go index 8ca0eebf1e810..ea269b5d65571 100644 --- a/pkg/executor/infoschema_reader.go +++ b/pkg/executor/infoschema_reader.go @@ -48,6 +48,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" plannercore "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/privilege" "github.com/pingcap/tidb/pkg/privilege/privileges" "github.com/pingcap/tidb/pkg/session/txninfo" @@ -858,7 +859,7 @@ func (e *hugeMemTableRetriever) dataForColumnsInTable(ctx context.Context, sctx e.viewMu.Lock() _, ok := e.viewSchemaMap[tbl.ID] if !ok { - var viewLogicalPlan plannercore.Plan + var viewLogicalPlan base.Plan internalCtx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnOthers) // Build plan is not thread safe, there will be concurrency on sessionctx. if err := runWithSystemSession(internalCtx, sctx, func(s sessionctx.Context) error { diff --git a/pkg/executor/internal/builder/BUILD.bazel b/pkg/executor/internal/builder/BUILD.bazel index 93eeff4a64c6a..071fbc2c3f697 100644 --- a/pkg/executor/internal/builder/BUILD.bazel +++ b/pkg/executor/internal/builder/BUILD.bazel @@ -9,7 +9,7 @@ go_library( "//pkg/distsql", "//pkg/kv", "//pkg/planner/context", - "//pkg/planner/core", + "//pkg/planner/core/base", "//pkg/sessionctx", "//pkg/sessionctx/variable", "//pkg/util/timeutil", diff --git a/pkg/executor/internal/builder/builder_utils.go b/pkg/executor/internal/builder/builder_utils.go index 4c8897a70d5c2..8178a6b8d67c6 100644 --- a/pkg/executor/internal/builder/builder_utils.go +++ b/pkg/executor/internal/builder/builder_utils.go @@ -18,7 +18,7 @@ import ( "github.com/pingcap/tidb/pkg/distsql" "github.com/pingcap/tidb/pkg/kv" planctx "github.com/pingcap/tidb/pkg/planner/context" - plannercore "github.com/pingcap/tidb/pkg/planner/core" + plannercore "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/util/timeutil" diff --git a/pkg/executor/internal/mpp/BUILD.bazel b/pkg/executor/internal/mpp/BUILD.bazel index c4217b71c3a58..a8ad35e867e6d 100644 --- a/pkg/executor/internal/mpp/BUILD.bazel +++ b/pkg/executor/internal/mpp/BUILD.bazel @@ -19,6 +19,7 @@ go_library( "//pkg/infoschema", "//pkg/kv", "//pkg/planner/core", + "//pkg/planner/core/base", "//pkg/sessionctx", "//pkg/sessionctx/variable", "//pkg/store/copr", diff --git a/pkg/executor/internal/mpp/executor_with_retry.go b/pkg/executor/internal/mpp/executor_with_retry.go index a9945798db1ce..fd2bd527dc48c 100644 --- a/pkg/executor/internal/mpp/executor_with_retry.go +++ b/pkg/executor/internal/mpp/executor_with_retry.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/tidb/pkg/executor/mppcoordmanager" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/kv" - plannercore "github.com/pingcap/tidb/pkg/planner/core" + plannercore "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/util/logutil" "github.com/pingcap/tidb/pkg/util/memory" diff --git a/pkg/executor/internal/mpp/local_mpp_coordinator.go b/pkg/executor/internal/mpp/local_mpp_coordinator.go index 874c3aa41d0fc..77fd00374e1ce 100644 --- a/pkg/executor/internal/mpp/local_mpp_coordinator.go +++ b/pkg/executor/internal/mpp/local_mpp_coordinator.go @@ -34,6 +34,7 @@ import ( "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/kv" plannercore "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/store/copr" @@ -106,7 +107,7 @@ type localMppCoordinator struct { ctx context.Context sessionCtx sessionctx.Context is infoschema.InfoSchema - originalPlan plannercore.PhysicalPlan + originalPlan base.PhysicalPlan reqMap map[int64]*mppRequestReport cancelFunc context.CancelFunc @@ -151,7 +152,7 @@ type localMppCoordinator struct { } // NewLocalMPPCoordinator creates a new localMppCoordinator instance -func NewLocalMPPCoordinator(ctx context.Context, sctx sessionctx.Context, is infoschema.InfoSchema, plan plannercore.PhysicalPlan, planIDs []int, startTS uint64, mppQueryID kv.MPPQueryID, gatherID uint64, coordinatorAddr string, memTracker *memory.Tracker) *localMppCoordinator { +func NewLocalMPPCoordinator(ctx context.Context, sctx sessionctx.Context, is infoschema.InfoSchema, plan base.PhysicalPlan, planIDs []int, startTS uint64, mppQueryID kv.MPPQueryID, gatherID uint64, coordinatorAddr string, memTracker *memory.Tracker) *localMppCoordinator { if sctx.GetSessionVars().ChooseMppVersion() < kv.MppVersionV2 { coordinatorAddr = "" } @@ -181,7 +182,7 @@ func NewLocalMPPCoordinator(ctx context.Context, sctx sessionctx.Context, is inf } func (c *localMppCoordinator) appendMPPDispatchReq(pf *plannercore.Fragment) error { - dagReq, err := builder.ConstructDAGReq(c.sessionCtx, []plannercore.PhysicalPlan{pf.ExchangeSender}, kv.TiFlash) + dagReq, err := builder.ConstructDAGReq(c.sessionCtx, []base.PhysicalPlan{pf.ExchangeSender}, kv.TiFlash) if err != nil { return errors.Trace(err) } @@ -336,7 +337,7 @@ func (c *localMppCoordinator) fixTaskForCTEStorageAndReader(exec *tipb.Executor, // DFS to check if plan needs report execution summary through ReportMPPTaskStatus mpp service // Currently, return true if plan contains limit operator -func needReportExecutionSummary(plan plannercore.PhysicalPlan) bool { +func needReportExecutionSummary(plan base.PhysicalPlan) bool { switch x := plan.(type) { case *plannercore.PhysicalLimit: return true diff --git a/pkg/executor/internal/testutil/BUILD.bazel b/pkg/executor/internal/testutil/BUILD.bazel index 62764c8c5c033..15815c6e1cef2 100644 --- a/pkg/executor/internal/testutil/BUILD.bazel +++ b/pkg/executor/internal/testutil/BUILD.bazel @@ -17,6 +17,7 @@ go_library( "//pkg/parser/ast", "//pkg/parser/mysql", "//pkg/planner/core", + "//pkg/planner/core/base", "//pkg/planner/property", "//pkg/sessionctx", "//pkg/sessionctx/variable", diff --git a/pkg/executor/internal/testutil/testutil.go b/pkg/executor/internal/testutil/testutil.go index 566202ca8aaa1..ae8dd25a62980 100644 --- a/pkg/executor/internal/testutil/testutil.go +++ b/pkg/executor/internal/testutil/testutil.go @@ -26,7 +26,7 @@ import ( "github.com/pingcap/tidb/pkg/executor/internal/exec" "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/parser/mysql" - plannercore "github.com/pingcap/tidb/pkg/planner/core" + plannercore "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/types" diff --git a/pkg/executor/mpp_gather.go b/pkg/executor/mpp_gather.go index 45160e22c4490..23fcc16da63a4 100644 --- a/pkg/executor/mpp_gather.go +++ b/pkg/executor/mpp_gather.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/model" plannercore "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/types" @@ -56,7 +57,7 @@ func getMPPQueryTS(ctx sessionctx.Context) uint64 { return mppQueryInfo.QueryTS.Load() } -func collectPlanIDs(plan plannercore.PhysicalPlan, ids []int) []int { +func collectPlanIDs(plan base.PhysicalPlan, ids []int) []int { ids = append(ids, plan.ID()) for _, child := range plan.Children() { ids = collectPlanIDs(child, ids) @@ -68,7 +69,7 @@ func collectPlanIDs(plan plannercore.PhysicalPlan, ids []int) []int { type MPPGather struct { exec.BaseExecutor is infoschema.InfoSchema - originalPlan plannercore.PhysicalPlan + originalPlan base.PhysicalPlan startTS uint64 mppQueryID kv.MPPQueryID respIter distsql.SelectResult diff --git a/pkg/executor/prepared.go b/pkg/executor/prepared.go index c9ae6edceb707..54fdc922aeb5f 100644 --- a/pkg/executor/prepared.go +++ b/pkg/executor/prepared.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" plannercore "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util" @@ -160,7 +161,7 @@ type ExecuteExec struct { usingVars []expression.Expression stmtExec exec.Executor stmt ast.StmtNode - plan plannercore.Plan + plan base.Plan lowerPriority bool outputNames []*types.FieldName } diff --git a/pkg/executor/resource_tag_test.go b/pkg/executor/resource_tag_test.go index 23fecdad837c0..a65a096afdb4e 100644 --- a/pkg/executor/resource_tag_test.go +++ b/pkg/executor/resource_tag_test.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/tidb/pkg/config" "github.com/pingcap/tidb/pkg/parser" plannercore "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/store/mockstore/unistore" "github.com/pingcap/tidb/pkg/tablecodec" "github.com/pingcap/tidb/pkg/testkit" @@ -193,7 +194,7 @@ func TestResourceGroupTag(t *testing.T) { if expectPlanDigest == nil { info := tk.Session().ShowProcess() require.NotNil(t, info) - p, ok := info.Plan.(plannercore.Plan) + p, ok := info.Plan.(base.Plan) require.True(t, ok) _, expectPlanDigest = plannercore.NormalizePlan(p) diff --git a/pkg/executor/table_reader.go b/pkg/executor/table_reader.go index 00364a5f3ddb5..75381bd7c3c45 100644 --- a/pkg/executor/table_reader.go +++ b/pkg/executor/table_reader.go @@ -38,6 +38,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/model" planctx "github.com/pingcap/tidb/pkg/planner/context" plannercore "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/table" @@ -161,8 +162,8 @@ type TableReaderExecutor struct { // resultHandler handles the order of the result. Since (MAXInt64, MAXUint64] stores before [0, MaxInt64] physically // for unsigned int. resultHandler *tableResultHandler - plans []plannercore.PhysicalPlan - tablePlan plannercore.PhysicalPlan + plans []base.PhysicalPlan + tablePlan base.PhysicalPlan memTracker *memory.Tracker selectResultHook // for testing diff --git a/pkg/executor/table_readers_required_rows_test.go b/pkg/executor/table_readers_required_rows_test.go index 18133245570b7..6be0105c8a097 100644 --- a/pkg/executor/table_readers_required_rows_test.go +++ b/pkg/executor/table_readers_required_rows_test.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/table/tables" "github.com/pingcap/tidb/pkg/types" @@ -141,7 +142,7 @@ func buildTableReader(sctx sessionctx.Context) exec.Executor { } func buildMockDAGRequest(sctx sessionctx.Context) *tipb.DAGRequest { - req, err := builder.ConstructDAGReq(sctx, []core.PhysicalPlan{&core.PhysicalTableScan{ + req, err := builder.ConstructDAGReq(sctx, []base.PhysicalPlan{&core.PhysicalTableScan{ Columns: []*model.ColumnInfo{}, Table: &model.TableInfo{ID: 12345, PKIsHandle: false}, Desc: false, diff --git a/pkg/executor/test/executor/BUILD.bazel b/pkg/executor/test/executor/BUILD.bazel index 9d84138fc3720..7862539266790 100644 --- a/pkg/executor/test/executor/BUILD.bazel +++ b/pkg/executor/test/executor/BUILD.bazel @@ -27,6 +27,7 @@ go_test( "//pkg/parser/terror", "//pkg/planner", "//pkg/planner/core", + "//pkg/planner/core/base", "//pkg/session", "//pkg/sessionctx", "//pkg/sessionctx/stmtctx", diff --git a/pkg/executor/test/executor/executor_test.go b/pkg/executor/test/executor/executor_test.go index c0591bde97465..40a79b3a81013 100644 --- a/pkg/executor/test/executor/executor_test.go +++ b/pkg/executor/test/executor/executor_test.go @@ -48,6 +48,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/planner" plannercore "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/session" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" @@ -721,7 +722,7 @@ func TestUnreasonablyClose(t *testing.T) { // To enable the shuffleExec operator. tk.MustExec("set @@tidb_merge_join_concurrency=4") - var opsNeedsCovered = []plannercore.PhysicalPlan{ + var opsNeedsCovered = []base.PhysicalPlan{ &plannercore.PhysicalHashJoin{}, &plannercore.PhysicalMergeJoin{}, &plannercore.PhysicalIndexJoin{}, @@ -788,8 +789,8 @@ func TestUnreasonablyClose(t *testing.T) { // This for loop level traverses the plan tree to get which operators are covered. var hasCTE bool - for child := []plannercore.PhysicalPlan{p.(plannercore.PhysicalPlan)}; len(child) != 0; { - newChild := make([]plannercore.PhysicalPlan, 0, len(child)) + for child := []base.PhysicalPlan{p.(base.PhysicalPlan)}; len(child) != 0; { + newChild := make([]base.PhysicalPlan, 0, len(child)) for _, ch := range child { found := false for k, t := range opsNeedsCovered { @@ -1370,7 +1371,7 @@ func TestCollectDMLRuntimeStats(t *testing.T) { getRootStats := func() string { info := tk.Session().ShowProcess() require.NotNil(t, info) - p, ok := info.Plan.(plannercore.Plan) + p, ok := info.Plan.(base.Plan) require.True(t, ok) stats := tk.Session().GetSessionVars().StmtCtx.RuntimeStatsColl.GetRootStats(p.ID()) return stats.String() @@ -1499,7 +1500,7 @@ func TestGetResultRowsCount(t *testing.T) { } info := tk.Session().ShowProcess() require.NotNil(t, info) - p, ok := info.Plan.(plannercore.Plan) + p, ok := info.Plan.(base.Plan) require.True(t, ok) cnt := executor.GetResultRowsCount(tk.Session().GetSessionVars().StmtCtx, p) require.Equal(t, ca.row, cnt, fmt.Sprintf("sql: %v", ca.sql)) @@ -2625,7 +2626,7 @@ func TestIsFastPlan(t *testing.T) { } info := tk.Session().ShowProcess() require.NotNil(t, info) - p, ok := info.Plan.(plannercore.Plan) + p, ok := info.Plan.(base.Plan) require.True(t, ok) ok = executor.IsFastPlan(p) require.Equal(t, ca.isFastPlan, ok) diff --git a/pkg/planner/BUILD.bazel b/pkg/planner/BUILD.bazel index 4c9c7a478df2d..118d329f7dd6a 100644 --- a/pkg/planner/BUILD.bazel +++ b/pkg/planner/BUILD.bazel @@ -15,6 +15,7 @@ go_library( "//pkg/planner/cascades", "//pkg/planner/context", "//pkg/planner/core", + "//pkg/planner/core/base", "//pkg/planner/util/debugtrace", "//pkg/privilege", "//pkg/sessionctx", diff --git a/pkg/planner/cascades/BUILD.bazel b/pkg/planner/cascades/BUILD.bazel index 25d33ca880bfe..e39e66add7f6e 100644 --- a/pkg/planner/cascades/BUILD.bazel +++ b/pkg/planner/cascades/BUILD.bazel @@ -19,6 +19,7 @@ go_library( "//pkg/parser/mysql", "//pkg/planner/context", "//pkg/planner/core", + "//pkg/planner/core/base", "//pkg/planner/implementation", "//pkg/planner/memo", "//pkg/planner/pattern", diff --git a/pkg/planner/cascades/optimize.go b/pkg/planner/cascades/optimize.go index 71a832b7f2350..d580b227b58b5 100644 --- a/pkg/planner/cascades/optimize.go +++ b/pkg/planner/cascades/optimize.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/tidb/pkg/expression" plannercore "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/memo" "github.com/pingcap/tidb/pkg/planner/pattern" "github.com/pingcap/tidb/pkg/planner/property" @@ -98,7 +99,7 @@ func (opt *Optimizer) GetImplementationRules(node plannercore.LogicalPlan) []Imp // for each expression in each group under the required physical property. A // memo structure is used for a group to reduce the repeated search on the same // required physical property. -func (opt *Optimizer) FindBestPlan(sctx plannercore.PlanContext, logical plannercore.LogicalPlan) (p plannercore.PhysicalPlan, cost float64, err error) { +func (opt *Optimizer) FindBestPlan(sctx base.PlanContext, logical plannercore.LogicalPlan) (p base.PhysicalPlan, cost float64, err error) { logical, err = opt.onPhasePreprocessing(sctx, logical) if err != nil { return nil, 0, err @@ -116,7 +117,7 @@ func (opt *Optimizer) FindBestPlan(sctx plannercore.PlanContext, logical planner return p, cost, err } -func (*Optimizer) onPhasePreprocessing(_ plannercore.PlanContext, plan plannercore.LogicalPlan) (plannercore.LogicalPlan, error) { +func (*Optimizer) onPhasePreprocessing(_ base.PlanContext, plan plannercore.LogicalPlan) (plannercore.LogicalPlan, error) { var err error plan, err = plan.PruneColumns(plan.Schema().Columns, nil) if err != nil { @@ -125,7 +126,7 @@ func (*Optimizer) onPhasePreprocessing(_ plannercore.PlanContext, plan plannerco return plan, nil } -func (opt *Optimizer) onPhaseExploration(_ plannercore.PlanContext, g *memo.Group) error { +func (opt *Optimizer) onPhaseExploration(_ base.PlanContext, g *memo.Group) error { for round, ruleBatch := range opt.transformationRuleBatches { for !g.Explored(round) { err := opt.exploreGroup(g, round, ruleBatch) @@ -242,7 +243,7 @@ func (opt *Optimizer) fillGroupStats(g *memo.Group) (err error) { } // onPhaseImplementation starts implementation physical operators from given root Group. -func (opt *Optimizer) onPhaseImplementation(_ plannercore.PlanContext, g *memo.Group) (plannercore.PhysicalPlan, float64, error) { +func (opt *Optimizer) onPhaseImplementation(_ base.PlanContext, g *memo.Group) (base.PhysicalPlan, float64, error) { prop := &property.PhysicalProperty{ ExpectedCnt: math.MaxFloat64, } diff --git a/pkg/planner/core/BUILD.bazel b/pkg/planner/core/BUILD.bazel index 9812a7bbc04ae..2eac2f143b97a 100644 --- a/pkg/planner/core/BUILD.bazel +++ b/pkg/planner/core/BUILD.bazel @@ -30,7 +30,6 @@ go_library( "pb_to_plan.go", "physical_plans.go", "plan.go", - "plan_base.go", "plan_cache.go", "plan_cache_lru.go", "plan_cache_param.go", @@ -113,6 +112,7 @@ go_library( "//pkg/parser/types", "//pkg/planner/cardinality", "//pkg/planner/context", + "//pkg/planner/core/base", "//pkg/planner/core/internal", "//pkg/planner/core/internal/base", "//pkg/planner/core/metrics", @@ -250,6 +250,7 @@ go_test( "//pkg/parser/mysql", "//pkg/parser/terror", "//pkg/planner", + "//pkg/planner/core/base", "//pkg/planner/core/internal", "//pkg/planner/property", "//pkg/planner/util", diff --git a/pkg/planner/core/access_object.go b/pkg/planner/core/access_object.go index e8a62dbce985d..edcafd0cafe5c 100644 --- a/pkg/planner/core/access_object.go +++ b/pkg/planner/core/access_object.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tipb/go-tipb" ) @@ -39,7 +40,7 @@ type dataAccesser interface { } type partitionAccesser interface { - accessObject(PlanContext) AccessObject + accessObject(base.PlanContext) AccessObject } // AccessObject represents what is accessed by an operator. @@ -371,7 +372,7 @@ func (p *BatchPointGetPlan) AccessObject() AccessObject { return res } -func getDynamicAccessPartition(sctx PlanContext, tblInfo *model.TableInfo, physPlanPartInfo *PhysPlanPartInfo, asName string) (res *DynamicPartitionAccessObject) { +func getDynamicAccessPartition(sctx base.PlanContext, tblInfo *model.TableInfo, physPlanPartInfo *PhysPlanPartInfo, asName string) (res *DynamicPartitionAccessObject) { pi := tblInfo.GetPartitionInfo() if pi == nil || !sctx.GetSessionVars().StmtCtx.UseDynamicPartitionPrune() { return nil @@ -412,7 +413,7 @@ func getDynamicAccessPartition(sctx PlanContext, tblInfo *model.TableInfo, physP return res } -func (p *PhysicalTableReader) accessObject(sctx PlanContext) AccessObject { +func (p *PhysicalTableReader) accessObject(sctx base.PlanContext) AccessObject { if !sctx.GetSessionVars().StmtCtx.UseDynamicPartitionPrune() { return DynamicPartitionAccessObjects(nil) } @@ -466,7 +467,7 @@ func (p *PhysicalTableReader) accessObject(sctx PlanContext) AccessObject { return res } -func (p *PhysicalIndexReader) accessObject(sctx PlanContext) AccessObject { +func (p *PhysicalIndexReader) accessObject(sctx base.PlanContext) AccessObject { if !sctx.GetSessionVars().StmtCtx.UseDynamicPartitionPrune() { return DynamicPartitionAccessObjects(nil) } @@ -482,7 +483,7 @@ func (p *PhysicalIndexReader) accessObject(sctx PlanContext) AccessObject { return DynamicPartitionAccessObjects{res} } -func (p *PhysicalIndexLookUpReader) accessObject(sctx PlanContext) AccessObject { +func (p *PhysicalIndexLookUpReader) accessObject(sctx base.PlanContext) AccessObject { if !sctx.GetSessionVars().StmtCtx.UseDynamicPartitionPrune() { return DynamicPartitionAccessObjects(nil) } @@ -498,7 +499,7 @@ func (p *PhysicalIndexLookUpReader) accessObject(sctx PlanContext) AccessObject return DynamicPartitionAccessObjects{res} } -func (p *PhysicalIndexMergeReader) accessObject(sctx PlanContext) AccessObject { +func (p *PhysicalIndexMergeReader) accessObject(sctx base.PlanContext) AccessObject { if !sctx.GetSessionVars().StmtCtx.UseDynamicPartitionPrune() { return DynamicPartitionAccessObjects(nil) } diff --git a/pkg/planner/core/base/BUILD.bazel b/pkg/planner/core/base/BUILD.bazel new file mode 100644 index 0000000000000..67cda54d2f0bb --- /dev/null +++ b/pkg/planner/core/base/BUILD.bazel @@ -0,0 +1,22 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "base", + srcs = [ + "plan_base.go", + "task_base.go", + ], + importpath = "github.com/pingcap/tidb/pkg/planner/core/base", + visibility = ["//visibility:public"], + deps = [ + "//pkg/expression", + "//pkg/kv", + "//pkg/planner/context", + "//pkg/planner/property", + "//pkg/planner/util/coreusage", + "//pkg/types", + "//pkg/util/execdetails", + "//pkg/util/tracing", + "@com_github_pingcap_tipb//go-tipb", + ], +) diff --git a/pkg/planner/core/plan_base.go b/pkg/planner/core/base/plan_base.go similarity index 95% rename from pkg/planner/core/plan_base.go rename to pkg/planner/core/base/plan_base.go index ae8327c177953..e7b34db82722d 100644 --- a/pkg/planner/core/plan_base.go +++ b/pkg/planner/core/base/plan_base.go @@ -12,13 +12,14 @@ // See the License for the specific language governing permissions and // limitations under the License. -package core +package base import ( "fmt" "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/planner/context" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util/coreusage" "github.com/pingcap/tidb/pkg/types" @@ -27,6 +28,12 @@ import ( "github.com/pingcap/tipb/go-tipb" ) +// PlanContext is the context for building plan. +type PlanContext = context.PlanContext + +// BuildPBContext is the context for building `*tipb.Executor`. +type BuildPBContext = context.BuildPBContext + // Plan is the description of an execution flow. // It is created from ast.Node first, then optimized by the optimizer, // finally used by the executor to create a Cursor which executes the statement. diff --git a/pkg/planner/core/base/task_base.go b/pkg/planner/core/base/task_base.go new file mode 100644 index 0000000000000..44de858fe4961 --- /dev/null +++ b/pkg/planner/core/base/task_base.go @@ -0,0 +1,34 @@ +// Copyright 2024 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package base + +// Task is a new version of `PhysicalPlanInfo`. It stores cost information for a task. +// A task may be CopTask, RootTask, MPPTaskMeta or a ParallelTask. +type Task interface { + // Count returns current task's row count. + Count() float64 + // Copy return a shallow copy of current task with the same pointer to p. + Copy() Task + // Plan returns current task's plan. + Plan() PhysicalPlan + // Invalid returns whether current task is invalid. + Invalid() bool + // ConvertToRootTask will convert current task as root type. + // Here we change return type as interface to avoid import cycle. + // Basic interface definition shouldn't depend on concrete implementation structure. + ConvertToRootTask(ctx PlanContext) Task + // MemoryUsage returns the memory usage of current task. + MemoryUsage() int64 +} diff --git a/pkg/planner/core/casetest/BUILD.bazel b/pkg/planner/core/casetest/BUILD.bazel index 2baabf61e3ca3..bf8d00caf5079 100644 --- a/pkg/planner/core/casetest/BUILD.bazel +++ b/pkg/planner/core/casetest/BUILD.bazel @@ -18,6 +18,7 @@ go_test( "//pkg/parser", "//pkg/parser/model", "//pkg/planner/core", + "//pkg/planner/core/base", "//pkg/planner/property", "//pkg/testkit", "//pkg/testkit/testdata", diff --git a/pkg/planner/core/casetest/plan_test.go b/pkg/planner/core/casetest/plan_test.go index ce5ef805f7d11..46347dc353c01 100644 --- a/pkg/planner/core/casetest/plan_test.go +++ b/pkg/planner/core/casetest/plan_test.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/testdata" "github.com/pingcap/tidb/pkg/util/plancodec" @@ -80,7 +81,7 @@ func TestPreferRangeScan(t *testing.T) { tk.MustExec(tt) info := tk.Session().ShowProcess() require.NotNil(t, info) - p, ok := info.Plan.(core.Plan) + p, ok := info.Plan.(base.Plan) require.True(t, ok) normalized, digest := core.NormalizePlan(p) @@ -131,7 +132,7 @@ func TestNormalizedPlan(t *testing.T) { tk.MustExec(tt) info := tk.Session().ShowProcess() require.NotNil(t, info) - p, ok := info.Plan.(core.Plan) + p, ok := info.Plan.(base.Plan) require.True(t, ok) normalized, digest := core.NormalizePlan(p) @@ -177,13 +178,13 @@ func TestPlanDigest4InList(t *testing.T) { tk.MustExec(query1) info1 := tk.Session().ShowProcess() require.NotNil(t, info1) - p1, ok := info1.Plan.(core.Plan) + p1, ok := info1.Plan.(base.Plan) require.True(t, ok) _, digest1 := core.NormalizePlan(p1) tk.MustExec(query2) info2 := tk.Session().ShowProcess() require.NotNil(t, info2) - p2, ok := info2.Plan.(core.Plan) + p2, ok := info2.Plan.(base.Plan) require.True(t, ok) _, digest2 := core.NormalizePlan(p2) require.Equal(t, digest1, digest2) @@ -215,13 +216,13 @@ func TestIssue47634(t *testing.T) { tk.MustExec(query1) info1 := tk.Session().ShowProcess() require.NotNil(t, info1) - p1, ok := info1.Plan.(core.Plan) + p1, ok := info1.Plan.(base.Plan) require.True(t, ok) _, digest1 := core.NormalizePlan(p1) tk.MustExec(query2) info2 := tk.Session().ShowProcess() require.NotNil(t, info2) - p2, ok := info2.Plan.(core.Plan) + p2, ok := info2.Plan.(base.Plan) require.True(t, ok) _, digest2 := core.NormalizePlan(p2) require.Equal(t, digest1, digest2) diff --git a/pkg/planner/core/casetest/planstats/BUILD.bazel b/pkg/planner/core/casetest/planstats/BUILD.bazel index b8753e55159b9..8d738edc3b502 100644 --- a/pkg/planner/core/casetest/planstats/BUILD.bazel +++ b/pkg/planner/core/casetest/planstats/BUILD.bazel @@ -18,6 +18,7 @@ go_test( "//pkg/parser/model", "//pkg/planner", "//pkg/planner/core", + "//pkg/planner/core/base", "//pkg/sessionctx", "//pkg/sessionctx/stmtctx", "//pkg/statistics", diff --git a/pkg/planner/core/casetest/planstats/plan_stats_test.go b/pkg/planner/core/casetest/planstats/plan_stats_test.go index 5727858447e77..5f7c4f05d4dcb 100644 --- a/pkg/planner/core/casetest/planstats/plan_stats_test.go +++ b/pkg/planner/core/casetest/planstats/plan_stats_test.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/planner" plannercore "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" "github.com/pingcap/tidb/pkg/statistics" @@ -66,11 +67,11 @@ func TestPlanStatsLoad(t *testing.T) { testCases := []struct { sql string skip bool - check func(p plannercore.Plan, tableInfo *model.TableInfo) + check func(p base.Plan, tableInfo *model.TableInfo) }{ { // DataSource sql: "select * from t where c>1", - check: func(p plannercore.Plan, tableInfo *model.TableInfo) { + check: func(p base.Plan, tableInfo *model.TableInfo) { switch pp := p.(type) { case *plannercore.PhysicalTableReader: stats := pp.StatsInfo().HistColl @@ -83,7 +84,7 @@ func TestPlanStatsLoad(t *testing.T) { }, { // PartitionTable sql: "select * from pt where a < 15 and c > 1", - check: func(p plannercore.Plan, tableInfo *model.TableInfo) { + check: func(p base.Plan, tableInfo *model.TableInfo) { pua, ok := p.(*plannercore.PhysicalUnionAll) require.True(t, ok) for _, child := range pua.Children() { @@ -93,8 +94,8 @@ func TestPlanStatsLoad(t *testing.T) { }, { // Join sql: "select * from t t1 inner join t t2 on t1.b=t2.b where t1.d=3", - check: func(p plannercore.Plan, tableInfo *model.TableInfo) { - pp, ok := p.(plannercore.PhysicalPlan) + check: func(p base.Plan, tableInfo *model.TableInfo) { + pp, ok := p.(base.PhysicalPlan) require.True(t, ok) require.Greater(t, countFullStats(pp.Children()[0].StatsInfo().HistColl, tableInfo.Columns[3].ID), 0) require.Greater(t, countFullStats(pp.Children()[1].StatsInfo().HistColl, tableInfo.Columns[3].ID), 0) @@ -102,7 +103,7 @@ func TestPlanStatsLoad(t *testing.T) { }, { // Apply sql: "select * from t t1 where t1.b > (select count(*) from t t2 where t2.c > t1.a and t2.d>1) and t1.c>2", - check: func(p plannercore.Plan, tableInfo *model.TableInfo) { + check: func(p base.Plan, tableInfo *model.TableInfo) { pp, ok := p.(*plannercore.PhysicalProjection) require.True(t, ok) pa, ok := pp.Children()[0].(*plannercore.PhysicalApply) @@ -115,7 +116,7 @@ func TestPlanStatsLoad(t *testing.T) { }, { // > Any sql: "select * from t where t.b > any(select d from t where t.c > 2)", - check: func(p plannercore.Plan, tableInfo *model.TableInfo) { + check: func(p base.Plan, tableInfo *model.TableInfo) { ph, ok := p.(*plannercore.PhysicalHashJoin) require.True(t, ok) ptr, ok := ph.Children()[0].(*plannercore.PhysicalTableReader) @@ -125,7 +126,7 @@ func TestPlanStatsLoad(t *testing.T) { }, { // in sql: "select * from t where t.b in (select d from t where t.c > 2)", - check: func(p plannercore.Plan, tableInfo *model.TableInfo) { + check: func(p base.Plan, tableInfo *model.TableInfo) { ph, ok := p.(*plannercore.PhysicalHashJoin) require.True(t, ok) ptr, ok := ph.Children()[1].(*plannercore.PhysicalTableReader) @@ -135,7 +136,7 @@ func TestPlanStatsLoad(t *testing.T) { }, { // not in sql: "select * from t where t.b not in (select d from t where t.c > 2)", - check: func(p plannercore.Plan, tableInfo *model.TableInfo) { + check: func(p base.Plan, tableInfo *model.TableInfo) { ph, ok := p.(*plannercore.PhysicalHashJoin) require.True(t, ok) ptr, ok := ph.Children()[1].(*plannercore.PhysicalTableReader) @@ -145,7 +146,7 @@ func TestPlanStatsLoad(t *testing.T) { }, { // exists sql: "select * from t t1 where exists (select * from t t2 where t1.b > t2.d and t2.c>1)", - check: func(p plannercore.Plan, tableInfo *model.TableInfo) { + check: func(p base.Plan, tableInfo *model.TableInfo) { ph, ok := p.(*plannercore.PhysicalHashJoin) require.True(t, ok) ptr, ok := ph.Children()[1].(*plannercore.PhysicalTableReader) @@ -155,7 +156,7 @@ func TestPlanStatsLoad(t *testing.T) { }, { // not exists sql: "select * from t t1 where not exists (select * from t t2 where t1.b > t2.d and t2.c>1)", - check: func(p plannercore.Plan, tableInfo *model.TableInfo) { + check: func(p base.Plan, tableInfo *model.TableInfo) { ph, ok := p.(*plannercore.PhysicalHashJoin) require.True(t, ok) ptr, ok := ph.Children()[1].(*plannercore.PhysicalTableReader) @@ -165,7 +166,7 @@ func TestPlanStatsLoad(t *testing.T) { }, { // CTE sql: "with cte(x, y) as (select d + 1, b from t where c > 1) select * from cte where x < 3", - check: func(p plannercore.Plan, tableInfo *model.TableInfo) { + check: func(p base.Plan, tableInfo *model.TableInfo) { ps, ok := p.(*plannercore.PhysicalProjection) require.True(t, ok) pc, ok := ps.Children()[0].(*plannercore.PhysicalTableReader) @@ -179,7 +180,7 @@ func TestPlanStatsLoad(t *testing.T) { }, { // recursive CTE sql: "with recursive cte(x, y) as (select a, b from t where c > 1 union select x + 1, y from cte where x < 5) select * from cte", - check: func(p plannercore.Plan, tableInfo *model.TableInfo) { + check: func(p base.Plan, tableInfo *model.TableInfo) { pc, ok := p.(*plannercore.PhysicalCTE) require.True(t, ok) pp, ok := pc.SeedPlan.(*plannercore.PhysicalProjection) @@ -191,7 +192,7 @@ func TestPlanStatsLoad(t *testing.T) { }, { // check idx(b) sql: "select * from t USE INDEX(idx) where b >= 10", - check: func(p plannercore.Plan, tableInfo *model.TableInfo) { + check: func(p base.Plan, tableInfo *model.TableInfo) { pr, ok := p.(*plannercore.PhysicalIndexLookUpReader) require.True(t, ok) pis, ok := pr.IndexPlans[0].(*plannercore.PhysicalIndexScan) diff --git a/pkg/planner/core/casetest/tiflash_selection_late_materialization_test.go b/pkg/planner/core/casetest/tiflash_selection_late_materialization_test.go index c85a3608b901d..10744b3be2dfc 100644 --- a/pkg/planner/core/casetest/tiflash_selection_late_materialization_test.go +++ b/pkg/planner/core/casetest/tiflash_selection_late_materialization_test.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/testdata" "github.com/pingcap/tidb/pkg/util/plancodec" @@ -67,7 +68,7 @@ func TestTiFlashLateMaterialization(t *testing.T) { tk.MustExec(tt) info := tk.Session().ShowProcess() require.NotNil(t, info) - p, ok := info.Plan.(core.Plan) + p, ok := info.Plan.(base.Plan) require.True(t, ok) normalized, digest := core.NormalizePlan(p) diff --git a/pkg/planner/core/common_plans.go b/pkg/planner/core/common_plans.go index 38e4c42d43dd8..392755ecfb9f1 100644 --- a/pkg/planner/core/common_plans.go +++ b/pkg/planner/core/common_plans.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util/coreusage" "github.com/pingcap/tidb/pkg/sessionctx/variable" @@ -191,12 +192,12 @@ type Execute struct { Params []expression.Expression PrepStmt *PlanCacheStmt Stmt ast.StmtNode - Plan Plan + Plan base.Plan } // Check if result of GetVar expr is BinaryLiteral // Because GetVar use String to represent BinaryLiteral, here we need to convert string back to BinaryLiteral. -func isGetVarBinaryLiteral(sctx PlanContext, expr expression.Expression) (res bool) { +func isGetVarBinaryLiteral(sctx base.PlanContext, expr expression.Expression) (res bool) { scalarFunc, ok := expr.(*expression.ScalarFunction) if ok && scalarFunc.FuncName.L == ast.GetVar { name, isNull, err := scalarFunc.GetArgs()[0].EvalString(sctx.GetExprCtx().GetEvalCtx(), chunk.Row{}) @@ -358,7 +359,7 @@ type Insert struct { GenCols InsertGeneratedColumns - SelectPlan PhysicalPlan + SelectPlan base.PhysicalPlan IsReplace bool @@ -424,7 +425,7 @@ type Update struct { VirtualAssignmentsOffset int - SelectPlan PhysicalPlan + SelectPlan base.PhysicalPlan TblColPosInfos TblColPosInfoSlice @@ -472,7 +473,7 @@ type Delete struct { IsMultiTable bool - SelectPlan PhysicalPlan + SelectPlan base.PhysicalPlan TblColPosInfos TblColPosInfoSlice @@ -587,7 +588,7 @@ type ImportInto struct { GenCols InsertGeneratedColumns Stmt string - SelectPlan PhysicalPlan + SelectPlan base.PhysicalPlan } // LoadStats represents a load stats plan. @@ -678,7 +679,7 @@ type DDL struct { type SelectInto struct { baseSchemaProducer - TargetPlan Plan + TargetPlan base.Plan IntoOpt *ast.SelectIntoOption LineFieldsInfo } @@ -762,7 +763,7 @@ func JSONToString(j []*ExplainInfoForEncode) (string, error) { type Explain struct { baseSchemaProducer - TargetPlan Plan + TargetPlan base.Plan Format string Analyze bool ExecStmt ast.StmtNode @@ -773,7 +774,7 @@ type Explain struct { } // GetExplainRowsForPlan get explain rows for plan. -func GetExplainRowsForPlan(plan Plan) (rows [][]string) { +func GetExplainRowsForPlan(plan base.Plan) (rows [][]string) { explain := &Explain{ TargetPlan: plan, Format: types.ExplainFormatROW, @@ -856,7 +857,7 @@ func (e *Explain) RenderResult() error { if e.Analyze && strings.ToLower(e.Format) == types.ExplainFormatTrueCardCost { // true_card_cost mode is used to calibrate the cost model. - pp, ok := e.TargetPlan.(PhysicalPlan) + pp, ok := e.TargetPlan.(base.PhysicalPlan) if ok { if _, err := getPlanCost(pp, property.RootTaskType, coreusage.NewDefaultPlanCostOption().WithCostFlag(coreusage.CostFlagRecalculate|coreusage.CostFlagUseTrueCardinality|coreusage.CostFlagTrace)); err != nil { @@ -895,7 +896,7 @@ func (e *Explain) RenderResult() error { } if strings.ToLower(e.Format) == types.ExplainFormatCostTrace { - if pp, ok := e.TargetPlan.(PhysicalPlan); ok { + if pp, ok := e.TargetPlan.(base.PhysicalPlan); ok { // trigger getPlanCost again with CostFlagTrace to record all cost formulas if _, err := getPlanCost(pp, property.RootTaskType, coreusage.NewDefaultPlanCostOption().WithCostFlag(coreusage.CostFlagRecalculate|coreusage.CostFlagTrace)); err != nil { @@ -918,7 +919,7 @@ func (e *Explain) RenderResult() error { } } case types.ExplainFormatDOT: - if physicalPlan, ok := e.TargetPlan.(PhysicalPlan); ok { + if physicalPlan, ok := e.TargetPlan.(base.PhysicalPlan); ok { e.prepareDotInfo(physicalPlan) } case types.ExplainFormatHint: @@ -1018,7 +1019,7 @@ func (e *Explain) explainFlatOpInRowFormat(flatOp *FlatOperator) { e.prepareOperatorInfo(flatOp.Origin, taskTp, textTreeExplainID) } -func getRuntimeInfoStr(ctx PlanContext, p Plan, runtimeStatsColl *execdetails.RuntimeStatsColl) (actRows, analyzeInfo, memoryInfo, diskInfo string) { +func getRuntimeInfoStr(ctx base.PlanContext, p base.Plan, runtimeStatsColl *execdetails.RuntimeStatsColl) (actRows, analyzeInfo, memoryInfo, diskInfo string) { if runtimeStatsColl == nil { runtimeStatsColl = ctx.GetSessionVars().StmtCtx.RuntimeStatsColl if runtimeStatsColl == nil { @@ -1049,7 +1050,7 @@ func getRuntimeInfoStr(ctx PlanContext, p Plan, runtimeStatsColl *execdetails.Ru return } -func getRuntimeInfo(ctx PlanContext, p Plan, runtimeStatsColl *execdetails.RuntimeStatsColl) ( +func getRuntimeInfo(ctx base.PlanContext, p base.Plan, runtimeStatsColl *execdetails.RuntimeStatsColl) ( rootStats *execdetails.RootRuntimeStats, copStats *execdetails.CopRuntimeStats, memTracker *memory.Tracker, @@ -1074,7 +1075,7 @@ func getRuntimeInfo(ctx PlanContext, p Plan, runtimeStatsColl *execdetails.Runti // prepareOperatorInfo generates the following information for every plan: // operator id, estimated rows, task type, access object and other operator info. -func (e *Explain) prepareOperatorInfo(p Plan, taskType, id string) { +func (e *Explain) prepareOperatorInfo(p base.Plan, taskType, id string) { if p.ExplainID().String() == "_0" { return } @@ -1106,7 +1107,7 @@ func (e *Explain) prepareOperatorInfo(p Plan, taskType, id string) { e.Rows = append(e.Rows, row) } -func (e *Explain) prepareOperatorInfoForJSONFormat(p Plan, taskType, id string, explainID string) *ExplainInfoForEncode { +func (e *Explain) prepareOperatorInfoForJSONFormat(p base.Plan, taskType, id string, explainID string) *ExplainInfoForEncode { if p.ExplainID().String() == "_0" { return nil } @@ -1127,7 +1128,7 @@ func (e *Explain) prepareOperatorInfoForJSONFormat(p Plan, taskType, id string, return jsonRow } -func (e *Explain) getOperatorInfo(p Plan, id string) (estRows, estCost, costFormula, accessObject, operatorInfo string) { +func (e *Explain) getOperatorInfo(p base.Plan, id string) (estRows, estCost, costFormula, accessObject, operatorInfo string) { // For `explain for connection` statement, `e.ExplainRows` will be set. for _, row := range e.ExplainRows { if len(row) < 5 { @@ -1138,7 +1139,7 @@ func (e *Explain) getOperatorInfo(p Plan, id string) (estRows, estCost, costForm } } - pp, isPhysicalPlan := p.(PhysicalPlan) + pp, isPhysicalPlan := p.(base.PhysicalPlan) estRows = "N/A" estCost = "N/A" costFormula = "N/A" @@ -1171,7 +1172,7 @@ func (e *Explain) getOperatorInfo(p Plan, id string) (estRows, estCost, costForm } // BinaryPlanStrFromFlatPlan generates the compressed and encoded binary plan from a FlatPhysicalPlan. -func BinaryPlanStrFromFlatPlan(explainCtx PlanContext, flat *FlatPhysicalPlan) string { +func BinaryPlanStrFromFlatPlan(explainCtx base.PlanContext, flat *FlatPhysicalPlan) string { binary := binaryDataFromFlatPlan(explainCtx, flat) if binary == nil { return "" @@ -1184,7 +1185,7 @@ func BinaryPlanStrFromFlatPlan(explainCtx PlanContext, flat *FlatPhysicalPlan) s return str } -func binaryDataFromFlatPlan(explainCtx PlanContext, flat *FlatPhysicalPlan) *tipb.ExplainData { +func binaryDataFromFlatPlan(explainCtx base.PlanContext, flat *FlatPhysicalPlan) *tipb.ExplainData { if len(flat.Main) == 0 { return nil } @@ -1209,7 +1210,7 @@ func binaryDataFromFlatPlan(explainCtx PlanContext, flat *FlatPhysicalPlan) *tip return res } -func binaryOpTreeFromFlatOps(explainCtx PlanContext, ops FlatPlanTree) *tipb.ExplainOperator { +func binaryOpTreeFromFlatOps(explainCtx base.PlanContext, ops FlatPlanTree) *tipb.ExplainOperator { s := make([]tipb.ExplainOperator, len(ops)) for i, op := range ops { binaryOpFromFlatOp(explainCtx, op, &s[i]) @@ -1220,7 +1221,7 @@ func binaryOpTreeFromFlatOps(explainCtx PlanContext, ops FlatPlanTree) *tipb.Exp return &s[0] } -func binaryOpFromFlatOp(explainCtx PlanContext, fop *FlatOperator, out *tipb.ExplainOperator) { +func binaryOpFromFlatOp(explainCtx base.PlanContext, fop *FlatOperator, out *tipb.ExplainOperator) { out.Name = fop.Origin.ExplainID().String() switch fop.Label { case BuildSide: @@ -1254,7 +1255,7 @@ func binaryOpFromFlatOp(explainCtx PlanContext, fop *FlatOperator, out *tipb.Exp } if fop.IsPhysicalPlan { - p := fop.Origin.(PhysicalPlan) + p := fop.Origin.(base.PhysicalPlan) out.Cost, _ = getPlanCost(p, property.RootTaskType, coreusage.NewDefaultPlanCostOption()) out.EstRows = p.GetEstRowCountForDisplay() } else if statsInfo := fop.Origin.StatsInfo(); statsInfo != nil { @@ -1313,7 +1314,7 @@ func binaryOpFromFlatOp(explainCtx PlanContext, fop *FlatOperator, out *tipb.Exp } } -func (e *Explain) prepareDotInfo(p PhysicalPlan) { +func (e *Explain) prepareDotInfo(p base.PhysicalPlan) { buffer := bytes.NewBufferString("") fmt.Fprintf(buffer, "\ndigraph %s {\n", p.ExplainID()) e.prepareTaskDot(p, "root", buffer) @@ -1322,7 +1323,7 @@ func (e *Explain) prepareDotInfo(p PhysicalPlan) { e.Rows = append(e.Rows, []string{buffer.String()}) } -func (e *Explain) prepareTaskDot(p PhysicalPlan, taskTp string, buffer *bytes.Buffer) { +func (e *Explain) prepareTaskDot(p base.PhysicalPlan, taskTp string, buffer *bytes.Buffer) { fmt.Fprintf(buffer, "subgraph cluster%v{\n", p.ID()) buffer.WriteString("node [style=filled, color=lightgrey]\n") buffer.WriteString("color=black\n") @@ -1336,10 +1337,10 @@ func (e *Explain) prepareTaskDot(p PhysicalPlan, taskTp string, buffer *bytes.Bu fmt.Fprintf(buffer, "\"%s\"\n", p.ExplainID()) } - var copTasks []PhysicalPlan + var copTasks []base.PhysicalPlan var pipelines []string - for planQueue := []PhysicalPlan{p}; len(planQueue) > 0; planQueue = planQueue[1:] { + for planQueue := []base.PhysicalPlan{p}; len(planQueue) > 0; planQueue = planQueue[1:] { curPlan := planQueue[0] switch copPlan := curPlan.(type) { case *PhysicalTableReader: @@ -1383,7 +1384,7 @@ func (e *Explain) prepareTaskDot(p PhysicalPlan, taskTp string, buffer *bytes.Bu // 1. ctx is auto commit tagged // 2. session is not InTxn // 3. plan is point get by pk, or point get by unique index (no double read) -func IsPointGetWithPKOrUniqueKeyByAutoCommit(vars *variable.SessionVars, p Plan) (bool, error) { +func IsPointGetWithPKOrUniqueKeyByAutoCommit(vars *variable.SessionVars, p base.Plan) (bool, error) { if !IsAutoCommitTxn(vars) { return false, nil } diff --git a/pkg/planner/core/debugtrace.go b/pkg/planner/core/debugtrace.go index fe7b2e195c2e8..19843e1824f3f 100644 --- a/pkg/planner/core/debugtrace.go +++ b/pkg/planner/core/debugtrace.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/context" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/planner/util/debugtrace" "github.com/pingcap/tidb/pkg/statistics" @@ -72,7 +73,7 @@ func (info *binaryParamInfo) MarshalJSON() ([]byte, error) { } // DebugTraceReceivedCommand records the received command from the client to the debug trace. -func DebugTraceReceivedCommand(s PlanContext, cmd byte, stmtNode ast.StmtNode) { +func DebugTraceReceivedCommand(s base.PlanContext, cmd byte, stmtNode ast.StmtNode) { sessionVars := s.GetSessionVars() trace := debugtrace.GetOrInitDebugTraceRoot(s) traceInfo := new(receivedCmdInfo) @@ -172,7 +173,7 @@ type getStatsTblInfo struct { } func debugTraceGetStatsTbl( - s PlanContext, + s base.PlanContext, tblInfo *model.TableInfo, pid int64, handleIsNil, @@ -249,7 +250,7 @@ func convertAccessPathForDebugTrace(path *util.AccessPath, out *accessPathForDeb } } -func debugTraceAccessPaths(s PlanContext, paths []*util.AccessPath) { +func debugTraceAccessPaths(s base.PlanContext, paths []*util.AccessPath) { root := debugtrace.GetOrInitDebugTraceRoot(s) traceInfo := make([]accessPathForDebugTrace, len(paths)) for i, partialPath := range paths { diff --git a/pkg/planner/core/encode.go b/pkg/planner/core/encode.go index b11d8fff4da11..af67053afa477 100644 --- a/pkg/planner/core/encode.go +++ b/pkg/planner/core/encode.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/util/plancodec" ) @@ -70,7 +71,7 @@ func EncodeFlatPlan(flat *FlatPhysicalPlan) string { actRows, analyzeInfo, memoryInfo, diskInfo := getRuntimeInfoStr(p.SCtx(), p, nil) var estRows float64 if fop.IsPhysicalPlan { - estRows = fop.Origin.(PhysicalPlan).GetEstRowCountForDisplay() + estRows = fop.Origin.(base.PhysicalPlan).GetEstRowCountForDisplay() } else if statsInfo := p.StatsInfo(); statsInfo != nil { estRows = statsInfo.RowCount } @@ -102,7 +103,7 @@ func encodeFlatPlanTree(flatTree FlatPlanTree, offset int, buf *bytes.Buffer) { actRows, analyzeInfo, memoryInfo, diskInfo := getRuntimeInfoStr(p.SCtx(), p, nil) var estRows float64 if fop.IsPhysicalPlan { - estRows = fop.Origin.(PhysicalPlan).GetEstRowCountForDisplay() + estRows = fop.Origin.(base.PhysicalPlan).GetEstRowCountForDisplay() } else if statsInfo := p.StatsInfo(); statsInfo != nil { estRows = statsInfo.RowCount } @@ -152,7 +153,7 @@ type planEncoder struct { // EncodePlan is used to encodePlan the plan to the plan tree with compressing. // Deprecated: FlattenPhysicalPlan() + EncodeFlatPlan() is preferred. -func EncodePlan(p Plan) string { +func EncodePlan(p base.Plan) string { if explain, ok := p.(*Explain); ok { p = explain.TargetPlan } @@ -170,7 +171,7 @@ func EncodePlan(p Plan) string { return pn.encodePlanTree(p) } -func (pn *planEncoder) encodePlanTree(p Plan) string { +func (pn *planEncoder) encodePlanTree(p base.Plan) string { pn.encodedPlans = make(map[int]bool) pn.buf.Reset() pn.ctes = pn.ctes[:0] @@ -205,11 +206,11 @@ func (pn *planEncoder) encodeCTEPlan() { } } -func (pn *planEncoder) encodePlan(p Plan, isRoot bool, store kv.StoreType, depth int) { +func (pn *planEncoder) encodePlan(p base.Plan, isRoot bool, store kv.StoreType, depth int) { taskTypeInfo := plancodec.EncodeTaskType(isRoot, store) actRows, analyzeInfo, memoryInfo, diskInfo := getRuntimeInfoStr(p.SCtx(), p, nil) rowCount := 0.0 - if pp, ok := p.(PhysicalPlan); ok { + if pp, ok := p.(base.PhysicalPlan); ok { rowCount = pp.GetEstRowCountForDisplay() } else if statsInfo := p.StatsInfo(); statsInfo != nil { rowCount = statsInfo.RowCount @@ -285,7 +286,7 @@ func NormalizeFlatPlan(flat *FlatPhysicalPlan) (normalized string, digest *parse d.buf.Grow(30 * len(selectPlan)) for _, fop := range selectPlan { taskTypeInfo := plancodec.EncodeTaskTypeForNormalize(fop.IsRoot, fop.StoreType) - p := fop.Origin.(PhysicalPlan) + p := fop.Origin.(base.PhysicalPlan) plancodec.NormalizePlanNode( int(fop.Depth-uint32(selectPlanOffset)), fop.Origin.TP(), @@ -308,7 +309,7 @@ func NormalizeFlatPlan(flat *FlatPhysicalPlan) (normalized string, digest *parse // NormalizePlan is used to normalize the plan and generate plan digest. // Deprecated: FlattenPhysicalPlan() + NormalizeFlatPlan() is preferred. -func NormalizePlan(p Plan) (normalized string, digest *parser.Digest) { +func NormalizePlan(p base.Plan) (normalized string, digest *parser.Digest) { selectPlan := getSelectPlan(p) if selectPlan == nil { return "", parser.NewDigest(nil) @@ -329,13 +330,13 @@ func NormalizePlan(p Plan) (normalized string, digest *parser.Digest) { return } -func (d *planDigester) normalizePlanTree(p PhysicalPlan) { +func (d *planDigester) normalizePlanTree(p base.PhysicalPlan) { d.encodedPlans = make(map[int]bool) d.buf.Reset() d.normalizePlan(p, true, kv.TiKV, 0) } -func (d *planDigester) normalizePlan(p PhysicalPlan, isRoot bool, store kv.StoreType, depth int) { +func (d *planDigester) normalizePlan(p base.PhysicalPlan, isRoot bool, store kv.StoreType, depth int) { taskTypeInfo := plancodec.EncodeTaskTypeForNormalize(isRoot, store) plancodec.NormalizePlanNode(depth, p.TP(), taskTypeInfo, p.ExplainNormalizedInfo(), &d.buf) d.encodedPlans[p.ID()] = true @@ -365,9 +366,9 @@ func (d *planDigester) normalizePlan(p PhysicalPlan, isRoot bool, store kv.Store } } -func getSelectPlan(p Plan) PhysicalPlan { - var selectPlan PhysicalPlan - if physicalPlan, ok := p.(PhysicalPlan); ok { +func getSelectPlan(p base.Plan) base.PhysicalPlan { + var selectPlan base.PhysicalPlan + if physicalPlan, ok := p.(base.PhysicalPlan); ok { selectPlan = physicalPlan } else { switch x := p.(type) { diff --git a/pkg/planner/core/exhaust_physical_plans.go b/pkg/planner/core/exhaust_physical_plans.go index c93c9ae618d35..fa77ed8d8a414 100644 --- a/pkg/planner/core/exhaust_physical_plans.go +++ b/pkg/planner/core/exhaust_physical_plans.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/cardinality" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/planner/util/fixcontrol" @@ -49,7 +50,7 @@ import ( "go.uber.org/zap" ) -func (p *LogicalUnionScan) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { +func (p *LogicalUnionScan) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) { if prop.IsFlashProp() { p.SCtx().GetSessionVars().RaiseWarningWhenMPPEnforced( "MPP mode may be blocked because operator `UnionScan` is not supported now.") @@ -60,7 +61,7 @@ func (p *LogicalUnionScan) exhaustPhysicalPlans(prop *property.PhysicalProperty) Conditions: p.conditions, HandleCols: p.handleCols, }.Init(p.SCtx(), p.StatsInfo(), p.QueryBlockOffset(), childProp) - return []PhysicalPlan{us}, true, nil + return []base.PhysicalPlan{us}, true, nil } func getMaxSortPrefix(sortCols, allCols []*expression.Column) []int { @@ -156,8 +157,8 @@ func (*LogicalJoin) checkJoinKeyCollation(leftKeys, rightKeys []*expression.Colu } // GetMergeJoin convert the logical join to physical merge join based on the physical property. -func (p *LogicalJoin) GetMergeJoin(prop *property.PhysicalProperty, schema *expression.Schema, statsInfo *property.StatsInfo, leftStatsInfo *property.StatsInfo, rightStatsInfo *property.StatsInfo) []PhysicalPlan { - joins := make([]PhysicalPlan, 0, len(p.leftProperties)+1) +func (p *LogicalJoin) GetMergeJoin(prop *property.PhysicalProperty, schema *expression.Schema, statsInfo *property.StatsInfo, leftStatsInfo *property.StatsInfo, rightStatsInfo *property.StatsInfo) []base.PhysicalPlan { + joins := make([]base.PhysicalPlan, 0, len(p.leftProperties)+1) // The leftProperties caches all the possible properties that are provided by its children. leftJoinKeys, rightJoinKeys, isNullEQ, hasNullEQ := p.GetJoinKeys() @@ -292,7 +293,7 @@ func getNewNullEQByOffsets(oldNullEQ []bool, offsets []int) []bool { return newNullEQ } -func (p *LogicalJoin) getEnforcedMergeJoin(prop *property.PhysicalProperty, schema *expression.Schema, statsInfo *property.StatsInfo) []PhysicalPlan { +func (p *LogicalJoin) getEnforcedMergeJoin(prop *property.PhysicalProperty, schema *expression.Schema, statsInfo *property.StatsInfo) []base.PhysicalPlan { // Check whether SMJ can satisfy the required property leftJoinKeys, rightJoinKeys, isNullEQ, hasNullEQ := p.GetJoinKeys() // TODO: support null equal join keys for merge join @@ -373,7 +374,7 @@ func (p *LogicalJoin) getEnforcedMergeJoin(prop *property.PhysicalProperty, sche enforcedPhysicalMergeJoin.SetSchema(schema) enforcedPhysicalMergeJoin.childrenReqProps = []*property.PhysicalProperty{lProp, rProp} enforcedPhysicalMergeJoin.initCompareFuncs() - return []PhysicalPlan{enforcedPhysicalMergeJoin} + return []base.PhysicalPlan{enforcedPhysicalMergeJoin} } func (p *PhysicalMergeJoin) initCompareFuncs() { @@ -387,7 +388,7 @@ func (p *LogicalJoin) shouldSkipHashJoin() bool { return (p.preferJoinType&h.PreferNoHashJoin) > 0 || (p.SCtx().GetSessionVars().DisableHashJoin) } -func (p *LogicalJoin) getHashJoins(prop *property.PhysicalProperty) (joins []PhysicalPlan, forced bool) { +func (p *LogicalJoin) getHashJoins(prop *property.PhysicalProperty) (joins []base.PhysicalPlan, forced bool) { if !prop.IsSortItemEmpty() { // hash join doesn't promise any orders return } @@ -400,7 +401,7 @@ func (p *LogicalJoin) getHashJoins(prop *property.PhysicalProperty) (joins []Phy forceRightToBuild = false } - joins = make([]PhysicalPlan, 0, 2) + joins = make([]base.PhysicalPlan, 0, 2) switch p.JoinType { case SemiJoin, AntiSemiJoin, LeftOuterSemiJoin, AntiLeftOuterSemiJoin: joins = append(joins, p.getHashJoin(prop, 1, false)) @@ -464,13 +465,13 @@ func (p *LogicalJoin) getHashJoin(prop *property.PhysicalProperty, innerIdx int, func (p *LogicalJoin) constructIndexJoin( prop *property.PhysicalProperty, outerIdx int, - innerTask Task, + innerTask base.Task, ranges ranger.MutableRanges, keyOff2IdxOff []int, path *util.AccessPath, compareFilters *ColWithCmpFuncManager, extractOtherEQ bool, -) []PhysicalPlan { +) []base.PhysicalPlan { if ranges == nil { ranges = ranger.Ranges{} // empty range } @@ -572,24 +573,24 @@ func (p *LogicalJoin) constructIndexJoin( join.IdxColLens = path.IdxColLens } join.SetSchema(p.schema) - return []PhysicalPlan{join} + return []base.PhysicalPlan{join} } func (p *LogicalJoin) constructIndexMergeJoin( prop *property.PhysicalProperty, outerIdx int, - innerTask Task, + innerTask base.Task, ranges ranger.MutableRanges, keyOff2IdxOff []int, path *util.AccessPath, compareFilters *ColWithCmpFuncManager, -) []PhysicalPlan { +) []base.PhysicalPlan { hintExists := false if (outerIdx == 1 && (p.preferJoinType&h.PreferLeftAsINLMJInner) > 0) || (outerIdx == 0 && (p.preferJoinType&h.PreferRightAsINLMJInner) > 0) { hintExists = true } indexJoins := p.constructIndexJoin(prop, outerIdx, innerTask, ranges, keyOff2IdxOff, path, compareFilters, !hintExists) - indexMergeJoins := make([]PhysicalPlan, 0, len(indexJoins)) + indexMergeJoins := make([]base.PhysicalPlan, 0, len(indexJoins)) for _, plan := range indexJoins { join := plan.(*PhysicalIndexJoin) // Index merge join can't handle hash keys. So we ban it heuristically. @@ -685,14 +686,14 @@ func (p *LogicalJoin) constructIndexMergeJoin( func (p *LogicalJoin) constructIndexHashJoin( prop *property.PhysicalProperty, outerIdx int, - innerTask Task, + innerTask base.Task, ranges ranger.MutableRanges, keyOff2IdxOff []int, path *util.AccessPath, compareFilters *ColWithCmpFuncManager, -) []PhysicalPlan { +) []base.PhysicalPlan { indexJoins := p.constructIndexJoin(prop, outerIdx, innerTask, ranges, keyOff2IdxOff, path, compareFilters, true) - indexHashJoins := make([]PhysicalPlan, 0, len(indexJoins)) + indexHashJoins := make([]base.PhysicalPlan, 0, len(indexJoins)) for _, plan := range indexJoins { join := plan.(*PhysicalIndexJoin) indexHashJoin := PhysicalIndexHashJoin{ @@ -710,7 +711,7 @@ func (p *LogicalJoin) constructIndexHashJoin( // First of all, we'll check whether the inner child is DataSource. // Then, we will extract the join keys of p's equal conditions. Then check whether all of them are just the primary key // or match some part of on index. If so we will choose the best one and construct a index join. -func (p *LogicalJoin) getIndexJoinByOuterIdx(prop *property.PhysicalProperty, outerIdx int) (joins []PhysicalPlan) { +func (p *LogicalJoin) getIndexJoinByOuterIdx(prop *property.PhysicalProperty, outerIdx int) (joins []base.PhysicalPlan) { outerChild, innerChild := p.children[outerIdx], p.children[1-outerIdx] all, _ := prop.AllSameOrder() // If the order by columns are not all from outer child, index join cannot promise the order. @@ -818,7 +819,7 @@ func (p *LogicalJoin) getIndexJoinBuildHelper(ds *DataSource, innerJoinKeys []*e // promised to be no worse than building IndexScan as the inner child. func (p *LogicalJoin) buildIndexJoinInner2TableScan( prop *property.PhysicalProperty, wrapper *indexJoinInnerChildWrapper, innerJoinKeys, outerJoinKeys []*expression.Column, - outerIdx int, avgInnerRowCnt float64) (joins []PhysicalPlan) { + outerIdx int, avgInnerRowCnt float64) (joins []base.PhysicalPlan) { ds := wrapper.ds var tblPath *util.AccessPath for _, path := range ds.possibleAccessPaths { @@ -833,7 +834,7 @@ func (p *LogicalJoin) buildIndexJoinInner2TableScan( keyOff2IdxOff := make([]int, len(innerJoinKeys)) newOuterJoinKeys := make([]*expression.Column, 0) var ranges ranger.MutableRanges = ranger.Ranges{} - var innerTask, innerTask2 Task + var innerTask, innerTask2 base.Task var helper *indexJoinBuildHelper if ds.tableInfo.IsCommonHandle { helper, keyOff2IdxOff = p.getIndexJoinBuildHelper(ds, innerJoinKeys, func(path *util.AccessPath) bool { return path.IsCommonHandlePath }, outerJoinKeys) @@ -898,7 +899,7 @@ func (p *LogicalJoin) buildIndexJoinInner2TableScan( path = helper.chosenPath lastColMng = helper.lastColManager } - joins = make([]PhysicalPlan, 0, 3) + joins = make([]base.PhysicalPlan, 0, 3) failpoint.Inject("MockOnlyEnableIndexHashJoin", func(val failpoint.Value) { if val.(bool) && !p.SCtx().GetSessionVars().InRestrictedSQL { failpoint.Return(p.constructIndexHashJoin(prop, outerIdx, innerTask, nil, keyOff2IdxOff, path, lastColMng)) @@ -916,7 +917,7 @@ func (p *LogicalJoin) buildIndexJoinInner2TableScan( func (p *LogicalJoin) buildIndexJoinInner2IndexScan( prop *property.PhysicalProperty, wrapper *indexJoinInnerChildWrapper, innerJoinKeys, outerJoinKeys []*expression.Column, - outerIdx int, avgInnerRowCnt float64) (joins []PhysicalPlan) { + outerIdx int, avgInnerRowCnt float64) (joins []base.PhysicalPlan) { ds := wrapper.ds indexValid := func(path *util.AccessPath) bool { if path.IsTablePath() { @@ -936,7 +937,7 @@ func (p *LogicalJoin) buildIndexJoinInner2IndexScan( if helper == nil { return nil } - joins = make([]PhysicalPlan, 0, 3) + joins = make([]base.PhysicalPlan, 0, 3) rangeInfo := helper.buildRangeDecidedByInformation(helper.chosenPath.IdxCols, outerJoinKeys) maxOneRow := false if helper.chosenPath.Index.Unique && helper.usedColsLen == len(helper.chosenPath.FullIdxCols) { @@ -1025,7 +1026,7 @@ func (p *LogicalJoin) constructInnerTableScanTask( keepOrder bool, desc bool, rowCount float64, -) Task { +) base.Task { ds := wrapper.ds // If `ds.tableInfo.GetPartitionInfo() != nil`, // it means the data source is a partition table reader. @@ -1090,13 +1091,13 @@ func (p *LogicalJoin) constructInnerTableScanTask( ts.PlanPartInfo = copTask.physPlanPartInfo selStats := ts.StatsInfo().Scale(selectivity) ts.addPushedDownSelection(copTask, selStats) - t := copTask.ConvertToRootTask(ds.SCtx()) + t := copTask.ConvertToRootTask(ds.SCtx()).(*RootTask) reader := t.GetPlan() t.SetPlan(p.constructInnerByWrapper(wrapper, reader)) return t } -func (p *LogicalJoin) constructInnerByWrapper(wrapper *indexJoinInnerChildWrapper, child PhysicalPlan) PhysicalPlan { +func (p *LogicalJoin) constructInnerByWrapper(wrapper *indexJoinInnerChildWrapper, child base.PhysicalPlan) base.PhysicalPlan { for i := len(wrapper.zippedChildren) - 1; i >= 0; i-- { switch x := wrapper.zippedChildren[i].(type) { case *LogicalUnionScan: @@ -1110,7 +1111,7 @@ func (p *LogicalJoin) constructInnerByWrapper(wrapper *indexJoinInnerChildWrappe return child } -func (*LogicalJoin) constructInnerSel(sel *LogicalSelection, child PhysicalPlan) PhysicalPlan { +func (*LogicalJoin) constructInnerSel(sel *LogicalSelection, child base.PhysicalPlan) base.PhysicalPlan { if sel == nil { return child } @@ -1121,7 +1122,7 @@ func (*LogicalJoin) constructInnerSel(sel *LogicalSelection, child PhysicalPlan) return physicalSel } -func (*LogicalJoin) constructInnerProj(proj *LogicalProjection, child PhysicalPlan) PhysicalPlan { +func (*LogicalJoin) constructInnerProj(proj *LogicalProjection, child base.PhysicalPlan) base.PhysicalPlan { if proj == nil { return child } @@ -1135,7 +1136,7 @@ func (*LogicalJoin) constructInnerProj(proj *LogicalProjection, child PhysicalPl return physicalProj } -func (*LogicalJoin) constructInnerUnionScan(us *LogicalUnionScan, reader PhysicalPlan) PhysicalPlan { +func (*LogicalJoin) constructInnerUnionScan(us *LogicalUnionScan, reader base.PhysicalPlan) base.PhysicalPlan { if us == nil { return reader } @@ -1211,7 +1212,7 @@ func (p *LogicalJoin) constructInnerIndexScanTask( desc bool, rowCount float64, maxOneRow bool, -) Task { +) base.Task { ds := wrapper.ds // If `ds.tableInfo.GetPartitionInfo() != nil`, // it means the data source is a partition table reader. @@ -1378,7 +1379,7 @@ func (p *LogicalJoin) constructInnerIndexScanTask( } finalStats := ds.tableStats.ScaleByExpectCnt(rowCount) is.addPushedDownSelection(cop, ds, tmpPath, finalStats) - t := cop.ConvertToRootTask(ds.SCtx()) + t := cop.ConvertToRootTask(ds.SCtx()).(*RootTask) reader := t.GetPlan() t.SetPlan(p.constructInnerByWrapper(wrapper, reader)) return t @@ -1934,7 +1935,7 @@ func (ijHelper *indexJoinBuildHelper) buildTemplateRange(matchedKeyCnt int, eqAn return } -func filterIndexJoinBySessionVars(sc PlanContext, indexJoins []PhysicalPlan) []PhysicalPlan { +func filterIndexJoinBySessionVars(sc base.PlanContext, indexJoins []base.PhysicalPlan) []base.PhysicalPlan { if sc.GetSessionVars().EnableIndexMergeJoin { return indexJoins } @@ -1963,7 +1964,7 @@ const ( indexMergeJoinMethod = 2 ) -func (*LogicalJoin) getIndexJoinSideAndMethod(join PhysicalPlan) (innerSide, joinMethod int, ok bool) { +func (*LogicalJoin) getIndexJoinSideAndMethod(join base.PhysicalPlan) (innerSide, joinMethod int, ok bool) { var innerIdx int switch ij := join.(type) { case *PhysicalIndexJoin: @@ -1987,7 +1988,7 @@ func (*LogicalJoin) getIndexJoinSideAndMethod(join PhysicalPlan) (innerSide, joi } // tryToGetIndexJoin returns all available index join plans, and the second returned value indicates whether this plan is enforced by hints. -func (p *LogicalJoin) tryToGetIndexJoin(prop *property.PhysicalProperty) (indexJoins []PhysicalPlan, canForced bool) { +func (p *LogicalJoin) tryToGetIndexJoin(prop *property.PhysicalProperty) (indexJoins []base.PhysicalPlan, canForced bool) { // supportLeftOuter and supportRightOuter indicates whether this type of join // supports the left side or right side to be the outer side. var supportLeftOuter, supportRightOuter bool @@ -1999,7 +2000,7 @@ func (p *LogicalJoin) tryToGetIndexJoin(prop *property.PhysicalProperty) (indexJ case InnerJoin: supportLeftOuter, supportRightOuter = true, true } - candidates := make([]PhysicalPlan, 0, 2) + candidates := make([]base.PhysicalPlan, 0, 2) if supportLeftOuter { candidates = append(candidates, p.getIndexJoinByOuterIdx(prop, 0)...) } @@ -2029,11 +2030,11 @@ func (p *LogicalJoin) tryToGetIndexJoin(prop *property.PhysicalProperty) (indexJ return filterIndexJoinBySessionVars(p.SCtx(), candidates), false } -func (p *LogicalJoin) handleFilterIndexJoinHints(candidates []PhysicalPlan) []PhysicalPlan { +func (p *LogicalJoin) handleFilterIndexJoinHints(candidates []base.PhysicalPlan) []base.PhysicalPlan { if !p.preferAny(h.PreferNoIndexJoin, h.PreferNoIndexHashJoin, h.PreferNoIndexMergeJoin) { return candidates // no filter index join hints } - filtered := make([]PhysicalPlan, 0, len(candidates)) + filtered := make([]base.PhysicalPlan, 0, len(candidates)) for _, candidate := range candidates { _, joinMethod, ok := p.getIndexJoinSideAndMethod(candidate) if !ok { @@ -2050,12 +2051,12 @@ func (p *LogicalJoin) handleFilterIndexJoinHints(candidates []PhysicalPlan) []Ph } // handleForceIndexJoinHints handles the force index join hints and returns all plans that can satisfy the hints. -func (p *LogicalJoin) handleForceIndexJoinHints(prop *property.PhysicalProperty, candidates []PhysicalPlan) (indexJoins []PhysicalPlan, canForced bool) { +func (p *LogicalJoin) handleForceIndexJoinHints(prop *property.PhysicalProperty, candidates []base.PhysicalPlan) (indexJoins []base.PhysicalPlan, canForced bool) { if !p.preferAny(h.PreferRightAsINLJInner, h.PreferRightAsINLHJInner, h.PreferRightAsINLMJInner, h.PreferLeftAsINLJInner, h.PreferLeftAsINLHJInner, h.PreferLeftAsINLMJInner) { return candidates, false // no force index join hints } - forced := make([]PhysicalPlan, 0, len(candidates)) + forced := make([]base.PhysicalPlan, 0, len(candidates)) for _, candidate := range candidates { innerSide, joinMethod, ok := p.getIndexJoinSideAndMethod(candidate) if !ok { @@ -2103,7 +2104,7 @@ func (p *LogicalJoin) handleForceIndexJoinHints(prop *property.PhysicalProperty, return candidates, false } -func checkChildFitBC(p Plan) bool { +func checkChildFitBC(p base.Plan) bool { if p.StatsInfo().HistColl == nil { return p.SCtx().GetSessionVars().BroadcastJoinThresholdCount == -1 || p.StatsInfo().Count() < p.SCtx().GetSessionVars().BroadcastJoinThresholdCount } @@ -2112,7 +2113,7 @@ func checkChildFitBC(p Plan) bool { return p.SCtx().GetSessionVars().BroadcastJoinThresholdSize == -1 || sz < float64(p.SCtx().GetSessionVars().BroadcastJoinThresholdSize) } -func calcBroadcastExchangeSize(p Plan, mppStoreCnt int) (row float64, size float64, hasSize bool) { +func calcBroadcastExchangeSize(p base.Plan, mppStoreCnt int) (row float64, size float64, hasSize bool) { s := p.StatsInfo() row = float64(s.Count()) * float64(mppStoreCnt-1) if s.HistColl == nil { @@ -2123,7 +2124,7 @@ func calcBroadcastExchangeSize(p Plan, mppStoreCnt int) (row float64, size float return row, size, true } -func calcBroadcastExchangeSizeByChild(p1 Plan, p2 Plan, mppStoreCnt int) (row float64, size float64, hasSize bool) { +func calcBroadcastExchangeSizeByChild(p1 base.Plan, p2 base.Plan, mppStoreCnt int) (row float64, size float64, hasSize bool) { row1, size1, hasSize1 := calcBroadcastExchangeSize(p1, mppStoreCnt) row2, size2, hasSize2 := calcBroadcastExchangeSize(p2, mppStoreCnt) @@ -2139,7 +2140,7 @@ func calcBroadcastExchangeSizeByChild(p1 Plan, p2 Plan, mppStoreCnt int) (row fl return math.Min(row1, row2), 0, false } -func calcHashExchangeSize(p Plan, mppStoreCnt int) (row float64, sz float64, hasSize bool) { +func calcHashExchangeSize(p base.Plan, mppStoreCnt int) (row float64, sz float64, hasSize bool) { s := p.StatsInfo() row = float64(s.Count()) * float64(mppStoreCnt-1) / float64(mppStoreCnt) if s.HistColl == nil { @@ -2150,7 +2151,7 @@ func calcHashExchangeSize(p Plan, mppStoreCnt int) (row float64, sz float64, has return row, sz, true } -func calcHashExchangeSizeByChild(p1 Plan, p2 Plan, mppStoreCnt int) (row float64, size float64, hasSize bool) { +func calcHashExchangeSizeByChild(p1 base.Plan, p2 base.Plan, mppStoreCnt int) (row float64, size float64, hasSize bool) { row1, size1, hasSize1 := calcHashExchangeSize(p1, mppStoreCnt) row2, size2, hasSize2 := calcHashExchangeSize(p2, mppStoreCnt) @@ -2231,7 +2232,7 @@ func (p *LogicalJoin) preferMppBCJ() bool { // Firstly we check the hint, if hint is figured by user, we force to choose the corresponding physical plan. // If the hint is not matched, it will get other candidates. // If the hint is not figured, we will pick all candidates. -func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { +func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) { failpoint.Inject("MockOnlyEnableIndexHashJoin", func(val failpoint.Value) { if val.(bool) && !p.SCtx().GetSessionVars().InRestrictedSQL { indexJoins, _ := p.tryToGetIndexJoin(prop) @@ -2255,7 +2256,7 @@ func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]P if prop.MPPPartitionTp == property.BroadcastType { return nil, false, nil } - joins := make([]PhysicalPlan, 0, 8) + joins := make([]base.PhysicalPlan, 0, 8) canPushToTiFlash := p.canPushToCop(kv.TiFlash) if p.SCtx().GetSessionVars().IsMPPAllowed() && canPushToTiFlash { if (p.preferJoinType & h.PreferShuffleJoin) > 0 { @@ -2348,7 +2349,7 @@ func canExprsInJoinPushdown(p *LogicalJoin, storeType kv.StoreType) bool { return true } -func (p *LogicalJoin) tryToGetMppHashJoin(prop *property.PhysicalProperty, useBCJ bool) []PhysicalPlan { +func (p *LogicalJoin) tryToGetMppHashJoin(prop *property.PhysicalProperty, useBCJ bool) []base.PhysicalPlan { if !prop.IsSortItemEmpty() { return nil } @@ -2537,7 +2538,7 @@ func (p *LogicalJoin) tryToGetMppHashJoin(prop *property.PhysicalProperty, useBC // Mpp Join has quite heavy cost. Even limit might not suspend it in time, so we don't scale the count. }.Init(p.SCtx(), p.StatsInfo(), p.QueryBlockOffset(), childrenProps...) join.SetSchema(p.schema) - return []PhysicalPlan{join} + return []base.PhysicalPlan{join} } func choosePartitionKeys(keys []*property.MPPPartitionColumn, matches []int) []*property.MPPPartitionColumn { @@ -2568,7 +2569,7 @@ func (p *LogicalProjection) TryToGetChildProp(prop *property.PhysicalProperty) ( } // exhaustPhysicalPlans enumerate all the possible physical plan for expand operator (currently only mpp case is supported) -func (p *LogicalExpand) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { +func (p *LogicalExpand) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) { // under the mpp task type, if the sort item is not empty, refuse it, cause expanded data doesn't support any sort items. if !prop.IsSortItemEmpty() { // false, meaning we can add a sort enforcer. @@ -2594,13 +2595,13 @@ func (p *LogicalExpand) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([ ExtraGroupingColNames: p.ExtraGroupingColNames, }.Init(p.SCtx(), p.StatsInfo().ScaleByExpectCnt(prop.ExpectedCnt), p.QueryBlockOffset(), mppProp) expand.SetSchema(p.Schema()) - return []PhysicalPlan{expand}, true, nil + return []base.PhysicalPlan{expand}, true, nil } // if MPP switch is shutdown, nothing can be generated. return nil, true, nil } -func (p *LogicalProjection) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { +func (p *LogicalProjection) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) { newProp, ok := p.TryToGetChildProp(prop) if !ok { return nil, true, nil @@ -2622,7 +2623,7 @@ func (p *LogicalProjection) exhaustPhysicalPlans(prop *property.PhysicalProperty newProps = append(newProps, copProp) } - ret := make([]PhysicalPlan, 0, len(newProps)) + ret := make([]base.PhysicalPlan, 0, len(newProps)) for _, newProp := range newProps { proj := PhysicalProjection{ Exprs: p.Exprs, @@ -2662,7 +2663,7 @@ func pushLimitOrTopNForcibly(p LogicalPlan) bool { return false } -func (lt *LogicalTopN) getPhysTopN(prop *property.PhysicalProperty) []PhysicalPlan { +func (lt *LogicalTopN) getPhysTopN(prop *property.PhysicalProperty) []base.PhysicalPlan { allTaskTypes := []property.TaskType{property.CopSingleReadTaskType, property.CopMultiReadTaskType} if !pushLimitOrTopNForcibly(lt) { allTaskTypes = append(allTaskTypes, property.RootTaskType) @@ -2670,7 +2671,7 @@ func (lt *LogicalTopN) getPhysTopN(prop *property.PhysicalProperty) []PhysicalPl if lt.SCtx().GetSessionVars().IsMPPAllowed() { allTaskTypes = append(allTaskTypes, property.MppTaskType) } - ret := make([]PhysicalPlan, 0, len(allTaskTypes)) + ret := make([]base.PhysicalPlan, 0, len(allTaskTypes)) for _, tp := range allTaskTypes { resultProp := &property.PhysicalProperty{TaskTp: tp, ExpectedCnt: math.MaxFloat64, CTEProducerStatus: prop.CTEProducerStatus} topN := PhysicalTopN{ @@ -2684,7 +2685,7 @@ func (lt *LogicalTopN) getPhysTopN(prop *property.PhysicalProperty) []PhysicalPl return ret } -func (lt *LogicalTopN) getPhysLimits(prop *property.PhysicalProperty) []PhysicalPlan { +func (lt *LogicalTopN) getPhysLimits(prop *property.PhysicalProperty) []base.PhysicalPlan { p, canPass := GetPropByOrderByItems(lt.ByItems) if !canPass { return nil @@ -2694,7 +2695,7 @@ func (lt *LogicalTopN) getPhysLimits(prop *property.PhysicalProperty) []Physical if !pushLimitOrTopNForcibly(lt) { allTaskTypes = append(allTaskTypes, property.RootTaskType) } - ret := make([]PhysicalPlan, 0, len(allTaskTypes)) + ret := make([]base.PhysicalPlan, 0, len(allTaskTypes)) for _, tp := range allTaskTypes { resultProp := &property.PhysicalProperty{TaskTp: tp, ExpectedCnt: float64(lt.Count + lt.Offset), SortItems: p.SortItems, CTEProducerStatus: prop.CTEProducerStatus} limit := PhysicalLimit{ @@ -2722,7 +2723,7 @@ func MatchItems(p *property.PhysicalProperty, items []*util.ByItems) bool { return true } -func (lt *LogicalTopN) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { +func (lt *LogicalTopN) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) { if MatchItems(prop, lt.ByItems) { return append(lt.getPhysTopN(prop), lt.getPhysLimits(prop)...), true, nil } @@ -2734,7 +2735,7 @@ func (la *LogicalApply) GetHashJoin(prop *property.PhysicalProperty) *PhysicalHa return la.LogicalJoin.getHashJoin(prop, 1, false) } -func (la *LogicalApply) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { +func (la *LogicalApply) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) { if !prop.AllColsFromSchema(la.children[0].Schema()) || prop.IsFlashProp() { // for convenient, we don't pass through any prop la.SCtx().GetSessionVars().RaiseWarningWhenMPPEnforced( "MPP mode may be blocked because operator `Apply` is not supported now.") @@ -2775,7 +2776,7 @@ func (la *LogicalApply) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([ &property.PhysicalProperty{ExpectedCnt: math.MaxFloat64, SortItems: prop.SortItems, CTEProducerStatus: prop.CTEProducerStatus}, &property.PhysicalProperty{ExpectedCnt: math.MaxFloat64, CTEProducerStatus: prop.CTEProducerStatus}) apply.SetSchema(la.schema) - return []PhysicalPlan{apply}, true, nil + return []base.PhysicalPlan{apply}, true, nil } func disableAggPushDownToCop(p LogicalPlan) { @@ -2815,7 +2816,7 @@ func (lw *LogicalWindow) checkComparisonForTiFlash(frameBound *FrameBound) bool return true } -func (lw *LogicalWindow) tryToGetMppWindows(prop *property.PhysicalProperty) []PhysicalPlan { +func (lw *LogicalWindow) tryToGetMppWindows(prop *property.PhysicalProperty) []base.PhysicalPlan { if !prop.IsSortItemAllForPartition() { return nil } @@ -2909,11 +2910,11 @@ func (lw *LogicalWindow) tryToGetMppWindows(prop *property.PhysicalProperty) []P }.Init(lw.SCtx(), lw.StatsInfo().ScaleByExpectCnt(prop.ExpectedCnt), lw.QueryBlockOffset(), childProperty) window.SetSchema(lw.Schema()) - return []PhysicalPlan{window} + return []base.PhysicalPlan{window} } -func (lw *LogicalWindow) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { - windows := make([]PhysicalPlan, 0, 2) +func (lw *LogicalWindow) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) { + windows := make([]base.PhysicalPlan, 0, 2) canPushToTiFlash := lw.canPushToCop(kv.TiFlash) if lw.SCtx().GetSessionVars().IsMPPAllowed() && canPushToTiFlash { @@ -2945,7 +2946,7 @@ func (lw *LogicalWindow) exhaustPhysicalPlans(prop *property.PhysicalProperty) ( } // exhaustPhysicalPlans is only for implementing interface. DataSource and Dual generate task in `findBestTask` directly. -func (*baseLogicalPlan) exhaustPhysicalPlans(*property.PhysicalProperty) ([]PhysicalPlan, bool, error) { +func (*baseLogicalPlan) exhaustPhysicalPlans(*property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) { panic("baseLogicalPlan.exhaustPhysicalPlans() should never be called.") } @@ -3039,13 +3040,13 @@ func (la *LogicalAggregation) canPushToCop(storeTp kv.StoreType) bool { return la.baseLogicalPlan.canPushToCop(storeTp) && !la.noCopPushDown } -func (la *LogicalAggregation) getEnforcedStreamAggs(prop *property.PhysicalProperty) []PhysicalPlan { +func (la *LogicalAggregation) getEnforcedStreamAggs(prop *property.PhysicalProperty) []base.PhysicalPlan { if prop.IsFlashProp() { return nil } _, desc := prop.AllSameOrder() allTaskTypes := prop.GetAllPossibleChildTaskTypes() - enforcedAggs := make([]PhysicalPlan, 0, len(allTaskTypes)) + enforcedAggs := make([]base.PhysicalPlan, 0, len(allTaskTypes)) childProp := &property.PhysicalProperty{ ExpectedCnt: math.Max(prop.ExpectedCnt*la.inputCount/la.StatsInfo().RowCount, prop.ExpectedCnt), CanAddEnforcer: true, @@ -3097,7 +3098,7 @@ func (la *LogicalAggregation) distinctArgsMeetsProperty() bool { return true } -func (la *LogicalAggregation) getStreamAggs(prop *property.PhysicalProperty) []PhysicalPlan { +func (la *LogicalAggregation) getStreamAggs(prop *property.PhysicalProperty) []base.PhysicalPlan { // TODO: support CopTiFlash task type in stream agg if prop.IsFlashProp() { return nil @@ -3119,7 +3120,7 @@ func (la *LogicalAggregation) getStreamAggs(prop *property.PhysicalProperty) []P } allTaskTypes := prop.GetAllPossibleChildTaskTypes() - streamAggs := make([]PhysicalPlan, 0, len(la.possibleProperties)*(len(allTaskTypes)-1)+len(allTaskTypes)) + streamAggs := make([]base.PhysicalPlan, 0, len(la.possibleProperties)*(len(allTaskTypes)-1)+len(allTaskTypes)) childProp := &property.PhysicalProperty{ ExpectedCnt: math.Max(prop.ExpectedCnt*la.inputCount/la.StatsInfo().RowCount, prop.ExpectedCnt), } @@ -3201,7 +3202,7 @@ func (la *LogicalAggregation) checkCanPushDownToMPP() bool { return CheckAggCanPushCop(la.SCtx(), la.AggFuncs, la.GroupByItems, kv.TiFlash) } -func (la *LogicalAggregation) tryToGetMppHashAggs(prop *property.PhysicalProperty) (hashAggs []PhysicalPlan) { +func (la *LogicalAggregation) tryToGetMppHashAggs(prop *property.PhysicalProperty) (hashAggs []base.PhysicalPlan) { if !prop.IsSortItemEmpty() { return nil } @@ -3308,7 +3309,7 @@ func (la *LogicalAggregation) tryToGetMppHashAggs(prop *property.PhysicalPropert preferMode, prefer = Mpp2Phase, true } if prefer { - var preferPlans []PhysicalPlan + var preferPlans []base.PhysicalPlan for _, agg := range hashAggs { if hg, ok := agg.(*PhysicalHashAgg); ok && hg.MppRunMode == preferMode { preferPlans = append(preferPlans, hg) @@ -3328,14 +3329,14 @@ func (la *LogicalAggregation) tryToGetMppHashAggs(prop *property.PhysicalPropert // for 2, the final result for this physical operator enumeration is chosen or rejected is according to more factors later (hint/variable/partition/virtual-col/cost) // // That is to say, the non-complete positive judgement of canPushDownToMPP/canPushDownToTiFlash/canPushDownToTiKV is not that for sure here. -func (la *LogicalAggregation) getHashAggs(prop *property.PhysicalProperty) []PhysicalPlan { +func (la *LogicalAggregation) getHashAggs(prop *property.PhysicalProperty) []base.PhysicalPlan { if !prop.IsSortItemEmpty() { return nil } if prop.TaskTp == property.MppTaskType && !la.checkCanPushDownToMPP() { return nil } - hashAggs := make([]PhysicalPlan, 0, len(prop.GetAllPossibleChildTaskTypes())) + hashAggs := make([]base.PhysicalPlan, 0, len(prop.GetAllPossibleChildTaskTypes())) taskTypes := []property.TaskType{property.CopSingleReadTaskType, property.CopMultiReadTaskType} canPushDownToTiFlash := la.canPushToCop(kv.TiFlash) canPushDownToMPP := canPushDownToTiFlash && la.SCtx().GetSessionVars().IsMPPAllowed() && la.checkCanPushDownToMPP() @@ -3401,7 +3402,7 @@ func (la *LogicalAggregation) ResetHintIfConflicted() (preferHash bool, preferSt return } -func (la *LogicalAggregation) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { +func (la *LogicalAggregation) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) { if la.PreferAggToCop { if !la.canPushToCop(kv.TiKV) { la.SCtx().GetSessionVars().StmtCtx.SetHintWarning( @@ -3431,7 +3432,7 @@ func (la *LogicalAggregation) exhaustPhysicalPlans(prop *property.PhysicalProper return aggs, !(preferStream || preferHash), nil } -func (p *LogicalSelection) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { +func (p *LogicalSelection) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) { newProps := make([]*property.PhysicalProperty, 0, 2) childProp := prop.CloneEssentialFields() newProps = append(newProps, childProp) @@ -3444,7 +3445,7 @@ func (p *LogicalSelection) exhaustPhysicalPlans(prop *property.PhysicalProperty) newProps = append(newProps, childPropMpp) } - ret := make([]PhysicalPlan, 0, len(newProps)) + ret := make([]base.PhysicalPlan, 0, len(newProps)) for _, newProp := range newProps { sel := PhysicalSelection{ Conditions: p.Conditions, @@ -3461,7 +3462,7 @@ func (p *LogicalSelection) canPushDown(storeTp kv.StoreType) bool { expression.CanExprsPushDown(GetPushDownCtx(p.SCtx()), p.Conditions, storeTp) } -func (p *LogicalLimit) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { +func (p *LogicalLimit) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) { if !prop.IsSortItemEmpty() { return nil, true, nil } @@ -3473,7 +3474,7 @@ func (p *LogicalLimit) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([] if p.canPushToCop(kv.TiFlash) && p.SCtx().GetSessionVars().IsMPPAllowed() { allTaskTypes = append(allTaskTypes, property.MppTaskType) } - ret := make([]PhysicalPlan, 0, len(allTaskTypes)) + ret := make([]base.PhysicalPlan, 0, len(allTaskTypes)) for _, tp := range allTaskTypes { resultProp := &property.PhysicalProperty{TaskTp: tp, ExpectedCnt: float64(p.Count + p.Offset), CTEProducerStatus: prop.CTEProducerStatus} limit := PhysicalLimit{ @@ -3487,7 +3488,7 @@ func (p *LogicalLimit) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([] return ret, true, nil } -func (p *LogicalLock) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { +func (p *LogicalLock) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) { if prop.IsFlashProp() { p.SCtx().GetSessionVars().RaiseWarningWhenMPPEnforced( "MPP mode may be blocked because operator `Lock` is not supported now.") @@ -3499,10 +3500,10 @@ func (p *LogicalLock) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]P TblID2Handle: p.tblID2Handle, TblID2PhysTblIDCol: p.tblID2PhysTblIDCol, }.Init(p.SCtx(), p.StatsInfo().ScaleByExpectCnt(prop.ExpectedCnt), childProp) - return []PhysicalPlan{lock}, true, nil + return []base.PhysicalPlan{lock}, true, nil } -func (p *LogicalUnionAll) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { +func (p *LogicalUnionAll) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) { // TODO: UnionAll can not pass any order, but we can change it to sort merge to keep order. if !prop.IsSortItemEmpty() || (prop.IsFlashProp() && prop.TaskTp != property.MppTaskType) { return nil, true, nil @@ -3541,12 +3542,12 @@ func (p *LogicalUnionAll) exhaustPhysicalPlans(prop *property.PhysicalProperty) } mppUA := PhysicalUnionAll{mpp: true}.Init(p.SCtx(), p.StatsInfo().ScaleByExpectCnt(prop.ExpectedCnt), p.QueryBlockOffset(), chReqProps...) mppUA.SetSchema(p.Schema()) - return []PhysicalPlan{ua, mppUA}, true, nil + return []base.PhysicalPlan{ua, mppUA}, true, nil } - return []PhysicalPlan{ua}, true, nil + return []base.PhysicalPlan{ua}, true, nil } -func (p *LogicalPartitionUnionAll) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { +func (p *LogicalPartitionUnionAll) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) { uas, flagHint, err := p.LogicalUnionAll.exhaustPhysicalPlans(prop) if err != nil { return nil, false, err @@ -3574,10 +3575,10 @@ func (ls *LogicalSort) getNominalSort(reqProp *property.PhysicalProperty) *Nomin return ps } -func (ls *LogicalSort) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { +func (ls *LogicalSort) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) { if prop.TaskTp == property.RootTaskType { if MatchItems(prop, ls.ByItems) { - ret := make([]PhysicalPlan, 0, 2) + ret := make([]base.PhysicalPlan, 0, 2) ret = append(ret, ls.getPhysicalSort(prop)) ns := ls.getNominalSort(prop) if ns != nil { @@ -3591,22 +3592,22 @@ func (ls *LogicalSort) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([] newProp.RejectSort = true ps := NominalSort{OnlyColumn: true, ByItems: ls.ByItems}.Init( ls.SCtx(), ls.StatsInfo().ScaleByExpectCnt(prop.ExpectedCnt), ls.QueryBlockOffset(), newProp) - return []PhysicalPlan{ps}, true, nil + return []base.PhysicalPlan{ps}, true, nil } } return nil, true, nil } -func (p *LogicalMaxOneRow) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { +func (p *LogicalMaxOneRow) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) { if !prop.IsSortItemEmpty() || prop.IsFlashProp() { p.SCtx().GetSessionVars().RaiseWarningWhenMPPEnforced("MPP mode may be blocked because operator `MaxOneRow` is not supported now.") return nil, true, nil } mor := PhysicalMaxOneRow{}.Init(p.SCtx(), p.StatsInfo(), p.QueryBlockOffset(), &property.PhysicalProperty{ExpectedCnt: 2, CTEProducerStatus: prop.CTEProducerStatus}) - return []PhysicalPlan{mor}, true, nil + return []base.PhysicalPlan{mor}, true, nil } -func (p *LogicalCTE) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { +func (p *LogicalCTE) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) { pcte := PhysicalCTE{CTE: p.cte}.Init(p.SCtx(), p.StatsInfo()) if prop.IsFlashProp() { pcte.storageSender = PhysicalExchangeSender{ @@ -3615,10 +3616,10 @@ func (p *LogicalCTE) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]Ph } pcte.SetSchema(p.schema) pcte.childrenReqProps = []*property.PhysicalProperty{prop.CloneEssentialFields()} - return []PhysicalPlan{(*PhysicalCTEStorage)(pcte)}, true, nil + return []base.PhysicalPlan{(*PhysicalCTEStorage)(pcte)}, true, nil } -func (p *LogicalSequence) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { +func (p *LogicalSequence) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) { possibleChildrenProps := make([][]*property.PhysicalProperty, 0, 2) anyType := &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.AnyType, CanAddEnforcer: true, RejectSort: true, CTEProducerStatus: prop.CTEProducerStatus} if prop.TaskTp == property.MppTaskType { @@ -3637,7 +3638,7 @@ func (p *LogicalSequence) exhaustPhysicalPlans(prop *property.PhysicalProperty) p.SCtx().GetSessionVars().IsMPPAllowed() && prop.IsSortItemEmpty() { possibleChildrenProps = append(possibleChildrenProps, []*property.PhysicalProperty{anyType, anyType.CloneEssentialFields()}) } - seqs := make([]PhysicalPlan, 0, 2) + seqs := make([]base.PhysicalPlan, 0, 2) for _, propChoice := range possibleChildrenProps { childReqs := make([]*property.PhysicalProperty, 0, len(p.children)) for i := 0; i < len(p.children)-1; i++ { diff --git a/pkg/planner/core/exhaust_physical_plans_test.go b/pkg/planner/core/exhaust_physical_plans_test.go index 006da7e8dc7bb..6b281d4a9f533 100644 --- a/pkg/planner/core/exhaust_physical_plans_test.go +++ b/pkg/planner/core/exhaust_physical_plans_test.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/charset" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/statistics" @@ -339,7 +340,7 @@ func TestIndexJoinAnalyzeLookUpFilters(t *testing.T) { } } -func checkRangeFallbackAndReset(t *testing.T, ctx PlanContext, expectedRangeFallback bool) { +func checkRangeFallbackAndReset(t *testing.T, ctx base.PlanContext, expectedRangeFallback bool) { require.Equal(t, expectedRangeFallback, ctx.GetSessionVars().StmtCtx.RangeFallback) ctx.GetSessionVars().StmtCtx.RangeFallback = false } diff --git a/pkg/planner/core/expression_rewriter.go b/pkg/planner/core/expression_rewriter.go index 6b179f8788035..5f9d57bcc5dd5 100644 --- a/pkg/planner/core/expression_rewriter.go +++ b/pkg/planner/core/expression_rewriter.go @@ -32,6 +32,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/opcode" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/table/tables" @@ -49,12 +50,12 @@ import ( ) // EvalSubqueryFirstRow evaluates incorrelated subqueries once, and get first row. -var EvalSubqueryFirstRow func(ctx context.Context, p PhysicalPlan, is infoschema.InfoSchema, sctx PlanContext) (row []types.Datum, err error) +var EvalSubqueryFirstRow func(ctx context.Context, p base.PhysicalPlan, is infoschema.InfoSchema, sctx base.PlanContext) (row []types.Datum, err error) // evalAstExprWithPlanCtx evaluates ast expression with plan context. // Different with expression.EvalSimpleAst, it uses planner context and is more powerful to build some special expressions // like subquery, window function, etc. -func evalAstExprWithPlanCtx(sctx PlanContext, expr ast.ExprNode) (types.Datum, error) { +func evalAstExprWithPlanCtx(sctx base.PlanContext, expr ast.ExprNode) (types.Datum, error) { if val, ok := expr.(*driver.ValueExpr); ok { return val.Datum, nil } @@ -80,7 +81,7 @@ func evalAstExpr(ctx expression.BuildContext, expr ast.ExprNode) (types.Datum, e // rewriteAstExprWithPlanCtx rewrites ast expression directly. // Different with expression.BuildSimpleExpr, it uses planner context and is more powerful to build some special expressions // like subquery, window function, etc. -func rewriteAstExprWithPlanCtx(sctx PlanContext, expr ast.ExprNode, schema *expression.Schema, names types.NameSlice, allowCastArray bool) (expression.Expression, error) { +func rewriteAstExprWithPlanCtx(sctx base.PlanContext, expr ast.ExprNode, schema *expression.Schema, names types.NameSlice, allowCastArray bool) (expression.Expression, error) { var is infoschema.InfoSchema // in tests, it may be null if s, ok := sctx.GetInfoSchema().(infoschema.InfoSchema); ok { diff --git a/pkg/planner/core/find_best_task.go b/pkg/planner/core/find_best_task.go index dccfae4f463db..81ce43388c1d5 100644 --- a/pkg/planner/core/find_best_task.go +++ b/pkg/planner/core/find_best_task.go @@ -30,6 +30,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/cardinality" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/planner/util/coreusage" @@ -142,7 +143,7 @@ func GetPropByOrderByItemsContainScalarFunc(items []*util.ByItems) (*property.Ph return &property.PhysicalProperty{SortItems: propItems}, true, onlyColumn } -func (p *LogicalTableDual) findBestTask(prop *property.PhysicalProperty, planCounter *PlanCounterTp, opt *coreusage.PhysicalOptimizeOp) (Task, int64, error) { +func (p *LogicalTableDual) findBestTask(prop *property.PhysicalProperty, planCounter *PlanCounterTp, opt *coreusage.PhysicalOptimizeOp) (base.Task, int64, error) { // If the required property is not empty and the row count > 1, // we cannot ensure this required property. // But if the row count is 0 or 1, we don't need to care about the property. @@ -161,7 +162,7 @@ func (p *LogicalTableDual) findBestTask(prop *property.PhysicalProperty, planCou return rt, 1, nil } -func (p *LogicalShow) findBestTask(prop *property.PhysicalProperty, planCounter *PlanCounterTp, _ *coreusage.PhysicalOptimizeOp) (Task, int64, error) { +func (p *LogicalShow) findBestTask(prop *property.PhysicalProperty, planCounter *PlanCounterTp, _ *coreusage.PhysicalOptimizeOp) (base.Task, int64, error) { if !prop.IsSortItemEmpty() || planCounter.Empty() { return invalidTask, 0, nil } @@ -173,7 +174,7 @@ func (p *LogicalShow) findBestTask(prop *property.PhysicalProperty, planCounter return rt, 1, nil } -func (p *LogicalShowDDLJobs) findBestTask(prop *property.PhysicalProperty, planCounter *PlanCounterTp, _ *coreusage.PhysicalOptimizeOp) (Task, int64, error) { +func (p *LogicalShowDDLJobs) findBestTask(prop *property.PhysicalProperty, planCounter *PlanCounterTp, _ *coreusage.PhysicalOptimizeOp) (base.Task, int64, error) { if !prop.IsSortItemEmpty() || planCounter.Empty() { return invalidTask, 0, nil } @@ -186,7 +187,7 @@ func (p *LogicalShowDDLJobs) findBestTask(prop *property.PhysicalProperty, planC } // rebuildChildTasks rebuilds the childTasks to make the clock_th combination. -func (p *baseLogicalPlan) rebuildChildTasks(childTasks *[]Task, pp PhysicalPlan, childCnts []int64, planCounter int64, ts uint64, opt *coreusage.PhysicalOptimizeOp) error { +func (p *baseLogicalPlan) rebuildChildTasks(childTasks *[]base.Task, pp base.PhysicalPlan, childCnts []int64, planCounter int64, ts uint64, opt *coreusage.PhysicalOptimizeOp) error { // The taskMap of children nodes should be rolled back first. for _, child := range p.children { child.rollBackTaskMap(ts) @@ -218,16 +219,16 @@ func (p *baseLogicalPlan) rebuildChildTasks(childTasks *[]Task, pp PhysicalPlan, } func (p *baseLogicalPlan) enumeratePhysicalPlans4Task( - physicalPlans []PhysicalPlan, + physicalPlans []base.PhysicalPlan, prop *property.PhysicalProperty, addEnforcer bool, planCounter *PlanCounterTp, opt *coreusage.PhysicalOptimizeOp, -) (Task, int64, error) { - var bestTask Task = invalidTask +) (base.Task, int64, error) { + var bestTask base.Task = invalidTask var curCntPlan, cntPlan int64 var err error - childTasks := make([]Task, 0, len(p.children)) + childTasks := make([]base.Task, 0, len(p.children)) childCnts := make([]int64, len(p.children)) cntPlan = 0 iteration := p.iteratePhysicalPlan @@ -301,12 +302,12 @@ func (p *baseLogicalPlan) enumeratePhysicalPlans4Task( // iteratePhysicalPlan is used to iterate the physical plan and get all child tasks. func (p *baseLogicalPlan) iteratePhysicalPlan( - selfPhysicalPlan PhysicalPlan, - childTasks []Task, + selfPhysicalPlan base.PhysicalPlan, + childTasks []base.Task, childCnts []int64, _ *property.PhysicalProperty, opt *coreusage.PhysicalOptimizeOp, -) ([]Task, int64, []int64, error) { +) ([]base.Task, int64, []int64, error) { // Find best child tasks firstly. childTasks = childTasks[:0] // The curCntPlan records the number of possible plans for pp @@ -334,12 +335,12 @@ func (p *baseLogicalPlan) iteratePhysicalPlan( // iterateChildPlan does the special part for sequence. We need to iterate its child one by one to check whether the former child is a valid plan and then go to the nex func (p *LogicalSequence) iterateChildPlan( - selfPhysicalPlan PhysicalPlan, - childTasks []Task, + selfPhysicalPlan base.PhysicalPlan, + childTasks []base.Task, childCnts []int64, prop *property.PhysicalProperty, opt *coreusage.PhysicalOptimizeOp, -) ([]Task, int64, []int64, error) { +) ([]base.Task, int64, []int64, error) { // Find best child tasks firstly. childTasks = childTasks[:0] // The curCntPlan records the number of possible plans for pp @@ -391,7 +392,7 @@ func (p *LogicalSequence) iterateChildPlan( } // compareTaskCost compares cost of curTask and bestTask and returns whether curTask's cost is smaller than bestTask's. -func compareTaskCost(curTask, bestTask Task, op *coreusage.PhysicalOptimizeOp) (curIsBetter bool, err error) { +func compareTaskCost(curTask, bestTask base.Task, op *coreusage.PhysicalOptimizeOp) (curIsBetter bool, err error) { curCost, curInvalid, err := getTaskPlanCost(curTask, op) if err != nil { return false, err @@ -412,7 +413,7 @@ func compareTaskCost(curTask, bestTask Task, op *coreusage.PhysicalOptimizeOp) ( // getTaskPlanCost returns the cost of this task. // The new cost interface will be used if EnableNewCostInterface is true. // The second returned value indicates whether this task is valid. -func getTaskPlanCost(t Task, pop *coreusage.PhysicalOptimizeOp) (float64, bool, error) { +func getTaskPlanCost(t base.Task, pop *coreusage.PhysicalOptimizeOp) (float64, bool, error) { if t.Invalid() { return math.MaxFloat64, true, nil } @@ -495,7 +496,7 @@ func getTaskPlanCost(t Task, pop *coreusage.PhysicalOptimizeOp) (float64, bool, return cost + indexPartialCost, false, err } -func appendCandidate4PhysicalOptimizeOp(pop *coreusage.PhysicalOptimizeOp, lp LogicalPlan, pp PhysicalPlan, prop *property.PhysicalProperty) { +func appendCandidate4PhysicalOptimizeOp(pop *coreusage.PhysicalOptimizeOp, lp LogicalPlan, pp base.PhysicalPlan, prop *property.PhysicalProperty) { if pop == nil || pop.GetTracer() == nil || pp == nil { return } @@ -510,7 +511,7 @@ func appendCandidate4PhysicalOptimizeOp(pop *coreusage.PhysicalOptimizeOp, lp Lo // that will cause no physical plan when the logic plan got selected. // the fix to add innerTask.plan() to planTree and mapping correct logic plan index := -1 - var plan PhysicalPlan + var plan base.PhysicalPlan switch join := pp.(type) { case *PhysicalIndexMergeJoin: index = join.InnerChildIdx @@ -541,7 +542,7 @@ func appendPlanCostDetail4PhysicalOptimizeOp(pop *coreusage.PhysicalOptimizeOp, } // findBestTask implements LogicalPlan interface. -func (p *baseLogicalPlan) findBestTask(prop *property.PhysicalProperty, planCounter *PlanCounterTp, opt *coreusage.PhysicalOptimizeOp) (bestTask Task, cntPlan int64, err error) { +func (p *baseLogicalPlan) findBestTask(prop *property.PhysicalProperty, planCounter *PlanCounterTp, opt *coreusage.PhysicalOptimizeOp) (bestTask base.Task, cntPlan int64, err error) { // If p is an inner plan in an IndexJoin, the IndexJoin will generate an inner plan by itself, // and set inner child prop nil, so here we do nothing. if prop == nil { @@ -567,7 +568,7 @@ func (p *baseLogicalPlan) findBestTask(prop *property.PhysicalProperty, planCoun // prop should be read only because its cached hashcode might be not consistent // when it is changed. So we clone a new one for the temporary changes. newProp := prop.CloneEssentialFields() - var plansFitsProp, plansNeedEnforce []PhysicalPlan + var plansFitsProp, plansNeedEnforce []base.PhysicalPlan var hintWorksWithProp bool // Maybe the plan can satisfy the required property, // so we try to get the task without the enforced sort first. @@ -610,7 +611,7 @@ func (p *baseLogicalPlan) findBestTask(prop *property.PhysicalProperty, planCoun } var cnt int64 - var curTask Task + var curTask base.Task if bestTask, cnt, err = p.enumeratePhysicalPlans4Task(plansFitsProp, newProp, false, planCounter, opt); err != nil { return nil, 0, err } @@ -640,7 +641,7 @@ END: return bestTask, cntPlan, nil } -func (p *LogicalMemTable) findBestTask(prop *property.PhysicalProperty, planCounter *PlanCounterTp, opt *coreusage.PhysicalOptimizeOp) (t Task, cntPlan int64, err error) { +func (p *LogicalMemTable) findBestTask(prop *property.PhysicalProperty, planCounter *PlanCounterTp, opt *coreusage.PhysicalOptimizeOp) (t base.Task, cntPlan int64, err error) { if prop.MPPPartitionTp != property.AnyType { return invalidTask, 0, nil } @@ -694,7 +695,7 @@ func (p *LogicalMemTable) findBestTask(prop *property.PhysicalProperty, planCoun } // tryToGetDualTask will check if the push down predicate has false constant. If so, it will return table dual. -func (ds *DataSource) tryToGetDualTask() (Task, error) { +func (ds *DataSource) tryToGetDualTask() (base.Task, error) { for _, cond := range ds.pushedDownConds { if con, ok := cond.(*expression.Constant); ok && con.DeferredExpr == nil && con.ParamMarker == nil { result, _, err := expression.EvalBool(ds.SCtx().GetExprCtx().GetEvalCtx(), []expression.Expression{cond}, chunk.Row{}) @@ -743,7 +744,7 @@ func compareIndexBack(lhs, rhs *candidatePath) (int, bool) { // compareCandidates is the core of skyline pruning, which is used to decide which candidate path is better. // The return value is 1 if lhs is better, -1 if rhs is better, 0 if they are equivalent or not comparable. -func compareCandidates(sctx PlanContext, prop *property.PhysicalProperty, lhs, rhs *candidatePath) int { +func compareCandidates(sctx base.PlanContext, prop *property.PhysicalProperty, lhs, rhs *candidatePath) int { // Due to #50125, full scan on MVIndex has been disabled, so MVIndex path might lead to 'can't find a proper plan' error at the end. // Avoid MVIndex path to exclude all other paths and leading to 'can't find a proper plan' error, see #49438 for an example. if isMVIndexPath(lhs.path) || isMVIndexPath(rhs.path) { @@ -1273,7 +1274,7 @@ func (ds *DataSource) exploreEnforcedPlan() bool { // findBestTask implements the PhysicalPlan interface. // It will enumerate all the available indices and choose a plan with least cost. -func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter *PlanCounterTp, opt *coreusage.PhysicalOptimizeOp) (t Task, cntPlan int64, err error) { +func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter *PlanCounterTp, opt *coreusage.PhysicalOptimizeOp) (t base.Task, cntPlan int64, err error) { // If ds is an inner plan in an IndexJoin, the IndexJoin will generate an inner plan by itself, // and set inner child prop nil, so here we do nothing. if prop == nil { @@ -1311,7 +1312,7 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter return } var cnt int64 - var unenforcedTask Task + var unenforcedTask base.Task // If prop.CanAddEnforcer is true, the prop.SortItems need to be set nil for ds.findBestTask. // Before function return, reset it for enforcing task prop and storing map. oldProp := prop.CloneEssentialFields() @@ -1494,7 +1495,7 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter } } if allRangeIsPoint { - var pointGetTask Task + var pointGetTask base.Task if len(path.Ranges) == 1 { pointGetTask = ds.convertToPointGet(prop, candidate) } else { @@ -1532,7 +1533,7 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter if ds.preferStoreType&h.PreferTiKV != 0 && path.StoreType == kv.TiFlash { continue } - var tblTask Task + var tblTask base.Task if ds.SampleInfo != nil { tblTask, err = ds.convertToSampleTable(prop, candidate, opt) } else { @@ -1587,7 +1588,7 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter } // convertToIndexMergeScan builds the index merge scan for intersection or union cases. -func (ds *DataSource) convertToIndexMergeScan(prop *property.PhysicalProperty, candidate *candidatePath, _ *coreusage.PhysicalOptimizeOp) (task Task, err error) { +func (ds *DataSource) convertToIndexMergeScan(prop *property.PhysicalProperty, candidate *candidatePath, _ *coreusage.PhysicalOptimizeOp) (task base.Task, err error) { if prop.IsFlashProp() || prop.TaskTp == property.CopSingleReadTaskType { return invalidTask, nil } @@ -1609,7 +1610,7 @@ func (ds *DataSource) convertToIndexMergeScan(prop *property.PhysicalProperty, c } }) path := candidate.path - scans := make([]PhysicalPlan, 0, len(path.PartialIndexPaths)) + scans := make([]base.PhysicalPlan, 0, len(path.PartialIndexPaths)) cop := &CopTask{ indexPlanFinished: false, tblColHists: ds.TblColHists, @@ -1629,7 +1630,7 @@ func (ds *DataSource) convertToIndexMergeScan(prop *property.PhysicalProperty, c }) } for _, partPath := range path.PartialIndexPaths { - var scan PhysicalPlan + var scan base.PhysicalPlan if partPath.IsTablePath() { scan = ds.convertToPartialTableScan(prop, partPath, candidate.isMatchProp, byItems) } else { @@ -1678,7 +1679,7 @@ func (ds *DataSource) convertToIndexMergeScan(prop *property.PhysicalProperty, c return task, nil } -func (ds *DataSource) convertToPartialIndexScan(prop *property.PhysicalProperty, path *util.AccessPath, matchProp bool, byItems []*util.ByItems) (indexPlan PhysicalPlan) { +func (ds *DataSource) convertToPartialIndexScan(prop *property.PhysicalProperty, path *util.AccessPath, matchProp bool, byItems []*util.ByItems) (indexPlan base.PhysicalPlan) { is := ds.getOriginalPhysicalIndexScan(prop, path, matchProp, false) // TODO: Consider using isIndexCoveringColumns() to avoid another TableRead indexConds := path.IndexFilters @@ -1717,7 +1718,7 @@ func checkColinSchema(cols []*expression.Column, schema *expression.Schema) bool return true } -func (ds *DataSource) convertToPartialTableScan(prop *property.PhysicalProperty, path *util.AccessPath, matchProp bool, byItems []*util.ByItems) (tablePlan PhysicalPlan) { +func (ds *DataSource) convertToPartialTableScan(prop *property.PhysicalProperty, path *util.AccessPath, matchProp bool, byItems []*util.ByItems) (tablePlan base.PhysicalPlan) { ts, rowCount := ds.getOriginalPhysicalTableScan(prop, path, matchProp) overwritePartialTableScanSchema(ds, ts) // remove ineffetive filter condition after overwriting physicalscan schema @@ -1790,7 +1791,7 @@ func setIndexMergeTableScanHandleCols(ds *DataSource, ts *PhysicalTableScan) (er // buildIndexMergeTableScan() returns Selection that will be pushed to TiKV. // Filters that cannot be pushed to TiKV are also returned, and an extra Selection above IndexMergeReader will be constructed later. func (ds *DataSource) buildIndexMergeTableScan(tableFilters []expression.Expression, - totalRowCount float64, matchProp bool) (PhysicalPlan, []expression.Expression, bool, error) { + totalRowCount float64, matchProp bool) (base.PhysicalPlan, []expression.Expression, bool, error) { ts := PhysicalTableScan{ Table: ds.tableInfo, Columns: slices.Clone(ds.Columns), @@ -1815,7 +1816,7 @@ func (ds *DataSource) buildIndexMergeTableScan(tableFilters []expression.Express if ds.statisticTable.Pseudo { ts.StatsInfo().StatsVersion = statistics.PseudoVersion } - var currentTopPlan PhysicalPlan = ts + var currentTopPlan base.PhysicalPlan = ts if len(tableFilters) > 0 { pushedFilters, remainingFilters := extractFiltersForIndexMerge(GetPushDownCtx(ds.SCtx()), tableFilters) pushedFilters1, remainingFilters1 := SplitSelCondsWithVirtualColumn(pushedFilters) @@ -1990,7 +1991,7 @@ func (ts *PhysicalTableScan) appendExtraHandleCol(ds *DataSource) (*expression.C // convertToIndexScan converts the DataSource to index scan with idx. func (ds *DataSource) convertToIndexScan(prop *property.PhysicalProperty, - candidate *candidatePath, _ *coreusage.PhysicalOptimizeOp) (task Task, err error) { + candidate *candidatePath, _ *coreusage.PhysicalOptimizeOp) (task base.Task, err error) { if candidate.path.Index.MVIndex { // MVIndex is special since different index rows may return the same _row_id and this can break some assumptions of IndexReader. // Currently only support using IndexMerge to access MVIndex instead of IndexReader. @@ -2273,7 +2274,7 @@ func SplitSelCondsWithVirtualColumn(conds []expression.Expression) (withoutVirt return withoutVirt, withVirt } -func matchIndicesProp(sctx PlanContext, idxCols []*expression.Column, colLens []int, propItems []property.SortItem) bool { +func matchIndicesProp(sctx base.PlanContext, idxCols []*expression.Column, colLens []int, propItems []property.SortItem) bool { if len(idxCols) < len(propItems) { return false } @@ -2380,7 +2381,7 @@ func (ds *DataSource) isPointGetPath(path *util.AccessPath) bool { } // convertToTableScan converts the DataSource to table scan. -func (ds *DataSource) convertToTableScan(prop *property.PhysicalProperty, candidate *candidatePath, _ *coreusage.PhysicalOptimizeOp) (Task, error) { +func (ds *DataSource) convertToTableScan(prop *property.PhysicalProperty, candidate *candidatePath, _ *coreusage.PhysicalOptimizeOp) (base.Task, error) { // It will be handled in convertToIndexScan. if prop.TaskTp == property.CopMultiReadTaskType { return invalidTask, nil @@ -2455,7 +2456,7 @@ func (ds *DataSource) convertToTableScan(prop *property.PhysicalProperty, candid ColumnNames: ds.names, } mppTask = ts.addPushedDownSelectionToMppTask(mppTask, ds.StatsInfo().ScaleByExpectCnt(prop.ExpectedCnt)) - var task Task = mppTask + var task base.Task = mppTask if !mppTask.Invalid() { if prop.TaskTp == property.MppTaskType && len(mppTask.rootTaskConds) > 0 { // If got filters cannot be pushed down to tiflash, we have to make sure it will be executed in TiDB, @@ -2489,7 +2490,7 @@ func (ds *DataSource) convertToTableScan(prop *property.PhysicalProperty, candid ColumnNames: ds.names, } ts.PlanPartInfo = copTask.physPlanPartInfo - var task Task = copTask + var task base.Task = copTask if candidate.isMatchProp { copTask.keepOrder = true if ds.tableInfo.GetPartitionInfo() != nil { @@ -2521,7 +2522,7 @@ func (ds *DataSource) convertToTableScan(prop *property.PhysicalProperty, candid } func (ds *DataSource) convertToSampleTable(prop *property.PhysicalProperty, - candidate *candidatePath, _ *coreusage.PhysicalOptimizeOp) (Task, error) { + candidate *candidatePath, _ *coreusage.PhysicalOptimizeOp) (base.Task, error) { if prop.TaskTp == property.CopMultiReadTaskType { return invalidTask, nil } @@ -2544,7 +2545,7 @@ func (ds *DataSource) convertToSampleTable(prop *property.PhysicalProperty, return rt, nil } -func (ds *DataSource) convertToPointGet(prop *property.PhysicalProperty, candidate *candidatePath) Task { +func (ds *DataSource) convertToPointGet(prop *property.PhysicalProperty, candidate *candidatePath) base.Task { if !prop.IsSortItemEmpty() && !candidate.isMatchProp { return invalidTask } @@ -2620,7 +2621,7 @@ func (ds *DataSource) convertToPointGet(prop *property.PhysicalProperty, candida return rTsk } -func (ds *DataSource) convertToBatchPointGet(prop *property.PhysicalProperty, candidate *candidatePath) Task { +func (ds *DataSource) convertToBatchPointGet(prop *property.PhysicalProperty, candidate *candidatePath) base.Task { if !prop.IsSortItemEmpty() && !candidate.isMatchProp { return invalidTask } @@ -2838,7 +2839,7 @@ func (ds *DataSource) getOriginalPhysicalIndexScan(prop *property.PhysicalProper return is } -func (p *LogicalCTE) findBestTask(prop *property.PhysicalProperty, counter *PlanCounterTp, pop *coreusage.PhysicalOptimizeOp) (t Task, cntPlan int64, err error) { +func (p *LogicalCTE) findBestTask(prop *property.PhysicalProperty, counter *PlanCounterTp, pop *coreusage.PhysicalOptimizeOp) (t base.Task, cntPlan int64, err error) { if len(p.children) > 0 { return p.baseLogicalPlan.findBestTask(prop, counter, pop) } @@ -2871,7 +2872,7 @@ func (p *LogicalCTE) findBestTask(prop *property.PhysicalProperty, counter *Plan return t, 1, nil } -func (p *LogicalCTETable) findBestTask(prop *property.PhysicalProperty, _ *PlanCounterTp, _ *coreusage.PhysicalOptimizeOp) (t Task, cntPlan int64, err error) { +func (p *LogicalCTETable) findBestTask(prop *property.PhysicalProperty, _ *PlanCounterTp, _ *coreusage.PhysicalOptimizeOp) (t base.Task, cntPlan int64, err error) { if !prop.IsSortItemEmpty() { return nil, 1, nil } @@ -2884,7 +2885,7 @@ func (p *LogicalCTETable) findBestTask(prop *property.PhysicalProperty, _ *PlanC return t, 1, nil } -func appendCandidate(lp LogicalPlan, task Task, prop *property.PhysicalProperty, opt *coreusage.PhysicalOptimizeOp) { +func appendCandidate(lp LogicalPlan, task base.Task, prop *property.PhysicalProperty, opt *coreusage.PhysicalOptimizeOp) { if task == nil || task.Invalid() { return } @@ -2900,7 +2901,7 @@ func pushDownNot(ctx expression.BuildContext, conds []expression.Expression) []e return conds } -func validateTableSamplePlan(ds *DataSource, t Task, err error) error { +func validateTableSamplePlan(ds *DataSource, t base.Task, err error) error { if err != nil { return err } diff --git a/pkg/planner/core/find_best_task_test.go b/pkg/planner/core/find_best_task_test.go index 6124f5898062b..1296f649c111c 100644 --- a/pkg/planner/core/find_best_task_test.go +++ b/pkg/planner/core/find_best_task_test.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/expression" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util/coreusage" "github.com/stretchr/testify/require" @@ -29,12 +30,12 @@ type mockDataSource struct { baseLogicalPlan } -func (ds mockDataSource) Init(ctx PlanContext) *mockDataSource { +func (ds mockDataSource) Init(ctx base.PlanContext) *mockDataSource { ds.baseLogicalPlan = newBaseLogicalPlan(ctx, "mockDS", &ds, 0) return &ds } -func (ds *mockDataSource) findBestTask(prop *property.PhysicalProperty, planCounter *PlanCounterTp, opt *coreusage.PhysicalOptimizeOp) (Task, int64, error) { +func (ds *mockDataSource) findBestTask(prop *property.PhysicalProperty, planCounter *PlanCounterTp, opt *coreusage.PhysicalOptimizeOp) (base.Task, int64, error) { // It can satisfy any of the property! // Just use a TableDual for convenience. p := PhysicalTableDual{}.Init(ds.SCtx(), &property.StatsInfo{RowCount: 1}, 0) @@ -66,12 +67,12 @@ type mockLogicalPlan4Test struct { costOverflow bool } -func (p mockLogicalPlan4Test) Init(ctx PlanContext) *mockLogicalPlan4Test { +func (p mockLogicalPlan4Test) Init(ctx base.PlanContext) *mockLogicalPlan4Test { p.baseLogicalPlan = newBaseLogicalPlan(ctx, "mockPlan", &p, 0) return &p } -func (p *mockLogicalPlan4Test) getPhysicalPlan1(prop *property.PhysicalProperty) PhysicalPlan { +func (p *mockLogicalPlan4Test) getPhysicalPlan1(prop *property.PhysicalProperty) base.PhysicalPlan { physicalPlan1 := mockPhysicalPlan4Test{planType: 1}.Init(p.SCtx()) physicalPlan1.SetStats(&property.StatsInfo{RowCount: 1}) physicalPlan1.childrenReqProps = make([]*property.PhysicalProperty, 1) @@ -79,7 +80,7 @@ func (p *mockLogicalPlan4Test) getPhysicalPlan1(prop *property.PhysicalProperty) return physicalPlan1 } -func (p *mockLogicalPlan4Test) getPhysicalPlan2(prop *property.PhysicalProperty) PhysicalPlan { +func (p *mockLogicalPlan4Test) getPhysicalPlan2(prop *property.PhysicalProperty) base.PhysicalPlan { physicalPlan2 := mockPhysicalPlan4Test{planType: 2}.Init(p.SCtx()) physicalPlan2.SetStats(&property.StatsInfo{RowCount: 1}) physicalPlan2.childrenReqProps = make([]*property.PhysicalProperty, 1) @@ -87,9 +88,9 @@ func (p *mockLogicalPlan4Test) getPhysicalPlan2(prop *property.PhysicalProperty) return physicalPlan2 } -func (p *mockLogicalPlan4Test) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { - plan1 := make([]PhysicalPlan, 0, 1) - plan2 := make([]PhysicalPlan, 0, 1) +func (p *mockLogicalPlan4Test) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) { + plan1 := make([]base.PhysicalPlan, 0, 1) + plan2 := make([]base.PhysicalPlan, 0, 1) if prop.IsSortItemEmpty() && p.canGeneratePlan2 { // Generate PhysicalPlan2 when the property is empty. plan2 = append(plan2, p.getPhysicalPlan2(prop)) @@ -118,13 +119,13 @@ type mockPhysicalPlan4Test struct { planType int } -func (p mockPhysicalPlan4Test) Init(ctx PlanContext) *mockPhysicalPlan4Test { +func (p mockPhysicalPlan4Test) Init(ctx base.PlanContext) *mockPhysicalPlan4Test { p.basePhysicalPlan = newBasePhysicalPlan(ctx, "mockPlan", &p, 0) return &p } // Attach2Task implements the PhysicalPlan interface. -func (p *mockPhysicalPlan4Test) Attach2Task(tasks ...Task) Task { +func (p *mockPhysicalPlan4Test) Attach2Task(tasks ...base.Task) base.Task { t := tasks[0].Copy() attachPlan2Task(p, t) return t diff --git a/pkg/planner/core/flat_plan.go b/pkg/planner/core/flat_plan.go index 5dc4e4c3170e4..e84ac2942b66d 100644 --- a/pkg/planner/core/flat_plan.go +++ b/pkg/planner/core/flat_plan.go @@ -19,6 +19,7 @@ import ( "slices" "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/util/logutil" "github.com/pingcap/tidb/pkg/util/texttree" "go.uber.org/zap" @@ -92,7 +93,7 @@ func (e FlatPlanTree) GetSelectPlan() (FlatPlanTree, int) { // It contains a reference to the original operator and some usually needed information. type FlatOperator struct { // A reference to the original operator. - Origin Plan + Origin base.Plan // With ChildrenIdx and ChildrenEndIdx, we can locate every children subtrees of this operator in the FlatPlanTree. // For example, the first children subtree is flatTree[ChildrenIdx[0] : ChildrenIdx[1]], the last children subtree @@ -171,7 +172,7 @@ type operatorCtx struct { } // FlattenPhysicalPlan generates a FlatPhysicalPlan from a PhysicalPlan, Insert, Delete, Update, Explain or Execute. -func FlattenPhysicalPlan(p Plan, buildSideFirst bool) *FlatPhysicalPlan { +func FlattenPhysicalPlan(p base.Plan, buildSideFirst bool) *FlatPhysicalPlan { if p == nil { return nil } @@ -216,7 +217,7 @@ func FlattenPhysicalPlan(p Plan, buildSideFirst bool) *FlatPhysicalPlan { return res } -func (*FlatPhysicalPlan) flattenSingle(p Plan, info *operatorCtx) *FlatOperator { +func (*FlatPhysicalPlan) flattenSingle(p base.Plan, info *operatorCtx) *FlatOperator { // Some operators are not initialized and given an ExplainID. So their explain IDs are "_0" // (when in EXPLAIN FORMAT = 'brief' it will be ""), we skip such operators. // Examples: Explain, Execute @@ -234,14 +235,14 @@ func (*FlatPhysicalPlan) flattenSingle(p Plan, info *operatorCtx) *FlatOperator IsLastChild: info.isLastChild, } - if _, ok := p.(PhysicalPlan); ok { + if _, ok := p.(base.PhysicalPlan); ok { res.IsPhysicalPlan = true } return res } // Note that info should not be modified in this method. -func (f *FlatPhysicalPlan) flattenRecursively(p Plan, info *operatorCtx, target FlatPlanTree) (res FlatPlanTree, idx int) { +func (f *FlatPhysicalPlan) flattenRecursively(p base.Plan, info *operatorCtx, target FlatPlanTree) (res FlatPlanTree, idx int) { idx = -1 flat := f.flattenSingle(p, info) if flat != nil { @@ -259,7 +260,7 @@ func (f *FlatPhysicalPlan) flattenRecursively(p Plan, info *operatorCtx, target } // For physical operators, we just enumerate their children and collect their information. // Note that some physical operators are special, and they are handled below this part. - if physPlan, ok := p.(PhysicalPlan); ok { + if physPlan, ok := p.(base.PhysicalPlan); ok { label := make([]OperatorLabel, len(physPlan.Children())) switch plan := physPlan.(type) { @@ -293,7 +294,7 @@ func (f *FlatPhysicalPlan) flattenRecursively(p Plan, info *operatorCtx, target label[1-plan.InnerChildIdx] = BuildSide } - children := make([]PhysicalPlan, len(physPlan.Children())) + children := make([]base.PhysicalPlan, len(physPlan.Children())) copy(children, physPlan.Children()) if len(label) == 2 && label[0] == ProbeSide && diff --git a/pkg/planner/core/foreign_key.go b/pkg/planner/core/foreign_key.go index 653956d0cba9c..15c8444dad9c2 100644 --- a/pkg/planner/core/foreign_key.go +++ b/pkg/planner/core/foreign_key.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/util/dbterror/plannererrors" ) @@ -54,7 +55,7 @@ type FKCascade struct { // CascadePlans contains the child cascade plan. // CascadePlans will be filled during execution, so only `explain analyze` statement result contains the cascade plan, // `explain` statement result doesn't contain the cascade plan. - CascadePlans []Plan + CascadePlans []base.Plan } // FKCascadeType indicates in which (delete/update) statements. @@ -140,7 +141,7 @@ func (f *FKCascade) MemoryUsage() (sum int64) { return } -func (p *Insert) buildOnInsertFKTriggers(ctx PlanContext, is infoschema.InfoSchema, dbName string) error { +func (p *Insert) buildOnInsertFKTriggers(ctx base.PlanContext, is infoschema.InfoSchema, dbName string) error { if !ctx.GetSessionVars().ForeignKeyChecks { return nil } @@ -197,7 +198,7 @@ func (p *Insert) buildOnDuplicateUpdateColumns() map[string]struct{} { return m } -func (*Insert) buildOnReplaceReferredFKTriggers(ctx PlanContext, is infoschema.InfoSchema, dbName string, tblInfo *model.TableInfo) ([]*FKCheck, []*FKCascade, error) { +func (*Insert) buildOnReplaceReferredFKTriggers(ctx base.PlanContext, is infoschema.InfoSchema, dbName string, tblInfo *model.TableInfo) ([]*FKCheck, []*FKCascade, error) { referredFKs := is.GetTableReferredForeignKeys(dbName, tblInfo.Name.L) fkChecks := make([]*FKCheck, 0, len(referredFKs)) fkCascades := make([]*FKCascade, 0, len(referredFKs)) @@ -216,7 +217,7 @@ func (*Insert) buildOnReplaceReferredFKTriggers(ctx PlanContext, is infoschema.I return fkChecks, fkCascades, nil } -func (updt *Update) buildOnUpdateFKTriggers(ctx PlanContext, is infoschema.InfoSchema, tblID2table map[int64]table.Table) error { +func (updt *Update) buildOnUpdateFKTriggers(ctx base.PlanContext, is infoschema.InfoSchema, tblID2table map[int64]table.Table) error { if !ctx.GetSessionVars().ForeignKeyChecks { return nil } @@ -257,7 +258,7 @@ func (updt *Update) buildOnUpdateFKTriggers(ctx PlanContext, is infoschema.InfoS return nil } -func (del *Delete) buildOnDeleteFKTriggers(ctx PlanContext, is infoschema.InfoSchema, tblID2table map[int64]table.Table) error { +func (del *Delete) buildOnDeleteFKTriggers(ctx base.PlanContext, is infoschema.InfoSchema, tblID2table map[int64]table.Table) error { if !ctx.GetSessionVars().ForeignKeyChecks { return nil } @@ -288,7 +289,7 @@ func (del *Delete) buildOnDeleteFKTriggers(ctx PlanContext, is infoschema.InfoSc return nil } -func buildOnUpdateReferredFKTriggers(ctx PlanContext, is infoschema.InfoSchema, dbName string, tblInfo *model.TableInfo, updateCols map[string]struct{}) ([]*FKCheck, []*FKCascade, error) { +func buildOnUpdateReferredFKTriggers(ctx base.PlanContext, is infoschema.InfoSchema, dbName string, tblInfo *model.TableInfo, updateCols map[string]struct{}) ([]*FKCheck, []*FKCascade, error) { referredFKs := is.GetTableReferredForeignKeys(dbName, tblInfo.Name.L) fkChecks := make([]*FKCheck, 0, len(referredFKs)) fkCascades := make([]*FKCascade, 0, len(referredFKs)) @@ -310,7 +311,7 @@ func buildOnUpdateReferredFKTriggers(ctx PlanContext, is infoschema.InfoSchema, return fkChecks, fkCascades, nil } -func buildOnUpdateChildFKChecks(ctx PlanContext, is infoschema.InfoSchema, dbName string, tblInfo *model.TableInfo, updateCols map[string]struct{}) ([]*FKCheck, error) { +func buildOnUpdateChildFKChecks(ctx base.PlanContext, is infoschema.InfoSchema, dbName string, tblInfo *model.TableInfo, updateCols map[string]struct{}) ([]*FKCheck, error) { fkChecks := make([]*FKCheck, 0, len(tblInfo.ForeignKeys)) for _, fk := range tblInfo.ForeignKeys { if fk.Version < 1 { @@ -365,7 +366,7 @@ func (updt *Update) buildTbl2UpdateColumns() map[int64]map[string]struct{} { return tblID2UpdateColumns } -func buildOnDeleteOrUpdateFKTrigger(ctx PlanContext, is infoschema.InfoSchema, referredFK *model.ReferredFKInfo, tp FKCascadeType) (*FKCheck, *FKCascade, error) { +func buildOnDeleteOrUpdateFKTrigger(ctx base.PlanContext, is infoschema.InfoSchema, referredFK *model.ReferredFKInfo, tp FKCascadeType) (*FKCheck, *FKCascade, error) { childTable, err := is.TableByName(referredFK.ChildSchema, referredFK.ChildTable) if err != nil { return nil, nil, nil @@ -405,7 +406,7 @@ func isMapContainAnyCols(colsMap map[string]struct{}, cols ...model.CIStr) bool return false } -func buildFKCheckOnModifyChildTable(ctx PlanContext, is infoschema.InfoSchema, fk *model.FKInfo, failedErr error) (*FKCheck, error) { +func buildFKCheckOnModifyChildTable(ctx base.PlanContext, is infoschema.InfoSchema, fk *model.FKInfo, failedErr error) (*FKCheck, error) { referTable, err := is.TableByName(fk.RefSchema, fk.RefTable) if err != nil { return nil, nil @@ -419,7 +420,7 @@ func buildFKCheckOnModifyChildTable(ctx PlanContext, is infoschema.InfoSchema, f return fkCheck, nil } -func buildFKCheckForReferredFK(ctx PlanContext, childTable table.Table, fk *model.FKInfo, referredFK *model.ReferredFKInfo) (*FKCheck, error) { +func buildFKCheckForReferredFK(ctx base.PlanContext, childTable table.Table, fk *model.FKInfo, referredFK *model.ReferredFKInfo) (*FKCheck, error) { failedErr := plannererrors.ErrRowIsReferenced2.GenWithStackByArgs(fk.String(referredFK.ChildSchema.L, referredFK.ChildTable.L)) fkCheck, err := buildFKCheck(ctx, childTable, fk.Cols, failedErr) if err != nil { @@ -430,7 +431,7 @@ func buildFKCheckForReferredFK(ctx PlanContext, childTable table.Table, fk *mode return fkCheck, nil } -func buildFKCheck(ctx PlanContext, tbl table.Table, cols []model.CIStr, failedErr error) (*FKCheck, error) { +func buildFKCheck(ctx base.PlanContext, tbl table.Table, cols []model.CIStr, failedErr error) (*FKCheck, error) { tblInfo := tbl.Meta() if tblInfo.PKIsHandle && len(cols) == 1 { refColInfo := model.FindColumnInfo(tblInfo.Columns, cols[0].L) @@ -467,7 +468,7 @@ func buildFKCheck(ctx PlanContext, tbl table.Table, cols []model.CIStr, failedEr }.Init(ctx), nil } -func buildFKCascade(ctx PlanContext, tp FKCascadeType, referredFK *model.ReferredFKInfo, childTable table.Table, fk *model.FKInfo) (*FKCascade, error) { +func buildFKCascade(ctx base.PlanContext, tp FKCascadeType, referredFK *model.ReferredFKInfo, childTable table.Table, fk *model.FKInfo) (*FKCascade, error) { cols := make([]*model.ColumnInfo, len(fk.Cols)) childTableColumns := childTable.Meta().Columns for i, c := range fk.Cols { diff --git a/pkg/planner/core/fragment.go b/pkg/planner/core/fragment.go index 4db7eeaba2023..bbbbf810a8b38 100644 --- a/pkg/planner/core/fragment.go +++ b/pkg/planner/core/fragment.go @@ -30,6 +30,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/types" @@ -219,7 +220,7 @@ func (e *mppTaskGenerator) constructMPPTasksByChildrenTasks(tasks []*kv.MPPTask, return newTasks } -func (f *Fragment) init(p PhysicalPlan) error { +func (f *Fragment) init(p base.PhysicalPlan) error { switch x := p.(type) { case *PhysicalTableScan: if f.TableScan != nil { @@ -252,7 +253,7 @@ func (f *Fragment) init(p PhysicalPlan) error { // after untwist, there will be two plans in `forest` slice: // - ExchangeSender -> Projection (c1) -> TableScan(t) // - ExchangeSender -> Projection (c2) -> TableScan(s) -func (e *mppTaskGenerator) untwistPlanAndRemoveUnionAll(stack []PhysicalPlan, forest *[]*PhysicalExchangeSender) error { +func (e *mppTaskGenerator) untwistPlanAndRemoveUnionAll(stack []base.PhysicalPlan, forest *[]*PhysicalExchangeSender) error { cur := stack[len(stack)-1] switch x := cur.(type) { case *PhysicalTableScan, *PhysicalExchangeReceiver, *PhysicalCTE: // This should be the leave node. @@ -330,7 +331,7 @@ func (e *mppTaskGenerator) untwistPlanAndRemoveUnionAll(stack []PhysicalPlan, fo func (e *mppTaskGenerator) buildFragments(s *PhysicalExchangeSender) ([]*Fragment, error) { forest := make([]*PhysicalExchangeSender, 0, 1) - err := e.untwistPlanAndRemoveUnionAll([]PhysicalPlan{s}, &forest) + err := e.untwistPlanAndRemoveUnionAll([]base.PhysicalPlan{s}, &forest) if err != nil { return nil, errors.Trace(err) } @@ -427,8 +428,8 @@ func (e *mppTaskGenerator) generateMPPTasksForFragment(f *Fragment) (tasks []*kv // flipCTEReader fix the plan tree. In the func generateTasksForCTEReader, we create the plan tree like ParentPlan->CTEConsumer->ExchangeReceiver. // The CTEConsumer has no real meaning in MPP's execution. We prune it to make the plan become ParentPlan->ExchangeReceiver. // But the Receiver needs a schema since itself doesn't hold the schema. So the final plan become ParentPlan->ExchangeReceiver->CTEConsumer. -func (f *Fragment) flipCTEReader(currentPlan PhysicalPlan) { - newChildren := make([]PhysicalPlan, len(currentPlan.Children())) +func (f *Fragment) flipCTEReader(currentPlan base.PhysicalPlan) { + newChildren := make([]base.PhysicalPlan, len(currentPlan.Children())) for i := 0; i < len(currentPlan.Children()); i++ { child := currentPlan.Children()[i] newChildren[i] = child @@ -487,7 +488,7 @@ func (e *mppTaskGenerator) addReaderTasksForCTEStorage(storageID int, tasks ...* } } -func partitionPruning(ctx PlanContext, tbl table.PartitionedTable, conds []expression.Expression, partitionNames []model.CIStr, +func partitionPruning(ctx base.PlanContext, tbl table.PartitionedTable, conds []expression.Expression, partitionNames []model.CIStr, columns []*expression.Column, columnNames types.NameSlice) ([]table.PhysicalTable, error) { idxArr, err := PartitionPruning(ctx, tbl, conds, partitionNames, columns, columnNames) if err != nil { diff --git a/pkg/planner/core/hint_utils.go b/pkg/planner/core/hint_utils.go index 6d8a17d9bd54e..76faa02452f24 100644 --- a/pkg/planner/core/hint_utils.go +++ b/pkg/planner/core/hint_utils.go @@ -18,6 +18,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/planner/core/base" h "github.com/pingcap/tidb/pkg/util/hint" ) @@ -39,7 +40,7 @@ func GenHintsFromFlatPlan(flat *FlatPhysicalPlan) []*ast.TableOptimizerHint { return nil } for _, fop := range selectPlan { - p := fop.Origin.(PhysicalPlan) + p := fop.Origin.(base.PhysicalPlan) hints = genHintsFromSingle(p, nodeTp, fop.StoreType, hints) } for _, cte := range flat.CTEs { @@ -47,7 +48,7 @@ func GenHintsFromFlatPlan(flat *FlatPhysicalPlan) []*ast.TableOptimizerHint { if i == 0 || !fop.IsRoot { continue } - p := fop.Origin.(PhysicalPlan) + p := fop.Origin.(base.PhysicalPlan) hints = genHintsFromSingle(p, nodeTp, fop.StoreType, hints) } } @@ -55,7 +56,7 @@ func GenHintsFromFlatPlan(flat *FlatPhysicalPlan) []*ast.TableOptimizerHint { } // GenHintsFromPhysicalPlan generates hints from physical plan. -func GenHintsFromPhysicalPlan(p Plan) []*ast.TableOptimizerHint { +func GenHintsFromPhysicalPlan(p base.Plan) []*ast.TableOptimizerHint { flat := FlattenPhysicalPlan(p, false) return GenHintsFromFlatPlan(flat) } @@ -67,7 +68,7 @@ func getTableName(tblName model.CIStr, asName *model.CIStr) model.CIStr { return tblName } -func extractTableAsName(p PhysicalPlan) (*model.CIStr, *model.CIStr) { +func extractTableAsName(p base.PhysicalPlan) (*model.CIStr, *model.CIStr) { if len(p.Children()) > 1 { return nil, nil } @@ -96,7 +97,7 @@ func extractTableAsName(p PhysicalPlan) (*model.CIStr, *model.CIStr) { return nil, nil } -func getJoinHints(sctx PlanContext, joinType string, parentOffset int, nodeType h.NodeType, children ...PhysicalPlan) (res []*ast.TableOptimizerHint) { +func getJoinHints(sctx base.PlanContext, joinType string, parentOffset int, nodeType h.NodeType, children ...base.PhysicalPlan) (res []*ast.TableOptimizerHint) { if parentOffset == -1 { return res } @@ -137,7 +138,7 @@ func getJoinHints(sctx PlanContext, joinType string, parentOffset int, nodeType return res } -func genHintsFromSingle(p PhysicalPlan, nodeType h.NodeType, storeType kv.StoreType, res []*ast.TableOptimizerHint) []*ast.TableOptimizerHint { +func genHintsFromSingle(p base.PhysicalPlan, nodeType h.NodeType, storeType kv.StoreType, res []*ast.TableOptimizerHint) []*ast.TableOptimizerHint { qbName, err := h.GenerateQBName(nodeType, p.QueryBlockOffset()) if err != nil { return res diff --git a/pkg/planner/core/indexmerge_path.go b/pkg/planner/core/indexmerge_path.go index a7995c8eaef42..285a38349b63f 100644 --- a/pkg/planner/core/indexmerge_path.go +++ b/pkg/planner/core/indexmerge_path.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/cardinality" "github.com/pingcap/tidb/pkg/planner/context" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/planner/util/debugtrace" "github.com/pingcap/tidb/pkg/statistics" @@ -1370,7 +1371,7 @@ func collectFilters4MVIndex( // accessFilters: [x=1, (2 member of a), z=1], remainingFilters: [x+z>0], mvColOffset: 1, mvFilterMutations[(2 member of a), (1 member of a)] // // the outer usage will be: accessFilter[mvColOffset] = each element of mvFilterMutations to get the mv access filters mutation combination. -func CollectFilters4MVIndexMutations(sctx PlanContext, filters []expression.Expression, +func CollectFilters4MVIndexMutations(sctx base.PlanContext, filters []expression.Expression, idxCols []*expression.Column) (accessFilters, remainingFilters []expression.Expression, mvColOffset int, mvFilterMutations []expression.Expression) { usedAsAccess := make([]bool, len(filters)) // accessFilters [x, a, z] @@ -1481,7 +1482,7 @@ const ( // Though this function is introduced for MV index, it can also be used for normal index // If the return value ok is false, the type must be unspecifiedFilterTp. func checkAccessFilter4IdxCol( - sctx PlanContext, + sctx base.PlanContext, filter expression.Expression, idxCol *expression.Column, ) ( diff --git a/pkg/planner/core/initialize.go b/pkg/planner/core/initialize.go index 834d221196746..15d7cbc991028 100644 --- a/pkg/planner/core/initialize.go +++ b/pkg/planner/core/initialize.go @@ -17,6 +17,7 @@ package core import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/kv" + base2 "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/core/internal/base" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/types" @@ -25,55 +26,55 @@ import ( ) // Init initializes LogicalAggregation. -func (la LogicalAggregation) Init(ctx PlanContext, offset int) *LogicalAggregation { +func (la LogicalAggregation) Init(ctx base2.PlanContext, offset int) *LogicalAggregation { la.baseLogicalPlan = newBaseLogicalPlan(ctx, plancodec.TypeAgg, &la, offset) return &la } // Init initializes LogicalJoin. -func (p LogicalJoin) Init(ctx PlanContext, offset int) *LogicalJoin { +func (p LogicalJoin) Init(ctx base2.PlanContext, offset int) *LogicalJoin { p.baseLogicalPlan = newBaseLogicalPlan(ctx, plancodec.TypeJoin, &p, offset) return &p } // Init initializes DataSource. -func (ds DataSource) Init(ctx PlanContext, offset int) *DataSource { +func (ds DataSource) Init(ctx base2.PlanContext, offset int) *DataSource { ds.baseLogicalPlan = newBaseLogicalPlan(ctx, plancodec.TypeDataSource, &ds, offset) return &ds } // Init initializes TiKVSingleGather. -func (sg TiKVSingleGather) Init(ctx PlanContext, offset int) *TiKVSingleGather { +func (sg TiKVSingleGather) Init(ctx base2.PlanContext, offset int) *TiKVSingleGather { sg.baseLogicalPlan = newBaseLogicalPlan(ctx, plancodec.TypeTiKVSingleGather, &sg, offset) return &sg } // Init initializes LogicalTableScan. -func (ts LogicalTableScan) Init(ctx PlanContext, offset int) *LogicalTableScan { +func (ts LogicalTableScan) Init(ctx base2.PlanContext, offset int) *LogicalTableScan { ts.baseLogicalPlan = newBaseLogicalPlan(ctx, plancodec.TypeTableScan, &ts, offset) return &ts } // Init initializes LogicalIndexScan. -func (is LogicalIndexScan) Init(ctx PlanContext, offset int) *LogicalIndexScan { +func (is LogicalIndexScan) Init(ctx base2.PlanContext, offset int) *LogicalIndexScan { is.baseLogicalPlan = newBaseLogicalPlan(ctx, plancodec.TypeIdxScan, &is, offset) return &is } // Init initializes LogicalApply. -func (la LogicalApply) Init(ctx PlanContext, offset int) *LogicalApply { +func (la LogicalApply) Init(ctx base2.PlanContext, offset int) *LogicalApply { la.baseLogicalPlan = newBaseLogicalPlan(ctx, plancodec.TypeApply, &la, offset) return &la } // Init initializes LogicalSelection. -func (p LogicalSelection) Init(ctx PlanContext, qbOffset int) *LogicalSelection { +func (p LogicalSelection) Init(ctx base2.PlanContext, qbOffset int) *LogicalSelection { p.baseLogicalPlan = newBaseLogicalPlan(ctx, plancodec.TypeSel, &p, qbOffset) return &p } // Init initializes PhysicalSelection. -func (p PhysicalSelection) Init(ctx PlanContext, stats *property.StatsInfo, qbOffset int, props ...*property.PhysicalProperty) *PhysicalSelection { +func (p PhysicalSelection) Init(ctx base2.PlanContext, stats *property.StatsInfo, qbOffset int, props ...*property.PhysicalProperty) *PhysicalSelection { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeSel, &p, qbOffset) p.childrenReqProps = props p.SetStats(stats) @@ -81,25 +82,25 @@ func (p PhysicalSelection) Init(ctx PlanContext, stats *property.StatsInfo, qbOf } // Init initializes LogicalUnionScan. -func (p LogicalUnionScan) Init(ctx PlanContext, qbOffset int) *LogicalUnionScan { +func (p LogicalUnionScan) Init(ctx base2.PlanContext, qbOffset int) *LogicalUnionScan { p.baseLogicalPlan = newBaseLogicalPlan(ctx, plancodec.TypeUnionScan, &p, qbOffset) return &p } // Init initializes LogicalProjection. -func (p LogicalProjection) Init(ctx PlanContext, qbOffset int) *LogicalProjection { +func (p LogicalProjection) Init(ctx base2.PlanContext, qbOffset int) *LogicalProjection { p.baseLogicalPlan = newBaseLogicalPlan(ctx, plancodec.TypeProj, &p, qbOffset) return &p } // Init initializes LogicalProjection. -func (p LogicalExpand) Init(ctx PlanContext, offset int) *LogicalExpand { +func (p LogicalExpand) Init(ctx base2.PlanContext, offset int) *LogicalExpand { p.baseLogicalPlan = newBaseLogicalPlan(ctx, plancodec.TypeExpand, &p, offset) return &p } // Init initializes PhysicalProjection. -func (p PhysicalProjection) Init(ctx PlanContext, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalProjection { +func (p PhysicalProjection) Init(ctx base2.PlanContext, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalProjection { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeProj, &p, offset) p.childrenReqProps = props p.SetStats(stats) @@ -107,19 +108,19 @@ func (p PhysicalProjection) Init(ctx PlanContext, stats *property.StatsInfo, off } // Init initializes LogicalUnionAll. -func (p LogicalUnionAll) Init(ctx PlanContext, offset int) *LogicalUnionAll { +func (p LogicalUnionAll) Init(ctx base2.PlanContext, offset int) *LogicalUnionAll { p.baseLogicalPlan = newBaseLogicalPlan(ctx, plancodec.TypeUnion, &p, offset) return &p } // Init initializes LogicalPartitionUnionAll. -func (p LogicalPartitionUnionAll) Init(ctx PlanContext, offset int) *LogicalPartitionUnionAll { +func (p LogicalPartitionUnionAll) Init(ctx base2.PlanContext, offset int) *LogicalPartitionUnionAll { p.baseLogicalPlan = newBaseLogicalPlan(ctx, plancodec.TypePartitionUnion, &p, offset) return &p } // Init initializes PhysicalUnionAll. -func (p PhysicalUnionAll) Init(ctx PlanContext, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalUnionAll { +func (p PhysicalUnionAll) Init(ctx base2.PlanContext, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalUnionAll { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeUnion, &p, offset) p.childrenReqProps = props p.SetStats(stats) @@ -127,13 +128,13 @@ func (p PhysicalUnionAll) Init(ctx PlanContext, stats *property.StatsInfo, offse } // Init initializes LogicalSort. -func (ls LogicalSort) Init(ctx PlanContext, offset int) *LogicalSort { +func (ls LogicalSort) Init(ctx base2.PlanContext, offset int) *LogicalSort { ls.baseLogicalPlan = newBaseLogicalPlan(ctx, plancodec.TypeSort, &ls, offset) return &ls } // Init initializes PhysicalSort. -func (p PhysicalSort) Init(ctx PlanContext, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalSort { +func (p PhysicalSort) Init(ctx base2.PlanContext, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalSort { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeSort, &p, offset) p.childrenReqProps = props p.SetStats(stats) @@ -141,7 +142,7 @@ func (p PhysicalSort) Init(ctx PlanContext, stats *property.StatsInfo, offset in } // Init initializes NominalSort. -func (p NominalSort) Init(ctx PlanContext, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *NominalSort { +func (p NominalSort) Init(ctx base2.PlanContext, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *NominalSort { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeSort, &p, offset) p.childrenReqProps = props p.SetStats(stats) @@ -149,13 +150,13 @@ func (p NominalSort) Init(ctx PlanContext, stats *property.StatsInfo, offset int } // Init initializes LogicalTopN. -func (lt LogicalTopN) Init(ctx PlanContext, offset int) *LogicalTopN { +func (lt LogicalTopN) Init(ctx base2.PlanContext, offset int) *LogicalTopN { lt.baseLogicalPlan = newBaseLogicalPlan(ctx, plancodec.TypeTopN, <, offset) return < } // Init initializes PhysicalTopN. -func (p PhysicalTopN) Init(ctx PlanContext, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalTopN { +func (p PhysicalTopN) Init(ctx base2.PlanContext, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalTopN { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeTopN, &p, offset) p.childrenReqProps = props p.SetStats(stats) @@ -163,13 +164,13 @@ func (p PhysicalTopN) Init(ctx PlanContext, stats *property.StatsInfo, offset in } // Init initializes LogicalLimit. -func (p LogicalLimit) Init(ctx PlanContext, offset int) *LogicalLimit { +func (p LogicalLimit) Init(ctx base2.PlanContext, offset int) *LogicalLimit { p.baseLogicalPlan = newBaseLogicalPlan(ctx, plancodec.TypeLimit, &p, offset) return &p } // Init initializes PhysicalLimit. -func (p PhysicalLimit) Init(ctx PlanContext, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalLimit { +func (p PhysicalLimit) Init(ctx base2.PlanContext, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalLimit { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeLimit, &p, offset) p.childrenReqProps = props p.SetStats(stats) @@ -177,26 +178,26 @@ func (p PhysicalLimit) Init(ctx PlanContext, stats *property.StatsInfo, offset i } // Init initializes LogicalTableDual. -func (p LogicalTableDual) Init(ctx PlanContext, offset int) *LogicalTableDual { +func (p LogicalTableDual) Init(ctx base2.PlanContext, offset int) *LogicalTableDual { p.baseLogicalPlan = newBaseLogicalPlan(ctx, plancodec.TypeDual, &p, offset) return &p } // Init initializes PhysicalTableDual. -func (p PhysicalTableDual) Init(ctx PlanContext, stats *property.StatsInfo, offset int) *PhysicalTableDual { +func (p PhysicalTableDual) Init(ctx base2.PlanContext, stats *property.StatsInfo, offset int) *PhysicalTableDual { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeDual, &p, offset) p.SetStats(stats) return &p } // Init initializes LogicalMaxOneRow. -func (p LogicalMaxOneRow) Init(ctx PlanContext, offset int) *LogicalMaxOneRow { +func (p LogicalMaxOneRow) Init(ctx base2.PlanContext, offset int) *LogicalMaxOneRow { p.baseLogicalPlan = newBaseLogicalPlan(ctx, plancodec.TypeMaxOneRow, &p, offset) return &p } // Init initializes PhysicalMaxOneRow. -func (p PhysicalMaxOneRow) Init(ctx PlanContext, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalMaxOneRow { +func (p PhysicalMaxOneRow) Init(ctx base2.PlanContext, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalMaxOneRow { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeMaxOneRow, &p, offset) p.childrenReqProps = props p.SetStats(stats) @@ -204,13 +205,13 @@ func (p PhysicalMaxOneRow) Init(ctx PlanContext, stats *property.StatsInfo, offs } // Init initializes LogicalWindow. -func (p LogicalWindow) Init(ctx PlanContext, offset int) *LogicalWindow { +func (p LogicalWindow) Init(ctx base2.PlanContext, offset int) *LogicalWindow { p.baseLogicalPlan = newBaseLogicalPlan(ctx, plancodec.TypeWindow, &p, offset) return &p } // Init initializes PhysicalWindow. -func (p PhysicalWindow) Init(ctx PlanContext, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalWindow { +func (p PhysicalWindow) Init(ctx base2.PlanContext, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalWindow { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeWindow, &p, offset) p.childrenReqProps = props p.SetStats(stats) @@ -218,7 +219,7 @@ func (p PhysicalWindow) Init(ctx PlanContext, stats *property.StatsInfo, offset } // Init initializes PhysicalShuffle. -func (p PhysicalShuffle) Init(ctx PlanContext, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalShuffle { +func (p PhysicalShuffle) Init(ctx base2.PlanContext, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalShuffle { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeShuffle, &p, offset) p.childrenReqProps = props p.SetStats(stats) @@ -226,7 +227,7 @@ func (p PhysicalShuffle) Init(ctx PlanContext, stats *property.StatsInfo, offset } // Init initializes PhysicalShuffleReceiverStub. -func (p PhysicalShuffleReceiverStub) Init(ctx PlanContext, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalShuffleReceiverStub { +func (p PhysicalShuffleReceiverStub) Init(ctx base2.PlanContext, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalShuffleReceiverStub { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeShuffleReceiver, &p, offset) p.childrenReqProps = props p.SetStats(stats) @@ -234,49 +235,49 @@ func (p PhysicalShuffleReceiverStub) Init(ctx PlanContext, stats *property.Stats } // Init initializes Update. -func (p Update) Init(ctx PlanContext) *Update { +func (p Update) Init(ctx base2.PlanContext) *Update { p.Plan = base.NewBasePlan(ctx, plancodec.TypeUpdate, 0) return &p } // Init initializes Delete. -func (p Delete) Init(ctx PlanContext) *Delete { +func (p Delete) Init(ctx base2.PlanContext) *Delete { p.Plan = base.NewBasePlan(ctx, plancodec.TypeDelete, 0) return &p } // Init initializes Insert. -func (p Insert) Init(ctx PlanContext) *Insert { +func (p Insert) Init(ctx base2.PlanContext) *Insert { p.Plan = base.NewBasePlan(ctx, plancodec.TypeInsert, 0) return &p } // Init initializes LoadData. -func (p LoadData) Init(ctx PlanContext) *LoadData { +func (p LoadData) Init(ctx base2.PlanContext) *LoadData { p.Plan = base.NewBasePlan(ctx, plancodec.TypeLoadData, 0) return &p } // Init initializes ImportInto. -func (p ImportInto) Init(ctx PlanContext) *ImportInto { +func (p ImportInto) Init(ctx base2.PlanContext) *ImportInto { p.Plan = base.NewBasePlan(ctx, plancodec.TypeImportInto, 0) return &p } // Init initializes LogicalShow. -func (p LogicalShow) Init(ctx PlanContext) *LogicalShow { +func (p LogicalShow) Init(ctx base2.PlanContext) *LogicalShow { p.baseLogicalPlan = newBaseLogicalPlan(ctx, plancodec.TypeShow, &p, 0) return &p } // Init initializes LogicalShowDDLJobs. -func (p LogicalShowDDLJobs) Init(ctx PlanContext) *LogicalShowDDLJobs { +func (p LogicalShowDDLJobs) Init(ctx base2.PlanContext) *LogicalShowDDLJobs { p.baseLogicalPlan = newBaseLogicalPlan(ctx, plancodec.TypeShowDDLJobs, &p, 0) return &p } // Init initializes PhysicalShow. -func (p PhysicalShow) Init(ctx PlanContext) *PhysicalShow { +func (p PhysicalShow) Init(ctx base2.PlanContext) *PhysicalShow { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeShow, &p, 0) // Just use pseudo stats to avoid panic. p.SetStats(&property.StatsInfo{RowCount: 1}) @@ -284,7 +285,7 @@ func (p PhysicalShow) Init(ctx PlanContext) *PhysicalShow { } // Init initializes PhysicalShowDDLJobs. -func (p PhysicalShowDDLJobs) Init(ctx PlanContext) *PhysicalShowDDLJobs { +func (p PhysicalShowDDLJobs) Init(ctx base2.PlanContext) *PhysicalShowDDLJobs { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeShowDDLJobs, &p, 0) // Just use pseudo stats to avoid panic. p.SetStats(&property.StatsInfo{RowCount: 1}) @@ -292,13 +293,13 @@ func (p PhysicalShowDDLJobs) Init(ctx PlanContext) *PhysicalShowDDLJobs { } // Init initializes LogicalLock. -func (p LogicalLock) Init(ctx PlanContext) *LogicalLock { +func (p LogicalLock) Init(ctx base2.PlanContext) *LogicalLock { p.baseLogicalPlan = newBaseLogicalPlan(ctx, plancodec.TypeLock, &p, 0) return &p } // Init initializes PhysicalLock. -func (p PhysicalLock) Init(ctx PlanContext, stats *property.StatsInfo, props ...*property.PhysicalProperty) *PhysicalLock { +func (p PhysicalLock) Init(ctx base2.PlanContext, stats *property.StatsInfo, props ...*property.PhysicalProperty) *PhysicalLock { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeLock, &p, 0) p.childrenReqProps = props p.SetStats(stats) @@ -306,32 +307,32 @@ func (p PhysicalLock) Init(ctx PlanContext, stats *property.StatsInfo, props ... } // Init initializes PhysicalTableScan. -func (p PhysicalTableScan) Init(ctx PlanContext, offset int) *PhysicalTableScan { +func (p PhysicalTableScan) Init(ctx base2.PlanContext, offset int) *PhysicalTableScan { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeTableScan, &p, offset) return &p } // Init initializes PhysicalIndexScan. -func (p PhysicalIndexScan) Init(ctx PlanContext, offset int) *PhysicalIndexScan { +func (p PhysicalIndexScan) Init(ctx base2.PlanContext, offset int) *PhysicalIndexScan { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeIdxScan, &p, offset) return &p } // Init initializes LogicalMemTable. -func (p LogicalMemTable) Init(ctx PlanContext, offset int) *LogicalMemTable { +func (p LogicalMemTable) Init(ctx base2.PlanContext, offset int) *LogicalMemTable { p.baseLogicalPlan = newBaseLogicalPlan(ctx, plancodec.TypeMemTableScan, &p, offset) return &p } // Init initializes PhysicalMemTable. -func (p PhysicalMemTable) Init(ctx PlanContext, stats *property.StatsInfo, offset int) *PhysicalMemTable { +func (p PhysicalMemTable) Init(ctx base2.PlanContext, stats *property.StatsInfo, offset int) *PhysicalMemTable { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeMemTableScan, &p, offset) p.SetStats(stats) return &p } // Init initializes PhysicalHashJoin. -func (p PhysicalHashJoin) Init(ctx PlanContext, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalHashJoin { +func (p PhysicalHashJoin) Init(ctx base2.PlanContext, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalHashJoin { tp := plancodec.TypeHashJoin p.basePhysicalPlan = newBasePhysicalPlan(ctx, tp, &p, offset) p.childrenReqProps = props @@ -340,20 +341,20 @@ func (p PhysicalHashJoin) Init(ctx PlanContext, stats *property.StatsInfo, offse } // Init initializes PhysicalMergeJoin. -func (p PhysicalMergeJoin) Init(ctx PlanContext, stats *property.StatsInfo, offset int) *PhysicalMergeJoin { +func (p PhysicalMergeJoin) Init(ctx base2.PlanContext, stats *property.StatsInfo, offset int) *PhysicalMergeJoin { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeMergeJoin, &p, offset) p.SetStats(stats) return &p } // Init initializes basePhysicalAgg. -func (base basePhysicalAgg) Init(ctx PlanContext, stats *property.StatsInfo, offset int) *basePhysicalAgg { +func (base basePhysicalAgg) Init(ctx base2.PlanContext, stats *property.StatsInfo, offset int) *basePhysicalAgg { base.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeHashAgg, &base, offset) base.SetStats(stats) return &base } -func (base basePhysicalAgg) initForHash(ctx PlanContext, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalHashAgg { +func (base basePhysicalAgg) initForHash(ctx base2.PlanContext, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalHashAgg { p := &PhysicalHashAgg{base} p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeHashAgg, p, offset) p.childrenReqProps = props @@ -361,7 +362,7 @@ func (base basePhysicalAgg) initForHash(ctx PlanContext, stats *property.StatsIn return p } -func (base basePhysicalAgg) initForStream(ctx PlanContext, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalStreamAgg { +func (base basePhysicalAgg) initForStream(ctx base2.PlanContext, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalStreamAgg { p := &PhysicalStreamAgg{base} p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeStreamAgg, p, offset) p.childrenReqProps = props @@ -370,7 +371,7 @@ func (base basePhysicalAgg) initForStream(ctx PlanContext, stats *property.Stats } // Init initializes PhysicalApply. -func (p PhysicalApply) Init(ctx PlanContext, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalApply { +func (p PhysicalApply) Init(ctx base2.PlanContext, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalApply { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeApply, &p, offset) p.childrenReqProps = props p.SetStats(stats) @@ -378,7 +379,7 @@ func (p PhysicalApply) Init(ctx PlanContext, stats *property.StatsInfo, offset i } // Init initializes PhysicalUnionScan. -func (p PhysicalUnionScan) Init(ctx PlanContext, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalUnionScan { +func (p PhysicalUnionScan) Init(ctx base2.PlanContext, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalUnionScan { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeUnionScan, &p, offset) p.childrenReqProps = props p.SetStats(stats) @@ -386,7 +387,7 @@ func (p PhysicalUnionScan) Init(ctx PlanContext, stats *property.StatsInfo, offs } // Init initializes PhysicalIndexLookUpReader. -func (p PhysicalIndexLookUpReader) Init(ctx PlanContext, offset int) *PhysicalIndexLookUpReader { +func (p PhysicalIndexLookUpReader) Init(ctx base2.PlanContext, offset int) *PhysicalIndexLookUpReader { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeIndexLookUp, &p, offset) p.TablePlans = flattenPushDownPlan(p.tablePlan) p.IndexPlans = flattenPushDownPlan(p.indexPlan) @@ -395,7 +396,7 @@ func (p PhysicalIndexLookUpReader) Init(ctx PlanContext, offset int) *PhysicalIn } // Init initializes PhysicalIndexMergeReader. -func (p PhysicalIndexMergeReader) Init(ctx PlanContext, offset int) *PhysicalIndexMergeReader { +func (p PhysicalIndexMergeReader) Init(ctx base2.PlanContext, offset int) *PhysicalIndexMergeReader { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeIndexMerge, &p, offset) if p.tablePlan != nil { p.SetStats(p.tablePlan.StatsInfo()) @@ -407,7 +408,7 @@ func (p PhysicalIndexMergeReader) Init(ctx PlanContext, offset int) *PhysicalInd p.SetStats(p.partialPlans[0].StatsInfo().ScaleByExpectCnt(totalRowCount)) p.StatsInfo().StatsVersion = p.partialPlans[0].StatsInfo().StatsVersion } - p.PartialPlans = make([][]PhysicalPlan, 0, len(p.partialPlans)) + p.PartialPlans = make([][]base2.PhysicalPlan, 0, len(p.partialPlans)) for _, partialPlan := range p.partialPlans { tempPlans := flattenPushDownPlan(partialPlan) p.PartialPlans = append(p.PartialPlans, tempPlans) @@ -436,7 +437,7 @@ func (p PhysicalIndexMergeReader) Init(ctx PlanContext, offset int) *PhysicalInd return &p } -func (p *PhysicalTableReader) adjustReadReqType(ctx PlanContext) { +func (p *PhysicalTableReader) adjustReadReqType(ctx base2.PlanContext) { if p.StoreType == kv.TiFlash { _, ok := p.tablePlan.(*PhysicalExchangeSender) if ok { @@ -472,7 +473,7 @@ func (p *PhysicalTableReader) adjustReadReqType(ctx PlanContext) { } // Init initializes PhysicalTableReader. -func (p PhysicalTableReader) Init(ctx PlanContext, offset int) *PhysicalTableReader { +func (p PhysicalTableReader) Init(ctx base2.PlanContext, offset int) *PhysicalTableReader { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeTableReader, &p, offset) p.ReadReqType = Cop if p.tablePlan == nil { @@ -488,7 +489,7 @@ func (p PhysicalTableReader) Init(ctx PlanContext, offset int) *PhysicalTableRea } // Init initializes PhysicalTableSample. -func (p PhysicalTableSample) Init(ctx PlanContext, offset int) *PhysicalTableSample { +func (p PhysicalTableSample) Init(ctx base2.PlanContext, offset int) *PhysicalTableSample { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeTableSample, &p, offset) p.SetStats(&property.StatsInfo{RowCount: 1}) return &p @@ -508,14 +509,14 @@ func (p *PhysicalTableSample) MemoryUsage() (sum int64) { } // Init initializes PhysicalIndexReader. -func (p PhysicalIndexReader) Init(ctx PlanContext, offset int) *PhysicalIndexReader { +func (p PhysicalIndexReader) Init(ctx base2.PlanContext, offset int) *PhysicalIndexReader { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeIndexReader, &p, offset) p.SetSchema(nil) return &p } // Init initializes PhysicalIndexJoin. -func (p PhysicalIndexJoin) Init(ctx PlanContext, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalIndexJoin { +func (p PhysicalIndexJoin) Init(ctx base2.PlanContext, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalIndexJoin { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeIndexJoin, &p, offset) p.childrenReqProps = props p.SetStats(stats) @@ -523,7 +524,7 @@ func (p PhysicalIndexJoin) Init(ctx PlanContext, stats *property.StatsInfo, offs } // Init initializes PhysicalIndexMergeJoin. -func (p PhysicalIndexMergeJoin) Init(ctx PlanContext) *PhysicalIndexMergeJoin { +func (p PhysicalIndexMergeJoin) Init(ctx base2.PlanContext) *PhysicalIndexMergeJoin { p.SetTP(plancodec.TypeIndexMergeJoin) p.SetID(int(ctx.GetSessionVars().PlanID.Add(1))) p.SetSCtx(ctx) @@ -532,7 +533,7 @@ func (p PhysicalIndexMergeJoin) Init(ctx PlanContext) *PhysicalIndexMergeJoin { } // Init initializes PhysicalIndexHashJoin. -func (p PhysicalIndexHashJoin) Init(ctx PlanContext) *PhysicalIndexHashJoin { +func (p PhysicalIndexHashJoin) Init(ctx base2.PlanContext) *PhysicalIndexHashJoin { p.SetTP(plancodec.TypeIndexHashJoin) p.SetID(int(ctx.GetSessionVars().PlanID.Add(1))) p.SetSCtx(ctx) @@ -541,7 +542,7 @@ func (p PhysicalIndexHashJoin) Init(ctx PlanContext) *PhysicalIndexHashJoin { } // Init initializes BatchPointGetPlan. -func (p *BatchPointGetPlan) Init(ctx PlanContext, stats *property.StatsInfo, schema *expression.Schema, names []*types.FieldName, offset int) *BatchPointGetPlan { +func (p *BatchPointGetPlan) Init(ctx base2.PlanContext, stats *property.StatsInfo, schema *expression.Schema, names []*types.FieldName, offset int) *BatchPointGetPlan { p.Plan = base.NewBasePlan(ctx, plancodec.TypeBatchPointGet, offset) p.schema = schema p.names = names @@ -552,7 +553,7 @@ func (p *BatchPointGetPlan) Init(ctx PlanContext, stats *property.StatsInfo, sch } // Init initializes PointGetPlan. -func (p PointGetPlan) Init(ctx PlanContext, stats *property.StatsInfo, offset int, _ ...*property.PhysicalProperty) *PointGetPlan { +func (p PointGetPlan) Init(ctx base2.PlanContext, stats *property.StatsInfo, offset int, _ ...*property.PhysicalProperty) *PointGetPlan { p.Plan = base.NewBasePlan(ctx, plancodec.TypePointGet, offset) p.SetStats(stats) p.Columns = ExpandVirtualColumn(p.Columns, p.schema, p.TblInfo.Columns) @@ -560,20 +561,20 @@ func (p PointGetPlan) Init(ctx PlanContext, stats *property.StatsInfo, offset in } // Init only assigns type and context. -func (p PhysicalExchangeSender) Init(ctx PlanContext, stats *property.StatsInfo) *PhysicalExchangeSender { +func (p PhysicalExchangeSender) Init(ctx base2.PlanContext, stats *property.StatsInfo) *PhysicalExchangeSender { p.Plan = base.NewBasePlan(ctx, plancodec.TypeExchangeSender, 0) p.SetStats(stats) return &p } // Init only assigns type and context. -func (p PhysicalExchangeReceiver) Init(ctx PlanContext, stats *property.StatsInfo) *PhysicalExchangeReceiver { +func (p PhysicalExchangeReceiver) Init(ctx base2.PlanContext, stats *property.StatsInfo) *PhysicalExchangeReceiver { p.Plan = base.NewBasePlan(ctx, plancodec.TypeExchangeReceiver, 0) p.SetStats(stats) return &p } -func flattenTreePlan(plan PhysicalPlan, plans []PhysicalPlan) []PhysicalPlan { +func flattenTreePlan(plan base2.PhysicalPlan, plans []base2.PhysicalPlan) []base2.PhysicalPlan { plans = append(plans, plan) for _, child := range plan.Children() { plans = flattenTreePlan(child, plans) @@ -582,8 +583,8 @@ func flattenTreePlan(plan PhysicalPlan, plans []PhysicalPlan) []PhysicalPlan { } // flattenPushDownPlan converts a plan tree to a list, whose head is the leaf node like table scan. -func flattenPushDownPlan(p PhysicalPlan) []PhysicalPlan { - plans := make([]PhysicalPlan, 0, 5) +func flattenPushDownPlan(p base2.PhysicalPlan) []base2.PhysicalPlan { + plans := make([]base2.PhysicalPlan, 0, 5) plans = flattenTreePlan(p, plans) for i := 0; i < len(plans)/2; i++ { j := len(plans) - i - 1 @@ -593,53 +594,53 @@ func flattenPushDownPlan(p PhysicalPlan) []PhysicalPlan { } // Init only assigns type and context. -func (p LogicalCTE) Init(ctx PlanContext, offset int) *LogicalCTE { +func (p LogicalCTE) Init(ctx base2.PlanContext, offset int) *LogicalCTE { p.baseLogicalPlan = newBaseLogicalPlan(ctx, plancodec.TypeCTE, &p, offset) return &p } // Init only assigns type and context. -func (p PhysicalCTE) Init(ctx PlanContext, stats *property.StatsInfo) *PhysicalCTE { +func (p PhysicalCTE) Init(ctx base2.PlanContext, stats *property.StatsInfo) *PhysicalCTE { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeCTE, &p, 0) p.SetStats(stats) return &p } // Init only assigns type and context. -func (p LogicalCTETable) Init(ctx PlanContext, offset int) *LogicalCTETable { +func (p LogicalCTETable) Init(ctx base2.PlanContext, offset int) *LogicalCTETable { p.baseLogicalPlan = newBaseLogicalPlan(ctx, plancodec.TypeCTETable, &p, offset) return &p } // Init only assigns type and context. -func (p PhysicalCTETable) Init(ctx PlanContext, stats *property.StatsInfo) *PhysicalCTETable { +func (p PhysicalCTETable) Init(ctx base2.PlanContext, stats *property.StatsInfo) *PhysicalCTETable { p.Plan = base.NewBasePlan(ctx, plancodec.TypeCTETable, 0) p.SetStats(stats) return &p } // Init initializes FKCheck. -func (p FKCheck) Init(ctx PlanContext) *FKCheck { +func (p FKCheck) Init(ctx base2.PlanContext) *FKCheck { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeForeignKeyCheck, &p, 0) p.SetStats(&property.StatsInfo{}) return &p } // Init initializes FKCascade -func (p FKCascade) Init(ctx PlanContext) *FKCascade { +func (p FKCascade) Init(ctx base2.PlanContext) *FKCascade { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeForeignKeyCascade, &p, 0) p.SetStats(&property.StatsInfo{}) return &p } // Init initializes LogicalSequence -func (p LogicalSequence) Init(ctx PlanContext, offset int) *LogicalSequence { +func (p LogicalSequence) Init(ctx base2.PlanContext, offset int) *LogicalSequence { p.baseLogicalPlan = newBaseLogicalPlan(ctx, plancodec.TypeSequence, &p, offset) return &p } // Init initializes PhysicalSequence -func (p PhysicalSequence) Init(ctx PlanContext, stats *property.StatsInfo, blockOffset int, props ...*property.PhysicalProperty) *PhysicalSequence { +func (p PhysicalSequence) Init(ctx base2.PlanContext, stats *property.StatsInfo, blockOffset int, props ...*property.PhysicalProperty) *PhysicalSequence { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeSequence, &p, blockOffset) p.SetStats(stats) p.childrenReqProps = props @@ -647,7 +648,7 @@ func (p PhysicalSequence) Init(ctx PlanContext, stats *property.StatsInfo, block } // Init initializes ScalarSubqueryEvalCtx -func (p ScalarSubqueryEvalCtx) Init(ctx PlanContext, offset int) *ScalarSubqueryEvalCtx { +func (p ScalarSubqueryEvalCtx) Init(ctx base2.PlanContext, offset int) *ScalarSubqueryEvalCtx { p.Plan = base.NewBasePlan(ctx, plancodec.TypeScalarSubQuery, offset) return &p } diff --git a/pkg/planner/core/issuetest/BUILD.bazel b/pkg/planner/core/issuetest/BUILD.bazel index a08f63c0df355..41e03b9c6d4e3 100644 --- a/pkg/planner/core/issuetest/BUILD.bazel +++ b/pkg/planner/core/issuetest/BUILD.bazel @@ -14,6 +14,7 @@ go_test( "//pkg/parser", "//pkg/planner", "//pkg/planner/core", + "//pkg/planner/core/base", "//pkg/testkit", "//pkg/testkit/testdata", "//pkg/testkit/testmain", diff --git a/pkg/planner/core/issuetest/planner_issue_test.go b/pkg/planner/core/issuetest/planner_issue_test.go index be9e5ef29d971..68abab69dc621 100644 --- a/pkg/planner/core/issuetest/planner_issue_test.go +++ b/pkg/planner/core/issuetest/planner_issue_test.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/planner" "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/testkit" "github.com/stretchr/testify/require" ) @@ -48,7 +49,7 @@ func TestIssue43461(t *testing.T) { if ok { break } - p = p.(core.PhysicalPlan).Children()[0] + p = p.(base.PhysicalPlan).Children()[0] } require.True(t, ok) diff --git a/pkg/planner/core/logical_plan_builder.go b/pkg/planner/core/logical_plan_builder.go index 7c3eefc2692c2..c62139bdb47f6 100644 --- a/pkg/planner/core/logical_plan_builder.go +++ b/pkg/planner/core/logical_plan_builder.go @@ -40,6 +40,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/opcode" "github.com/pingcap/tidb/pkg/parser/terror" + "github.com/pingcap/tidb/pkg/planner/core/base" core_metrics "github.com/pingcap/tidb/pkg/planner/core/metrics" fd "github.com/pingcap/tidb/pkg/planner/funcdep" "github.com/pingcap/tidb/pkg/planner/property" @@ -81,7 +82,7 @@ const ( // aggOrderByResolver is currently resolving expressions of order by clause // in aggregate function GROUP_CONCAT. type aggOrderByResolver struct { - ctx PlanContext + ctx base.PlanContext err error args []ast.ExprNode exprDepth int // exprDepth is the depth of current expression in expression tree. @@ -625,7 +626,7 @@ func (p *LogicalJoin) ExtractOnCondition( // extractTableAlias returns table alias of the LogicalPlan's columns. // It will return nil when there are multiple table alias, because the alias is only used to check if // the logicalPlan Match some optimizer hints, and hints are not expected to take effect in this case. -func extractTableAlias(p Plan, parentOffset int) *h.HintedTable { +func extractTableAlias(p base.Plan, parentOffset int) *h.HintedTable { if len(p.OutputNames()) > 0 && p.OutputNames()[0].TblName.L != "" { firstName := p.OutputNames()[0] for _, name := range p.OutputNames() { @@ -2378,7 +2379,7 @@ CheckReferenced: // getUintFromNode gets uint64 value from ast.Node. // For ordinary statement, node should be uint64 constant value. // For prepared statement, node is string. We should convert it to uint64. -func getUintFromNode(ctx PlanContext, n ast.Node, mustInt64orUint64 bool) (uVal uint64, isNull bool, isExpectedType bool) { +func getUintFromNode(ctx base.PlanContext, n ast.Node, mustInt64orUint64 bool) (uVal uint64, isNull bool, isExpectedType bool) { var val any switch v := n.(type) { case *driver.ValueExpr: @@ -2440,7 +2441,7 @@ func CheckParamTypeInt64orUint64(param *driver.ParamMarkerExpr) (bool, uint64) { return false, 0 } -func extractLimitCountOffset(ctx PlanContext, limit *ast.Limit) (count uint64, +func extractLimitCountOffset(ctx base.PlanContext, limit *ast.Limit) (count uint64, offset uint64, err error) { var isExpectedType bool if limit.Count != nil { @@ -3197,7 +3198,7 @@ func (b *PlanBuilder) resolveCorrelatedAggregates(ctx context.Context, sel *ast. // gbyResolver resolves group by items from select fields. type gbyResolver struct { - ctx PlanContext + ctx base.PlanContext fields []*ast.SelectField schema *expression.Schema names []*types.FieldName @@ -4425,7 +4426,7 @@ func (ds *DataSource) AddExtraPhysTblIDColumn() *expression.Column { // 2. table row count from statistics is zero. // 3. statistics is outdated. // Note: please also update getLatestVersionFromStatsTable() when logic in this function changes. -func getStatsTable(ctx PlanContext, tblInfo *model.TableInfo, pid int64) *statistics.Table { +func getStatsTable(ctx base.PlanContext, tblInfo *model.TableInfo, pid int64) *statistics.Table { statsHandle := domain.GetDomain(ctx).StatsHandle() var usePartitionStats, countIs0, pseudoStatsForUninitialized, pseudoStatsForOutdated bool var statsTbl *statistics.Table @@ -5479,7 +5480,7 @@ func (b *PlanBuilder) BuildDataSourceFromView(ctx context.Context, dbName model. return b.buildProjUponView(ctx, dbName, tableInfo, selectLogicalPlan) } -func (b *PlanBuilder) buildProjUponView(_ context.Context, dbName model.CIStr, tableInfo *model.TableInfo, selectLogicalPlan Plan) (LogicalPlan, error) { +func (b *PlanBuilder) buildProjUponView(_ context.Context, dbName model.CIStr, tableInfo *model.TableInfo, selectLogicalPlan base.Plan) (LogicalPlan, error) { columnInfo := tableInfo.Cols() cols := selectLogicalPlan.Schema().Clone().Columns outputNamesOfUnderlyingSelect := selectLogicalPlan.OutputNames().Shallow() @@ -5725,7 +5726,7 @@ func buildColumns2Handle( return cols2Handles, nil } -func (b *PlanBuilder) buildUpdate(ctx context.Context, update *ast.UpdateStmt) (Plan, error) { +func (b *PlanBuilder) buildUpdate(ctx context.Context, update *ast.UpdateStmt) (base.Plan, error) { b.pushSelectOffset(0) b.pushTableHints(update.TableHints, 0) defer func() { @@ -6132,7 +6133,7 @@ func IsDefaultExprSameColumn(names types.NameSlice, node ast.ExprNode) bool { return false } -func (b *PlanBuilder) buildDelete(ctx context.Context, ds *ast.DeleteStmt) (Plan, error) { +func (b *PlanBuilder) buildDelete(ctx context.Context, ds *ast.DeleteStmt) (base.Plan, error) { b.pushSelectOffset(0) b.pushTableHints(ds.TableHints, 0) defer func() { diff --git a/pkg/planner/core/logical_plans.go b/pkg/planner/core/logical_plans.go index 5934c3cf78a63..3def077d74f0d 100644 --- a/pkg/planner/core/logical_plans.go +++ b/pkg/planner/core/logical_plans.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/cardinality" + "github.com/pingcap/tidb/pkg/planner/core/base" fd "github.com/pingcap/tidb/pkg/planner/funcdep" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util" @@ -1190,7 +1191,7 @@ func extractNotNullFromConds(conditions []expression.Expression, p LogicalPlan) return notnullColsUniqueIDs } -func extractConstantCols(conditions []expression.Expression, sctx PlanContext, fds *fd.FDSet) intset.FastIntSet { +func extractConstantCols(conditions []expression.Expression, sctx base.PlanContext, fds *fd.FDSet) intset.FastIntSet { // extract constant cols // eg: where a=1 and b is null and (1+c)=5. // TODO: Some columns can only be determined to be constant from multiple constraints (e.g. x <= 1 AND x >= 1) @@ -1217,7 +1218,7 @@ func extractConstantCols(conditions []expression.Expression, sctx PlanContext, f return constUniqueIDs } -func extractEquivalenceCols(conditions []expression.Expression, sctx PlanContext, fds *fd.FDSet) [][]intset.FastIntSet { +func extractEquivalenceCols(conditions []expression.Expression, sctx base.PlanContext, fds *fd.FDSet) [][]intset.FastIntSet { var equivObjsPair [][]expression.Expression equivObjsPair = expression.ExtractEquivalenceColumns(equivObjsPair, conditions) equivUniqueIDs := make([][]intset.FastIntSet, 0, len(equivObjsPair)) @@ -1622,7 +1623,7 @@ func (ds *DataSource) Convert2Gathers() (gathers []LogicalPlan) { } func detachCondAndBuildRangeForPath( - sctx PlanContext, + sctx base.PlanContext, path *util.AccessPath, conds []expression.Expression, histColl *statistics.HistColl, @@ -2231,7 +2232,7 @@ func extractCorColumnsBySchema4LogicalPlan(p LogicalPlan, schema *expression.Sch // ExtractCorColumnsBySchema4PhysicalPlan only extracts the correlated columns that match the specified schema. // e.g. If the correlated columns from plan are [t1.a, t2.a, t3.a] and specified schema is [t2.a, t2.b, t2.c], // only [t2.a] is returned. -func ExtractCorColumnsBySchema4PhysicalPlan(p PhysicalPlan, schema *expression.Schema) []*expression.CorrelatedColumn { +func ExtractCorColumnsBySchema4PhysicalPlan(p base.PhysicalPlan, schema *expression.Schema) []*expression.CorrelatedColumn { corCols := ExtractCorrelatedCols4PhysicalPlan(p) return ExtractCorColumnsBySchema(corCols, schema, true) } @@ -2297,8 +2298,8 @@ type CTEClass struct { seedPartLogicalPlan LogicalPlan recursivePartLogicalPlan LogicalPlan // seedPartPhysicalPlan and recursivePartPhysicalPlan are the physical plans for the seed part and recursive part of this CTE. - seedPartPhysicalPlan PhysicalPlan - recursivePartPhysicalPlan PhysicalPlan + seedPartPhysicalPlan base.PhysicalPlan + recursivePartPhysicalPlan base.PhysicalPlan // storageID for this CTE. IDForStorage int // optFlag is the optFlag for the whole CTE. diff --git a/pkg/planner/core/logical_plans_test.go b/pkg/planner/core/logical_plans_test.go index bcc6f77ba2a2d..b35f647b5a967 100644 --- a/pkg/planner/core/logical_plans_test.go +++ b/pkg/planner/core/logical_plans_test.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/sessionctx" @@ -47,7 +48,7 @@ type plannerSuite struct { p *parser.Parser is infoschema.InfoSchema sctx sessionctx.Context - ctx PlanContext + ctx base.PlanContext } func (p *plannerSuite) GetParser() *parser.Parser { @@ -392,7 +393,7 @@ func TestExtraPKNotNullFlag(t *testing.T) { require.Equal(t, mysql.PriKeyFlag|mysql.NotNullFlag, ds.schema.Columns[2].RetType.GetFlag()) } -func buildLogicPlan4GroupBy(s *plannerSuite, t *testing.T, sql string) (Plan, error) { +func buildLogicPlan4GroupBy(s *plannerSuite, t *testing.T, sql string) (base.Plan, error) { sqlMode := s.ctx.GetSessionVars().SQLMode mockedTableInfo := MockSignedTable() // mock the table info here for later use @@ -1845,7 +1846,7 @@ func (s *plannerSuiteWithOptimizeVars) doTestWindowFunction(t *testing.T, input, } } -func (s *plannerSuiteWithOptimizeVars) optimize(ctx context.Context, sql string) (PhysicalPlan, ast.Node, error) { +func (s *plannerSuiteWithOptimizeVars) optimize(ctx context.Context, sql string) (base.PhysicalPlan, ast.Node, error) { stmt, err := s.p.ParseOneStmt(sql, "", "") if err != nil { return nil, nil, err @@ -1875,7 +1876,7 @@ func (s *plannerSuiteWithOptimizeVars) optimize(ctx context.Context, sql string) return nil, nil, err } p, _, err = physicalOptimize(p.(LogicalPlan), &PlanCounterDisabled) - return p.(PhysicalPlan), stmt, err + return p.(base.PhysicalPlan), stmt, err } func byItemsToProperty(byItems []*util.ByItems) *property.PhysicalProperty { diff --git a/pkg/planner/core/memtable_predicate_extractor.go b/pkg/planner/core/memtable_predicate_extractor.go index 762c1782548af..aa48a274a539e 100644 --- a/pkg/planner/core/memtable_predicate_extractor.go +++ b/pkg/planner/core/memtable_predicate_extractor.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/tablecodec" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" @@ -58,7 +59,7 @@ import ( // 4. Executor sends requests to the target components instead of all of the components type MemTablePredicateExtractor interface { // Extracts predicates which can be pushed down and returns the remained predicates - Extract(PlanContext, *expression.Schema, []*types.FieldName, []expression.Expression) (remained []expression.Expression) + Extract(base.PlanContext, *expression.Schema, []*types.FieldName, []expression.Expression) (remained []expression.Expression) explainInfo(p *PhysicalMemTable) string } @@ -529,7 +530,7 @@ func (extractHelper) getStringFunctionName(fn *expression.ScalarFunction) string // SELECT * FROM t WHERE time='2019-10-10 10:10:10' // SELECT * FROM t WHERE time>'2019-10-10 10:10:10' AND time<'2019-10-11 10:10:10' func (helper extractHelper) extractTimeRange( - ctx PlanContext, + ctx base.PlanContext, schema *expression.Schema, names []*types.FieldName, predicates []expression.Expression, @@ -718,7 +719,7 @@ type ClusterTableExtractor struct { } // Extract implements the MemTablePredicateExtractor Extract interface -func (e *ClusterTableExtractor) Extract(_ PlanContext, +func (e *ClusterTableExtractor) Extract(_ base.PlanContext, schema *expression.Schema, names []*types.FieldName, predicates []expression.Expression, @@ -784,7 +785,7 @@ type ClusterLogTableExtractor struct { } // Extract implements the MemTablePredicateExtractor Extract interface -func (e *ClusterLogTableExtractor) Extract(ctx PlanContext, +func (e *ClusterLogTableExtractor) Extract(ctx base.PlanContext, schema *expression.Schema, names []*types.FieldName, predicates []expression.Expression, @@ -897,7 +898,7 @@ type HotRegionsHistoryTableExtractor struct { } // Extract implements the MemTablePredicateExtractor Extract interface -func (e *HotRegionsHistoryTableExtractor) Extract(ctx PlanContext, +func (e *HotRegionsHistoryTableExtractor) Extract(ctx base.PlanContext, schema *expression.Schema, names []*types.FieldName, predicates []expression.Expression, @@ -1015,7 +1016,7 @@ func newMetricTableExtractor() *MetricTableExtractor { } // Extract implements the MemTablePredicateExtractor Extract interface -func (e *MetricTableExtractor) Extract(ctx PlanContext, +func (e *MetricTableExtractor) Extract(ctx base.PlanContext, schema *expression.Schema, names []*types.FieldName, predicates []expression.Expression, @@ -1085,7 +1086,7 @@ func (e *MetricTableExtractor) explainInfo(p *PhysicalMemTable) string { } // GetMetricTablePromQL uses to get the promQL of metric table. -func (e *MetricTableExtractor) GetMetricTablePromQL(sctx PlanContext, lowerTableName string) string { +func (e *MetricTableExtractor) GetMetricTablePromQL(sctx base.PlanContext, lowerTableName string) string { quantiles := e.Quantiles def, err := infoschema.GetMetricTableDef(lowerTableName) if err != nil { @@ -1115,7 +1116,7 @@ type MetricSummaryTableExtractor struct { } // Extract implements the MemTablePredicateExtractor Extract interface -func (e *MetricSummaryTableExtractor) Extract(_ PlanContext, +func (e *MetricSummaryTableExtractor) Extract(_ base.PlanContext, schema *expression.Schema, names []*types.FieldName, predicates []expression.Expression, @@ -1147,7 +1148,7 @@ type InspectionResultTableExtractor struct { } // Extract implements the MemTablePredicateExtractor Extract interface -func (e *InspectionResultTableExtractor) Extract(_ PlanContext, +func (e *InspectionResultTableExtractor) Extract(_ base.PlanContext, schema *expression.Schema, names []*types.FieldName, predicates []expression.Expression, @@ -1184,7 +1185,7 @@ type InspectionSummaryTableExtractor struct { } // Extract implements the MemTablePredicateExtractor Extract interface -func (e *InspectionSummaryTableExtractor) Extract(_ PlanContext, +func (e *InspectionSummaryTableExtractor) Extract(_ base.PlanContext, schema *expression.Schema, names []*types.FieldName, predicates []expression.Expression, @@ -1242,7 +1243,7 @@ type InspectionRuleTableExtractor struct { } // Extract implements the MemTablePredicateExtractor Extract interface -func (e *InspectionRuleTableExtractor) Extract(_ PlanContext, +func (e *InspectionRuleTableExtractor) Extract(_ base.PlanContext, schema *expression.Schema, names []*types.FieldName, predicates []expression.Expression, @@ -1286,7 +1287,7 @@ type TimeRange struct { } // Extract implements the MemTablePredicateExtractor Extract interface -func (e *SlowQueryExtractor) Extract(ctx PlanContext, +func (e *SlowQueryExtractor) Extract(ctx base.PlanContext, schema *expression.Schema, names []*types.FieldName, predicates []expression.Expression, @@ -1389,7 +1390,7 @@ type TableStorageStatsExtractor struct { } // Extract implements the MemTablePredicateExtractor Extract interface. -func (e *TableStorageStatsExtractor) Extract(_ PlanContext, +func (e *TableStorageStatsExtractor) Extract(_ base.PlanContext, schema *expression.Schema, names []*types.FieldName, predicates []expression.Expression, @@ -1459,7 +1460,7 @@ type TiFlashSystemTableExtractor struct { } // Extract implements the MemTablePredicateExtractor Extract interface -func (e *TiFlashSystemTableExtractor) Extract(_ PlanContext, +func (e *TiFlashSystemTableExtractor) Extract(_ base.PlanContext, schema *expression.Schema, names []*types.FieldName, predicates []expression.Expression, @@ -1521,7 +1522,7 @@ type StatementsSummaryExtractor struct { } // Extract implements the MemTablePredicateExtractor Extract interface -func (e *StatementsSummaryExtractor) Extract(sctx PlanContext, +func (e *StatementsSummaryExtractor) Extract(sctx base.PlanContext, schema *expression.Schema, names []*types.FieldName, predicates []expression.Expression, @@ -1574,7 +1575,7 @@ func (e *StatementsSummaryExtractor) explainInfo(p *PhysicalMemTable) string { } func (e *StatementsSummaryExtractor) findCoarseTimeRange( - sctx PlanContext, + sctx base.PlanContext, schema *expression.Schema, names []*types.FieldName, predicates []expression.Expression, @@ -1622,7 +1623,7 @@ type TikvRegionPeersExtractor struct { } // Extract implements the MemTablePredicateExtractor Extract interface -func (e *TikvRegionPeersExtractor) Extract(_ PlanContext, +func (e *TikvRegionPeersExtractor) Extract(_ base.PlanContext, schema *expression.Schema, names []*types.FieldName, predicates []expression.Expression, @@ -1680,7 +1681,7 @@ type ColumnsTableExtractor struct { } // Extract implements the MemTablePredicateExtractor Extract interface -func (e *ColumnsTableExtractor) Extract(_ PlanContext, +func (e *ColumnsTableExtractor) Extract(_ base.PlanContext, schema *expression.Schema, names []*types.FieldName, predicates []expression.Expression, @@ -1743,7 +1744,7 @@ type TiKVRegionStatusExtractor struct { } // Extract implements the MemTablePredicateExtractor Extract interface -func (e *TiKVRegionStatusExtractor) Extract(_ PlanContext, +func (e *TiKVRegionStatusExtractor) Extract(_ base.PlanContext, schema *expression.Schema, names []*types.FieldName, predicates []expression.Expression, @@ -1797,7 +1798,7 @@ type InfoSchemaTablesExtractor struct { } // Extract implements the MemTablePredicateExtractor Extract interface -func (e *InfoSchemaTablesExtractor) Extract(_ PlanContext, +func (e *InfoSchemaTablesExtractor) Extract(_ base.PlanContext, schema *expression.Schema, names []*types.FieldName, predicates []expression.Expression, diff --git a/pkg/planner/core/optimizer.go b/pkg/planner/core/optimizer.go index 1ea82e71ba9aa..d0573f6d23069 100644 --- a/pkg/planner/core/optimizer.go +++ b/pkg/planner/core/optimizer.go @@ -38,6 +38,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/auth" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/property" plannerutil "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/planner/util/coreusage" @@ -58,7 +59,7 @@ import ( ) // OptimizeAstNode optimizes the query to a physical plan directly. -var OptimizeAstNode func(ctx context.Context, sctx sessionctx.Context, node ast.Node, is infoschema.InfoSchema) (Plan, types.NameSlice, error) +var OptimizeAstNode func(ctx context.Context, sctx sessionctx.Context, node ast.Node, is infoschema.InfoSchema) (base.Plan, types.NameSlice, error) // AllowCartesianProduct means whether tidb allows cartesian join without equal conditions. var AllowCartesianProduct = atomic.NewBool(true) @@ -143,7 +144,7 @@ type logicalOptRule interface { } // BuildLogicalPlanForTest builds a logical plan for testing purpose from ast.Node. -func BuildLogicalPlanForTest(ctx context.Context, sctx sessionctx.Context, node ast.Node, infoSchema infoschema.InfoSchema) (Plan, error) { +func BuildLogicalPlanForTest(ctx context.Context, sctx sessionctx.Context, node ast.Node, infoSchema infoschema.InfoSchema) (base.Plan, error) { sctx.GetSessionVars().PlanID.Store(0) sctx.GetSessionVars().PlanColumnID.Store(0) builder, _ := NewPlanBuilder().Init(sctx.GetPlanCtx(), infoSchema, utilhint.NewQBHintHandler(nil)) @@ -267,7 +268,7 @@ func CheckTableLock(ctx tablelock.TableLockReadContext, is infoschema.InfoSchema return nil } -func checkStableResultMode(sctx PlanContext) bool { +func checkStableResultMode(sctx base.PlanContext) bool { s := sctx.GetSessionVars() st := s.StmtCtx return s.EnableStableResultMode && (!st.InInsertStmt && !st.InUpdateStmt && !st.InDeleteStmt && !st.InLoadDataStmt) @@ -278,10 +279,10 @@ func checkStableResultMode(sctx PlanContext) bool { // The returned logical plan is necessary for generating plans for Common Table Expressions (CTEs). func doOptimize( ctx context.Context, - sctx PlanContext, + sctx base.PlanContext, flag uint64, logic LogicalPlan, -) (LogicalPlan, PhysicalPlan, float64, error) { +) (LogicalPlan, base.PhysicalPlan, float64, error) { sessVars := sctx.GetSessionVars() flag = adjustOptimizationFlags(flag, logic) logic, err := logicalOptimize(ctx, flag, logic) @@ -337,10 +338,10 @@ func adjustOptimizationFlags(flag uint64, logic LogicalPlan) uint64 { // DoOptimize optimizes a logical plan to a physical plan. func DoOptimize( ctx context.Context, - sctx PlanContext, + sctx base.PlanContext, flag uint64, logic LogicalPlan, -) (PhysicalPlan, float64, error) { +) (base.PhysicalPlan, float64, error) { sessVars := sctx.GetSessionVars() if sessVars.StmtCtx.EnableOptimizerDebugTrace { debugtrace.EnterContextCommon(sctx) @@ -352,7 +353,7 @@ func DoOptimize( // refineCETrace will adjust the content of CETrace. // Currently, it will (1) deduplicate trace records, (2) sort the trace records (to make it easier in the tests) and (3) fill in the table name. -func refineCETrace(sctx PlanContext) { +func refineCETrace(sctx base.PlanContext) { stmtCtx := sctx.GetSessionVars().StmtCtx stmtCtx.OptimizerCETrace = tracing.DedupCETrace(stmtCtx.OptimizerCETrace) slices.SortFunc(stmtCtx.OptimizerCETrace, func(i, j *tracing.CETraceRecord) int { @@ -388,7 +389,7 @@ func refineCETrace(sctx PlanContext) { } // mergeContinuousSelections merge continuous selections which may occur after changing plans. -func mergeContinuousSelections(p PhysicalPlan) { +func mergeContinuousSelections(p base.PhysicalPlan) { if sel, ok := p.(*PhysicalSelection); ok { for { childSel := sel.children[0] @@ -411,7 +412,7 @@ func mergeContinuousSelections(p PhysicalPlan) { } } -func postOptimize(ctx context.Context, sctx PlanContext, plan PhysicalPlan) (PhysicalPlan, error) { +func postOptimize(ctx context.Context, sctx base.PlanContext, plan base.PhysicalPlan) (base.PhysicalPlan, error) { // some cases from update optimize will require avoiding projection elimination. // see comments ahead of call of DoOptimize in function of buildUpdate(). err := prunePhysicalColumns(sctx, plan) @@ -432,7 +433,7 @@ func postOptimize(ctx context.Context, sctx PlanContext, plan PhysicalPlan) (Phy return plan, nil } -func generateRuntimeFilter(sctx PlanContext, plan PhysicalPlan) { +func generateRuntimeFilter(sctx base.PlanContext, plan base.PhysicalPlan) { if !sctx.GetSessionVars().IsRuntimeFilterEnabled() || sctx.GetSessionVars().InRestrictedSQL { return } @@ -451,7 +452,7 @@ func generateRuntimeFilter(sctx PlanContext, plan PhysicalPlan) { // prunePhysicalColumns currently only work for MPP(HashJoin<-Exchange). // Here add projection instead of pruning columns directly for safety considerations. // And projection is cheap here for it saves the network cost and work in memory. -func prunePhysicalColumns(sctx PlanContext, plan PhysicalPlan) error { +func prunePhysicalColumns(sctx base.PlanContext, plan base.PhysicalPlan) error { if tableReader, ok := plan.(*PhysicalTableReader); ok { if _, isExchangeSender := tableReader.tablePlan.(*PhysicalExchangeSender); isExchangeSender { err := prunePhysicalColumnsInternal(sctx, tableReader.tablePlan) @@ -495,7 +496,7 @@ func (p *PhysicalHashJoin) extractUsedCols(parentUsedCols []*expression.Column) return leftCols, rightCols } -func prunePhysicalColumnForHashJoinChild(sctx PlanContext, hashJoin *PhysicalHashJoin, joinUsedCols []*expression.Column, sender *PhysicalExchangeSender) error { +func prunePhysicalColumnForHashJoinChild(sctx base.PlanContext, hashJoin *PhysicalHashJoin, joinUsedCols []*expression.Column, sender *PhysicalExchangeSender) error { var err error evalCtx := sctx.GetExprCtx().GetEvalCtx() joinUsed := expression.GetUsedList(evalCtx, joinUsedCols, sender.Schema()) @@ -544,7 +545,7 @@ func prunePhysicalColumnForHashJoinChild(sctx PlanContext, hashJoin *PhysicalHas return err } -func prunePhysicalColumnsInternal(sctx PlanContext, plan PhysicalPlan) error { +func prunePhysicalColumnsInternal(sctx base.PlanContext, plan base.PhysicalPlan) error { var err error switch x := plan.(type) { case *PhysicalHashJoin: @@ -596,7 +597,7 @@ func prunePhysicalColumnsInternal(sctx PlanContext, plan PhysicalPlan) error { // - Only the filter conditions with high selectivity should be pushed down. // - The filter conditions which contain heavy cost functions should not be pushed down. // - Filter conditions that apply to the same column are either pushed down or not pushed down at all. -func tryEnableLateMaterialization(sctx PlanContext, plan PhysicalPlan) { +func tryEnableLateMaterialization(sctx base.PlanContext, plan base.PhysicalPlan) { // check if EnableLateMaterialization is set if sctx.GetSessionVars().EnableLateMaterialization && !sctx.GetSessionVars().TiFlashFastScan { predicatePushDownToTableScan(sctx, plan) @@ -637,7 +638,7 @@ countStarRewrite: datasource pick k1 instead of row_id Rewritten Query: select count(k1) from table */ -func countStarRewrite(plan PhysicalPlan) { +func countStarRewrite(plan base.PhysicalPlan) { countStarRewriteInternal(plan) if tableReader, ok := plan.(*PhysicalTableReader); ok { countStarRewrite(tableReader.tablePlan) @@ -648,7 +649,7 @@ func countStarRewrite(plan PhysicalPlan) { } } -func countStarRewriteInternal(plan PhysicalPlan) { +func countStarRewriteInternal(plan base.PhysicalPlan) { // match pattern any agg(count(constant)) -> tablefullscan(tiflash) var physicalAgg *basePhysicalAgg switch x := plan.(type) { @@ -726,7 +727,7 @@ func rewriteTableScanAndAggArgs(physicalTableScan *PhysicalTableScan, aggFuncs [ // < 0: fine grained shuffle is disabled. // > 0: use TiFlashFineGrainedShuffleStreamCount as stream count. // == 0: use TiFlashMaxThreads as stream count when it's greater than 0. Otherwise set status as uninitialized. -func handleFineGrainedShuffle(ctx context.Context, sctx PlanContext, plan PhysicalPlan) { +func handleFineGrainedShuffle(ctx context.Context, sctx base.PlanContext, plan base.PhysicalPlan) { streamCount := sctx.GetSessionVars().TiFlashFineGrainedShuffleStreamCount if streamCount < 0 { return @@ -746,7 +747,7 @@ func handleFineGrainedShuffle(ctx context.Context, sctx PlanContext, plan Physic setupFineGrainedShuffle(ctx, sctx, &streamCountInfo, &tiflashServerCountInfo, plan) } -func setupFineGrainedShuffle(ctx context.Context, sctx PlanContext, streamCountInfo *tiflashClusterInfo, tiflashServerCountInfo *tiflashClusterInfo, plan PhysicalPlan) { +func setupFineGrainedShuffle(ctx context.Context, sctx base.PlanContext, streamCountInfo *tiflashClusterInfo, tiflashServerCountInfo *tiflashClusterInfo, plan base.PhysicalPlan) { if tableReader, ok := plan.(*PhysicalTableReader); ok { if _, isExchangeSender := tableReader.tablePlan.(*PhysicalExchangeSender); isExchangeSender { helper := fineGrainedShuffleHelper{shuffleTarget: unknown, plans: make([]*basePhysicalPlan, 1)} @@ -800,7 +801,7 @@ func (h *fineGrainedShuffleHelper) updateTarget(t shuffleTarget, p *basePhysical // calculateTiFlashStreamCountUsingMinLogicalCores uses minimal logical cpu cores among tiflash servers, and divide by 2 // return false, 0 if any err happens -func calculateTiFlashStreamCountUsingMinLogicalCores(ctx context.Context, sctx PlanContext, serversInfo []infoschema.ServerInfo) (bool, uint64) { +func calculateTiFlashStreamCountUsingMinLogicalCores(ctx context.Context, sctx base.PlanContext, serversInfo []infoschema.ServerInfo) (bool, uint64) { failpoint.Inject("mockTiFlashStreamCountUsingMinLogicalCores", func(val failpoint.Value) { intVal, err := strconv.Atoi(val.(string)) if err == nil { @@ -837,7 +838,7 @@ func calculateTiFlashStreamCountUsingMinLogicalCores(ctx context.Context, sctx P return false, 0 } -func checkFineGrainedShuffleForJoinAgg(ctx context.Context, sctx PlanContext, streamCountInfo *tiflashClusterInfo, tiflashServerCountInfo *tiflashClusterInfo, exchangeColCount int, splitLimit uint64) (applyFlag bool, streamCount uint64) { +func checkFineGrainedShuffleForJoinAgg(ctx context.Context, sctx base.PlanContext, streamCountInfo *tiflashClusterInfo, tiflashServerCountInfo *tiflashClusterInfo, exchangeColCount int, splitLimit uint64) (applyFlag bool, streamCount uint64) { switch (*streamCountInfo).itemStatus { case unInitialized: streamCount = 4 // assume 8c node in cluster as minimal, stream count is 8 / 2 = 4 @@ -897,7 +898,7 @@ func checkFineGrainedShuffleForJoinAgg(ctx context.Context, sctx PlanContext, st return applyFlag, streamCount } -func inferFineGrainedShuffleStreamCountForWindow(ctx context.Context, sctx PlanContext, streamCountInfo *tiflashClusterInfo, tiflashServerCountInfo *tiflashClusterInfo) (streamCount uint64) { +func inferFineGrainedShuffleStreamCountForWindow(ctx context.Context, sctx base.PlanContext, streamCountInfo *tiflashClusterInfo, tiflashServerCountInfo *tiflashClusterInfo) (streamCount uint64) { switch (*streamCountInfo).itemStatus { case unInitialized: if (*tiflashServerCountInfo).itemStatus == failed { @@ -943,7 +944,7 @@ func setDefaultStreamCount(streamCountInfo *tiflashClusterInfo) { (*streamCountInfo).itemValue = variable.DefStreamCountWhenMaxThreadsNotSet } -func setupFineGrainedShuffleInternal(ctx context.Context, sctx PlanContext, plan PhysicalPlan, helper *fineGrainedShuffleHelper, streamCountInfo *tiflashClusterInfo, tiflashServerCountInfo *tiflashClusterInfo) { +func setupFineGrainedShuffleInternal(ctx context.Context, sctx base.PlanContext, plan base.PhysicalPlan, helper *fineGrainedShuffleHelper, streamCountInfo *tiflashClusterInfo, tiflashServerCountInfo *tiflashClusterInfo) { switch x := plan.(type) { case *PhysicalWindow: // Do not clear the plans because window executor will keep the data partition. @@ -1045,7 +1046,7 @@ func setupFineGrainedShuffleInternal(ctx context.Context, sctx PlanContext, plan // propagateProbeParents doesn't affect the execution plan, it only sets the probeParents field of a PhysicalPlan. // It's for handling the inconsistency between row count in the statsInfo and the recorded actual row count. Please // see comments in PhysicalPlan for details. -func propagateProbeParents(plan PhysicalPlan, probeParents []PhysicalPlan) { +func propagateProbeParents(plan base.PhysicalPlan, probeParents []base.PhysicalPlan) { plan.SetProbeParents(probeParents) switch x := plan.(type) { case *PhysicalApply, *PhysicalIndexJoin, *PhysicalIndexHashJoin, *PhysicalIndexMergeJoin: @@ -1054,7 +1055,7 @@ func propagateProbeParents(plan PhysicalPlan, probeParents []PhysicalPlan) { // The core logic of this method: // Record every Apply and Index Join we met, record it in a slice, and set it in their inner children. - newParents := make([]PhysicalPlan, len(probeParents), len(probeParents)+1) + newParents := make([]base.PhysicalPlan, len(probeParents), len(probeParents)+1) copy(newParents, probeParents) newParents = append(newParents, plan) propagateProbeParents(plan.Children()[join.getInnerChildIdx()], newParents) @@ -1078,7 +1079,7 @@ func propagateProbeParents(plan PhysicalPlan, probeParents []PhysicalPlan) { } } -func enableParallelApply(sctx PlanContext, plan PhysicalPlan) PhysicalPlan { +func enableParallelApply(sctx base.PlanContext, plan base.PhysicalPlan) base.PhysicalPlan { if !sctx.GetSessionVars().EnableParallelApply { return plan } @@ -1175,7 +1176,7 @@ func isLogicalRuleDisabled(r logicalOptRule) bool { return disabled } -func physicalOptimize(logic LogicalPlan, planCounter *PlanCounterTp) (plan PhysicalPlan, cost float64, err error) { +func physicalOptimize(logic LogicalPlan, planCounter *PlanCounterTp) (plan base.PhysicalPlan, cost float64, err error) { if logic.SCtx().GetSessionVars().StmtCtx.EnableOptimizerDebugTrace { debugtrace.EnterContextCommon(logic.SCtx()) defer debugtrace.LeaveContextCommon(logic.SCtx()) @@ -1235,12 +1236,12 @@ func physicalOptimize(logic LogicalPlan, planCounter *PlanCounterTp) (plan Physi } // eliminateUnionScanAndLock set lock property for PointGet and BatchPointGet and eliminates UnionScan and Lock. -func eliminateUnionScanAndLock(sctx PlanContext, p PhysicalPlan) PhysicalPlan { +func eliminateUnionScanAndLock(sctx base.PlanContext, p base.PhysicalPlan) base.PhysicalPlan { var pointGet *PointGetPlan var batchPointGet *BatchPointGetPlan var physLock *PhysicalLock var unionScan *PhysicalUnionScan - iteratePhysicalPlan(p, func(p PhysicalPlan) bool { + iteratePhysicalPlan(p, func(p base.PhysicalPlan) bool { if len(p.Children()) > 1 { return false } @@ -1275,7 +1276,7 @@ func eliminateUnionScanAndLock(sctx PlanContext, p PhysicalPlan) PhysicalPlan { batchPointGet.LockWaitTime = waitTime } } - return transformPhysicalPlan(p, func(p PhysicalPlan) PhysicalPlan { + return transformPhysicalPlan(p, func(p base.PhysicalPlan) base.PhysicalPlan { if p == physLock { return p.Children()[0] } @@ -1286,7 +1287,7 @@ func eliminateUnionScanAndLock(sctx PlanContext, p PhysicalPlan) PhysicalPlan { }) } -func iteratePhysicalPlan(p PhysicalPlan, f func(p PhysicalPlan) bool) { +func iteratePhysicalPlan(p base.PhysicalPlan, f func(p base.PhysicalPlan) bool) { if !f(p) { return } @@ -1295,7 +1296,7 @@ func iteratePhysicalPlan(p PhysicalPlan, f func(p PhysicalPlan) bool) { } } -func transformPhysicalPlan(p PhysicalPlan, f func(p PhysicalPlan) PhysicalPlan) PhysicalPlan { +func transformPhysicalPlan(p base.PhysicalPlan, f func(p base.PhysicalPlan) base.PhysicalPlan) base.PhysicalPlan { for i, child := range p.Children() { p.Children()[i] = transformPhysicalPlan(child, f) } @@ -1327,7 +1328,7 @@ func init() { DefaultDisabledLogicalRulesList.Store(set.NewStringSet()) } -func disableReuseChunkIfNeeded(sctx PlanContext, plan PhysicalPlan) { +func disableReuseChunkIfNeeded(sctx base.PlanContext, plan base.PhysicalPlan) { if !sctx.GetSessionVars().IsAllocValid() { return } @@ -1342,7 +1343,7 @@ func disableReuseChunkIfNeeded(sctx PlanContext, plan PhysicalPlan) { } // checkOverlongColType Check if read field type is long field. -func checkOverlongColType(sctx PlanContext, plan PhysicalPlan) bool { +func checkOverlongColType(sctx base.PlanContext, plan base.PhysicalPlan) bool { if plan == nil { return false } diff --git a/pkg/planner/core/optimizer_test.go b/pkg/planner/core/optimizer_test.go index 9bf60e6a32a0c..f5f5dec8bfaca 100644 --- a/pkg/planner/core/optimizer_test.go +++ b/pkg/planner/core/optimizer_test.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tipb/go-tipb" @@ -145,8 +146,8 @@ func TestHandleFineGrainedShuffle(t *testing.T) { p.TiFlashFineGrainedShuffleStreamCount = 0 } } - var check func(p PhysicalPlan, expStreamCount int64, expChildCount int, curChildCount int) - check = func(p PhysicalPlan, expStreamCount int64, expChildCount int, curChildCount int) { + var check func(p base.PhysicalPlan, expStreamCount int64, expChildCount int, curChildCount int) + check = func(p base.PhysicalPlan, expStreamCount int64, expChildCount int, curChildCount int) { if len(p.Children()) == 0 { require.Equal(t, expChildCount, curChildCount) _, isTableScan := p.(*PhysicalTableScan) @@ -167,7 +168,7 @@ func TestHandleFineGrainedShuffle(t *testing.T) { defer func() { domain.GetDomain(sctx).StatsHandle().Close() }() - start := func(p PhysicalPlan, expStreamCount int64, expChildCount int, curChildCount int) { + start := func(p base.PhysicalPlan, expStreamCount int64, expChildCount int, curChildCount int) { handleFineGrainedShuffle(nil, sctx.GetPlanCtx(), tableReader) check(p, expStreamCount, expChildCount, curChildCount) clear(plans) @@ -175,29 +176,29 @@ func TestHandleFineGrainedShuffle(t *testing.T) { // Window <- Sort <- ExchangeReceiver <- ExchangeSender tableReader.tablePlan = passSender - passSender.children = []PhysicalPlan{partWindow} - partWindow.children = []PhysicalPlan{partialSort} - partialSort.children = []PhysicalPlan{recv} - recv.children = []PhysicalPlan{hashSender} - hashSender.children = []PhysicalPlan{tableScan} + passSender.children = []base.PhysicalPlan{partWindow} + partWindow.children = []base.PhysicalPlan{partialSort} + partialSort.children = []base.PhysicalPlan{recv} + recv.children = []base.PhysicalPlan{hashSender} + hashSender.children = []base.PhysicalPlan{tableScan} start(partWindow, expStreamCount, 4, 0) // Window <- ExchangeReceiver <- ExchangeSender tableReader.tablePlan = passSender - passSender.children = []PhysicalPlan{partWindow} - partWindow.children = []PhysicalPlan{recv} - recv.children = []PhysicalPlan{hashSender} - hashSender.children = []PhysicalPlan{tableScan} + passSender.children = []base.PhysicalPlan{partWindow} + partWindow.children = []base.PhysicalPlan{recv} + recv.children = []base.PhysicalPlan{hashSender} + hashSender.children = []base.PhysicalPlan{tableScan} start(partWindow, expStreamCount, 3, 0) // Window <- Sort(x) <- ExchangeReceiver <- ExchangeSender // Fine-grained shuffle is disabled because sort is not partial. tableReader.tablePlan = passSender - passSender.children = []PhysicalPlan{partWindow} - partWindow.children = []PhysicalPlan{sort} - sort.children = []PhysicalPlan{recv} - recv.children = []PhysicalPlan{hashSender} - hashSender.children = []PhysicalPlan{tableScan} + passSender.children = []base.PhysicalPlan{partWindow} + partWindow.children = []base.PhysicalPlan{sort} + sort.children = []base.PhysicalPlan{recv} + recv.children = []base.PhysicalPlan{hashSender} + hashSender.children = []base.PhysicalPlan{tableScan} start(partWindow, 0, 4, 0) // Window <- Sort <- Window <- Sort <- ExchangeReceiver <- ExchangeSender @@ -209,13 +210,13 @@ func TestHandleFineGrainedShuffle(t *testing.T) { IsPartialSort: true, } tableReader.tablePlan = passSender - passSender.children = []PhysicalPlan{partWindow} - partWindow.children = []PhysicalPlan{partialSort} - partialSort.children = []PhysicalPlan{partWindow1} - partWindow1.children = []PhysicalPlan{partialSort1} - partialSort1.children = []PhysicalPlan{recv} - recv.children = []PhysicalPlan{hashSender} - hashSender.children = []PhysicalPlan{tableScan} + passSender.children = []base.PhysicalPlan{partWindow} + partWindow.children = []base.PhysicalPlan{partialSort} + partialSort.children = []base.PhysicalPlan{partWindow1} + partWindow1.children = []base.PhysicalPlan{partialSort1} + partialSort1.children = []base.PhysicalPlan{recv} + recv.children = []base.PhysicalPlan{hashSender} + hashSender.children = []base.PhysicalPlan{tableScan} start(partWindow, expStreamCount, 6, 0) // Window <- Sort <- Window(x) <- Sort <- ExchangeReceiver <- ExchangeSender(x) @@ -225,74 +226,74 @@ func TestHandleFineGrainedShuffle(t *testing.T) { IsPartialSort: true, } tableReader.tablePlan = passSender - passSender.children = []PhysicalPlan{partWindow} - partWindow.children = []PhysicalPlan{partialSort} - partialSort.children = []PhysicalPlan{nonPartWindow} - nonPartWindow.children = []PhysicalPlan{partialSort1} - partialSort1.children = []PhysicalPlan{recv} - recv.children = []PhysicalPlan{passSender} - passSender.children = []PhysicalPlan{tableScan} + passSender.children = []base.PhysicalPlan{partWindow} + partWindow.children = []base.PhysicalPlan{partialSort} + partialSort.children = []base.PhysicalPlan{nonPartWindow} + nonPartWindow.children = []base.PhysicalPlan{partialSort1} + partialSort1.children = []base.PhysicalPlan{recv} + recv.children = []base.PhysicalPlan{passSender} + passSender.children = []base.PhysicalPlan{tableScan} start(partWindow, 0, 6, 0) // HashAgg <- Window <- ExchangeReceiver <- ExchangeSender hashAgg := &PhysicalHashAgg{} tableReader.tablePlan = passSender - passSender.children = []PhysicalPlan{hashAgg} - hashAgg.children = []PhysicalPlan{partWindow} - partWindow.children = []PhysicalPlan{recv} - recv.children = []PhysicalPlan{hashSender} - hashSender.children = []PhysicalPlan{tableScan} + passSender.children = []base.PhysicalPlan{hashAgg} + hashAgg.children = []base.PhysicalPlan{partWindow} + partWindow.children = []base.PhysicalPlan{recv} + recv.children = []base.PhysicalPlan{hashSender} + hashSender.children = []base.PhysicalPlan{tableScan} require.Equal(t, uint64(0), hashAgg.TiFlashFineGrainedShuffleStreamCount) start(partWindow, expStreamCount, 3, 0) // Window <- HashAgg(x) <- ExchangeReceiver <- ExchangeSender tableReader.tablePlan = passSender - passSender.children = []PhysicalPlan{partWindow} + passSender.children = []base.PhysicalPlan{partWindow} hashAgg = &PhysicalHashAgg{} - partWindow.children = []PhysicalPlan{hashAgg} - hashAgg.children = []PhysicalPlan{recv} - recv.children = []PhysicalPlan{hashSender} - hashSender.children = []PhysicalPlan{tableScan} + partWindow.children = []base.PhysicalPlan{hashAgg} + hashAgg.children = []base.PhysicalPlan{recv} + recv.children = []base.PhysicalPlan{hashSender} + hashSender.children = []base.PhysicalPlan{tableScan} start(partWindow, 0, 4, 0) // Window <- Join(x) <- ExchangeReceiver <- ExchangeSender // <- ExchangeReceiver <- ExchangeSender tableReader.tablePlan = passSender - passSender.children = []PhysicalPlan{partWindow} + passSender.children = []base.PhysicalPlan{partWindow} hashJoin := &PhysicalHashJoin{} recv1 := &PhysicalExchangeReceiver{} tableScan1 := &PhysicalTableScan{} - partWindow.children = []PhysicalPlan{hashJoin} + partWindow.children = []base.PhysicalPlan{hashJoin} hashSender1 := &PhysicalExchangeSender{ ExchangeType: tipb.ExchangeType_Hash, } - hashJoin.children = []PhysicalPlan{recv, recv1} - recv.children = []PhysicalPlan{hashSender} - recv1.children = []PhysicalPlan{hashSender1} - hashSender.children = []PhysicalPlan{tableScan} - hashSender1.children = []PhysicalPlan{tableScan1} + hashJoin.children = []base.PhysicalPlan{recv, recv1} + recv.children = []base.PhysicalPlan{hashSender} + recv1.children = []base.PhysicalPlan{hashSender1} + hashSender.children = []base.PhysicalPlan{tableScan} + hashSender1.children = []base.PhysicalPlan{tableScan1} start(partWindow, 0, 4, 0) // Join <- ExchangeReceiver <- ExchangeSender <- Window <- ExchangeReceiver(2) <- ExchangeSender(2) // <- ExchangeReceiver(1) <- ExchangeSender(1) tableReader.tablePlan = passSender - passSender.children = []PhysicalPlan{partWindow} + passSender.children = []base.PhysicalPlan{partWindow} hashJoin = &PhysicalHashJoin{} recv1 = &PhysicalExchangeReceiver{} - hashJoin.children = []PhysicalPlan{recv, recv1} - recv.children = []PhysicalPlan{hashSender} - hashSender.children = []PhysicalPlan{partWindow} + hashJoin.children = []base.PhysicalPlan{recv, recv1} + recv.children = []base.PhysicalPlan{hashSender} + hashSender.children = []base.PhysicalPlan{partWindow} recv2 := &PhysicalExchangeReceiver{} hashSender2 := &PhysicalExchangeSender{ ExchangeType: tipb.ExchangeType_Hash, } tableScan2 := &PhysicalTableScan{} - partWindow.children = []PhysicalPlan{recv2} - recv2.children = []PhysicalPlan{hashSender2} - hashSender2.children = []PhysicalPlan{tableScan2} - recv1.children = []PhysicalPlan{hashSender1} + partWindow.children = []base.PhysicalPlan{recv2} + recv2.children = []base.PhysicalPlan{hashSender2} + hashSender2.children = []base.PhysicalPlan{tableScan2} + recv1.children = []base.PhysicalPlan{hashSender1} tableScan1 = &PhysicalTableScan{} - hashSender1.children = []PhysicalPlan{tableScan1} + hashSender1.children = []base.PhysicalPlan{tableScan1} start(partWindow, expStreamCount, 3, 0) instances := []string{ @@ -324,10 +325,10 @@ func TestHandleFineGrainedShuffle(t *testing.T) { // HashAgg(x) <- ExchangeReceiver <- ExchangeSender tableReader.tablePlan = passSender hashAgg = &PhysicalHashAgg{} - passSender.children = []PhysicalPlan{hashAgg} - hashAgg.children = []PhysicalPlan{recv} - recv.children = []PhysicalPlan{hashSender} - hashSender.children = []PhysicalPlan{tableScan} + passSender.children = []base.PhysicalPlan{hashAgg} + hashAgg.children = []base.PhysicalPlan{recv} + recv.children = []base.PhysicalPlan{hashSender} + hashSender.children = []base.PhysicalPlan{tableScan} tableScan.Schema().Columns = append(tableScan.Schema().Columns, col0) start(hashAgg, 8, 3, 0) @@ -338,7 +339,7 @@ func TestHandleFineGrainedShuffle(t *testing.T) { hashJoin.EqualConditions = append(hashJoin.EqualConditions, sf) hashJoin.RightJoinKeys = append(hashJoin.RightJoinKeys, col0) hashJoin.InnerChildIdx = 1 - passSender.children = []PhysicalPlan{hashJoin} + passSender.children = []base.PhysicalPlan{hashJoin} recv = &PhysicalExchangeReceiver{} recv1 = &PhysicalExchangeReceiver{} tableScan = &PhysicalTableScan{} @@ -349,11 +350,11 @@ func TestHandleFineGrainedShuffle(t *testing.T) { hashSender1 = &PhysicalExchangeSender{ ExchangeType: tipb.ExchangeType_Hash, } - hashJoin.children = []PhysicalPlan{recv, recv1} - recv.children = []PhysicalPlan{hashSender} - recv1.children = []PhysicalPlan{hashSender1} - hashSender.children = []PhysicalPlan{tableScan} - hashSender1.children = []PhysicalPlan{tableScan1} + hashJoin.children = []base.PhysicalPlan{recv, recv1} + recv.children = []base.PhysicalPlan{hashSender} + recv1.children = []base.PhysicalPlan{hashSender1} + hashSender.children = []base.PhysicalPlan{tableScan} + hashSender1.children = []base.PhysicalPlan{tableScan1} hashSender1.HashCols = partitionCols tableScan1.Schema().Columns = append(tableScan1.Schema().Columns, col0) handleFineGrainedShuffle(nil, sctx.GetPlanCtx(), tableReader) @@ -369,10 +370,10 @@ func TestHandleFineGrainedShuffle(t *testing.T) { // HashAgg(x) <- ExchangeReceiver <- ExchangeSender, exceed splitLimit tableReader.tablePlan = passSender hashAgg = &PhysicalHashAgg{} - passSender.children = []PhysicalPlan{hashAgg} - hashAgg.children = []PhysicalPlan{recv} - recv.children = []PhysicalPlan{hashSender} - hashSender.children = []PhysicalPlan{tableScan} + passSender.children = []base.PhysicalPlan{hashAgg} + hashAgg.children = []base.PhysicalPlan{recv} + recv.children = []base.PhysicalPlan{hashSender} + hashSender.children = []base.PhysicalPlan{tableScan} tableScan.Schema().Columns = append(tableScan.Schema().Columns, col0) start(hashAgg, 0, 3, 0) @@ -384,17 +385,17 @@ func TestHandleFineGrainedShuffle(t *testing.T) { hashJoin.EqualConditions = append(hashJoin.EqualConditions, sf) hashJoin.LeftJoinKeys = append(hashJoin.LeftJoinKeys, col0) hashJoin.InnerChildIdx = 1 - passSender.children = []PhysicalPlan{hashJoin} + passSender.children = []base.PhysicalPlan{hashJoin} recv1 = &PhysicalExchangeReceiver{} tableScan1 = &PhysicalTableScan{} hashSender1 = &PhysicalExchangeSender{ ExchangeType: tipb.ExchangeType_Hash, } - hashJoin.children = []PhysicalPlan{recv, recv1} - recv.children = []PhysicalPlan{hashSender} - recv1.children = []PhysicalPlan{hashSender1} - hashSender.children = []PhysicalPlan{tableScan} - hashSender1.children = []PhysicalPlan{tableScan1} + hashJoin.children = []base.PhysicalPlan{recv, recv1} + recv.children = []base.PhysicalPlan{hashSender} + recv1.children = []base.PhysicalPlan{hashSender1} + hashSender.children = []base.PhysicalPlan{tableScan} + hashSender1.children = []base.PhysicalPlan{tableScan1} hashSender1.HashCols = partitionCols tableScan1.Schema().Columns = append(tableScan1.Schema().Columns, col0) start(hashJoin, 0, 3, 0) @@ -444,8 +445,8 @@ func TestPrunePhysicalColumns(t *testing.T) { tableScan1 := &PhysicalTableScan{} tableReader.tablePlan = passSender - passSender.children = []PhysicalPlan{hashJoin} - hashJoin.children = []PhysicalPlan{recv, recv1} + passSender.children = []base.PhysicalPlan{hashJoin} + hashJoin.children = []base.PhysicalPlan{recv, recv1} selection := &PhysicalSelection{} cond, err := expression.NewFunction(sctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), col2, col3) diff --git a/pkg/planner/core/partition_prune.go b/pkg/planner/core/partition_prune.go index 957bceccf94b1..2f9381304f61e 100644 --- a/pkg/planner/core/partition_prune.go +++ b/pkg/planner/core/partition_prune.go @@ -17,6 +17,7 @@ package core import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/types" ) @@ -24,7 +25,7 @@ import ( // PartitionPruning finds all used partitions according to query conditions, it will // return nil if condition match none of partitions. The return value is a array of the // idx in the partition definitions array, use pi.Definitions[idx] to get the partition ID -func PartitionPruning(ctx PlanContext, tbl table.PartitionedTable, conds []expression.Expression, partitionNames []model.CIStr, +func PartitionPruning(ctx base.PlanContext, tbl table.PartitionedTable, conds []expression.Expression, partitionNames []model.CIStr, columns []*expression.Column, names types.NameSlice) ([]int, error) { s := partitionProcessor{} pi := tbl.Meta().Partition diff --git a/pkg/planner/core/pb_to_plan.go b/pkg/planner/core/pb_to_plan.go index af455809ee23a..9ad0a4a953524 100644 --- a/pkg/planner/core/pb_to_plan.go +++ b/pkg/planner/core/pb_to_plan.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/types" @@ -32,20 +33,20 @@ import ( // PBPlanBuilder uses to build physical plan from dag protocol buffers. type PBPlanBuilder struct { - sctx PlanContext + sctx base.PlanContext tps []*types.FieldType is infoschema.InfoSchema ranges []*coprocessor.KeyRange } // NewPBPlanBuilder creates a new pb plan builder. -func NewPBPlanBuilder(sctx PlanContext, is infoschema.InfoSchema, ranges []*coprocessor.KeyRange) *PBPlanBuilder { +func NewPBPlanBuilder(sctx base.PlanContext, is infoschema.InfoSchema, ranges []*coprocessor.KeyRange) *PBPlanBuilder { return &PBPlanBuilder{sctx: sctx, is: is, ranges: ranges} } // Build builds physical plan from dag protocol buffers. -func (b *PBPlanBuilder) Build(executors []*tipb.Executor) (p PhysicalPlan, err error) { - var src PhysicalPlan +func (b *PBPlanBuilder) Build(executors []*tipb.Executor) (p base.PhysicalPlan, err error) { + var src base.PhysicalPlan for i := 0; i < len(executors); i++ { curr, err := b.pbToPhysicalPlan(executors[i], src) if err != nil { @@ -57,7 +58,7 @@ func (b *PBPlanBuilder) Build(executors []*tipb.Executor) (p PhysicalPlan, err e return src, nil } -func (b *PBPlanBuilder) pbToPhysicalPlan(e *tipb.Executor, subPlan PhysicalPlan) (p PhysicalPlan, err error) { +func (b *PBPlanBuilder) pbToPhysicalPlan(e *tipb.Executor, subPlan base.PhysicalPlan) (p base.PhysicalPlan, err error) { switch e.Tp { case tipb.ExecType_TypeTableScan: p, err = b.pbToTableScan(e) @@ -91,7 +92,7 @@ func (b *PBPlanBuilder) pbToPhysicalPlan(e *tipb.Executor, subPlan PhysicalPlan) return p, err } -func (b *PBPlanBuilder) pbToTableScan(e *tipb.Executor) (PhysicalPlan, error) { +func (b *PBPlanBuilder) pbToTableScan(e *tipb.Executor) (base.PhysicalPlan, error) { tblScan := e.TblScan tbl, ok := b.is.TableByID(tblScan.TableId) if !ok { @@ -151,7 +152,7 @@ func (b *PBPlanBuilder) buildTableScanSchema(tblInfo *model.TableInfo, columns [ return schema } -func (b *PBPlanBuilder) pbToSelection(e *tipb.Executor) (PhysicalPlan, error) { +func (b *PBPlanBuilder) pbToSelection(e *tipb.Executor) (base.PhysicalPlan, error) { conds, err := expression.PBToExprs(b.sctx.GetExprCtx(), e.Selection.Conditions, b.tps) if err != nil { return nil, err @@ -162,7 +163,7 @@ func (b *PBPlanBuilder) pbToSelection(e *tipb.Executor) (PhysicalPlan, error) { return p, nil } -func (b *PBPlanBuilder) pbToTopN(e *tipb.Executor) (PhysicalPlan, error) { +func (b *PBPlanBuilder) pbToTopN(e *tipb.Executor) (base.PhysicalPlan, error) { topN := e.TopN byItems := make([]*util.ByItems, 0, len(topN.OrderBy)) exprCtx := b.sctx.GetExprCtx() @@ -180,14 +181,14 @@ func (b *PBPlanBuilder) pbToTopN(e *tipb.Executor) (PhysicalPlan, error) { return p, nil } -func (b *PBPlanBuilder) pbToLimit(e *tipb.Executor) (PhysicalPlan, error) { +func (b *PBPlanBuilder) pbToLimit(e *tipb.Executor) (base.PhysicalPlan, error) { p := PhysicalLimit{ Count: e.Limit.Limit, }.Init(b.sctx, &property.StatsInfo{}, 0, &property.PhysicalProperty{}) return p, nil } -func (b *PBPlanBuilder) pbToAgg(e *tipb.Executor, isStreamAgg bool) (PhysicalPlan, error) { +func (b *PBPlanBuilder) pbToAgg(e *tipb.Executor, isStreamAgg bool) (base.PhysicalPlan, error) { aggFuncs, groupBys, err := b.getAggInfo(e) if err != nil { return nil, errors.Trace(err) @@ -198,7 +199,7 @@ func (b *PBPlanBuilder) pbToAgg(e *tipb.Executor, isStreamAgg bool) (PhysicalPla GroupByItems: groupBys, } baseAgg.schema = schema - var partialAgg PhysicalPlan + var partialAgg base.PhysicalPlan if isStreamAgg { partialAgg = baseAgg.initForStream(b.sctx, &property.StatsInfo{}, 0, &property.PhysicalProperty{}) } else { @@ -258,7 +259,7 @@ func (b *PBPlanBuilder) convertColumnInfo(tblInfo *model.TableInfo, pbColumns [] return columns, nil } -func (*PBPlanBuilder) pbToKill(e *tipb.Executor) (PhysicalPlan, error) { +func (*PBPlanBuilder) pbToKill(e *tipb.Executor) (base.PhysicalPlan, error) { node := &ast.KillStmt{ ConnectionID: e.Kill.ConnID, Query: e.Kill.Query, @@ -267,7 +268,7 @@ func (*PBPlanBuilder) pbToKill(e *tipb.Executor) (PhysicalPlan, error) { return &PhysicalSimpleWrapper{Inner: simple}, nil } -func (b *PBPlanBuilder) predicatePushDown(physicalPlan PhysicalPlan, predicates []expression.Expression) ([]expression.Expression, PhysicalPlan) { +func (b *PBPlanBuilder) predicatePushDown(physicalPlan base.PhysicalPlan, predicates []expression.Expression) ([]expression.Expression, base.PhysicalPlan) { if physicalPlan == nil { return predicates, physicalPlan } diff --git a/pkg/planner/core/physical_plan_test.go b/pkg/planner/core/physical_plan_test.go index 06e47ab5fb210..928241461eabf 100644 --- a/pkg/planner/core/physical_plan_test.go +++ b/pkg/planner/core/physical_plan_test.go @@ -30,6 +30,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/planner" "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/core/internal" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util" @@ -392,7 +393,7 @@ func TestDAGPlanBuilderSplitAvg(t *testing.T) { require.NoError(t, err, comment) require.Equal(t, tt.plan, core.ToString(p), comment) - root, ok := p.(core.PhysicalPlan) + root, ok := p.(base.PhysicalPlan) if !ok { continue } @@ -400,7 +401,7 @@ func TestDAGPlanBuilderSplitAvg(t *testing.T) { } } -func testDAGPlanBuilderSplitAvg(t *testing.T, root core.PhysicalPlan) { +func testDAGPlanBuilderSplitAvg(t *testing.T, root base.PhysicalPlan) { if p, ok := root.(*core.PhysicalTableReader); ok { if p.TablePlans != nil { baseAgg := p.TablePlans[len(p.TablePlans)-1] @@ -457,11 +458,11 @@ func TestPhysicalTableScanExtractCorrelatedCols(t *testing.T) { tk.MustExec(sql) info := tk.Session().ShowProcess() require.NotNil(t, info) - p, ok := info.Plan.(core.Plan) + p, ok := info.Plan.(base.Plan) require.True(t, ok) - var findSelection func(p core.Plan) *core.PhysicalSelection - findSelection = func(p core.Plan) *core.PhysicalSelection { + var findSelection func(p base.Plan) *core.PhysicalSelection + findSelection = func(p base.Plan) *core.PhysicalSelection { if p == nil { return nil } @@ -481,7 +482,7 @@ func TestPhysicalTableScanExtractCorrelatedCols(t *testing.T) { } return nil default: - physicayPlan := p.(core.PhysicalPlan) + physicayPlan := p.(base.PhysicalPlan) for _, child := range physicayPlan.Children() { if sel := findSelection(child); sel != nil { return sel diff --git a/pkg/planner/core/physical_plans.go b/pkg/planner/core/physical_plans.go index 1be00cb580356..3f74d7b54b5ac 100644 --- a/pkg/planner/core/physical_plans.go +++ b/pkg/planner/core/physical_plans.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/cardinality" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/planner/util/coreusage" @@ -46,34 +47,34 @@ import ( ) var ( - _ PhysicalPlan = &PhysicalSelection{} - _ PhysicalPlan = &PhysicalProjection{} - _ PhysicalPlan = &PhysicalTopN{} - _ PhysicalPlan = &PhysicalMaxOneRow{} - _ PhysicalPlan = &PhysicalTableDual{} - _ PhysicalPlan = &PhysicalUnionAll{} - _ PhysicalPlan = &PhysicalSort{} - _ PhysicalPlan = &NominalSort{} - _ PhysicalPlan = &PhysicalLock{} - _ PhysicalPlan = &PhysicalLimit{} - _ PhysicalPlan = &PhysicalIndexScan{} - _ PhysicalPlan = &PhysicalTableScan{} - _ PhysicalPlan = &PhysicalTableReader{} - _ PhysicalPlan = &PhysicalIndexReader{} - _ PhysicalPlan = &PhysicalIndexLookUpReader{} - _ PhysicalPlan = &PhysicalIndexMergeReader{} - _ PhysicalPlan = &PhysicalHashAgg{} - _ PhysicalPlan = &PhysicalStreamAgg{} - _ PhysicalPlan = &PhysicalApply{} - _ PhysicalPlan = &PhysicalIndexJoin{} - _ PhysicalPlan = &PhysicalHashJoin{} - _ PhysicalPlan = &PhysicalMergeJoin{} - _ PhysicalPlan = &PhysicalUnionScan{} - _ PhysicalPlan = &PhysicalWindow{} - _ PhysicalPlan = &PhysicalShuffle{} - _ PhysicalPlan = &PhysicalShuffleReceiverStub{} - _ PhysicalPlan = &BatchPointGetPlan{} - _ PhysicalPlan = &PhysicalTableSample{} + _ base.PhysicalPlan = &PhysicalSelection{} + _ base.PhysicalPlan = &PhysicalProjection{} + _ base.PhysicalPlan = &PhysicalTopN{} + _ base.PhysicalPlan = &PhysicalMaxOneRow{} + _ base.PhysicalPlan = &PhysicalTableDual{} + _ base.PhysicalPlan = &PhysicalUnionAll{} + _ base.PhysicalPlan = &PhysicalSort{} + _ base.PhysicalPlan = &NominalSort{} + _ base.PhysicalPlan = &PhysicalLock{} + _ base.PhysicalPlan = &PhysicalLimit{} + _ base.PhysicalPlan = &PhysicalIndexScan{} + _ base.PhysicalPlan = &PhysicalTableScan{} + _ base.PhysicalPlan = &PhysicalTableReader{} + _ base.PhysicalPlan = &PhysicalIndexReader{} + _ base.PhysicalPlan = &PhysicalIndexLookUpReader{} + _ base.PhysicalPlan = &PhysicalIndexMergeReader{} + _ base.PhysicalPlan = &PhysicalHashAgg{} + _ base.PhysicalPlan = &PhysicalStreamAgg{} + _ base.PhysicalPlan = &PhysicalApply{} + _ base.PhysicalPlan = &PhysicalIndexJoin{} + _ base.PhysicalPlan = &PhysicalHashJoin{} + _ base.PhysicalPlan = &PhysicalMergeJoin{} + _ base.PhysicalPlan = &PhysicalUnionScan{} + _ base.PhysicalPlan = &PhysicalWindow{} + _ base.PhysicalPlan = &PhysicalShuffle{} + _ base.PhysicalPlan = &PhysicalShuffleReceiverStub{} + _ base.PhysicalPlan = &BatchPointGetPlan{} + _ base.PhysicalPlan = &PhysicalTableSample{} ) type tableScanAndPartitionInfo struct { @@ -124,8 +125,8 @@ type PhysicalTableReader struct { physicalSchemaProducer // TablePlans flats the tablePlan to construct executor pb. - TablePlans []PhysicalPlan - tablePlan PhysicalPlan + TablePlans []base.PhysicalPlan + tablePlan base.PhysicalPlan // StoreType indicates table read from which type of store. StoreType kv.StoreType @@ -175,7 +176,7 @@ func (pi *PhysPlanPartInfo) MemoryUsage() (sum int64) { } // GetTablePlan exports the tablePlan. -func (p *PhysicalTableReader) GetTablePlan() PhysicalPlan { +func (p *PhysicalTableReader) GetTablePlan() base.PhysicalPlan { return p.tablePlan } @@ -223,7 +224,7 @@ func (p *PhysicalTableReader) MemoryUsage() (sum int64) { } // setMppOrBatchCopForTableScan set IsMPPOrBatchCop for all TableScan. -func setMppOrBatchCopForTableScan(curPlan PhysicalPlan) { +func setMppOrBatchCopForTableScan(curPlan base.PhysicalPlan) { if ts, ok := curPlan.(*PhysicalTableScan); ok { ts.IsMPPOrBatchCop = true } @@ -258,7 +259,7 @@ func (sg *TiKVSingleGather) GetPhysicalIndexReader(schema *expression.Schema, st } // Clone implements op.PhysicalPlan interface. -func (p *PhysicalTableReader) Clone() (PhysicalPlan, error) { +func (p *PhysicalTableReader) Clone() (base.PhysicalPlan, error) { cloned := new(PhysicalTableReader) base, err := p.physicalSchemaProducer.cloneWithSelf(cloned) if err != nil { @@ -277,7 +278,7 @@ func (p *PhysicalTableReader) Clone() (PhysicalPlan, error) { } // SetChildren overrides op.PhysicalPlan SetChildren interface. -func (p *PhysicalTableReader) SetChildren(children ...PhysicalPlan) { +func (p *PhysicalTableReader) SetChildren(children ...base.PhysicalPlan) { p.tablePlan = children[0] p.TablePlans = flattenPushDownPlan(p.tablePlan) } @@ -310,8 +311,8 @@ type PhysicalIndexReader struct { physicalSchemaProducer // IndexPlans flats the indexPlan to construct executor pb. - IndexPlans []PhysicalPlan - indexPlan PhysicalPlan + IndexPlans []base.PhysicalPlan + indexPlan base.PhysicalPlan // OutputColumns represents the columns that index reader should return. OutputColumns []*expression.Column @@ -321,7 +322,7 @@ type PhysicalIndexReader struct { } // Clone implements op.PhysicalPlan interface. -func (p *PhysicalIndexReader) Clone() (PhysicalPlan, error) { +func (p *PhysicalIndexReader) Clone() (base.PhysicalPlan, error) { cloned := new(PhysicalIndexReader) base, err := p.physicalSchemaProducer.cloneWithSelf(cloned) if err != nil { @@ -354,7 +355,7 @@ func (p *PhysicalIndexReader) SetSchema(_ *expression.Schema) { } // SetChildren overrides op.PhysicalPlan SetChildren interface. -func (p *PhysicalIndexReader) SetChildren(children ...PhysicalPlan) { +func (p *PhysicalIndexReader) SetChildren(children ...base.PhysicalPlan) { p.indexPlan = children[0] p.SetSchema(nil) } @@ -439,11 +440,11 @@ type PhysicalIndexLookUpReader struct { physicalSchemaProducer // IndexPlans flats the indexPlan to construct executor pb. - IndexPlans []PhysicalPlan + IndexPlans []base.PhysicalPlan // TablePlans flats the tablePlan to construct executor pb. - TablePlans []PhysicalPlan - indexPlan PhysicalPlan - tablePlan PhysicalPlan + TablePlans []base.PhysicalPlan + indexPlan base.PhysicalPlan + tablePlan base.PhysicalPlan Paging bool ExtraHandleCol *expression.Column @@ -461,7 +462,7 @@ type PhysicalIndexLookUpReader struct { } // Clone implements op.PhysicalPlan interface. -func (p *PhysicalIndexLookUpReader) Clone() (PhysicalPlan, error) { +func (p *PhysicalIndexLookUpReader) Clone() (base.PhysicalPlan, error) { cloned := new(PhysicalIndexLookUpReader) base, err := p.physicalSchemaProducer.cloneWithSelf(cloned) if err != nil { @@ -589,13 +590,13 @@ type PhysicalIndexMergeReader struct { ByItems []*util.ByItems // PartialPlans flats the partialPlans to construct executor pb. - PartialPlans [][]PhysicalPlan + PartialPlans [][]base.PhysicalPlan // TablePlans flats the tablePlan to construct executor pb. - TablePlans []PhysicalPlan + TablePlans []base.PhysicalPlan // partialPlans are the partial plans that have not been flatted. The type of each element is permitted PhysicalIndexScan or PhysicalTableScan. - partialPlans []PhysicalPlan + partialPlans []base.PhysicalPlan // tablePlan is a PhysicalTableScan to get the table tuples. Current, it must be not nil. - tablePlan PhysicalPlan + tablePlan base.PhysicalPlan // Used by partition table. PlanPartInfo PhysPlanPartInfo @@ -737,7 +738,7 @@ type PhysicalIndexScan struct { } // Clone implements op.PhysicalPlan interface. -func (p *PhysicalIndexScan) Clone() (PhysicalPlan, error) { +func (p *PhysicalIndexScan) Clone() (base.PhysicalPlan, error) { cloned := new(PhysicalIndexScan) *cloned = *p base, err := p.physicalSchemaProducer.cloneWithSelf(cloned) @@ -815,7 +816,7 @@ func (p *PhysicalIndexScan) MemoryUsage() (sum int64) { // For keepOrder with partition table, // we need use partitionHandle to distinct two handles, // the `_tidb_rowid` in different partitions can have the same value. -func AddExtraPhysTblIDColumn(sctx PlanContext, columns []*model.ColumnInfo, schema *expression.Schema) ([]*model.ColumnInfo, *expression.Schema, bool) { +func AddExtraPhysTblIDColumn(sctx base.PlanContext, columns []*model.ColumnInfo, schema *expression.Schema) ([]*model.ColumnInfo, *expression.Schema, bool) { // Not adding the ExtraPhysTblID if already exists if FindColumnInfoByID(columns, model.ExtraPhysTblID) != nil { return columns, schema, false @@ -919,7 +920,7 @@ type PhysicalTableScan struct { } // Clone implements op.PhysicalPlan interface. -func (ts *PhysicalTableScan) Clone() (PhysicalPlan, error) { +func (ts *PhysicalTableScan) Clone() (base.PhysicalPlan, error) { clonedScan := new(PhysicalTableScan) *clonedScan = *ts prod, err := ts.physicalSchemaProducer.cloneWithSelf(clonedScan) @@ -1083,7 +1084,7 @@ type PhysicalProjection struct { } // Clone implements op.PhysicalPlan interface. -func (p *PhysicalProjection) Clone() (PhysicalPlan, error) { +func (p *PhysicalProjection) Clone() (base.PhysicalPlan, error) { cloned := new(PhysicalProjection) *cloned = *p base, err := p.physicalSchemaProducer.cloneWithSelf(cloned) @@ -1133,7 +1134,7 @@ func (lt *PhysicalTopN) GetPartitionBy() []property.SortItem { } // Clone implements op.PhysicalPlan interface. -func (lt *PhysicalTopN) Clone() (PhysicalPlan, error) { +func (lt *PhysicalTopN) Clone() (base.PhysicalPlan, error) { cloned := new(PhysicalTopN) *cloned = *lt base, err := lt.basePhysicalPlan.cloneWithSelf(cloned) @@ -1187,7 +1188,7 @@ type PhysicalApply struct { } // Clone implements op.PhysicalPlan interface. -func (la *PhysicalApply) Clone() (PhysicalPlan, error) { +func (la *PhysicalApply) Clone() (base.PhysicalPlan, error) { cloned := new(PhysicalApply) base, err := la.PhysicalHashJoin.Clone() if err != nil { @@ -1257,7 +1258,7 @@ func (p *basePhysicalJoin) getInnerChildIdx() int { return p.InnerChildIdx } -func (p *basePhysicalJoin) cloneWithSelf(newSelf PhysicalPlan) (*basePhysicalJoin, error) { +func (p *basePhysicalJoin) cloneWithSelf(newSelf base.PhysicalPlan) (*basePhysicalJoin, error) { cloned := new(basePhysicalJoin) base, err := p.physicalSchemaProducer.cloneWithSelf(newSelf) if err != nil { @@ -1364,7 +1365,7 @@ type PhysicalHashJoin struct { } // Clone implements op.PhysicalPlan interface. -func (p *PhysicalHashJoin) Clone() (PhysicalPlan, error) { +func (p *PhysicalHashJoin) Clone() (base.PhysicalPlan, error) { cloned := new(PhysicalHashJoin) base, err := p.basePhysicalJoin.cloneWithSelf(cloned) if err != nil { @@ -1464,7 +1465,7 @@ func NewPhysicalHashJoin(p *LogicalJoin, innerIdx int, useOuterToBuild bool, new type PhysicalIndexJoin struct { basePhysicalJoin - innerTask Task + innerTask base.Task // Ranges stores the IndexRanges when the inner plan is index scan. Ranges ranger.MutableRanges @@ -1585,7 +1586,7 @@ type PhysicalExchangeReceiver struct { } // Clone implment op.PhysicalPlan interface. -func (p *PhysicalExchangeReceiver) Clone() (PhysicalPlan, error) { +func (p *PhysicalExchangeReceiver) Clone() (base.PhysicalPlan, error) { np := new(PhysicalExchangeReceiver) base, err := p.basePhysicalPlan.cloneWithSelf(np) if err != nil { @@ -1635,7 +1636,7 @@ type PhysicalExpand struct { } // Init only assigns type and context. -func (p PhysicalExpand) Init(ctx PlanContext, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalExpand { +func (p PhysicalExpand) Init(ctx base.PlanContext, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalExpand { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeExpand, &p, offset) p.childrenReqProps = props p.SetStats(stats) @@ -1643,7 +1644,7 @@ func (p PhysicalExpand) Init(ctx PlanContext, stats *property.StatsInfo, offset } // Clone implements op.PhysicalPlan interface. -func (p *PhysicalExpand) Clone() (PhysicalPlan, error) { +func (p *PhysicalExpand) Clone() (base.PhysicalPlan, error) { if len(p.LevelExprs) > 0 { return p.cloneV2() } @@ -1665,7 +1666,7 @@ func (p *PhysicalExpand) Clone() (PhysicalPlan, error) { return np, nil } -func (p *PhysicalExpand) cloneV2() (PhysicalPlan, error) { +func (p *PhysicalExpand) cloneV2() (base.PhysicalPlan, error) { np := new(PhysicalExpand) base, err := p.physicalSchemaProducer.cloneWithSelf(np) if err != nil { @@ -1712,7 +1713,7 @@ type PhysicalExchangeSender struct { } // Clone implements op.PhysicalPlan interface. -func (p *PhysicalExchangeSender) Clone() (PhysicalPlan, error) { +func (p *PhysicalExchangeSender) Clone() (base.PhysicalPlan, error) { np := new(PhysicalExchangeSender) base, err := p.basePhysicalPlan.cloneWithSelf(np) if err != nil { @@ -1740,7 +1741,7 @@ func (p *PhysicalExchangeSender) MemoryUsage() (sum int64) { } // Clone implements op.PhysicalPlan interface. -func (p *PhysicalMergeJoin) Clone() (PhysicalPlan, error) { +func (p *PhysicalMergeJoin) Clone() (base.PhysicalPlan, error) { cloned := new(PhysicalMergeJoin) base, err := p.basePhysicalJoin.cloneWithSelf(cloned) if err != nil { @@ -1800,7 +1801,7 @@ func (p *PhysicalLimit) GetPartitionBy() []property.SortItem { } // Clone implements op.PhysicalPlan interface. -func (p *PhysicalLimit) Clone() (PhysicalPlan, error) { +func (p *PhysicalLimit) Clone() (base.PhysicalPlan, error) { cloned := new(PhysicalLimit) *cloned = *p base, err := p.physicalSchemaProducer.cloneWithSelf(cloned) @@ -1833,7 +1834,7 @@ type PhysicalUnionAll struct { } // Clone implements op.PhysicalPlan interface. -func (p *PhysicalUnionAll) Clone() (PhysicalPlan, error) { +func (p *PhysicalUnionAll) Clone() (base.PhysicalPlan, error) { cloned := new(PhysicalUnionAll) base, err := p.physicalSchemaProducer.cloneWithSelf(cloned) if err != nil { @@ -1886,7 +1887,7 @@ func (p *basePhysicalAgg) IsFinalAgg() bool { return false } -func (p *basePhysicalAgg) cloneWithSelf(newSelf PhysicalPlan) (*basePhysicalAgg, error) { +func (p *basePhysicalAgg) cloneWithSelf(newSelf base.PhysicalPlan) (*basePhysicalAgg, error) { cloned := new(basePhysicalAgg) base, err := p.physicalSchemaProducer.cloneWithSelf(newSelf) if err != nil { @@ -1976,7 +1977,7 @@ func (p *PhysicalHashAgg) getPointer() *basePhysicalAgg { } // Clone implements op.PhysicalPlan interface. -func (p *PhysicalHashAgg) Clone() (PhysicalPlan, error) { +func (p *PhysicalHashAgg) Clone() (base.PhysicalPlan, error) { cloned := new(PhysicalHashAgg) base, err := p.basePhysicalAgg.cloneWithSelf(cloned) if err != nil { @@ -2024,7 +2025,7 @@ func (p *PhysicalStreamAgg) getPointer() *basePhysicalAgg { } // Clone implements op.PhysicalPlan interface. -func (p *PhysicalStreamAgg) Clone() (PhysicalPlan, error) { +func (p *PhysicalStreamAgg) Clone() (base.PhysicalPlan, error) { cloned := new(PhysicalStreamAgg) base, err := p.basePhysicalAgg.cloneWithSelf(cloned) if err != nil { @@ -2054,7 +2055,7 @@ type PhysicalSort struct { } // Clone implements op.PhysicalPlan interface. -func (ls *PhysicalSort) Clone() (PhysicalPlan, error) { +func (ls *PhysicalSort) Clone() (base.PhysicalPlan, error) { cloned := new(PhysicalSort) cloned.IsPartialSort = ls.IsPartialSort base, err := ls.basePhysicalPlan.cloneWithSelf(cloned) @@ -2184,7 +2185,7 @@ type PhysicalSelection struct { } // Clone implements op.PhysicalPlan interface. -func (p *PhysicalSelection) Clone() (PhysicalPlan, error) { +func (p *PhysicalSelection) Clone() (base.PhysicalPlan, error) { cloned := new(PhysicalSelection) base, err := p.basePhysicalPlan.cloneWithSelf(cloned) if err != nil { @@ -2223,7 +2224,7 @@ type PhysicalMaxOneRow struct { } // Clone implements op.PhysicalPlan interface. -func (p *PhysicalMaxOneRow) Clone() (PhysicalPlan, error) { +func (p *PhysicalMaxOneRow) Clone() (base.PhysicalPlan, error) { cloned := new(PhysicalMaxOneRow) base, err := p.basePhysicalPlan.cloneWithSelf(cloned) if err != nil { @@ -2313,7 +2314,7 @@ func (p *PhysicalWindow) ExtractCorrelatedCols() []*expression.CorrelatedColumn } // Clone implements op.PhysicalPlan interface. -func (p *PhysicalWindow) Clone() (PhysicalPlan, error) { +func (p *PhysicalWindow) Clone() (base.PhysicalPlan, error) { cloned := new(PhysicalWindow) *cloned = *p base, err := p.physicalSchemaProducer.cloneWithSelf(cloned) @@ -2376,8 +2377,8 @@ type PhysicalShuffle struct { basePhysicalPlan Concurrency int - Tails []PhysicalPlan - DataSources []PhysicalPlan + Tails []base.PhysicalPlan + DataSources []base.PhysicalPlan SplitterType PartitionSplitterType ByItemArrays [][]expression.Expression @@ -2425,7 +2426,7 @@ type PhysicalShuffleReceiverStub struct { // Receiver points to `executor.shuffleReceiver`. Receiver unsafe.Pointer // DataSource is the op.PhysicalPlan of the Receiver. - DataSource PhysicalPlan + DataSource base.PhysicalPlan } // MemoryUsage return the memory usage of PhysicalShuffleReceiverStub @@ -2442,7 +2443,7 @@ func (p *PhysicalShuffleReceiverStub) MemoryUsage() (sum int64) { } // CollectPlanStatsVersion uses to collect the statistics version of the plan. -func CollectPlanStatsVersion(plan PhysicalPlan, statsInfos map[string]uint64) map[string]uint64 { +func CollectPlanStatsVersion(plan base.PhysicalPlan, statsInfos map[string]uint64) map[string]uint64 { for _, child := range plan.Children() { statsInfos = CollectPlanStatsVersion(child, statsInfos) } @@ -2499,7 +2500,7 @@ func (p *PhysicalShowDDLJobs) MemoryUsage() (sum int64) { } // BuildMergeJoinPlan builds a PhysicalMergeJoin from the given fields. Currently, it is only used for test purpose. -func BuildMergeJoinPlan(ctx PlanContext, joinType JoinType, leftKeys, rightKeys []*expression.Column) *PhysicalMergeJoin { +func BuildMergeJoinPlan(ctx base.PlanContext, joinType JoinType, leftKeys, rightKeys []*expression.Column) *PhysicalMergeJoin { baseJoin := basePhysicalJoin{ JoinType: joinType, DefaultValues: []types.Datum{types.NewDatum(1), types.NewDatum(1)}, @@ -2510,7 +2511,7 @@ func BuildMergeJoinPlan(ctx PlanContext, joinType JoinType, leftKeys, rightKeys } // SafeClone clones this op.PhysicalPlan and handles its panic. -func SafeClone(v PhysicalPlan) (_ PhysicalPlan, err error) { +func SafeClone(v base.PhysicalPlan) (_ base.PhysicalPlan, err error) { defer func() { if r := recover(); r != nil { err = errors.Errorf("%v", r) @@ -2568,8 +2569,8 @@ func NewTableSampleInfo(node *ast.TableSample, fullSchema *expression.Schema, pt type PhysicalCTE struct { physicalSchemaProducer - SeedPlan PhysicalPlan - RecurPlan PhysicalPlan + SeedPlan base.PhysicalPlan + RecurPlan base.PhysicalPlan CTE *CTEClass cteAsName model.CIStr cteName model.CIStr @@ -2615,7 +2616,7 @@ func (p *PhysicalCTE) ExplainID() fmt.Stringer { } // Clone implements op.PhysicalPlan interface. -func (p *PhysicalCTE) Clone() (PhysicalPlan, error) { +func (p *PhysicalCTE) Clone() (base.PhysicalPlan, error) { cloned := new(PhysicalCTE) base, err := p.physicalSchemaProducer.cloneWithSelf(cloned) if err != nil { @@ -2758,7 +2759,7 @@ func (p *PhysicalCTEStorage) MemoryUsage() (sum int64) { } // Clone implements op.PhysicalPlan interface. -func (p *PhysicalCTEStorage) Clone() (PhysicalPlan, error) { +func (p *PhysicalCTEStorage) Clone() (base.PhysicalPlan, error) { cloned, err := (*PhysicalCTE)(p).Clone() if err != nil { return nil, err @@ -2766,7 +2767,7 @@ func (p *PhysicalCTEStorage) Clone() (PhysicalPlan, error) { return (*PhysicalCTEStorage)(cloned.(*PhysicalCTE)), nil } -func appendChildCandidate(origin PhysicalPlan, pp PhysicalPlan, op *coreusage.PhysicalOptimizeOp) { +func appendChildCandidate(origin base.PhysicalPlan, pp base.PhysicalPlan, op *coreusage.PhysicalOptimizeOp) { candidate := &tracing.CandidatePlanTrace{ PlanTrace: &tracing.PlanTrace{ ID: pp.ID(), @@ -2813,7 +2814,7 @@ func (*PhysicalSequence) ExplainInfo() string { } // Clone implements op.PhysicalPlan interface. -func (p *PhysicalSequence) Clone() (PhysicalPlan, error) { +func (p *PhysicalSequence) Clone() (base.PhysicalPlan, error) { cloned := new(PhysicalSequence) base, err := p.physicalSchemaProducer.cloneWithSelf(cloned) if err != nil { diff --git a/pkg/planner/core/plan.go b/pkg/planner/core/plan.go index 364aabdfe0f7d..0d0a5d7e19130 100644 --- a/pkg/planner/core/plan.go +++ b/pkg/planner/core/plan.go @@ -21,7 +21,7 @@ import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/planner/cardinality" - "github.com/pingcap/tidb/pkg/planner/context" + base2 "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/core/internal/base" fd "github.com/pingcap/tidb/pkg/planner/funcdep" "github.com/pingcap/tidb/pkg/planner/property" @@ -34,14 +34,8 @@ import ( "github.com/pingcap/tidb/pkg/util/tracing" ) -// PlanContext is the context for building plan. -type PlanContext = context.PlanContext - -// BuildPBContext is the context for building `*tipb.Executor`. -type BuildPBContext = context.BuildPBContext - // AsSctx converts PlanContext to sessionctx.Context. -func AsSctx(pctx PlanContext) (sessionctx.Context, error) { +func AsSctx(pctx base2.PlanContext) (sessionctx.Context, error) { sctx, ok := pctx.(sessionctx.Context) if !ok { return nil, errors.New("the current PlanContext cannot be converted to sessionctx.Context") @@ -49,7 +43,7 @@ func AsSctx(pctx PlanContext) (sessionctx.Context, error) { return sctx, nil } -func enforceProperty(p *property.PhysicalProperty, tsk Task, ctx PlanContext) Task { +func enforceProperty(p *property.PhysicalProperty, tsk base2.Task, ctx base2.PlanContext) base2.Task { if p.TaskTp == property.MppTaskType { mpp, ok := tsk.(*MppTask) if !ok || mpp.Invalid() { @@ -81,7 +75,7 @@ func enforceProperty(p *property.PhysicalProperty, tsk Task, ctx PlanContext) Ta } // optimizeByShuffle insert `PhysicalShuffle` to optimize performance by running in a parallel manner. -func optimizeByShuffle(tsk Task, ctx PlanContext) Task { +func optimizeByShuffle(tsk base2.Task, ctx base2.PlanContext) base2.Task { if tsk.Plan() == nil { return tsk } @@ -103,7 +97,7 @@ func optimizeByShuffle(tsk Task, ctx PlanContext) Task { return tsk } -func optimizeByShuffle4Window(pp *PhysicalWindow, ctx PlanContext) *PhysicalShuffle { +func optimizeByShuffle4Window(pp *PhysicalWindow, ctx base2.PlanContext) *PhysicalShuffle { concurrency := ctx.GetSessionVars().WindowConcurrency() if concurrency <= 1 { return nil @@ -134,15 +128,15 @@ func optimizeByShuffle4Window(pp *PhysicalWindow, ctx PlanContext) *PhysicalShuf reqProp := &property.PhysicalProperty{ExpectedCnt: math.MaxFloat64} shuffle := PhysicalShuffle{ Concurrency: concurrency, - Tails: []PhysicalPlan{tail}, - DataSources: []PhysicalPlan{dataSource}, + Tails: []base2.PhysicalPlan{tail}, + DataSources: []base2.PhysicalPlan{dataSource}, SplitterType: PartitionHashSplitterType, ByItemArrays: [][]expression.Expression{byItems}, }.Init(ctx, pp.StatsInfo(), pp.QueryBlockOffset(), reqProp) return shuffle } -func optimizeByShuffle4StreamAgg(pp *PhysicalStreamAgg, ctx PlanContext) *PhysicalShuffle { +func optimizeByShuffle4StreamAgg(pp *PhysicalStreamAgg, ctx base2.PlanContext) *PhysicalShuffle { concurrency := ctx.GetSessionVars().StreamAggConcurrency() if concurrency <= 1 { return nil @@ -171,23 +165,23 @@ func optimizeByShuffle4StreamAgg(pp *PhysicalStreamAgg, ctx PlanContext) *Physic reqProp := &property.PhysicalProperty{ExpectedCnt: math.MaxFloat64} shuffle := PhysicalShuffle{ Concurrency: concurrency, - Tails: []PhysicalPlan{tail}, - DataSources: []PhysicalPlan{dataSource}, + Tails: []base2.PhysicalPlan{tail}, + DataSources: []base2.PhysicalPlan{dataSource}, SplitterType: PartitionHashSplitterType, ByItemArrays: [][]expression.Expression{util.CloneExprs(pp.GroupByItems)}, }.Init(ctx, pp.StatsInfo(), pp.QueryBlockOffset(), reqProp) return shuffle } -func optimizeByShuffle4MergeJoin(pp *PhysicalMergeJoin, ctx PlanContext) *PhysicalShuffle { +func optimizeByShuffle4MergeJoin(pp *PhysicalMergeJoin, ctx base2.PlanContext) *PhysicalShuffle { concurrency := ctx.GetSessionVars().MergeJoinConcurrency() if concurrency <= 1 { return nil } children := pp.Children() - dataSources := make([]PhysicalPlan, len(children)) - tails := make([]PhysicalPlan, len(children)) + dataSources := make([]base2.PhysicalPlan, len(children)) + tails := make([]base2.PhysicalPlan, len(children)) for i := range children { sort, ok := children[i].(*PhysicalSort) @@ -221,7 +215,7 @@ func optimizeByShuffle4MergeJoin(pp *PhysicalMergeJoin, ctx PlanContext) *Physic // LogicalPlan is a tree of logical operators. // We can do a lot of logical optimizations to it, like predicate pushdown and column pruning. type LogicalPlan interface { - Plan + base2.Plan // HashCode encodes a LogicalPlan to fast compare whether a LogicalPlan equals to another. // We use a strict encode method here which ensures there is no conflict. @@ -243,7 +237,7 @@ type LogicalPlan interface { // If planCounter > 0, the clock_th plan generated in this function will be returned. // If planCounter = 0, the plan generated in this function will not be considered. // If planCounter = -1, then we will not force plan. - findBestTask(prop *property.PhysicalProperty, planCounter *PlanCounterTp, op *coreusage.PhysicalOptimizeOp) (Task, int64, error) + findBestTask(prop *property.PhysicalProperty, planCounter *PlanCounterTp, op *coreusage.PhysicalOptimizeOp) (base2.Task, int64, error) // BuildKeyInfo will collect the information of unique keys into schema. // Because this method is also used in cascades planner, we cannot use @@ -289,7 +283,7 @@ type LogicalPlan interface { // It will return: // 1. All possible plans that can match the required property. // 2. Whether the SQL hint can work. Return true if there is no hint. - exhaustPhysicalPlans(*property.PhysicalProperty) (physicalPlans []PhysicalPlan, hintCanWork bool, err error) + exhaustPhysicalPlans(*property.PhysicalProperty) (physicalPlans []base2.PhysicalPlan, hintCanWork bool, err error) // ExtractCorrelatedCols extracts correlated columns inside the LogicalPlan. ExtractCorrelatedCols() []*expression.CorrelatedColumn @@ -319,7 +313,7 @@ type LogicalPlan interface { type baseLogicalPlan struct { base.Plan - taskMap map[string]Task + taskMap map[string]base2.Task // taskMapBak forms a backlog stack of taskMap, used to roll back the taskMap. taskMapBak []string // taskMapBakTS stores the timestamps of logs. @@ -355,7 +349,7 @@ func (*baseLogicalPlan) ExplainInfo() string { return "" } -func getEstimatedProbeCntFromProbeParents(probeParents []PhysicalPlan) float64 { +func getEstimatedProbeCntFromProbeParents(probeParents []base2.PhysicalPlan) float64 { res := float64(1) for _, pp := range probeParents { switch pp.(type) { @@ -369,7 +363,7 @@ func getEstimatedProbeCntFromProbeParents(probeParents []PhysicalPlan) float64 { return res } -func getActualProbeCntFromProbeParents(pps []PhysicalPlan, statsColl *execdetails.RuntimeStatsColl) int64 { +func getActualProbeCntFromProbeParents(pps []base2.PhysicalPlan, statsColl *execdetails.RuntimeStatsColl) int64 { res := int64(1) for _, pp := range pps { switch pp.(type) { @@ -395,8 +389,8 @@ type basePhysicalPlan struct { base.Plan childrenReqProps []*property.PhysicalProperty - self PhysicalPlan - children []PhysicalPlan + self base2.PhysicalPlan + children []base2.PhysicalPlan // used by the new cost interface planCostInit bool @@ -405,7 +399,7 @@ type basePhysicalPlan struct { // probeParents records the IndexJoins and Applys with this operator in their inner children. // Please see comments in op.PhysicalPlan for details. - probeParents []PhysicalPlan + probeParents []base2.PhysicalPlan // Only for MPP. If TiFlashFineGrainedShuffleStreamCount > 0: // 1. For ExchangeSender, means its output will be partitioned by hash key. @@ -413,7 +407,7 @@ type basePhysicalPlan struct { TiFlashFineGrainedShuffleStreamCount uint64 } -func (p *basePhysicalPlan) cloneWithSelf(newSelf PhysicalPlan) (*basePhysicalPlan, error) { +func (p *basePhysicalPlan) cloneWithSelf(newSelf base2.PhysicalPlan) (*basePhysicalPlan, error) { base := &basePhysicalPlan{ Plan: p.Plan, self: newSelf, @@ -437,7 +431,7 @@ func (p *basePhysicalPlan) cloneWithSelf(newSelf PhysicalPlan) (*basePhysicalPla } // Clone implements op.PhysicalPlan interface. -func (p *basePhysicalPlan) Clone() (PhysicalPlan, error) { +func (p *basePhysicalPlan) Clone() (base2.PhysicalPlan, error) { return nil, errors.Errorf("%T doesn't support cloning", p.self) } @@ -493,7 +487,7 @@ func (p *basePhysicalPlan) GetActualProbeCnt(statsColl *execdetails.RuntimeStats return getActualProbeCntFromProbeParents(p.probeParents, statsColl) } -func (p *basePhysicalPlan) SetProbeParents(probeParents []PhysicalPlan) { +func (p *basePhysicalPlan) SetProbeParents(probeParents []base2.PhysicalPlan) { p.probeParents = probeParents } @@ -531,12 +525,12 @@ func (p *baseLogicalPlan) rollBackTaskMap(ts uint64) { } } -func (p *baseLogicalPlan) getTask(prop *property.PhysicalProperty) Task { +func (p *baseLogicalPlan) getTask(prop *property.PhysicalProperty) base2.Task { key := prop.HashCode() return p.taskMap[string(key)] } -func (p *baseLogicalPlan) storeTask(prop *property.PhysicalProperty, task Task) { +func (p *baseLogicalPlan) storeTask(prop *property.PhysicalProperty, task base2.Task) { key := prop.HashCode() if p.SCtx().GetSessionVars().StmtCtx.StmtHints.TaskMapNeedBackUp() { // Empty string for useless change. @@ -603,9 +597,9 @@ func (p *logicalSchemaProducer) BuildKeyInfo(selfSchema *expression.Schema, chil } } -func newBaseLogicalPlan(ctx PlanContext, tp string, self LogicalPlan, qbOffset int) baseLogicalPlan { +func newBaseLogicalPlan(ctx base2.PlanContext, tp string, self LogicalPlan, qbOffset int) baseLogicalPlan { return baseLogicalPlan{ - taskMap: make(map[string]Task), + taskMap: make(map[string]base2.Task), taskMapBak: make([]string, 0, 10), taskMapBakTS: make([]uint64, 0, 10), Plan: base.NewBasePlan(ctx, tp, qbOffset), @@ -613,7 +607,7 @@ func newBaseLogicalPlan(ctx PlanContext, tp string, self LogicalPlan, qbOffset i } } -func newBasePhysicalPlan(ctx PlanContext, tp string, self PhysicalPlan, offset int) basePhysicalPlan { +func newBasePhysicalPlan(ctx base2.PlanContext, tp string, self base2.PhysicalPlan, offset int) basePhysicalPlan { return basePhysicalPlan{ Plan: base.NewBasePlan(ctx, tp, offset), self: self, @@ -661,7 +655,7 @@ func (p *baseLogicalPlan) Children() []LogicalPlan { } // Children implements op.PhysicalPlan Children interface. -func (p *basePhysicalPlan) Children() []PhysicalPlan { +func (p *basePhysicalPlan) Children() []base2.PhysicalPlan { return p.children } @@ -671,7 +665,7 @@ func (p *baseLogicalPlan) SetChildren(children ...LogicalPlan) { } // SetChildren implements op.PhysicalPlan SetChildren interface. -func (p *basePhysicalPlan) SetChildren(children ...PhysicalPlan) { +func (p *basePhysicalPlan) SetChildren(children ...base2.PhysicalPlan) { p.children = children } @@ -681,7 +675,7 @@ func (p *baseLogicalPlan) SetChild(i int, child LogicalPlan) { } // SetChild implements op.PhysicalPlan SetChild interface. -func (p *basePhysicalPlan) SetChild(i int, child PhysicalPlan) { +func (p *basePhysicalPlan) SetChild(i int, child base2.PhysicalPlan) { p.children[i] = child } diff --git a/pkg/planner/core/plan_cache.go b/pkg/planner/core/plan_cache.go index 2749711ea30b7..a5b4473e96917 100644 --- a/pkg/planner/core/plan_cache.go +++ b/pkg/planner/core/plan_cache.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/pkg/metrics" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/planner/core/base" core_metrics "github.com/pingcap/tidb/pkg/planner/core/metrics" "github.com/pingcap/tidb/pkg/planner/util/debugtrace" "github.com/pingcap/tidb/pkg/planner/util/fixcontrol" @@ -55,7 +56,7 @@ type PlanCacheKeyTestIssue46760 struct{} type PlanCacheKeyTestIssue47133 struct{} // SetParameterValuesIntoSCtx sets these parameters into session context. -func SetParameterValuesIntoSCtx(sctx PlanContext, isNonPrep bool, markers []ast.ParamMarkerExpr, params []expression.Expression) error { +func SetParameterValuesIntoSCtx(sctx base.PlanContext, isNonPrep bool, markers []ast.ParamMarkerExpr, params []expression.Expression) error { vars := sctx.GetSessionVars() vars.PlanCacheParams.Reset() for i, usingParam := range params { @@ -147,7 +148,7 @@ func planCachePreprocess(ctx context.Context, sctx sessionctx.Context, isNonPrep // isNonPrepared indicates whether to use the non-prepared plan cache or the prepared plan cache. func GetPlanFromSessionPlanCache(ctx context.Context, sctx sessionctx.Context, isNonPrepared bool, is infoschema.InfoSchema, stmt *PlanCacheStmt, - params []expression.Expression) (plan Plan, names []*types.FieldName, err error) { + params []expression.Expression) (plan base.Plan, names []*types.FieldName, err error) { if err := planCachePreprocess(ctx, sctx, isNonPrepared, is, stmt, params); err != nil { return nil, nil, err } @@ -233,14 +234,14 @@ func parseParamTypes(sctx sessionctx.Context, params []expression.Expression) (p return } -func getCachedPointPlan(stmt *PlanCacheStmt, sessVars *variable.SessionVars, stmtCtx *stmtctx.StatementContext) (Plan, +func getCachedPointPlan(stmt *PlanCacheStmt, sessVars *variable.SessionVars, stmtCtx *stmtctx.StatementContext) (base.Plan, []*types.FieldName, bool, error) { // short path for point-get plans // Rewriting the expression in the select.where condition will convert its // type from "paramMarker" to "Constant".When Point Select queries are executed, // the expression in the where condition will not be evaluated, // so you don't need to consider whether prepared.useCache is enabled. - plan := stmt.PointGet.Plan.(Plan) + plan := stmt.PointGet.Plan.(base.Plan) names := stmt.PointGet.ColumnNames.(types.NameSlice) if !RebuildPlan4CachedPlan(plan) { return nil, nil, false, nil @@ -260,7 +261,7 @@ func getCachedPointPlan(stmt *PlanCacheStmt, sessVars *variable.SessionVars, stm } func getCachedPlan(sctx sessionctx.Context, isNonPrepared bool, cacheKey kvcache.Key, bindSQL string, - is infoschema.InfoSchema, stmt *PlanCacheStmt, matchOpts *utilpc.PlanCacheMatchOpts) (Plan, + is infoschema.InfoSchema, stmt *PlanCacheStmt, matchOpts *utilpc.PlanCacheMatchOpts) (base.Plan, []*types.FieldName, bool, error) { sessVars := sctx.GetSessionVars() stmtCtx := sessVars.StmtCtx @@ -304,7 +305,7 @@ func getCachedPlan(sctx sessionctx.Context, isNonPrepared bool, cacheKey kvcache // and try to add it to cache func generateNewPlan(ctx context.Context, sctx sessionctx.Context, isNonPrepared bool, is infoschema.InfoSchema, stmt *PlanCacheStmt, cacheKey kvcache.Key, latestSchemaVersion int64, bindSQL string, - matchOpts *utilpc.PlanCacheMatchOpts) (Plan, []*types.FieldName, error) { + matchOpts *utilpc.PlanCacheMatchOpts) (base.Plan, []*types.FieldName, error) { stmtAst := stmt.PreparedAst sessVars := sctx.GetSessionVars() stmtCtx := sessVars.StmtCtx @@ -350,7 +351,7 @@ func generateNewPlan(ctx context.Context, sctx sessionctx.Context, isNonPrepared } // RebuildPlan4CachedPlan will rebuild this plan under current user parameters. -func RebuildPlan4CachedPlan(p Plan) (ok bool) { +func RebuildPlan4CachedPlan(p base.Plan) (ok bool) { sc := p.SCtx().GetSessionVars().StmtCtx if !sc.UseCache { return false // plan-cache is disabled for this query @@ -370,7 +371,7 @@ func RebuildPlan4CachedPlan(p Plan) (ok bool) { return true } -func updateRange(p PhysicalPlan, ranges ranger.Ranges, rangeInfo string) { +func updateRange(p base.PhysicalPlan, ranges ranger.Ranges, rangeInfo string) { switch x := p.(type) { case *PhysicalTableScan: x.Ranges = ranges @@ -396,7 +397,7 @@ func updateRange(p PhysicalPlan, ranges ranger.Ranges, rangeInfo string) { // generated, its complete ranges are ['a','a'], ['b','b'], ['c','c'], whose mem usage is under range mem limit 100B. // When the cached plan is hit, the complete ranges may become ['aaa','aaa'], ['bbb','bbb'], ['ccc','ccc'], whose mem // usage exceeds range mem limit 100B, and range fallback happens and tidb may fetch more rows than users expect. -func rebuildRange(p Plan) error { +func rebuildRange(p base.Plan) error { sctx := p.SCtx() sc := p.SCtx().GetSessionVars().StmtCtx var err error @@ -607,7 +608,7 @@ func rebuildRange(p Plan) error { } // We don't need to handle the indexMerge.TablePlans, because the tablePlans // only can be (Selection) + TableRowIDScan. There have no range need to rebuild. - case PhysicalPlan: + case base.PhysicalPlan: for _, child := range x.Children() { err = rebuildRange(child) if err != nil { @@ -630,7 +631,7 @@ func rebuildRange(p Plan) error { return nil } -func convertConstant2Datum(ctx PlanContext, con *expression.Constant, target *types.FieldType) (*types.Datum, error) { +func convertConstant2Datum(ctx base.PlanContext, con *expression.Constant, target *types.FieldType) (*types.Datum, error) { val, err := con.Eval(ctx.GetExprCtx().GetEvalCtx(), chunk.Row{}) if err != nil { return nil, err @@ -648,7 +649,7 @@ func convertConstant2Datum(ctx PlanContext, con *expression.Constant, target *ty return &dVal, nil } -func buildRangeForTableScan(sctx PlanContext, ts *PhysicalTableScan) (err error) { +func buildRangeForTableScan(sctx base.PlanContext, ts *PhysicalTableScan) (err error) { if ts.Table.IsCommonHandle { pk := tables.FindPrimaryIndex(ts.Table) pkCols := make([]*expression.Column, 0, len(pk.Columns)) @@ -713,7 +714,7 @@ func buildRangeForTableScan(sctx PlanContext, ts *PhysicalTableScan) (err error) return } -func buildRangeForIndexScan(sctx PlanContext, is *PhysicalIndexScan) (err error) { +func buildRangeForIndexScan(sctx base.PlanContext, is *PhysicalIndexScan) (err error) { if len(is.IdxCols) == 0 { if ranger.HasFullRange(is.Ranges, false) { // the original range is already a full-range. is.Ranges = ranger.FullRange() @@ -770,8 +771,8 @@ func CheckPreparedPriv(sctx sessionctx.Context, stmt *PlanCacheStmt, is infosche // tryCachePointPlan will try to cache point execution plan, there may be some // short paths for these executions, currently "point select" and "point update" -func tryCachePointPlan(_ context.Context, sctx PlanContext, - stmt *PlanCacheStmt, p Plan, names types.NameSlice) error { +func tryCachePointPlan(_ context.Context, sctx base.PlanContext, + stmt *PlanCacheStmt, p base.Plan, names types.NameSlice) error { if !sctx.GetSessionVars().StmtCtx.UseCache { return nil } diff --git a/pkg/planner/core/plan_cache_lru_test.go b/pkg/planner/core/plan_cache_lru_test.go index 0efe8a49e5ca9..d737fa55a4ad1 100644 --- a/pkg/planner/core/plan_cache_lru_test.go +++ b/pkg/planner/core/plan_cache_lru_test.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/hack" "github.com/pingcap/tidb/pkg/util/kvcache" @@ -37,7 +38,7 @@ func randomPlanCacheKey() *planCacheKey { } func randomPlanCacheValue(types []*types.FieldType) *PlanCacheValue { - plans := []Plan{&Insert{}, &Update{}, &Delete{}, &PhysicalTableScan{}, &PhysicalTableDual{}, &PhysicalTableReader{}, + plans := []base.Plan{&Insert{}, &Update{}, &Delete{}, &PhysicalTableScan{}, &PhysicalTableDual{}, &PhysicalTableReader{}, &PhysicalTableScan{}, &PhysicalIndexJoin{}, &PhysicalIndexHashJoin{}, &PhysicalIndexMergeJoin{}, &PhysicalIndexMergeReader{}, &PhysicalIndexLookUpReader{}, &PhysicalApply{}, &PhysicalApply{}, &PhysicalLimit{}} random := rand.New(rand.NewSource(time.Now().UnixNano())) diff --git a/pkg/planner/core/plan_cache_utils.go b/pkg/planner/core/plan_cache_utils.go index da551e1465268..04ae7a2b8dcbb 100644 --- a/pkg/planner/core/plan_cache_utils.go +++ b/pkg/planner/core/plan_cache_utils.go @@ -32,6 +32,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/planner/util/fixcontrol" "github.com/pingcap/tidb/pkg/sessionctx" @@ -78,7 +79,7 @@ func (e *paramMarkerExtractor) Leave(in ast.Node) (ast.Node, bool) { // paramSQL is the corresponding parameterized sql like 'select * from t where a?'. // paramStmt is the Node of paramSQL. func GeneratePlanCacheStmtWithAST(ctx context.Context, sctx sessionctx.Context, isPrepStmt bool, - paramSQL string, paramStmt ast.StmtNode, is infoschema.InfoSchema) (*PlanCacheStmt, Plan, int, error) { + paramSQL string, paramStmt ast.StmtNode, is infoschema.InfoSchema) (*PlanCacheStmt, base.Plan, int, error) { vars := sctx.GetSessionVars() var extractor paramMarkerExtractor paramStmt.Accept(&extractor) @@ -163,7 +164,7 @@ func GeneratePlanCacheStmtWithAST(ctx context.Context, sctx sessionctx.Context, } } - var p Plan + var p base.Plan destBuilder, _ := NewPlanBuilder().Init(sctx.GetPlanCtx(), ret.InfoSchema, hint.NewQBHintHandler(nil)) p, err = destBuilder.Build(ctx, paramStmt) if err != nil { @@ -343,7 +344,7 @@ func NewPlanCacheKey(sessionVars *variable.SessionVars, stmtText, stmtDB string, // PlanCacheValue stores the cached Statement and StmtNode. type PlanCacheValue struct { - Plan Plan + Plan base.Plan OutPutNames []*types.FieldName TblInfo2UnionScan map[*model.TableInfo]bool memoryUsage int64 @@ -368,7 +369,7 @@ func (v *PlanCacheValue) MemoryUsage() (sum int64) { return v.memoryUsage } switch x := v.Plan.(type) { - case PhysicalPlan: + case base.PhysicalPlan: sum = x.MemoryUsage() case *Insert: sum = x.MemoryUsage() @@ -397,7 +398,7 @@ func (v *PlanCacheValue) MemoryUsage() (sum int64) { } // NewPlanCacheValue creates a SQLCacheValue. -func NewPlanCacheValue(plan Plan, names []*types.FieldName, srcMap map[*model.TableInfo]bool, +func NewPlanCacheValue(plan base.Plan, names []*types.FieldName, srcMap map[*model.TableInfo]bool, matchOpts *utilpc.PlanCacheMatchOpts, stmtHints *hint.StmtHints) *PlanCacheValue { dstMap := make(map[*model.TableInfo]bool) for k, v := range srcMap { @@ -581,7 +582,7 @@ func checkTypesCompatibility4PC(tpsExpected, tpsActual []*types.FieldType) bool return true } -func isSafePointGetPath4PlanCache(sctx PlanContext, path *util.AccessPath) bool { +func isSafePointGetPath4PlanCache(sctx base.PlanContext, path *util.AccessPath) bool { // PointGet might contain some over-optimized assumptions, like `a>=1 and a<=1` --> `a=1`, but // these assumptions may be broken after parameters change. diff --git a/pkg/planner/core/plan_cacheable_checker.go b/pkg/planner/core/plan_cacheable_checker.go index b0568cfdcd646..861fe625c443e 100644 --- a/pkg/planner/core/plan_cacheable_checker.go +++ b/pkg/planner/core/plan_cacheable_checker.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/planner/core/base" core_metrics "github.com/pingcap/tidb/pkg/planner/core/metrics" "github.com/pingcap/tidb/pkg/planner/util/fixcontrol" "github.com/pingcap/tidb/pkg/types" @@ -48,14 +49,14 @@ func Cacheable(node ast.Node, is infoschema.InfoSchema) bool { // CacheableWithCtx checks whether the input ast(query) is cacheable. // TODO: only for test, remove this function later on. -func CacheableWithCtx(sctx PlanContext, node ast.Node, is infoschema.InfoSchema) (bool, string) { +func CacheableWithCtx(sctx base.PlanContext, node ast.Node, is infoschema.InfoSchema) (bool, string) { return IsASTCacheable(nil, sctx, node, is) } // IsASTCacheable checks whether the input ast(query) is cacheable. // Handle "ignore_plan_cache()" hint // If there are multiple hints, only one will take effect -func IsASTCacheable(ctx context.Context, sctx PlanContext, node ast.Node, is infoschema.InfoSchema) (bool, string) { +func IsASTCacheable(ctx context.Context, sctx base.PlanContext, node ast.Node, is infoschema.InfoSchema) (bool, string) { switch node.(type) { case *ast.SelectStmt, *ast.UpdateStmt, *ast.InsertStmt, *ast.DeleteStmt, *ast.SetOprStmt: default: @@ -76,7 +77,7 @@ func IsASTCacheable(ctx context.Context, sctx PlanContext, node ast.Node, is inf // cacheableChecker checks whether a query can be cached: type cacheableChecker struct { ctx context.Context - sctx PlanContext + sctx base.PlanContext cacheable bool schema infoschema.InfoSchema reason string // reason why cannot use plan-cache @@ -212,7 +213,7 @@ func (checker *cacheableChecker) Leave(in ast.Node) (out ast.Node, ok bool) { var nonPrepCacheCheckerPool = &sync.Pool{New: func() any { return &nonPreparedPlanCacheableChecker{} }} // NonPreparedPlanCacheableWithCtx checks whether this SQL is cacheable for non-prepared plan cache. -func NonPreparedPlanCacheableWithCtx(sctx PlanContext, node ast.Node, is infoschema.InfoSchema) (ok bool, reason string) { +func NonPreparedPlanCacheableWithCtx(sctx base.PlanContext, node ast.Node, is infoschema.InfoSchema) (ok bool, reason string) { selStmt, isSelect := node.(*ast.SelectStmt) if !sctx.GetSessionVars().EnableNonPreparedPlanCacheForDML && (!isSelect || selStmt.LockInfo != nil) { @@ -302,7 +303,7 @@ func NonPreparedPlanCacheableWithCtx(sctx PlanContext, node ast.Node, is infosch } // isSelectStmtNonPrepCacheableFastCheck checks whether the input select statement is cacheable for non-prepared plan cache. -func isSelectStmtNonPrepCacheableFastCheck(sctx PlanContext, selectStmt *ast.SelectStmt) (names []*ast.TableName, ok bool, reason string) { +func isSelectStmtNonPrepCacheableFastCheck(sctx base.PlanContext, selectStmt *ast.SelectStmt) (names []*ast.TableName, ok bool, reason string) { if selectStmt.Kind != ast.SelectStmtKindSelect { return nil, false, "not a select statement" } @@ -370,7 +371,7 @@ func extractTableNames(node ast.ResultSetNode, names []*ast.TableName) ([]*ast.T // nonPreparedPlanCacheableChecker checks whether a query's plan can be cached for non-prepared plan cache. // NOTE: we can add more rules in the future. type nonPreparedPlanCacheableChecker struct { - sctx PlanContext + sctx base.PlanContext cacheable bool reason string // reason why this statement cannot hit the cache schema infoschema.InfoSchema @@ -383,7 +384,7 @@ type nonPreparedPlanCacheableChecker struct { maxNumberParam int // the maximum number of parameters for a query to be cached. } -func (checker *nonPreparedPlanCacheableChecker) reset(sctx PlanContext, schema infoschema.InfoSchema, tableNodes []*ast.TableName, maxNumberParam int) { +func (checker *nonPreparedPlanCacheableChecker) reset(sctx base.PlanContext, schema infoschema.InfoSchema, tableNodes []*ast.TableName, maxNumberParam int) { checker.sctx = sctx checker.cacheable = true checker.schema = schema @@ -531,8 +532,8 @@ func getColType(schema infoschema.InfoSchema, tbl *ast.TableName, col *ast.Colum } // isPlanCacheable returns whether this plan is cacheable and the reason if not. -func isPlanCacheable(sctx PlanContext, p Plan, paramNum, limitParamNum int, hasSubQuery bool) (cacheable bool, reason string) { - var pp PhysicalPlan +func isPlanCacheable(sctx base.PlanContext, p base.Plan, paramNum, limitParamNum int, hasSubQuery bool) (cacheable bool, reason string) { + var pp base.PhysicalPlan switch x := p.(type) { case *Insert: pp = x.SelectPlan @@ -540,7 +541,7 @@ func isPlanCacheable(sctx PlanContext, p Plan, paramNum, limitParamNum int, hasS pp = x.SelectPlan case *Delete: pp = x.SelectPlan - case PhysicalPlan: + case base.PhysicalPlan: pp = x default: return false, fmt.Sprintf("unexpected un-cacheable plan %v", p.ExplainID().String()) @@ -561,8 +562,8 @@ func isPlanCacheable(sctx PlanContext, p Plan, paramNum, limitParamNum int, hasS } // isPhysicalPlanCacheable returns whether this physical plan is cacheable and return the reason if not. -func isPhysicalPlanCacheable(sctx PlanContext, p PhysicalPlan, paramNum, limitParamNum int, underIndexMerge bool) (cacheable bool, reason string) { - var subPlans []PhysicalPlan +func isPhysicalPlanCacheable(sctx base.PlanContext, p base.PhysicalPlan, paramNum, limitParamNum int, underIndexMerge bool) (cacheable bool, reason string) { + var subPlans []base.PhysicalPlan switch x := p.(type) { case *PhysicalTableDual: if paramNum > 0 { @@ -604,7 +605,7 @@ func isPhysicalPlanCacheable(sctx PlanContext, p PhysicalPlan, paramNum, limitPa } // getMaxParamLimit returns the maximum number of parameters for a query that can be cached in the Plan Cache. -func getMaxParamLimit(sctx PlanContext) int { +func getMaxParamLimit(sctx base.PlanContext) int { v := 200 if sctx == nil || sctx.GetSessionVars() == nil || sctx.GetSessionVars().OptimizerFixControl == nil { return v @@ -619,7 +620,7 @@ func getMaxParamLimit(sctx PlanContext) int { return v } -func enablePlanCacheForGeneratedCols(sctx PlanContext) bool { +func enablePlanCacheForGeneratedCols(sctx base.PlanContext) bool { // disable this by default since it's not well tested. defaultVal := true if sctx == nil || sctx.GetSessionVars() == nil || sctx.GetSessionVars().GetOptimizerFixControlMap() == nil { @@ -629,7 +630,7 @@ func enablePlanCacheForGeneratedCols(sctx PlanContext) bool { } // checkTableCacheable checks whether a query accessing this table is cacheable. -func checkTableCacheable(ctx context.Context, sctx PlanContext, schema infoschema.InfoSchema, node *ast.TableName, isNonPrep bool) (cacheable bool, reason string) { +func checkTableCacheable(ctx context.Context, sctx base.PlanContext, schema infoschema.InfoSchema, node *ast.TableName, isNonPrep bool) (cacheable bool, reason string) { tableSchema := node.Schema if tableSchema.L == "" { tableSchema.O = sctx.GetSessionVars().CurrentDB diff --git a/pkg/planner/core/plan_cost_detail.go b/pkg/planner/core/plan_cost_detail.go index 27b8bc2cffd33..a5c6b35597267 100644 --- a/pkg/planner/core/plan_cost_detail.go +++ b/pkg/planner/core/plan_cost_detail.go @@ -18,6 +18,7 @@ import ( "fmt" "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/util/coreusage" "github.com/pingcap/tidb/pkg/util/tracing" ) @@ -117,7 +118,7 @@ func setBatchPointGetPlanCostDetail(p *BatchPointGetPlan, opt *coreusage.Physica appendPlanCostDetail4PhysicalOptimizeOp(opt, detail) } -func setPhysicalTableOrIndexScanCostDetail(p PhysicalPlan, opt *coreusage.PhysicalOptimizeOp, +func setPhysicalTableOrIndexScanCostDetail(p base.PhysicalPlan, opt *coreusage.PhysicalOptimizeOp, rowCount, rowSize, scanFactor float64, costModelVersion int) { if opt == nil { return diff --git a/pkg/planner/core/plan_cost_ver1.go b/pkg/planner/core/plan_cost_ver1.go index 18a60c1c15b0e..99ac1fdde0264 100644 --- a/pkg/planner/core/plan_cost_ver1.go +++ b/pkg/planner/core/plan_cost_ver1.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/planner/cardinality" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util/coreusage" "github.com/pingcap/tidb/pkg/sessionctx/variable" @@ -171,7 +172,7 @@ func (p *PhysicalIndexLookUpReader) GetPlanCostVer1(_ property.TaskType, option p.planCost = 0 // child's cost - for _, child := range []PhysicalPlan{p.indexPlan, p.tablePlan} { + for _, child := range []base.PhysicalPlan{p.indexPlan, p.tablePlan} { childCost, err := child.GetPlanCostVer1(property.CopMultiReadTaskType, option) if err != nil { return 0, err @@ -406,7 +407,7 @@ func (p *PhysicalIndexMergeReader) GetPlanCostVer1(_ property.TaskType, option * } // GetPartialReaderNetDataSize returns the estimated total response data size of a partial read. -func (p *PhysicalIndexMergeReader) GetPartialReaderNetDataSize(plan PhysicalPlan) float64 { +func (p *PhysicalIndexMergeReader) GetPartialReaderNetDataSize(plan base.PhysicalPlan) float64 { _, isIdxScan := plan.(*PhysicalIndexScan) return plan.StatsCount() * cardinality.GetAvgRowSize(p.SCtx(), getTblStats(plan), plan.Schema().Columns, isIdxScan, false) } @@ -1243,7 +1244,7 @@ func (p *PhysicalExchangeReceiver) GetPlanCostVer1(taskType property.TaskType, o return p.planCost, nil } -func getOperatorActRows(operator PhysicalPlan) float64 { +func getOperatorActRows(operator base.PhysicalPlan) float64 { if operator == nil { return 0 } @@ -1260,7 +1261,7 @@ func getOperatorActRows(operator PhysicalPlan) float64 { return actRows } -func getCardinality(operator PhysicalPlan, costFlag uint64) float64 { +func getCardinality(operator base.PhysicalPlan, costFlag uint64) float64 { if hasCostFlag(costFlag, coreusage.CostFlagUseTrueCardinality) { actualProbeCnt := operator.GetActualProbeCnt(operator.SCtx().GetSessionVars().StmtCtx.RuntimeStatsColl) if actualProbeCnt == 0 { @@ -1279,7 +1280,7 @@ func getCardinality(operator PhysicalPlan, costFlag uint64) float64 { // estimateNetSeekCost calculates the net seek cost for the plan. // for TiKV, it's len(access-range) * seek-factor, // and for TiFlash, it's len(access-range) * len(access-column) * seek-factor. -func estimateNetSeekCost(copTaskPlan PhysicalPlan) float64 { +func estimateNetSeekCost(copTaskPlan base.PhysicalPlan) float64 { switch x := copTaskPlan.(type) { case *PhysicalTableScan: if x.StoreType == kv.TiFlash { // the old TiFlash interface uses cop-task protocol @@ -1294,7 +1295,7 @@ func estimateNetSeekCost(copTaskPlan PhysicalPlan) float64 { } // getTblStats returns the tbl-stats of this plan, which contains all columns before pruning. -func getTblStats(copTaskPlan PhysicalPlan) *statistics.HistColl { +func getTblStats(copTaskPlan base.PhysicalPlan) *statistics.HistColl { switch x := copTaskPlan.(type) { case *PhysicalTableScan: return x.tblColHists @@ -1306,7 +1307,7 @@ func getTblStats(copTaskPlan PhysicalPlan) *statistics.HistColl { } // getTableNetFactor returns the corresponding net factor of this table, it's mainly for temporary tables -func getTableNetFactor(copTaskPlan PhysicalPlan) float64 { +func getTableNetFactor(copTaskPlan base.PhysicalPlan) float64 { switch x := copTaskPlan.(type) { case *PhysicalTableScan: return x.SCtx().GetSessionVars().GetNetworkFactor(x.Table) diff --git a/pkg/planner/core/plan_cost_ver2.go b/pkg/planner/core/plan_cost_ver2.go index d644464b14ebb..176d0f3577f26 100644 --- a/pkg/planner/core/plan_cost_ver2.go +++ b/pkg/planner/core/plan_cost_ver2.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/planner/cardinality" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/planner/util/coreusage" @@ -32,14 +33,14 @@ import ( ) // GetPlanCost returns the cost of this plan. -func GetPlanCost(p PhysicalPlan, taskType property.TaskType, option *coreusage.PlanCostOption) (float64, error) { +func GetPlanCost(p base.PhysicalPlan, taskType property.TaskType, option *coreusage.PlanCostOption) (float64, error) { return getPlanCost(p, taskType, option) } // GenPlanCostTrace define a hook function to customize the cost calculation. -var GenPlanCostTrace func(p PhysicalPlan, costV *coreusage.CostVer2, taskType property.TaskType, option *coreusage.PlanCostOption) +var GenPlanCostTrace func(p base.PhysicalPlan, costV *coreusage.CostVer2, taskType property.TaskType, option *coreusage.PlanCostOption) -func getPlanCost(p PhysicalPlan, taskType property.TaskType, option *coreusage.PlanCostOption) (float64, error) { +func getPlanCost(p base.PhysicalPlan, taskType property.TaskType, option *coreusage.PlanCostOption) (float64, error) { if p.SCtx().GetSessionVars().CostModelVersion == modelVer2 { planCost, err := p.GetPlanCostVer2(taskType, option) if coreusage.TraceCost(option) && GenPlanCostTrace != nil { @@ -934,7 +935,7 @@ var defaultVer2Factors = costVer2Factors{ TiDBRequest: coreusage.CostVer2Factor{Name: "tidb_request_factor", Value: 6000000.00}, } -func getTaskCPUFactorVer2(_ PhysicalPlan, taskType property.TaskType) coreusage.CostVer2Factor { +func getTaskCPUFactorVer2(_ base.PhysicalPlan, taskType property.TaskType) coreusage.CostVer2Factor { switch taskType { case property.RootTaskType: // TiDB return defaultVer2Factors.TiDBCPU @@ -945,7 +946,7 @@ func getTaskCPUFactorVer2(_ PhysicalPlan, taskType property.TaskType) coreusage. } } -func getTaskMemFactorVer2(_ PhysicalPlan, taskType property.TaskType) coreusage.CostVer2Factor { +func getTaskMemFactorVer2(_ base.PhysicalPlan, taskType property.TaskType) coreusage.CostVer2Factor { switch taskType { case property.RootTaskType: // TiDB return defaultVer2Factors.TiDBMem @@ -956,7 +957,7 @@ func getTaskMemFactorVer2(_ PhysicalPlan, taskType property.TaskType) coreusage. } } -func getTaskScanFactorVer2(p PhysicalPlan, storeType kv.StoreType, taskType property.TaskType) coreusage.CostVer2Factor { +func getTaskScanFactorVer2(p base.PhysicalPlan, storeType kv.StoreType, taskType property.TaskType) coreusage.CostVer2Factor { if isTemporaryTable(getTableInfo(p)) { return defaultVer2Factors.TiDBTemp } @@ -981,7 +982,7 @@ func getTaskScanFactorVer2(p PhysicalPlan, storeType kv.StoreType, taskType prop } } -func getTaskNetFactorVer2(p PhysicalPlan, _ property.TaskType) coreusage.CostVer2Factor { +func getTaskNetFactorVer2(p base.PhysicalPlan, _ property.TaskType) coreusage.CostVer2Factor { if isTemporaryTable(getTableInfo(p)) { return defaultVer2Factors.TiDBTemp } @@ -996,7 +997,7 @@ func getTaskNetFactorVer2(p PhysicalPlan, _ property.TaskType) coreusage.CostVer return defaultVer2Factors.TiDB2KVNet } -func getTaskRequestFactorVer2(p PhysicalPlan, _ property.TaskType) coreusage.CostVer2Factor { +func getTaskRequestFactorVer2(p base.PhysicalPlan, _ property.TaskType) coreusage.CostVer2Factor { if isTemporaryTable(getTableInfo(p)) { return defaultVer2Factors.TiDBTemp } @@ -1007,7 +1008,7 @@ func isTemporaryTable(tbl *model.TableInfo) bool { return tbl != nil && tbl.TempTableType != model.TempTableNone } -func getTableInfo(p PhysicalPlan) *model.TableInfo { +func getTableInfo(p base.PhysicalPlan) *model.TableInfo { switch x := p.(type) { case *PhysicalIndexReader: return getTableInfo(x.indexPlan) diff --git a/pkg/planner/core/plan_cost_ver2_test.go b/pkg/planner/core/plan_cost_ver2_test.go index 7bab1efc2a9bf..12e718fe88ff0 100644 --- a/pkg/planner/core/plan_cost_ver2_test.go +++ b/pkg/planner/core/plan_cost_ver2_test.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/planner" "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util/coreusage" "github.com/pingcap/tidb/pkg/sessiontxn" @@ -147,7 +148,7 @@ func BenchmarkGetPlanCost(b *testing.B) { if err != nil { b.Fatal(err) } - phyPlan := plan.(core.PhysicalPlan) + phyPlan := plan.(base.PhysicalPlan) _, err = core.GetPlanCost(phyPlan, property.RootTaskType, coreusage.NewDefaultPlanCostOption().WithCostFlag(coreusage.CostFlagRecalculate)) if err != nil { b.Fatal(err) diff --git a/pkg/planner/core/plan_test.go b/pkg/planner/core/plan_test.go index bb494131f18e9..05db519444c5f 100644 --- a/pkg/planner/core/plan_test.go +++ b/pkg/planner/core/plan_test.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/testkit" @@ -52,7 +53,7 @@ func TestEncodeDecodePlan(t *testing.T) { getPlanTree := func() (str1, str2 string) { info := tk.Session().ShowProcess() require.NotNil(t, info) - p, ok := info.Plan.(core.Plan) + p, ok := info.Plan.(base.Plan) require.True(t, ok) encodeStr := core.EncodePlan(p) planTree, err := plancodec.DecodePlan(encodeStr) @@ -327,7 +328,7 @@ func testNormalizeDigest(tk *testkit.TestKit, t *testing.T, sql1, sql2 string, i tk.MustQuery(sql1) info := tk.Session().ShowProcess() require.NotNil(t, info) - physicalPlan, ok := info.Plan.(core.PhysicalPlan) + physicalPlan, ok := info.Plan.(base.PhysicalPlan) require.True(t, ok) normalized1, digest1 := core.NormalizePlan(physicalPlan) @@ -340,7 +341,7 @@ func testNormalizeDigest(tk *testkit.TestKit, t *testing.T, sql1, sql2 string, i tk.MustQuery(sql2) info = tk.Session().ShowProcess() require.NotNil(t, info) - physicalPlan, ok = info.Plan.(core.PhysicalPlan) + physicalPlan, ok = info.Plan.(base.PhysicalPlan) require.True(t, ok) normalized2, digest2 := core.NormalizePlan(physicalPlan) @@ -412,7 +413,7 @@ func BenchmarkDecodePlan(b *testing.B) { tk.MustExec(query) info := tk.Session().ShowProcess() require.NotNil(b, info) - p, ok := info.Plan.(core.PhysicalPlan) + p, ok := info.Plan.(base.PhysicalPlan) require.True(b, ok) // TODO: optimize the encode plan performance when encode plan with runtimeStats tk.Session().GetSessionVars().StmtCtx.RuntimeStatsColl = nil @@ -439,7 +440,7 @@ func BenchmarkEncodePlan(b *testing.B) { tk.MustExec(query) info := tk.Session().ShowProcess() require.NotNil(b, info) - p, ok := info.Plan.(core.PhysicalPlan) + p, ok := info.Plan.(base.PhysicalPlan) require.True(b, ok) tk.Session().GetSessionVars().StmtCtx.RuntimeStatsColl = nil b.ResetTimer() @@ -463,7 +464,7 @@ func BenchmarkEncodeFlatPlan(b *testing.B) { tk.MustExec(query) info := tk.Session().ShowProcess() require.NotNil(b, info) - p, ok := info.Plan.(core.PhysicalPlan) + p, ok := info.Plan.(base.PhysicalPlan) require.True(b, ok) tk.Session().GetSessionVars().StmtCtx.RuntimeStatsColl = nil b.ResetTimer() @@ -534,7 +535,7 @@ func TestCopPaging(t *testing.T) { } func TestBuildFinalModeAggregation(t *testing.T) { - aggSchemaBuilder := func(sctx core.PlanContext, aggFuncs []*aggregation.AggFuncDesc) *expression.Schema { + aggSchemaBuilder := func(sctx base.PlanContext, aggFuncs []*aggregation.AggFuncDesc) *expression.Schema { schema := expression.NewSchema(make([]*expression.Column, 0, len(aggFuncs))...) for _, agg := range aggFuncs { newCol := &expression.Column{ @@ -548,7 +549,7 @@ func TestBuildFinalModeAggregation(t *testing.T) { isFinalAggMode := func(mode aggregation.AggFunctionMode) bool { return mode == aggregation.FinalMode || mode == aggregation.CompleteMode } - checkResult := func(sctx core.PlanContext, aggFuncs []*aggregation.AggFuncDesc, groubyItems []expression.Expression) { + checkResult := func(sctx base.PlanContext, aggFuncs []*aggregation.AggFuncDesc, groubyItems []expression.Expression) { for partialIsCop := 0; partialIsCop < 2; partialIsCop++ { for isMPPTask := 0; isMPPTask < 2; isMPPTask++ { partial, final, _ := core.BuildFinalModeAggregation(sctx, &core.AggInfo{ diff --git a/pkg/planner/core/plan_to_pb.go b/pkg/planner/core/plan_to_pb.go index e6990c44a7f45..802b0445709e7 100644 --- a/pkg/planner/core/plan_to_pb.go +++ b/pkg/planner/core/plan_to_pb.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/tidb/pkg/expression/aggregation" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/table/tables" "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/ranger" @@ -27,12 +28,12 @@ import ( ) // ToPB implements PhysicalPlan ToPB interface. -func (p *basePhysicalPlan) ToPB(_ *BuildPBContext, _ kv.StoreType) (*tipb.Executor, error) { +func (p *basePhysicalPlan) ToPB(_ *base.BuildPBContext, _ kv.StoreType) (*tipb.Executor, error) { return nil, errors.Errorf("plan %s fails converts to PB", p.Plan.ExplainID()) } // ToPB implements PhysicalPlan ToPB interface. -func (p *PhysicalExpand) ToPB(ctx *BuildPBContext, storeType kv.StoreType) (*tipb.Executor, error) { +func (p *PhysicalExpand) ToPB(ctx *base.BuildPBContext, storeType kv.StoreType) (*tipb.Executor, error) { if len(p.LevelExprs) > 0 { return p.toPBV2(ctx, storeType) } @@ -56,7 +57,7 @@ func (p *PhysicalExpand) ToPB(ctx *BuildPBContext, storeType kv.StoreType) (*tip return &tipb.Executor{Tp: tipb.ExecType_TypeExpand, Expand: expand, ExecutorId: &executorID}, nil } -func (p *PhysicalExpand) toPBV2(ctx *BuildPBContext, storeType kv.StoreType) (*tipb.Executor, error) { +func (p *PhysicalExpand) toPBV2(ctx *base.BuildPBContext, storeType kv.StoreType) (*tipb.Executor, error) { client := ctx.GetClient() projExprsPB := make([]*tipb.ExprSlice, 0, len(p.LevelExprs)) evalCtx := ctx.GetExprCtx().GetEvalCtx() @@ -84,7 +85,7 @@ func (p *PhysicalExpand) toPBV2(ctx *BuildPBContext, storeType kv.StoreType) (*t } // ToPB implements PhysicalPlan ToPB interface. -func (p *PhysicalHashAgg) ToPB(ctx *BuildPBContext, storeType kv.StoreType) (*tipb.Executor, error) { +func (p *PhysicalHashAgg) ToPB(ctx *base.BuildPBContext, storeType kv.StoreType) (*tipb.Executor, error) { client := ctx.GetClient() groupByExprs, err := expression.ExpressionsToPBList(ctx.GetExprCtx().GetEvalCtx(), p.GroupByItems, client) if err != nil { @@ -120,7 +121,7 @@ func (p *PhysicalHashAgg) ToPB(ctx *BuildPBContext, storeType kv.StoreType) (*ti } // ToPB implements PhysicalPlan ToPB interface. -func (p *PhysicalStreamAgg) ToPB(ctx *BuildPBContext, storeType kv.StoreType) (*tipb.Executor, error) { +func (p *PhysicalStreamAgg) ToPB(ctx *base.BuildPBContext, storeType kv.StoreType) (*tipb.Executor, error) { client := ctx.GetClient() evalCtx := ctx.GetExprCtx().GetEvalCtx() pushDownCtx := GetPushDownCtxFromBuildPBContext(ctx) @@ -151,7 +152,7 @@ func (p *PhysicalStreamAgg) ToPB(ctx *BuildPBContext, storeType kv.StoreType) (* } // ToPB implements PhysicalPlan ToPB interface. -func (p *PhysicalSelection) ToPB(ctx *BuildPBContext, storeType kv.StoreType) (*tipb.Executor, error) { +func (p *PhysicalSelection) ToPB(ctx *base.BuildPBContext, storeType kv.StoreType) (*tipb.Executor, error) { client := ctx.GetClient() conditions, err := expression.ExpressionsToPBList(ctx.GetExprCtx().GetEvalCtx(), p.Conditions, client) if err != nil { @@ -173,7 +174,7 @@ func (p *PhysicalSelection) ToPB(ctx *BuildPBContext, storeType kv.StoreType) (* } // ToPB implements PhysicalPlan ToPB interface. -func (p *PhysicalProjection) ToPB(ctx *BuildPBContext, storeType kv.StoreType) (*tipb.Executor, error) { +func (p *PhysicalProjection) ToPB(ctx *base.BuildPBContext, storeType kv.StoreType) (*tipb.Executor, error) { client := ctx.GetClient() exprs, err := expression.ExpressionsToPBList(ctx.GetExprCtx().GetEvalCtx(), p.Exprs, client) if err != nil { @@ -195,7 +196,7 @@ func (p *PhysicalProjection) ToPB(ctx *BuildPBContext, storeType kv.StoreType) ( } // ToPB implements PhysicalPlan ToPB interface. -func (p *PhysicalTopN) ToPB(ctx *BuildPBContext, storeType kv.StoreType) (*tipb.Executor, error) { +func (p *PhysicalTopN) ToPB(ctx *base.BuildPBContext, storeType kv.StoreType) (*tipb.Executor, error) { client := ctx.GetClient() topNExec := &tipb.TopN{ Limit: p.Count, @@ -220,7 +221,7 @@ func (p *PhysicalTopN) ToPB(ctx *BuildPBContext, storeType kv.StoreType) (*tipb. } // ToPB implements PhysicalPlan ToPB interface. -func (p *PhysicalLimit) ToPB(ctx *BuildPBContext, storeType kv.StoreType) (*tipb.Executor, error) { +func (p *PhysicalLimit) ToPB(ctx *base.BuildPBContext, storeType kv.StoreType) (*tipb.Executor, error) { client := ctx.GetClient() limitExec := &tipb.Limit{ Limit: p.Count, @@ -241,7 +242,7 @@ func (p *PhysicalLimit) ToPB(ctx *BuildPBContext, storeType kv.StoreType) (*tipb } // ToPB implements PhysicalPlan ToPB interface. -func (p *PhysicalTableScan) ToPB(ctx *BuildPBContext, storeType kv.StoreType) (*tipb.Executor, error) { +func (p *PhysicalTableScan) ToPB(ctx *base.BuildPBContext, storeType kv.StoreType) (*tipb.Executor, error) { if storeType == kv.TiFlash && p.Table.GetPartitionInfo() != nil && p.IsMPPOrBatchCop && p.SCtx().GetSessionVars().StmtCtx.UseDynamicPartitionPrune() { return p.partitionTableScanToPBForFlash(ctx) } @@ -278,7 +279,7 @@ func (p *PhysicalTableScan) ToPB(ctx *BuildPBContext, storeType kv.StoreType) (* return &tipb.Executor{Tp: tipb.ExecType_TypeTableScan, TblScan: tsExec, ExecutorId: &executorID}, err } -func (p *PhysicalTableScan) partitionTableScanToPBForFlash(ctx *BuildPBContext) (*tipb.Executor, error) { +func (p *PhysicalTableScan) partitionTableScanToPBForFlash(ctx *base.BuildPBContext) (*tipb.Executor, error) { ptsExec := tables.BuildPartitionTableScanFromInfos(p.Table, p.Columns, ctx.TiFlashFastScan) if len(p.LateMaterializationFilterCondition) > 0 { @@ -342,7 +343,7 @@ func FindColumnInfoByID(colInfos []*model.ColumnInfo, id int64) *model.ColumnInf } // ToPB generates the pb structure. -func (e *PhysicalExchangeSender) ToPB(ctx *BuildPBContext, storeType kv.StoreType) (*tipb.Executor, error) { +func (e *PhysicalExchangeSender) ToPB(ctx *base.BuildPBContext, storeType kv.StoreType) (*tipb.Executor, error) { child, err := e.Children()[0].ToPB(ctx, kv.TiFlash) if err != nil { return nil, errors.Trace(err) @@ -417,7 +418,7 @@ func (e *PhysicalExchangeSender) ToPB(ctx *BuildPBContext, storeType kv.StoreTyp } // ToPB generates the pb structure. -func (e *PhysicalExchangeReceiver) ToPB(ctx *BuildPBContext, _ kv.StoreType) (*tipb.Executor, error) { +func (e *PhysicalExchangeReceiver) ToPB(ctx *base.BuildPBContext, _ kv.StoreType) (*tipb.Executor, error) { encodedTask := make([][]byte, 0, len(e.Tasks)) for _, task := range e.Tasks { @@ -456,7 +457,7 @@ func (e *PhysicalExchangeReceiver) ToPB(ctx *BuildPBContext, _ kv.StoreType) (*t } // ToPB implements PhysicalPlan ToPB interface. -func (p *PhysicalIndexScan) ToPB(_ *BuildPBContext, _ kv.StoreType) (*tipb.Executor, error) { +func (p *PhysicalIndexScan) ToPB(_ *base.BuildPBContext, _ kv.StoreType) (*tipb.Executor, error) { columns := make([]*model.ColumnInfo, 0, p.schema.Len()) tableColumns := p.Table.Cols() for _, col := range p.schema.Columns { @@ -490,7 +491,7 @@ func (p *PhysicalIndexScan) ToPB(_ *BuildPBContext, _ kv.StoreType) (*tipb.Execu } // ToPB implements PhysicalPlan ToPB interface. -func (p *PhysicalHashJoin) ToPB(ctx *BuildPBContext, storeType kv.StoreType) (*tipb.Executor, error) { +func (p *PhysicalHashJoin) ToPB(ctx *base.BuildPBContext, storeType kv.StoreType) (*tipb.Executor, error) { client := ctx.GetClient() if len(p.LeftJoinKeys) > 0 && len(p.LeftNAJoinKeys) > 0 { @@ -638,7 +639,7 @@ func (p *PhysicalHashJoin) ToPB(ctx *BuildPBContext, storeType kv.StoreType) (*t } // ToPB converts FrameBound to tipb structure. -func (fb *FrameBound) ToPB(ctx *BuildPBContext) (*tipb.WindowFrameBound, error) { +func (fb *FrameBound) ToPB(ctx *base.BuildPBContext) (*tipb.WindowFrameBound, error) { pbBound := &tipb.WindowFrameBound{ Type: tipb.WindowBoundType(fb.Type), Unbounded: fb.UnBounded, @@ -660,7 +661,7 @@ func (fb *FrameBound) ToPB(ctx *BuildPBContext) (*tipb.WindowFrameBound, error) } // ToPB implements PhysicalPlan ToPB interface. -func (p *PhysicalWindow) ToPB(ctx *BuildPBContext, storeType kv.StoreType) (*tipb.Executor, error) { +func (p *PhysicalWindow) ToPB(ctx *base.BuildPBContext, storeType kv.StoreType) (*tipb.Executor, error) { client := ctx.GetClient() windowExec := &tipb.Window{} @@ -713,7 +714,7 @@ func (p *PhysicalWindow) ToPB(ctx *BuildPBContext, storeType kv.StoreType) (*tip } // ToPB implements PhysicalPlan ToPB interface. -func (p *PhysicalSort) ToPB(ctx *BuildPBContext, storeType kv.StoreType) (*tipb.Executor, error) { +func (p *PhysicalSort) ToPB(ctx *base.BuildPBContext, storeType kv.StoreType) (*tipb.Executor, error) { if !p.IsPartialSort { return nil, errors.Errorf("sort %s can't convert to pb, because it isn't a partial sort", p.Plan.ExplainID()) } diff --git a/pkg/planner/core/planbuilder.go b/pkg/planner/core/planbuilder.go index fc2240147a45d..527ed8f6977bc 100644 --- a/pkg/planner/core/planbuilder.go +++ b/pkg/planner/core/planbuilder.go @@ -41,6 +41,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/opcode" "github.com/pingcap/tidb/pkg/parser/terror" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/privilege" @@ -200,7 +201,7 @@ const ( // PlanBuilder builds Plan from an ast.Node. // It just builds the ast node straightforwardly. type PlanBuilder struct { - ctx PlanContext + ctx base.PlanContext is infoschema.InfoSchema outerSchemas []*expression.Schema outerNames [][]*types.FieldName @@ -444,7 +445,7 @@ func NewPlanBuilder(opts ...PlanBuilderOpt) *PlanBuilder { // PlannerSelectBlockAsName should be restored after using this builder. // This is The comman code pattern to use it: // NewPlanBuilder().Init(sctx, is, processor) -func (b *PlanBuilder) Init(sctx PlanContext, is infoschema.InfoSchema, processor *hint.QBHintHandler) (*PlanBuilder, []ast.HintTable) { +func (b *PlanBuilder) Init(sctx base.PlanContext, is infoschema.InfoSchema, processor *hint.QBHintHandler) (*PlanBuilder, []ast.HintTable) { savedBlockNames := sctx.GetSessionVars().PlannerSelectBlockAsName.Load() if processor == nil { sctx.GetSessionVars().PlannerSelectBlockAsName.Store(&[]ast.HintTable{}) @@ -496,7 +497,7 @@ func (b *PlanBuilder) ResetForReuse() *PlanBuilder { } // Build builds the ast node to a Plan. -func (b *PlanBuilder) Build(ctx context.Context, node ast.Node) (Plan, error) { +func (b *PlanBuilder) Build(ctx context.Context, node ast.Node) (base.Plan, error) { b.optFlag |= flagPrunColumns switch x := node.(type) { case *ast.AdminStmt: @@ -575,7 +576,7 @@ func (b *PlanBuilder) Build(ctx context.Context, node ast.Node) (Plan, error) { return nil, plannererrors.ErrUnsupportedType.GenWithStack("Unsupported type %T", node) } -func (b *PlanBuilder) buildSetConfig(ctx context.Context, v *ast.SetConfigStmt) (Plan, error) { +func (b *PlanBuilder) buildSetConfig(ctx context.Context, v *ast.SetConfigStmt) (base.Plan, error) { privErr := plannererrors.ErrSpecificAccessDenied.GenWithStackByArgs("CONFIG") b.visitInfo = appendVisitInfo(b.visitInfo, mysql.ConfigPriv, "", "", "", privErr) mockTablePlan := LogicalTableDual{}.Init(b.ctx, b.getSelectOffset()) @@ -586,14 +587,14 @@ func (b *PlanBuilder) buildSetConfig(ctx context.Context, v *ast.SetConfigStmt) return &SetConfig{Name: v.Name, Type: v.Type, Instance: v.Instance, Value: expr}, err } -func (*PlanBuilder) buildChange(v *ast.ChangeStmt) (Plan, error) { +func (*PlanBuilder) buildChange(v *ast.ChangeStmt) (base.Plan, error) { exe := &Change{ ChangeStmt: v, } return exe, nil } -func (b *PlanBuilder) buildExecute(ctx context.Context, v *ast.ExecuteStmt) (Plan, error) { +func (b *PlanBuilder) buildExecute(ctx context.Context, v *ast.ExecuteStmt) (base.Plan, error) { vars := make([]expression.Expression, 0, len(v.UsingVars)) for _, expr := range v.UsingVars { newExpr, _, err := b.rewrite(ctx, expr, nil, nil, true) @@ -614,7 +615,7 @@ func (b *PlanBuilder) buildExecute(ctx context.Context, v *ast.ExecuteStmt) (Pla return exe, nil } -func (b *PlanBuilder) buildDo(ctx context.Context, v *ast.DoStmt) (Plan, error) { +func (b *PlanBuilder) buildDo(ctx context.Context, v *ast.DoStmt) (base.Plan, error) { var p LogicalPlan dual := LogicalTableDual{RowCount: 1}.Init(b.ctx, b.getSelectOffset()) dual.SetSchema(expression.NewSchema()) @@ -672,7 +673,7 @@ func (b *PlanBuilder) buildDo(ctx context.Context, v *ast.DoStmt) (Plan, error) return proj, nil } -func (b *PlanBuilder) buildSet(ctx context.Context, v *ast.SetStmt) (Plan, error) { +func (b *PlanBuilder) buildSet(ctx context.Context, v *ast.SetStmt) (base.Plan, error) { p := &Set{} for _, vars := range v.Variables { if vars.IsGlobal { @@ -733,7 +734,7 @@ func (b *PlanBuilder) buildSet(ctx context.Context, v *ast.SetStmt) (Plan, error return p, nil } -func (b *PlanBuilder) buildDropBindPlan(v *ast.DropBindingStmt) (Plan, error) { +func (b *PlanBuilder) buildDropBindPlan(v *ast.DropBindingStmt) (base.Plan, error) { var p *SQLBindPlan if v.OriginNode != nil { normdOrigSQL, sqlDigestWithDB := norm.NormalizeStmtForBinding(v.OriginNode, norm.WithSpecifiedDB(b.ctx.GetSessionVars().CurrentDB)) @@ -758,7 +759,7 @@ func (b *PlanBuilder) buildDropBindPlan(v *ast.DropBindingStmt) (Plan, error) { return p, nil } -func (b *PlanBuilder) buildSetBindingStatusPlan(v *ast.SetBindingStmt) (Plan, error) { +func (b *PlanBuilder) buildSetBindingStatusPlan(v *ast.SetBindingStmt) (base.Plan, error) { var p *SQLBindPlan if v.OriginNode != nil { p = &SQLBindPlan{ @@ -806,7 +807,7 @@ func checkHintedSQL(sql, charset, collation, db string) error { return nil } -func fetchRecordFromClusterStmtSummary(sctx PlanContext, planDigest string) ([]chunk.Row, error) { +func fetchRecordFromClusterStmtSummary(sctx base.PlanContext, planDigest string) ([]chunk.Row, error) { ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnBindInfo) exec := sctx.GetSQLExecutor() fields := "stmt_type, schema_name, digest_text, sample_user, prepared, query_sample_text, charset, collation, plan_hint, plan_digest" @@ -829,7 +830,7 @@ func fetchRecordFromClusterStmtSummary(sctx PlanContext, planDigest string) ([]c return rows, nil } -func (b *PlanBuilder) buildCreateBindPlanFromPlanDigest(v *ast.CreateBindingStmt) (Plan, error) { +func (b *PlanBuilder) buildCreateBindPlanFromPlanDigest(v *ast.CreateBindingStmt) (base.Plan, error) { if v.PlanDigest == "" { return nil, errors.New("plan digest is empty") } @@ -879,7 +880,7 @@ func (b *PlanBuilder) buildCreateBindPlanFromPlanDigest(v *ast.CreateBindingStmt return p, nil } -func (b *PlanBuilder) buildCreateBindPlan(v *ast.CreateBindingStmt) (Plan, error) { +func (b *PlanBuilder) buildCreateBindPlan(v *ast.CreateBindingStmt) (base.Plan, error) { if v.OriginNode == nil { return b.buildCreateBindPlanFromPlanDigest(v) } @@ -1034,7 +1035,7 @@ func isForUpdateReadSelectLock(lock *ast.SelectLockInfo) bool { // getLatestIndexInfo gets the index info of latest schema version from given table id, // it returns nil if the schema version is not changed -func getLatestIndexInfo(ctx PlanContext, id int64, startVer int64) (map[int64]*model.IndexInfo, bool, error) { +func getLatestIndexInfo(ctx base.PlanContext, id int64, startVer int64) (map[int64]*model.IndexInfo, bool, error) { dom := domain.GetDomain(ctx) if dom == nil { return nil, false, errors.New("domain not found for ctx") @@ -1055,7 +1056,7 @@ func getLatestIndexInfo(ctx PlanContext, id int64, startVer int64) (map[int64]*m return latestIndexes, true, nil } -func getPossibleAccessPaths(ctx PlanContext, tableHints *hint.PlanHints, indexHints []*ast.IndexHint, tbl table.Table, dbName, tblName model.CIStr, check bool, hasFlagPartitionProcessor bool) ([]*util.AccessPath, error) { +func getPossibleAccessPaths(ctx base.PlanContext, tableHints *hint.PlanHints, indexHints []*ast.IndexHint, tbl table.Table, dbName, tblName model.CIStr, check bool, hasFlagPartitionProcessor bool) ([]*util.AccessPath, error) { tblInfo := tbl.Meta() publicPaths := make([]*util.AccessPath, 0, len(tblInfo.Indices)+2) tp := kv.TiKV @@ -1234,7 +1235,7 @@ func getPossibleAccessPaths(ctx PlanContext, tableHints *hint.PlanHints, indexHi return available, nil } -func filterPathByIsolationRead(ctx PlanContext, paths []*util.AccessPath, tblName model.CIStr, dbName model.CIStr) ([]*util.AccessPath, error) { +func filterPathByIsolationRead(ctx base.PlanContext, paths []*util.AccessPath, tblName model.CIStr, dbName model.CIStr) ([]*util.AccessPath, error) { // TODO: filter paths with isolation read locations. if util2.IsSysDB(dbName.L) { return paths, nil @@ -1342,7 +1343,7 @@ func setExtraPhysTblIDColsOnDataSource(p LogicalPlan, tblID2PhysTblIDCol map[int } } -func (b *PlanBuilder) buildPrepare(x *ast.PrepareStmt) Plan { +func (b *PlanBuilder) buildPrepare(x *ast.PrepareStmt) base.Plan { p := &Prepare{ Name: x.Name, } @@ -1363,8 +1364,8 @@ func (b *PlanBuilder) buildPrepare(x *ast.PrepareStmt) Plan { return p } -func (b *PlanBuilder) buildAdmin(ctx context.Context, as *ast.AdminStmt) (Plan, error) { - var ret Plan +func (b *PlanBuilder) buildAdmin(ctx context.Context, as *ast.AdminStmt) (base.Plan, error) { + var ret base.Plan var err error switch as.Tp { case ast.AdminCheckTable, ast.AdminCheckIndex: @@ -1478,7 +1479,7 @@ func (b *PlanBuilder) buildAdmin(ctx context.Context, as *ast.AdminStmt) (Plan, return ret, nil } -func (b *PlanBuilder) buildPhysicalIndexLookUpReader(_ context.Context, dbName model.CIStr, tbl table.Table, idx *model.IndexInfo) (Plan, error) { +func (b *PlanBuilder) buildPhysicalIndexLookUpReader(_ context.Context, dbName model.CIStr, tbl table.Table, idx *model.IndexInfo) (base.Plan, error) { tblInfo := tbl.Meta() physicalID, isPartition := getPhysicalID(tbl) fullExprCols, _, err := expression.TableInfo2SchemaAndNames(b.ctx.GetExprCtx(), dbName, tblInfo) @@ -1562,7 +1563,7 @@ func (b *PlanBuilder) buildPhysicalIndexLookUpReader(_ context.Context, dbName m extraHandleCol: extraCol, commonHandleCols: commonCols, } - rootT := cop.ConvertToRootTask(b.ctx) + rootT := cop.ConvertToRootTask(b.ctx).(*RootTask) if err := rootT.GetPlan().ResolveIndices(); err != nil { return nil, err } @@ -1635,11 +1636,11 @@ func tryGetPkHandleCol(tblInfo *model.TableInfo, allColSchema *expression.Schema return nil, nil, false } -func (b *PlanBuilder) buildPhysicalIndexLookUpReaders(ctx context.Context, dbName model.CIStr, tbl table.Table, indices []table.Index) ([]Plan, []*model.IndexInfo, error) { +func (b *PlanBuilder) buildPhysicalIndexLookUpReaders(ctx context.Context, dbName model.CIStr, tbl table.Table, indices []table.Index) ([]base.Plan, []*model.IndexInfo, error) { tblInfo := tbl.Meta() // get index information indexInfos := make([]*model.IndexInfo, 0, len(tblInfo.Indices)) - indexLookUpReaders := make([]Plan, 0, len(tblInfo.Indices)) + indexLookUpReaders := make([]base.Plan, 0, len(tblInfo.Indices)) check := b.isForUpdateRead || b.ctx.GetSessionVars().IsIsolation(ast.ReadCommitted) check = check && b.ctx.GetSessionVars().ConnectionID > 0 @@ -1716,7 +1717,7 @@ func (b *PlanBuilder) buildAdminCheckTable(ctx context.Context, as *ast.AdminStm DBName: tblName.Schema.O, Table: tbl, } - var readerPlans []Plan + var readerPlans []base.Plan var indexInfos []*model.IndexInfo var err error if as.Tp == ast.AdminCheckIndex { @@ -1813,7 +1814,7 @@ func getColsInfo(tn *ast.TableName) (indicesInfo []*model.IndexInfo, colsInfo [] } // BuildHandleColsForAnalyze returns HandleCols for ANALYZE. -func BuildHandleColsForAnalyze(ctx PlanContext, tblInfo *model.TableInfo, allColumns bool, colsInfo []*model.ColumnInfo) HandleCols { +func BuildHandleColsForAnalyze(ctx base.PlanContext, tblInfo *model.TableInfo, allColumns bool, colsInfo []*model.ColumnInfo) HandleCols { var handleCols HandleCols switch { case tblInfo.PKIsHandle: @@ -2480,7 +2481,7 @@ func pickColumnList(astColChoice model.ColumnChoice, astColList []*model.ColumnI } // buildAnalyzeTable constructs analyze tasks for each table. -func (b *PlanBuilder) buildAnalyzeTable(as *ast.AnalyzeTableStmt, opts map[ast.AnalyzeOptionType]uint64, version int) (Plan, error) { +func (b *PlanBuilder) buildAnalyzeTable(as *ast.AnalyzeTableStmt, opts map[ast.AnalyzeOptionType]uint64, version int) (base.Plan, error) { p := &Analyze{Opts: opts} p.OptionsMap = make(map[int64]V2AnalyzeOptions) usePersistedOptions := variable.PersistAnalyzeOptions.Load() @@ -2555,7 +2556,7 @@ func (b *PlanBuilder) buildAnalyzeTable(as *ast.AnalyzeTableStmt, opts map[ast.A return p, nil } -func (b *PlanBuilder) buildAnalyzeIndex(as *ast.AnalyzeTableStmt, opts map[ast.AnalyzeOptionType]uint64, version int) (Plan, error) { +func (b *PlanBuilder) buildAnalyzeIndex(as *ast.AnalyzeTableStmt, opts map[ast.AnalyzeOptionType]uint64, version int) (base.Plan, error) { p := &Analyze{Opts: opts} statsHandle := domain.GetDomain(b.ctx).StatsHandle() if statsHandle == nil { @@ -2607,7 +2608,7 @@ func (b *PlanBuilder) buildAnalyzeIndex(as *ast.AnalyzeTableStmt, opts map[ast.A return p, nil } -func (b *PlanBuilder) buildAnalyzeAllIndex(as *ast.AnalyzeTableStmt, opts map[ast.AnalyzeOptionType]uint64, version int) (Plan, error) { +func (b *PlanBuilder) buildAnalyzeAllIndex(as *ast.AnalyzeTableStmt, opts map[ast.AnalyzeOptionType]uint64, version int) (base.Plan, error) { p := &Analyze{Opts: opts} statsHandle := domain.GetDomain(b.ctx).StatsHandle() if statsHandle == nil { @@ -2828,7 +2829,7 @@ func handleAnalyzeOptions(opts []ast.AnalyzeOpt, statsVer int) (map[ast.AnalyzeO return optMap, nil } -func (b *PlanBuilder) buildAnalyze(as *ast.AnalyzeTableStmt) (Plan, error) { +func (b *PlanBuilder) buildAnalyze(as *ast.AnalyzeTableStmt) (base.Plan, error) { if as.NoWriteToBinLog { return nil, dbterror.ErrNotSupportedYet.GenWithStackByArgs("[NO_WRITE_TO_BINLOG | LOCAL]") } @@ -3121,7 +3122,7 @@ func splitWhere(where ast.ExprNode) []ast.ExprNode { return conditions } -func (b *PlanBuilder) buildShow(ctx context.Context, show *ast.ShowStmt) (Plan, error) { +func (b *PlanBuilder) buildShow(ctx context.Context, show *ast.ShowStmt) (base.Plan, error) { p := LogicalShow{ ShowContents: ShowContents{ Tp: show.Tp, @@ -3289,7 +3290,7 @@ func (b *PlanBuilder) buildShow(ctx context.Context, show *ast.ShowStmt) (Plan, return np, nil } -func (b *PlanBuilder) buildSimple(ctx context.Context, node ast.StmtNode) (Plan, error) { +func (b *PlanBuilder) buildSimple(ctx context.Context, node ast.StmtNode) (base.Plan, error) { p := &Simple{Statement: node} switch raw := node.(type) { @@ -3409,7 +3410,7 @@ func (b *PlanBuilder) buildSimple(ctx context.Context, node ast.StmtNode) (Plan, return p, nil } -func collectVisitInfoFromRevokeStmt(sctx PlanContext, vi []visitInfo, stmt *ast.RevokeStmt) ([]visitInfo, error) { +func collectVisitInfoFromRevokeStmt(sctx base.PlanContext, vi []visitInfo, stmt *ast.RevokeStmt) ([]visitInfo, error) { // To use REVOKE, you must have the GRANT OPTION privilege, // and you must have the privileges that you are granting. dbName := stmt.Level.DBName @@ -3461,7 +3462,7 @@ func collectVisitInfoFromRevokeStmt(sctx PlanContext, vi []visitInfo, stmt *ast. // appendVisitInfoIsRestrictedUser appends additional visitInfo if the user has a // special privilege called "RESTRICTED_USER_ADMIN". It only applies when SEM is enabled. -func appendVisitInfoIsRestrictedUser(visitInfo []visitInfo, sctx PlanContext, user *auth.UserIdentity, priv string) []visitInfo { +func appendVisitInfoIsRestrictedUser(visitInfo []visitInfo, sctx base.PlanContext, user *auth.UserIdentity, priv string) []visitInfo { if !sem.IsEnabled() { return visitInfo } @@ -3473,7 +3474,7 @@ func appendVisitInfoIsRestrictedUser(visitInfo []visitInfo, sctx PlanContext, us return visitInfo } -func collectVisitInfoFromGrantStmt(sctx PlanContext, vi []visitInfo, stmt *ast.GrantStmt) ([]visitInfo, error) { +func collectVisitInfoFromGrantStmt(sctx base.PlanContext, vi []visitInfo, stmt *ast.GrantStmt) ([]visitInfo, error) { // To use GRANT, you must have the GRANT OPTION privilege, // and you must have the privileges that you are granting. dbName := stmt.Level.DBName @@ -3532,7 +3533,7 @@ func collectVisitInfoFromGrantStmt(sctx PlanContext, vi []visitInfo, stmt *ast.G return vi, nil } -func genAuthErrForGrantStmt(sctx PlanContext, dbName string) error { +func genAuthErrForGrantStmt(sctx base.PlanContext, dbName string) error { if !strings.EqualFold(dbName, variable.PerformanceSchema) { return nil } @@ -3608,7 +3609,7 @@ func (b *PlanBuilder) resolveGeneratedColumns(ctx context.Context, columns []*ta return igc, nil } -func (b *PlanBuilder) buildInsert(ctx context.Context, insert *ast.InsertStmt) (Plan, error) { +func (b *PlanBuilder) buildInsert(ctx context.Context, insert *ast.InsertStmt) (base.Plan, error) { ts, ok := insert.Table.TableRefs.Left.(*ast.TableSource) if !ok { return nil, infoschema.ErrTableNotExists.FastGenByArgs() @@ -4078,7 +4079,7 @@ func (b *PlanBuilder) buildSelectPlanOfInsert(ctx context.Context, insert *ast.I return nil } -func (b *PlanBuilder) buildLoadData(ctx context.Context, ld *ast.LoadDataStmt) (Plan, error) { +func (b *PlanBuilder) buildLoadData(ctx context.Context, ld *ast.LoadDataStmt) (base.Plan, error) { mockTablePlan := LogicalTableDual{}.Init(b.ctx, b.getSelectOffset()) var ( err error @@ -4148,7 +4149,7 @@ var ( ImportIntoDataSource = "data source" ) -func (b *PlanBuilder) buildImportInto(ctx context.Context, ld *ast.ImportIntoStmt) (Plan, error) { +func (b *PlanBuilder) buildImportInto(ctx context.Context, ld *ast.ImportIntoStmt) (base.Plan, error) { mockTablePlan := LogicalTableDual{}.Init(b.ctx, b.getSelectOffset()) var ( err error @@ -4260,13 +4261,13 @@ func (b *PlanBuilder) buildImportInto(ctx context.Context, ld *ast.ImportIntoStm return p, nil } -func (*PlanBuilder) buildLoadStats(ld *ast.LoadStatsStmt) Plan { +func (*PlanBuilder) buildLoadStats(ld *ast.LoadStatsStmt) base.Plan { p := &LoadStats{Path: ld.Path} return p } // buildLockStats requires INSERT and SELECT privilege for the tables same as buildAnalyze. -func (b *PlanBuilder) buildLockStats(ld *ast.LockStatsStmt) Plan { +func (b *PlanBuilder) buildLockStats(ld *ast.LockStatsStmt) base.Plan { p := &LockStats{ Tables: ld.Tables, } @@ -4277,7 +4278,7 @@ func (b *PlanBuilder) buildLockStats(ld *ast.LockStatsStmt) Plan { } // buildUnlockStats requires INSERT and SELECT privilege for the tables same as buildAnalyze. -func (b *PlanBuilder) buildUnlockStats(ld *ast.UnlockStatsStmt) Plan { +func (b *PlanBuilder) buildUnlockStats(ld *ast.UnlockStatsStmt) base.Plan { p := &UnlockStats{ Tables: ld.Tables, } @@ -4300,7 +4301,7 @@ func (b *PlanBuilder) requireInsertAndSelectPriv(tables []*ast.TableName) { } } -func (*PlanBuilder) buildIndexAdvise(node *ast.IndexAdviseStmt) Plan { +func (*PlanBuilder) buildIndexAdvise(node *ast.IndexAdviseStmt) base.Plan { p := &IndexAdvise{ IsLocal: node.IsLocal, Path: node.Path, @@ -4311,7 +4312,7 @@ func (*PlanBuilder) buildIndexAdvise(node *ast.IndexAdviseStmt) Plan { return p } -func (b *PlanBuilder) buildSplitRegion(node *ast.SplitRegionStmt) (Plan, error) { +func (b *PlanBuilder) buildSplitRegion(node *ast.SplitRegionStmt) (base.Plan, error) { if node.Table.TableInfo.TempTableType != model.TempTableNone { return nil, plannererrors.ErrOptOnTemporaryTable.GenWithStackByArgs("split table") } @@ -4324,7 +4325,7 @@ func (b *PlanBuilder) buildSplitRegion(node *ast.SplitRegionStmt) (Plan, error) return b.buildSplitTableRegion(node) } -func (b *PlanBuilder) buildSplitIndexRegion(node *ast.SplitRegionStmt) (Plan, error) { +func (b *PlanBuilder) buildSplitIndexRegion(node *ast.SplitRegionStmt) (base.Plan, error) { tblInfo := node.Table.TableInfo if node.IndexName.L == strings.ToLower(mysql.PrimaryKeyName) && (tblInfo.IsCommonHandle || tblInfo.PKIsHandle) { @@ -4447,7 +4448,7 @@ func (b *PlanBuilder) convertValue(valueItem ast.ExprNode, mockTablePlan Logical return d, nil } -func (b *PlanBuilder) buildSplitTableRegion(node *ast.SplitRegionStmt) (Plan, error) { +func (b *PlanBuilder) buildSplitTableRegion(node *ast.SplitRegionStmt) (base.Plan, error) { tblInfo := node.Table.TableInfo handleColInfos := buildHandleColumnInfos(tblInfo) mockTablePlan := LogicalTableDual{}.Init(b.ctx, b.getSelectOffset()) @@ -4548,7 +4549,7 @@ func convertValueListToData(valueList []ast.ExprNode, handleColInfos []*model.Co return data, nil } -func (b *PlanBuilder) buildDDL(ctx context.Context, node ast.DDLNode) (Plan, error) { +func (b *PlanBuilder) buildDDL(ctx context.Context, node ast.DDLNode) (base.Plan, error) { var authErr error switch v := node.(type) { case *ast.AlterDatabaseStmt: @@ -4895,7 +4896,7 @@ const ( // buildTrace builds a trace plan. Inside this method, it first optimize the // underlying query and then constructs a schema, which will be used to constructs // rows result. -func (*PlanBuilder) buildTrace(trace *ast.TraceStmt) (Plan, error) { +func (*PlanBuilder) buildTrace(trace *ast.TraceStmt) (base.Plan, error) { p := &Trace{ StmtNode: trace.Stmt, Format: trace.Format, @@ -4952,7 +4953,7 @@ func (*PlanBuilder) buildTrace(trace *ast.TraceStmt) (Plan, error) { return p, nil } -func (b *PlanBuilder) buildExplainPlan(targetPlan Plan, format string, explainRows [][]string, analyze bool, execStmt ast.StmtNode, runtimeStats *execdetails.RuntimeStatsColl) (Plan, error) { +func (b *PlanBuilder) buildExplainPlan(targetPlan base.Plan, format string, explainRows [][]string, analyze bool, execStmt ast.StmtNode, runtimeStats *execdetails.RuntimeStatsColl) (base.Plan, error) { if strings.ToLower(format) == types.ExplainFormatTrueCardCost && !analyze { return nil, errors.Errorf("'explain format=%v' cannot work without 'analyze', please use 'explain analyze format=%v'", format, format) } @@ -4971,7 +4972,7 @@ func (b *PlanBuilder) buildExplainPlan(targetPlan Plan, format string, explainRo // buildExplainFor gets *last* (maybe running or finished) query plan from connection #connection id. // See https://dev.mysql.com/doc/refman/8.0/en/explain-for-connection.html. -func (b *PlanBuilder) buildExplainFor(explainFor *ast.ExplainForStmt) (Plan, error) { +func (b *PlanBuilder) buildExplainFor(explainFor *ast.ExplainForStmt) (base.Plan, error) { processInfo, ok := b.ctx.GetSessionManager().GetProcessInfo(explainFor.ConnectionID) if !ok { return nil, plannererrors.ErrNoSuchThread.GenWithStackByArgs(explainFor.ConnectionID) @@ -4984,7 +4985,7 @@ func (b *PlanBuilder) buildExplainFor(explainFor *ast.ExplainForStmt) (Plan, err } } - targetPlan, ok := processInfo.Plan.(Plan) + targetPlan, ok := processInfo.Plan.(base.Plan) if !ok || targetPlan == nil { return &Explain{Format: explainFor.Format}, nil } @@ -4995,7 +4996,7 @@ func (b *PlanBuilder) buildExplainFor(explainFor *ast.ExplainForStmt) (Plan, err return b.buildExplainPlan(targetPlan, explainFor.Format, explainRows, false, nil, processInfo.RuntimeStatsColl) } -func (b *PlanBuilder) buildExplain(ctx context.Context, explain *ast.ExplainStmt) (Plan, error) { +func (b *PlanBuilder) buildExplain(ctx context.Context, explain *ast.ExplainStmt) (base.Plan, error) { if show, ok := explain.Stmt.(*ast.ShowStmt); ok { return b.buildShow(ctx, show) } @@ -5013,7 +5014,7 @@ func (b *PlanBuilder) buildExplain(ctx context.Context, explain *ast.ExplainStmt return b.buildExplainPlan(targetPlan, explain.Format, nil, explain.Analyze, explain.Stmt, nil) } -func (b *PlanBuilder) buildSelectInto(ctx context.Context, sel *ast.SelectStmt) (Plan, error) { +func (b *PlanBuilder) buildSelectInto(ctx context.Context, sel *ast.SelectStmt) (base.Plan, error) { if sem.IsEnabled() { return nil, plannererrors.ErrNotSupportedWithSem.GenWithStackByArgs("SELECT INTO") } @@ -5312,7 +5313,7 @@ func convert2OutputSchemasAndNames(names []string, ftypes []byte) (schema *expre return } -func (b *PlanBuilder) buildPlanReplayer(pc *ast.PlanReplayerStmt) Plan { +func (b *PlanBuilder) buildPlanReplayer(pc *ast.PlanReplayerStmt) base.Plan { p := &PlanReplayer{ExecStmt: pc.Stmt, Analyze: pc.Analyze, Load: pc.Load, File: pc.File, Capture: pc.Capture, Remove: pc.Remove, SQLDigest: pc.SQLDigest, PlanDigest: pc.PlanDigest} @@ -5327,7 +5328,7 @@ func (b *PlanBuilder) buildPlanReplayer(pc *ast.PlanReplayerStmt) Plan { return p } -func calcTSForPlanReplayer(sctx PlanContext, tsExpr ast.ExprNode) uint64 { +func calcTSForPlanReplayer(sctx base.PlanContext, tsExpr ast.ExprNode) uint64 { tsVal, err := evalAstExprWithPlanCtx(sctx, tsExpr) if err != nil { sctx.GetSessionVars().StmtCtx.AppendWarning(err) @@ -5404,7 +5405,7 @@ func findStmtAsViewSchema(stmt ast.Node) *ast.SelectStmt { } // buildCompactTable builds a plan for the "ALTER TABLE [NAME] COMPACT ..." statement. -func (b *PlanBuilder) buildCompactTable(node *ast.CompactTableStmt) (Plan, error) { +func (b *PlanBuilder) buildCompactTable(node *ast.CompactTableStmt) (base.Plan, error) { var authErr error if b.ctx.GetSessionVars().User != nil { authErr = plannererrors.ErrTableaccessDenied.GenWithStackByArgs("ALTER", b.ctx.GetSessionVars().User.AuthUsername, diff --git a/pkg/planner/core/planbuilder_test.go b/pkg/planner/core/planbuilder_test.go index c09a0e8e20a09..1fb7193c53f1a 100644 --- a/pkg/planner/core/planbuilder_test.go +++ b/pkg/planner/core/planbuilder_test.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/statistics" @@ -205,7 +206,7 @@ func TestDeepClone(t *testing.T) { byItems := []*util.ByItems{{Expr: expr}} sort1 := &PhysicalSort{ByItems: byItems} sort2 := &PhysicalSort{ByItems: byItems} - checkDeepClone := func(p1, p2 PhysicalPlan) error { + checkDeepClone := func(p1, p2 base.PhysicalPlan) error { whiteList := []string{"*property.StatsInfo", "*sessionctx.Context", "*mock.Context"} return checkDeepClonedCore(reflect.ValueOf(p1), reflect.ValueOf(p2), typeName(reflect.TypeOf(p1)), whiteList, nil) } @@ -251,7 +252,7 @@ func TestTablePlansAndTablePlanInPhysicalTableReaderClone(t *testing.T) { // table reader tableReader := &PhysicalTableReader{ tablePlan: tableScan, - TablePlans: []PhysicalPlan{tableScan}, + TablePlans: []base.PhysicalPlan{tableScan}, StoreType: kv.TiFlash, } tableReader = tableReader.Init(ctx, 0) @@ -287,7 +288,7 @@ func TestPhysicalPlanClone(t *testing.T) { // table reader tableReader := &PhysicalTableReader{ tablePlan: tableScan, - TablePlans: []PhysicalPlan{tableScan}, + TablePlans: []base.PhysicalPlan{tableScan}, StoreType: kv.TiFlash, } tableReader = tableReader.Init(ctx, 0) @@ -307,7 +308,7 @@ func TestPhysicalPlanClone(t *testing.T) { // index reader indexReader := &PhysicalIndexReader{ indexPlan: indexScan, - IndexPlans: []PhysicalPlan{indexScan}, + IndexPlans: []base.PhysicalPlan{indexScan}, OutputColumns: []*expression.Column{col, col}, } indexReader = indexReader.Init(ctx, 0) @@ -315,9 +316,9 @@ func TestPhysicalPlanClone(t *testing.T) { // index lookup indexLookup := &PhysicalIndexLookUpReader{ - IndexPlans: []PhysicalPlan{indexReader}, + IndexPlans: []base.PhysicalPlan{indexReader}, indexPlan: indexScan, - TablePlans: []PhysicalPlan{tableReader}, + TablePlans: []base.PhysicalPlan{tableReader}, tablePlan: tableScan, ExtraHandleCol: col, PushedLimit: &PushedDownLimit{1, 2}, @@ -402,7 +403,7 @@ func typeName(t reflect.Type) string { return tmp[len(tmp)-1] } -func checkPhysicalPlanClone(p PhysicalPlan) error { +func checkPhysicalPlanClone(p base.PhysicalPlan) error { cloned, err := p.Clone() if err != nil { return err diff --git a/pkg/planner/core/point_get_plan.go b/pkg/planner/core/point_get_plan.go index 54fc451a93ece..342c58c1b3c19 100644 --- a/pkg/planner/core/point_get_plan.go +++ b/pkg/planner/core/point_get_plan.go @@ -33,6 +33,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/opcode" "github.com/pingcap/tidb/pkg/parser/terror" ptypes "github.com/pingcap/tidb/pkg/parser/types" + base2 "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/core/internal/base" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util/coreusage" @@ -84,7 +85,7 @@ type PointGetPlan struct { IdxCols []*expression.Column IdxColLens []int AccessConditions []expression.Expression - ctx PlanContext + ctx base2.PlanContext UnsignedHandle bool IsTableDual bool Lock bool @@ -102,7 +103,7 @@ type PointGetPlan struct { // probeParents records the IndexJoins and Applys with this operator in their inner children. // Please see comments in PhysicalPlan for details. - probeParents []PhysicalPlan + probeParents []base2.PhysicalPlan // stmtHints should restore in executing context. stmtHints *hint.StmtHints // explicit partition selection @@ -126,7 +127,7 @@ func (p *PointGetPlan) GetActualProbeCnt(statsColl *execdetails.RuntimeStatsColl } // SetProbeParents implements PhysicalPlan interface. -func (p *PointGetPlan) SetProbeParents(probeParents []PhysicalPlan) { +func (p *PointGetPlan) SetProbeParents(probeParents []base2.PhysicalPlan) { p.probeParents = probeParents } @@ -154,17 +155,17 @@ func (p *PointGetPlan) SetCost(cost float64) { // Attach2Task makes the current physical plan as the father of task's physicalPlan and updates the cost of // current task. If the child's task is cop task, some operator may close this task and return a new rootTask. -func (*PointGetPlan) Attach2Task(...Task) Task { +func (*PointGetPlan) Attach2Task(...base2.Task) base2.Task { return nil } // ToPB converts physical plan to tipb executor. -func (*PointGetPlan) ToPB(_ *BuildPBContext, _ kv.StoreType) (*tipb.Executor, error) { +func (*PointGetPlan) ToPB(_ *base2.BuildPBContext, _ kv.StoreType) (*tipb.Executor, error) { return nil, nil } // Clone implements PhysicalPlan interface. -func (p *PointGetPlan) Clone() (PhysicalPlan, error) { +func (p *PointGetPlan) Clone() (base2.PhysicalPlan, error) { return nil, errors.Errorf("%T doesn't support cloning", p) } @@ -239,15 +240,15 @@ func (p *PointGetPlan) StatsInfo() *property.StatsInfo { } // Children gets all the children. -func (*PointGetPlan) Children() []PhysicalPlan { +func (*PointGetPlan) Children() []base2.PhysicalPlan { return nil } // SetChildren sets the children for the plan. -func (*PointGetPlan) SetChildren(...PhysicalPlan) {} +func (*PointGetPlan) SetChildren(...base2.PhysicalPlan) {} // SetChild sets a specific child for the plan. -func (*PointGetPlan) SetChild(_ int, _ PhysicalPlan) {} +func (*PointGetPlan) SetChild(_ int, _ base2.PhysicalPlan) {} // ResolveIndices resolves the indices for columns. After doing this, the columns can evaluate the rows by their indices. func (p *PointGetPlan) ResolveIndices() error { @@ -409,7 +410,7 @@ func (p *PointGetPlan) PrunePartitions(sctx sessionctx.Context) bool { type BatchPointGetPlan struct { baseSchemaProducer - ctx PlanContext + ctx base2.PlanContext dbName string TblInfo *model.TableInfo IndexInfo *model.IndexInfo @@ -445,7 +446,7 @@ type BatchPointGetPlan struct { // probeParents records the IndexJoins and Applys with this operator in their inner children. // Please see comments in PhysicalPlan for details. - probeParents []PhysicalPlan + probeParents []base2.PhysicalPlan // explicit partition selection PartitionNames []model.CIStr } @@ -467,7 +468,7 @@ func (p *BatchPointGetPlan) GetActualProbeCnt(statsColl *execdetails.RuntimeStat } // SetProbeParents implements PhysicalPlan interface. -func (p *BatchPointGetPlan) SetProbeParents(probeParents []PhysicalPlan) { +func (p *BatchPointGetPlan) SetProbeParents(probeParents []base2.PhysicalPlan) { p.probeParents = probeParents } @@ -482,7 +483,7 @@ func (p *BatchPointGetPlan) SetCost(cost float64) { } // Clone implements PhysicalPlan interface. -func (p *BatchPointGetPlan) Clone() (PhysicalPlan, error) { +func (p *BatchPointGetPlan) Clone() (base2.PhysicalPlan, error) { return nil, errors.Errorf("%T doesn't support cloning", p) } @@ -493,12 +494,12 @@ func (*BatchPointGetPlan) ExtractCorrelatedCols() []*expression.CorrelatedColumn // Attach2Task makes the current physical plan as the father of task's physicalPlan and updates the cost of // current task. If the child's task is cop task, some operator may close this task and return a new rootTask. -func (*BatchPointGetPlan) Attach2Task(...Task) Task { +func (*BatchPointGetPlan) Attach2Task(...base2.Task) base2.Task { return nil } // ToPB converts physical plan to tipb executor. -func (*BatchPointGetPlan) ToPB(_ *BuildPBContext, _ kv.StoreType) (*tipb.Executor, error) { +func (*BatchPointGetPlan) ToPB(_ *base2.BuildPBContext, _ kv.StoreType) (*tipb.Executor, error) { return nil, nil } @@ -555,15 +556,15 @@ func (p *BatchPointGetPlan) StatsInfo() *property.StatsInfo { } // Children gets all the children. -func (*BatchPointGetPlan) Children() []PhysicalPlan { +func (*BatchPointGetPlan) Children() []base2.PhysicalPlan { return nil } // SetChildren sets the children for the plan. -func (*BatchPointGetPlan) SetChildren(...PhysicalPlan) {} +func (*BatchPointGetPlan) SetChildren(...base2.PhysicalPlan) {} // SetChild sets a specific child for the plan. -func (*BatchPointGetPlan) SetChild(_ int, _ PhysicalPlan) {} +func (*BatchPointGetPlan) SetChild(_ int, _ base2.PhysicalPlan) {} // ResolveIndices resolves the indices for columns. After doing this, the columns can evaluate the rows by their indices. func (p *BatchPointGetPlan) ResolveIndices() error { @@ -859,11 +860,11 @@ const PointPlanKey = stringutil.StringerStr("pointPlanKey") // PointPlanVal is used to store point plan that is pre-built for multi-statement query. // Save the plan in a struct so even if the point plan is nil, we don't need to try again. type PointPlanVal struct { - Plan Plan + Plan base2.Plan } // TryFastPlan tries to use the PointGetPlan for the query. -func TryFastPlan(ctx PlanContext, node ast.Node) (p Plan) { +func TryFastPlan(ctx base2.PlanContext, node ast.Node) (p base2.Plan) { if checkStableResultMode(ctx) { // the rule of stabilizing results has not taken effect yet, so cannot generate a plan here in this mode return nil @@ -936,7 +937,7 @@ func IsSelectForUpdateLockType(lockType ast.SelectLockType) bool { return false } -func getLockWaitTime(ctx PlanContext, lockInfo *ast.SelectLockInfo) (lock bool, waitTime int64) { +func getLockWaitTime(ctx base2.PlanContext, lockInfo *ast.SelectLockInfo) (lock bool, waitTime int64) { if lockInfo != nil { if IsSelectForUpdateLockType(lockInfo.LockType) { // Locking of rows for update using SELECT FOR UPDATE only applies when autocommit @@ -960,7 +961,7 @@ func getLockWaitTime(ctx PlanContext, lockInfo *ast.SelectLockInfo) (lock bool, } func newBatchPointGetPlan( - ctx PlanContext, patternInExpr *ast.PatternInExpr, + ctx base2.PlanContext, patternInExpr *ast.PatternInExpr, handleCol *model.ColumnInfo, tbl *model.TableInfo, schema *expression.Schema, names []*types.FieldName, whereColNames []string, indexHints []*ast.IndexHint, ) *BatchPointGetPlan { @@ -1198,7 +1199,7 @@ func newBatchPointGetPlan( return p.Init(ctx, statsInfo, schema, names, 0) } -func tryWhereIn2BatchPointGet(ctx PlanContext, selStmt *ast.SelectStmt) *BatchPointGetPlan { +func tryWhereIn2BatchPointGet(ctx base2.PlanContext, selStmt *ast.SelectStmt) *BatchPointGetPlan { if selStmt.OrderBy != nil || selStmt.GroupBy != nil || selStmt.Limit != nil || selStmt.Having != nil || selStmt.Distinct || len(selStmt.WindowSpecs) > 0 { @@ -1299,7 +1300,7 @@ func tryWhereIn2BatchPointGet(ctx PlanContext, selStmt *ast.SelectStmt) *BatchPo // 2. It must be a single table select. // 3. All the columns must be public and not generated. // 4. The condition is an access path that the range is a unique key. -func tryPointGetPlan(ctx PlanContext, selStmt *ast.SelectStmt, check bool) *PointGetPlan { +func tryPointGetPlan(ctx base2.PlanContext, selStmt *ast.SelectStmt, check bool) *PointGetPlan { if selStmt.Having != nil || selStmt.OrderBy != nil { return nil } else if selStmt.Limit != nil { @@ -1369,7 +1370,7 @@ func tryPointGetPlan(ctx PlanContext, selStmt *ast.SelectStmt, check bool) *Poin return checkTblIndexForPointPlan(ctx, tblName, schema, names, pairs, isTableDual, check) } -func checkTblIndexForPointPlan(ctx PlanContext, tblName *ast.TableName, schema *expression.Schema, +func checkTblIndexForPointPlan(ctx base2.PlanContext, tblName *ast.TableName, schema *expression.Schema, names []*types.FieldName, pairs []nameValuePair, isTableDual, check bool) *PointGetPlan { check = check || ctx.GetSessionVars().IsIsolation(ast.ReadCommitted) check = check && ctx.GetSessionVars().ConnectionID > 0 @@ -1475,7 +1476,7 @@ func indexIsAvailableByHints(idxInfo *model.IndexInfo, idxHints []*ast.IndexHint return isIgnore } -func newPointGetPlan(ctx PlanContext, dbName string, schema *expression.Schema, tbl *model.TableInfo, names []*types.FieldName) *PointGetPlan { +func newPointGetPlan(ctx base2.PlanContext, dbName string, schema *expression.Schema, tbl *model.TableInfo, names []*types.FieldName) *PointGetPlan { p := &PointGetPlan{ Plan: base.NewBasePlan(ctx, plancodec.TypePointGet, 0), dbName: dbName, @@ -1488,7 +1489,7 @@ func newPointGetPlan(ctx PlanContext, dbName string, schema *expression.Schema, return p } -func checkFastPlanPrivilege(ctx PlanContext, dbName, tableName string, checkTypes ...mysql.PrivilegeType) error { +func checkFastPlanPrivilege(ctx base2.PlanContext, dbName, tableName string, checkTypes ...mysql.PrivilegeType) error { pm := privilege.GetPrivilegeManager(ctx) visitInfos := make([]visitInfo, 0, len(checkTypes)) for _, checkType := range checkTypes { @@ -1639,7 +1640,7 @@ func getSingleTableNameAndAlias(tableRefs *ast.TableRefsClause) (tblName *ast.Ta } // getNameValuePairs extracts `column = constant/paramMarker` conditions from expr as name value pairs. -func getNameValuePairs(ctx PlanContext, tbl *model.TableInfo, tblName model.CIStr, nvPairs []nameValuePair, expr ast.ExprNode) ( +func getNameValuePairs(ctx base2.PlanContext, tbl *model.TableInfo, tblName model.CIStr, nvPairs []nameValuePair, expr ast.ExprNode) ( pairs []nameValuePair, isTableDual bool) { stmtCtx := ctx.GetSessionVars().StmtCtx binOp, ok := expr.(*ast.BinaryOperationExpr) @@ -1881,7 +1882,7 @@ func checkIfAssignmentListHasSubQuery(list []*ast.Assignment) bool { return false } -func tryUpdatePointPlan(ctx PlanContext, updateStmt *ast.UpdateStmt) Plan { +func tryUpdatePointPlan(ctx base2.PlanContext, updateStmt *ast.UpdateStmt) base2.Plan { // Avoid using the point_get when assignment_list contains the sub-query in the UPDATE. if checkIfAssignmentListHasSubQuery(updateStmt.List) { return nil @@ -1916,7 +1917,7 @@ func tryUpdatePointPlan(ctx PlanContext, updateStmt *ast.UpdateStmt) Plan { return nil } -func buildPointUpdatePlan(ctx PlanContext, pointPlan PhysicalPlan, dbName string, tbl *model.TableInfo, updateStmt *ast.UpdateStmt) Plan { +func buildPointUpdatePlan(ctx base2.PlanContext, pointPlan base2.PhysicalPlan, dbName string, tbl *model.TableInfo, updateStmt *ast.UpdateStmt) base2.Plan { if checkFastPlanPrivilege(ctx, dbName, tbl.Name.L, mysql.SelectPriv, mysql.UpdatePriv) != nil { return nil } @@ -1972,7 +1973,7 @@ func buildPointUpdatePlan(ctx PlanContext, pointPlan PhysicalPlan, dbName string return updatePlan } -func buildOrderedList(ctx PlanContext, plan Plan, list []*ast.Assignment, +func buildOrderedList(ctx base2.PlanContext, plan base2.Plan, list []*ast.Assignment, ) (orderedList []*expression.Assignment, allAssignmentsAreConstant bool) { orderedList = make([]*expression.Assignment, 0, len(list)) allAssignmentsAreConstant = true @@ -2009,7 +2010,7 @@ func buildOrderedList(ctx PlanContext, plan Plan, list []*ast.Assignment, return orderedList, allAssignmentsAreConstant } -func tryDeletePointPlan(ctx PlanContext, delStmt *ast.DeleteStmt) Plan { +func tryDeletePointPlan(ctx base2.PlanContext, delStmt *ast.DeleteStmt) base2.Plan { if delStmt.IsMultiTable { return nil } @@ -2040,7 +2041,7 @@ func tryDeletePointPlan(ctx PlanContext, delStmt *ast.DeleteStmt) Plan { return nil } -func buildPointDeletePlan(ctx PlanContext, pointPlan PhysicalPlan, dbName string, tbl *model.TableInfo) Plan { +func buildPointDeletePlan(ctx base2.PlanContext, pointPlan base2.PhysicalPlan, dbName string, tbl *model.TableInfo) base2.Plan { if checkFastPlanPrivilege(ctx, dbName, tbl.Name.L, mysql.SelectPriv, mysql.DeletePriv) != nil { return nil } @@ -2093,7 +2094,7 @@ func colInfoToColumn(col *model.ColumnInfo, idx int) *expression.Column { } } -func buildHandleCols(ctx PlanContext, tbl *model.TableInfo, schema *expression.Schema) HandleCols { +func buildHandleCols(ctx base2.PlanContext, tbl *model.TableInfo, schema *expression.Schema) HandleCols { // fields len is 0 for update and delete. if tbl.PKIsHandle { for i, col := range tbl.Columns { @@ -2115,7 +2116,7 @@ func buildHandleCols(ctx PlanContext, tbl *model.TableInfo, schema *expression.S // TODO: Remove this, by enabling all types of partitioning // and update/add tests -func getHashOrKeyPartitionColumnName(ctx PlanContext, tbl *model.TableInfo) *model.CIStr { +func getHashOrKeyPartitionColumnName(ctx base2.PlanContext, tbl *model.TableInfo) *model.CIStr { pi := tbl.GetPartitionInfo() if pi == nil { return nil diff --git a/pkg/planner/core/preprocess.go b/pkg/planner/core/preprocess.go index d2631d02ba1dc..13b5fb0483072 100644 --- a/pkg/planner/core/preprocess.go +++ b/pkg/planner/core/preprocess.go @@ -34,6 +34,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/privilege" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" @@ -1795,7 +1796,7 @@ func (p *preprocessor) hasAutoConvertWarning(colDef *ast.ColumnDef) bool { return false } -func tryLockMDLAndUpdateSchemaIfNecessary(sctx PlanContext, dbName model.CIStr, tbl table.Table, is infoschema.InfoSchema) (table.Table, error) { +func tryLockMDLAndUpdateSchemaIfNecessary(sctx base.PlanContext, dbName model.CIStr, tbl table.Table, is infoschema.InfoSchema) (table.Table, error) { if !sctx.GetSessionVars().TxnCtx.EnableMDL { return tbl, nil } diff --git a/pkg/planner/core/rule_aggregation_push_down.go b/pkg/planner/core/rule_aggregation_push_down.go index 95398693c2ee5..286048a34c824 100644 --- a/pkg/planner/core/rule_aggregation_push_down.go +++ b/pkg/planner/core/rule_aggregation_push_down.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/tidb/pkg/expression/aggregation" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/planner/util/coreusage" "github.com/pingcap/tidb/pkg/types" @@ -193,7 +194,7 @@ func (a *aggregationPushDownSolver) splitAggFuncsAndGbyCols(agg *LogicalAggregat } // addGbyCol adds a column to gbyCols. If a group by column has existed, it will not be added repeatedly. -func (*aggregationPushDownSolver) addGbyCol(ctx PlanContext, gbyCols []*expression.Column, cols ...*expression.Column) []*expression.Column { +func (*aggregationPushDownSolver) addGbyCol(ctx base.PlanContext, gbyCols []*expression.Column, cols ...*expression.Column) []*expression.Column { for _, c := range cols { duplicate := false for _, gbyCol := range gbyCols { @@ -216,7 +217,7 @@ func (*aggregationPushDownSolver) checkValidJoin(join *LogicalJoin) bool { // decompose splits an aggregate function to two parts: a final mode function and a partial mode function. Currently // there are no differences between partial mode and complete mode, so we can confuse them. -func (*aggregationPushDownSolver) decompose(ctx PlanContext, aggFunc *aggregation.AggFuncDesc, +func (*aggregationPushDownSolver) decompose(ctx base.PlanContext, aggFunc *aggregation.AggFuncDesc, schema *expression.Schema, nullGenerating bool) ([]*aggregation.AggFuncDesc, *expression.Schema) { // Result is a slice because avg should be decomposed to sum and count. Currently we don't process this case. result := []*aggregation.AggFuncDesc{aggFunc.Clone()} @@ -325,7 +326,7 @@ func (*aggregationPushDownSolver) checkAllArgsColumn(fun *aggregation.AggFuncDes // TODO: // 1. https://github.com/pingcap/tidb/issues/16355, push avg & distinct functions across join // 2. remove this method and use splitPartialAgg instead for clean code. -func (a *aggregationPushDownSolver) makeNewAgg(ctx PlanContext, aggFuncs []*aggregation.AggFuncDesc, +func (a *aggregationPushDownSolver) makeNewAgg(ctx base.PlanContext, aggFuncs []*aggregation.AggFuncDesc, gbyCols []*expression.Column, preferAggType uint, preferAggToCop bool, blockOffset int, nullGenerating bool) (*LogicalAggregation, error) { agg := LogicalAggregation{ GroupByItems: expression.Column2Exprs(gbyCols), diff --git a/pkg/planner/core/rule_collect_plan_stats.go b/pkg/planner/core/rule_collect_plan_stats.go index 88c5fa762bfaf..6b11912bf1f88 100644 --- a/pkg/planner/core/rule_collect_plan_stats.go +++ b/pkg/planner/core/rule_collect_plan_stats.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/util/coreusage" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/statistics" @@ -95,7 +96,7 @@ func (syncWaitStatsLoadPoint) name() string { } // RequestLoadStats send load column/index stats requests to stats handle -func RequestLoadStats(ctx PlanContext, neededHistItems []model.StatsLoadItem, syncWait int64) error { +func RequestLoadStats(ctx base.PlanContext, neededHistItems []model.StatsLoadItem, syncWait int64) error { maxExecutionTime := ctx.GetSessionVars().GetMaxExecutionTime() if maxExecutionTime > 0 && maxExecutionTime < uint64(syncWait) { syncWait = int64(maxExecutionTime) @@ -227,7 +228,7 @@ func CollectDependingVirtualCols(tblID2Tbl map[int64]table.Table, neededItems [] // 1. the indices contained the any one of histNeededColumns, eg: histNeededColumns contained A,B columns, and idx_a is // composed up by A column, then we thought the idx_a should be collected // 2. The stats condition of idx_a can't meet IsFullLoad, which means its stats was evicted previously -func collectSyncIndices(ctx PlanContext, +func collectSyncIndices(ctx base.PlanContext, histNeededColumns []model.StatsLoadItem, tblID2Tbl map[int64]table.Table, ) map[model.TableItemID]struct{} { @@ -276,7 +277,7 @@ func collectHistNeededItems(histNeededColumns []model.StatsLoadItem, histNeededI return } -func recordTableRuntimeStats(sctx PlanContext, tbls map[int64]struct{}) { +func recordTableRuntimeStats(sctx base.PlanContext, tbls map[int64]struct{}) { tblStats := sctx.GetSessionVars().StmtCtx.TableStats if tblStats == nil { tblStats = map[int64]any{} @@ -294,7 +295,7 @@ func recordTableRuntimeStats(sctx PlanContext, tbls map[int64]struct{}) { sctx.GetSessionVars().StmtCtx.TableStats = tblStats } -func recordSingleTableRuntimeStats(sctx PlanContext, tblID int64) (stats *statistics.Table, skip bool, err error) { +func recordSingleTableRuntimeStats(sctx base.PlanContext, tblID int64) (stats *statistics.Table, skip bool, err error) { dom := domain.GetDomain(sctx) statsHandle := dom.StatsHandle() is := sctx.GetDomainInfoSchema().(infoschema.InfoSchema) diff --git a/pkg/planner/core/rule_decorrelate.go b/pkg/planner/core/rule_decorrelate.go index 10ddf00790484..918bed2d3c20f 100644 --- a/pkg/planner/core/rule_decorrelate.go +++ b/pkg/planner/core/rule_decorrelate.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/expression/aggregation" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/util/coreusage" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/plancodec" @@ -98,7 +99,7 @@ func ExtractCorrelatedCols4LogicalPlan(p LogicalPlan) []*expression.CorrelatedCo // ExtractCorrelatedCols4PhysicalPlan recursively extracts all of the correlated columns // from a plan tree by calling PhysicalPlan.ExtractCorrelatedCols. -func ExtractCorrelatedCols4PhysicalPlan(p PhysicalPlan) []*expression.CorrelatedColumn { +func ExtractCorrelatedCols4PhysicalPlan(p base.PhysicalPlan) []*expression.CorrelatedColumn { corCols := p.ExtractCorrelatedCols() for _, child := range p.Children() { corCols = append(corCols, ExtractCorrelatedCols4PhysicalPlan(child)...) @@ -129,11 +130,11 @@ func ExtractCorrelatedCols4PhysicalPlan(p PhysicalPlan) []*expression.Correlated // |_ Apply_3 // |_ outerSide // |_ innerSide(cor_col_3) -func ExtractOuterApplyCorrelatedCols(p PhysicalPlan) []*expression.CorrelatedColumn { +func ExtractOuterApplyCorrelatedCols(p base.PhysicalPlan) []*expression.CorrelatedColumn { return extractOuterApplyCorrelatedColsHelper(p, []*expression.Schema{}) } -func extractOuterApplyCorrelatedColsHelper(p PhysicalPlan, outerSchemas []*expression.Schema) []*expression.CorrelatedColumn { +func extractOuterApplyCorrelatedColsHelper(p base.PhysicalPlan, outerSchemas []*expression.Schema) []*expression.CorrelatedColumn { if p == nil { return nil } @@ -156,7 +157,7 @@ func extractOuterApplyCorrelatedColsHelper(p PhysicalPlan, outerSchemas []*expre switch v := p.(type) { case *PhysicalApply: - var outerPlan PhysicalPlan + var outerPlan base.PhysicalPlan if v.InnerChildIdx == 0 { outerPlan = v.Children()[1] } else { diff --git a/pkg/planner/core/rule_eliminate_projection.go b/pkg/planner/core/rule_eliminate_projection.go index 0cab6be023909..4618fdf6f7b00 100644 --- a/pkg/planner/core/rule_eliminate_projection.go +++ b/pkg/planner/core/rule_eliminate_projection.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/util/coreusage" ) @@ -120,7 +121,7 @@ func ResolveExprAndReplace(origin expression.Expression, replace map[string]*exp } } -func doPhysicalProjectionElimination(p PhysicalPlan) PhysicalPlan { +func doPhysicalProjectionElimination(p base.PhysicalPlan) base.PhysicalPlan { for i, child := range p.Children() { p.Children()[i] = doPhysicalProjectionElimination(child) } @@ -146,7 +147,7 @@ func doPhysicalProjectionElimination(p PhysicalPlan) PhysicalPlan { // eliminatePhysicalProjection should be called after physical optimization to // eliminate the redundant projection left after logical projection elimination. -func eliminatePhysicalProjection(p PhysicalPlan) PhysicalPlan { +func eliminatePhysicalProjection(p base.PhysicalPlan) base.PhysicalPlan { failpoint.Inject("DisableProjectionPostOptimization", func(val failpoint.Value) { if val.(bool) { failpoint.Return(p) diff --git a/pkg/planner/core/rule_inject_extra_projection.go b/pkg/planner/core/rule_inject_extra_projection.go index 0119fa3684f37..144e6e4860382 100644 --- a/pkg/planner/core/rule_inject_extra_projection.go +++ b/pkg/planner/core/rule_inject_extra_projection.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/tidb/pkg/expression/aggregation" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/core/internal" "github.com/pingcap/tidb/pkg/planner/util" ) @@ -32,7 +33,7 @@ import ( // 1. In postOptimize. // 2. TiDB can be used as a coprocessor, when a plan tree been pushed down to // TiDB, we need to inject extra projections for the plan tree as well. -func InjectExtraProjection(plan PhysicalPlan) PhysicalPlan { +func InjectExtraProjection(plan base.PhysicalPlan) base.PhysicalPlan { failpoint.Inject("DisableProjectionPostOptimization", func(val failpoint.Value) { if val.(bool) { failpoint.Return(plan) @@ -50,7 +51,7 @@ func NewProjInjector() *projInjector { return &projInjector{} } -func (pe *projInjector) inject(plan PhysicalPlan) PhysicalPlan { +func (pe *projInjector) inject(plan base.PhysicalPlan) base.PhysicalPlan { for i, child := range plan.Children() { plan.Children()[i] = pe.inject(child) } @@ -112,7 +113,7 @@ func injectProjBelowUnion(un *PhysicalUnionAll) *PhysicalUnionAll { // scalar functions in aggregation may speed up by vectorized evaluation in // the `proj`. If all the args of `aggFuncs`, and all the item of `groupByItems` // are columns or constants, we do not need to build the `proj`. -func InjectProjBelowAgg(aggPlan PhysicalPlan, aggFuncs []*aggregation.AggFuncDesc, groupByItems []expression.Expression) PhysicalPlan { +func InjectProjBelowAgg(aggPlan base.PhysicalPlan, aggFuncs []*aggregation.AggFuncDesc, groupByItems []expression.Expression) base.PhysicalPlan { hasScalarFunc := false internal.WrapCastForAggFuncs(aggPlan.SCtx().GetExprCtx(), aggFuncs) @@ -204,7 +205,7 @@ func InjectProjBelowAgg(aggPlan PhysicalPlan, aggFuncs []*aggregation.AggFuncDes // PhysicalTopN, some extra columns will be added into the schema of the // Projection, thus we need to add another Projection upon them to prune the // redundant columns. -func InjectProjBelowSort(p PhysicalPlan, orderByItems []*util.ByItems) PhysicalPlan { +func InjectProjBelowSort(p base.PhysicalPlan, orderByItems []*util.ByItems) base.PhysicalPlan { hasScalarFunc, numOrderByItems := false, len(orderByItems) for i := 0; !hasScalarFunc && i < numOrderByItems; i++ { _, isScalarFunc := orderByItems[i].Expr.(*expression.ScalarFunction) @@ -271,7 +272,7 @@ func InjectProjBelowSort(p PhysicalPlan, orderByItems []*util.ByItems) PhysicalP // TurnNominalSortIntoProj will turn nominal sort into two projections. This is to check if the scalar functions will // overflow. -func TurnNominalSortIntoProj(p PhysicalPlan, onlyColumn bool, orderByItems []*util.ByItems) PhysicalPlan { +func TurnNominalSortIntoProj(p base.PhysicalPlan, onlyColumn bool, orderByItems []*util.ByItems) base.PhysicalPlan { if onlyColumn { return p.Children()[0] } diff --git a/pkg/planner/core/rule_join_reorder.go b/pkg/planner/core/rule_join_reorder.go index ccc6831b1d64c..96dbf3f31a642 100644 --- a/pkg/planner/core/rule_join_reorder.go +++ b/pkg/planner/core/rule_join_reorder.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/parser/ast" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/util/coreusage" h "github.com/pingcap/tidb/pkg/util/hint" "github.com/pingcap/tidb/pkg/util/plancodec" @@ -234,7 +235,7 @@ func (s *joinReOrderSolver) optimize(_ context.Context, p LogicalPlan, opt *core } // optimizeRecursive recursively collects join groups and applies join reorder algorithm for each group. -func (s *joinReOrderSolver) optimizeRecursive(ctx PlanContext, p LogicalPlan, tracer *joinReorderTrace) (LogicalPlan, error) { +func (s *joinReOrderSolver) optimizeRecursive(ctx base.PlanContext, p LogicalPlan, tracer *joinReorderTrace) (LogicalPlan, error) { if _, ok := p.(*LogicalCTE); ok { return p, nil } @@ -393,7 +394,7 @@ type joinGroupResult struct { // nolint:structcheck type baseSingleGroupJoinOrderSolver struct { - ctx PlanContext + ctx base.PlanContext curJoinGroup []*jrNode leadingJoinGroup LogicalPlan *basicJoinGroupInfo diff --git a/pkg/planner/core/rule_join_reorder_dp_test.go b/pkg/planner/core/rule_join_reorder_dp_test.go index 219e2fce92b13..b3380c9e9ba1f 100644 --- a/pkg/planner/core/rule_join_reorder_dp_test.go +++ b/pkg/planner/core/rule_join_reorder_dp_test.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/types" "github.com/stretchr/testify/require" @@ -35,7 +36,7 @@ type mockLogicalJoin struct { JoinType JoinType } -func (mj mockLogicalJoin) init(ctx PlanContext) *mockLogicalJoin { +func (mj mockLogicalJoin) init(ctx base.PlanContext) *mockLogicalJoin { mj.baseLogicalPlan = newBaseLogicalPlan(ctx, "MockLogicalJoin", &mj, 0) return &mj } @@ -47,7 +48,7 @@ func (mj *mockLogicalJoin) recursiveDeriveStats(_ [][]*expression.Column) (*prop return mj.statsMap[mj.involvedNodeSet], nil } -func newMockJoin(ctx PlanContext, statsMap map[int]*property.StatsInfo) func(lChild, rChild LogicalPlan, _ []*expression.ScalarFunction, _, _, _ []expression.Expression, joinType JoinType) LogicalPlan { +func newMockJoin(ctx base.PlanContext, statsMap map[int]*property.StatsInfo) func(lChild, rChild LogicalPlan, _ []*expression.ScalarFunction, _, _, _ []expression.Expression, joinType JoinType) LogicalPlan { return func(lChild, rChild LogicalPlan, _ []*expression.ScalarFunction, _, _, _ []expression.Expression, joinType JoinType) LogicalPlan { retJoin := mockLogicalJoin{}.init(ctx) retJoin.schema = expression.MergeSchema(lChild.Schema(), rChild.Schema()) @@ -133,7 +134,7 @@ func makeStatsMapForTPCHQ5() map[int]*property.StatsInfo { return statsMap } -func newDataSource(ctx PlanContext, name string, count int) LogicalPlan { +func newDataSource(ctx base.PlanContext, name string, count int) LogicalPlan { ds := DataSource{}.Init(ctx, 0) tan := model.NewCIStr(name) ds.TableAsName = &tan diff --git a/pkg/planner/core/rule_partition_processor.go b/pkg/planner/core/rule_partition_processor.go index a7edaabfd1319..6d185d02649ca 100644 --- a/pkg/planner/core/rule_partition_processor.go +++ b/pkg/planner/core/rule_partition_processor.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/util/coreusage" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/table/tables" @@ -119,7 +120,7 @@ type partitionTable interface { PartitionExpr() *tables.PartitionExpr } -func generateHashPartitionExpr(ctx PlanContext, pi *model.PartitionInfo, columns []*expression.Column, names types.NameSlice) (expression.Expression, error) { +func generateHashPartitionExpr(ctx base.PlanContext, pi *model.PartitionInfo, columns []*expression.Column, names types.NameSlice) (expression.Expression, error) { schema := expression.NewSchema(columns...) // Increase the PlanID to make sure some tests will pass. The old implementation to rewrite AST builds a `TableDual` // that causes the `PlanID` increases, and many test cases hardcoded the output plan ID in the expected result. @@ -145,7 +146,7 @@ func getPartColumnsForHashPartition(hashExpr expression.Expression) ([]*expressi return partCols, colLen } -func (s *partitionProcessor) getUsedHashPartitions(ctx PlanContext, +func (s *partitionProcessor) getUsedHashPartitions(ctx base.PlanContext, tbl table.Table, partitionNames []model.CIStr, columns []*expression.Column, conds []expression.Expression, names types.NameSlice) ([]int, error) { pi := tbl.Meta().Partition @@ -263,7 +264,7 @@ func (s *partitionProcessor) getUsedHashPartitions(ctx PlanContext, return used, nil } -func (s *partitionProcessor) getUsedKeyPartitions(ctx PlanContext, +func (s *partitionProcessor) getUsedKeyPartitions(ctx base.PlanContext, tbl table.Table, partitionNames []model.CIStr, columns []*expression.Column, conds []expression.Expression, _ types.NameSlice) ([]int, error) { pi := tbl.Meta().Partition @@ -371,7 +372,7 @@ func (s *partitionProcessor) getUsedKeyPartitions(ctx PlanContext, } // getUsedPartitions is used to get used partitions for hash or key partition tables -func (s *partitionProcessor) getUsedPartitions(ctx PlanContext, tbl table.Table, +func (s *partitionProcessor) getUsedPartitions(ctx base.PlanContext, tbl table.Table, partitionNames []model.CIStr, columns []*expression.Column, conds []expression.Expression, names types.NameSlice, partType model.PartitionType) ([]int, error) { if partType == model.PartitionTypeHash { @@ -382,7 +383,7 @@ func (s *partitionProcessor) getUsedPartitions(ctx PlanContext, tbl table.Table, // findUsedPartitions is used to get used partitions for hash or key partition tables. // The first returning is the used partition index set pruned by `conds`. -func (s *partitionProcessor) findUsedPartitions(ctx PlanContext, +func (s *partitionProcessor) findUsedPartitions(ctx base.PlanContext, tbl table.Table, partitionNames []model.CIStr, conds []expression.Expression, columns []*expression.Column, names types.NameSlice) ([]int, error) { pi := tbl.Meta().Partition @@ -439,7 +440,7 @@ func convertToRangeOr(used []int, pi *model.PartitionInfo) partitionRangeOR { } // pruneHashOrKeyPartition is used to prune hash or key partition tables -func (s *partitionProcessor) pruneHashOrKeyPartition(ctx PlanContext, tbl table.Table, partitionNames []model.CIStr, +func (s *partitionProcessor) pruneHashOrKeyPartition(ctx base.PlanContext, tbl table.Table, partitionNames []model.CIStr, conds []expression.Expression, columns []*expression.Column, names types.NameSlice) ([]int, error) { used, err := s.findUsedPartitions(ctx, tbl, partitionNames, conds, columns, names) if err != nil { @@ -525,14 +526,14 @@ func (s *partitionProcessor) processHashOrKeyPartition(ds *DataSource, pi *model // listPartitionPruner uses to prune partition for list partition. type listPartitionPruner struct { *partitionProcessor - ctx PlanContext + ctx base.PlanContext pi *model.PartitionInfo partitionNames []model.CIStr fullRange map[int]struct{} listPrune *tables.ForListPruning } -func newListPartitionPruner(ctx PlanContext, tbl table.Table, partitionNames []model.CIStr, s *partitionProcessor, pruneList *tables.ForListPruning, columns []*expression.Column) *listPartitionPruner { +func newListPartitionPruner(ctx base.PlanContext, tbl table.Table, partitionNames []model.CIStr, s *partitionProcessor, pruneList *tables.ForListPruning, columns []*expression.Column) *listPartitionPruner { pruneList = pruneList.Clone() for i := range pruneList.PruneExprCols { for j := range columns { @@ -787,7 +788,7 @@ func (l *listPartitionPruner) findUsedListPartitions(conds []expression.Expressi return used, nil } -func (s *partitionProcessor) findUsedListPartitions(ctx PlanContext, tbl table.Table, partitionNames []model.CIStr, +func (s *partitionProcessor) findUsedListPartitions(ctx base.PlanContext, tbl table.Table, partitionNames []model.CIStr, conds []expression.Expression, columns []*expression.Column) ([]int, error) { pi := tbl.Meta().Partition partExpr := tbl.(partitionTable).PartitionExpr() @@ -815,7 +816,7 @@ func (s *partitionProcessor) findUsedListPartitions(ctx PlanContext, tbl table.T return ret, nil } -func (s *partitionProcessor) pruneListPartition(ctx PlanContext, tbl table.Table, partitionNames []model.CIStr, +func (s *partitionProcessor) pruneListPartition(ctx base.PlanContext, tbl table.Table, partitionNames []model.CIStr, conds []expression.Expression, columns []*expression.Column) ([]int, error) { used, err := s.findUsedListPartitions(ctx, tbl, partitionNames, conds, columns) if err != nil { @@ -1001,7 +1002,7 @@ func intersectionRange(start, end, newStart, newEnd int) (s int, e int) { return s, e } -func (s *partitionProcessor) pruneRangePartition(ctx PlanContext, pi *model.PartitionInfo, tbl table.PartitionedTable, conds []expression.Expression, +func (s *partitionProcessor) pruneRangePartition(ctx base.PlanContext, pi *model.PartitionInfo, tbl table.PartitionedTable, conds []expression.Expression, columns []*expression.Column, names types.NameSlice) (partitionRangeOR, error) { partExpr := tbl.(partitionTable).PartitionExpr() @@ -1054,7 +1055,7 @@ func (s *partitionProcessor) processListPartition(ds *DataSource, pi *model.Part } // makePartitionByFnCol extracts the column and function information in 'partition by ... fn(col)'. -func makePartitionByFnCol(sctx PlanContext, columns []*expression.Column, names types.NameSlice, partitionExpr string) (*expression.Column, *expression.ScalarFunction, monotoneMode, error) { +func makePartitionByFnCol(sctx base.PlanContext, columns []*expression.Column, names types.NameSlice, partitionExpr string) (*expression.Column, *expression.ScalarFunction, monotoneMode, error) { monotonous := monotoneModeInvalid schema := expression.NewSchema(columns...) // Increase the PlanID to make sure some tests will pass. The old implementation to rewrite AST builds a `TableDual` @@ -1100,7 +1101,7 @@ func makePartitionByFnCol(sctx PlanContext, columns []*expression.Column, names return col, fn, monotonous, nil } -func minCmp(ctx PlanContext, lowVal []types.Datum, columnsPruner *rangeColumnsPruner, comparer []collate.Collator, lowExclude bool, gotError *bool) func(i int) bool { +func minCmp(ctx base.PlanContext, lowVal []types.Datum, columnsPruner *rangeColumnsPruner, comparer []collate.Collator, lowExclude bool, gotError *bool) func(i int) bool { return func(i int) bool { for j := range lowVal { expr := columnsPruner.lessThan[i][j] @@ -1180,7 +1181,7 @@ func minCmp(ctx PlanContext, lowVal []types.Datum, columnsPruner *rangeColumnsPr } } -func maxCmp(ctx PlanContext, hiVal []types.Datum, columnsPruner *rangeColumnsPruner, comparer []collate.Collator, hiExclude bool, gotError *bool) func(i int) bool { +func maxCmp(ctx base.PlanContext, hiVal []types.Datum, columnsPruner *rangeColumnsPruner, comparer []collate.Collator, hiExclude bool, gotError *bool) func(i int) bool { return func(i int) bool { for j := range hiVal { expr := columnsPruner.lessThan[i][j] @@ -1219,7 +1220,7 @@ func maxCmp(ctx PlanContext, hiVal []types.Datum, columnsPruner *rangeColumnsPru } } -func multiColumnRangeColumnsPruner(sctx PlanContext, exprs []expression.Expression, +func multiColumnRangeColumnsPruner(sctx base.PlanContext, exprs []expression.Expression, columnsPruner *rangeColumnsPruner, result partitionRangeOR) partitionRangeOR { lens := make([]int, 0, len(columnsPruner.partCols)) for i := range columnsPruner.partCols { @@ -1268,7 +1269,7 @@ func multiColumnRangeColumnsPruner(sctx PlanContext, exprs []expression.Expressi return result.intersection(rangeOr).simplify() } -func partitionRangeForCNFExpr(sctx PlanContext, exprs []expression.Expression, +func partitionRangeForCNFExpr(sctx base.PlanContext, exprs []expression.Expression, pruner partitionRangePruner, result partitionRangeOR) partitionRangeOR { // TODO: When the ranger/detacher handles varchar_col_general_ci cmp constant bin collation // remove the check for single column RANGE COLUMNS and remove the single column implementation @@ -1282,7 +1283,7 @@ func partitionRangeForCNFExpr(sctx PlanContext, exprs []expression.Expression, } // partitionRangeForExpr calculate the partitions for the expression. -func partitionRangeForExpr(sctx PlanContext, expr expression.Expression, +func partitionRangeForExpr(sctx base.PlanContext, expr expression.Expression, pruner partitionRangePruner, result partitionRangeOR) partitionRangeOR { // Handle AND, OR respectively. if op, ok := expr.(*expression.ScalarFunction); ok { @@ -1314,7 +1315,7 @@ func partitionRangeForExpr(sctx PlanContext, expr expression.Expression, } type partitionRangePruner interface { - partitionRangeForExpr(PlanContext, expression.Expression) (start, end int, succ bool) + partitionRangeForExpr(base.PlanContext, expression.Expression) (start, end int, succ bool) fullRange() partitionRangeOR } @@ -1329,7 +1330,7 @@ type rangePruner struct { monotonous monotoneMode } -func (p *rangePruner) partitionRangeForExpr(sctx PlanContext, expr expression.Expression) (start int, end int, ok bool) { +func (p *rangePruner) partitionRangeForExpr(sctx base.PlanContext, expr expression.Expression) (start int, end int, ok bool) { if constExpr, ok := expr.(*expression.Constant); ok { if b, err := constExpr.Value.ToBool(sctx.GetSessionVars().StmtCtx.TypeCtx()); err == nil && b == 0 { // A constant false expression. @@ -1351,14 +1352,14 @@ func (p *rangePruner) fullRange() partitionRangeOR { } // partitionRangeForOrExpr calculate the partitions for or(expr1, expr2) -func partitionRangeForOrExpr(sctx PlanContext, expr1, expr2 expression.Expression, +func partitionRangeForOrExpr(sctx base.PlanContext, expr1, expr2 expression.Expression, pruner partitionRangePruner) partitionRangeOR { tmp1 := partitionRangeForExpr(sctx, expr1, pruner, pruner.fullRange()) tmp2 := partitionRangeForExpr(sctx, expr2, pruner, pruner.fullRange()) return tmp1.union(tmp2) } -func partitionRangeColumnForInExpr(sctx PlanContext, args []expression.Expression, +func partitionRangeColumnForInExpr(sctx base.PlanContext, args []expression.Expression, pruner *rangeColumnsPruner) partitionRangeOR { col, ok := args[0].(*expression.Column) if !ok || col.ID != pruner.partCols[0].ID { @@ -1395,7 +1396,7 @@ func partitionRangeColumnForInExpr(sctx PlanContext, args []expression.Expressio return result.simplify() } -func partitionRangeForInExpr(sctx PlanContext, args []expression.Expression, +func partitionRangeForInExpr(sctx base.PlanContext, args []expression.Expression, pruner *rangePruner) partitionRangeOR { col, ok := args[0].(*expression.Column) if !ok || col.ID != pruner.col.ID { @@ -1472,7 +1473,7 @@ type dataForPrune struct { // extractDataForPrune extracts data from the expression for pruning. // The expression should have this form: 'f(x) op const', otherwise it can't be pruned. -func (p *rangePruner) extractDataForPrune(sctx PlanContext, expr expression.Expression) (dataForPrune, bool) { +func (p *rangePruner) extractDataForPrune(sctx base.PlanContext, expr expression.Expression) (dataForPrune, bool) { var ret dataForPrune op, ok := expr.(*expression.ScalarFunction) if !ok { @@ -1743,7 +1744,7 @@ func checkTableHintsApplicableForPartition(partitions []model.CIStr, partitionSe return unknownPartitions } -func appendWarnForUnknownPartitions(ctx PlanContext, hintName string, unknownPartitions []string) { +func appendWarnForUnknownPartitions(ctx base.PlanContext, hintName string, unknownPartitions []string) { if len(unknownPartitions) == 0 { return } @@ -1824,7 +1825,7 @@ func (s *partitionProcessor) makeUnionAllChildren(ds *DataSource, pi *model.Part return unionAll, nil } -func (*partitionProcessor) pruneRangeColumnsPartition(ctx PlanContext, conds []expression.Expression, pi *model.PartitionInfo, pe *tables.PartitionExpr, columns []*expression.Column) (partitionRangeOR, error) { +func (*partitionProcessor) pruneRangeColumnsPartition(ctx base.PlanContext, conds []expression.Expression, pi *model.PartitionInfo, pe *tables.PartitionExpr, columns []*expression.Column) (partitionRangeOR, error) { result := fullRange(len(pi.Definitions)) if len(pi.Columns) < 1 { @@ -1884,7 +1885,7 @@ func (p *rangeColumnsPruner) getPartCol(colID int64) *expression.Column { return nil } -func (p *rangeColumnsPruner) partitionRangeForExpr(sctx PlanContext, expr expression.Expression) (start int, end int, ok bool) { +func (p *rangeColumnsPruner) partitionRangeForExpr(sctx base.PlanContext, expr expression.Expression) (start int, end int, ok bool) { op, ok := expr.(*expression.ScalarFunction) if !ok { return 0, len(p.lessThan), false @@ -1941,7 +1942,7 @@ func (p *rangeColumnsPruner) partitionRangeForExpr(sctx PlanContext, expr expres // pruneUseBinarySearch returns the start and end of which partitions will match. // If no match (i.e. value > last partition) the start partition will be the number of partition, not the first partition! -func (p *rangeColumnsPruner) pruneUseBinarySearch(sctx PlanContext, op string, data *expression.Constant) (start int, end int) { +func (p *rangeColumnsPruner) pruneUseBinarySearch(sctx base.PlanContext, op string, data *expression.Constant) (start int, end int) { var savedError error var isNull bool if len(p.partCols) > 1 { diff --git a/pkg/planner/core/rule_predicate_push_down.go b/pkg/planner/core/rule_predicate_push_down.go index 4217672c558f5..604235a43a7f7 100644 --- a/pkg/planner/core/rule_predicate_push_down.go +++ b/pkg/planner/core/rule_predicate_push_down.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/planner/util/coreusage" "github.com/pingcap/tidb/pkg/types" @@ -36,7 +37,7 @@ type ppdSolver struct{} // exprPrefixAdder is the wrapper struct to add tidb_shard(x) = val for `OrigConds` // `cols` is the index columns for a unique shard index type exprPrefixAdder struct { - sctx PlanContext + sctx base.PlanContext OrigConds []expression.Expression cols []*expression.Column lengths []int @@ -422,7 +423,7 @@ func simplifyOuterJoin(p *LogicalJoin, predicates []expression.Expression) { // If it is a predicate containing a reference to an inner table that evaluates to UNKNOWN or FALSE when one of its arguments is NULL. // If it is a conjunction containing a null-rejected condition as a conjunct. // If it is a disjunction of null-rejected conditions. -func isNullRejected(ctx PlanContext, schema *expression.Schema, expr expression.Expression) bool { +func isNullRejected(ctx base.PlanContext, schema *expression.Schema, expr expression.Expression) bool { exprCtx := ctx.GetExprCtx() expr = expression.PushDownNot(exprCtx, expr) if expression.ContainOuterNot(expr) { @@ -454,14 +455,14 @@ func isNullRejected(ctx PlanContext, schema *expression.Schema, expr expression. // isNullRejectedSpecially handles some null-rejected cases specially, since the current in // EvaluateExprWithNull is too strict for some cases, e.g. #49616. -func isNullRejectedSpecially(ctx PlanContext, schema *expression.Schema, expr expression.Expression) bool { +func isNullRejectedSpecially(ctx base.PlanContext, schema *expression.Schema, expr expression.Expression) bool { return specialNullRejectedCase1(ctx, schema, expr) // only 1 case now } // specialNullRejectedCase1 is mainly for #49616. // Case1 specially handles `null-rejected OR (null-rejected AND {others})`, then no matter what the result // of `{others}` is (True, False or Null), the result of this predicate is null, so this predicate is null-rejected. -func specialNullRejectedCase1(ctx PlanContext, schema *expression.Schema, expr expression.Expression) bool { +func specialNullRejectedCase1(ctx base.PlanContext, schema *expression.Schema, expr expression.Expression) bool { isFunc := func(e expression.Expression, lowerFuncName string) *expression.ScalarFunction { f, ok := e.(*expression.ScalarFunction) if !ok { @@ -712,7 +713,7 @@ func DeriveOtherConditions( // deriveNotNullExpr generates a new expression `not(isnull(col))` given `col1 op col2`, // in which `col` is in specified schema. Caller guarantees that only one of `col1` or // `col2` is in schema. -func deriveNotNullExpr(ctx PlanContext, expr expression.Expression, schema *expression.Schema) expression.Expression { +func deriveNotNullExpr(ctx base.PlanContext, expr expression.Expression, schema *expression.Schema) expression.Expression { binop, ok := expr.(*expression.ScalarFunction) if !ok || len(binop.GetArgs()) != 2 { return nil @@ -918,7 +919,7 @@ func appendAddSelectionTraceStep(p LogicalPlan, child LogicalPlan, sel *LogicalS // "SELECT * FROM test WHERE tidb_shard(a) = val1 AND a = 10 OR tidb_shard(a) = val2 AND a = 20" // @param[in] conds the original condtion of this datasource // @retval - the new condition after adding expression prefix -func (ds *DataSource) AddPrefix4ShardIndexes(sc PlanContext, conds []expression.Expression) []expression.Expression { +func (ds *DataSource) AddPrefix4ShardIndexes(sc base.PlanContext, conds []expression.Expression) []expression.Expression { if !ds.containExprPrefixUk { return conds } @@ -945,7 +946,7 @@ func (ds *DataSource) AddPrefix4ShardIndexes(sc PlanContext, conds []expression. return newConds } -func (ds *DataSource) addExprPrefixCond(sc PlanContext, path *util.AccessPath, +func (ds *DataSource) addExprPrefixCond(sc base.PlanContext, path *util.AccessPath, conds []expression.Expression) ([]expression.Expression, error) { idxCols, idxColLens := expression.IndexInfo2PrefixCols(ds.Columns, ds.schema.Columns, path.Index) diff --git a/pkg/planner/core/rule_predicate_simplification.go b/pkg/planner/core/rule_predicate_simplification.go index a3e3ddc02bab3..3ef885a8a0437 100644 --- a/pkg/planner/core/rule_predicate_simplification.go +++ b/pkg/planner/core/rule_predicate_simplification.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/parser/ast" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/util/coreusage" ) @@ -81,7 +82,7 @@ func (s *baseLogicalPlan) predicateSimplification(opt *coreusage.LogicalOptimize // updateInPredicate applies intersection of an in list with <> value. It returns updated In list and a flag for // a special case if an element in the inlist is not removed to keep the list not empty. -func updateInPredicate(ctx PlanContext, inPredicate expression.Expression, notEQPredicate expression.Expression) (expression.Expression, bool) { +func updateInPredicate(ctx base.PlanContext, inPredicate expression.Expression, notEQPredicate expression.Expression) (expression.Expression, bool) { _, inPredicateType := findPredicateType(inPredicate) _, notEQPredicateType := findPredicateType(notEQPredicate) if inPredicateType != inListPredicate || notEQPredicateType != notEqualPredicate { @@ -117,7 +118,7 @@ func updateInPredicate(ctx PlanContext, inPredicate expression.Expression, notEQ return newPred, specialCase } -func applyPredicateSimplification(sctx PlanContext, predicates []expression.Expression) []expression.Expression { +func applyPredicateSimplification(sctx base.PlanContext, predicates []expression.Expression) []expression.Expression { if len(predicates) <= 1 { return predicates } diff --git a/pkg/planner/core/runtime_filter.go b/pkg/planner/core/runtime_filter.go index 370764abf3bc6..ec3fb0a433497 100644 --- a/pkg/planner/core/runtime_filter.go +++ b/pkg/planner/core/runtime_filter.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/dbterror/plannererrors" @@ -201,7 +202,7 @@ func (rf *RuntimeFilter) Clone() *RuntimeFilter { } // RuntimeFilterListToPB convert runtime filter list to PB list -func RuntimeFilterListToPB(ctx *BuildPBContext, runtimeFilterList []*RuntimeFilter, client kv.Client) ([]*tipb.RuntimeFilter, error) { +func RuntimeFilterListToPB(ctx *base.BuildPBContext, runtimeFilterList []*RuntimeFilter, client kv.Client) ([]*tipb.RuntimeFilter, error) { result := make([]*tipb.RuntimeFilter, 0, len(runtimeFilterList)) for _, runtimeFilter := range runtimeFilterList { rfPB, err := runtimeFilter.ToPB(ctx, client) @@ -214,7 +215,7 @@ func RuntimeFilterListToPB(ctx *BuildPBContext, runtimeFilterList []*RuntimeFilt } // ToPB convert runtime filter to PB -func (rf *RuntimeFilter) ToPB(ctx *BuildPBContext, client kv.Client) (*tipb.RuntimeFilter, error) { +func (rf *RuntimeFilter) ToPB(ctx *base.BuildPBContext, client kv.Client) (*tipb.RuntimeFilter, error) { pc := expression.NewPBConverter(client, ctx.GetExprCtx().GetEvalCtx()) srcExprListPB := make([]*tipb.Expr, 0, len(rf.srcExprList)) for _, srcExpr := range rf.srcExprList { diff --git a/pkg/planner/core/runtime_filter_generator.go b/pkg/planner/core/runtime_filter_generator.go index 385cff8ca04d6..dea46e0fcfb9e 100644 --- a/pkg/planner/core/runtime_filter_generator.go +++ b/pkg/planner/core/runtime_filter_generator.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/logutil" @@ -29,7 +30,7 @@ import ( type RuntimeFilterGenerator struct { rfIDGenerator *util.IDGenerator columnUniqueIDToRF map[int64][]*RuntimeFilter - parentPhysicalPlan PhysicalPlan + parentPhysicalPlan base.PhysicalPlan childIdxForParentPhysicalPlan int } @@ -55,7 +56,7 @@ PhysicalPlanTree: TableScan ExchangeNode (assign RF1) */ -func (generator *RuntimeFilterGenerator) GenerateRuntimeFilter(plan PhysicalPlan) { +func (generator *RuntimeFilterGenerator) GenerateRuntimeFilter(plan base.PhysicalPlan) { switch physicalPlan := plan.(type) { case *PhysicalHashJoin: generator.generateRuntimeFilterInterval(physicalPlan) @@ -227,7 +228,7 @@ func (generator *RuntimeFilterGenerator) calculateRFMode(buildNode *PhysicalHash return variable.RFGlobal } -func (generator *RuntimeFilterGenerator) belongsToSameFragment(currentNode PhysicalPlan, targetNode *PhysicalTableScan) bool { +func (generator *RuntimeFilterGenerator) belongsToSameFragment(currentNode base.PhysicalPlan, targetNode *PhysicalTableScan) bool { switch currentNode.(type) { case *PhysicalExchangeReceiver: // terminal traversal diff --git a/pkg/planner/core/scalar_subq_expression.go b/pkg/planner/core/scalar_subq_expression.go index d51617aa6669b..54f5c60093cc2 100644 --- a/pkg/planner/core/scalar_subq_expression.go +++ b/pkg/planner/core/scalar_subq_expression.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/infoschema" + base2 "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/core/internal/base" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" @@ -33,7 +34,7 @@ type ScalarSubqueryEvalCtx struct { base.Plan // The context for evaluating the subquery. - scalarSubQuery PhysicalPlan + scalarSubQuery base2.PhysicalPlan ctx context.Context is infoschema.InfoSchema evalErr error diff --git a/pkg/planner/core/stats.go b/pkg/planner/core/stats.go index 09b44a5ebfd1a..5c49479c2821f 100644 --- a/pkg/planner/core/stats.go +++ b/pkg/planner/core/stats.go @@ -30,6 +30,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/cardinality" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/planner/util/debugtrace" @@ -220,7 +221,7 @@ func init() { } // getTblInfoForUsedStatsByPhysicalID get table name, partition name and HintedTable that will be used to record used stats. -func getTblInfoForUsedStatsByPhysicalID(sctx PlanContext, id int64) (fullName string, tblInfo *model.TableInfo) { +func getTblInfoForUsedStatsByPhysicalID(sctx base.PlanContext, id int64) (fullName string, tblInfo *model.TableInfo) { fullName = "tableID " + strconv.FormatInt(id, 10) is := domain.GetDomain(sctx).InfoSchema() diff --git a/pkg/planner/core/stringer.go b/pkg/planner/core/stringer.go index 0dd0abbf0e556..b07ee6f0792e5 100644 --- a/pkg/planner/core/stringer.go +++ b/pkg/planner/core/stringer.go @@ -19,11 +19,12 @@ import ( "fmt" "strings" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/util/plancodec" ) // ToString explains a Plan, returns description string. -func ToString(p Plan) string { +func ToString(p base.Plan) string { strs, _ := toString(p, []string{}, []int{}) return strings.Join(strs, "->") } @@ -37,7 +38,7 @@ func FDToString(p LogicalPlan) string { return strings.Join(strs, " >>> ") } -func needIncludeChildrenString(plan Plan) bool { +func needIncludeChildrenString(plan base.Plan) bool { switch x := plan.(type) { case *LogicalUnionAll, *PhysicalUnionAll, *LogicalPartitionUnionAll: // after https://github.com/pingcap/tidb/pull/25218, the union may contain less than 2 children, @@ -45,7 +46,7 @@ func needIncludeChildrenString(plan Plan) bool { return true case LogicalPlan: return len(x.Children()) > 1 - case PhysicalPlan: + case base.PhysicalPlan: return len(x.Children()) > 1 default: return false @@ -75,7 +76,7 @@ func fdToString(in LogicalPlan, strs []string, idxs []int) ([]string, []int) { return strs, idxs } -func toString(in Plan, strs []string, idxs []int) ([]string, []int) { +func toString(in base.Plan, strs []string, idxs []int) ([]string, []int) { switch x := in.(type) { case LogicalPlan: if needIncludeChildrenString(in) { @@ -86,7 +87,7 @@ func toString(in Plan, strs []string, idxs []int) ([]string, []int) { strs, idxs = toString(c, strs, idxs) } case *PhysicalExchangeReceiver: // do nothing - case PhysicalPlan: + case base.PhysicalPlan: if needIncludeChildrenString(in) { idxs = append(idxs, len(strs)) } diff --git a/pkg/planner/core/task.go b/pkg/planner/core/task.go index 1967f16f271c4..6412b6079c47d 100644 --- a/pkg/planner/core/task.go +++ b/pkg/planner/core/task.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/charset" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/cardinality" + base2 "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/core/internal/base" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util" @@ -38,7 +39,7 @@ import ( "go.uber.org/zap" ) -func attachPlan2Task(p PhysicalPlan, t Task) Task { +func attachPlan2Task(p base2.PhysicalPlan, t base2.Task) base2.Task { switch v := t.(type) { case *CopTask: if v.indexPlanFinished { @@ -95,13 +96,13 @@ func (t *CopTask) getStoreType() kv.StoreType { } // Attach2Task implements PhysicalPlan interface. -func (p *basePhysicalPlan) Attach2Task(tasks ...Task) Task { +func (p *basePhysicalPlan) Attach2Task(tasks ...base2.Task) base2.Task { t := tasks[0].ConvertToRootTask(p.SCtx()) return attachPlan2Task(p.self, t) } // Attach2Task implements PhysicalPlan interface. -func (p *PhysicalUnionScan) Attach2Task(tasks ...Task) Task { +func (p *PhysicalUnionScan) Attach2Task(tasks ...base2.Task) base2.Task { // We need to pull the projection under unionScan upon unionScan. // Since the projection only prunes columns, it's ok the put it upon unionScan. if sel, ok := tasks[0].Plan().(*PhysicalSelection); ok { @@ -130,7 +131,7 @@ func (p *PhysicalUnionScan) Attach2Task(tasks ...Task) Task { } // Attach2Task implements PhysicalPlan interface. -func (p *PhysicalApply) Attach2Task(tasks ...Task) Task { +func (p *PhysicalApply) Attach2Task(tasks ...base2.Task) base2.Task { lTask := tasks[0].ConvertToRootTask(p.SCtx()) rTask := tasks[1].ConvertToRootTask(p.SCtx()) p.SetChildren(lTask.Plan(), rTask.Plan()) @@ -141,7 +142,7 @@ func (p *PhysicalApply) Attach2Task(tasks ...Task) Task { } // Attach2Task implements PhysicalPlan interface. -func (p *PhysicalIndexMergeJoin) Attach2Task(tasks ...Task) Task { +func (p *PhysicalIndexMergeJoin) Attach2Task(tasks ...base2.Task) base2.Task { innerTask := p.innerTask outerTask := tasks[1-p.InnerChildIdx].ConvertToRootTask(p.SCtx()) if p.InnerChildIdx == 1 { @@ -155,7 +156,7 @@ func (p *PhysicalIndexMergeJoin) Attach2Task(tasks ...Task) Task { } // Attach2Task implements PhysicalPlan interface. -func (p *PhysicalIndexHashJoin) Attach2Task(tasks ...Task) Task { +func (p *PhysicalIndexHashJoin) Attach2Task(tasks ...base2.Task) base2.Task { innerTask := p.innerTask outerTask := tasks[1-p.InnerChildIdx].ConvertToRootTask(p.SCtx()) if p.InnerChildIdx == 1 { @@ -169,7 +170,7 @@ func (p *PhysicalIndexHashJoin) Attach2Task(tasks ...Task) Task { } // Attach2Task implements PhysicalPlan interface. -func (p *PhysicalIndexJoin) Attach2Task(tasks ...Task) Task { +func (p *PhysicalIndexJoin) Attach2Task(tasks ...base2.Task) base2.Task { innerTask := p.innerTask outerTask := tasks[1-p.InnerChildIdx].ConvertToRootTask(p.SCtx()) if p.InnerChildIdx == 1 { @@ -196,7 +197,7 @@ func getAvgRowSize(stats *property.StatsInfo, cols []*expression.Column) (size f } // Attach2Task implements PhysicalPlan interface. -func (p *PhysicalHashJoin) Attach2Task(tasks ...Task) Task { +func (p *PhysicalHashJoin) Attach2Task(tasks ...base2.Task) base2.Task { if p.storeTp == kv.TiFlash { return p.attach2TaskForTiFlash(tasks...) } @@ -268,7 +269,7 @@ func negotiateCommonType(lType, rType *types.FieldType) (*types.FieldType, bool, return commonType, needConvert(lType, commonType), needConvert(rType, commonType) } -func getProj(ctx PlanContext, p PhysicalPlan) *PhysicalProjection { +func getProj(ctx base2.PlanContext, p base2.PhysicalPlan) *PhysicalProjection { proj := PhysicalProjection{ Exprs: make([]expression.Expression, 0, len(p.Schema().Columns)), }.Init(ctx, p.StatsInfo(), p.QueryBlockOffset()) @@ -381,7 +382,7 @@ func (p *PhysicalHashJoin) convertPartitionKeysIfNeed(lTask, rTask *MppTask) (*M return lTask, rTask } -func (p *PhysicalHashJoin) attach2TaskForMpp(tasks ...Task) Task { +func (p *PhysicalHashJoin) attach2TaskForMpp(tasks ...base2.Task) base2.Task { lTask, lok := tasks[0].(*MppTask) rTask, rok := tasks[1].(*MppTask) if !lok || !rok { @@ -453,7 +454,7 @@ func (p *PhysicalHashJoin) attach2TaskForMpp(tasks ...Task) Task { return task } -func (p *PhysicalHashJoin) attach2TaskForTiFlash(tasks ...Task) Task { +func (p *PhysicalHashJoin) attach2TaskForTiFlash(tasks ...base2.Task) base2.Task { lTask, lok := tasks[0].(*CopTask) rTask, rok := tasks[1].(*CopTask) if !lok || !rok { @@ -477,7 +478,7 @@ func (p *PhysicalHashJoin) attach2TaskForTiFlash(tasks ...Task) Task { } // Attach2Task implements PhysicalPlan interface. -func (p *PhysicalMergeJoin) Attach2Task(tasks ...Task) Task { +func (p *PhysicalMergeJoin) Attach2Task(tasks ...base2.Task) base2.Task { lTask := tasks[0].ConvertToRootTask(p.SCtx()) rTask := tasks[1].ConvertToRootTask(p.SCtx()) p.SetChildren(lTask.Plan(), rTask.Plan()) @@ -486,7 +487,7 @@ func (p *PhysicalMergeJoin) Attach2Task(tasks ...Task) Task { return t } -func buildIndexLookUpTask(ctx PlanContext, t *CopTask) *RootTask { +func buildIndexLookUpTask(ctx base2.PlanContext, t *CopTask) *RootTask { newTask := &RootTask{} p := PhysicalIndexLookUpReader{ tablePlan: t.tablePlan, @@ -521,7 +522,7 @@ func buildIndexLookUpTask(ctx PlanContext, t *CopTask) *RootTask { return newTask } -func extractRows(p PhysicalPlan) float64 { +func extractRows(p base2.PhysicalPlan) float64 { f := float64(0) for _, c := range p.Children() { if len(c.Children()) != 0 { @@ -534,7 +535,7 @@ func extractRows(p PhysicalPlan) float64 { } // calcPagingCost calculates the cost for paging processing which may increase the seekCnt and reduce scanned rows. -func calcPagingCost(ctx PlanContext, indexPlan PhysicalPlan, expectCnt uint64) float64 { +func calcPagingCost(ctx base2.PlanContext, indexPlan base2.PhysicalPlan, expectCnt uint64) float64 { sessVars := ctx.GetSessionVars() indexRows := indexPlan.StatsCount() sourceRows := extractRows(indexPlan) @@ -555,7 +556,7 @@ func calcPagingCost(ctx PlanContext, indexPlan PhysicalPlan, expectCnt uint64) f return math.Max(pagingCst-sessVars.GetSeekFactor(nil), 0) } -func (t *CopTask) handleRootTaskConds(ctx PlanContext, newTask *RootTask) { +func (t *CopTask) handleRootTaskConds(ctx base2.PlanContext, newTask *RootTask) { if len(t.rootTaskConds) > 0 { selectivity, _, err := cardinality.Selectivity(ctx, t.tblColHists, t.rootTaskConds, nil) if err != nil { @@ -570,7 +571,7 @@ func (t *CopTask) handleRootTaskConds(ctx PlanContext, newTask *RootTask) { } // setTableScanToTableRowIDScan is to update the isChildOfIndexLookUp attribute of PhysicalTableScan child -func setTableScanToTableRowIDScan(p PhysicalPlan) { +func setTableScanToTableRowIDScan(p base2.PhysicalPlan) { if ts, ok := p.(*PhysicalTableScan); ok { ts.SetIsChildOfIndexLookUp(true) } else { @@ -596,7 +597,7 @@ func setTableScanToTableRowIDScan(p PhysicalPlan) { // // 4: attach the limit to the TOP of root index merge operator if there is some root condition exists for index merge // intersection/union case. -func (p *PhysicalLimit) Attach2Task(tasks ...Task) Task { +func (p *PhysicalLimit) Attach2Task(tasks ...base2.Task) base2.Task { t := tasks[0].Copy() newPartitionBy := make([]property.SortItem, 0, len(p.GetPartitionBy())) for _, expr := range p.GetPartitionBy() { @@ -643,7 +644,7 @@ func (p *PhysicalLimit) Attach2Task(tasks ...Task) Task { } else { // cop.indexPlanFinished = false indicates the table side is a pure table-scan, sink the limit to the index merge index side. newCount := p.Offset + p.Count - limitChildren := make([]PhysicalPlan, 0, len(cop.idxMergePartPlans)) + limitChildren := make([]base2.PhysicalPlan, 0, len(cop.idxMergePartPlans)) for _, partialScan := range cop.idxMergePartPlans { childProfile := partialScan.StatsInfo() stats := deriveLimitStats(childProfile, float64(newCount)) @@ -703,7 +704,7 @@ func (p *PhysicalLimit) Attach2Task(tasks ...Task) Task { return attachPlan2Task(p, t) } -func (p *PhysicalLimit) sinkIntoIndexLookUp(t Task) bool { +func (p *PhysicalLimit) sinkIntoIndexLookUp(t base2.Task) bool { root := t.(*RootTask) reader, isDoubleRead := root.GetPlan().(*PhysicalIndexLookUpReader) proj, isProj := root.GetPlan().(*PhysicalProjection) @@ -755,7 +756,7 @@ func (p *PhysicalLimit) sinkIntoIndexLookUp(t Task) bool { return true } -func (p *PhysicalLimit) sinkIntoIndexMerge(t Task) bool { +func (p *PhysicalLimit) sinkIntoIndexMerge(t base2.Task) bool { root := t.(*RootTask) imReader, isIm := root.GetPlan().(*PhysicalIndexMergeReader) proj, isProj := root.GetPlan().(*PhysicalProjection) @@ -809,14 +810,14 @@ func (p *PhysicalLimit) sinkIntoIndexMerge(t Task) bool { } // Attach2Task implements PhysicalPlan interface. -func (p *PhysicalSort) Attach2Task(tasks ...Task) Task { +func (p *PhysicalSort) Attach2Task(tasks ...base2.Task) base2.Task { t := tasks[0].Copy() t = attachPlan2Task(p, t) return t } // Attach2Task implements PhysicalPlan interface. -func (p *NominalSort) Attach2Task(tasks ...Task) Task { +func (p *NominalSort) Attach2Task(tasks ...base2.Task) base2.Task { if p.OnlyColumn { return tasks[0] } @@ -825,7 +826,7 @@ func (p *NominalSort) Attach2Task(tasks ...Task) Task { return t } -func (p *PhysicalTopN) getPushedDownTopN(childPlan PhysicalPlan) *PhysicalTopN { +func (p *PhysicalTopN) getPushedDownTopN(childPlan base2.PhysicalPlan) *PhysicalTopN { newByItems := make([]*util.ByItems, 0, len(p.ByItems)) for _, expr := range p.ByItems { newByItems = append(newByItems, expr.Clone()) @@ -850,7 +851,7 @@ func (p *PhysicalTopN) getPushedDownTopN(childPlan PhysicalPlan) *PhysicalTopN { // canPushToIndexPlan checks if this TopN can be pushed to the index side of copTask. // It can be pushed to the index side when all columns used by ByItems are available from the index side and there's no prefix index column. -func (*PhysicalTopN) canPushToIndexPlan(indexPlan PhysicalPlan, byItemCols []*expression.Column) bool { +func (*PhysicalTopN) canPushToIndexPlan(indexPlan base2.PhysicalPlan, byItemCols []*expression.Column) bool { // If we call canPushToIndexPlan and there's no index plan, we should go into the index merge case. // Index merge case is specially handled for now. So we directly return false here. // So we directly return false. @@ -931,7 +932,7 @@ func (p *PhysicalTopN) canPushDownToTiFlash(mppTask *MppTask) bool { } // Attach2Task implements physical plan -func (p *PhysicalTopN) Attach2Task(tasks ...Task) Task { +func (p *PhysicalTopN) Attach2Task(tasks ...base2.Task) base2.Task { t := tasks[0].Copy() cols := make([]*expression.Column, 0, len(p.ByItems)) for _, item := range p.ByItems { @@ -965,7 +966,7 @@ func (p *PhysicalTopN) Attach2Task(tasks ...Task) Task { } // Attach2Task implements the PhysicalPlan interface. -func (p *PhysicalExpand) Attach2Task(tasks ...Task) Task { +func (p *PhysicalExpand) Attach2Task(tasks ...base2.Task) base2.Task { t := tasks[0].Copy() // current expand can only be run in MPP TiFlash mode. if mpp, ok := t.(*MppTask); ok { @@ -977,7 +978,7 @@ func (p *PhysicalExpand) Attach2Task(tasks ...Task) Task { } // Attach2Task implements PhysicalPlan interface. -func (p *PhysicalProjection) Attach2Task(tasks ...Task) Task { +func (p *PhysicalProjection) Attach2Task(tasks ...base2.Task) base2.Task { t := tasks[0].Copy() if cop, ok := t.(*CopTask); ok { if (len(cop.rootTaskConds) == 0 && len(cop.idxMergePartPlans) == 0) && expression.CanExprsPushDown(GetPushDownCtx(p.SCtx()), p.Exprs, cop.getStoreType()) { @@ -999,9 +1000,9 @@ func (p *PhysicalProjection) Attach2Task(tasks ...Task) Task { return t } -func (p *PhysicalUnionAll) attach2MppTasks(tasks ...Task) Task { +func (p *PhysicalUnionAll) attach2MppTasks(tasks ...base2.Task) base2.Task { t := &MppTask{p: p} - childPlans := make([]PhysicalPlan, 0, len(tasks)) + childPlans := make([]base2.PhysicalPlan, 0, len(tasks)) for _, tk := range tasks { if mpp, ok := tk.(*MppTask); ok && !tk.Invalid() { childPlans = append(childPlans, mpp.Plan()) @@ -1019,7 +1020,7 @@ func (p *PhysicalUnionAll) attach2MppTasks(tasks ...Task) Task { } // Attach2Task implements PhysicalPlan interface. -func (p *PhysicalUnionAll) Attach2Task(tasks ...Task) Task { +func (p *PhysicalUnionAll) Attach2Task(tasks ...base2.Task) base2.Task { for _, t := range tasks { if _, ok := t.(*MppTask); ok { if p.TP() == plancodec.TypePartitionUnion { @@ -1033,7 +1034,7 @@ func (p *PhysicalUnionAll) Attach2Task(tasks ...Task) Task { } t := &RootTask{} t.SetPlan(p) - childPlans := make([]PhysicalPlan, 0, len(tasks)) + childPlans := make([]base2.PhysicalPlan, 0, len(tasks)) for _, task := range tasks { task = task.ConvertToRootTask(p.SCtx()) childPlans = append(childPlans, task.Plan()) @@ -1043,7 +1044,7 @@ func (p *PhysicalUnionAll) Attach2Task(tasks ...Task) Task { } // Attach2Task implements PhysicalPlan interface. -func (sel *PhysicalSelection) Attach2Task(tasks ...Task) Task { +func (sel *PhysicalSelection) Attach2Task(tasks ...base2.Task) base2.Task { if mppTask, _ := tasks[0].(*MppTask); mppTask != nil { // always push to mpp task. if expression.CanExprsPushDown(GetPushDownCtx(sel.SCtx()), sel.Conditions, kv.TiFlash) { return attachPlan2Task(sel, mppTask.Copy()) @@ -1055,7 +1056,7 @@ func (sel *PhysicalSelection) Attach2Task(tasks ...Task) Task { // CheckAggCanPushCop checks whether the aggFuncs and groupByItems can // be pushed down to coprocessor. -func CheckAggCanPushCop(sctx PlanContext, aggFuncs []*aggregation.AggFuncDesc, groupByItems []expression.Expression, storeType kv.StoreType) bool { +func CheckAggCanPushCop(sctx base2.PlanContext, aggFuncs []*aggregation.AggFuncDesc, groupByItems []expression.Expression, storeType kv.StoreType) bool { sc := sctx.GetSessionVars().StmtCtx ret := true reason := "" @@ -1134,7 +1135,7 @@ type AggInfo struct { // building the aggregate executor(e.g. buildHashAgg will split the AggDesc further for parallel executing). // firstRowFuncMap is a map between partial first_row to final first_row, will be used in RemoveUnnecessaryFirstRow func BuildFinalModeAggregation( - sctx PlanContext, original *AggInfo, partialIsCop bool, isMPPTask bool) (partial, final *AggInfo, firstRowFuncMap map[*aggregation.AggFuncDesc]*aggregation.AggFuncDesc) { + sctx base2.PlanContext, original *AggInfo, partialIsCop bool, isMPPTask bool) (partial, final *AggInfo, firstRowFuncMap map[*aggregation.AggFuncDesc]*aggregation.AggFuncDesc) { firstRowFuncMap = make(map[*aggregation.AggFuncDesc]*aggregation.AggFuncDesc, len(original.AggFuncs)) partial = &AggInfo{ AggFuncs: make([]*aggregation.AggFuncDesc, 0, len(original.AggFuncs)), @@ -1290,7 +1291,7 @@ func BuildFinalModeAggregation( if aggregation.NeedCount(finalAggFunc.Name) { // only Avg and Count need count if isMPPTask && finalAggFunc.Name == ast.AggFuncCount { - // For MPP Task, the final count() is changed to sum(). + // For MPP base2.Task, the final count() is changed to sum(). // Note: MPP mode does not run avg() directly, instead, avg() -> sum()/(case when count() = 0 then 1 else count() end), // so we do not process it here. finalAggFunc.Name = ast.AggFuncSum @@ -1457,7 +1458,7 @@ func (p *basePhysicalAgg) convertAvgForMPP() *PhysicalProjection { return proj } -func (p *basePhysicalAgg) newPartialAggregate(copTaskType kv.StoreType, isMPPTask bool) (partial, final PhysicalPlan) { +func (p *basePhysicalAgg) newPartialAggregate(copTaskType kv.StoreType, isMPPTask bool) (partial, final base2.PhysicalPlan) { // Check if this aggregation can push down. if !CheckAggCanPushCop(p.SCtx(), p.AggFuncs, p.GroupByItems, copTaskType) { return nil, p.self @@ -1618,7 +1619,7 @@ func (p *basePhysicalAgg) canUse3Stage4SingleDistinctAgg() bool { return num == 1 } -func genFirstRowAggForGroupBy(ctx PlanContext, groupByItems []expression.Expression) ([]*aggregation.AggFuncDesc, error) { +func genFirstRowAggForGroupBy(ctx base2.PlanContext, groupByItems []expression.Expression) ([]*aggregation.AggFuncDesc, error) { aggFuncs := make([]*aggregation.AggFuncDesc, 0, len(groupByItems)) for _, groupBy := range groupByItems { agg, err := aggregation.NewAggFuncDesc(ctx.GetExprCtx(), ast.AggFuncFirstRow, []expression.Expression{groupBy}, false) @@ -1638,7 +1639,7 @@ func genFirstRowAggForGroupBy(ctx PlanContext, groupByItems []expression.Express // The schema is [firstrow(a), count(b), a]. The column firstrow(a) is unnecessary. // Can optimize the schema to [count(b), a] , and change the index to get value. func RemoveUnnecessaryFirstRow( - sctx PlanContext, + sctx base2.PlanContext, finalGbyItems []expression.Expression, partialAggFuncs []*aggregation.AggFuncDesc, partialGbyItems []expression.Expression, @@ -1698,7 +1699,7 @@ func computePartialCursorOffset(name string) int { } // Attach2Task implements PhysicalPlan interface. -func (p *PhysicalStreamAgg) Attach2Task(tasks ...Task) Task { +func (p *PhysicalStreamAgg) Attach2Task(tasks ...base2.Task) base2.Task { t := tasks[0].Copy() if cop, ok := t.(*CopTask); ok { // We should not push agg down across @@ -1764,7 +1765,7 @@ func (p *PhysicalHashAgg) cpuCostDivisor(hasDistinct bool) (divisor, con float64 return math.Min(float64(finalCon), float64(partialCon)), float64(finalCon + partialCon) } -func (p *PhysicalHashAgg) attach2TaskForMpp1Phase(mpp *MppTask) Task { +func (p *PhysicalHashAgg) attach2TaskForMpp1Phase(mpp *MppTask) base2.Task { // 1-phase agg: when the partition columns can be satisfied, where the plan does not need to enforce Exchange // only push down the original agg proj := p.convertAvgForMPP() @@ -1890,8 +1891,8 @@ func (p *PhysicalHashAgg) scaleStats4GroupingSets(groupingSets expression.Groupi // +- HashAgg count(c) #4, group by a,b,groupingID -> partial agg // +- Expand {}, {} -> expand // +- TableScan foo -func (p *PhysicalHashAgg) adjust3StagePhaseAgg(partialAgg, finalAgg PhysicalPlan, canUse3StageAgg bool, - groupingSets expression.GroupingSets, mpp *MppTask) (final, mid, part, proj4Part PhysicalPlan, _ error) { +func (p *PhysicalHashAgg) adjust3StagePhaseAgg(partialAgg, finalAgg base2.PhysicalPlan, canUse3StageAgg bool, + groupingSets expression.GroupingSets, mpp *MppTask) (final, mid, part, proj4Part base2.PhysicalPlan, _ error) { if !(partialAgg != nil && canUse3StageAgg) { // quick path: return the original finalAgg and partiAgg. return finalAgg, nil, partialAgg, nil, nil @@ -2074,7 +2075,7 @@ func (p *PhysicalHashAgg) adjust3StagePhaseAgg(partialAgg, finalAgg PhysicalPlan return finalHashAgg, middleHashAgg, partialHashAgg, proj4Partial, nil } -func (p *PhysicalHashAgg) attach2TaskForMpp(tasks ...Task) Task { +func (p *PhysicalHashAgg) attach2TaskForMpp(tasks ...base2.Task) base2.Task { t := tasks[0].Copy() mpp, ok := t.(*MppTask) if !ok { @@ -2202,7 +2203,7 @@ func (p *PhysicalHashAgg) attach2TaskForMpp(tasks ...Task) Task { } // Attach2Task implements the PhysicalPlan interface. -func (p *PhysicalHashAgg) Attach2Task(tasks ...Task) Task { +func (p *PhysicalHashAgg) Attach2Task(tasks ...base2.Task) base2.Task { t := tasks[0].Copy() if cop, ok := t.(*CopTask); ok { if len(cop.rootTaskConds) == 0 && len(cop.idxMergePartPlans) == 0 { @@ -2245,7 +2246,7 @@ func (p *PhysicalHashAgg) Attach2Task(tasks ...Task) Task { return t } -func (p *PhysicalWindow) attach2TaskForMPP(mpp *MppTask) Task { +func (p *PhysicalWindow) attach2TaskForMPP(mpp *MppTask) base2.Task { // FIXME: currently, tiflash's join has different schema with TiDB, // so we have to rebuild the schema of join and operators which may inherit schema from join. // for window, we take the sub-plan's schema, and the schema generated by windowDescs. @@ -2262,7 +2263,7 @@ func (p *PhysicalWindow) attach2TaskForMPP(mpp *MppTask) Task { } // Attach2Task implements the PhysicalPlan interface. -func (p *PhysicalWindow) Attach2Task(tasks ...Task) Task { +func (p *PhysicalWindow) Attach2Task(tasks ...base2.Task) base2.Task { if mpp, ok := tasks[0].Copy().(*MppTask); ok && p.storeTp == kv.TiFlash { return p.attach2TaskForMPP(mpp) } @@ -2271,7 +2272,7 @@ func (p *PhysicalWindow) Attach2Task(tasks ...Task) Task { } // Attach2Task implements the PhysicalPlan interface. -func (p *PhysicalCTEStorage) Attach2Task(tasks ...Task) Task { +func (p *PhysicalCTEStorage) Attach2Task(tasks ...base2.Task) base2.Task { t := tasks[0].Copy() if mpp, ok := t.(*MppTask); ok { p.SetChildren(t.Plan()) @@ -2290,7 +2291,7 @@ func (p *PhysicalCTEStorage) Attach2Task(tasks ...Task) Task { } // Attach2Task implements the PhysicalPlan interface. -func (p *PhysicalSequence) Attach2Task(tasks ...Task) Task { +func (p *PhysicalSequence) Attach2Task(tasks ...base2.Task) base2.Task { for _, t := range tasks { _, isMpp := t.(*MppTask) if !isMpp { @@ -2300,7 +2301,7 @@ func (p *PhysicalSequence) Attach2Task(tasks ...Task) Task { lastTask := tasks[len(tasks)-1].(*MppTask) - children := make([]PhysicalPlan, 0, len(tasks)) + children := make([]base2.PhysicalPlan, 0, len(tasks)) for _, t := range tasks { children = append(children, t.Plan()) } @@ -2316,7 +2317,7 @@ func (p *PhysicalSequence) Attach2Task(tasks ...Task) Task { return mppTask } -func collectPartitionInfosFromMPPPlan(p *PhysicalTableReader, mppPlan PhysicalPlan) { +func collectPartitionInfosFromMPPPlan(p *PhysicalTableReader, mppPlan base2.PhysicalPlan) { switch x := mppPlan.(type) { case *PhysicalTableScan: p.TableScanAndPartitionInfos = append(p.TableScanAndPartitionInfos, tableScanAndPartitionInfo{x, x.PlanPartInfo}) @@ -2327,14 +2328,14 @@ func collectPartitionInfosFromMPPPlan(p *PhysicalTableReader, mppPlan PhysicalPl } } -func collectRowSizeFromMPPPlan(mppPlan PhysicalPlan) (rowSize float64) { +func collectRowSizeFromMPPPlan(mppPlan base2.PhysicalPlan) (rowSize float64) { if mppPlan != nil && mppPlan.StatsInfo() != nil && mppPlan.StatsInfo().HistColl != nil { return cardinality.GetAvgRowSize(mppPlan.SCtx(), mppPlan.StatsInfo().HistColl, mppPlan.Schema().Columns, false, false) } return 1 // use 1 as lower-bound for safety } -func accumulateNetSeekCost4MPP(p PhysicalPlan) (cost float64) { +func accumulateNetSeekCost4MPP(p base2.PhysicalPlan) (cost float64) { if ts, ok := p.(*PhysicalTableScan); ok { return float64(len(ts.Ranges)) * float64(len(ts.Columns)) * ts.SCtx().GetSessionVars().GetSeekFactor(ts.Table) } @@ -2344,7 +2345,7 @@ func accumulateNetSeekCost4MPP(p PhysicalPlan) (cost float64) { return } -func tryExpandVirtualColumn(p PhysicalPlan) { +func tryExpandVirtualColumn(p base2.PhysicalPlan) { if ts, ok := p.(*PhysicalTableScan); ok { ts.Columns = ExpandVirtualColumn(ts.Columns, ts.schema, ts.Table.Columns) return diff --git a/pkg/planner/core/task_base.go b/pkg/planner/core/task_base.go index d4c172132409b..1d8f5080427fe 100644 --- a/pkg/planner/core/task_base.go +++ b/pkg/planner/core/task_base.go @@ -18,6 +18,7 @@ import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/planner/cardinality" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/util/logutil" @@ -27,44 +28,27 @@ import ( ) var ( - _ Task = &RootTask{} - _ Task = &MppTask{} - _ Task = &CopTask{} + _ base.Task = &RootTask{} + _ base.Task = &MppTask{} + _ base.Task = &CopTask{} ) -// Task is a new version of `PhysicalPlanInfo`. It stores cost information for a task. -// A task may be CopTask, RootTask, MPPTaskMeta or a ParallelTask. -type Task interface { - // Count returns current task's row count. - Count() float64 - // Copy return a shallow copy of current task with the same pointer to p. - Copy() Task - // Plan returns current task's plan. - Plan() PhysicalPlan - // Invalid returns whether current task is invalid. - Invalid() bool - // ConvertToRootTask will convert current task as root type. - ConvertToRootTask(ctx PlanContext) *RootTask - // MemoryUsage returns the memory usage of current task. - MemoryUsage() int64 -} - // ************************************* RootTask Start ****************************************** // RootTask is the final sink node of a plan graph. It should be a single goroutine on tidb. type RootTask struct { - p PhysicalPlan + p base.PhysicalPlan isEmpty bool // isEmpty indicates if this task contains a dual table and returns empty data. // TODO: The flag 'isEmpty' is only checked by Projection and UnionAll. We should support more cases in the future. } // GetPlan returns the root task's plan. -func (t *RootTask) GetPlan() PhysicalPlan { +func (t *RootTask) GetPlan() base.PhysicalPlan { return t.p } // SetPlan sets the root task' plan. -func (t *RootTask) SetPlan(p PhysicalPlan) { +func (t *RootTask) SetPlan(p base.PhysicalPlan) { t.p = p } @@ -79,14 +63,14 @@ func (t *RootTask) SetEmpty(x bool) { } // Copy implements Task interface. -func (t *RootTask) Copy() Task { +func (t *RootTask) Copy() base.Task { return &RootTask{ p: t.p, } } // ConvertToRootTask implements Task interface. -func (t *RootTask) ConvertToRootTask(_ PlanContext) *RootTask { +func (t *RootTask) ConvertToRootTask(_ base.PlanContext) base.Task { return t.Copy().(*RootTask) } @@ -101,7 +85,7 @@ func (t *RootTask) Count() float64 { } // Plan implements Task interface. -func (t *RootTask) Plan() PhysicalPlan { +func (t *RootTask) Plan() base.PhysicalPlan { return t.p } @@ -127,7 +111,7 @@ func (t *RootTask) MemoryUsage() (sum int64) { // 3. consider virtual columns. // 4. TODO: partition prune after close type MppTask struct { - p PhysicalPlan + p base.PhysicalPlan partTp property.MPPPartitionType hashCols []*property.MPPPartitionColumn @@ -152,13 +136,13 @@ func (t *MppTask) Count() float64 { } // Copy implements Task interface. -func (t *MppTask) Copy() Task { +func (t *MppTask) Copy() base.Task { nt := *t return &nt } // Plan implements Task interface. -func (t *MppTask) Plan() PhysicalPlan { +func (t *MppTask) Plan() base.PhysicalPlan { return t.p } @@ -168,7 +152,7 @@ func (t *MppTask) Invalid() bool { } // ConvertToRootTask implements Task interface. -func (t *MppTask) ConvertToRootTask(ctx PlanContext) *RootTask { +func (t *MppTask) ConvertToRootTask(ctx base.PlanContext) base.Task { return t.Copy().(*MppTask).ConvertToRootTaskImpl(ctx) } @@ -186,7 +170,7 @@ func (t *MppTask) MemoryUsage() (sum int64) { } // ConvertToRootTaskImpl implements Task interface. -func (t *MppTask) ConvertToRootTaskImpl(ctx PlanContext) *RootTask { +func (t *MppTask) ConvertToRootTaskImpl(ctx base.PlanContext) *RootTask { // In disaggregated-tiflash mode, need to consider generated column. tryExpandVirtualColumn(t.p) sender := PhysicalExchangeSender{ @@ -238,8 +222,8 @@ func (t *MppTask) ConvertToRootTaskImpl(ctx PlanContext) *RootTask { // CopTask is a task that runs in a distributed kv store. // TODO: In future, we should split copTask to indexTask and tableTask. type CopTask struct { - indexPlan PhysicalPlan - tablePlan PhysicalPlan + indexPlan base.PhysicalPlan + tablePlan base.PhysicalPlan // indexPlanFinished means we have finished index plan. indexPlanFinished bool // keepOrder indicates if the plan scans data by order. @@ -259,7 +243,7 @@ type CopTask struct { // is used to compute average row width when computing scan cost. tblCols []*expression.Column - idxMergePartPlans []PhysicalPlan + idxMergePartPlans []base.PhysicalPlan idxMergeIsIntersection bool idxMergeAccessMVIndex bool @@ -289,7 +273,7 @@ func (t *CopTask) Count() float64 { } // Copy implements Task interface. -func (t *CopTask) Copy() Task { +func (t *CopTask) Copy() base.Task { nt := *t return &nt } @@ -297,7 +281,7 @@ func (t *CopTask) Copy() Task { // Plan implements Task interface. // copTask plan should be careful with indexMergeReader, whose real plan is stored in // idxMergePartPlans, when its indexPlanFinished is marked with false. -func (t *CopTask) Plan() PhysicalPlan { +func (t *CopTask) Plan() base.PhysicalPlan { if t.indexPlanFinished { return t.tablePlan } @@ -341,12 +325,12 @@ func (t *CopTask) MemoryUsage() (sum int64) { } // ConvertToRootTask implements Task interface. -func (t *CopTask) ConvertToRootTask(ctx PlanContext) *RootTask { +func (t *CopTask) ConvertToRootTask(ctx base.PlanContext) base.Task { // copy one to avoid changing itself. return t.Copy().(*CopTask).convertToRootTaskImpl(ctx) } -func (t *CopTask) convertToRootTaskImpl(ctx PlanContext) *RootTask { +func (t *CopTask) convertToRootTaskImpl(ctx base.PlanContext) *RootTask { // copTasks are run in parallel, to make the estimated cost closer to execution time, we amortize // the cost to cop iterator workers. According to `CopClient::Send`, the concurrency // is Min(DistSQLScanConcurrency, numRegionsInvolvedInScan), since we cannot infer diff --git a/pkg/planner/core/tiflash_selection_late_materialization.go b/pkg/planner/core/tiflash_selection_late_materialization.go index 4211c8183ee90..da09afef5dbe8 100644 --- a/pkg/planner/core/tiflash_selection_late_materialization.go +++ b/pkg/planner/core/tiflash_selection_late_materialization.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/planner/cardinality" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/util/logutil" "go.uber.org/zap" ) @@ -47,7 +48,7 @@ type expressionGroup struct { // predicatePushDownToTableScan is used find the selection just above the table scan // and try to push down the predicates to the table scan. // Used for TiFlash late materialization. -func predicatePushDownToTableScan(sctx PlanContext, plan PhysicalPlan) PhysicalPlan { +func predicatePushDownToTableScan(sctx base.PlanContext, plan base.PhysicalPlan) base.PhysicalPlan { switch p := plan.(type) { case *PhysicalSelection: if physicalTableScan, ok := plan.Children()[0].(*PhysicalTableScan); ok && physicalTableScan.StoreType == kv.TiFlash { @@ -57,7 +58,7 @@ func predicatePushDownToTableScan(sctx PlanContext, plan PhysicalPlan) PhysicalP return p.Children()[0] } } else if !ok { - newChildren := make([]PhysicalPlan, 0, len(plan.Children())) + newChildren := make([]base.PhysicalPlan, 0, len(plan.Children())) for _, child := range plan.Children() { newChildren = append(newChildren, predicatePushDownToTableScan(sctx, child)) } @@ -67,7 +68,7 @@ func predicatePushDownToTableScan(sctx PlanContext, plan PhysicalPlan) PhysicalP p.tablePlan = predicatePushDownToTableScan(sctx, p.tablePlan) default: if len(plan.Children()) > 0 { - newChildren := make([]PhysicalPlan, 0, len(plan.Children())) + newChildren := make([]base.PhysicalPlan, 0, len(plan.Children())) for _, child := range plan.Children() { newChildren = append(newChildren, predicatePushDownToTableScan(sctx, child)) } @@ -103,7 +104,7 @@ func transformColumnsToCode(cols []*expression.Column, totalColumnCount int) str // @example: conds = [a > 1, b > 1, a > 2, c > 1, a > 3, b > 2], return = [[a > 3, a > 2, a > 1], [b > 2, b > 1], [c > 1]] // @note: when the selectivity of one group is larger than the threshold, we will remove it from the returned result. // @note: when the number of columns of one group is larger than the threshold, we will remove it from the returned result. -func groupByColumnsSortBySelectivity(sctx PlanContext, conds []expression.Expression, physicalTableScan *PhysicalTableScan) []expressionGroup { +func groupByColumnsSortBySelectivity(sctx base.PlanContext, conds []expression.Expression, physicalTableScan *PhysicalTableScan) []expressionGroup { // Create a map to store the groupMap of conditions keyed by the columns groupMap := make(map[string][]expression.Expression) @@ -204,7 +205,7 @@ func removeSpecificExprsFromSelection(physicalSelection *PhysicalSelection, expr // @param: sctx: the session context // @param: physicalSelection: the PhysicalSelection containing the conditions to be pushed down // @param: physicalTableScan: the PhysicalTableScan to be pushed down to -func predicatePushDownToTableScanImpl(sctx PlanContext, physicalSelection *PhysicalSelection, physicalTableScan *PhysicalTableScan) { +func predicatePushDownToTableScanImpl(sctx base.PlanContext, physicalSelection *PhysicalSelection, physicalTableScan *PhysicalTableScan) { // When the table is small, there is no need to push down the conditions. if physicalTableScan.tblColHists.RealtimeCount <= tiflashDataPackSize || physicalTableScan.KeepOrder { return diff --git a/pkg/planner/core/util.go b/pkg/planner/core/util.go index 7e8e21afa2e27..3ed107a56e272 100644 --- a/pkg/planner/core/util.go +++ b/pkg/planner/core/util.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" + base2 "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/core/internal/base" "github.com/pingcap/tidb/pkg/planner/util/coreusage" "github.com/pingcap/tidb/pkg/sessionctx" @@ -155,7 +156,7 @@ type physicalSchemaProducer struct { basePhysicalPlan } -func (s *physicalSchemaProducer) cloneWithSelf(newSelf PhysicalPlan) (*physicalSchemaProducer, error) { +func (s *physicalSchemaProducer) cloneWithSelf(newSelf base2.PhysicalPlan) (*physicalSchemaProducer, error) { base, err := s.basePhysicalPlan.cloneWithSelf(newSelf) if err != nil { return nil, err @@ -272,7 +273,7 @@ func buildLogicalJoinSchema(joinType JoinType, join LogicalPlan) *expression.Sch } // BuildPhysicalJoinSchema builds the schema of PhysicalJoin from it's children's schema. -func BuildPhysicalJoinSchema(joinType JoinType, join PhysicalPlan) *expression.Schema { +func BuildPhysicalJoinSchema(joinType JoinType, join base2.PhysicalPlan) *expression.Schema { leftSchema := join.Children()[0].Schema() switch joinType { case SemiJoin, AntiSemiJoin: @@ -317,8 +318,8 @@ func GetStatsInfo(i any) map[string]uint64 { // To entirely fix this, uncomment the assertion in TestPreparedIssue17419 return nil } - p := i.(Plan) - var physicalPlan PhysicalPlan + p := i.(base2.Plan) + var physicalPlan base2.PhysicalPlan switch x := p.(type) { case *Insert: physicalPlan = x.SelectPlan @@ -326,7 +327,7 @@ func GetStatsInfo(i any) map[string]uint64 { physicalPlan = x.SelectPlan case *Delete: physicalPlan = x.SelectPlan - case PhysicalPlan: + case base2.PhysicalPlan: physicalPlan = x } @@ -387,7 +388,7 @@ func extractStringFromBoolSlice(slice []bool) string { return strings.Join(l, ",") } -func tableHasDirtyContent(ctx PlanContext, tableInfo *model.TableInfo) bool { +func tableHasDirtyContent(ctx base2.PlanContext, tableInfo *model.TableInfo) bool { pi := tableInfo.GetPartitionInfo() if pi == nil { return ctx.HasDirtyContent(tableInfo.ID) @@ -402,8 +403,8 @@ func tableHasDirtyContent(ctx PlanContext, tableInfo *model.TableInfo) bool { return false } -func clonePhysicalPlan(plans []PhysicalPlan) ([]PhysicalPlan, error) { - cloned := make([]PhysicalPlan, 0, len(plans)) +func clonePhysicalPlan(plans []base2.PhysicalPlan) ([]base2.PhysicalPlan, error) { + cloned := make([]base2.PhysicalPlan, 0, len(plans)) for _, p := range plans { c, err := p.Clone() if err != nil { @@ -472,11 +473,11 @@ func EncodeUniqueIndexValuesForKey(ctx sessionctx.Context, tblInfo *model.TableI } // GetPushDownCtx creates a PushDownContext from PlanContext -func GetPushDownCtx(pctx PlanContext) expression.PushDownContext { +func GetPushDownCtx(pctx base2.PlanContext) expression.PushDownContext { return GetPushDownCtxFromBuildPBContext(pctx.GetBuildPBCtx()) } // GetPushDownCtxFromBuildPBContext creates a PushDownContext from BuildPBContext -func GetPushDownCtxFromBuildPBContext(bctx *BuildPBContext) expression.PushDownContext { +func GetPushDownCtxFromBuildPBContext(bctx *base2.BuildPBContext) expression.PushDownContext { return expression.NewPushDownContext(bctx.GetExprCtx().GetEvalCtx(), bctx.GetClient(), bctx.InExplainStmt, bctx.AppendWarning, bctx.AppendExtraWarning, bctx.GroupConcatMaxLen) } diff --git a/pkg/planner/implementation/BUILD.bazel b/pkg/planner/implementation/BUILD.bazel index 77ac6b0767bff..37e1667760c84 100644 --- a/pkg/planner/implementation/BUILD.bazel +++ b/pkg/planner/implementation/BUILD.bazel @@ -17,6 +17,7 @@ go_library( "//pkg/parser/model", "//pkg/planner/cardinality", "//pkg/planner/core", + "//pkg/planner/core/base", "//pkg/planner/memo", "//pkg/statistics", ], diff --git a/pkg/planner/implementation/base.go b/pkg/planner/implementation/base.go index 37bd970dc381d..aeac3558babcf 100644 --- a/pkg/planner/implementation/base.go +++ b/pkg/planner/implementation/base.go @@ -15,7 +15,7 @@ package implementation import ( - plannercore "github.com/pingcap/tidb/pkg/planner/core" + plannercore "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/memo" ) diff --git a/pkg/planner/memo/BUILD.bazel b/pkg/planner/memo/BUILD.bazel index 720862b8983ba..188c164fed8ea 100644 --- a/pkg/planner/memo/BUILD.bazel +++ b/pkg/planner/memo/BUILD.bazel @@ -13,6 +13,7 @@ go_library( deps = [ "//pkg/expression", "//pkg/planner/core", + "//pkg/planner/core/base", "//pkg/planner/pattern", "//pkg/planner/property", ], @@ -37,6 +38,7 @@ go_test( "//pkg/parser", "//pkg/parser/model", "//pkg/planner/core", + "//pkg/planner/core/base", "//pkg/planner/pattern", "//pkg/planner/property", "//pkg/sessionctx/variable", diff --git a/pkg/planner/memo/group_test.go b/pkg/planner/memo/group_test.go index ae4e676b17df7..423d5eb09b2a0 100644 --- a/pkg/planner/memo/group_test.go +++ b/pkg/planner/memo/group_test.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/model" plannercore "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/pattern" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/sessionctx/variable" @@ -177,13 +178,13 @@ func TestGroupGetFirstElem(t *testing.T) { } type fakeImpl struct { - plan plannercore.PhysicalPlan + plan base.PhysicalPlan } func (impl *fakeImpl) CalcCost(float64, ...Implementation) float64 { return 0 } func (impl *fakeImpl) SetCost(float64) {} func (impl *fakeImpl) GetCost() float64 { return 0 } -func (impl *fakeImpl) GetPlan() plannercore.PhysicalPlan { return impl.plan } +func (impl *fakeImpl) GetPlan() base.PhysicalPlan { return impl.plan } func (impl *fakeImpl) AttachChildren(...Implementation) Implementation { return nil } func (impl *fakeImpl) GetCostLimit(float64, ...Implementation) float64 { return 0 } diff --git a/pkg/planner/memo/implementation.go b/pkg/planner/memo/implementation.go index e0552150e8b1b..a7417a040c5c4 100644 --- a/pkg/planner/memo/implementation.go +++ b/pkg/planner/memo/implementation.go @@ -15,7 +15,7 @@ package memo import ( - plannercore "github.com/pingcap/tidb/pkg/planner/core" + plannercore "github.com/pingcap/tidb/pkg/planner/core/base" ) // Implementation defines the interface for cost of physical plan. diff --git a/pkg/planner/optimize.go b/pkg/planner/optimize.go index 915fc537fcbc9..23946a250e400 100644 --- a/pkg/planner/optimize.go +++ b/pkg/planner/optimize.go @@ -32,6 +32,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/cascades" pctx "github.com/pingcap/tidb/pkg/planner/context" "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/util/debugtrace" "github.com/pingcap/tidb/pkg/privilege" "github.com/pingcap/tidb/pkg/sessionctx" @@ -59,7 +60,7 @@ func IsReadOnly(node ast.Node, vars *variable.SessionVars) bool { } // getPlanFromNonPreparedPlanCache tries to get an available cached plan from the NonPrepared Plan Cache for this stmt. -func getPlanFromNonPreparedPlanCache(ctx context.Context, sctx sessionctx.Context, stmt ast.StmtNode, is infoschema.InfoSchema) (p core.Plan, ns types.NameSlice, ok bool, err error) { +func getPlanFromNonPreparedPlanCache(ctx context.Context, sctx sessionctx.Context, stmt ast.StmtNode, is infoschema.InfoSchema) (p base.Plan, ns types.NameSlice, ok bool, err error) { stmtCtx := sctx.GetSessionVars().StmtCtx _, isExplain := stmt.(*ast.ExplainStmt) if !sctx.GetSessionVars().EnableNonPreparedPlanCache || // disabled @@ -125,7 +126,7 @@ func getPlanFromNonPreparedPlanCache(ctx context.Context, sctx sessionctx.Contex } // Optimize does optimization and creates a Plan. -func Optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is infoschema.InfoSchema) (plan core.Plan, slice types.NameSlice, retErr error) { +func Optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is infoschema.InfoSchema) (plan base.Plan, slice types.NameSlice, retErr error) { sessVars := sctx.GetSessionVars() pctx := sctx.GetPlanCtx() if sessVars.StmtCtx.EnableOptimizerDebugTrace { @@ -200,7 +201,7 @@ func Optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is in } if _, isolationReadContainTiKV := sessVars.IsolationReadEngines[kv.TiKV]; isolationReadContainTiKV { - var fp core.Plan + var fp base.Plan if fpv, ok := sctx.Value(core.PointPlanKey).(core.PointPlanVal); ok { // point plan is already tried in a multi-statement query. fp = fpv.Plan @@ -264,7 +265,7 @@ func Optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is in var ( names types.NameSlice - bestPlan, bestPlanFromBind core.Plan + bestPlan, bestPlanFromBind base.Plan chosenBinding bindinfo.Binding err error ) @@ -371,7 +372,7 @@ func Optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is in // OptimizeForForeignKeyCascade does optimization and creates a Plan for foreign key cascade. // Compare to Optimize, OptimizeForForeignKeyCascade only build plan by StmtNode, // doesn't consider plan cache and plan binding, also doesn't do privilege check. -func OptimizeForForeignKeyCascade(ctx context.Context, sctx pctx.PlanContext, node ast.StmtNode, is infoschema.InfoSchema) (core.Plan, error) { +func OptimizeForForeignKeyCascade(ctx context.Context, sctx pctx.PlanContext, node ast.StmtNode, is infoschema.InfoSchema) (base.Plan, error) { builder := planBuilderPool.Get().(*core.PlanBuilder) defer planBuilderPool.Put(builder.ResetForReuse()) hintProcessor := hint.NewQBHintHandler(sctx.GetSessionVars().StmtCtx) @@ -435,7 +436,7 @@ var planBuilderPool = sync.Pool{ // optimizeCnt is a global variable only used for test. var optimizeCnt int -func optimize(ctx context.Context, sctx pctx.PlanContext, node ast.Node, is infoschema.InfoSchema) (core.Plan, types.NameSlice, float64, error) { +func optimize(ctx context.Context, sctx pctx.PlanContext, node ast.Node, is infoschema.InfoSchema) (base.Plan, types.NameSlice, float64, error) { failpoint.Inject("checkOptimizeCountOne", func(val failpoint.Value) { // only count the optimization for SQL with specified text if testSQL, ok := val.(string); ok && testSQL == node.OriginalText() { @@ -508,7 +509,7 @@ func optimize(ctx context.Context, sctx pctx.PlanContext, node ast.Node, is info // OptimizeExecStmt to handle the "execute" statement func OptimizeExecStmt(ctx context.Context, sctx sessionctx.Context, - execAst *ast.ExecuteStmt, is infoschema.InfoSchema) (core.Plan, types.NameSlice, error) { + execAst *ast.ExecuteStmt, is infoschema.InfoSchema) (base.Plan, types.NameSlice, error) { builder := planBuilderPool.Get().(*core.PlanBuilder) defer planBuilderPool.Put(builder.ResetForReuse()) pctx := sctx.GetPlanCtx() @@ -532,7 +533,7 @@ func OptimizeExecStmt(ctx context.Context, sctx sessionctx.Context, return exec, names, nil } -func buildLogicalPlan(ctx context.Context, sctx pctx.PlanContext, node ast.Node, builder *core.PlanBuilder) (core.Plan, error) { +func buildLogicalPlan(ctx context.Context, sctx pctx.PlanContext, node ast.Node, builder *core.PlanBuilder) (base.Plan, error) { sctx.GetSessionVars().PlanID.Store(0) sctx.GetSessionVars().PlanColumnID.Store(0) sctx.GetSessionVars().MapScalarSubQ = nil diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel index c6b7ad1e527f7..223002db73edc 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -44,6 +44,7 @@ go_library( "//pkg/parser/mysql", "//pkg/parser/terror", "//pkg/planner/core", + "//pkg/planner/core/base", "//pkg/plugin", "//pkg/privilege", "//pkg/privilege/conn", diff --git a/pkg/server/conn.go b/pkg/server/conn.go index 884141069c0e8..46ed2038fb836 100644 --- a/pkg/server/conn.go +++ b/pkg/server/conn.go @@ -74,6 +74,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" plannercore "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/plugin" "github.com/pingcap/tidb/pkg/privilege" "github.com/pingcap/tidb/pkg/privilege/conn" @@ -1714,7 +1715,7 @@ func (cc *clientConn) handleQuery(ctx context.Context, sql string) (err error) { warns := sc.GetWarnings() parserWarns := warns[len(prevWarns):] - var pointPlans []plannercore.Plan + var pointPlans []base.Plan cc.ctx.GetSessionVars().InMultiStmts = false if len(stmts) > 1 { // The client gets to choose if it allows multi-statements, and @@ -1816,7 +1817,7 @@ func (cc *clientConn) handleQuery(ctx context.Context, sql string) (err error) { // prefetchPointPlanKeys extracts the point keys in multi-statement query, // use BatchGet to get the keys, so the values will be cached in the snapshot cache, save RPC call cost. // For pessimistic transaction, the keys will be batch locked. -func (cc *clientConn) prefetchPointPlanKeys(ctx context.Context, stmts []ast.StmtNode, sqls string) ([]plannercore.Plan, error) { +func (cc *clientConn) prefetchPointPlanKeys(ctx context.Context, stmts []ast.StmtNode, sqls string) ([]base.Plan, error) { txn, err := cc.ctx.Txn(false) if err != nil { return nil, err @@ -1837,12 +1838,12 @@ func (cc *clientConn) prefetchPointPlanKeys(ctx context.Context, stmts []ast.Stm return nil, nil } } - pointPlans := make([]plannercore.Plan, len(stmts)) + pointPlans := make([]base.Plan, len(stmts)) var idxKeys []kv.Key //nolint: prealloc var rowKeys []kv.Key //nolint: prealloc isCommonHandle := make(map[string]bool, 0) - handlePlan := func(sctx sessionctx.Context, p plannercore.PhysicalPlan, resetStmtCtxFn func()) error { + handlePlan := func(sctx sessionctx.Context, p base.PhysicalPlan, resetStmtCtxFn func()) error { var tableID int64 switch v := p.(type) { case *plannercore.PointGetPlan: diff --git a/pkg/session/BUILD.bazel b/pkg/session/BUILD.bazel index 12b10d2cb8655..142dc508662a8 100644 --- a/pkg/session/BUILD.bazel +++ b/pkg/session/BUILD.bazel @@ -59,6 +59,7 @@ go_library( "//pkg/planner/context", "//pkg/planner/contextimpl", "//pkg/planner/core", + "//pkg/planner/core/base", "//pkg/plugin", "//pkg/privilege", "//pkg/privilege/conn", diff --git a/pkg/session/session.go b/pkg/session/session.go index d201a30781882..c761b0399f9bf 100644 --- a/pkg/session/session.go +++ b/pkg/session/session.go @@ -73,6 +73,7 @@ import ( "github.com/pingcap/tidb/pkg/planner" planctx "github.com/pingcap/tidb/pkg/planner/context" plannercore "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/plugin" "github.com/pingcap/tidb/pkg/privilege" "github.com/pingcap/tidb/pkg/privilege/conn" @@ -172,7 +173,7 @@ type session struct { } currentCtx context.Context // only use for runtime.trace, Please NEVER use it. - currentPlan plannercore.Plan + currentPlan base.Plan store kv.Storage diff --git a/pkg/sessiontxn/isolation/BUILD.bazel b/pkg/sessiontxn/isolation/BUILD.bazel index a2ac53f3abe88..afaafcf40a420 100644 --- a/pkg/sessiontxn/isolation/BUILD.bazel +++ b/pkg/sessiontxn/isolation/BUILD.bazel @@ -20,6 +20,7 @@ go_library( "//pkg/parser/mysql", "//pkg/parser/terror", "//pkg/planner/core", + "//pkg/planner/core/base", "//pkg/sessionctx", "//pkg/sessionctx/binloginfo", "//pkg/sessionctx/variable", diff --git a/pkg/sessiontxn/isolation/optimistic.go b/pkg/sessiontxn/isolation/optimistic.go index c68ef59a6f5fb..a0b6554bc4d59 100644 --- a/pkg/sessiontxn/isolation/optimistic.go +++ b/pkg/sessiontxn/isolation/optimistic.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/mysql" plannercore "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/sessiontxn" @@ -120,7 +121,7 @@ func (p *OptimisticTxnContextProvider) AdviseOptimizeWithPlan(plan any) (err err return nil } - realPlan, ok := plan.(plannercore.Plan) + realPlan, ok := plan.(base.Plan) if !ok { return nil } diff --git a/pkg/sessiontxn/isolation/readcommitted.go b/pkg/sessiontxn/isolation/readcommitted.go index 6c60f8e2c885a..7cd946df70ac9 100644 --- a/pkg/sessiontxn/isolation/readcommitted.go +++ b/pkg/sessiontxn/isolation/readcommitted.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/terror" plannercore "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/sessiontxn" @@ -263,11 +264,11 @@ func (p *PessimisticRCTxnContextProvider) AdviseWarmup() error { } // planSkipGetTsoFromPD identifies the plans which don't need get newest ts from PD. -func planSkipGetTsoFromPD(sctx sessionctx.Context, plan plannercore.Plan, inLockOrWriteStmt bool) bool { +func planSkipGetTsoFromPD(sctx sessionctx.Context, plan base.Plan, inLockOrWriteStmt bool) bool { switch v := plan.(type) { case *plannercore.PointGetPlan: return sctx.GetSessionVars().RcWriteCheckTS && (v.Lock || inLockOrWriteStmt) - case plannercore.PhysicalPlan: + case base.PhysicalPlan: if len(v.Children()) == 0 { return false } @@ -302,7 +303,7 @@ func (p *PessimisticRCTxnContextProvider) AdviseOptimizeWithPlan(val any) (err e return nil } - plan, ok := val.(plannercore.Plan) + plan, ok := val.(base.Plan) if !ok { return nil } diff --git a/pkg/sessiontxn/isolation/repeatable_read.go b/pkg/sessiontxn/isolation/repeatable_read.go index 1303687724b32..48b71d488ef92 100644 --- a/pkg/sessiontxn/isolation/repeatable_read.go +++ b/pkg/sessiontxn/isolation/repeatable_read.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/terror" plannercore "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/sessiontxn" @@ -185,7 +186,7 @@ func (p *PessimisticRRTxnContextProvider) AdviseOptimizeWithPlan(val any) (err e return nil } - plan, ok := val.(plannercore.Plan) + plan, ok := val.(base.Plan) if !ok { return nil } @@ -203,7 +204,7 @@ func (p *PessimisticRRTxnContextProvider) AdviseOptimizeWithPlan(val any) (err e // Note: For point get and batch point get (name it plan), if one of the ancestor node is update/delete/physicalLock, // we should check whether the plan.Lock is true or false. See comments in needNotToBeOptimized. // inLockOrWriteStmt = true means one of the ancestor node is update/delete/physicalLock. -func notNeedGetLatestTSFromPD(plan plannercore.Plan, inLockOrWriteStmt bool) bool { +func notNeedGetLatestTSFromPD(plan base.Plan, inLockOrWriteStmt bool) bool { switch v := plan.(type) { case *plannercore.PointGetPlan: // We do not optimize the point get/ batch point get if plan.lock = false and inLockOrWriteStmt = true. @@ -213,7 +214,7 @@ func notNeedGetLatestTSFromPD(plan plannercore.Plan, inLockOrWriteStmt bool) boo return !inLockOrWriteStmt || v.Lock case *plannercore.BatchPointGetPlan: return !inLockOrWriteStmt || v.Lock - case plannercore.PhysicalPlan: + case base.PhysicalPlan: if len(v.Children()) == 0 { return false }