Skip to content

Commit

Permalink
planner: remove some useless function pointer after logicalop are all…
Browse files Browse the repository at this point in the history
… migrated. (#56393)

ref #51664, ref #52714
  • Loading branch information
AilinKid authored Sep 30, 2024
1 parent dc03726 commit c46e4bc
Show file tree
Hide file tree
Showing 9 changed files with 39 additions and 65 deletions.
1 change: 0 additions & 1 deletion pkg/planner/core/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -122,7 +122,6 @@ go_library(
"//pkg/planner/cardinality",
"//pkg/planner/cascades/base",
"//pkg/planner/core/base",
"//pkg/planner/core/constraint",
"//pkg/planner/core/cost",
"//pkg/planner/core/metrics",
"//pkg/planner/core/operator/baseimpl",
Expand Down
3 changes: 0 additions & 3 deletions pkg/planner/core/core_init.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,10 +27,8 @@ import (

func init() {
// For code refactor init.
utilfuncp.AddSelection = addSelection
utilfuncp.FindBestTask = findBestTask
utilfuncp.PruneByItems = pruneByItems
utilfuncp.GetTaskPlanCost = getTaskPlanCost
utilfuncp.CanPushToCopImpl = canPushToCopImpl
utilfuncp.PushDownTopNForBaseLogicalPlan = pushDownTopNForBaseLogicalPlan
utilfuncp.FindBestTask4LogicalCTE = findBestTask4LogicalCTE
Expand Down Expand Up @@ -60,7 +58,6 @@ func init() {

utilfuncp.GetActualProbeCntFromProbeParents = getActualProbeCntFromProbeParents
utilfuncp.GetEstimatedProbeCntFromProbeParents = getEstimatedProbeCntFromProbeParents
utilfuncp.AppendCandidate4PhysicalOptimizeOp = appendCandidate4PhysicalOptimizeOp

utilfuncp.DoOptimize = doOptimize
utilfuncp.IsSingleScan = isSingleScan
Expand Down
15 changes: 7 additions & 8 deletions pkg/planner/core/find_best_task.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,6 @@ 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 +102,7 @@ func findBestTask4LogicalTableDual(lp base.LogicalPlan, prop *property.PhysicalP
}.Init(p.SCtx(), p.StatsInfo(), p.QueryBlockOffset())
dual.SetSchema(p.Schema())
planCounter.Dec(1)
utilfuncp.AppendCandidate4PhysicalOptimizeOp(opt, p, dual, prop)
appendCandidate4PhysicalOptimizeOp(opt, p, dual, prop)
rt := &RootTask{}
rt.SetPlan(dual)
rt.SetEmpty(p.RowCount == 0)
Expand Down Expand Up @@ -240,7 +239,7 @@ func enumeratePhysicalPlans4Task(
bestTask = curTask
break
}
utilfuncp.AppendCandidate4PhysicalOptimizeOp(opt, p, curTask.Plan(), prop)
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 @@ -346,11 +345,11 @@ func iterateChildPlan4LogicalSequence(

// compareTaskCost compares cost of curTask and bestTask and returns whether curTask's cost is smaller than bestTask's.
func compareTaskCost(curTask, bestTask base.Task, op *optimizetrace.PhysicalOptimizeOp) (curIsBetter bool, err error) {
curCost, curInvalid, err := utilfuncp.GetTaskPlanCost(curTask, op)
curCost, curInvalid, err := getTaskPlanCost(curTask, op)
if err != nil {
return false, err
}
bestCost, bestInvalid, err := utilfuncp.GetTaskPlanCost(bestTask, op)
bestCost, bestInvalid, err := getTaskPlanCost(bestTask, op)
if err != nil {
return false, err
}
Expand Down Expand Up @@ -586,7 +585,7 @@ func findBestTask(lp base.LogicalPlan, prop *property.PhysicalProperty, planCoun
bestTask = curTask
goto END
}
utilfuncp.AppendCandidate4PhysicalOptimizeOp(opt, p, curTask.Plan(), prop)
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 @@ -646,7 +645,7 @@ func findBestTask4LogicalMemTable(lp base.LogicalPlan, prop *property.PhysicalPr
}.Init(p.SCtx(), p.StatsInfo(), p.QueryBlockOffset())
memTable.SetSchema(p.Schema())
planCounter.Dec(1)
utilfuncp.AppendCandidate4PhysicalOptimizeOp(opt, p, memTable, prop)
appendCandidate4PhysicalOptimizeOp(opt, p, memTable, prop)
rt := &RootTask{}
rt.SetPlan(memTable)
return rt, 1, nil
Expand Down Expand Up @@ -2984,7 +2983,7 @@ func appendCandidate(lp base.LogicalPlan, task base.Task, prop *property.Physica
if task == nil || task.Invalid() {
return
}
utilfuncp.AppendCandidate4PhysicalOptimizeOp(opt, lp, task.Plan(), prop)
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
2 changes: 1 addition & 1 deletion pkg/planner/core/operator/logicalop/base_logical_plan.go
Original file line number Diff line number Diff line change
Expand Up @@ -122,7 +122,7 @@ func (p *BaseLogicalPlan) PredicatePushDown(predicates []expression.Expression,
}
child := p.children[0]
rest, newChild := child.PredicatePushDown(predicates, opt)
utilfuncp.AddSelection(p.self, newChild, rest, 0, opt)
addSelection(p.self, newChild, rest, 0, opt)
return nil, p.self
}

Expand Down
4 changes: 2 additions & 2 deletions pkg/planner/core/operator/logicalop/logical_join.go
Original file line number Diff line number Diff line change
Expand Up @@ -284,8 +284,8 @@ func (p *LogicalJoin) PredicatePushDown(predicates []expression.Expression, opt
rightCond = expression.RemoveDupExprs(rightCond)
leftRet, lCh := p.Children()[0].PredicatePushDown(leftCond, opt)
rightRet, rCh := p.Children()[1].PredicatePushDown(rightCond, opt)
utilfuncp.AddSelection(p, lCh, leftRet, 0, opt)
utilfuncp.AddSelection(p, rCh, rightRet, 1, opt)
addSelection(p, lCh, leftRet, 0, opt)
addSelection(p, rCh, rightRet, 1, opt)
p.updateEQCond()
ruleutil.BuildKeyInfoPortal(p)
return ret, p.Self()
Expand Down
28 changes: 28 additions & 0 deletions pkg/planner/core/operator/logicalop/logical_plans_misc.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,10 @@
package logicalop

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

var (
Expand Down Expand Up @@ -70,3 +73,28 @@ func HasMaxOneRow(p base.LogicalPlan, childMaxOneRow []bool) bool {
}
return false
}

func addSelection(p base.LogicalPlan, child base.LogicalPlan, conditions []expression.Expression, chIdx int, opt *optimizetrace.LogicalOptimizeOp) {
if len(conditions) == 0 {
p.Children()[chIdx] = child
return
}
conditions = expression.PropagateConstant(p.SCtx().GetExprCtx(), conditions)
// Return table dual when filter is constant false or null.
dual := Conds2TableDual(child, conditions)
if dual != nil {
p.Children()[chIdx] = dual
AppendTableDualTraceStep(child, dual, conditions, opt)
return
}

conditions = constraint.DeleteTrueExprs(p, conditions)
if len(conditions) == 0 {
p.Children()[chIdx] = child
return
}
selection := LogicalSelection{Conditions: conditions}.Init(p.SCtx(), p.QueryBlockOffset())
selection.SetChildren(child)
p.Children()[chIdx] = selection
AppendAddSelectionTraceStep(p, child, selection, opt)
}
2 changes: 1 addition & 1 deletion pkg/planner/core/operator/logicalop/logical_union_all.go
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,7 @@ func (p *LogicalUnionAll) PredicatePushDown(predicates []expression.Expression,
newExprs := make([]expression.Expression, 0, len(predicates))
newExprs = append(newExprs, predicates...)
retCond, newChild := proj.PredicatePushDown(newExprs, opt)
utilfuncp.AddSelection(p, newChild, retCond, i, opt)
addSelection(p, newChild, retCond, i, opt)
}
return nil, p
}
Expand Down
26 changes: 0 additions & 26 deletions pkg/planner/core/rule_predicate_push_down.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@ 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/core/constraint"
"github.com/pingcap/tidb/pkg/planner/core/operator/logicalop"
"github.com/pingcap/tidb/pkg/planner/util"
"github.com/pingcap/tidb/pkg/planner/util/optimizetrace"
Expand Down Expand Up @@ -48,31 +47,6 @@ func (*PPDSolver) Optimize(_ context.Context, lp base.LogicalPlan, opt *optimize
return p, planChanged, nil
}

func addSelection(p base.LogicalPlan, child base.LogicalPlan, conditions []expression.Expression, chIdx int, opt *optimizetrace.LogicalOptimizeOp) {
if len(conditions) == 0 {
p.Children()[chIdx] = child
return
}
conditions = expression.PropagateConstant(p.SCtx().GetExprCtx(), conditions)
// Return table dual when filter is constant false or null.
dual := logicalop.Conds2TableDual(child, conditions)
if dual != nil {
p.Children()[chIdx] = dual
logicalop.AppendTableDualTraceStep(child, dual, conditions, opt)
return
}

conditions = constraint.DeleteTrueExprs(p, conditions)
if len(conditions) == 0 {
p.Children()[chIdx] = child
return
}
selection := logicalop.LogicalSelection{Conditions: conditions}.Init(p.SCtx(), p.QueryBlockOffset())
selection.SetChildren(child)
p.Children()[chIdx] = selection
logicalop.AppendAddSelectionTraceStep(p, child, selection, opt)
}

// Name implements base.LogicalOptRule.<1st> interface.
func (*PPDSolver) Name() string {
return "predicate_push_down"
Expand Down
23 changes: 0 additions & 23 deletions pkg/planner/util/utilfuncp/func_pointer_misc.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,29 +28,6 @@ import (

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

// 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)

// 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.
// todo: (3) arenatlx, remove this func pointer when Task pkg is moved out of core, and
// getTaskPlanCost can be some member function usage of its family.
var GetTaskPlanCost func(t base.Task, pop *optimizetrace.PhysicalOptimizeOp) (float64, bool, error)

// AddSelection will add a selection if necessary.
// This function is util function pointer that initialized by core functionality.
// todo: (4) arenatlx, remove this func pointer when inside referred LogicalSelection is moved out of core.
var AddSelection func(p base.LogicalPlan, child base.LogicalPlan, conditions []expression.Expression,
chIdx int, opt *optimizetrace.LogicalOptimizeOp)

// PushDownTopNForBaseLogicalPlan will be called by baseLogicalPlan in logicalOp pkg. While the implementation
// of pushDownTopNForBaseLogicalPlan depends on concrete logical operators.
// todo: (5) arenatlx, Remove this util func pointer when logical operators are moved from core to logicalop.
Expand Down

0 comments on commit c46e4bc

Please sign in to comment.