Skip to content

Commit

Permalink
This is an automated cherry-pick of #52373
Browse files Browse the repository at this point in the history
Signed-off-by: ti-chi-bot <ti-community-prow-bot@tidb.io>
  • Loading branch information
qw4990 authored and ti-chi-bot committed May 31, 2024
1 parent d9c5bf6 commit fee09c4
Show file tree
Hide file tree
Showing 8 changed files with 165 additions and 18 deletions.
2 changes: 1 addition & 1 deletion executor/adapter.go
Original file line number Diff line number Diff line change
Expand Up @@ -312,7 +312,7 @@ func (a *ExecStmt) PointGet(ctx context.Context) (*recordSet, error) {
a.PsStmt.Executor = nil
} else {
// CachedPlan type is already checked in last step
pointGetPlan := a.PsStmt.PreparedAst.CachedPlan.(*plannercore.PointGetPlan)
pointGetPlan := a.PsStmt.CachedPlan.(*plannercore.PointGetPlan)
exec.Init(pointGetPlan)
a.PsStmt.Executor = exec
pointExecutor = exec
Expand Down
2 changes: 1 addition & 1 deletion executor/compiler.go
Original file line number Diff line number Diff line change
Expand Up @@ -152,7 +152,7 @@ func (c *Compiler) Compile(ctx context.Context, stmtNode ast.StmtNode) (_ *ExecS
stmt.PsStmt = preparedObj
} else {
// invalid the previous cached point plan
preparedObj.PreparedAst.CachedPlan = nil
preparedObj.CachedPlan = nil
}
}
}
Expand Down
8 changes: 7 additions & 1 deletion executor/prepared.go
Original file line number Diff line number Diff line change
Expand Up @@ -206,12 +206,18 @@ func (e *DeallocateExec) Next(ctx context.Context, req *chunk.Chunk) error {
if !ok {
return errors.Errorf("invalid PlanCacheStmt type")
}
prepared := preparedObj.PreparedAst
delete(vars.PreparedStmtNameToID, e.Name)
<<<<<<< HEAD:executor/prepared.go
if e.ctx.GetSessionVars().EnablePreparedPlanCache {
bindSQL, _ := plannercore.GetBindSQL4PlanCache(e.ctx, preparedObj)
cacheKey, err := plannercore.NewPlanCacheKey(vars, preparedObj.StmtText, preparedObj.StmtDB, prepared.SchemaVersion,
0, bindSQL)
=======
if e.Ctx().GetSessionVars().EnablePreparedPlanCache {
bindSQL, _ := bindinfo.MatchSQLBindingForPlanCache(e.Ctx(), preparedObj.PreparedAst.Stmt, &preparedObj.BindingInfo)
cacheKey, err := plannercore.NewPlanCacheKey(vars, preparedObj.StmtText, preparedObj.StmtDB, preparedObj.SchemaVersion,
0, bindSQL, expression.ExprPushDownBlackListReloadTimeStamp.Load())
>>>>>>> 62d6f4737bf (planner: move fields from ast.Prepared to planner.PlanCacheStmt (#52373)):pkg/executor/prepared.go
if err != nil {
return err
}
Expand Down
5 changes: 5 additions & 0 deletions parser/ast/misc.go
Original file line number Diff line number Diff line change
Expand Up @@ -516,13 +516,18 @@ func (n *DeallocateStmt) Accept(v Visitor) (Node, bool) {

// Prepared represents a prepared statement.
type Prepared struct {
<<<<<<< HEAD:parser/ast/misc.go
Stmt StmtNode
StmtType string
Params []ParamMarkerExpr
SchemaVersion int64
UseCache bool
CachedPlan interface{}
CachedNames interface{}
=======
Stmt StmtNode
StmtType string
>>>>>>> 62d6f4737bf (planner: move fields from ast.Prepared to planner.PlanCacheStmt (#52373)):pkg/parser/ast/misc.go
}

// ExecuteStmt is a statement to execute PreparedStmt.
Expand Down
103 changes: 93 additions & 10 deletions planner/core/plan_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -71,18 +71,53 @@ func planCachePreprocess(ctx context.Context, sctx sessionctx.Context, isGeneral
}
val.SetBinaryLiteral(binVal)
}
<<<<<<< HEAD:planner/core/plan_cache.go
param.Datum = val
param.InExecute = true
vars.PreparedParams = append(vars.PreparedParams, val)
=======
if markers != nil {
param := markers[i].(*driver.ParamMarkerExpr)
param.Datum = val
param.InExecute = true
}
vars.PlanCacheParams.Append(val)
}
if vars.StmtCtx.EnableOptimizerDebugTrace && len(vars.PlanCacheParams.AllParamValues()) > 0 {
vals := vars.PlanCacheParams.AllParamValues()
valStrs := make([]string, len(vals))
for i, val := range vals {
valStrs[i] = val.String()
}
debugtrace.RecordAnyValuesWithNames(sctx, "Parameter datums for EXECUTE", valStrs)
}
vars.PlanCacheParams.SetForNonPrepCache(isNonPrep)
return nil
}

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

// step 1: check parameter number
if len(stmt.Params) != len(params) {
return errors.Trace(plannererrors.ErrWrongParamCount)
}

// step 2: set parameter values
if err := SetParameterValuesIntoSCtx(sctx.GetPlanCtx(), isNonPrepared, stmt.Params, params); err != nil {
return errors.Trace(err)
>>>>>>> 62d6f4737bf (planner: move fields from ast.Prepared to planner.PlanCacheStmt (#52373)):pkg/planner/core/plan_cache.go
}

// step 3: check schema version
if stmtAst.SchemaVersion != is.SchemaMetaVersion() {
if stmt.SchemaVersion != is.SchemaMetaVersion() {
// In order to avoid some correctness issues, we have to clear the
// cached plan once the schema version is changed.
// Cached plan in prepared struct does NOT have a "cache key" with
// schema version like prepared plan cache key
stmtAst.CachedPlan = nil
stmt.CachedPlan = nil
stmt.Executor = nil
stmt.ColumnInfos = nil
// If the schema version has changed we need to preprocess it again,
Expand All @@ -96,7 +131,7 @@ func planCachePreprocess(ctx context.Context, sctx sessionctx.Context, isGeneral
if err != nil {
return ErrSchemaChanged.GenWithStack("Schema change caused error: %s", err.Error())
}
stmtAst.SchemaVersion = is.SchemaMetaVersion()
stmt.SchemaVersion = is.SchemaMetaVersion()
}

// step 4: handle expiration
Expand All @@ -105,9 +140,15 @@ func planCachePreprocess(ctx context.Context, sctx sessionctx.Context, isGeneral
// So we need to clear the current session's plan cache.
// And update lastUpdateTime to the newest one.
expiredTimeStamp4PC := domain.GetDomain(sctx).ExpiredTimeStamp4PC()
<<<<<<< HEAD:planner/core/plan_cache.go
if stmtAst.UseCache && expiredTimeStamp4PC.Compare(vars.LastUpdateTime4PC) > 0 {
sctx.GetPlanCache(isGeneralPlanCache).DeleteAll()
stmtAst.CachedPlan = nil
=======
if stmt.StmtCacheable && expiredTimeStamp4PC.Compare(vars.LastUpdateTime4PC) > 0 {
sctx.GetSessionPlanCache().DeleteAll()
stmt.CachedPlan = nil
>>>>>>> 62d6f4737bf (planner: move fields from ast.Prepared to planner.PlanCacheStmt (#52373)):pkg/planner/core/plan_cache.go
vars.LastUpdateTime4PC = expiredTimeStamp4PC
}
return nil
Expand All @@ -127,8 +168,23 @@ func GetPlanFromSessionPlanCache(ctx context.Context, sctx sessionctx.Context,
var cacheKey kvcache.Key
sessVars := sctx.GetSessionVars()
stmtCtx := sessVars.StmtCtx
<<<<<<< HEAD:planner/core/plan_cache.go
stmtAst := stmt.PreparedAst
stmtCtx.UseCache = stmtAst.UseCache
=======
cacheEnabled := false
if isNonPrepared {
stmtCtx.CacheType = stmtctx.SessionNonPrepared
cacheEnabled = sctx.GetSessionVars().EnableNonPreparedPlanCache // plan-cache might be disabled after prepare.
} else {
stmtCtx.CacheType = stmtctx.SessionPrepared
cacheEnabled = sctx.GetSessionVars().EnablePreparedPlanCache
}
stmtCtx.UseCache = stmt.StmtCacheable && cacheEnabled
if stmt.UncacheableReason != "" {
stmtCtx.ForceSetSkipPlanCache(errors.NewNoStackError(stmt.UncacheableReason))
}
>>>>>>> 62d6f4737bf (planner: move fields from ast.Prepared to planner.PlanCacheStmt (#52373)):pkg/planner/core/plan_cache.go

var bindSQL string
if stmtCtx.UseCache {
Expand All @@ -151,15 +207,24 @@ func GetPlanFromSessionPlanCache(ctx context.Context, sctx sessionctx.Context,
latestSchemaVersion = domain.GetDomain(sctx).InfoSchema().SchemaMetaVersion()
}
if cacheKey, err = NewPlanCacheKey(sctx.GetSessionVars(), stmt.StmtText,
<<<<<<< HEAD:planner/core/plan_cache.go
stmt.StmtDB, stmtAst.SchemaVersion, latestSchemaVersion, bindSQL); err != nil {
=======
stmt.StmtDB, stmt.SchemaVersion, latestSchemaVersion, bindSQL, expression.ExprPushDownBlackListReloadTimeStamp.Load()); err != nil {
>>>>>>> 62d6f4737bf (planner: move fields from ast.Prepared to planner.PlanCacheStmt (#52373)):pkg/planner/core/plan_cache.go
return nil, nil, err
}
}

<<<<<<< HEAD:planner/core/plan_cache.go
paramTypes := parseParamTypes(sctx, params)

if stmtCtx.UseCache && stmtAst.CachedPlan != nil { // for point query plan
if plan, names, ok, err := getPointQueryPlan(stmtAst, sessVars, stmtCtx); ok {
=======
if stmtCtx.UseCache && stmt.CachedPlan != nil { // special code path for fast point plan
if plan, names, ok, err := getCachedPointPlan(stmt, sessVars, stmtCtx); ok {
>>>>>>> 62d6f4737bf (planner: move fields from ast.Prepared to planner.PlanCacheStmt (#52373)):pkg/planner/core/plan_cache.go
return plan, names, err
}
}
Expand Down Expand Up @@ -193,7 +258,11 @@ func parseParamTypes(sctx sessionctx.Context, params []expression.Expression) (p
return
}

<<<<<<< HEAD:planner/core/plan_cache.go
func getPointQueryPlan(stmt *ast.Prepared, sessVars *variable.SessionVars, stmtCtx *stmtctx.StatementContext) (Plan,
=======
func getCachedPointPlan(stmt *PlanCacheStmt, sessVars *variable.SessionVars, stmtCtx *stmtctx.StatementContext) (Plan,
>>>>>>> 62d6f4737bf (planner: move fields from ast.Prepared to planner.PlanCacheStmt (#52373)):pkg/planner/core/plan_cache.go
[]*types.FieldName, bool, error) {
// short path for point-get plans
// Rewriting the expression in the select.where condition will convert its
Expand Down Expand Up @@ -293,7 +362,11 @@ func generateNewPlan(ctx context.Context, sctx sessionctx.Context, isGeneralPlan
if _, isolationReadContainTiFlash := sessVars.IsolationReadEngines[kv.TiFlash]; isolationReadContainTiFlash && !IsReadOnly(stmtAst.Stmt, sessVars) {
delete(sessVars.IsolationReadEngines, kv.TiFlash)
if cacheKey, err = NewPlanCacheKey(sessVars, stmt.StmtText, stmt.StmtDB,
<<<<<<< HEAD:planner/core/plan_cache.go
stmtAst.SchemaVersion, latestSchemaVersion, bindSQL); err != nil {
=======
stmt.SchemaVersion, latestSchemaVersion, bindSQL, expression.ExprPushDownBlackListReloadTimeStamp.Load()); err != nil {
>>>>>>> 62d6f4737bf (planner: move fields from ast.Prepared to planner.PlanCacheStmt (#52373)):pkg/planner/core/plan_cache.go
return nil, nil, err
}
sessVars.IsolationReadEngines[kv.TiFlash] = struct{}{}
Expand Down Expand Up @@ -690,10 +763,15 @@ func tryCachePointPlan(_ context.Context, sctx sessionctx.Context,
return nil
}
var (
<<<<<<< HEAD:planner/core/plan_cache.go
stmtAst = stmt.PreparedAst
ok bool
err error
names types.NameSlice
=======
ok bool
err error
>>>>>>> 62d6f4737bf (planner: move fields from ast.Prepared to planner.PlanCacheStmt (#52373)):pkg/planner/core/plan_cache.go
)

if plan, _ok := p.(*PointGetPlan); _ok {
Expand All @@ -709,8 +787,8 @@ func tryCachePointPlan(_ context.Context, sctx sessionctx.Context,

if ok {
// just cache point plan now
stmtAst.CachedPlan = p
stmtAst.CachedNames = names
stmt.CachedPlan = p
stmt.CachedNames = names
stmt.NormalizedPlan, stmt.PlanDigest = NormalizePlan(p)
sctx.GetSessionVars().StmtCtx.SetPlan(p)
sctx.GetSessionVars().StmtCtx.SetPlanDigest(stmt.NormalizedPlan, stmt.PlanDigest)
Expand Down Expand Up @@ -805,33 +883,38 @@ func GetBindSQL4PlanCache(sctx sessionctx.Context, stmt *PlanCacheStmt) (string,
// IsPointPlanShortPathOK 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
<<<<<<< HEAD:planner/core/plan_cache.go
func IsPointPlanShortPathOK(sctx sessionctx.Context, is infoschema.InfoSchema, stmt *PlanCacheStmt) (bool, error) {
stmtAst := stmt.PreparedAst
if stmtAst.CachedPlan == nil || staleread.IsStmtStaleness(sctx) {
=======
func IsPointGetPlanShortPathOK(sctx sessionctx.Context, is infoschema.InfoSchema, stmt *PlanCacheStmt) (bool, error) {
if stmt.CachedPlan == nil || staleread.IsStmtStaleness(sctx) {
>>>>>>> 62d6f4737bf (planner: move fields from ast.Prepared to planner.PlanCacheStmt (#52373)):pkg/planner/core/plan_cache.go
return false, nil
}
// check auto commit
if !IsAutoCommitTxn(sctx) {
return false, nil
}
if stmtAst.SchemaVersion != is.SchemaMetaVersion() {
stmtAst.CachedPlan = nil
if stmt.SchemaVersion != is.SchemaMetaVersion() {
stmt.CachedPlan = nil
stmt.ColumnInfos = nil
return false, nil
}
// maybe we'd better check cached plan type here, current
// only point select/update will be cached, see "getPhysicalPlan" func
var ok bool
var err error
switch stmtAst.CachedPlan.(type) {
switch stmt.CachedPlan.(type) {
case *PointGetPlan:
ok = true
case *Update:
pointUpdate := stmtAst.CachedPlan.(*Update)
pointUpdate := stmt.CachedPlan.(*Update)
_, ok = pointUpdate.SelectPlan.(*PointGetPlan)
if !ok {
err = errors.Errorf("cached update plan not point update")
stmtAst.CachedPlan = nil
stmt.CachedPlan = nil
return false, err
}
default:
Expand Down
44 changes: 44 additions & 0 deletions planner/core/plan_cache_utils.go
Original file line number Diff line number Diff line change
Expand Up @@ -110,10 +110,15 @@ func GeneratePlanCacheStmtWithAST(ctx context.Context, sctx sessionctx.Context,
}

prepared := &ast.Prepared{
<<<<<<< HEAD:planner/core/plan_cache_utils.go
Stmt: stmt,
StmtType: ast.GetStmtLabel(stmt),
Params: extractor.markers,
SchemaVersion: ret.InfoSchema.SchemaMetaVersion(),
=======
Stmt: paramStmt,
StmtType: ast.GetStmtLabel(paramStmt),
>>>>>>> 62d6f4737bf (planner: move fields from ast.Prepared to planner.PlanCacheStmt (#52373)):pkg/planner/core/plan_cache_utils.go
}
normalizedSQL, digest := parser.NormalizeDigest(prepared.Stmt.Text())

Expand All @@ -136,11 +141,23 @@ func GeneratePlanCacheStmtWithAST(ctx context.Context, sctx sessionctx.Context,
}
}

<<<<<<< HEAD:planner/core/plan_cache_utils.go
// We try to build the real statement of preparedStmt.
for i := range prepared.Params {
param := prepared.Params[i].(*driver.ParamMarkerExpr)
param.Datum.SetNull()
param.InExecute = false
=======
// For prepared statements like `prepare st from 'select * from t where a<?'`,
// parameters are unknown here, so regard them all as NULL.
// For non-prepared statements, all parameters are already initialized at `ParameterizeAST`, so no need to set NULL.
if isPrepStmt {
for i := range extractor.markers {
param := extractor.markers[i].(*driver.ParamMarkerExpr)
param.Datum.SetNull()
param.InExecute = false
}
>>>>>>> 62d6f4737bf (planner: move fields from ast.Prepared to planner.PlanCacheStmt (#52373)):pkg/planner/core/plan_cache_utils.go
}

var p Plan
Expand All @@ -159,8 +176,16 @@ func GeneratePlanCacheStmtWithAST(ctx context.Context, sctx sessionctx.Context,
SQLDigest: digest,
ForUpdateRead: destBuilder.GetIsForUpdateRead(),
SnapshotTSEvaluator: ret.SnapshotTSEvaluator,
<<<<<<< HEAD:planner/core/plan_cache_utils.go
NormalizedSQL4PC: normalizedSQL4PC,
SQLDigest4PC: digest4PC,
=======
StmtCacheable: cacheable,
UncacheableReason: reason,
QueryFeatures: features,
SchemaVersion: ret.InfoSchema.SchemaMetaVersion(),
Params: extractor.markers,
>>>>>>> 62d6f4737bf (planner: move fields from ast.Prepared to planner.PlanCacheStmt (#52373)):pkg/planner/core/plan_cache_utils.go
}
if err = CheckPreparedPriv(sctx, preparedObj, ret.InfoSchema); err != nil {
return nil, nil, 0, err
Expand Down Expand Up @@ -427,11 +452,30 @@ type PlanCacheStmt struct {
PreparedAst *ast.Prepared
StmtDB string // which DB the statement will be processed over
VisitInfos []visitInfo
<<<<<<< HEAD:planner/core/plan_cache_utils.go
ColumnInfos interface{}
// Executor is only used for point get scene.
// Notice that we should only cache the PointGetExecutor that have a snapshot with MaxTS in it.
// If the current plan is not PointGet or does not use MaxTS optimization, this value should be nil here.
Executor interface{}
=======
ColumnInfos any
Params []ast.ParamMarkerExpr
// Executor is only used for point get scene.
// Notice that we should only cache the PointGetExecutor that have a snapshot with MaxTS in it.
// If the current plan is not PointGet or does not use MaxTS optimization, this value should be nil here.
Executor any

// below fields are for PointGet short path
SchemaVersion int64
CachedPlan any
CachedNames any

StmtCacheable bool // Whether this stmt is cacheable.
UncacheableReason string // Why this stmt is uncacheable.
QueryFeatures *PlanCacheQueryFeatures

>>>>>>> 62d6f4737bf (planner: move fields from ast.Prepared to planner.PlanCacheStmt (#52373)):pkg/planner/core/plan_cache_utils.go
NormalizedSQL string
NormalizedPlan string
SQLDigest *parser.Digest
Expand Down
4 changes: 4 additions & 0 deletions server/driver_tidb.go
Original file line number Diff line number Diff line change
Expand Up @@ -204,7 +204,11 @@ func (ts *TiDBStatement) Close() error {
}
bindSQL, _ := core.GetBindSQL4PlanCache(ts.ctx, preparedObj)
cacheKey, err := core.NewPlanCacheKey(ts.ctx.GetSessionVars(), preparedObj.StmtText, preparedObj.StmtDB,
<<<<<<< HEAD:server/driver_tidb.go
preparedObj.PreparedAst.SchemaVersion, 0, bindSQL)
=======
preparedObj.SchemaVersion, 0, bindSQL, expression.ExprPushDownBlackListReloadTimeStamp.Load())
>>>>>>> 62d6f4737bf (planner: move fields from ast.Prepared to planner.PlanCacheStmt (#52373)):pkg/server/driver_tidb.go
if err != nil {
return err
}
Expand Down
Loading

0 comments on commit fee09c4

Please sign in to comment.