Skip to content

Commit

Permalink
planner: deprecate index lookup merge join. (#54681)
Browse files Browse the repository at this point in the history
close #54064
  • Loading branch information
AilinKid authored Jul 25, 2024
1 parent 1acb8f7 commit 628b7ed
Show file tree
Hide file tree
Showing 21 changed files with 133 additions and 116 deletions.
5 changes: 0 additions & 5 deletions pkg/executor/join/index_lookup_merge_join.go
Original file line number Diff line number Diff line change
Expand Up @@ -443,11 +443,6 @@ func (imw *innerMergeWorker) handleTask(ctx context.Context, task *lookUpMergeJo
}
}
task.memTracker.Consume(int64(cap(task.outerOrderIdx)))
failpoint.Inject("IndexMergeJoinMockOOM", func(val failpoint.Value) {
if val.(bool) {
panic("OOM test index merge join doesn't hang here.")
}
})
// NeedOuterSort means the outer side property items can't guarantee the order of join keys.
// Because the necessary condition of merge join is both outer and inner keep order of join keys.
// In this case, we need sort the outer side.
Expand Down
34 changes: 15 additions & 19 deletions pkg/executor/join/index_lookup_merge_join_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,25 +22,6 @@ import (
"github.com/stretchr/testify/require"
)

func TestIndexLookupMergeJoinHang(t *testing.T) {
require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/executor/join/IndexMergeJoinMockOOM", `return(true)`))
defer func() {
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/executor/join/IndexMergeJoinMockOOM"))
}()
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t1, t2")
tk.MustExec("create table t1 (a int,b int,index idx(a))")
tk.MustExec("create table t2 (a int,b int,index idx(a))")
tk.MustExec("insert into t1 values (1,1),(2,2),(3,3),(2000,2000)")
tk.MustExec("insert into t2 values (1,1),(2,2),(3,3),(2000,2000)")
// Do not hang in index merge join when OOM occurs.
err := tk.QueryToErr("select /*+ INL_MERGE_JOIN(t1, t2) */ * from t1, t2 where t1.a = t2.a")
require.Error(t, err)
require.Equal(t, "OOM test index merge join doesn't hang here.", err.Error())
}

