planner: refactor plan cache value code (#40656)
ref pingcap/tidb#40219
This commit is contained in:
@ -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
|
||||
}
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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,
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user