Skip to content

Commit

Permalink
planner: refactor planner module's warning and note generation logic …
Browse files Browse the repository at this point in the history
…(get rid of unnecessary stack info) (#49390)

close #49291
  • Loading branch information
AilinKid authored Dec 14, 2023
1 parent d076bc1 commit 6aea95e
Show file tree
Hide file tree
Showing 24 changed files with 120 additions and 123 deletions.
12 changes: 6 additions & 6 deletions DEPS.bzl
Original file line number Diff line number Diff line change
Expand Up @@ -5784,13 +5784,13 @@ def go_deps():
name = "com_github_pingcap_errors",
build_file_proto_mode = "disable_global",
importpath = "github.com/pingcap/errors",
sha256 = "4dadf9dc507b4187a70b78e49d572bc0e8f89a7b4a8974d6a978f72620526996",
strip_prefix = "github.com/pingcap/[email protected].20221009092201-b66cddb77c32",
sha256 = "b4db3d3c222d9039c84baacbbd9c46aa0346f3f04d2577a77475a64ecfefebf9",
strip_prefix = "github.com/pingcap/[email protected].20231212100244-799fae176cfb",
urls = [
"http://bazel-cache.pingcap.net:8080/gomod/github.com/pingcap/errors/com_github_pingcap_errors-v0.11.5-0.20221009092201-b66cddb77c32.zip",
"http://ats.apps.svc/gomod/github.com/pingcap/errors/com_github_pingcap_errors-v0.11.5-0.20221009092201-b66cddb77c32.zip",
"https://cache.hawkingrei.com/gomod/github.com/pingcap/errors/com_github_pingcap_errors-v0.11.5-0.20221009092201-b66cddb77c32.zip",
"https://storage.googleapis.com/pingcapmirror/gomod/github.com/pingcap/errors/com_github_pingcap_errors-v0.11.5-0.20221009092201-b66cddb77c32.zip",
"http://bazel-cache.pingcap.net:8080/gomod/github.com/pingcap/errors/com_github_pingcap_errors-v0.11.5-0.20231212100244-799fae176cfb.zip",
"http://ats.apps.svc/gomod/github.com/pingcap/errors/com_github_pingcap_errors-v0.11.5-0.20231212100244-799fae176cfb.zip",
"https://cache.hawkingrei.com/gomod/github.com/pingcap/errors/com_github_pingcap_errors-v0.11.5-0.20231212100244-799fae176cfb.zip",
"https://storage.googleapis.com/pingcapmirror/gomod/github.com/pingcap/errors/com_github_pingcap_errors-v0.11.5-0.20231212100244-799fae176cfb.zip",
],
)
go_repository(
Expand Down
2 changes: 1 addition & 1 deletion go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -78,7 +78,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 v1.0.0
github.com/pingcap/kvproto v0.0.0-20231122054644-fb0f5c2a0a10
Expand Down
4 changes: 2 additions & 2 deletions go.sum
Original file line number Diff line number Diff line change
Expand Up @@ -699,8 +699,8 @@ github.com/pingcap/badger v1.5.1-0.20230103063557-828f39b09b6d/go.mod h1:p8QnkZn
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 v1.0.0 h1:CyA6AxcOZkQh52wIqYlAmaVmF6EvrcqFywP463pjA8g=
Expand Down
4 changes: 2 additions & 2 deletions pkg/planner/cardinality/selectivity.go
Original file line number Diff line number Diff line change
Expand Up @@ -359,7 +359,7 @@ OUTER:
for i, scalarCond := range notCoveredStrMatch {
ok, sel, err := GetSelectivityByFilter(ctx, coll, []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
Expand All @@ -374,7 +374,7 @@ OUTER:
for i, scalarCond := range notCoveredNegateStrMatch {
ok, sel, err := GetSelectivityByFilter(ctx, coll, []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
Expand Down
12 changes: 6 additions & 6 deletions pkg/planner/core/common_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -874,12 +874,12 @@ func (e *Explain) RenderResult() error {
cost, _ := pp.getPlanCostVer2(property.RootTaskType, NewDefaultPlanCostOption())
if cost.trace != nil {
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()
Expand All @@ -890,14 +890,14 @@ 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"))
}
}

Expand Down
41 changes: 19 additions & 22 deletions pkg/planner/core/exhaust_physical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -236,7 +236,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"))
}

Expand Down Expand Up @@ -401,7 +401,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.SCtx().GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("Some HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are conflicts, please check the hints"))
p.SCtx().GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen("Some HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are conflicts, please check the hints"))
forceLeftToBuild = false
forceRightToBuild = false
}
Expand All @@ -412,7 +412,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.SCtx().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.SCtx().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
}
Expand Down Expand Up @@ -457,7 +457,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."))
}
Expand Down Expand Up @@ -2039,13 +2039,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)
Expand Down Expand Up @@ -2125,7 +2125,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
}
Expand Down Expand Up @@ -2269,7 +2269,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.
Expand Down Expand Up @@ -2314,8 +2314,7 @@ func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]P
hasMppHints = true
}
if hasMppHints {
warning := ErrInternal.GenWithStack(errMsg)
p.SCtx().GetSessionVars().StmtCtx.AppendWarning(warning)
p.SCtx().GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen(errMsg))
}
}
if prop.IsFlashProp() {
Expand Down Expand Up @@ -2437,7 +2436,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.SCtx().GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("Some HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are conflicts, please check the hints"))
p.SCtx().GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen("Some HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are conflicts, please check the hints"))
forceLeftToBuild = false
forceRightToBuild = false
}
Expand Down Expand Up @@ -2485,7 +2484,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.SCtx().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.SCtx().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
Expand Down Expand Up @@ -2680,8 +2679,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
}
}
Expand Down Expand Up @@ -2768,7 +2766,7 @@ func (la *LogicalApply) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([
return nil, true, nil
}
if !prop.IsSortItemEmpty() && la.SCtx().GetSessionVars().EnableParallelApply {
la.SCtx().GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("Parallel Apply rejects the possible order properties of its outer child currently"))
la.SCtx().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])
Expand Down Expand Up @@ -3216,7 +3214,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.SCtx().GetSessionVars().StmtCtx.InExplainStmt {
la.SCtx().GetSessionVars().StmtCtx.AppendWarning(warnErr)
} else {
Expand Down Expand Up @@ -3392,8 +3390,7 @@ func (la *LogicalAggregation) getHashAggs(prop *property.PhysicalProperty) []Phy
hasMppHints = true
}
if hasMppHints {
warning := ErrInternal.GenWithStack(errMsg)
la.SCtx().GetSessionVars().StmtCtx.AppendWarning(warning)
la.SCtx().GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen(errMsg))
}
}
if prop.IsFlashProp() {
Expand Down Expand Up @@ -3422,7 +3419,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.SCtx().GetSessionVars().StmtCtx.AppendWarning(warning)
la.aggHints.preferAggType = 0
preferHash, preferStream = false, false
Expand All @@ -3434,7 +3431,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.SCtx().GetSessionVars().StmtCtx.AppendWarning(warning)
la.aggHints.preferAggToCop = false
}
Expand All @@ -3456,7 +3453,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.SCtx().GetSessionVars().StmtCtx.AppendWarning(warning)
}

Expand Down
Loading

0 comments on commit 6aea95e

Please sign in to comment.