diff --git a/cmd/explaintest/r/black_list.result b/cmd/explaintest/r/black_list.result new file mode 100644 index 0000000000000..465490c35bfdb --- /dev/null +++ b/cmd/explaintest/r/black_list.result @@ -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; + diff --git a/cmd/explaintest/t/black_list.test b/cmd/explaintest/t/black_list.test new file mode 100644 index 0000000000000..27a41226295cf --- /dev/null +++ b/cmd/explaintest/t/black_list.test @@ -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; \ No newline at end of file diff --git a/executor/analyze_test.go b/executor/analyze_test.go index 50577ca7c50ba..ce0f579e56757 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -265,7 +265,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"+ diff --git a/executor/builder.go b/executor/builder.go index 0b4de52b007a5..d2d048616bd93 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -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: @@ -469,6 +471,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} } diff --git a/executor/opt_rule_blacklist.go b/executor/opt_rule_blacklist.go new file mode 100644 index 0000000000000..a6a4d37ec28ef --- /dev/null +++ b/executor/opt_rule_blacklist.go @@ -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 +} diff --git a/executor/reload_expr_pushdown_blacklist.go b/executor/reload_expr_pushdown_blacklist.go index e80a9ea9cf65e..90bbcae1362fa 100644 --- a/executor/reload_expr_pushdown_blacklist.go +++ b/executor/reload_expr_pushdown_blacklist.go @@ -15,7 +15,6 @@ package executor import ( "context" - "strings" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/sessionctx" @@ -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 diff --git a/executor/reload_expr_pushdown_blacklist_test.go b/executor/reload_expr_pushdown_blacklist_test.go deleted file mode 100644 index a31c444511aec..0000000000000 --- a/executor/reload_expr_pushdown_blacklist_test.go +++ /dev/null @@ -1,42 +0,0 @@ -// 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_test - -import ( - . "github.com/pingcap/check" - "github.com/pingcap/tidb/util/testkit" -) - -func (s *testSuite2) TestReloadExprPushdownBlacklist(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("create database expr_pushdown_blacklist") - tk.MustExec("use expr_pushdown_blacklist") - tk.MustExec("create table t (a int)") - tk.MustQuery("desc select * from t where a < 1").Check(testkit.Rows( - "TableReader_7 3323.33 root data:Selection_6", - "└─Selection_6 3323.33 cop lt(expr_pushdown_blacklist.t.a, 1)", - " └─TableScan_5 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo")) - - tk.MustExec("insert into mysql.expr_pushdown_blacklist values('lt')") - tk.MustQuery("desc select * from t where a < 1").Check(testkit.Rows( - "TableReader_7 3323.33 root data:Selection_6", - "└─Selection_6 3323.33 cop lt(expr_pushdown_blacklist.t.a, 1)", - " └─TableScan_5 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo")) - - tk.MustExec("admin reload expr_pushdown_blacklist") - tk.MustQuery("desc select * from t where a < 1").Check(testkit.Rows( - "Selection_5 8000.00 root lt(expr_pushdown_blacklist.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")) -} diff --git a/expression/integration_test.go b/expression/integration_test.go index 3e551f0199e2f..4b55bc284bc8e 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -4458,6 +4458,11 @@ func (s *testIntegrationSuite) TestExprPushdownBlacklist(c *C) { tk.MustQuery(`select * from mysql.expr_pushdown_blacklist`).Check(testkit.Rows()) } +func (s *testIntegrationSuite) TestOptRuleBlacklist(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustQuery(`select * from mysql.opt_rule_blacklist`).Check(testkit.Rows()) +} + func (s *testIntegrationSuite) TestIssue10804(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustQuery(`SELECT @@information_schema_stats_expiry`).Check(testkit.Rows(`86400`)) diff --git a/go.mod b/go.mod index 5950f1dc2f5c0..ba19c42dd4d89 100644 --- a/go.mod +++ b/go.mod @@ -68,7 +68,6 @@ require ( golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e golang.org/x/sys v0.0.0-20190109145017-48ac38b7c8cb 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 diff --git a/go.sum b/go.sum index 10bd3f3646065..b9ccf000166a7 100644 --- a/go.sum +++ b/go.sum @@ -277,9 +277,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= diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index c43f03bf67b50..8a458ffc90bc6 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -133,6 +133,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 diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index ab7d09117fe19..f164dbb5a92a4 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -26,6 +26,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" ) @@ -68,6 +69,7 @@ var optRuleList = []logicalOptRule{ // logicalOptRule means a logical optimizing rule, which contains decorrelate, ppd, column pruning, etc. type logicalOptRule interface { optimize(LogicalPlan) (LogicalPlan, error) + name() string } // BuildLogicalPlan used to build logical plan from ast.Node. @@ -139,7 +141,7 @@ func logicalOptimize(flag uint64, logic LogicalPlan) (LogicalPlan, error) { // 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<