planner: refactor some code of binding cache (#58504)
ref pingcap/tidb#51347
This commit is contained in:
@ -101,15 +101,6 @@ func (b *Binding) SinceUpdateTime() (time.Duration, error) {
|
||||
return time.Since(updateTime), nil
|
||||
}
|
||||
|
||||
// Bindings represents a sql bind record retrieved from the storage.
|
||||
type Bindings []Binding
|
||||
|
||||
// Copy get the copy of bindings
|
||||
func (br Bindings) Copy() Bindings {
|
||||
nbr := append(make(Bindings, 0, len(br)), br...)
|
||||
return nbr
|
||||
}
|
||||
|
||||
// prepareHints builds ID and Hint for Bindings. If sctx is not nil, we check if
|
||||
// the BindSQL is still valid.
|
||||
func prepareHints(sctx sessionctx.Context, binding *Binding) (rerr error) {
|
||||
@ -165,19 +156,19 @@ func prepareHints(sctx sessionctx.Context, binding *Binding) (rerr error) {
|
||||
}
|
||||
|
||||
// `merge` merges two Bindings. It will replace old bindings with new bindings if there are new updates.
|
||||
func merge(lBindings, rBindings Bindings) Bindings {
|
||||
func merge(lBindings, rBindings []*Binding) []*Binding {
|
||||
if lBindings == nil {
|
||||
return rBindings
|
||||
}
|
||||
if rBindings == nil {
|
||||
return lBindings
|
||||
}
|
||||
result := lBindings.Copy()
|
||||
result := lBindings
|
||||
for i := range rBindings {
|
||||
rbind := rBindings[i]
|
||||
found := false
|
||||
for j, lbind := range lBindings {
|
||||
if lbind.isSame(&rbind) {
|
||||
if lbind.isSame(rbind) {
|
||||
found = true
|
||||
if rbind.UpdateTime.Compare(lbind.UpdateTime) >= 0 {
|
||||
result[j] = rbind
|
||||
@ -192,8 +183,8 @@ func merge(lBindings, rBindings Bindings) Bindings {
|
||||
return result
|
||||
}
|
||||
|
||||
func removeDeletedBindings(br Bindings) Bindings {
|
||||
result := make(Bindings, 0, len(br))
|
||||
func removeDeletedBindings(br []*Binding) []*Binding {
|
||||
result := make([]*Binding, 0, len(br))
|
||||
for _, binding := range br {
|
||||
if binding.Status != deleted {
|
||||
result = append(result, binding)
|
||||
@ -202,15 +193,6 @@ func removeDeletedBindings(br Bindings) Bindings {
|
||||
return result
|
||||
}
|
||||
|
||||
// size calculates the memory size of a Bindings.
|
||||
func (br Bindings) size() float64 {
|
||||
mem := float64(0)
|
||||
for _, binding := range br {
|
||||
mem += binding.size()
|
||||
}
|
||||
return mem
|
||||
}
|
||||
|
||||
// size calculates the memory size of a bind info.
|
||||
func (b *Binding) size() float64 {
|
||||
res := len(b.OriginalSQL) + len(b.Db) + len(b.BindSQL) + len(b.Status) + 2*int(unsafe.Sizeof(b.CreateTime)) + len(b.Charset) + len(b.Collation) + len(b.ID)
|
||||
|
||||
@ -141,13 +141,13 @@ func (b *digestBiMapImpl) SQLDigest2NoDBDigest(sqlDigest string) string {
|
||||
// BindingCache is the interface for the cache of the SQL plan bindings.
|
||||
type BindingCache interface {
|
||||
// MatchingBinding supports cross-db matching on bindings.
|
||||
MatchingBinding(sctx sessionctx.Context, noDBDigest string, tableNames []*ast.TableName) (bindings Binding, isMatched bool)
|
||||
MatchingBinding(sctx sessionctx.Context, noDBDigest string, tableNames []*ast.TableName) (binding *Binding, isMatched bool)
|
||||
// GetBinding gets the binding for the specified sqlDigest.
|
||||
GetBinding(sqlDigest string) Bindings
|
||||
GetBinding(sqlDigest string) []*Binding
|
||||
// GetAllBindings gets all the bindings in the cache.
|
||||
GetAllBindings() Bindings
|
||||
GetAllBindings() []*Binding
|
||||
// SetBinding sets the binding for the specified sqlDigest.
|
||||
SetBinding(sqlDigest string, meta Bindings) (err error)
|
||||
SetBinding(sqlDigest string, bindings []*Binding) (err error)
|
||||
// RemoveBinding removes the binding for the specified sqlDigest.
|
||||
RemoveBinding(sqlDigest string)
|
||||
// SetMemCapacity sets the memory capacity for the cache.
|
||||
@ -170,16 +170,20 @@ type bindingCache struct {
|
||||
cache *ristretto.Cache // the underlying cache to store the bindings.
|
||||
|
||||
// loadBindingFromStorageFunc is used to load binding from storage if cache miss.
|
||||
loadBindingFromStorageFunc func(sctx sessionctx.Context, sqlDigest string) (Bindings, error)
|
||||
loadBindingFromStorageFunc func(sctx sessionctx.Context, sqlDigest string) ([]*Binding, error)
|
||||
}
|
||||
|
||||
func newBindCache(bindingLoad func(sctx sessionctx.Context, sqlDigest string) (Bindings, error)) BindingCache {
|
||||
func newBindCache(bindingLoad func(sctx sessionctx.Context, sqlDigest string) ([]*Binding, error)) BindingCache {
|
||||
cache, _ := ristretto.NewCache(&ristretto.Config{
|
||||
NumCounters: 1e6,
|
||||
MaxCost: variable.MemQuotaBindingCache.Load(),
|
||||
BufferItems: 64,
|
||||
Cost: func(value any) int64 {
|
||||
return int64(value.(Bindings).size())
|
||||
var cost int64
|
||||
for _, binding := range value.([]*Binding) {
|
||||
cost += int64(binding.size())
|
||||
}
|
||||
return cost
|
||||
},
|
||||
Metrics: true,
|
||||
IgnoreInternalCost: true,
|
||||
@ -196,7 +200,7 @@ func (c *bindingCache) shouldMetric() bool {
|
||||
return c.loadBindingFromStorageFunc != nil // only metric for GlobalBindingCache, whose loadBindingFromStorageFunc is not nil.
|
||||
}
|
||||
|
||||
func (c *bindingCache) MatchingBinding(sctx sessionctx.Context, noDBDigest string, tableNames []*ast.TableName) (matchedBinding Binding, isMatched bool) {
|
||||
func (c *bindingCache) MatchingBinding(sctx sessionctx.Context, noDBDigest string, tableNames []*ast.TableName) (matchedBinding *Binding, isMatched bool) {
|
||||
matchedBinding, isMatched, missingSQLDigest := c.getFromMemory(sctx, noDBDigest, tableNames)
|
||||
if len(missingSQLDigest) == 0 {
|
||||
if c.shouldMetric() && isMatched {
|
||||
@ -215,7 +219,7 @@ func (c *bindingCache) MatchingBinding(sctx sessionctx.Context, noDBDigest strin
|
||||
return
|
||||
}
|
||||
|
||||
func (c *bindingCache) getFromMemory(sctx sessionctx.Context, noDBDigest string, tableNames []*ast.TableName) (matchedBinding Binding, isMatched bool, missingSQLDigest []string) {
|
||||
func (c *bindingCache) getFromMemory(sctx sessionctx.Context, noDBDigest string, tableNames []*ast.TableName) (matchedBinding *Binding, isMatched bool, missingSQLDigest []string) {
|
||||
if c.Size() == 0 {
|
||||
return
|
||||
}
|
||||
@ -290,20 +294,20 @@ func (c *bindingCache) loadFromStore(sctx sessionctx.Context, missingSQLDigest [
|
||||
// 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 *bindingCache) GetBinding(sqlDigest string) Bindings {
|
||||
func (c *bindingCache) GetBinding(sqlDigest string) []*Binding {
|
||||
v, ok := c.cache.Get(sqlDigest)
|
||||
if !ok {
|
||||
return nil
|
||||
}
|
||||
return v.(Bindings)
|
||||
return v.([]*Binding)
|
||||
}
|
||||
|
||||
// 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 *bindingCache) GetAllBindings() Bindings {
|
||||
func (c *bindingCache) GetAllBindings() []*Binding {
|
||||
sqlDigests := c.digestBiMap.All()
|
||||
bindings := make(Bindings, 0, len(sqlDigests))
|
||||
bindings := make([]*Binding, 0, len(sqlDigests))
|
||||
for _, sqlDigest := range sqlDigests {
|
||||
bindings = append(bindings, c.GetBinding(sqlDigest)...)
|
||||
}
|
||||
@ -312,7 +316,7 @@ func (c *bindingCache) GetAllBindings() Bindings {
|
||||
|
||||
// SetBinding sets the Bindings to the cache.
|
||||
// The function is thread-safe.
|
||||
func (c *bindingCache) SetBinding(sqlDigest string, bindings Bindings) (err error) {
|
||||
func (c *bindingCache) SetBinding(sqlDigest string, bindings []*Binding) (err error) {
|
||||
// prepare noDBDigests for all bindings
|
||||
noDBDigests := make([]string, 0, len(bindings))
|
||||
p := parser.New()
|
||||
|
||||
@ -24,7 +24,7 @@ import (
|
||||
"github.com/stretchr/testify/require"
|
||||
)
|
||||
|
||||
func bindingNoDBDigest(t *testing.T, b Binding) string {
|
||||
func bindingNoDBDigest(t *testing.T, b *Binding) string {
|
||||
p := parser.New()
|
||||
stmt, err := p.ParseOneStmt(b.BindSQL, b.Charset, b.Collation)
|
||||
require.NoError(t, err)
|
||||
@ -34,16 +34,16 @@ func bindingNoDBDigest(t *testing.T, b Binding) string {
|
||||
|
||||
func TestCrossDBBindingCache(t *testing.T) {
|
||||
fbc := newBindCache(nil).(*bindingCache)
|
||||
b1 := Binding{BindSQL: "SELECT * FROM db1.t1", SQLDigest: "b1"}
|
||||
b1 := &Binding{BindSQL: "SELECT * FROM db1.t1", SQLDigest: "b1"}
|
||||
fDigest1 := bindingNoDBDigest(t, b1)
|
||||
b2 := Binding{BindSQL: "SELECT * FROM db2.t1", SQLDigest: "b2"}
|
||||
b3 := Binding{BindSQL: "SELECT * FROM db2.t3", SQLDigest: "b3"}
|
||||
b2 := &Binding{BindSQL: "SELECT * FROM db2.t1", SQLDigest: "b2"}
|
||||
b3 := &Binding{BindSQL: "SELECT * FROM db2.t3", SQLDigest: "b3"}
|
||||
fDigest3 := bindingNoDBDigest(t, b3)
|
||||
|
||||
// add 3 bindings and b1 and b2 have the same noDBDigest
|
||||
require.NoError(t, fbc.SetBinding(b1.SQLDigest, []Binding{b1}))
|
||||
require.NoError(t, fbc.SetBinding(b2.SQLDigest, []Binding{b2}))
|
||||
require.NoError(t, fbc.SetBinding(b3.SQLDigest, []Binding{b3}))
|
||||
require.NoError(t, fbc.SetBinding(b1.SQLDigest, []*Binding{b1}))
|
||||
require.NoError(t, fbc.SetBinding(b2.SQLDigest, []*Binding{b2}))
|
||||
require.NoError(t, fbc.SetBinding(b3.SQLDigest, []*Binding{b3}))
|
||||
require.Equal(t, len(fbc.digestBiMap.(*digestBiMapImpl).noDBDigest2SQLDigest), 2) // b1 and b2 have the same noDBDigest
|
||||
require.Equal(t, len(fbc.digestBiMap.NoDBDigest2SQLDigest(fDigest1)), 2)
|
||||
require.Equal(t, len(fbc.digestBiMap.NoDBDigest2SQLDigest(fDigest3)), 1)
|
||||
@ -70,8 +70,8 @@ func TestCrossDBBindingCache(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestBindCache(t *testing.T) {
|
||||
bindings := Bindings{{BindSQL: "SELECT * FROM t1"}}
|
||||
kvSize := int(bindings.size())
|
||||
bindings := []*Binding{{BindSQL: "SELECT * FROM t1"}}
|
||||
kvSize := int(bindings[0].size())
|
||||
defer func(v int64) {
|
||||
variable.MemQuotaBindingCache.Store(v)
|
||||
}(variable.MemQuotaBindingCache.Load())
|
||||
|
||||
@ -49,11 +49,11 @@ func MatchSQLBindingForPlanCache(sctx sessionctx.Context, stmtNode ast.StmtNode,
|
||||
}
|
||||
|
||||
// MatchSQLBinding returns the matched binding for this statement.
|
||||
func MatchSQLBinding(sctx sessionctx.Context, stmtNode ast.StmtNode) (binding Binding, matched bool, scope string) {
|
||||
func MatchSQLBinding(sctx sessionctx.Context, stmtNode ast.StmtNode) (binding *Binding, matched bool, scope string) {
|
||||
return matchSQLBinding(sctx, stmtNode, nil)
|
||||
}
|
||||
|
||||
func matchSQLBinding(sctx sessionctx.Context, stmtNode ast.StmtNode, info *BindingMatchInfo) (binding Binding, matched bool, scope string) {
|
||||
func matchSQLBinding(sctx sessionctx.Context, stmtNode ast.StmtNode, info *BindingMatchInfo) (binding *Binding, matched bool, scope string) {
|
||||
useBinding := sctx.GetSessionVars().UsePlanBaselines
|
||||
if !useBinding || stmtNode == nil {
|
||||
return
|
||||
|
||||
@ -50,10 +50,10 @@ 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)
|
||||
MatchGlobalBinding(sctx sessionctx.Context, noDBDigest string, tableNames []*ast.TableName) (matchedBinding *Binding, isMatched bool)
|
||||
|
||||
// GetAllGlobalBindings returns all bind records in cache.
|
||||
GetAllGlobalBindings() (bindings Bindings)
|
||||
GetAllGlobalBindings() (bindings []*Binding)
|
||||
|
||||
// CreateGlobalBinding creates a Bindings to the storage and the cache.
|
||||
// It replaces all the exists bindings for the same normalized SQL.
|
||||
@ -201,7 +201,7 @@ func (h *globalBindingHandle) LoadFromStorageToCache(fullLoad bool) (err error)
|
||||
}
|
||||
|
||||
oldBinding := h.bindingCache.GetBinding(sqlDigest)
|
||||
newBinding := removeDeletedBindings(merge(oldBinding, []Binding{binding}))
|
||||
newBinding := removeDeletedBindings(merge(oldBinding, []*Binding{binding}))
|
||||
if len(newBinding) > 0 {
|
||||
err = h.bindingCache.SetBinding(sqlDigest, newBinding)
|
||||
if err != nil {
|
||||
@ -401,12 +401,12 @@ func lockBindInfoTable(sctx sessionctx.Context) error {
|
||||
}
|
||||
|
||||
// MatchGlobalBinding returns the matched binding for this statement.
|
||||
func (h *globalBindingHandle) MatchGlobalBinding(sctx sessionctx.Context, noDBDigest string, tableNames []*ast.TableName) (matchedBinding Binding, isMatched bool) {
|
||||
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 Bindings) {
|
||||
func (h *globalBindingHandle) GetAllGlobalBindings() (bindings []*Binding) {
|
||||
return h.bindingCache.GetAllBindings()
|
||||
}
|
||||
|
||||
@ -427,13 +427,13 @@ func (h *globalBindingHandle) GetMemCapacity() (memCapacity int64) {
|
||||
}
|
||||
|
||||
// newBinding builds Bindings from a tuple in storage.
|
||||
func newBinding(sctx sessionctx.Context, row chunk.Row) (string, Binding, error) {
|
||||
func newBinding(sctx sessionctx.Context, row chunk.Row) (string, *Binding, error) {
|
||||
status := row.GetString(3)
|
||||
// For compatibility, the 'Using' status binding will be converted to the 'Enabled' status binding.
|
||||
if status == Using {
|
||||
status = Enabled
|
||||
}
|
||||
binding := Binding{
|
||||
binding := &Binding{
|
||||
OriginalSQL: row.GetString(0),
|
||||
Db: strings.ToLower(row.GetString(2)),
|
||||
BindSQL: row.GetString(1),
|
||||
@ -447,7 +447,7 @@ func newBinding(sctx sessionctx.Context, row chunk.Row) (string, Binding, error)
|
||||
PlanDigest: row.GetString(10),
|
||||
}
|
||||
sqlDigest := parser.DigestNormalized(binding.OriginalSQL)
|
||||
err := prepareHints(sctx, &binding)
|
||||
err := prepareHints(sctx, binding)
|
||||
sctx.GetSessionVars().CurrentDB = binding.Db
|
||||
return sqlDigest.String(), binding, err
|
||||
}
|
||||
@ -615,7 +615,7 @@ func (h *globalBindingHandle) CloseCache() {
|
||||
}
|
||||
|
||||
// LoadBindingsFromStorageToCache loads global bindings from storage to the memory cache.
|
||||
func (h *globalBindingHandle) LoadBindingsFromStorage(sctx sessionctx.Context, sqlDigest string) (Bindings, error) {
|
||||
func (h *globalBindingHandle) LoadBindingsFromStorage(sctx sessionctx.Context, sqlDigest string) ([]*Binding, error) {
|
||||
if sqlDigest == "" {
|
||||
return nil, nil
|
||||
}
|
||||
@ -632,7 +632,7 @@ func (h *globalBindingHandle) LoadBindingsFromStorage(sctx sessionctx.Context, s
|
||||
if bindings == nil {
|
||||
return nil, nil
|
||||
}
|
||||
return bindings.(Bindings), nil
|
||||
return bindings.([]*Binding), nil
|
||||
case <-time.After(timeout):
|
||||
return nil, errors.New("load bindings from storage timeout")
|
||||
}
|
||||
@ -642,14 +642,14 @@ func (h *globalBindingHandle) loadBindingsFromStorageInternal(sqlDigest string)
|
||||
failpoint.Inject("load_bindings_from_storage_internal_timeout", func() {
|
||||
time.Sleep(time.Second)
|
||||
})
|
||||
var bindings Bindings
|
||||
var bindings []*Binding
|
||||
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 where sql_digest = '%s'", sqlDigest)
|
||||
err := h.callWithSCtx(false, func(sctx sessionctx.Context) error {
|
||||
rows, _, err := execRows(sctx, selectStmt)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
bindings = make([]Binding, 0, len(rows))
|
||||
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 {
|
||||
|
||||
@ -40,10 +40,10 @@ type SessionBindingHandle interface {
|
||||
DropSessionBinding(sqlDigests []string) error
|
||||
|
||||
// MatchSessionBinding returns the matched binding for this statement.
|
||||
MatchSessionBinding(sctx sessionctx.Context, noDBDigest string, tableNames []*ast.TableName) (matchedBinding Binding, isMatched bool)
|
||||
MatchSessionBinding(sctx sessionctx.Context, noDBDigest string, tableNames []*ast.TableName) (matchedBinding *Binding, isMatched bool)
|
||||
|
||||
// GetAllSessionBindings return all bindings.
|
||||
GetAllSessionBindings() (bindings Bindings)
|
||||
GetAllSessionBindings() (bindings []*Binding)
|
||||
|
||||
// Close closes the SessionBindingHandle.
|
||||
Close()
|
||||
@ -54,12 +54,12 @@ type SessionBindingHandle interface {
|
||||
// sessionBindingHandle is used to handle all session sql bind operations.
|
||||
type sessionBindingHandle struct {
|
||||
mu sync.RWMutex
|
||||
bindings map[string]Bindings // sqlDigest --> Bindings
|
||||
bindings map[string][]*Binding // sqlDigest --> Bindings
|
||||
}
|
||||
|
||||
// NewSessionBindingHandle creates a new SessionBindingHandle.
|
||||
func NewSessionBindingHandle() SessionBindingHandle {
|
||||
return &sessionBindingHandle{bindings: make(map[string]Bindings)}
|
||||
return &sessionBindingHandle{bindings: make(map[string][]*Binding)}
|
||||
}
|
||||
|
||||
// CreateSessionBinding creates a Bindings to the cache.
|
||||
@ -83,7 +83,7 @@ func (h *sessionBindingHandle) CreateSessionBinding(sctx sessionctx.Context, bin
|
||||
binding.UpdateTime = now
|
||||
|
||||
// update the BindMeta to the cache.
|
||||
h.bindings[parser.DigestNormalized(binding.OriginalSQL).String()] = []Binding{*binding}
|
||||
h.bindings[parser.DigestNormalized(binding.OriginalSQL).String()] = []*Binding{binding}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
@ -99,7 +99,7 @@ func (h *sessionBindingHandle) DropSessionBinding(sqlDigests []string) error {
|
||||
}
|
||||
|
||||
// MatchSessionBinding returns the matched binding for this statement.
|
||||
func (h *sessionBindingHandle) MatchSessionBinding(sctx sessionctx.Context, noDBDigest string, tableNames []*ast.TableName) (matchedBinding Binding, isMatched bool) {
|
||||
func (h *sessionBindingHandle) MatchSessionBinding(sctx sessionctx.Context, noDBDigest string, tableNames []*ast.TableName) (matchedBinding *Binding, isMatched bool) {
|
||||
h.mu.RLock()
|
||||
defer h.mu.RUnlock()
|
||||
p := parser.New()
|
||||
@ -133,7 +133,7 @@ func (h *sessionBindingHandle) MatchSessionBinding(sctx sessionctx.Context, noDB
|
||||
}
|
||||
|
||||
// GetAllSessionBindings return all session bind info.
|
||||
func (h *sessionBindingHandle) GetAllSessionBindings() (bindings Bindings) {
|
||||
func (h *sessionBindingHandle) GetAllSessionBindings() (bindings []*Binding) {
|
||||
h.mu.RLock()
|
||||
defer h.mu.RUnlock()
|
||||
for _, bind := range h.bindings {
|
||||
@ -148,7 +148,7 @@ func (h *sessionBindingHandle) EncodeSessionStates(_ context.Context, _ sessionc
|
||||
if len(bindings) == 0 {
|
||||
return nil
|
||||
}
|
||||
bytes, err := json.Marshal([]Binding(bindings))
|
||||
bytes, err := json.Marshal(bindings)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -172,7 +172,7 @@ func (h *sessionBindingHandle) DecodeSessionStates(_ context.Context, sctx sessi
|
||||
return nil
|
||||
}
|
||||
|
||||
var records []Binding
|
||||
var records []*Binding
|
||||
// Key "Bindings" only exists in old versions.
|
||||
if _, ok := m[0]["Bindings"]; ok {
|
||||
err = h.decodeOldStyleSessionStates(bindingBytes, &records)
|
||||
@ -185,27 +185,27 @@ func (h *sessionBindingHandle) DecodeSessionStates(_ context.Context, sctx sessi
|
||||
|
||||
for _, record := range records {
|
||||
// Restore hints and ID because hints are hard to encode.
|
||||
if err = prepareHints(sctx, &record); err != nil {
|
||||
if err = prepareHints(sctx, record); err != nil {
|
||||
return err
|
||||
}
|
||||
h.bindings[parser.DigestNormalized(record.OriginalSQL).String()] = []Binding{record}
|
||||
h.bindings[parser.DigestNormalized(record.OriginalSQL).String()] = []*Binding{record}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// Before v8.0.0, the data structure is different. We need to adapt to the old structure so that the sessions
|
||||
// can be migrated from an old version to a new version.
|
||||
func (*sessionBindingHandle) decodeOldStyleSessionStates(bindingBytes []byte, bindings *[]Binding) error {
|
||||
func (*sessionBindingHandle) decodeOldStyleSessionStates(bindingBytes []byte, bindings *[]*Binding) error {
|
||||
type bindRecord struct {
|
||||
OriginalSQL string
|
||||
Db string
|
||||
Bindings []Binding
|
||||
Bindings []*Binding
|
||||
}
|
||||
var records []bindRecord
|
||||
if err := json.Unmarshal(bindingBytes, &records); err != nil {
|
||||
return err
|
||||
}
|
||||
*bindings = make([]Binding, 0, len(records))
|
||||
*bindings = make([]*Binding, 0, len(records))
|
||||
for _, record := range records {
|
||||
for _, binding := range record.Bindings {
|
||||
binding.OriginalSQL = record.OriginalSQL
|
||||
|
||||
@ -571,7 +571,7 @@ type PlanReplayerDumpTask struct {
|
||||
|
||||
// variables used to dump the plan
|
||||
StartTS uint64
|
||||
SessionBindings []bindinfo.Bindings
|
||||
SessionBindings [][]*bindinfo.Binding
|
||||
EncodedPlan string
|
||||
SessionVars *variable.SessionVars
|
||||
ExecStmts []ast.StmtNode
|
||||
|
||||
@ -623,7 +623,7 @@ func dumpVariables(sctx sessionctx.Context, sessionVars *variable.SessionVars, z
|
||||
return nil
|
||||
}
|
||||
|
||||
func dumpSessionBindRecords(records []bindinfo.Bindings, zw *zip.Writer) error {
|
||||
func dumpSessionBindRecords(records [][]*bindinfo.Binding, zw *zip.Writer) error {
|
||||
sRows := make([][]string, 0)
|
||||
for _, bindData := range records {
|
||||
for _, hint := range bindData {
|
||||
|
||||
@ -2285,7 +2285,7 @@ func sendPlanReplayerDumpTask(key replayer.PlanReplayerTaskKey, sctx sessionctx.
|
||||
PlanReplayerTaskKey: key,
|
||||
StartTS: startTS,
|
||||
TblStats: stmtCtx.TableStats,
|
||||
SessionBindings: []bindinfo.Bindings{bindings},
|
||||
SessionBindings: [][]*bindinfo.Binding{bindings},
|
||||
SessionVars: sctx.GetSessionVars(),
|
||||
ExecStmts: []ast.StmtNode{stmtNode},
|
||||
DebugTrace: []any{stmtCtx.OptimizerDebugTrace},
|
||||
|
||||
@ -317,7 +317,7 @@ func (*visibleChecker) Leave(in ast.Node) (out ast.Node, ok bool) {
|
||||
}
|
||||
|
||||
func (e *ShowExec) fetchShowBind() error {
|
||||
var bindings []bindinfo.Binding
|
||||
var bindings []*bindinfo.Binding
|
||||
if !e.GlobalScope {
|
||||
handle := e.Ctx().Value(bindinfo.SessionBindInfoKeyType).(bindinfo.SessionBindingHandle)
|
||||
bindings = handle.GetAllSessionBindings()
|
||||
|
||||
@ -241,32 +241,13 @@ func Optimize(ctx context.Context, sctx sessionctx.Context, node *resolve.NodeW,
|
||||
|
||||
enableUseBinding := sessVars.UsePlanBaselines
|
||||
stmtNode, isStmtNode := node.Node.(ast.StmtNode)
|
||||
binding, match, scope := bindinfo.MatchSQLBinding(sctx, stmtNode)
|
||||
var bindings bindinfo.Bindings
|
||||
binding, match, _ := bindinfo.MatchSQLBinding(sctx, stmtNode)
|
||||
var bindings []*bindinfo.Binding
|
||||
if match {
|
||||
bindings = []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) && bindings != nil {
|
||||
bindings = bindings.Copy()
|
||||
for i := range bindings {
|
||||
bindings[i].CreateTime = types.ZeroTime
|
||||
bindings[i].UpdateTime = types.ZeroTime
|
||||
}
|
||||
}
|
||||
})
|
||||
debugtrace.RecordAnyValuesWithNames(pctx,
|
||||
"Used binding", useBinding,
|
||||
"Enable binding", enableUseBinding,
|
||||
"IsStmtNode", isStmtNode,
|
||||
"Matched", match,
|
||||
"Scope", scope,
|
||||
"Matched bindings", bindings,
|
||||
)
|
||||
}
|
||||
if isStmtNode {
|
||||
// add the extra Limit after matching the bind record
|
||||
stmtNode = core.TryAddExtraLimit(sctx, stmtNode)
|
||||
@ -289,7 +270,7 @@ func Optimize(ctx context.Context, sctx sessionctx.Context, node *resolve.NodeW,
|
||||
var (
|
||||
names types.NameSlice
|
||||
bestPlan, bestPlanFromBind base.Plan
|
||||
chosenBinding bindinfo.Binding
|
||||
chosenBinding *bindinfo.Binding
|
||||
err error
|
||||
)
|
||||
if useBinding {
|
||||
|
||||
@ -180,7 +180,6 @@ go_test(
|
||||
"//pkg/session",
|
||||
"//pkg/sessionctx/variable",
|
||||
"//pkg/sessiontxn",
|
||||
"//pkg/statistics/handle/ddl/testutil",
|
||||
"//pkg/store/mockstore",
|
||||
"//pkg/store/mockstore/unistore",
|
||||
"//pkg/testkit",
|
||||
|
||||
@ -18,18 +18,15 @@ import (
|
||||
"bufio"
|
||||
"bytes"
|
||||
"context"
|
||||
"encoding/json"
|
||||
"os"
|
||||
"path/filepath"
|
||||
"testing"
|
||||
|
||||
"github.com/pingcap/failpoint"
|
||||
"github.com/pingcap/tidb/pkg/parser/auth"
|
||||
"github.com/pingcap/tidb/pkg/parser/mysql"
|
||||
"github.com/pingcap/tidb/pkg/server/internal"
|
||||
"github.com/pingcap/tidb/pkg/server/internal/testutil"
|
||||
"github.com/pingcap/tidb/pkg/server/internal/util"
|
||||
statstestutil "github.com/pingcap/tidb/pkg/statistics/handle/ddl/testutil"
|
||||
"github.com/pingcap/tidb/pkg/testkit"
|
||||
"github.com/pingcap/tidb/pkg/testkit/testdata"
|
||||
"github.com/pingcap/tidb/pkg/util/arena"
|
||||
@ -38,77 +35,6 @@ import (
|
||||
"github.com/stretchr/testify/require"
|
||||
)
|
||||
|
||||
// cmd: go test -run=^TestOptimizerDebugTrace$ --tags=intest github.com/pingcap/tidb/pkg/server
|
||||
// If you want to update the test result, please run the following command:
|
||||
// cmd: go test -run=^TestOptimizerDebugTrace$ --tags=intest github.com/pingcap/tidb/pkg/server --record
|
||||
func TestOptimizerDebugTrace(t *testing.T) {
|
||||
require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/planner/SetBindingTimeToZero", `return(true)`))
|
||||
require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/planner/core/DebugTraceStableStatsTbl", `return(true)`))
|
||||
defer func() {
|
||||
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/planner/SetBindingTimeToZero"))
|
||||
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/planner/core/DebugTraceStableStatsTbl"))
|
||||
}()
|
||||
store, dom := testkit.CreateMockStoreAndDomain(t)
|
||||
tk := testkit.NewTestKit(t, store)
|
||||
tidbdrv := NewTiDBDriver(store)
|
||||
cfg := util.NewTestConfig()
|
||||
cfg.Port, cfg.Status.StatusPort = 0, 0
|
||||
cfg.Status.ReportStatus = false
|
||||
server, err := NewServer(cfg, tidbdrv)
|
||||
require.NoError(t, err)
|
||||
defer server.Close()
|
||||
cc := &clientConn{
|
||||
server: server,
|
||||
alloc: arena.NewAllocator(1024),
|
||||
chunkAlloc: chunk.NewAllocator(),
|
||||
pkt: internal.NewPacketIOForTest(bufio.NewWriter(bytes.NewBuffer(nil))),
|
||||
}
|
||||
ctx := context.Background()
|
||||
cc.SetCtx(&TiDBContext{Session: tk.Session(), stmts: make(map[int]*TiDBStatement)})
|
||||
|
||||
tk.MustExec("use test")
|
||||
tk.MustExec("create table t (col1 int, index i(col1))")
|
||||
h := dom.StatsHandle()
|
||||
err = statstestutil.HandleNextDDLEventWithTxn(h)
|
||||
require.NoError(t, err)
|
||||
|
||||
tk.MustExec("plan replayer capture '0595c79f25d183319d0830ff8ca538c9054cbf407e5e27488b5dc40e4738a7c8' '*'")
|
||||
tk.MustExec("plan replayer capture 'c0fcc0abbaaffcaafe21115a3c67ae5d96a188cc197559953d2865ea6852d3cc' '*'")
|
||||
tk.MustExec("plan replayer capture '58fcbdd56a722c02225488c89a782cd2d626f8219c8ef8f57cd3bcdb6eb7c1b2' '*'")
|
||||
require.NoError(t, cc.HandleStmtPrepare(ctx, "select sum(col1) from t where col1 < ? and col1 > 100"))
|
||||
tk.MustExec("prepare stmt from 'select * from t where col1 in (?, 2, 3)'")
|
||||
tk.MustExec("set @a = 1")
|
||||
var (
|
||||
in []string
|
||||
out []any
|
||||
)
|
||||
optSuiteData := testDataMap["optimizer_suite"]
|
||||
optSuiteData.LoadTestCases(t, &in, &out)
|
||||
for i, cmdString := range in {
|
||||
require.NoError(t, cc.dispatch(ctx, []byte(cmdString)))
|
||||
traceInfo := cc.ctx.Session.GetSessionVars().StmtCtx.OptimizerDebugTrace
|
||||
var buf bytes.Buffer
|
||||
encoder := json.NewEncoder(&buf)
|
||||
encoder.SetEscapeHTML(false)
|
||||
require.NoError(t, encoder.Encode(traceInfo))
|
||||
var res any
|
||||
require.NoError(t, json.Unmarshal(buf.Bytes(), &res))
|
||||
testdata.OnRecord(func() {
|
||||
out[i] = res
|
||||
})
|
||||
require.Equal(t, out[i], res, cmdString)
|
||||
}
|
||||
|
||||
prHandle := dom.GetPlanReplayerHandle()
|
||||
worker := prHandle.GetWorker()
|
||||
for i := 0; i < 3; i++ {
|
||||
task := prHandle.DrainTask()
|
||||
success := worker.HandleTask(task)
|
||||
require.True(t, success)
|
||||
require.NoError(t, os.Remove(filepath.Join(replayer.GetPlanReplayerDirName(), task.FileName)))
|
||||
}
|
||||
}
|
||||
|
||||
func TestIssue46197(t *testing.T) {
|
||||
store := testkit.CreateMockStore(t)
|
||||
tk := testkit.NewTestKit(t, store)
|
||||
|
||||
Reference in New Issue
Block a user