diff --git a/bindinfo/bind.go b/bindinfo/bind.go index 41719749bd559..2f2250d371442 100644 --- a/bindinfo/bind.go +++ b/bindinfo/bind.go @@ -25,22 +25,30 @@ type hintProcessor struct { *HintsSet // bindHint2Ast indicates the behavior of the processor, `true` for bind hint to ast, `false` for extract hint from ast. bindHint2Ast bool - tableCounter int64 - indexCounter int64 + tableCounter int + indexCounter int } func (hp *hintProcessor) Enter(in ast.Node) (ast.Node, bool) { switch v := in.(type) { case *ast.SelectStmt: if hp.bindHint2Ast { - v.TableHints = hp.tableHints[hp.tableCounter] + if hp.tableCounter < len(hp.tableHints) { + v.TableHints = hp.tableHints[hp.tableCounter] + } else { + v.TableHints = nil + } hp.tableCounter++ } else { hp.tableHints = append(hp.tableHints, v.TableHints) } case *ast.TableName: if hp.bindHint2Ast { - v.IndexHints = hp.indexHints[hp.indexCounter] + if hp.indexCounter < len(hp.indexHints) { + v.IndexHints = hp.indexHints[hp.indexCounter] + } else { + v.IndexHints = nil + } hp.indexCounter++ } else { hp.indexHints = append(hp.indexHints, v.IndexHints) diff --git a/bindinfo/bind_test.go b/bindinfo/bind_test.go index 167fa4898f5f6..32ca0cfcbf49f 100644 --- a/bindinfo/bind_test.go +++ b/bindinfo/bind_test.go @@ -460,7 +460,7 @@ func (s *testSuite) TestUseMultiplyBindings(c *C) { tk.MustExec("analyze table t") tk.MustExec("create binding for select * from t where a >= 1 and b >= 1 and c = 0 using select * from t use index(idx_a) where a >= 1 and b >= 1 and c = 0") tk.MustExec("create binding for select * from t where a >= 1 and b >= 1 and c = 0 using select * from t use index(idx_b) where a >= 1 and b >= 1 and c = 0") - // It cannot choose `idx_c` although it has lowest cost. + // It cannot choose table path although it has lowest cost. tk.MustQuery("select * from t where a >= 4 and b >= 1 and c = 0") c.Assert(tk.Se.GetSessionVars().StmtCtx.IndexNames[0], Equals, "t:idx_a") tk.MustQuery("select * from t where a >= 1 and b >= 4 and c = 0") @@ -504,3 +504,23 @@ func (s *testSuite) TestDropSingleBindings(c *C) { rows = tk.MustQuery("show global bindings").Rows() c.Assert(len(rows), Equals, 0) } + +func (s *testSuite) TestAddEvolveTasks(c *C) { + tk := testkit.NewTestKit(c, s.store) + s.cleanBindingEnv(tk) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b int, c int, index idx_a(a), index idx_b(b), index idx_c(c))") + tk.MustExec("insert into t values (1,1,1), (2,2,2), (3,3,3), (4,4,4), (5,5,5)") + tk.MustExec("analyze table t") + tk.MustExec("create global binding for select * from t where a >= 1 and b >= 1 and c = 0 using select * from t use index(idx_a) where a >= 1 and b >= 1 and c = 0") + tk.MustExec("set @@tidb_evolve_plan_baselines=1") + // It cannot choose table path although it has lowest cost. + tk.MustQuery("select * from t where a >= 4 and b >= 1 and c = 0") + c.Assert(tk.Se.GetSessionVars().StmtCtx.IndexNames[0], Equals, "t:idx_a") + domain.GetDomain(tk.Se).BindHandle().SaveEvolveTasksToStore() + rows := tk.MustQuery("show global bindings").Rows() + c.Assert(len(rows), Equals, 2) + c.Assert(rows[1][1], Equals, "SELECT /*+ USE_INDEX(@`sel_1` `test`.`t` )*/ * FROM `test`.`t` WHERE `a`>=4 AND `b`>=1 AND `c`=0") + c.Assert(rows[1][3], Equals, "pending verify") +} diff --git a/bindinfo/cache.go b/bindinfo/cache.go index a6389a14fb19f..69ea15915fd0e 100644 --- a/bindinfo/cache.go +++ b/bindinfo/cache.go @@ -31,6 +31,8 @@ const ( deleted = "deleted" // Invalid is the bind info's invalid status. Invalid = "invalid" + // PendingVerify means the bind info needs to be verified. + PendingVerify = "pending verify" ) // Binding stores the basic bind hint info. @@ -47,6 +49,7 @@ type Binding struct { // Hint is the parsed hints, it is used to bind hints to stmt node. Hint *HintsSet // id is the string form of all hints. It is used to uniquely identify different hints. + // It would be non-empty only when the status is `Using` or `PendingVerify`. id string } @@ -71,10 +74,10 @@ func (br *BindRecord) HasUsingBinding() bool { return false } -// FindUsingBinding find bindings with status `Using` in BindRecord. -func (br *BindRecord) FindUsingBinding(hint string) *Binding { +// FindBinding find bindings in BindRecord. +func (br *BindRecord) FindBinding(hint string) *Binding { for _, binding := range br.Bindings { - if binding.Status == Using && binding.id == hint { + if binding.id == hint { return &binding } } @@ -84,7 +87,7 @@ func (br *BindRecord) FindUsingBinding(hint string) *Binding { func (br *BindRecord) prepareHints(sctx sessionctx.Context, is infoschema.InfoSchema) error { p := parser.New() for i, bind := range br.Bindings { - if bind.Hint != nil { + if bind.Hint != nil || bind.id != "" { continue } stmtNode, err := p.ParseOneStmt(bind.BindSQL, bind.Charset, bind.Collation) diff --git a/bindinfo/handle.go b/bindinfo/handle.go index 4d9196d0d6a44..e2cd6d495e234 100644 --- a/bindinfo/handle.go +++ b/bindinfo/handle.go @@ -70,10 +70,10 @@ type BindHandle struct { // 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 struct { - sync.Mutex - atomic.Value - } + invalidBindRecordMap tmpBindRecordMap + + // pendingVerifyBindRecordMap indicates the pending verify bind records that found during query. + pendingVerifyBindRecordMap tmpBindRecordMap lastUpdateTime types.Time @@ -83,9 +83,9 @@ type BindHandle struct { // Lease influences the duration of loading bind info and handling invalid bind. var Lease = 3 * time.Second -type invalidBindRecordMap struct { - bindRecord *BindRecord - droppedTime time.Time +type bindRecordUpdate struct { + bindRecord *BindRecord + updateTime time.Time } // NewBindHandle creates a new BindHandle. @@ -95,7 +95,18 @@ func NewBindHandle(ctx sessionctx.Context) *BindHandle { handle.bindInfo.Value.Store(make(cache, 32)) handle.bindInfo.parser = parser.New() handle.parser4Baseline = parser.New() - handle.invalidBindRecordMap.Value.Store(make(map[string]*invalidBindRecordMap)) + handle.invalidBindRecordMap.Value.Store(make(map[string]*bindRecordUpdate)) + handle.invalidBindRecordMap.flushFunc = func(record *BindRecord) error { + // We do not need the first two parameters because they are only use to generate hint, + // and we already have the hint. + return handle.DropBindRecord(nil, nil, record) + } + handle.pendingVerifyBindRecordMap.Value.Store(make(map[string]*bindRecordUpdate)) + handle.pendingVerifyBindRecordMap.flushFunc = func(record *BindRecord) error { + // We do not need the first two parameters because they are only use to generate hint, + // and we already have the hint. + return handle.AddBindRecord(nil, nil, record) + } return handle } @@ -150,6 +161,21 @@ func (h *BindHandle) AddBindRecord(sctx sessionctx.Context, is infoschema.InfoSc if err != nil { return err } + + br := h.GetBindRecord(parser.DigestHash(record.OriginalSQL), record.OriginalSQL, record.Db) + var duplicateBinding string + if br != nil { + binding := br.FindBinding(record.Bindings[0].id) + if binding != nil { + // There is already a binding with status `Using` or `PendingVerify`, we could directly cancel the job. + if record.Bindings[0].Status == PendingVerify { + return nil + } + // Otherwise, we need to remove it before insert. + duplicateBinding = binding.BindSQL + } + } + exec, _ := h.sctx.Context.(sqlexec.SQLExecutor) h.sctx.Lock() _, err = exec.Execute(context.TODO(), "BEGIN") @@ -179,18 +205,10 @@ func (h *BindHandle) AddBindRecord(sctx sessionctx.Context, is infoschema.InfoSc h.bindInfo.Unlock() }() - oldBindRecord := h.GetBindRecord(parser.DigestHash(record.OriginalSQL), record.OriginalSQL, record.Db) - if oldBindRecord != nil { - for _, newBinding := range record.Bindings { - binding := oldBindRecord.FindUsingBinding(newBinding.id) - if binding == nil { - continue - } - // Remove duplicates before insert. - _, err = exec.Execute(context.TODO(), h.deleteBindInfoSQL(record.OriginalSQL, record.Db, binding.BindSQL)) - if err != nil { - return err - } + if duplicateBinding != "" { + _, err = exec.Execute(context.TODO(), h.deleteBindInfoSQL(record.OriginalSQL, record.Db, duplicateBinding)) + if err != nil { + return err } } @@ -205,7 +223,6 @@ func (h *BindHandle) AddBindRecord(sctx sessionctx.Context, is infoschema.InfoSc Fsp: 3, } record.Bindings[i].UpdateTime = record.Bindings[0].CreateTime - record.Bindings[i].Status = Using // insert the BindRecord to the storage. _, err = exec.Execute(context.TODO(), h.insertBindInfoSQL(record.OriginalSQL, record.Db, record.Bindings[i])) @@ -266,7 +283,7 @@ func (h *BindHandle) DropBindRecord(sctx sessionctx.Context, is infoschema.InfoS if oldBindRecord == nil { continue } - binding := oldBindRecord.FindUsingBinding(record.Bindings[i].id) + binding := oldBindRecord.FindBinding(record.Bindings[i].id) if binding != nil { bindingSQLs = append(bindingSQLs, binding.BindSQL) } @@ -276,44 +293,60 @@ func (h *BindHandle) DropBindRecord(sctx sessionctx.Context, is infoschema.InfoS return err } -// DropInvalidBindRecord execute the drop bindRecord task. -func (h *BindHandle) DropInvalidBindRecord() { - invalidBindRecordMap := copyInvalidBindRecordMap(h.invalidBindRecordMap.Load().(map[string]*invalidBindRecordMap)) - for key, invalidBindRecord := range invalidBindRecordMap { - if invalidBindRecord.droppedTime.IsZero() { - err := h.DropBindRecord(nil, nil, invalidBindRecord.bindRecord) +// 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 +} + +func (tmpMap *tmpBindRecordMap) flushToStore() { + 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("DropInvalidBindRecord failed", zap.Error(err)) + logutil.BgLogger().Error("flush bind record failed", zap.Error(err)) } - invalidBindRecord.droppedTime = time.Now() + bindRecord.updateTime = time.Now() continue } - if time.Since(invalidBindRecord.droppedTime) > 6*time.Second { - delete(invalidBindRecordMap, key) - updateMetrics(metrics.ScopeGlobal, invalidBindRecord.bindRecord, nil, false) + if time.Since(bindRecord.updateTime) > 6*time.Second { + delete(newMap, key) + updateMetrics(metrics.ScopeGlobal, bindRecord.bindRecord, nil, false) } } - h.invalidBindRecordMap.Store(invalidBindRecordMap) + tmpMap.Store(newMap) } -// AddDropInvalidBindTask add bindRecord to invalidBindRecordMap when the bindRecord need to be deleted. -func (h *BindHandle) AddDropInvalidBindTask(invalidBindRecord *BindRecord) { - key := invalidBindRecord.OriginalSQL + ":" + invalidBindRecord.Db - if _, ok := h.invalidBindRecordMap.Value.Load().(map[string]*invalidBindRecordMap)[key]; ok { +func (tmpMap *tmpBindRecordMap) saveToCache(bindRecord *BindRecord) { + key := bindRecord.OriginalSQL + ":" + bindRecord.Db + ":" + bindRecord.Bindings[0].id + if _, ok := tmpMap.Load().(map[string]*bindRecordUpdate)[key]; ok { return } - h.invalidBindRecordMap.Lock() - defer h.invalidBindRecordMap.Unlock() - if _, ok := h.invalidBindRecordMap.Value.Load().(map[string]*invalidBindRecordMap)[key]; ok { + tmpMap.Lock() + defer tmpMap.Unlock() + if _, ok := tmpMap.Load().(map[string]*bindRecordUpdate)[key]; ok { return } - newMap := copyInvalidBindRecordMap(h.invalidBindRecordMap.Value.Load().(map[string]*invalidBindRecordMap)) - newMap[key] = &invalidBindRecordMap{ - bindRecord: invalidBindRecord, + newMap := copyBindRecordUpdateMap(tmpMap.Load().(map[string]*bindRecordUpdate)) + newMap[key] = &bindRecordUpdate{ + bindRecord: bindRecord, } - h.invalidBindRecordMap.Store(newMap) - updateMetrics(metrics.ScopeGlobal, nil, invalidBindRecord, false) + tmpMap.Store(newMap) + updateMetrics(metrics.ScopeGlobal, nil, bindRecord, false) +} + +// DropInvalidBindRecord execute the drop bindRecord task. +func (h *BindHandle) DropInvalidBindRecord() { + h.invalidBindRecordMap.flushToStore() +} + +// AddDropInvalidBindTask add bindRecord to invalidBindRecordMap when the bindRecord need to be deleted. +func (h *BindHandle) AddDropInvalidBindTask(invalidBindRecord *BindRecord) { + h.invalidBindRecordMap.saveToCache(invalidBindRecord) } // Size return the size of bind info cache. @@ -360,6 +393,7 @@ func (h *BindHandle) newBindRecord(row chunk.Row) (string, *BindRecord, error) { if err != nil { return "", nil, err } + h.sctx.GetSessionVars().StmtCtx.TimeZone = h.sctx.GetSessionVars().TimeZone h.sctx.GetSessionVars().CurrentDB = bindRecord.Db err = bindRecord.prepareHints(h.sctx.Context, h.sctx.GetSessionVars().TxnCtx.InfoSchema.(infoschema.InfoSchema)) return hash, bindRecord, err @@ -431,8 +465,8 @@ func (c cache) copy() cache { return newCache } -func copyInvalidBindRecordMap(oldMap map[string]*invalidBindRecordMap) map[string]*invalidBindRecordMap { - newMap := make(map[string]*invalidBindRecordMap, len(oldMap)) +func copyBindRecordUpdateMap(oldMap map[string]*bindRecordUpdate) map[string]*bindRecordUpdate { + newMap := make(map[string]*bindRecordUpdate, len(oldMap)) for k, v := range oldMap { newMap[k] = v } @@ -453,7 +487,7 @@ func (c cache) getBindRecord(hash, normdOrigSQL, db string) *BindRecord { 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`, + `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), @@ -536,9 +570,25 @@ func (h *BindHandle) CaptureBaselines() { } } +// 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, planHint string) { + binding.id = planHint + br := &BindRecord{ + OriginalSQL: originalSQL, + Db: DB, + Bindings: []Binding{binding}, + } + h.pendingVerifyBindRecordMap.saveToCache(br) +} + +// SaveEvolveTasksToStore saves the evolve task into store. +func (h *BindHandle) SaveEvolveTasksToStore() { + h.pendingVerifyBindRecordMap.flushToStore() +} + // Clear resets the bind handle. It is used for test. func (h *BindHandle) Clear() { h.bindInfo.Store(make(cache)) - h.invalidBindRecordMap.Store(make(map[string]*invalidBindRecordMap)) + h.invalidBindRecordMap.Store(make(map[string]*bindRecordUpdate)) h.lastUpdateTime = types.ZeroTimestamp } diff --git a/domain/domain.go b/domain/domain.go index 4f46f4562b483..5ebe342804a8d 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -863,6 +863,7 @@ func (do *Domain) globalBindHandleWorkerLoop() { continue } do.bindHandle.CaptureBaselines() + do.bindHandle.SaveEvolveTasksToStore() } } }() diff --git a/planner/optimize.go b/planner/optimize.go index d8c9296b9be70..47961882ac6fd 100644 --- a/planner/optimize.go +++ b/planner/optimize.go @@ -15,12 +15,14 @@ package planner import ( "context" + "fmt" "math" "strings" "github.com/pingcap/errors" "github.com/pingcap/parser" "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/format" "github.com/pingcap/tidb/bindinfo" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/infoschema" @@ -30,7 +32,9 @@ import ( "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" + driver "github.com/pingcap/tidb/types/parser_driver" "github.com/pingcap/tidb/util/logutil" + "go.uber.org/zap" ) // Optimize does optimization and creates a Plan. @@ -47,8 +51,11 @@ func Optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is in sctx.PrepareTxnFuture(ctx) bestPlan, names, _, err := optimize(ctx, sctx, node, is) - if err != nil || !sctx.GetSessionVars().UsePlanBaselines { - return bestPlan, names, err + if err != nil { + return nil, nil, err + } + if !(sctx.GetSessionVars().UsePlanBaselines || sctx.GetSessionVars().EvolvePlanBaselines) { + return bestPlan, names, nil } stmtNode, ok := node.(ast.StmtNode) if !ok { @@ -59,8 +66,9 @@ func Optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is in return bestPlan, names, nil } bestPlanHint := plannercore.GenHintsFromPhysicalPlan(bestPlan) + binding := bindRecord.FindBinding(bestPlanHint) // If the best bestPlan is in baselines, just use it. - if bindRecord.FindUsingBinding(bestPlanHint) != nil { + if binding != nil && binding.Status == bindinfo.Using { return bestPlan, names, nil } bestCostAmongHints := math.MaxFloat64 @@ -88,10 +96,13 @@ func Optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is in bestPlanAmongHints = plan } } + // If there is already a evolution task, we do not need to handle it again. + if sctx.GetSessionVars().EvolvePlanBaselines && binding == nil { + handleEvolveTasks(ctx, sctx, bindRecord, stmtNode, bestPlanHint) + } // Restore the hint to avoid changing the stmt node. bindinfo.BindHint(stmtNode, originHints) - // TODO: Evolve the plan baselines using best plan. - if bestPlanAmongHints != nil { + if sctx.GetSessionVars().UsePlanBaselines && bestPlanAmongHints != nil { return bestPlanAmongHints, names, nil } return bestPlan, names, nil @@ -206,6 +217,50 @@ func handleInvalidBindRecord(ctx context.Context, sctx sessionctx.Context, level globalHandle.AddDropInvalidBindTask(&bindRecord) } +func handleEvolveTasks(ctx context.Context, sctx sessionctx.Context, br *bindinfo.BindRecord, stmtNode ast.StmtNode, planHint 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. + bindinfo.BindHint(stmtNode, &bindinfo.HintsSet{}) + var sb strings.Builder + restoreCtx := format.NewRestoreCtx(format.DefaultRestoreFlags, &sb) + err := stmtNode.Restore(restoreCtx) + if err != nil { + logutil.Logger(ctx).Info("Restore SQL failed", zap.Error(err)) + } + bindsql := strings.Replace(sb.String(), "SELECT", fmt.Sprintf("SELECT /*+ %s*/", planHint), 1) + globalHandle := domain.GetDomain(sctx).BindHandle() + charset, collation := sctx.GetSessionVars().GetCharsetInfo() + binding := bindinfo.Binding{BindSQL: bindsql, Status: bindinfo.PendingVerify, Charset: charset, Collation: collation} + globalHandle.AddEvolvePlanTask(br.OriginalSQL, br.Db, binding, planHint) +} + +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 +} + // isPointGetWithoutDoubleRead returns true when meets following conditions: // 1. ctx is auto commit tagged. // 2. plan is point get by pk. diff --git a/session/session.go b/session/session.go index 70fec5373d495..b9ab74336b2f1 100644 --- a/session/session.go +++ b/session/session.go @@ -1839,6 +1839,7 @@ var builtinGlobalVariable = []string{ variable.TiDBMaxDeltaSchemaCount, variable.TiDBCapturePlanBaseline, variable.TiDBUsePlanBaselines, + variable.TiDBEvolvePlanBaselines, variable.TiDBIsolationReadEngines, } diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index ec3a59511cb9c..e009ec0f0e7e8 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -432,6 +432,9 @@ type SessionVars struct { // UsePlanBaselines indicates whether we will use plan baselines to adjust plan. UsePlanBaselines bool + // EvolvePlanBaselines indicates whether we will evolve the plan baselines. + EvolvePlanBaselines bool + // Unexported fields should be accessed and set through interfaces like GetReplicaRead() and SetReplicaRead(). // allowInSubqToJoinAndAgg can be set to false to forbid rewriting the semi join to inner join with agg. @@ -527,6 +530,7 @@ func NewSessionVars() *SessionVars { replicaRead: kv.ReplicaReadLeader, AllowRemoveAutoInc: DefTiDBAllowRemoveAutoInc, UsePlanBaselines: DefTiDBUsePlanBaselines, + EvolvePlanBaselines: DefTiDBEvolvePlanBaselines, isolationReadEngines: map[kv.StoreType]struct{}{kv.TiKV: {}, kv.TiFlash: {}}, LockWaitTimeout: DefInnodbLockWaitTimeout * 1000, } @@ -973,6 +977,8 @@ func (s *SessionVars) SetSystemVar(name string, val string) error { SetMaxDeltaSchemaCount(tidbOptInt64(val, DefTiDBMaxDeltaSchemaCount)) case TiDBUsePlanBaselines: s.UsePlanBaselines = TiDBOptOn(val) + case TiDBEvolvePlanBaselines: + s.EvolvePlanBaselines = TiDBOptOn(val) case TiDBIsolationReadEngines: s.isolationReadEngines = make(map[kv.StoreType]struct{}) for _, engine := range strings.Split(val, ",") { diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 3954d7f6027d1..4d85ed00c420f 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -718,6 +718,7 @@ var defaultSysVars = []*SysVar{ {ScopeGlobal | ScopeSession, TiDBEnableStmtSummary, "0"}, {ScopeGlobal | ScopeSession, TiDBCapturePlanBaseline, "0"}, {ScopeGlobal | ScopeSession, TiDBUsePlanBaselines, BoolToIntStr(DefTiDBUsePlanBaselines)}, + {ScopeGlobal | ScopeSession, TiDBEvolvePlanBaselines, BoolToIntStr(DefTiDBEvolvePlanBaselines)}, {ScopeGlobal | ScopeSession, TiDBIsolationReadEngines, "tikv,tiflash"}, } diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index a095485cdd1f2..4c9a73b429c8b 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -334,6 +334,9 @@ const ( // TiDBUsePlanBaselines indicates whether the use of plan baselines is enabled. TiDBUsePlanBaselines = "tidb_use_plan_baselines" + // TiDBEvolvePlanBaselines indicates whether the evolution of plan baselines is enabled. + TiDBEvolvePlanBaselines = "tidb_evolve_plan_baselines" + // TiDBIsolationReadEngines indicates the tidb only read from the stores whose engine type is involved in IsolationReadEngines. // Now, only support TiKV and TiFlash. TiDBIsolationReadEngines = "tidb_isolation_read_engines" @@ -413,6 +416,7 @@ const ( DefTiDBEnableNoopFuncs = false DefTiDBAllowRemoveAutoInc = false DefTiDBUsePlanBaselines = true + DefTiDBEvolvePlanBaselines = false DefInnodbLockWaitTimeout = 50 // 50s ) diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index b6fd23d72eae4..f1d077b831013 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -608,7 +608,7 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string) (string, default: return value, ErrWrongValueForVar.GenWithStackByArgs(TiDBTxnMode, value) } - case TiDBAllowRemoveAutoInc, TiDBUsePlanBaselines: + case TiDBAllowRemoveAutoInc, TiDBUsePlanBaselines, TiDBEvolvePlanBaselines: switch { case strings.EqualFold(value, "ON") || value == "1": return "on", nil