planner: support set binding status by sql digest (#39517)
ref pingcap/tidb#39199
This commit is contained in:
@ -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()
|
||||
|
||||
@ -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'.
|
||||
|
||||
@ -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'")
|
||||
}
|
||||
|
||||
@ -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.
|
||||
|
||||
@ -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:
|
||||
|
||||
Reference in New Issue
Block a user