planner: refactor some code of binding cache (#58504)

ref pingcap/tidb#51347
This commit is contained in:
Yuanjia Zhang
2024-12-24 17:57:15 +08:00
committed by GitHub
parent 8ecdb54c51
commit 457dcc6bbd
13 changed files with 68 additions and 176 deletions

View File

@ -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)

View File

@ -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()

View File

@ -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())

View File

@ -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

View File

@ -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 {

View File

@ -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

View File

@ -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

View File

@ -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 {

View File

@ -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},

View File

@ -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()

View File

@ -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 {

View File

@ -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",

View File

@ -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)