planner: refactor some code related to binding (#59883)
ref pingcap/tidb#51347
This commit is contained in:
@ -151,7 +151,7 @@ func matchSQLBinding(sctx sessionctx.Context, stmtNode ast.StmtNode, info *Bindi
|
||||
if globalHandle == nil {
|
||||
return
|
||||
}
|
||||
binding, matched = globalHandle.MatchGlobalBinding(sctx, noDBDigest, tableNames)
|
||||
binding, matched = globalHandle.MatchingBinding(sctx, noDBDigest, tableNames)
|
||||
if matched {
|
||||
return binding, matched, metrics.ScopeGlobal
|
||||
}
|
||||
|
||||
@ -15,15 +15,128 @@
|
||||
package bindinfo
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
|
||||
"github.com/dgraph-io/ristretto"
|
||||
"github.com/pingcap/tidb/pkg/metrics"
|
||||
"github.com/pingcap/tidb/pkg/parser"
|
||||
"github.com/pingcap/tidb/pkg/parser/ast"
|
||||
"github.com/pingcap/tidb/pkg/sessionctx"
|
||||
"github.com/pingcap/tidb/pkg/sessionctx/vardef"
|
||||
"github.com/pingcap/tidb/pkg/types"
|
||||
"github.com/pingcap/tidb/pkg/util"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
// BindingCacheUpdater maintains the binding cache and provide update APIs.
|
||||
type BindingCacheUpdater interface {
|
||||
BindingCache
|
||||
|
||||
// LoadFromStorageToCache loads global bindings from storage to the memory cache.
|
||||
LoadFromStorageToCache(fullLoad bool) (err error)
|
||||
|
||||
// LastUpdateTime returns the last update time.
|
||||
LastUpdateTime() types.Time
|
||||
}
|
||||
|
||||
type bindingCacheUpdater struct {
|
||||
BindingCache
|
||||
|
||||
sPool util.DestroyableSessionPool
|
||||
|
||||
// lastTaskTime records the last update time for the global sql bind cache.
|
||||
// This value is used to avoid reload duplicated bindings from storage.
|
||||
lastUpdateTime atomic.Value
|
||||
}
|
||||
|
||||
// LoadFromStorageToCache loads bindings from the storage into the cache.
|
||||
func (u *bindingCacheUpdater) LoadFromStorageToCache(fullLoad bool) (err error) {
|
||||
var lastUpdateTime types.Time
|
||||
var timeCondition string
|
||||
if fullLoad {
|
||||
lastUpdateTime = types.ZeroTimestamp
|
||||
timeCondition = ""
|
||||
} else {
|
||||
lastUpdateTime = u.lastUpdateTime.Load().(types.Time)
|
||||
timeCondition = fmt.Sprintf("WHERE update_time>'%s'", lastUpdateTime.String())
|
||||
}
|
||||
condition := fmt.Sprintf(`%s ORDER BY update_time, create_time`, timeCondition)
|
||||
bindings, err := u.readBindingsFromStorage(condition)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
for _, binding := range bindings {
|
||||
// Update lastUpdateTime to the newest one.
|
||||
// Even if this one is an invalid bind.
|
||||
if binding.UpdateTime.Compare(lastUpdateTime) > 0 {
|
||||
lastUpdateTime = binding.UpdateTime
|
||||
}
|
||||
|
||||
oldBinding := u.GetBinding(binding.SQLDigest)
|
||||
cachedBinding := pickCachedBinding(oldBinding, binding)
|
||||
if cachedBinding != nil {
|
||||
err = u.SetBinding(binding.SQLDigest, cachedBinding)
|
||||
if err != nil {
|
||||
bindingLogger().Warn("BindingHandle.Update", zap.Error(err))
|
||||
}
|
||||
} else {
|
||||
u.RemoveBinding(binding.SQLDigest)
|
||||
}
|
||||
}
|
||||
|
||||
// update last-update-time and metrics
|
||||
u.lastUpdateTime.Store(lastUpdateTime)
|
||||
metrics.BindingCacheMemUsage.Set(float64(u.GetMemUsage()))
|
||||
metrics.BindingCacheMemLimit.Set(float64(u.GetMemCapacity()))
|
||||
metrics.BindingCacheNumBindings.Set(float64(len(u.GetAllBindings())))
|
||||
return nil
|
||||
}
|
||||
|
||||
// LastUpdateTime returns the last update time.
|
||||
func (u *bindingCacheUpdater) LastUpdateTime() types.Time {
|
||||
return u.lastUpdateTime.Load().(types.Time)
|
||||
}
|
||||
|
||||
func (u *bindingCacheUpdater) readBindingsFromStorage(condition string) (bindings []*Binding, err error) {
|
||||
selectStmt := fmt.Sprintf(`SELECT original_sql, bind_sql, default_db, status, create_time,
|
||||
update_time, charset, collation, source, sql_digest, plan_digest FROM mysql.bind_info
|
||||
%s`, condition)
|
||||
|
||||
err = callWithSCtx(u.sPool, false, func(sctx sessionctx.Context) error {
|
||||
rows, _, err := execRows(sctx, selectStmt)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
bindings = make([]*Binding, 0, len(rows))
|
||||
for _, row := range rows {
|
||||
// Skip the builtin record which is designed for binding synchronization.
|
||||
if row.GetString(0) == BuiltinPseudoSQL4BindLock {
|
||||
continue
|
||||
}
|
||||
binding := newBindingFromStorage(row)
|
||||
if hErr := prepareHints(sctx, binding); hErr != nil {
|
||||
bindingLogger().Warn("failed to generate bind record from data row", zap.Error(hErr))
|
||||
continue
|
||||
}
|
||||
bindings = append(bindings, binding)
|
||||
}
|
||||
return nil
|
||||
})
|
||||
return
|
||||
}
|
||||
|
||||
// NewBindingCacheUpdater creates a new BindingCacheUpdater.
|
||||
func NewBindingCacheUpdater(sPool util.DestroyableSessionPool) BindingCacheUpdater {
|
||||
u := new(bindingCacheUpdater)
|
||||
u.lastUpdateTime.Store(types.ZeroTimestamp)
|
||||
u.sPool = sPool
|
||||
u.BindingCache = newBindCache()
|
||||
return u
|
||||
}
|
||||
|
||||
// digestBiMap represents a bidirectional map between noDBDigest and sqlDigest, used to support cross-db binding.
|
||||
// One noDBDigest can map to multiple sqlDigests, but one sqlDigest can only map to one noDBDigest.
|
||||
type digestBiMap interface {
|
||||
|
||||
@ -17,12 +17,10 @@ package bindinfo
|
||||
import (
|
||||
"fmt"
|
||||
"strings"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/failpoint"
|
||||
"github.com/pingcap/tidb/pkg/metrics"
|
||||
"github.com/pingcap/tidb/pkg/parser/ast"
|
||||
"github.com/pingcap/tidb/pkg/parser/format"
|
||||
"github.com/pingcap/tidb/pkg/parser/mysql"
|
||||
@ -38,13 +36,7 @@ import (
|
||||
|
||||
// GlobalBindingHandle is used to handle all global sql bind operations.
|
||||
type GlobalBindingHandle interface {
|
||||
// Methods for create, get, drop global sql bindings.
|
||||
|
||||
// MatchGlobalBinding returns the matched binding for this statement.
|
||||
MatchGlobalBinding(sctx sessionctx.Context, noDBDigest string, tableNames []*ast.TableName) (matchedBinding *Binding, isMatched bool)
|
||||
|
||||
// GetAllGlobalBindings returns all bind records in cache.
|
||||
GetAllGlobalBindings() (bindings []*Binding)
|
||||
BindingCacheUpdater
|
||||
|
||||
// CreateGlobalBinding creates a Bindings to the storage and the cache.
|
||||
// It replaces all the exists bindings for the same normalized SQL.
|
||||
@ -56,36 +48,16 @@ type GlobalBindingHandle interface {
|
||||
// SetGlobalBindingStatus set a Bindings's status to the storage and bind cache.
|
||||
SetGlobalBindingStatus(newStatus, sqlDigest string) (ok bool, err error)
|
||||
|
||||
// Methods for load and clear global sql bindings.
|
||||
|
||||
// LoadFromStorageToCache loads global bindings from storage to the memory cache.
|
||||
LoadFromStorageToCache(fullLoad bool) (err error)
|
||||
|
||||
// GCGlobalBinding physically removes the deleted bind records in mysql.bind_info.
|
||||
GCGlobalBinding() (err error)
|
||||
|
||||
// Methods for memory control.
|
||||
|
||||
// GetMemUsage returns the memory usage for the bind cache.
|
||||
GetMemUsage() (memUsage int64)
|
||||
|
||||
// GetMemCapacity returns the memory capacity for the bind cache.
|
||||
GetMemCapacity() (memCapacity int64)
|
||||
|
||||
// Close closes the binding handler.
|
||||
Close()
|
||||
|
||||
variable.Statistics
|
||||
}
|
||||
|
||||
// globalBindingHandle is used to handle all global sql bind operations.
|
||||
type globalBindingHandle struct {
|
||||
sPool util.DestroyableSessionPool
|
||||
bindingCache BindingCache
|
||||
|
||||
// lastTaskTime records the last update time for the global sql bind cache.
|
||||
// This value is used to avoid reload duplicated bindings from storage.
|
||||
lastUpdateTime atomic.Value
|
||||
BindingCacheUpdater
|
||||
sPool util.DestroyableSessionPool
|
||||
}
|
||||
|
||||
// Lease influences the duration of loading bind info and handling invalid bind.
|
||||
@ -113,85 +85,12 @@ const (
|
||||
|
||||
// NewGlobalBindingHandle creates a new GlobalBindingHandle.
|
||||
func NewGlobalBindingHandle(sPool util.DestroyableSessionPool) GlobalBindingHandle {
|
||||
h := &globalBindingHandle{sPool: sPool}
|
||||
h.lastUpdateTime.Store(types.ZeroTimestamp)
|
||||
h.bindingCache = newBindCache()
|
||||
cache := NewBindingCacheUpdater(sPool)
|
||||
h := &globalBindingHandle{sPool: sPool, BindingCacheUpdater: cache}
|
||||
variable.RegisterStatistics(h)
|
||||
return h
|
||||
}
|
||||
|
||||
// LoadFromStorageToCache loads bindings from the storage into the cache.
|
||||
func (h *globalBindingHandle) LoadFromStorageToCache(fullLoad bool) (err error) {
|
||||
var lastUpdateTime types.Time
|
||||
var timeCondition string
|
||||
if fullLoad {
|
||||
lastUpdateTime = types.ZeroTimestamp
|
||||
timeCondition = ""
|
||||
} else {
|
||||
lastUpdateTime = h.lastUpdateTime.Load().(types.Time)
|
||||
timeCondition = fmt.Sprintf("WHERE update_time>'%s'", lastUpdateTime.String())
|
||||
}
|
||||
condition := fmt.Sprintf(`%s ORDER BY update_time, create_time`, timeCondition)
|
||||
bindings, err := h.readBindingsFromStorage(condition)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
for _, binding := range bindings {
|
||||
// Update lastUpdateTime to the newest one.
|
||||
// Even if this one is an invalid bind.
|
||||
if binding.UpdateTime.Compare(lastUpdateTime) > 0 {
|
||||
lastUpdateTime = binding.UpdateTime
|
||||
}
|
||||
|
||||
oldBinding := h.bindingCache.GetBinding(binding.SQLDigest)
|
||||
cachedBinding := pickCachedBinding(oldBinding, binding)
|
||||
if cachedBinding != nil {
|
||||
err = h.bindingCache.SetBinding(binding.SQLDigest, cachedBinding)
|
||||
if err != nil {
|
||||
bindingLogger().Warn("BindingHandle.Update", zap.Error(err))
|
||||
}
|
||||
} else {
|
||||
h.bindingCache.RemoveBinding(binding.SQLDigest)
|
||||
}
|
||||
}
|
||||
|
||||
// update last-update-time and metrics
|
||||
h.lastUpdateTime.Store(lastUpdateTime)
|
||||
metrics.BindingCacheMemUsage.Set(float64(h.GetMemUsage()))
|
||||
metrics.BindingCacheMemLimit.Set(float64(h.GetMemCapacity()))
|
||||
metrics.BindingCacheNumBindings.Set(float64(len(h.bindingCache.GetAllBindings())))
|
||||
return nil
|
||||
}
|
||||
|
||||
func (h *globalBindingHandle) readBindingsFromStorage(condition string) (bindings []*Binding, err error) {
|
||||
selectStmt := fmt.Sprintf(`SELECT original_sql, bind_sql, default_db, status, create_time,
|
||||
update_time, charset, collation, source, sql_digest, plan_digest FROM mysql.bind_info
|
||||
%s`, condition)
|
||||
|
||||
err = callWithSCtx(h.sPool, false, func(sctx sessionctx.Context) error {
|
||||
rows, _, err := execRows(sctx, selectStmt)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
bindings = make([]*Binding, 0, len(rows))
|
||||
for _, row := range rows {
|
||||
// Skip the builtin record which is designed for binding synchronization.
|
||||
if row.GetString(0) == BuiltinPseudoSQL4BindLock {
|
||||
continue
|
||||
}
|
||||
binding := newBindingFromStorage(row)
|
||||
if hErr := prepareHints(sctx, binding); hErr != nil {
|
||||
bindingLogger().Warn("failed to generate bind record from data row", zap.Error(hErr))
|
||||
continue
|
||||
}
|
||||
bindings = append(bindings, binding)
|
||||
}
|
||||
return nil
|
||||
})
|
||||
return
|
||||
}
|
||||
|
||||
// CreateGlobalBinding creates a Bindings to the storage and the cache.
|
||||
// It replaces all the exists bindings for the same normalized SQL.
|
||||
func (h *globalBindingHandle) CreateGlobalBinding(sctx sessionctx.Context, bindings []*Binding) (err error) {
|
||||
@ -370,26 +269,6 @@ func lockBindInfoTable(sctx sessionctx.Context) error {
|
||||
return err
|
||||
}
|
||||
|
||||
// MatchGlobalBinding returns the matched binding for this statement.
|
||||
func (h *globalBindingHandle) MatchGlobalBinding(sctx sessionctx.Context, noDBDigest string, tableNames []*ast.TableName) (matchedBinding *Binding, isMatched bool) {
|
||||
return h.bindingCache.MatchingBinding(sctx, noDBDigest, tableNames)
|
||||
}
|
||||
|
||||
// GetAllGlobalBindings returns all bind records in cache.
|
||||
func (h *globalBindingHandle) GetAllGlobalBindings() (bindings []*Binding) {
|
||||
return h.bindingCache.GetAllBindings()
|
||||
}
|
||||
|
||||
// GetMemUsage returns the memory usage for the bind cache.
|
||||
func (h *globalBindingHandle) GetMemUsage() (memUsage int64) {
|
||||
return h.bindingCache.GetMemUsage()
|
||||
}
|
||||
|
||||
// GetMemCapacity returns the memory capacity for the bind cache.
|
||||
func (h *globalBindingHandle) GetMemCapacity() (memCapacity int64) {
|
||||
return h.bindingCache.GetMemCapacity()
|
||||
}
|
||||
|
||||
// newBindingFromStorage builds Bindings from a tuple in storage.
|
||||
func newBindingFromStorage(row chunk.Row) *Binding {
|
||||
status := row.GetString(3)
|
||||
@ -482,12 +361,12 @@ func (*globalBindingHandle) GetScope(_ string) vardef.ScopeFlag {
|
||||
// Stats returns the server statistics.
|
||||
func (h *globalBindingHandle) Stats(_ *variable.SessionVars) (map[string]any, error) {
|
||||
m := make(map[string]any)
|
||||
m[lastPlanBindingUpdateTime] = h.lastUpdateTime.Load().(types.Time).String()
|
||||
m[lastPlanBindingUpdateTime] = h.LastUpdateTime().String()
|
||||
return m, nil
|
||||
}
|
||||
|
||||
// Close closes the binding handler.
|
||||
func (h *globalBindingHandle) Close() {
|
||||
h.bindingCache.Close()
|
||||
h.BindingCacheUpdater.Close()
|
||||
h.sPool.Close()
|
||||
}
|
||||
|
||||
@ -52,7 +52,7 @@ func TestBindingCache(t *testing.T) {
|
||||
|
||||
tk.MustExec("drop global binding for select * from t;")
|
||||
require.Nil(t, dom.BindingHandle().LoadFromStorageToCache(false))
|
||||
require.Equal(t, 1, len(dom.BindingHandle().GetAllGlobalBindings()))
|
||||
require.Equal(t, 1, len(dom.BindingHandle().GetAllBindings()))
|
||||
}
|
||||
|
||||
func TestBindingLastUpdateTime(t *testing.T) {
|
||||
@ -73,7 +73,7 @@ func TestBindingLastUpdateTime(t *testing.T) {
|
||||
require.NoError(t, err)
|
||||
|
||||
_, noDBDigest := bindinfo.NormalizeStmtForBinding(stmt, "", true)
|
||||
binding, matched := bindHandle.MatchGlobalBinding(tk.Session(), noDBDigest, bindinfo.CollectTableNames(stmt))
|
||||
binding, matched := bindHandle.MatchingBinding(tk.Session(), noDBDigest, bindinfo.CollectTableNames(stmt))
|
||||
require.True(t, matched)
|
||||
updateTime := binding.UpdateTime.String()
|
||||
|
||||
@ -130,12 +130,12 @@ func TestBindParse(t *testing.T) {
|
||||
bindHandle := bindinfo.NewGlobalBindingHandle(&mockSessionPool{tk.Session()})
|
||||
err := bindHandle.LoadFromStorageToCache(true)
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, 1, len(bindHandle.GetAllGlobalBindings()))
|
||||
require.Equal(t, 1, len(bindHandle.GetAllBindings()))
|
||||
|
||||
stmt, err := parser.New().ParseOneStmt("select * from test . t", "", "")
|
||||
require.NoError(t, err)
|
||||
_, noDBDigest := bindinfo.NormalizeStmtForBinding(stmt, "", true)
|
||||
binding, matched := bindHandle.MatchGlobalBinding(tk.Session(), noDBDigest, bindinfo.CollectTableNames(stmt))
|
||||
binding, matched := bindHandle.MatchingBinding(tk.Session(), noDBDigest, bindinfo.CollectTableNames(stmt))
|
||||
require.True(t, matched)
|
||||
require.Equal(t, "select * from `test` . `t`", binding.OriginalSQL)
|
||||
require.Equal(t, "select * from `test` . `t` use index(index_t)", binding.BindSQL)
|
||||
@ -431,7 +431,7 @@ func TestGlobalBinding(t *testing.T) {
|
||||
stmt, _, _ := utilNormalizeWithDefaultDB(t, testSQL.querySQL)
|
||||
|
||||
_, noDBDigest := bindinfo.NormalizeStmtForBinding(stmt, "", true)
|
||||
binding, matched := dom.BindingHandle().MatchGlobalBinding(tk.Session(), noDBDigest, bindinfo.CollectTableNames(stmt))
|
||||
binding, matched := dom.BindingHandle().MatchingBinding(tk.Session(), noDBDigest, bindinfo.CollectTableNames(stmt))
|
||||
require.True(t, matched)
|
||||
require.Equal(t, testSQL.originSQL, binding.OriginalSQL)
|
||||
require.Equal(t, testSQL.bindSQL, binding.BindSQL)
|
||||
@ -461,10 +461,10 @@ func TestGlobalBinding(t *testing.T) {
|
||||
bindHandle := bindinfo.NewGlobalBindingHandle(&mockSessionPool{tk.Session()})
|
||||
err = bindHandle.LoadFromStorageToCache(true)
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, 1, len(bindHandle.GetAllGlobalBindings()))
|
||||
require.Equal(t, 1, len(bindHandle.GetAllBindings()))
|
||||
|
||||
_, noDBDigest = bindinfo.NormalizeStmtForBinding(stmt, "", true)
|
||||
binding, matched = dom.BindingHandle().MatchGlobalBinding(tk.Session(), noDBDigest, bindinfo.CollectTableNames(stmt))
|
||||
binding, matched = dom.BindingHandle().MatchingBinding(tk.Session(), noDBDigest, bindinfo.CollectTableNames(stmt))
|
||||
require.True(t, matched)
|
||||
require.Equal(t, testSQL.originSQL, binding.OriginalSQL)
|
||||
require.Equal(t, testSQL.bindSQL, binding.BindSQL)
|
||||
@ -479,15 +479,15 @@ func TestGlobalBinding(t *testing.T) {
|
||||
require.Equal(t, uint64(1), tk.Session().AffectedRows())
|
||||
require.NoError(t, err)
|
||||
_, noDBDigest = bindinfo.NormalizeStmtForBinding(stmt, "", true)
|
||||
_, matched = dom.BindingHandle().MatchGlobalBinding(tk.Session(), noDBDigest, bindinfo.CollectTableNames(stmt))
|
||||
_, matched = dom.BindingHandle().MatchingBinding(tk.Session(), noDBDigest, bindinfo.CollectTableNames(stmt))
|
||||
require.False(t, matched) // dropped
|
||||
bindHandle = bindinfo.NewGlobalBindingHandle(&mockSessionPool{tk.Session()})
|
||||
err = bindHandle.LoadFromStorageToCache(true)
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, 0, len(bindHandle.GetAllGlobalBindings()))
|
||||
require.Equal(t, 0, len(bindHandle.GetAllBindings()))
|
||||
|
||||
_, noDBDigest = bindinfo.NormalizeStmtForBinding(stmt, "", true)
|
||||
_, matched = dom.BindingHandle().MatchGlobalBinding(tk.Session(), noDBDigest, bindinfo.CollectTableNames(stmt))
|
||||
_, matched = dom.BindingHandle().MatchingBinding(tk.Session(), noDBDigest, bindinfo.CollectTableNames(stmt))
|
||||
require.False(t, matched) // dropped
|
||||
|
||||
rs, err = tk.Exec("show global bindings")
|
||||
@ -692,7 +692,7 @@ func TestHintsSetID(t *testing.T) {
|
||||
stmt, err := parser.New().ParseOneStmt("select * from t where a > ?", "", "")
|
||||
require.NoError(t, err)
|
||||
_, noDBDigest := bindinfo.NormalizeStmtForBinding(stmt, "", true)
|
||||
binding, matched := dom.BindingHandle().MatchGlobalBinding(tk.Session(), noDBDigest, bindinfo.CollectTableNames(stmt))
|
||||
binding, matched := dom.BindingHandle().MatchingBinding(tk.Session(), noDBDigest, bindinfo.CollectTableNames(stmt))
|
||||
require.True(t, matched)
|
||||
require.Equal(t, "select * from `test` . `t` where `a` > ?", binding.OriginalSQL)
|
||||
require.Equal(t, "use_index(@`sel_1` `test`.`t` `idx_a`)", binding.ID)
|
||||
@ -700,7 +700,7 @@ func TestHintsSetID(t *testing.T) {
|
||||
utilCleanBindingEnv(tk)
|
||||
tk.MustExec("create global binding for select * from t where a > 10 using select /*+ use_index(t, idx_a) */ * from t where a > 10")
|
||||
_, noDBDigest = bindinfo.NormalizeStmtForBinding(stmt, "", true)
|
||||
binding, matched = dom.BindingHandle().MatchGlobalBinding(tk.Session(), noDBDigest, bindinfo.CollectTableNames(stmt))
|
||||
binding, matched = dom.BindingHandle().MatchingBinding(tk.Session(), noDBDigest, bindinfo.CollectTableNames(stmt))
|
||||
require.True(t, matched)
|
||||
require.Equal(t, "select * from `test` . `t` where `a` > ?", binding.OriginalSQL)
|
||||
require.Equal(t, "use_index(@`sel_1` `test`.`t` `idx_a`)", binding.ID)
|
||||
@ -708,7 +708,7 @@ func TestHintsSetID(t *testing.T) {
|
||||
utilCleanBindingEnv(tk)
|
||||
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")
|
||||
_, noDBDigest = bindinfo.NormalizeStmtForBinding(stmt, "", true)
|
||||
binding, matched = dom.BindingHandle().MatchGlobalBinding(tk.Session(), noDBDigest, bindinfo.CollectTableNames(stmt))
|
||||
binding, matched = dom.BindingHandle().MatchingBinding(tk.Session(), noDBDigest, bindinfo.CollectTableNames(stmt))
|
||||
require.True(t, matched)
|
||||
require.Equal(t, "select * from `test` . `t` where `a` > ?", binding.OriginalSQL)
|
||||
require.Equal(t, "use_index(@`sel_1` `test`.`t` `idx_a`)", binding.ID)
|
||||
@ -716,7 +716,7 @@ func TestHintsSetID(t *testing.T) {
|
||||
utilCleanBindingEnv(tk)
|
||||
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")
|
||||
_, noDBDigest = bindinfo.NormalizeStmtForBinding(stmt, "", true)
|
||||
binding, matched = dom.BindingHandle().MatchGlobalBinding(tk.Session(), noDBDigest, bindinfo.CollectTableNames(stmt))
|
||||
binding, matched = dom.BindingHandle().MatchingBinding(tk.Session(), noDBDigest, bindinfo.CollectTableNames(stmt))
|
||||
require.True(t, matched)
|
||||
require.Equal(t, "select * from `test` . `t` where `a` > ?", binding.OriginalSQL)
|
||||
require.Equal(t, "use_index(@`sel_1` `test`.`t` `idx_a`)", binding.ID)
|
||||
@ -724,7 +724,7 @@ func TestHintsSetID(t *testing.T) {
|
||||
utilCleanBindingEnv(tk)
|
||||
tk.MustExec("create global binding for select * from t where a > 10 using select /*+ use_index(T, IDX_A) */ * from t where a > 10")
|
||||
_, noDBDigest = bindinfo.NormalizeStmtForBinding(stmt, "", true)
|
||||
binding, matched = dom.BindingHandle().MatchGlobalBinding(tk.Session(), noDBDigest, bindinfo.CollectTableNames(stmt))
|
||||
binding, matched = dom.BindingHandle().MatchingBinding(tk.Session(), noDBDigest, bindinfo.CollectTableNames(stmt))
|
||||
require.True(t, matched)
|
||||
require.Equal(t, "select * from `test` . `t` where `a` > ?", binding.OriginalSQL)
|
||||
require.Equal(t, "use_index(@`sel_1` `test`.`t` `idx_a`)", binding.ID)
|
||||
@ -734,7 +734,7 @@ func TestHintsSetID(t *testing.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")
|
||||
_, noDBDigest = bindinfo.NormalizeStmtForBinding(stmt, "", true)
|
||||
binding, matched = dom.BindingHandle().MatchGlobalBinding(tk.Session(), noDBDigest, bindinfo.CollectTableNames(stmt))
|
||||
binding, matched = dom.BindingHandle().MatchingBinding(tk.Session(), noDBDigest, bindinfo.CollectTableNames(stmt))
|
||||
require.True(t, matched)
|
||||
require.Equal(t, "select * from `test` . `t` where `a` > ?", binding.OriginalSQL)
|
||||
}
|
||||
@ -757,7 +757,7 @@ func TestErrorBind(t *testing.T) {
|
||||
stmt, err := parser.New().ParseOneStmt("select * from test . t where i > ?", "", "")
|
||||
require.NoError(t, err)
|
||||
_, noDBDigest := bindinfo.NormalizeStmtForBinding(stmt, "", true)
|
||||
binding, matched := dom.BindingHandle().MatchGlobalBinding(tk.Session(), noDBDigest, bindinfo.CollectTableNames(stmt))
|
||||
binding, matched := dom.BindingHandle().MatchingBinding(tk.Session(), noDBDigest, bindinfo.CollectTableNames(stmt))
|
||||
require.True(t, matched)
|
||||
require.Equal(t, "select * from `test` . `t` where `i` > ?", binding.OriginalSQL)
|
||||
require.Equal(t, "SELECT * FROM `test`.`t` USE INDEX (`index_t`) WHERE `i` > 100", binding.BindSQL)
|
||||
@ -797,7 +797,7 @@ func TestBestPlanInBaselines(t *testing.T) {
|
||||
stmt, _, _ := utilNormalizeWithDefaultDB(t, "select a, b from t where a = 1 limit 0, 1")
|
||||
|
||||
_, noDBDigest := bindinfo.NormalizeStmtForBinding(stmt, "", true)
|
||||
binding, matched := dom.BindingHandle().MatchGlobalBinding(tk.Session(), noDBDigest, bindinfo.CollectTableNames(stmt))
|
||||
binding, matched := dom.BindingHandle().MatchingBinding(tk.Session(), noDBDigest, bindinfo.CollectTableNames(stmt))
|
||||
require.True(t, matched)
|
||||
require.Equal(t, "select `a` , `b` from `test` . `t` where `a` = ? limit ...", binding.OriginalSQL)
|
||||
require.Equal(t, "SELECT /*+ use_index(@`sel_1` `test`.`t` `ia`)*/ `a`,`b` FROM `test`.`t` WHERE `a` = 1 LIMIT 0,1", binding.BindSQL)
|
||||
@ -840,7 +840,7 @@ func TestBindingSymbolList(t *testing.T) {
|
||||
require.NoError(t, err)
|
||||
|
||||
_, noDBDigest := bindinfo.NormalizeStmtForBinding(stmt, "", true)
|
||||
binding, matched := dom.BindingHandle().MatchGlobalBinding(tk.Session(), noDBDigest, bindinfo.CollectTableNames(stmt))
|
||||
binding, matched := dom.BindingHandle().MatchingBinding(tk.Session(), noDBDigest, bindinfo.CollectTableNames(stmt))
|
||||
require.True(t, matched)
|
||||
require.Equal(t, "select `a` , `b` from `test` . `t` where `a` = ? limit ...", binding.OriginalSQL)
|
||||
require.Equal(t, "SELECT `a`,`b` FROM `test`.`t` USE INDEX (`ib`) WHERE `a` = 1 LIMIT 0,1", binding.BindSQL)
|
||||
@ -908,7 +908,7 @@ func TestBindingInListWithSingleLiteral(t *testing.T) {
|
||||
require.NoError(t, err)
|
||||
|
||||
_, noDBDigest := bindinfo.NormalizeStmtForBinding(stmt, "", true)
|
||||
binding, matched := dom.BindingHandle().MatchGlobalBinding(tk.Session(), noDBDigest, bindinfo.CollectTableNames(stmt))
|
||||
binding, matched := dom.BindingHandle().MatchingBinding(tk.Session(), noDBDigest, bindinfo.CollectTableNames(stmt))
|
||||
require.True(t, matched)
|
||||
require.Equal(t, "select `a` , `b` from `test` . `t` where `a` in ( ... )", binding.OriginalSQL)
|
||||
require.Equal(t, "SELECT `a`,`b` FROM `test`.`t` USE INDEX (`ib`) WHERE `a` IN (1,2,3)", binding.BindSQL)
|
||||
|
||||
@ -324,7 +324,7 @@ func (e *ShowExec) fetchShowBind() error {
|
||||
handle := e.Ctx().Value(bindinfo.SessionBindInfoKeyType).(bindinfo.SessionBindingHandle)
|
||||
bindings = handle.GetAllSessionBindings()
|
||||
} else {
|
||||
bindings = domain.GetDomain(e.Ctx()).BindingHandle().GetAllGlobalBindings()
|
||||
bindings = domain.GetDomain(e.Ctx()).BindingHandle().GetAllBindings()
|
||||
}
|
||||
// Remove the invalid bindings.
|
||||
parser := parser.New()
|
||||
@ -383,7 +383,7 @@ func (e *ShowExec) fetchShowBindingCacheStatus(ctx context.Context) error {
|
||||
|
||||
handle := domain.GetDomain(e.Ctx()).BindingHandle()
|
||||
|
||||
bindings := handle.GetAllGlobalBindings()
|
||||
bindings := handle.GetAllBindings()
|
||||
numBindings := 0
|
||||
for _, binding := range bindings {
|
||||
if binding.IsBindingEnabled() {
|
||||
|
||||
Reference in New Issue
Block a user