Skip to content

Commit

Permalink
planner: small refactor for plan cache (#53028)
Browse files Browse the repository at this point in the history
ref #50618
  • Loading branch information
qw4990 authored May 6, 2024
1 parent 659f32a commit 17d3f7f
Show file tree
Hide file tree
Showing 4 changed files with 8 additions and 12 deletions.
3 changes: 1 addition & 2 deletions pkg/expression/util.go
Original file line number Diff line number Diff line change
Expand Up @@ -1166,11 +1166,10 @@ func DatumToConstant(d types.Datum, tp byte, flag uint) *Constant {
// ParamMarkerExpression generate a getparam function expression.
func ParamMarkerExpression(ctx variable.SessionVarsProvider, v *driver.ParamMarkerExpr, needParam bool) (*Constant, error) {
useCache := ctx.GetSessionVars().StmtCtx.UseCache()
isPointExec := ctx.GetSessionVars().StmtCtx.PointExec
tp := types.NewFieldType(mysql.TypeUnspecified)
types.InferParamTypeFromDatum(&v.Datum, tp)
value := &Constant{Value: v.Datum, RetType: tp}
if useCache || isPointExec || needParam {
if useCache || needParam {
value.ParamMarker = &ParamMarker{
order: v.Order,
ctx: ctx,
Expand Down
8 changes: 3 additions & 5 deletions pkg/planner/core/plan_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,6 @@ 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/stmtctx"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
"github.com/pingcap/tidb/pkg/sessiontxn/staleread"
"github.com/pingcap/tidb/pkg/table/tables"
Expand Down Expand Up @@ -225,7 +224,7 @@ 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, stmtCtx); ok {
if plan, names, ok, err := getCachedPointPlan(stmt, sessVars); ok {
return plan, names, err
}
}
Expand Down Expand Up @@ -263,7 +262,7 @@ func parseParamTypes(sctx sessionctx.Context, params []expression.Expression) (p
return
}

func getCachedPointPlan(stmt *PlanCacheStmt, sessVars *variable.SessionVars, stmtCtx *stmtctx.StatementContext) (base.Plan,
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
Expand All @@ -282,7 +281,6 @@ func getCachedPointPlan(stmt *PlanCacheStmt, sessVars *variable.SessionVars, stm
core_metrics.GetPlanCacheHitCounter(false).Inc()
}
sessVars.FoundInPlanCache = true
stmtCtx.PointExec = true
if pointGetPlan, ok := plan.(*PointGetPlan); ok && pointGetPlan != nil && pointGetPlan.stmtHints != nil {
sessVars.StmtCtx.StmtHints = *pointGetPlan.stmtHints
}
Expand Down Expand Up @@ -327,7 +325,7 @@ func getCachedPlan(sctx sessionctx.Context, isNonPrepared bool, cacheKey kvcache
}
stmtCtx.SetPlanDigest(stmt.NormalizedPlan, stmt.PlanDigest)
stmtCtx.StmtHints = *cachedVal.stmtHints
return cachedVal.Plan, cachedVal.OutPutNames, true, nil
return cachedVal.Plan, cachedVal.OutputColumns, true, nil
}

// generateNewPlan call the optimizer to generate a new plan for current statement
Expand Down
8 changes: 4 additions & 4 deletions pkg/planner/core/plan_cache_utils.go
Original file line number Diff line number Diff line change
Expand Up @@ -394,7 +394,7 @@ func NewPlanCacheKey(sessionVars *variable.SessionVars, stmtText, stmtDB string,
// PlanCacheValue stores the cached Statement and StmtNode.
type PlanCacheValue struct {
Plan base.Plan
OutPutNames []*types.FieldName
OutputColumns types.NameSlice
TblInfo2UnionScan map[*model.TableInfo]bool
memoryUsage int64

Expand Down Expand Up @@ -430,7 +430,7 @@ func (v *PlanCacheValue) MemoryUsage() (sum int64) {
sum = unKnownMemoryUsage
}

sum += size.SizeOfInterface + size.SizeOfSlice*2 + int64(cap(v.OutPutNames))*size.SizeOfPointer +
sum += size.SizeOfInterface + size.SizeOfSlice*2 + int64(cap(v.OutputColumns))*size.SizeOfPointer +
size.SizeOfMap + int64(len(v.TblInfo2UnionScan))*(size.SizeOfPointer+size.SizeOfBool) + size.SizeOfInt64*2
if v.matchOpts != nil {
sum += int64(cap(v.matchOpts.ParamTypes)) * size.SizeOfPointer
Expand All @@ -439,7 +439,7 @@ func (v *PlanCacheValue) MemoryUsage() (sum int64) {
}
}

for _, name := range v.OutPutNames {
for _, name := range v.OutputColumns {
sum += name.MemoryUsage()
}
v.memoryUsage = sum
Expand All @@ -459,7 +459,7 @@ func NewPlanCacheValue(plan base.Plan, names []*types.FieldName, srcMap map[*mod
}
return &PlanCacheValue{
Plan: plan,
OutPutNames: names,
OutputColumns: names,
TblInfo2UnionScan: dstMap,
matchOpts: matchOpts,
stmtHints: stmtHints.Clone(),
Expand Down
1 change: 0 additions & 1 deletion pkg/sessionctx/stmtctx/stmtctx.go
Original file line number Diff line number Diff line change
Expand Up @@ -269,7 +269,6 @@ type StatementContext struct {
plan any

Tables []TableEntry
PointExec bool // for point update cached execution, Constant expression need to set "paramMarker"
lockWaitStartTime int64 // LockWaitStartTime stores the pessimistic lock wait start time
PessimisticLockWaited int32
LockKeysDuration int64
Expand Down

0 comments on commit 17d3f7f

Please sign in to comment.