diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 77cc854995465..f984f2e12b2cc 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1978,76 +1978,39 @@ func (p *LogicalJoin) preferAny(joinFlags ...uint) bool { return false } -// satisfyIndexJoinHint returns whether this join plan can satisfy current index join hints. -func (p *LogicalJoin) satisfyIndexJoinHint(join PhysicalPlan) bool { - const left, right = 0, 1 - const indexJoin, indexHashJoin, indexMergeJoin = 0, 1, 2 - var innerSide, innerIdx, joinMethod int +const ( + joinLeft = 0 + joinRight = 1 + indexJoinMethod = 0 + indexHashJoinMethod = 1 + indexMergeJoinMethod = 2 +) + +func (*LogicalJoin) getIndexJoinSideAndMethod(join PhysicalPlan) (innerSide, joinMethod int, ok bool) { + var innerIdx int switch ij := join.(type) { case *PhysicalIndexJoin: innerIdx = ij.getInnerChildIdx() - joinMethod = indexJoin + joinMethod = indexJoinMethod case *PhysicalIndexHashJoin: innerIdx = ij.getInnerChildIdx() - joinMethod = indexHashJoin + joinMethod = indexHashJoinMethod case *PhysicalIndexMergeJoin: innerIdx = ij.getInnerChildIdx() - joinMethod = indexMergeJoin + joinMethod = indexMergeJoinMethod default: - return false + return 0, 0, false } - innerSide = left + ok = true + innerSide = joinLeft if innerIdx == 1 { - innerSide = right + innerSide = joinRight } - - if (p.preferAny(preferLeftAsINLJInner) && innerSide == left && joinMethod == indexJoin) || - (p.preferAny(preferRightAsINLJInner) && innerSide == right && joinMethod == indexJoin) || - (p.preferAny(preferLeftAsINLHJInner) && innerSide == left && joinMethod == indexHashJoin) || - (p.preferAny(preferRightAsINLHJInner) && innerSide == right && joinMethod == indexHashJoin) || - (p.preferAny(preferLeftAsINLMJInner) && innerSide == left && joinMethod == indexMergeJoin) || - (p.preferAny(preferRightAsINLMJInner) && innerSide == right && joinMethod == indexMergeJoin) { - return true - } - return false + return } -// tryToGetIndexJoin will get index join by hints. If we can generate a valid index join by hint, the second return value -// will be true, which means we force to choose this index join. Otherwise we will select a join algorithm with min-cost. +// tryToGetIndexJoin returns all available index join plans, and the second returned value indicates whether this plan is enforced by hints. func (p *LogicalJoin) tryToGetIndexJoin(prop *property.PhysicalProperty) (indexJoins []PhysicalPlan, canForced bool) { - forceLeftOuter := p.preferAny(preferRightAsINLJInner, preferRightAsINLHJInner, preferRightAsINLMJInner) // left as outer == right as inner - forceRightOuter := p.preferAny(preferLeftAsINLJInner, preferLeftAsINLHJInner, preferLeftAsINLMJInner) // right as outer == left as inner - needForced := forceLeftOuter || forceRightOuter - - defer func() { - // Print warning message if any hints cannot work. - // If the required property is not empty, we will enforce it and try the hint again. - // So we only need to generate warning message when the property is empty. - if !canForced && needForced && prop.IsSortItemEmpty() { - // Construct warning message prefix. - var indexJoinTables, indexHashJoinTables, indexMergeJoinTables []hintTableInfo - if p.hintInfo != nil { - t := p.hintInfo.indexNestedLoopJoinTables - indexJoinTables, indexHashJoinTables, indexMergeJoinTables = t.inljTables, t.inlhjTables, t.inlmjTables - } - var errMsg string - switch { - case p.preferAny(preferLeftAsINLJInner, preferRightAsINLJInner): // prefer index join - errMsg = fmt.Sprintf("Optimizer Hint %s or %s is inapplicable", restore2JoinHint(HintINLJ, indexJoinTables), restore2JoinHint(TiDBIndexNestedLoopJoin, indexJoinTables)) - case p.preferAny(preferLeftAsINLHJInner, preferRightAsINLHJInner): // prefer index hash join - errMsg = fmt.Sprintf("Optimizer Hint %s is inapplicable", restore2JoinHint(HintINLHJ, indexHashJoinTables)) - case p.preferAny(preferLeftAsINLMJInner, preferRightAsINLMJInner): // prefer index merge join - errMsg = fmt.Sprintf("Optimizer Hint %s is inapplicable", restore2JoinHint(HintINLMJ, indexMergeJoinTables)) - } - // Append inapplicable reason. - if len(p.EqualConditions) == 0 { - errMsg += " without column equal ON condition" - } - // Generate warning message to client. - p.SCtx().GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack(errMsg)) - } - }() - // supportLeftOuter and supportRightOuter indicates whether this type of join // supports the left side or right side to be the outer side. var supportLeftOuter, supportRightOuter bool @@ -2059,47 +2022,75 @@ func (p *LogicalJoin) tryToGetIndexJoin(prop *property.PhysicalProperty) (indexJ case InnerJoin: supportLeftOuter, supportRightOuter = true, true } - - var allLeftOuterJoins, allRightOuterJoins, forcedLeftOuterJoins, forcedRightOuterJoins []PhysicalPlan + candidates := make([]PhysicalPlan, 0, 2) if supportLeftOuter { - allLeftOuterJoins = p.getIndexJoinByOuterIdx(prop, 0) - forcedLeftOuterJoins = make([]PhysicalPlan, 0, len(allLeftOuterJoins)) - for _, j := range allLeftOuterJoins { - if p.satisfyIndexJoinHint(j) { - forcedLeftOuterJoins = append(forcedLeftOuterJoins, j) - } + candidates = append(candidates, p.getIndexJoinByOuterIdx(prop, 0)...) + } + if supportRightOuter { + candidates = append(candidates, p.getIndexJoinByOuterIdx(prop, 1)...) + } + + // handle hints and variables about index join. + // the priority is: force hints like TIDB_INLJ > filter hints like NO_INDEX_JOIN > variables. + candidates, canForced = p.handleForceIndexJoinHints(prop, candidates) + if canForced { + return candidates, canForced + } + return filterIndexJoinBySessionVars(p.SCtx(), candidates), false +} + +// handleForceIndexJoinHints handles the force index join hints and returns all plans that can satisfy the hints. +func (p *LogicalJoin) handleForceIndexJoinHints(prop *property.PhysicalProperty, candidates []PhysicalPlan) (indexJoins []PhysicalPlan, canForced bool) { + if !p.preferAny(preferRightAsINLJInner, preferRightAsINLHJInner, preferRightAsINLMJInner, + preferLeftAsINLJInner, preferLeftAsINLHJInner, preferLeftAsINLMJInner) { + return candidates, false // no force index join hints + } + forced := make([]PhysicalPlan, 0, len(candidates)) + for _, candidate := range candidates { + innerSide, joinMethod, ok := p.getIndexJoinSideAndMethod(candidate) + if !ok { + continue } - switch { - case len(forcedLeftOuterJoins) == 0 && !supportRightOuter: - return filterIndexJoinBySessionVars(p.SCtx(), allLeftOuterJoins), false - case len(forcedLeftOuterJoins) != 0 && (!supportRightOuter || (forceLeftOuter && !forceRightOuter)): - return forcedLeftOuterJoins, true + if (p.preferAny(preferLeftAsINLJInner) && innerSide == joinLeft && joinMethod == indexJoinMethod) || + (p.preferAny(preferRightAsINLJInner) && innerSide == joinRight && joinMethod == indexJoinMethod) || + (p.preferAny(preferLeftAsINLHJInner) && innerSide == joinLeft && joinMethod == indexHashJoinMethod) || + (p.preferAny(preferRightAsINLHJInner) && innerSide == joinRight && joinMethod == indexHashJoinMethod) || + (p.preferAny(preferLeftAsINLMJInner) && innerSide == joinLeft && joinMethod == indexMergeJoinMethod) || + (p.preferAny(preferRightAsINLMJInner) && innerSide == joinRight && joinMethod == indexMergeJoinMethod) { + forced = append(forced, candidate) } } - if supportRightOuter { - allRightOuterJoins = p.getIndexJoinByOuterIdx(prop, 1) - forcedRightOuterJoins = make([]PhysicalPlan, 0, len(allRightOuterJoins)) - for _, j := range allRightOuterJoins { - if p.satisfyIndexJoinHint(j) { - forcedRightOuterJoins = append(forcedRightOuterJoins, j) - } + if len(forced) > 0 { + return forced, true + } + // Cannot find any valid index join plan with these force hints. + // Print warning message if any hints cannot work. + // If the required property is not empty, we will enforce it and try the hint again. + // So we only need to generate warning message when the property is empty. + if prop.IsSortItemEmpty() { + var indexJoinTables, indexHashJoinTables, indexMergeJoinTables []hintTableInfo + if p.hintInfo != nil { + t := p.hintInfo.indexNestedLoopJoinTables + indexJoinTables, indexHashJoinTables, indexMergeJoinTables = t.inljTables, t.inlhjTables, t.inlmjTables } + var errMsg string switch { - case len(forcedRightOuterJoins) == 0 && !supportLeftOuter: - return filterIndexJoinBySessionVars(p.SCtx(), allRightOuterJoins), false - case len(forcedRightOuterJoins) != 0 && (!supportLeftOuter || (forceRightOuter && !forceLeftOuter)): - return forcedRightOuterJoins, true + case p.preferAny(preferLeftAsINLJInner, preferRightAsINLJInner): // prefer index join + errMsg = fmt.Sprintf("Optimizer Hint %s or %s is inapplicable", restore2JoinHint(HintINLJ, indexJoinTables), restore2JoinHint(TiDBIndexNestedLoopJoin, indexJoinTables)) + case p.preferAny(preferLeftAsINLHJInner, preferRightAsINLHJInner): // prefer index hash join + errMsg = fmt.Sprintf("Optimizer Hint %s is inapplicable", restore2JoinHint(HintINLHJ, indexHashJoinTables)) + case p.preferAny(preferLeftAsINLMJInner, preferRightAsINLMJInner): // prefer index merge join + errMsg = fmt.Sprintf("Optimizer Hint %s is inapplicable", restore2JoinHint(HintINLMJ, indexMergeJoinTables)) } + // Append inapplicable reason. + if len(p.EqualConditions) == 0 { + errMsg += " without column equal ON condition" + } + // Generate warning message to client. + p.SCtx().GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack(errMsg)) } - - canForceLeft := len(forcedLeftOuterJoins) != 0 && forceLeftOuter - canForceRight := len(forcedRightOuterJoins) != 0 && forceRightOuter - canForced = canForceLeft || canForceRight - if canForced { - return append(forcedLeftOuterJoins, forcedRightOuterJoins...), true - } - return filterIndexJoinBySessionVars(p.SCtx(), append(allLeftOuterJoins, allRightOuterJoins...)), false + return candidates, false } func checkChildFitBC(p Plan) bool {