diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 38ba8df4e37b9..2d73534fc2e1e 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -2017,7 +2017,7 @@ func getUintFromNode(ctx sessionctx.Context, n ast.Node, mustInt64orUint64 bool) return 0, false, true } if mustInt64orUint64 { - if expected := checkParamTypeInt64orUint64(v); !expected { + if expected, _ := CheckParamTypeInt64orUint64(v); !expected { return 0, false, false } } @@ -2054,19 +2054,19 @@ func getUintFromNode(ctx sessionctx.Context, n ast.Node, mustInt64orUint64 bool) return 0, false, false } -// check param type for plan cache limit, only allow int64 and uint64 now +// CheckParamTypeInt64orUint64 check param type for plan cache limit, only allow int64 and uint64 now // eg: set @a = 1; -func checkParamTypeInt64orUint64(param *driver.ParamMarkerExpr) bool { +func CheckParamTypeInt64orUint64(param *driver.ParamMarkerExpr) (bool, uint64) { val := param.GetValue() switch v := val.(type) { case int64: if v >= 0 { - return true + return true, uint64(v) } case uint64: - return true + return true, v } - return false + return false, 0 } func extractLimitCountOffset(ctx sessionctx.Context, limit *ast.Limit) (count uint64, diff --git a/planner/core/plan_cache.go b/planner/core/plan_cache.go index 677b9afe4e29d..70ae7d3481616 100644 --- a/planner/core/plan_cache.go +++ b/planner/core/plan_cache.go @@ -165,15 +165,18 @@ func GetPlanFromSessionPlanCache(ctx context.Context, sctx sessionctx.Context, return plan, names, err } } - + limitCountAndOffset, paramErr := ExtractLimitFromAst(stmt.PreparedAst.Stmt, sctx) + if paramErr != nil { + return nil, nil, paramErr + } if stmtCtx.UseCache { // for non-point plans if plan, names, ok, err := getCachedPlan(sctx, isNonPrepared, cacheKey, bindSQL, is, stmt, - paramTypes); err != nil || ok { + paramTypes, limitCountAndOffset); err != nil || ok { return plan, names, err } } - return generateNewPlan(ctx, sctx, isNonPrepared, is, stmt, cacheKey, latestSchemaVersion, paramNum, paramTypes, bindSQL) + return generateNewPlan(ctx, sctx, isNonPrepared, is, stmt, cacheKey, latestSchemaVersion, paramNum, paramTypes, bindSQL, limitCountAndOffset) } // parseParamTypes get parameters' types in PREPARE statement @@ -221,12 +224,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) (Plan, + is infoschema.InfoSchema, stmt *PlanCacheStmt, paramTypes []*types.FieldType, limitParams []uint64) (Plan, []*types.FieldName, bool, error) { sessVars := sctx.GetSessionVars() stmtCtx := sessVars.StmtCtx - candidate, exist := sctx.GetPlanCache(isNonPrepared).Get(cacheKey, paramTypes) + candidate, exist := sctx.GetPlanCache(isNonPrepared).Get(cacheKey, paramTypes, limitParams) if !exist { return nil, nil, false, nil } @@ -265,7 +268,7 @@ 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, paramNum int, - paramTypes []*types.FieldType, bindSQL string) (Plan, []*types.FieldName, error) { + paramTypes []*types.FieldType, bindSQL string, limitParams []uint64) (Plan, []*types.FieldName, error) { stmtAst := stmt.PreparedAst sessVars := sctx.GetSessionVars() stmtCtx := sessVars.StmtCtx @@ -296,11 +299,11 @@ func generateNewPlan(ctx context.Context, sctx sessionctx.Context, isNonPrepared } sessVars.IsolationReadEngines[kv.TiFlash] = struct{}{} } - cached := NewPlanCacheValue(p, names, stmtCtx.TblInfo2UnionScan, paramTypes) + cached := NewPlanCacheValue(p, names, stmtCtx.TblInfo2UnionScan, paramTypes, limitParams) stmt.NormalizedPlan, stmt.PlanDigest = NormalizePlan(p) stmtCtx.SetPlan(p) stmtCtx.SetPlanDigest(stmt.NormalizedPlan, stmt.PlanDigest) - sctx.GetPlanCache(isNonPrepared).Put(cacheKey, cached, paramTypes) + sctx.GetPlanCache(isNonPrepared).Put(cacheKey, cached, paramTypes, limitParams) } sessVars.FoundInPlanCache = false return p, names, err diff --git a/planner/core/plan_cache_lru.go b/planner/core/plan_cache_lru.go index 413dd37e8f5a2..20fa4c3f5c85c 100644 --- a/planner/core/plan_cache_lru.go +++ b/planner/core/plan_cache_lru.go @@ -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) (*list.Element, bool) + pickFromBucket func(map[*list.Element]struct{}, []*types.FieldType, []uint64) (*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) (*list.Element, bool), sctx sessionctx.Context) *LRUPlanCache { + pickFromBucket func(map[*list.Element]struct{}, []*types.FieldType, []uint64) (*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") @@ -94,13 +94,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) (value kvcache.Value, ok bool) { +func (l *LRUPlanCache) Get(key kvcache.Key, paramTypes []*types.FieldType, limitParams []uint64) (value kvcache.Value, ok bool) { l.lock.Lock() defer l.lock.Unlock() bucket, bucketExist := l.buckets[strHashKey(key, false)] if bucketExist { - if element, exist := l.pickFromBucket(bucket, paramTypes); exist { + if element, exist := l.pickFromBucket(bucket, paramTypes, limitParams); exist { l.lruList.MoveToFront(element) return element.Value.(*planCacheEntry).PlanValue, true } @@ -109,14 +109,14 @@ func (l *LRUPlanCache) Get(key kvcache.Key, paramTypes []*types.FieldType) (valu } // Put puts the (key, value) pair into the LRU Cache. -func (l *LRUPlanCache) Put(key kvcache.Key, value kvcache.Value, paramTypes []*types.FieldType) { +func (l *LRUPlanCache) Put(key kvcache.Key, value kvcache.Value, paramTypes []*types.FieldType, limitParams []uint64) { l.lock.Lock() defer l.lock.Unlock() hash := strHashKey(key, true) bucket, bucketExist := l.buckets[hash] if bucketExist { - if element, exist := l.pickFromBucket(bucket, paramTypes); exist { + if element, exist := l.pickFromBucket(bucket, paramTypes, limitParams); exist { l.updateInstanceMetric(&planCacheEntry{PlanKey: key, PlanValue: value}, element.Value.(*planCacheEntry)) element.Value.(*planCacheEntry).PlanValue = value l.lruList.MoveToFront(element) @@ -252,16 +252,36 @@ func (l *LRUPlanCache) memoryControl() { } // PickPlanFromBucket pick one plan from bucket -func PickPlanFromBucket(bucket map[*list.Element]struct{}, paramTypes []*types.FieldType) (*list.Element, bool) { +func PickPlanFromBucket(bucket map[*list.Element]struct{}, paramTypes []*types.FieldType, limitParams []uint64) (*list.Element, bool) { for k := range bucket { plan := k.Value.(*planCacheEntry).PlanValue.(*PlanCacheValue) - if plan.ParamTypes.CheckTypesCompatibility4PC(paramTypes) { + ok1 := plan.ParamTypes.CheckTypesCompatibility4PC(paramTypes) + if !ok1 { + continue + } + ok2 := checkUint64SliceIfEqual(plan.limitOffsetAndCount, limitParams) + if ok2 { return k, true } } return nil, false } +func checkUint64SliceIfEqual(a, b []uint64) bool { + if (a == nil && b != nil) || (a != nil && b == nil) { + return false + } + if len(a) != len(b) { + return false + } + for i := range a { + if a[i] != b[i] { + return false + } + } + return true +} + // updateInstanceMetric update the memory usage and plan num for show in grafana func (l *LRUPlanCache) updateInstanceMetric(in, out *planCacheEntry) { updateInstancePlanNum(in, out) diff --git a/planner/core/plan_cache_lru_test.go b/planner/core/plan_cache_lru_test.go index 74b6b2c92c3bb..f51480401ce62 100644 --- a/planner/core/plan_cache_lru_test.go +++ b/planner/core/plan_cache_lru_test.go @@ -65,14 +65,18 @@ func TestLRUPCPut(t *testing.T) { {types.NewFieldType(mysql.TypeFloat), types.NewFieldType(mysql.TypeLong)}, {types.NewFieldType(mysql.TypeFloat), types.NewFieldType(mysql.TypeInt24)}, } + limitParams := [][]uint64{ + {1}, {2}, {3}, {4}, {5}, + } // 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], + ParamTypes: pTypes[i], + limitOffsetAndCount: limitParams[i], } - lru.Put(keys[i], vals[i], pTypes[i]) + lru.Put(keys[i], vals[i], pTypes[i], limitParams[i]) } require.Equal(t, lru.size, lru.capacity) require.Equal(t, uint(3), lru.size) @@ -103,7 +107,7 @@ 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]) + element, exist := lru.pickFromBucket(bucket, pTypes[i], limitParams[i]) require.NotNil(t, element) require.True(t, exist) require.Equal(t, root, element) @@ -131,22 +135,25 @@ func TestLRUPCGet(t *testing.T) { {types.NewFieldType(mysql.TypeFloat), types.NewFieldType(mysql.TypeLong)}, {types.NewFieldType(mysql.TypeFloat), types.NewFieldType(mysql.TypeInt24)}, } + limitParams := [][]uint64{ + {1}, {2}, {3}, {4}, {5}, + } // 5 bucket for i := 0; i < 5; i++ { keys[i] = &planCacheKey{database: strconv.FormatInt(int64(i%4), 10)} - vals[i] = &PlanCacheValue{ParamTypes: pTypes[i]} - lru.Put(keys[i], vals[i], pTypes[i]) + vals[i] = &PlanCacheValue{ParamTypes: pTypes[i], limitOffsetAndCount: limitParams[i]} + lru.Put(keys[i], vals[i], pTypes[i], limitParams[i]) } // test for non-existent elements for i := 0; i < 2; i++ { - value, exists := lru.Get(keys[i], pTypes[i]) + value, exists := lru.Get(keys[i], pTypes[i], limitParams[i]) require.False(t, exists) require.Nil(t, value) } for i := 2; i < 5; i++ { - value, exists := lru.Get(keys[i], pTypes[i]) + value, exists := lru.Get(keys[i], pTypes[i], limitParams[i]) require.True(t, exists) require.NotNil(t, value) require.Equal(t, vals[i], value) @@ -175,23 +182,29 @@ func TestLRUPCDelete(t *testing.T) { {types.NewFieldType(mysql.TypeFloat), types.NewFieldType(mysql.TypeEnum)}, {types.NewFieldType(mysql.TypeFloat), types.NewFieldType(mysql.TypeDate)}, } + limitParams := [][]uint64{ + {1}, {2}, {3}, + } for i := 0; i < 3; i++ { keys[i] = &planCacheKey{database: strconv.FormatInt(int64(i), 10)} - vals[i] = &PlanCacheValue{ParamTypes: pTypes[i]} - lru.Put(keys[i], vals[i], pTypes[i]) + vals[i] = &PlanCacheValue{ + ParamTypes: pTypes[i], + limitOffsetAndCount: limitParams[i], + } + lru.Put(keys[i], vals[i], pTypes[i], []uint64{}) } require.Equal(t, 3, int(lru.size)) lru.Delete(keys[1]) - value, exists := lru.Get(keys[1], pTypes[1]) + value, exists := lru.Get(keys[1], pTypes[1], limitParams[1]) require.False(t, exists) require.Nil(t, value) require.Equal(t, 2, int(lru.size)) - _, exists = lru.Get(keys[0], pTypes[0]) + _, exists = lru.Get(keys[0], pTypes[0], limitParams[0]) require.True(t, exists) - _, exists = lru.Get(keys[2], pTypes[2]) + _, exists = lru.Get(keys[2], pTypes[2], limitParams[2]) require.True(t, exists) } @@ -207,14 +220,14 @@ 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]} - lru.Put(keys[i], vals[i], pTypes[i]) + lru.Put(keys[i], vals[i], pTypes[i], []uint64{}) } require.Equal(t, 3, int(lru.size)) lru.DeleteAll() for i := 0; i < 3; i++ { - value, exists := lru.Get(keys[i], pTypes[i]) + value, exists := lru.Get(keys[i], pTypes[i], []uint64{}) require.False(t, exists) require.Nil(t, value) require.Equal(t, 0, int(lru.size)) @@ -242,7 +255,7 @@ func TestLRUPCSetCapacity(t *testing.T) { for i := 0; i < 5; i++ { keys[i] = &planCacheKey{database: strconv.FormatInt(int64(1), 10)} vals[i] = &PlanCacheValue{ParamTypes: pTypes[i]} - lru.Put(keys[i], vals[i], pTypes[i]) + lru.Put(keys[i], vals[i], pTypes[i], []uint64{}) } require.Equal(t, lru.size, lru.capacity) require.Equal(t, uint(5), lru.size) @@ -292,7 +305,7 @@ func TestIssue37914(t *testing.T) { val := &PlanCacheValue{ParamTypes: pTypes} require.NotPanics(t, func() { - lru.Put(key, val, pTypes) + lru.Put(key, val, pTypes, []uint64{}) }) } @@ -313,7 +326,7 @@ func TestIssue38244(t *testing.T) { for i := 0; i < 5; i++ { keys[i] = &planCacheKey{database: strconv.FormatInt(int64(i), 10)} vals[i] = &PlanCacheValue{ParamTypes: pTypes[i]} - lru.Put(keys[i], vals[i], pTypes[i]) + lru.Put(keys[i], vals[i], pTypes[i], []uint64{}) } require.Equal(t, lru.size, lru.capacity) require.Equal(t, uint(3), lru.size) @@ -334,7 +347,7 @@ func TestLRUPlanCacheMemoryUsage(t *testing.T) { for i := 0; i < 3; i++ { k := randomPlanCacheKey() v := randomPlanCacheValue(pTypes) - lru.Put(k, v, pTypes) + lru.Put(k, v, pTypes, []uint64{}) res += k.MemoryUsage() + v.MemoryUsage() require.Equal(t, lru.MemoryUsage(), res) } @@ -342,7 +355,7 @@ func TestLRUPlanCacheMemoryUsage(t *testing.T) { p := &PhysicalTableScan{} k := &planCacheKey{database: "3"} v := &PlanCacheValue{Plan: p} - lru.Put(k, v, pTypes) + lru.Put(k, v, pTypes, []uint64{}) res += k.MemoryUsage() + v.MemoryUsage() for kk, vv := range evict { res -= kk.(*planCacheKey).MemoryUsage() + vv.(*PlanCacheValue).MemoryUsage() diff --git a/planner/core/plan_cache_test.go b/planner/core/plan_cache_test.go index fe76180291edf..8acc28b7b0062 100644 --- a/planner/core/plan_cache_test.go +++ b/planner/core/plan_cache_test.go @@ -385,12 +385,6 @@ func TestPlanCacheDiagInfo(t *testing.T) { tk.MustExec("prepare stmt from 'select /*+ ignore_plan_cache() */ * from t'") tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip plan-cache: ignore plan cache by hint")) - tk.MustExec("prepare stmt from 'select * from t limit ?'") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip plan-cache: query has 'limit ?' is un-cacheable")) - - tk.MustExec("prepare stmt from 'select * from t limit ?, 1'") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip plan-cache: query has 'limit ?, 10' is un-cacheable")) - tk.MustExec("prepare stmt from 'select * from t order by ?'") tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip plan-cache: query has 'order by ?' is un-cacheable")) @@ -463,18 +457,49 @@ func TestIssue40225(t *testing.T) { tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("1")) } -func TestUncacheableReason(t *testing.T) { +func TestPlanCacheWithLimit(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - tk.MustExec("create table t (a int)") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int primary key, b int)") + + testCases := []struct { + sql string + params []int + }{ + {"prepare stmt from 'select * from t limit ?'", []int{1}}, + {"prepare stmt from 'select * from t limit ?, ?'", []int{1, 2}}, + {"prepare stmt from 'delete from t order by a limit ?'", []int{1}}, + {"prepare stmt from 'insert into t select * from t order by a desc limit ?'", []int{1}}, + {"prepare stmt from 'insert into t select * from t order by a desc limit ?, ?'", []int{1, 2}}, + {"prepare stmt from 'update t set a = 1 limit ?'", []int{1}}, + {"prepare stmt from '(select * from t order by a limit ?) union (select * from t order by a desc limit ?)'", []int{1, 2}}, + {"prepare stmt from 'select * from t where a = ? limit ?, ?'", []int{1, 1, 1}}, + {"prepare stmt from 'select * from t where a in (?, ?) limit ?, ?'", []int{1, 2, 1, 1}}, + } + + for idx, testCase := range testCases { + tk.MustExec(testCase.sql) + var using []string + for i, p := range testCase.params { + tk.MustExec(fmt.Sprintf("set @a%d = %d", i, p)) + using = append(using, fmt.Sprintf("@a%d", i)) + } - tk.MustExec("prepare st from 'select * from t limit ?'") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip plan-cache: query has 'limit ?' is un-cacheable")) + tk.MustExec("execute stmt using " + strings.Join(using, ", ")) + tk.MustExec("execute stmt using " + strings.Join(using, ", ")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) - tk.MustExec("set @a=1") - tk.MustQuery("execute st using @a").Check(testkit.Rows()) - tk.MustExec("prepare st from 'select * from t limit ?'") - // show the corresponding un-cacheable reason at execute-stage as well - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip plan-cache: query has 'limit ?' is un-cacheable")) + if idx < 6 { + tk.MustExec("set @a0 = 6") + tk.MustExec("execute stmt using " + strings.Join(using, ", ")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) + } + } + + tk.MustExec("prepare stmt from 'select * from t limit ?'") + tk.MustExec("set @a = 10001") + tk.MustExec("execute stmt using @a") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip plan-cache: limit count more than 10000")) } diff --git a/planner/core/plan_cache_utils.go b/planner/core/plan_cache_utils.go index 3fe4ee38bfe45..082637506e590 100644 --- a/planner/core/plan_cache_utils.go +++ b/planner/core/plan_cache_utils.go @@ -343,6 +343,9 @@ type PlanCacheValue struct { 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 } func (v *PlanCacheValue) varTypesUnchanged(txtVarTps []*types.FieldType) bool { @@ -390,7 +393,7 @@ func (v *PlanCacheValue) MemoryUsage() (sum int64) { // NewPlanCacheValue creates a SQLCacheValue. func NewPlanCacheValue(plan Plan, names []*types.FieldName, srcMap map[*model.TableInfo]bool, - paramTypes []*types.FieldType) *PlanCacheValue { + paramTypes []*types.FieldType, limitParams []uint64) *PlanCacheValue { dstMap := make(map[*model.TableInfo]bool) for k, v := range srcMap { dstMap[k] = v @@ -400,10 +403,11 @@ 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, + Plan: plan, + OutPutNames: names, + TblInfo2UnionScan: dstMap, + ParamTypes: userParamTypes, + limitOffsetAndCount: limitParams, } } @@ -453,3 +457,69 @@ func GetPreparedStmt(stmt *ast.ExecuteStmt, vars *variable.SessionVars) (*PlanCa } return nil, ErrStmtNotFound } + +type limitExtractor struct { + cacheable bool // For safety considerations, check if limit count less than 10000 + offsetAndCount []uint64 + unCacheableReason string + paramTypeErr error +} + +// Enter implements Visitor interface. +func (checker *limitExtractor) Enter(in ast.Node) (out ast.Node, skipChildren bool) { + switch node := in.(type) { + case *ast.Limit: + if node.Count != nil { + if count, isParamMarker := node.Count.(*driver.ParamMarkerExpr); isParamMarker { + typeExpected, val := CheckParamTypeInt64orUint64(count) + if typeExpected { + if val > 10000 { + checker.cacheable = false + checker.unCacheableReason = "limit count more than 10000" + return in, true + } + checker.offsetAndCount = append(checker.offsetAndCount, val) + } else { + checker.paramTypeErr = ErrWrongArguments.GenWithStackByArgs("LIMIT") + return in, true + } + } + } + if node.Offset != nil { + if offset, isParamMarker := node.Offset.(*driver.ParamMarkerExpr); isParamMarker { + typeExpected, val := CheckParamTypeInt64orUint64(offset) + if typeExpected { + checker.offsetAndCount = append(checker.offsetAndCount, val) + } else { + checker.paramTypeErr = ErrWrongArguments.GenWithStackByArgs("LIMIT") + return in, true + } + } + } + } + return in, false +} + +// Leave implements Visitor interface. +func (checker *limitExtractor) Leave(in ast.Node) (out ast.Node, ok bool) { + return in, checker.cacheable +} + +// ExtractLimitFromAst extract limit offset and count from ast for plan cache key encode +func ExtractLimitFromAst(node ast.Node, sctx sessionctx.Context) ([]uint64, error) { + if node == nil { + return nil, nil + } + checker := limitExtractor{ + cacheable: true, + offsetAndCount: []uint64{}, + } + node.Accept(&checker) + if checker.paramTypeErr != nil { + return nil, checker.paramTypeErr + } + if sctx != nil && !checker.cacheable { + sctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.New("skip plan-cache: " + checker.unCacheableReason)) + } + return checker.offsetAndCount, nil +} diff --git a/planner/core/plan_cacheable_checker.go b/planner/core/plan_cacheable_checker.go index 0074cff434221..2ff9e51823ee2 100644 --- a/planner/core/plan_cacheable_checker.go +++ b/planner/core/plan_cacheable_checker.go @@ -135,21 +135,22 @@ func (checker *cacheableChecker) Enter(in ast.Node) (out ast.Node, skipChildren return in, true } } - case *ast.Limit: - if node.Count != nil { - if _, isParamMarker := node.Count.(*driver.ParamMarkerExpr); isParamMarker { - checker.cacheable = false - checker.reason = "query has 'limit ?' is un-cacheable" - return in, true - } - } - if node.Offset != nil { - if _, isParamMarker := node.Offset.(*driver.ParamMarkerExpr); isParamMarker { - checker.cacheable = false - checker.reason = "query has 'limit ?, 10' is un-cacheable" - return in, true - } - } + // todo: these comment is used to add switch in the later pr + //case *ast.Limit: + // if node.Count != nil { + // if _, isParamMarker := node.Count.(*driver.ParamMarkerExpr); isParamMarker { + // checker.cacheable = false + // checker.reason = "query has 'limit ?' is un-cacheable" + // return in, true + // } + // } + // if node.Offset != nil { + // if _, isParamMarker := node.Offset.(*driver.ParamMarkerExpr); isParamMarker { + // checker.cacheable = false + // checker.reason = "query has 'limit ?, 10' is un-cacheable" + // return in, true + // } + // } case *ast.FrameBound: if _, ok := node.Expr.(*driver.ParamMarkerExpr); ok { checker.cacheable = false diff --git a/planner/core/plan_cacheable_checker_test.go b/planner/core/plan_cacheable_checker_test.go index e87a08592eb16..7d417e377888f 100644 --- a/planner/core/plan_cacheable_checker_test.go +++ b/planner/core/plan_cacheable_checker_test.go @@ -87,7 +87,7 @@ func TestCacheable(t *testing.T) { TableRefs: tableRefsClause, Limit: limitStmt, } - require.False(t, core.Cacheable(stmt, is)) + require.True(t, core.Cacheable(stmt, is)) limitStmt = &ast.Limit{ Offset: &driver.ParamMarkerExpr{}, @@ -96,7 +96,7 @@ func TestCacheable(t *testing.T) { TableRefs: tableRefsClause, Limit: limitStmt, } - require.False(t, core.Cacheable(stmt, is)) + require.True(t, core.Cacheable(stmt, is)) limitStmt = &ast.Limit{} stmt = &ast.DeleteStmt{ @@ -139,7 +139,7 @@ func TestCacheable(t *testing.T) { TableRefs: tableRefsClause, Limit: limitStmt, } - require.False(t, core.Cacheable(stmt, is)) + require.True(t, core.Cacheable(stmt, is)) limitStmt = &ast.Limit{ Offset: &driver.ParamMarkerExpr{}, @@ -148,7 +148,7 @@ func TestCacheable(t *testing.T) { TableRefs: tableRefsClause, Limit: limitStmt, } - require.False(t, core.Cacheable(stmt, is)) + require.True(t, core.Cacheable(stmt, is)) limitStmt = &ast.Limit{} stmt = &ast.UpdateStmt{ @@ -188,7 +188,7 @@ func TestCacheable(t *testing.T) { stmt = &ast.SelectStmt{ Limit: limitStmt, } - require.False(t, core.Cacheable(stmt, is)) + require.True(t, core.Cacheable(stmt, is)) limitStmt = &ast.Limit{ Offset: &driver.ParamMarkerExpr{}, @@ -196,7 +196,7 @@ func TestCacheable(t *testing.T) { stmt = &ast.SelectStmt{ Limit: limitStmt, } - require.False(t, core.Cacheable(stmt, is)) + require.True(t, core.Cacheable(stmt, is)) limitStmt = &ast.Limit{} stmt = &ast.SelectStmt{ diff --git a/sessionctx/context.go b/sessionctx/context.go index 0e38fbdaba3d5..0999b2396cae0 100644 --- a/sessionctx/context.go +++ b/sessionctx/context.go @@ -54,8 +54,8 @@ type SessionStatesHandler interface { // PlanCache is an interface for prepare and non-prepared plan cache type PlanCache interface { - Get(key kvcache.Key, paramTypes []*types.FieldType) (value kvcache.Value, ok bool) - Put(key kvcache.Key, value kvcache.Value, paramTypes []*types.FieldType) + Get(key kvcache.Key, paramTypes []*types.FieldType, limitParams []uint64) (value kvcache.Value, ok bool) + Put(key kvcache.Key, value kvcache.Value, paramTypes []*types.FieldType, limitParams []uint64) Delete(key kvcache.Key) DeleteAll() Size() int