Skip to content

Commit

Permalink
bindinfo: SetSkipPlanCache use nostack errors (#50585)
Browse files Browse the repository at this point in the history
close #49291
  • Loading branch information
hawkingrei authored Jan 19, 2024
1 parent 1442f07 commit bfa10bd
Show file tree
Hide file tree
Showing 10 changed files with 16 additions and 16 deletions.
2 changes: 1 addition & 1 deletion pkg/executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -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())
Expand Down
4 changes: 2 additions & 2 deletions pkg/expression/builtin_compare.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
Expand All @@ -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
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/expression/builtin_other.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/expression/expression.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
2 changes: 1 addition & 1 deletion pkg/planner/core/expression_rewriter.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
4 changes: 2 additions & 2 deletions pkg/planner/core/find_best_task.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down Expand Up @@ -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)
Expand Down
2 changes: 1 addition & 1 deletion pkg/planner/core/plan_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
6 changes: 3 additions & 3 deletions pkg/planner/core/plan_cache_utils.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand All @@ -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)
Expand Down
2 changes: 1 addition & 1 deletion pkg/planner/optimize.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
6 changes: 3 additions & 3 deletions pkg/util/ranger/detacher.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
Expand All @@ -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 {
Expand All @@ -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"))
}
}
}
Expand Down

0 comments on commit bfa10bd

Please sign in to comment.