Skip to content

Commit

Permalink
planner: simplify the bindinfo package (#58664)
Browse files Browse the repository at this point in the history
ref #51347
  • Loading branch information
qw4990 authored Jan 3, 2025
1 parent 65e514e commit 0662241
Show file tree
Hide file tree
Showing 7 changed files with 19 additions and 82 deletions.
1 change: 0 additions & 1 deletion pkg/bindinfo/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@ go_library(
"//pkg/sessionctx/sessionstates",
"//pkg/sessionctx/variable",
"//pkg/types",
"//pkg/types/parser_driver",
"//pkg/util",
"//pkg/util/chunk",
"//pkg/util/hack",
Expand Down
18 changes: 4 additions & 14 deletions pkg/bindinfo/binding.go
Original file line number Diff line number Diff line change
Expand Up @@ -264,7 +264,7 @@ func (*tableNameCollector) Leave(in ast.Node) (out ast.Node, ok bool) {

// prepareHints builds ID and Hint for Bindings. If sctx is not nil, we check if
// the BindSQL is still valid.
func prepareHints(sctx sessionctx.Context, binding *Binding) (rerr error) {
func prepareHints(_ sessionctx.Context, binding *Binding) (rerr error) {
defer func() {
if r := recover(); r != nil {
rerr = errors.Errorf("panic when preparing hints for binding %v, panic: %v", binding.BindSQL, r)
Expand All @@ -281,25 +281,15 @@ func prepareHints(sctx sessionctx.Context, binding *Binding) (rerr error) {
return err
}
tableNames := CollectTableNames(bindingStmt)
isFuzzy := isCrossDBBinding(bindingStmt)
if isFuzzy {
isCrossDB := isCrossDBBinding(bindingStmt)
if isCrossDB {
dbName = "*" // ues '*' for universal bindings
}

hintsSet, stmt, warns, err := hint.ParseHintsSet(p, binding.BindSQL, binding.Charset, binding.Collation, dbName)
hintsSet, _, warns, err := hint.ParseHintsSet(p, binding.BindSQL, binding.Charset, binding.Collation, dbName)
if err != nil {
return err
}
if sctx != nil && !isFuzzy {
paramChecker := &paramMarkerChecker{}
stmt.Accept(paramChecker)
if !paramChecker.hasParamMarker {
_, err = getHintsForSQL(sctx, binding.BindSQL)
if err != nil {
return err
}
}
}
hintsStr, err := hintsSet.Restore()
if err != nil {
return err
Expand Down
57 changes: 1 addition & 56 deletions pkg/bindinfo/global_handle.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,6 @@ import (
"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
"github.com/pingcap/tidb/pkg/types"
driver "github.com/pingcap/tidb/pkg/types/parser_driver"
"github.com/pingcap/tidb/pkg/util"
"github.com/pingcap/tidb/pkg/util/chunk"
"github.com/pingcap/tidb/pkg/util/hint"
Expand Down Expand Up @@ -418,50 +417,12 @@ func newBindingFromStorage(sctx sessionctx.Context, row chunk.Row) (string, *Bin
return sqlDigest.String(), binding, err
}

func getHintsForSQL(sctx sessionctx.Context, sql string) (string, error) {
origVals := sctx.GetSessionVars().UsePlanBaselines
sctx.GetSessionVars().UsePlanBaselines = false

// Usually passing a sprintf to ExecuteInternal is not recommended, but in this case
// it is safe because ExecuteInternal does not permit MultiStatement execution. Thus,
// the statement won't be able to "break out" from EXPLAIN.
rs, err := exec(sctx, fmt.Sprintf("EXPLAIN FORMAT='hint' %s", sql))
sctx.GetSessionVars().UsePlanBaselines = origVals
if rs != nil {
defer func() {
// Audit log is collected in Close(), set InRestrictedSQL to avoid 'create sql binding' been recorded as 'explain'.
origin := sctx.GetSessionVars().InRestrictedSQL
sctx.GetSessionVars().InRestrictedSQL = true
terror.Call(rs.Close)
sctx.GetSessionVars().InRestrictedSQL = origin
}()
}
if err != nil {
return "", err
}
chk := rs.NewChunk(nil)
err = rs.Next(context.TODO(), chk)
if err != nil {
return "", err
}
return chk.GetRow(0).GetString(0), nil
}

// GenerateBindingSQL generates binding sqls from stmt node and plan hints.
func GenerateBindingSQL(stmtNode ast.StmtNode, planHint string, skipCheckIfHasParam bool, defaultDB string) string {
func GenerateBindingSQL(stmtNode ast.StmtNode, planHint string, defaultDB string) string {
// If would be nil for very simple cases such as point get, we do not need to evolve for them.
if planHint == "" {
return ""
}
if !skipCheckIfHasParam {
paramChecker := &paramMarkerChecker{}
stmtNode.Accept(paramChecker)
// We need to evolve on current sql, but we cannot restore values for paramMarkers yet,
// so just ignore them now.
if paramChecker.hasParamMarker {
return ""
}
}
// We need to evolve plan based on the current sql, not the original sql which may have different parameters.
// So here we would remove the hint and inject the current best plan hint.
hint.BindHint(stmtNode, &hint.HintsSet{})
Expand Down Expand Up @@ -514,22 +475,6 @@ func GenerateBindingSQL(stmtNode ast.StmtNode, planHint string, skipCheckIfHasPa
return ""
}

type paramMarkerChecker struct {
hasParamMarker bool
}

func (e *paramMarkerChecker) Enter(in ast.Node) (ast.Node, bool) {
if _, ok := in.(*driver.ParamMarkerExpr); ok {
e.hasParamMarker = true
return in, true
}
return in, false
}

func (*paramMarkerChecker) Leave(in ast.Node) (ast.Node, bool) {
return in, true
}

func (h *globalBindingHandle) callWithSCtx(wrapTxn bool, f func(sctx sessionctx.Context) error) (err error) {
resource, err := h.sPool.Get()
if err != nil {
Expand Down
5 changes: 2 additions & 3 deletions pkg/bindinfo/global_handle_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -97,11 +97,10 @@ func TestBindingLastUpdateTimeWithInvalidBind(t *testing.T) {
updateTime0 := rows0[0][1]
require.Equal(t, updateTime0, "0000-00-00 00:00:00")

tk.MustExec("insert into mysql.bind_info values('select * from `test` . `t`', 'select * from `test` . `t` use index(`idx`)', 'test', 'enabled', '2000-01-01 09:00:00', '2000-01-01 09:00:00', '', '','" +
tk.MustExec("insert into mysql.bind_info values('select * from `test` . `t`', 'invalid_binding', 'test', 'enabled', '2000-01-01 09:00:00', '2000-01-01 09:00:00', '', '','" +
bindinfo.Manual + "', '', '')")
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a int)")
tk.MustExec("admin reload bindings;")

rows1 := tk.MustQuery("show status like 'last_plan_binding_update_time';").Rows()
Expand Down Expand Up @@ -748,7 +747,7 @@ func TestErrorBind(t *testing.T) {

tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustGetErrMsg("create global binding for select * from t using select * from t", "[schema:1146]Table 'test.t' doesn't exist")
tk.MustContainErrMsg("create global binding for select * xxx", "You have an error in your SQL syntax")
tk.MustExec("drop table if exists t")
tk.MustExec("drop table if exists t1")
tk.MustExec("create table t(i int, s varchar(20))")
Expand Down
6 changes: 2 additions & 4 deletions pkg/bindinfo/tests/bind_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -348,12 +348,10 @@ func TestInvisibleIndex(t *testing.T) {
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a int, b int, unique idx_a(a), index idx_b(b) invisible)")
tk.MustGetErrMsg(
"create global binding for select * from t using select * from t use index(idx_b) ",
"[planner:1176]Key 'idx_b' doesn't exist in table 't'")

// Create bind using index
tk.MustExec("create global binding for select * from t using select * from t use index(idx_a) ")
tk.MustExec("create global binding for select * from t using select * from t use index(idx_b)")
tk.MustExec("create global binding for select * from t using select * from t use index(idx_a)")

tk.MustQuery("select * from t")
require.Equal(t, "t:idx_a", tk.Session().GetSessionVars().StmtCtx.IndexNames[0])
Expand Down
2 changes: 1 addition & 1 deletion pkg/planner/core/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -918,7 +918,7 @@ func constructSQLBindOPFromPlanDigest(
return nil, errors.NewNoStackErrorf("binding failed: %v. Plan Digest: %v", err, planDigest)
}
complete, reason := hint.CheckBindingFromHistoryComplete(originNode, bindableStmt.PlanHint)
bindSQL := bindinfo.GenerateBindingSQL(originNode, bindableStmt.PlanHint, true, bindableStmt.Schema)
bindSQL := bindinfo.GenerateBindingSQL(originNode, bindableStmt.PlanHint, bindableStmt.Schema)
var hintNode ast.StmtNode
hintNode, err = parser4binding.ParseOneStmt(bindSQL, bindableStmt.Charset, bindableStmt.Collation)
if err != nil {
Expand Down
12 changes: 9 additions & 3 deletions pkg/sessionctx/sessionstates/session_states_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1293,7 +1293,9 @@ func TestSQLBinding(t *testing.T) {
tk.MustExec("drop table test.t1")
return nil
},
restoreErr: errno.ErrNoSuchTable,
checkFunc: func(tk *testkit.TestKit, param any) {
require.Equal(t, 1, len(tk.MustQuery("show session bindings").Rows()))
},
cleanFunc: func(tk *testkit.TestKit) {
tk.MustExec("create table test.t1(id int primary key, name varchar(10), key(name))")
},
Expand All @@ -1308,7 +1310,9 @@ func TestSQLBinding(t *testing.T) {
tk.MustExec("drop database test1")
return nil
},
restoreErr: errno.ErrNoSuchTable,
checkFunc: func(tk *testkit.TestKit, param any) {
require.Equal(t, 1, len(tk.MustQuery("show session bindings").Rows()))
},
},
{
// alter the table
Expand All @@ -1317,7 +1321,9 @@ func TestSQLBinding(t *testing.T) {
tk.MustExec("alter table test.t1 drop index name")
return nil
},
restoreErr: errno.ErrKeyDoesNotExist,
checkFunc: func(tk *testkit.TestKit, param any) {
require.Equal(t, 1, len(tk.MustQuery("show session bindings").Rows()))
},
cleanFunc: func(tk *testkit.TestKit) {
tk.MustExec("alter table test.t1 add index name(name)")
},
Expand Down

0 comments on commit 0662241

Please sign in to comment.