planner: refactor a few code of plan cache (#54464)
ref pingcap/tidb#54057
This commit is contained in:
@ -221,11 +221,11 @@ func GetPlanFromPlanCache(ctx context.Context, sctx sessionctx.Context,
|
||||
}
|
||||
}
|
||||
|
||||
var matchOpts *PlanCacheMatchOpts
|
||||
var paramTypes []*types.FieldType
|
||||
if stmtCtx.UseCache() {
|
||||
var cacheVal kvcache.Value
|
||||
var hit, isPointPlan bool
|
||||
if stmt.PointGet.pointPlan != nil { // if it's PointGet Plan, no need to use MatchOpts
|
||||
if stmt.PointGet.pointPlan != nil { // if it's PointGet Plan, no need to use paramTypes
|
||||
cacheVal = &PlanCacheValue{
|
||||
Plan: stmt.PointGet.pointPlan,
|
||||
OutputColumns: stmt.PointGet.columnNames,
|
||||
@ -233,9 +233,9 @@ func GetPlanFromPlanCache(ctx context.Context, sctx sessionctx.Context,
|
||||
}
|
||||
isPointPlan, hit = true, true
|
||||
} else {
|
||||
matchOpts = GetMatchOpts(sctx, params)
|
||||
paramTypes = parseParamTypes(sctx, params)
|
||||
// TODO: consider instance-level plan cache
|
||||
cacheVal, hit = sctx.GetSessionPlanCache().Get(cacheKey, matchOpts)
|
||||
cacheVal, hit = sctx.GetSessionPlanCache().Get(cacheKey, paramTypes)
|
||||
}
|
||||
if hit {
|
||||
if intest.InTest && ctx.Value(PlanCacheKeyTestBeforeAdjust{}) != nil {
|
||||
@ -249,11 +249,11 @@ func GetPlanFromPlanCache(ctx context.Context, sctx sessionctx.Context,
|
||||
}
|
||||
}
|
||||
}
|
||||
if matchOpts == nil {
|
||||
matchOpts = GetMatchOpts(sctx, params)
|
||||
if paramTypes == nil {
|
||||
paramTypes = parseParamTypes(sctx, params)
|
||||
}
|
||||
|
||||
return generateNewPlan(ctx, sctx, isNonPrepared, is, stmt, cacheKey, matchOpts)
|
||||
return generateNewPlan(ctx, sctx, isNonPrepared, is, stmt, cacheKey, paramTypes)
|
||||
}
|
||||
|
||||
func adjustCachedPlan(sctx sessionctx.Context, cachedVal *PlanCacheValue, isNonPrepared, isPointPlan bool,
|
||||
@ -288,7 +288,7 @@ func adjustCachedPlan(sctx sessionctx.Context, cachedVal *PlanCacheValue, isNonP
|
||||
// generateNewPlan call the optimizer to generate a new plan for current statement
|
||||
// and try to add it to cache
|
||||
func generateNewPlan(ctx context.Context, sctx sessionctx.Context, isNonPrepared bool, is infoschema.InfoSchema,
|
||||
stmt *PlanCacheStmt, cacheKey string, matchOpts *PlanCacheMatchOpts) (base.Plan, []*types.FieldName, error) {
|
||||
stmt *PlanCacheStmt, cacheKey string, paramTypes []*types.FieldType) (base.Plan, []*types.FieldName, error) {
|
||||
stmtAst := stmt.PreparedAst
|
||||
sessVars := sctx.GetSessionVars()
|
||||
stmtCtx := sessVars.StmtCtx
|
||||
@ -303,18 +303,18 @@ func generateNewPlan(ctx context.Context, sctx sessionctx.Context, isNonPrepared
|
||||
|
||||
// check whether this plan is cacheable.
|
||||
if stmtCtx.UseCache() {
|
||||
if cacheable, reason := isPlanCacheable(sctx.GetPlanCtx(), p, len(matchOpts.ParamTypes), len(stmt.limits), stmt.hasSubquery); !cacheable {
|
||||
if cacheable, reason := isPlanCacheable(sctx.GetPlanCtx(), p, len(paramTypes), len(stmt.limits), stmt.hasSubquery); !cacheable {
|
||||
stmtCtx.SetSkipPlanCache(reason)
|
||||
}
|
||||
}
|
||||
|
||||
// put this plan into the plan cache.
|
||||
if stmtCtx.UseCache() {
|
||||
cached := NewPlanCacheValue(p, names, matchOpts, &stmtCtx.StmtHints)
|
||||
cached := NewPlanCacheValue(p, names, paramTypes, &stmtCtx.StmtHints)
|
||||
stmt.NormalizedPlan, stmt.PlanDigest = NormalizePlan(p)
|
||||
stmtCtx.SetPlan(p)
|
||||
stmtCtx.SetPlanDigest(stmt.NormalizedPlan, stmt.PlanDigest)
|
||||
sctx.GetSessionPlanCache().Put(cacheKey, cached, matchOpts)
|
||||
sctx.GetSessionPlanCache().Put(cacheKey, cached, paramTypes)
|
||||
if _, ok := p.(*PointGetPlan); ok {
|
||||
stmt.PointGet.pointPlan = p
|
||||
stmt.PointGet.columnNames = names
|
||||
|
||||
@ -74,22 +74,18 @@ func (pc *instancePlanCache) getHead(key string, create bool) *instancePCNode {
|
||||
return nil
|
||||
}
|
||||
|
||||
// Get gets the cached value according to key and opts.
|
||||
func (pc *instancePlanCache) Get(sctx sessionctx.Context, key string, opts any) (value any, ok bool) {
|
||||
// Get gets the cached value according to key and paramTypes.
|
||||
func (pc *instancePlanCache) Get(key string, paramTypes any) (value any, ok bool) {
|
||||
headNode := pc.getHead(key, false)
|
||||
if headNode == nil { // cache miss
|
||||
return nil, false
|
||||
}
|
||||
return pc.getPlanFromList(sctx, headNode, opts)
|
||||
return pc.getPlanFromList(headNode, paramTypes)
|
||||
}
|
||||
|
||||
func (*instancePlanCache) getPlanFromList(sctx sessionctx.Context, headNode *instancePCNode, opts any) (any, bool) {
|
||||
func (*instancePlanCache) getPlanFromList(headNode *instancePCNode, paramTypes any) (any, bool) {
|
||||
for node := headNode.next.Load(); node != nil; node = node.next.Load() {
|
||||
var matchOpts *PlanCacheMatchOpts
|
||||
if opts != nil {
|
||||
matchOpts = opts.(*PlanCacheMatchOpts)
|
||||
}
|
||||
if matchCachedPlan(sctx, node.value, matchOpts) { // v.Plan is read-only, no need to lock
|
||||
if checkTypesCompatibility4PC(node.value.paramTypes, paramTypes) { // v.Plan is read-only, no need to lock
|
||||
node.lastUsed.Store(time.Now()) // atomically update the lastUsed field
|
||||
return node.value, true
|
||||
}
|
||||
@ -99,7 +95,7 @@ func (*instancePlanCache) getPlanFromList(sctx sessionctx.Context, headNode *ins
|
||||
|
||||
// Put puts the key and values into the cache.
|
||||
// Due to some thread-safety issues, this Put operation might fail, use the returned succ to indicate it.
|
||||
func (pc *instancePlanCache) Put(sctx sessionctx.Context, key string, value, opts any) (succ bool) {
|
||||
func (pc *instancePlanCache) Put(key string, value, paramTypes any) (succ bool) {
|
||||
vMem := value.(*PlanCacheValue).MemoryUsage()
|
||||
if vMem+pc.totCost.Load() > pc.hardMemLimit.Load() {
|
||||
return // do nothing if it exceeds the hard limit
|
||||
@ -108,7 +104,7 @@ func (pc *instancePlanCache) Put(sctx sessionctx.Context, key string, value, opt
|
||||
if headNode == nil {
|
||||
return false // for safety
|
||||
}
|
||||
if _, ok := pc.getPlanFromList(sctx, headNode, opts); ok {
|
||||
if _, ok := pc.getPlanFromList(headNode, paramTypes); ok {
|
||||
return // some other thread has inserted the same plan before
|
||||
}
|
||||
|
||||
|
||||
@ -27,19 +27,19 @@ import (
|
||||
"github.com/stretchr/testify/require"
|
||||
)
|
||||
|
||||
func _put(sctx sessionctx.Context, pc sessionctx.InstancePlanCache, testKey, memUsage, statsHash int64) (succ bool) {
|
||||
v := &PlanCacheValue{testKey: testKey, memoryUsage: memUsage, matchOpts: &PlanCacheMatchOpts{}}
|
||||
return pc.Put(sctx, fmt.Sprintf("%v-%v", testKey, statsHash), v, &PlanCacheMatchOpts{})
|
||||
func _put(pc sessionctx.InstancePlanCache, testKey, memUsage, statsHash int64) (succ bool) {
|
||||
v := &PlanCacheValue{testKey: testKey, memoryUsage: memUsage}
|
||||
return pc.Put(fmt.Sprintf("%v-%v", testKey, statsHash), v, nil)
|
||||
}
|
||||
|
||||
func _hit(t *testing.T, sctx sessionctx.Context, pc sessionctx.InstancePlanCache, testKey, statsHash int) {
|
||||
v, ok := pc.Get(sctx, fmt.Sprintf("%v-%v", testKey, statsHash), &PlanCacheMatchOpts{})
|
||||
func _hit(t *testing.T, pc sessionctx.InstancePlanCache, testKey, statsHash int) {
|
||||
v, ok := pc.Get(fmt.Sprintf("%v-%v", testKey, statsHash), nil)
|
||||
require.True(t, ok)
|
||||
require.Equal(t, v.(*PlanCacheValue).testKey, int64(testKey))
|
||||
}
|
||||
|
||||
func _miss(t *testing.T, sctx sessionctx.Context, pc sessionctx.InstancePlanCache, testKey, statsHash int) {
|
||||
_, ok := pc.Get(sctx, fmt.Sprintf("%v-%v", testKey, statsHash), &PlanCacheMatchOpts{})
|
||||
func _miss(t *testing.T, pc sessionctx.InstancePlanCache, testKey, statsHash int) {
|
||||
_, ok := pc.Get(fmt.Sprintf("%v-%v", testKey, statsHash), nil)
|
||||
require.False(t, ok)
|
||||
}
|
||||
|
||||
@ -50,64 +50,64 @@ func TestInstancePlanCacheBasic(t *testing.T) {
|
||||
}()
|
||||
|
||||
pc := NewInstancePlanCache(1000, 1000)
|
||||
_put(sctx, pc, 1, 100, 0)
|
||||
_put(sctx, pc, 2, 100, 0)
|
||||
_put(sctx, pc, 3, 100, 0)
|
||||
_put(pc, 1, 100, 0)
|
||||
_put(pc, 2, 100, 0)
|
||||
_put(pc, 3, 100, 0)
|
||||
require.Equal(t, pc.MemUsage(), int64(300))
|
||||
_hit(t, sctx, pc, 1, 0)
|
||||
_hit(t, sctx, pc, 2, 0)
|
||||
_hit(t, sctx, pc, 3, 0)
|
||||
_hit(t, pc, 1, 0)
|
||||
_hit(t, pc, 2, 0)
|
||||
_hit(t, pc, 3, 0)
|
||||
|
||||
// exceed the hard limit during Put
|
||||
pc = NewInstancePlanCache(250, 250)
|
||||
_put(sctx, pc, 1, 100, 0)
|
||||
_put(sctx, pc, 2, 100, 0)
|
||||
_put(sctx, pc, 3, 100, 0)
|
||||
_put(pc, 1, 100, 0)
|
||||
_put(pc, 2, 100, 0)
|
||||
_put(pc, 3, 100, 0)
|
||||
require.Equal(t, pc.MemUsage(), int64(200))
|
||||
_hit(t, sctx, pc, 1, 0)
|
||||
_hit(t, sctx, pc, 2, 0)
|
||||
_miss(t, sctx, pc, 3, 0)
|
||||
_hit(t, pc, 1, 0)
|
||||
_hit(t, pc, 2, 0)
|
||||
_miss(t, pc, 3, 0)
|
||||
|
||||
// can't Put 2 same values
|
||||
pc = NewInstancePlanCache(250, 250)
|
||||
_put(sctx, pc, 1, 100, 0)
|
||||
_put(sctx, pc, 1, 101, 0)
|
||||
_put(pc, 1, 100, 0)
|
||||
_put(pc, 1, 101, 0)
|
||||
require.Equal(t, pc.MemUsage(), int64(100)) // the second one will be ignored
|
||||
|
||||
// eviction
|
||||
pc = NewInstancePlanCache(320, 500)
|
||||
_put(sctx, pc, 1, 100, 0)
|
||||
_put(sctx, pc, 2, 100, 0)
|
||||
_put(sctx, pc, 3, 100, 0)
|
||||
_put(sctx, pc, 4, 100, 0)
|
||||
_put(sctx, pc, 5, 100, 0)
|
||||
_hit(t, sctx, pc, 1, 0) // access 1-3 to refresh their last_used
|
||||
_hit(t, sctx, pc, 2, 0)
|
||||
_hit(t, sctx, pc, 3, 0)
|
||||
_put(pc, 1, 100, 0)
|
||||
_put(pc, 2, 100, 0)
|
||||
_put(pc, 3, 100, 0)
|
||||
_put(pc, 4, 100, 0)
|
||||
_put(pc, 5, 100, 0)
|
||||
_hit(t, pc, 1, 0) // access 1-3 to refresh their last_used
|
||||
_hit(t, pc, 2, 0)
|
||||
_hit(t, pc, 3, 0)
|
||||
require.Equal(t, pc.Evict(), true)
|
||||
require.Equal(t, pc.MemUsage(), int64(300))
|
||||
_hit(t, sctx, pc, 1, 0) // access 1-3 to refresh their last_used
|
||||
_hit(t, sctx, pc, 2, 0)
|
||||
_hit(t, sctx, pc, 3, 0)
|
||||
_miss(t, sctx, pc, 4, 0) // 4-5 have been evicted
|
||||
_miss(t, sctx, pc, 5, 0)
|
||||
_hit(t, pc, 1, 0) // access 1-3 to refresh their last_used
|
||||
_hit(t, pc, 2, 0)
|
||||
_hit(t, pc, 3, 0)
|
||||
_miss(t, pc, 4, 0) // 4-5 have been evicted
|
||||
_miss(t, pc, 5, 0)
|
||||
|
||||
// no need to eviction if mem < softLimit
|
||||
pc = NewInstancePlanCache(320, 500)
|
||||
_put(sctx, pc, 1, 100, 0)
|
||||
_put(sctx, pc, 2, 100, 0)
|
||||
_put(sctx, pc, 3, 100, 0)
|
||||
_put(pc, 1, 100, 0)
|
||||
_put(pc, 2, 100, 0)
|
||||
_put(pc, 3, 100, 0)
|
||||
require.Equal(t, pc.Evict(), false)
|
||||
require.Equal(t, pc.MemUsage(), int64(300))
|
||||
_hit(t, sctx, pc, 1, 0)
|
||||
_hit(t, sctx, pc, 2, 0)
|
||||
_hit(t, sctx, pc, 3, 0)
|
||||
_hit(t, pc, 1, 0)
|
||||
_hit(t, pc, 2, 0)
|
||||
_hit(t, pc, 3, 0)
|
||||
|
||||
// empty head should be dropped after eviction
|
||||
pc = NewInstancePlanCache(1, 500)
|
||||
_put(sctx, pc, 1, 100, 0)
|
||||
_put(sctx, pc, 2, 100, 0)
|
||||
_put(sctx, pc, 3, 100, 0)
|
||||
_put(pc, 1, 100, 0)
|
||||
_put(pc, 2, 100, 0)
|
||||
_put(pc, 3, 100, 0)
|
||||
require.Equal(t, pc.MemUsage(), int64(300))
|
||||
pcImpl := pc.(*instancePlanCache)
|
||||
numHeads := 0
|
||||
@ -129,58 +129,58 @@ func TestInstancePlanCacheWithMatchOpts(t *testing.T) {
|
||||
|
||||
// same key with different statsHash
|
||||
pc := NewInstancePlanCache(1000, 1000)
|
||||
_put(sctx, pc, 1, 100, 1)
|
||||
_put(sctx, pc, 1, 100, 2)
|
||||
_put(sctx, pc, 1, 100, 3)
|
||||
_hit(t, sctx, pc, 1, 1)
|
||||
_hit(t, sctx, pc, 1, 2)
|
||||
_hit(t, sctx, pc, 1, 3)
|
||||
_miss(t, sctx, pc, 1, 4)
|
||||
_miss(t, sctx, pc, 2, 1)
|
||||
_put(pc, 1, 100, 1)
|
||||
_put(pc, 1, 100, 2)
|
||||
_put(pc, 1, 100, 3)
|
||||
_hit(t, pc, 1, 1)
|
||||
_hit(t, pc, 1, 2)
|
||||
_hit(t, pc, 1, 3)
|
||||
_miss(t, pc, 1, 4)
|
||||
_miss(t, pc, 2, 1)
|
||||
|
||||
// multiple keys with same statsHash
|
||||
pc = NewInstancePlanCache(1000, 1000)
|
||||
_put(sctx, pc, 1, 100, 1)
|
||||
_put(sctx, pc, 1, 100, 2)
|
||||
_put(sctx, pc, 2, 100, 1)
|
||||
_put(sctx, pc, 2, 100, 2)
|
||||
_hit(t, sctx, pc, 1, 1)
|
||||
_hit(t, sctx, pc, 1, 2)
|
||||
_miss(t, sctx, pc, 1, 3)
|
||||
_hit(t, sctx, pc, 2, 1)
|
||||
_hit(t, sctx, pc, 2, 2)
|
||||
_miss(t, sctx, pc, 2, 3)
|
||||
_miss(t, sctx, pc, 3, 1)
|
||||
_miss(t, sctx, pc, 3, 2)
|
||||
_miss(t, sctx, pc, 3, 3)
|
||||
_put(pc, 1, 100, 1)
|
||||
_put(pc, 1, 100, 2)
|
||||
_put(pc, 2, 100, 1)
|
||||
_put(pc, 2, 100, 2)
|
||||
_hit(t, pc, 1, 1)
|
||||
_hit(t, pc, 1, 2)
|
||||
_miss(t, pc, 1, 3)
|
||||
_hit(t, pc, 2, 1)
|
||||
_hit(t, pc, 2, 2)
|
||||
_miss(t, pc, 2, 3)
|
||||
_miss(t, pc, 3, 1)
|
||||
_miss(t, pc, 3, 2)
|
||||
_miss(t, pc, 3, 3)
|
||||
|
||||
// hard limit can take effect in this case
|
||||
pc = NewInstancePlanCache(200, 200)
|
||||
_put(sctx, pc, 1, 100, 1)
|
||||
_put(sctx, pc, 1, 100, 2)
|
||||
_put(sctx, pc, 1, 100, 3) // the third one will be ignored
|
||||
_put(pc, 1, 100, 1)
|
||||
_put(pc, 1, 100, 2)
|
||||
_put(pc, 1, 100, 3) // the third one will be ignored
|
||||
require.Equal(t, pc.MemUsage(), int64(200))
|
||||
_hit(t, sctx, pc, 1, 1)
|
||||
_hit(t, sctx, pc, 1, 2)
|
||||
_miss(t, sctx, pc, 1, 3)
|
||||
_hit(t, pc, 1, 1)
|
||||
_hit(t, pc, 1, 2)
|
||||
_miss(t, pc, 1, 3)
|
||||
|
||||
// eviction this case
|
||||
pc = NewInstancePlanCache(300, 500)
|
||||
_put(sctx, pc, 1, 100, 1)
|
||||
_put(sctx, pc, 1, 100, 2)
|
||||
_put(sctx, pc, 1, 100, 3)
|
||||
_put(sctx, pc, 1, 100, 4)
|
||||
_put(sctx, pc, 1, 100, 5)
|
||||
_hit(t, sctx, pc, 1, 1) // refresh 1-3's last_used
|
||||
_hit(t, sctx, pc, 1, 2)
|
||||
_hit(t, sctx, pc, 1, 3)
|
||||
_put(pc, 1, 100, 1)
|
||||
_put(pc, 1, 100, 2)
|
||||
_put(pc, 1, 100, 3)
|
||||
_put(pc, 1, 100, 4)
|
||||
_put(pc, 1, 100, 5)
|
||||
_hit(t, pc, 1, 1) // refresh 1-3's last_used
|
||||
_hit(t, pc, 1, 2)
|
||||
_hit(t, pc, 1, 3)
|
||||
require.True(t, pc.Evict())
|
||||
require.Equal(t, pc.MemUsage(), int64(300))
|
||||
_hit(t, sctx, pc, 1, 1)
|
||||
_hit(t, sctx, pc, 1, 2)
|
||||
_hit(t, sctx, pc, 1, 3)
|
||||
_miss(t, sctx, pc, 1, 4)
|
||||
_miss(t, sctx, pc, 1, 5)
|
||||
_hit(t, pc, 1, 1)
|
||||
_hit(t, pc, 1, 2)
|
||||
_hit(t, pc, 1, 3)
|
||||
_miss(t, pc, 1, 4)
|
||||
_miss(t, pc, 1, 5)
|
||||
}
|
||||
|
||||
func TestInstancePlanCacheConcurrentRead(t *testing.T) {
|
||||
@ -195,7 +195,7 @@ func TestInstancePlanCacheConcurrentRead(t *testing.T) {
|
||||
for k := 0; k < 100; k++ {
|
||||
for statsHash := 0; statsHash < 100; statsHash++ {
|
||||
if rand.Intn(10) < 7 {
|
||||
_put(sctx, pc, int64(k), 1, int64(statsHash))
|
||||
_put(pc, int64(k), 1, int64(statsHash))
|
||||
flag[k][statsHash] = true
|
||||
}
|
||||
}
|
||||
@ -209,9 +209,9 @@ func TestInstancePlanCacheConcurrentRead(t *testing.T) {
|
||||
for i := 0; i < 10000; i++ {
|
||||
k, statsHash := rand.Intn(100), rand.Intn(100)
|
||||
if flag[k][statsHash] {
|
||||
_hit(t, sctx, pc, k, statsHash)
|
||||
_hit(t, pc, k, statsHash)
|
||||
} else {
|
||||
_miss(t, sctx, pc, k, statsHash)
|
||||
_miss(t, pc, k, statsHash)
|
||||
}
|
||||
time.Sleep(time.Nanosecond * 10)
|
||||
}
|
||||
@ -235,7 +235,7 @@ func TestInstancePlanCacheConcurrentWriteRead(t *testing.T) {
|
||||
defer wg.Done()
|
||||
for i := 0; i < 1000; i++ {
|
||||
k, statsHash := rand.Intn(100), rand.Intn(100)
|
||||
if _put(sctx, pc, int64(k), 1, int64(statsHash)) {
|
||||
if _put(pc, int64(k), 1, int64(statsHash)) {
|
||||
flag[k][statsHash].Store(true)
|
||||
}
|
||||
time.Sleep(time.Nanosecond * 10)
|
||||
@ -249,7 +249,7 @@ func TestInstancePlanCacheConcurrentWriteRead(t *testing.T) {
|
||||
for i := 0; i < 2000; i++ {
|
||||
k, statsHash := rand.Intn(100), rand.Intn(100)
|
||||
if flag[k][statsHash].Load() {
|
||||
_hit(t, sctx, pc, k, statsHash)
|
||||
_hit(t, pc, k, statsHash)
|
||||
}
|
||||
time.Sleep(time.Nanosecond * 5)
|
||||
}
|
||||
|
||||
@ -79,13 +79,13 @@ func NewLRUPlanCache(capacity uint, guard float64, quota uint64, sctx sessionctx
|
||||
}
|
||||
|
||||
// Get tries to find the corresponding value according to the given key.
|
||||
func (l *LRUPlanCache) Get(key string, opts any) (value any, ok bool) {
|
||||
func (l *LRUPlanCache) Get(key string, paramTypes any) (value any, ok bool) {
|
||||
l.lock.RLock()
|
||||
defer l.lock.RUnlock()
|
||||
|
||||
bucket, bucketExist := l.buckets[key]
|
||||
if bucketExist {
|
||||
if element, exist := l.pickFromBucket(bucket, opts); exist {
|
||||
if element, exist := l.pickFromBucket(bucket, paramTypes); exist {
|
||||
l.lruList.MoveToFront(element)
|
||||
return element.Value.(*planCacheEntry).PlanValue, true
|
||||
}
|
||||
@ -94,13 +94,13 @@ func (l *LRUPlanCache) Get(key string, opts any) (value any, ok bool) {
|
||||
}
|
||||
|
||||
// Put puts the (key, value) pair into the LRU Cache.
|
||||
func (l *LRUPlanCache) Put(key string, value, opts any) {
|
||||
func (l *LRUPlanCache) Put(key string, value, paramTypes any) {
|
||||
l.lock.Lock()
|
||||
defer l.lock.Unlock()
|
||||
|
||||
bucket, bucketExist := l.buckets[key]
|
||||
if bucketExist {
|
||||
if element, exist := l.pickFromBucket(bucket, opts); exist {
|
||||
if element, exist := l.pickFromBucket(bucket, paramTypes); exist {
|
||||
l.updateInstanceMetric(&planCacheEntry{PlanKey: key, PlanValue: value}, element.Value.(*planCacheEntry))
|
||||
element.Value.(*planCacheEntry).PlanValue = value
|
||||
l.lruList.MoveToFront(element)
|
||||
@ -239,13 +239,9 @@ func (l *LRUPlanCache) memoryControl() {
|
||||
}
|
||||
|
||||
// PickPlanFromBucket pick one plan from bucket
|
||||
func (l *LRUPlanCache) pickFromBucket(bucket map[*list.Element]struct{}, opts any) (*list.Element, bool) {
|
||||
var matchOpts *PlanCacheMatchOpts
|
||||
if opts != nil {
|
||||
matchOpts = opts.(*PlanCacheMatchOpts)
|
||||
}
|
||||
func (*LRUPlanCache) pickFromBucket(bucket map[*list.Element]struct{}, paramTypes any) (*list.Element, bool) {
|
||||
for k := range bucket {
|
||||
if matchCachedPlan(l.sctx, k.Value.(*planCacheEntry).PlanValue.(*PlanCacheValue), matchOpts) {
|
||||
if checkTypesCompatibility4PC(k.Value.(*planCacheEntry).PlanValue.(*PlanCacheValue).paramTypes, paramTypes) {
|
||||
return k, true
|
||||
}
|
||||
}
|
||||
|
||||
@ -37,8 +37,8 @@ func randomPlanCacheValue(types []*types.FieldType) *PlanCacheValue {
|
||||
&PhysicalIndexLookUpReader{}, &PhysicalApply{}, &PhysicalApply{}, &PhysicalLimit{}}
|
||||
random := rand.New(rand.NewSource(time.Now().UnixNano()))
|
||||
return &PlanCacheValue{
|
||||
Plan: plans[random.Int()%len(plans)],
|
||||
matchOpts: &PlanCacheMatchOpts{ParamTypes: types},
|
||||
Plan: plans[random.Int()%len(plans)],
|
||||
paramTypes: types,
|
||||
}
|
||||
}
|
||||
|
||||
@ -71,11 +71,9 @@ func TestLRUPCPut(t *testing.T) {
|
||||
// one key corresponding to multi values
|
||||
for i := 0; i < 5; i++ {
|
||||
keys[i] = "key-1"
|
||||
opts := &PlanCacheMatchOpts{
|
||||
ParamTypes: pTypes[i],
|
||||
}
|
||||
opts := pTypes[i]
|
||||
vals[i] = &PlanCacheValue{
|
||||
matchOpts: opts,
|
||||
paramTypes: opts,
|
||||
}
|
||||
lru.Put(keys[i], vals[i], opts)
|
||||
}
|
||||
@ -107,9 +105,7 @@ func TestLRUPCPut(t *testing.T) {
|
||||
|
||||
bucket, exist := lru.buckets[keys[i]]
|
||||
require.True(t, exist)
|
||||
matchOpts := &PlanCacheMatchOpts{
|
||||
ParamTypes: pTypes[i],
|
||||
}
|
||||
matchOpts := pTypes[i]
|
||||
element, exist := lru.pickFromBucket(bucket, matchOpts)
|
||||
require.NotNil(t, element)
|
||||
require.True(t, exist)
|
||||
@ -146,29 +142,23 @@ func TestLRUPCGet(t *testing.T) {
|
||||
// 5 bucket
|
||||
for i := 0; i < 5; i++ {
|
||||
keys[i] = fmt.Sprintf("key-%v", i%4)
|
||||
opts := &PlanCacheMatchOpts{
|
||||
ParamTypes: pTypes[i],
|
||||
}
|
||||
opts := pTypes[i]
|
||||
vals[i] = &PlanCacheValue{
|
||||
matchOpts: opts,
|
||||
paramTypes: opts,
|
||||
}
|
||||
lru.Put(keys[i], vals[i], opts)
|
||||
}
|
||||
|
||||
// test for non-existent elements
|
||||
for i := 0; i < 2; i++ {
|
||||
opts := &PlanCacheMatchOpts{
|
||||
ParamTypes: pTypes[i],
|
||||
}
|
||||
opts := pTypes[i]
|
||||
value, exists := lru.Get(keys[i], opts)
|
||||
require.False(t, exists)
|
||||
require.Nil(t, value)
|
||||
}
|
||||
|
||||
for i := 2; i < 5; i++ {
|
||||
opts := &PlanCacheMatchOpts{
|
||||
ParamTypes: pTypes[i],
|
||||
}
|
||||
opts := pTypes[i]
|
||||
value, exists := lru.Get(keys[i], opts)
|
||||
require.True(t, exists)
|
||||
require.NotNil(t, value)
|
||||
@ -205,11 +195,9 @@ func TestLRUPCDelete(t *testing.T) {
|
||||
}
|
||||
for i := 0; i < 3; i++ {
|
||||
keys[i] = fmt.Sprintf("key-%v", i)
|
||||
opts := &PlanCacheMatchOpts{
|
||||
ParamTypes: pTypes[i],
|
||||
}
|
||||
opts := pTypes[i]
|
||||
vals[i] = &PlanCacheValue{
|
||||
matchOpts: opts,
|
||||
paramTypes: opts,
|
||||
}
|
||||
lru.Put(keys[i], vals[i], opts)
|
||||
}
|
||||
@ -217,21 +205,15 @@ func TestLRUPCDelete(t *testing.T) {
|
||||
|
||||
lru.Delete(keys[1])
|
||||
|
||||
value, exists := lru.Get(keys[1], &PlanCacheMatchOpts{
|
||||
ParamTypes: pTypes[1],
|
||||
})
|
||||
value, exists := lru.Get(keys[1], pTypes[1])
|
||||
require.False(t, exists)
|
||||
require.Nil(t, value)
|
||||
require.Equal(t, 2, int(lru.size))
|
||||
|
||||
_, exists = lru.Get(keys[0], &PlanCacheMatchOpts{
|
||||
ParamTypes: pTypes[0],
|
||||
})
|
||||
_, exists = lru.Get(keys[0], pTypes[0])
|
||||
require.True(t, exists)
|
||||
|
||||
_, exists = lru.Get(keys[2], &PlanCacheMatchOpts{
|
||||
ParamTypes: pTypes[2],
|
||||
})
|
||||
_, exists = lru.Get(keys[2], pTypes[2])
|
||||
require.True(t, exists)
|
||||
}
|
||||
|
||||
@ -249,11 +231,9 @@ func TestLRUPCDeleteAll(t *testing.T) {
|
||||
}
|
||||
for i := 0; i < 3; i++ {
|
||||
keys[i] = fmt.Sprintf("key-%v", i)
|
||||
opts := &PlanCacheMatchOpts{
|
||||
ParamTypes: pTypes[i],
|
||||
}
|
||||
opts := pTypes[i]
|
||||
vals[i] = &PlanCacheValue{
|
||||
matchOpts: opts,
|
||||
paramTypes: opts,
|
||||
}
|
||||
lru.Put(keys[i], vals[i], opts)
|
||||
}
|
||||
@ -262,9 +242,7 @@ func TestLRUPCDeleteAll(t *testing.T) {
|
||||
lru.DeleteAll()
|
||||
|
||||
for i := 0; i < 3; i++ {
|
||||
opts := &PlanCacheMatchOpts{
|
||||
ParamTypes: pTypes[i],
|
||||
}
|
||||
opts := pTypes[i]
|
||||
value, exists := lru.Get(keys[i], opts)
|
||||
require.False(t, exists)
|
||||
require.Nil(t, value)
|
||||
@ -296,11 +274,9 @@ func TestLRUPCSetCapacity(t *testing.T) {
|
||||
// one key corresponding to multi values
|
||||
for i := 0; i < 5; i++ {
|
||||
keys[i] = "key-1"
|
||||
opts := &PlanCacheMatchOpts{
|
||||
ParamTypes: pTypes[i],
|
||||
}
|
||||
opts := pTypes[i]
|
||||
vals[i] = &PlanCacheValue{
|
||||
matchOpts: opts,
|
||||
paramTypes: opts,
|
||||
}
|
||||
lru.Put(keys[i], vals[i], opts)
|
||||
}
|
||||
@ -351,10 +327,8 @@ func TestIssue37914(t *testing.T) {
|
||||
}()
|
||||
pTypes := []*types.FieldType{types.NewFieldType(mysql.TypeFloat), types.NewFieldType(mysql.TypeDouble)}
|
||||
key := "key-1"
|
||||
opts := &PlanCacheMatchOpts{
|
||||
ParamTypes: pTypes,
|
||||
}
|
||||
val := &PlanCacheValue{matchOpts: opts}
|
||||
opts := pTypes
|
||||
val := &PlanCacheValue{paramTypes: opts}
|
||||
|
||||
require.NotPanics(t, func() {
|
||||
lru.Put(key, val, opts)
|
||||
@ -381,10 +355,8 @@ func TestIssue38244(t *testing.T) {
|
||||
// one key corresponding to multi values
|
||||
for i := 0; i < 5; i++ {
|
||||
keys[i] = fmt.Sprintf("key-%v", i)
|
||||
opts := &PlanCacheMatchOpts{
|
||||
ParamTypes: pTypes[i],
|
||||
}
|
||||
vals[i] = &PlanCacheValue{matchOpts: opts}
|
||||
opts := pTypes[i]
|
||||
vals[i] = &PlanCacheValue{paramTypes: opts}
|
||||
lru.Put(keys[i], vals[i], opts)
|
||||
}
|
||||
require.Equal(t, lru.size, lru.capacity)
|
||||
@ -409,9 +381,7 @@ func TestLRUPlanCacheMemoryUsage(t *testing.T) {
|
||||
for i := 0; i < 3; i++ {
|
||||
k := randomPlanCacheKey()
|
||||
v := randomPlanCacheValue(pTypes)
|
||||
opts := &PlanCacheMatchOpts{
|
||||
ParamTypes: pTypes,
|
||||
}
|
||||
opts := pTypes
|
||||
lru.Put(k, v, opts)
|
||||
res += int64(len(k)) + v.MemoryUsage()
|
||||
require.Equal(t, lru.MemoryUsage(), res)
|
||||
@ -420,9 +390,7 @@ func TestLRUPlanCacheMemoryUsage(t *testing.T) {
|
||||
p := &PhysicalTableScan{}
|
||||
k := "key-3"
|
||||
v := &PlanCacheValue{Plan: p}
|
||||
opts := &PlanCacheMatchOpts{
|
||||
ParamTypes: pTypes,
|
||||
}
|
||||
opts := pTypes
|
||||
lru.Put(k, v, opts)
|
||||
res += int64(len(k)) + v.MemoryUsage()
|
||||
for kk, vv := range evict {
|
||||
|
||||
@ -397,8 +397,8 @@ type PlanCacheValue struct {
|
||||
memoryUsage int64
|
||||
testKey int64 // this is only for test
|
||||
|
||||
// matchOpts stores some fields help to choose a suitable plan
|
||||
matchOpts *PlanCacheMatchOpts
|
||||
// paramTypes stores all parameters' FieldType, some different parameters may share same plan
|
||||
paramTypes []*types.FieldType
|
||||
// stmtHints stores the hints which set session variables, because the hints won't be processed using cached plan.
|
||||
stmtHints *hint.StmtHints
|
||||
}
|
||||
@ -431,9 +431,9 @@ func (v *PlanCacheValue) MemoryUsage() (sum int64) {
|
||||
|
||||
sum += size.SizeOfInterface + size.SizeOfSlice*2 + int64(cap(v.OutputColumns))*size.SizeOfPointer +
|
||||
size.SizeOfMap + size.SizeOfInt64*2
|
||||
if v.matchOpts != nil {
|
||||
sum += int64(cap(v.matchOpts.ParamTypes)) * size.SizeOfPointer
|
||||
for _, ft := range v.matchOpts.ParamTypes {
|
||||
if v.paramTypes != nil {
|
||||
sum += int64(cap(v.paramTypes)) * size.SizeOfPointer
|
||||
for _, ft := range v.paramTypes {
|
||||
sum += ft.MemoryUsage()
|
||||
}
|
||||
}
|
||||
@ -447,26 +447,19 @@ func (v *PlanCacheValue) MemoryUsage() (sum int64) {
|
||||
|
||||
// NewPlanCacheValue creates a SQLCacheValue.
|
||||
func NewPlanCacheValue(plan base.Plan, names []*types.FieldName,
|
||||
matchOpts *PlanCacheMatchOpts, stmtHints *hint.StmtHints) *PlanCacheValue {
|
||||
userParamTypes := make([]*types.FieldType, len(matchOpts.ParamTypes))
|
||||
for i, tp := range matchOpts.ParamTypes {
|
||||
paramTypes []*types.FieldType, stmtHints *hint.StmtHints) *PlanCacheValue {
|
||||
userParamTypes := make([]*types.FieldType, len(paramTypes))
|
||||
for i, tp := range paramTypes {
|
||||
userParamTypes[i] = tp.Clone()
|
||||
}
|
||||
return &PlanCacheValue{
|
||||
Plan: plan,
|
||||
OutputColumns: names,
|
||||
matchOpts: matchOpts,
|
||||
paramTypes: userParamTypes,
|
||||
stmtHints: stmtHints.Clone(),
|
||||
}
|
||||
}
|
||||
|
||||
// PlanCacheMatchOpts store some property used to fetch plan from plan cache
|
||||
// The structure set here is to avoid import cycle
|
||||
type PlanCacheMatchOpts struct {
|
||||
// paramTypes stores all parameters' FieldType, some different parameters may share same plan
|
||||
ParamTypes []*types.FieldType
|
||||
}
|
||||
|
||||
// planCacheStmtProcessor records all query features which may affect plan selection.
|
||||
type planCacheStmtProcessor struct {
|
||||
is infoschema.InfoSchema
|
||||
@ -570,17 +563,16 @@ func GetPreparedStmt(stmt *ast.ExecuteStmt, vars *variable.SessionVars) (*PlanCa
|
||||
return nil, plannererrors.ErrStmtNotFound
|
||||
}
|
||||
|
||||
// GetMatchOpts get options to fetch plan or generate new plan
|
||||
// we can add more options here
|
||||
func GetMatchOpts(sctx sessionctx.Context, params []expression.Expression) *PlanCacheMatchOpts {
|
||||
return &PlanCacheMatchOpts{ParamTypes: parseParamTypes(sctx, params)}
|
||||
}
|
||||
|
||||
// CheckTypesCompatibility4PC compares FieldSlice with []*types.FieldType
|
||||
// Currently this is only used in plan cache to check whether the types of parameters are compatible.
|
||||
// If the types of parameters are compatible, we can use the cached plan.
|
||||
// tpsExpected is types from cached plan
|
||||
func checkTypesCompatibility4PC(tpsExpected, tpsActual []*types.FieldType) bool {
|
||||
func checkTypesCompatibility4PC(expected, actual any) bool {
|
||||
if expected == nil || actual == nil {
|
||||
return true // no need to compare types
|
||||
}
|
||||
tpsExpected := expected.([]*types.FieldType)
|
||||
tpsActual := actual.([]*types.FieldType)
|
||||
if len(tpsExpected) != len(tpsActual) {
|
||||
return false
|
||||
}
|
||||
@ -711,14 +703,3 @@ func parseParamTypes(sctx sessionctx.Context, params []expression.Expression) (p
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
// matchCachedPlan checks whether this plan is matched with these match-options.
|
||||
func matchCachedPlan(_ sessionctx.Context, value *PlanCacheValue, matchOpts *PlanCacheMatchOpts) bool {
|
||||
if matchOpts == nil { // if PointGet, the matchOpts is nil
|
||||
return true
|
||||
}
|
||||
if !checkTypesCompatibility4PC(value.matchOpts.ParamTypes, matchOpts.ParamTypes) {
|
||||
return false
|
||||
}
|
||||
return true
|
||||
}
|
||||
|
||||
@ -54,8 +54,8 @@ type SessionStatesHandler interface {
|
||||
|
||||
// SessionPlanCache is an interface for prepare and non-prepared plan cache
|
||||
type SessionPlanCache interface {
|
||||
Get(key string, opts any) (value any, ok bool)
|
||||
Put(key string, value, opts any)
|
||||
Get(key string, paramTypes any) (value any, ok bool)
|
||||
Put(key string, value, paramTypes any)
|
||||
Delete(key string)
|
||||
DeleteAll()
|
||||
Size() int
|
||||
@ -67,9 +67,9 @@ type SessionPlanCache interface {
|
||||
// Value and Opts should always be *PlanCacheValue and *PlanCacheMatchOpts, use any to avoid cycle-import.
|
||||
type InstancePlanCache interface {
|
||||
// Get gets the cached value from the cache according to key and opts.
|
||||
Get(sctx Context, key string, opts any) (value any, ok bool)
|
||||
Get(key string, paramTypes any) (value any, ok bool)
|
||||
// Put puts the key and value into the cache.
|
||||
Put(sctx Context, key string, value, opts any) (succ bool)
|
||||
Put(key string, value, paramTypes any) (succ bool)
|
||||
// Evict evicts some cached values.
|
||||
Evict() (evicted bool)
|
||||
// MemUsage returns the total memory usage of this plan cache.
|
||||
|
||||
Reference in New Issue
Block a user