planner: support set binding status by sql digest (#39517)

ref pingcap/tidb#39199
This commit is contained in:
fzzf678
2022-12-21 17:28:54 +08:00
committed by GitHub
parent 785f515b3a
commit 08f23ef9fe
5 changed files with 67 additions and 4 deletions

View File

@ -533,6 +533,15 @@ func (h *BindHandle) SetBindRecordStatus(originalSQL string, binding *Binding, n
return
}
// SetBindRecordStatusByDigest set a BindRecord's status to the storage and bind cache.
func (h *BindHandle) SetBindRecordStatusByDigest(newStatus, sqlDigest string) (ok bool, err error) {
oldRecord, err := h.GetBindRecordBySQLDigest(sqlDigest)
if err != nil {
return false, err
}
return h.SetBindRecordStatus(oldRecord.OriginalSQL, nil, newStatus)
}
// GCBindRecord physically removes the deleted bind records in mysql.bind_info.
func (h *BindHandle) GCBindRecord() (err error) {
h.bindInfo.Lock()

View File

@ -63,6 +63,8 @@ func (e *SQLBindExec) Next(ctx context.Context, req *chunk.Chunk) error {
return e.reloadBindings()
case plannercore.OpSetBindingStatus:
return e.setBindingStatus()
case plannercore.OpSetBindingStatusByDigest:
return e.setBindingStatusByDigest()
default:
return errors.Errorf("unsupported SQL bind operation: %v", e.sqlBindOp)
}
@ -119,6 +121,15 @@ func (e *SQLBindExec) setBindingStatus() error {
return err
}
func (e *SQLBindExec) setBindingStatusByDigest() error {
ok, err := domain.GetDomain(e.ctx).BindHandle().SetBindRecordStatusByDigest(e.newStatus, e.sqlDigest)
if err == nil && !ok {
warningMess := errors.New("There are no bindings can be set the status. Please check the SQL text")
e.ctx.GetSessionVars().StmtCtx.AppendWarning(warningMess)
}
return err
}
func (e *SQLBindExec) createSQLBind() error {
// For audit log, SQLBindExec execute "explain" statement internally, save and recover stmtctx
// is necessary to avoid 'create binding' been recorded as 'explain'.

View File

@ -1028,3 +1028,34 @@ func TestBindingFromHistoryWithTiFlashBindable(t *testing.T) {
planDigest := tk.MustQuery(fmt.Sprintf("select plan_digest from information_schema.statements_summary where query_sample_text = '%s'", sql)).Rows()
tk.MustGetErrMsg(fmt.Sprintf("create binding from history using plan digest '%s'", planDigest[0][0]), "can't create binding for query with tiflash engine")
}
func TestSetBindingStatusBySQLDigest(t *testing.T) {
s := new(clusterTablesSuite)
s.store, s.dom = testkit.CreateMockStoreAndDomain(t)
s.rpcserver, s.listenAddr = s.setUpRPCService(t, "127.0.0.1:0", nil)
s.httpServer, s.mockAddr = s.setUpMockPDHTTPServer()
s.startTime = time.Now()
defer s.httpServer.Close()
defer s.rpcserver.Stop()
tk := s.newTestKitWithRoot(t)
require.NoError(t, tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil))
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(id int, a int, key(a))")
sql := "select /*+ ignore_index(t, a) */ * from t where t.a = 1"
tk.MustExec(sql)
planDigest := tk.MustQuery(fmt.Sprintf("select plan_digest from information_schema.cluster_statements_summary where query_sample_text = '%s'", sql)).Rows()
tk.MustExec(fmt.Sprintf("create global binding from history using plan digest '%s'", planDigest[0][0]))
sql = "select * from t where t.a = 1"
tk.MustExec(sql)
tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("1"))
sqlDigest := tk.MustQuery("show global bindings").Rows()
tk.MustExec(fmt.Sprintf("set binding disabled for sql digest '%s'", sqlDigest[0][9]))
tk.MustExec(sql)
tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("0"))
tk.MustExec(fmt.Sprintf("set binding enabled for sql digest '%s'", sqlDigest[0][9]))
tk.MustExec(sql)
tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("1"))
tk.MustGetErrMsg("set binding enabled for sql digest '2'", "can't find any binding for '2'")
}

View File

@ -252,6 +252,8 @@ const (
OpSetBindingStatus
// OpSQLBindDropByDigest is used to drop SQL binds by digest
OpSQLBindDropByDigest
// OpSetBindingStatusByDigest represents the operation to set SQL binding status by sql digest.
OpSetBindingStatusByDigest
)
// SQLBindPlan represents a plan for SQL bind.

View File

@ -1014,10 +1014,20 @@ func (b *PlanBuilder) buildDropBindPlan(v *ast.DropBindingStmt) (Plan, error) {
}
func (b *PlanBuilder) buildSetBindingStatusPlan(v *ast.SetBindingStmt) (Plan, error) {
p := &SQLBindPlan{
SQLBindOp: OpSetBindingStatus,
NormdOrigSQL: parser.Normalize(utilparser.RestoreWithDefaultDB(v.OriginNode, b.ctx.GetSessionVars().CurrentDB, v.OriginNode.Text())),
Db: utilparser.GetDefaultDB(v.OriginNode, b.ctx.GetSessionVars().CurrentDB),
var p *SQLBindPlan
if v.OriginNode != nil {
p = &SQLBindPlan{
SQLBindOp: OpSetBindingStatus,
NormdOrigSQL: parser.Normalize(utilparser.RestoreWithDefaultDB(v.OriginNode, b.ctx.GetSessionVars().CurrentDB, v.OriginNode.Text())),
Db: utilparser.GetDefaultDB(v.OriginNode, b.ctx.GetSessionVars().CurrentDB),
}
} else if v.SQLDigest != "" {
p = &SQLBindPlan{
SQLBindOp: OpSetBindingStatusByDigest,
SQLDigest: v.SQLDigest,
}
} else {
return nil, errors.New("sql digest is empty")
}
switch v.BindingStatusType {
case ast.BindingStatusTypeEnabled: