Skip to content

Commit

Permalink
Revert "planner: simplify plan cache for fast point get (pingcap#53094)"
Browse files Browse the repository at this point in the history
This reverts commit 50b8dfa.
  • Loading branch information
qw4990 committed May 29, 2024
1 parent e169d6f commit 2ae7691
Show file tree
Hide file tree
Showing 3 changed files with 72 additions and 33 deletions.
101 changes: 70 additions & 31 deletions pkg/planner/core/plan_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ import (
"github.com/pingcap/tidb/pkg/planner/util/fixcontrol"
"github.com/pingcap/tidb/pkg/privilege"
"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
"github.com/pingcap/tidb/pkg/sessiontxn/staleread"
"github.com/pingcap/tidb/pkg/table/tables"
"github.com/pingcap/tidb/pkg/types"
Expand Down Expand Up @@ -222,11 +223,17 @@ func GetPlanFromSessionPlanCache(ctx context.Context, sctx sessionctx.Context,
}
}

if stmtCtx.UseCache() && stmt.PointGet.Plan != nil { // special code path for fast point plan
if plan, names, ok, err := getCachedPointPlan(stmt, sessVars); ok {
return plan, names, err
}
}

matchOpts, err := GetMatchOpts(sctx, is, stmt, params)
if err != nil {
return nil, nil, err
}
if stmtCtx.UseCache() {
if stmtCtx.UseCache() { // for non-point plans
if plan, names, ok, err := getCachedPlan(sctx, isNonPrepared, cacheKey, bindSQL, is, stmt, matchOpts); err != nil || ok {
return plan, names, err
}
Expand Down Expand Up @@ -255,38 +262,43 @@ func parseParamTypes(sctx sessionctx.Context, params []expression.Expression) (p
return
}

func getCachedPointPlan(stmt *PlanCacheStmt, sessVars *variable.SessionVars) (base.Plan,
[]*types.FieldName, bool, error) {
// short path for point-get plans
// Rewriting the expression in the select.where condition will convert its
// type from "paramMarker" to "Constant".When Point Select queries are executed,
// the expression in the where condition will not be evaluated,
// so you don't need to consider whether prepared.useCache is enabled.
plan := stmt.PointGet.Plan.(base.Plan)
names := stmt.PointGet.ColumnNames.(types.NameSlice)
if !RebuildPlan4CachedPlan(plan) {
return nil, nil, false, nil
}
if metrics.ResettablePlanCacheCounterFortTest {
metrics.PlanCacheCounter.WithLabelValues("prepare").Inc()
} else {
// only for prepared plan cache
core_metrics.GetPlanCacheHitCounter(false).Inc()
}
sessVars.FoundInPlanCache = true
if pointGetPlan, ok := plan.(*PointGetPlan); ok && pointGetPlan != nil && pointGetPlan.stmtHints != nil {
sessVars.StmtCtx.StmtHints = *pointGetPlan.stmtHints
}
return plan, names, true, nil
}

func getCachedPlan(sctx sessionctx.Context, isNonPrepared bool, cacheKey kvcache.Key, bindSQL string,
is infoschema.InfoSchema, stmt *PlanCacheStmt, matchOpts *utilpc.PlanCacheMatchOpts) (base.Plan,
[]*types.FieldName, bool, error) {
sessVars := sctx.GetSessionVars()
stmtCtx := sessVars.StmtCtx

// handle PointGet Plan specially
if stmt.PointGet.Plan != nil { // TODO: remove this special handle for point-get plan
plan := stmt.PointGet.Plan.(base.Plan)
names := stmt.PointGet.ColumnNames.(types.NameSlice)
if !RebuildPlan4CachedPlan(plan) {
return nil, nil, false, nil
}
if metrics.ResettablePlanCacheCounterFortTest {
metrics.PlanCacheCounter.WithLabelValues("prepare").Inc()
} else {
// only for prepared plan cache
core_metrics.GetPlanCacheHitCounter(false).Inc()
}
sessVars.FoundInPlanCache = true
if pointGetPlan, ok := plan.(*PointGetPlan); ok && pointGetPlan != nil && pointGetPlan.stmtHints != nil {
stmtCtx.StmtHints = *pointGetPlan.stmtHints
}
return plan, names, true, nil
}

candidate, exist := sctx.GetSessionPlanCache().Get(cacheKey, matchOpts)
if !exist {
return nil, nil, false, nil
}
cachedVal := candidate.(*PlanCacheValue)
if err := checkPreparedPriv(sctx, stmt, is); err != nil {
if err := CheckPreparedPriv(sctx, stmt, is); err != nil {
return nil, nil, false, err
}
for tblInfo, unionScan := range cachedVal.TblInfo2UnionScan {
Expand Down Expand Up @@ -332,6 +344,10 @@ func generateNewPlan(ctx context.Context, sctx sessionctx.Context, isNonPrepared
if err != nil {
return nil, nil, err
}
err = tryCachePointPlan(ctx, sctx.GetPlanCtx(), stmt, p, names)
if err != nil {
return nil, nil, err
}

// check whether this plan is cacheable.
if stmtCtx.UseCache() {
Expand All @@ -342,13 +358,6 @@ func generateNewPlan(ctx context.Context, sctx sessionctx.Context, isNonPrepared

// put this plan into the plan cache.
if stmtCtx.UseCache() {
// update PointGet Plan specially
if pointGet, ok := p.(*PointGetPlan); ok {
pointGet.stmtHints = sctx.GetSessionVars().StmtCtx.StmtHints.Clone()
stmt.PointGet.Plan = p
stmt.PointGet.ColumnNames = names
}

// rebuild key to exclude kv.TiFlash when stmt is not read only
if _, isolationReadContainTiFlash := sessVars.IsolationReadEngines[kv.TiFlash]; isolationReadContainTiFlash && !IsReadOnly(stmtAst.Stmt, sessVars) {
delete(sessVars.IsolationReadEngines, kv.TiFlash)
Expand Down Expand Up @@ -775,8 +784,8 @@ func isSafeRange(accessConds []expression.Expression, rebuiltResult *ranger.Deta
return true
}

// checkPreparedPriv checks the privilege of the prepared statement
func checkPreparedPriv(sctx sessionctx.Context, stmt *PlanCacheStmt, is infoschema.InfoSchema) error {
// CheckPreparedPriv checks the privilege of the prepared statement
func CheckPreparedPriv(sctx sessionctx.Context, stmt *PlanCacheStmt, is infoschema.InfoSchema) error {
if pm := privilege.GetPrivilegeManager(sctx); pm != nil {
visitInfo := VisitInfo4PrivCheck(is, stmt.PreparedAst.Stmt, stmt.VisitInfos)
if err := CheckPrivilege(sctx.GetSessionVars().ActiveRoles, pm, visitInfo); err != nil {
Expand All @@ -787,6 +796,36 @@ func checkPreparedPriv(sctx sessionctx.Context, stmt *PlanCacheStmt, is infosche
return err
}

// tryCachePointPlan will try to cache point execution plan, there may be some
// short paths for these executions, currently "point select" and "point update"
func tryCachePointPlan(_ context.Context, sctx base.PlanContext,
stmt *PlanCacheStmt, p base.Plan, names types.NameSlice) error {
if !sctx.GetSessionVars().StmtCtx.UseCache() {
return nil
}
var (
ok bool
err error
)

if plan, _ok := p.(*PointGetPlan); _ok {
ok = IsPointGetWithPKOrUniqueKeyByAutoCommit(sctx.GetSessionVars(), p)
if ok {
plan.stmtHints = sctx.GetSessionVars().StmtCtx.StmtHints.Clone()
}
}

if ok {
// just cache point plan now
stmt.PointGet.Plan = p
stmt.PointGet.ColumnNames = names
stmt.NormalizedPlan, stmt.PlanDigest = NormalizePlan(p)
sctx.GetSessionVars().StmtCtx.SetPlan(p)
sctx.GetSessionVars().StmtCtx.SetPlanDigest(stmt.NormalizedPlan, stmt.PlanDigest)
}
return err
}

// IsPointGetPlanShortPathOK check if we can execute using plan cached in prepared structure
// Be careful with the short path, current precondition is ths cached plan satisfying
// IsPointGetWithPKOrUniqueKeyByAutoCommit
Expand Down
2 changes: 1 addition & 1 deletion pkg/planner/core/plan_cache_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1708,7 +1708,7 @@ func TestFixControl33031(t *testing.T) {
tk.MustExec(`set @a = 1`)
tk.MustQuery(`execute stmt using @a`).Check(testkit.Rows("1 1"))
require.False(t, tk.Session().GetSessionVars().FoundInPlanCache)
tk.MustQuery(`show warnings`).Check(testkit.Rows("Warning 1105 skip plan-cache: plan rebuild failed, Fix33031 fix-control set and partitioned table in cached Point Get plan"))
tk.MustQuery(`show warnings`).Check(testkit.Rows("Warning 1105 skip plan-cache: plan rebuild failed, Fix33031 fix-control set and partitioned table in cached Point Get plan", "Warning 1105 skip plan-cache: plan rebuild failed, Fix33031 fix-control set and partitioned table in cached Point Get plan"))
tk.MustExec(`set @@tidb_opt_fix_control = "33031:OFF"`)
tk.MustExec(`set @a = 2`)
tk.MustQuery(`execute stmt using @a`).Check(testkit.Rows("2 2"))
Expand Down
2 changes: 1 addition & 1 deletion pkg/planner/core/plan_cache_utils.go
Original file line number Diff line number Diff line change
Expand Up @@ -223,7 +223,7 @@ func GeneratePlanCacheStmtWithAST(ctx context.Context, sctx sessionctx.Context,
RelateVersion: relateVersion,
Params: extractor.markers,
}
if err = checkPreparedPriv(sctx, preparedObj, ret.InfoSchema); err != nil {
if err = CheckPreparedPriv(sctx, preparedObj, ret.InfoSchema); err != nil {
return nil, nil, 0, err
}
return preparedObj, p, paramCount, nil
Expand Down

0 comments on commit 2ae7691

Please sign in to comment.