planner: refactor plan cache value code (#40656)

ref pingcap/tidb#40219
This commit is contained in:
fzzf678
2023-01-18 17:21:51 +08:00
committed by GitHub
parent 294408188e
commit 970ac9150c
3 changed files with 68 additions and 31 deletions

View File

@ -53,7 +53,7 @@ type LRUPlanCache struct {
lock sync.Mutex
// pickFromBucket get one element from bucket. The LRUPlanCache can not work if it is nil
pickFromBucket func(map[*list.Element]struct{}, []*types.FieldType, []uint64) (*list.Element, bool)
pickFromBucket func(map[*list.Element]struct{}, *planCacheMatchOpts) (*list.Element, bool)
// onEvict will be called if any eviction happened, only for test use now
onEvict func(kvcache.Key, kvcache.Value)
@ -68,7 +68,7 @@ type LRUPlanCache struct {
// NewLRUPlanCache creates a PCLRUCache object, whose capacity is "capacity".
// NOTE: "capacity" should be a positive value.
func NewLRUPlanCache(capacity uint, guard float64, quota uint64,
pickFromBucket func(map[*list.Element]struct{}, []*types.FieldType, []uint64) (*list.Element, bool), sctx sessionctx.Context) *LRUPlanCache {
pickFromBucket func(map[*list.Element]struct{}, *planCacheMatchOpts) (*list.Element, bool), sctx sessionctx.Context) *LRUPlanCache {
if capacity < 1 {
capacity = 100
logutil.BgLogger().Info("capacity of LRU cache is less than 1, will use default value(100) init cache")
@ -100,7 +100,11 @@ func (l *LRUPlanCache) Get(key kvcache.Key, paramTypes []*types.FieldType, limit
bucket, bucketExist := l.buckets[strHashKey(key, false)]
if bucketExist {
if element, exist := l.pickFromBucket(bucket, paramTypes, limitParams); exist {
matchOpts := &planCacheMatchOpts{
paramTypes: paramTypes,
limitOffsetAndCount: limitParams,
}
if element, exist := l.pickFromBucket(bucket, matchOpts); exist {
l.lruList.MoveToFront(element)
return element.Value.(*planCacheEntry).PlanValue, true
}
@ -116,7 +120,11 @@ func (l *LRUPlanCache) Put(key kvcache.Key, value kvcache.Value, paramTypes []*t
hash := strHashKey(key, true)
bucket, bucketExist := l.buckets[hash]
if bucketExist {
if element, exist := l.pickFromBucket(bucket, paramTypes, limitParams); exist {
matchOpts := &planCacheMatchOpts{
paramTypes: paramTypes,
limitOffsetAndCount: limitParams,
}
if element, exist := l.pickFromBucket(bucket, matchOpts); exist {
l.updateInstanceMetric(&planCacheEntry{PlanKey: key, PlanValue: value}, element.Value.(*planCacheEntry))
element.Value.(*planCacheEntry).PlanValue = value
l.lruList.MoveToFront(element)
@ -252,14 +260,14 @@ func (l *LRUPlanCache) memoryControl() {
}
// PickPlanFromBucket pick one plan from bucket
func PickPlanFromBucket(bucket map[*list.Element]struct{}, paramTypes []*types.FieldType, limitParams []uint64) (*list.Element, bool) {
func PickPlanFromBucket(bucket map[*list.Element]struct{}, matchOpts *planCacheMatchOpts) (*list.Element, bool) {
for k := range bucket {
plan := k.Value.(*planCacheEntry).PlanValue.(*PlanCacheValue)
ok1 := plan.ParamTypes.CheckTypesCompatibility4PC(paramTypes)
ok1 := plan.matchOpts.paramTypes.CheckTypesCompatibility4PC(matchOpts.paramTypes)
if !ok1 {
continue
}
ok2 := checkUint64SliceIfEqual(plan.limitOffsetAndCount, limitParams)
ok2 := checkUint64SliceIfEqual(plan.matchOpts.limitOffsetAndCount, matchOpts.limitOffsetAndCount)
if ok2 {
return k, true
}

View File

@ -40,8 +40,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)],
ParamTypes: types,
Plan: plans[random.Int()%len(plans)],
matchOpts: planCacheMatchOpts{paramTypes: types},
}
}
@ -73,8 +73,10 @@ func TestLRUPCPut(t *testing.T) {
for i := 0; i < 5; i++ {
keys[i] = &planCacheKey{database: strconv.FormatInt(int64(1), 10)}
vals[i] = &PlanCacheValue{
ParamTypes: pTypes[i],
limitOffsetAndCount: limitParams[i],
matchOpts: planCacheMatchOpts{
paramTypes: pTypes[i],
limitOffsetAndCount: limitParams[i],
},
}
lru.Put(keys[i], vals[i], pTypes[i], limitParams[i])
}
@ -107,7 +109,11 @@ func TestLRUPCPut(t *testing.T) {
bucket, exist := lru.buckets[string(hack.String(keys[i].Hash()))]
require.True(t, exist)
element, exist := lru.pickFromBucket(bucket, pTypes[i], limitParams[i])
matchOpts := &planCacheMatchOpts{
paramTypes: pTypes[i],
limitOffsetAndCount: limitParams[i],
}
element, exist := lru.pickFromBucket(bucket, matchOpts)
require.NotNil(t, element)
require.True(t, exist)
require.Equal(t, root, element)
@ -141,7 +147,12 @@ func TestLRUPCGet(t *testing.T) {
// 5 bucket
for i := 0; i < 5; i++ {
keys[i] = &planCacheKey{database: strconv.FormatInt(int64(i%4), 10)}
vals[i] = &PlanCacheValue{ParamTypes: pTypes[i], limitOffsetAndCount: limitParams[i]}
vals[i] = &PlanCacheValue{
matchOpts: planCacheMatchOpts{
paramTypes: pTypes[i],
limitOffsetAndCount: limitParams[i],
},
}
lru.Put(keys[i], vals[i], pTypes[i], limitParams[i])
}
@ -188,8 +199,10 @@ func TestLRUPCDelete(t *testing.T) {
for i := 0; i < 3; i++ {
keys[i] = &planCacheKey{database: strconv.FormatInt(int64(i), 10)}
vals[i] = &PlanCacheValue{
ParamTypes: pTypes[i],
limitOffsetAndCount: limitParams[i],
matchOpts: planCacheMatchOpts{
paramTypes: pTypes[i],
limitOffsetAndCount: limitParams[i],
},
}
lru.Put(keys[i], vals[i], pTypes[i], []uint64{})
}
@ -219,7 +232,11 @@ func TestLRUPCDeleteAll(t *testing.T) {
}
for i := 0; i < 3; i++ {
keys[i] = &planCacheKey{database: strconv.FormatInt(int64(i), 10)}
vals[i] = &PlanCacheValue{ParamTypes: pTypes[i]}
vals[i] = &PlanCacheValue{
matchOpts: planCacheMatchOpts{
paramTypes: pTypes[i],
},
}
lru.Put(keys[i], vals[i], pTypes[i], []uint64{})
}
require.Equal(t, 3, int(lru.size))
@ -254,7 +271,10 @@ func TestLRUPCSetCapacity(t *testing.T) {
// one key corresponding to multi values
for i := 0; i < 5; i++ {
keys[i] = &planCacheKey{database: strconv.FormatInt(int64(1), 10)}
vals[i] = &PlanCacheValue{ParamTypes: pTypes[i]}
vals[i] = &PlanCacheValue{
matchOpts: planCacheMatchOpts{
paramTypes: pTypes[i],
}}
lru.Put(keys[i], vals[i], pTypes[i], []uint64{})
}
require.Equal(t, lru.size, lru.capacity)
@ -302,7 +322,7 @@ func TestIssue37914(t *testing.T) {
pTypes := []*types.FieldType{types.NewFieldType(mysql.TypeFloat), types.NewFieldType(mysql.TypeDouble)}
key := &planCacheKey{database: strconv.FormatInt(int64(1), 10)}
val := &PlanCacheValue{ParamTypes: pTypes}
val := &PlanCacheValue{matchOpts: planCacheMatchOpts{paramTypes: pTypes}}
require.NotPanics(t, func() {
lru.Put(key, val, pTypes, []uint64{})
@ -325,7 +345,7 @@ func TestIssue38244(t *testing.T) {
// one key corresponding to multi values
for i := 0; i < 5; i++ {
keys[i] = &planCacheKey{database: strconv.FormatInt(int64(i), 10)}
vals[i] = &PlanCacheValue{ParamTypes: pTypes[i]}
vals[i] = &PlanCacheValue{matchOpts: planCacheMatchOpts{paramTypes: pTypes[i]}}
lru.Put(keys[i], vals[i], pTypes[i], []uint64{})
}
require.Equal(t, lru.size, lru.capacity)

View File

@ -251,6 +251,14 @@ func (key *planCacheKey) MemoryUsage() (sum int64) {
return
}
type planCacheMatchOpts struct {
// paramTypes stores all parameters' FieldType, some different parameters may share same plan
paramTypes FieldSlice
// limitOffsetAndCount stores all the offset and key parameters extract from limit statement
// only used for cache and pick plan with parameters in limit
limitOffsetAndCount []uint64
}
// SetPstmtIDSchemaVersion implements PstmtCacheKeyMutator interface to change pstmtID and schemaVersion of cacheKey.
// so we can reuse Key instead of new every time.
func SetPstmtIDSchemaVersion(key kvcache.Key, stmtText string, schemaVersion int64, isolationReadEngines map[kv.StoreType]struct{}) {
@ -341,15 +349,14 @@ type PlanCacheValue struct {
Plan Plan
OutPutNames []*types.FieldName
TblInfo2UnionScan map[*model.TableInfo]bool
ParamTypes FieldSlice
memoryUsage int64
// limitOffsetAndCount stores all the offset and key parameters extract from limit statement
// only used for cache and pick plan with parameters
limitOffsetAndCount []uint64
// matchOpts stores some fields help to choose a suitable plan
matchOpts planCacheMatchOpts
}
func (v *PlanCacheValue) varTypesUnchanged(txtVarTps []*types.FieldType) bool {
return v.ParamTypes.CheckTypesCompatibility4PC(txtVarTps)
return v.matchOpts.paramTypes.CheckTypesCompatibility4PC(txtVarTps)
}
// unKnownMemoryUsage represent the memory usage of uncounted structure, maybe need implement later
@ -378,13 +385,13 @@ func (v *PlanCacheValue) MemoryUsage() (sum int64) {
sum = unKnownMemoryUsage
}
sum += size.SizeOfInterface + size.SizeOfSlice*2 + int64(cap(v.OutPutNames)+cap(v.ParamTypes))*size.SizeOfPointer +
sum += size.SizeOfInterface + size.SizeOfSlice*2 + int64(cap(v.OutPutNames)+cap(v.matchOpts.paramTypes))*size.SizeOfPointer +
size.SizeOfMap + int64(len(v.TblInfo2UnionScan))*(size.SizeOfPointer+size.SizeOfBool) + size.SizeOfInt64*2
for _, name := range v.OutPutNames {
sum += name.MemoryUsage()
}
for _, ft := range v.ParamTypes {
for _, ft := range v.matchOpts.paramTypes {
sum += ft.MemoryUsage()
}
v.memoryUsage = sum
@ -403,11 +410,13 @@ func NewPlanCacheValue(plan Plan, names []*types.FieldName, srcMap map[*model.Ta
userParamTypes[i] = tp.Clone()
}
return &PlanCacheValue{
Plan: plan,
OutPutNames: names,
TblInfo2UnionScan: dstMap,
ParamTypes: userParamTypes,
limitOffsetAndCount: limitParams,
Plan: plan,
OutPutNames: names,
TblInfo2UnionScan: dstMap,
matchOpts: planCacheMatchOpts{
paramTypes: userParamTypes,
limitOffsetAndCount: limitParams,
},
}
}