Skip to content

Commit

Permalink
planner: support no_index_join, no_index_hash_join, `no_index_mer…
Browse files Browse the repository at this point in the history
…ge_join` hints (#45633) (#45882)

ref #45520
  • Loading branch information
ti-chi-bot committed Aug 8, 2023
1 parent df7760f commit e21efe5
Show file tree
Hide file tree
Showing 7 changed files with 263 additions and 2 deletions.
12 changes: 12 additions & 0 deletions planner/core/casetest/rule_join_reorder_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -83,6 +83,18 @@ func TestNoMergeJoinHint(t *testing.T) {
runJoinReorderTestData(t, tk, "TestNoMergeJoinHint")
}

func TestNoIndexJoinHint(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec(`set tidb_enable_index_merge_join=true`)
tk.MustExec("use test")
tk.MustExec("create table t1(a int, key(a));")
tk.MustExec("create table t2(a int, key(a));")
tk.MustExec("create table t3(a int, key(a));")
tk.MustExec("create table t4(a int, key(a));")
runJoinReorderTestData(t, tk, "TestNoIndexJoinHint")
}

func TestLeadingJoinHint(t *testing.T) {
store := testkit.CreateMockStore(t)

Expand Down
16 changes: 16 additions & 0 deletions planner/core/casetest/testdata/join_reorder_suite_in.json
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,22 @@
"select /*+ leading(t1, t2, t3, t4), hash_join(t1, t2), no_hash_join(t3), hash_join(t4) */ * from t1, t2, t3, t4"
]
},
{
"name": "TestNoIndexJoinHint",
"cases": [
"select /*+ no_merge_join(t1), no_hash_join(t1), no_index_hash_join(t1), no_index_merge_join(t1) */ * from t1, t2 where t1.a=t2.a",
"select /*+ no_merge_join(t1), no_hash_join(t1), no_index_hash_join(t1), no_index_merge_join(t2) */ * from t1, t2 where t1.a=t2.a",
"select /*+ no_merge_join(t1), no_hash_join(t1), no_index_hash_join(t1, t2), no_index_merge_join(t2, t1) */ * from t1, t2 where t1.a=t2.a",
"select /*+ no_merge_join(t1), no_hash_join(t1), no_index_join(t1), no_index_merge_join(t1) */ * from t1, t2 where t1.a=t2.a",
"select /*+ no_merge_join(t1), no_hash_join(t1), no_index_join(t1), no_index_hash_join(t1) */ * from t1, t2 where t1.a=t2.a",
"select /*+ no_merge_join(t1), no_hash_join(t1), inl_join(t1), no_index_merge_join(t1), no_index_hash_join(t1) */ * from t1, t2 where t1.a=t2.a",
"select /*+ inl_join(t1), no_index_join(t1) */ * from t1, t2 where t1.a=t2.a",
"select /*+ inl_hash_join(t1), no_index_hash_join(t2) */ * from t1, t2 where t1.a=t2.a",
"select /*+ inl_merge_join(t1), no_index_merge_join(t1, t2) */ * from t1, t2 where t1.a=t2.a",
"select /*+ inl_join(t1), no_index_hash_join(t1) */ * from t1, t2 where t1.a=t2.a",
"select /*+ inl_join(t1), no_index_merge_join(t1) */ * from t1, t2 where t1.a=t2.a"
]
},
{
"name": "TestNoMergeJoinHint",
"cases": [
Expand Down
143 changes: 143 additions & 0 deletions planner/core/casetest/testdata/join_reorder_suite_out.json
Original file line number Diff line number Diff line change
Expand Up @@ -782,6 +782,149 @@
}
]
},
{
"Name": "TestNoIndexJoinHint",
"Cases": [
{
"SQL": "select /*+ no_merge_join(t1), no_hash_join(t1), no_index_hash_join(t1), no_index_merge_join(t1) */ * from t1, t2 where t1.a=t2.a",
"Plan": [
"IndexJoin 12487.50 root inner join, inner:IndexReader, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a)",
"├─IndexReader(Build) 9990.00 root index:IndexFullScan",
"│ └─IndexFullScan 9990.00 cop[tikv] table:t1, index:a(a) keep order:false, stats:pseudo",
"└─IndexReader(Probe) 12487.50 root index:Selection",
" └─Selection 12487.50 cop[tikv] not(isnull(test.t2.a))",
" └─IndexRangeScan 12500.00 cop[tikv] table:t2, index:a(a) range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo"
],
"Warning": null
},
{
"SQL": "select /*+ no_merge_join(t1), no_hash_join(t1), no_index_hash_join(t1), no_index_merge_join(t2) */ * from t1, t2 where t1.a=t2.a",
"Plan": [
"IndexJoin 12487.50 root inner join, inner:IndexReader, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a)",
"├─IndexReader(Build) 9990.00 root index:IndexFullScan",
"│ └─IndexFullScan 9990.00 cop[tikv] table:t1, index:a(a) keep order:false, stats:pseudo",
"└─IndexReader(Probe) 12487.50 root index:Selection",
" └─Selection 12487.50 cop[tikv] not(isnull(test.t2.a))",
" └─IndexRangeScan 12500.00 cop[tikv] table:t2, index:a(a) range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo"
],
"Warning": null
},
{
"SQL": "select /*+ no_merge_join(t1), no_hash_join(t1), no_index_hash_join(t1, t2), no_index_merge_join(t2, t1) */ * from t1, t2 where t1.a=t2.a",
"Plan": [
"IndexJoin 12487.50 root inner join, inner:IndexReader, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a)",
"├─IndexReader(Build) 9990.00 root index:IndexFullScan",
"│ └─IndexFullScan 9990.00 cop[tikv] table:t1, index:a(a) keep order:false, stats:pseudo",
"└─IndexReader(Probe) 12487.50 root index:Selection",
" └─Selection 12487.50 cop[tikv] not(isnull(test.t2.a))",
" └─IndexRangeScan 12500.00 cop[tikv] table:t2, index:a(a) range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo"
],
"Warning": null
},
{
"SQL": "select /*+ no_merge_join(t1), no_hash_join(t1), no_index_join(t1), no_index_merge_join(t1) */ * from t1, t2 where t1.a=t2.a",
"Plan": [
"IndexHashJoin 12487.50 root inner join, inner:IndexReader, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a)",
"├─IndexReader(Build) 9990.00 root index:IndexFullScan",
"│ └─IndexFullScan 9990.00 cop[tikv] table:t1, index:a(a) keep order:false, stats:pseudo",
"└─IndexReader(Probe) 12487.50 root index:Selection",
" └─Selection 12487.50 cop[tikv] not(isnull(test.t2.a))",
" └─IndexRangeScan 12500.00 cop[tikv] table:t2, index:a(a) range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo"
],
"Warning": null
},
{
"SQL": "select /*+ no_merge_join(t1), no_hash_join(t1), no_index_join(t1), no_index_hash_join(t1) */ * from t1, t2 where t1.a=t2.a",
"Plan": [
"IndexMergeJoin 12487.50 root inner join, inner:IndexReader, outer key:test.t1.a, inner key:test.t2.a",
"├─IndexReader(Build) 9990.00 root index:IndexFullScan",
"│ └─IndexFullScan 9990.00 cop[tikv] table:t1, index:a(a) keep order:false, stats:pseudo",
"└─IndexReader(Probe) 12487.50 root index:Selection",
" └─Selection 12487.50 cop[tikv] not(isnull(test.t2.a))",
" └─IndexRangeScan 12500.00 cop[tikv] table:t2, index:a(a) range: decided by [eq(test.t2.a, test.t1.a)], keep order:true, stats:pseudo"
],
"Warning": null
},
{
"SQL": "select /*+ no_merge_join(t1), no_hash_join(t1), inl_join(t1), no_index_merge_join(t1), no_index_hash_join(t1) */ * from t1, t2 where t1.a=t2.a",
"Plan": [
"IndexJoin 12487.50 root inner join, inner:IndexReader, outer key:test.t2.a, inner key:test.t1.a, equal cond:eq(test.t2.a, test.t1.a)",
"├─IndexReader(Build) 9990.00 root index:IndexFullScan",
"│ └─IndexFullScan 9990.00 cop[tikv] table:t2, index:a(a) keep order:false, stats:pseudo",
"└─IndexReader(Probe) 12487.50 root index:Selection",
" └─Selection 12487.50 cop[tikv] not(isnull(test.t1.a))",
" └─IndexRangeScan 12500.00 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, test.t2.a)], keep order:false, stats:pseudo"
],
"Warning": null
},
{
"SQL": "select /*+ inl_join(t1), no_index_join(t1) */ * from t1, t2 where t1.a=t2.a",
"Plan": [
"IndexJoin 12487.50 root inner join, inner:IndexReader, outer key:test.t2.a, inner key:test.t1.a, equal cond:eq(test.t2.a, test.t1.a)",
"├─IndexReader(Build) 9990.00 root index:IndexFullScan",
"│ └─IndexFullScan 9990.00 cop[tikv] table:t2, index:a(a) keep order:false, stats:pseudo",
"└─IndexReader(Probe) 12487.50 root index:Selection",
" └─Selection 12487.50 cop[tikv] not(isnull(test.t1.a))",
" └─IndexRangeScan 12500.00 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, test.t2.a)], keep order:false, stats:pseudo"
],
"Warning": [
"Warning 1815 Some INL_JOIN and NO_INDEX_JOIN hints conflict, NO_INDEX_JOIN may be ignored"
]
},
{
"SQL": "select /*+ inl_hash_join(t1), no_index_hash_join(t2) */ * from t1, t2 where t1.a=t2.a",
"Plan": [
"IndexHashJoin 12487.50 root inner join, inner:IndexReader, outer key:test.t2.a, inner key:test.t1.a, equal cond:eq(test.t2.a, test.t1.a)",
"├─IndexReader(Build) 9990.00 root index:IndexFullScan",
"│ └─IndexFullScan 9990.00 cop[tikv] table:t2, index:a(a) keep order:false, stats:pseudo",
"└─IndexReader(Probe) 12487.50 root index:Selection",
" └─Selection 12487.50 cop[tikv] not(isnull(test.t1.a))",
" └─IndexRangeScan 12500.00 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, test.t2.a)], keep order:false, stats:pseudo"
],
"Warning": [
"Warning 1815 Some INL_HASH_JOIN and NO_INDEX_HASH_JOIN hints conflict, NO_INDEX_HASH_JOIN may be ignored"
]
},
{
"SQL": "select /*+ inl_merge_join(t1), no_index_merge_join(t1, t2) */ * from t1, t2 where t1.a=t2.a",
"Plan": [
"IndexMergeJoin 12487.50 root inner join, inner:IndexReader, outer key:test.t2.a, inner key:test.t1.a",
"├─IndexReader(Build) 9990.00 root index:IndexFullScan",
"│ └─IndexFullScan 9990.00 cop[tikv] table:t2, index:a(a) keep order:false, stats:pseudo",
"└─IndexReader(Probe) 12487.50 root index:Selection",
" └─Selection 12487.50 cop[tikv] not(isnull(test.t1.a))",
" └─IndexRangeScan 12500.00 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, test.t2.a)], keep order:true, stats:pseudo"
],
"Warning": [
"Warning 1815 Some INL_MERGE_JOIN and NO_INDEX_MERGE_JOIN hints conflict, NO_INDEX_MERGE_JOIN may be ignored"
]
},
{
"SQL": "select /*+ inl_join(t1), no_index_hash_join(t1) */ * from t1, t2 where t1.a=t2.a",
"Plan": [
"IndexJoin 12487.50 root inner join, inner:IndexReader, outer key:test.t2.a, inner key:test.t1.a, equal cond:eq(test.t2.a, test.t1.a)",
"├─IndexReader(Build) 9990.00 root index:IndexFullScan",
"│ └─IndexFullScan 9990.00 cop[tikv] table:t2, index:a(a) keep order:false, stats:pseudo",
"└─IndexReader(Probe) 12487.50 root index:Selection",
" └─Selection 12487.50 cop[tikv] not(isnull(test.t1.a))",
" └─IndexRangeScan 12500.00 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, test.t2.a)], keep order:false, stats:pseudo"
],
"Warning": null
},
{
"SQL": "select /*+ inl_join(t1), no_index_merge_join(t1) */ * from t1, t2 where t1.a=t2.a",
"Plan": [
"IndexJoin 12487.50 root inner join, inner:IndexReader, outer key:test.t2.a, inner key:test.t1.a, equal cond:eq(test.t2.a, test.t1.a)",
"├─IndexReader(Build) 9990.00 root index:IndexFullScan",
"│ └─IndexFullScan 9990.00 cop[tikv] table:t2, index:a(a) keep order:false, stats:pseudo",
"└─IndexReader(Probe) 12487.50 root index:Selection",
" └─Selection 12487.50 cop[tikv] not(isnull(test.t1.a))",
" └─IndexRangeScan 12500.00 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, test.t2.a)], keep order:false, stats:pseudo"
],
"Warning": null
}
]
},
{
"Name": "TestNoMergeJoinHint",
"Cases": [
Expand Down
37 changes: 35 additions & 2 deletions planner/core/exhaust_physical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -2028,15 +2028,48 @@ func (p *LogicalJoin) tryToGetIndexJoin(prop *property.PhysicalProperty) (indexJ
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.
// Handle hints and variables about index join.
// The priority is: force hints like TIDB_INLJ > filter hints like NO_INDEX_JOIN > variables.
// 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"))
}
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"))
}
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"))
}

