Skip to content

Commit

Permalink
planner: clean BindRecord in the binding package (#50590)
Browse files Browse the repository at this point in the history
ref #48875
  • Loading branch information
qw4990 authored Jan 19, 2024
1 parent d8298d5 commit 4539d60
Show file tree
Hide file tree
Showing 7 changed files with 42 additions and 46 deletions.
24 changes: 12 additions & 12 deletions pkg/bindinfo/bind_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ import (
"github.com/pingcap/tidb/pkg/util/memory"
)

// bindCache uses the LRU cache to store the bindRecord.
// bindCache 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 {
Expand Down Expand Up @@ -81,10 +81,10 @@ func (c *bindCache) set(key bindCacheKey, value Bindings) (ok bool, err error) {
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")
return
}
bindRecords := c.get(key)
if bindRecords != nil {
bindings := c.get(key)
if bindings != nil {
// Remove the origin key-value pair.
mem -= calcBindCacheKVMem(key, bindRecords)
mem -= calcBindCacheKVMem(key, bindings)
}
for mem+c.memTracker.BytesConsumed() > c.memCapacity {
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")
Expand All @@ -103,9 +103,9 @@ 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 {
bindRecords := c.get(key)
if bindRecords != nil {
mem := calcBindCacheKVMem(key, bindRecords)
bindings := c.get(key)
if bindings != nil {
mem := calcBindCacheKVMem(key, bindings)
c.cache.Delete(key)
c.memTracker.Consume(-mem)
return true
Expand All @@ -122,18 +122,18 @@ func (c *bindCache) GetBinding(sqlDigest string) Bindings {
return c.get(bindCacheKey(sqlDigest))
}

// GetAllBindings return all the bindRecords from the bindCache.
// GetAllBindings return all the bindings 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) GetAllBindings() Bindings {
c.lock.Lock()
defer c.lock.Unlock()
values := c.cache.Values()
bindRecords := make(Bindings, 0, len(values))
bindings := make(Bindings, 0, len(values))
for _, vals := range values {
bindRecords = append(bindRecords, vals.(Bindings)...)
bindings = append(bindings, vals.(Bindings)...)
}
return bindRecords
return bindings
}

// SetBinding sets the Bindings to the cache.
Expand All @@ -159,7 +159,7 @@ func (c *bindCache) RemoveBinding(sqlDigest string) {
func (c *bindCache) SetMemCapacity(capacity int64) {
c.lock.Lock()
defer c.lock.Unlock()
// Only change the capacity size without affecting the cached bindRecord
// Only change the capacity size without affecting the cached bindings
c.memCapacity = capacity
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/bindinfo/binding.go
Original file line number Diff line number Diff line change
Expand Up @@ -108,7 +108,7 @@ func (b *Binding) SinceUpdateTime() (time.Duration, error) {
// Bindings represents a sql bind record retrieved from the storage.
type Bindings []Binding

// Copy get the copy of bindRecord
// Copy get the copy of bindings
func (br Bindings) Copy() Bindings {
nbr := append(make(Bindings, 0, len(br)), br...)
return nbr
Expand Down
18 changes: 9 additions & 9 deletions pkg/bindinfo/global_handle.go
Original file line number Diff line number Diff line change
Expand Up @@ -61,7 +61,7 @@ type GlobalBindingHandle interface {
// SetGlobalBindingStatus set a Bindings's status to the storage and bind cache.
SetGlobalBindingStatus(newStatus, sqlDigest string) (ok bool, err error)

// AddInvalidGlobalBinding adds Bindings which needs to be deleted into invalidBindRecordMap.
// AddInvalidGlobalBinding adds Bindings which needs to be deleted into invalidBindingCache.
AddInvalidGlobalBinding(invalidBinding Binding)

// DropInvalidGlobalBinding executes the drop Bindings tasks.
Expand Down Expand Up @@ -121,7 +121,7 @@ type globalBindingHandle struct {
// This value is used to avoid reload duplicated bindings from storage.
lastUpdateTime atomic.Value

// invalidBindRecordMap indicates the invalid bindings found during querying.
// invalidBindings indicates the invalid bindings found during querying.
// A binding will be deleted from this map, after 2 bind-lease, after it is dropped from the kv.
invalidBindings *invalidBindingCache
}
Expand Down Expand Up @@ -173,10 +173,10 @@ func (h *globalBindingHandle) setFuzzyDigestMap(m map[string][]string) {
h.fuzzyDigestMap.Store(m)
}

func buildFuzzyDigestMap(bindRecords Bindings) map[string][]string {
func buildFuzzyDigestMap(bindings Bindings) map[string][]string {
m := make(map[string][]string)
p := parser.New()
for _, binding := range bindRecords {
for _, binding := range bindings {
stmt, err := p.ParseOneStmt(binding.BindSQL, binding.Charset, binding.Collation)
if err != nil {
logutil.BgLogger().Warn("parse bindSQL failed", zap.String("bindSQL", binding.BindSQL), zap.Error(err))
Expand Down Expand Up @@ -289,7 +289,7 @@ func (h *globalBindingHandle) CreateGlobalBinding(sctx sessionctx.Context, bindi
}()

return h.callWithSCtx(true, func(sctx sessionctx.Context) error {
// Lock mysql.bind_info to synchronize with CreateBindRecord / AddBindRecord / DropBindRecord on other tidb instances.
// Lock mysql.bind_info to synchronize with CreateBinding / AddBinding / DropBinding on other tidb instances.
if err = lockBindInfoTable(sctx); err != nil {
return err
}
Expand Down Expand Up @@ -330,7 +330,7 @@ func (h *globalBindingHandle) CreateGlobalBinding(sctx sessionctx.Context, bindi
// dropGlobalBinding drops a Bindings to the storage and Bindings int the cache.
func (h *globalBindingHandle) dropGlobalBinding(sqlDigest string) (deletedRows uint64, err error) {
err = h.callWithSCtx(false, func(sctx sessionctx.Context) error {
// Lock mysql.bind_info to synchronize with CreateBindRecord / AddBindRecord / DropBindRecord on other tidb instances.
// Lock mysql.bind_info to synchronize with CreateBinding / AddBinding / DropBinding on other tidb instances.
if err = lockBindInfoTable(sctx); err != nil {
return err
}
Expand Down Expand Up @@ -403,7 +403,7 @@ func (h *globalBindingHandle) SetGlobalBindingStatus(newStatus, sqlDigest string
// GCGlobalBinding physically removes the deleted bind records in mysql.bind_info.
func (h *globalBindingHandle) GCGlobalBinding() (err error) {
return h.callWithSCtx(true, func(sctx sessionctx.Context) error {
// Lock mysql.bind_info to synchronize with CreateBindRecord / AddBindRecord / DropBindRecord on other tidb instances.
// Lock mysql.bind_info to synchronize with CreateBinding / AddBinding / DropBinding on other tidb instances.
if err = lockBindInfoTable(sctx); err != nil {
return err
}
Expand Down Expand Up @@ -480,7 +480,7 @@ func (h *globalBindingHandle) DropInvalidGlobalBinding() {
}
}

// AddInvalidGlobalBinding adds Bindings which needs to be deleted into invalidBindRecordMap.
// AddInvalidGlobalBinding adds Bindings which needs to be deleted into invalidBindings.
func (h *globalBindingHandle) AddInvalidGlobalBinding(invalidBinding Binding) {
h.invalidBindings.add(invalidBinding)
}
Expand Down Expand Up @@ -530,7 +530,7 @@ func (h *globalBindingHandle) GetAllGlobalBindings() (bindings Bindings) {
}

// SetBindCacheCapacity reset the capacity for the bindCache.
// It will not affect already cached BindRecords.
// It will not affect already cached Bindings.
func (h *globalBindingHandle) SetBindCacheCapacity(capacity int64) {
h.getCache().SetMemCapacity(capacity)
}
Expand Down
12 changes: 6 additions & 6 deletions pkg/bindinfo/session_handle.go
Original file line number Diff line number Diff line change
Expand Up @@ -72,7 +72,7 @@ func (h *sessionBindingHandle) appendSessionBinding(sqlDigest string, meta Bindi
oldBindings := h.ch.GetBinding(sqlDigest)
err := h.ch.SetBinding(sqlDigest, meta)
if err != nil {
logutil.BgLogger().Warn("SessionHandle.appendBindRecord", zap.String("category", "sql-bind"), zap.Error(err))
logutil.BgLogger().Warn("SessionHandle.appendSessionBinding", zap.String("category", "sql-bind"), zap.Error(err))
}
updateMetrics(metrics.ScopeSession, oldBindings, meta, false)
}
Expand Down Expand Up @@ -107,9 +107,9 @@ func (h *sessionBindingHandle) MatchSessionBinding(sctx sessionctx.Context, fuzz
// The current implementation is simplistic, but session binding is only for test purpose, so
// there shouldn't be many session bindings, and to keep it simple, this implementation is acceptable.
leastWildcards := len(tableNames) + 1
bindRecords := h.ch.GetAllBindings()
bindings := h.ch.GetAllBindings()
enableFuzzyBinding := sctx.GetSessionVars().EnableFuzzyBinding
for _, binding := range bindRecords {
for _, binding := range bindings {
bindingStmt, err := parser.New().ParseOneStmt(binding.BindSQL, binding.Charset, binding.Collation)
if err != nil {
return
Expand Down Expand Up @@ -140,11 +140,11 @@ func (h *sessionBindingHandle) GetAllSessionBindings() (bindings Bindings) {

// EncodeSessionStates implements SessionStatesHandler.EncodeSessionStates interface.
func (h *sessionBindingHandle) EncodeSessionStates(_ context.Context, _ sessionctx.Context, sessionStates *sessionstates.SessionStates) error {
bindRecords := h.ch.GetAllBindings()
if len(bindRecords) == 0 {
bindings := h.ch.GetAllBindings()
if len(bindings) == 0 {
return nil
}
bytes, err := json.Marshal([]Binding(bindRecords))
bytes, err := json.Marshal([]Binding(bindings))
if err != nil {
return err
}
Expand Down
6 changes: 1 addition & 5 deletions pkg/executor/adapter.go
Original file line number Diff line number Diff line change
Expand Up @@ -2159,15 +2159,11 @@ func sendPlanReplayerDumpTask(key replayer.PlanReplayerTaskKey, sctx sessionctx.
stmtCtx := sctx.GetSessionVars().StmtCtx
handle := sctx.Value(bindinfo.SessionBindInfoKeyType).(bindinfo.SessionBindingHandle)
bindings := handle.GetAllSessionBindings()
bindRecords := make([]bindinfo.Bindings, 0, len(bindings))
for _, binding := range bindings {
bindRecords = append(bindRecords, []bindinfo.Binding{binding})
}
dumpTask := &domain.PlanReplayerDumpTask{
PlanReplayerTaskKey: key,
StartTS: startTS,
TblStats: stmtCtx.TableStats,
SessionBindings: bindRecords,
SessionBindings: []bindinfo.Bindings{bindings},
SessionVars: sctx.GetSessionVars(),
ExecStmts: []ast.StmtNode{stmtNode},
DebugTrace: []interface{}{stmtCtx.OptimizerDebugTrace},
Expand Down
4 changes: 2 additions & 2 deletions pkg/executor/show.go
Original file line number Diff line number Diff line change
Expand Up @@ -328,9 +328,9 @@ func (e *ShowExec) fetchShowBind() error {
} else {
bindings = domain.GetDomain(e.Ctx()).BindHandle().GetAllGlobalBindings()
}
// Remove the invalid bindRecord.
// Remove the invalid bindings.
parser := parser.New()
// For the different origin_sql, sort the bindRecords according to their max update time.
// For the different origin_sql, sort the bindings according to their max update time.
sort.Slice(bindings, func(i int, j int) bool {
cmpResult := bindings[i].UpdateTime.Compare(bindings[j].UpdateTime)
if cmpResult == 0 {
Expand Down
22 changes: 11 additions & 11 deletions pkg/planner/optimize.go
Original file line number Diff line number Diff line change
Expand Up @@ -220,19 +220,19 @@ func Optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is in
enableUseBinding := sessVars.UsePlanBaselines
stmtNode, isStmtNode := node.(ast.StmtNode)
binding, match, scope := bindinfo.MatchSQLBinding(sctx, stmtNode)
var bindRecord bindinfo.Bindings
var bindings bindinfo.Bindings
if match {
bindRecord = []bindinfo.Binding{binding}
bindings = []bindinfo.Binding{binding}
}

useBinding := enableUseBinding && isStmtNode && match
if sessVars.StmtCtx.EnableOptimizerDebugTrace {
failpoint.Inject("SetBindingTimeToZero", func(val failpoint.Value) {
if val.(bool) && bindRecord != nil {
bindRecord = bindRecord.Copy()
for i := range bindRecord {
bindRecord[i].CreateTime = types.ZeroTime
bindRecord[i].UpdateTime = types.ZeroTime
if val.(bool) && bindings != nil {
bindings = bindings.Copy()
for i := range bindings {
bindings[i].CreateTime = types.ZeroTime
bindings[i].UpdateTime = types.ZeroTime
}
}
})
Expand All @@ -242,7 +242,7 @@ func Optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is in
"IsStmtNode", isStmtNode,
"Matched", match,
"Scope", scope,
"Matched bindings", bindRecord,
"Matched bindings", bindings,
)
}
if isStmtNode {
Expand Down Expand Up @@ -274,8 +274,8 @@ func Optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is in
minCost := math.MaxFloat64
var bindStmtHints stmtctx.StmtHints
originHints := hint.CollectHint(stmtNode)
// bindRecord must be not nil when coming here, try to find the best binding.
for _, binding := range bindRecord {
// bindings must be not nil when coming here, try to find the best binding.
for _, binding := range bindings {
if !binding.IsBindingEnabled() {
continue
}
Expand Down Expand Up @@ -352,7 +352,7 @@ func Optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is in
if sessVars.EvolvePlanBaselines && bestPlanFromBind != nil &&
sessVars.SelectLimit == math.MaxUint64 { // do not evolve this query if sql_select_limit is enabled
// Check bestPlanFromBind firstly to avoid nil stmtNode.
if _, ok := stmtNode.(*ast.SelectStmt); ok && !bindRecord[0].Hint.ContainTableHint(hint.HintReadFromStorage) {
if _, ok := stmtNode.(*ast.SelectStmt); ok && !bindings[0].Hint.ContainTableHint(hint.HintReadFromStorage) {
sessVars.StmtCtx.StmtHints = originStmtHints
defPlan, _, _, err := optimize(ctx, sctx, node, is)
if err != nil {
Expand Down

0 comments on commit 4539d60

Please sign in to comment.