Skip to content

Commit

Permalink
bindinfo: refacter the matcher (#50362)
Browse files Browse the repository at this point in the history
ref #48875
  • Loading branch information
hawkingrei authored Jan 15, 2024
1 parent 715fc0b commit d68acd6
Show file tree
Hide file tree
Showing 13 changed files with 176 additions and 111 deletions.
2 changes: 2 additions & 0 deletions pkg/bindinfo/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ go_library(
importpath = "github.com/pingcap/tidb/pkg/bindinfo",
visibility = ["//visibility:public"],
deps = [
"//pkg/bindinfo/norm",
"//pkg/kv",
"//pkg/metrics",
"//pkg/parser",
Expand Down Expand Up @@ -63,6 +64,7 @@ go_test(
shard_count = 50,
deps = [
"//pkg/bindinfo/internal",
"//pkg/bindinfo/norm",
"//pkg/config",
"//pkg/domain",
"//pkg/metrics",
Expand Down
90 changes: 2 additions & 88 deletions pkg/bindinfo/binding_match.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,15 +15,13 @@
package bindinfo

import (
"strings"
"sync"

"github.com/pingcap/tidb/pkg/bindinfo/norm"
"github.com/pingcap/tidb/pkg/metrics"
"github.com/pingcap/tidb/pkg/parser"
"github.com/pingcap/tidb/pkg/parser/ast"
"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/util/hint"
utilparser "github.com/pingcap/tidb/pkg/util/parser"
)

var (
Expand Down Expand Up @@ -74,7 +72,7 @@ func getBindRecord(sctx sessionctx.Context, stmtNode ast.StmtNode, info *Binding
var fuzzyDigest string
var tableNames []*ast.TableName
if info == nil || info.TableNames == nil || info.FuzzyDigest == "" {
_, fuzzyDigest = NormalizeStmtForFuzzyBinding(stmtNode)
_, fuzzyDigest = norm.NormalizeStmtForBinding(stmtNode, norm.WithFuzz(true))
tableNames = CollectTableNames(stmtNode)
if info != nil {
info.FuzzyDigest = fuzzyDigest
Expand All @@ -99,90 +97,6 @@ func getBindRecord(sctx sessionctx.Context, stmtNode ast.StmtNode, info *Binding
return nil, ""
}

func eraseLastSemicolon(stmt ast.StmtNode) {
sql := stmt.Text()
if len(sql) > 0 && sql[len(sql)-1] == ';' {
stmt.SetText(nil, sql[:len(sql)-1])
}
}

// NormalizeStmtForBinding normalizes a statement for binding.
// Schema names will be completed automatically: `select * from t` --> `select * from db . t`.
func NormalizeStmtForBinding(stmtNode ast.StmtNode, specifiedDB string) (normalizedStmt, exactSQLDigest string) {
return normalizeStmt(stmtNode, specifiedDB, false)
}

// NormalizeStmtForFuzzyBinding normalizes a statement for fuzzy matching.
// Schema names will be eliminated automatically: `select * from db . t` --> `select * from t`.
func NormalizeStmtForFuzzyBinding(stmtNode ast.StmtNode) (normalizedStmt, fuzzySQLDigest string) {
return normalizeStmt(stmtNode, "", true)
}

// NormalizeStmtForBinding normalizes a statement for binding.
// This function skips Explain automatically, and literals in in-lists will be normalized as '...'.
// For normal bindings, DB name will be completed automatically:
//
// e.g. `select * from t where a in (1, 2, 3)` --> `select * from test.t where a in (...)`
func normalizeStmt(stmtNode ast.StmtNode, specifiedDB string, fuzzy bool) (normalizedStmt, sqlDigest string) {
normalize := func(n ast.StmtNode) (normalizedStmt, sqlDigest string) {
eraseLastSemicolon(n)
var digest *parser.Digest
var normalizedSQL string
if !fuzzy {
normalizedSQL = utilparser.RestoreWithDefaultDB(n, specifiedDB, n.Text())
} else {
normalizedSQL = utilparser.RestoreWithoutDB(n)
}
normalizedStmt, digest = parser.NormalizeDigestForBinding(normalizedSQL)
return normalizedStmt, digest.String()
}

switch x := stmtNode.(type) {
case *ast.ExplainStmt:
// This function is only used to find bind record.
// For some SQLs, such as `explain select * from t`, they will be entered here many times,
// but some of them do not want to obtain bind record.
// The difference between them is whether len(x.Text()) is empty. They cannot be distinguished by stmt.restore.
// For these cases, we need return "" as normalize SQL and hash.
if len(x.Text()) == 0 {
return "", ""
}
switch x.Stmt.(type) {
case *ast.SelectStmt, *ast.DeleteStmt, *ast.UpdateStmt, *ast.InsertStmt:
normalizeSQL, digest := normalize(x.Stmt)
return normalizeSQL, digest
case *ast.SetOprStmt:
normalizeExplainSQL, _ := normalize(x)

idx := strings.Index(normalizeExplainSQL, "select")
parenthesesIdx := strings.Index(normalizeExplainSQL, "(")
if parenthesesIdx != -1 && parenthesesIdx < idx {
idx = parenthesesIdx
}
// If the SQL is `EXPLAIN ((VALUES ROW ()) ORDER BY 1);`, the idx will be -1.
if idx == -1 {
hash := parser.DigestNormalized(normalizeExplainSQL)
return normalizeExplainSQL, hash.String()
}
normalizeSQL := normalizeExplainSQL[idx:]
hash := parser.DigestNormalized(normalizeSQL)
return normalizeSQL, hash.String()
}
case *ast.SelectStmt, *ast.SetOprStmt, *ast.DeleteStmt, *ast.UpdateStmt, *ast.InsertStmt:
// This function is only used to find bind record.
// For some SQLs, such as `explain select * from t`, they will be entered here many times,
// but some of them do not want to obtain bind record.
// The difference between them is whether len(x.Text()) is empty. They cannot be distinguished by stmt.restore.
// For these cases, we need return "" as normalize SQL and hash.
if len(x.Text()) == 0 {
return "", ""
}
normalizedSQL, digest := normalize(x)
return normalizedSQL, digest
}
return "", ""
}

func fuzzyMatchBindingTableName(currentDB string, stmtTableNames, bindingTableNames []*ast.TableName) (numWildcards int, matched bool) {
if len(stmtTableNames) != len(bindingTableNames) {
return 0, false
Expand Down
5 changes: 3 additions & 2 deletions pkg/bindinfo/capture_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (

"github.com/pingcap/tidb/pkg/bindinfo"
"github.com/pingcap/tidb/pkg/bindinfo/internal"
"github.com/pingcap/tidb/pkg/bindinfo/norm"
"github.com/pingcap/tidb/pkg/config"
"github.com/pingcap/tidb/pkg/domain"
"github.com/pingcap/tidb/pkg/parser"
Expand Down Expand Up @@ -329,7 +330,7 @@ func TestBindingSource(t *testing.T) {
tk.MustExec("create global binding for select * from t where a > 10 using select * from t ignore index(idx_a) where a > 10")
bindHandle := dom.BindHandle()
stmt, _, _ := internal.UtilNormalizeWithDefaultDB(t, "select * from t where a > ?")
_, fuzzyDigest := bindinfo.NormalizeStmtForFuzzyBinding(stmt)
_, fuzzyDigest := norm.NormalizeStmtForBinding(stmt, norm.WithFuzz(true))
bindData, err := bindHandle.MatchGlobalBinding(tk.Session(), fuzzyDigest, bindinfo.CollectTableNames(stmt))
require.NoError(t, err)
require.NotNil(t, bindData)
Expand All @@ -351,7 +352,7 @@ func TestBindingSource(t *testing.T) {
tk.MustExec("admin capture bindings")
bindHandle.CaptureBaselines()
stmt, _, _ = internal.UtilNormalizeWithDefaultDB(t, "select * from t where a < ?")
_, fuzzyDigest = bindinfo.NormalizeStmtForFuzzyBinding(stmt)
_, fuzzyDigest = norm.NormalizeStmtForBinding(stmt, norm.WithFuzz(true))
bindData, err = bindHandle.MatchGlobalBinding(tk.Session(), fuzzyDigest, bindinfo.CollectTableNames(stmt))
require.NoError(t, err)
require.NotNil(t, bindData)
Expand Down
3 changes: 2 additions & 1 deletion pkg/bindinfo/global_handle.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
"time"

"github.com/pingcap/errors"
"github.com/pingcap/tidb/pkg/bindinfo/norm"
"github.com/pingcap/tidb/pkg/metrics"
"github.com/pingcap/tidb/pkg/parser"
"github.com/pingcap/tidb/pkg/parser/ast"
Expand Down Expand Up @@ -189,7 +190,7 @@ func buildFuzzyDigestMap(bindRecords []*BindRecord) map[string][]string {
p = parser.New()
continue
}
_, fuzzyDigest := NormalizeStmtForFuzzyBinding(stmt)
_, fuzzyDigest := norm.NormalizeStmtForBinding(stmt, norm.WithFuzz(true))
m[fuzzyDigest] = append(m[fuzzyDigest], binding.SQLDigest)
}
}
Expand Down
13 changes: 7 additions & 6 deletions pkg/bindinfo/global_handle_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (
"github.com/ngaut/pools"
"github.com/pingcap/tidb/pkg/bindinfo"
"github.com/pingcap/tidb/pkg/bindinfo/internal"
"github.com/pingcap/tidb/pkg/bindinfo/norm"
"github.com/pingcap/tidb/pkg/metrics"
"github.com/pingcap/tidb/pkg/parser"
sessiontypes "github.com/pingcap/tidb/pkg/session/types"
Expand Down Expand Up @@ -70,7 +71,7 @@ func TestBindingLastUpdateTime(t *testing.T) {
stmt, err := parser.New().ParseOneStmt("select * from test . t0", "", "")
require.NoError(t, err)

_, fuzzyDigest := bindinfo.NormalizeStmtForFuzzyBinding(stmt)
_, fuzzyDigest := norm.NormalizeStmtForBinding(stmt, norm.WithFuzz(true))
bindData, err := bindHandle.MatchGlobalBinding(tk.Session(), fuzzyDigest, bindinfo.CollectTableNames(stmt))
require.NoError(t, err)
require.Equal(t, 1, len(bindData.Bindings))
Expand Down Expand Up @@ -139,7 +140,7 @@ func TestBindParse(t *testing.T) {

stmt, err := parser.New().ParseOneStmt("select * from test . t", "", "")
require.NoError(t, err)
_, fuzzyDigest := bindinfo.NormalizeStmtForFuzzyBinding(stmt)
_, fuzzyDigest := norm.NormalizeStmtForBinding(stmt, norm.WithFuzz(true))
bindData, err := bindHandle.MatchGlobalBinding(tk.Session(), fuzzyDigest, bindinfo.CollectTableNames(stmt))
require.NoError(t, err)
require.NotNil(t, bindData)
Expand Down Expand Up @@ -441,7 +442,7 @@ func TestGlobalBinding(t *testing.T) {

stmt, _, _ := internal.UtilNormalizeWithDefaultDB(t, testSQL.querySQL)

_, fuzzyDigest := bindinfo.NormalizeStmtForFuzzyBinding(stmt)
_, fuzzyDigest := norm.NormalizeStmtForBinding(stmt, norm.WithFuzz(true))
bindData, err := dom.BindHandle().MatchGlobalBinding(tk.Session(), fuzzyDigest, bindinfo.CollectTableNames(stmt))
require.NoError(t, err)
require.NotNil(t, bindData)
Expand Down Expand Up @@ -476,7 +477,7 @@ func TestGlobalBinding(t *testing.T) {
require.NoError(t, err)
require.Equal(t, 1, bindHandle.Size())

_, fuzzyDigest = bindinfo.NormalizeStmtForFuzzyBinding(stmt)
_, fuzzyDigest = norm.NormalizeStmtForBinding(stmt, norm.WithFuzz(true))
bindData, err = dom.BindHandle().MatchGlobalBinding(tk.Session(), fuzzyDigest, bindinfo.CollectTableNames(stmt))
require.NoError(t, err)
require.NotNil(t, bindData)
Expand All @@ -493,7 +494,7 @@ func TestGlobalBinding(t *testing.T) {
_, err = tk.Exec("drop global " + testSQL.dropSQL)
require.Equal(t, uint64(1), tk.Session().AffectedRows())
require.NoError(t, err)
_, fuzzyDigest = bindinfo.NormalizeStmtForFuzzyBinding(stmt)
_, fuzzyDigest = norm.NormalizeStmtForBinding(stmt, norm.WithFuzz(true))
bindData, err = dom.BindHandle().MatchGlobalBinding(tk.Session(), fuzzyDigest, bindinfo.CollectTableNames(stmt))
require.NoError(t, err)
require.Nil(t, bindData)
Expand All @@ -503,7 +504,7 @@ func TestGlobalBinding(t *testing.T) {
require.NoError(t, err)
require.Equal(t, 0, bindHandle.Size())

_, fuzzyDigest = bindinfo.NormalizeStmtForFuzzyBinding(stmt)
_, fuzzyDigest = norm.NormalizeStmtForBinding(stmt, norm.WithFuzz(true))
bindData, err = dom.BindHandle().MatchGlobalBinding(tk.Session(), fuzzyDigest, bindinfo.CollectTableNames(stmt))
require.NoError(t, err)
require.Nil(t, bindData)
Expand Down
13 changes: 13 additions & 0 deletions pkg/bindinfo/norm/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -0,0 +1,13 @@
load("@io_bazel_rules_go//go:def.bzl", "go_library")

go_library(
name = "norm",
srcs = ["normalize.go"],
importpath = "github.com/pingcap/tidb/pkg/bindinfo/norm",
visibility = ["//visibility:public"],
deps = [
"//pkg/parser",
"//pkg/parser/ast",
"//pkg/util/parser",
],
)
127 changes: 127 additions & 0 deletions pkg/bindinfo/norm/normalize.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,127 @@
// 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 norm

import (
"strings"

"github.com/pingcap/tidb/pkg/parser"
"github.com/pingcap/tidb/pkg/parser/ast"
utilparser "github.com/pingcap/tidb/pkg/util/parser"
)

type option struct {
specifiedDB string
fuzz bool
}

type optionFunc func(*option)

// WithFuzz specifies whether to eliminate schema names.
func WithFuzz(fuzz bool) optionFunc {
return func(user *option) {
user.fuzz = fuzz
}
}

// WithSpecifiedDB specifies the specified DB name.
func WithSpecifiedDB(specifiedDB string) optionFunc {
return func(user *option) {
user.specifiedDB = specifiedDB
}
}

// NormalizeStmtForBinding normalizes a statement for binding.
// when fuzz is false, schema names will be completed automatically: `select * from t` --> `select * from db . t`.
// when fuzz is true, schema names will be eliminated automatically: `select * from db . t` --> `select * from t`.
func NormalizeStmtForBinding(stmtNode ast.StmtNode, options ...optionFunc) (normalizedStmt, exactSQLDigest string) {
opt := &option{}
for _, option := range options {
option(opt)
}
return normalizeStmt(stmtNode, opt.specifiedDB, opt.fuzz)
}

// NormalizeStmtForBinding normalizes a statement for binding.
// This function skips Explain automatically, and literals in in-lists will be normalized as '...'.
// For normal bindings, DB name will be completed automatically:
//
// e.g. `select * from t where a in (1, 2, 3)` --> `select * from test.t where a in (...)`
func normalizeStmt(stmtNode ast.StmtNode, specifiedDB string, fuzzy bool) (normalizedStmt, sqlDigest string) {
normalize := func(n ast.StmtNode) (normalizedStmt, sqlDigest string) {
eraseLastSemicolon(n)
var digest *parser.Digest
var normalizedSQL string
if !fuzzy {
normalizedSQL = utilparser.RestoreWithDefaultDB(n, specifiedDB, n.Text())
} else {
normalizedSQL = utilparser.RestoreWithoutDB(n)
}
normalizedStmt, digest = parser.NormalizeDigestForBinding(normalizedSQL)
return normalizedStmt, digest.String()
}

switch x := stmtNode.(type) {
case *ast.ExplainStmt:
// This function is only used to find bind record.
// For some SQLs, such as `explain select * from t`, they will be entered here many times,
// but some of them do not want to obtain bind record.
// The difference between them is whether len(x.Text()) is empty. They cannot be distinguished by stmt.restore.
// For these cases, we need return "" as normalize SQL and hash.
if len(x.Text()) == 0 {
return "", ""
}
switch x.Stmt.(type) {
case *ast.SelectStmt, *ast.DeleteStmt, *ast.UpdateStmt, *ast.InsertStmt:
normalizeSQL, digest := normalize(x.Stmt)
return normalizeSQL, digest
case *ast.SetOprStmt:
normalizeExplainSQL, _ := normalize(x)

idx := strings.Index(normalizeExplainSQL, "select")
parenthesesIdx := strings.Index(normalizeExplainSQL, "(")
if parenthesesIdx != -1 && parenthesesIdx < idx {
idx = parenthesesIdx
}
// If the SQL is `EXPLAIN ((VALUES ROW ()) ORDER BY 1);`, the idx will be -1.
if idx == -1 {
hash := parser.DigestNormalized(normalizeExplainSQL)
return normalizeExplainSQL, hash.String()
}
normalizeSQL := normalizeExplainSQL[idx:]
hash := parser.DigestNormalized(normalizeSQL)
return normalizeSQL, hash.String()
}
case *ast.SelectStmt, *ast.SetOprStmt, *ast.DeleteStmt, *ast.UpdateStmt, *ast.InsertStmt:
// This function is only used to find bind record.
// For some SQLs, such as `explain select * from t`, they will be entered here many times,
// but some of them do not want to obtain bind record.
// The difference between them is whether len(x.Text()) is empty. They cannot be distinguished by stmt.restore.
// For these cases, we need return "" as normalize SQL and hash.
if len(x.Text()) == 0 {
return "", ""
}
normalizedSQL, digest := normalize(x)
return normalizedSQL, digest
}
return "", ""
}

func eraseLastSemicolon(stmt ast.StmtNode) {
sql := stmt.Text()
if len(sql) > 0 && sql[len(sql)-1] == ';' {
stmt.SetText(nil, sql[:len(sql)-1])
}
}
3 changes: 2 additions & 1 deletion pkg/bindinfo/session_handle.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (
"time"

"github.com/pingcap/errors"
"github.com/pingcap/tidb/pkg/bindinfo/norm"
"github.com/pingcap/tidb/pkg/metrics"
"github.com/pingcap/tidb/pkg/parser"
"github.com/pingcap/tidb/pkg/parser/ast"
Expand Down Expand Up @@ -117,7 +118,7 @@ func (h *sessionBindingHandle) MatchSessionBinding(sctx sessionctx.Context, fuzz
if err != nil {
return nil, err
}
_, bindingFuzzyDigest := NormalizeStmtForFuzzyBinding(bindingStmt)
_, bindingFuzzyDigest := norm.NormalizeStmtForBinding(bindingStmt, norm.WithFuzz(true))
if bindingFuzzyDigest != fuzzyDigest {
continue
}
Expand Down
Loading

0 comments on commit d68acd6

Please sign in to comment.