Skip to content

Commit

Permalink
fix
Browse files Browse the repository at this point in the history
  • Loading branch information
fzzf678 committed Jan 16, 2023
1 parent 64c82da commit b877d21
Show file tree
Hide file tree
Showing 4 changed files with 43 additions and 26 deletions.
21 changes: 8 additions & 13 deletions planner/core/plan_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -165,19 +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
limitCountAndOffset, paramErr := ExtractLimitFromAst(stmt.PreparedAst.Stmt, sctx)
if paramErr != nil {
return nil, nil, paramErr
}
if plan, names, ok, err := getCachedPlan(sctx, isNonPrepared, cacheKey, bindSQL, is, stmt,
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
Expand Down Expand Up @@ -269,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
Expand Down Expand Up @@ -300,15 +299,11 @@ func generateNewPlan(ctx context.Context, sctx sessionctx.Context, isNonPrepared
}
sessVars.IsolationReadEngines[kv.TiFlash] = struct{}{}
}
limitCountAndOffset, paramErr := ExtractLimitFromAst(stmt.PreparedAst.Stmt, nil)
if paramErr != nil {
return nil, nil, paramErr
}
cached := NewPlanCacheValue(p, names, stmtCtx.TblInfo2UnionScan, paramTypes, limitCountAndOffset)
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, limitCountAndOffset)
sctx.GetPlanCache(isNonPrepared).Put(cacheKey, cached, paramTypes, limitParams)
}
sessVars.FoundInPlanCache = false
return p, names, err
Expand Down
16 changes: 10 additions & 6 deletions planner/core/plan_cache_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -462,7 +462,7 @@ func TestPlanCacheWithLimit(t *testing.T) {
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a int, b int, key(a))")
tk.MustExec("create table t(a int primary key, b int)")

testCases := []struct {
sql string
Expand All @@ -474,10 +474,12 @@ func TestPlanCacheWithLimit(t *testing.T) {
{"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 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 _, testCase := range testCases {
for idx, testCase := range testCases {
tk.MustExec(testCase.sql)
var using []string
for i, p := range testCase.params {
Expand All @@ -489,9 +491,11 @@ func TestPlanCacheWithLimit(t *testing.T) {
tk.MustExec("execute stmt using " + strings.Join(using, ", "))
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1"))

tk.MustExec("set @a0 = 6")
tk.MustExec("execute stmt using " + strings.Join(using, ", "))
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0"))
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 ?'")
Expand Down
16 changes: 9 additions & 7 deletions planner/core/plan_cache_utils.go
Original file line number Diff line number Diff line change
Expand Up @@ -338,11 +338,13 @@ func (s FieldSlice) CheckTypesCompatibility4PC(tps []*types.FieldType) bool {

// PlanCacheValue stores the cached Statement and StmtNode.
type PlanCacheValue struct {
Plan Plan
OutPutNames []*types.FieldName
TblInfo2UnionScan map[*model.TableInfo]bool
ParamTypes FieldSlice
memoryUsage int64
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
}

Expand Down Expand Up @@ -506,15 +508,15 @@ func (checker *limitExtractor) Leave(in ast.Node) (out ast.Node, ok bool) {
// 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 []uint64{}, nil
return nil, nil
}
checker := limitExtractor{
cacheable: true,
offsetAndCount: []uint64{},
}
node.Accept(&checker)
if checker.paramTypeErr != nil {
return []uint64{}, checker.paramTypeErr
return nil, checker.paramTypeErr
}
if sctx != nil && !checker.cacheable {
sctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.New("skip plan-cache: " + checker.unCacheableReason))
Expand Down
16 changes: 16 additions & 0 deletions planner/core/plan_cacheable_checker.go
Original file line number Diff line number Diff line change
Expand Up @@ -135,6 +135,22 @@ func (checker *cacheableChecker) Enter(in ast.Node) (out ast.Node, skipChildren
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
Expand Down

0 comments on commit b877d21

Please sign in to comment.