Skip to content

Commit

Permalink
*: add opt_rule_blacklist in mysql tables. (#11096) (#11658)
Browse files Browse the repository at this point in the history
  • Loading branch information
lzmhhh123 authored and sre-bot committed Aug 7, 2019
1 parent 8642e9b commit eca09b1
Show file tree
Hide file tree
Showing 26 changed files with 214 additions and 53 deletions.
35 changes: 35 additions & 0 deletions cmd/explaintest/r/black_list.result
Original file line number Diff line number Diff line change
@@ -0,0 +1,35 @@
use test;
drop table if exists t;
create table t (a int);
explain select * from t where a < 1;
id count task operator info
TableReader_7 3323.33 root data:Selection_6
└─Selection_6 3323.33 cop lt(test.t.a, 1)
└─TableScan_5 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo
insert into mysql.opt_rule_blacklist values('predicate_push_down');
admin reload opt_rule_blacklist;

explain select * from t where a < 1;
id count task operator info
Selection_5 8000.00 root lt(test.t.a, 1)
└─TableReader_7 10000.00 root data:TableScan_6
└─TableScan_6 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo
delete from mysql.opt_rule_blacklist where name='predicate_push_down';
admin reload opt_rule_blacklist;

explain select * from t where a < 1;
id count task operator info
TableReader_7 3323.33 root data:Selection_6
└─Selection_6 3323.33 cop lt(test.t.a, 1)
└─TableScan_5 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo
insert into mysql.expr_pushdown_blacklist values('lt');
admin reload expr_pushdown_blacklist;

explain select * from t where a < 1;
id count task operator info
Selection_5 8000.00 root lt(test.t.a, 1)
└─TableReader_7 10000.00 root data:TableScan_6
└─TableScan_6 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo
delete from mysql.expr_pushdown_blacklist where name='lt';
admin reload expr_pushdown_blacklist;

27 changes: 27 additions & 0 deletions cmd/explaintest/t/black_list.test
Original file line number Diff line number Diff line change
@@ -0,0 +1,27 @@
use test;
drop table if exists t;
create table t (a int);

explain select * from t where a < 1;

insert into mysql.opt_rule_blacklist values('predicate_push_down');

admin reload opt_rule_blacklist;

explain select * from t where a < 1;

delete from mysql.opt_rule_blacklist where name='predicate_push_down';

admin reload opt_rule_blacklist;

explain select * from t where a < 1;

insert into mysql.expr_pushdown_blacklist values('lt');

admin reload expr_pushdown_blacklist;

explain select * from t where a < 1;

delete from mysql.expr_pushdown_blacklist where name='lt';

admin reload expr_pushdown_blacklist;
2 changes: 1 addition & 1 deletion executor/analyze_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -255,7 +255,7 @@ func (s *testSuite1) TestFastAnalyze(c *C) {
c.Assert(err, IsNil)
tableInfo := table.Meta()
tbl := dom.StatsHandle().GetTableStats(tableInfo)
c.Assert(tbl.String(), Equals, "Table:41 Count:20\n"+
c.Assert(tbl.String(), Equals, "Table:43 Count:20\n"+
"column:1 ndv:20 totColSize:0\n"+
"num: 6 lower_bound: 3 upper_bound: 15 repeats: 1\n"+
"num: 7 lower_bound: 18 upper_bound: 33 repeats: 1\n"+
Expand Down
6 changes: 6 additions & 0 deletions executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -108,6 +108,8 @@ func (b *executorBuilder) build(p plannercore.Plan) Executor {
return b.buildChecksumTable(v)
case *plannercore.ReloadExprPushdownBlacklist:
return b.buildReloadExprPushdownBlacklist(v)
case *plannercore.ReloadOptRuleBlacklist:
return b.buildReloadOptRuleBlacklist(v)
case *plannercore.AdminPlugins:
return b.buildAdminPlugins(v)
case *plannercore.DDL:
Expand Down Expand Up @@ -516,6 +518,10 @@ func (b *executorBuilder) buildReloadExprPushdownBlacklist(v *plannercore.Reload
return &ReloadExprPushdownBlacklistExec{baseExecutor{ctx: b.ctx}}
}

func (b *executorBuilder) buildReloadOptRuleBlacklist(v *plannercore.ReloadOptRuleBlacklist) Executor {
return &ReloadOptRuleBlacklistExec{baseExecutor{ctx: b.ctx}}
}

func (b *executorBuilder) buildAdminPlugins(v *plannercore.AdminPlugins) Executor {
return &AdminPluginsExec{baseExecutor: baseExecutor{ctx: b.ctx}, Action: v.Action, Plugins: v.Plugins}
}
Expand Down
50 changes: 50 additions & 0 deletions executor/opt_rule_blacklist.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,50 @@
// Copyright 2019 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,
// See the License for the specific language governing permissions and
// limitations under the License.

package executor

import (
"context"

plannercore "github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/set"
"github.com/pingcap/tidb/util/sqlexec"
)

// ReloadOptRuleBlacklistExec indicates ReloadOptRuleBlacklist executor.
type ReloadOptRuleBlacklistExec struct {
baseExecutor
}

// Next implements the Executor Next interface.
func (e *ReloadOptRuleBlacklistExec) Next(ctx context.Context, _ *chunk.Chunk) error {
return LoadOptRuleBlacklist(e.ctx)
}

// LoadOptRuleBlacklist loads the latest data from table mysql.opt_rule_blacklist.
func LoadOptRuleBlacklist(ctx sessionctx.Context) (err error) {
sql := "select HIGH_PRIORITY name from mysql.opt_rule_blacklist"
rows, _, err := ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(ctx, sql)
if err != nil {
return err
}
newDisabledLogicalRules := set.NewStringSet()
for _, row := range rows {
name := row.GetString(0)
newDisabledLogicalRules.Insert(name)
}
plannercore.DefaultDisabledLogicalRulesList.Store(newDisabledLogicalRules)
return nil
}
3 changes: 1 addition & 2 deletions executor/reload_expr_pushdown_blacklist.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@ package executor

import (
"context"
"strings"

"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/sessionctx"
Expand Down Expand Up @@ -43,7 +42,7 @@ func LoadExprPushdownBlacklist(ctx sessionctx.Context) (err error) {
newBlacklist := make(map[string]struct{})
for _, row := range rows {
name := row.GetString(0)
newBlacklist[strings.ToLower(name)] = struct{}{}
newBlacklist[name] = struct{}{}
}
expression.DefaultExprPushdownBlacklist.Store(newBlacklist)
return nil
Expand Down
42 changes: 0 additions & 42 deletions executor/reload_expr_pushdown_blacklist_test.go

This file was deleted.

3 changes: 1 addition & 2 deletions go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,7 @@ require (
github.com/pingcap/goleveldb v0.0.0-20171020122428-b9ff6c35079e
github.com/pingcap/kvproto v0.0.0-20190703131923-d9830856b531
github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596
github.com/pingcap/parser v0.0.0-20190805123000-d4d07337cee2
github.com/pingcap/parser v0.0.0-20190806084718-1a31cabbaef2
github.com/pingcap/pd v0.0.0-20190711034019-ee98bf9063e9
github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible
github.com/pingcap/tipb v0.0.0-20190428032612-535e1abaa330
Expand All @@ -70,7 +70,6 @@ require (
golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e
golang.org/x/sys v0.0.0-20190109145017-48ac38b7c8cb // indirect
golang.org/x/text v0.3.0
golang.org/x/time v0.0.0-20190308202827-9d24e82272b4 // indirect
golang.org/x/tools v0.0.0-20190130214255-bb1329dc71a0
google.golang.org/genproto v0.0.0-20190108161440-ae2f86662275 // indirect
google.golang.org/grpc v1.17.0
Expand Down
7 changes: 3 additions & 4 deletions go.sum
Original file line number Diff line number Diff line change
Expand Up @@ -167,8 +167,8 @@ github.com/pingcap/kvproto v0.0.0-20190703131923-d9830856b531/go.mod h1:QMdbTAXC
github.com/pingcap/log v0.0.0-20190214045112-b37da76f67a7/go.mod h1:xsfkWVaFVV5B8e1K9seWfyJWFrIhbtUTAD8NV1Pq3+w=
github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596 h1:t2OQTpPJnrPDGlvA+3FwJptMTt6MEPdzK1Wt99oaefQ=
github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596/go.mod h1:WpHUKhNZ18v116SvGrmjkA9CBhYmuUTKL+p8JC9ANEw=
github.com/pingcap/parser v0.0.0-20190805123000-d4d07337cee2 h1:6xsMMlH8yvLr7rt2yxOjTx8bW3y3OSAXRTEvisTxVS8=
github.com/pingcap/parser v0.0.0-20190805123000-d4d07337cee2/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA=
github.com/pingcap/parser v0.0.0-20190806084718-1a31cabbaef2 h1:wPekJBaldlYsCQa6zkahYkTctfoQLGXf4eqCdtxlYQ0=
github.com/pingcap/parser v0.0.0-20190806084718-1a31cabbaef2/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA=
github.com/pingcap/pd v0.0.0-20190711034019-ee98bf9063e9 h1:sqqiviE8oEYXJh3Aq59HO/AhxjsvcRb9ETh0ivFOHXc=
github.com/pingcap/pd v0.0.0-20190711034019-ee98bf9063e9/go.mod h1:3DlDlFT7EF64A1bmb/tulZb6wbPSagm5G4p1AlhaEDs=
github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible h1:MkWCxgZpJBgY2f4HtwWMMFzSBb3+JPzeJgF3VrXE/bU=
Expand Down Expand Up @@ -280,9 +280,8 @@ golang.org/x/sys v0.0.0-20190109145017-48ac38b7c8cb h1:1w588/yEchbPNpa9sEvOcMZYb
golang.org/x/sys v0.0.0-20190109145017-48ac38b7c8cb/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/text v0.3.0 h1:g61tztE5qeGQ89tm6NTjjM9VPIm088od1l6aSorWRWg=
golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2 h1:+DCIGbF/swA92ohVg0//6X2IVY3KZs6p9mix0ziNYJM=
golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ=
golang.org/x/time v0.0.0-20190308202827-9d24e82272b4 h1:SvFZT6jyqRaOeXpc5h/JSfZenJ2O330aBsf7JfSUXmQ=
golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ=
golang.org/x/tools v0.0.0-20180828015842-6cd1fcedba52/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
golang.org/x/tools v0.0.0-20190130214255-bb1329dc71a0 h1:iRpjPej1fPzmfoBhMFkp3HdqzF+ytPmAwiQhJGV0zGw=
golang.org/x/tools v0.0.0-20190130214255-bb1329dc71a0/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
Expand Down
5 changes: 5 additions & 0 deletions planner/core/common_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -135,6 +135,11 @@ type ReloadExprPushdownBlacklist struct {
baseSchemaProducer
}

// ReloadOptRuleBlacklist reloads the data from opt_rule_blacklist table.
type ReloadOptRuleBlacklist struct {
baseSchemaProducer
}

// AdminPluginsAction indicate action will be taken on plugins.
type AdminPluginsAction int

Expand Down
14 changes: 13 additions & 1 deletion planner/core/optimizer.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import (
"github.com/pingcap/tidb/planner/property"
"github.com/pingcap/tidb/privilege"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/util/set"
"go.uber.org/atomic"
)

Expand Down Expand Up @@ -67,6 +68,7 @@ var optRuleList = []logicalOptRule{
// logicalOptRule means a logical optimizing rule, which contains decorrelate, ppd, column pruning, etc.
type logicalOptRule interface {
optimize(context.Context, LogicalPlan) (LogicalPlan, error)
name() string
}

// BuildLogicalPlan used to build logical plan from ast.Node.
Expand Down Expand Up @@ -127,7 +129,7 @@ func logicalOptimize(ctx context.Context, flag uint64, logic LogicalPlan) (Logic
// 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 {
if flag&(1<<uint(i)) == 0 || isLogicalRuleDisabled(rule) {
continue
}
logic, err = rule.optimize(ctx, logic)
Expand All @@ -138,6 +140,11 @@ func logicalOptimize(ctx context.Context, flag uint64, logic LogicalPlan) (Logic
return logic, err
}

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

func physicalOptimize(logic LogicalPlan) (PhysicalPlan, error) {
if _, err := logic.recursiveDeriveStats(); err != nil {
return nil, err
Expand Down Expand Up @@ -174,6 +181,11 @@ func existsCartesianProduct(p LogicalPlan) bool {
return false
}

// DefaultDisabledLogicalRulesList indicates the logical rules which should be banned.
var DefaultDisabledLogicalRulesList *atomic.Value

func init() {
expression.EvalAstExpr = evalAstExpr
DefaultDisabledLogicalRulesList = new(atomic.Value)
DefaultDisabledLogicalRulesList.Store(set.NewStringSet())
}
2 changes: 2 additions & 0 deletions planner/core/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -638,6 +638,8 @@ func (b *PlanBuilder) buildAdmin(ctx context.Context, as *ast.AdminStmt) (Plan,
ret = p
case ast.AdminReloadExprPushdownBlacklist:
return &ReloadExprPushdownBlacklist{}, nil
case ast.AdminReloadOptRuleBlacklist:
return &ReloadOptRuleBlacklist{}, nil
case ast.AdminPluginEnable:
return &AdminPlugins{Action: Enable, Plugins: as.Plugins}, nil
case ast.AdminPluginDisable:
Expand Down
4 changes: 4 additions & 0 deletions planner/core/rule_aggregation_elimination.go
Original file line number Diff line number Diff line change
Expand Up @@ -156,3 +156,7 @@ func (a *aggregationEliminator) optimize(ctx context.Context, p LogicalPlan) (Lo
}
return p, nil
}

func (*aggregationEliminator) name() string {
return "aggregation_eliminate"
}
4 changes: 4 additions & 0 deletions planner/core/rule_aggregation_push_down.go
Original file line number Diff line number Diff line change
Expand Up @@ -405,3 +405,7 @@ func (a *aggregationPushDownSolver) aggPushDown(p LogicalPlan) (_ LogicalPlan, e
p.SetChildren(newChildren...)
return p, nil
}

func (*aggregationPushDownSolver) name() string {
return "aggregation_push_down"
}
4 changes: 4 additions & 0 deletions planner/core/rule_build_key_info.go
Original file line number Diff line number Diff line change
Expand Up @@ -220,3 +220,7 @@ func (ds *DataSource) buildKeyInfo() {
}
}
}

func (*buildKeySolver) name() string {
return "build_keys"
}
4 changes: 4 additions & 0 deletions planner/core/rule_column_pruning.go
Original file line number Diff line number Diff line change
Expand Up @@ -414,3 +414,7 @@ func (p *LogicalWindow) extractUsedCols(parentUsedCols []*expression.Column) []*
}
return parentUsedCols
}

func (*columnPruner) name() string {
return "column_prune"
}
4 changes: 4 additions & 0 deletions planner/core/rule_decorrelate.go
Original file line number Diff line number Diff line change
Expand Up @@ -256,3 +256,7 @@ func (s *decorrelateSolver) optimize(ctx context.Context, p LogicalPlan) (Logica
p.SetChildren(newChildren...)
return p, nil
}

func (*decorrelateSolver) name() string {
return "decorrelate"
}
4 changes: 4 additions & 0 deletions planner/core/rule_eliminate_projection.go
Original file line number Diff line number Diff line change
Expand Up @@ -222,3 +222,7 @@ func (p *LogicalWindow) replaceExprColumns(replace map[string]*expression.Column
resolveColumnAndReplace(item.Col, replace)
}
}

func (*projectionEliminater) name() string {
return "projection_eliminate"
}
4 changes: 4 additions & 0 deletions planner/core/rule_join_elimination.go
Original file line number Diff line number Diff line change
Expand Up @@ -237,3 +237,7 @@ func (o *outerJoinEliminator) doOptimize(p LogicalPlan, aggCols []*expression.Co
func (o *outerJoinEliminator) optimize(ctx context.Context, p LogicalPlan) (LogicalPlan, error) {
return o.doOptimize(p, nil, nil)
}

func (*outerJoinEliminator) name() string {
return "outer_join_eliminate"
}
4 changes: 4 additions & 0 deletions planner/core/rule_join_reorder.go
Original file line number Diff line number Diff line change
Expand Up @@ -168,3 +168,7 @@ func (s *baseSingleGroupJoinOrderSolver) newJoinWithEdges(lChild, rChild Logical
func (s *baseSingleGroupJoinOrderSolver) calcJoinCumCost(join LogicalPlan, lNode, rNode *jrNode) float64 {
return join.statsInfo().RowCount + lNode.cumCost + rNode.cumCost
}

func (*joinReOrderSolver) name() string {
return "join_reorder"
}
4 changes: 4 additions & 0 deletions planner/core/rule_max_min_eliminate.go
Original file line number Diff line number Diff line change
Expand Up @@ -84,3 +84,7 @@ func (a *maxMinEliminator) eliminateMaxMin(p LogicalPlan) {
a.eliminateMaxMin(child)
}
}

func (*maxMinEliminator) name() string {
return "max_min_eliminate"
}
4 changes: 4 additions & 0 deletions planner/core/rule_partition_processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -206,3 +206,7 @@ func (s *partitionProcessor) findByName(partitionNames []model.CIStr, partitionN
}
return false
}

func (*partitionProcessor) name() string {
return "partition_processor"
}
Loading

0 comments on commit eca09b1

Please sign in to comment.