// Copyright 2019 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, // See the License for the specific language governing permissions and // limitations under the License. package bindinfo import ( "context" "fmt" "runtime" "strconv" "strings" "sync" "sync/atomic" "time" "github.com/pingcap/parser" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/format" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/types" driver "github.com/pingcap/tidb/types/parser_driver" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/hint" "github.com/pingcap/tidb/util/logutil" utilparser "github.com/pingcap/tidb/util/parser" "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/stmtsummary" "github.com/pingcap/tidb/util/timeutil" "go.uber.org/zap" ) // BindHandle is used to handle all global sql bind operations. type BindHandle struct { sctx struct { sync.Mutex sessionctx.Context } // bindInfo caches the sql bind info from storage. // // The Mutex protects that there is only one goroutine changes the content // of atmoic.Value. // // NOTE: Concurrent Value Write: // // bindInfo.Lock() // newCache := bindInfo.Value.Load() // do the write operation on the newCache // bindInfo.Value.Store(newCache) // bindInfo.Unlock() // // NOTE: Concurrent Value Read: // // cache := bindInfo.Load(). // read the content // bindInfo struct { sync.Mutex atomic.Value parser *parser.Parser lastUpdateTime types.Time } // invalidBindRecordMap indicates the invalid bind records found during querying. // A record will be deleted from this map, after 2 bind-lease, after it is dropped from the kv. invalidBindRecordMap tmpBindRecordMap // pendingVerifyBindRecordMap indicates the pending verify bind records that found during query. pendingVerifyBindRecordMap tmpBindRecordMap } // Lease influences the duration of loading bind info and handling invalid bind. var Lease = 3 * time.Second const ( // OwnerKey is the bindinfo owner path that is saved to etcd. OwnerKey = "/tidb/bindinfo/owner" // Prompt is the prompt for bindinfo owner manager. Prompt = "bindinfo" ) type bindRecordUpdate struct { bindRecord *BindRecord updateTime time.Time } // NewBindHandle creates a new BindHandle. func NewBindHandle(ctx sessionctx.Context) *BindHandle { handle := &BindHandle{} handle.sctx.Context = ctx handle.bindInfo.Value.Store(make(cache, 32)) handle.bindInfo.parser = parser.New() handle.invalidBindRecordMap.Value.Store(make(map[string]*bindRecordUpdate)) handle.invalidBindRecordMap.flushFunc = func(record *BindRecord) error { return handle.DropBindRecord(record.OriginalSQL, record.Db, &record.Bindings[0]) } handle.pendingVerifyBindRecordMap.Value.Store(make(map[string]*bindRecordUpdate)) handle.pendingVerifyBindRecordMap.flushFunc = func(record *BindRecord) error { // BindSQL has already been validated when coming here, so we use nil sctx parameter. return handle.AddBindRecord(nil, record) } return handle } // Update updates the global sql bind cache. func (h *BindHandle) Update(fullLoad bool) (err error) { h.bindInfo.Lock() lastUpdateTime := h.bindInfo.lastUpdateTime h.bindInfo.Unlock() sql := "select original_sql, bind_sql, default_db, status, create_time, update_time, charset, collation, source from mysql.bind_info" if !fullLoad { sql += " where update_time > \"" + lastUpdateTime.String() + "\"" } // We need to apply the updates by order, wrong apply order of same original sql may cause inconsistent state. sql += " order by update_time" // No need to acquire the session context lock for ExecRestrictedSQL, it // uses another background session. rows, _, err := h.sctx.Context.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(sql) if err != nil { return err } // Make sure there is only one goroutine writes the cache. h.bindInfo.Lock() newCache := h.bindInfo.Value.Load().(cache).copy() defer func() { h.bindInfo.lastUpdateTime = lastUpdateTime h.bindInfo.Value.Store(newCache) h.bindInfo.Unlock() }() for _, row := range rows { hash, meta, err := h.newBindRecord(row) // Update lastUpdateTime to the newest one. if meta.Bindings[0].UpdateTime.Compare(lastUpdateTime) > 0 { lastUpdateTime = meta.Bindings[0].UpdateTime } if err != nil { logutil.BgLogger().Info("update bindinfo failed", zap.Error(err)) continue } oldRecord := newCache.getBindRecord(hash, meta.OriginalSQL, meta.Db) newRecord := merge(oldRecord, meta).removeDeletedBindings() if len(newRecord.Bindings) > 0 { newCache.setBindRecord(hash, newRecord) } else { newCache.removeDeletedBindRecord(hash, newRecord) } updateMetrics(metrics.ScopeGlobal, oldRecord, newCache.getBindRecord(hash, meta.OriginalSQL, meta.Db), true) } return nil } // CreateBindRecord creates a BindRecord to the storage and the cache. // It replaces all the exists bindings for the same normalized SQL. func (h *BindHandle) CreateBindRecord(sctx sessionctx.Context, record *BindRecord) (err error) { err = record.prepareHints(sctx) if err != nil { return err } exec, _ := h.sctx.Context.(sqlexec.SQLExecutor) h.sctx.Lock() _, err = exec.ExecuteInternal(context.TODO(), "BEGIN") if err != nil { h.sctx.Unlock() return } normalizedSQL := parser.DigestNormalized(record.OriginalSQL) oldRecord := h.GetBindRecord(normalizedSQL, record.OriginalSQL, record.Db) defer func() { if err != nil { _, err1 := exec.ExecuteInternal(context.TODO(), "ROLLBACK") h.sctx.Unlock() terror.Log(err1) return } _, err = exec.ExecuteInternal(context.TODO(), "COMMIT") h.sctx.Unlock() if err != nil { return } // Make sure there is only one goroutine writes the cache and uses parser. h.bindInfo.Lock() if oldRecord != nil { h.removeBindRecord(normalizedSQL, oldRecord) } h.appendBindRecord(normalizedSQL, record) h.bindInfo.Unlock() }() txn, err1 := h.sctx.Context.Txn(true) if err1 != nil { return err1 } now := types.NewTime(types.FromGoTime(oracle.GetTimeFromTS(txn.StartTS())), mysql.TypeTimestamp, 3) if oldRecord != nil { for _, binding := range oldRecord.Bindings { _, err1 = exec.ExecuteInternal(context.TODO(), h.logicalDeleteBindInfoSQL(record.OriginalSQL, record.Db, now, binding.BindSQL)) if err != nil { return err1 } } } for i := range record.Bindings { record.Bindings[i].CreateTime = now record.Bindings[i].UpdateTime = now // insert the BindRecord to the storage. _, err = exec.ExecuteInternal(context.TODO(), h.insertBindInfoSQL(record.OriginalSQL, record.Db, record.Bindings[i])) if err != nil { return err } } return nil } // AddBindRecord adds a BindRecord to the storage and BindRecord to the cache. func (h *BindHandle) AddBindRecord(sctx sessionctx.Context, record *BindRecord) (err error) { err = record.prepareHints(sctx) if err != nil { return err } oldRecord := h.GetBindRecord(parser.DigestNormalized(record.OriginalSQL), record.OriginalSQL, record.Db) var duplicateBinding *Binding if oldRecord != nil { binding := oldRecord.FindBinding(record.Bindings[0].ID) if binding != nil { // There is already a binding with status `Using`, `PendingVerify` or `Rejected`, we could directly cancel the job. if record.Bindings[0].Status == PendingVerify { return nil } // Otherwise, we need to remove it before insert. duplicateBinding = binding } } exec, _ := h.sctx.Context.(sqlexec.SQLExecutor) h.sctx.Lock() _, err = exec.ExecuteInternal(context.TODO(), "BEGIN") if err != nil { h.sctx.Unlock() return } defer func() { if err != nil { _, err1 := exec.ExecuteInternal(context.TODO(), "ROLLBACK") h.sctx.Unlock() terror.Log(err1) return } _, err = exec.ExecuteInternal(context.TODO(), "COMMIT") h.sctx.Unlock() if err != nil { return } // Make sure there is only one goroutine writes the cache and uses parser. h.bindInfo.Lock() h.appendBindRecord(parser.DigestNormalized(record.OriginalSQL), record) h.bindInfo.Unlock() }() txn, err1 := h.sctx.Context.Txn(true) if err1 != nil { return err1 } if duplicateBinding != nil { _, err = exec.ExecuteInternal(context.TODO(), h.deleteBindInfoSQL(record.OriginalSQL, record.Db, duplicateBinding.BindSQL)) if err != nil { return err } } now := types.NewTime(types.FromGoTime(oracle.GetTimeFromTS(txn.StartTS())), mysql.TypeTimestamp, 3) for i := range record.Bindings { if duplicateBinding != nil { record.Bindings[i].CreateTime = duplicateBinding.CreateTime } else { record.Bindings[i].CreateTime = now } record.Bindings[i].UpdateTime = now // insert the BindRecord to the storage. _, err = exec.ExecuteInternal(context.TODO(), h.insertBindInfoSQL(record.OriginalSQL, record.Db, record.Bindings[i])) if err != nil { return err } } return nil } // DropBindRecord drops a BindRecord to the storage and BindRecord int the cache. func (h *BindHandle) DropBindRecord(originalSQL, db string, binding *Binding) (err error) { exec, _ := h.sctx.Context.(sqlexec.SQLExecutor) h.sctx.Lock() _, err = exec.ExecuteInternal(context.TODO(), "BEGIN") if err != nil { h.sctx.Unlock() return } defer func() { if err != nil { _, err1 := exec.ExecuteInternal(context.TODO(), "ROLLBACK") h.sctx.Unlock() terror.Log(err1) return } _, err = exec.ExecuteInternal(context.TODO(), "COMMIT") h.sctx.Unlock() if err != nil { return } record := &BindRecord{OriginalSQL: originalSQL, Db: db} if binding != nil { record.Bindings = append(record.Bindings, *binding) } // Make sure there is only one goroutine writes the cache and uses parser. h.bindInfo.Lock() h.removeBindRecord(parser.DigestNormalized(originalSQL), record) h.bindInfo.Unlock() }() txn, err1 := h.sctx.Context.Txn(true) if err1 != nil { return err1 } updateTs := types.NewTime(types.FromGoTime(oracle.GetTimeFromTS(txn.StartTS())), mysql.TypeTimestamp, 3) bindSQL := "" if binding != nil { bindSQL = binding.BindSQL } _, err = exec.ExecuteInternal(context.TODO(), h.logicalDeleteBindInfoSQL(originalSQL, db, updateTs, bindSQL)) return err } // tmpBindRecordMap is used to temporarily save bind record changes. // Those changes will be flushed into store periodically. type tmpBindRecordMap struct { sync.Mutex atomic.Value flushFunc func(record *BindRecord) error } // flushToStore calls flushFunc for items in tmpBindRecordMap and removes them with a delay. func (tmpMap *tmpBindRecordMap) flushToStore() { tmpMap.Lock() defer tmpMap.Unlock() newMap := copyBindRecordUpdateMap(tmpMap.Load().(map[string]*bindRecordUpdate)) for key, bindRecord := range newMap { if bindRecord.updateTime.IsZero() { err := tmpMap.flushFunc(bindRecord.bindRecord) if err != nil { logutil.BgLogger().Error("flush bind record failed", zap.Error(err)) } bindRecord.updateTime = time.Now() continue } if time.Since(bindRecord.updateTime) > 6*time.Second { delete(newMap, key) updateMetrics(metrics.ScopeGlobal, bindRecord.bindRecord, nil, false) } } tmpMap.Store(newMap) } // Add puts a BindRecord into tmpBindRecordMap. func (tmpMap *tmpBindRecordMap) Add(bindRecord *BindRecord) { key := bindRecord.OriginalSQL + ":" + bindRecord.Db + ":" + bindRecord.Bindings[0].ID if _, ok := tmpMap.Load().(map[string]*bindRecordUpdate)[key]; ok { return } tmpMap.Lock() defer tmpMap.Unlock() if _, ok := tmpMap.Load().(map[string]*bindRecordUpdate)[key]; ok { return } newMap := copyBindRecordUpdateMap(tmpMap.Load().(map[string]*bindRecordUpdate)) newMap[key] = &bindRecordUpdate{ bindRecord: bindRecord, } tmpMap.Store(newMap) updateMetrics(metrics.ScopeGlobal, nil, bindRecord, false) } // DropInvalidBindRecord executes the drop BindRecord tasks. func (h *BindHandle) DropInvalidBindRecord() { h.invalidBindRecordMap.flushToStore() } // AddDropInvalidBindTask adds BindRecord which needs to be deleted into invalidBindRecordMap. func (h *BindHandle) AddDropInvalidBindTask(invalidBindRecord *BindRecord) { h.invalidBindRecordMap.Add(invalidBindRecord) } // Size returns the size of bind info cache. func (h *BindHandle) Size() int { size := 0 for _, bindRecords := range h.bindInfo.Load().(cache) { size += len(bindRecords) } return size } // GetBindRecord returns the BindRecord of the (normdOrigSQL,db) if BindRecord exist. func (h *BindHandle) GetBindRecord(hash, normdOrigSQL, db string) *BindRecord { return h.bindInfo.Load().(cache).getBindRecord(hash, normdOrigSQL, db) } // GetAllBindRecord returns all bind records in cache. func (h *BindHandle) GetAllBindRecord() (bindRecords []*BindRecord) { bindRecordMap := h.bindInfo.Load().(cache) for _, bindRecord := range bindRecordMap { bindRecords = append(bindRecords, bindRecord...) } return bindRecords } // newBindRecord builds BindRecord from a tuple in storage. func (h *BindHandle) newBindRecord(row chunk.Row) (string, *BindRecord, error) { hint := Binding{ BindSQL: row.GetString(1), Status: row.GetString(3), CreateTime: row.GetTime(4), UpdateTime: row.GetTime(5), Charset: row.GetString(6), Collation: row.GetString(7), Source: row.GetString(8), } bindRecord := &BindRecord{ OriginalSQL: row.GetString(0), Db: row.GetString(2), Bindings: []Binding{hint}, } hash := parser.DigestNormalized(bindRecord.OriginalSQL) h.sctx.Lock() defer h.sctx.Unlock() h.sctx.GetSessionVars().CurrentDB = bindRecord.Db err := bindRecord.prepareHints(h.sctx.Context) return hash, bindRecord, err } // appendBindRecord addes the BindRecord to the cache, all the stale BindRecords are // removed from the cache after this operation. func (h *BindHandle) appendBindRecord(hash string, meta *BindRecord) { newCache := h.bindInfo.Value.Load().(cache).copy() oldRecord := newCache.getBindRecord(hash, meta.OriginalSQL, meta.Db) newRecord := merge(oldRecord, meta) newCache.setBindRecord(hash, newRecord) h.bindInfo.Value.Store(newCache) updateMetrics(metrics.ScopeGlobal, oldRecord, newRecord, false) } // removeBindRecord removes the BindRecord from the cache. func (h *BindHandle) removeBindRecord(hash string, meta *BindRecord) { newCache := h.bindInfo.Value.Load().(cache).copy() oldRecord := newCache.getBindRecord(hash, meta.OriginalSQL, meta.Db) newCache.removeDeletedBindRecord(hash, meta) h.bindInfo.Value.Store(newCache) updateMetrics(metrics.ScopeGlobal, oldRecord, newCache.getBindRecord(hash, meta.OriginalSQL, meta.Db), false) } // removeDeletedBindRecord removes the BindRecord which has same originSQL and db with specified BindRecord. func (c cache) removeDeletedBindRecord(hash string, meta *BindRecord) { metas, ok := c[hash] if !ok { return } for i := len(metas) - 1; i >= 0; i-- { if metas[i].isSame(meta) { metas[i] = metas[i].remove(meta) if len(metas[i].Bindings) == 0 { metas = append(metas[:i], metas[i+1:]...) } if len(metas) == 0 { delete(c, hash) return } } } c[hash] = metas } func (c cache) setBindRecord(hash string, meta *BindRecord) { metas := c[hash] for i := range metas { if metas[i].Db == meta.Db && metas[i].OriginalSQL == meta.OriginalSQL { metas[i] = meta return } } c[hash] = append(c[hash], meta) } func (c cache) copy() cache { newCache := make(cache, len(c)) for k, v := range c { bindRecords := make([]*BindRecord, len(v)) copy(bindRecords, v) newCache[k] = bindRecords } return newCache } func copyBindRecordUpdateMap(oldMap map[string]*bindRecordUpdate) map[string]*bindRecordUpdate { newMap := make(map[string]*bindRecordUpdate, len(oldMap)) for k, v := range oldMap { newMap[k] = v } return newMap } func (c cache) getBindRecord(hash, normdOrigSQL, db string) *BindRecord { bindRecords := c[hash] for _, bindRecord := range bindRecords { if bindRecord.OriginalSQL == normdOrigSQL && bindRecord.Db == db { return bindRecord } } return nil } func (h *BindHandle) deleteBindInfoSQL(normdOrigSQL, db, bindSQL string) string { return fmt.Sprintf( `DELETE FROM mysql.bind_info WHERE original_sql=%s AND default_db=%s AND bind_sql=%s`, expression.Quote(normdOrigSQL), expression.Quote(db), expression.Quote(bindSQL), ) } func (h *BindHandle) insertBindInfoSQL(orignalSQL string, db string, info Binding) string { return fmt.Sprintf(`INSERT INTO mysql.bind_info VALUES (%s, %s, %s, %s, %s, %s, %s, %s, %s)`, expression.Quote(orignalSQL), expression.Quote(info.BindSQL), expression.Quote(db), expression.Quote(info.Status), expression.Quote(info.CreateTime.String()), expression.Quote(info.UpdateTime.String()), expression.Quote(info.Charset), expression.Quote(info.Collation), expression.Quote(info.Source), ) } func (h *BindHandle) logicalDeleteBindInfoSQL(originalSQL, db string, updateTs types.Time, bindingSQL string) string { sql := fmt.Sprintf(`UPDATE mysql.bind_info SET status=%s,update_time=%s WHERE original_sql=%s and default_db=%s`, expression.Quote(deleted), expression.Quote(updateTs.String()), expression.Quote(originalSQL), expression.Quote(db)) if bindingSQL == "" { return sql } return sql + fmt.Sprintf(` and bind_sql = %s`, expression.Quote(bindingSQL)) } // CaptureBaselines is used to automatically capture plan baselines. func (h *BindHandle) CaptureBaselines() { parser4Capture := parser.New() schemas, sqls := stmtsummary.StmtSummaryByDigestMap.GetMoreThanOnceBindableStmt() for i := range sqls { stmt, err := parser4Capture.ParseOneStmt(sqls[i], "", "") if err != nil { logutil.BgLogger().Debug("parse SQL failed", zap.String("SQL", sqls[i]), zap.Error(err)) continue } if insertStmt, ok := stmt.(*ast.InsertStmt); ok && insertStmt.Select == nil { continue } normalizedSQL, digest := parser.NormalizeDigest(sqls[i]) dbName := utilparser.GetDefaultDB(stmt, schemas[i]) if r := h.GetBindRecord(digest, normalizedSQL, dbName); r != nil && r.HasUsingBinding() { continue } h.sctx.Lock() h.sctx.GetSessionVars().CurrentDB = schemas[i] oriIsolationRead := h.sctx.GetSessionVars().IsolationReadEngines // TODO: support all engines plan hint in capture baselines. h.sctx.GetSessionVars().IsolationReadEngines = map[kv.StoreType]struct{}{kv.TiKV: {}} hints, err := getHintsForSQL(h.sctx.Context, sqls[i]) h.sctx.GetSessionVars().IsolationReadEngines = oriIsolationRead h.sctx.Unlock() if err != nil { logutil.BgLogger().Debug("generate hints failed", zap.String("SQL", sqls[i]), zap.Error(err)) continue } bindSQL := GenerateBindSQL(context.TODO(), stmt, hints) if bindSQL == "" { continue } charset, collation := h.sctx.GetSessionVars().GetCharsetInfo() binding := Binding{ BindSQL: bindSQL, Status: Using, Charset: charset, Collation: collation, Source: Capture, } // We don't need to pass the `sctx` because the BindSQL has been validated already. err = h.AddBindRecord(nil, &BindRecord{OriginalSQL: normalizedSQL, Db: dbName, Bindings: []Binding{binding}}) if err != nil { logutil.BgLogger().Info("capture baseline failed", zap.String("SQL", sqls[i]), zap.Error(err)) } } } func getHintsForSQL(sctx sessionctx.Context, sql string) (string, error) { oriVals := sctx.GetSessionVars().UsePlanBaselines sctx.GetSessionVars().UsePlanBaselines = false recordSets, err := sctx.(sqlexec.SQLExecutor).ExecuteInternal(context.TODO(), fmt.Sprintf("explain format='hint' %s", sql)) sctx.GetSessionVars().UsePlanBaselines = oriVals if len(recordSets) > 0 { defer terror.Log(recordSets[0].Close()) } if err != nil { return "", err } chk := recordSets[0].NewChunk() err = recordSets[0].Next(context.TODO(), chk) if err != nil { return "", err } return chk.GetRow(0).GetString(0), nil } // GenerateBindSQL generates binding sqls from stmt node and plan hints. func GenerateBindSQL(ctx context.Context, stmtNode ast.StmtNode, planHint 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 "" } paramChecker := ¶mMarkerChecker{} 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{}) var sb strings.Builder restoreCtx := format.NewRestoreCtx(format.DefaultRestoreFlags, &sb) err := stmtNode.Restore(restoreCtx) if err != nil { logutil.Logger(ctx).Warn("Restore SQL failed", zap.Error(err)) } bindSQL := sb.String() switch n := stmtNode.(type) { case *ast.DeleteStmt: deleteIdx := strings.Index(bindSQL, "DELETE") // Remove possible `explain` prefix. bindSQL = bindSQL[deleteIdx:] return strings.Replace(bindSQL, "DELETE", fmt.Sprintf("DELETE /*+ %s*/", planHint), 1) case *ast.UpdateStmt: updateIdx := strings.Index(bindSQL, "UPDATE") // Remove possible `explain` prefix. bindSQL = bindSQL[updateIdx:] return strings.Replace(bindSQL, "UPDATE", fmt.Sprintf("UPDATE /*+ %s*/", planHint), 1) case *ast.SelectStmt: selectIdx := strings.Index(bindSQL, "SELECT") // Remove possible `explain` prefix. bindSQL = bindSQL[selectIdx:] return strings.Replace(bindSQL, "SELECT", fmt.Sprintf("SELECT /*+ %s*/", planHint), 1) case *ast.InsertStmt: insertIdx := int(0) if n.IsReplace { insertIdx = strings.Index(bindSQL, "REPLACE") } else { insertIdx = strings.Index(bindSQL, "INSERT") } // Remove possible `explain` prefix. bindSQL = bindSQL[insertIdx:] return strings.Replace(bindSQL, "SELECT", fmt.Sprintf("SELECT /*+ %s*/", planHint), 1) } logutil.Logger(ctx).Warn("Unexpected statement type") 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 (e *paramMarkerChecker) Leave(in ast.Node) (ast.Node, bool) { return in, true } // AddEvolvePlanTask adds the evolve plan task into memory cache. It would be flushed to store periodically. func (h *BindHandle) AddEvolvePlanTask(originalSQL, DB string, binding Binding) { br := &BindRecord{ OriginalSQL: originalSQL, Db: DB, Bindings: []Binding{binding}, } h.pendingVerifyBindRecordMap.Add(br) } // SaveEvolveTasksToStore saves the evolve task into store. func (h *BindHandle) SaveEvolveTasksToStore() { h.pendingVerifyBindRecordMap.flushToStore() } func getEvolveParameters(ctx sessionctx.Context) (time.Duration, time.Time, time.Time, error) { sql := fmt.Sprintf("select variable_name, variable_value from mysql.global_variables where variable_name in ('%s', '%s', '%s')", variable.TiDBEvolvePlanTaskMaxTime, variable.TiDBEvolvePlanTaskStartTime, variable.TiDBEvolvePlanTaskEndTime) rows, _, err := ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(sql) if err != nil { return 0, time.Time{}, time.Time{}, err } maxTime, startTimeStr, endTimeStr := int64(variable.DefTiDBEvolvePlanTaskMaxTime), variable.DefTiDBEvolvePlanTaskStartTime, variable.DefAutoAnalyzeEndTime for _, row := range rows { switch row.GetString(0) { case variable.TiDBEvolvePlanTaskMaxTime: maxTime, err = strconv.ParseInt(row.GetString(1), 10, 64) if err != nil { return 0, time.Time{}, time.Time{}, err } case variable.TiDBEvolvePlanTaskStartTime: startTimeStr = row.GetString(1) case variable.TiDBEvolvePlanTaskEndTime: endTimeStr = row.GetString(1) } } startTime, err := time.ParseInLocation(variable.FullDayTimeFormat, startTimeStr, time.UTC) if err != nil { return 0, time.Time{}, time.Time{}, err } endTime, err := time.ParseInLocation(variable.FullDayTimeFormat, endTimeStr, time.UTC) if err != nil { return 0, time.Time{}, time.Time{}, err } return time.Duration(maxTime) * time.Second, startTime, endTime, nil } const ( // acceptFactor is the factor to decide should we accept the pending verified plan. // A pending verified plan will be accepted if it performs at least `acceptFactor` times better than the accepted plans. acceptFactor = 1.5 // verifyTimeoutFactor is how long to wait to verify the pending plan. // For debugging purposes it is useful to wait a few times longer than the current execution time so that // an informative error can be written to the log. verifyTimeoutFactor = 2.0 // nextVerifyDuration is the duration that we will retry the rejected plans. nextVerifyDuration = 7 * 24 * time.Hour ) func (h *BindHandle) getOnePendingVerifyJob() (string, string, Binding) { cache := h.bindInfo.Value.Load().(cache) for _, bindRecords := range cache { for _, bindRecord := range bindRecords { for _, bind := range bindRecord.Bindings { if bind.Status == PendingVerify { return bindRecord.OriginalSQL, bindRecord.Db, bind } if bind.Status != Rejected { continue } dur, err := bind.SinceUpdateTime() // Should not happen. if err != nil { continue } // Rejected and retry it now. if dur > nextVerifyDuration { return bindRecord.OriginalSQL, bindRecord.Db, bind } } } } return "", "", Binding{} } func (h *BindHandle) getRunningDuration(sctx sessionctx.Context, db, sql string, maxTime time.Duration) (time.Duration, error) { ctx := context.TODO() if db != "" { _, err := sctx.(sqlexec.SQLExecutor).ExecuteInternal(ctx, fmt.Sprintf("use `%s`", db)) if err != nil { return 0, err } } ctx, cancelFunc := context.WithCancel(ctx) timer := time.NewTimer(maxTime) resultChan := make(chan error) startTime := time.Now() go runSQL(ctx, sctx, sql, resultChan) select { case err := <-resultChan: cancelFunc() if err != nil { return 0, err } return time.Since(startTime), nil case <-timer.C: cancelFunc() logutil.BgLogger().Warn("plan verification timed out", zap.Duration("timeElapsed", time.Since(startTime))) } <-resultChan return -1, nil } func runSQL(ctx context.Context, sctx sessionctx.Context, sql string, resultChan chan<- error) { defer func() { if r := recover(); r != nil { buf := make([]byte, 4096) stackSize := runtime.Stack(buf, false) buf = buf[:stackSize] resultChan <- fmt.Errorf("run sql panicked: %v", string(buf)) } }() recordSets, err := sctx.(sqlexec.SQLExecutor).ExecuteInternal(ctx, sql) if err != nil { if len(recordSets) > 0 { terror.Call(recordSets[0].Close) } resultChan <- err return } recordSet := recordSets[0] chk := recordSets[0].NewChunk() for { err = recordSet.Next(ctx, chk) if err != nil || chk.NumRows() == 0 { break } } terror.Call(recordSets[0].Close) resultChan <- err } // HandleEvolvePlanTask tries to evolve one plan task. // It only handle one tasks once because we want each task could use the latest parameters. func (h *BindHandle) HandleEvolvePlanTask(sctx sessionctx.Context, adminEvolve bool) error { originalSQL, db, binding := h.getOnePendingVerifyJob() if originalSQL == "" { return nil } maxTime, startTime, endTime, err := getEvolveParameters(sctx) if err != nil { return err } if maxTime == 0 || (!timeutil.WithinDayTimePeriod(startTime, endTime, time.Now()) && !adminEvolve) { return nil } sctx.GetSessionVars().UsePlanBaselines = true currentPlanTime, err := h.getRunningDuration(sctx, db, binding.BindSQL, maxTime) // If we just return the error to the caller, this job will be retried again and again and cause endless logs, // since it is still in the bind record. Now we just drop it and if it is actually retryable, // we will hope for that we can capture this evolve task again. if err != nil { return h.DropBindRecord(originalSQL, db, &binding) } // If the accepted plan timeouts, it is hard to decide the timeout for verify plan. // Currently we simply mark the verify plan as `using` if it could run successfully within maxTime. if currentPlanTime > 0 { maxTime = time.Duration(float64(currentPlanTime) * verifyTimeoutFactor) } sctx.GetSessionVars().UsePlanBaselines = false verifyPlanTime, err := h.getRunningDuration(sctx, db, binding.BindSQL, maxTime) if err != nil { return h.DropBindRecord(originalSQL, db, &binding) } if verifyPlanTime == -1 || (float64(verifyPlanTime)*acceptFactor > float64(currentPlanTime)) { binding.Status = Rejected digestText, _ := parser.NormalizeDigest(binding.BindSQL) // for log desensitization logutil.BgLogger().Warn("new plan rejected", zap.Duration("currentPlanTime", currentPlanTime), zap.Duration("verifyPlanTime", verifyPlanTime), zap.String("digestText", digestText), ) } else { binding.Status = Using } // We don't need to pass the `sctx` because the BindSQL has been validated already. return h.AddBindRecord(nil, &BindRecord{OriginalSQL: originalSQL, Db: db, Bindings: []Binding{binding}}) } // Clear resets the bind handle. It is only used for test. func (h *BindHandle) Clear() { h.bindInfo.Lock() h.bindInfo.Store(make(cache)) h.bindInfo.lastUpdateTime = types.ZeroTimestamp h.bindInfo.Unlock() h.invalidBindRecordMap.Store(make(map[string]*bindRecordUpdate)) h.pendingVerifyBindRecordMap.Store(make(map[string]*bindRecordUpdate)) } // FlushBindings flushes the BindRecord in temp maps to storage and loads them into cache. func (h *BindHandle) FlushBindings() error { h.DropInvalidBindRecord() h.SaveEvolveTasksToStore() return h.Update(false) } // ReloadBindings clears existing binding cache and do a full load from mysql.bind_info. // It is used to maintain consistency between cache and mysql.bind_info if the table is deleted or truncated. func (h *BindHandle) ReloadBindings() error { h.bindInfo.Lock() h.bindInfo.Store(make(cache)) h.bindInfo.lastUpdateTime = types.ZeroTimestamp h.bindInfo.Unlock() return h.Update(true) }