Skip to content

Commit

Permalink
*: add a blacklist to disallow pushing down specific expressions (#10688
Browse files Browse the repository at this point in the history
)
  • Loading branch information
XuHuaiyu authored and qw4990 committed Jun 12, 2019
1 parent 3f4f3d0 commit cd8c4e6
Show file tree
Hide file tree
Showing 12 changed files with 147 additions and 8 deletions.
2 changes: 1 addition & 1 deletion executor/analyze_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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:39 Count:3000\n"+
c.Assert(tbl.String(), Equals, "Table:41 Count:3000\n"+
"column:1 ndv:3000 totColSize:0\n"+
"num: 603 lower_bound: 0 upper_bound: 658 repeats: 1\n"+
"num: 603 lower_bound: 663 upper_bound: 1248 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 @@ -106,6 +106,8 @@ func (b *executorBuilder) build(p plannercore.Plan) Executor {
return b.buildCheckIndexRange(v)
case *plannercore.ChecksumTable:
return b.buildChecksumTable(v)
case *plannercore.ReloadExprPushdownBlacklist:
return b.buildReloadExprPushdownBlacklist(v)
case *plannercore.DDL:
return b.buildDDL(v)
case *plannercore.Deallocate:
Expand Down Expand Up @@ -461,6 +463,10 @@ func (b *executorBuilder) buildChecksumTable(v *plannercore.ChecksumTable) Execu
return e
}

func (b *executorBuilder) buildReloadExprPushdownBlacklist(v *plannercore.ReloadExprPushdownBlacklist) Executor {
return &ReloadExprPushdownBlacklistExec{baseExecutor{ctx: b.ctx}}
}

func (b *executorBuilder) buildDeallocate(v *plannercore.Deallocate) Executor {
base := newBaseExecutor(b.ctx, nil, v.ExplainID())
base.initCap = chunk.ZeroCapacity
Expand Down
50 changes: 50 additions & 0 deletions executor/reload_expr_pushdown_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"
"strings"

"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/sqlexec"
)

// ReloadExprPushdownBlacklistExec indicates ReloadExprPushdownBlacklist executor.
type ReloadExprPushdownBlacklistExec struct {
baseExecutor
}

// Next implements the Executor Next interface.
func (e *ReloadExprPushdownBlacklistExec) Next(ctx context.Context, _ *chunk.RecordBatch) error {
return LoadExprPushdownBlacklist(e.ctx)
}

// LoadExprPushdownBlacklist loads the latest data from table mysql.expr_pushdown_blacklist.
func LoadExprPushdownBlacklist(ctx sessionctx.Context) (err error) {
sql := "select HIGH_PRIORITY name from mysql.expr_pushdown_blacklist"
rows, _, err := ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(ctx, sql)
if err != nil {
return err
}
newBlacklist := make(map[string]struct{})
for _, row := range rows {
name := row.GetString(0)
newBlacklist[strings.ToLower(name)] = struct{}{}
}
expression.DefaultExprPushdownBlacklist.Store(newBlacklist)
return nil
}
42 changes: 42 additions & 0 deletions executor/reload_expr_pushdown_blacklist_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,42 @@
// 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"))
}
13 changes: 11 additions & 2 deletions expression/expr_to_pb.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ package expression

import (
"context"
"sync/atomic"

"github.com/pingcap/parser/ast"
"github.com/pingcap/parser/charset"
Expand Down Expand Up @@ -321,8 +322,16 @@ func (pc PbConverter) canFuncBePushed(sf *ScalarFunction) bool {

// date functions.
ast.DateFormat:

return true
_, disallowPushdown := DefaultExprPushdownBlacklist.Load().(map[string]struct{})[sf.FuncName.L]
return true && !disallowPushdown
}
return false
}

// DefaultExprPushdownBlacklist indicates the expressions which can not be pushed down to TiKV.
var DefaultExprPushdownBlacklist *atomic.Value

