Skip to content

Commit

Permalink
planner: move loical plan functionality AppendCandidate4PhysicalOptim…
Browse files Browse the repository at this point in the history
…izeOp to util (#53198)

ref #51664, ref #52714
  • Loading branch information
AilinKid authored May 13, 2024
1 parent e5651c4 commit fa94f49
Show file tree
Hide file tree
Showing 12 changed files with 89 additions and 40 deletions.
2 changes: 2 additions & 0 deletions pkg/planner/core/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,7 @@ go_library(
"access_object.go",
"collect_column_stats_usage.go",
"common_plans.go",
"core_init.go",
"debugtrace.go",
"encode.go",
"exhaust_physical_plans.go",
Expand Down Expand Up @@ -125,6 +126,7 @@ go_library(
"//pkg/planner/util/fixcontrol",
"//pkg/planner/util/optimizetrace",
"//pkg/planner/util/tablesampler",
"//pkg/planner/util/utilfuncp",
"//pkg/privilege",
"//pkg/sessionctx",
"//pkg/sessionctx/stmtctx",
Expand Down
46 changes: 46 additions & 0 deletions pkg/planner/core/core_init.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,46 @@
// 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 core

import (
"github.com/pingcap/tidb/pkg/expression"
"github.com/pingcap/tidb/pkg/planner/cardinality"
plannerutil "github.com/pingcap/tidb/pkg/planner/util"
"github.com/pingcap/tidb/pkg/planner/util/utilfuncp"
"github.com/pingcap/tidb/pkg/statistics"
"github.com/pingcap/tidb/pkg/util/set"
"go.uber.org/atomic"
)

func init() {
// For code refactor init.
utilfuncp.HasMaxOneRowUtil = HasMaxOneRow
utilfuncp.AppendCandidate4PhysicalOptimizeOp = appendCandidate4PhysicalOptimizeOp

// For mv index init.
cardinality.GetTblInfoForUsedStatsByPhysicalID = getTblInfoForUsedStatsByPhysicalID
cardinality.CollectFilters4MVIndex = collectFilters4MVIndex
cardinality.BuildPartialPaths4MVIndex = buildPartialPaths4MVIndex
statistics.PrepareCols4MVIndex = PrepareIdxColsAndUnwrapArrayType

// For basic optimizer init.
expression.EvalSimpleAst = evalAstExpr
expression.BuildSimpleExpr = buildSimpleExpr
expression.DecodeKeyFromString = decodeKeyFromString
plannerutil.EvalAstExprWithPlanCtx = evalAstExprWithPlanCtx
plannerutil.RewriteAstExprWithPlanCtx = rewriteAstExprWithPlanCtx
DefaultDisabledLogicalRulesList = new(atomic.Value)
DefaultDisabledLogicalRulesList.Store(set.NewStringSet())
}
11 changes: 6 additions & 5 deletions pkg/planner/core/find_best_task.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@ import (
"github.com/pingcap/tidb/pkg/planner/util"
"github.com/pingcap/tidb/pkg/planner/util/fixcontrol"
"github.com/pingcap/tidb/pkg/planner/util/optimizetrace"
"github.com/pingcap/tidb/pkg/planner/util/utilfuncp"
"github.com/pingcap/tidb/pkg/statistics"
"github.com/pingcap/tidb/pkg/types"
tidbutil "github.com/pingcap/tidb/pkg/util"
Expand Down Expand Up @@ -103,7 +104,7 @@ func (p *LogicalTableDual) FindBestTask(prop *property.PhysicalProperty, planCou
}.Init(p.SCtx(), p.StatsInfo(), p.QueryBlockOffset())
dual.SetSchema(p.schema)
planCounter.Dec(1)
appendCandidate4PhysicalOptimizeOp(opt, p, dual, prop)
utilfuncp.AppendCandidate4PhysicalOptimizeOp(opt, p, dual, prop)
rt := &RootTask{}
rt.SetPlan(dual)
rt.SetEmpty(p.RowCount == 0)
Expand Down Expand Up @@ -239,7 +240,7 @@ func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(
bestTask = curTask
break
}
appendCandidate4PhysicalOptimizeOp(opt, p, curTask.Plan(), prop)
utilfuncp.AppendCandidate4PhysicalOptimizeOp(opt, p, curTask.Plan(), prop)
// Get the most efficient one.
if curIsBetter, err := compareTaskCost(curTask, bestTask, opt); err != nil {
return nil, 0, err
Expand Down Expand Up @@ -579,7 +580,7 @@ func (p *baseLogicalPlan) FindBestTask(prop *property.PhysicalProperty, planCoun
bestTask = curTask
goto END
}
appendCandidate4PhysicalOptimizeOp(opt, p, curTask.Plan(), prop)
utilfuncp.AppendCandidate4PhysicalOptimizeOp(opt, p, curTask.Plan(), prop)
if curIsBetter, err := compareTaskCost(curTask, bestTask, opt); err != nil {
return nil, 0, err
} else if curIsBetter {
Expand Down Expand Up @@ -639,7 +640,7 @@ func (p *LogicalMemTable) FindBestTask(prop *property.PhysicalProperty, planCoun
}.Init(p.SCtx(), p.StatsInfo(), p.QueryBlockOffset())
memTable.SetSchema(p.schema)
planCounter.Dec(1)
appendCandidate4PhysicalOptimizeOp(opt, p, memTable, prop)
utilfuncp.AppendCandidate4PhysicalOptimizeOp(opt, p, memTable, prop)
rt := &RootTask{}
rt.SetPlan(memTable)
return rt, 1, nil
Expand Down Expand Up @@ -2926,7 +2927,7 @@ func appendCandidate(lp base.LogicalPlan, task base.Task, prop *property.Physica
if task == nil || task.Invalid() {
return
}
appendCandidate4PhysicalOptimizeOp(opt, lp, task.Plan(), prop)
utilfuncp.AppendCandidate4PhysicalOptimizeOp(opt, lp, task.Plan(), prop)
}

// PushDownNot here can convert condition 'not (a != 1)' to 'a = 1'. When we build range from conds, the condition like
Expand Down
6 changes: 0 additions & 6 deletions pkg/planner/core/indexmerge_path.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,12 +43,6 @@ import (
"golang.org/x/exp/maps"
)

func init() {
cardinality.CollectFilters4MVIndex = collectFilters4MVIndex
cardinality.BuildPartialPaths4MVIndex = buildPartialPaths4MVIndex
statistics.PrepareCols4MVIndex = PrepareIdxColsAndUnwrapArrayType
}

// generateIndexMergePath generates IndexMerge AccessPaths on this DataSource.
func (ds *DataSource) generateIndexMergePath() error {
if ds.SCtx().GetSessionVars().StmtCtx.EnableOptimizerDebugTrace {
Expand Down
11 changes: 0 additions & 11 deletions pkg/planner/core/optimizer.go
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,6 @@ import (
"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/debugtrace"
"github.com/pingcap/tidb/pkg/planner/util/optimizetrace"
"github.com/pingcap/tidb/pkg/privilege"
Expand Down Expand Up @@ -1172,16 +1171,6 @@ func existsCartesianProduct(p base.LogicalPlan) bool {
// DefaultDisabledLogicalRulesList indicates the logical rules which should be banned.
var DefaultDisabledLogicalRulesList *atomic.Value

func init() {
expression.EvalSimpleAst = evalAstExpr
expression.BuildSimpleExpr = buildSimpleExpr
expression.DecodeKeyFromString = decodeKeyFromString
plannerutil.EvalAstExprWithPlanCtx = evalAstExprWithPlanCtx
plannerutil.RewriteAstExprWithPlanCtx = rewriteAstExprWithPlanCtx
DefaultDisabledLogicalRulesList = new(atomic.Value)
DefaultDisabledLogicalRulesList.Store(set.NewStringSet())
}

func disableReuseChunkIfNeeded(sctx base.PlanContext, plan base.PhysicalPlan) {
if !sctx.GetSessionVars().IsAllocValid() {
return
Expand Down
7 changes: 2 additions & 5 deletions pkg/planner/core/plan.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,17 +27,14 @@ import (
"github.com/pingcap/tidb/pkg/planner/util"
"github.com/pingcap/tidb/pkg/planner/util/costusage"
"github.com/pingcap/tidb/pkg/planner/util/optimizetrace"
"github.com/pingcap/tidb/pkg/planner/util/utilfuncp"
"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/types"
"github.com/pingcap/tidb/pkg/util/execdetails"
"github.com/pingcap/tidb/pkg/util/size"
"github.com/pingcap/tidb/pkg/util/tracing"
)

func init() {
util.HasMaxOneRowUtil = HasMaxOneRow
}

// AsSctx converts PlanContext to sessionctx.Context.
func AsSctx(pctx base.PlanContext) (sessionctx.Context, error) {
sctx, ok := pctx.(sessionctx.Context)
Expand Down Expand Up @@ -479,7 +476,7 @@ func (p *baseLogicalPlan) BuildKeyInfo(_ *expression.Schema, _ []*expression.Sch
for i := range p.children {
childMaxOneRow[i] = p.children[i].MaxOneRow()
}
p.maxOneRow = util.HasMaxOneRowUtil(p.self, childMaxOneRow)
p.maxOneRow = utilfuncp.HasMaxOneRowUtil(p.self, childMaxOneRow)
}

// BuildKeyInfo implements LogicalPlan BuildKeyInfo interface.
Expand Down
5 changes: 0 additions & 5 deletions pkg/planner/core/stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -217,11 +217,6 @@ func (ds *DataSource) getGroupNDVs(colGroups [][]*expression.Column) []property.
return ndvs
}

func init() {
// To handle cycle import, we have to define this function here.
cardinality.GetTblInfoForUsedStatsByPhysicalID = getTblInfoForUsedStatsByPhysicalID
}

// getTblInfoForUsedStatsByPhysicalID get table name, partition name and HintedTable that will be used to record used stats.
func getTblInfoForUsedStatsByPhysicalID(sctx base.PlanContext, id int64) (fullName string, tblInfo *model.TableInfo) {
fullName = "tableID " + strconv.FormatInt(id, 10)
Expand Down
2 changes: 1 addition & 1 deletion pkg/planner/memo/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@ go_library(
"//pkg/planner/core/base",
"//pkg/planner/pattern",
"//pkg/planner/property",
"//pkg/planner/util",
"//pkg/planner/util/utilfuncp",
],
)

Expand Down
4 changes: 2 additions & 2 deletions pkg/planner/memo/group.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ import (
"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/planner/util"
"github.com/pingcap/tidb/pkg/planner/util/utilfuncp"
)

// ExploreMark is uses to mark whether a Group or GroupExpr has
Expand Down Expand Up @@ -223,5 +223,5 @@ func (g *Group) BuildKeyInfo() {
g.Prop.Schema.Keys = childSchema[0].Keys
}
e.ExprNode.BuildKeyInfo(g.Prop.Schema, childSchema)
g.Prop.MaxOneRow = e.ExprNode.MaxOneRow() || util.HasMaxOneRowUtil(e.ExprNode, childMaxOneRow)
g.Prop.MaxOneRow = e.ExprNode.MaxOneRow() || utilfuncp.HasMaxOneRowUtil(e.ExprNode, childMaxOneRow)
}
2 changes: 0 additions & 2 deletions pkg/planner/util/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -5,7 +5,6 @@ go_library(
srcs = [
"byitem.go",
"expression.go",
"func_pointer_misc.go",
"handle_cols.go",
"misc.go",
"null_misc.go",
Expand All @@ -20,7 +19,6 @@ go_library(
"//pkg/parser/model",
"//pkg/parser/mysql",
"//pkg/planner/context",
"//pkg/planner/core/base",
"//pkg/sessionctx/stmtctx",
"//pkg/tablecodec",
"//pkg/types",
Expand Down
13 changes: 13 additions & 0 deletions pkg/planner/util/utilfuncp/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -0,0 +1,13 @@
load("@io_bazel_rules_go//go:def.bzl", "go_library")

go_library(
name = "utilfuncp",
srcs = ["func_pointer_misc.go"],
importpath = "github.com/pingcap/tidb/pkg/planner/util/utilfuncp",
visibility = ["//visibility:public"],
deps = [
"//pkg/planner/core/base",
"//pkg/planner/property",
"//pkg/planner/util/optimizetrace",
],
)
Original file line number Diff line number Diff line change
Expand Up @@ -12,14 +12,28 @@
// See the License for the specific language governing permissions and
// limitations under the License.

package util
package utilfuncp

import "github.com/pingcap/tidb/pkg/planner/core/base"
import (
"github.com/pingcap/tidb/pkg/planner/core/base"
"github.com/pingcap/tidb/pkg/planner/property"
"github.com/pingcap/tidb/pkg/planner/util/optimizetrace"
)

// this file is used for passing function pointer at init(){} to avoid some import cycles.

// HasMaxOneRowUtil is used in baseLogicalPlan implementation of LogicalPlan interface, while
// the original HasMaxOneRowUtil has some dependency of original core pkg: like Datasource which
// hasn't been moved out of core pkg, so associative func pointer is introduced.
// todo: (1) arenatlx, remove this func pointer when concrete Logical Operators moved out.
// todo: (1) arenatlx, remove this func pointer when concrete Logical Operators moved out of core.
var HasMaxOneRowUtil func(p base.LogicalPlan, childMaxOneRow []bool) bool

// AppendCandidate4PhysicalOptimizeOp is used in all logicalOp's findBestTask to trace the physical
// optimizing steps. Since we try to move baseLogicalPlan out of core, then other concrete logical
// operators, this appendCandidate4PhysicalOptimizeOp will make logicalOp/pkg back import core/pkg;
// if we move appendCandidate4PhysicalOptimizeOp together with baseLogicalPlan to logicalOp/pkg, it
// will heavily depend on concrete other logical operators inside, which are still defined in core/pkg
// too.
// todo: (2) arenatlx, remove this func pointer when concrete Logical Operators moved out of core.
var AppendCandidate4PhysicalOptimizeOp func(pop *optimizetrace.PhysicalOptimizeOp, lp base.LogicalPlan,
pp base.PhysicalPlan, prop *property.PhysicalProperty)

0 comments on commit fa94f49

Please sign in to comment.