diff --git a/executor/prepared.go b/executor/prepared.go index 4fc7f8403258..2d4ff1789fab 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -316,7 +316,7 @@ func (e *DeallocateExec) Next(ctx context.Context, req *chunk.Chunk) error { return err } if !vars.IgnorePreparedCacheCloseStmt { // keep the plan in cache - e.ctx.PreparedPlanCache().Delete(cacheKey) + e.ctx.GetPlanCache(false).Delete(cacheKey) } } vars.RemovePreparedStmt(id) diff --git a/executor/seqtest/prepared_test.go b/executor/seqtest/prepared_test.go index d5364068aae3..f8176bbc9cc9 100644 --- a/executor/seqtest/prepared_test.go +++ b/executor/seqtest/prepared_test.go @@ -620,7 +620,7 @@ func TestPrepareDealloc(t *testing.T) { tk.MustExec("drop table if exists prepare_test") tk.MustExec("create table prepare_test (id int PRIMARY KEY, c1 int)") - require.Equal(t, 0, tk.Session().PreparedPlanCache().Size()) + require.Equal(t, 0, tk.Session().GetPlanCache(false).Size()) tk.MustExec(`prepare stmt1 from 'select id from prepare_test'`) tk.MustExec("execute stmt1") tk.MustExec(`prepare stmt2 from 'select c1 from prepare_test'`) @@ -629,20 +629,20 @@ func TestPrepareDealloc(t *testing.T) { tk.MustExec("execute stmt3") tk.MustExec(`prepare stmt4 from 'select * from prepare_test'`) tk.MustExec("execute stmt4") - require.Equal(t, 3, tk.Session().PreparedPlanCache().Size()) + require.Equal(t, 3, tk.Session().GetPlanCache(false).Size()) tk.MustExec("deallocate prepare stmt1") - require.Equal(t, 3, tk.Session().PreparedPlanCache().Size()) + require.Equal(t, 3, tk.Session().GetPlanCache(false).Size()) tk.MustExec("deallocate prepare stmt2") tk.MustExec("deallocate prepare stmt3") tk.MustExec("deallocate prepare stmt4") - require.Equal(t, 0, tk.Session().PreparedPlanCache().Size()) + require.Equal(t, 0, tk.Session().GetPlanCache(false).Size()) tk.MustExec(`prepare stmt1 from 'select * from prepare_test'`) tk.MustExec(`execute stmt1`) tk.MustExec(`prepare stmt2 from 'select * from prepare_test'`) tk.MustExec(`execute stmt2`) - require.Equal(t, 1, tk.Session().PreparedPlanCache().Size()) // use the same cached plan since they have the same statement + require.Equal(t, 1, tk.Session().GetPlanCache(false).Size()) // use the same cached plan since they have the same statement tk.MustExec(`drop database if exists plan_cache`) tk.MustExec(`create database plan_cache`) @@ -650,7 +650,7 @@ func TestPrepareDealloc(t *testing.T) { tk.MustExec(`create table prepare_test (id int PRIMARY KEY, c1 int)`) tk.MustExec(`prepare stmt3 from 'select * from prepare_test'`) tk.MustExec(`execute stmt3`) - require.Equal(t, 2, tk.Session().PreparedPlanCache().Size()) // stmt3 has different DB + require.Equal(t, 2, tk.Session().GetPlanCache(false).Size()) // stmt3 has different DB } func TestPreparedIssue8153(t *testing.T) { diff --git a/executor/simple.go b/executor/simple.go index df0fb6608acb..b75044df703b 100644 --- a/executor/simple.go +++ b/executor/simple.go @@ -1739,7 +1739,7 @@ func (e *SimpleExec) executeAdminFlushPlanCache(s *ast.AdminStmt) error { } now := types.NewTime(types.FromGoTime(time.Now().In(e.ctx.GetSessionVars().StmtCtx.TimeZone)), mysql.TypeTimestamp, 3) e.ctx.GetSessionVars().LastUpdateTime4PC = now - e.ctx.PreparedPlanCache().DeleteAll() + e.ctx.GetPlanCache(false).DeleteAll() if s.StatementScope == ast.StatementScopeInstance { // Record the timestamp. When other sessions want to use the plan cache, // it will check the timestamp first to decide whether the plan cache should be flushed. diff --git a/parser/ast/misc.go b/parser/ast/misc.go index 1c1e1804814c..4a0fc7201de7 100644 --- a/parser/ast/misc.go +++ b/parser/ast/misc.go @@ -508,6 +508,10 @@ type ExecuteStmt struct { BinaryArgs interface{} PrepStmt interface{} // the corresponding prepared statement IdxInMulti int + + // FromGeneralStmt indicates whether this execute-stmt is converted from a general query. + // e.g. select * from t where a>2 --> execute 'select * from t where a>?' using 2 + FromGeneralStmt bool } // Restore implements Node interface. diff --git a/planner/core/plan_cache.go b/planner/core/plan_cache.go index 6031fb3c8d4c..5a656e1a8ef3 100644 --- a/planner/core/plan_cache.go +++ b/planner/core/plan_cache.go @@ -42,7 +42,7 @@ import ( "go.uber.org/zap" ) -func planCachePreprocess(sctx sessionctx.Context, is infoschema.InfoSchema, +func planCachePreprocess(sctx sessionctx.Context, isGeneralPlanCache bool, is infoschema.InfoSchema, stmt *PlanCacheStmt, params []expression.Expression) error { vars := sctx.GetSessionVars() stmtAst := stmt.PreparedAst @@ -102,7 +102,7 @@ func planCachePreprocess(sctx sessionctx.Context, is infoschema.InfoSchema, // And update lastUpdateTime to the newest one. expiredTimeStamp4PC := domain.GetDomain(sctx).ExpiredTimeStamp4PC() if stmtAst.UseCache && expiredTimeStamp4PC.Compare(vars.LastUpdateTime4PC) > 0 { - sctx.PreparedPlanCache().DeleteAll() + sctx.GetPlanCache(isGeneralPlanCache).DeleteAll() stmtAst.CachedPlan = nil vars.LastUpdateTime4PC = expiredTimeStamp4PC } @@ -112,9 +112,11 @@ func planCachePreprocess(sctx sessionctx.Context, is infoschema.InfoSchema, // GetPlanFromSessionPlanCache is the entry point of Plan Cache. // It tries to get a valid cached plan from this session's plan cache. // If there is no such a plan, it'll call the optimizer to generate a new one. -func GetPlanFromSessionPlanCache(ctx context.Context, sctx sessionctx.Context, is infoschema.InfoSchema, stmt *PlanCacheStmt, +// isGeneralPlanCache indicates whether to use the general plan cache or the prepared plan cache. +func GetPlanFromSessionPlanCache(ctx context.Context, sctx sessionctx.Context, + isGeneralPlanCache bool, is infoschema.InfoSchema, stmt *PlanCacheStmt, params []expression.Expression) (plan Plan, names []*types.FieldName, err error) { - if err := planCachePreprocess(sctx, is, stmt, params); err != nil { + if err := planCachePreprocess(sctx, isGeneralPlanCache, is, stmt, params); err != nil { return nil, nil, err } @@ -154,13 +156,13 @@ func GetPlanFromSessionPlanCache(ctx context.Context, sctx sessionctx.Context, i } if stmtAst.UseCache && !ignorePlanCache { // for general plans - if plan, names, ok, err := getGeneralPlan(sctx, cacheKey, bindSQL, is, stmt, + if plan, names, ok, err := getGeneralPlan(sctx, isGeneralPlanCache, cacheKey, bindSQL, is, stmt, paramTypes); err != nil || ok { return plan, names, err } } - return generateNewPlan(ctx, sctx, is, stmt, ignorePlanCache, cacheKey, + return generateNewPlan(ctx, sctx, isGeneralPlanCache, is, stmt, ignorePlanCache, cacheKey, latestSchemaVersion, paramNum, paramTypes, bindSQL) } @@ -208,13 +210,13 @@ func getPointQueryPlan(stmt *ast.Prepared, sessVars *variable.SessionVars, stmtC return plan, names, true, nil } -func getGeneralPlan(sctx sessionctx.Context, cacheKey kvcache.Key, bindSQL string, +func getGeneralPlan(sctx sessionctx.Context, isGeneralPlanCache bool, cacheKey kvcache.Key, bindSQL string, is infoschema.InfoSchema, stmt *PlanCacheStmt, paramTypes []*types.FieldType) (Plan, []*types.FieldName, bool, error) { sessVars := sctx.GetSessionVars() stmtCtx := sessVars.StmtCtx - cachedVal, exist := getValidPlanFromCache(sctx, cacheKey, paramTypes) + cachedVal, exist := getValidPlanFromCache(sctx, isGeneralPlanCache, cacheKey, paramTypes) if !exist { return nil, nil, false, nil } @@ -225,7 +227,7 @@ func getGeneralPlan(sctx sessionctx.Context, cacheKey kvcache.Key, bindSQL strin if !unionScan && tableHasDirtyContent(sctx, tblInfo) { // TODO we can inject UnionScan into cached plan to avoid invalidating it, though // rebuilding the filters in UnionScan is pretty trivial. - sctx.PreparedPlanCache().Delete(cacheKey) + sctx.GetPlanCache(isGeneralPlanCache).Delete(cacheKey) return nil, nil, false, nil } } @@ -251,7 +253,7 @@ func getGeneralPlan(sctx sessionctx.Context, cacheKey kvcache.Key, bindSQL strin // 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, is infoschema.InfoSchema, stmt *PlanCacheStmt, +func generateNewPlan(ctx context.Context, sctx sessionctx.Context, isGeneralPlanCache bool, is infoschema.InfoSchema, stmt *PlanCacheStmt, ignorePlanCache bool, cacheKey kvcache.Key, latestSchemaVersion int64, paramNum int, paramTypes []*types.FieldType, bindSQL string) (Plan, []*types.FieldName, error) { stmtAst := stmt.PreparedAst @@ -286,7 +288,7 @@ func generateNewPlan(ctx context.Context, sctx sessionctx.Context, is infoschema stmt.NormalizedPlan, stmt.PlanDigest = NormalizePlan(p) stmtCtx.SetPlan(p) stmtCtx.SetPlanDigest(stmt.NormalizedPlan, stmt.PlanDigest) - putPlanIntoCache(sctx, cacheKey, cached) + putPlanIntoCache(sctx, isGeneralPlanCache, cacheKey, cached) } sessVars.FoundInPlanCache = false return p, names, err diff --git a/planner/core/plan_cache_utils.go b/planner/core/plan_cache_utils.go index 91d2d2635f97..f4b524507b86 100644 --- a/planner/core/plan_cache_utils.go +++ b/planner/core/plan_cache_utils.go @@ -39,8 +39,8 @@ var ( PreparedPlanCacheMaxMemory = *atomic2.NewUint64(math.MaxUint64) ) -func getValidPlanFromCache(sctx sessionctx.Context, key kvcache.Key, paramTypes []*types.FieldType) (*PlanCacheValue, bool) { - cache := sctx.PreparedPlanCache() +func getValidPlanFromCache(sctx sessionctx.Context, isGeneralPlanCache bool, key kvcache.Key, paramTypes []*types.FieldType) (*PlanCacheValue, bool) { + cache := sctx.GetPlanCache(isGeneralPlanCache) val, exist := cache.Get(key) if !exist { return nil, exist @@ -54,8 +54,8 @@ func getValidPlanFromCache(sctx sessionctx.Context, key kvcache.Key, paramTypes return nil, false } -func putPlanIntoCache(sctx sessionctx.Context, key kvcache.Key, plan *PlanCacheValue) { - cache := sctx.PreparedPlanCache() +func putPlanIntoCache(sctx sessionctx.Context, isGeneralPlanCache bool, key kvcache.Key, plan *PlanCacheValue) { + cache := sctx.GetPlanCache(isGeneralPlanCache) val, exist := cache.Get(key) if !exist { cache.Put(key, []*PlanCacheValue{plan}) diff --git a/planner/core/prepare_test.go b/planner/core/prepare_test.go index 0ec103835ee8..c9975aae026f 100644 --- a/planner/core/prepare_test.go +++ b/planner/core/prepare_test.go @@ -690,7 +690,7 @@ func TestPrepareCacheDeferredFunction(t *testing.T) { require.True(t, ok) err = executor.ResetContextOfStmt(tk.Session(), stmt) require.NoError(t, err) - plan, _, err := core.GetPlanFromSessionPlanCache(ctx, tk.Session(), is, execPlan.PrepStmt, execPlan.Params) + plan, _, err := core.GetPlanFromSessionPlanCache(ctx, tk.Session(), false, is, execPlan.PrepStmt, execPlan.Params) require.NoError(t, err) planStr[i] = core.ToString(plan) require.Regexpf(t, expectedPattern, planStr[i], "for %dth %s", i, sql1) diff --git a/planner/optimize.go b/planner/optimize.go index 5d809eb31091..e859d78fd5b6 100644 --- a/planner/optimize.go +++ b/planner/optimize.go @@ -374,7 +374,7 @@ func OptimizeExecStmt(ctx context.Context, sctx sessionctx.Context, if !ok { return nil, nil, errors.Errorf("invalid result plan type, should be Execute") } - plan, names, err := core.GetPlanFromSessionPlanCache(ctx, sctx, is, exec.PrepStmt, exec.Params) + plan, names, err := core.GetPlanFromSessionPlanCache(ctx, sctx, execAst.FromGeneralStmt, is, exec.PrepStmt, exec.Params) if err != nil { return nil, nil, err } diff --git a/server/driver_tidb.go b/server/driver_tidb.go index bfd76ff0552a..ff70903c0104 100644 --- a/server/driver_tidb.go +++ b/server/driver_tidb.go @@ -176,7 +176,7 @@ func (ts *TiDBStatement) Close() error { return err } if !ts.ctx.GetSessionVars().IgnorePreparedCacheCloseStmt { // keep the plan in cache - ts.ctx.PreparedPlanCache().Delete(cacheKey) + ts.ctx.GetPlanCache(false).Delete(cacheKey) } } ts.ctx.GetSessionVars().RemovePreparedStmt(ts.id) diff --git a/session/session.go b/session/session.go index f087108bc016..4f2fae4ce696 100644 --- a/session/session.go +++ b/session/session.go @@ -232,6 +232,7 @@ type session struct { store kv.Storage preparedPlanCache *kvcache.SimpleLRUCache + generalPlanCache *kvcache.SimpleLRUCache sessionVars *variable.SessionVars sessionManager util.SessionManager @@ -371,7 +372,7 @@ func (s *session) cleanRetryInfo() { plannercore.SetPstmtIDSchemaVersion(cacheKey, stmtText, preparedAst.SchemaVersion, s.sessionVars.IsolationReadEngines) } if !s.sessionVars.IgnorePreparedCacheCloseStmt { // keep the plan in cache - s.PreparedPlanCache().Delete(cacheKey) + s.GetPlanCache(false).Delete(cacheKey) } } s.sessionVars.RemovePreparedStmt(stmtID) @@ -430,7 +431,19 @@ func (s *session) SetCollation(coID int) error { return s.sessionVars.SetSystemVarWithoutValidation(variable.CollationConnection, co) } -func (s *session) PreparedPlanCache() *kvcache.SimpleLRUCache { +func (s *session) GetPlanCache(isGeneralPlanCache bool) *kvcache.SimpleLRUCache { + if isGeneralPlanCache { // use the general plan cache + if !s.GetSessionVars().EnableGeneralPlanCache { + return nil + } + if s.generalPlanCache == nil { // lazy construction + s.generalPlanCache = kvcache.NewSimpleLRUCache(uint(s.GetSessionVars().GeneralPlanCacheSize), + variable.PreparedPlanCacheMemoryGuardRatio.Load(), plannercore.PreparedPlanCacheMaxMemory.Load()) + } + return s.generalPlanCache + } + + // use the prepared plan cache if !s.GetSessionVars().EnablePreparedPlanCache { return nil } diff --git a/sessionctx/context.go b/sessionctx/context.go index d5f627920773..7ab28480eb86 100644 --- a/sessionctx/context.go +++ b/sessionctx/context.go @@ -106,8 +106,9 @@ type Context interface { // GetStore returns the store of session. GetStore() kv.Storage - // PreparedPlanCache returns the cache of the physical plan - PreparedPlanCache() *kvcache.SimpleLRUCache + // GetPlanCache returns the cache of the physical plan. + // generalPlanCache indicates to return the general plan cache or the prepared plan cache. + GetPlanCache(isGeneralPlanCache bool) *kvcache.SimpleLRUCache // StoreQueryFeedback stores the query feedback. StoreQueryFeedback(feedback interface{}) diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index a9d395bae52f..7b4d00dfadde 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -1866,8 +1866,7 @@ func (k planCacheStmtKey) Hash() []byte { // AddGeneralPlanCacheStmt adds this PlanCacheStmt into general-plan-cache-stmt cache func (s *SessionVars) AddGeneralPlanCacheStmt(sql string, stmt interface{}) { if s.generalPlanCacheStmts == nil { - // TODO: make it configurable - s.generalPlanCacheStmts = kvcache.NewSimpleLRUCache(100, 0, 0) + s.generalPlanCacheStmts = kvcache.NewSimpleLRUCache(uint(s.GeneralPlanCacheSize), 0, 0) } s.generalPlanCacheStmts.Put(planCacheStmtKey(sql), stmt) } diff --git a/sessionctx/variable/session_test.go b/sessionctx/variable/session_test.go index 7e714d1bc9ac..0776dd6083fa 100644 --- a/sessionctx/variable/session_test.go +++ b/sessionctx/variable/session_test.go @@ -387,6 +387,7 @@ func TestTransactionContextSavepoint(t *testing.T) { func TestGeneralPlanCacheStmt(t *testing.T) { sessVars := variable.NewSessionVars() + sessVars.GeneralPlanCacheSize = 100 sql1 := "select * from t where a>?" sql2 := "select * from t where a