Skip to content

Commit

Permalink
bindinfo: add SetHintWarning to trace the hint warning (#50666)
Browse files Browse the repository at this point in the history
close #48875
  • Loading branch information
hawkingrei committed Jan 24, 2024
1 parent d6ef1c7 commit 230e1d0
Show file tree
Hide file tree
Showing 8 changed files with 47 additions and 44 deletions.
37 changes: 18 additions & 19 deletions pkg/planner/core/exhaust_physical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -237,7 +237,7 @@ func (p *LogicalJoin) GetMergeJoin(prop *property.PhysicalProperty, schema *expr
if p.preferJoinType&h.PreferMergeJoin == 0 {
return nil
}
p.SCtx().GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen(
p.SCtx().GetSessionVars().StmtCtx.SetHintWarning(ErrInternal.FastGen(
"Some MERGE_JOIN and NO_MERGE_JOIN hints conflict, NO_MERGE_JOIN is ignored"))
}

Expand Down Expand Up @@ -402,7 +402,7 @@ func (p *LogicalJoin) getHashJoins(prop *property.PhysicalProperty) (joins []Phy
forceLeftToBuild := ((p.preferJoinType & h.PreferLeftAsHJBuild) > 0) || ((p.preferJoinType & h.PreferRightAsHJProbe) > 0)
forceRightToBuild := ((p.preferJoinType & h.PreferRightAsHJBuild) > 0) || ((p.preferJoinType & h.PreferLeftAsHJProbe) > 0)
if forceLeftToBuild && forceRightToBuild {
p.SCtx().GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen("Some HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are conflicts, please check the hints"))
p.SCtx().GetSessionVars().StmtCtx.SetHintWarning(ErrInternal.FastGen("Some HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are conflicts, please check the hints"))
forceLeftToBuild = false
forceRightToBuild = false
}
Expand All @@ -413,7 +413,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.FastGen(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.SetHintWarning(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 @@ -458,7 +458,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.FastGen(
p.SCtx().GetSessionVars().StmtCtx.SetHintWarning(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 @@ -2054,13 +2054,13 @@ func (p *LogicalJoin) tryToGetIndexJoin(prop *property.PhysicalProperty) (indexJ
// Handle hints conflict first.
stmtCtx := p.SCtx().GetSessionVars().StmtCtx
if p.preferAny(h.PreferLeftAsINLJInner, h.PreferRightAsINLJInner) && p.preferAny(h.PreferNoIndexJoin) {
stmtCtx.AppendWarning(ErrInternal.FastGen("Some INL_JOIN and NO_INDEX_JOIN hints conflict, NO_INDEX_JOIN may be ignored"))
stmtCtx.SetHintWarning(ErrInternal.FastGen("Some INL_JOIN and NO_INDEX_JOIN hints conflict, NO_INDEX_JOIN may be ignored"))
}
if p.preferAny(h.PreferLeftAsINLHJInner, h.PreferRightAsINLHJInner) && p.preferAny(h.PreferNoIndexHashJoin) {
stmtCtx.AppendWarning(ErrInternal.FastGen("Some INL_HASH_JOIN and NO_INDEX_HASH_JOIN hints conflict, NO_INDEX_HASH_JOIN may be ignored"))
stmtCtx.SetHintWarning(ErrInternal.FastGen("Some INL_HASH_JOIN and NO_INDEX_HASH_JOIN hints conflict, NO_INDEX_HASH_JOIN may be ignored"))
}
if p.preferAny(h.PreferLeftAsINLMJInner, h.PreferRightAsINLMJInner) && p.preferAny(h.PreferNoIndexMergeJoin) {
stmtCtx.AppendWarning(ErrInternal.FastGen("Some INL_MERGE_JOIN and NO_INDEX_MERGE_JOIN hints conflict, NO_INDEX_MERGE_JOIN may be ignored"))
stmtCtx.SetHintWarning(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 @@ -2140,7 +2140,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.FastGen(errMsg))
p.SCtx().GetSessionVars().StmtCtx.SetHintWarning(ErrInternal.FastGen(errMsg))
}
return candidates, false
}
Expand Down Expand Up @@ -2284,7 +2284,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.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.SCtx().GetSessionVars().StmtCtx.SetHintWarning(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 @@ -2329,7 +2329,7 @@ func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]P
hasMppHints = true
}
if hasMppHints {
p.SCtx().GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen(errMsg))
p.SCtx().GetSessionVars().StmtCtx.SetHintWarning(ErrInternal.FastGen(errMsg))
}
}
if prop.IsFlashProp() {
Expand Down Expand Up @@ -2451,7 +2451,7 @@ func (p *LogicalJoin) tryToGetMppHashJoin(prop *property.PhysicalProperty, useBC
forceLeftToBuild := ((p.preferJoinType & h.PreferLeftAsHJBuild) > 0) || ((p.preferJoinType & h.PreferRightAsHJProbe) > 0)
forceRightToBuild := ((p.preferJoinType & h.PreferRightAsHJBuild) > 0) || ((p.preferJoinType & h.PreferLeftAsHJProbe) > 0)
if forceLeftToBuild && forceRightToBuild {
p.SCtx().GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen("Some HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are conflicts, please check the hints"))
p.SCtx().GetSessionVars().StmtCtx.SetHintWarning(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 @@ -2499,7 +2499,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.FastGen("Some HASH_JOIN_BUILD and HASH_JOIN_PROBE hints cannot be utilized for MPP joins, please check the hints"))
p.SCtx().GetSessionVars().StmtCtx.SetHintWarning(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 @@ -2694,7 +2694,7 @@ func pushLimitOrTopNForcibly(p LogicalPlan) bool {
}
if *preferPushDown {
errMsg := "Optimizer Hint LIMIT_TO_COP is inapplicable"
p.SCtx().GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen(errMsg))
p.SCtx().GetSessionVars().StmtCtx.SetHintWarning(ErrInternal.FastGen(errMsg))
*preferPushDown = false
}
}
Expand Down Expand Up @@ -3405,7 +3405,7 @@ func (la *LogicalAggregation) getHashAggs(prop *property.PhysicalProperty) []Phy
hasMppHints = true
}
if hasMppHints {
la.SCtx().GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen(errMsg))
la.SCtx().GetSessionVars().StmtCtx.SetHintWarning(ErrInternal.FastGen(errMsg))
}
}
if prop.IsFlashProp() {
Expand Down Expand Up @@ -3435,7 +3435,7 @@ func (la *LogicalAggregation) ResetHintIfConflicted() (preferHash bool, preferSt
if preferHash && preferStream {
errMsg := "Optimizer aggregation hints are conflicted"
warning := ErrInternal.FastGen(errMsg)
la.SCtx().GetSessionVars().StmtCtx.AppendWarning(warning)
la.SCtx().GetSessionVars().StmtCtx.SetHintWarning(warning)
la.aggHints.PreferAggType = 0
preferHash, preferStream = false, false
}
Expand All @@ -3445,9 +3445,8 @@ func (la *LogicalAggregation) ResetHintIfConflicted() (preferHash bool, preferSt
func (la *LogicalAggregation) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) {
if la.aggHints.PreferAggToCop {
if !la.canPushToCop(kv.TiKV) {
errMsg := "Optimizer Hint AGG_TO_COP is inapplicable"
warning := ErrInternal.FastGen(errMsg)
la.SCtx().GetSessionVars().StmtCtx.AppendWarning(warning)
la.SCtx().GetSessionVars().StmtCtx.SetHintWarning(
ErrInternal.FastGen("Optimizer Hint AGG_TO_COP is inapplicable"))
la.aggHints.PreferAggToCop = false
}
}
Expand All @@ -3469,7 +3468,7 @@ func (la *LogicalAggregation) exhaustPhysicalPlans(prop *property.PhysicalProper
if streamAggs == nil && preferStream && !prop.IsSortItemEmpty() {
errMsg := "Optimizer Hint STREAM_AGG is inapplicable"
warning := ErrInternal.FastGen(errMsg)
la.SCtx().GetSessionVars().StmtCtx.AppendWarning(warning)
la.SCtx().GetSessionVars().StmtCtx.SetHintWarning(warning)
}

return aggs, !(preferStream || preferHash), nil
Expand Down
8 changes: 4 additions & 4 deletions pkg/planner/core/expression_rewriter.go
Original file line number Diff line number Diff line change
Expand Up @@ -653,7 +653,7 @@ func (er *expressionRewriter) handleCompareSubquery(ctx context.Context, planCtx

noDecorrelate := hintFlags&hint.HintFlagNoDecorrelate > 0
if noDecorrelate && len(extractCorColumnsBySchema4LogicalPlan(np, planCtx.plan.Schema())) == 0 {
er.sctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen(
er.sctx.GetSessionVars().StmtCtx.SetHintWarning(ErrInternal.FastGen(
"NO_DECORRELATE() is inapplicable because there are no correlated columns."))
noDecorrelate = false
}
Expand Down Expand Up @@ -962,13 +962,13 @@ func (er *expressionRewriter) handleExistSubquery(ctx context.Context, planCtx *

noDecorrelate := hintFlags&hint.HintFlagNoDecorrelate > 0
if noDecorrelate && len(extractCorColumnsBySchema4LogicalPlan(np, planCtx.plan.Schema())) == 0 {
er.sctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen(
er.sctx.GetSessionVars().StmtCtx.SetHintWarning(ErrInternal.FastGen(
"NO_DECORRELATE() is inapplicable because there are no correlated columns."))
noDecorrelate = false
}
semiJoinRewrite := hintFlags&hint.HintFlagSemiJoinRewrite > 0
if semiJoinRewrite && noDecorrelate {
er.sctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen(
er.sctx.GetSessionVars().StmtCtx.SetHintWarning(ErrInternal.FastGen(
"NO_DECORRELATE() and SEMI_JOIN_REWRITE() are in conflict. Both will be ineffective."))
noDecorrelate = false
semiJoinRewrite = false
Expand Down Expand Up @@ -1138,7 +1138,7 @@ func (er *expressionRewriter) handleInSubquery(ctx context.Context, planCtx *exp
noDecorrelate := hintFlags&hint.HintFlagNoDecorrelate > 0
corCols := extractCorColumnsBySchema4LogicalPlan(np, planCtx.plan.Schema())
if len(corCols) == 0 && noDecorrelate {
er.sctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen(
er.sctx.GetSessionVars().StmtCtx.SetHintWarning(ErrInternal.FastGen(
"NO_DECORRELATE() is inapplicable because there are no correlated columns."))
noDecorrelate = false
}
Expand Down
25 changes: 12 additions & 13 deletions pkg/planner/core/logical_plan_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -779,9 +779,8 @@ func (p *LogicalJoin) setPreferredJoinTypeAndOrder(hintInfo *h.TableHintInfo) {
}
}
if hasConflict {
errMsg := "Join hints are conflict, you can only specify one type of join"
warning := ErrInternal.FastGen(errMsg)
p.SCtx().GetSessionVars().StmtCtx.AppendWarning(warning)
p.SCtx().GetSessionVars().StmtCtx.SetHintWarning(
ErrInternal.FastGen("Join hints are conflict, you can only specify one type of join"))
p.preferJoinType = 0
}
// set the join order
Expand Down Expand Up @@ -851,7 +850,7 @@ func (p *LogicalJoin) setPreferredJoinType() {
if containDifferentJoinTypes(p.preferJoinType) {
errMsg := "Join hints conflict after join reorder phase, you can only specify one type of join"
warning := ErrInternal.FastGen(errMsg)
p.SCtx().GetSessionVars().StmtCtx.AppendWarning(warning)
p.SCtx().GetSessionVars().StmtCtx.SetHintWarning(warning)
p.preferJoinType = 0
}
}
Expand Down Expand Up @@ -880,7 +879,7 @@ func (ds *DataSource) setPreferredStoreType(hintInfo *h.TableHintInfo) {
"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.SCtx().GetSessionVars().GetIsolationReadEngines())
warning := ErrInternal.FastGen(errMsg)
ds.SCtx().GetSessionVars().StmtCtx.AppendWarning(warning)
ds.SCtx().GetSessionVars().StmtCtx.SetHintWarning(warning)
} else {
ds.SCtx().GetSessionVars().RaiseWarningWhenMPPEnforced("MPP mode may be blocked because you have set a hint to read table `" + hintTbl.TblName.O + "` from TiKV.")
}
Expand All @@ -892,7 +891,7 @@ func (ds *DataSource) setPreferredStoreType(hintInfo *h.TableHintInfo) {
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.FastGen(errMsg)
ds.SCtx().GetSessionVars().StmtCtx.AppendWarning(warning)
ds.SCtx().GetSessionVars().StmtCtx.SetHintWarning(warning)
ds.preferStoreType = 0
return
}
Expand All @@ -908,7 +907,7 @@ func (ds *DataSource) setPreferredStoreType(hintInfo *h.TableHintInfo) {
"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.SCtx().GetSessionVars().GetIsolationReadEngines())
warning := ErrInternal.FastGen(errMsg)
ds.SCtx().GetSessionVars().StmtCtx.AppendWarning(warning)
ds.SCtx().GetSessionVars().StmtCtx.SetHintWarning(warning)
}
}
}
Expand Down Expand Up @@ -4080,7 +4079,7 @@ func (b *PlanBuilder) pushTableHints(hints []*ast.TableOptimizerHint, currentLev
limitHints.PreferLimitToCop = true
case h.HintMerge:
if hint.Tables != nil {
b.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen("The MERGE hint is not used correctly, maybe it inputs a table name."))
b.ctx.GetSessionVars().StmtCtx.SetHintWarning(ErrInternal.FastGen("The MERGE hint is not used correctly, maybe it inputs a table name."))
continue
}
MergeHints.PreferMerge = true
Expand All @@ -4091,13 +4090,13 @@ func (b *PlanBuilder) pushTableHints(hints []*ast.TableOptimizerHint, currentLev
leadingHintCnt++
case h.HintSemiJoinRewrite:
if b.subQueryCtx != handlingExistsSubquery {
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."))
b.ctx.GetSessionVars().StmtCtx.SetHintWarning(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 |= h.HintFlagSemiJoinRewrite
case h.HintNoDecorrelate:
if b.subQueryCtx == notHandlingSubquery {
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."))
b.ctx.GetSessionVars().StmtCtx.SetHintWarning(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 |= h.HintFlagNoDecorrelate
Expand All @@ -4109,9 +4108,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.FastGen("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.SetHintWarning(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.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.ctx.GetSessionVars().StmtCtx.SetHintWarning(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, &h.TableHintInfo{
Expand Down Expand Up @@ -4147,7 +4146,7 @@ func (b *PlanBuilder) popVisitInfo() {
func (b *PlanBuilder) popTableHints() {
hintInfo := b.tableHintInfo[len(b.tableHintInfo)-1]
for _, warning := range h.CollectUnmatchedHintWarnings(hintInfo) {
b.ctx.GetSessionVars().StmtCtx.AppendWarning(warning)
b.ctx.GetSessionVars().StmtCtx.SetHintWarning(warning)
}
b.tableHintInfo = b.tableHintInfo[:len(b.tableHintInfo)-1]
}
Expand Down
8 changes: 4 additions & 4 deletions pkg/planner/core/rule_join_reorder.go
Original file line number Diff line number Diff line change
Expand Up @@ -271,19 +271,19 @@ func (s *joinReOrderSolver) optimizeRecursive(ctx sessionctx.Context, p LogicalP

leadingHintInfo, hasDiffLeadingHint := checkAndGenerateLeadingHint(joinOrderHintInfo)
if hasDiffLeadingHint {
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"))
ctx.GetSessionVars().StmtCtx.SetHintWarning(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.FastGen("leading hint is inapplicable, check if the leading hint table is valid"))
ctx.GetSessionVars().StmtCtx.SetHintWarning(ErrInternal.FastGen("leading hint is inapplicable, check if the leading hint table is valid"))
} else {
curJoinGroup = leftJoinGroup
}
} else {
ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen("leading hint is inapplicable for the DP join reorder algorithm"))
ctx.GetSessionVars().StmtCtx.SetHintWarning(ErrInternal.FastGen("leading hint is inapplicable for the DP join reorder algorithm"))
}
}

Expand Down Expand Up @@ -325,7 +325,7 @@ func (s *joinReOrderSolver) optimizeRecursive(ctx sessionctx.Context, p LogicalP
return p, nil
}
if len(curJoinGroup) == 1 && joinOrderHintInfo != nil {
ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen("leading hint is inapplicable, check the join type or the join algorithm hint"))
ctx.GetSessionVars().StmtCtx.SetHintWarning(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() {
Expand Down
2 changes: 1 addition & 1 deletion pkg/planner/core/rule_join_reorder_greedy.go
Original file line number Diff line number Diff line change
Expand Up @@ -82,7 +82,7 @@ func (s *joinReorderGreedySolver) solve(joinNodePlans []LogicalPlan, tracer *joi
// TODO(hawkingrei): we find the problem in the TestHint.
// `select * from t1, t2, t3 union all select /*+ leading(t3, t2) */ * from t1, t2, t3 union all select * from t1, t2, t3`
// this sql should not return the warning. but It will not affect the result. so we will fix it as soon as possible.
s.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen("leading hint is inapplicable, check if the leading hint table has join conditions with other tables"))
s.ctx.GetSessionVars().StmtCtx.SetHintWarning(ErrInternal.FastGen("leading hint is inapplicable, check if the leading hint table has join conditions with other tables"))
}
cartesianGroup = append(cartesianGroup, newNode.p)
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/planner/core/rule_partition_processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -1731,7 +1731,7 @@ func appendWarnForUnknownPartitions(ctx sessionctx.Context, hintName string, unk
}

warning := fmt.Errorf("unknown partitions (%s) in optimizer hint %s", strings.Join(unknownPartitions, ","), hintName)
ctx.GetSessionVars().StmtCtx.AppendWarning(warning)
ctx.GetSessionVars().StmtCtx.SetHintWarning(warning)
}

func (*partitionProcessor) checkHintsApplicable(ds *DataSource, partitionSet set.StringSet) {
Expand Down
Loading

0 comments on commit 230e1d0

Please sign in to comment.