Skip to content

Commit

Permalink
planner: classify logical optimizing rule interface and files. (#55226)
Browse files Browse the repository at this point in the history
ref #51664, ref #52714
  • Loading branch information
AilinKid authored Aug 7, 2024
1 parent 395c6e7 commit 3117d3f
Show file tree
Hide file tree
Showing 36 changed files with 415 additions and 254 deletions.
2 changes: 1 addition & 1 deletion pkg/planner/core/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -68,7 +68,6 @@ go_library(
"rule_aggregation_elimination.go",
"rule_aggregation_push_down.go",
"rule_aggregation_skew_rewrite.go",
"rule_build_key_info.go",
"rule_collect_plan_stats.go",
"rule_column_pruning.go",
"rule_constant_propagation.go",
Expand Down Expand Up @@ -140,6 +139,7 @@ go_library(
"//pkg/planner/core/metrics",
"//pkg/planner/core/operator/baseimpl",
"//pkg/planner/core/operator/logicalop",
"//pkg/planner/core/rule",
"//pkg/planner/core/rule/util",
"//pkg/planner/funcdep",
"//pkg/planner/property",
Expand Down
1 change: 1 addition & 0 deletions pkg/planner/core/base/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,7 @@ go_library(
"doc.go",
"misc_base.go",
"plan_base.go",
"rule_base.go",
"task_base.go",
],
importpath = "github.com/pingcap/tidb/pkg/planner/core/base",
Expand Down
33 changes: 33 additions & 0 deletions pkg/planner/core/base/rule_base.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
// 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

import (
"context"

"github.com/pingcap/tidb/pkg/planner/util/optimizetrace"
)

// LogicalOptRule means a logical optimizing rule, which contains de-correlate, ppd, column pruning, etc.
type LogicalOptRule interface {
// Optimize return parameters:
// 1. base.LogicalPlan: The optimized base.LogicalPlan after rule is applied
// 2. bool: Used to judge whether the plan is changed or not by logical rule.
// If the plan is changed, it will return true.
// The default value is false. It means that no interaction rule will be triggered.
// 3. error: If there is error during the rule optimizer, it will be thrown
Optimize(context.Context, LogicalPlan, *optimizetrace.LogicalOptimizeOp) (LogicalPlan, bool, error)
Name() string
}
3 changes: 2 additions & 1 deletion pkg/planner/core/logical_datasource.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ import (
"github.com/pingcap/tidb/pkg/planner/core/base"
"github.com/pingcap/tidb/pkg/planner/core/cost"
"github.com/pingcap/tidb/pkg/planner/core/operator/logicalop"
ruleutil "github.com/pingcap/tidb/pkg/planner/core/rule/util"
fd "github.com/pingcap/tidb/pkg/planner/funcdep"
"github.com/pingcap/tidb/pkg/planner/property"
"github.com/pingcap/tidb/pkg/planner/util"
Expand Down Expand Up @@ -257,7 +258,7 @@ func (ds *DataSource) BuildKeyInfo(selfSchema *expression.Schema, _ []*expressio
} else if index.State != model.StatePublic {
continue
}
if uniqueKey, newKey := checkIndexCanBeKey(index, ds.Columns, selfSchema); newKey != nil {
if uniqueKey, newKey := ruleutil.CheckIndexCanBeKey(index, ds.Columns, selfSchema); newKey != nil {
selfSchema.Keys = append(selfSchema.Keys, newKey)
} else if uniqueKey != nil {
selfSchema.UniqueKeys = append(selfSchema.UniqueKeys, uniqueKey)
Expand Down
3 changes: 2 additions & 1 deletion pkg/planner/core/logical_index_scan.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (
"github.com/pingcap/tidb/pkg/parser/mysql"
"github.com/pingcap/tidb/pkg/planner/core/base"
"github.com/pingcap/tidb/pkg/planner/core/operator/logicalop"
ruleutil "github.com/pingcap/tidb/pkg/planner/core/rule/util"
"github.com/pingcap/tidb/pkg/planner/property"
"github.com/pingcap/tidb/pkg/types"
"github.com/pingcap/tidb/pkg/util/plancodec"
Expand Down Expand Up @@ -98,7 +99,7 @@ func (is *LogicalIndexScan) BuildKeyInfo(selfSchema *expression.Schema, _ []*exp
if path.IsTablePath() {
continue
}
if uniqueKey, newKey := checkIndexCanBeKey(path.Index, is.Columns, selfSchema); newKey != nil {
if uniqueKey, newKey := ruleutil.CheckIndexCanBeKey(path.Index, is.Columns, selfSchema); newKey != nil {
selfSchema.Keys = append(selfSchema.Keys, newKey)
} else if uniqueKey != nil {
selfSchema.UniqueKeys = append(selfSchema.UniqueKeys, uniqueKey)
Expand Down
2 changes: 1 addition & 1 deletion pkg/planner/core/logical_join.go
Original file line number Diff line number Diff line change
Expand Up @@ -288,7 +288,7 @@ func (p *LogicalJoin) PredicatePushDown(predicates []expression.Expression, opt
utilfuncp.AddSelection(p, lCh, leftRet, 0, opt)
utilfuncp.AddSelection(p, rCh, rightRet, 1, opt)
p.updateEQCond()
buildKeyInfo(p)
ruleutil.BuildKeyInfoPortal(p)
return ret, p.Self()
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/planner/core/logical_selection.go
Original file line number Diff line number Diff line change
Expand Up @@ -152,7 +152,7 @@ func (p *LogicalSelection) BuildKeyInfo(selfSchema *expression.Schema, childSche
}
}
}
p.SetMaxOneRow(checkMaxOneRowCond(eqCols, childSchema[0]))
p.SetMaxOneRow(ruleutil.CheckMaxOneRowCond(eqCols, childSchema[0]))
}

// PushDownTopN inherits BaseLogicalPlan.<5th> implementation.
Expand Down
82 changes: 35 additions & 47 deletions pkg/planner/core/optimizer.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@ import (
"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/core/rule"
"github.com/pingcap/tidb/pkg/planner/property"
"github.com/pingcap/tidb/pkg/planner/util/debugtrace"
"github.com/pingcap/tidb/pkg/planner/util/optimizetrace"
Expand Down Expand Up @@ -96,53 +97,40 @@ const (
flagResolveExpand
)

var optRuleList = []logicalOptRule{
&gcSubstituter{},
&columnPruner{},
&resultReorder{},
&buildKeySolver{},
&decorrelateSolver{},
&semiJoinRewriter{},
&aggregationEliminator{},
&skewDistinctAggRewriter{},
&projectionEliminator{},
&maxMinEliminator{},
&constantPropagationSolver{},
&convertOuterToInnerJoin{},
&ppdSolver{},
&outerJoinEliminator{},
&partitionProcessor{},
&collectPredicateColumnsPoint{},
&aggregationPushDownSolver{},
&deriveTopNFromWindow{},
&predicateSimplification{},
&pushDownTopNOptimizer{},
&syncWaitStatsLoadPoint{},
&joinReOrderSolver{},
&columnPruner{}, // column pruning again at last, note it will mess up the results of buildKeySolver
&pushDownSequenceSolver{},
&resolveExpand{},
var optRuleList = []base.LogicalOptRule{
&GcSubstituter{},
&ColumnPruner{},
&ResultReorder{},
&rule.BuildKeySolver{},
&DecorrelateSolver{},
&SemiJoinRewriter{},
&AggregationEliminator{},
&SkewDistinctAggRewriter{},
&ProjectionEliminator{},
&MaxMinEliminator{},
&ConstantPropagationSolver{},
&ConvertOuterToInnerJoin{},
&PPDSolver{},
&OuterJoinEliminator{},
&PartitionProcessor{},
&CollectPredicateColumnsPoint{},
&AggregationPushDownSolver{},
&DeriveTopNFromWindow{},
&PredicateSimplification{},
&PushDownTopNOptimizer{},
&SyncWaitStatsLoadPoint{},
&JoinReOrderSolver{},
&ColumnPruner{}, // column pruning again at last, note it will mess up the results of buildKeySolver
&PushDownSequenceSolver{},
&ResolveExpand{},
}

// Interaction Rule List
/* The interaction rule will be trigger when it satisfies following conditions:
1. The related rule has been trigger and changed the plan
2. The interaction rule is enabled
*/
var optInteractionRuleList = map[logicalOptRule]logicalOptRule{}

// logicalOptRule means a logical optimizing rule, which contains decorrelate, ppd, column pruning, etc.
type logicalOptRule interface {
/* Return Parameters:
1. base.LogicalPlan: The optimized base.LogicalPlan after rule is applied
2. bool: Used to judge whether the plan is changed or not by logical rule.
If the plan is changed, it will return true.
The default value is false. It means that no interaction rule will be triggered.
3. error: If there is error during the rule optimizer, it will be thrown
*/
optimize(context.Context, base.LogicalPlan, *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, bool, error)
name() string
}
var optInteractionRuleList = map[base.LogicalOptRule]base.LogicalOptRule{}

// 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) (base.Plan, error) {
Expand Down Expand Up @@ -999,17 +987,17 @@ func logicalOptimize(ctx context.Context, flag uint64, logic base.LogicalPlan) (
}()
}
var err error
var againRuleList []logicalOptRule
var againRuleList []base.LogicalOptRule
for i, rule := range optRuleList {
// The order of flags is same as the order of optRule in the list.
// We use a bitmask to record which opt rules should be used. If the i-th bit is 1, it means we should
// apply i-th optimizing rule.
if flag&(1<<uint(i)) == 0 || isLogicalRuleDisabled(rule) {
continue
}
opt.AppendBeforeRuleOptimize(i, rule.name(), logic.BuildPlanTrace)
opt.AppendBeforeRuleOptimize(i, rule.Name(), logic.BuildPlanTrace)
var planChanged bool
logic, planChanged, err = rule.optimize(ctx, logic, opt)
logic, planChanged, err = rule.Optimize(ctx, logic, opt)
if err != nil {
return nil, err
}
Expand All @@ -1022,8 +1010,8 @@ func logicalOptimize(ctx context.Context, flag uint64, logic base.LogicalPlan) (

// Trigger the interaction rule
for i, rule := range againRuleList {
opt.AppendBeforeRuleOptimize(i, rule.name(), logic.BuildPlanTrace)
logic, _, err = rule.optimize(ctx, logic, opt)
opt.AppendBeforeRuleOptimize(i, rule.Name(), logic.BuildPlanTrace)
logic, _, err = rule.Optimize(ctx, logic, opt)
if err != nil {
return nil, err
}
Expand All @@ -1033,8 +1021,8 @@ func logicalOptimize(ctx context.Context, flag uint64, logic base.LogicalPlan) (
return logic, err
}

func isLogicalRuleDisabled(r logicalOptRule) bool {
disabled := DefaultDisabledLogicalRulesList.Load().(set.StringSet).Exist(r.name())
func isLogicalRuleDisabled(r base.LogicalOptRule) bool {
disabled := DefaultDisabledLogicalRulesList.Load().(set.StringSet).Exist(r.Name())
return disabled
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/planner/core/partition_prune.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@ import (
// idx in the partition definitions array, use pi.Definitions[idx] to get the partition ID
func PartitionPruning(ctx base.PlanContext, tbl table.PartitionedTable, conds []expression.Expression, partitionNames []model.CIStr,
columns []*expression.Column, names types.NameSlice) ([]int, error) {
s := partitionProcessor{}
s := PartitionProcessor{}
pi := tbl.Meta().Partition
switch pi.Type {
case model.PartitionTypeHash, model.PartitionTypeKey:
Expand Down
17 changes: 17 additions & 0 deletions pkg/planner/core/rule/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -0,0 +1,17 @@
load("@io_bazel_rules_go//go:def.bzl", "go_library")

go_library(
name = "rule",
srcs = [
"rule_build_key_info.go",
"rule_init.go",
],
importpath = "github.com/pingcap/tidb/pkg/planner/core/rule",
visibility = ["//visibility:public"],
deps = [
"//pkg/expression",
"//pkg/planner/core/base",
"//pkg/planner/core/rule/util",
"//pkg/planner/util/optimizetrace",
],
)
54 changes: 54 additions & 0 deletions pkg/planner/core/rule/rule_build_key_info.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,54 @@
// 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 rule

import (
"context"

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

// BuildKeySolver is used to build key info for logical plan.
type BuildKeySolver struct{}

// *************************** start implementation of LogicalOptRule interface ***************************

// Name implements base.LogicalOptRule.<0th> interface.
func (*BuildKeySolver) Name() string {
return "build_keys"
}

// Optimize implements base.LogicalOptRule.<1st> interface.
func (*BuildKeySolver) Optimize(_ context.Context, p base.LogicalPlan, _ *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, bool, error) {
planChanged := false
buildKeyInfo(p)
return p, planChanged, nil
}

// **************************** end implementation of LogicalOptRule interface ****************************

// buildKeyInfo recursively calls base.LogicalPlan's BuildKeyInfo method.
func buildKeyInfo(lp base.LogicalPlan) {
for _, child := range lp.Children() {
buildKeyInfo(child)
}
childSchema := make([]*expression.Schema, len(lp.Children()))
for i, child := range lp.Children() {
childSchema[i] = child.Schema()
}
lp.BuildKeyInfo(lp.Schema(), childSchema)
}
26 changes: 26 additions & 0 deletions pkg/planner/core/rule/rule_init.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,26 @@
// 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 rule

import "github.com/pingcap/tidb/pkg/planner/core/rule/util"

// rule/pkg should rely on operator/pkg to do type check and dig in and out,
// rule/util doesn't have to rely on rule/pkg, but it can be put with rule
// handling logic, and be referenced by operator/pkg.
// the core usage only care and call about the rule/pkg and operator/pkg.

func init() {
util.BuildKeyInfoPortal = buildKeyInfo
}
12 changes: 10 additions & 2 deletions pkg/planner/core/rule/util/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -2,8 +2,16 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library")

go_library(
name = "util",
srcs = ["misc.go"],
srcs = [
"build_key_info_misc.go",
"misc.go",
],
importpath = "github.com/pingcap/tidb/pkg/planner/core/rule/util",
visibility = ["//visibility:public"],
deps = ["//pkg/expression"],
deps = [
"//pkg/expression",
"//pkg/parser/model",
"//pkg/parser/mysql",
"//pkg/planner/core/base",
],
)
Loading

0 comments on commit 3117d3f

Please sign in to comment.