func init() {
DefaultExprPushdownBlacklist = new(atomic.Value)
DefaultExprPushdownBlacklist.Store(make(map[string]struct{}))
}
5 changes: 5 additions & 0 deletions expression/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -4398,3 +4398,8 @@ func (s *testIntegrationSuite) TestIssue10181(c *C) {
tk.MustExec(`insert into t values(9223372036854775807), (18446744073709551615)`)
tk.MustQuery(`select * from t where a > 9223372036854775807-0.5 order by a`).Check(testkit.Rows(`9223372036854775807`, `18446744073709551615`))
}

func (s *testIntegrationSuite) TestExprPushdownBlacklist(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustQuery(`select * from mysql.expr_pushdown_blacklist`).Check(testkit.Rows())
}
2 changes: 1 addition & 1 deletion go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -54,7 +54,7 @@ require (
github.com/pingcap/goleveldb v0.0.0-20171020122428-b9ff6c35079e
github.com/pingcap/kvproto v0.0.0-20190528074401-b942b3f4108f
github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596
github.com/pingcap/parser v0.0.0-20190611090107-2bb2c0e8f340
github.com/pingcap/parser v0.0.0-20190612052718-3b36f86d9b7b
github.com/pingcap/pd v2.1.11+incompatible
github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible
github.com/pingcap/tipb v0.0.0-20190428032612-535e1abaa330
Expand Down
5 changes: 2 additions & 3 deletions go.sum
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,6 @@ github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973 h1:xJ4a3vCFaGF/jqvzLM
github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q=
github.com/blacktear23/go-proxyprotocol v0.0.0-20180807104634-af7a81e8dd0d h1:rQlvB2AYWme2bIB18r/SipGiMEVJYE9U0z+MGoU/LtQ=
github.com/blacktear23/go-proxyprotocol v0.0.0-20180807104634-af7a81e8dd0d/go.mod h1:VKt7CNAQxpFpSDz3sXyj9hY/GbVsQCr0sB3w59nE7lU=
github.com/client9/misspell v0.3.4 h1:ta993UF76GwbvJcIo3Y68y/M3WxlpEHPWIGDkJYwzJI=
github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw=
github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd h1:qMd81Ts1T2OTKmB4acZcyKaMtRnY5Y44NuXGX2GFJ1w=
github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd/go.mod h1:sE/e/2PUdi/liOCUjSTXgM1o87ZssimdTWN964YiIeI=
Expand Down Expand Up @@ -133,8 +132,8 @@ github.com/pingcap/kvproto v0.0.0-20190528074401-b942b3f4108f h1:EXZvZmZl+n4PGSR
github.com/pingcap/kvproto v0.0.0-20190528074401-b942b3f4108f/go.mod h1:QMdbTAXCHzzygQzqcG9uVUgU2fKeSN1GmfMiykdSzzY=
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-20190611090107-2bb2c0e8f340 h1:WZGc0+xpKCnjuVdMX9YW7763/HO0Ey4Dn9nEbUfq5bc=
github.com/pingcap/parser v0.0.0-20190611090107-2bb2c0e8f340/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA=
github.com/pingcap/parser v0.0.0-20190612052718-3b36f86d9b7b h1:4/+CIoFd4AMLZbjDpqhoa9IByT/lVcg+13/W/UgNVXM=
github.com/pingcap/parser v0.0.0-20190612052718-3b36f86d9b7b/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA=
github.com/pingcap/pd v2.1.11+incompatible h1:LHn92NDzkkpivZj+hyyuXIoSdyMMQbLRqQZg8ZQcz6o=
github.com/pingcap/pd v2.1.11+incompatible/go.mod h1:nD3+EoYes4+aNNODO99ES59V83MZSI+dFbhyr667a0E=
github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible h1:MkWCxgZpJBgY2f4HtwWMMFzSBb3+JPzeJgF3VrXE/bU=
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 @@ -128,6 +128,11 @@ type CancelDDLJobs struct {
JobIDs []int64
}

// ReloadExprPushdownBlacklist reloads the data from expr_pushdown_blacklist table.
type ReloadExprPushdownBlacklist struct {
baseSchemaProducer
}

// Change represents a change plan.
type Change struct {
baseSchemaProducer
Expand Down
2 changes: 2 additions & 0 deletions planner/core/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -668,6 +668,8 @@ func (b *PlanBuilder) buildAdmin(as *ast.AdminStmt) (Plan, error) {
p := &ShowSlow{ShowSlow: as.ShowSlow}
p.SetSchema(buildShowSlowSchema())
ret = p
case ast.AdminReloadExprPushdownBlacklist:
return &ReloadExprPushdownBlacklist{}, nil
default:
return nil, ErrUnsupportedType.GenWithStack("Unsupported ast.AdminStmt(%T) for buildAdmin", as)
}
Expand Down
16 changes: 16 additions & 0 deletions session/bootstrap.go
Original file line number Diff line number Diff line change
Expand Up @@ -259,6 +259,11 @@ const (
count bigint(64) UNSIGNED NOT NULL,
index tbl(table_id, is_index, hist_id)
);`

// CreateExprPushdownBlacklist stores the expressions which are not allowed to be pushed down.
CreateExprPushdownBlacklist = `CREATE TABLE IF NOT EXISTS mysql.expr_pushdown_blacklist (
name char(100) NOT NULL
);`
)

// bootstrap initiates system DB for a store.
Expand Down Expand Up @@ -336,6 +341,7 @@ const (
version30 = 30
version31 = 31
version32 = 32
version33 = 33
)

func checkBootstrapped(s Session) (bool, error) {
Expand Down Expand Up @@ -523,6 +529,10 @@ func upgrade(s Session) {
upgradeToVer29(s)
}

if ver < version33 {
upgradeToVer33(s)
}

updateBootstrapVer(s)
_, err = s.Execute(context.Background(), "COMMIT")

Expand Down Expand Up @@ -825,6 +835,10 @@ func upgradeToVer32(s Session) {
doReentrantDDL(s, "ALTER TABLE mysql.tables_priv MODIFY table_priv SET('Select','Insert','Update','Delete','Create','Drop','Grant', 'Index', 'Alter', 'Create View', 'Show View', 'Trigger', 'References')")
}

func upgradeToVer33(s Session) {
doReentrantDDL(s, CreateExprPushdownBlacklist)
}

// updateBootstrapVer updates bootstrap version variable in mysql.TiDB table.
func updateBootstrapVer(s Session) {
// Update bootstrap version.
Expand Down Expand Up @@ -883,6 +897,8 @@ func doDDLWorks(s Session) {
mustExecute(s, CreateBindInfoTable)
// Create stats_topn_store table.
mustExecute(s, CreateStatsTopNTable)
// Create expr_pushdown_blacklist table.
mustExecute(s, CreateExprPushdownBlacklist)
}

// doDMLWorks executes DML statements in bootstrap stage.
Expand Down
7 changes: 6 additions & 1 deletion session/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -1463,6 +1463,11 @@ func BootstrapSession(store kv.Storage) (*domain.Domain, error) {
}
}

err = executor.LoadExprPushdownBlacklist(se)
if err != nil {
return nil, err
}

se1, err := createSession(store)
if err != nil {
return nil, err
Expand Down Expand Up @@ -1567,7 +1572,7 @@ func createSessionWithDomain(store kv.Storage, dom *domain.Domain) (*session, er

const (
notBootstrapped = 0
currentBootstrapVersion = 32
currentBootstrapVersion = 33
)

func getStoreBootstrapVersion(store kv.Storage) int64 {
Expand Down

0 comments on commit cd8c4e6

Please sign in to comment.