diff --git a/pkg/bindinfo/BUILD.bazel b/pkg/bindinfo/BUILD.bazel index aac506a9b45fc..c62004f988526 100644 --- a/pkg/bindinfo/BUILD.bazel +++ b/pkg/bindinfo/BUILD.bazel @@ -3,8 +3,8 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "bindinfo", srcs = [ - "bind_cache.go", "binding.go", + "binding_cache.go", "binding_match.go", "capture.go", "global_handle.go", @@ -49,7 +49,7 @@ go_test( name = "bindinfo_test", timeout = "moderate", srcs = [ - "bind_cache_test.go", + "binding_cache_test.go", "binding_match_test.go", "capture_test.go", "fuzzy_binding_test.go", diff --git a/pkg/bindinfo/bind_cache.go b/pkg/bindinfo/binding_cache.go similarity index 76% rename from pkg/bindinfo/bind_cache.go rename to pkg/bindinfo/binding_cache.go index c5701e2bf96ac..6148e4cb5f6bc 100644 --- a/pkg/bindinfo/bind_cache.go +++ b/pkg/bindinfo/binding_cache.go @@ -25,33 +25,33 @@ import ( "github.com/pingcap/tidb/pkg/util/memory" ) -// bindCache uses the LRU cache to store the bindings. +// bindingCache uses the LRU cache to store the bindings. // The key of the LRU cache is original sql, the value is a slice of Bindings. -// Note: The bindCache should be accessed with lock. -type bindCache struct { +// Note: The bindingCache should be accessed with lock. +type bindingCache struct { lock sync.Mutex cache *kvcache.SimpleLRUCache memCapacity int64 memTracker *memory.Tracker // track memory usage. } -type bindCacheKey string +type bindingCacheKey string -func (key bindCacheKey) Hash() []byte { +func (key bindingCacheKey) Hash() []byte { return hack.Slice(string(key)) } -func calcBindCacheKVMem(key bindCacheKey, value Bindings) int64 { +func calcBindCacheKVMem(key bindingCacheKey, value Bindings) int64 { var valMem int64 valMem += int64(value.size()) return int64(len(key.Hash())) + valMem } -func newBindCache() *bindCache { - // since bindCache controls the memory usage by itself, set the capacity of +func newBindCache() *bindingCache { + // since bindingCache controls the memory usage by itself, set the capacity of // the underlying LRUCache to max to close its memory control cache := kvcache.NewSimpleLRUCache(mathutil.MaxUint, 0, 0) - c := bindCache{ + c := bindingCache{ cache: cache, memCapacity: variable.MemQuotaBindingCache.Load(), memTracker: memory.NewTracker(memory.LabelForBindCache, -1), @@ -60,10 +60,10 @@ func newBindCache() *bindCache { } // get gets a cache item according to cache key. It's not thread-safe. -// Note: Only other functions of the bindCache file can use this function. +// Note: Only other functions of the bindingCache file can use this function. // Don't use this function directly in other files in bindinfo package. // The return value is not read-only, but it is only can be used in other functions which are also in the bind_cache.go. -func (c *bindCache) get(key bindCacheKey) Bindings { +func (c *bindingCache) get(key bindingCacheKey) Bindings { value, hit := c.cache.Get(key) if !hit { return nil @@ -73,9 +73,9 @@ func (c *bindCache) get(key bindCacheKey) Bindings { } // set inserts an item to the cache. It's not thread-safe. -// Only other functions of the bindCache can use this function. +// Only other functions of the bindingCache can use this function. // The set operation will return error message when the memory usage of binding_cache exceeds its capacity. -func (c *bindCache) set(key bindCacheKey, value Bindings) (ok bool, err error) { +func (c *bindingCache) set(key bindingCacheKey, value Bindings) (ok bool, err error) { mem := calcBindCacheKVMem(key, value) if mem > c.memCapacity { // ignore this kv pair if its size is too large err = errors.New("The memory usage of all available bindings exceeds the cache's mem quota. As a result, all available bindings cannot be held on the cache. Please increase the value of the system variable 'tidb_mem_quota_binding_cache' and execute 'admin reload bindings' to ensure that all bindings exist in the cache and can be used normally") @@ -92,7 +92,7 @@ func (c *bindCache) set(key bindCacheKey, value Bindings) (ok bool, err error) { if !evicted { return } - c.memTracker.Consume(-calcBindCacheKVMem(evictedKey.(bindCacheKey), evictedValue.(Bindings))) + c.memTracker.Consume(-calcBindCacheKVMem(evictedKey.(bindingCacheKey), evictedValue.(Bindings))) } c.memTracker.Consume(mem) c.cache.Put(key, value) @@ -101,8 +101,8 @@ func (c *bindCache) set(key bindCacheKey, value Bindings) (ok bool, err error) { } // delete remove an item from the cache. It's not thread-safe. -// Only other functions of the bindCache can use this function. -func (c *bindCache) delete(key bindCacheKey) bool { +// Only other functions of the bindingCache can use this function. +func (c *bindingCache) delete(key bindingCacheKey) bool { bindings := c.get(key) if bindings != nil { mem := calcBindCacheKVMem(key, bindings) @@ -116,16 +116,16 @@ func (c *bindCache) delete(key bindCacheKey) bool { // GetBinding gets the Bindings 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) GetBinding(sqlDigest string) Bindings { +func (c *bindingCache) GetBinding(sqlDigest string) Bindings { c.lock.Lock() defer c.lock.Unlock() - return c.get(bindCacheKey(sqlDigest)) + return c.get(bindingCacheKey(sqlDigest)) } -// GetAllBindings return all the bindings from the bindCache. +// GetAllBindings return all the bindings from the bindingCache. // 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) GetAllBindings() Bindings { +func (c *bindingCache) GetAllBindings() Bindings { c.lock.Lock() defer c.lock.Unlock() values := c.cache.Values() @@ -138,25 +138,25 @@ func (c *bindCache) GetAllBindings() Bindings { // SetBinding sets the Bindings to the cache. // The function is thread-safe. -func (c *bindCache) SetBinding(sqlDigest string, meta Bindings) (err error) { +func (c *bindingCache) SetBinding(sqlDigest string, meta Bindings) (err error) { c.lock.Lock() defer c.lock.Unlock() - cacheKey := bindCacheKey(sqlDigest) + cacheKey := bindingCacheKey(sqlDigest) _, err = c.set(cacheKey, meta) return } // RemoveBinding removes the Bindings which has same originSQL with specified Bindings. // The function is thread-safe. -func (c *bindCache) RemoveBinding(sqlDigest string) { +func (c *bindingCache) RemoveBinding(sqlDigest string) { c.lock.Lock() defer c.lock.Unlock() - c.delete(bindCacheKey(sqlDigest)) + c.delete(bindingCacheKey(sqlDigest)) } // SetMemCapacity sets the memory capacity for the cache. // The function is thread-safe. -func (c *bindCache) SetMemCapacity(capacity int64) { +func (c *bindingCache) SetMemCapacity(capacity int64) { c.lock.Lock() defer c.lock.Unlock() // Only change the capacity size without affecting the cached bindings @@ -165,7 +165,7 @@ func (c *bindCache) SetMemCapacity(capacity int64) { // GetMemUsage get the memory Usage for the cache. // The function is thread-safe. -func (c *bindCache) GetMemUsage() int64 { +func (c *bindingCache) GetMemUsage() int64 { c.lock.Lock() defer c.lock.Unlock() return c.memTracker.BytesConsumed() @@ -173,15 +173,15 @@ func (c *bindCache) GetMemUsage() int64 { // GetMemCapacity get the memory capacity for the cache. // The function is thread-safe. -func (c *bindCache) GetMemCapacity() int64 { +func (c *bindingCache) GetMemCapacity() int64 { c.lock.Lock() defer c.lock.Unlock() return c.memCapacity } -// Copy copies a new bindCache from the origin cache. +// Copy copies a new bindingCache from the origin cache. // The function is thread-safe. -func (c *bindCache) Copy() (newCache *bindCache, err error) { +func (c *bindingCache) Copy() (newCache *bindingCache, err error) { c.lock.Lock() defer c.lock.Unlock() newCache = newBindCache() @@ -190,7 +190,7 @@ func (c *bindCache) Copy() (newCache *bindCache, err error) { } keys := c.cache.Keys() for _, key := range keys { - cacheKey := key.(bindCacheKey) + cacheKey := key.(bindingCacheKey) v := c.get(cacheKey) if _, err := newCache.set(cacheKey, v); err != nil { return nil, err @@ -199,7 +199,7 @@ func (c *bindCache) Copy() (newCache *bindCache, err error) { return newCache, err } -func (c *bindCache) Size() int { +func (c *bindingCache) Size() int { c.lock.Lock() defer c.lock.Unlock() return c.cache.Size() diff --git a/pkg/bindinfo/bind_cache_test.go b/pkg/bindinfo/binding_cache_test.go similarity index 93% rename from pkg/bindinfo/bind_cache_test.go rename to pkg/bindinfo/binding_cache_test.go index c9ec09e1069fa..3eed5f94fe3e5 100644 --- a/pkg/bindinfo/bind_cache_test.go +++ b/pkg/bindinfo/binding_cache_test.go @@ -29,11 +29,11 @@ func TestBindCache(t *testing.T) { bindCache := newBindCache() value := make([]Bindings, 3) - key := make([]bindCacheKey, 3) + key := make([]bindingCacheKey, 3) var bigKey string for i := 0; i < 3; i++ { cacheKey := strings.Repeat(strconv.Itoa(i), 50) - key[i] = bindCacheKey(hack.Slice(cacheKey)) + key[i] = bindingCacheKey(hack.Slice(cacheKey)) value[i] = []Binding{{OriginalSQL: cacheKey}} bigKey += cacheKey @@ -66,7 +66,7 @@ func TestBindCache(t *testing.T) { result = bindCache.get(key[1]) require.NotNil(t, result) - bigBindCacheKey := bindCacheKey(hack.Slice(bigKey)) + bigBindCacheKey := bindingCacheKey(hack.Slice(bigKey)) bigBindCacheValue := []Binding{{OriginalSQL: strings.Repeat("x", 100)}} require.Equal(t, int64(266), calcBindCacheKVMem(bigBindCacheKey, bigBindCacheValue)) ok, err = bindCache.set(bigBindCacheKey, bigBindCacheValue) diff --git a/pkg/bindinfo/capture.go b/pkg/bindinfo/capture.go index f65db8b73d409..8e30a295848c4 100644 --- a/pkg/bindinfo/capture.go +++ b/pkg/bindinfo/capture.go @@ -174,7 +174,7 @@ func (h *globalBindingHandle) CaptureBaselines() { if r := h.getCache().GetBinding(digest.String()); HasAvailableBinding(r) { continue } - bindSQL := GenerateBindSQL(context.TODO(), stmt, bindableStmt.PlanHint, true, dbName) + bindSQL := GenerateBindingSQL(context.TODO(), stmt, bindableStmt.PlanHint, true, dbName) if bindSQL == "" { continue } diff --git a/pkg/bindinfo/global_handle.go b/pkg/bindinfo/global_handle.go index f28f99cb9c70e..9df12fa50d3ba 100644 --- a/pkg/bindinfo/global_handle.go +++ b/pkg/bindinfo/global_handle.go @@ -83,8 +83,8 @@ type GlobalBindingHandle interface { // Size returns the size of bind info cache. Size() int - // SetBindCacheCapacity reset the capacity for the bindCache. - SetBindCacheCapacity(capacity int64) + // SetBindingCacheCapacity reset the capacity for the bindingCache. + SetBindingCacheCapacity(capacity int64) // GetMemUsage returns the memory usage for the bind cache. GetMemUsage() (memUsage int64) @@ -110,7 +110,7 @@ type GlobalBindingHandle interface { type globalBindingHandle struct { sPool SessionPool - bindingCache atomic.Pointer[bindCache] + bindingCache atomic.Pointer[bindingCache] // fuzzyDigestMap is used to support fuzzy matching. // fuzzyDigest is the digest calculated after eliminating all DB names, e.g. `select * from test.t` -> `select * from t` -> fuzzyDigest. @@ -156,11 +156,11 @@ func NewGlobalBindingHandle(sPool SessionPool) GlobalBindingHandle { return handle } -func (h *globalBindingHandle) getCache() *bindCache { +func (h *globalBindingHandle) getCache() *bindingCache { return h.bindingCache.Load() } -func (h *globalBindingHandle) setCache(c *bindCache) { +func (h *globalBindingHandle) setCache(c *bindingCache) { // TODO: update the global cache in-place instead of replacing it and remove this function. h.bindingCache.Store(c) } @@ -209,7 +209,7 @@ func (h *globalBindingHandle) setLastUpdateTime(t types.Time) { func (h *globalBindingHandle) LoadFromStorageToCache(fullLoad bool) (err error) { var lastUpdateTime types.Time var timeCondition string - var newCache *bindCache + var newCache *bindingCache if fullLoad { lastUpdateTime = types.ZeroTimestamp timeCondition = "" @@ -257,10 +257,10 @@ func (h *globalBindingHandle) LoadFromStorageToCache(fullLoad bool) (err error) continue } - oldRecord := newCache.GetBinding(sqlDigest) - newRecord := removeDeletedBindings(merge(oldRecord, []Binding{binding})) - if len(newRecord) > 0 { - err = newCache.SetBinding(sqlDigest, newRecord) + oldBinding := newCache.GetBinding(sqlDigest) + newBinding := removeDeletedBindings(merge(oldBinding, []Binding{binding})) + if len(newBinding) > 0 { + err = newCache.SetBinding(sqlDigest, newBinding) if err != nil { // When the memory capacity of bing_cache is not enough, // there will be some memory-related errors in multiple places. @@ -270,7 +270,7 @@ func (h *globalBindingHandle) LoadFromStorageToCache(fullLoad bool) (err error) } else { newCache.RemoveBinding(sqlDigest) } - updateMetrics(metrics.ScopeGlobal, oldRecord, newCache.GetBinding(sqlDigest), true) + updateMetrics(metrics.ScopeGlobal, oldBinding, newCache.GetBinding(sqlDigest), true) } return nil }) @@ -529,9 +529,9 @@ func (h *globalBindingHandle) GetAllGlobalBindings() (bindings Bindings) { return h.getCache().GetAllBindings() } -// SetBindCacheCapacity reset the capacity for the bindCache. +// SetBindingCacheCapacity reset the capacity for the bindingCache. // It will not affect already cached Bindings. -func (h *globalBindingHandle) SetBindCacheCapacity(capacity int64) { +func (h *globalBindingHandle) SetBindingCacheCapacity(capacity int64) { h.getCache().SetMemCapacity(capacity) } @@ -611,8 +611,8 @@ func getHintsForSQL(sctx sessionctx.Context, sql string) (string, error) { 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, skipCheckIfHasParam bool, defaultDB string) string { +// GenerateBindingSQL generates binding sqls from stmt node and plan hints. +func GenerateBindingSQL(ctx context.Context, stmtNode ast.StmtNode, planHint string, skipCheckIfHasParam bool, defaultDB 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 "" diff --git a/pkg/bindinfo/session_handle.go b/pkg/bindinfo/session_handle.go index 98d8a43f893bf..0328b9d79844d 100644 --- a/pkg/bindinfo/session_handle.go +++ b/pkg/bindinfo/session_handle.go @@ -56,7 +56,7 @@ type SessionBindingHandle interface { // sessionBindingHandle is used to handle all session sql bind operations. type sessionBindingHandle struct { - ch *bindCache + ch *bindingCache } // NewSessionBindingHandle creates a new SessionBindingHandle. diff --git a/pkg/planner/core/planbuilder.go b/pkg/planner/core/planbuilder.go index f60679ea5362c..a07577ae57a4f 100644 --- a/pkg/planner/core/planbuilder.go +++ b/pkg/planner/core/planbuilder.go @@ -850,7 +850,7 @@ func (b *PlanBuilder) buildCreateBindPlanFromPlanDigest(v *ast.CreateBindingStmt if err = hint.CheckBindingFromHistoryBindable(originNode, bindableStmt.PlanHint); err != nil { return nil, err } - bindSQL := bindinfo.GenerateBindSQL(context.TODO(), originNode, bindableStmt.PlanHint, true, bindableStmt.Schema) + bindSQL := bindinfo.GenerateBindingSQL(context.TODO(), originNode, bindableStmt.PlanHint, true, bindableStmt.Schema) var hintNode ast.StmtNode hintNode, err = parser4binding.ParseOneStmt(bindSQL, bindableStmt.Charset, bindableStmt.Collation) if err != nil {