Skip to content

Commit

Permalink
planner: rename general plan cache to non-prepared plan cache (#3…
Browse files Browse the repository at this point in the history
  • Loading branch information
qw4990 committed Dec 13, 2022
1 parent 760dbe1 commit aeceb22
Show file tree
Hide file tree
Showing 14 changed files with 85 additions and 124 deletions.
10 changes: 2 additions & 8 deletions executor/prepared.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down Expand Up @@ -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)
}

Expand Down
4 changes: 2 additions & 2 deletions expression/function_traits.go
Original file line number Diff line number Diff line change
Expand Up @@ -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: {},
Expand Down
30 changes: 15 additions & 15 deletions planner/core/plan_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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
}
Expand All @@ -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
}

Expand Down Expand Up @@ -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)
}

Expand All @@ -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
Expand All @@ -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
}
Expand All @@ -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
}
}
Expand All @@ -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
Expand Down Expand Up @@ -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
Expand Down
4 changes: 2 additions & 2 deletions planner/core/plan_cache_param.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
Expand Down Expand Up @@ -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 {
Expand Down
2 changes: 1 addition & 1 deletion planner/core/plan_cache_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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`)
Expand Down
15 changes: 0 additions & 15 deletions planner/core/plan_cache_utils.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down
24 changes: 12 additions & 12 deletions planner/core/plan_cacheable_checker.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -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) {
Expand All @@ -217,7 +217,7 @@ func GeneralPlanCacheableWithCtx(sctx sessionctx.Context, node ast.Node, is info
}
}

checker := generalPlanCacheableChecker{
checker := nonPreparedPlanCacheableChecker{
sctx: sctx,
cacheable: true,
schema: is,
Expand All @@ -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
}
Expand All @@ -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
}

Expand Down
6 changes: 3 additions & 3 deletions planner/core/plan_cacheable_checker_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down Expand Up @@ -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))
}
}
18 changes: 9 additions & 9 deletions planner/optimize.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down Expand Up @@ -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
}
Expand Down Expand Up @@ -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
}
Expand Down
Loading

0 comments on commit aeceb22

Please sign in to comment.