candidates, canForced = p.handleForceIndexJoinHints(prop, candidates)
if canForced {
return candidates, canForced
}
candidates = p.handleFilterIndexJoinHints(candidates)
return filterIndexJoinBySessionVars(p.SCtx(), candidates), false
}

func (p *LogicalJoin) handleFilterIndexJoinHints(candidates []PhysicalPlan) []PhysicalPlan {
if !p.preferAny(preferNoIndexJoin, preferNoIndexHashJoin, preferNoIndexMergeJoin) {
return candidates // no filter index join hints
}
filtered := make([]PhysicalPlan, 0, len(candidates))
for _, candidate := range candidates {
_, joinMethod, ok := p.getIndexJoinSideAndMethod(candidate)
if !ok {
continue
}
if (p.preferAny(preferNoIndexJoin) && joinMethod == indexJoinMethod) ||
(p.preferAny(preferNoIndexHashJoin) && joinMethod == indexHashJoinMethod) ||
(p.preferAny(preferNoIndexMergeJoin) && joinMethod == indexMergeJoinMethod) {
continue
}
filtered = append(filtered, candidate)
}
return filtered
}

// 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,
Expand Down
41 changes: 41 additions & 0 deletions planner/core/logical_plan_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -96,6 +96,12 @@ const (
HintINLHJ = "inl_hash_join"
// HintINLMJ is hint enforce index nested loop merge join.
HintINLMJ = "inl_merge_join"
// HintNoIndexJoin is the hint to enforce the query not to use index join.
HintNoIndexJoin = "no_index_join"
// HintNoIndexHashJoin is the hint to enforce the query not to use index hash join.
HintNoIndexHashJoin = "no_index_hash_join"
// HintNoIndexMergeJoin is the hint to enforce the query not to use index merge join.
HintNoIndexMergeJoin = "no_index_merge_join"
// TiDBHashJoin is hint enforce hash join.
TiDBHashJoin = "tidb_hj"
// HintNoHashJoin is the hint to enforce the query not to use hash join.
Expand Down Expand Up @@ -669,6 +675,30 @@ func (p *LogicalJoin) setPreferredJoinTypeAndOrder(hintInfo *tableHintInfo) {
p.preferJoinType |= preferRightAsINLMJInner
p.rightPreferJoinType |= preferINLMJ
}
if hintInfo.ifPreferNoIndexJoin(lhsAlias) {
p.preferJoinType |= preferNoIndexJoin
p.leftPreferJoinType |= preferNoIndexJoin
}
if hintInfo.ifPreferNoIndexJoin(rhsAlias) {
p.preferJoinType |= preferNoIndexJoin
p.rightPreferJoinType |= preferNoIndexJoin
}
if hintInfo.ifPreferNoIndexHashJoin(lhsAlias) {
p.preferJoinType |= preferNoIndexHashJoin
p.leftPreferJoinType |= preferNoIndexHashJoin
}
if hintInfo.ifPreferNoIndexHashJoin(rhsAlias) {
p.preferJoinType |= preferNoIndexHashJoin
p.rightPreferJoinType |= preferNoIndexHashJoin
}
if hintInfo.ifPreferNoIndexMergeJoin(lhsAlias) {
p.preferJoinType |= preferNoIndexMergeJoin
p.leftPreferJoinType |= preferNoIndexMergeJoin
}
if hintInfo.ifPreferNoIndexMergeJoin(rhsAlias) {
p.preferJoinType |= preferNoIndexMergeJoin
p.rightPreferJoinType |= preferNoIndexMergeJoin
}
if hintInfo.ifPreferHJBuild(lhsAlias) {
p.preferJoinType |= preferLeftAsHJBuild
p.leftPreferJoinType |= preferHJBuild
Expand Down Expand Up @@ -3730,6 +3760,7 @@ func (b *PlanBuilder) pushTableHints(hints []*ast.TableOptimizerHint, currentLev
hints = b.hintProcessor.GetCurrentStmtHints(hints, currentLevel)
var (
sortMergeTables, inljTables, inlhjTables, inlmjTables, hashJoinTables, bcTables []hintTableInfo
noIndexJoinTables, noIndexHashJoinTables, noIndexMergeJoinTables []hintTableInfo
noHashJoinTables, noMergeJoinTables []hintTableInfo
shuffleJoinTables []hintTableInfo
indexHintList, indexMergeHintList []indexHintInfo
Expand Down Expand Up @@ -3772,6 +3803,12 @@ func (b *PlanBuilder) pushTableHints(hints []*ast.TableOptimizerHint, currentLev
noHashJoinTables = append(noHashJoinTables, tableNames2HintTableInfo(b.ctx, hint.HintName.L, hint.Tables, b.hintProcessor, currentLevel)...)
case HintNoMergeJoin:
noMergeJoinTables = append(noMergeJoinTables, tableNames2HintTableInfo(b.ctx, hint.HintName.L, hint.Tables, b.hintProcessor, currentLevel)...)
case HintNoIndexJoin:
noIndexJoinTables = append(noIndexJoinTables, tableNames2HintTableInfo(b.ctx, hint.HintName.L, hint.Tables, b.hintProcessor, currentLevel)...)
case HintNoIndexHashJoin:
noIndexHashJoinTables = append(noIndexHashJoinTables, tableNames2HintTableInfo(b.ctx, hint.HintName.L, hint.Tables, b.hintProcessor, currentLevel)...)
case HintNoIndexMergeJoin:
noIndexMergeJoinTables = append(noIndexMergeJoinTables, tableNames2HintTableInfo(b.ctx, hint.HintName.L, hint.Tables, b.hintProcessor, currentLevel)...)
case HintMPP1PhaseAgg:
aggHints.preferAggType |= preferMPP1PhaseAgg
case HintMPP2PhaseAgg:
Expand Down Expand Up @@ -3881,6 +3918,7 @@ func (b *PlanBuilder) pushTableHints(hints []*ast.TableOptimizerHint, currentLev
broadcastJoinTables: bcTables,
shuffleJoinTables: shuffleJoinTables,
indexNestedLoopJoinTables: indexNestedLoopJoinTables{inljTables, inlhjTables, inlmjTables},
noIndexJoinTables: indexNestedLoopJoinTables{noIndexJoinTables, noIndexHashJoinTables, noIndexMergeJoinTables},
hashJoinTables: hashJoinTables,
noHashJoinTables: noHashJoinTables,
noMergeJoinTables: noMergeJoinTables,
Expand Down Expand Up @@ -7091,6 +7129,9 @@ func getInnerFromParenthesesAndUnaryPlus(expr ast.ExprNode) ast.ExprNode {
func containDifferentJoinTypes(preferJoinType uint) bool {
preferJoinType &= ^preferNoHashJoin
preferJoinType &= ^preferNoMergeJoin
preferJoinType &= ^preferNoIndexJoin
preferJoinType &= ^preferNoIndexHashJoin
preferJoinType &= ^preferNoIndexMergeJoin

inlMask := preferRightAsINLJInner ^ preferLeftAsINLJInner
inlhjMask := preferRightAsINLHJInner ^ preferLeftAsINLHJInner
Expand Down
3 changes: 3 additions & 0 deletions planner/core/logical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -121,6 +121,9 @@ const (
preferNoHashJoin
preferMergeJoin
preferNoMergeJoin
preferNoIndexJoin
preferNoIndexHashJoin
preferNoIndexMergeJoin
preferBCJoin
preferShuffleJoin
preferRewriteSemiJoin
Expand Down
Loading

0 comments on commit e21efe5

Please sign in to comment.