Skip to content

Commit

Permalink
bindinfo: set correct default_db for captured bindings (pingcap#15367) (
Browse files Browse the repository at this point in the history
pingcap#15635)

Co-authored-by: Kenan Yao <cauchy1992@gmail.com>
  • Loading branch information
sre-bot and eurekaka authored Mar 26, 2020
1 parent 4189951 commit 4ba7ac7
Show file tree
Hide file tree
Showing 4 changed files with 81 additions and 32 deletions.
27 changes: 27 additions & 0 deletions bindinfo/bind_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -498,6 +498,33 @@ func (s *testSuite) TestCapturePlanBaseline(c *C) {
c.Assert(rows[0][1], Equals, "SELECT /*+ USE_INDEX(@`sel_1` `test`.`t` )*/ * FROM `t` WHERE `a`>10")
}

func (s *testSuite) TestCaptureBaselinesDefaultDB(c *C) {
tk := testkit.NewTestKit(c, s.store)
s.cleanBindingEnv(tk)
stmtsummary.StmtSummaryByDigestMap.Clear()
tk.MustExec(" set @@tidb_capture_plan_baselines = on")
defer func() {
tk.MustExec(" set @@tidb_capture_plan_baselines = off")
}()
tk.MustExec("use test")
tk.MustExec("drop database if exists spm")
tk.MustExec("create database spm")
tk.MustExec("create table spm.t(a int, index idx_a(a))")
c.Assert(tk.Se.Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil), IsTrue)
tk.MustExec("select * from spm.t ignore index(idx_a) where a > 10")
tk.MustExec("select * from spm.t ignore index(idx_a) where a > 10")
tk.MustExec("admin capture bindings")
rows := tk.MustQuery("show global bindings").Rows()
c.Assert(len(rows), Equals, 1)
// Default DB should be "" when all columns have explicit database name.
c.Assert(rows[0][2], Equals, "")
c.Assert(rows[0][3], Equals, "using")
tk.MustExec("use spm")
tk.MustExec("select * from spm.t where a > 10")
// Should use TableScan because of the "ignore index" binding.
c.Assert(len(tk.Se.GetSessionVars().StmtCtx.IndexNames), Equals, 0)
}

func (s *testSuite) TestUseMultiplyBindings(c *C) {
tk := testkit.NewTestKit(c, s.store)
s.cleanBindingEnv(tk)
Expand Down
6 changes: 4 additions & 2 deletions bindinfo/handle.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@ import (
driver "github.com/pingcap/tidb/types/parser_driver"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/logutil"
utilparser "github.com/pingcap/tidb/util/parser"
"github.com/pingcap/tidb/util/sqlexec"
"github.com/pingcap/tidb/util/stmtsummary"
"github.com/pingcap/tidb/util/timeutil"
Expand Down Expand Up @@ -529,7 +530,8 @@ func (h *BindHandle) CaptureBaselines() {
continue
}
normalizedSQL, digiest := parser.NormalizeDigest(sqls[i])
if r := h.GetBindRecord(digiest, normalizedSQL, schemas[i]); r != nil && r.HasUsingBinding() {
dbName := utilparser.GetDefaultDB(stmt, schemas[i])
if r := h.GetBindRecord(digiest, normalizedSQL, dbName); r != nil && r.HasUsingBinding() {
continue
}
h.sctx.Lock()
Expand Down Expand Up @@ -559,7 +561,7 @@ func (h *BindHandle) CaptureBaselines() {
Collation: collation,
}
// We don't need to pass the `sctx` because they are used to generate hints and we already filled hints in.
err = h.AddBindRecord(nil, &BindRecord{OriginalSQL: normalizedSQL, Db: schemas[i], Bindings: []Binding{binding}})
err = h.AddBindRecord(nil, &BindRecord{OriginalSQL: normalizedSQL, Db: dbName, Bindings: []Binding{binding}})
if err != nil {
logutil.BgLogger().Info("capture baseline failed", zap.String("SQL", sqls[i]), zap.Error(err))
}
Expand Down
33 changes: 3 additions & 30 deletions planner/core/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@ import (
util2 "github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/logutil"
utilparser "github.com/pingcap/tidb/util/parser"
"github.com/pingcap/tidb/util/ranger"
"github.com/pingcap/tidb/util/set"

Expand Down Expand Up @@ -558,7 +559,7 @@ func (b *PlanBuilder) buildDropBindPlan(v *ast.DropBindingStmt) (Plan, error) {
SQLBindOp: OpSQLBindDrop,
NormdOrigSQL: parser.Normalize(v.OriginSel.Text()),
IsGlobal: v.GlobalScope,
Db: getDefaultDB(b.ctx, v.OriginSel),
Db: utilparser.GetDefaultDB(v.OriginSel, b.ctx.GetSessionVars().CurrentDB),
}
if v.HintedSel != nil {
p.BindSQL = v.HintedSel.Text()
Expand All @@ -575,42 +576,14 @@ func (b *PlanBuilder) buildCreateBindPlan(v *ast.CreateBindingStmt) (Plan, error
BindSQL: v.HintedSel.Text(),
IsGlobal: v.GlobalScope,
BindStmt: v.HintedSel,
Db: getDefaultDB(b.ctx, v.OriginSel),
Db: utilparser.GetDefaultDB(v.OriginSel, b.ctx.GetSessionVars().CurrentDB),
Charset: charSet,
Collation: collation,
}
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SuperPriv, "", "", "", nil)
return p, nil
}

func getDefaultDB(ctx sessionctx.Context, sel ast.StmtNode) string {
implicitDB := &implicitDatabase{}
sel.Accept(implicitDB)
if implicitDB.hasImplicit {
return ctx.GetSessionVars().CurrentDB
}
return ""
}

type implicitDatabase struct {
hasImplicit bool
}

func (i *implicitDatabase) Enter(in ast.Node) (out ast.Node, skipChildren bool) {
switch x := in.(type) {
case *ast.TableName:
if x.Schema.L == "" {
i.hasImplicit = true
}
return in, true
}
return in, false
}

func (i *implicitDatabase) Leave(in ast.Node) (out ast.Node, ok bool) {
return in, true
}

// detectSelectAgg detects an aggregate function or GROUP BY clause.
func (b *PlanBuilder) detectSelectAgg(sel *ast.SelectStmt) bool {
if sel.GroupBy != nil {
Expand Down
47 changes: 47 additions & 0 deletions util/parser/ast.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,47 @@
// Copyright 2020 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 parser

import (
"github.com/pingcap/parser/ast"
)

// GetDefaultDB checks if all columns in the AST have explicit DBName. If not, return specified DBName.
func GetDefaultDB(sel ast.StmtNode, dbName string) string {
implicitDB := &implicitDatabase{}
sel.Accept(implicitDB)
if implicitDB.hasImplicit {
return dbName
}
return ""
}

type implicitDatabase struct {
hasImplicit bool
}

func (i *implicitDatabase) Enter(in ast.Node) (out ast.Node, skipChildren bool) {
switch x := in.(type) {
case *ast.TableName:
if x.Schema.L == "" {
i.hasImplicit = true
}
return in, true
}
return in, i.hasImplicit
}

func (i *implicitDatabase) Leave(in ast.Node) (out ast.Node, ok bool) {
return in, true
}

0 comments on commit 4ba7ac7

Please sign in to comment.