func TestIssue18068(t *testing.T) {
require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/executor/join/testIssue18068", `return(true)`))
defer func() {
Expand All @@ -64,3 +45,18 @@ func TestIssue18068(t *testing.T) {
tk.MustExec("select /*+ inl_merge_join(s)*/ 1 from t join s on t.a = s.a limit 1")
tk.MustExec("select /*+ inl_merge_join(s)*/ 1 from t join s on t.a = s.a limit 1")
}

func TestIssue54064(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("drop table if exists A, B")
tk.MustExec("create table A\n(id int primary key nonclustered auto_increment,\nx varchar(32) not null,\ny char(5) not null,\nz varchar(25) not null,\nkey idx_sub_tsk(z,x,y)\n)")
tk.MustExec("create table B\n( y char(5) not null,\nz varchar(25) not null,\nx varchar(32) not null,\nprimary key(z, x, y) nonclustered\n)\n")
tk.MustExec("insert into A (y, z, x) values\n('CN000', '123', 'RW '),\n('CN000', '456', '123');")
tk.MustExec("insert into B values\n('CN000', '123', 'RW '),\n('CN000', '456', '123');")
tk.MustQuery("select /*+ inl_merge_join(a, b) */\na.*\nfrom a join b on a.y=b.y and a.z=b.z and a.x = b.x\nwhere a.y='CN000' order by 1,2;").Check(
testkit.Rows("1 RW CN000 123", "2 123 CN000 456"))
res := tk.MustQuery("explain format='brief' select /*+ inl_merge_join(a, b) */\na.*\nfrom a join b on a.y=b.y and a.z=b.z and a.x = b.x\nwhere a.y='CN000' order by 1,2;")
require.NotRegexp(t, "IndexMergeJoin_.*", res.Rows()[0][0])
}
16 changes: 8 additions & 8 deletions pkg/executor/partition_table_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1184,49 +1184,49 @@ func TestPartitionTableWithDifferentJoin(t *testing.T) {
// tk.MustHavePlan(queryHash, "IndexMergeJoin")
// tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows())
tk.MustQuery(queryHash)
tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|1815|Optimizer Hint /*+ INL_MERGE_JOIN(trange, trange2) */ is inapplicable"))
tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|1815|The INDEX MERGE JOIN hint is deprecated for usage, try other hints."))

queryHash = fmt.Sprintf("select /*+ inl_merge_join(trange, trange2) */ * from trange, trange2 where trange.a=trange2.a and trange.a > %v and trange2.a > %v;", x1, x2)
// queryRegular = fmt.Sprintf("select /*+ inl_merge_join(tregular2, tregular4) */ * from tregular2, tregular4 where tregular2.a=tregular4.a and tregular2.a > %v and tregular4.a > %v;", x1, x2)
// tk.MustHavePlan(queryHash, "IndexMergeJoin")
// tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows())
tk.MustQuery(queryHash)
tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|1815|Optimizer Hint /*+ INL_MERGE_JOIN(trange, trange2) */ is inapplicable"))
tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|1815|The INDEX MERGE JOIN hint is deprecated for usage, try other hints."))

queryHash = fmt.Sprintf("select /*+ inl_merge_join(trange, trange2) */ * from trange, trange2 where trange.a=trange2.a and trange.a > %v and trange.b > %v;", x1, x2)
// queryRegular = fmt.Sprintf("select /*+ inl_merge_join(tregular2, tregular4) */ * from tregular2, tregular4 where tregular2.a=tregular4.a and tregular2.a > %v and tregular2.b > %v;", x1, x2)
// tk.MustHavePlan(queryHash, "IndexMergeJoin")
// tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows())
tk.MustQuery(queryHash)
tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|1815|Optimizer Hint /*+ INL_MERGE_JOIN(trange, trange2) */ is inapplicable"))
tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|1815|The INDEX MERGE JOIN hint is deprecated for usage, try other hints."))

queryHash = fmt.Sprintf("select /*+ inl_merge_join(trange, trange2) */ * from trange, trange2 where trange.a=trange2.a and trange.a > %v and trange2.b > %v;", x1, x2)
// queryRegular = fmt.Sprintf("select /*+ inl_merge_join(tregular2, tregular4) */ * from tregular2, tregular4 where tregular2.a=tregular4.a and tregular2.a > %v and tregular4.b > %v;", x1, x2)
// tk.MustHavePlan(queryHash, "IndexMergeJoin")
// tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows())
tk.MustQuery(queryHash)
tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|1815|Optimizer Hint /*+ INL_MERGE_JOIN(trange, trange2) */ is inapplicable"))
tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|1815|The INDEX MERGE JOIN hint is deprecated for usage, try other hints."))

// group 6
// index_merge_join range partition and regualr table
queryHash = fmt.Sprintf("select /*+ inl_merge_join(trange, tregular4) */ * from trange, tregular4 where trange.a=tregular4.a and trange.a > %v;", x1)
queryRegular = fmt.Sprintf("select /*+ inl_merge_join(tregular2, tregular4) */ * from tregular2, tregular4 where tregular2.a=tregular4.a and tregular2.a > %v;", x1)
tk.MustHavePlan(queryHash, "IndexMergeJoin")
tk.MustNotHavePlan(queryHash, "IndexMergeJoin")
tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows())

queryHash = fmt.Sprintf("select /*+ inl_merge_join(trange, tregular4) */ * from trange, tregular4 where trange.a=tregular4.a and trange.a > %v and tregular4.a > %v;", x1, x2)
queryRegular = fmt.Sprintf("select /*+ inl_merge_join(tregular2, tregular4) */ * from tregular2, tregular4 where tregular2.a=tregular4.a and tregular2.a > %v and tregular4.a > %v;", x1, x2)
tk.MustHavePlan(queryHash, "IndexMergeJoin")
tk.MustNotHavePlan(queryHash, "IndexMergeJoin")
tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows())

queryHash = fmt.Sprintf("select /*+ inl_merge_join(trange, tregular4) */ * from trange, tregular4 where trange.a=tregular4.a and trange.a > %v and trange.b > %v;", x1, x2)
queryRegular = fmt.Sprintf("select /*+ inl_merge_join(tregular2, tregular4) */ * from tregular2, tregular4 where tregular2.a=tregular4.a and tregular2.a > %v and tregular2.b > %v;", x1, x2)
tk.MustHavePlan(queryHash, "IndexMergeJoin")
tk.MustNotHavePlan(queryHash, "IndexMergeJoin")
tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows())

