Skip to content

Commit

Permalink
planner: make clear for MaybeOverOptimized4PlanCache (#29782)
Browse files Browse the repository at this point in the history
  • Loading branch information
Reminiscent authored Nov 25, 2021
1 parent fe5733c commit 8dc59e6
Show file tree
Hide file tree
Showing 7 changed files with 36 additions and 35 deletions.
5 changes: 4 additions & 1 deletion expression/builtin_compare.go
Original file line number Diff line number Diff line change
Expand Up @@ -1461,11 +1461,14 @@ func (c *compareFunctionClass) refineArgs(ctx sessionctx.Context, args []Express
// To keep the result be compatible with MySQL, refine `int non-constant <cmp> str constant`
// here and skip this refine operation in all other cases for safety.
if (arg0IsInt && !arg0IsCon && arg1IsString && arg1IsCon) || (arg1IsInt && !arg1IsCon && arg0IsString && arg0IsCon) {
ctx.GetSessionVars().StmtCtx.MaybeOverOptimized4PlanCache = true
ctx.GetSessionVars().StmtCtx.SkipPlanCache = true
RemoveMutableConst(ctx, args)
} else {
return args
}
} else if ctx.GetSessionVars().StmtCtx.SkipPlanCache {
// We should remove the mutable constant for correctness, because its value may be changed.
RemoveMutableConst(ctx, args)
}
// int non-constant [cmp] non-int constant
if arg0IsInt && !arg0IsCon && !arg1IsInt && arg1IsCon {
Expand Down
7 changes: 1 addition & 6 deletions expression/util.go
Original file line number Diff line number Diff line change
Expand Up @@ -920,12 +920,7 @@ func ContainCorrelatedColumn(exprs []Expression) bool {
// TODO: Do more careful check here.
func MaybeOverOptimized4PlanCache(ctx sessionctx.Context, exprs []Expression) bool {
// If we do not enable plan cache, all the optimization can work correctly.
if !ctx.GetSessionVars().StmtCtx.UseCache {
return false
}
if ctx.GetSessionVars().StmtCtx.MaybeOverOptimized4PlanCache {
// If the current statement can not be cached. We should remove the mutable constant.
RemoveMutableConst(ctx, exprs)
if !ctx.GetSessionVars().StmtCtx.UseCache || ctx.GetSessionVars().StmtCtx.SkipPlanCache {
return false
}
return containMutableConst(ctx, exprs)
Expand Down
4 changes: 2 additions & 2 deletions planner/core/common_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -496,7 +496,7 @@ REBUILD:
e.names = names
e.Plan = p
_, isTableDual := p.(*PhysicalTableDual)
if !isTableDual && prepared.UseCache && !stmtCtx.MaybeOverOptimized4PlanCache {
if !isTableDual && prepared.UseCache && !stmtCtx.SkipPlanCache {
// rebuild key to exclude kv.TiFlash when stmt is not read only
if _, isolationReadContainTiFlash := sessVars.IsolationReadEngines[kv.TiFlash]; isolationReadContainTiFlash && !IsReadOnly(stmt, sessVars) {
delete(sessVars.IsolationReadEngines, kv.TiFlash)
Expand Down Expand Up @@ -531,7 +531,7 @@ REBUILD:
// short paths for these executions, currently "point select" and "point update"
func (e *Execute) tryCachePointPlan(ctx context.Context, sctx sessionctx.Context,
preparedStmt *CachedPrepareStmt, is infoschema.InfoSchema, p Plan) error {
if sctx.GetSessionVars().StmtCtx.MaybeOverOptimized4PlanCache {
if sctx.GetSessionVars().StmtCtx.SkipPlanCache {
return nil
}
var (
Expand Down
5 changes: 4 additions & 1 deletion planner/core/expression_rewriter.go
Original file line number Diff line number Diff line change
Expand Up @@ -1454,11 +1454,14 @@ func (er *expressionRewriter) inToExpression(lLen int, not bool, tp *types.Field
if c.GetType().EvalType() == types.ETString {
// To keep the result be compatible with MySQL, refine `int non-constant <cmp> str constant`
// here and skip this refine operation in all other cases for safety.
er.sctx.GetSessionVars().StmtCtx.MaybeOverOptimized4PlanCache = true
er.sctx.GetSessionVars().StmtCtx.SkipPlanCache = true
expression.RemoveMutableConst(er.sctx, []expression.Expression{c})
} else {
continue
}
} else if er.sctx.GetSessionVars().StmtCtx.SkipPlanCache {
// We should remove the mutable constant for correctness, because its value may be changed.
expression.RemoveMutableConst(er.sctx, []expression.Expression{c})
}
args[i], isExceptional = expression.RefineComparedConstant(er.sctx, *leftFt, c, opcode.EQ)
if isExceptional {
Expand Down
2 changes: 1 addition & 1 deletion planner/core/find_best_task.go
Original file line number Diff line number Diff line change
Expand Up @@ -806,7 +806,7 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter
if len(path.Ranges) == 0 {
// We should uncache the tableDual plan.
if expression.MaybeOverOptimized4PlanCache(ds.ctx, path.AccessConds) {
ds.ctx.GetSessionVars().StmtCtx.MaybeOverOptimized4PlanCache = true
ds.ctx.GetSessionVars().StmtCtx.SkipPlanCache = true
}
dual := PhysicalTableDual{}.Init(ds.ctx, ds.stats, ds.blockOffset)
dual.SetSchema(ds.schema)
Expand Down
2 changes: 1 addition & 1 deletion planner/core/optimizer.go
Original file line number Diff line number Diff line change
Expand Up @@ -321,7 +321,7 @@ func postOptimize(sctx sessionctx.Context, plan PhysicalPlan) PhysicalPlan {
// Todo: make more careful check here.
func checkPlanCacheable(sctx sessionctx.Context, plan PhysicalPlan) {
if sctx.GetSessionVars().StmtCtx.UseCache && useTiFlash(plan) {
sctx.GetSessionVars().StmtCtx.MaybeOverOptimized4PlanCache = true
sctx.GetSessionVars().StmtCtx.SkipPlanCache = true
}
}

Expand Down
46 changes: 23 additions & 23 deletions sessionctx/stmtctx/stmtctx.go
Original file line number Diff line number Diff line change
Expand Up @@ -67,29 +67,29 @@ type StatementContext struct {

// IsDDLJobInQueue is used to mark whether the DDL job is put into the queue.
// If IsDDLJobInQueue is true, it means the DDL job is in the queue of storage, and it can be handled by the DDL worker.
IsDDLJobInQueue bool
InInsertStmt bool
InUpdateStmt bool
InDeleteStmt bool
InSelectStmt bool
InLoadDataStmt bool
InExplainStmt bool
InCreateOrAlterStmt bool
IgnoreTruncate bool
IgnoreZeroInDate bool
DupKeyAsWarning bool
BadNullAsWarning bool
DividedByZeroAsWarning bool
TruncateAsWarning bool
OverflowAsWarning bool
InShowWarning bool
UseCache bool
BatchCheck bool
InNullRejectCheck bool
AllowInvalidDate bool
IgnoreNoPartition bool
MaybeOverOptimized4PlanCache bool
IgnoreExplainIDSuffix bool
IsDDLJobInQueue bool
InInsertStmt bool
InUpdateStmt bool
InDeleteStmt bool
InSelectStmt bool
InLoadDataStmt bool
InExplainStmt bool
InCreateOrAlterStmt bool
IgnoreTruncate bool
IgnoreZeroInDate bool
DupKeyAsWarning bool
BadNullAsWarning bool
DividedByZeroAsWarning bool
TruncateAsWarning bool
OverflowAsWarning bool
InShowWarning bool
UseCache bool
BatchCheck bool
InNullRejectCheck bool
AllowInvalidDate bool
IgnoreNoPartition bool
SkipPlanCache bool
IgnoreExplainIDSuffix bool
// If the select statement was like 'select * from t as of timestamp ...' or in a stale read transaction
// or is affected by the tidb_read_staleness session variable, then the statement will be makred as isStaleness
// in stmtCtx
Expand Down

0 comments on commit 8dc59e6

Please sign in to comment.