Skip to content

Commit

Permalink
planner: refactor some code of the binding package (#48959)
Browse files Browse the repository at this point in the history
ref #48875
  • Loading branch information
qw4990 committed Nov 28, 2023
1 parent e116f6c commit 68271e9
Show file tree
Hide file tree
Showing 10 changed files with 203 additions and 781 deletions.
4 changes: 2 additions & 2 deletions pkg/bindinfo/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -5,6 +5,7 @@ go_library(
srcs = [
"bind_cache.go",
"bind_record.go",
"capture.go",
"handle.go",
"session_handle.go",
"stat.go",
Expand Down Expand Up @@ -37,7 +38,6 @@ go_library(
"//pkg/util/sqlexec",
"//pkg/util/stmtsummary/v2:stmtsummary",
"//pkg/util/table-filter",
"//pkg/util/timeutil",
"@org_golang_x_exp//maps",
"@org_uber_go_zap//:zap",
],
Expand All @@ -57,7 +57,7 @@ go_test(
embed = [":bindinfo"],
flaky = True,
race = "on",
shard_count = 41,
shard_count = 40,
deps = [
"//pkg/bindinfo/internal",
"//pkg/config",
Expand Down
197 changes: 197 additions & 0 deletions pkg/bindinfo/capture.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,197 @@
// Copyright 2023 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 bindinfo

import (
"context"
"strconv"
"strings"

"github.com/pingcap/tidb/pkg/kv"
"github.com/pingcap/tidb/pkg/parser"
"github.com/pingcap/tidb/pkg/parser/ast"
"github.com/pingcap/tidb/pkg/sessionctx/stmtctx"
"github.com/pingcap/tidb/pkg/util/logutil"
utilparser "github.com/pingcap/tidb/pkg/util/parser"
"github.com/pingcap/tidb/pkg/util/sqlexec"
stmtsummaryv2 "github.com/pingcap/tidb/pkg/util/stmtsummary/v2"
tablefilter "github.com/pingcap/tidb/pkg/util/table-filter"
"go.uber.org/zap"
)

type captureFilter struct {
frequency int64
tables []tablefilter.Filter // `schema.table`
users map[string]struct{}

fail bool
currentDB string
}

func (cf *captureFilter) Enter(in ast.Node) (out ast.Node, skipChildren bool) {
if x, ok := in.(*ast.TableName); ok {
tblEntry := stmtctx.TableEntry{
DB: x.Schema.L,
Table: x.Name.L,
}
if x.Schema.L == "" {
tblEntry.DB = cf.currentDB
}
for _, tableFilter := range cf.tables {
if tableFilter.MatchTable(tblEntry.DB, tblEntry.Table) {
cf.fail = true // some filter is matched
}
}
}
return in, cf.fail
}

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

func (cf *captureFilter) isEmpty() bool {
return len(cf.tables) == 0 && len(cf.users) == 0
}

// ParseCaptureTableFilter checks whether this filter is valid and parses it.
func ParseCaptureTableFilter(tableFilter string) (f tablefilter.Filter, valid bool) {
// forbid wildcards '!' and '@' for safety,
// please see https://github.com/pingcap/tidb-tools/tree/master/pkg/table-filter for more details.
tableFilter = strings.TrimLeft(tableFilter, " \t")
if tableFilter == "" {
return nil, false
}
if tableFilter[0] == '!' || tableFilter[0] == '@' {
return nil, false
}
var err error
f, err = tablefilter.Parse([]string{tableFilter})
if err != nil {
return nil, false
}
return f, true
}

func (h *BindHandle) extractCaptureFilterFromStorage() (filter *captureFilter) {
filter = &captureFilter{
frequency: 1,
users: make(map[string]struct{}),
}
exec := h.sctx.Context.(sqlexec.RestrictedSQLExecutor)
ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnBindInfo)
// No need to acquire the session context lock for ExecRestrictedSQL, it
// uses another background session.
rows, _, err := exec.ExecRestrictedSQL(ctx, nil, `SELECT filter_type, filter_value FROM mysql.capture_plan_baselines_blacklist order by filter_type`)
if err != nil {
logutil.BgLogger().Warn("failed to load mysql.capture_plan_baselines_blacklist", zap.String("category", "sql-bind"), zap.Error(err))
return
}
for _, row := range rows {
filterTp := strings.ToLower(row.GetString(0))
valStr := strings.ToLower(row.GetString(1))
switch filterTp {
case "table":
tfilter, valid := ParseCaptureTableFilter(valStr)
if !valid {
logutil.BgLogger().Warn("capture table filter is invalid, ignore it", zap.String("category", "sql-bind"), zap.String("filter_value", valStr))
continue
}
filter.tables = append(filter.tables, tfilter)
case "user":
filter.users[valStr] = struct{}{}
case "frequency":
f, err := strconv.ParseInt(valStr, 10, 64)
if err != nil {
logutil.BgLogger().Warn("failed to parse frequency type value, ignore it", zap.String("category", "sql-bind"), zap.String("filter_value", valStr), zap.Error(err))
continue
}
if f < 1 {
logutil.BgLogger().Warn("frequency threshold is less than 1, ignore it", zap.String("category", "sql-bind"), zap.Int64("frequency", f))
continue
}
if f > filter.frequency {
filter.frequency = f
}
default:
logutil.BgLogger().Warn("unknown capture filter type, ignore it", zap.String("category", "sql-bind"), zap.String("filter_type", filterTp))
}
}
return
}

// CaptureBaselines is used to automatically capture plan baselines.
func (h *BindHandle) CaptureBaselines() {
parser4Capture := parser.New()
captureFilter := h.extractCaptureFilterFromStorage()
emptyCaptureFilter := captureFilter.isEmpty()
bindableStmts := stmtsummaryv2.GetMoreThanCntBindableStmt(captureFilter.frequency)
for _, bindableStmt := range bindableStmts {
stmt, err := parser4Capture.ParseOneStmt(bindableStmt.Query, bindableStmt.Charset, bindableStmt.Collation)
if err != nil {
logutil.BgLogger().Debug("parse SQL failed in baseline capture", zap.String("category", "sql-bind"), zap.String("SQL", bindableStmt.Query), zap.Error(err))
continue
}
if insertStmt, ok := stmt.(*ast.InsertStmt); ok && insertStmt.Select == nil {
continue
}
if !emptyCaptureFilter {
captureFilter.fail = false
captureFilter.currentDB = bindableStmt.Schema
stmt.Accept(captureFilter)
if captureFilter.fail {
continue
}

if len(captureFilter.users) > 0 {
filteredByUser := true
for user := range bindableStmt.Users {
if _, ok := captureFilter.users[user]; !ok {
filteredByUser = false // some user not in the black-list has processed this stmt
break
}
}
if filteredByUser {
continue
}
}
}
dbName := utilparser.GetDefaultDB(stmt, bindableStmt.Schema)
normalizedSQL, digest := parser.NormalizeDigest(utilparser.RestoreWithDefaultDB(stmt, dbName, bindableStmt.Query))
if r := h.GetGlobalBinding(digest.String(), normalizedSQL, dbName); r != nil && r.HasAvailableBinding() {
continue
}
bindSQL := GenerateBindSQL(context.TODO(), stmt, bindableStmt.PlanHint, true, dbName)
if bindSQL == "" {
continue
}
h.sctx.Lock()
charset, collation := h.sctx.GetSessionVars().GetCharsetInfo()
h.sctx.Unlock()
binding := Binding{
BindSQL: bindSQL,
Status: Enabled,
Charset: charset,
Collation: collation,
Source: Capture,
SQLDigest: digest.String(),
}
// We don't need to pass the `sctx` because the BindSQL has been validated already.
err = h.CreateGlobalBinding(nil, &BindRecord{OriginalSQL: normalizedSQL, Db: dbName, Bindings: []Binding{binding}})
if err != nil {
logutil.BgLogger().Debug("create bind record failed in baseline capture", zap.String("category", "sql-bind"), zap.String("SQL", bindableStmt.Query), zap.Error(err))
}
}
}
38 changes: 0 additions & 38 deletions pkg/bindinfo/capture_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -336,19 +336,6 @@ func TestBindingSource(t *testing.T) {
bind := bindData.Bindings[0]
require.Equal(t, bindinfo.Manual, bind.Source)

// Test Source for evolved sql
tk.MustExec("set @@tidb_evolve_plan_baselines=1")
tk.MustQuery("select * from t where a > 10")
bindHandle.SaveEvolveTasksToStore()
sql, sqlDigest = internal.UtilNormalizeWithDefaultDB(t, "select * from t where a > ?")
bindData = bindHandle.GetGlobalBinding(sqlDigest, sql, "test")
require.NotNil(t, bindData)
require.Equal(t, "select * from `test` . `t` where `a` > ?", bindData.OriginalSQL)
require.Len(t, bindData.Bindings, 2)
bind = bindData.Bindings[1]
require.Equal(t, bindinfo.Evolve, bind.Source)
tk.MustExec("set @@tidb_evolve_plan_baselines=0")

// Test Source for captured sqls
stmtsummary.StmtSummaryByDigestMap.Clear()
tk.MustExec("SET GLOBAL tidb_capture_plan_baselines = on")
Expand Down Expand Up @@ -492,31 +479,6 @@ func TestIssue20417(t *testing.T) {
require.Equal(t, "select * from `test` . `t` where `b` = ? and `c` = ?", rows[0][0])
require.Equal(t, "SELECT /*+ use_index(@`sel_1` `test`.`t` `idxb`), no_order_index(@`sel_1` `test`.`t` `idxb`)*/ * FROM `test`.`t` WHERE `b` = 2 AND `c` = 213124", rows[0][1])
tk.MustExec("SET GLOBAL tidb_capture_plan_baselines = off")

// Test for evolve baseline
internal.UtilCleanBindingEnv(tk, dom)
tk.MustExec("set @@tidb_evolve_plan_baselines=1")
tk.MustExec("create global binding for select * from t WHERE c=3924541 using select /*+ use_index(@sel_1 test.t idxb) */ * from t WHERE c=3924541")
rows = tk.MustQuery("show global bindings").Rows()
require.Len(t, rows, 1)
require.Equal(t, "select * from `test` . `t` where `c` = ?", rows[0][0])
require.Equal(t, "SELECT /*+ use_index(@`sel_1` `test`.`t` `idxb`)*/ * FROM `test`.`t` WHERE `c` = 3924541", rows[0][1])
tk.MustExec("select /*+ use_index(t idxc)*/ * from t where c=3924541")
require.Equal(t, "t:idxb", tk.Session().GetSessionVars().StmtCtx.IndexNames[0])
tk.MustExec("admin flush bindings")
rows = tk.MustQuery("show global bindings").Rows()
require.Len(t, rows, 2)
require.Equal(t, "select * from `test` . `t` where `c` = ?", rows[0][0])
require.Equal(t, "SELECT /*+ use_index(@`sel_1` `test`.`t` `idxc`), no_order_index(@`sel_1` `test`.`t` `idxc`)*/ * FROM `test`.`t` WHERE `c` = 3924541", rows[0][1])
require.Equal(t, "pending verify", rows[0][3])
tk.MustExec("admin evolve bindings")
rows = tk.MustQuery("show global bindings").Rows()
require.Len(t, rows, 2)
require.Equal(t, "select * from `test` . `t` where `c` = ?", rows[0][0])
require.Equal(t, "SELECT /*+ use_index(@`sel_1` `test`.`t` `idxc`), no_order_index(@`sel_1` `test`.`t` `idxc`)*/ * FROM `test`.`t` WHERE `c` = 3924541", rows[0][1])
status := rows[0][3].(string)
require.True(t, status == bindinfo.Enabled || status == bindinfo.Rejected)
tk.MustExec("set @@tidb_evolve_plan_baselines=0")
}

func TestCaptureWithZeroSlowLogThreshold(t *testing.T) {
Expand Down
Loading

0 comments on commit 68271e9

Please sign in to comment.