diff --git a/executor/prepared.go b/executor/prepared.go index ae742a30d8bb5..a9dd9452e3c99 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -53,8 +53,6 @@ type PrepareExec struct { Fields []*ast.ResultField Stmt interface{} - IsGeneralStmt bool - // If it's generated from executing "prepare stmt from '...'", the process is parse -> plan -> executor // If it's generated from the prepare protocol, the process is session.PrepareStmt -> NewPrepareExec // They both generate a PrepareExec struct, but the second case needs to reset the statement context while the first already do that. @@ -135,19 +133,15 @@ func (e *PrepareExec) Next(ctx context.Context, req *chunk.Chunk) error { if !isNoResultPlan(p) { e.Fields = colNames2ResultFields(p.Schema(), p.OutputNames(), vars.CurrentDB) } - if e.ID == 0 && !e.IsGeneralStmt { + if e.ID == 0 { e.ID = vars.GetNextPreparedStmtID() } - if e.name != "" && !e.IsGeneralStmt { + if e.name != "" { vars.PreparedStmtNameToID[e.name] = e.ID } e.ParamCount = paramCnt e.Stmt = stmt - if e.IsGeneralStmt { - vars.AddGeneralPlanCacheStmt(e.sqlText, stmt) - return nil - } return vars.AddPreparedStmt(e.ID, stmt) } diff --git a/expression/function_traits.go b/expression/function_traits.go index aba61d9b2d92a..4d6fa98da6a99 100644 --- a/expression/function_traits.go +++ b/expression/function_traits.go @@ -19,8 +19,8 @@ import ( "github.com/pingcap/tidb/parser/opcode" ) -// GeneralPlanCacheableOp stores function which can be cached to general plan cache. -var GeneralPlanCacheableOp = map[string]struct{}{ +// NonPreparedPlanCacheableOp stores function which can be cached to non-prepared plan cache. +var NonPreparedPlanCacheableOp = map[string]struct{}{ ast.LogicAnd: {}, ast.LogicOr: {}, ast.GE: {}, diff --git a/planner/core/plan_cache.go b/planner/core/plan_cache.go index e58e8b6d91708..d9df658aaa46e 100644 --- a/planner/core/plan_cache.go +++ b/planner/core/plan_cache.go @@ -42,7 +42,7 @@ import ( "go.uber.org/zap" ) -func planCachePreprocess(ctx context.Context, sctx sessionctx.Context, isGeneralPlanCache bool, is infoschema.InfoSchema, stmt *PlanCacheStmt, params []expression.Expression) error { +func planCachePreprocess(ctx context.Context, sctx sessionctx.Context, isNonPrepared bool, is infoschema.InfoSchema, stmt *PlanCacheStmt, params []expression.Expression) error { vars := sctx.GetSessionVars() stmtAst := stmt.PreparedAst vars.StmtCtx.StmtType = stmtAst.StmtType @@ -101,7 +101,7 @@ func planCachePreprocess(ctx context.Context, sctx sessionctx.Context, isGeneral // And update lastUpdateTime to the newest one. expiredTimeStamp4PC := domain.GetDomain(sctx).ExpiredTimeStamp4PC() if stmtAst.UseCache && expiredTimeStamp4PC.Compare(vars.LastUpdateTime4PC) > 0 { - sctx.GetPlanCache(isGeneralPlanCache).DeleteAll() + sctx.GetPlanCache(isNonPrepared).DeleteAll() stmtAst.CachedPlan = nil vars.LastUpdateTime4PC = expiredTimeStamp4PC } @@ -111,11 +111,11 @@ func planCachePreprocess(ctx context.Context, sctx sessionctx.Context, isGeneral // 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. -// isGeneralPlanCache indicates whether to use the general plan cache or the prepared plan cache. +// isNonPrepared indicates whether to use the non-prepared plan cache or the prepared plan cache. func GetPlanFromSessionPlanCache(ctx context.Context, sctx sessionctx.Context, - isGeneralPlanCache bool, is infoschema.InfoSchema, stmt *PlanCacheStmt, + isNonPrepared bool, is infoschema.InfoSchema, stmt *PlanCacheStmt, params []expression.Expression) (plan Plan, names []*types.FieldName, err error) { - if err := planCachePreprocess(ctx, sctx, isGeneralPlanCache, is, stmt, params); err != nil { + if err := planCachePreprocess(ctx, sctx, isNonPrepared, is, stmt, params); err != nil { return nil, nil, err } @@ -149,19 +149,19 @@ func GetPlanFromSessionPlanCache(ctx context.Context, sctx sessionctx.Context, paramNum, paramTypes := parseParamTypes(sctx, params) if stmtAst.UseCache && stmtAst.CachedPlan != nil && !ignorePlanCache { // for point query plan - if plan, names, ok, err := getPointQueryPlan(stmtAst, sessVars, stmtCtx); ok { + if plan, names, ok, err := getCachedPointPlan(stmtAst, sessVars, stmtCtx); ok { return plan, names, err } } - if stmtAst.UseCache && !ignorePlanCache { // for general plans - if plan, names, ok, err := getGeneralPlan(sctx, isGeneralPlanCache, cacheKey, bindSQL, is, stmt, + if stmtAst.UseCache && !ignorePlanCache { // for non-point plans + if plan, names, ok, err := getCachedPlan(sctx, isNonPrepared, cacheKey, bindSQL, is, stmt, paramTypes); err != nil || ok { return plan, names, err } } - return generateNewPlan(ctx, sctx, isGeneralPlanCache, is, stmt, ignorePlanCache, cacheKey, + return generateNewPlan(ctx, sctx, isNonPrepared, is, stmt, ignorePlanCache, cacheKey, latestSchemaVersion, paramNum, paramTypes, bindSQL) } @@ -185,7 +185,7 @@ func parseParamTypes(sctx sessionctx.Context, params []expression.Expression) (p return } -func getPointQueryPlan(stmt *ast.Prepared, sessVars *variable.SessionVars, stmtCtx *stmtctx.StatementContext) (Plan, +func getCachedPointPlan(stmt *ast.Prepared, sessVars *variable.SessionVars, stmtCtx *stmtctx.StatementContext) (Plan, []*types.FieldName, bool, error) { // short path for point-get plans // Rewriting the expression in the select.where condition will convert its @@ -209,13 +209,13 @@ func getPointQueryPlan(stmt *ast.Prepared, sessVars *variable.SessionVars, stmtC return plan, names, true, nil } -func getGeneralPlan(sctx sessionctx.Context, isGeneralPlanCache bool, cacheKey kvcache.Key, bindSQL string, +func getCachedPlan(sctx sessionctx.Context, isNonPrepared 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 - candidate, exist := sctx.GetPlanCache(isGeneralPlanCache).Get(cacheKey, paramTypes) + candidate, exist := sctx.GetPlanCache(isNonPrepared).Get(cacheKey, paramTypes) if !exist { return nil, nil, false, nil } @@ -227,7 +227,7 @@ func getGeneralPlan(sctx sessionctx.Context, isGeneralPlanCache bool, cacheKey k 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.GetPlanCache(isGeneralPlanCache).Delete(cacheKey) + sctx.GetPlanCache(isNonPrepared).Delete(cacheKey) return nil, nil, false, nil } } @@ -253,7 +253,7 @@ func getGeneralPlan(sctx sessionctx.Context, isGeneralPlanCache bool, cacheKey k // 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, isGeneralPlanCache bool, is infoschema.InfoSchema, stmt *PlanCacheStmt, +func generateNewPlan(ctx context.Context, sctx sessionctx.Context, isNonPrepared 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 @@ -290,7 +290,7 @@ func generateNewPlan(ctx context.Context, sctx sessionctx.Context, isGeneralPlan stmt.NormalizedPlan, stmt.PlanDigest = NormalizePlan(p) stmtCtx.SetPlan(p) stmtCtx.SetPlanDigest(stmt.NormalizedPlan, stmt.PlanDigest) - sctx.GetPlanCache(isGeneralPlanCache).Put(cacheKey, cached, paramTypes) + sctx.GetPlanCache(isNonPrepared).Put(cacheKey, cached, paramTypes) } sessVars.FoundInPlanCache = false return p, names, err diff --git a/planner/core/plan_cache_param.go b/planner/core/plan_cache_param.go index a1e4b5a3f6703..7c79b2a6416a0 100644 --- a/planner/core/plan_cache_param.go +++ b/planner/core/plan_cache_param.go @@ -54,7 +54,7 @@ func (pr *paramReplacer) Enter(in ast.Node) (out ast.Node, skipChildren bool) { switch n := in.(type) { case *driver.ValueExpr: pr.params = append(pr.params, n) - // offset is used as order in general plan cache. + // offset is used as order in non-prepared plan cache. param := ast.NewParamMarkerExpr(len(pr.params) - 1) return param, true } @@ -100,7 +100,7 @@ func (pr *paramRestorer) Enter(in ast.Node) (out ast.Node, skipChildren bool) { pr.err = errors.New("failed to restore ast.Node") return nil, true } - // offset is used as order in general plan cache. + // offset is used as order in non-prepared plan cache. return pr.params[n.Offset], true } if pr.err != nil { diff --git a/planner/core/plan_cache_test.go b/planner/core/plan_cache_test.go index 868f35229ce6e..26647de63ab2a 100644 --- a/planner/core/plan_cache_test.go +++ b/planner/core/plan_cache_test.go @@ -81,7 +81,7 @@ func TestInitLRUWithSystemVar(t *testing.T) { require.NotNil(t, lru) } -func TestGeneralPlanCacheBasically(t *testing.T) { +func TestNonPreparedPlanCacheBasically(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec(`use test`) diff --git a/planner/core/plan_cache_utils.go b/planner/core/plan_cache_utils.go index 4229e2b134f06..2b1621857b9ca 100644 --- a/planner/core/plan_cache_utils.go +++ b/planner/core/plan_cache_utils.go @@ -161,21 +161,6 @@ func GeneratePlanCacheStmtWithAST(ctx context.Context, sctx sessionctx.Context, return preparedObj, p, ParamCount, nil } -func getValidPlanFromCache(sctx sessionctx.Context, isGeneralPlanCache bool, key kvcache.Key, paramTypes []*types.FieldType) (*PlanCacheValue, bool) { - cache := sctx.GetPlanCache(isGeneralPlanCache) - val, exist := cache.Get(key, paramTypes) - if !exist { - return nil, exist - } - candidate := val.(*PlanCacheValue) - return candidate, true -} - -func putPlanIntoCache(sctx sessionctx.Context, isGeneralPlanCache bool, key kvcache.Key, plan *PlanCacheValue, paramTypes []*types.FieldType) { - cache := sctx.GetPlanCache(isGeneralPlanCache) - cache.Put(key, plan, paramTypes) -} - // planCacheKey is used to access Plan Cache. We put some variables that do not affect the plan into planCacheKey, such as the sql text. // Put the parameters that may affect the plan in planCacheValue. // However, due to some compatibility reasons, we will temporarily keep some system variable-related values in planCacheKey. diff --git a/planner/core/plan_cacheable_checker.go b/planner/core/plan_cacheable_checker.go index 55b1e45cd0e99..3412ace379271 100644 --- a/planner/core/plan_cacheable_checker.go +++ b/planner/core/plan_cacheable_checker.go @@ -175,16 +175,16 @@ func (checker *cacheableChecker) Leave(in ast.Node) (out ast.Node, ok bool) { return in, checker.cacheable } -// GeneralPlanCacheable checks whether the input ast is cacheable for general plan cache with empty session context, which is mainly for testing. -func GeneralPlanCacheable(node ast.Node, is infoschema.InfoSchema) bool { - return GeneralPlanCacheableWithCtx(nil, node, is) +// NonPreparedPlanCacheable checks whether the input ast is cacheable for non-prepared plan cache with empty session context, which is mainly for testing. +func NonPreparedPlanCacheable(node ast.Node, is infoschema.InfoSchema) bool { + return NonPreparedPlanCacheableWithCtx(nil, node, is) } -// GeneralPlanCacheableWithCtx checks whether the input ast is cacheable for general plan cache. +// NonPreparedPlanCacheableWithCtx checks whether the input ast is cacheable for non-prepared plan cache. // Only support: select {field} from {single-table} where {cond} and {cond} ... // {cond}: {col} {op} {val} // {op}: >, <, = -func GeneralPlanCacheableWithCtx(sctx sessionctx.Context, node ast.Node, is infoschema.InfoSchema) bool { +func NonPreparedPlanCacheableWithCtx(sctx sessionctx.Context, node ast.Node, is infoschema.InfoSchema) bool { selectStmt, isSelect := node.(*ast.SelectStmt) if !isSelect { // only support select statement now return false @@ -206,7 +206,7 @@ func GeneralPlanCacheableWithCtx(sctx sessionctx.Context, node ast.Node, is info } tableRefs := from.TableRefs if tableRefs.Right != nil { - // We don't support the join for the general plan cache now. + // We don't support the join for the non-prepared plan cache now. return false } switch x := tableRefs.Left.(type) { @@ -217,7 +217,7 @@ func GeneralPlanCacheableWithCtx(sctx sessionctx.Context, node ast.Node, is info } } - checker := generalPlanCacheableChecker{ + checker := nonPreparedPlanCacheableChecker{ sctx: sctx, cacheable: true, schema: is, @@ -226,19 +226,19 @@ func GeneralPlanCacheableWithCtx(sctx sessionctx.Context, node ast.Node, is info return checker.cacheable } -// generalPlanCacheableChecker checks whether a query's plan can be cached for general plan cache. +// nonPreparedPlanCacheableChecker checks whether a query's plan can be cached for non-prepared plan cache. // NOTE: we can add more rules in the future. -type generalPlanCacheableChecker struct { +type nonPreparedPlanCacheableChecker struct { sctx sessionctx.Context cacheable bool schema infoschema.InfoSchema } // Enter implements Visitor interface. -func (checker *generalPlanCacheableChecker) Enter(in ast.Node) (out ast.Node, skipChildren bool) { +func (checker *nonPreparedPlanCacheableChecker) Enter(in ast.Node) (out ast.Node, skipChildren bool) { switch node := in.(type) { case *ast.BinaryOperationExpr: - if _, found := expression.GeneralPlanCacheableOp[node.Op.String()]; !found { + if _, found := expression.NonPreparedPlanCacheableOp[node.Op.String()]; !found { checker.cacheable = false return in, true } @@ -265,7 +265,7 @@ func (checker *generalPlanCacheableChecker) Enter(in ast.Node) (out ast.Node, sk } // Leave implements Visitor interface. -func (checker *generalPlanCacheableChecker) Leave(in ast.Node) (out ast.Node, ok bool) { +func (checker *nonPreparedPlanCacheableChecker) Leave(in ast.Node) (out ast.Node, ok bool) { return in, checker.cacheable } diff --git a/planner/core/plan_cacheable_checker_test.go b/planner/core/plan_cacheable_checker_test.go index a32294e34c54c..658537bde4188 100644 --- a/planner/core/plan_cacheable_checker_test.go +++ b/planner/core/plan_cacheable_checker_test.go @@ -247,7 +247,7 @@ func TestCacheable(t *testing.T) { require.True(t, core.Cacheable(stmt, is)) } -func TestGeneralPlanCacheable(t *testing.T) { +func TestNonPreparedPlanCacheable(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -297,12 +297,12 @@ func TestGeneralPlanCacheable(t *testing.T) { for _, q := range unsupported { stmt, err := p.ParseOneStmt(q, charset, collation) require.NoError(t, err) - require.False(t, core.GeneralPlanCacheable(stmt, is)) + require.False(t, core.NonPreparedPlanCacheable(stmt, is)) } for _, q := range supported { stmt, err := p.ParseOneStmt(q, charset, collation) require.NoError(t, err) - require.True(t, core.GeneralPlanCacheable(stmt, is)) + require.True(t, core.NonPreparedPlanCacheable(stmt, is)) } } diff --git a/planner/optimize.go b/planner/optimize.go index d5ee997057180..3a6804d5fa319 100644 --- a/planner/optimize.go +++ b/planner/optimize.go @@ -73,23 +73,23 @@ func matchSQLBinding(sctx sessionctx.Context, stmtNode ast.StmtNode) (bindRecord return bindRecord, scope, true } -// getPlanFromGeneralPlanCache tries to get an available cached plan from the General Plan Cache for this stmt. -func getPlanFromGeneralPlanCache(ctx context.Context, sctx sessionctx.Context, stmt ast.StmtNode, is infoschema.InfoSchema) (core.Plan, types.NameSlice, bool, error) { +// getPlanFromNonPreparedPlanCache tries to get an available cached plan from the NonPrepared Plan Cache for this stmt. +func getPlanFromNonPreparedPlanCache(ctx context.Context, sctx sessionctx.Context, stmt ast.StmtNode, is infoschema.InfoSchema) (core.Plan, types.NameSlice, bool, error) { if sctx.GetSessionVars().StmtCtx.InPreparedPlanBuilding || // already in cached plan rebuilding phase - !core.GeneralPlanCacheableWithCtx(sctx, stmt, is) { + !core.NonPreparedPlanCacheableWithCtx(sctx, stmt, is) { return nil, nil, false, nil } paramSQL, params, err := core.ParameterizeAST(sctx, stmt) if err != nil { return nil, nil, false, err } - val := sctx.GetSessionVars().GetGeneralPlanCacheStmt(paramSQL) + val := sctx.GetSessionVars().GetNonPreparedPlanCacheStmt(paramSQL) if val == nil { cachedStmt, _, _, err := core.GeneratePlanCacheStmtWithAST(ctx, sctx, stmt) if err != nil { return nil, nil, false, err } - sctx.GetSessionVars().AddGeneralPlanCacheStmt(paramSQL, cachedStmt) + sctx.GetSessionVars().AddNonPreparedPlanCacheStmt(paramSQL, cachedStmt) val = cachedStmt } cachedStmt := val.(*core.PlanCacheStmt) @@ -176,11 +176,11 @@ func Optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is in node = stmtNode } - // try to get Plan from the General Plan Cache - if sctx.GetSessionVars().EnableGeneralPlanCache && + // try to get Plan from the NonPrepared Plan Cache + if sctx.GetSessionVars().EnableNonPreparedPlanCache && isStmtNode && !useBinding { // TODO: support binding - cachedPlan, names, ok, err := getPlanFromGeneralPlanCache(ctx, sctx, stmtNode, is) + cachedPlan, names, ok, err := getPlanFromNonPreparedPlanCache(ctx, sctx, stmtNode, is) if err != nil { return nil, nil, err } @@ -436,7 +436,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, execAst.FromGeneralStmt, is, exec.PrepStmt, exec.Params) + plan, names, err := core.GetPlanFromSessionPlanCache(ctx, sctx, false, is, exec.PrepStmt, exec.Params) if err != nil { return nil, nil, err } diff --git a/session/session.go b/session/session.go index df01e592f7506..40b3921a4145c 100644 --- a/session/session.go +++ b/session/session.go @@ -175,8 +175,6 @@ type Session interface { RollbackTxn(context.Context) // PrepareStmt executes prepare statement in binary protocol. PrepareStmt(sql string) (stmtID uint32, paramCount int, fields []*ast.ResultField, err error) - // CacheGeneralStmt parses the sql, generates the corresponding PlanCacheStmt and cache it. - CacheGeneralStmt(sql string) (interface{}, error) // ExecutePreparedStmt executes a prepared statement. // Deprecated: please use ExecuteStmt, this function is left for testing only. // TODO: remove ExecutePreparedStmt. @@ -254,8 +252,8 @@ type session struct { store kv.Storage - preparedPlanCache sessionctx.PlanCache - generalPlanCache sessionctx.PlanCache + preparedPlanCache sessionctx.PlanCache + nonPreparedPlanCache sessionctx.PlanCache sessionVars *variable.SessionVars sessionManager util.SessionManager @@ -458,17 +456,17 @@ func (s *session) SetCollation(coID int) error { return s.sessionVars.SetSystemVarWithoutValidation(variable.CollationConnection, co) } -func (s *session) GetPlanCache(isGeneralPlanCache bool) sessionctx.PlanCache { - if isGeneralPlanCache { // use the general plan cache - if !s.GetSessionVars().EnableGeneralPlanCache { +func (s *session) GetPlanCache(isNonPrepared bool) sessionctx.PlanCache { + if isNonPrepared { // use the non-prepared plan cache + if !s.GetSessionVars().EnableNonPreparedPlanCache { return nil } - if s.generalPlanCache == nil { // lazy construction - s.generalPlanCache = plannercore.NewLRUPlanCache(uint(s.GetSessionVars().GeneralPlanCacheSize), + if s.nonPreparedPlanCache == nil { // lazy construction + s.nonPreparedPlanCache = plannercore.NewLRUPlanCache(uint(s.GetSessionVars().NonPreparedPlanCacheSize), variable.PreparedPlanCacheMemoryGuardRatio.Load(), plannercore.PreparedPlanCacheMaxMemory.Load(), plannercore.PickPlanFromBucket, s) } - return s.generalPlanCache + return s.nonPreparedPlanCache } // use the prepared plan cache @@ -2443,22 +2441,6 @@ func (s *session) rollbackOnError(ctx context.Context) { } } -// CacheGeneralStmt parses the sql, generates the corresponding PlanCacheStmt and cache it. -// The sql have to be parameterized, e.g. select * from t where a>?. -func (s *session) CacheGeneralStmt(sql string) (interface{}, error) { - if stmt := s.sessionVars.GetGeneralPlanCacheStmt(sql); stmt != nil { - // skip this step if there is already a PlanCacheStmt for this ql - return stmt, nil - } - - prepareExec := executor.NewPrepareExec(s, sql) - prepareExec.IsGeneralStmt = true - if err := prepareExec.Next(context.Background(), nil); err != nil { - return nil, err - } - return prepareExec.Stmt, nil -} - // PrepareStmt is used for executing prepare statement in binary protocol func (s *session) PrepareStmt(sql string) (stmtID uint32, paramCount int, fields []*ast.ResultField, err error) { if s.sessionVars.TxnCtx.InfoSchema == nil { diff --git a/sessionctx/context.go b/sessionctx/context.go index f39d3a82a8f38..4cc201206df07 100644 --- a/sessionctx/context.go +++ b/sessionctx/context.go @@ -52,7 +52,7 @@ type SessionStatesHandler interface { DecodeSessionStates(context.Context, Context, *sessionstates.SessionStates) error } -// PlanCache is an interface for prepare and general plan cache +// 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) @@ -120,8 +120,8 @@ type Context interface { GetStore() kv.Storage // GetPlanCache returns the cache of the physical plan. - // generalPlanCache indicates to return the general plan cache or the prepared plan cache. - GetPlanCache(isGeneralPlanCache bool) PlanCache + // isNonPrepared indicates to return the non-prepared plan cache or the prepared plan cache. + GetPlanCache(isNonPrepared bool) PlanCache // StoreQueryFeedback stores the query feedback. StoreQueryFeedback(feedback interface{}) diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 8c977aa832579..ad60ccba2cfa4 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -638,8 +638,8 @@ type SessionVars struct { SysWarningCount int // SysErrorCount is the system variable "error_count", because it is on the hot path, so we extract it from the systems SysErrorCount uint16 - // generalPlanCacheStmts stores PlanCacheStmts for general plan cache. - generalPlanCacheStmts *kvcache.SimpleLRUCache + // nonPreparedPlanCacheStmts stores PlanCacheStmts for non-prepared plan cache. + nonPreparedPlanCacheStmts *kvcache.SimpleLRUCache // PreparedStmts stores prepared statement. PreparedStmts map[uint32]interface{} PreparedStmtNameToID map[string]uint32 @@ -1255,17 +1255,17 @@ type SessionVars struct { // EnablePreparedPlanCache indicates whether to enable prepared plan cache. EnablePreparedPlanCache bool - // GeneralPlanCacheSize controls the size of general plan cache. + // PreparedPlanCacheSize controls the size of prepared plan cache. PreparedPlanCacheSize uint64 // PreparedPlanCacheMonitor indicates whether to enable prepared plan cache monitor. EnablePreparedPlanCacheMemoryMonitor bool - // EnableGeneralPlanCache indicates whether to enable general plan cache. - EnableGeneralPlanCache bool + // EnableNonPreparedPlanCache indicates whether to enable non-prepared plan cache. + EnableNonPreparedPlanCache bool - // GeneralPlanCacheSize controls the size of general plan cache. - GeneralPlanCacheSize uint64 + // NonPreparedPlanCacheSize controls the size of non-prepared plan cache. + NonPreparedPlanCacheSize uint64 // ConstraintCheckInPlacePessimistic controls whether to skip the locking of some keys in pessimistic transactions. // Postpone the conflict check and constraint check to prewrite or later pessimistic locking requests. @@ -2038,20 +2038,20 @@ func (k planCacheStmtKey) Hash() []byte { return []byte(k) } -// AddGeneralPlanCacheStmt adds this PlanCacheStmt into general-plan-cache-stmt cache -func (s *SessionVars) AddGeneralPlanCacheStmt(sql string, stmt interface{}) { - if s.generalPlanCacheStmts == nil { - s.generalPlanCacheStmts = kvcache.NewSimpleLRUCache(uint(s.GeneralPlanCacheSize), 0, 0) +// AddNonPreparedPlanCacheStmt adds this PlanCacheStmt into non-preapred plan-cache stmt cache +func (s *SessionVars) AddNonPreparedPlanCacheStmt(sql string, stmt interface{}) { + if s.nonPreparedPlanCacheStmts == nil { + s.nonPreparedPlanCacheStmts = kvcache.NewSimpleLRUCache(uint(s.NonPreparedPlanCacheSize), 0, 0) } - s.generalPlanCacheStmts.Put(planCacheStmtKey(sql), stmt) + s.nonPreparedPlanCacheStmts.Put(planCacheStmtKey(sql), stmt) } -// GetGeneralPlanCacheStmt gets the PlanCacheStmt. -func (s *SessionVars) GetGeneralPlanCacheStmt(sql string) interface{} { - if s.generalPlanCacheStmts == nil { +// GetNonPreparedPlanCacheStmt gets the PlanCacheStmt. +func (s *SessionVars) GetNonPreparedPlanCacheStmt(sql string) interface{} { + if s.nonPreparedPlanCacheStmts == nil { return nil } - stmt, _ := s.generalPlanCacheStmts.Get(planCacheStmtKey(sql)) + stmt, _ := s.nonPreparedPlanCacheStmts.Get(planCacheStmtKey(sql)) return stmt } diff --git a/sessionctx/variable/session_test.go b/sessionctx/variable/session_test.go index 732ce4ad606cf..67483e0c4f75a 100644 --- a/sessionctx/variable/session_test.go +++ b/sessionctx/variable/session_test.go @@ -398,21 +398,21 @@ func TestTransactionContextSavepoint(t *testing.T) { require.Equal(t, 0, len(tc.Savepoints)) } -func TestGeneralPlanCacheStmt(t *testing.T) { +func TestNonPreparedPlanCacheStmt(t *testing.T) { sessVars := variable.NewSessionVars(nil) - sessVars.GeneralPlanCacheSize = 100 + sessVars.NonPreparedPlanCacheSize = 100 sql1 := "select * from t where a>?" sql2 := "select * from t where a