queryHash = fmt.Sprintf("select /*+ inl_merge_join(trange, tregular4) */ * from trange, tregular4 where trange.a=tregular4.a and trange.a > %v and tregular4.b > %v;", x1, x2)
queryRegular = fmt.Sprintf("select /*+ inl_merge_join(tregular2, tregular4) */ * from tregular2, tregular4 where tregular2.a=tregular4.a and tregular2.a > %v and tregular4.b > %v;", x1, x2)
tk.MustHavePlan(queryHash, "IndexMergeJoin")
tk.MustNotHavePlan(queryHash, "IndexMergeJoin")
tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows())

// group 7
Expand Down
5 changes: 0 additions & 5 deletions pkg/executor/test/jointest/hashjoin/hash_join_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -378,11 +378,6 @@ func TestExplainAnalyzeJoin(t *testing.T) {
require.Equal(t, 7, len(rows))
require.Regexp(t, "HashJoin.*", rows[0][0])
require.Regexp(t, "time:.*, loops:.*, build_hash_table:{total:.*, fetch:.*, build:.*}, probe:{concurrency:5, total:.*, max:.*, probe:.*, fetch and wait:.*}", rows[0][5])
// Test for index merge join.
rows = tk.MustQuery("explain analyze select /*+ INL_MERGE_JOIN(t1, t2) */ * from t1,t2 where t1.a=t2.a;").Rows()
require.Len(t, rows, 9)
require.Regexp(t, "IndexMergeJoin_.*", rows[0][0])
require.Regexp(t, fmt.Sprintf(".*Concurrency:%v.*", tk.Session().GetSessionVars().IndexLookupJoinConcurrency()), rows[0][5])

// TestExplainAnalyzeIndexHashJoin
// Issue 43597
Expand Down
4 changes: 2 additions & 2 deletions pkg/executor/test/jointest/join_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -136,10 +136,10 @@ func TestJoin2(t *testing.T) {
// The physical plans of the two sql are tested at physical_plan_test.go
tk.MustQuery("select /*+ INL_JOIN(t, t1) */ * from t join t1 on t.a=t1.a").Check(testkit.Rows("1 1 1 2", "1 1 1 3", "1 1 1 4", "3 3 3 4"))
tk.MustQuery("select /*+ INL_HASH_JOIN(t, t1) */ * from t join t1 on t.a=t1.a").Sort().Check(testkit.Rows("1 1 1 2", "1 1 1 3", "1 1 1 4", "3 3 3 4"))
tk.MustQuery("select /*+ INL_MERGE_JOIN(t, t1) */ * from t join t1 on t.a=t1.a").Check(testkit.Rows("1 1 1 2", "1 1 1 3", "1 1 1 4", "3 3 3 4"))
tk.MustQuery("select /*+ INL_MERGE_JOIN(t, t1) */ * from t join t1 on t.a=t1.a").Check(testkit.Rows("1 1 1 4", "1 1 1 3", "1 1 1 2", "3 3 3 4"))
tk.MustQuery("select /*+ INL_JOIN(t) */ * from t1 join t on t.a=t1.a and t.a < t1.b").Check(testkit.Rows("1 2 1 1", "1 3 1 1", "1 4 1 1", "3 4 3 3"))
tk.MustQuery("select /*+ INL_HASH_JOIN(t) */ * from t1 join t on t.a=t1.a and t.a < t1.b").Sort().Check(testkit.Rows("1 2 1 1", "1 3 1 1", "1 4 1 1", "3 4 3 3"))
tk.MustQuery("select /*+ INL_MERGE_JOIN(t) */ * from t1 join t on t.a=t1.a and t.a < t1.b").Check(testkit.Rows("1 2 1 1", "1 3 1 1", "1 4 1 1", "3 4 3 3"))
tk.MustQuery("select /*+ INL_MERGE_JOIN(t) */ * from t1 join t on t.a=t1.a and t.a < t1.b").Check(testkit.Rows("1 4 1 1", "1 3 1 1", "1 2 1 1", "3 4 3 3"))
// Test single index reader.
tk.MustQuery("select /*+ INL_JOIN(t, t1) */ t1.b from t1 join t on t.b=t1.b").Check(testkit.Rows("2", "3"))
tk.MustQuery("select /*+ INL_HASH_JOIN(t, t1) */ t1.b from t1 join t on t.b=t1.b").Sort().Check(testkit.Rows("2", "3"))
Expand Down
17 changes: 15 additions & 2 deletions pkg/planner/core/casetest/physicalplantest/physical_plan_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1114,8 +1114,9 @@ func TestIndexJoinHint(t *testing.T) {

var input []string
var output []struct {
SQL string
Plan string
SQL string
Plan string
Warns []string
}

is := domain.GetDomain(tk.Session()).InfoSchema()
Expand All @@ -1124,6 +1125,16 @@ func TestIndexJoinHint(t *testing.T) {

planSuiteData := GetPlanSuiteData()
planSuiteData.LoadTestCases(t, &input, &output)
filterWarnings := func(originalWarnings []contextutil.SQLWarn) []contextutil.SQLWarn {
warnings := make([]contextutil.SQLWarn, 0, 4)
for _, warning := range originalWarnings {
// filter out warning about skyline pruning
if !strings.Contains(warning.Err.Error(), "remain after pruning paths for") {
warnings = append(warnings, warning)
}
}
return warnings
}
for i, tt := range input {
comment := fmt.Sprintf("case:%v sql: %s", i, tt)
stmt, err := p.ParseOneStmt(tt, "", "")
Expand All @@ -1133,7 +1144,9 @@ func TestIndexJoinHint(t *testing.T) {
testdata.OnRecord(func() {
output[i].SQL = tt
output[i].Plan = core.ToString(p)
output[i].Warns = testdata.ConvertSQLWarnToStrings(filterWarnings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()))
})
tk.Session().GetSessionVars().StmtCtx.TruncateWarnings(0)
require.Equal(t, output[i].Plan, core.ToString(p), comment)
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2543,23 +2543,34 @@
"Cases": [
{
"SQL": "select /*+ INL_JOIN(t1) */ * from t1 join t2 on t1.a = t2.a;",
"Plan": "IndexJoin{IndexLookUp(Index(t1.idx_a)[[NULL,NULL]]->Sel([not(isnull(test.t1.a))]), Table(t1))->TableReader(Table(t2)->Sel([not(isnull(test.t2.a))]))}(test.t2.a,test.t1.a)"
"Plan": "IndexJoin{IndexLookUp(Index(t1.idx_a)[[NULL,NULL]]->Sel([not(isnull(test.t1.a))]), Table(t1))->TableReader(Table(t2)->Sel([not(isnull(test.t2.a))]))}(test.t2.a,test.t1.a)",
"Warns": null
},
{
"SQL": "select /*+ INL_HASH_JOIN(t1) */ * from t1 join t2 on t1.a = t2.a;",
"Plan": "IndexHashJoin{IndexLookUp(Index(t1.idx_a)[[NULL,NULL]]->Sel([not(isnull(test.t1.a))]), Table(t1))->TableReader(Table(t2)->Sel([not(isnull(test.t2.a))]))}(test.t2.a,test.t1.a)"
"Plan": "IndexHashJoin{IndexLookUp(Index(t1.idx_a)[[NULL,NULL]]->Sel([not(isnull(test.t1.a))]), Table(t1))->TableReader(Table(t2)->Sel([not(isnull(test.t2.a))]))}(test.t2.a,test.t1.a)",
"Warns": null
},
{
"SQL": "select /*+ INL_MERGE_JOIN(t1) */ * from t1 join t2 on t1.a = t2.a;",
"Plan": "IndexMergeJoin{IndexLookUp(Index(t1.idx_a)[[NULL,NULL]]->Sel([not(isnull(test.t1.a))]), Table(t1))->Projection->TableReader(Table(t2)->Sel([not(isnull(test.t2.a))]))}(test.t2.a,test.t1.a)"
"Plan": "LeftHashJoin{TableReader(Table(t1)->Sel([not(isnull(test.t1.a))]))->TableReader(Table(t2)->Sel([not(isnull(test.t2.a))]))}(test.t1.a,test.t2.a)",
"Warns": [
"[planner:1815]The INDEX MERGE JOIN hint is deprecated for usage, try other hints."
]
},
{
"SQL": "select /*+ inl_merge_join(t2) */ t1.a, t2.a from t t1 left join t t2 use index(g_2) on t1.g=t2.g",
"Plan": "IndexMergeJoin{IndexReader(Index(t.g_2)[[NULL,+inf]])->IndexReader(Index(t.g_2)[[NULL,NULL]]->Sel([not(isnull(test.t.g))]))}(test.t.g,test.t.g)"
"Plan": "MergeLeftOuterJoin{IndexReader(Index(t.g_2)[[NULL,+inf]])->IndexReader(Index(t.g_2)[[-inf,+inf]])}(test.t.g,test.t.g)",
"Warns": [
"[planner:1815]The INDEX MERGE JOIN hint is deprecated for usage, try other hints."
]
},
{
"SQL": "select /*+inl_merge_join(t2)*/ t1.a, t2.a from t t1 left join t t2 use index(g_2) on t1.g=t2.g order by t1.a",
"Plan": "IndexMergeJoin{IndexReader(Index(t.g_2)[[NULL,+inf]])->IndexReader(Index(t.g_2)[[NULL,NULL]]->Sel([not(isnull(test.t.g))]))}(test.t.g,test.t.g)->Sort"
"Plan": "IndexHashJoin{TableReader(Table(t))->IndexReader(Index(t.g_2)[[NULL,NULL]]->Sel([not(isnull(test.t.g))]))}(test.t.g,test.t.g)",
"Warns": [
"[planner:1815]The INDEX MERGE JOIN hint is deprecated for usage, try other hints."
]
}
]
},
Expand Down Expand Up @@ -2686,8 +2697,8 @@
},
{
"SQL": "select /*+ INL_MERGE_JOIN(t1) */ t1.b, t2.b from t1 inner join t2 on t1.a = t2.a;",
"Plan": "IndexMergeJoin{IndexLookUp(Index(t1.idx_a)[[NULL,NULL]]->Sel([not(isnull(test.t1.a))]), Table(t1))->Projection->TableReader(Table(t2)->Sel([not(isnull(test.t2.a))]))}(test.t2.a,test.t1.a)",
"Hints": "inl_merge_join(@`sel_1` `test`.`t1`), use_index(@`sel_1` `test`.`t1` `idx_a`), order_index(@`sel_1` `test`.`t1` `idx_a`), use_index(@`sel_1` `test`.`t2` )"
"Plan": "LeftHashJoin{TableReader(Table(t1)->Sel([not(isnull(test.t1.a))]))->TableReader(Table(t2)->Sel([not(isnull(test.t2.a))]))}(test.t1.a,test.t2.a)",
"Hints": "hash_join(@`sel_1` `test`.`t1`), use_index(@`sel_1` `test`.`t1` ), use_index(@`sel_1` `test`.`t2` )"
},
{
"SQL": "select /*+ MERGE_JOIN(t1) */ t1.b, t2.b from t1 inner join t2 on t1.a = t2.a;",
Expand Down
7 changes: 5 additions & 2 deletions pkg/util/hint/hint.go
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,7 @@ const (
HintINLJ = "inl_join"
// HintINLHJ is hint enforce index nested loop hash join.
HintINLHJ = "inl_hash_join"
// HintINLMJ is hint enforce index nested loop merge join.
// Deprecated: 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"
Expand Down Expand Up @@ -776,7 +776,10 @@ func ParsePlanHints(hints []*ast.TableOptimizerHint,
case HintINLHJ:
inlhjTables = append(inlhjTables, tableNames2HintTableInfo(currentDB, hint.HintName.L, hint.Tables, hintProcessor, currentLevel, warnHandler)...)
case HintINLMJ:
inlmjTables = append(inlmjTables, tableNames2HintTableInfo(currentDB, hint.HintName.L, hint.Tables, hintProcessor, currentLevel, warnHandler)...)
if hint.Tables != nil {
warnHandler.SetHintWarning("The INDEX MERGE JOIN hint is deprecated for usage, try other hints.")
continue
}
case TiDBHashJoin, HintHJ:
hashJoinTables = append(hashJoinTables, tableNames2HintTableInfo(currentDB, hint.HintName.L, hint.Tables, hintProcessor, currentLevel, warnHandler)...)
case HintNoHashJoin:
Expand Down
8 changes: 4 additions & 4 deletions tests/integrationtest/r/executor/index_lookup_join.result
Original file line number Diff line number Diff line change
Expand Up @@ -90,12 +90,12 @@ select /*+ INL_MERGE_JOIN(t1, t2) */ * from t1, t2;
a b a b
show warnings;
Level Code Message
Warning 1815 Optimizer Hint /*+ INL_MERGE_JOIN(t1, t2) */ is inapplicable without column equal ON condition
Warning 1815 The INDEX MERGE JOIN hint is deprecated for usage, try other hints.
select /*+ INL_MERGE_JOIN(t1, t2) */ * from t1 join t2 on t1.a=t2.a;
a b a b
show warnings;
Level Code Message
Warning 1815 Optimizer Hint /*+ INL_MERGE_JOIN(t1, t2) */ is inapplicable
Warning 1815 The INDEX MERGE JOIN hint is deprecated for usage, try other hints.
drop table if exists t1, t2;
create table t1(a bigint, b bigint, index idx_a(a));
create table t2(a bigint, b bigint);
Expand Down Expand Up @@ -123,12 +123,12 @@ select /*+ INL_MERGE_JOIN(t1) */ * from t1 left join t2 on t1.a=t2.a;
a b a b
show warnings;
Level Code Message
Warning 1815 Optimizer Hint /*+ INL_MERGE_JOIN(t1) */ is inapplicable
Warning 1815 The INDEX MERGE JOIN hint is deprecated for usage, try other hints.
select /*+ INL_MERGE_JOIN(t2) */ * from t1 right join t2 on t1.a=t2.a;
a b a b
show warnings;
Level Code Message
Warning 1815 Optimizer Hint /*+ INL_MERGE_JOIN(t2) */ is inapplicable
Warning 1815 The INDEX MERGE JOIN hint is deprecated for usage, try other hints.
drop table if exists t1, t2;
create table t1 (a int, key(a));
create table t2 (a int, key(a));
Expand Down
Loading

0 comments on commit 628b7ed

Please sign in to comment.