diff --git a/pkg/bindinfo/bind_cache.go b/pkg/bindinfo/bind_cache.go index 83b9086af94f0..92ed09d43e67f 100644 --- a/pkg/bindinfo/bind_cache.go +++ b/pkg/bindinfo/bind_cache.go @@ -131,10 +131,10 @@ func (c *bindCache) delete(key bindCacheKey) bool { return false } -// GetBindRecord gets the BindRecord from the cache. +// GetBinding gets the BindRecord from the cache. // The return value is not read-only, but it shouldn't be changed in the caller functions. // The function is thread-safe. -func (c *bindCache) GetBindRecord(sqlDigest, normalizedSQL, _ string) *BindRecord { +func (c *bindCache) GetBinding(sqlDigest, normalizedSQL, _ string) *BindRecord { c.lock.Lock() defer c.lock.Unlock() bindRecords := c.get(bindCacheKey(sqlDigest)) @@ -146,10 +146,10 @@ func (c *bindCache) GetBindRecord(sqlDigest, normalizedSQL, _ string) *BindRecor return nil } -// GetBindRecordBySQLDigest gets the BindRecord from the cache. +// GetBindingBySQLDigest gets the BindRecord from the cache. // The return value is not read-only, but it shouldn't be changed in the caller functions. // The function is thread-safe. -func (c *bindCache) GetBindRecordBySQLDigest(sqlDigest string) (*BindRecord, error) { +func (c *bindCache) GetBindingBySQLDigest(sqlDigest string) (*BindRecord, error) { c.lock.Lock() defer c.lock.Unlock() bindings := c.get(bindCacheKey(sqlDigest)) @@ -163,10 +163,10 @@ func (c *bindCache) GetBindRecordBySQLDigest(sqlDigest string) (*BindRecord, err return bindings[0], nil } -// GetAllBindRecords return all the bindRecords from the bindCache. +// GetAllBindings return all the bindRecords from the bindCache. // The return value is not read-only, but it shouldn't be changed in the caller functions. // The function is thread-safe. -func (c *bindCache) GetAllBindRecords() []*BindRecord { +func (c *bindCache) GetAllBindings() []*BindRecord { c.lock.Lock() defer c.lock.Unlock() values := c.cache.Values() @@ -178,9 +178,9 @@ func (c *bindCache) GetAllBindRecords() []*BindRecord { return bindRecords } -// SetBindRecord sets the BindRecord to the cache. +// SetBinding sets the BindRecord to the cache. // The function is thread-safe. -func (c *bindCache) SetBindRecord(sqlDigest string, meta *BindRecord) (err error) { +func (c *bindCache) SetBinding(sqlDigest string, meta *BindRecord) (err error) { c.lock.Lock() defer c.lock.Unlock() cacheKey := bindCacheKey(sqlDigest) @@ -194,9 +194,9 @@ func (c *bindCache) SetBindRecord(sqlDigest string, meta *BindRecord) (err error return } -// RemoveBindRecord removes the BindRecord which has same originSQL with specified BindRecord. +// RemoveBinding removes the BindRecord which has same originSQL with specified BindRecord. // The function is thread-safe. -func (c *bindCache) RemoveBindRecord(sqlDigest string, meta *BindRecord) { +func (c *bindCache) RemoveBinding(sqlDigest string, meta *BindRecord) { c.lock.Lock() defer c.lock.Unlock() metas := c.getCopiedVal(bindCacheKey(sqlDigest)) diff --git a/pkg/bindinfo/capture_test.go b/pkg/bindinfo/capture_test.go index b65ae573f627d..9d4abe3d39822 100644 --- a/pkg/bindinfo/capture_test.go +++ b/pkg/bindinfo/capture_test.go @@ -329,7 +329,7 @@ func TestBindingSource(t *testing.T) { tk.MustExec("create global binding for select * from t where a > 10 using select * from t ignore index(idx_a) where a > 10") bindHandle := dom.BindHandle() sql, sqlDigest := internal.UtilNormalizeWithDefaultDB(t, "select * from t where a > ?") - bindData := bindHandle.GetBindRecord(sqlDigest, sql, "test") + bindData := bindHandle.GetGlobalBinding(sqlDigest, sql, "test") require.NotNil(t, bindData) require.Equal(t, "select * from `test` . `t` where `a` > ?", bindData.OriginalSQL) require.Len(t, bindData.Bindings, 1) @@ -341,7 +341,7 @@ func TestBindingSource(t *testing.T) { tk.MustQuery("select * from t where a > 10") bindHandle.SaveEvolveTasksToStore() sql, sqlDigest = internal.UtilNormalizeWithDefaultDB(t, "select * from t where a > ?") - bindData = bindHandle.GetBindRecord(sqlDigest, sql, "test") + bindData = bindHandle.GetGlobalBinding(sqlDigest, sql, "test") require.NotNil(t, bindData) require.Equal(t, "select * from `test` . `t` where `a` > ?", bindData.OriginalSQL) require.Len(t, bindData.Bindings, 2) @@ -362,7 +362,7 @@ func TestBindingSource(t *testing.T) { tk.MustExec("admin capture bindings") bindHandle.CaptureBaselines() sql, sqlDigest = internal.UtilNormalizeWithDefaultDB(t, "select * from t where a < ?") - bindData = bindHandle.GetBindRecord(sqlDigest, sql, "test") + bindData = bindHandle.GetGlobalBinding(sqlDigest, sql, "test") require.NotNil(t, bindData) require.Equal(t, "select * from `test` . `t` where `a` < ?", bindData.OriginalSQL) require.Len(t, bindData.Bindings, 1) diff --git a/pkg/bindinfo/handle.go b/pkg/bindinfo/handle.go index 8a549c4464521..5932ad93a9fab 100644 --- a/pkg/bindinfo/handle.go +++ b/pkg/bindinfo/handle.go @@ -77,7 +77,6 @@ type BindHandle struct { bindInfo struct { sync.Mutex atomic.Value - parser *parser.Parser lastUpdateTime types.Time } @@ -127,16 +126,15 @@ func (h *BindHandle) Reset(ctx sessionctx.Context) { defer h.bindInfo.Unlock() h.sctx.Context = ctx h.bindInfo.Value.Store(newBindCache()) - h.bindInfo.parser = parser.New() h.invalidBindRecordMap.Value.Store(make(map[string]*bindRecordUpdate)) h.invalidBindRecordMap.flushFunc = func(record *BindRecord) error { - _, err := h.DropBindRecord(record.OriginalSQL, record.Db, &record.Bindings[0]) + _, err := h.DropGlobalBinding(record.OriginalSQL, record.Db, &record.Bindings[0]) return err } h.pendingVerifyBindRecordMap.Value.Store(make(map[string]*bindRecordUpdate)) h.pendingVerifyBindRecordMap.flushFunc = func(record *BindRecord) error { // BindSQL has already been validated when coming here, so we use nil sctx parameter. - return h.AddBindRecord(nil, record) + return h.AddGlobalBinding(nil, record) } variable.RegisterStatistics(h) } @@ -194,17 +192,17 @@ func (h *BindHandle) Update(fullLoad bool) (err error) { continue } - oldRecord := newCache.GetBindRecord(sqlDigest, meta.OriginalSQL, meta.Db) + oldRecord := newCache.GetBinding(sqlDigest, meta.OriginalSQL, meta.Db) newRecord := merge(oldRecord, meta).removeDeletedBindings() if len(newRecord.Bindings) > 0 { - err = newCache.SetBindRecord(sqlDigest, newRecord) + err = newCache.SetBinding(sqlDigest, newRecord) if err != nil { memExceededErr = err } } else { - newCache.RemoveBindRecord(sqlDigest, newRecord) + newCache.RemoveBinding(sqlDigest, newRecord) } - updateMetrics(metrics.ScopeGlobal, oldRecord, newCache.GetBindRecord(sqlDigest, meta.OriginalSQL, meta.Db), true) + updateMetrics(metrics.ScopeGlobal, oldRecord, newCache.GetBinding(sqlDigest, meta.OriginalSQL, meta.Db), true) } if memExceededErr != nil { // When the memory capacity of bing_cache is not enough, @@ -215,9 +213,9 @@ func (h *BindHandle) Update(fullLoad bool) (err error) { return nil } -// CreateBindRecord creates a BindRecord to the storage and the cache. +// CreateGlobalBinding 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) { +func (h *BindHandle) CreateGlobalBinding(sctx sessionctx.Context, record *BindRecord) (err error) { err = record.prepareHints(sctx) if err != nil { return err @@ -250,7 +248,7 @@ func (h *BindHandle) CreateBindRecord(sctx sessionctx.Context, record *BindRecor } sqlDigest := parser.DigestNormalized(record.OriginalSQL) - h.setBindRecord(sqlDigest.String(), record) + h.setGlobalCacheBinding(sqlDigest.String(), record) }() // Lock mysql.bind_info to synchronize with CreateBindRecord / AddBindRecord / DropBindRecord on other tidb instances. @@ -292,15 +290,15 @@ func (h *BindHandle) CreateBindRecord(sctx sessionctx.Context, record *BindRecor 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) { +// AddGlobalBinding adds a BindRecord to the storage and BindRecord to the cache. +func (h *BindHandle) AddGlobalBinding(sctx sessionctx.Context, record *BindRecord) (err error) { err = record.prepareHints(sctx) if err != nil { return err } record.Db = strings.ToLower(record.Db) - oldRecord := h.GetBindRecord(parser.DigestNormalized(record.OriginalSQL).String(), record.OriginalSQL, record.Db) + oldRecord := h.GetGlobalBinding(parser.DigestNormalized(record.OriginalSQL).String(), record.OriginalSQL, record.Db) var duplicateBinding *Binding if oldRecord != nil { binding := oldRecord.FindBinding(record.Bindings[0].ID) @@ -339,7 +337,7 @@ func (h *BindHandle) AddBindRecord(sctx sessionctx.Context, record *BindRecord) return } - h.appendBindRecord(parser.DigestNormalized(record.OriginalSQL).String(), record) + h.appendGlobalCacheBinding(parser.DigestNormalized(record.OriginalSQL).String(), record) }() // Lock mysql.bind_info to synchronize with CreateBindRecord / AddBindRecord / DropBindRecord on other tidb instances. @@ -393,8 +391,8 @@ func (h *BindHandle) AddBindRecord(sctx sessionctx.Context, record *BindRecord) return nil } -// DropBindRecord drops a BindRecord to the storage and BindRecord int the cache. -func (h *BindHandle) DropBindRecord(originalSQL, db string, binding *Binding) (deletedRows uint64, err error) { +// DropGlobalBinding drops a BindRecord to the storage and BindRecord int the cache. +func (h *BindHandle) DropGlobalBinding(originalSQL, db string, binding *Binding) (deletedRows uint64, err error) { db = strings.ToLower(db) h.bindInfo.Lock() h.sctx.Lock() @@ -424,7 +422,7 @@ func (h *BindHandle) DropBindRecord(originalSQL, db string, binding *Binding) (d if binding != nil { record.Bindings = append(record.Bindings, *binding) } - h.removeBindRecord(parser.DigestNormalized(originalSQL).String(), record) + h.removeGlobalCacheBinding(parser.DigestNormalized(originalSQL).String(), record) }() // Lock mysql.bind_info to synchronize with CreateBindRecord / AddBindRecord / DropBindRecord on other tidb instances. @@ -448,17 +446,17 @@ func (h *BindHandle) DropBindRecord(originalSQL, db string, binding *Binding) (d return h.sctx.Context.GetSessionVars().StmtCtx.AffectedRows(), nil } -// DropBindRecordByDigest drop BindRecord to the storage and BindRecord int the cache. -func (h *BindHandle) DropBindRecordByDigest(sqlDigest string) (deletedRows uint64, err error) { - oldRecord, err := h.GetBindRecordBySQLDigest(sqlDigest) +// DropGlobalBindingByDigest drop BindRecord to the storage and BindRecord int the cache. +func (h *BindHandle) DropGlobalBindingByDigest(sqlDigest string) (deletedRows uint64, err error) { + oldRecord, err := h.GetGlobalBindingBySQLDigest(sqlDigest) if err != nil { return 0, err } - return h.DropBindRecord(oldRecord.OriginalSQL, strings.ToLower(oldRecord.Db), nil) + return h.DropGlobalBinding(oldRecord.OriginalSQL, strings.ToLower(oldRecord.Db), nil) } -// SetBindRecordStatus set a BindRecord's status to the storage and bind cache. -func (h *BindHandle) SetBindRecordStatus(originalSQL string, binding *Binding, newStatus string) (ok bool, err error) { +// SetGlobalBindingStatus set a BindRecord's status to the storage and bind cache. +func (h *BindHandle) SetGlobalBindingStatus(originalSQL string, binding *Binding, newStatus string) (ok bool, err error) { h.bindInfo.Lock() h.sctx.Lock() defer func() { @@ -505,7 +503,7 @@ func (h *BindHandle) SetBindRecordStatus(originalSQL string, binding *Binding, n ok = true record := &BindRecord{OriginalSQL: originalSQL} sqlDigest := parser.DigestNormalized(record.OriginalSQL) - oldRecord := h.GetBindRecord(sqlDigest.String(), originalSQL, "") + oldRecord := h.GetGlobalBinding(sqlDigest.String(), originalSQL, "") setBindingStatusInCacheSucc := false if oldRecord != nil && len(oldRecord.Bindings) > 0 { record.Bindings = make([]Binding, len(oldRecord.Bindings)) @@ -521,7 +519,7 @@ func (h *BindHandle) SetBindRecordStatus(originalSQL string, binding *Binding, n } } if setBindingStatusInCacheSucc { - h.setBindRecord(sqlDigest.String(), record) + h.setGlobalCacheBinding(sqlDigest.String(), record) } }() @@ -544,17 +542,17 @@ 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) +// SetGlobalBindingStatusByDigest set a BindRecord's status to the storage and bind cache. +func (h *BindHandle) SetGlobalBindingStatusByDigest(newStatus, sqlDigest string) (ok bool, err error) { + oldRecord, err := h.GetGlobalBindingBySQLDigest(sqlDigest) if err != nil { return false, err } - return h.SetBindRecordStatus(oldRecord.OriginalSQL, nil, newStatus) + return h.SetGlobalBindingStatus(oldRecord.OriginalSQL, nil, newStatus) } -// GCBindRecord physically removes the deleted bind records in mysql.bind_info. -func (h *BindHandle) GCBindRecord() (err error) { +// GCGlobalBinding physically removes the deleted bind records in mysql.bind_info. +func (h *BindHandle) GCGlobalBinding() (err error) { h.bindInfo.Lock() h.sctx.Lock() defer func() { @@ -666,35 +664,35 @@ func (tmpMap *tmpBindRecordMap) Add(bindRecord *BindRecord) { updateMetrics(metrics.ScopeGlobal, nil, bindRecord, false) } -// DropInvalidBindRecord executes the drop BindRecord tasks. -func (h *BindHandle) DropInvalidBindRecord() { +// DropInvalidGlobalBinding executes the drop BindRecord tasks. +func (h *BindHandle) DropInvalidGlobalBinding() { h.invalidBindRecordMap.flushToStore() } -// AddDropInvalidBindTask adds BindRecord which needs to be deleted into invalidBindRecordMap. -func (h *BindHandle) AddDropInvalidBindTask(invalidBindRecord *BindRecord) { +// AddInvalidGlobalBinding adds BindRecord which needs to be deleted into invalidBindRecordMap. +func (h *BindHandle) AddInvalidGlobalBinding(invalidBindRecord *BindRecord) { h.invalidBindRecordMap.Add(invalidBindRecord) } // Size returns the size of bind info cache. func (h *BindHandle) Size() int { - size := len(h.bindInfo.Load().(*bindCache).GetAllBindRecords()) + size := len(h.bindInfo.Load().(*bindCache).GetAllBindings()) return size } -// GetBindRecord returns the BindRecord of the (normalizedSQL,db) if BindRecord exist. -func (h *BindHandle) GetBindRecord(sqlDigest, normalizedSQL, db string) *BindRecord { - return h.bindInfo.Load().(*bindCache).GetBindRecord(sqlDigest, normalizedSQL, db) +// GetGlobalBinding returns the BindRecord of the (normalizedSQL,db) if BindRecord exist. +func (h *BindHandle) GetGlobalBinding(sqlDigest, normalizedSQL, db string) *BindRecord { + return h.bindInfo.Load().(*bindCache).GetBinding(sqlDigest, normalizedSQL, db) } -// GetBindRecordBySQLDigest returns the BindRecord of the sql digest. -func (h *BindHandle) GetBindRecordBySQLDigest(sqlDigest string) (*BindRecord, error) { - return h.bindInfo.Load().(*bindCache).GetBindRecordBySQLDigest(sqlDigest) +// GetGlobalBindingBySQLDigest returns the BindRecord of the sql digest. +func (h *BindHandle) GetGlobalBindingBySQLDigest(sqlDigest string) (*BindRecord, error) { + return h.bindInfo.Load().(*bindCache).GetBindingBySQLDigest(sqlDigest) } -// GetAllBindRecord returns all bind records in cache. -func (h *BindHandle) GetAllBindRecord() (bindRecords []*BindRecord) { - return h.bindInfo.Load().(*bindCache).GetAllBindRecords() +// GetAllGlobalBinding returns all bind records in cache. +func (h *BindHandle) GetAllGlobalBinding() (bindRecords []*BindRecord) { + return h.bindInfo.Load().(*bindCache).GetAllBindings() } // SetBindCacheCapacity reset the capacity for the bindCache. @@ -744,32 +742,32 @@ func (h *BindHandle) newBindRecord(row chunk.Row) (string, *BindRecord, error) { return sqlDigest.String(), bindRecord, err } -// setBindRecord sets the BindRecord to the cache, if there already exists a BindRecord, +// setGlobalCacheBinding sets the BindRecord to the cache, if there already exists a BindRecord, // it will be overridden. -func (h *BindHandle) setBindRecord(sqlDigest string, meta *BindRecord) { +func (h *BindHandle) setGlobalCacheBinding(sqlDigest string, meta *BindRecord) { newCache, err0 := h.bindInfo.Value.Load().(*bindCache).Copy() if err0 != nil { - logutil.BgLogger().Warn("BindHandle.setBindRecord", zap.String("category", "sql-bind"), zap.Error(err0)) + logutil.BgLogger().Warn("BindHandle.setGlobalCacheBindRecord", zap.String("category", "sql-bind"), zap.Error(err0)) } - oldRecord := newCache.GetBindRecord(sqlDigest, meta.OriginalSQL, meta.Db) - err1 := newCache.SetBindRecord(sqlDigest, meta) + oldRecord := newCache.GetBinding(sqlDigest, meta.OriginalSQL, meta.Db) + err1 := newCache.SetBinding(sqlDigest, meta) if err1 != nil && err0 == nil { - logutil.BgLogger().Warn("BindHandle.setBindRecord", zap.String("category", "sql-bind"), zap.Error(err1)) + logutil.BgLogger().Warn("BindHandle.setGlobalCacheBindRecord", zap.String("category", "sql-bind"), zap.Error(err1)) } h.bindInfo.Value.Store(newCache) updateMetrics(metrics.ScopeGlobal, oldRecord, meta, false) } -// appendBindRecord adds the BindRecord to the cache, all the stale BindRecords are +// appendGlobalCacheBinding adds the BindRecord to the cache, all the stale BindRecords are // removed from the cache after this operation. -func (h *BindHandle) appendBindRecord(sqlDigest string, meta *BindRecord) { +func (h *BindHandle) appendGlobalCacheBinding(sqlDigest string, meta *BindRecord) { newCache, err0 := h.bindInfo.Value.Load().(*bindCache).Copy() if err0 != nil { logutil.BgLogger().Warn("BindHandle.appendBindRecord", zap.String("category", "sql-bind"), zap.Error(err0)) } - oldRecord := newCache.GetBindRecord(sqlDigest, meta.OriginalSQL, meta.Db) + oldRecord := newCache.GetBinding(sqlDigest, meta.OriginalSQL, meta.Db) newRecord := merge(oldRecord, meta) - err1 := newCache.SetBindRecord(sqlDigest, newRecord) + err1 := newCache.SetBinding(sqlDigest, newRecord) if err1 != nil && err0 == nil { // Only need to handle the error once. logutil.BgLogger().Warn("BindHandle.appendBindRecord", zap.String("category", "sql-bind"), zap.Error(err1)) @@ -778,16 +776,16 @@ func (h *BindHandle) appendBindRecord(sqlDigest string, meta *BindRecord) { updateMetrics(metrics.ScopeGlobal, oldRecord, newRecord, false) } -// removeBindRecord removes the BindRecord from the cache. -func (h *BindHandle) removeBindRecord(sqlDigest string, meta *BindRecord) { +// removeGlobalCacheBinding removes the BindRecord from the cache. +func (h *BindHandle) removeGlobalCacheBinding(sqlDigest string, meta *BindRecord) { newCache, err := h.bindInfo.Value.Load().(*bindCache).Copy() if err != nil { logutil.BgLogger().Warn("", zap.String("category", "sql-bind"), zap.Error(err)) } - oldRecord := newCache.GetBindRecord(sqlDigest, meta.OriginalSQL, meta.Db) - newCache.RemoveBindRecord(sqlDigest, meta) + oldRecord := newCache.GetBinding(sqlDigest, meta.OriginalSQL, meta.Db) + newCache.RemoveBinding(sqlDigest, meta) h.bindInfo.Value.Store(newCache) - updateMetrics(metrics.ScopeGlobal, oldRecord, newCache.GetBindRecord(sqlDigest, meta.OriginalSQL, meta.Db), false) + updateMetrics(metrics.ScopeGlobal, oldRecord, newCache.GetBinding(sqlDigest, meta.OriginalSQL, meta.Db), false) } func copyBindRecordUpdateMap(oldMap map[string]*bindRecordUpdate) map[string]*bindRecordUpdate { @@ -935,7 +933,7 @@ func (h *BindHandle) CaptureBaselines() { } dbName := utilparser.GetDefaultDB(stmt, bindableStmt.Schema) normalizedSQL, digest := parser.NormalizeDigest(utilparser.RestoreWithDefaultDB(stmt, dbName, bindableStmt.Query)) - if r := h.GetBindRecord(digest.String(), normalizedSQL, dbName); r != nil && r.HasAvailableBinding() { + if r := h.GetGlobalBinding(digest.String(), normalizedSQL, dbName); r != nil && r.HasAvailableBinding() { continue } bindSQL := GenerateBindSQL(context.TODO(), stmt, bindableStmt.PlanHint, true, dbName) @@ -954,7 +952,7 @@ func (h *BindHandle) CaptureBaselines() { SQLDigest: digest.String(), } // We don't need to pass the `sctx` because the BindSQL has been validated already. - err = h.CreateBindRecord(nil, &BindRecord{OriginalSQL: normalizedSQL, Db: dbName, Bindings: []Binding{binding}}) + err = h.CreateGlobalBinding(nil, &BindRecord{OriginalSQL: normalizedSQL, Db: dbName, Bindings: []Binding{binding}}) if err != nil { logutil.BgLogger().Debug("create bind record failed in baseline capture", zap.String("category", "sql-bind"), zap.String("SQL", bindableStmt.Query), zap.Error(err)) } @@ -1141,7 +1139,7 @@ const ( func (h *BindHandle) getOnePendingVerifyJob() (originalSQL, db string, binding Binding) { cache := h.bindInfo.Value.Load().(*bindCache) - for _, bindRecord := range cache.GetAllBindRecords() { + for _, bindRecord := range cache.GetAllBindings() { for _, bind := range bindRecord.Bindings { if bind.Status == PendingVerify { return bindRecord.OriginalSQL, bindRecord.Db, bind @@ -1240,7 +1238,7 @@ func (h *BindHandle) HandleEvolvePlanTask(sctx sessionctx.Context, adminEvolve b // 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 { - _, err = h.DropBindRecord(originalSQL, db, &binding) + _, err = h.DropGlobalBinding(originalSQL, db, &binding) return err } // If the accepted plan timeouts, it is hard to decide the timeout for verify plan. @@ -1251,7 +1249,7 @@ func (h *BindHandle) HandleEvolvePlanTask(sctx sessionctx.Context, adminEvolve b sctx.GetSessionVars().UsePlanBaselines = false verifyPlanTime, err := h.getRunningDuration(sctx, db, binding.BindSQL, maxTime) if err != nil { - _, err = h.DropBindRecord(originalSQL, db, &binding) + _, err = h.DropGlobalBinding(originalSQL, db, &binding) return err } if verifyPlanTime == -1 || (float64(verifyPlanTime)*acceptFactor > float64(currentPlanTime)) { @@ -1266,7 +1264,7 @@ func (h *BindHandle) HandleEvolvePlanTask(sctx sessionctx.Context, adminEvolve b binding.Status = Enabled } // 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}}) + return h.AddGlobalBinding(nil, &BindRecord{OriginalSQL: originalSQL, Db: db, Bindings: []Binding{binding}}) } // Clear resets the bind handle. It is only used for test. @@ -1279,16 +1277,16 @@ func (h *BindHandle) Clear() { 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() +// FlushGlobalBindings flushes the BindRecord in temp maps to storage and loads them into cache. +func (h *BindHandle) FlushGlobalBindings() error { + h.DropInvalidGlobalBinding() h.SaveEvolveTasksToStore() return h.Update(false) } -// ReloadBindings clears existing binding cache and do a full load from mysql.bind_info. +// ReloadGlobalBindings 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 { +func (h *BindHandle) ReloadGlobalBindings() error { h.bindInfo.Lock() h.bindInfo.Store(newBindCache()) h.bindInfo.lastUpdateTime = types.ZeroTimestamp diff --git a/pkg/bindinfo/handle_test.go b/pkg/bindinfo/handle_test.go index 7a95e3ecb1ce2..263e501445968 100644 --- a/pkg/bindinfo/handle_test.go +++ b/pkg/bindinfo/handle_test.go @@ -50,7 +50,7 @@ func TestBindingCache(t *testing.T) { tk.MustExec("drop global binding for select * from t;") require.Nil(t, dom.BindHandle().Update(false)) - require.Equal(t, 1, len(dom.BindHandle().GetAllBindRecord())) + require.Equal(t, 1, len(dom.BindHandle().GetAllGlobalBinding())) } func TestBindingLastUpdateTime(t *testing.T) { @@ -68,7 +68,7 @@ func TestBindingLastUpdateTime(t *testing.T) { err := bindHandle.Update(true) require.NoError(t, err) sql, sqlDigest := parser.NormalizeDigest("select * from test . t0") - bindData := bindHandle.GetBindRecord(sqlDigest.String(), sql, "test") + bindData := bindHandle.GetGlobalBinding(sqlDigest.String(), sql, "test") require.Equal(t, 1, len(bindData.Bindings)) bind := bindData.Bindings[0] updateTime := bind.UpdateTime.String() @@ -133,7 +133,7 @@ func TestBindParse(t *testing.T) { require.Equal(t, 1, bindHandle.Size()) sql, sqlDigest := parser.NormalizeDigest("select * from test . t") - bindData := bindHandle.GetBindRecord(sqlDigest.String(), sql, "test") + bindData := bindHandle.GetGlobalBinding(sqlDigest.String(), sql, "test") require.NotNil(t, bindData) require.Equal(t, "select * from `test` . `t`", bindData.OriginalSQL) bind := bindData.Bindings[0] @@ -490,7 +490,7 @@ func TestGlobalBinding(t *testing.T) { sql, sqlDigest := internal.UtilNormalizeWithDefaultDB(t, testSQL.querySQL) - bindData := dom.BindHandle().GetBindRecord(sqlDigest, sql, "test") + bindData := dom.BindHandle().GetGlobalBinding(sqlDigest, sql, "test") require.NotNil(t, bindData) require.Equal(t, testSQL.originSQL, bindData.OriginalSQL) bind := bindData.Bindings[0] @@ -523,7 +523,7 @@ func TestGlobalBinding(t *testing.T) { require.NoError(t, err) require.Equal(t, 1, bindHandle.Size()) - bindData = bindHandle.GetBindRecord(sqlDigest, sql, "test") + bindData = bindHandle.GetGlobalBinding(sqlDigest, sql, "test") require.NotNil(t, bindData) require.Equal(t, testSQL.originSQL, bindData.OriginalSQL) bind = bindData.Bindings[0] @@ -538,7 +538,7 @@ func TestGlobalBinding(t *testing.T) { _, err = tk.Exec("drop global " + testSQL.dropSQL) require.Equal(t, uint64(1), tk.Session().AffectedRows()) require.NoError(t, err) - bindData = dom.BindHandle().GetBindRecord(sqlDigest, sql, "test") + bindData = dom.BindHandle().GetGlobalBinding(sqlDigest, sql, "test") require.Nil(t, bindData) err = metrics.BindTotalGauge.WithLabelValues(metrics.ScopeGlobal, bindinfo.Enabled).Write(pb) @@ -554,7 +554,7 @@ func TestGlobalBinding(t *testing.T) { require.NoError(t, err) require.Equal(t, 0, bindHandle.Size()) - bindData = bindHandle.GetBindRecord(sqlDigest, sql, "test") + bindData = bindHandle.GetGlobalBinding(sqlDigest, sql, "test") require.Nil(t, bindData) rs, err = tk.Exec("show global bindings") diff --git a/pkg/bindinfo/session_handle.go b/pkg/bindinfo/session_handle.go index fb54f11ba06c6..777664e21ee50 100644 --- a/pkg/bindinfo/session_handle.go +++ b/pkg/bindinfo/session_handle.go @@ -43,20 +43,20 @@ func NewSessionBindHandle() *SessionHandle { return sessionHandle } -// appendBindRecord adds the BindRecord to the cache, all the stale bindMetas are +// appendSessionBinding adds the BindRecord to the cache, all the stale bindMetas are // removed from the cache after this operation. -func (h *SessionHandle) appendBindRecord(sqlDigest string, meta *BindRecord) { - oldRecord := h.ch.GetBindRecord(sqlDigest, meta.OriginalSQL, meta.Db) - err := h.ch.SetBindRecord(sqlDigest, meta) +func (h *SessionHandle) appendSessionBinding(sqlDigest string, meta *BindRecord) { + oldRecord := h.ch.GetBinding(sqlDigest, meta.OriginalSQL, meta.Db) + err := h.ch.SetBinding(sqlDigest, meta) if err != nil { logutil.BgLogger().Warn("SessionHandle.appendBindRecord", zap.String("category", "sql-bind"), zap.Error(err)) } updateMetrics(metrics.ScopeSession, oldRecord, meta, false) } -// CreateBindRecord creates a BindRecord to the cache. +// CreateSessionBinding creates a BindRecord to the cache. // It replaces all the exists bindings for the same normalized SQL. -func (h *SessionHandle) CreateBindRecord(sctx sessionctx.Context, record *BindRecord) (err error) { +func (h *SessionHandle) CreateSessionBinding(sctx sessionctx.Context, record *BindRecord) (err error) { err = record.prepareHints(sctx) if err != nil { return err @@ -69,15 +69,15 @@ func (h *SessionHandle) CreateBindRecord(sctx sessionctx.Context, record *BindRe } // update the BindMeta to the cache. - h.appendBindRecord(parser.DigestNormalized(record.OriginalSQL).String(), record) + h.appendSessionBinding(parser.DigestNormalized(record.OriginalSQL).String(), record) return nil } -// DropBindRecord drops a BindRecord in the cache. -func (h *SessionHandle) DropBindRecord(originalSQL, db string, binding *Binding) error { +// DropSessionBinding drops a BindRecord in the cache. +func (h *SessionHandle) DropSessionBinding(originalSQL, db string, binding *Binding) error { db = strings.ToLower(db) sqlDigest := parser.DigestNormalized(originalSQL).String() - oldRecord := h.GetBindRecord(sqlDigest, originalSQL, db) + oldRecord := h.GetSessionBinding(sqlDigest, originalSQL, db) var newRecord *BindRecord record := &BindRecord{OriginalSQL: originalSQL, Db: db} if binding != nil { @@ -88,7 +88,7 @@ func (h *SessionHandle) DropBindRecord(originalSQL, db string, binding *Binding) } else { newRecord = record } - err := h.ch.SetBindRecord(sqlDigest, newRecord) + err := h.ch.SetBinding(sqlDigest, newRecord) if err != nil { // Should never reach here, just return an error for safety return err @@ -97,33 +97,33 @@ func (h *SessionHandle) DropBindRecord(originalSQL, db string, binding *Binding) return nil } -// DropBindRecordByDigest drop BindRecord in the cache. -func (h *SessionHandle) DropBindRecordByDigest(sqlDigest string) error { - oldRecord, err := h.GetBindRecordBySQLDigest(sqlDigest) +// DropSessionBindingByDigest drop BindRecord in the cache. +func (h *SessionHandle) DropSessionBindingByDigest(sqlDigest string) error { + oldRecord, err := h.GetSessionBindingBySQLDigest(sqlDigest) if err != nil { return err } - return h.DropBindRecord(oldRecord.OriginalSQL, strings.ToLower(oldRecord.Db), nil) + return h.DropSessionBinding(oldRecord.OriginalSQL, strings.ToLower(oldRecord.Db), nil) } -// GetBindRecord return the BindMeta of the (normdOrigSQL,db) if BindMeta exist. -func (h *SessionHandle) GetBindRecord(sqlDigest, normdOrigSQL, db string) *BindRecord { - return h.ch.GetBindRecord(sqlDigest, normdOrigSQL, db) +// GetSessionBinding return the BindMeta of the (normdOrigSQL,db) if BindMeta exist. +func (h *SessionHandle) GetSessionBinding(sqlDigest, normdOrigSQL, db string) *BindRecord { + return h.ch.GetBinding(sqlDigest, normdOrigSQL, db) } -// GetBindRecordBySQLDigest return all BindMeta corresponding to sqlDigest. -func (h *SessionHandle) GetBindRecordBySQLDigest(sqlDigest string) (*BindRecord, error) { - return h.ch.GetBindRecordBySQLDigest(sqlDigest) +// GetSessionBindingBySQLDigest return all BindMeta corresponding to sqlDigest. +func (h *SessionHandle) GetSessionBindingBySQLDigest(sqlDigest string) (*BindRecord, error) { + return h.ch.GetBindingBySQLDigest(sqlDigest) } -// GetAllBindRecord return all session bind info. -func (h *SessionHandle) GetAllBindRecord() (bindRecords []*BindRecord) { - return h.ch.GetAllBindRecords() +// GetAllSessionBindRecord return all session bind info. +func (h *SessionHandle) GetAllSessionBindRecord() (bindRecords []*BindRecord) { + return h.ch.GetAllBindings() } // EncodeSessionStates implements SessionStatesHandler.EncodeSessionStates interface. func (h *SessionHandle) EncodeSessionStates(_ context.Context, _ sessionctx.Context, sessionStates *sessionstates.SessionStates) error { - bindRecords := h.ch.GetAllBindRecords() + bindRecords := h.ch.GetAllBindings() if len(bindRecords) == 0 { return nil } @@ -149,14 +149,14 @@ func (h *SessionHandle) DecodeSessionStates(_ context.Context, sctx sessionctx.C if err := record.prepareHints(sctx); err != nil { return err } - h.appendBindRecord(parser.DigestNormalized(record.OriginalSQL).String(), record) + h.appendSessionBinding(parser.DigestNormalized(record.OriginalSQL).String(), record) } return nil } // Close closes the session handle. func (h *SessionHandle) Close() { - for _, bindRecord := range h.ch.GetAllBindRecords() { + for _, bindRecord := range h.ch.GetAllBindings() { updateMetrics(metrics.ScopeSession, bindRecord, nil, false) } } diff --git a/pkg/bindinfo/session_handle_test.go b/pkg/bindinfo/session_handle_test.go index c8edef8538c9e..7cadd0e2f71da 100644 --- a/pkg/bindinfo/session_handle_test.go +++ b/pkg/bindinfo/session_handle_test.go @@ -118,7 +118,7 @@ func TestSessionBinding(t *testing.T) { handle := tk.Session().Value(bindinfo.SessionBindInfoKeyType).(*bindinfo.SessionHandle) sqlDigest := parser.DigestNormalized(testSQL.originSQL).String() - bindData := handle.GetBindRecord(sqlDigest, testSQL.originSQL, "test") + bindData := handle.GetSessionBinding(sqlDigest, testSQL.originSQL, "test") require.NotNil(t, bindData) require.Equal(t, testSQL.originSQL, bindData.OriginalSQL) bind := bindData.Bindings[0] @@ -155,7 +155,7 @@ func TestSessionBinding(t *testing.T) { _, err = tk.Exec("drop session " + testSQL.dropSQL) require.NoError(t, err) - bindData = handle.GetBindRecord(sqlDigest, testSQL.originSQL, "test") + bindData = handle.GetSessionBinding(sqlDigest, testSQL.originSQL, "test") require.NotNil(t, bindData) require.Equal(t, testSQL.originSQL, bindData.OriginalSQL) require.Len(t, bindData.Bindings, 0) diff --git a/pkg/bindinfo/tests/bind_test.go b/pkg/bindinfo/tests/bind_test.go index 67ac70bc764d5..8e5e2d5ad6022 100644 --- a/pkg/bindinfo/tests/bind_test.go +++ b/pkg/bindinfo/tests/bind_test.go @@ -320,7 +320,7 @@ func TestBindingSymbolList(t *testing.T) { // Normalize sql, hash := parser.NormalizeDigestForBinding("select a, b from test . t where a = 1 limit 0, 1") - bindData := dom.BindHandle().GetBindRecord(hash.String(), sql, "test") + bindData := dom.BindHandle().GetGlobalBinding(hash.String(), sql, "test") require.NotNil(t, bindData) require.Equal(t, "select `a` , `b` from `test` . `t` where `a` = ? limit ...", bindData.OriginalSQL) bind := bindData.Bindings[0] @@ -364,7 +364,7 @@ func TestBindingInListWithSingleLiteral(t *testing.T) { // Normalize sql, hash := parser.NormalizeDigestForBinding("select a, b from test . t where a in (1)") - bindData := dom.BindHandle().GetBindRecord(hash.String(), sql, "test") + bindData := dom.BindHandle().GetGlobalBinding(hash.String(), sql, "test") require.NotNil(t, bindData) require.Equal(t, "select `a` , `b` from `test` . `t` where `a` in ( ... )", bindData.OriginalSQL) bind := bindData.Bindings[0] @@ -399,7 +399,7 @@ func TestBestPlanInBaselines(t *testing.T) { tk.MustExec(`create global binding for select a, b from t where b = 1 limit 0, 1 using select /*+ use_index(@sel_1 test.t ib) */ a, b from t where b = 1 limit 0, 1`) sql, hash := internal.UtilNormalizeWithDefaultDB(t, "select a, b from t where a = 1 limit 0, 1") - bindData := dom.BindHandle().GetBindRecord(hash, sql, "test") + bindData := dom.BindHandle().GetGlobalBinding(hash, sql, "test") require.NotNil(t, bindData) require.Equal(t, "select `a` , `b` from `test` . `t` where `a` = ? limit ...", bindData.OriginalSQL) bind := bindData.Bindings[0] @@ -432,7 +432,7 @@ func TestErrorBind(t *testing.T) { require.NoError(t, err, "err %v", err) sql, hash := parser.NormalizeDigestForBinding("select * from test . t where i > ?") - bindData := dom.BindHandle().GetBindRecord(hash.String(), sql, "test") + bindData := dom.BindHandle().GetGlobalBinding(hash.String(), sql, "test") require.NotNil(t, bindData) require.Equal(t, "select * from `test` . `t` where `i` > ?", bindData.OriginalSQL) bind := bindData.Bindings[0] @@ -449,7 +449,7 @@ func TestErrorBind(t *testing.T) { require.NoError(t, err) rs.Close() - dom.BindHandle().DropInvalidBindRecord() + dom.BindHandle().DropInvalidGlobalBinding() rs, err = tk.Exec("show global bindings") require.NoError(t, err) @@ -621,7 +621,7 @@ func TestHintsSetEvolveTask(t *testing.T) { bindHandle.SaveEvolveTasksToStore() // Verify the added Binding for evolution contains valid ID and Hint, otherwise, panic may happen. sql, hash := internal.UtilNormalizeWithDefaultDB(t, "select * from t where a > ?") - bindData := bindHandle.GetBindRecord(hash, sql, "test") + bindData := bindHandle.GetGlobalBinding(hash, sql, "test") require.NotNil(t, bindData) require.Equal(t, "select * from `test` . `t` where `a` > ?", bindData.OriginalSQL) require.Len(t, bindData.Bindings, 2) @@ -642,7 +642,7 @@ func TestHintsSetID(t *testing.T) { bindHandle := dom.BindHandle() // Verify the added Binding contains ID with restored query block. sql, hash := internal.UtilNormalizeWithDefaultDB(t, "select * from t where a > ?") - bindData := bindHandle.GetBindRecord(hash, sql, "test") + bindData := bindHandle.GetGlobalBinding(hash, sql, "test") require.NotNil(t, bindData) require.Equal(t, "select * from `test` . `t` where `a` > ?", bindData.OriginalSQL) require.Len(t, bindData.Bindings, 1) @@ -651,7 +651,7 @@ func TestHintsSetID(t *testing.T) { internal.UtilCleanBindingEnv(tk, dom) tk.MustExec("create global binding for select * from t where a > 10 using select /*+ use_index(t, idx_a) */ * from t where a > 10") - bindData = bindHandle.GetBindRecord(hash, sql, "test") + bindData = bindHandle.GetGlobalBinding(hash, sql, "test") require.NotNil(t, bindData) require.Equal(t, "select * from `test` . `t` where `a` > ?", bindData.OriginalSQL) require.Len(t, bindData.Bindings, 1) @@ -660,7 +660,7 @@ func TestHintsSetID(t *testing.T) { internal.UtilCleanBindingEnv(tk, dom) tk.MustExec("create global binding for select * from t where a > 10 using select /*+ use_index(@sel_1 t, idx_a) */ * from t where a > 10") - bindData = bindHandle.GetBindRecord(hash, sql, "test") + bindData = bindHandle.GetGlobalBinding(hash, sql, "test") require.NotNil(t, bindData) require.Equal(t, "select * from `test` . `t` where `a` > ?", bindData.OriginalSQL) require.Len(t, bindData.Bindings, 1) @@ -669,7 +669,7 @@ func TestHintsSetID(t *testing.T) { internal.UtilCleanBindingEnv(tk, dom) tk.MustExec("create global binding for select * from t where a > 10 using select /*+ use_index(@qb1 t, idx_a) qb_name(qb1) */ * from t where a > 10") - bindData = bindHandle.GetBindRecord(hash, sql, "test") + bindData = bindHandle.GetGlobalBinding(hash, sql, "test") require.NotNil(t, bindData) require.Equal(t, "select * from `test` . `t` where `a` > ?", bindData.OriginalSQL) require.Len(t, bindData.Bindings, 1) @@ -678,7 +678,7 @@ func TestHintsSetID(t *testing.T) { internal.UtilCleanBindingEnv(tk, dom) tk.MustExec("create global binding for select * from t where a > 10 using select /*+ use_index(T, IDX_A) */ * from t where a > 10") - bindData = bindHandle.GetBindRecord(hash, sql, "test") + bindData = bindHandle.GetGlobalBinding(hash, sql, "test") require.NotNil(t, bindData) require.Equal(t, "select * from `test` . `t` where `a` > ?", bindData.OriginalSQL) require.Len(t, bindData.Bindings, 1) @@ -689,7 +689,7 @@ func TestHintsSetID(t *testing.T) { err := tk.ExecToErr("create global binding for select * from t using select /*+ non_exist_hint() */ * from t") require.True(t, terror.ErrorEqual(err, parser.ErrParse)) tk.MustExec("create global binding for select * from t where a > 10 using select * from t where a > 10") - bindData = bindHandle.GetBindRecord(hash, sql, "test") + bindData = bindHandle.GetGlobalBinding(hash, sql, "test") require.NotNil(t, bindData) require.Equal(t, "select * from `test` . `t` where `a` > ?", bindData.OriginalSQL) require.Len(t, bindData.Bindings, 1) @@ -896,7 +896,7 @@ func TestGCBindRecord(t *testing.T) { h := dom.BindHandle() // bindinfo.Lease is set to 0 for test env in SetUpSuite. - require.NoError(t, h.GCBindRecord()) + require.NoError(t, h.GCGlobalBinding()) rows = tk.MustQuery("show global bindings").Rows() require.Len(t, rows, 1) require.Equal(t, "select * from `test` . `t` where `a` = ?", rows[0][0]) @@ -910,7 +910,7 @@ func TestGCBindRecord(t *testing.T) { tk.MustQuery("select status from mysql.bind_info where original_sql = 'select * from `test` . `t` where `a` = ?'").Check(testkit.Rows( "deleted", )) - require.NoError(t, h.GCBindRecord()) + require.NoError(t, h.GCGlobalBinding()) tk.MustQuery("show global bindings").Check(testkit.Rows()) tk.MustQuery("select status from mysql.bind_info where original_sql = 'select * from `test` . `t` where `a` = ?'").Check(testkit.Rows()) } @@ -1057,15 +1057,15 @@ func TestDropBindBySQLDigest(t *testing.T) { internal.UtilCleanBindingEnv(tk, dom) sql := "create global binding for " + c.origin + " using " + c.hint tk.MustExec(sql) - h.ReloadBindings() + h.ReloadGlobalBindings() res := tk.MustQuery(`show global bindings`).Rows() require.Equal(t, len(res), 1) require.Equal(t, len(res[0]), 11) drop := fmt.Sprintf("drop global binding for sql digest '%s'", res[0][9]) tk.MustExec(drop) - require.NoError(t, h.GCBindRecord()) - h.ReloadBindings() + require.NoError(t, h.GCGlobalBinding()) + h.ReloadGlobalBindings() tk.MustQuery("show global bindings").Check(testkit.Rows()) } @@ -1080,7 +1080,7 @@ func TestDropBindBySQLDigest(t *testing.T) { require.Equal(t, len(res[0]), 11) drop := fmt.Sprintf("drop binding for sql digest '%s'", res[0][9]) tk.MustExec(drop) - require.NoError(t, h.GCBindRecord()) + require.NoError(t, h.GCGlobalBinding()) tk.MustQuery("show bindings").Check(testkit.Rows()) } diff --git a/pkg/domain/domain.go b/pkg/domain/domain.go index f24d22574ca5f..6af0a74e2161a 100644 --- a/pkg/domain/domain.go +++ b/pkg/domain/domain.go @@ -1879,7 +1879,7 @@ func (do *Domain) globalBindHandleWorkerLoop(owner owner.Manager) { if err != nil { logutil.BgLogger().Error("update bindinfo failed", zap.Error(err)) } - bindHandle.DropInvalidBindRecord() + bindHandle.DropInvalidGlobalBinding() // Get Global optVal, err := do.GetGlobalVar(variable.TiDBCapturePlanBaseline) if err == nil && variable.TiDBOptOn(optVal) { @@ -1890,7 +1890,7 @@ func (do *Domain) globalBindHandleWorkerLoop(owner owner.Manager) { if !owner.IsOwner() { continue } - err := do.bindHandle.Load().GCBindRecord() + err := do.bindHandle.Load().GCGlobalBinding() if err != nil { logutil.BgLogger().Error("GC bind record failed", zap.Error(err)) } diff --git a/pkg/executor/adapter.go b/pkg/executor/adapter.go index 44bb8de5bf3b6..903883f6ac065 100644 --- a/pkg/executor/adapter.go +++ b/pkg/executor/adapter.go @@ -2099,7 +2099,7 @@ func sendPlanReplayerDumpTask(key replayer.PlanReplayerTaskKey, sctx sessionctx. PlanReplayerTaskKey: key, StartTS: startTS, TblStats: stmtCtx.TableStats, - SessionBindings: handle.GetAllBindRecord(), + SessionBindings: handle.GetAllSessionBindRecord(), SessionVars: sctx.GetSessionVars(), ExecStmts: []ast.StmtNode{stmtNode}, DebugTrace: []interface{}{stmtCtx.OptimizerDebugTrace}, diff --git a/pkg/executor/bind.go b/pkg/executor/bind.go index d4d30e74a1362..97c222ff2714d 100644 --- a/pkg/executor/bind.go +++ b/pkg/executor/bind.go @@ -83,10 +83,10 @@ func (e *SQLBindExec) dropSQLBind() error { } if !e.isGlobal { handle := e.Ctx().Value(bindinfo.SessionBindInfoKeyType).(*bindinfo.SessionHandle) - err := handle.DropBindRecord(e.normdOrigSQL, e.db, bindInfo) + err := handle.DropSessionBinding(e.normdOrigSQL, e.db, bindInfo) return err } - affectedRows, err := domain.GetDomain(e.Ctx()).BindHandle().DropBindRecord(e.normdOrigSQL, e.db, bindInfo) + affectedRows, err := domain.GetDomain(e.Ctx()).BindHandle().DropGlobalBinding(e.normdOrigSQL, e.db, bindInfo) e.Ctx().GetSessionVars().StmtCtx.AddAffectedRows(affectedRows) return err } @@ -97,10 +97,10 @@ func (e *SQLBindExec) dropSQLBindByDigest() error { } if !e.isGlobal { handle := e.Ctx().Value(bindinfo.SessionBindInfoKeyType).(*bindinfo.SessionHandle) - err := handle.DropBindRecordByDigest(e.sqlDigest) + err := handle.DropSessionBindingByDigest(e.sqlDigest) return err } - affectedRows, err := domain.GetDomain(e.Ctx()).BindHandle().DropBindRecordByDigest(e.sqlDigest) + affectedRows, err := domain.GetDomain(e.Ctx()).BindHandle().DropGlobalBindingByDigest(e.sqlDigest) e.Ctx().GetSessionVars().StmtCtx.AddAffectedRows(affectedRows) return err } @@ -114,7 +114,7 @@ func (e *SQLBindExec) setBindingStatus() error { Collation: e.collation, } } - ok, err := domain.GetDomain(e.Ctx()).BindHandle().SetBindRecordStatus(e.normdOrigSQL, bindInfo, e.newStatus) + ok, err := domain.GetDomain(e.Ctx()).BindHandle().SetGlobalBindingStatus(e.normdOrigSQL, bindInfo, e.newStatus) 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) @@ -123,7 +123,7 @@ func (e *SQLBindExec) setBindingStatus() error { } func (e *SQLBindExec) setBindingStatusByDigest() error { - ok, err := domain.GetDomain(e.Ctx()).BindHandle().SetBindRecordStatusByDigest(e.newStatus, e.sqlDigest) + ok, err := domain.GetDomain(e.Ctx()).BindHandle().SetGlobalBindingStatusByDigest(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) @@ -159,13 +159,13 @@ func (e *SQLBindExec) createSQLBind() error { } if !e.isGlobal { handle := e.Ctx().Value(bindinfo.SessionBindInfoKeyType).(*bindinfo.SessionHandle) - return handle.CreateBindRecord(e.Ctx(), record) + return handle.CreateSessionBinding(e.Ctx(), record) } - return domain.GetDomain(e.Ctx()).BindHandle().CreateBindRecord(e.Ctx(), record) + return domain.GetDomain(e.Ctx()).BindHandle().CreateGlobalBinding(e.Ctx(), record) } func (e *SQLBindExec) flushBindings() error { - return domain.GetDomain(e.Ctx()).BindHandle().FlushBindings() + return domain.GetDomain(e.Ctx()).BindHandle().FlushGlobalBindings() } func (e *SQLBindExec) captureBindings() { @@ -177,5 +177,5 @@ func (e *SQLBindExec) evolveBindings() error { } func (e *SQLBindExec) reloadBindings() error { - return domain.GetDomain(e.Ctx()).BindHandle().ReloadBindings() + return domain.GetDomain(e.Ctx()).BindHandle().ReloadGlobalBindings() } diff --git a/pkg/executor/show.go b/pkg/executor/show.go index 0ac276bbea05c..cbb924464f1f3 100644 --- a/pkg/executor/show.go +++ b/pkg/executor/show.go @@ -324,9 +324,9 @@ func (e *ShowExec) fetchShowBind() error { var tmp []*bindinfo.BindRecord if !e.GlobalScope { handle := e.Ctx().Value(bindinfo.SessionBindInfoKeyType).(*bindinfo.SessionHandle) - tmp = handle.GetAllBindRecord() + tmp = handle.GetAllSessionBindRecord() } else { - tmp = domain.GetDomain(e.Ctx()).BindHandle().GetAllBindRecord() + tmp = domain.GetDomain(e.Ctx()).BindHandle().GetAllGlobalBinding() } bindRecords := make([]*bindinfo.BindRecord, 0) for _, bindRecord := range tmp { @@ -408,7 +408,7 @@ func (e *ShowExec) fetchShowBindingCacheStatus(ctx context.Context) error { handle := domain.GetDomain(e.Ctx()).BindHandle() - bindRecords := handle.GetAllBindRecord() + bindRecords := handle.GetAllGlobalBinding() numBindings := 0 for _, bindRecord := range bindRecords { for _, binding := range bindRecord.Bindings { diff --git a/pkg/planner/core/plan_cache.go b/pkg/planner/core/plan_cache.go index a696f2eaefbae..ace9376fafae3 100644 --- a/pkg/planner/core/plan_cache.go +++ b/pkg/planner/core/plan_cache.go @@ -797,7 +797,7 @@ func GetBindSQL4PlanCache(sctx sessionctx.Context, stmt *PlanCacheStmt) (string, return "", ignore } sessionHandle := sctx.Value(bindinfo.SessionBindInfoKeyType).(*bindinfo.SessionHandle) - bindRecord := sessionHandle.GetBindRecord(stmt.SQLDigest4PC, stmt.NormalizedSQL4PC, "") + bindRecord := sessionHandle.GetSessionBinding(stmt.SQLDigest4PC, stmt.NormalizedSQL4PC, "") if bindRecord != nil { enabledBinding := bindRecord.FindEnabledBinding() if enabledBinding != nil { @@ -809,7 +809,7 @@ func GetBindSQL4PlanCache(sctx sessionctx.Context, stmt *PlanCacheStmt) (string, if globalHandle == nil { return "", ignore } - bindRecord = globalHandle.GetBindRecord(stmt.SQLDigest4PC, stmt.NormalizedSQL4PC, "") + bindRecord = globalHandle.GetGlobalBinding(stmt.SQLDigest4PC, stmt.NormalizedSQL4PC, "") if bindRecord != nil { enabledBinding := bindRecord.FindEnabledBinding() if enabledBinding != nil { diff --git a/pkg/planner/optimize.go b/pkg/planner/optimize.go index 95643fc7d400d..849de12b966a6 100644 --- a/pkg/planner/optimize.go +++ b/pkg/planner/optimize.go @@ -309,7 +309,7 @@ func Optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is in plan, curNames, cost, err := optimize(ctx, sctx, node, is) if err != nil { binding.Status = bindinfo.Invalid - handleInvalidBindRecord(ctx, sctx, scope, bindinfo.BindRecord{ + handleInvalidBinding(ctx, sctx, scope, bindinfo.BindRecord{ OriginalSQL: bindRecord.OriginalSQL, Db: bindRecord.Db, Bindings: []bindinfo.Binding{binding}, @@ -675,7 +675,7 @@ func getBindRecord(ctx sessionctx.Context, stmt ast.StmtNode) (*bindinfo.BindRec return nil, "", err } sessionHandle := ctx.Value(bindinfo.SessionBindInfoKeyType).(*bindinfo.SessionHandle) - bindRecord := sessionHandle.GetBindRecord(sqlDigest, normalizedSQL, "") + bindRecord := sessionHandle.GetSessionBinding(sqlDigest, normalizedSQL, "") if bindRecord != nil { if bindRecord.HasEnabledBinding() { return bindRecord, metrics.ScopeSession, nil @@ -686,13 +686,13 @@ func getBindRecord(ctx sessionctx.Context, stmt ast.StmtNode) (*bindinfo.BindRec if globalHandle == nil { return nil, "", nil } - bindRecord = globalHandle.GetBindRecord(sqlDigest, normalizedSQL, "") + bindRecord = globalHandle.GetGlobalBinding(sqlDigest, normalizedSQL, "") return bindRecord, metrics.ScopeGlobal, nil } -func handleInvalidBindRecord(ctx context.Context, sctx sessionctx.Context, level string, bindRecord bindinfo.BindRecord) { +func handleInvalidBinding(ctx context.Context, sctx sessionctx.Context, level string, bindRecord bindinfo.BindRecord) { sessionHandle := sctx.Value(bindinfo.SessionBindInfoKeyType).(*bindinfo.SessionHandle) - err := sessionHandle.DropBindRecord(bindRecord.OriginalSQL, bindRecord.Db, &bindRecord.Bindings[0]) + err := sessionHandle.DropSessionBinding(bindRecord.OriginalSQL, bindRecord.Db, &bindRecord.Bindings[0]) if err != nil { logutil.Logger(ctx).Info("drop session bindings failed") } @@ -701,7 +701,7 @@ func handleInvalidBindRecord(ctx context.Context, sctx sessionctx.Context, level } globalHandle := domain.GetDomain(sctx).BindHandle() - globalHandle.AddDropInvalidBindTask(&bindRecord) + globalHandle.AddInvalidGlobalBinding(&bindRecord) } func handleEvolveTasks(ctx context.Context, sctx sessionctx.Context, br *bindinfo.BindRecord, stmtNode ast.StmtNode, planHint string) {