planner: refactor a few code of plan cache (#54464)

ref pingcap/tidb#54057
This commit is contained in:
Yuanjia Zhang
2024-07-05 11:50:58 +08:00
committed by GitHub
parent 0c9a679117
commit d19fc99ec6
7 changed files with 156 additions and 215 deletions

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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