Skip to content

Commit

Permalink
planner: refactor plan cache LRU code (#41618)
Browse files Browse the repository at this point in the history
ref #40219
  • Loading branch information
fzzf678 authored Feb 22, 2023
1 parent 5b0315e commit 12107e3
Show file tree
Hide file tree
Showing 9 changed files with 210 additions and 126 deletions.
2 changes: 2 additions & 0 deletions planner/core/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -133,6 +133,7 @@ go_library(
"//util/mock",
"//util/paging",
"//util/parser",
"//util/plancache",
"//util/plancodec",
"//util/ranger",
"//util/rowcodec",
Expand Down Expand Up @@ -260,6 +261,7 @@ go_test(
"//util/kvcache",
"//util/logutil",
"//util/mock",
"//util/plancache",
"//util/plancodec",
"//util/ranger",
"//util/set",
Expand Down
29 changes: 14 additions & 15 deletions planner/core/plan_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@ import (
"github.com/pingcap/tidb/util/collate"
"github.com/pingcap/tidb/util/kvcache"
"github.com/pingcap/tidb/util/logutil"
utilpc "github.com/pingcap/tidb/util/plancache"
"github.com/pingcap/tidb/util/ranger"
"go.uber.org/zap"
)
Expand Down Expand Up @@ -158,25 +159,23 @@ func GetPlanFromSessionPlanCache(ctx context.Context, sctx sessionctx.Context,
}
}

paramTypes := parseParamTypes(sctx, params)

if stmtCtx.UseCache && stmtAst.CachedPlan != nil { // special code path for fast point plan
if plan, names, ok, err := getCachedPointPlan(stmtAst, sessVars, stmtCtx); ok {
return plan, names, err
}
}
limitCountAndOffset, paramErr := ExtractLimitFromAst(stmt.PreparedAst.Stmt, sctx)
if paramErr != nil {
return nil, nil, paramErr

matchOpts, err := GetMatchOpts(sctx, stmt.PreparedAst.Stmt, params)
if err != nil {
return nil, nil, err
}
if stmtCtx.UseCache { // for non-point plans
if plan, names, ok, err := getCachedPlan(sctx, isNonPrepared, cacheKey, bindSQL, is, stmt,
paramTypes, limitCountAndOffset); err != nil || ok {
if plan, names, ok, err := getCachedPlan(sctx, isNonPrepared, cacheKey, bindSQL, is, stmt, matchOpts); err != nil || ok {
return plan, names, err
}
}

return generateNewPlan(ctx, sctx, isNonPrepared, is, stmt, cacheKey, latestSchemaVersion, paramTypes, bindSQL, limitCountAndOffset)
return generateNewPlan(ctx, sctx, isNonPrepared, is, stmt, cacheKey, latestSchemaVersion, bindSQL, matchOpts)
}

// parseParamTypes get parameters' types in PREPARE statement
Expand Down Expand Up @@ -223,12 +222,12 @@ func getCachedPointPlan(stmt *ast.Prepared, sessVars *variable.SessionVars, stmt
}

func getCachedPlan(sctx sessionctx.Context, isNonPrepared bool, cacheKey kvcache.Key, bindSQL string,
is infoschema.InfoSchema, stmt *PlanCacheStmt, paramTypes []*types.FieldType, limitParams []uint64) (Plan,
is infoschema.InfoSchema, stmt *PlanCacheStmt, matchOpts *utilpc.PlanCacheMatchOpts) (Plan,
[]*types.FieldName, bool, error) {
sessVars := sctx.GetSessionVars()
stmtCtx := sessVars.StmtCtx

candidate, exist := sctx.GetPlanCache(isNonPrepared).Get(cacheKey, paramTypes, limitParams)
candidate, exist := sctx.GetPlanCache(isNonPrepared).Get(cacheKey, matchOpts)
if !exist {
return nil, nil, false, nil
}
Expand Down Expand Up @@ -267,8 +266,8 @@ func getCachedPlan(sctx sessionctx.Context, isNonPrepared bool, cacheKey kvcache
// 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 kvcache.Key, latestSchemaVersion int64, paramTypes []*types.FieldType,
bindSQL string, limitParams []uint64) (Plan, []*types.FieldName, error) {
stmt *PlanCacheStmt, cacheKey kvcache.Key, latestSchemaVersion int64, bindSQL string,
matchOpts *utilpc.PlanCacheMatchOpts) (Plan, []*types.FieldName, error) {
stmtAst := stmt.PreparedAst
sessVars := sctx.GetSessionVars()
stmtCtx := sessVars.StmtCtx
Expand All @@ -287,7 +286,7 @@ func generateNewPlan(ctx context.Context, sctx sessionctx.Context, isNonPrepared

// check whether this plan is cacheable.
if stmtCtx.UseCache {
if cacheable, reason := isPlanCacheable(sctx, p, len(paramTypes), len(limitParams)); !cacheable {
if cacheable, reason := isPlanCacheable(sctx, p, len(matchOpts.ParamTypes), len(matchOpts.LimitOffsetAndCount)); !cacheable {
stmtCtx.SetSkipPlanCache(errors.Errorf(reason))
}
}
Expand All @@ -303,11 +302,11 @@ func generateNewPlan(ctx context.Context, sctx sessionctx.Context, isNonPrepared
}
sessVars.IsolationReadEngines[kv.TiFlash] = struct{}{}
}
cached := NewPlanCacheValue(p, names, stmtCtx.TblInfo2UnionScan, paramTypes, limitParams)
cached := NewPlanCacheValue(p, names, stmtCtx.TblInfo2UnionScan, matchOpts)
stmt.NormalizedPlan, stmt.PlanDigest = NormalizePlan(p)
stmtCtx.SetPlan(p)
stmtCtx.SetPlanDigest(stmt.NormalizedPlan, stmt.PlanDigest)
sctx.GetPlanCache(isNonPrepared).Put(cacheKey, cached, paramTypes, limitParams)
sctx.GetPlanCache(isNonPrepared).Put(cacheKey, cached, matchOpts)
}
sessVars.FoundInPlanCache = false
return p, names, err
Expand Down
26 changes: 9 additions & 17 deletions planner/core/plan_cache_lru.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,11 +20,11 @@ import (
"github.com/pingcap/errors"
"github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/hack"
"github.com/pingcap/tidb/util/kvcache"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/memory"
utilpc "github.com/pingcap/tidb/util/plancache"
)

// planCacheEntry wraps Key and Value. It's the value of list.Element.
Expand Down Expand Up @@ -89,17 +89,13 @@ func strHashKey(key kvcache.Key, deepCopy bool) string {
}

// Get tries to find the corresponding value according to the given key.
func (l *LRUPlanCache) Get(key kvcache.Key, paramTypes []*types.FieldType, limitParams []uint64) (value kvcache.Value, ok bool) {
func (l *LRUPlanCache) Get(key kvcache.Key, opts *utilpc.PlanCacheMatchOpts) (value kvcache.Value, ok bool) {
l.lock.Lock()
defer l.lock.Unlock()

bucket, bucketExist := l.buckets[strHashKey(key, false)]
if bucketExist {
matchOpts := &planCacheMatchOpts{
paramTypes: paramTypes,
limitOffsetAndCount: limitParams,
}
if element, exist := l.pickFromBucket(bucket, matchOpts); exist {
if element, exist := l.pickFromBucket(bucket, opts); exist {
l.lruList.MoveToFront(element)
return element.Value.(*planCacheEntry).PlanValue, true
}
Expand All @@ -108,18 +104,14 @@ func (l *LRUPlanCache) Get(key kvcache.Key, paramTypes []*types.FieldType, limit
}

// Put puts the (key, value) pair into the LRU Cache.
func (l *LRUPlanCache) Put(key kvcache.Key, value kvcache.Value, paramTypes []*types.FieldType, limitParams []uint64) {
func (l *LRUPlanCache) Put(key kvcache.Key, value kvcache.Value, opts *utilpc.PlanCacheMatchOpts) {
l.lock.Lock()
defer l.lock.Unlock()

hash := strHashKey(key, true)
bucket, bucketExist := l.buckets[hash]
if bucketExist {
matchOpts := &planCacheMatchOpts{
paramTypes: paramTypes,
limitOffsetAndCount: limitParams,
}
if element, exist := l.pickFromBucket(bucket, matchOpts); exist {
if element, exist := l.pickFromBucket(bucket, opts); exist {
l.updateInstanceMetric(&planCacheEntry{PlanKey: key, PlanValue: value}, element.Value.(*planCacheEntry))
element.Value.(*planCacheEntry).PlanValue = value
l.lruList.MoveToFront(element)
Expand Down Expand Up @@ -255,21 +247,21 @@ func (l *LRUPlanCache) memoryControl() {
}

// PickPlanFromBucket pick one plan from bucket
func (l *LRUPlanCache) pickFromBucket(bucket map[*list.Element]struct{}, matchOpts *planCacheMatchOpts) (*list.Element, bool) {
func (l *LRUPlanCache) pickFromBucket(bucket map[*list.Element]struct{}, matchOpts *utilpc.PlanCacheMatchOpts) (*list.Element, bool) {
for k := range bucket {
plan := k.Value.(*planCacheEntry).PlanValue.(*PlanCacheValue)
// check param types' compatibility
ok1 := plan.matchOpts.paramTypes.CheckTypesCompatibility4PC(matchOpts.paramTypes)
ok1 := checkTypesCompatibility4PC(plan.matchOpts.ParamTypes, matchOpts.ParamTypes)
if !ok1 {
continue
}

// check limit offset and key if equal and check switch if enabled
ok2 := checkUint64SliceIfEqual(plan.matchOpts.limitOffsetAndCount, matchOpts.limitOffsetAndCount)
ok2 := checkUint64SliceIfEqual(plan.matchOpts.LimitOffsetAndCount, matchOpts.LimitOffsetAndCount)
if !ok2 {
continue
}
if len(plan.matchOpts.limitOffsetAndCount) > 0 && !l.sctx.GetSessionVars().EnablePlanCacheForParamLimit {
if len(plan.matchOpts.LimitOffsetAndCount) > 0 && !l.sctx.GetSessionVars().EnablePlanCacheForParamLimit {
// offset and key slice matched, but it is a plan with param limit and the switch is disabled
continue
}
Expand Down
Loading

0 comments on commit 12107e3

Please sign in to comment.