diff --git a/expression/builtin_compare.go b/expression/builtin_compare.go index 1090ca0e14eea..e4edf23ab719f 100644 --- a/expression/builtin_compare.go +++ b/expression/builtin_compare.go @@ -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 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 { diff --git a/expression/util.go b/expression/util.go index 43af814cc0742..7b34ef442067b 100644 --- a/expression/util.go +++ b/expression/util.go @@ -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) diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index 10241797b6fe9..6419f4a04c19f 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -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) @@ -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 ( diff --git a/planner/core/expression_rewriter.go b/planner/core/expression_rewriter.go index 5bea6442b9101..ab57b29c2096d 100644 --- a/planner/core/expression_rewriter.go +++ b/planner/core/expression_rewriter.go @@ -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 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 { diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index e0abd33d197b7..baaf5d17f483c 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -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) diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index 8677f16e47871..f800973cd2f73 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -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 } } diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index 1a295e2a20d55..bdf93278a59d5 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -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