diff --git a/planner/core/casetest/rule/BUILD.bazel b/planner/core/casetest/rule/BUILD.bazel new file mode 100644 index 0000000000000..b735973b2d3bd --- /dev/null +++ b/planner/core/casetest/rule/BUILD.bazel @@ -0,0 +1,35 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_test") + +go_test( + name = "rule_test", + timeout = "short", + srcs = [ + "main_test.go", + "rule_derive_topn_from_window_test.go", + "rule_inject_extra_projection_test.go", + "rule_join_reorder_test.go", + "rule_result_reorder_test.go", + ], + data = glob(["testdata/**"]), + flaky = True, + shard_count = 21, + deps = [ + "//domain", + "//expression", + "//expression/aggregation", + "//parser/ast", + "//parser/model", + "//parser/mysql", + "//planner/core/internal", + "//sessionctx/variable", + "//testkit", + "//testkit/testdata", + "//testkit/testmain", + "//testkit/testsetup", + "//types", + "//util/mock", + "@com_github_pingcap_failpoint//:failpoint", + "@com_github_stretchr_testify//require", + "@org_uber_go_goleak//:goleak", + ], +) diff --git a/planner/core/casetest/rule/rule_join_reorder_test.go b/planner/core/casetest/rule/rule_join_reorder_test.go new file mode 100644 index 0000000000000..140740aebf679 --- /dev/null +++ b/planner/core/casetest/rule/rule_join_reorder_test.go @@ -0,0 +1,316 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package rule + +import ( + "testing" + + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/testdata" + "github.com/stretchr/testify/require" +) + +func runJoinReorderTestData(t *testing.T, tk *testkit.TestKit, name string) { + var input []string + var output []struct { + SQL string + Plan []string + Warning []string + } + joinReorderSuiteData := GetJoinReorderSuiteData() + joinReorderSuiteData.LoadTestCasesByName(name, t, &input, &output) + require.Equal(t, len(input), len(output)) + for i := range input { + testdata.OnRecord(func() { + output[i].SQL = input[i] + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + input[i]).Rows()) + output[i].Warning = testdata.ConvertRowsToStrings(tk.MustQuery("show warnings").Rows()) + }) + tk.MustQuery("explain format = 'brief' " + input[i]).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery("show warnings").Check(testkit.Rows(output[i].Warning...)) + } +} + +func TestStraightJoinHint(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("drop table if exists t, t1, t2, t3, t4;") + tk.MustExec("create table t(a int, b int, key(a));") + tk.MustExec("create table t1(a int, b int, key(a));") + tk.MustExec("create table t2(a int, b int, key(a));") + tk.MustExec("create table t3(a int, b int, key(a));") + tk.MustExec("create table t4(a int, b int, key(a));") + runJoinReorderTestData(t, tk, "TestStraightJoinHint") +} + +func TestNoHashJoinHint(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t1(a int, b int, key(a));") + tk.MustExec("create table t2(a int, b int, key(a));") + tk.MustExec("create table t3(a int, b int, key(a));") + tk.MustExec("create table t4(a int, b int, key(a));") + runJoinReorderTestData(t, tk, "TestNoHashJoinHint") +} + +func TestLeadingJoinHint(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("drop table if exists t, t1, t2, t3, t4, t5, t6, t7, t8;") + tk.MustExec("create table t(a int, b int, key(a));") + tk.MustExec("create table t1(a int, b int, key(a));") + tk.MustExec("create table t2(a int, b int, key(a));") + tk.MustExec("create table t3(a int, b int, key(a));") + tk.MustExec("create table t4(a int, b int, key(a));") + tk.MustExec("create table t5(a int, b int, key(a));") + tk.MustExec("create table t6(a int, b int, key(a));") + tk.MustExec("create table t7(a int, b int, key(a));") + tk.MustExec("create table t8(a int, b int, key(a));") + runJoinReorderTestData(t, tk, "TestLeadingJoinHint") + + // test cases for multiple leading hints + tk.MustExec("select /*+ leading(t1) leading(t2) */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid")) +} + +func TestJoinOrderHint(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t, t1, t2, t3, t4, t5, t6, t7, t8;") + tk.MustExec("create table t(a int, b int, key(a));") + tk.MustExec("create table t1(a int, b int, key(a));") + tk.MustExec("create table t2(a int, b int, key(a));") + tk.MustExec("create table t3(a int, b int, key(a));") + tk.MustExec("create table t4(a int, b int, key(a));") + tk.MustExec("create table t5(a int, b int, key(a));") + tk.MustExec("create table t6(a int, b int, key(a));") + tk.MustExec("create table t7(a int, b int, key(a));") + tk.MustExec("create table t8(a int, b int, key(a));") + + // test cases for using the leading hint and straight_join hint at the same time + tk.MustExec("select /*+ leading(t1) straight_join() */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 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")) + + tk.MustExec("select /*+ straight_join() leading(t1) */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 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")) + + // more join order hints appear in the same time + tk.MustExec("select /*+ leading(t1) leading(t1) */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid")) + + tk.MustExec("select /*+ leading(t1) leading(t2) */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid")) + + tk.MustExec("select /*+ straight_join() straight_join() */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 STRAIGHT_JOIN() is defined more than once, only the last definition takes effect")) + + // test cases for table name in hint + // the same table appears in the leading hint + tk.MustExec("select /*+ leading(t1, t1) */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 There are no matching table names for (t1) in optimizer hint /*+ LEADING(t1, t1) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid")) + + tk.MustExec("select /*+ leading(t1, t2, t1) */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 There are no matching table names for (t1) in optimizer hint /*+ LEADING(t1, t2, t1) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid")) + + // the wrong table appears in the leading hint + tk.MustExec("select /*+ leading(t) */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 There are no matching table names for (t) in optimizer hint /*+ LEADING(t) */. Maybe you can use the table alias name")) + + tk.MustExec("select /*+ leading(t1, t2, t) */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 There are no matching table names for (t) in optimizer hint /*+ LEADING(t1, t2, t) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid")) + + // table alias in the leading hint + tk.MustExec("select /*+ leading(t) */ * from t1 t join t2 on t.a=t2.a join t3 on t2.b=t3.b") + tk.MustQuery("show warnings").Check(testkit.Rows()) + + tk.MustExec("select /*+ leading(t1) */ * from t1 t join t2 on t.a=t2.a join t3 on t2.b=t3.b") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 There are no matching table names for (t1) in optimizer hint /*+ LEADING(t1) */. Maybe you can use the table alias name")) + + tk.MustExec("select /*+ leading(t2, t) */ * from t1 t join t2 on t.a=t2.a join t3 on t2.b=t3.b") + tk.MustQuery("show warnings").Check(testkit.Rows()) + + tk.MustExec("select /*+ leading(t2, t1) */ * from t1 t join t2 on t.a=t2.a join t3 on t2.b=t3.b") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 There are no matching table names for (t1) in optimizer hint /*+ LEADING(t2, t1) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid")) + + // table name in leading hint cross query block + // Todo: Can not handle this case yet. Because when we extract the join group, it will get the join group {t1, t2, t3}. + // So the table 't4' can not be used. + tk.MustExec("select /*+ leading(t4) */ * from (select t2.b from t1 join t2 on t1.a=t2.a) t4 join t3 on t4.b=t3.b") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 leading hint is inapplicable, check if the leading hint table is valid")) + + tk.MustExec("select /*+ leading(t3, t2@sel_2) */ * from (select t2.b from t1 join t2 on t1.a=t2.a) t4 join t3 on t4.b=t3.b") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 There are no matching table names for (t2) in optimizer hint /*+ LEADING(t3, t2) */. Maybe you can use the table alias name")) + + tk.MustExec("select * from (select /*+ leading(t1, t3@sel_1) */ t2.b from t1 join t2 on t1.a=t2.a) t4 join t3 on t4.b=t3.b") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t1, t3) */. Maybe you can use the table alias name")) + + tk.MustExec("select /*+ leading(t3) */ * from (select /*+ leading(t1) */ t2.b from t1 join t2 on t1.a=t2.a) t4 join t3 on t4.b=t3.b") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid")) + + runJoinReorderTestData(t, tk, "TestJoinOrderHint") +} + +func TestJoinOrderHint4StaticPartitionTable(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("drop table if exists t, t1, t2, t3;") + tk.MustExec(`create table t(a int, b int) partition by hash(a) partitions 3`) + tk.MustExec(`create table t1(a int, b int) partition by hash(a) partitions 4`) + tk.MustExec(`create table t2(a int, b int) partition by hash(a) partitions 5`) + tk.MustExec(`create table t3(a int, b int) partition by hash(b) partitions 3`) + tk.MustExec(`create table t4(a int, b int) partition by hash(a) partitions 4`) + tk.MustExec(`create table t5(a int, b int) partition by hash(a) partitions 5`) + tk.MustExec(`create table t6(a int, b int) partition by hash(b) partitions 3`) + + tk.MustExec(`set @@tidb_partition_prune_mode="static"`) + tk.MustExec("set @@tidb_enable_outer_join_reorder=true") + runJoinReorderTestData(t, tk, "TestJoinOrderHint4StaticPartitionTable") +} + +func TestJoinOrderHint4DynamicPartitionTable(t *testing.T) { + failpoint.Enable("github.com/pingcap/tidb/planner/core/forceDynamicPrune", `return(true)`) + defer failpoint.Disable("github.com/pingcap/tidb/planner/core/forceDynamicPrune") + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t, t1, t2, t3;") + tk.MustExec(`create table t(a int, b int) partition by hash(a) partitions 3`) + tk.MustExec(`create table t1(a int, b int) partition by hash(a) partitions 4`) + tk.MustExec(`create table t2(a int, b int) partition by hash(a) partitions 5`) + tk.MustExec(`create table t3(a int, b int) partition by hash(b) partitions 3`) + tk.MustExec(`create table t4(a int, b int) partition by hash(a) partitions 4`) + tk.MustExec(`create table t5(a int, b int) partition by hash(a) partitions 5`) + tk.MustExec(`create table t6(a int, b int) partition by hash(b) partitions 3`) + + tk.MustExec(`set @@tidb_partition_prune_mode="dynamic"`) + tk.MustExec("set @@tidb_enable_outer_join_reorder=true") + runJoinReorderTestData(t, tk, "TestJoinOrderHint4DynamicPartitionTable") +} + +func TestJoinOrderHint4DifferentJoinType(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("drop table if exists t, t1, t2, t3, t4, t5, t6, t7, t8;") + tk.MustExec("create table t(a int, b int, key(a));") + tk.MustExec("create table t1(a int, b int, key(a));") + tk.MustExec("create table t2(a int, b int, key(a));") + tk.MustExec("create table t3(a int, b int, key(a));") + tk.MustExec("create table t4(a int, b int, key(a));") + tk.MustExec("create table t5(a int, b int, key(a));") + tk.MustExec("create table t6(a int, b int, key(a));") + tk.MustExec("create table t7(a int, b int, key(a));") + tk.MustExec("create table t8(a int, b int, key(a));") + tk.MustExec("set @@tidb_enable_outer_join_reorder=true") + + runJoinReorderTestData(t, tk, "TestJoinOrderHint4DifferentJoinType") +} + +func TestJoinOrderHint4TiFlash(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t, t1, t2, t3;") + tk.MustExec("create table t(a int, b int, key(a));") + tk.MustExec("create table t1(a int, b int, key(a));") + tk.MustExec("create table t2(a int, b int, key(a));") + tk.MustExec("create table t3(a int, b int, key(a));") + tk.MustExec("create table t4(a int, b int, key(a));") + tk.MustExec("create table t5(a int, b int, key(a));") + tk.MustExec("create table t6(a int, b int, key(a));") + tk.MustExec("set @@tidb_enable_outer_join_reorder=true") + + // Create virtual tiflash replica info. + dom := domain.GetDomain(tk.Session()) + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + tableName := tblInfo.Name.L + if tableName == "t" || tableName == "t1" || tableName == "t2" || tableName == "t3" || tableName == "t4" || tableName == "t5" || tableName == "t6" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1;") + runJoinReorderTestData(t, tk, "TestJoinOrderHint4TiFlash") +} + +func TestJoinOrderHint4Subquery(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("drop table if exists t, t1, t2, t3, t4, t5, t6, t7, t8;") + tk.MustExec("create table t(a int, b int, key(a));") + tk.MustExec("create table t1(a int, b int, key(a));") + tk.MustExec("create table t2(a int, b int, key(a));") + tk.MustExec("create table t3(a int, b int, key(a));") + tk.MustExec("create table t4(a int, b int, key(a));") + tk.MustExec("create table t5(a int, b int, key(a));") + tk.MustExec("create table t6(a int, b int, key(a));") + tk.MustExec("create table t7(a int, b int, key(a));") + tk.MustExec("create table t8(a int, b int, key(a));") + tk.MustExec("insert into t3 values(1, 1), (2, 2), (3, 3);") + tk.MustExec("analyze table t3;") + + runJoinReorderTestData(t, tk, "TestJoinOrderHint4Subquery") +} + +func TestLeadingJoinHint4OuterJoin(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("drop table if exists t, t1, t2, t3, t4, t5, t6, t7, t8;") + tk.MustExec("create table t(a int, b int, key(a));") + tk.MustExec("create table t1(a int, b int, key(a));") + tk.MustExec("create table t2(a int, b int, key(a));") + tk.MustExec("create table t3(a int, b int, key(a));") + tk.MustExec("create table t4(a int, b int, key(a));") + tk.MustExec("create table t5(a int, b int, key(a));") + tk.MustExec("create table t6(a int, b int, key(a));") + tk.MustExec("create table t7(a int, b int, key(a));") + tk.MustExec("create table t8(a int, b int, key(a));") + tk.MustExec("set @@tidb_enable_outer_join_reorder=true") + runJoinReorderTestData(t, tk, "TestLeadingJoinHint4OuterJoin") +} diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 27867f444331d..dc8f692fedf7e 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -229,7 +229,8 @@ func (p *LogicalJoin) GetMergeJoin(prop *property.PhysicalProperty, schema *expr } // If TiDB_SMJ hint is existed, it should consider enforce merge join, // because we can't trust lhsChildProperty completely. - if (p.preferJoinType & preferMergeJoin) > 0 { + if (p.preferJoinType&preferMergeJoin) > 0 || + (p.preferJoinType&preferNoHashJoin) > 0 { // if hash join is not allowed, generate as many other types of join as possible to avoid 'cant-find-plan' error. joins = append(joins, p.getEnforcedMergeJoin(prop, schema, statsInfo)...) } @@ -387,6 +388,7 @@ func (p *LogicalJoin) getHashJoins(prop *property.PhysicalProperty) (joins []Phy forceLeftToBuild = false forceRightToBuild = false } + joins = make([]PhysicalPlan, 0, 2) switch p.JoinType { case SemiJoin, AntiSemiJoin, LeftOuterSemiJoin, AntiLeftOuterSemiJoin: @@ -433,7 +435,15 @@ func (p *LogicalJoin) getHashJoins(prop *property.PhysicalProperty) (joins []Phy } } } + forced = (p.preferJoinType&preferHashJoin > 0) || forceLeftToBuild || forceRightToBuild + noHashJoin := (p.preferJoinType & preferNoHashJoin) > 0 + if !forced && noHashJoin { + return nil, false + } else if forced && noHashJoin { + p.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack( + "Some HASH_JOIN and NO_HASH_JOIN hints conflict, NO_HASH_JOIN is ignored")) + } return } diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 5e5e2edc01b0e..b51e7479cffa9 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -95,6 +95,8 @@ const ( HintINLMJ = "inl_merge_join" // TiDBHashJoin is hint enforce hash join. TiDBHashJoin = "tidb_hj" + // HintNoHashJoin is the hint to enforce the query not to use hash join. + HintNoHashJoin = "no_hash_join" // HintHJ is hint enforce hash join. HintHJ = "hash_join" // HintHashJoinBuild is hint enforce hash join's build side @@ -603,6 +605,14 @@ func (p *LogicalJoin) setPreferredJoinTypeAndOrder(hintInfo *tableHintInfo) { if hintInfo.ifPreferHashJoin(lhsAlias, rhsAlias) { p.preferJoinType |= preferHashJoin } + if hintInfo.ifPreferNoHashJoin(lhsAlias) { + p.preferJoinType |= preferNoHashJoin + p.leftPreferJoinType |= preferNoHashJoin + } + if hintInfo.ifPreferNoHashJoin(rhsAlias) { + p.preferJoinType |= preferNoHashJoin + p.rightPreferJoinType |= preferNoHashJoin + } if hintInfo.ifPreferINLJ(lhsAlias) { p.preferJoinType |= preferLeftAsINLJInner } @@ -3579,6 +3589,7 @@ func (b *PlanBuilder) pushTableHints(hints []*ast.TableOptimizerHint, currentLev hints = b.hintProcessor.GetCurrentStmtHints(hints, currentLevel) var ( sortMergeTables, inljTables, inlhjTables, inlmjTables, hashJoinTables, bcTables []hintTableInfo + noHashJoinTables []hintTableInfo shuffleJoinTables []hintTableInfo indexHintList, indexMergeHintList []indexHintInfo tiflashTables, tikvTables []hintTableInfo @@ -3593,7 +3604,7 @@ func (b *PlanBuilder) pushTableHints(hints []*ast.TableOptimizerHint, currentLev for _, hint := range hints { // Set warning for the hint that requires the table name. switch hint.HintName.L { - case TiDBMergeJoin, HintSMJ, TiDBIndexNestedLoopJoin, HintINLJ, HintINLHJ, HintINLMJ, + case TiDBMergeJoin, HintSMJ, TiDBIndexNestedLoopJoin, HintINLJ, HintINLHJ, HintINLMJ, HintNoHashJoin, TiDBHashJoin, HintHJ, HintUseIndex, HintIgnoreIndex, HintForceIndex, HintOrderIndex, HintNoOrderIndex, HintIndexMerge, HintLeading: if len(hint.Tables) == 0 { b.pushHintWithoutTableWarning(hint) @@ -3616,6 +3627,8 @@ func (b *PlanBuilder) pushTableHints(hints []*ast.TableOptimizerHint, currentLev inlmjTables = append(inlmjTables, tableNames2HintTableInfo(b.ctx, hint.HintName.L, hint.Tables, b.hintProcessor, currentLevel)...) case TiDBHashJoin, HintHJ: hashJoinTables = append(hashJoinTables, tableNames2HintTableInfo(b.ctx, hint.HintName.L, hint.Tables, b.hintProcessor, currentLevel)...) + case HintNoHashJoin: + noHashJoinTables = append(noHashJoinTables, tableNames2HintTableInfo(b.ctx, hint.HintName.L, hint.Tables, b.hintProcessor, currentLevel)...) case HintMPP1PhaseAgg: aggHints.preferAggType |= preferMPP1PhaseAgg case HintMPP2PhaseAgg: @@ -3726,6 +3739,7 @@ func (b *PlanBuilder) pushTableHints(hints []*ast.TableOptimizerHint, currentLev shuffleJoinTables: shuffleJoinTables, indexNestedLoopJoinTables: indexNestedLoopJoinTables{inljTables, inlhjTables, inlmjTables}, hashJoinTables: hashJoinTables, + noHashJoinTables: noHashJoinTables, indexHintList: indexHintList, tiflashTables: tiflashTables, tikvTables: tikvTables, @@ -6940,6 +6954,8 @@ func getInnerFromParenthesesAndUnaryPlus(expr ast.ExprNode) ast.ExprNode { // containDifferentJoinTypes checks whether `preferJoinType` contains different // join types. func containDifferentJoinTypes(preferJoinType uint) bool { + preferJoinType &= ^preferNoHashJoin + inlMask := preferRightAsINLJInner ^ preferLeftAsINLJInner inlhjMask := preferRightAsINLHJInner ^ preferLeftAsINLHJInner inlmjMask := preferRightAsINLMJInner ^ preferLeftAsINLMJInner diff --git a/planner/core/logical_plans.go b/planner/core/logical_plans.go index 8a9cfcf62e41c..dd4e187ac9573 100644 --- a/planner/core/logical_plans.go +++ b/planner/core/logical_plans.go @@ -110,7 +110,25 @@ func (tp JoinType) String() string { } const ( +<<<<<<< HEAD preferLeftAsINLJInner uint = 1 << iota +======= + // Hint flag for join + preferINLJ uint = 1 << iota + preferINLHJ + preferINLMJ + preferHJBuild + preferHJProbe + preferHashJoin + preferNoHashJoin + preferMergeJoin + preferBCJoin + preferShuffleJoin + preferRewriteSemiJoin + + // Hint flag to specify the join with direction + preferLeftAsINLJInner +>>>>>>> 7c2dbbe34e4 (planner: support `no_hash_join` hint on optimizer (#45538)) preferRightAsINLJInner preferLeftAsINLHJInner preferRightAsINLHJInner diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index d48764de0e672..1257e82b3e91c 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -93,6 +93,7 @@ type tableHintInfo struct { broadcastJoinTables []hintTableInfo shuffleJoinTables []hintTableInfo hashJoinTables []hintTableInfo + noHashJoinTables []hintTableInfo indexHintList []indexHintInfo tiflashTables []hintTableInfo tikvTables []hintTableInfo @@ -237,6 +238,10 @@ func (info *tableHintInfo) ifPreferHashJoin(tableNames ...*hintTableInfo) bool { return info.matchTableName(tableNames, info.hashJoinTables) } +func (info *tableHintInfo) ifPreferNoHashJoin(tableNames ...*hintTableInfo) bool { + return info.matchTableName(tableNames, info.noHashJoinTables) +} + func (info *tableHintInfo) ifPreferHJBuild(tableNames ...*hintTableInfo) bool { return info.matchTableName(tableNames, info.hjBuildTables) } diff --git a/planner/core/testdata/join_reorder_suite_in.json b/planner/core/testdata/join_reorder_suite_in.json index 5ef0edb64351b..4beb64b905bfc 100644 --- a/planner/core/testdata/join_reorder_suite_in.json +++ b/planner/core/testdata/join_reorder_suite_in.json @@ -30,6 +30,25 @@ "select /*+ straight_join() */ * from ((select t3.a, t3.b from t3, t2, t1, t where t.a = t1.a and t1.b=t2.b) t3 join t4 on t3.a=t4.a) join (t1 join t2 on t1.a=t2.a) on t1.a=t4.a;" ] }, + { + "name": "TestNoHashJoinHint", + "cases": [ + "select /*+ no_hash_join() */ * from t1, t2", + "select /*+ no_hash_join(t1), hash_join(t1) */ * from t1, t2", + "select /*+ no_hash_join(t1), hash_join(t2) */ * from t1, t2", + "select /*+ no_hash_join(t1) */ * from t1, t2", + "select /*+ no_hash_join(t1, t2) */ * from t1, t2", + "select /*+ no_hash_join(t1) */ * from t1, t2 where t1.a=t2.a", + "select /*+ no_hash_join(t1, t2) */ * from t1, t2 where t1.b=t2.b", + "select /*+ no_hash_join(t1) */ * from t1, t2 where t1.a=t2.a and t1.b=t2.b", + "select /*+ no_hash_join(t2) */ * from t1 left join t2 on t1.b=t2.b", + "select /*+ no_hash_join(t2) */ * from t1 left join t2 on t1.a=t2.a", + "select /*+ no_hash_join(t2) */ * from t1 right join t2 on t1.b=t2.b", + "select /*+ no_hash_join(t2) */ * from t1 right join t2 on t1.a=t2.a", + "select /*+ leading(t4, t3, t2, t1), no_hash_join(t2, t3) */ * from t1, t2, t3, t4", + "select /*+ leading(t1, t2, t3, t4), hash_join(t1, t2), no_hash_join(t3), hash_join(t4) */ * from t1, t2, t3, t4" + ] + }, { "name": "TestLeadingJoinHint", "cases": [ @@ -97,7 +116,16 @@ "select /*+ leading(t3) */ * from ((select /*+ leading(t5) */ t8.a, t8.b from t8, t7, t6, t5 where t5.a = t6.a and t6.b=t7.b) t3 join t4 on t3.a=t4.a) join (t1 join t2 on t1.a=t2.a) on t1.a=t4.a;", "select /*+ leading(t3, t1) */ * from ((select /*+ leading(t7) */ t8.a, t8.b from t8, t7, t6, t5 where t5.a = t6.a and t6.b=t7.b) t3 join t4 on t3.a=t4.a) join (t1 join t2 on t1.a=t2.a) on t1.a=t4.a;", "select /*+ leading(t3, t1, t2) */ * from ((select /*+ leading(t6, t7) */ t8.a, t8.b from t8, t7, t6, t5 where t5.a = t6.a and t6.b=t7.b) t3 join t4 on t3.a=t4.a) join (t1 join t2 on t1.a=t2.a) on t1.a=t4.a;", +<<<<<<< HEAD:planner/core/testdata/join_reorder_suite_in.json "select /*+ leading(t3, t4) */ * from ((select /*+ leading(t5, t7, t8) */ t8.a, t8.b from t8, t7, t6, t5 where t5.a = t6.a and t6.b=t7.b) t3 join t4 on t3.a=t4.a) join (t1 join t2 on t1.a=t2.a) on t1.a=t4.a;" +======= + "select /*+ leading(t3, t4) */ * from ((select /*+ leading(t5, t7, t8) */ t8.a, t8.b from t8, t7, t6, t5 where t5.a = t6.a and t6.b=t7.b) t3 join t4 on t3.a=t4.a) join (t1 join t2 on t1.a=t2.a) on t1.a=t4.a;", + // block name + "select /*+ leading(tx, t1, t3) */ * from t1, (select * from t2) tx, t3 where t1.a=tx.a and tx.a=t3.a;", + "select /*+ leading(txx, tx, t1) */ * from t1, (select * from t2) tx, (select * from t3) txx where t1.a=tx.a and tx.a=txx.a;", + "select /*+ leading(txx, tx, t) */ * from (select * from t1) t, (select * from t2) tx, (select * from t3) txx where t.a=tx.a and tx.a=txx.a;", + "select /*+ leading(tx, t1, t3) */ * from t1, (select t2.* from t2, t4 where t2.a=t4.a) tx, t3 where t1.a=tx.a and tx.a=t3.a;" +>>>>>>> 7c2dbbe34e4 (planner: support `no_hash_join` hint on optimizer (#45538)):planner/core/casetest/rule/testdata/join_reorder_suite_in.json ] }, { @@ -122,7 +150,6 @@ "select /*+ straight_join() */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b and t2.b=t3.b;", "select /*+ leading(t1) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b and t2.b=t3.b;", "select /*+ leading(t1, t3) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b and t2.b=t3.b;", - // outer join // left join "select /*+ leading(t1, t2) */ * from t4 join t on t4.a=t.a left join t1 on t.a = t1.a join t2 on t.b = t2.b join t3 on t2.b=t3.b;", @@ -130,28 +157,21 @@ "select /*+ leading(t4, t3, t2, t, t1) */ * from t4 join t on t4.a=t.a left join t1 on t.a = t1.a join t2 on t.b = t2.b join t3 on t2.b=t3.b;", "select /*+ leading(t4, t3, t2, t) */ * from t4 join t on t4.a=t.a left join t1 on t.a = t1.a join t2 on t.b = t2.b join t3 on t2.b=t3.b;", "select /*+ leading(t3, t2, t) */ * from t4 join t on t4.a=t.a left join t1 on t.a = t1.a join t2 on t.b = t2.b join t3 on t2.b=t3.b;", - "select /*+ leading(t3) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.b=t1.b;", "select /*+ leading(t2, t1, t3) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.b=t1.b;", - "select /*+ leading(t2, t3) */ * from t2 left join (t1 join t3 on t1.a=t3.a join t4 on t3.b = t4.b) on t2.b=t1.b;", "select /*+ leading(t3, t4) */ * from t2 left join (t1 join t3 on t1.a=t3.a join t4 on t3.b = t4.b) on t2.b=t1.b;", - "select /*+ leading(t3, t4) */ * from t2 left join (t1 join t3 on t1.a=t3.a join t4 on t3.b = t4.b) on t2.b=t1.b join t5 on t2.a = t5.a join t6 on t5.b=t6.b;", "select /*+ leading(t3, t4) leading(t5, t6) */ * from t2 left join (t1 join t3 on t1.a=t3.a join t4 on t3.b = t4.b) on t2.b=t1.b join t5 on t2.a = t5.a join t6 on t5.b=t6.b;", // should support this case later "select /*+ leading(t5, t6, t3, t4) */ * from t2 left join (t1 join t3 on t1.a=t3.a join t4 on t3.b = t4.b) on t2.b=t1.b join t5 on t2.a = t5.a join t6 on t5.b=t6.b;", - // right join "select /*+ leading(t1, t2) */ * from t4 join t on t4.a=t.a right join t1 on t.a = t1.a join t2 on t1.b = t2.b join t3 on t2.b=t3.b;", "select /*+ leading(t2, t3) */ * from t4 join t on t4.a=t.a right join t1 on t.a = t1.a join t2 on t1.b = t2.b join t3 on t2.b=t3.b;", "select /*+ leading(t1, t3) */ * from t4 join t on t4.a=t.a right join t1 on t.a = t1.a join t2 on t1.b = t2.b join t3 on t2.b=t3.b;", - "select /*+ leading(t3) */ * from t2 right join (t1 left join t3 on t1.a=t3.a) on t2.b=t1.b;", "select /*+ leading(t2, t1, t3) */ * from t2 right join (t1 left join t3 on t1.a=t3.a) on t2.b=t1.b;", - "select /*+ leading(t2, t3) */ * from t2 right join (t1 join t3 on t1.a=t3.a join t4 on t3.b = t4.b) on t2.b=t1.b;", "select /*+ leading(t3, t4) */ * from t2 right join (t1 join t3 on t1.a=t3.a join t4 on t3.b = t4.b) on t2.b=t1.b;", - "select /*+ leading(t3, t4) */ * from t2 right join (t1 join t3 on t1.a=t3.a join t4 on t3.b = t4.b) on t2.b=t1.b join t5 on t2.a = t5.a join t6 on t5.b=t6.b;", "select /*+ leading(t3, t4) leading(t5, t6) */ * from t2 right join (t1 join t3 on t1.a=t3.a join t4 on t3.b = t4.b) on t2.b=t1.b join t5 on t2.a = t5.a join t6 on t5.b=t6.b;", "select /*+ leading(t3, t4, t5, t6) */ * from t2 right join (t1 join t3 on t1.a=t3.a join t4 on t3.b = t4.b) on t2.b=t1.b join t5 on t2.a = t5.a join t6 on t5.b=t6.b;" @@ -164,7 +184,6 @@ "select /*+ straight_join() */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b and t2.b=t3.b;", "select /*+ leading(t1) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b and t2.b=t3.b;", "select /*+ leading(t1, t3) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b and t2.b=t3.b;", - // outer join // left join "select /*+ leading(t1, t2) */ * from t4 join t on t4.a=t.a left join t1 on t.a = t1.a join t2 on t.b = t2.b join t3 on t2.b=t3.b;", @@ -172,28 +191,21 @@ "select /*+ leading(t4, t3, t2, t, t1) */ * from t4 join t on t4.a=t.a left join t1 on t.a = t1.a join t2 on t.b = t2.b join t3 on t2.b=t3.b;", "select /*+ leading(t4, t3, t2, t) */ * from t4 join t on t4.a=t.a left join t1 on t.a = t1.a join t2 on t.b = t2.b join t3 on t2.b=t3.b;", "select /*+ leading(t3, t2, t) */ * from t4 join t on t4.a=t.a left join t1 on t.a = t1.a join t2 on t.b = t2.b join t3 on t2.b=t3.b;", - "select /*+ leading(t3) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.b=t1.b;", "select /*+ leading(t2, t1, t3) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.b=t1.b;", - "select /*+ leading(t2, t3) */ * from t2 left join (t1 join t3 on t1.a=t3.a join t4 on t3.b = t4.b) on t2.b=t1.b;", "select /*+ leading(t3, t4) */ * from t2 left join (t1 join t3 on t1.a=t3.a join t4 on t3.b = t4.b) on t2.b=t1.b;", - "select /*+ leading(t3, t4) */ * from t2 left join (t1 join t3 on t1.a=t3.a join t4 on t3.b = t4.b) on t2.b=t1.b join t5 on t2.a = t5.a join t6 on t5.b=t6.b;", "select /*+ leading(t3, t4) leading(t5, t6) */ * from t2 left join (t1 join t3 on t1.a=t3.a join t4 on t3.b = t4.b) on t2.b=t1.b join t5 on t2.a = t5.a join t6 on t5.b=t6.b;", // should support this case ;ater "select /*+ leading(t5, t6, t3, t4) */ * from t2 left join (t1 join t3 on t1.a=t3.a join t4 on t3.b = t4.b) on t2.b=t1.b join t5 on t2.a = t5.a join t6 on t5.b=t6.b;", - // right join "select /*+ leading(t1, t2) */ * from t4 join t on t4.a=t.a right join t1 on t.a = t1.a join t2 on t1.b = t2.b join t3 on t2.b=t3.b;", "select /*+ leading(t2, t3) */ * from t4 join t on t4.a=t.a right join t1 on t.a = t1.a join t2 on t1.b = t2.b join t3 on t2.b=t3.b;", "select /*+ leading(t1, t3) */ * from t4 join t on t4.a=t.a right join t1 on t.a = t1.a join t2 on t1.b = t2.b join t3 on t2.b=t3.b;", - "select /*+ leading(t3) */ * from t2 right join (t1 left join t3 on t1.a=t3.a) on t2.b=t1.b;", "select /*+ leading(t2, t1, t3) */ * from t2 right join (t1 left join t3 on t1.a=t3.a) on t2.b=t1.b;", - "select /*+ leading(t2, t3) */ * from t2 right join (t1 join t3 on t1.a=t3.a join t4 on t3.b = t4.b) on t2.b=t1.b;", "select /*+ leading(t3, t4) */ * from t2 right join (t1 join t3 on t1.a=t3.a join t4 on t3.b = t4.b) on t2.b=t1.b;", - "select /*+ leading(t3, t4) */ * from t2 right join (t1 join t3 on t1.a=t3.a join t4 on t3.b = t4.b) on t2.b=t1.b join t5 on t2.a = t5.a join t6 on t5.b=t6.b;", "select /*+ leading(t3, t4) leading(t5, t6) */ * from t2 right join (t1 join t3 on t1.a=t3.a join t4 on t3.b = t4.b) on t2.b=t1.b join t5 on t2.a = t5.a join t6 on t5.b=t6.b;", "select /*+ leading(t3, t4, t5, t6) */ * from t2 right join (t1 join t3 on t1.a=t3.a join t4 on t3.b = t4.b) on t2.b=t1.b join t5 on t2.a = t5.a join t6 on t5.b=t6.b;" @@ -207,42 +219,36 @@ "select /*+ straight_join() */ * from t1 join t2 on t1.a=t2.a left join t3 on t2.b=t3.b", "select /*+ straight_join() */ * from t1 join t2 on t1.a=t2.a right join t3 on t2.b=t3.b", "select /*+ straight_join() */ * from t1 join t2 on t1.a=t2.a cross join t3", - // inner join "select /*+ leading(t2) */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b", "select /*+ leading(t3) */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b", "select /*+ leading(t2, t3) */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b", "select /*+ leading(t3, t2) */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b", "select /*+ leading(t3, t1) */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b", - // straight join "select /*+ leading(t2) */ * from t1 straight_join t2 straight_join t3", "select /*+ leading(t3) */ * from t1 straight_join t2 straight_join t3", "select /*+ leading(t2, t3) */ * from t1 straight_join t2 straight_join t3", "select /*+ leading(t3, t2) */ * from t1 straight_join t2 straight_join t3", "select /*+ leading(t3, t1) */ * from t1 straight_join t2 straight_join t3", - // left outer join "select /*+ leading(t2) */ * from t1 join t2 on t1.a=t2.a left join t3 on t2.b=t3.b", "select /*+ leading(t3) */ * from t1 join t2 on t1.a=t2.a left join t3 on t2.b=t3.b", "select /*+ leading(t2, t3) */ * from t1 join t2 on t1.a=t2.a left join t3 on t2.b=t3.b", "select /*+ leading(t3, t2) */ * from t1 join t2 on t1.a=t2.a left join t3 on t2.b=t3.b", "select /*+ leading(t3, t1) */ * from t1 join t2 on t1.a=t2.a left join t3 on t2.b=t3.b", - // right outer join "select /*+ leading(t2) */ * from t1 join t2 on t1.a=t2.a right join t3 on t2.b=t3.b", "select /*+ leading(t3) */ * from t1 join t2 on t1.a=t2.a right join t3 on t2.b=t3.b", "select /*+ leading(t2, t3) */ * from t1 join t2 on t1.a=t2.a right join t3 on t2.b=t3.b", "select /*+ leading(t3, t2) */ * from t1 join t2 on t1.a=t2.a right join t3 on t2.b=t3.b", "select /*+ leading(t3, t1) */ * from t1 join t2 on t1.a=t2.a right join t3 on t2.b=t3.b", - // cross join "select /*+ leading(t2) */ * from t1 join t2 on t1.a=t2.a cross join t3", "select /*+ leading(t3) */ * from t1 join t2 on t1.a=t2.a cross join t3", "select /*+ leading(t2, t3) */ * from t1 join t2 on t1.a=t2.a cross join t3", "select /*+ leading(t3, t2) */ * from t1 join t2 on t1.a=t2.a cross join t3", "select /*+ leading(t3, t1) */ * from t1 join t2 on t1.a=t2.a cross join t3", - // outer join + outer join types "select /*+ leading(t2, t1) */ * from t1 left join t2 on t1.a=t2.a cross join t3", "select /*+ leading(t2, t1) */ * from t1 right join t2 on t1.a=t2.a cross join t3", @@ -262,7 +268,6 @@ "select /*+ leading(t2, t3) */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b", "select /*+ leading(t3, t2) */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b", "select /*+ leading(t3, t1) */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b", - // outer join // left join "select /*+ leading(t1, t2) */ * from t4 join t on t4.a=t.a left join t1 on t.a = t1.a join t2 on t.b = t2.b join t3 on t2.b=t3.b;", @@ -270,28 +275,21 @@ "select /*+ leading(t4, t3, t2, t, t1) */ * from t4 join t on t4.a=t.a left join t1 on t.a = t1.a join t2 on t.b = t2.b join t3 on t2.b=t3.b;", "select /*+ leading(t4, t3, t2, t) */ * from t4 join t on t4.a=t.a left join t1 on t.a = t1.a join t2 on t.b = t2.b join t3 on t2.b=t3.b;", "select /*+ leading(t3, t2, t) */ * from t4 join t on t4.a=t.a left join t1 on t.a = t1.a join t2 on t.b = t2.b join t3 on t2.b=t3.b;", - "select /*+ leading(t3) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.b=t1.b;", "select /*+ leading(t2, t1, t3) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.b=t1.b;", - "select /*+ leading(t2, t3) */ * from t2 left join (t1 join t3 on t1.a=t3.a join t4 on t3.b = t4.b) on t2.b=t1.b;", "select /*+ leading(t3, t4) */ * from t2 left join (t1 join t3 on t1.a=t3.a join t4 on t3.b = t4.b) on t2.b=t1.b;", - "select /*+ leading(t3, t4) */ * from t2 left join (t1 join t3 on t1.a=t3.a join t4 on t3.b = t4.b) on t2.b=t1.b join t5 on t2.a = t5.a join t6 on t5.b=t6.b;", "select /*+ leading(t3, t4) leading(t5, t6) */ * from t2 left join (t1 join t3 on t1.a=t3.a join t4 on t3.b = t4.b) on t2.b=t1.b join t5 on t2.a = t5.a join t6 on t5.b=t6.b;", // should support this case ;ater "select /*+ leading(t5, t6, t3, t4) */ * from t2 left join (t1 join t3 on t1.a=t3.a join t4 on t3.b = t4.b) on t2.b=t1.b join t5 on t2.a = t5.a join t6 on t5.b=t6.b;", - // right join "select /*+ leading(t1, t2) */ * from t4 join t on t4.a=t.a right join t1 on t.a = t1.a join t2 on t1.b = t2.b join t3 on t2.b=t3.b;", "select /*+ leading(t2, t3) */ * from t4 join t on t4.a=t.a right join t1 on t.a = t1.a join t2 on t1.b = t2.b join t3 on t2.b=t3.b;", "select /*+ leading(t1, t3) */ * from t4 join t on t4.a=t.a right join t1 on t.a = t1.a join t2 on t1.b = t2.b join t3 on t2.b=t3.b;", - "select /*+ leading(t3) */ * from t2 right join (t1 left join t3 on t1.a=t3.a) on t2.b=t1.b;", "select /*+ leading(t2, t1, t3) */ * from t2 right join (t1 left join t3 on t1.a=t3.a) on t2.b=t1.b;", - "select /*+ leading(t2, t3) */ * from t2 right join (t1 join t3 on t1.a=t3.a join t4 on t3.b = t4.b) on t2.b=t1.b;", "select /*+ leading(t3, t4) */ * from t2 right join (t1 join t3 on t1.a=t3.a join t4 on t3.b = t4.b) on t2.b=t1.b;", - "select /*+ leading(t3, t4) */ * from t2 right join (t1 join t3 on t1.a=t3.a join t4 on t3.b = t4.b) on t2.b=t1.b join t5 on t2.a = t5.a join t6 on t5.b=t6.b;", "select /*+ leading(t3, t4) leading(t5, t6) */ * from t2 right join (t1 join t3 on t1.a=t3.a join t4 on t3.b = t4.b) on t2.b=t1.b join t5 on t2.a = t5.a join t6 on t5.b=t6.b;", "select /*+ leading(t3, t4, t5, t6) */ * from t2 right join (t1 join t3 on t1.a=t3.a join t4 on t3.b = t4.b) on t2.b=t1.b join t5 on t2.a = t5.a join t6 on t5.b=t6.b;" @@ -307,7 +305,6 @@ "select /*+ straight_join() */ * from t1 join t2 on t1.a=t2.a where not exists (select t3.a from t3 where t1.b = t3.b)", "select /*+ straight_join() */ t1.a, (select min(t1.a) from t2 where t2.a > t1.a) from t1;", "select /*+ straight_join() */ t1.a, (select min(t1.a) from t2 where t2.a > t1.a) from t1 join t3 on t1.a = t3.a;", - // straight join hint + uncorrelated subquery "select /*+ straight_join() */ * from t1 join t2 on t1.a=t2.a where t1.a in (select t3.a from t3)", "select /*+ straight_join() */ * from t1 join t2 on t1.a=t2.a where t1.a not in (select t3.a from t3)", @@ -318,7 +315,6 @@ "select /*+ straight_join() */ * from t1 join t2 on t1.a=t2.a join (select * from t4) t3 on t2.b=t3.b;", "select /*+ straight_join() */ t1.a, (select min(t2.a) from t2) from t1;", "select /*+ straight_join() */ t1.a, (select min(t2.a) from t2) from t1 join t3 on t1.a = t3.a;", - // leading hint + correlated subquery "select /*+ leading(t4) */ * from t1 join t2 on t1.a=t2.a join t4 on t1.b = t4.b where t1.a in (select t3.a from t3 where t1.b = t3.b)", "select /*+ leading(t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where t1.a in (select t3.a from t3 where t1.b = t3.b)", @@ -326,45 +322,38 @@ "select /*+ leading(t1, t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where t1.a in (select t3.a from t3 where t1.b = t3.b)", "select /*+ leading(t3@sel_2, t2) */ * from t1 join t2 on t1.a=t2.a where t1.a in (select t3.a from t3 where t1.b = t3.b)", "select /*+ leading(t3@sel_2, t1) */ * from t1 join t2 on t1.a=t2.a where t1.a in (select t3.a from t3 where t1.b = t3.b)", - "select /*+ leading(t4) */ * from t1 join t2 on t1.a=t2.a join t4 on t1.b = t4.b where t1.a not in (select t3.a from t3 where t1.b = t3.b)", "select /*+ leading(t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where t1.a not in (select t3.a from t3 where t1.b = t3.b)", "select /*+ leading(t2, t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where t1.a not in (select t3.a from t3 where t1.b = t3.b)", "select /*+ leading(t1, t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where t1.a not in (select t3.a from t3 where t1.b = t3.b)", "select /*+ leading(t3@sel_2, t2) */ * from t1 join t2 on t1.a=t2.a where t1.a not in (select t3.a from t3 where t1.b = t3.b)", "select /*+ leading(t3@sel_2, t1) */ * from t1 join t2 on t1.a=t2.a where t1.a not in (select t3.a from t3 where t1.b = t3.b)", - "select /*+ leading(t4) */ * from t1 join t2 on t1.a=t2.a join t4 on t1.b = t4.b where exists (select t3.a from t3 where t1.b = t3.b)", "select /*+ leading(t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where exists (select t3.a from t3 where t1.b = t3.b)", "select /*+ leading(t2, t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where exists (select t3.a from t3 where t1.b = t3.b)", "select /*+ leading(t1, t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where exists (select t3.a from t3 where t1.b = t3.b)", "select /*+ leading(t3@sel_2, t2) */ * from t1 join t2 on t1.a=t2.a where exists (select t3.a from t3 where t1.b = t3.b)", "select /*+ leading(t3@sel_2, t1) */ * from t1 join t2 on t1.a=t2.a where exists (select t3.a from t3 where t1.b = t3.b)", - "select /*+ leading(t4) */ * from t1 join t2 on t1.a=t2.a join t4 on t1.b = t4.b where not exists (select t3.a from t3 where t1.b = t3.b)", "select /*+ leading(t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where not exists (select t3.a from t3 where t1.b = t3.b)", "select /*+ leading(t2, t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where not exists (select t3.a from t3 where t1.b = t3.b)", "select /*+ leading(t1, t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where not exists (select t3.a from t3 where t1.b = t3.b)", "select /*+ leading(t3@sel_2, t2) */ * from t1 join t2 on t1.a=t2.a where not exists (select t3.a from t3 where t1.b = t3.b)", "select /*+ leading(t3@sel_2, t1) */ * from t1 join t2 on t1.a=t2.a where not exists (select t3.a from t3 where t1.b = t3.b)", - "select /*+ leading(t1) */ t1.a, (select min(t1.a) from t2 where t2.a > t1.a) from t1 join t3 on t1.a = t3.a;", "select /*+ leading(t3@sel_2) */ t1.a, (select min(t1.a) from t2 where t2.a > t1.a) from t1 join t3 on t1.a = t3.a;", "select /*+ leading(t2, t3@sel_2) */ t1.a, (select min(t1.a) from t2 where t2.a > t1.a) from t1 join t3 on t1.a = t3.a;", "select /*+ leading(t1, t3@sel_2) */ t1.a, (select min(t1.a) from t2 where t2.a > t1.a) from t1 join t3 on t1.a = t3.a;", "select /*+ leading(t3@sel_2, t2) */ t1.a, (select min(t1.a) from t2 where t2.a > t1.a) from t1 join t3 on t1.a = t3.a;", "select /*+ leading(t3@sel_2, t1) */ t1.a, (select min(t1.a) from t2 where t2.a > t1.a) from t1 join t3 on t1.a = t3.a;", - "select /*+ leading(t4, t3@sel_2) */ * from t1 join t2 on t1.a=t2.a join t4 on t1.b = t4.b where t1.a = (select max(t3.a) from t3 where t1.b = t3.b)", "select /*+ leading(t4) */ * from t1 join t2 on t1.a=t2.a join t4 on t1.b = t4.b where t1.a = (select max(t3.a) from t3 where t1.b = t3.b)", "select /*+ leading(t3@sel_2) */ * from t1 join t2 on t1.a=t2.a join t4 on t1.b = t4.b where t1.a = (select max(t3.a) from t3 where t1.b = t3.b)", "select /*+ leading(t3@sel_2, t2) */ * from t1 join t2 on t1.a=t2.a join t4 on t1.b = t4.b where t1.a = (select max(t3.a) from t3 where t1.b = t3.b)", - "select /*+ leading(t4, t3@sel_2) */ * from t1 join t2 on t1.a=t2.a join t4 on t1.b = t4.b where t1.a > (select min(t3.a) from t3 where t1.b = t3.b)", "select /*+ leading(t4) */ * from t1 join t2 on t1.a=t2.a join t4 on t1.b = t4.b where t1.a > (select min(t3.a) from t3 where t1.b = t3.b)", "select /*+ leading(t3@sel_2) */ * from t1 join t2 on t1.a=t2.a join t4 on t1.b = t4.b where t1.a > (select min(t3.a) from t3 where t1.b = t3.b)", "select /*+ leading(t3@sel_2, t2) */ * from t1 join t2 on t1.a=t2.a join t4 on t1.b = t4.b where t1.a > (select min(t3.a) from t3 where t1.b = t3.b)", - // leading hint + uncorrelated subquery "select /*+ leading(t4) */ * from t1 join t2 on t1.a=t2.a join t4 on t1.b = t4.b where t1.a in (select t3.a from t3)", "select /*+ leading(t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where t1.a in (select t3.a from t3)", @@ -372,32 +361,27 @@ "select /*+ leading(t1, t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where t1.a in (select t3.a from t3)", "select /*+ leading(t3@sel_2, t2) */ * from t1 join t2 on t1.a=t2.a where t1.a in (select t3.a from t3)", "select /*+ leading(t3@sel_2, t1) */ * from t1 join t2 on t1.a=t2.a where t1.a in (select t3.a from t3)", - "select /*+ leading(t4) */ * from t1 join t2 on t1.a=t2.a join t4 on t1.b = t4.b where t1.a not in (select t3.a from t3)", "select /*+ leading(t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where t1.a not in (select t3.a from t3)", "select /*+ leading(t2, t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where t1.a not in (select t3.a from t3)", "select /*+ leading(t1, t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where t1.a not in (select t3.a from t3)", "select /*+ leading(t3@sel_2, t2) */ * from t1 join t2 on t1.a=t2.a where t1.a not in (select t3.a from t3)", "select /*+ leading(t3@sel_2, t1) */ * from t1 join t2 on t1.a=t2.a where t1.a not in (select t3.a from t3)", - "select /*+ leading(t4) */ * from t1 join t2 on t1.a=t2.a join t4 on t1.b = t4.b where exists (select t3.a from t3)", "select /*+ leading(t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where exists (select t3.a from t3)", "select /*+ leading(t2, t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where exists (select t3.a from t3)", "select /*+ leading(t1, t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where exists (select t3.a from t3)", "select /*+ leading(t3@sel_2, t2) */ * from t1 join t2 on t1.a=t2.a where exists (select t3.a from t3)", "select /*+ leading(t3@sel_2, t1) */ * from t1 join t2 on t1.a=t2.a where exists (select t3.a from t3)", - "select /*+ leading(t4) */ * from t1 join t2 on t1.a=t2.a join t4 on t1.b = t4.b where not exists (select t3.a from t3)", "select /*+ leading(t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where not exists (select t3.a from t3)", "select /*+ leading(t2, t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where not exists (select t3.a from t3)", "select /*+ leading(t1, t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where not exists (select t3.a from t3)", "select /*+ leading(t3@sel_2, t2) */ * from t1 join t2 on t1.a=t2.a where not exists (select t3.a from t3)", "select /*+ leading(t3@sel_2, t1) */ * from t1 join t2 on t1.a=t2.a where not exists (select t3.a from t3)", - "select /*+ leading(t4@sel_2, t1) */ * from t1 join (select * from t4) t2 on t1.a=t2.a join t3 on t2.b=t3.b;", "select /*+ leading(t2, t4@sel_2) */ * from (select * from t4) t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b;", "select /*+ leading(t3) */ * from t1 join t2 on t1.a=t2.a join (select * from t4) t3 on t2.b=t3.b;", - "select /*+ leading(t1) */ t1.a, (select min(t2.a) from t2) from t1 join t3 on t1.a = t3.a;", "select /*+ leading(t1, t2@sel_2) */ t1.a, (select min(t2.a) from t2) from t1 join t3 on t1.a = t3.a;", "select /*+ leading(t1, t3) */ t1.a, (select min(t2.a) from t2) from t1 join t3 on t1.a = t3.a;", @@ -417,32 +401,24 @@ "select /*+ leading(t4, t3, t2, t, t1) */ * from t4 join t on t4.a=t.a left join t1 on t.a = t1.a join t2 on t.b = t2.b join t3 on t2.b=t3.b;", "select /*+ leading(t4, t3, t2, t) */ * from t4 join t on t4.a=t.a left join t1 on t.a = t1.a join t2 on t.b = t2.b join t3 on t2.b=t3.b;", "select /*+ leading(t3, t2, t) */ * from t4 join t on t4.a=t.a left join t1 on t.a = t1.a join t2 on t.b = t2.b join t3 on t2.b=t3.b;", - "select /*+ leading(t3) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.b=t1.b;", "select /*+ leading(t2, t1, t3) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.b=t1.b;", - "select /*+ leading(t2, t3) */ * from t2 left join (t1 join t3 on t1.a=t3.a join t4 on t3.b = t4.b) on t2.b=t1.b;", "select /*+ leading(t3, t4) */ * from t2 left join (t1 join t3 on t1.a=t3.a join t4 on t3.b = t4.b) on t2.b=t1.b;", - "select /*+ leading(t3, t4) */ * from t2 left join (t1 join t3 on t1.a=t3.a join t4 on t3.b = t4.b) on t2.b=t1.b join t5 on t2.a = t5.a join t6 on t5.b=t6.b;", "select /*+ leading(t3, t4) leading(t5, t6) */ * from t2 left join (t1 join t3 on t1.a=t3.a join t4 on t3.b = t4.b) on t2.b=t1.b join t5 on t2.a = t5.a join t6 on t5.b=t6.b;", // should support this case ;ater "select /*+ leading(t5, t6, t3, t4) */ * from t2 left join (t1 join t3 on t1.a=t3.a join t4 on t3.b = t4.b) on t2.b=t1.b join t5 on t2.a = t5.a join t6 on t5.b=t6.b;", - // right join "select /*+ leading(t1, t2) */ * from t4 join t on t4.a=t.a right join t1 on t.a = t1.a join t2 on t1.b = t2.b join t3 on t2.b=t3.b;", "select /*+ leading(t2, t3) */ * from t4 join t on t4.a=t.a right join t1 on t.a = t1.a join t2 on t1.b = t2.b join t3 on t2.b=t3.b;", "select /*+ leading(t1, t3) */ * from t4 join t on t4.a=t.a right join t1 on t.a = t1.a join t2 on t1.b = t2.b join t3 on t2.b=t3.b;", - "select /*+ leading(t3) */ * from t2 right join (t1 left join t3 on t1.a=t3.a) on t2.b=t1.b;", "select /*+ leading(t2, t1, t3) */ * from t2 right join (t1 left join t3 on t1.a=t3.a) on t2.b=t1.b;", - "select /*+ leading(t2, t3) */ * from t2 right join (t1 join t3 on t1.a=t3.a join t4 on t3.b = t4.b) on t2.b=t1.b;", "select /*+ leading(t3, t4) */ * from t2 right join (t1 join t3 on t1.a=t3.a join t4 on t3.b = t4.b) on t2.b=t1.b;", - "select /*+ leading(t3, t4) */ * from t2 right join (t1 join t3 on t1.a=t3.a join t4 on t3.b = t4.b) on t2.b=t1.b join t5 on t2.a = t5.a join t6 on t5.b=t6.b;", "select /*+ leading(t3, t4) leading(t5, t6) */ * from t2 right join (t1 join t3 on t1.a=t3.a join t4 on t3.b = t4.b) on t2.b=t1.b join t5 on t2.a = t5.a join t6 on t5.b=t6.b;", "select /*+ leading(t3, t4, t5, t6) */ * from t2 right join (t1 join t3 on t1.a=t3.a join t4 on t3.b = t4.b) on t2.b=t1.b join t5 on t2.a = t5.a join t6 on t5.b=t6.b;", - // test cases for using the join order hint and join algorithm hint "select /*+ leading(t2) hash_join(t2) */ * from t left join t1 on t.a = t1.a right join t2 on t1.b = t2.b join t3 on t2.b = t3.b ;", "select /*+ leading(t2) hash_join(t1) */ * from t left join t1 on t.a = t1.a right join t2 on t1.b = t2.b join t3 on t2.b = t3.b ;", @@ -463,59 +439,46 @@ "select /*+ leading(t3) merge_join(t3) */ * from t right join t1 on t.a = t1.a join t2 on t1.b = t2.b join t3 on t2.b = t3.b ;", "select /*+ leading(t3) merge_join(t3) */ * from t left join t1 on t.a = t1.a left join t2 on t1.b = t2.b join t3 on t2.b = t3.b ;", "select /*+ leading(t2) INL_JOIN(t1) */ * from t join t1 on t.a = t1.a left join t2 on t1.b = t2.b join t3 on t2.b = t3.b ;", - // leading hint + correlated subquery "select /*+ leading(t4) */ * from t1 join t2 on t1.a=t2.a right join t4 on t1.b = t4.b where t1.a in (select t3.a from t3 where t1.b = t3.b)", "select /*+ leading(t3@sel_2) */ * from t1 left join t2 on t1.a=t2.a where t1.a in (select t3.a from t3 where t1.b = t3.b)", "select /*+ leading(t2, t3@sel_2) */ * from t1 right join t2 on t1.a=t2.a where t1.a in (select t3.a from t3 where t1.b = t3.b)", - "select /*+ leading(t4) */ * from t1 join t2 on t1.a=t2.a right join t4 on t1.b = t4.b where t1.a not in (select t3.a from t3 where t1.b = t3.b)", "select /*+ leading(t3@sel_2) */ * from t1 left join t2 on t1.a=t2.a where t1.a not in (select t3.a from t3 where t1.b = t3.b)", "select /*+ leading(t2, t3@sel_2) */ * from t1 right join t2 on t1.a=t2.a where t1.a not in (select t3.a from t3 where t1.b = t3.b)", - "select /*+ leading(t4) */ * from t1 join t2 on t1.a=t2.a right join t4 on t1.b = t4.b where exists (select t3.a from t3 where t1.b = t3.b)", "select /*+ leading(t3@sel_2) */ * from t1 left join t2 on t1.a=t2.a where exists (select t3.a from t3 where t1.b = t3.b)", "select /*+ leading(t2, t3@sel_2) */ * from t1 right join t2 on t1.a=t2.a where exists (select t3.a from t3 where t1.b = t3.b)", - "select /*+ leading(t4) */ * from t1 join t2 on t1.a=t2.a right join t4 on t1.b = t4.b where not exists (select t3.a from t3 where t1.b = t3.b)", "select /*+ leading(t3@sel_2) */ * from t1 left join t2 on t1.a=t2.a where not exists (select t3.a from t3 where t1.b = t3.b)", "select /*+ leading(t2, t3@sel_2) */ * from t1 right join t2 on t1.a=t2.a where not exists (select t3.a from t3 where t1.b = t3.b)", - "select /*+ leading(t1) */ t1.a, (select min(t1.a) from t2 where t2.a > t1.a) from t1 join t3 on t1.a = t3.a;", "select /*+ leading(t3@sel_2) */ t1.a, (select min(t1.a) from t2 where t2.a > t1.a) from t1 join t3 on t1.a = t3.a;", "select /*+ leading(t2, t3@sel_2) */ t1.a, (select min(t1.a) from t2 where t2.a > t1.a) from t1 join t3 on t1.a = t3.a;", - "select /*+ leading(t4, t3@sel_2) */ * from t1 right join t2 on t1.a=t2.a join t4 on t1.b = t4.b where t1.a = (select max(t3.a) from t3 where t1.b = t3.b)", "select /*+ leading(t4) */ * from t1 right join t2 on t1.a=t2.a join t4 on t1.b = t4.b where t1.a = (select max(t3.a) from t3 where t1.b = t3.b)", "select /*+ leading(t3@sel_2) */ * from t1 left join t2 on t1.a=t2.a join t4 on t1.b = t4.b where t1.a = (select max(t3.a) from t3 where t1.b = t3.b)", "select /*+ leading(t3@sel_2, t1) */ * from t1 left join t2 on t1.a=t2.a join t4 on t1.b = t4.b where t1.a = (select max(t3.a) from t3 where t1.b = t3.b)", - "select /*+ leading(t4, t3@sel_2) */ * from t1 left join t2 on t1.a=t2.a right join t4 on t1.b = t4.b where t1.a > (select min(t3.a) from t3 where t1.b = t3.b)", "select /*+ leading(t4) */ * from t1 left join t2 on t1.a=t2.a right join t4 on t1.b = t4.b where t1.a > (select min(t3.a) from t3 where t1.b = t3.b)", "select /*+ leading(t3@sel_2) */ * from t1 left join t2 on t1.a=t2.a join t4 on t1.b = t4.b where t1.a > (select min(t3.a) from t3 where t1.b = t3.b)", "select /*+ leading(t3@sel_2, t2) */ * from t1 right join t2 on t1.a=t2.a join t4 on t1.b = t4.b where t1.a > (select min(t3.a) from t3 where t1.b = t3.b)", - // leading hint + uncorrelated subquery "select /*+ leading(t4) */ * from t1 join t2 on t1.a=t2.a right join t4 on t1.b = t4.b where t1.a in (select t3.a from t3)", "select /*+ leading(t3@sel_2) */ * from t1 left join t2 on t1.a=t2.a where t1.a in (select t3.a from t3)", "select /*+ leading(t2, t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where t1.a in (select t3.a from t3)", - "select /*+ leading(t4) */ * from t1 left join t2 on t1.a=t2.a right join t4 on t1.b = t4.b where t1.a not in (select t3.a from t3)", "select /*+ leading(t3@sel_2) */ * from t1 left join t2 on t1.a=t2.a where t1.a not in (select t3.a from t3)", "select /*+ leading(t2, t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where t1.a not in (select t3.a from t3)", - "select /*+ leading(t4) */ * from t1 left join t2 on t1.a=t2.a right join t4 on t1.b = t4.b where exists (select t3.a from t3)", "select /*+ leading(t3@sel_2) */ * from t1 left join t2 on t1.a=t2.a where exists (select t3.a from t3)", "select /*+ leading(t2, t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where exists (select t3.a from t3)", - "select /*+ leading(t4) */ * from t1 join t2 on t1.a=t2.a right join t4 on t1.b = t4.b where not exists (select t3.a from t3)", "select /*+ leading(t3@sel_2) */ * from t1 left join t2 on t1.a=t2.a where not exists (select t3.a from t3)", "select /*+ leading(t2, t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where not exists (select t3.a from t3)", - "select /*+ leading(t4@sel_2, t1) */ * from t1 join (select * from t4) t2 on t1.a=t2.a join t3 on t2.b=t3.b;", "select /*+ leading(t2, t4@sel_2) */ * from (select * from t4) t1 right join t2 on t1.a=t2.a join t3 on t2.b=t3.b;", "select /*+ leading(t3) */ * from t1 join t2 on t1.a=t2.a right join (select * from t4) t3 on t2.b=t3.b;", - "select /*+ leading(t1) */ t1.a, (select min(t2.a) from t2) from t1 join t3 on t1.a = t3.a;", "select /*+ leading(t1, t2@sel_2) */ t1.a, (select min(t2.a) from t2) from t1 join t3 on t1.a = t3.a;", "select /*+ leading(t1, t3) */ t1.a, (select min(t2.a) from t2) from t1 join t3 on t1.a = t3.a;", diff --git a/planner/core/testdata/join_reorder_suite_out.json b/planner/core/testdata/join_reorder_suite_out.json index e2a6c562726d4..a9930db1d3a8d 100644 --- a/planner/core/testdata/join_reorder_suite_out.json +++ b/planner/core/testdata/join_reorder_suite_out.json @@ -583,6 +583,205 @@ } ] }, + { + "Name": "TestNoHashJoinHint", + "Cases": [ + { + "SQL": "select /*+ no_hash_join() */ * from t1, t2", + "Plan": [ + "HashJoin 100000000.00 root CARTESIAN inner join", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 Hint no_hash_join() is inapplicable. Please specify the table names in the arguments." + ] + }, + { + "SQL": "select /*+ no_hash_join(t1), hash_join(t1) */ * from t1, t2", + "Plan": [ + "HashJoin 100000000.00 root CARTESIAN inner join", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 Some HASH_JOIN and NO_HASH_JOIN hints conflict, NO_HASH_JOIN is ignored" + ] + }, + { + "SQL": "select /*+ no_hash_join(t1), hash_join(t2) */ * from t1, t2", + "Plan": [ + "HashJoin 100000000.00 root CARTESIAN inner join", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 Some HASH_JOIN and NO_HASH_JOIN hints conflict, NO_HASH_JOIN is ignored" + ] + }, + { + "SQL": "select /*+ no_hash_join(t1) */ * from t1, t2", + "Plan": [ + "MergeJoin 100000000.00 root inner join", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ no_hash_join(t1, t2) */ * from t1, t2", + "Plan": [ + "MergeJoin 100000000.00 root inner join", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ no_hash_join(t1) */ * from t1, t2 where t1.a=t2.a", + "Plan": [ + "IndexHashJoin 12487.50 root inner join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a)", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─IndexLookUp(Probe) 12487.50 root ", + " ├─Selection(Build) 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", + " └─TableRowIDScan(Probe) 12487.50 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ no_hash_join(t1, t2) */ * from t1, t2 where t1.b=t2.b", + "Plan": [ + "MergeJoin 12487.50 root inner join, left key:test.t1.b, right key:test.t2.b", + "├─Sort(Build) 9990.00 root test.t2.b", + "│ └─TableReader 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─Sort(Probe) 9990.00 root test.t1.b", + " └─TableReader 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ no_hash_join(t1) */ * from t1, t2 where t1.a=t2.a and t1.b=t2.b", + "Plan": [ + "IndexHashJoin 12475.01 root inner join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a), eq(test.t1.b, test.t2.b)", + "├─TableReader(Build) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t1.a)), not(isnull(test.t1.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─IndexLookUp(Probe) 12475.01 root ", + " ├─Selection(Build) 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", + " └─Selection(Probe) 12475.01 cop[tikv] not(isnull(test.t2.b))", + " └─TableRowIDScan 12487.50 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ no_hash_join(t2) */ * from t1 left join t2 on t1.b=t2.b", + "Plan": [ + "MergeJoin 12487.50 root left outer join, left key:test.t1.b, right key:test.t2.b", + "├─Sort(Build) 9990.00 root test.t2.b", + "│ └─TableReader 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─Sort(Probe) 10000.00 root test.t1.b", + " └─TableReader 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ no_hash_join(t2) */ * from t1 left join t2 on t1.a=t2.a", + "Plan": [ + "IndexHashJoin 12487.50 root left outer join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a)", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─IndexLookUp(Probe) 12487.50 root ", + " ├─Selection(Build) 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", + " └─TableRowIDScan(Probe) 12487.50 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ no_hash_join(t2) */ * from t1 right join t2 on t1.b=t2.b", + "Plan": [ + "MergeJoin 12487.50 root right outer join, left key:test.t1.b, right key:test.t2.b", + "├─Sort(Build) 9990.00 root test.t1.b", + "│ └─TableReader 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─Sort(Probe) 10000.00 root test.t2.b", + " └─TableReader 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ no_hash_join(t2) */ * from t1 right join t2 on t1.a=t2.a", + "Plan": [ + "IndexHashJoin 12487.50 root right outer join, inner:IndexLookUp, outer key:test.t2.a, inner key:test.t1.a, equal cond:eq(test.t2.a, test.t1.a)", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─IndexLookUp(Probe) 12487.50 root ", + " ├─Selection(Build) 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", + " └─TableRowIDScan(Probe) 12487.50 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t4, t3, t2, t1), no_hash_join(t2, t3) */ * from t1, t2, t3, t4", + "Plan": [ + "Projection 10000000000000000.00 root test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b, test.t4.a, test.t4.b", + "└─HashJoin 10000000000000000.00 root CARTESIAN inner join", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─MergeJoin(Probe) 1000000000000.00 root inner join", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─MergeJoin(Probe) 100000000.00 root inner join", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t1, t2, t3, t4), hash_join(t1, t2), no_hash_join(t3), hash_join(t4) */ * from t1, t2, t3, t4", + "Plan": [ + "HashJoin 10000000000000000.00 root CARTESIAN inner join", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + "└─MergeJoin(Probe) 1000000000000.00 root inner join", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 100000000.00 root CARTESIAN inner join", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": null + } + ] + }, { "Name": "TestLeadingJoinHint", "Cases": [