From 9d7608201e61ebfe594adfc7753496c8fc759c9d Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Fri, 19 Jan 2024 12:44:44 +0800 Subject: [PATCH 1/2] bindinfo: SetSkipPlanCache use nostack errors Signed-off-by: Weizhen Wang --- pkg/executor/executor.go | 2 +- pkg/expression/builtin_compare.go | 4 ++-- pkg/expression/builtin_other.go | 2 +- pkg/expression/constant_propagation.go | 4 ++-- pkg/expression/expression.go | 2 +- pkg/planner/core/find_best_task.go | 4 ++-- pkg/planner/core/plan_cache.go | 2 +- pkg/planner/core/plan_cache_utils.go | 6 +++--- pkg/planner/core/rule_predicate_simplification.go | 4 ++-- pkg/planner/optimize.go | 2 +- 10 files changed, 16 insertions(+), 16 deletions(-) diff --git a/pkg/executor/executor.go b/pkg/executor/executor.go index abe6e8be19d9a..573e2cc56acf6 100644 --- a/pkg/executor/executor.go +++ b/pkg/executor/executor.go @@ -1460,7 +1460,7 @@ func init() { plannercore.EvalSubqueryFirstRow = func(ctx context.Context, p plannercore.PhysicalPlan, is infoschema.InfoSchema, sctx sessionctx.Context) ([]types.Datum, error) { defer func(begin time.Time) { s := sctx.GetSessionVars() - s.StmtCtx.SetSkipPlanCache(errors.New("query has uncorrelated sub-queries is un-cacheable")) + s.StmtCtx.SetSkipPlanCache(errors.NewNoStackError("query has uncorrelated sub-queries is un-cacheable")) s.RewritePhaseInfo.PreprocessSubQueries++ s.RewritePhaseInfo.DurationPreprocessSubQuery += time.Since(begin) }(time.Now()) diff --git a/pkg/expression/builtin_compare.go b/pkg/expression/builtin_compare.go index 8f12b837e9050..3e53d42ce24f6 100644 --- a/pkg/expression/builtin_compare.go +++ b/pkg/expression/builtin_compare.go @@ -1535,7 +1535,7 @@ func allowCmpArgsRefining4PlanCache(ctx sessionctx.Context, args []Expression) ( exprType := args[1-conIdx].GetType() exprEvalType := exprType.EvalType() if exprType.GetType() == mysql.TypeYear { - reason := errors.Errorf("'%v' may be converted to INT", args[conIdx].String()) + reason := errors.NewNoStackErrorf("'%v' may be converted to INT", args[conIdx].String()) ctx.GetSessionVars().StmtCtx.SetSkipPlanCache(reason) return true } @@ -1545,7 +1545,7 @@ func allowCmpArgsRefining4PlanCache(ctx sessionctx.Context, args []Expression) ( conEvalType := args[conIdx].GetType().EvalType() if exprEvalType == types.ETInt && (conEvalType == types.ETString || conEvalType == types.ETReal || conEvalType == types.ETDecimal) { - reason := errors.Errorf("'%v' may be converted to INT", args[conIdx].String()) + reason := errors.NewNoStackErrorf("'%v' may be converted to INT", args[conIdx].String()) ctx.GetSessionVars().StmtCtx.SetSkipPlanCache(reason) return true } diff --git a/pkg/expression/builtin_other.go b/pkg/expression/builtin_other.go index 7f4a1c35b4501..ed30fee6f3431 100644 --- a/pkg/expression/builtin_other.go +++ b/pkg/expression/builtin_other.go @@ -165,7 +165,7 @@ func (c *inFunctionClass) verifyArgs(ctx sessionctx.Context, args []Expression) case columnType.GetType() == mysql.TypeBit && constant.Value.Kind() == types.KindInt64: if constant.Value.GetInt64() < 0 { if MaybeOverOptimized4PlanCache(ctx, args) { - ctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.Errorf("Bit Column in (%v)", constant.Value.GetInt64())) + ctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.NewNoStackErrorf("Bit Column in (%v)", constant.Value.GetInt64())) } continue } diff --git a/pkg/expression/constant_propagation.go b/pkg/expression/constant_propagation.go index 5898b2b76b4a0..51bb85a7a47e4 100644 --- a/pkg/expression/constant_propagation.go +++ b/pkg/expression/constant_propagation.go @@ -282,7 +282,7 @@ func (s *propConstSolver) propagateColumnEQ() { func (s *propConstSolver) setConds2ConstFalse() { if MaybeOverOptimized4PlanCache(s.ctx, s.conditions) { - s.ctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.New("some parameters may be overwritten when constant propagation")) + s.ctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.NewNoStackError("some parameters may be overwritten when constant propagation")) } s.conditions = []Expression{&Constant{ Value: types.NewDatum(false), @@ -398,7 +398,7 @@ func (s *basePropConstSolver) dealWithPossibleHybridType(col *Column, con *Const return nil, false } if MaybeOverOptimized4PlanCache(s.ctx, []Expression{con}) { - s.ctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.New("Skip plan cache since mutable constant is restored and propagated")) + s.ctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.NewNoStackError("Skip plan cache since mutable constant is restored and propagated")) } switch d.Kind() { case types.KindInt64: diff --git a/pkg/expression/expression.go b/pkg/expression/expression.go index 97c83bf45e033..4cb273d178cee 100644 --- a/pkg/expression/expression.go +++ b/pkg/expression/expression.go @@ -804,7 +804,7 @@ func SplitDNFItems(onExpr Expression) []Expression { // If the Expression is a non-constant value, it means the result is unknown. func EvaluateExprWithNull(ctx sessionctx.Context, schema *Schema, expr Expression) Expression { if MaybeOverOptimized4PlanCache(ctx, []Expression{expr}) { - ctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.New("%v affects null check")) + ctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.NewNoStackError("%v affects null check")) } if ctx.GetSessionVars().StmtCtx.InNullRejectCheck { expr, _ = evaluateExprWithNullInNullRejectCheck(ctx, schema, expr) diff --git a/pkg/planner/core/find_best_task.go b/pkg/planner/core/find_best_task.go index 55247b0979f49..f94a3a996c536 100644 --- a/pkg/planner/core/find_best_task.go +++ b/pkg/planner/core/find_best_task.go @@ -1158,7 +1158,7 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter if len(path.Ranges) == 0 { // We should uncache the tableDual plan. if expression.MaybeOverOptimized4PlanCache(ds.SCtx(), path.AccessConds) { - ds.SCtx().GetSessionVars().StmtCtx.SetSkipPlanCache(errors.Errorf("get a TableDual plan")) + ds.SCtx().GetSessionVars().StmtCtx.SetSkipPlanCache(errors.NewNoStackError("get a TableDual plan")) } dual := PhysicalTableDual{}.Init(ds.SCtx(), ds.StatsInfo(), ds.QueryBlockOffset()) dual.SetSchema(ds.schema) @@ -1236,7 +1236,7 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter // Batch/PointGet plans may be over-optimized, like `a>=1(?) and a<=1(?)` --> `a=1` --> PointGet(a=1). // For safety, prevent these plans from the plan cache here. if !pointGetTask.invalid() && expression.MaybeOverOptimized4PlanCache(ds.SCtx(), candidate.path.AccessConds) && !isSafePointGetPath4PlanCache(ds.SCtx(), candidate.path) { - ds.SCtx().GetSessionVars().StmtCtx.SetSkipPlanCache(errors.New("Batch/PointGet plans may be over-optimized")) + ds.SCtx().GetSessionVars().StmtCtx.SetSkipPlanCache(errors.NewNoStackError("Batch/PointGet plans may be over-optimized")) } appendCandidate(ds, pointGetTask, prop, opt) diff --git a/pkg/planner/core/plan_cache.go b/pkg/planner/core/plan_cache.go index b2a96ceb327a4..fa3833a71c979 100644 --- a/pkg/planner/core/plan_cache.go +++ b/pkg/planner/core/plan_cache.go @@ -164,7 +164,7 @@ func GetPlanFromSessionPlanCache(ctx context.Context, sctx sessionctx.Context, } stmtCtx.UseCache = stmt.StmtCacheable && cacheEnabled if stmt.UncacheableReason != "" { - stmtCtx.ForceSetSkipPlanCache(errors.New(stmt.UncacheableReason)) + stmtCtx.ForceSetSkipPlanCache(errors.NewNoStackError(stmt.UncacheableReason)) } var bindSQL string diff --git a/pkg/planner/core/plan_cache_utils.go b/pkg/planner/core/plan_cache_utils.go index f180833eeae5b..c081936f2d0c8 100644 --- a/pkg/planner/core/plan_cache_utils.go +++ b/pkg/planner/core/plan_cache_utils.go @@ -502,11 +502,11 @@ func GetMatchOpts(sctx sessionctx.Context, is infoschema.InfoSchema, stmt *PlanC if count, isParamMarker := node.Count.(*driver.ParamMarkerExpr); isParamMarker { typeExpected, val := CheckParamTypeInt64orUint64(count) if !typeExpected { - sctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.New("unexpected value after LIMIT")) + sctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.NewNoStackError("unexpected value after LIMIT")) break } if val > MaxCacheableLimitCount { - sctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.New("limit count is too large")) + sctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.NewNoStackError("limit count is too large")) break } limitOffsetAndCount = append(limitOffsetAndCount, val) @@ -516,7 +516,7 @@ func GetMatchOpts(sctx sessionctx.Context, is infoschema.InfoSchema, stmt *PlanC if offset, isParamMarker := node.Offset.(*driver.ParamMarkerExpr); isParamMarker { typeExpected, val := CheckParamTypeInt64orUint64(offset) if !typeExpected { - sctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.New("unexpected value after LIMIT")) + sctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.NewNoStackError("unexpected value after LIMIT")) break } limitOffsetAndCount = append(limitOffsetAndCount, val) diff --git a/pkg/planner/core/rule_predicate_simplification.go b/pkg/planner/core/rule_predicate_simplification.go index 501c78767012e..02274c9782b6b 100644 --- a/pkg/planner/core/rule_predicate_simplification.go +++ b/pkg/planner/core/rule_predicate_simplification.go @@ -132,13 +132,13 @@ func applyPredicateSimplification(sctx sessionctx.Context, predicates []expressi if iCol == jCol { if iType == notEqualPredicate && jType == inListPredicate { predicates[j], specialCase = updateInPredicate(sctx, jthPredicate, ithPredicate) - sctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.New("NE/INList simplification is triggered")) + sctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.NewNoStackError("NE/INList simplification is triggered")) if !specialCase { removeValues = append(removeValues, i) } } else if iType == inListPredicate && jType == notEqualPredicate { predicates[i], specialCase = updateInPredicate(sctx, ithPredicate, jthPredicate) - sctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.New("NE/INList simplification is triggered")) + sctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.NewNoStackError("NE/INList simplification is triggered")) if !specialCase { removeValues = append(removeValues, j) } diff --git a/pkg/planner/optimize.go b/pkg/planner/optimize.go index 5e18c9ed6c28f..013285d93f654 100644 --- a/pkg/planner/optimize.go +++ b/pkg/planner/optimize.go @@ -197,7 +197,7 @@ func Optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is in sessVars.StmtCtx.AddSetVarHintRestore(name, oldV) } if len(sessVars.StmtCtx.StmtHints.SetVars) > 0 { - sctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.Errorf("SET_VAR is used in the SQL")) + sctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.NewNoStackError("SET_VAR is used in the SQL")) } txnManger := sessiontxn.GetTxnManager(sctx) From cfa375063c23d2b53d9ac17bf91f6ca272e11b44 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Fri, 19 Jan 2024 12:49:20 +0800 Subject: [PATCH 2/2] bindinfo: SetSkipPlanCache use nostack errors Signed-off-by: Weizhen Wang --- pkg/expression/constant_propagation.go | 4 ++-- pkg/planner/core/expression_rewriter.go | 2 +- pkg/planner/core/rule_predicate_simplification.go | 4 ++-- pkg/util/ranger/detacher.go | 6 +++--- 4 files changed, 8 insertions(+), 8 deletions(-) diff --git a/pkg/expression/constant_propagation.go b/pkg/expression/constant_propagation.go index 51bb85a7a47e4..5898b2b76b4a0 100644 --- a/pkg/expression/constant_propagation.go +++ b/pkg/expression/constant_propagation.go @@ -282,7 +282,7 @@ func (s *propConstSolver) propagateColumnEQ() { func (s *propConstSolver) setConds2ConstFalse() { if MaybeOverOptimized4PlanCache(s.ctx, s.conditions) { - s.ctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.NewNoStackError("some parameters may be overwritten when constant propagation")) + s.ctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.New("some parameters may be overwritten when constant propagation")) } s.conditions = []Expression{&Constant{ Value: types.NewDatum(false), @@ -398,7 +398,7 @@ func (s *basePropConstSolver) dealWithPossibleHybridType(col *Column, con *Const return nil, false } if MaybeOverOptimized4PlanCache(s.ctx, []Expression{con}) { - s.ctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.NewNoStackError("Skip plan cache since mutable constant is restored and propagated")) + s.ctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.New("Skip plan cache since mutable constant is restored and propagated")) } switch d.Kind() { case types.KindInt64: diff --git a/pkg/planner/core/expression_rewriter.go b/pkg/planner/core/expression_rewriter.go index 4933104015590..259226a4513c3 100644 --- a/pkg/planner/core/expression_rewriter.go +++ b/pkg/planner/core/expression_rewriter.go @@ -1710,7 +1710,7 @@ func (er *expressionRewriter) inToExpression(lLen int, not bool, tp *types.Field if c.GetType().EvalType() == types.ETInt { continue // no need to refine it } - er.sctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.Errorf("'%v' may be converted to INT", c.String())) + er.sctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.NewNoStackErrorf("'%v' may be converted to INT", c.String())) if err := expression.RemoveMutableConst(er.sctx, []expression.Expression{c}); err != nil { er.err = err return diff --git a/pkg/planner/core/rule_predicate_simplification.go b/pkg/planner/core/rule_predicate_simplification.go index 02274c9782b6b..501c78767012e 100644 --- a/pkg/planner/core/rule_predicate_simplification.go +++ b/pkg/planner/core/rule_predicate_simplification.go @@ -132,13 +132,13 @@ func applyPredicateSimplification(sctx sessionctx.Context, predicates []expressi if iCol == jCol { if iType == notEqualPredicate && jType == inListPredicate { predicates[j], specialCase = updateInPredicate(sctx, jthPredicate, ithPredicate) - sctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.NewNoStackError("NE/INList simplification is triggered")) + sctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.New("NE/INList simplification is triggered")) if !specialCase { removeValues = append(removeValues, i) } } else if iType == inListPredicate && jType == notEqualPredicate { predicates[i], specialCase = updateInPredicate(sctx, ithPredicate, jthPredicate) - sctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.NewNoStackError("NE/INList simplification is triggered")) + sctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.New("NE/INList simplification is triggered")) if !specialCase { removeValues = append(removeValues, j) } diff --git a/pkg/util/ranger/detacher.go b/pkg/util/ranger/detacher.go index 37ff8ab1ddc85..b1b67222c7765 100644 --- a/pkg/util/ranger/detacher.go +++ b/pkg/util/ranger/detacher.go @@ -640,7 +640,7 @@ func ExtractEqAndInCondition(sctx sessionctx.Context, conditions []expression.Ex if len(points[offset]) == 0 { // Early termination if false expression found if expression.MaybeOverOptimized4PlanCache(sctx, conditions) { // `a>@x and a<@y` --> `invalid-range if @x>=@y` - sctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.Errorf("some parameters may be overwritten")) + sctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.NewNoStackErrorf("some parameters may be overwritten")) } return nil, nil, nil, nil, true } @@ -664,7 +664,7 @@ func ExtractEqAndInCondition(sctx sessionctx.Context, conditions []expression.Ex } else if len(points[i]) == 0 { // Early termination if false expression found if expression.MaybeOverOptimized4PlanCache(sctx, conditions) { // `a>@x and a<@y` --> `invalid-range if @x>=@y` - sctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.Errorf("some parameters may be overwritten")) + sctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.NewNoStackErrorf("some parameters may be overwritten")) } return nil, nil, nil, nil, true } else { @@ -678,7 +678,7 @@ func ExtractEqAndInCondition(sctx sessionctx.Context, conditions []expression.Ex } if expression.MaybeOverOptimized4PlanCache(sctx, conditions) { // `a=@x and a=@y` --> `a=@x if @x==@y` - sctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.Errorf("some parameters may be overwritten")) + sctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.NewNoStackErrorf("some parameters may be overwritten")) } } }