diff --git a/DEPS.bzl b/DEPS.bzl index 0344082d6e12f..4b5c656182dd3 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -3392,8 +3392,8 @@ def go_deps(): name = "com_github_pingcap_errors", build_file_proto_mode = "disable_global", importpath = "github.com/pingcap/errors", - sum = "h1:m5ZsBa5o/0CkzZXfXLaThzKuR85SnHHetqBCpzQ30h8=", - version = "v0.11.5-0.20221009092201-b66cddb77c32", + sum = "h1:yqyP+k0mgRPpXJQDOCrtaG2YZym0ZDD+vt5JzlBUkrw=", + version = "v0.11.5-0.20231212100244-799fae176cfb", ) go_repository( name = "com_github_pingcap_failpoint", diff --git a/go.mod b/go.mod index 413a6b174a56d..15ccd9fbd6d3f 100644 --- a/go.mod +++ b/go.mod @@ -70,7 +70,7 @@ require ( github.com/opentracing/opentracing-go v1.2.0 github.com/phayes/freeport v0.0.0-20180830031419-95f893ade6f2 github.com/pingcap/badger v1.5.1-0.20230103063557-828f39b09b6d - github.com/pingcap/errors v0.11.5-0.20221009092201-b66cddb77c32 + github.com/pingcap/errors v0.11.5-0.20231212100244-799fae176cfb github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059 github.com/pingcap/kvproto v0.0.0-20231011074246-fa00d2b03372 diff --git a/go.sum b/go.sum index 0245d880aeeb2..ad5883c8e337c 100644 --- a/go.sum +++ b/go.sum @@ -780,8 +780,8 @@ github.com/pingcap/check v0.0.0-20200212061837-5e12011dc712/go.mod h1:PYMCGwN0JH github.com/pingcap/errors v0.11.0/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= github.com/pingcap/errors v0.11.4/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= github.com/pingcap/errors v0.11.5-0.20190809092503-95897b64e011/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= -github.com/pingcap/errors v0.11.5-0.20221009092201-b66cddb77c32 h1:m5ZsBa5o/0CkzZXfXLaThzKuR85SnHHetqBCpzQ30h8= -github.com/pingcap/errors v0.11.5-0.20221009092201-b66cddb77c32/go.mod h1:X2r9ueLEUZgtx2cIogM0v4Zj5uvvzhuuiu7Pn8HzMPg= +github.com/pingcap/errors v0.11.5-0.20231212100244-799fae176cfb h1:yqyP+k0mgRPpXJQDOCrtaG2YZym0ZDD+vt5JzlBUkrw= +github.com/pingcap/errors v0.11.5-0.20231212100244-799fae176cfb/go.mod h1:X2r9ueLEUZgtx2cIogM0v4Zj5uvvzhuuiu7Pn8HzMPg= github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c h1:CgbKAHto5CQgWM9fSBIvaxsJHuGP0uM74HXtv3MyyGQ= github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c/go.mod h1:4qGtCB0QK0wBzKtFEGDhxXnSnbQApw1gc9siScUl8ew= github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059 h1:Pe2LbxRmbTfAoKJ65bZLmhahmvHm7n9DUxGRQT00208= diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index 3ca2e25f10afd..fcda463cd40b4 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -860,12 +860,12 @@ func (e *Explain) RenderResult() error { // output cost formula and factor costs through warning under model ver2 and true_card_cost mode for cost calibration. cost, _ := pp.getPlanCostVer2(property.RootTaskType, NewDefaultPlanCostOption()) trace := cost.trace - pp.SCtx().GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("cost formula: %v", trace.formula)) + pp.SCtx().GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackErrorf("cost formula: %v", trace.formula)) data, err := json.Marshal(trace.factorCosts) if err != nil { - pp.SCtx().GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("marshal factor costs error %v", err)) + pp.SCtx().GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackErrorf("marshal factor costs error %v", err)) } - pp.SCtx().GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("factor costs: %v", string(data))) + pp.SCtx().GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackErrorf("factor costs: %v", string(data))) // output cost factor weights for cost calibration factors := defaultVer2Factors.tolist() @@ -876,13 +876,13 @@ func (e *Explain) RenderResult() error { } } if wstr, err := json.Marshal(weights); err != nil { - pp.SCtx().GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("marshal weights error %v", err)) + pp.SCtx().GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackErrorf("marshal weights error %v", err)) } else { - pp.SCtx().GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("factor weights: %v", string(wstr))) + pp.SCtx().GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackErrorf("factor weights: %v", string(wstr))) } } } else { - e.SCtx().GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("'explain format=true_card_cost' cannot support this plan")) + e.SCtx().GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackError("'explain format=true_card_cost' cannot support this plan")) } } diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index d59ada0a22b3c..21b388778ca08 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -234,7 +234,7 @@ func (p *LogicalJoin) GetMergeJoin(prop *property.PhysicalProperty, schema *expr if p.preferJoinType&preferMergeJoin == 0 { return nil } - p.SCtx().GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack( + p.SCtx().GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen( "Some MERGE_JOIN and NO_MERGE_JOIN hints conflict, NO_MERGE_JOIN is ignored")) } @@ -399,7 +399,7 @@ func (p *LogicalJoin) getHashJoins(prop *property.PhysicalProperty) (joins []Phy forceLeftToBuild := ((p.preferJoinType & preferLeftAsHJBuild) > 0) || ((p.preferJoinType & preferRightAsHJProbe) > 0) forceRightToBuild := ((p.preferJoinType & preferRightAsHJBuild) > 0) || ((p.preferJoinType & preferLeftAsHJProbe) > 0) if forceLeftToBuild && forceRightToBuild { - p.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("Some HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are conflicts, please check the hints")) + p.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen("Some HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are conflicts, please check the hints")) forceLeftToBuild = false forceRightToBuild = false } @@ -410,7 +410,7 @@ func (p *LogicalJoin) getHashJoins(prop *property.PhysicalProperty) (joins []Phy joins = append(joins, p.getHashJoin(prop, 1, false)) if forceLeftToBuild || forceRightToBuild { // Do not support specifying the build side. - p.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack(fmt.Sprintf("We can't use the HASH_JOIN_BUILD or HASH_JOIN_PROBE hint for %s, please check the hint", p.JoinType))) + p.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen(fmt.Sprintf("We can't use the HASH_JOIN_BUILD or HASH_JOIN_PROBE hint for %s, please check the hint", p.JoinType))) forceLeftToBuild = false forceRightToBuild = false } @@ -455,7 +455,7 @@ func (p *LogicalJoin) getHashJoins(prop *property.PhysicalProperty) (joins []Phy if !forced && p.shouldSkipHashJoin() { return nil, false } else if forced && p.shouldSkipHashJoin() { - p.SCtx().GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack( + p.SCtx().GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen( "A conflict between the HASH_JOIN hint and the NO_HASH_JOIN hint, " + "or the tidb_opt_enable_hash_join system variable, the HASH_JOIN hint will take precedence.")) } @@ -2037,13 +2037,13 @@ func (p *LogicalJoin) tryToGetIndexJoin(prop *property.PhysicalProperty) (indexJ // Handle hints conflict first. stmtCtx := p.SCtx().GetSessionVars().StmtCtx if p.preferAny(preferLeftAsINLJInner, preferRightAsINLJInner) && p.preferAny(preferNoIndexJoin) { - stmtCtx.AppendWarning(ErrInternal.GenWithStack("Some INL_JOIN and NO_INDEX_JOIN hints conflict, NO_INDEX_JOIN may be ignored")) + stmtCtx.AppendWarning(ErrInternal.FastGen("Some INL_JOIN and NO_INDEX_JOIN hints conflict, NO_INDEX_JOIN may be ignored")) } if p.preferAny(preferLeftAsINLHJInner, preferRightAsINLHJInner) && p.preferAny(preferNoIndexHashJoin) { - stmtCtx.AppendWarning(ErrInternal.GenWithStack("Some INL_HASH_JOIN and NO_INDEX_HASH_JOIN hints conflict, NO_INDEX_HASH_JOIN may be ignored")) + stmtCtx.AppendWarning(ErrInternal.FastGen("Some INL_HASH_JOIN and NO_INDEX_HASH_JOIN hints conflict, NO_INDEX_HASH_JOIN may be ignored")) } if p.preferAny(preferLeftAsINLMJInner, preferRightAsINLMJInner) && p.preferAny(preferNoIndexMergeJoin) { - stmtCtx.AppendWarning(ErrInternal.GenWithStack("Some INL_MERGE_JOIN and NO_INDEX_MERGE_JOIN hints conflict, NO_INDEX_MERGE_JOIN may be ignored")) + stmtCtx.AppendWarning(ErrInternal.FastGen("Some INL_MERGE_JOIN and NO_INDEX_MERGE_JOIN hints conflict, NO_INDEX_MERGE_JOIN may be ignored")) } candidates, canForced = p.handleForceIndexJoinHints(prop, candidates) @@ -2123,7 +2123,7 @@ func (p *LogicalJoin) handleForceIndexJoinHints(prop *property.PhysicalProperty, errMsg += " without column equal ON condition" } // Generate warning message to client. - p.SCtx().GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack(errMsg)) + p.SCtx().GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen(errMsg)) } return candidates, false } @@ -2268,7 +2268,7 @@ func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]P if !isJoinHintSupportedInMPPMode(p.preferJoinType) { if hasMPPJoinHints(p.preferJoinType) { // If there are MPP hints but has some conflicts join method hints, all the join hints are invalid. - p.SCtx().GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("The MPP join hints are in conflict, and you can only specify join method hints that are currently supported by MPP mode now")) + p.SCtx().GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen("The MPP join hints are in conflict, and you can only specify join method hints that are currently supported by MPP mode now")) p.preferJoinType = 0 } else { // If there are no MPP hints but has some conflicts join method hints, the MPP mode will be blocked. @@ -2313,7 +2313,7 @@ func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]P hasMppHints = true } if hasMppHints { - warning := ErrInternal.GenWithStack(errMsg) + warning := ErrInternal.FastGen(errMsg) p.ctx.GetSessionVars().StmtCtx.AppendWarning(warning) } } @@ -2435,7 +2435,7 @@ func (p *LogicalJoin) tryToGetMppHashJoin(prop *property.PhysicalProperty, useBC forceLeftToBuild := ((p.preferJoinType & preferLeftAsHJBuild) > 0) || ((p.preferJoinType & preferRightAsHJProbe) > 0) forceRightToBuild := ((p.preferJoinType & preferRightAsHJBuild) > 0) || ((p.preferJoinType & preferLeftAsHJProbe) > 0) if forceLeftToBuild && forceRightToBuild { - p.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("Some HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are conflicts, please check the hints")) + p.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen("Some HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are conflicts, please check the hints")) forceLeftToBuild = false forceRightToBuild = false } @@ -2482,7 +2482,7 @@ func (p *LogicalJoin) tryToGetMppHashJoin(prop *property.PhysicalProperty, useBC if !match { if fixedBuildSide { // A warning will be generated if the build side is fixed, but we attempt to change it using the hint. - p.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("Some HASH_JOIN_BUILD and HASH_JOIN_PROBE hints cannot be utilized for MPP joins, please check the hints")) + p.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen("Some HASH_JOIN_BUILD and HASH_JOIN_PROBE hints cannot be utilized for MPP joins, please check the hints")) } else { // The HASH_JOIN_BUILD OR HASH_JOIN_PROBE hints can take effective. preferredBuildIndex = 1 - preferredBuildIndex @@ -2636,8 +2636,7 @@ func pushLimitOrTopNForcibly(p LogicalPlan) bool { } if *preferPushDown { errMsg := "Optimizer Hint LIMIT_TO_COP is inapplicable" - warning := ErrInternal.GenWithStack(errMsg) - p.SCtx().GetSessionVars().StmtCtx.AppendWarning(warning) + p.SCtx().GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen(errMsg)) *preferPushDown = false } } @@ -2724,7 +2723,7 @@ func (la *LogicalApply) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([ return nil, true, nil } if !prop.IsSortItemEmpty() && la.SCtx().GetSessionVars().EnableParallelApply { - la.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("Parallel Apply rejects the possible order properties of its outer child currently")) + la.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackErrorf("Parallel Apply rejects the possible order properties of its outer child currently")) return nil, true, nil } disableAggPushDownToCop(la.children[0]) @@ -3126,7 +3125,7 @@ func (la *LogicalAggregation) checkCanPushDownToMPP() bool { } } if hasUnsupportedDistinct { - warnErr := errors.New("Aggregation can not be pushed to storage layer in mpp mode because it contains agg function with distinct") + warnErr := errors.NewNoStackError("Aggregation can not be pushed to storage layer in mpp mode because it contains agg function with distinct") if la.ctx.GetSessionVars().StmtCtx.InExplainStmt { la.ctx.GetSessionVars().StmtCtx.AppendWarning(warnErr) } else { @@ -3295,7 +3294,7 @@ func (la *LogicalAggregation) getHashAggs(prop *property.PhysicalProperty) []Phy hasMppHints = true } if hasMppHints { - warning := ErrInternal.GenWithStack(errMsg) + warning := ErrInternal.FastGen(errMsg) la.ctx.GetSessionVars().StmtCtx.AppendWarning(warning) } } @@ -3325,7 +3324,7 @@ func (la *LogicalAggregation) ResetHintIfConflicted() (preferHash bool, preferSt preferStream = (la.aggHints.preferAggType & preferStreamAgg) > 0 if preferHash && preferStream { errMsg := "Optimizer aggregation hints are conflicted" - warning := ErrInternal.GenWithStack(errMsg) + warning := ErrInternal.FastGen(errMsg) la.ctx.GetSessionVars().StmtCtx.AppendWarning(warning) la.aggHints.preferAggType = 0 preferHash, preferStream = false, false @@ -3337,7 +3336,7 @@ func (la *LogicalAggregation) exhaustPhysicalPlans(prop *property.PhysicalProper if la.aggHints.preferAggToCop { if !la.canPushToCop(kv.TiKV) { errMsg := "Optimizer Hint AGG_TO_COP is inapplicable" - warning := ErrInternal.GenWithStack(errMsg) + warning := ErrInternal.FastGen(errMsg) la.ctx.GetSessionVars().StmtCtx.AppendWarning(warning) la.aggHints.preferAggToCop = false } @@ -3359,7 +3358,7 @@ func (la *LogicalAggregation) exhaustPhysicalPlans(prop *property.PhysicalProper if streamAggs == nil && preferStream && !prop.IsSortItemEmpty() { errMsg := "Optimizer Hint STREAM_AGG is inapplicable" - warning := ErrInternal.GenWithStack(errMsg) + warning := ErrInternal.FastGen(errMsg) la.ctx.GetSessionVars().StmtCtx.AppendWarning(warning) } diff --git a/planner/core/expression_rewriter.go b/planner/core/expression_rewriter.go index 41e77c48b89f8..2594c4e1f14e8 100644 --- a/planner/core/expression_rewriter.go +++ b/planner/core/expression_rewriter.go @@ -538,7 +538,7 @@ func (er *expressionRewriter) handleCompareSubquery(ctx context.Context, v *ast. noDecorrelate := hintFlags&HintFlagNoDecorrelate > 0 if noDecorrelate && len(extractCorColumnsBySchema4LogicalPlan(np, er.p.Schema())) == 0 { - er.sctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack( + er.sctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen( "NO_DECORRELATE() is inapplicable because there are no correlated columns.")) noDecorrelate = false } @@ -841,13 +841,13 @@ func (er *expressionRewriter) handleExistSubquery(ctx context.Context, v *ast.Ex noDecorrelate := hintFlags&HintFlagNoDecorrelate > 0 if noDecorrelate && len(extractCorColumnsBySchema4LogicalPlan(np, er.p.Schema())) == 0 { - er.sctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack( + er.sctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen( "NO_DECORRELATE() is inapplicable because there are no correlated columns.")) noDecorrelate = false } semiJoinRewrite := hintFlags&HintFlagSemiJoinRewrite > 0 if semiJoinRewrite && noDecorrelate { - er.sctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack( + er.sctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen( "NO_DECORRELATE() and SEMI_JOIN_REWRITE() are in conflict. Both will be ineffective.")) noDecorrelate = false semiJoinRewrite = false @@ -988,7 +988,7 @@ func (er *expressionRewriter) handleInSubquery(ctx context.Context, v *ast.Patte noDecorrelate := hintFlags&HintFlagNoDecorrelate > 0 corCols := extractCorColumnsBySchema4LogicalPlan(np, er.p.Schema()) if len(corCols) == 0 && noDecorrelate { - er.sctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack( + er.sctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen( "NO_DECORRELATE() is inapplicable because there are no correlated columns.")) noDecorrelate = false } @@ -1048,7 +1048,7 @@ func (er *expressionRewriter) handleScalarSubquery(ctx context.Context, v *ast.S noDecorrelate := hintFlags&HintFlagNoDecorrelate > 0 if noDecorrelate && len(extractCorColumnsBySchema4LogicalPlan(np, er.p.Schema())) == 0 { - er.sctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack( + er.sctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen( "NO_DECORRELATE() is inapplicable because there are no correlated columns.")) noDecorrelate = false } @@ -1396,7 +1396,7 @@ func (er *expressionRewriter) rewriteVariable(v *ast.VariableExpr) { } if sysVar.IsNoop && !variable.EnableNoopVariables.Load() { // The variable does nothing, append a warning to the statement output. - sessionVars.StmtCtx.AppendWarning(ErrGettingNoopVariable.GenWithStackByArgs(sysVar.Name)) + sessionVars.StmtCtx.AppendWarning(ErrGettingNoopVariable.FastGenByArgs(sysVar.Name)) } if sem.IsEnabled() && sem.IsInvisibleSysVar(sysVar.Name) { err := ErrSpecificAccessDenied.GenWithStackByArgs("RESTRICTED_VARIABLES_ADMIN") @@ -2180,7 +2180,7 @@ func decodeKeyFromString(ctx sessionctx.Context, s string) string { sc := ctx.GetSessionVars().StmtCtx key, err := hex.DecodeString(s) if err != nil { - sc.AppendWarning(errors.Errorf("invalid key: %X", key)) + sc.AppendWarning(errors.NewNoStackErrorf("invalid key: %X", key)) return s } // Auto decode byte if needed. @@ -2190,17 +2190,17 @@ func decodeKeyFromString(ctx sessionctx.Context, s string) string { } tableID := tablecodec.DecodeTableID(key) if tableID == 0 { - sc.AppendWarning(errors.Errorf("invalid key: %X", key)) + sc.AppendWarning(errors.NewNoStackErrorf("invalid key: %X", key)) return s } dm := domain.GetDomain(ctx) if dm == nil { - sc.AppendWarning(errors.Errorf("domain not found when decoding key: %X", key)) + sc.AppendWarning(errors.NewNoStackErrorf("domain not found when decoding key: %X", key)) return s } is := dm.InfoSchema() if is == nil { - sc.AppendWarning(errors.Errorf("infoschema not found when decoding key: %X", key)) + sc.AppendWarning(errors.NewNoStackErrorf("infoschema not found when decoding key: %X", key)) return s } tbl, _ := is.TableByID(tableID) @@ -2230,7 +2230,7 @@ func decodeKeyFromString(ctx sessionctx.Context, s string) string { } return ret } - sc.AppendWarning(errors.Errorf("invalid key: %X", key)) + sc.AppendWarning(errors.NewNoStackErrorf("invalid key: %X", key)) return s } diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index ed34cf8057fb1..ac2fefbe4115e 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -950,7 +950,7 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter pruningInfo := ds.getPruningInfo(candidates, prop) defer func() { if err == nil && t != nil && !t.invalid() && pruningInfo != "" { - warnErr := errors.New(pruningInfo) + warnErr := errors.NewNoStackError(pruningInfo) if ds.ctx.GetSessionVars().StmtCtx.InVerboseExplain { ds.ctx.GetSessionVars().StmtCtx.AppendNote(warnErr) } else { diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index ffa2d6de0a67c..8b4a551c42e9f 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -741,7 +741,7 @@ func (p *LogicalJoin) setPreferredJoinTypeAndOrder(hintInfo *tableHintInfo) { } if hasConflict { errMsg := "Join hints are conflict, you can only specify one type of join" - warning := ErrInternal.GenWithStack(errMsg) + warning := ErrInternal.FastGen(errMsg) p.ctx.GetSessionVars().StmtCtx.AppendWarning(warning) p.preferJoinType = 0 } @@ -811,7 +811,7 @@ func (p *LogicalJoin) setPreferredJoinType() { p.preferJoinType = setPreferredJoinTypeFromOneSide(p.leftPreferJoinType, true) | setPreferredJoinTypeFromOneSide(p.rightPreferJoinType, false) if containDifferentJoinTypes(p.preferJoinType) { errMsg := "Join hints conflict after join reorder phase, you can only specify one type of join" - warning := ErrInternal.GenWithStack(errMsg) + warning := ErrInternal.FastGen(errMsg) p.ctx.GetSessionVars().StmtCtx.AppendWarning(warning) p.preferJoinType = 0 } @@ -840,7 +840,7 @@ func (ds *DataSource) setPreferredStoreType(hintInfo *tableHintInfo) { errMsg := fmt.Sprintf("No available path for table %s.%s with the store type %s of the hint /*+ read_from_storage */, "+ "please check the status of the table replica and variable value of tidb_isolation_read_engines(%v)", ds.DBName.O, ds.table.Meta().Name.O, kv.TiKV.Name(), ds.ctx.GetSessionVars().GetIsolationReadEngines()) - warning := ErrInternal.GenWithStack(errMsg) + warning := ErrInternal.FastGen(errMsg) ds.ctx.GetSessionVars().StmtCtx.AppendWarning(warning) } else { ds.ctx.GetSessionVars().RaiseWarningWhenMPPEnforced("MPP mode may be blocked because you have set a hint to read table `" + hintTbl.tblName.O + "` from TiKV.") @@ -852,7 +852,7 @@ func (ds *DataSource) setPreferredStoreType(hintInfo *tableHintInfo) { if ds.preferStoreType != 0 { errMsg := fmt.Sprintf("Storage hints are conflict, you can only specify one storage type of table %s.%s", alias.dbName.L, alias.tblName.L) - warning := ErrInternal.GenWithStack(errMsg) + warning := ErrInternal.FastGen(errMsg) ds.ctx.GetSessionVars().StmtCtx.AppendWarning(warning) ds.preferStoreType = 0 return @@ -868,7 +868,7 @@ func (ds *DataSource) setPreferredStoreType(hintInfo *tableHintInfo) { errMsg := fmt.Sprintf("No available path for table %s.%s with the store type %s of the hint /*+ read_from_storage */, "+ "please check the status of the table replica and variable value of tidb_isolation_read_engines(%v)", ds.DBName.O, ds.table.Meta().Name.O, kv.TiFlash.Name(), ds.ctx.GetSessionVars().GetIsolationReadEngines()) - warning := ErrInternal.GenWithStack(errMsg) + warning := ErrInternal.FastGen(errMsg) ds.ctx.GetSessionVars().StmtCtx.AppendWarning(warning) } } @@ -3767,7 +3767,7 @@ func (b *PlanBuilder) pushHintWithoutTableWarning(hint *ast.TableOptimizerHint) return } errMsg := fmt.Sprintf("Hint %s is inapplicable. Please specify the table names in the arguments.", sb.String()) - b.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack(errMsg)) + b.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen(errMsg)) } func (b *PlanBuilder) pushTableHints(hints []*ast.TableOptimizerHint, currentLevel int) { @@ -3893,7 +3893,7 @@ func (b *PlanBuilder) pushTableHints(hints []*ast.TableOptimizerHint, currentLev limitHints.preferLimitToCop = true case HintMerge: if hint.Tables != nil { - b.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("The MERGE hint is not used correctly, maybe it inputs a table name.")) + b.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen("The MERGE hint is not used correctly, maybe it inputs a table name.")) continue } MergeHints.preferMerge = true @@ -3904,13 +3904,13 @@ func (b *PlanBuilder) pushTableHints(hints []*ast.TableOptimizerHint, currentLev leadingHintCnt++ case HintSemiJoinRewrite: if b.subQueryCtx != handlingExistsSubquery { - b.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("The SEMI_JOIN_REWRITE hint is not used correctly, maybe it's not in a subquery or the subquery is not EXISTS clause.")) + b.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen("The SEMI_JOIN_REWRITE hint is not used correctly, maybe it's not in a subquery or the subquery is not EXISTS clause.")) continue } b.subQueryHintFlags |= HintFlagSemiJoinRewrite case HintNoDecorrelate: if b.subQueryCtx == notHandlingSubquery { - b.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("NO_DECORRELATE() is inapplicable because it's not in an IN subquery, an EXISTS subquery, an ANY/ALL/SOME subquery or a scalar subquery.")) + b.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen("NO_DECORRELATE() is inapplicable because it's not in an IN subquery, an EXISTS subquery, an ANY/ALL/SOME subquery or a scalar subquery.")) continue } b.subQueryHintFlags |= HintFlagNoDecorrelate @@ -3922,9 +3922,9 @@ func (b *PlanBuilder) pushTableHints(hints []*ast.TableOptimizerHint, currentLev // If there are more leading hints or the straight_join hint existes, all leading hints will be invalid. leadingJoinOrder = leadingJoinOrder[:0] if leadingHintCnt > 1 { - b.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid")) + b.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen("We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid")) } else if b.ctx.GetSessionVars().StmtCtx.StraightJoinOrder { - b.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("We can only use the straight_join hint, when we use the leading hint and straight_join hint at the same time, all leading hints will be invalid")) + b.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen("We can only use the straight_join hint, when we use the leading hint and straight_join hint at the same time, all leading hints will be invalid")) } } b.tableHintInfo = append(b.tableHintInfo, tableHintInfo{ @@ -3990,7 +3990,7 @@ func (b *PlanBuilder) appendUnmatchedIndexHintWarning(indexHints []indexHintInfo hint.dbName, hint.tblName, ) - b.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack(errMsg)) + b.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen(errMsg)) } } } @@ -6812,7 +6812,7 @@ func (b *PlanBuilder) handleDefaultFrame(spec *ast.WindowSpec, windowFuncName st // For functions that operate on the entire partition, the frame clause will be ignored. if spec.Frame != nil { specName := spec.Name.O - b.ctx.GetSessionVars().StmtCtx.AppendNote(ErrWindowFunctionIgnoresFrame.GenWithStackByArgs(windowFuncName, getWindowName(specName))) + b.ctx.GetSessionVars().StmtCtx.AppendNote(ErrWindowFunctionIgnoresFrame.FastGenByArgs(windowFuncName, getWindowName(specName))) newSpec.Frame = nil updated = true } diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index c8119c90b831b..44a68a0c3f66f 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -570,7 +570,7 @@ func tryEnableLateMaterialization(sctx sessionctx.Context, plan PhysicalPlan) { } if sctx.GetSessionVars().EnableLateMaterialization && sctx.GetSessionVars().TiFlashFastScan { sc := sctx.GetSessionVars().StmtCtx - sc.AppendWarning(errors.New("FastScan is not compatible with late materialization, late materialization is disabled")) + sc.AppendWarning(errors.NewNoStackError("FastScan is not compatible with late materialization, late materialization is disabled")) } } @@ -1063,7 +1063,7 @@ func enableParallelApply(sctx sessionctx.Context, plan PhysicalPlan) PhysicalPla if noOrder && supportClone { apply.Concurrency = sctx.GetSessionVars().ExecutorConcurrency } else { - sctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("Some apply operators can not be executed in parallel")) + sctx.GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackErrorf("Some apply operators can not be executed in parallel")) } // because of the limitation 3, we cannot parallelize Apply operators in this Apply's inner size, @@ -1160,7 +1160,7 @@ func physicalOptimize(logic LogicalPlan, planCounter *PlanCounterTp) (plan Physi return nil, 0, err } if *planCounter > 0 { - logic.SCtx().GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("The parameter of nth_plan() is out of range")) + logic.SCtx().GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackErrorf("The parameter of nth_plan() is out of range")) } if t.invalid() { return nil, 0, ErrInternal.GenWithStackByArgs("Can't find a proper physical plan for this query") diff --git a/planner/core/plan_cache.go b/planner/core/plan_cache.go index b168506a46787..f468f39568e72 100644 --- a/planner/core/plan_cache.go +++ b/planner/core/plan_cache.go @@ -353,7 +353,7 @@ func RebuildPlan4CachedPlan(p Plan) (ok bool) { sc.InPreparedPlanBuilding = true defer func() { sc.InPreparedPlanBuilding = false }() if err := rebuildRange(p); err != nil { - sc.AppendWarning(errors.Errorf("skip plan-cache: plan rebuild failed, %s", err.Error())) + sc.AppendWarning(errors.NewNoStackErrorf("skip plan-cache: plan rebuild failed, %s", err.Error())) return false // fail to rebuild ranges } if !sc.UseCache { diff --git a/planner/core/plan_cache_utils.go b/planner/core/plan_cache_utils.go index 39fa8c6b3a9fb..e427a601b21fd 100644 --- a/planner/core/plan_cache_utils.go +++ b/planner/core/plan_cache_utils.go @@ -142,7 +142,7 @@ func GeneratePlanCacheStmtWithAST(ctx context.Context, sctx sessionctx.Context, cacheable = true // it is already checked here } if !cacheable { - sctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("skip prepared plan-cache: " + reason)) + sctx.GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackErrorf("skip prepared plan-cache: " + reason)) } selectStmtNode, normalizedSQL4PC, digest4PC, err = ExtractSelectAndNormalizeDigest(paramStmt, vars.CurrentDB) if err != nil || selectStmtNode == nil { diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 803c424fdf5ea..97cbd8ae284d4 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1456,7 +1456,7 @@ func getPossibleAccessPaths(ctx sessionctx.Context, tableHints *tableHintInfo, i for _, idxName := range hint.IndexNames { path := getPathByIndexName(publicPaths, idxName, tblInfo) if path == nil { - err := ErrKeyDoesNotExist.GenWithStackByArgs(idxName, tblInfo.Name) + err := ErrKeyDoesNotExist.FastGenByArgs(idxName, tblInfo.Name) // if hint is from comment-style sql hints, we should throw a warning instead of error. if i < indexHintsLen { return nil, err @@ -2263,7 +2263,7 @@ func (b *PlanBuilder) getPredicateColumns(tbl *ast.TableName, cols *calcOnceMap) return nil, err } if len(colList) == 0 { - b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("No predicate column has been collected yet for table %s.%s so all columns are analyzed", tbl.Schema.L, tbl.Name.L)) + b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackErrorf("No predicate column has been collected yet for table %s.%s so all columns are analyzed", tbl.Schema.L, tbl.Name.L)) for _, colInfo := range tblInfo.Columns { cols.data[colInfo.ID] = struct{}{} } @@ -2300,7 +2300,7 @@ func (b *PlanBuilder) getFullAnalyzeColumnsInfo( warning bool, ) ([]*model.ColumnInfo, []*model.ColumnInfo, error) { if mustAllColumns && warning && (columnChoice == model.PredicateColumns || columnChoice == model.ColumnList) { - b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("Table %s.%s has version 1 statistics so all the columns must be analyzed to overwrite the current statistics", tbl.Schema.L, tbl.Name.L)) + b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackErrorf("Table %s.%s has version 1 statistics so all the columns must be analyzed to overwrite the current statistics", tbl.Schema.L, tbl.Name.L)) } colSet2colList := func(colSet map[int64]struct{}) []*model.ColumnInfo { colList := make([]*model.ColumnInfo, 0, len(colSet)) @@ -2357,7 +2357,7 @@ func (b *PlanBuilder) getFullAnalyzeColumnsInfo( missingNames = append(missingNames, col.Name.O) } } - b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("Columns %s are missing in ANALYZE but their stats are needed for calculating stats for indexes/primary key/extended stats", strings.Join(missingNames, ","))) + b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackErrorf("Columns %s are missing in ANALYZE but their stats are needed for calculating stats for indexes/primary key/extended stats", strings.Join(missingNames, ","))) } } for colID := range mustAnalyzed { @@ -2394,7 +2394,7 @@ func getModifiedIndexesInfoForAnalyze(sctx sessionctx.Context, tblInfo *model.Ta continue } if originIdx.MVIndex { - sctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("analyzing multi-valued indexes is not supported, skip %s", originIdx.Name.L)) + sctx.GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackErrorf("analyzing multi-valued indexes is not supported, skip %s", originIdx.Name.L)) continue } if allColumns { @@ -2424,7 +2424,7 @@ func (b *PlanBuilder) buildAnalyzeFullSamplingTask( rsOptionsMap map[int64]V2AnalyzeOptions, ) ([]AnalyzeColumnsTask, error) { if as.Incremental { - b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("The version 2 stats would ignore the INCREMENTAL keyword and do full sampling")) + b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackErrorf("The version 2 stats would ignore the INCREMENTAL keyword and do full sampling")) } astOpts, err := parseAnalyzeOptionsV2(as.AnalyzeOpts) if err != nil { @@ -2513,7 +2513,7 @@ func (b *PlanBuilder) genV2AnalyzeOptions( astOpts = make(map[ast.AnalyzeOptionType]uint64, 0) astColChoice = model.DefaultChoice astColList = make([]*model.ColumnInfo, 0) - b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.New("Ignore columns and options when analyze partition in dynamic mode")) + b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackError("Ignore columns and options when analyze partition in dynamic mode")) } tblSavedOpts, tblSavedColChoice, tblSavedColList, err := b.getSavedAnalyzeOpts(tbl.TableInfo.ID, tbl.TableInfo) if err != nil { @@ -2703,7 +2703,7 @@ func (b *PlanBuilder) buildAnalyzeTable(as *ast.AnalyzeTableStmt, opts map[ast.A continue } if idx.MVIndex { - b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("analyzing multi-valued indexes is not supported, skip %s", idx.Name.L)) + b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackErrorf("analyzing multi-valued indexes is not supported, skip %s", idx.Name.L)) continue } for i, id := range physicalIDs { @@ -2768,10 +2768,10 @@ func (b *PlanBuilder) buildAnalyzeIndex(as *ast.AnalyzeTableStmt, opts map[ast.A if b.ctx.GetSessionVars().EnableFastAnalyze { return nil, errors.Errorf("Fast analyze hasn't reached General Availability and only support analyze version 1 currently. But the existing statistics of the table is not version 1") } - b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("The analyze version from the session is not compatible with the existing statistics of the table. Use the existing version instead")) + b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackErrorf("The analyze version from the session is not compatible with the existing statistics of the table. Use the existing version instead")) } if version == statistics.Version2 { - b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("The version 2 would collect all statistics not only the selected indexes")) + b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackErrorf("The version 2 would collect all statistics not only the selected indexes")) return b.buildAnalyzeTable(as, opts, version) } for _, idxName := range as.IndexNames { @@ -2800,7 +2800,7 @@ func (b *PlanBuilder) buildAnalyzeIndex(as *ast.AnalyzeTableStmt, opts map[ast.A return nil, ErrAnalyzeMissIndex.GenWithStackByArgs(idxName.O, tblInfo.Name.O) } if idx.MVIndex { - b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("analyzing multi-valued indexes is not supported, skip %s", idx.Name.L)) + b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackErrorf("analyzing multi-valued indexes is not supported, skip %s", idx.Name.L)) continue } for i, id := range physicalIDs { @@ -2837,16 +2837,16 @@ func (b *PlanBuilder) buildAnalyzeAllIndex(as *ast.AnalyzeTableStmt, opts map[as if b.ctx.GetSessionVars().EnableFastAnalyze { return nil, errors.Errorf("Fast analyze hasn't reached General Availability and only support analyze version 1 currently. But the existing statistics of the table is not version 1") } - b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("The analyze version from the session is not compatible with the existing statistics of the table. Use the existing version instead")) + b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackErrorf("The analyze version from the session is not compatible with the existing statistics of the table. Use the existing version instead")) } if version == statistics.Version2 { - b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("The version 2 would collect all statistics not only the selected indexes")) + b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackErrorf("The version 2 would collect all statistics not only the selected indexes")) return b.buildAnalyzeTable(as, opts, version) } for _, idx := range tblInfo.Indices { if idx.State == model.StatePublic { if idx.MVIndex { - b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("analyzing multi-valued indexes is not supported, skip %s", idx.Name.L)) + b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackErrorf("analyzing multi-valued indexes is not supported, skip %s", idx.Name.L)) continue } diff --git a/planner/core/point_get_plan.go b/planner/core/point_get_plan.go index f831f76fda833..90de4ab838041 100644 --- a/planner/core/point_get_plan.go +++ b/planner/core/point_get_plan.go @@ -545,7 +545,7 @@ func TryFastPlan(ctx sessionctx.Context, node ast.Node) (p Plan) { defer func() { vars := ctx.GetSessionVars() if vars.SelectLimit != math2.MaxUint64 && p != nil { - ctx.GetSessionVars().StmtCtx.AppendWarning(errors.New("sql_select_limit is set, so point get plan is not activated")) + ctx.GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackError("sql_select_limit is set, so point get plan is not activated")) p = nil } if vars.StmtCtx.EnableOptimizeTrace && p != nil { diff --git a/planner/core/preprocess.go b/planner/core/preprocess.go index b4094199bc3f5..3a551f19acb64 100644 --- a/planner/core/preprocess.go +++ b/planner/core/preprocess.go @@ -1775,9 +1775,9 @@ func (p *preprocessor) hasAutoConvertWarning(colDef *ast.ColumnDef) bool { if !sessVars.SQLMode.HasStrictMode() && colDef.Tp.GetType() == mysql.TypeVarchar { colDef.Tp.SetType(mysql.TypeBlob) if colDef.Tp.GetCharset() == charset.CharsetBin { - sessVars.StmtCtx.AppendWarning(dbterror.ErrAutoConvert.GenWithStackByArgs(colDef.Name.Name.O, "VARBINARY", "BLOB")) + sessVars.StmtCtx.AppendWarning(dbterror.ErrAutoConvert.FastGenByArgs(colDef.Name.Name.O, "VARBINARY", "BLOB")) } else { - sessVars.StmtCtx.AppendWarning(dbterror.ErrAutoConvert.GenWithStackByArgs(colDef.Name.Name.O, "VARCHAR", "TEXT")) + sessVars.StmtCtx.AppendWarning(dbterror.ErrAutoConvert.FastGenByArgs(colDef.Name.Name.O, "VARCHAR", "TEXT")) } return true } diff --git a/planner/core/rule_join_reorder.go b/planner/core/rule_join_reorder.go index 7c25ef91d0063..ad1ebcddd0b1c 100644 --- a/planner/core/rule_join_reorder.go +++ b/planner/core/rule_join_reorder.go @@ -265,19 +265,19 @@ func (s *joinReOrderSolver) optimizeRecursive(ctx sessionctx.Context, p LogicalP leadingHintInfo, hasDiffLeadingHint := checkAndGenerateLeadingHint(joinOrderHintInfo) if hasDiffLeadingHint { - ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid")) + ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen("We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid")) } if leadingHintInfo != nil && leadingHintInfo.leadingJoinOrder != nil { if useGreedy { ok, leftJoinGroup := baseGroupSolver.generateLeadingJoinGroup(curJoinGroup, leadingHintInfo, hasOuterJoin) if !ok { - ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("leading hint is inapplicable, check if the leading hint table is valid")) + ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen("leading hint is inapplicable, check if the leading hint table is valid")) } else { curJoinGroup = leftJoinGroup } } else { - ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("leading hint is inapplicable for the DP join reorder algorithm")) + ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen("leading hint is inapplicable for the DP join reorder algorithm")) } } @@ -319,7 +319,7 @@ func (s *joinReOrderSolver) optimizeRecursive(ctx sessionctx.Context, p LogicalP return p, nil } if len(curJoinGroup) == 1 && joinOrderHintInfo != nil { - ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("leading hint is inapplicable, check the join type or the join algorithm hint")) + ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen("leading hint is inapplicable, check the join type or the join algorithm hint")) } newChildren := make([]LogicalPlan, 0, len(p.Children())) for _, child := range p.Children() { diff --git a/planner/core/rule_join_reorder_greedy.go b/planner/core/rule_join_reorder_greedy.go index 4b648a9254960..7bd3faac386cf 100644 --- a/planner/core/rule_join_reorder_greedy.go +++ b/planner/core/rule_join_reorder_greedy.go @@ -79,7 +79,7 @@ func (s *joinReorderGreedySolver) solve(joinNodePlans []LogicalPlan, tracer *joi // Getting here means that there is no join condition between the table used in the leading hint and other tables // For example: select /*+ leading(t3) */ * from t1 join t2 on t1.a=t2.a cross join t3 // We can not let table t3 join first. - s.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("leading hint is inapplicable, check if the leading hint table has join conditions with other tables")) + s.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen("leading hint is inapplicable, check if the leading hint table has join conditions with other tables")) } cartesianGroup = append(cartesianGroup, newNode.p) } diff --git a/planner/core/rule_partition_processor.go b/planner/core/rule_partition_processor.go index 0521dd7f69b2d..7221ba9e6536a 100644 --- a/planner/core/rule_partition_processor.go +++ b/planner/core/rule_partition_processor.go @@ -1659,7 +1659,7 @@ func (s *partitionProcessor) resolveOptimizeHint(ds *DataSource, partitionName m } if ds.preferStoreType&preferTiFlash != 0 && ds.preferStoreType&preferTiKV != 0 { ds.ctx.GetSessionVars().StmtCtx.AppendWarning( - errors.New("hint `read_from_storage` has conflict storage type for the partition " + partitionName.L)) + errors.NewNoStackError("hint `read_from_storage` has conflict storage type for the partition " + partitionName.L)) } return s.resolveAccessPaths(ds) diff --git a/planner/core/rule_semi_join_rewrite.go b/planner/core/rule_semi_join_rewrite.go index e38155d8ef455..b49cd6cd7ff47 100644 --- a/planner/core/rule_semi_join_rewrite.go +++ b/planner/core/rule_semi_join_rewrite.go @@ -53,7 +53,7 @@ func (smj *semiJoinRewriter) recursivePlan(p LogicalPlan) (LogicalPlan, error) { join.preferJoinType &= ^preferRewriteSemiJoin if join.JoinType == LeftOuterSemiJoin { - p.SCtx().GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("SEMI_JOIN_REWRITE() is inapplicable for LeftOuterSemiJoin.")) + p.SCtx().GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen("SEMI_JOIN_REWRITE() is inapplicable for LeftOuterSemiJoin.")) return p, nil } @@ -61,7 +61,7 @@ func (smj *semiJoinRewriter) recursivePlan(p LogicalPlan) (LogicalPlan, error) { // If there's left condition or other condition, we cannot rewrite if len(join.LeftConditions) > 0 || len(join.OtherConditions) > 0 { - p.SCtx().GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("SEMI_JOIN_REWRITE() is inapplicable for SemiJoin with left conditions or other conditions.")) + p.SCtx().GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen("SEMI_JOIN_REWRITE() is inapplicable for SemiJoin with left conditions or other conditions.")) return p, nil } diff --git a/planner/core/stats.go b/planner/core/stats.go index 2cf9960f3bd6b..3baad8e473323 100644 --- a/planner/core/stats.go +++ b/planner/core/stats.go @@ -473,9 +473,9 @@ func (ds *DataSource) derivePathStatsAndTryHeuristics() error { } } if ds.ctx.GetSessionVars().StmtCtx.InVerboseExplain { - ds.ctx.GetSessionVars().StmtCtx.AppendNote(errors.New(sb.String())) + ds.ctx.GetSessionVars().StmtCtx.AppendNote(errors.NewNoStackError(sb.String())) } else { - ds.ctx.GetSessionVars().StmtCtx.AppendExtraNote(errors.New(sb.String())) + ds.ctx.GetSessionVars().StmtCtx.AppendExtraNote(errors.NewNoStackError(sb.String())) } } return nil diff --git a/planner/core/task.go b/planner/core/task.go index dc77efda97971..c22db57756a69 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -1605,7 +1605,7 @@ func CheckAggCanPushCop(sctx sessionctx.Context, aggFuncs []*aggregation.AggFunc if storeType == kv.UnSpecified { storageName = "storage layer" } - warnErr := errors.New("Aggregation can not be pushed to " + storageName + " because " + reason) + warnErr := errors.NewNoStackError("Aggregation can not be pushed to " + storageName + " because " + reason) if sc.InExplainStmt { sc.AppendWarning(warnErr) } else { @@ -2056,7 +2056,7 @@ func (p *basePhysicalAgg) canUse3Stage4MultiDistinctAgg() (can bool, gss express } compressed := groupingSets.Merge() if len(compressed) != len(groupingSets) { - p.SCtx().GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("Some grouping sets should be merged")) + p.SCtx().GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackErrorf("Some grouping sets should be merged")) // todo arenatlx: some grouping set should be merged which is not supported by now temporarily. return false, nil } @@ -2072,7 +2072,7 @@ func (p *basePhysicalAgg) canUse3Stage4MultiDistinctAgg() (can bool, gss express groupingSetOffset := groupingSets.TargetOne(fun.Args) if groupingSetOffset == -1 { // todo: if we couldn't find a existed current valid group layout, we need to copy the column out from being filled with null value. - p.SCtx().GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("couldn't find a proper group set for normal agg")) + p.SCtx().GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackErrorf("couldn't find a proper group set for normal agg")) return false, nil } // starting with 1 diff --git a/planner/optimize.go b/planner/optimize.go index 2758407661b85..9b78c703ecef8 100644 --- a/planner/optimize.go +++ b/planner/optimize.go @@ -92,7 +92,7 @@ func getPlanFromNonPreparedPlanCache(ctx context.Context, sctx sessionctx.Contex ok, reason := core.NonPreparedPlanCacheableWithCtx(sctx, stmt, is) if !ok { if !isExplain && stmtCtx.InExplainStmt && stmtCtx.ExplainFormat == types.ExplainFormatPlanCache { - stmtCtx.AppendWarning(errors.Errorf("skip non-prepared plan-cache: %s", reason)) + stmtCtx.AppendWarning(errors.NewNoStackErrorf("skip non-prepared plan-cache: %s", reason)) } return nil, nil, false, nil } @@ -306,7 +306,7 @@ func Optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is in } } if bestPlanFromBind == nil { - sessVars.StmtCtx.AppendWarning(errors.New("no plan generated from bindings")) + sessVars.StmtCtx.AppendWarning(errors.NewNoStackError("no plan generated from bindings")) } else { bestPlan = bestPlanFromBind sessVars.StmtCtx.StmtHints = bindStmtHints @@ -316,12 +316,12 @@ func Optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is in sessVars.StmtCtx.BindSQL = chosenBinding.BindSQL sessVars.FoundInBinding = true if sessVars.StmtCtx.InVerboseExplain { - sessVars.StmtCtx.AppendNote(errors.Errorf("Using the bindSQL: %v", chosenBinding.BindSQL)) + sessVars.StmtCtx.AppendNote(errors.NewNoStackErrorf("Using the bindSQL: %v", chosenBinding.BindSQL)) } else { - sessVars.StmtCtx.AppendExtraNote(errors.Errorf("Using the bindSQL: %v", chosenBinding.BindSQL)) + sessVars.StmtCtx.AppendExtraNote(errors.NewNoStackErrorf("Using the bindSQL: %v", chosenBinding.BindSQL)) } if len(tableHints) > 0 { - sessVars.StmtCtx.AppendWarning(errors.Errorf("The system ignores the hints in the current query and uses the hints specified in the bindSQL: %v", chosenBinding.BindSQL)) + sessVars.StmtCtx.AppendWarning(errors.NewNoStackErrorf("The system ignores the hints in the current query and uses the hints specified in the bindSQL: %v", chosenBinding.BindSQL)) } } // Restore the hint to avoid changing the stmt node. @@ -723,17 +723,17 @@ func handleStmtHints(hints []*ast.TableOptimizerHint) (stmtHints stmtctx.StmtHin // Not all session variables are permitted for use with SET_VAR sysVar := variable.GetSysVar(setVarHint.VarName) if sysVar == nil { - warns = append(warns, core.ErrUnresolvedHintName.GenWithStackByArgs(setVarHint.VarName, hint.HintName.String())) + warns = append(warns, core.ErrUnresolvedHintName.FastGenByArgs(setVarHint.VarName, hint.HintName.String())) continue } if !sysVar.IsHintUpdatable { - warns = append(warns, core.ErrNotHintUpdatable.GenWithStackByArgs(setVarHint.VarName)) + warns = append(warns, core.ErrNotHintUpdatable.FastGenByArgs(setVarHint.VarName)) continue } // If several hints with the same variable name appear in the same statement, the first one is applied and the others are ignored with a warning if _, ok := setVars[setVarHint.VarName]; ok { msg := fmt.Sprintf("%s(%s=%s)", hint.HintName.String(), setVarHint.VarName, setVarHint.Value) - warns = append(warns, core.ErrWarnConflictingHint.GenWithStackByArgs(msg)) + warns = append(warns, core.ErrWarnConflictingHint.FastGenByArgs(msg)) continue } setVars[setVarHint.VarName] = setVarHint.Value @@ -747,19 +747,19 @@ func handleStmtHints(hints []*ast.TableOptimizerHint) (stmtHints stmtctx.StmtHin if memoryQuotaHintCnt != 0 { memoryQuotaHint := hints[hintOffs["memory_quota"]] if memoryQuotaHintCnt > 1 { - warn := errors.Errorf("MEMORY_QUOTA() is defined more than once, only the last definition takes effect: MEMORY_QUOTA(%v)", memoryQuotaHint.HintData.(int64)) + warn := errors.NewNoStackErrorf("MEMORY_QUOTA() is defined more than once, only the last definition takes effect: MEMORY_QUOTA(%v)", memoryQuotaHint.HintData.(int64)) warns = append(warns, warn) } // Executor use MemoryQuota <= 0 to indicate no memory limit, here use < 0 to handle hint syntax error. if memoryQuota := memoryQuotaHint.HintData.(int64); memoryQuota < 0 { delete(hintOffs, "memory_quota") - warn := errors.New("The use of MEMORY_QUOTA hint is invalid, valid usage: MEMORY_QUOTA(10 MB) or MEMORY_QUOTA(10 GB)") + warn := errors.NewNoStackError("The use of MEMORY_QUOTA hint is invalid, valid usage: MEMORY_QUOTA(10 MB) or MEMORY_QUOTA(10 GB)") warns = append(warns, warn) } else { stmtHints.HasMemQuotaHint = true stmtHints.MemQuotaQuery = memoryQuota if memoryQuota == 0 { - warn := errors.New("Setting the MEMORY_QUOTA to 0 means no memory limit") + warn := errors.NewNoStackError("Setting the MEMORY_QUOTA to 0 means no memory limit") warns = append(warns, warn) } } @@ -768,7 +768,7 @@ func handleStmtHints(hints []*ast.TableOptimizerHint) (stmtHints stmtctx.StmtHin if useToJAHintCnt != 0 { useToJAHint := hints[hintOffs["use_toja"]] if useToJAHintCnt > 1 { - warn := errors.Errorf("USE_TOJA() is defined more than once, only the last definition takes effect: USE_TOJA(%v)", useToJAHint.HintData.(bool)) + warn := errors.NewNoStackErrorf("USE_TOJA() is defined more than once, only the last definition takes effect: USE_TOJA(%v)", useToJAHint.HintData.(bool)) warns = append(warns, warn) } stmtHints.HasAllowInSubqToJoinAndAggHint = true @@ -778,7 +778,7 @@ func handleStmtHints(hints []*ast.TableOptimizerHint) (stmtHints stmtctx.StmtHin if useCascadesHintCnt != 0 { useCascadesHint := hints[hintOffs["use_cascades"]] if useCascadesHintCnt > 1 { - warn := errors.Errorf("USE_CASCADES() is defined more than once, only the last definition takes effect: USE_CASCADES(%v)", useCascadesHint.HintData.(bool)) + warn := errors.NewNoStackErrorf("USE_CASCADES() is defined more than once, only the last definition takes effect: USE_CASCADES(%v)", useCascadesHint.HintData.(bool)) warns = append(warns, warn) } stmtHints.HasEnableCascadesPlannerHint = true @@ -787,7 +787,7 @@ func handleStmtHints(hints []*ast.TableOptimizerHint) (stmtHints stmtctx.StmtHin // Handle NO_INDEX_MERGE if noIndexMergeHintCnt != 0 { if noIndexMergeHintCnt > 1 { - warn := errors.New("NO_INDEX_MERGE() is defined more than once, only the last definition takes effect") + warn := errors.NewNoStackError("NO_INDEX_MERGE() is defined more than once, only the last definition takes effect") warns = append(warns, warn) } stmtHints.NoIndexMergeHint = true @@ -795,7 +795,7 @@ func handleStmtHints(hints []*ast.TableOptimizerHint) (stmtHints stmtctx.StmtHin // Handle straight_join if straightJoinHintCnt != 0 { if straightJoinHintCnt > 1 { - warn := errors.New("STRAIGHT_JOIN() is defined more than once, only the last definition takes effect") + warn := errors.NewNoStackError("STRAIGHT_JOIN() is defined more than once, only the last definition takes effect") warns = append(warns, warn) } stmtHints.StraightJoinOrder = true @@ -803,7 +803,7 @@ func handleStmtHints(hints []*ast.TableOptimizerHint) (stmtHints stmtctx.StmtHin // Handle READ_CONSISTENT_REPLICA if readReplicaHintCnt != 0 { if readReplicaHintCnt > 1 { - warn := errors.New("READ_CONSISTENT_REPLICA() is defined more than once, only the last definition takes effect") + warn := errors.NewNoStackError("READ_CONSISTENT_REPLICA() is defined more than once, only the last definition takes effect") warns = append(warns, warn) } stmtHints.HasReplicaReadHint = true @@ -813,7 +813,7 @@ func handleStmtHints(hints []*ast.TableOptimizerHint) (stmtHints stmtctx.StmtHin if maxExecutionTimeCnt != 0 { maxExecutionTime := hints[hintOffs["max_execution_time"]] if maxExecutionTimeCnt > 1 { - warn := errors.Errorf("MAX_EXECUTION_TIME() is defined more than once, only the last definition takes effect: MAX_EXECUTION_TIME(%v)", maxExecutionTime.HintData.(uint64)) + warn := errors.NewNoStackErrorf("MAX_EXECUTION_TIME() is defined more than once, only the last definition takes effect: MAX_EXECUTION_TIME(%v)", maxExecutionTime.HintData.(uint64)) warns = append(warns, warn) } stmtHints.HasMaxExecutionTime = true @@ -823,7 +823,7 @@ func handleStmtHints(hints []*ast.TableOptimizerHint) (stmtHints stmtctx.StmtHin if resourceGroupHintCnt != 0 { resourceGroup := hints[hintOffs["resource_group"]] if resourceGroupHintCnt > 1 { - warn := errors.Errorf("RESOURCE_GROUP() is defined more than once, only the last definition takes effect: RESOURCE_GROUP(%v)", resourceGroup.HintData.(string)) + warn := errors.NewNoStackErrorf("RESOURCE_GROUP() is defined more than once, only the last definition takes effect: RESOURCE_GROUP(%v)", resourceGroup.HintData.(string)) warns = append(warns, warn) } stmtHints.HasResourceGroup = true @@ -832,13 +832,13 @@ func handleStmtHints(hints []*ast.TableOptimizerHint) (stmtHints stmtctx.StmtHin // Handle NTH_PLAN if forceNthPlanCnt != 0 { if forceNthPlanCnt > 1 { - warn := errors.Errorf("NTH_PLAN() is defined more than once, only the last definition takes effect: NTH_PLAN(%v)", forceNthPlan.HintData.(int64)) + warn := errors.NewNoStackErrorf("NTH_PLAN() is defined more than once, only the last definition takes effect: NTH_PLAN(%v)", forceNthPlan.HintData.(int64)) warns = append(warns, warn) } stmtHints.ForceNthPlan = forceNthPlan.HintData.(int64) if stmtHints.ForceNthPlan < 1 { stmtHints.ForceNthPlan = -1 - warn := errors.Errorf("the hintdata for NTH_PLAN() is too small, hint ignored") + warn := errors.NewNoStackError("the hintdata for NTH_PLAN() is too small, hint ignored") warns = append(warns, warn) } } else { diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index e2e02f35abd1e..fc6c86fc11e3a 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -685,13 +685,13 @@ func (sc *StatementContext) SetSkipPlanCache(reason error) { sc.UseCache = false switch sc.CacheType { case DefaultNoCache: - sc.AppendWarning(errors.New("unknown cache type")) + sc.AppendWarning(errors.NewNoStackError("unknown cache type")) case SessionPrepared: - sc.AppendWarning(errors.Errorf("skip prepared plan-cache: %s", reason.Error())) + sc.AppendWarning(errors.NewNoStackErrorf("skip prepared plan-cache: %s", reason.Error())) case SessionNonPrepared: if sc.InExplainStmt && sc.ExplainFormat == "plan_cache" { // use "plan_cache" rather than types.ExplainFormatPlanCache to avoid import cycle - sc.AppendWarning(errors.Errorf("skip non-prepared plan-cache: %s", reason.Error())) + sc.AppendWarning(errors.NewNoStackErrorf("skip non-prepared plan-cache: %s", reason.Error())) } } } diff --git a/statistics/selectivity.go b/statistics/selectivity.go index 76fc3cd9106c5..8205286880513 100644 --- a/statistics/selectivity.go +++ b/statistics/selectivity.go @@ -491,7 +491,7 @@ OUTER: for i, scalarCond := range notCoveredStrMatch { ok, sel, err := coll.GetSelectivityByFilter(ctx, []expression.Expression{scalarCond}) if err != nil { - sc.AppendWarning(errors.New("Error when using TopN-assisted estimation: " + err.Error())) + sc.AppendWarning(errors.NewNoStackError("Error when using TopN-assisted estimation: " + err.Error())) } if !ok { continue @@ -506,7 +506,7 @@ OUTER: for i, scalarCond := range notCoveredNegateStrMatch { ok, sel, err := coll.GetSelectivityByFilter(ctx, []expression.Expression{scalarCond}) if err != nil { - sc.AppendWarning(errors.New("Error when using TopN-assisted estimation: " + err.Error())) + sc.AppendWarning(errors.NewNoStackError("Error when using TopN-assisted estimation: " + err.Error())) } if !ok { continue