diff --git a/planner/core/casetest/rule/BUILD.bazel b/planner/core/casetest/rule/BUILD.bazel new file mode 100644 index 0000000000000..7949277ddcb3b --- /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 = 23, + 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..5701fcae99279 --- /dev/null +++ b/planner/core/casetest/rule/rule_join_reorder_test.go @@ -0,0 +1,339 @@ +// 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 TestNoMergeJoinHint(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + 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, "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) + + 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/casetest/rule/testdata/join_reorder_suite_in.json b/planner/core/casetest/rule/testdata/join_reorder_suite_in.json new file mode 100644 index 0000000000000..1201331e17a43 --- /dev/null +++ b/planner/core/casetest/rule/testdata/join_reorder_suite_in.json @@ -0,0 +1,518 @@ +[ + { + "name": "TestStraightJoinHint", + "cases": [ + "select /*+ straight_join() */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", + "select /*+ straight_join() */ * from t2, t1, t3, t where t.a = t1.a and t1.b=t2.b;", + "select /*+ straight_join() */ * from t3, t2, t1, t where t.a = t1.a and t1.b=t2.b;", + "select /*+ straight_join() */ * from t3, t1, t, t2 where t.a = t1.a and t1.b=t2.b;", + "select /*+ straight_join() */ * from t2 join t1 on t2.a=t1.a join t3 on t1.b=t3.b;", + "select /*+ straight_join() */ * from t3 join t1 on t1.b=t3.b join t2 on t2.a=t1.a;", + "select /*+ straight_join() */ * from t3 join t2 on t2.b=t3.b join t1 on t2.a=t1.a;", + "select /*+ straight_join() */ * from t2 join (t1 join t3 on t1.a=t3.a) on t2.a=1;", + "select /*+ straight_join() */ * from t2 join (t1 join t3 on t1.a=t3.a) on t2.a=t3.a;", + "select /*+ straight_join() */ * from t1 join (t2 join t3 on t2.a=t3.a) on t1.a=t3.a;", + "select /*+ straight_join() */ * from t3 join (t1 join t2 on t1.a=t2.a) on t2.a=t3.a;", + "select /*+ straight_join() */ * from t2 join t1 on t1.a=t2.a join t3 on t2.b=t3.b;", + "select /*+ straight_join() */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b;", + "select /*+ straight_join() */ * from t2 join t3 on t3.a=t2.a join t1 on t2.a=t1.a;", + "select /*+ straight_join() */ * from (t1 join t2 on t1.a=t2.a) join (t3 join t4 on t3.a=t4.a) on t2.a=t4.a;", + "select /*+ straight_join() */ * from (t1 join t2 on t1.a=t2.a) join (t3 join t4 on t3.a=t4.a) on t2.a=t3.a;", + "select /*+ straight_join() */ * from (t1 join t2 on t1.a=t2.a) join (t3 join t4 on t3.a=t4.a) on t1.a=t4.a;", + "select /*+ straight_join() */ * from (t3 join t4 on t3.a=t4.a) join (t1 join t2 on t1.a=t2.a) on t2.a=t4.a;", + "select /*+ straight_join() */ * from (t3 join t4 on t3.a=t4.a) join (t1 join t2 on t1.a=t2.a) on t2.a=t3.a;", + "select /*+ straight_join() */ * from ((select t3.a, t3.b from t, t1, t2, t3 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;", + "select /*+ straight_join() */ * from (t1 join t2 on t1.a=t2.a) join ((select t3.a, t3.b from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b) t3 join t4 on t3.a=t4.a) on t2.a=t4.a;", + "select /*+ straight_join() */ * from (t1 join t2 on t1.a=t2.a) join ((select t3.a, t3.b from t1, t3, t2, t where t.a = t1.a and t1.b=t2.b) t3 join t4 on t3.a=t4.a) on t2.a=t3.a;", + "select /*+ straight_join() */ * from (t1 join t2 on t1.a=t2.a) join ((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) on t1.a=t4.a;", + "select /*+ straight_join() */ * from ((select t3.a, t3.b from t1, t2, t, t3 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 t2.a=t4.a;", + "select /*+ straight_join() */ * from ((select t3.a, t3.b from t2, t1, t, t3 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 t2.a=t3.a;", + "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": "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": [ + "select /*+ no_merge_join() */ * from t1, t2 where t1.a=t2.a", + "select /*+ no_merge_join(t1), merge_join(t1) */ * from t1, t2 where t1.a=t2.a", + "select /*+ no_merge_join(t1), merge_join(t2) */ * from t1, t2 where t1.a=t2.a", + "select /*+ no_merge_join(t1) */ * from t1, t2 where t1.a=t2.a", + "select /*+ no_merge_join(t1, t2) */ * from t1, t2 where t1.a=t2.a", + "select /*+ no_merge_join(t2) */ * from t1 right join t2 on t1.a=t2.a", + "select /*+ leading(t4, t3, t2, t1), no_merge_join(t2, t3) */ * from t1, t2, t3, t4 where t1.a=t2.a and t2.a=t3.a and t3.a=t4.a", + "select /*+ leading(t1, t2, t3, t4), merge_join(t1, t2), no_merge_join(t3), merge_join(t4) */ * from t1, t2, t3, t4 where t1.a=t2.a and t2.a=t3.a and t3.a=t4.a" + ] + }, + { + "name": "TestLeadingJoinHint", + "cases": [ + "select /*+ leading(t, t1) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", + "select /*+ leading(t2, t1) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", + "select /*+ leading(t3, t1) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", + "select /*+ leading(t2, t3) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", + "select /*+ leading(t, t1, t2) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", + "select /*+ leading(t2, t1, t3) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", + "select /*+ leading(t3, t1, t) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", + "select /*+ leading(t2, t3, t) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", + "select /*+ leading(t2, t3, t, t4) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", + "select /*+ leading(t3, t2) */ * from t2 join t1 on t2.a=t1.a join t3 on t1.b=t3.b;", + "select /*+ leading(t3, t1) */ * from t2 join t1 on t2.a=t1.a join t3 on t1.b=t3.b;", + "select /*+ leading(t1, t2) */ * from t2 join t1 on t2.a=t1.a join t3 on t1.b=t3.b;", + "select /*+ leading(t3) */ * from t2 join t1 on t2.a=t1.a join t3 on t1.b=t3.b;", + "select /*+ leading(t2) */ * from t2 join t1 on t2.a=t1.a join t3 on t1.b=t3.b;", + "select /*+ leading(t1) */ * from t2 join t1 on t2.a=t1.a join t3 on t1.b=t3.b;", + "select /*+ leading(t3, t2, t1) */ * from t2 join t1 on t2.a=t1.a join t3 on t1.b=t3.b;", + "select /*+ leading(t1, t2, t3) */ * from t2 join t1 on t2.a=t1.a join t3 on t1.b=t3.b;", + "select /*+ leading(t3, t1, t2) */ * from t2 join t1 on t2.a=t1.a join t3 on t1.b=t3.b;", + "select /*+ leading(t2) */ * from t2 join (t1 join t3 on t1.a=t3.a) on t2.a=1;", + "select /*+ leading(t1) */ * from t2 join (t1 join t3 on t1.a=t3.a) on t2.a=1;", + "select /*+ leading(t3) */ * from t2 join (t1 join t3 on t1.a=t3.a) on t2.a=1;", + "select /*+ leading(t2, t1) */ * from t2 join (t1 join t3 on t1.a=t3.a) on t2.a=1;", + "select /*+ leading(t2, t3) */ * from t2 join (t1 join t3 on t1.a=t3.a) on t2.a=1;", + "select /*+ leading(t3, t1) */ * from t2 join (t1 join t3 on t1.a=t3.a) on t2.a=1;", + "select /*+ leading(t2, t1, t3) */ * from t2 join (t1 join t3 on t1.a=t3.a) on t2.a=1;", + "select /*+ leading(t1, t3, t2) */ * from t2 join (t1 join t3 on t1.a=t3.a) on t2.a=1;", + "select /*+ leading(t2, t3, t1) */ * from t2 join (t1 join t3 on t1.a=t3.a) on t2.a=1;", + "select /*+ leading(t1) */ * from t2 join (t1 join t3 on t1.a=t3.a) on t2.a=t3.a;", + "select /*+ leading(t2) */ * from t2 join (t1 join t3 on t1.a=t3.a) on t2.a=t3.a;", + "select /*+ leading(t3) */ * from t2 join (t1 join t3 on t1.a=t3.a) on t2.a=t3.a;", + "select /*+ leading(t1, t2) */ * from t2 join (t1 join t3 on t1.a=t3.a) on t2.a=t3.a;", + "select /*+ leading(t3, t2) */ * from t2 join (t1 join t3 on t1.a=t3.a) on t2.a=t3.a;", + "select /*+ leading(t1, t3) */ * from t2 join (t1 join t3 on t1.a=t3.a) on t2.a=t3.a;", + "select /*+ leading(t1, t2, t3) */ * from t2 join (t1 join t3 on t1.a=t3.a) on t2.a=t3.a;", + "select /*+ leading(t3, t1, t2) */ * from t2 join (t1 join t3 on t1.a=t3.a) on t2.a=t3.a;", + "select /*+ leading(t1) */ * from (t1 join t2 on t1.a=t2.a) join (t3 join t4 on t3.a=t4.a) on t2.a=t4.a;", + "select /*+ leading(t2) */ * from (t1 join t2 on t1.a=t2.a) join (t3 join t4 on t3.a=t4.a) on t2.a=t4.a;", + "select /*+ leading(t3) */ * from (t1 join t2 on t1.a=t2.a) join (t3 join t4 on t3.a=t4.a) on t2.a=t4.a;", + "select /*+ leading(t4) */ * from (t1 join t2 on t1.a=t2.a) join (t3 join t4 on t3.a=t4.a) on t2.a=t4.a;", + "select /*+ leading(t1, t2) */ * from (t1 join t2 on t1.a=t2.a) join (t3 join t4 on t3.a=t4.a) on t2.a=t4.a;", + "select /*+ leading(t1, t3) */ * from (t1 join t2 on t1.a=t2.a) join (t3 join t4 on t3.a=t4.a) on t2.a=t4.a;", + "select /*+ leading(t4, t1) */ * from (t1 join t2 on t1.a=t2.a) join (t3 join t4 on t3.a=t4.a) on t2.a=t4.a;", + "select /*+ leading(t4, t2) */ * from (t1 join t2 on t1.a=t2.a) join (t3 join t4 on t3.a=t4.a) on t2.a=t4.a;", + "select /*+ leading(t3, t2) */ * from (t1 join t2 on t1.a=t2.a) join (t3 join t4 on t3.a=t4.a) on t2.a=t4.a;", + "select /*+ leading(t3, t4) */ * from (t1 join t2 on t1.a=t2.a) join (t3 join t4 on t3.a=t4.a) on t2.a=t4.a;", + "select /*+ leading(t1, t2, t3) */ * from (t1 join t2 on t1.a=t2.a) join (t3 join t4 on t3.a=t4.a) on t2.a=t4.a;", + "select /*+ leading(t1, t4, t3) */ * from (t1 join t2 on t1.a=t2.a) join (t3 join t4 on t3.a=t4.a) on t2.a=t4.a;", + "select /*+ leading(t4, t2, t3) */ * from (t1 join t2 on t1.a=t2.a) join (t3 join t4 on t3.a=t4.a) on t2.a=t4.a;", + "select /*+ leading(t1, t2, t3, t4) */ * from (t1 join t2 on t1.a=t2.a) join (t3 join t4 on t3.a=t4.a) on t2.a=t4.a;", + "select /*+ leading(t1) */ * from ((select 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(t2) */ * from ((select 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) */ * from ((select 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(t4) */ * from ((select 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(t2, t1) */ * from ((select 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(t2, t3) */ * from ((select 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(t4, t1) */ * from ((select 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 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 * 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 * from ((select /*+ leading(t6) */ 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 * from ((select /*+ leading(t5, 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 * from ((select /*+ leading(t6, t8, 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) */ * 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;", + "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;" + ] + }, + { + "name": "TestJoinOrderHint", + "cases": [ + // test cases for using the join order hint and join algorithm hint + "select /*+ straight_join() hash_join(t) */ * from t, t1, t2 where t.a = t1.a and t1.b=t2.b;", + "select /*+ straight_join() merge_join(t1) */ * from t, t1, t2 where t.a = t1.a and t1.b=t2.b;", + "select /*+ straight_join() INL_JOIN(t1) */ * from t, t1, t2 where t.a = t1.a and t1.b=t2.b;", + "select /*+ leading(t2) hash_join(t2) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", + "select /*+ leading(t2) hash_join(t1) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", + "select /*+ leading(t2) INL_JOIN(t1) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", + "select /*+ leading(t2) merge_join(t2) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", + "select /*+ leading(t1) merge_join(t2) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", + "select /*+ leading(t2) INL_JOIN(t1) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;" + ] + }, + { + "name": "TestJoinOrderHint4StaticPartitionTable", + "cases": [ + "select /*+ straight_join() */ * from t, t1, t2 where t.a = t1.a and t1.b=t2.b;", + "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;", + "select /*+ leading(t2, t3) */ * 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, 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;", + // should support this case later + "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;", + "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;" + ] + }, + { + "name": "TestJoinOrderHint4DynamicPartitionTable", + "cases": [ + "select /*+ straight_join() */ * from t, t1, t2 where t.a = t1.a and t1.b=t2.b;", + "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;", + "select /*+ leading(t2, t3) */ * 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, 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;", + // should support this case later + "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;", + "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;" + ] + }, + { + "name": "TestJoinOrderHint4DifferentJoinType", + "cases": [ + "select /*+ straight_join() */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b", + "select /*+ straight_join() */ * from t1 straight_join t2 straight_join t3", + "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", + "select /*+ leading(t3, t1) */ * from t1 right join t2 on t1.a=t2.a right join t3 on t2.b=t3.b", + "select /*+ leading(t1) */ * from t1 left join t2 on t1.a=t2.a straight_join t3", + "select /*+ leading(t1) */ * from t1 right join t2 on t1.a=t2.a straight_join t3", + "select /*+ leading(t1) */ * from t1 left join t2 on t1.a=t2.a cross join t3", + "select /*+ leading(t1) */ * from t1 right join t2 on t1.a=t2.a cross join t3" + ] + }, + { + "name": "TestJoinOrderHint4TiFlash", + "cases": [ + "select /*+ straight_join() */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b", + "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", + // 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;", + "select /*+ leading(t2, t3) */ * 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, 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;", + // should support this case later + "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;", + "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;" + ] + }, + { + "name": "TestJoinOrderHint4Subquery", + "cases": [ + // straight join hint + correlated subquery + "select /*+ straight_join() */ * from t1 join t2 on t1.a=t2.a where t1.a in (select t3.a from t3 where t1.b = t3.b)", + "select /*+ straight_join() */ * 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 /*+ straight_join() */ * from t1 join t2 on t1.a=t2.a where exists (select t3.a from t3 where t1.b = t3.b)", + "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)", + "select /*+ straight_join() */ * from t1 join t2 on t1.a=t2.a where exists (select t3.a from t3)", + "select /*+ straight_join() */ * from t1 join t2 on t1.a=t2.a where not exists (select t3.a from t3)", + "select /*+ straight_join() */ * from t1 join (select * from t4) t2 on t1.a=t2.a join t3 on t2.b=t3.b;", + "select /*+ straight_join() */ * from (select * from t4) t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b;", + "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)", + "select /*+ leading(t2, 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(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)", + "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(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;", + "select /*+ leading(t2@sel_2, t1) */ t1.a, (select min(t2.a) from t2) from t1 join t3 on t1.a = t3.a;", + "select /*+ leading(t2@sel_2, t3) */ 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(t3, t2@sel_2) */ t1.a, (select min(t2.a) from t2) from t1 join t3 on t1.a = t3.a;" + ] + }, + { + "name": "TestLeadingJoinHint4OuterJoin", + "cases": [ + // left 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;", + "select /*+ leading(t2, t3) */ * 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, 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;", + // should support this case later + "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;", + "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 ;", + "select /*+ leading(t2) hash_join(t3) */ * 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(t) hash_join(t2) */ * 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(t) hash_join(t1) */ * 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(t) hash_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 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) INL_JOIN(t3) */ * 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(t3) INL_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 ;", + "select /*+ leading(t3) INL_JOIN(t3) */ * 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) merge_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(t1) merge_join(t2) */ * from t 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) merge_join(t3) */ * 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(t3) merge_join(t3) */ * from t 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(t2) */ * 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(t2) */ * 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 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;", + "select /*+ leading(t2@sel_2, t1) */ t1.a, (select min(t2.a) from t2) from t1 left join t3 on t1.a = t3.a;", + "select /*+ leading(t3, t2@sel_2) */ t1.a, (select min(t2.a) from t2) from t1 right join t3 on t1.a = t3.a;" + ] + } +] diff --git a/planner/core/casetest/rule/testdata/join_reorder_suite_out.json b/planner/core/casetest/rule/testdata/join_reorder_suite_out.json new file mode 100644 index 0000000000000..fd4c581d478ee --- /dev/null +++ b/planner/core/casetest/rule/testdata/join_reorder_suite_out.json @@ -0,0 +1,10168 @@ +[ + { + "Name": "TestStraightJoinHint", + "Cases": [ + { + "SQL": "select /*+ straight_join() */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", + "Plan": [ + "HashJoin 155937656.25 root CARTESIAN inner join", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t1.b, test.t2.b)]", + " ├─TableReader(Build) 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", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.a, test.t1.a)]", + " ├─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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ straight_join() */ * from t2, t1, t3, t where t.a = t1.a and t1.b=t2.b;", + "Plan": [ + "HashJoin 155937656.25 root inner join, equal:[eq(test.t1.a, test.t.a)]", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "└─HashJoin(Probe) 124750125.00 root CARTESIAN inner join", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t1.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", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ straight_join() */ * from t3, t2, t1, t where t.a = t1.a and t1.b=t2.b;", + "Plan": [ + "HashJoin 155937656.25 root inner join, equal:[eq(test.t1.a, test.t.a)]", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "└─HashJoin(Probe) 124750125.00 root inner join, equal:[eq(test.t2.b, test.t1.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", + " └─HashJoin(Probe) 99900000.00 root CARTESIAN inner join", + " ├─TableReader(Build) 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", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ straight_join() */ * from t3, t1, t, t2 where t.a = t1.a and t1.b=t2.b;", + "Plan": [ + "HashJoin 155937656.25 root inner join, equal:[eq(test.t1.b, test.t2.b)]", + "├─TableReader(Build) 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", + "└─HashJoin(Probe) 124750125.00 root inner join, equal:[eq(test.t1.a, test.t.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + " └─HashJoin(Probe) 99800100.00 root CARTESIAN inner join", + " ├─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", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ straight_join() */ * from t2 join t1 on t2.a=t1.a join t3 on t1.b=t3.b;", + "Plan": [ + "HashJoin 15593.77 root inner join, equal:[eq(test.t1.b, test.t3.b)]", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.a, test.t1.a)]", + " ├─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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ straight_join() */ * from t3 join t1 on t1.b=t3.b join t2 on t2.a=t1.a;", + "Plan": [ + "HashJoin 15593.77 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t3.b, test.t1.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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ straight_join() */ * from t3 join t2 on t2.b=t3.b join t1 on t2.a=t1.a;", + "Plan": [ + "HashJoin 15593.77 root inner join, equal:[eq(test.t2.a, test.t1.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", + "└─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t3.b, test.t2.b)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ straight_join() */ * from t2 join (t1 join t3 on t1.a=t3.a) on t2.a=1;", + "Plan": [ + "HashJoin 124875.00 root CARTESIAN inner join", + "├─IndexLookUp(Build) 10.00 root ", + "│ ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t2, index:a(a) range:[1,1], keep order:false, stats:pseudo", + "│ └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t3.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ straight_join() */ * from t2 join (t1 join t3 on t1.a=t3.a) on t2.a=t3.a;", + "Plan": [ + "HashJoin 15609.38 root inner join, equal:[eq(test.t2.a, test.t3.a)]", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t3.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ straight_join() */ * from t1 join (t2 join t3 on t2.a=t3.a) on t1.a=t3.a;", + "Plan": [ + "HashJoin 15609.38 root inner join, equal:[eq(test.t1.a, test.t3.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", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t2.a, test.t3.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ straight_join() */ * from t3 join (t1 join t2 on t1.a=t2.a) on t2.a=t3.a;", + "Plan": [ + "HashJoin 15609.38 root inner join, equal:[eq(test.t3.a, test.t2.a)]", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ straight_join() */ * from t2 join t1 on t1.a=t2.a join t3 on t2.b=t3.b;", + "Plan": [ + "HashJoin 15593.77 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.a, test.t1.a)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ straight_join() */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b;", + "Plan": [ + "HashJoin 15593.77 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ straight_join() */ * from t2 join t3 on t3.a=t2.a join t1 on t2.a=t1.a;", + "Plan": [ + "HashJoin 15609.38 root inner join, equal:[eq(test.t2.a, test.t1.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", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t2.a, test.t3.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ straight_join() */ * from (t1 join t2 on t1.a=t2.a) join (t3 join t4 on t3.a=t4.a) on t2.a=t4.a;", + "Plan": [ + "HashJoin 19511.72 root inner join, equal:[eq(test.t2.a, test.t4.a)]", + "├─HashJoin(Build) 12487.50 root inner join, equal:[eq(test.t3.a, test.t4.a)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ straight_join() */ * from (t1 join t2 on t1.a=t2.a) join (t3 join t4 on t3.a=t4.a) on t2.a=t3.a;", + "Plan": [ + "HashJoin 19511.72 root inner join, equal:[eq(test.t2.a, test.t3.a)]", + "├─HashJoin(Build) 12487.50 root inner join, equal:[eq(test.t3.a, test.t4.a)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ straight_join() */ * from (t1 join t2 on t1.a=t2.a) join (t3 join t4 on t3.a=t4.a) on t1.a=t4.a;", + "Plan": [ + "HashJoin 19511.72 root inner join, equal:[eq(test.t1.a, test.t4.a)]", + "├─HashJoin(Build) 12487.50 root inner join, equal:[eq(test.t3.a, test.t4.a)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ straight_join() */ * from (t3 join t4 on t3.a=t4.a) join (t1 join t2 on t1.a=t2.a) on t2.a=t4.a;", + "Plan": [ + "HashJoin 19511.72 root inner join, equal:[eq(test.t4.a, test.t2.a)]", + "├─HashJoin(Build) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 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", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t3.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ straight_join() */ * from (t3 join t4 on t3.a=t4.a) join (t1 join t2 on t1.a=t2.a) on t2.a=t3.a;", + "Plan": [ + "HashJoin 19511.72 root inner join, equal:[eq(test.t3.a, test.t2.a)]", + "├─HashJoin(Build) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 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", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t3.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ straight_join() */ * from ((select t3.a, t3.b from t, t1, t2, t3 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;", + "Plan": [ + "HashJoin 304261169.13 root inner join, equal:[eq(test.t4.a, test.t1.a)]", + "├─HashJoin(Build) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 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", + "└─HashJoin(Probe) 194727148.24 root inner join, equal:[eq(test.t3.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 155781718.59 root CARTESIAN inner join", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t1.b, test.t2.b)]", + " ├─TableReader(Build) 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", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.a, test.t1.a)]", + " ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + " │ └─IndexFullScan 9990.00 cop[tikv] table:t, index:a(a) keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ straight_join() */ * from (t1 join t2 on t1.a=t2.a) join ((select t3.a, t3.b from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b) t3 join t4 on t3.a=t4.a) on t2.a=t4.a;", + "Plan": [ + "HashJoin 304261169.13 root inner join, equal:[eq(test.t2.a, test.t4.a)]", + "├─HashJoin(Build) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 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", + "└─HashJoin(Probe) 194727148.24 root inner join, equal:[eq(test.t3.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 155781718.59 root CARTESIAN inner join", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t1.b, test.t2.b)]", + " ├─TableReader(Build) 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", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.a, test.t1.a)]", + " ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + " │ └─IndexFullScan 9990.00 cop[tikv] table:t, index:a(a) keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ straight_join() */ * from (t1 join t2 on t1.a=t2.a) join ((select t3.a, t3.b from t1, t3, t2, t where t.a = t1.a and t1.b=t2.b) t3 join t4 on t3.a=t4.a) on t2.a=t3.a;", + "Plan": [ + "HashJoin 304261169.13 root inner join, equal:[eq(test.t2.a, test.t3.a)]", + "├─HashJoin(Build) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 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", + "└─HashJoin(Probe) 194727148.24 root inner join, equal:[eq(test.t3.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 155781718.59 root inner join, equal:[eq(test.t1.a, test.t.a)]", + " ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + " │ └─IndexFullScan 9990.00 cop[tikv] table:t, index:a(a) keep order:false, stats:pseudo", + " └─HashJoin(Probe) 124625374.88 root inner join, equal:[eq(test.t1.b, test.t2.b)]", + " ├─TableReader(Build) 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", + " └─HashJoin(Probe) 99700299.90 root CARTESIAN inner join", + " ├─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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ straight_join() */ * from (t1 join t2 on t1.a=t2.a) join ((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) on t1.a=t4.a;", + "Plan": [ + "HashJoin 304261169.13 root inner join, equal:[eq(test.t1.a, test.t4.a)]", + "├─HashJoin(Build) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 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", + "└─HashJoin(Probe) 194727148.24 root inner join, equal:[eq(test.t3.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 155781718.59 root inner join, equal:[eq(test.t1.a, test.t.a)]", + " ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + " │ └─IndexFullScan 9990.00 cop[tikv] table:t, index:a(a) keep order:false, stats:pseudo", + " └─HashJoin(Probe) 124625374.88 root inner join, equal:[eq(test.t2.b, test.t1.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", + " └─HashJoin(Probe) 99800100.00 root CARTESIAN inner join", + " ├─TableReader(Build) 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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ straight_join() */ * from ((select t3.a, t3.b from t1, t2, t, t3 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 t2.a=t4.a;", + "Plan": [ + "HashJoin 304261169.13 root inner join, equal:[eq(test.t4.a, test.t2.a)]", + "├─HashJoin(Build) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 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", + "└─HashJoin(Probe) 194727148.24 root inner join, equal:[eq(test.t3.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 155781718.59 root CARTESIAN inner join", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t1.a, test.t.a)]", + " ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + " │ └─IndexFullScan 9990.00 cop[tikv] table:t, index:a(a) keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t1.b, test.t2.b)]", + " ├─TableReader(Build) 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", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ straight_join() */ * from ((select t3.a, t3.b from t2, t1, t, t3 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 t2.a=t3.a;", + "Plan": [ + "HashJoin 304261169.13 root inner join, equal:[eq(test.t3.a, test.t2.a)]", + "├─HashJoin(Build) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 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", + "└─HashJoin(Probe) 194727148.24 root inner join, equal:[eq(test.t3.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 155781718.59 root CARTESIAN inner join", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t1.a, test.t.a)]", + " ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + " │ └─IndexFullScan 9990.00 cop[tikv] table:t, index:a(a) keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t1.b)]", + " ├─TableReader(Build) 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", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "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;", + "Plan": [ + "HashJoin 304261169.13 root inner join, equal:[eq(test.t4.a, test.t1.a)]", + "├─HashJoin(Build) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 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", + "└─HashJoin(Probe) 194727148.24 root inner join, equal:[eq(test.t3.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 155781718.59 root inner join, equal:[eq(test.t1.a, test.t.a)]", + " ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + " │ └─IndexFullScan 9990.00 cop[tikv] table:t, index:a(a) keep order:false, stats:pseudo", + " └─HashJoin(Probe) 124625374.88 root inner join, equal:[eq(test.t2.b, test.t1.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", + " └─HashJoin(Probe) 99800100.00 root CARTESIAN inner join", + " ├─TableReader(Build) 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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": null + } + ] + }, + { + "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": "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": [ + { + "SQL": "select /*+ no_merge_join() */ * from t1, t2 where t1.a=t2.a", + "Plan": [ + "MergeJoin 12487.50 root inner join, left key:test.t1.a, right key:test.t2.a", + "├─IndexReader(Build) 9990.00 root index:IndexFullScan", + "│ └─IndexFullScan 9990.00 cop[tikv] table:t2, index:a(a) keep order:true, stats:pseudo", + "└─IndexReader(Probe) 9990.00 root index:IndexFullScan", + " └─IndexFullScan 9990.00 cop[tikv] table:t1, index:a(a) keep order:true, stats:pseudo" + ], + "Warning": [ + "Warning 1815 Hint no_merge_join() is inapplicable. Please specify the table names in the arguments." + ] + }, + { + "SQL": "select /*+ no_merge_join(t1), merge_join(t1) */ * from t1, t2 where t1.a=t2.a", + "Plan": [ + "MergeJoin 12487.50 root inner join, left key:test.t1.a, right key:test.t2.a", + "├─IndexReader(Build) 9990.00 root index:IndexFullScan", + "│ └─IndexFullScan 9990.00 cop[tikv] table:t2, index:a(a) keep order:true, stats:pseudo", + "└─IndexReader(Probe) 9990.00 root index:IndexFullScan", + " └─IndexFullScan 9990.00 cop[tikv] table:t1, index:a(a) keep order:true, stats:pseudo" + ], + "Warning": [ + "Warning 1815 Some MERGE_JOIN and NO_MERGE_JOIN hints conflict, NO_MERGE_JOIN is ignored" + ] + }, + { + "SQL": "select /*+ no_merge_join(t1), merge_join(t2) */ * from t1, t2 where t1.a=t2.a", + "Plan": [ + "MergeJoin 12487.50 root inner join, left key:test.t1.a, right key:test.t2.a", + "├─IndexReader(Build) 9990.00 root index:IndexFullScan", + "│ └─IndexFullScan 9990.00 cop[tikv] table:t2, index:a(a) keep order:true, stats:pseudo", + "└─IndexReader(Probe) 9990.00 root index:IndexFullScan", + " └─IndexFullScan 9990.00 cop[tikv] table:t1, index:a(a) keep order:true, stats:pseudo" + ], + "Warning": [ + "Warning 1815 Some MERGE_JOIN and NO_MERGE_JOIN hints conflict, NO_MERGE_JOIN is ignored" + ] + }, + { + "SQL": "select /*+ no_merge_join(t1) */ * from t1, t2 where t1.a=t2.a", + "Plan": [ + "HashJoin 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.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) 9990.00 root index:IndexFullScan", + " └─IndexFullScan 9990.00 cop[tikv] table:t1, index:a(a) keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ no_merge_join(t1, t2) */ * from t1, t2 where t1.a=t2.a", + "Plan": [ + "HashJoin 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.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) 9990.00 root index:IndexFullScan", + " └─IndexFullScan 9990.00 cop[tikv] table:t1, index:a(a) keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ no_merge_join(t2) */ * from t1 right join t2 on t1.a=t2.a", + "Plan": [ + "HashJoin 12487.50 root right outer join, equal:[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) 10000.00 root index:IndexFullScan", + " └─IndexFullScan 10000.00 cop[tikv] table:t2, index:a(a) keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t4, t3, t2, t1), no_merge_join(t2, t3) */ * from t1, t2, t3, t4 where t1.a=t2.a and t2.a=t3.a and t3.a=t4.a", + "Plan": [ + "Projection 19511.72 root test.t1.a, test.t2.a, test.t3.a, test.t4.a", + "└─HashJoin 19511.72 root inner join, equal:[eq(test.t2.a, test.t1.a)]", + " ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + " │ └─IndexFullScan 9990.00 cop[tikv] table:t1, index:a(a) keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15609.38 root inner join, equal:[eq(test.t3.a, test.t2.a)]", + " ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + " │ └─IndexFullScan 9990.00 cop[tikv] table:t2, index:a(a) keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t4.a, test.t3.a)]", + " ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + " │ └─IndexFullScan 9990.00 cop[tikv] table:t3, index:a(a) keep order:false, stats:pseudo", + " └─IndexReader(Probe) 9990.00 root index:IndexFullScan", + " └─IndexFullScan 9990.00 cop[tikv] table:t4, index:a(a) keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t1, t2, t3, t4), merge_join(t1, t2), no_merge_join(t3), merge_join(t4) */ * from t1, t2, t3, t4 where t1.a=t2.a and t2.a=t3.a and t3.a=t4.a", + "Plan": [ + "MergeJoin 19511.72 root inner join, left key:test.t3.a, right key:test.t4.a", + "├─IndexReader(Build) 9990.00 root index:IndexFullScan", + "│ └─IndexFullScan 9990.00 cop[tikv] table:t4, index:a(a) keep order:true, stats:pseudo", + "└─Sort(Probe) 15609.38 root test.t3.a", + " └─HashJoin 15609.38 root inner join, equal:[eq(test.t2.a, test.t3.a)]", + " ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + " │ └─IndexFullScan 9990.00 cop[tikv] table:t3, index:a(a) keep order:false, stats:pseudo", + " └─MergeJoin(Probe) 12487.50 root inner join, left key:test.t1.a, right key:test.t2.a", + " ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + " │ └─IndexFullScan 9990.00 cop[tikv] table:t2, index:a(a) keep order:true, stats:pseudo", + " └─IndexReader(Probe) 9990.00 root index:IndexFullScan", + " └─IndexFullScan 9990.00 cop[tikv] table:t1, index:a(a) keep order:true, stats:pseudo" + ], + "Warning": null + } + ] + }, + { + "Name": "TestLeadingJoinHint", + "Cases": [ + { + "SQL": "select /*+ leading(t, t1) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", + "Plan": [ + "HashJoin 155937656.25 root CARTESIAN inner join", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t1.b, test.t2.b)]", + " ├─TableReader(Build) 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", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.a, test.t1.a)]", + " ├─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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2, t1) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", + "Plan": [ + "Projection 155937656.25 root test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 155937656.25 root CARTESIAN inner join", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t1.a, test.t.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t1.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", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3, t1) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", + "Plan": [ + "Projection 155937656.25 root test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 155937656.25 root inner join, equal:[eq(test.t1.b, test.t2.b)]", + " ├─TableReader(Build) 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", + " └─HashJoin(Probe) 124750125.00 root inner join, equal:[eq(test.t1.a, test.t.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + " └─HashJoin(Probe) 99800100.00 root CARTESIAN inner join", + " ├─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", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2, t3) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", + "Plan": [ + "Projection 155937656.25 root test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 155937656.25 root inner join, equal:[eq(test.t1.a, test.t.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + " └─HashJoin(Probe) 124750125.00 root inner join, equal:[eq(test.t2.b, test.t1.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", + " └─HashJoin(Probe) 99900000.00 root CARTESIAN inner join", + " ├─TableReader(Build) 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", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t, t1, t2) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", + "Plan": [ + "HashJoin 155937656.25 root CARTESIAN inner join", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t1.b, test.t2.b)]", + " ├─TableReader(Build) 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", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.a, test.t1.a)]", + " ├─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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table has join conditions with other tables" + ] + }, + { + "SQL": "select /*+ leading(t2, t1, t3) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", + "Plan": [ + "Projection 155937656.25 root test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 155937656.25 root inner join, equal:[eq(test.t1.a, test.t.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + " └─HashJoin(Probe) 124750125.00 root CARTESIAN inner join", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t1.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", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3, t1, t) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", + "Plan": [ + "Projection 155937656.25 root test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 155937656.25 root inner join, equal:[eq(test.t1.b, test.t2.b)]", + " ├─TableReader(Build) 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", + " └─HashJoin(Probe) 124750125.00 root inner join, equal:[eq(test.t1.a, test.t.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + " └─HashJoin(Probe) 99800100.00 root CARTESIAN inner join", + " ├─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", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2, t3, t) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", + "Plan": [ + "Projection 1246253748750.00 root test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 1246253748750.00 root inner join, equal:[eq(test.t.a, test.t1.a) eq(test.t2.b, test.t1.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", + " └─HashJoin(Probe) 998001000000.00 root CARTESIAN inner join", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + " └─HashJoin(Probe) 99900000.00 root CARTESIAN inner join", + " ├─TableReader(Build) 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", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2, t3, t, t4) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", + "Plan": [ + "Projection 155937656.25 root test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 155937656.25 root CARTESIAN inner join", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t1.b, test.t2.b)]", + " ├─TableReader(Build) 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", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t1.a, test.t.a)]", + " ├─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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t4) in optimizer hint /*+ LEADING(t2, t3, t, t4) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t3, t2) */ * from t2 join t1 on t2.a=t1.a join t3 on t1.b=t3.b;", + "Plan": [ + "Projection 124625374.88 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b", + "└─HashJoin 124625374.88 root inner join, equal:[eq(test.t2.a, test.t1.a) eq(test.t3.b, test.t1.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", + " └─HashJoin(Probe) 99800100.00 root CARTESIAN inner join", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3, t1) */ * from t2 join t1 on t2.a=t1.a join t3 on t1.b=t3.b;", + "Plan": [ + "Projection 15593.77 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b", + "└─HashJoin 15593.77 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t3.b, test.t1.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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t1, t2) */ * from t2 join t1 on t2.a=t1.a join t3 on t1.b=t3.b;", + "Plan": [ + "Projection 15593.77 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b", + "└─HashJoin 15593.77 root inner join, equal:[eq(test.t1.b, test.t3.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3) */ * from t2 join t1 on t2.a=t1.a join t3 on t1.b=t3.b;", + "Plan": [ + "Projection 15593.77 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b", + "└─HashJoin 15593.77 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t3.b, test.t1.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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2) */ * from t2 join t1 on t2.a=t1.a join t3 on t1.b=t3.b;", + "Plan": [ + "HashJoin 15593.77 root inner join, equal:[eq(test.t1.b, test.t3.b)]", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.a, test.t1.a)]", + " ├─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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t1) */ * from t2 join t1 on t2.a=t1.a join t3 on t1.b=t3.b;", + "Plan": [ + "Projection 15593.77 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b", + "└─HashJoin 15593.77 root inner join, equal:[eq(test.t1.b, test.t3.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3, t2, t1) */ * from t2 join t1 on t2.a=t1.a join t3 on t1.b=t3.b;", + "Plan": [ + "Projection 124625374.88 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b", + "└─HashJoin 124625374.88 root inner join, equal:[eq(test.t2.a, test.t1.a) eq(test.t3.b, test.t1.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", + " └─HashJoin(Probe) 99800100.00 root CARTESIAN inner join", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t1, t2, t3) */ * from t2 join t1 on t2.a=t1.a join t3 on t1.b=t3.b;", + "Plan": [ + "Projection 15593.77 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b", + "└─HashJoin 15593.77 root inner join, equal:[eq(test.t1.b, test.t3.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3, t1, t2) */ * from t2 join t1 on t2.a=t1.a join t3 on t1.b=t3.b;", + "Plan": [ + "Projection 15593.77 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b", + "└─HashJoin 15593.77 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t3.b, test.t1.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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2) */ * from t2 join (t1 join t3 on t1.a=t3.a) on t2.a=1;", + "Plan": [ + "HashJoin 124875.00 root CARTESIAN inner join", + "├─IndexLookUp(Build) 10.00 root ", + "│ ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t2, index:a(a) range:[1,1], keep order:false, stats:pseudo", + "│ └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t3.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table has join conditions with other tables" + ] + }, + { + "SQL": "select /*+ leading(t1) */ * from t2 join (t1 join t3 on t1.a=t3.a) on t2.a=1;", + "Plan": [ + "Projection 124875.00 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b", + "└─HashJoin 124875.00 root CARTESIAN inner join", + " ├─IndexLookUp(Build) 10.00 root ", + " │ ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t2, index:a(a) range:[1,1], keep order:false, stats:pseudo", + " │ └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t3.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3) */ * from t2 join (t1 join t3 on t1.a=t3.a) on t2.a=1;", + "Plan": [ + "Projection 124875.00 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b", + "└─HashJoin 124875.00 root CARTESIAN inner join", + " ├─IndexLookUp(Build) 10.00 root ", + " │ ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t2, index:a(a) range:[1,1], keep order:false, stats:pseudo", + " │ └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t3.a, test.t1.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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2, t1) */ * from t2 join (t1 join t3 on t1.a=t3.a) on t2.a=1;", + "Plan": [ + "HashJoin 124875.00 root inner join, equal:[eq(test.t1.a, test.t3.a)]", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 99900.00 root CARTESIAN inner join", + " ├─IndexLookUp(Build) 10.00 root ", + " │ ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t2, index:a(a) range:[1,1], keep order:false, stats:pseudo", + " │ └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2, t3) */ * from t2 join (t1 join t3 on t1.a=t3.a) on t2.a=1;", + "Plan": [ + "Projection 124875.00 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b", + "└─HashJoin 124875.00 root inner join, equal:[eq(test.t3.a, test.t1.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", + " └─HashJoin(Probe) 99900.00 root CARTESIAN inner join", + " ├─IndexLookUp(Build) 10.00 root ", + " │ ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t2, index:a(a) range:[1,1], keep order:false, stats:pseudo", + " │ └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3, t1) */ * from t2 join (t1 join t3 on t1.a=t3.a) on t2.a=1;", + "Plan": [ + "Projection 124875.00 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b", + "└─HashJoin 124875.00 root CARTESIAN inner join", + " ├─IndexLookUp(Build) 10.00 root ", + " │ ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t2, index:a(a) range:[1,1], keep order:false, stats:pseudo", + " │ └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t3.a, test.t1.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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table has join conditions with other tables" + ] + }, + { + "SQL": "select /*+ leading(t2, t1, t3) */ * from t2 join (t1 join t3 on t1.a=t3.a) on t2.a=1;", + "Plan": [ + "HashJoin 124875.00 root inner join, equal:[eq(test.t1.a, test.t3.a)]", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 99900.00 root CARTESIAN inner join", + " ├─IndexLookUp(Build) 10.00 root ", + " │ ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t2, index:a(a) range:[1,1], keep order:false, stats:pseudo", + " │ └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t1, t3, t2) */ * from t2 join (t1 join t3 on t1.a=t3.a) on t2.a=1;", + "Plan": [ + "Projection 124875.00 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b", + "└─HashJoin 124875.00 root CARTESIAN inner join", + " ├─IndexLookUp(Build) 10.00 root ", + " │ ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t2, index:a(a) range:[1,1], keep order:false, stats:pseudo", + " │ └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t3.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2, t3, t1) */ * from t2 join (t1 join t3 on t1.a=t3.a) on t2.a=1;", + "Plan": [ + "Projection 124875.00 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b", + "└─HashJoin 124875.00 root inner join, equal:[eq(test.t3.a, test.t1.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", + " └─HashJoin(Probe) 99900.00 root CARTESIAN inner join", + " ├─IndexLookUp(Build) 10.00 root ", + " │ ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t2, index:a(a) range:[1,1], keep order:false, stats:pseudo", + " │ └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t1) */ * from t2 join (t1 join t3 on t1.a=t3.a) on t2.a=t3.a;", + "Plan": [ + "Projection 15609.38 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b", + "└─HashJoin 15609.38 root inner join, equal:[eq(test.t3.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t3.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2) */ * from t2 join (t1 join t3 on t1.a=t3.a) on t2.a=t3.a;", + "Plan": [ + "Projection 15609.38 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b", + "└─HashJoin 15609.38 root inner join, equal:[eq(test.t3.a, test.t1.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", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t2.a, test.t3.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3) */ * from t2 join (t1 join t3 on t1.a=t3.a) on t2.a=t3.a;", + "Plan": [ + "Projection 15609.38 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b", + "└─HashJoin 15609.38 root inner join, equal:[eq(test.t3.a, test.t1.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", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t3.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t1, t2) */ * from t2 join (t1 join t3 on t1.a=t3.a) on t2.a=t3.a;", + "Plan": [ + "Projection 124750125.00 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b", + "└─HashJoin 124750125.00 root inner join, equal:[eq(test.t1.a, test.t3.a) eq(test.t2.a, test.t3.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 99800100.00 root CARTESIAN inner join", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3, t2) */ * from t2 join (t1 join t3 on t1.a=t3.a) on t2.a=t3.a;", + "Plan": [ + "Projection 15609.38 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b", + "└─HashJoin 15609.38 root inner join, equal:[eq(test.t3.a, test.t1.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", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t3.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t1, t3) */ * from t2 join (t1 join t3 on t1.a=t3.a) on t2.a=t3.a;", + "Plan": [ + "Projection 15609.38 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b", + "└─HashJoin 15609.38 root inner join, equal:[eq(test.t3.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t3.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t1, t2, t3) */ * from t2 join (t1 join t3 on t1.a=t3.a) on t2.a=t3.a;", + "Plan": [ + "Projection 124750125.00 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b", + "└─HashJoin 124750125.00 root inner join, equal:[eq(test.t1.a, test.t3.a) eq(test.t2.a, test.t3.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 99800100.00 root CARTESIAN inner join", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3, t1, t2) */ * from t2 join (t1 join t3 on t1.a=t3.a) on t2.a=t3.a;", + "Plan": [ + "Projection 15609.38 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b", + "└─HashJoin 15609.38 root inner join, equal:[eq(test.t3.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t3.a, test.t1.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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t1) */ * from (t1 join t2 on t1.a=t2.a) join (t3 join t4 on t3.a=t4.a) on t2.a=t4.a;", + "Plan": [ + "Projection 19511.72 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 19511.72 root inner join, equal:[eq(test.t4.a, test.t3.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15609.38 root inner join, equal:[eq(test.t2.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2) */ * from (t1 join t2 on t1.a=t2.a) join (t3 join t4 on t3.a=t4.a) on t2.a=t4.a;", + "Plan": [ + "Projection 19511.72 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 19511.72 root inner join, equal:[eq(test.t4.a, test.t3.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15609.38 root inner join, equal:[eq(test.t2.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t2.a, test.t1.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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3) */ * from (t1 join t2 on t1.a=t2.a) join (t3 join t4 on t3.a=t4.a) on t2.a=t4.a;", + "Plan": [ + "Projection 19511.72 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 19511.72 root inner join, equal:[eq(test.t2.a, test.t1.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", + " └─HashJoin(Probe) 15609.38 root inner join, equal:[eq(test.t4.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t3.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t4) */ * from (t1 join t2 on t1.a=t2.a) join (t3 join t4 on t3.a=t4.a) on t2.a=t4.a;", + "Plan": [ + "Projection 19511.72 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 19511.72 root inner join, equal:[eq(test.t4.a, test.t3.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15609.38 root inner join, equal:[eq(test.t2.a, test.t1.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", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t4.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t1, t2) */ * from (t1 join t2 on t1.a=t2.a) join (t3 join t4 on t3.a=t4.a) on t2.a=t4.a;", + "Plan": [ + "Projection 19511.72 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 19511.72 root inner join, equal:[eq(test.t4.a, test.t3.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15609.38 root inner join, equal:[eq(test.t2.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t1, t3) */ * from (t1 join t2 on t1.a=t2.a) join (t3 join t4 on t3.a=t4.a) on t2.a=t4.a;", + "Plan": [ + "Projection 155937656.25 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 155937656.25 root inner join, equal:[eq(test.t3.a, test.t4.a) eq(test.t2.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 124750125.00 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 99800100.00 root CARTESIAN inner join", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t4, t1) */ * from (t1 join t2 on t1.a=t2.a) join (t3 join t4 on t3.a=t4.a) on t2.a=t4.a;", + "Plan": [ + "Projection 155937656.25 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 155937656.25 root inner join, equal:[eq(test.t4.a, test.t3.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 124750125.00 root inner join, equal:[eq(test.t1.a, test.t2.a) eq(test.t4.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 99800100.00 root CARTESIAN inner join", + " ├─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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t4, t2) */ * from (t1 join t2 on t1.a=t2.a) join (t3 join t4 on t3.a=t4.a) on t2.a=t4.a;", + "Plan": [ + "Projection 19511.72 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 19511.72 root inner join, equal:[eq(test.t4.a, test.t3.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15609.38 root inner join, equal:[eq(test.t2.a, test.t1.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", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t4.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3, t2) */ * from (t1 join t2 on t1.a=t2.a) join (t3 join t4 on t3.a=t4.a) on t2.a=t4.a;", + "Plan": [ + "Projection 155937656.25 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 155937656.25 root inner join, equal:[eq(test.t3.a, test.t4.a) eq(test.t2.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 124750125.00 root inner join, equal:[eq(test.t2.a, test.t1.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", + " └─HashJoin(Probe) 99800100.00 root CARTESIAN inner join", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3, t4) */ * from (t1 join t2 on t1.a=t2.a) join (t3 join t4 on t3.a=t4.a) on t2.a=t4.a;", + "Plan": [ + "Projection 19511.72 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 19511.72 root inner join, equal:[eq(test.t2.a, test.t1.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", + " └─HashJoin(Probe) 15609.38 root inner join, equal:[eq(test.t4.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t3.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t1, t2, t3) */ * from (t1 join t2 on t1.a=t2.a) join (t3 join t4 on t3.a=t4.a) on t2.a=t4.a;", + "Plan": [ + "HashJoin 155937656.25 root inner join, equal:[eq(test.t3.a, test.t4.a) eq(test.t2.a, test.t4.a)]", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 124750125.00 root CARTESIAN inner join", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t1, t4, t3) */ * from (t1 join t2 on t1.a=t2.a) join (t3 join t4 on t3.a=t4.a) on t2.a=t4.a;", + "Plan": [ + "Projection 155937656.25 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 155937656.25 root inner join, equal:[eq(test.t1.a, test.t2.a) eq(test.t4.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 124750125.00 root inner join, equal:[eq(test.t4.a, test.t3.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 99800100.00 root CARTESIAN inner join", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t4, t2, t3) */ * from (t1 join t2 on t1.a=t2.a) join (t3 join t4 on t3.a=t4.a) on t2.a=t4.a;", + "Plan": [ + "Projection 19511.72 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 19511.72 root inner join, equal:[eq(test.t2.a, test.t1.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", + " └─HashJoin(Probe) 15609.38 root inner join, equal:[eq(test.t4.a, test.t3.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t4.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t1, t2, t3, t4) */ * from (t1 join t2 on t1.a=t2.a) join (t3 join t4 on t3.a=t4.a) on t2.a=t4.a;", + "Plan": [ + "HashJoin 155937656.25 root inner join, equal:[eq(test.t3.a, test.t4.a) eq(test.t2.a, test.t4.a)]", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 124750125.00 root CARTESIAN inner join", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t1) */ * from ((select 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;", + "Plan": [ + "Projection 304261169.13 root test.t8.a, test.t8.b, test.t4.a, test.t4.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b", + "└─HashJoin 304261169.13 root CARTESIAN inner join", + " ├─HashJoin(Build) 15593.77 root inner join, equal:[eq(test.t6.a, test.t5.a)]", + " │ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + " │ │ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", + " │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t6.b, test.t7.b)]", + " │ ├─TableReader(Build) 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + " │ └─TableReader(Probe) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 19511.72 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15609.38 root inner join, equal:[eq(test.t4.a, test.t8.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t8.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2) */ * from ((select 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;", + "Plan": [ + "Projection 304261169.13 root test.t8.a, test.t8.b, test.t4.a, test.t4.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b", + "└─HashJoin 304261169.13 root CARTESIAN inner join", + " ├─HashJoin(Build) 15593.77 root inner join, equal:[eq(test.t6.a, test.t5.a)]", + " │ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + " │ │ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", + " │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t6.b, test.t7.b)]", + " │ ├─TableReader(Build) 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + " │ └─TableReader(Probe) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 19511.72 root inner join, equal:[eq(test.t4.a, test.t8.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t8.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15609.38 root inner join, equal:[eq(test.t1.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t2.a, test.t1.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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3) */ * from ((select 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;", + "Plan": [ + "HashJoin 304261169.13 root CARTESIAN inner join", + "├─HashJoin(Build) 15593.77 root inner join, equal:[eq(test.t6.a, test.t5.a)]", + "│ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + "│ │ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", + "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t6.b, test.t7.b)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 19511.72 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15609.38 root inner join, equal:[eq(test.t4.a, test.t1.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", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t8.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t8.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t4) */ * from ((select 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;", + "Plan": [ + "Projection 304261169.13 root test.t8.a, test.t8.b, test.t4.a, test.t4.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b", + "└─HashJoin 304261169.13 root CARTESIAN inner join", + " ├─HashJoin(Build) 15593.77 root inner join, equal:[eq(test.t6.a, test.t5.a)]", + " │ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + " │ │ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", + " │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t6.b, test.t7.b)]", + " │ ├─TableReader(Build) 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + " │ └─TableReader(Probe) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 19511.72 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15609.38 root inner join, equal:[eq(test.t4.a, test.t1.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", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t4.a, test.t8.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t8.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2, t1) */ * from ((select 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;", + "Plan": [ + "Projection 304261169.13 root test.t8.a, test.t8.b, test.t4.a, test.t4.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b", + "└─HashJoin 304261169.13 root CARTESIAN inner join", + " ├─HashJoin(Build) 15593.77 root inner join, equal:[eq(test.t6.a, test.t5.a)]", + " │ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + " │ │ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", + " │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t6.b, test.t7.b)]", + " │ ├─TableReader(Build) 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + " │ └─TableReader(Probe) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 19511.72 root inner join, equal:[eq(test.t4.a, test.t8.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t8.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15609.38 root inner join, equal:[eq(test.t1.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t2.a, test.t1.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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2, t3) */ * from ((select 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;", + "Plan": [ + "HashJoin 304261169.13 root CARTESIAN inner join", + "├─HashJoin(Build) 15593.77 root inner join, equal:[eq(test.t6.a, test.t5.a)]", + "│ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + "│ │ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", + "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t6.b, test.t7.b)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 19511.72 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15609.38 root inner join, equal:[eq(test.t4.a, test.t1.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", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t8.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t8.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t4, t1) */ * from ((select 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;", + "Plan": [ + "Projection 304261169.13 root test.t8.a, test.t8.b, test.t4.a, test.t4.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b", + "└─HashJoin 304261169.13 root CARTESIAN inner join", + " ├─HashJoin(Build) 15593.77 root inner join, equal:[eq(test.t6.a, test.t5.a)]", + " │ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + " │ │ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", + " │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t6.b, test.t7.b)]", + " │ ├─TableReader(Build) 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + " │ └─TableReader(Probe) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 19511.72 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15609.38 root inner join, equal:[eq(test.t4.a, test.t8.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t8.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t4.a, test.t1.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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3, t1) */ * from ((select 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;", + "Plan": [ + "HashJoin 304261169.13 root CARTESIAN inner join", + "├─HashJoin(Build) 15593.77 root inner join, equal:[eq(test.t6.a, test.t5.a)]", + "│ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + "│ │ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", + "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t6.b, test.t7.b)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 19511.72 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15609.38 root inner join, equal:[eq(test.t4.a, test.t1.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", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t8.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t8.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select * 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;", + "Plan": [ + "HashJoin 304261169.13 root CARTESIAN inner join", + "├─HashJoin(Build) 15593.77 root inner join, equal:[eq(test.t6.b, test.t7.b)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t5.a, test.t6.a)]", + "│ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + "│ │ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 19511.72 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15609.38 root inner join, equal:[eq(test.t4.a, test.t1.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", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t8.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t8.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select * from ((select /*+ leading(t6) */ 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;", + "Plan": [ + "HashJoin 304261169.13 root CARTESIAN inner join", + "├─HashJoin(Build) 15593.77 root inner join, equal:[eq(test.t6.a, test.t5.a)]", + "│ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + "│ │ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", + "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t6.b, test.t7.b)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 19511.72 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15609.38 root inner join, equal:[eq(test.t4.a, test.t1.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", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t8.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t8.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select * from ((select /*+ leading(t5, 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;", + "Plan": [ + "HashJoin 2431655263674.32 root CARTESIAN inner join", + "├─HashJoin(Build) 19511.72 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "│ └─HashJoin(Probe) 15609.38 root inner join, equal:[eq(test.t4.a, test.t1.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", + "│ └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t8.a, test.t4.a)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t8.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 124625374.88 root inner join, equal:[eq(test.t7.b, test.t6.b) eq(test.t5.a, test.t6.a)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 99800100.00 root CARTESIAN inner join", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + " └─IndexReader(Probe) 9990.00 root index:IndexFullScan", + " └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select * from ((select /*+ leading(t6, t8, 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;", + "Plan": [ + "HashJoin 304261169.13 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 243408935.30 root inner join, equal:[eq(test.t4.a, test.t1.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", + " └─HashJoin(Probe) 194727148.24 root inner join, equal:[eq(test.t8.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 155781718.59 root inner join, equal:[eq(test.t6.a, test.t5.a)]", + " ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + " │ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", + " └─HashJoin(Probe) 124625374.88 root inner join, equal:[eq(test.t6.b, test.t7.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 99700299.90 root CARTESIAN inner join", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t8.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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;", + "Plan": [ + "HashJoin 304261169.13 root CARTESIAN inner join", + "├─HashJoin(Build) 15593.77 root inner join, equal:[eq(test.t6.a, test.t5.a)]", + "│ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + "│ │ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", + "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t6.b, test.t7.b)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 19511.72 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15609.38 root inner join, equal:[eq(test.t4.a, test.t1.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", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t8.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t8.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "HashJoin 304261169.13 root CARTESIAN inner join", + "├─HashJoin(Build) 15593.77 root inner join, equal:[eq(test.t6.a, test.t5.a)]", + "│ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + "│ │ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", + "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t6.b, test.t7.b)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 19511.72 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15609.38 root inner join, equal:[eq(test.t4.a, test.t1.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", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t8.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t8.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "HashJoin 304261169.13 root CARTESIAN inner join", + "├─HashJoin(Build) 15593.77 root inner join, equal:[eq(test.t6.a, test.t5.a)]", + "│ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + "│ │ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", + "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t6.b, test.t7.b)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 19511.72 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15609.38 root inner join, equal:[eq(test.t4.a, test.t1.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", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t8.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t8.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "HashJoin 304261169.13 root CARTESIAN inner join", + "├─HashJoin(Build) 15593.77 root inner join, equal:[eq(test.t6.a, test.t5.a)]", + "│ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + "│ │ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", + "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t6.b, test.t7.b)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 19511.72 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15609.38 root inner join, equal:[eq(test.t4.a, test.t1.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", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t8.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t8.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid" + ] + }, + { + "SQL": "select /*+ leading(tx, t1, t3) */ * from t1, (select * from t2) tx, t3 where t1.a=tx.a and tx.a=t3.a;", + "Plan": [ + "Projection 15609.38 root test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 15609.38 root inner join, equal:[eq(test.t2.a, test.t3.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t2.a, test.t1.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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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;", + "Plan": [ + "Projection 15609.38 root test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 15609.38 root inner join, equal:[eq(test.t2.a, test.t1.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", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t3.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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;", + "Plan": [ + "Projection 15609.38 root test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 15609.38 root inner join, equal:[eq(test.t2.a, test.t1.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", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t3.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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;", + "Plan": [ + "HashJoin 19511.72 root inner join, equal:[eq(test.t2.a, test.t3.a)]", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 15609.38 root inner join, equal:[eq(test.t2.a, test.t4.a)]", + " ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + " │ └─IndexFullScan 9990.00 cop[tikv] table:t4, index:a(a) keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + } + ] + }, + { + "Name": "TestJoinOrderHint", + "Cases": [ + { + "SQL": "select /*+ straight_join() hash_join(t) */ * from t, t1, t2 where t.a = t1.a and t1.b=t2.b;", + "Plan": [ + "HashJoin 15593.77 root inner join, equal:[eq(test.t1.b, test.t2.b)]", + "├─TableReader(Build) 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", + "└─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.a, test.t1.a)]", + " ├─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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ straight_join() merge_join(t1) */ * from t, t1, t2 where t.a = t1.a and t1.b=t2.b;", + "Plan": [ + "HashJoin 15593.77 root inner join, equal:[eq(test.t1.b, test.t2.b)]", + "├─TableReader(Build) 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", + "└─MergeJoin(Probe) 12475.01 root inner join, left key:test.t.a, right key:test.t1.a", + " ├─Projection(Build) 9980.01 root test.t1.a, test.t1.b", + " │ └─IndexLookUp 9980.01 root ", + " │ ├─IndexFullScan(Build) 9990.00 cop[tikv] table:t1, index:a(a) keep order:true, stats:pseudo", + " │ └─Selection(Probe) 9980.01 cop[tikv] not(isnull(test.t1.b))", + " │ └─TableRowIDScan 9990.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─Projection(Probe) 9990.00 root test.t.a, test.t.b", + " └─IndexLookUp 9990.00 root ", + " ├─IndexFullScan(Build) 9990.00 cop[tikv] table:t, index:a(a) keep order:true, stats:pseudo", + " └─TableRowIDScan(Probe) 9990.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ straight_join() INL_JOIN(t1) */ * from t, t1, t2 where t.a = t1.a and t1.b=t2.b;", + "Plan": [ + "HashJoin 15593.77 root inner join, equal:[eq(test.t1.b, test.t2.b)]", + "├─TableReader(Build) 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", + "└─IndexJoin(Probe) 12475.01 root inner join, inner:IndexLookUp, outer key:test.t.a, inner key:test.t1.a, equal cond:eq(test.t.a, test.t1.a)", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + " └─IndexLookUp(Probe) 12475.01 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.t.a)], keep order:false, stats:pseudo", + " └─Selection(Probe) 12475.01 cop[tikv] not(isnull(test.t1.b))", + " └─TableRowIDScan 12487.50 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2) hash_join(t2) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", + "Plan": [ + "Projection 155937656.25 root test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 155937656.25 root CARTESIAN inner join", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t1.a, test.t.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t1.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", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2) hash_join(t1) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", + "Plan": [ + "Projection 155937656.25 root test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 155937656.25 root CARTESIAN inner join", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t1.a, test.t.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t1.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", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2) INL_JOIN(t1) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", + "Plan": [ + "Projection 155937656.25 root test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 155937656.25 root CARTESIAN inner join", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t1.a, test.t.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t1.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", + " └─TableReader(Probe) 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" + ], + "Warning": [ + "Warning 1815 Optimizer Hint /*+ INL_JOIN(t1) */ or /*+ TIDB_INLJ(t1) */ is inapplicable", + "Warning 1815 Optimizer Hint /*+ INL_JOIN(t1) */ or /*+ TIDB_INLJ(t1) */ is inapplicable" + ] + }, + { + "SQL": "select /*+ leading(t2) merge_join(t2) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", + "Plan": [ + "Projection 155937656.25 root test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 155937656.25 root CARTESIAN inner join", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t1.a, test.t.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + " └─MergeJoin(Probe) 12475.01 root inner join, left key:test.t2.b, right key:test.t1.b", + " ├─Sort(Build) 9980.01 root test.t1.b", + " │ └─TableReader 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", + " └─Sort(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t1) merge_join(t2) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", + "Plan": [ + "Projection 155937656.25 root test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 155937656.25 root CARTESIAN inner join", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─MergeJoin(Probe) 15593.77 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) 12475.01 root test.t1.b", + " └─HashJoin 12475.01 root inner join, equal:[eq(test.t1.a, test.t.a)]", + " ├─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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2) INL_JOIN(t1) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", + "Plan": [ + "Projection 155937656.25 root test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 155937656.25 root CARTESIAN inner join", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t1.a, test.t.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t1.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", + " └─TableReader(Probe) 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" + ], + "Warning": [ + "Warning 1815 Optimizer Hint /*+ INL_JOIN(t1) */ or /*+ TIDB_INLJ(t1) */ is inapplicable", + "Warning 1815 Optimizer Hint /*+ INL_JOIN(t1) */ or /*+ TIDB_INLJ(t1) */ is inapplicable" + ] + } + ] + }, + { + "Name": "TestJoinOrderHint4StaticPartitionTable", + "Cases": [ + { + "SQL": "select /*+ straight_join() */ * from t, t1, t2 where t.a = t1.a and t1.b=t2.b;", + "Plan": [ + "HashJoin 46828.12 root inner join, equal:[eq(test.t1.b, test.t2.b)]", + "├─HashJoin(Build) 37462.50 root inner join, equal:[eq(test.t.a, test.t1.a)]", + "│ ├─PartitionUnion(Build) 29970.00 root ", + "│ │ ├─TableReader 9990.00 root data:Selection", + "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo", + "│ │ ├─TableReader 9990.00 root data:Selection", + "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p1 keep order:false, stats:pseudo", + "│ │ └─TableReader 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo", + "│ └─PartitionUnion(Probe) 39920.04 root ", + "│ ├─TableReader 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, partition:p0 keep order:false, stats:pseudo", + "│ ├─TableReader 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, partition:p1 keep order:false, stats:pseudo", + "│ ├─TableReader 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, partition:p2 keep order:false, stats:pseudo", + "│ └─TableReader 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, partition:p3 keep order:false, stats:pseudo", + "└─PartitionUnion(Probe) 49950.00 root ", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p0 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p1 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p2 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p3 keep order:false, stats:pseudo", + " └─TableReader 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p4 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ straight_join() */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b and t2.b=t3.b;", + "Plan": [ + "HashJoin 35121.09 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + "├─PartitionUnion(Build) 29970.00 root ", + "│ ├─TableReader 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p0 keep order:false, stats:pseudo", + "│ ├─TableReader 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p1 keep order:false, stats:pseudo", + "│ └─TableReader 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p2 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 46828.12 root inner join, equal:[eq(test.t1.b, test.t2.b)]", + " ├─HashJoin(Build) 37462.50 root inner join, equal:[eq(test.t.a, test.t1.a)]", + " │ ├─PartitionUnion(Build) 29970.00 root ", + " │ │ ├─TableReader 9990.00 root data:Selection", + " │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo", + " │ │ ├─TableReader 9990.00 root data:Selection", + " │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p1 keep order:false, stats:pseudo", + " │ │ └─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo", + " │ └─PartitionUnion(Probe) 39920.04 root ", + " │ ├─TableReader 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, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 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, partition:p1 keep order:false, stats:pseudo", + " │ ├─TableReader 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, partition:p2 keep order:false, stats:pseudo", + " │ └─TableReader 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, partition:p3 keep order:false, stats:pseudo", + " └─PartitionUnion(Probe) 49950.00 root ", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p0 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p1 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p2 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p3 keep order:false, stats:pseudo", + " └─TableReader 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p4 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t1) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b and t2.b=t3.b;", + "Plan": [ + "HashJoin 35121.09 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + "├─PartitionUnion(Build) 29970.00 root ", + "│ ├─TableReader 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p0 keep order:false, stats:pseudo", + "│ ├─TableReader 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p1 keep order:false, stats:pseudo", + "│ └─TableReader 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p2 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 46828.12 root inner join, equal:[eq(test.t1.b, test.t2.b)]", + " ├─HashJoin(Build) 37462.50 root inner join, equal:[eq(test.t.a, test.t1.a)]", + " │ ├─PartitionUnion(Build) 29970.00 root ", + " │ │ ├─TableReader 9990.00 root data:Selection", + " │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo", + " │ │ ├─TableReader 9990.00 root data:Selection", + " │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p1 keep order:false, stats:pseudo", + " │ │ └─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo", + " │ └─PartitionUnion(Probe) 39920.04 root ", + " │ ├─TableReader 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, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 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, partition:p1 keep order:false, stats:pseudo", + " │ ├─TableReader 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, partition:p2 keep order:false, stats:pseudo", + " │ └─TableReader 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, partition:p3 keep order:false, stats:pseudo", + " └─PartitionUnion(Probe) 49950.00 root ", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p0 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p1 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p2 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p3 keep order:false, stats:pseudo", + " └─TableReader 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p4 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t1, t3) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b and t2.b=t3.b;", + "Plan": [ + "HashJoin 35121.09 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + "├─PartitionUnion(Build) 29970.00 root ", + "│ ├─TableReader 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p0 keep order:false, stats:pseudo", + "│ ├─TableReader 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p1 keep order:false, stats:pseudo", + "│ └─TableReader 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p2 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 46828.12 root inner join, equal:[eq(test.t1.b, test.t2.b)]", + " ├─HashJoin(Build) 37462.50 root inner join, equal:[eq(test.t.a, test.t1.a)]", + " │ ├─PartitionUnion(Build) 29970.00 root ", + " │ │ ├─TableReader 9990.00 root data:Selection", + " │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo", + " │ │ ├─TableReader 9990.00 root data:Selection", + " │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p1 keep order:false, stats:pseudo", + " │ │ └─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo", + " │ └─PartitionUnion(Probe) 39920.04 root ", + " │ ├─TableReader 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, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 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, partition:p1 keep order:false, stats:pseudo", + " │ ├─TableReader 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, partition:p2 keep order:false, stats:pseudo", + " │ └─TableReader 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, partition:p3 keep order:false, stats:pseudo", + " └─PartitionUnion(Probe) 49950.00 root ", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p0 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p1 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p2 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p3 keep order:false, stats:pseudo", + " └─TableReader 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p4 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "Projection 43857.47 root test.t4.a, test.t4.b, test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 43857.47 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─PartitionUnion(Build) 29970.00 root ", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p1 keep order:false, stats:pseudo", + " │ └─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 58476.62 root inner join, equal:[eq(test.t.b, test.t2.b)]", + " ├─PartitionUnion(Build) 49950.00 root ", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p1 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p2 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p3 keep order:false, stats:pseudo", + " │ └─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 46781.30 root left outer join, equal:[eq(test.t.a, test.t1.a)]", + " ├─HashJoin(Build) 37425.04 root inner join, equal:[eq(test.t.a, test.t4.a)]", + " │ ├─PartitionUnion(Build) 29940.03 root ", + " │ │ ├─TableReader 9980.01 root data:Selection", + " │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo", + " │ │ ├─TableReader 9980.01 root data:Selection", + " │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p1 keep order:false, stats:pseudo", + " │ │ └─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo", + " │ └─PartitionUnion(Probe) 39960.00 root ", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p1 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p2 keep order:false, stats:pseudo", + " │ └─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p3 keep order:false, stats:pseudo", + " └─PartitionUnion(Probe) 39960.00 root ", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p0 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p1 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p2 keep order:false, stats:pseudo", + " └─TableReader 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p3 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t2, t3) */ * 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;", + "Plan": [ + "Projection 43857.47 root test.t4.a, test.t4.b, test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 43857.47 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─PartitionUnion(Build) 29970.00 root ", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p1 keep order:false, stats:pseudo", + " │ └─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 58476.62 root inner join, equal:[eq(test.t.b, test.t2.b)]", + " ├─PartitionUnion(Build) 49950.00 root ", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p1 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p2 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p3 keep order:false, stats:pseudo", + " │ └─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 46781.30 root left outer join, equal:[eq(test.t.a, test.t1.a)]", + " ├─HashJoin(Build) 37425.04 root inner join, equal:[eq(test.t.a, test.t4.a)]", + " │ ├─PartitionUnion(Build) 29940.03 root ", + " │ │ ├─TableReader 9980.01 root data:Selection", + " │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo", + " │ │ ├─TableReader 9980.01 root data:Selection", + " │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p1 keep order:false, stats:pseudo", + " │ │ └─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo", + " │ └─PartitionUnion(Probe) 39960.00 root ", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p1 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p2 keep order:false, stats:pseudo", + " │ └─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p3 keep order:false, stats:pseudo", + " └─PartitionUnion(Probe) 39960.00 root ", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p0 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p1 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p2 keep order:false, stats:pseudo", + " └─TableReader 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p3 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "Projection 43857.47 root test.t4.a, test.t4.b, test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 43857.47 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─PartitionUnion(Build) 29970.00 root ", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p1 keep order:false, stats:pseudo", + " │ └─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 58476.62 root inner join, equal:[eq(test.t.b, test.t2.b)]", + " ├─PartitionUnion(Build) 49950.00 root ", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p1 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p2 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p3 keep order:false, stats:pseudo", + " │ └─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 46781.30 root left outer join, equal:[eq(test.t.a, test.t1.a)]", + " ├─HashJoin(Build) 37425.04 root inner join, equal:[eq(test.t.a, test.t4.a)]", + " │ ├─PartitionUnion(Build) 29940.03 root ", + " │ │ ├─TableReader 9980.01 root data:Selection", + " │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo", + " │ │ ├─TableReader 9980.01 root data:Selection", + " │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p1 keep order:false, stats:pseudo", + " │ │ └─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo", + " │ └─PartitionUnion(Probe) 39960.00 root ", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p1 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p2 keep order:false, stats:pseudo", + " │ └─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p3 keep order:false, stats:pseudo", + " └─PartitionUnion(Probe) 39960.00 root ", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p0 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p1 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p2 keep order:false, stats:pseudo", + " └─TableReader 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p3 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "Projection 43857.47 root test.t4.a, test.t4.b, test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 43857.47 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─PartitionUnion(Build) 29970.00 root ", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p1 keep order:false, stats:pseudo", + " │ └─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 58476.62 root inner join, equal:[eq(test.t.b, test.t2.b)]", + " ├─PartitionUnion(Build) 49950.00 root ", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p1 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p2 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p3 keep order:false, stats:pseudo", + " │ └─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 46781.30 root left outer join, equal:[eq(test.t.a, test.t1.a)]", + " ├─HashJoin(Build) 37425.04 root inner join, equal:[eq(test.t.a, test.t4.a)]", + " │ ├─PartitionUnion(Build) 29940.03 root ", + " │ │ ├─TableReader 9980.01 root data:Selection", + " │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo", + " │ │ ├─TableReader 9980.01 root data:Selection", + " │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p1 keep order:false, stats:pseudo", + " │ │ └─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo", + " │ └─PartitionUnion(Probe) 39960.00 root ", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p1 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p2 keep order:false, stats:pseudo", + " │ └─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p3 keep order:false, stats:pseudo", + " └─PartitionUnion(Probe) 39960.00 root ", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p0 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p1 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p2 keep order:false, stats:pseudo", + " └─TableReader 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p3 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "Projection 43857.47 root test.t4.a, test.t4.b, test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 43857.47 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─PartitionUnion(Build) 29970.00 root ", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p1 keep order:false, stats:pseudo", + " │ └─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 58476.62 root inner join, equal:[eq(test.t.b, test.t2.b)]", + " ├─PartitionUnion(Build) 49950.00 root ", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p1 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p2 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p3 keep order:false, stats:pseudo", + " │ └─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 46781.30 root left outer join, equal:[eq(test.t.a, test.t1.a)]", + " ├─HashJoin(Build) 37425.04 root inner join, equal:[eq(test.t.a, test.t4.a)]", + " │ ├─PartitionUnion(Build) 29940.03 root ", + " │ │ ├─TableReader 9980.01 root data:Selection", + " │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo", + " │ │ ├─TableReader 9980.01 root data:Selection", + " │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p1 keep order:false, stats:pseudo", + " │ │ └─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo", + " │ └─PartitionUnion(Probe) 39960.00 root ", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p1 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p2 keep order:false, stats:pseudo", + " │ └─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p3 keep order:false, stats:pseudo", + " └─PartitionUnion(Probe) 39960.00 root ", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p0 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p1 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p2 keep order:false, stats:pseudo", + " └─TableReader 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p3 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t3) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.b=t1.b;", + "Plan": [ + "HashJoin 50000.00 root left outer join, equal:[eq(test.t2.b, test.t1.b)]", + "├─HashJoin(Build) 39960.00 root left outer join, equal:[eq(test.t1.a, test.t3.a)]", + "│ ├─PartitionUnion(Build) 29970.00 root ", + "│ │ ├─TableReader 9990.00 root data:Selection", + "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p0 keep order:false, stats:pseudo", + "│ │ ├─TableReader 9990.00 root data:Selection", + "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p1 keep order:false, stats:pseudo", + "│ │ └─TableReader 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p2 keep order:false, stats:pseudo", + "│ └─PartitionUnion(Probe) 39960.00 root ", + "│ ├─TableReader 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p0 keep order:false, stats:pseudo", + "│ ├─TableReader 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p1 keep order:false, stats:pseudo", + "│ ├─TableReader 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p2 keep order:false, stats:pseudo", + "│ └─TableReader 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p3 keep order:false, stats:pseudo", + "└─PartitionUnion(Probe) 50000.00 root ", + " ├─TableReader 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p0 keep order:false, stats:pseudo", + " ├─TableReader 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p1 keep order:false, stats:pseudo", + " ├─TableReader 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p2 keep order:false, stats:pseudo", + " ├─TableReader 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p3 keep order:false, stats:pseudo", + " └─TableReader 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p4 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t2, t1, t3) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.b=t1.b;", + "Plan": [ + "HashJoin 50000.00 root left outer join, equal:[eq(test.t2.b, test.t1.b)]", + "├─HashJoin(Build) 39960.00 root left outer join, equal:[eq(test.t1.a, test.t3.a)]", + "│ ├─PartitionUnion(Build) 29970.00 root ", + "│ │ ├─TableReader 9990.00 root data:Selection", + "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p0 keep order:false, stats:pseudo", + "│ │ ├─TableReader 9990.00 root data:Selection", + "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p1 keep order:false, stats:pseudo", + "│ │ └─TableReader 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p2 keep order:false, stats:pseudo", + "│ └─PartitionUnion(Probe) 39960.00 root ", + "│ ├─TableReader 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p0 keep order:false, stats:pseudo", + "│ ├─TableReader 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p1 keep order:false, stats:pseudo", + "│ ├─TableReader 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p2 keep order:false, stats:pseudo", + "│ └─TableReader 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p3 keep order:false, stats:pseudo", + "└─PartitionUnion(Probe) 50000.00 root ", + " ├─TableReader 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p0 keep order:false, stats:pseudo", + " ├─TableReader 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p1 keep order:false, stats:pseudo", + " ├─TableReader 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p2 keep order:false, stats:pseudo", + " ├─TableReader 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p3 keep order:false, stats:pseudo", + " └─TableReader 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p4 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "HashJoin 58476.62 root left outer join, equal:[eq(test.t2.b, test.t1.b)]", + "├─PartitionUnion(Build) 50000.00 root ", + "│ ├─TableReader 10000.00 root data:TableFullScan", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p0 keep order:false, stats:pseudo", + "│ ├─TableReader 10000.00 root data:TableFullScan", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p1 keep order:false, stats:pseudo", + "│ ├─TableReader 10000.00 root data:TableFullScan", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p2 keep order:false, stats:pseudo", + "│ ├─TableReader 10000.00 root data:TableFullScan", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p3 keep order:false, stats:pseudo", + "│ └─TableReader 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p4 keep order:false, stats:pseudo", + "└─Projection(Probe) 46781.30 root test.t1.a, test.t1.b, test.t3.a, test.t3.b, test.t4.a, test.t4.b", + " └─HashJoin 46781.30 root inner join, equal:[eq(test.t3.b, test.t4.b)]", + " ├─HashJoin(Build) 37425.04 root inner join, equal:[eq(test.t3.a, test.t1.a)]", + " │ ├─PartitionUnion(Build) 29940.03 root ", + " │ │ ├─TableReader 9980.01 root data:Selection", + " │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p0 keep order:false, stats:pseudo", + " │ │ ├─TableReader 9980.01 root data:Selection", + " │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p1 keep order:false, stats:pseudo", + " │ │ └─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p2 keep order:false, stats:pseudo", + " │ └─PartitionUnion(Probe) 39920.04 root ", + " │ ├─TableReader 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, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 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, partition:p1 keep order:false, stats:pseudo", + " │ ├─TableReader 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, partition:p2 keep order:false, stats:pseudo", + " │ └─TableReader 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, partition:p3 keep order:false, stats:pseudo", + " └─PartitionUnion(Probe) 39960.00 root ", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p0 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p1 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p2 keep order:false, stats:pseudo", + " └─TableReader 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p3 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "HashJoin 58476.62 root left outer join, equal:[eq(test.t2.b, test.t1.b)]", + "├─PartitionUnion(Build) 50000.00 root ", + "│ ├─TableReader 10000.00 root data:TableFullScan", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p0 keep order:false, stats:pseudo", + "│ ├─TableReader 10000.00 root data:TableFullScan", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p1 keep order:false, stats:pseudo", + "│ ├─TableReader 10000.00 root data:TableFullScan", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p2 keep order:false, stats:pseudo", + "│ ├─TableReader 10000.00 root data:TableFullScan", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p3 keep order:false, stats:pseudo", + "│ └─TableReader 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p4 keep order:false, stats:pseudo", + "└─Projection(Probe) 46781.30 root test.t1.a, test.t1.b, test.t3.a, test.t3.b, test.t4.a, test.t4.b", + " └─HashJoin 46781.30 root inner join, equal:[eq(test.t3.b, test.t4.b)]", + " ├─HashJoin(Build) 37425.04 root inner join, equal:[eq(test.t3.a, test.t1.a)]", + " │ ├─PartitionUnion(Build) 29940.03 root ", + " │ │ ├─TableReader 9980.01 root data:Selection", + " │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p0 keep order:false, stats:pseudo", + " │ │ ├─TableReader 9980.01 root data:Selection", + " │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p1 keep order:false, stats:pseudo", + " │ │ └─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p2 keep order:false, stats:pseudo", + " │ └─PartitionUnion(Probe) 39920.04 root ", + " │ ├─TableReader 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, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 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, partition:p1 keep order:false, stats:pseudo", + " │ ├─TableReader 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, partition:p2 keep order:false, stats:pseudo", + " │ └─TableReader 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, partition:p3 keep order:false, stats:pseudo", + " └─PartitionUnion(Probe) 39960.00 root ", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p0 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p1 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p2 keep order:false, stats:pseudo", + " └─TableReader 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p3 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "Projection 54821.83 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b, test.t4.a, test.t4.b, test.t5.a, test.t5.b, test.t6.a, test.t6.b", + "└─HashJoin 54821.83 root left outer join, equal:[eq(test.t2.b, test.t1.b)]", + " ├─Projection(Build) 46781.30 root test.t1.a, test.t1.b, test.t3.a, test.t3.b, test.t4.a, test.t4.b", + " │ └─HashJoin 46781.30 root inner join, equal:[eq(test.t3.b, test.t4.b)]", + " │ ├─HashJoin(Build) 37425.04 root inner join, equal:[eq(test.t3.a, test.t1.a)]", + " │ │ ├─PartitionUnion(Build) 29940.03 root ", + " │ │ │ ├─TableReader 9980.01 root data:Selection", + " │ │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p0 keep order:false, stats:pseudo", + " │ │ │ ├─TableReader 9980.01 root data:Selection", + " │ │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p1 keep order:false, stats:pseudo", + " │ │ │ └─TableReader 9980.01 root data:Selection", + " │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p2 keep order:false, stats:pseudo", + " │ │ └─PartitionUnion(Probe) 39920.04 root ", + " │ │ ├─TableReader 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, partition:p0 keep order:false, stats:pseudo", + " │ │ ├─TableReader 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, partition:p1 keep order:false, stats:pseudo", + " │ │ ├─TableReader 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, partition:p2 keep order:false, stats:pseudo", + " │ │ └─TableReader 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, partition:p3 keep order:false, stats:pseudo", + " │ └─PartitionUnion(Probe) 39960.00 root ", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p1 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p2 keep order:false, stats:pseudo", + " │ └─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 46828.12 root inner join, equal:[eq(test.t5.a, test.t2.a)]", + " ├─HashJoin(Build) 37462.50 root inner join, equal:[eq(test.t6.b, test.t5.b)]", + " │ ├─PartitionUnion(Build) 29970.00 root ", + " │ │ ├─TableReader 9990.00 root data:Selection", + " │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t6.b))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t6, partition:p0 keep order:false, stats:pseudo", + " │ │ ├─TableReader 9990.00 root data:Selection", + " │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t6.b))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t6, partition:p1 keep order:false, stats:pseudo", + " │ │ └─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t6.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t6, partition:p2 keep order:false, stats:pseudo", + " │ └─PartitionUnion(Probe) 49900.05 root ", + " │ ├─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t5, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t5, partition:p1 keep order:false, stats:pseudo", + " │ ├─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t5, partition:p2 keep order:false, stats:pseudo", + " │ ├─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t5, partition:p3 keep order:false, stats:pseudo", + " │ └─TableReader 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t5, partition:p4 keep order:false, stats:pseudo", + " └─PartitionUnion(Probe) 49950.00 root ", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p0 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p1 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p2 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p3 keep order:false, stats:pseudo", + " └─TableReader 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p4 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "Projection 54821.83 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b, test.t4.a, test.t4.b, test.t5.a, test.t5.b, test.t6.a, test.t6.b", + "└─HashJoin 54821.83 root left outer join, equal:[eq(test.t2.b, test.t1.b)]", + " ├─Projection(Build) 46781.30 root test.t1.a, test.t1.b, test.t3.a, test.t3.b, test.t4.a, test.t4.b", + " │ └─HashJoin 46781.30 root inner join, equal:[eq(test.t3.b, test.t4.b)]", + " │ ├─HashJoin(Build) 37425.04 root inner join, equal:[eq(test.t3.a, test.t1.a)]", + " │ │ ├─PartitionUnion(Build) 29940.03 root ", + " │ │ │ ├─TableReader 9980.01 root data:Selection", + " │ │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p0 keep order:false, stats:pseudo", + " │ │ │ ├─TableReader 9980.01 root data:Selection", + " │ │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p1 keep order:false, stats:pseudo", + " │ │ │ └─TableReader 9980.01 root data:Selection", + " │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p2 keep order:false, stats:pseudo", + " │ │ └─PartitionUnion(Probe) 39920.04 root ", + " │ │ ├─TableReader 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, partition:p0 keep order:false, stats:pseudo", + " │ │ ├─TableReader 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, partition:p1 keep order:false, stats:pseudo", + " │ │ ├─TableReader 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, partition:p2 keep order:false, stats:pseudo", + " │ │ └─TableReader 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, partition:p3 keep order:false, stats:pseudo", + " │ └─PartitionUnion(Probe) 39960.00 root ", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p1 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p2 keep order:false, stats:pseudo", + " │ └─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 46828.12 root inner join, equal:[eq(test.t5.a, test.t2.a)]", + " ├─HashJoin(Build) 37462.50 root inner join, equal:[eq(test.t6.b, test.t5.b)]", + " │ ├─PartitionUnion(Build) 29970.00 root ", + " │ │ ├─TableReader 9990.00 root data:Selection", + " │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t6.b))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t6, partition:p0 keep order:false, stats:pseudo", + " │ │ ├─TableReader 9990.00 root data:Selection", + " │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t6.b))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t6, partition:p1 keep order:false, stats:pseudo", + " │ │ └─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t6.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t6, partition:p2 keep order:false, stats:pseudo", + " │ └─PartitionUnion(Probe) 49900.05 root ", + " │ ├─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t5, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t5, partition:p1 keep order:false, stats:pseudo", + " │ ├─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t5, partition:p2 keep order:false, stats:pseudo", + " │ ├─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t5, partition:p3 keep order:false, stats:pseudo", + " │ └─TableReader 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t5, partition:p4 keep order:false, stats:pseudo", + " └─PartitionUnion(Probe) 49950.00 root ", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p0 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p1 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p2 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p3 keep order:false, stats:pseudo", + " └─TableReader 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p4 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "Projection 54821.83 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b, test.t4.a, test.t4.b, test.t5.a, test.t5.b, test.t6.a, test.t6.b", + "└─HashJoin 54821.83 root left outer join, equal:[eq(test.t2.b, test.t1.b)]", + " ├─Projection(Build) 46781.30 root test.t1.a, test.t1.b, test.t3.a, test.t3.b, test.t4.a, test.t4.b", + " │ └─HashJoin 46781.30 root inner join, equal:[eq(test.t3.b, test.t4.b)]", + " │ ├─HashJoin(Build) 37425.04 root inner join, equal:[eq(test.t3.a, test.t1.a)]", + " │ │ ├─PartitionUnion(Build) 29940.03 root ", + " │ │ │ ├─TableReader 9980.01 root data:Selection", + " │ │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p0 keep order:false, stats:pseudo", + " │ │ │ ├─TableReader 9980.01 root data:Selection", + " │ │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p1 keep order:false, stats:pseudo", + " │ │ │ └─TableReader 9980.01 root data:Selection", + " │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p2 keep order:false, stats:pseudo", + " │ │ └─PartitionUnion(Probe) 39920.04 root ", + " │ │ ├─TableReader 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, partition:p0 keep order:false, stats:pseudo", + " │ │ ├─TableReader 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, partition:p1 keep order:false, stats:pseudo", + " │ │ ├─TableReader 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, partition:p2 keep order:false, stats:pseudo", + " │ │ └─TableReader 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, partition:p3 keep order:false, stats:pseudo", + " │ └─PartitionUnion(Probe) 39960.00 root ", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p1 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p2 keep order:false, stats:pseudo", + " │ └─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 46828.12 root inner join, equal:[eq(test.t5.a, test.t2.a)]", + " ├─HashJoin(Build) 37462.50 root inner join, equal:[eq(test.t6.b, test.t5.b)]", + " │ ├─PartitionUnion(Build) 29970.00 root ", + " │ │ ├─TableReader 9990.00 root data:Selection", + " │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t6.b))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t6, partition:p0 keep order:false, stats:pseudo", + " │ │ ├─TableReader 9990.00 root data:Selection", + " │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t6.b))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t6, partition:p1 keep order:false, stats:pseudo", + " │ │ └─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t6.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t6, partition:p2 keep order:false, stats:pseudo", + " │ └─PartitionUnion(Probe) 49900.05 root ", + " │ ├─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t5, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t5, partition:p1 keep order:false, stats:pseudo", + " │ ├─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t5, partition:p2 keep order:false, stats:pseudo", + " │ ├─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t5, partition:p3 keep order:false, stats:pseudo", + " │ └─TableReader 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t5, partition:p4 keep order:false, stats:pseudo", + " └─PartitionUnion(Probe) 49950.00 root ", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p0 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p1 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p2 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p3 keep order:false, stats:pseudo", + " └─TableReader 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p4 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "Projection 46828.12 root test.t4.a, test.t4.b, test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 46828.12 root right outer join, equal:[eq(test.t.a, test.t1.a)]", + " ├─Projection(Build) 37462.50 root test.t4.a, test.t4.b, test.t.a, test.t.b", + " │ └─HashJoin 37462.50 root inner join, equal:[eq(test.t.a, test.t4.a)]", + " │ ├─PartitionUnion(Build) 29970.00 root ", + " │ │ ├─TableReader 9990.00 root data:Selection", + " │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo", + " │ │ ├─TableReader 9990.00 root data:Selection", + " │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p1 keep order:false, stats:pseudo", + " │ │ └─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo", + " │ └─PartitionUnion(Probe) 39960.00 root ", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p1 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p2 keep order:false, stats:pseudo", + " │ └─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 39960.00 root inner join, equal:[eq(test.t2.b, test.t1.b)]", + " ├─HashJoin(Build) 37462.50 root inner join, equal:[eq(test.t3.b, test.t2.b)]", + " │ ├─PartitionUnion(Build) 29970.00 root ", + " │ │ ├─TableReader 9990.00 root data:Selection", + " │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p0 keep order:false, stats:pseudo", + " │ │ ├─TableReader 9990.00 root data:Selection", + " │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p1 keep order:false, stats:pseudo", + " │ │ └─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p2 keep order:false, stats:pseudo", + " │ └─PartitionUnion(Probe) 49950.00 root ", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p1 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p2 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p3 keep order:false, stats:pseudo", + " │ └─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p4 keep order:false, stats:pseudo", + " └─PartitionUnion(Probe) 39960.00 root ", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p0 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p1 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p2 keep order:false, stats:pseudo", + " └─TableReader 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p3 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "Projection 46828.12 root test.t4.a, test.t4.b, test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 46828.12 root right outer join, equal:[eq(test.t.a, test.t1.a)]", + " ├─Projection(Build) 37462.50 root test.t4.a, test.t4.b, test.t.a, test.t.b", + " │ └─HashJoin 37462.50 root inner join, equal:[eq(test.t.a, test.t4.a)]", + " │ ├─PartitionUnion(Build) 29970.00 root ", + " │ │ ├─TableReader 9990.00 root data:Selection", + " │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo", + " │ │ ├─TableReader 9990.00 root data:Selection", + " │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p1 keep order:false, stats:pseudo", + " │ │ └─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo", + " │ └─PartitionUnion(Probe) 39960.00 root ", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p1 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p2 keep order:false, stats:pseudo", + " │ └─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 39960.00 root inner join, equal:[eq(test.t2.b, test.t1.b)]", + " ├─HashJoin(Build) 37462.50 root inner join, equal:[eq(test.t3.b, test.t2.b)]", + " │ ├─PartitionUnion(Build) 29970.00 root ", + " │ │ ├─TableReader 9990.00 root data:Selection", + " │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p0 keep order:false, stats:pseudo", + " │ │ ├─TableReader 9990.00 root data:Selection", + " │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p1 keep order:false, stats:pseudo", + " │ │ └─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p2 keep order:false, stats:pseudo", + " │ └─PartitionUnion(Probe) 49950.00 root ", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p1 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p2 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p3 keep order:false, stats:pseudo", + " │ └─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p4 keep order:false, stats:pseudo", + " └─PartitionUnion(Probe) 39960.00 root ", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p0 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p1 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p2 keep order:false, stats:pseudo", + " └─TableReader 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p3 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "Projection 46828.12 root test.t4.a, test.t4.b, test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 46828.12 root right outer join, equal:[eq(test.t.a, test.t1.a)]", + " ├─Projection(Build) 37462.50 root test.t4.a, test.t4.b, test.t.a, test.t.b", + " │ └─HashJoin 37462.50 root inner join, equal:[eq(test.t.a, test.t4.a)]", + " │ ├─PartitionUnion(Build) 29970.00 root ", + " │ │ ├─TableReader 9990.00 root data:Selection", + " │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo", + " │ │ ├─TableReader 9990.00 root data:Selection", + " │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p1 keep order:false, stats:pseudo", + " │ │ └─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo", + " │ └─PartitionUnion(Probe) 39960.00 root ", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p1 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p2 keep order:false, stats:pseudo", + " │ └─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 39960.00 root inner join, equal:[eq(test.t2.b, test.t1.b)]", + " ├─HashJoin(Build) 37462.50 root inner join, equal:[eq(test.t3.b, test.t2.b)]", + " │ ├─PartitionUnion(Build) 29970.00 root ", + " │ │ ├─TableReader 9990.00 root data:Selection", + " │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p0 keep order:false, stats:pseudo", + " │ │ ├─TableReader 9990.00 root data:Selection", + " │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p1 keep order:false, stats:pseudo", + " │ │ └─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p2 keep order:false, stats:pseudo", + " │ └─PartitionUnion(Probe) 49950.00 root ", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p1 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p2 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p3 keep order:false, stats:pseudo", + " │ └─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p4 keep order:false, stats:pseudo", + " └─PartitionUnion(Probe) 39960.00 root ", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p0 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p1 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p2 keep order:false, stats:pseudo", + " └─TableReader 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p3 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t3) */ * from t2 right join (t1 left join t3 on t1.a=t3.a) on t2.b=t1.b;", + "Plan": [ + "HashJoin 50000.00 root right outer join, equal:[eq(test.t2.b, test.t1.b)]", + "├─HashJoin(Build) 40000.00 root left outer join, equal:[eq(test.t1.a, test.t3.a)]", + "│ ├─PartitionUnion(Build) 29970.00 root ", + "│ │ ├─TableReader 9990.00 root data:Selection", + "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p0 keep order:false, stats:pseudo", + "│ │ ├─TableReader 9990.00 root data:Selection", + "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p1 keep order:false, stats:pseudo", + "│ │ └─TableReader 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p2 keep order:false, stats:pseudo", + "│ └─PartitionUnion(Probe) 40000.00 root ", + "│ ├─TableReader 10000.00 root data:TableFullScan", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p0 keep order:false, stats:pseudo", + "│ ├─TableReader 10000.00 root data:TableFullScan", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p1 keep order:false, stats:pseudo", + "│ ├─TableReader 10000.00 root data:TableFullScan", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p2 keep order:false, stats:pseudo", + "│ └─TableReader 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p3 keep order:false, stats:pseudo", + "└─PartitionUnion(Probe) 49950.00 root ", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p0 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p1 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p2 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p3 keep order:false, stats:pseudo", + " └─TableReader 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p4 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t2, t1, t3) */ * from t2 right join (t1 left join t3 on t1.a=t3.a) on t2.b=t1.b;", + "Plan": [ + "HashJoin 50000.00 root right outer join, equal:[eq(test.t2.b, test.t1.b)]", + "├─HashJoin(Build) 40000.00 root left outer join, equal:[eq(test.t1.a, test.t3.a)]", + "│ ├─PartitionUnion(Build) 29970.00 root ", + "│ │ ├─TableReader 9990.00 root data:Selection", + "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p0 keep order:false, stats:pseudo", + "│ │ ├─TableReader 9990.00 root data:Selection", + "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p1 keep order:false, stats:pseudo", + "│ │ └─TableReader 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p2 keep order:false, stats:pseudo", + "│ └─PartitionUnion(Probe) 40000.00 root ", + "│ ├─TableReader 10000.00 root data:TableFullScan", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p0 keep order:false, stats:pseudo", + "│ ├─TableReader 10000.00 root data:TableFullScan", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p1 keep order:false, stats:pseudo", + "│ ├─TableReader 10000.00 root data:TableFullScan", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p2 keep order:false, stats:pseudo", + "│ └─TableReader 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p3 keep order:false, stats:pseudo", + "└─PartitionUnion(Probe) 49950.00 root ", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p0 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p1 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p2 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p3 keep order:false, stats:pseudo", + " └─TableReader 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p4 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "Projection 58476.62 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b, test.t4.a, test.t4.b", + "└─HashJoin 58476.62 root right outer join, equal:[eq(test.t2.b, test.t1.b)]", + " ├─PartitionUnion(Build) 49950.00 root ", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p1 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p2 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p3 keep order:false, stats:pseudo", + " │ └─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 46781.30 root inner join, equal:[eq(test.t3.b, test.t4.b)]", + " ├─HashJoin(Build) 37425.04 root inner join, equal:[eq(test.t3.a, test.t1.a)]", + " │ ├─PartitionUnion(Build) 29940.03 root ", + " │ │ ├─TableReader 9980.01 root data:Selection", + " │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p0 keep order:false, stats:pseudo", + " │ │ ├─TableReader 9980.01 root data:Selection", + " │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p1 keep order:false, stats:pseudo", + " │ │ └─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p2 keep order:false, stats:pseudo", + " │ └─PartitionUnion(Probe) 39960.00 root ", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p1 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p2 keep order:false, stats:pseudo", + " │ └─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p3 keep order:false, stats:pseudo", + " └─PartitionUnion(Probe) 39960.00 root ", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p0 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p1 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p2 keep order:false, stats:pseudo", + " └─TableReader 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p3 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "Projection 58476.62 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b, test.t4.a, test.t4.b", + "└─HashJoin 58476.62 root right outer join, equal:[eq(test.t2.b, test.t1.b)]", + " ├─PartitionUnion(Build) 49950.00 root ", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p1 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p2 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p3 keep order:false, stats:pseudo", + " │ └─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 46781.30 root inner join, equal:[eq(test.t3.b, test.t4.b)]", + " ├─HashJoin(Build) 37425.04 root inner join, equal:[eq(test.t3.a, test.t1.a)]", + " │ ├─PartitionUnion(Build) 29940.03 root ", + " │ │ ├─TableReader 9980.01 root data:Selection", + " │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p0 keep order:false, stats:pseudo", + " │ │ ├─TableReader 9980.01 root data:Selection", + " │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p1 keep order:false, stats:pseudo", + " │ │ └─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p2 keep order:false, stats:pseudo", + " │ └─PartitionUnion(Probe) 39960.00 root ", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p1 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p2 keep order:false, stats:pseudo", + " │ └─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1, partition:p3 keep order:false, stats:pseudo", + " └─PartitionUnion(Probe) 39960.00 root ", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p0 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p1 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p2 keep order:false, stats:pseudo", + " └─TableReader 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p3 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "Projection 54876.71 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b, test.t4.a, test.t4.b, test.t5.a, test.t5.b, test.t6.a, test.t6.b", + "└─HashJoin 54876.71 root inner join, equal:[eq(test.t5.b, test.t6.b)]", + " ├─PartitionUnion(Build) 29970.00 root ", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t6.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t6, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t6.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t6, partition:p1 keep order:false, stats:pseudo", + " │ └─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t6.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t6, partition:p2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 73095.78 root inner join, equal:[eq(test.t2.a, test.t5.a)]", + " ├─PartitionUnion(Build) 49900.05 root ", + " │ ├─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t5, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t5, partition:p1 keep order:false, stats:pseudo", + " │ ├─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t5, partition:p2 keep order:false, stats:pseudo", + " │ ├─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t5, partition:p3 keep order:false, stats:pseudo", + " │ └─TableReader 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t5, partition:p4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 58476.62 root inner join, equal:[eq(test.t1.b, test.t2.b)]", + " ├─PartitionUnion(Build) 49900.05 root ", + " │ ├─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p1 keep order:false, stats:pseudo", + " │ ├─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p2 keep order:false, stats:pseudo", + " │ ├─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p3 keep order:false, stats:pseudo", + " │ └─TableReader 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 46781.30 root inner join, equal:[eq(test.t3.b, test.t4.b)]", + " ├─HashJoin(Build) 37425.04 root inner join, equal:[eq(test.t3.a, test.t1.a)]", + " │ ├─PartitionUnion(Build) 29940.03 root ", + " │ │ ├─TableReader 9980.01 root data:Selection", + " │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p0 keep order:false, stats:pseudo", + " │ │ ├─TableReader 9980.01 root data:Selection", + " │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p1 keep order:false, stats:pseudo", + " │ │ └─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p2 keep order:false, stats:pseudo", + " │ └─PartitionUnion(Probe) 39920.04 root ", + " │ ├─TableReader 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, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 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, partition:p1 keep order:false, stats:pseudo", + " │ ├─TableReader 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, partition:p2 keep order:false, stats:pseudo", + " │ └─TableReader 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, partition:p3 keep order:false, stats:pseudo", + " └─PartitionUnion(Probe) 39960.00 root ", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p0 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p1 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p2 keep order:false, stats:pseudo", + " └─TableReader 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p3 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "Projection 54876.71 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b, test.t4.a, test.t4.b, test.t5.a, test.t5.b, test.t6.a, test.t6.b", + "└─HashJoin 54876.71 root inner join, equal:[eq(test.t5.b, test.t6.b)]", + " ├─PartitionUnion(Build) 29970.00 root ", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t6.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t6, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t6.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t6, partition:p1 keep order:false, stats:pseudo", + " │ └─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t6.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t6, partition:p2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 73095.78 root inner join, equal:[eq(test.t2.a, test.t5.a)]", + " ├─PartitionUnion(Build) 49900.05 root ", + " │ ├─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t5, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t5, partition:p1 keep order:false, stats:pseudo", + " │ ├─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t5, partition:p2 keep order:false, stats:pseudo", + " │ ├─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t5, partition:p3 keep order:false, stats:pseudo", + " │ └─TableReader 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t5, partition:p4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 58476.62 root inner join, equal:[eq(test.t1.b, test.t2.b)]", + " ├─PartitionUnion(Build) 49900.05 root ", + " │ ├─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p1 keep order:false, stats:pseudo", + " │ ├─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p2 keep order:false, stats:pseudo", + " │ ├─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p3 keep order:false, stats:pseudo", + " │ └─TableReader 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 46781.30 root inner join, equal:[eq(test.t3.b, test.t4.b)]", + " ├─HashJoin(Build) 37425.04 root inner join, equal:[eq(test.t3.a, test.t1.a)]", + " │ ├─PartitionUnion(Build) 29940.03 root ", + " │ │ ├─TableReader 9980.01 root data:Selection", + " │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p0 keep order:false, stats:pseudo", + " │ │ ├─TableReader 9980.01 root data:Selection", + " │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p1 keep order:false, stats:pseudo", + " │ │ └─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p2 keep order:false, stats:pseudo", + " │ └─PartitionUnion(Probe) 39920.04 root ", + " │ ├─TableReader 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, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 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, partition:p1 keep order:false, stats:pseudo", + " │ ├─TableReader 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, partition:p2 keep order:false, stats:pseudo", + " │ └─TableReader 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, partition:p3 keep order:false, stats:pseudo", + " └─PartitionUnion(Probe) 39960.00 root ", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p0 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p1 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p2 keep order:false, stats:pseudo", + " └─TableReader 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p3 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "Projection 54876.71 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b, test.t4.a, test.t4.b, test.t5.a, test.t5.b, test.t6.a, test.t6.b", + "└─HashJoin 54876.71 root inner join, equal:[eq(test.t5.b, test.t6.b)]", + " ├─PartitionUnion(Build) 29970.00 root ", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t6.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t6, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t6.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t6, partition:p1 keep order:false, stats:pseudo", + " │ └─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t6.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t6, partition:p2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 73095.78 root inner join, equal:[eq(test.t2.a, test.t5.a)]", + " ├─PartitionUnion(Build) 49900.05 root ", + " │ ├─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t5, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t5, partition:p1 keep order:false, stats:pseudo", + " │ ├─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t5, partition:p2 keep order:false, stats:pseudo", + " │ ├─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t5, partition:p3 keep order:false, stats:pseudo", + " │ └─TableReader 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t5, partition:p4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 58476.62 root inner join, equal:[eq(test.t1.b, test.t2.b)]", + " ├─PartitionUnion(Build) 49900.05 root ", + " │ ├─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p1 keep order:false, stats:pseudo", + " │ ├─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p2 keep order:false, stats:pseudo", + " │ ├─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p3 keep order:false, stats:pseudo", + " │ └─TableReader 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 46781.30 root inner join, equal:[eq(test.t3.b, test.t4.b)]", + " ├─HashJoin(Build) 37425.04 root inner join, equal:[eq(test.t3.a, test.t1.a)]", + " │ ├─PartitionUnion(Build) 29940.03 root ", + " │ │ ├─TableReader 9980.01 root data:Selection", + " │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p0 keep order:false, stats:pseudo", + " │ │ ├─TableReader 9980.01 root data:Selection", + " │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p1 keep order:false, stats:pseudo", + " │ │ └─TableReader 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p2 keep order:false, stats:pseudo", + " │ └─PartitionUnion(Probe) 39920.04 root ", + " │ ├─TableReader 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, partition:p0 keep order:false, stats:pseudo", + " │ ├─TableReader 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, partition:p1 keep order:false, stats:pseudo", + " │ ├─TableReader 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, partition:p2 keep order:false, stats:pseudo", + " │ └─TableReader 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, partition:p3 keep order:false, stats:pseudo", + " └─PartitionUnion(Probe) 39960.00 root ", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p0 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p1 keep order:false, stats:pseudo", + " ├─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p2 keep order:false, stats:pseudo", + " └─TableReader 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t4, partition:p3 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + } + ] + }, + { + "Name": "TestJoinOrderHint4DynamicPartitionTable", + "Cases": [ + { + "SQL": "select /*+ straight_join() */ * from t, t1, t2 where t.a = t1.a and t1.b=t2.b;", + "Plan": [ + "HashJoin 15593.77 root inner join, equal:[eq(test.t1.b, test.t2.b)]", + "├─TableReader(Build) 9990.00 root partition:all data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.a, test.t1.a)]", + " ├─TableReader(Build) 9980.01 root partition:all 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", + " └─TableReader(Probe) 9990.00 root partition:all data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ straight_join() */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b and t2.b=t3.b;", + "Plan": [ + "HashJoin 19492.21 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + "├─TableReader(Build) 9990.00 root partition:all data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t1.b, test.t2.b)]", + " ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.a, test.t1.a)]", + " ├─TableReader(Build) 9980.01 root partition:all 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", + " └─TableReader(Probe) 9990.00 root partition:all data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t1) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b and t2.b=t3.b;", + "Plan": [ + "Projection 19492.21 root test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 19492.21 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t1.b, test.t2.b)]", + " ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t1.a, test.t.a)]", + " ├─TableReader(Build) 9980.01 root partition:all 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", + " └─TableReader(Probe) 9990.00 root partition:all data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t1, t3) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b and t2.b=t3.b;", + "Plan": [ + "Projection 155781718.59 root test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 155781718.59 root inner join, equal:[eq(test.t1.b, test.t2.b) eq(test.t3.b, test.t2.b)]", + " ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 124625374.88 root inner join, equal:[eq(test.t1.a, test.t.a)]", + " ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + " └─HashJoin(Probe) 99700299.90 root CARTESIAN inner join", + " ├─TableReader(Build) 9980.01 root partition:all 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", + " └─TableReader(Probe) 9990.00 root partition:all data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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;", + "Plan": [ + "Projection 24365.26 root test.t4.a, test.t4.b, test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 24365.26 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 19492.21 root inner join, equal:[eq(test.t.b, test.t2.b)]", + " ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15593.77 root left outer join, equal:[eq(test.t.a, test.t1.a)]", + " ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.a, test.t4.a)]", + " ├─TableReader(Build) 9980.01 root partition:all data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root partition:all data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t2, t3) */ * 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;", + "Plan": [ + "Projection 24365.26 root test.t4.a, test.t4.b, test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 24365.26 root left outer join, equal:[eq(test.t.a, test.t1.a)]", + " ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 19492.21 root inner join, equal:[eq(test.t.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t2.b, test.t.b)]", + " ├─TableReader(Build) 9980.01 root partition:all data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root partition:all data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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;", + "Plan": [ + "Projection 24365.26 root test.t4.a, test.t4.b, test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 24365.26 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 19492.21 root inner join, equal:[eq(test.t.b, test.t2.b)]", + " ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15593.77 root left outer join, equal:[eq(test.t.a, test.t1.a)]", + " ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.a, test.t4.a)]", + " ├─TableReader(Build) 9980.01 root partition:all data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root partition:all data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "Projection 24365.26 root test.t4.a, test.t4.b, test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 24365.26 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 19492.21 root inner join, equal:[eq(test.t.b, test.t2.b)]", + " ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15593.77 root left outer join, equal:[eq(test.t.a, test.t1.a)]", + " ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.a, test.t4.a)]", + " ├─TableReader(Build) 9980.01 root partition:all data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root partition:all data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "Projection 24365.26 root test.t4.a, test.t4.b, test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 24365.26 root left outer join, equal:[eq(test.t.a, test.t1.a)]", + " ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 19492.21 root inner join, equal:[eq(test.t.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t2.b, test.t.b)]", + " ├─TableReader(Build) 9980.01 root partition:all data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t3.b, test.t2.b)]", + " ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root partition:all data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.b=t1.b;", + "Plan": [ + "HashJoin 15609.38 root left outer join, equal:[eq(test.t2.b, test.t1.b)]", + "├─TableReader(Build) 10000.00 root partition:all data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t3.a)]", + " ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root partition:all 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 /*+ leading(t2, t1, t3) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.b=t1.b;", + "Plan": [ + "HashJoin 15609.38 root left outer join, equal:[eq(test.t2.b, test.t1.b)]", + "├─TableReader(Build) 10000.00 root partition:all data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t3.a)]", + " ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root partition:all 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": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "HashJoin 19492.21 root left outer join, equal:[eq(test.t2.b, test.t1.b)]", + "├─TableReader(Build) 10000.00 root partition:all data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t3.b, test.t4.b)]", + " ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t1.a, test.t3.a)]", + " ├─TableReader(Build) 9980.01 root partition:all data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9980.01 root partition:all 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" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "HashJoin 19492.21 root left outer join, equal:[eq(test.t2.b, test.t1.b)]", + "├─TableReader(Build) 10000.00 root partition:all data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─Projection(Probe) 15593.77 root test.t1.a, test.t1.b, test.t3.a, test.t3.b, test.t4.a, test.t4.b", + " └─HashJoin 15593.77 root inner join, equal:[eq(test.t3.a, test.t1.a)]", + " ├─TableReader(Build) 9980.01 root partition:all 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", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t3.b, test.t4.b)]", + " ├─TableReader(Build) 9980.01 root partition:all data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root partition:all data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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;", + "Plan": [ + "Projection 30426.12 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b, test.t4.a, test.t4.b, test.t5.a, test.t5.b, test.t6.a, test.t6.b", + "└─HashJoin 30426.12 root left outer join, equal:[eq(test.t2.b, test.t1.b)]", + " ├─Projection(Build) 15593.77 root test.t1.a, test.t1.b, test.t3.a, test.t3.b, test.t4.a, test.t4.b", + " │ └─HashJoin 15593.77 root inner join, equal:[eq(test.t3.a, test.t1.a)]", + " │ ├─TableReader(Build) 9980.01 root partition:all 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", + " │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t3.b, test.t4.b)]", + " │ ├─TableReader(Build) 9980.01 root partition:all data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " │ └─TableReader(Probe) 9990.00 root partition:all data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t5.b, test.t6.b)]", + " ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t6.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t5.a, test.t2.a)]", + " ├─TableReader(Build) 9980.01 root partition:all data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t5 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root partition:all data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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;", + "Plan": [ + "Projection 30426.12 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b, test.t4.a, test.t4.b, test.t5.a, test.t5.b, test.t6.a, test.t6.b", + "└─HashJoin 30426.12 root left outer join, equal:[eq(test.t2.b, test.t1.b)]", + " ├─HashJoin(Build) 15593.77 root inner join, equal:[eq(test.t3.b, test.t4.b)]", + " │ ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t1.a, test.t3.a)]", + " │ ├─TableReader(Build) 9980.01 root partition:all data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " │ └─TableReader(Probe) 9980.01 root partition:all 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", + " └─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t5.b, test.t6.b)]", + " ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t6.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t5.a, test.t2.a)]", + " ├─TableReader(Build) 9980.01 root partition:all data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t5 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root partition:all data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "Projection 30426.12 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b, test.t4.a, test.t4.b, test.t5.a, test.t5.b, test.t6.a, test.t6.b", + "└─HashJoin 30426.12 root left outer join, equal:[eq(test.t2.b, test.t1.b)]", + " ├─HashJoin(Build) 15593.77 root inner join, equal:[eq(test.t3.b, test.t4.b)]", + " │ ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t1.a, test.t3.a)]", + " │ ├─TableReader(Build) 9980.01 root partition:all data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " │ └─TableReader(Probe) 9980.01 root partition:all 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", + " └─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t5.b, test.t6.b)]", + " ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t6.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t5.a, test.t2.a)]", + " ├─TableReader(Build) 9980.01 root partition:all data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t5 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root partition:all data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "HashJoin 24389.65 root right outer join, equal:[eq(test.t.a, test.t1.a)]", + "├─HashJoin(Build) 12487.50 root inner join, equal:[eq(test.t4.a, test.t.a)]", + "│ ├─TableReader(Build) 9990.00 root partition:all data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9990.00 root partition:all data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 15609.38 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.b, test.t2.b)]", + " ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root partition:all 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 /*+ 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;", + "Plan": [ + "Projection 24389.65 root test.t4.a, test.t4.b, test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 24389.65 root right outer join, equal:[eq(test.t.a, test.t1.a)]", + " ├─HashJoin(Build) 12487.50 root inner join, equal:[eq(test.t4.a, test.t.a)]", + " │ ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + " │ └─TableReader(Probe) 9990.00 root partition:all data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15609.38 root inner join, equal:[eq(test.t2.b, test.t1.b)]", + " ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root partition:all data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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;", + "Plan": [ + "HashJoin 24389.65 root right outer join, equal:[eq(test.t.a, test.t1.a)]", + "├─HashJoin(Build) 12487.50 root inner join, equal:[eq(test.t4.a, test.t.a)]", + "│ ├─TableReader(Build) 9990.00 root partition:all data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9990.00 root partition:all data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 15609.38 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.b, test.t2.b)]", + " ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root partition:all 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": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t3) */ * from t2 right join (t1 left join t3 on t1.a=t3.a) on t2.b=t1.b;", + "Plan": [ + "HashJoin 15593.77 root right outer join, equal:[eq(test.t2.b, test.t1.b)]", + "├─TableReader(Build) 9990.00 root partition:all data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t3.a)]", + " ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 10000.00 root partition:all data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2, t1, t3) */ * from t2 right join (t1 left join t3 on t1.a=t3.a) on t2.b=t1.b;", + "Plan": [ + "HashJoin 15593.77 root left outer join, equal:[eq(test.t1.a, test.t3.a)]", + "├─TableReader(Build) 9990.00 root partition:all data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root right outer join, equal:[eq(test.t2.b, test.t1.b)]", + " ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 10000.00 root partition:all data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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;", + "Plan": [ + "Projection 19492.21 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b, test.t4.a, test.t4.b", + "└─HashJoin 19492.21 root inner join, equal:[eq(test.t3.b, test.t4.b)]", + " ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15593.77 root right outer join, equal:[eq(test.t2.b, test.t1.b)]", + " ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t3.a, test.t1.a)]", + " ├─TableReader(Build) 9980.01 root partition:all data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root partition:all data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "Projection 19492.21 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b, test.t4.a, test.t4.b", + "└─HashJoin 19492.21 root right outer join, equal:[eq(test.t2.b, test.t1.b)]", + " ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t3.a, test.t1.a)]", + " ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t3.b, test.t4.b)]", + " ├─TableReader(Build) 9980.01 root partition:all data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root partition:all data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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;", + "Plan": [ + "Projection 30456.57 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b, test.t4.a, test.t4.b, test.t5.a, test.t5.b, test.t6.a, test.t6.b", + "└─HashJoin 30456.57 root inner join, equal:[eq(test.t5.b, test.t6.b)]", + " ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t6.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 24365.26 root inner join, equal:[eq(test.t2.a, test.t5.a)]", + " ├─TableReader(Build) 9980.01 root partition:all data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t5 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 19492.21 root inner join, equal:[eq(test.t1.b, test.t2.b)]", + " ├─TableReader(Build) 9980.01 root partition:all data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t3.a, test.t1.a)]", + " ├─TableReader(Build) 9980.01 root partition:all 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", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t3.b, test.t4.b)]", + " ├─TableReader(Build) 9980.01 root partition:all data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root partition:all data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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;", + "Plan": [ + "Projection 30456.57 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b, test.t4.a, test.t4.b, test.t5.a, test.t5.b, test.t6.a, test.t6.b", + "└─HashJoin 30456.57 root inner join, equal:[eq(test.t5.b, test.t6.b)]", + " ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t6.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 24365.26 root inner join, equal:[eq(test.t3.b, test.t4.b)]", + " ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 19492.21 root inner join, equal:[eq(test.t2.a, test.t5.a)]", + " ├─TableReader(Build) 9980.01 root partition:all data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t5 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t1.a, test.t3.a)]", + " ├─TableReader(Build) 9980.01 root partition:all data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t1.b)]", + " ├─TableReader(Build) 9980.01 root partition:all 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", + " └─TableReader(Probe) 9980.01 root partition:all data:Selection", + " └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "Projection 243165526.37 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b, test.t4.a, test.t4.b, test.t5.a, test.t5.b, test.t6.a, test.t6.b", + "└─HashJoin 243165526.37 root inner join, equal:[eq(test.t3.a, test.t1.a) eq(test.t2.b, test.t1.b)]", + " ├─TableReader(Build) 9980.01 root partition:all 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", + " └─HashJoin(Probe) 194532421.09 root inner join, equal:[eq(test.t5.a, test.t2.a)]", + " ├─TableReader(Build) 9980.01 root partition:all data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 155625936.88 root inner join, equal:[eq(test.t5.b, test.t6.b)]", + " ├─TableReader(Build) 9990.00 root partition:all data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t6.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 124500749.50 root CARTESIAN inner join", + " ├─TableReader(Build) 9980.01 root partition:all data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t5 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t3.b, test.t4.b)]", + " ├─TableReader(Build) 9980.01 root partition:all data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root partition:all data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warning": null + } + ] + }, + { + "Name": "TestJoinOrderHint4DifferentJoinType", + "Cases": [ + { + "SQL": "select /*+ straight_join() */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b", + "Plan": [ + "HashJoin 15593.77 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ straight_join() */ * from t1 straight_join t2 straight_join t3", + "Plan": [ + "HashJoin 1000000000000.00 root CARTESIAN 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 + }, + { + "SQL": "select /*+ straight_join() */ * from t1 join t2 on t1.a=t2.a left join t3 on t2.b=t3.b", + "Plan": [ + "HashJoin 15609.38 root left outer join, equal:[eq(test.t2.b, test.t3.b)]", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ straight_join() */ * from t1 join t2 on t1.a=t2.a right join t3 on t2.b=t3.b", + "Plan": [ + "HashJoin 15593.77 root right outer join, equal:[eq(test.t2.b, test.t3.b)]", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ straight_join() */ * from t1 join t2 on t1.a=t2.a cross join t3", + "Plan": [ + "HashJoin 124875000.00 root CARTESIAN inner join", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2) */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b", + "Plan": [ + "Projection 15593.77 root test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 15593.77 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.a, test.t1.a)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3) */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b", + "Plan": [ + "Projection 15593.77 root test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 15593.77 root inner join, equal:[eq(test.t2.a, test.t1.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", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t3.b, test.t2.b)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2, t3) */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b", + "Plan": [ + "Projection 15593.77 root test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 15593.77 root inner join, equal:[eq(test.t2.a, test.t1.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", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3, t2) */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b", + "Plan": [ + "Projection 15593.77 root test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 15593.77 root inner join, equal:[eq(test.t2.a, test.t1.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", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t3.b, test.t2.b)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3, t1) */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b", + "Plan": [ + "Projection 124625374.88 root test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 124625374.88 root inner join, equal:[eq(test.t1.a, test.t2.a) eq(test.t3.b, test.t2.b)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 99800100.00 root CARTESIAN inner join", + " ├─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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2) */ * from t1 straight_join t2 straight_join t3", + "Plan": [ + "HashJoin 1000000000000.00 root CARTESIAN 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": [ + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint" + ] + }, + { + "SQL": "select /*+ leading(t3) */ * from t1 straight_join t2 straight_join t3", + "Plan": [ + "HashJoin 1000000000000.00 root CARTESIAN 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": [ + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint" + ] + }, + { + "SQL": "select /*+ leading(t2, t3) */ * from t1 straight_join t2 straight_join t3", + "Plan": [ + "HashJoin 1000000000000.00 root CARTESIAN 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": [ + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint" + ] + }, + { + "SQL": "select /*+ leading(t3, t2) */ * from t1 straight_join t2 straight_join t3", + "Plan": [ + "HashJoin 1000000000000.00 root CARTESIAN 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": [ + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint" + ] + }, + { + "SQL": "select /*+ leading(t3, t1) */ * from t1 straight_join t2 straight_join t3", + "Plan": [ + "HashJoin 1000000000000.00 root CARTESIAN 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": [ + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint" + ] + }, + { + "SQL": "select /*+ leading(t2) */ * from t1 join t2 on t1.a=t2.a left join t3 on t2.b=t3.b", + "Plan": [ + "Projection 15609.38 root test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 15609.38 root left outer join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t2.a, test.t1.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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3) */ * from t1 join t2 on t1.a=t2.a left join t3 on t2.b=t3.b", + "Plan": [ + "Projection 15609.38 root test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 15609.38 root inner join, equal:[eq(test.t2.a, test.t1.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", + " └─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2, t3) */ * from t1 join t2 on t1.a=t2.a left join t3 on t2.b=t3.b", + "Plan": [ + "Projection 15609.38 root test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 15609.38 root inner join, equal:[eq(test.t2.a, test.t1.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", + " └─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3, t2) */ * from t1 join t2 on t1.a=t2.a left join t3 on t2.b=t3.b", + "Plan": [ + "Projection 15609.38 root test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 15609.38 root inner join, equal:[eq(test.t2.a, test.t1.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", + " └─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3, t1) */ * from t1 join t2 on t1.a=t2.a left join t3 on t2.b=t3.b", + "Plan": [ + "HashJoin 15609.38 root left outer join, equal:[eq(test.t2.b, test.t3.b)]", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t2) */ * from t1 join t2 on t1.a=t2.a right join t3 on t2.b=t3.b", + "Plan": [ + "HashJoin 15593.77 root right outer join, equal:[eq(test.t2.b, test.t3.b)]", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─Projection(Probe) 12475.01 root test.t1.a, test.t1.b, test.t2.a, test.t2.b", + " └─HashJoin 12475.01 root inner join, equal:[eq(test.t2.a, test.t1.a)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3) */ * from t1 join t2 on t1.a=t2.a right join t3 on t2.b=t3.b", + "Plan": [ + "HashJoin 15593.77 root right outer join, equal:[eq(test.t2.b, test.t3.b)]", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─Projection(Probe) 12475.01 root test.t1.a, test.t1.b, test.t2.a, test.t2.b", + " └─HashJoin 12475.01 root inner join, equal:[eq(test.t2.a, test.t1.a)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2, t3) */ * from t1 join t2 on t1.a=t2.a right join t3 on t2.b=t3.b", + "Plan": [ + "HashJoin 15593.77 root right outer join, equal:[eq(test.t2.b, test.t3.b)]", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─Projection(Probe) 12475.01 root test.t1.a, test.t1.b, test.t2.a, test.t2.b", + " └─HashJoin 12475.01 root inner join, equal:[eq(test.t2.a, test.t1.a)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t3, t2) */ * from t1 join t2 on t1.a=t2.a right join t3 on t2.b=t3.b", + "Plan": [ + "HashJoin 15593.77 root right outer join, equal:[eq(test.t2.b, test.t3.b)]", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─Projection(Probe) 12475.01 root test.t1.a, test.t1.b, test.t2.a, test.t2.b", + " └─HashJoin 12475.01 root inner join, equal:[eq(test.t2.a, test.t1.a)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t3, t1) */ * from t1 join t2 on t1.a=t2.a right join t3 on t2.b=t3.b", + "Plan": [ + "HashJoin 15593.77 root right outer join, equal:[eq(test.t2.b, test.t3.b)]", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─Projection(Probe) 12475.01 root test.t1.a, test.t1.b, test.t2.a, test.t2.b", + " └─HashJoin 12475.01 root inner join, equal:[eq(test.t2.a, test.t1.a)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t2) */ * from t1 join t2 on t1.a=t2.a cross join t3", + "Plan": [ + "Projection 124875000.00 root test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 124875000.00 root CARTESIAN inner join", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t2.a, test.t1.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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3) */ * from t1 join t2 on t1.a=t2.a cross join t3", + "Plan": [ + "Projection 124875000.00 root test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 124875000.00 root CARTESIAN inner join", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table has join conditions with other tables" + ] + }, + { + "SQL": "select /*+ leading(t2, t3) */ * from t1 join t2 on t1.a=t2.a cross join t3", + "Plan": [ + "Projection 124875000.00 root test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 124875000.00 root inner join, equal:[eq(test.t2.a, test.t1.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", + " └─HashJoin(Probe) 99900000.00 root CARTESIAN inner join", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─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:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3, t2) */ * from t1 join t2 on t1.a=t2.a cross join t3", + "Plan": [ + "Projection 124875000.00 root test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 124875000.00 root inner join, equal:[eq(test.t2.a, test.t1.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", + " └─HashJoin(Probe) 99900000.00 root CARTESIAN inner join", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─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:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3, t1) */ * from t1 join t2 on t1.a=t2.a cross join t3", + "Plan": [ + "Projection 124875000.00 root test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 124875000.00 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 99900000.00 root CARTESIAN inner join", + " ├─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", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2, t1) */ * from t1 left join t2 on t1.a=t2.a cross join t3", + "Plan": [ + "HashJoin 124875000.00 root CARTESIAN inner join", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─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 leading hint is inapplicable, check if the leading hint table has join conditions with other tables" + ] + }, + { + "SQL": "select /*+ leading(t2, t1) */ * from t1 right join t2 on t1.a=t2.a cross join t3", + "Plan": [ + "HashJoin 124875000.00 root CARTESIAN inner join", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root right outer join, equal:[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", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table has join conditions with other tables" + ] + }, + { + "SQL": "select /*+ leading(t3, t1) */ * from t1 right join t2 on t1.a=t2.a right join t3 on t2.b=t3.b", + "Plan": [ + "HashJoin 15609.38 root right outer join, equal:[eq(test.t2.b, test.t3.b)]", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root right outer join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 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", + " └─TableReader(Probe) 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" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t1) */ * from t1 left join t2 on t1.a=t2.a straight_join t3", + "Plan": [ + "HashJoin 124875000.00 root CARTESIAN inner join", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─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 /*+ leading(t1) */ * from t1 right join t2 on t1.a=t2.a straight_join t3", + "Plan": [ + "HashJoin 124875000.00 root CARTESIAN inner join", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root right outer join, equal:[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", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t1) */ * from t1 left join t2 on t1.a=t2.a cross join t3", + "Plan": [ + "HashJoin 124875000.00 root CARTESIAN inner join", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─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 /*+ leading(t1) */ * from t1 right join t2 on t1.a=t2.a cross join t3", + "Plan": [ + "HashJoin 124875000.00 root CARTESIAN inner join", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root right outer join, equal:[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", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": null + } + ] + }, + { + "Name": "TestJoinOrderHint4TiFlash", + "Cases": [ + { + "SQL": "select /*+ straight_join() */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b", + "Plan": [ + "TableReader 15593.77 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 15593.77 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 15593.77 mpp[tiflash] inner join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t3 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 mpp[tiflash] inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─ExchangeReceiver(Build) 9980.01 mpp[tiflash] ", + " │ └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t1.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2) */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b", + "Plan": [ + "TableReader 15593.77 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 15593.77 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 15593.77 mpp[tiflash] test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + " └─HashJoin 15593.77 mpp[tiflash] inner join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t3 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 mpp[tiflash] inner join, equal:[eq(test.t2.a, test.t1.a)]", + " ├─ExchangeReceiver(Build) 9980.01 mpp[tiflash] ", + " │ └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t1.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3) */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b", + "Plan": [ + "TableReader 15593.77 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 15593.77 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 15593.77 mpp[tiflash] test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + " └─HashJoin 15593.77 mpp[tiflash] inner join, equal:[eq(test.t2.a, test.t1.a)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 mpp[tiflash] inner join, equal:[eq(test.t3.b, test.t2.b)]", + " ├─ExchangeReceiver(Build) 9980.01 mpp[tiflash] ", + " │ └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t3.b))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t3 pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2, t3) */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b", + "Plan": [ + "TableReader 15593.77 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 15593.77 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 15593.77 mpp[tiflash] test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + " └─HashJoin 15593.77 mpp[tiflash] inner join, equal:[eq(test.t2.a, test.t1.a)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 mpp[tiflash] inner join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─ExchangeReceiver(Build) 9980.01 mpp[tiflash] ", + " │ └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t3.b))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t3 pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3, t2) */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b", + "Plan": [ + "TableReader 15593.77 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 15593.77 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 15593.77 mpp[tiflash] test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + " └─HashJoin 15593.77 mpp[tiflash] inner join, equal:[eq(test.t2.a, test.t1.a)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 mpp[tiflash] inner join, equal:[eq(test.t3.b, test.t2.b)]", + " ├─ExchangeReceiver(Build) 9980.01 mpp[tiflash] ", + " │ └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t3.b))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t3 pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3, t1) */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b", + "Plan": [ + "TableReader 124625374.88 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 124625374.88 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 124625374.88 mpp[tiflash] test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + " └─HashJoin 124625374.88 mpp[tiflash] inner join, equal:[eq(test.t1.a, test.t2.a) eq(test.t3.b, test.t2.b)]", + " ├─ExchangeReceiver(Build) 9980.01 mpp[tiflash] ", + " │ └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 99800100.00 mpp[tiflash] CARTESIAN inner join", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t3 pushed down filter:empty, keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t1.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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;", + "Plan": [ + "TableReader 24365.26 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 24365.26 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 24365.26 mpp[tiflash] test.t4.a, test.t4.b, test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + " └─HashJoin 24365.26 mpp[tiflash] inner join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t3 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 19492.21 mpp[tiflash] inner join, equal:[eq(test.t.b, test.t2.b)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15593.77 mpp[tiflash] left outer join, equal:[eq(test.t.a, test.t1.a)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t4.a)]", + " ├─ExchangeReceiver(Build) 9980.01 mpp[tiflash] ", + " │ └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t.a)), not(isnull(test.t.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t pushed down filter:empty, keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t4.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t4 pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t2, t3) */ * 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;", + "Plan": [ + "TableReader 24365.26 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 24365.26 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 24365.26 mpp[tiflash] test.t4.a, test.t4.b, test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + " └─HashJoin 24365.26 mpp[tiflash] left outer join, equal:[eq(test.t.a, test.t1.a)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 19492.21 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t4.a)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t4 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15593.77 mpp[tiflash] inner join, equal:[eq(test.t2.b, test.t.b)]", + " ├─ExchangeReceiver(Build) 9980.01 mpp[tiflash] ", + " │ └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t.a)), not(isnull(test.t.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 mpp[tiflash] inner join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t3.b))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t3 pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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;", + "Plan": [ + "TableReader 24365.26 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 24365.26 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 24365.26 mpp[tiflash] test.t4.a, test.t4.b, test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + " └─HashJoin 24365.26 mpp[tiflash] inner join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t3 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 19492.21 mpp[tiflash] inner join, equal:[eq(test.t.b, test.t2.b)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15593.77 mpp[tiflash] left outer join, equal:[eq(test.t.a, test.t1.a)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t4.a)]", + " ├─ExchangeReceiver(Build) 9980.01 mpp[tiflash] ", + " │ └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t.a)), not(isnull(test.t.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t pushed down filter:empty, keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t4.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t4 pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "TableReader 24365.26 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 24365.26 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 24365.26 mpp[tiflash] test.t4.a, test.t4.b, test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + " └─HashJoin 24365.26 mpp[tiflash] inner join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t3 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 19492.21 mpp[tiflash] inner join, equal:[eq(test.t.b, test.t2.b)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15593.77 mpp[tiflash] left outer join, equal:[eq(test.t.a, test.t1.a)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t4.a)]", + " ├─ExchangeReceiver(Build) 9980.01 mpp[tiflash] ", + " │ └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t.a)), not(isnull(test.t.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t pushed down filter:empty, keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t4.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t4 pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "TableReader 24365.26 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 24365.26 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 24365.26 mpp[tiflash] test.t4.a, test.t4.b, test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + " └─HashJoin 24365.26 mpp[tiflash] left outer join, equal:[eq(test.t.a, test.t1.a)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 19492.21 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t4.a)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t4 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15593.77 mpp[tiflash] inner join, equal:[eq(test.t2.b, test.t.b)]", + " ├─ExchangeReceiver(Build) 9980.01 mpp[tiflash] ", + " │ └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t.a)), not(isnull(test.t.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 mpp[tiflash] inner join, equal:[eq(test.t3.b, test.t2.b)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t3 pushed down filter:empty, keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t2.b))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.b=t1.b;", + "Plan": [ + "TableReader 15609.38 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 15609.38 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 15609.38 mpp[tiflash] left outer join, equal:[eq(test.t2.b, test.t1.b)]", + " ├─ExchangeReceiver(Build) 10000.00 mpp[tiflash] ", + " │ └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t2.b, collate: binary]", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 12487.50 mpp[tiflash] ", + " └─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.b, collate: binary]", + " └─HashJoin 12487.50 mpp[tiflash] left outer join, equal:[eq(test.t1.a, test.t3.a)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t3 pushed down filter:empty, keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t1.b))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2, t1, t3) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.b=t1.b;", + "Plan": [ + "TableReader 15609.38 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 15609.38 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 15609.38 mpp[tiflash] left outer join, equal:[eq(test.t2.b, test.t1.b)]", + " ├─ExchangeReceiver(Build) 10000.00 mpp[tiflash] ", + " │ └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t2.b, collate: binary]", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 12487.50 mpp[tiflash] ", + " └─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.b, collate: binary]", + " └─HashJoin 12487.50 mpp[tiflash] left outer join, equal:[eq(test.t1.a, test.t3.a)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t3 pushed down filter:empty, keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t1.b))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "TableReader 19492.21 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 19492.21 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 19492.21 mpp[tiflash] left outer join, equal:[eq(test.t2.b, test.t1.b)]", + " ├─ExchangeReceiver(Build) 10000.00 mpp[tiflash] ", + " │ └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t2.b, collate: binary]", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 15593.77 mpp[tiflash] ", + " └─ExchangeSender 15593.77 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.b, collate: binary]", + " └─HashJoin 15593.77 mpp[tiflash] inner join, equal:[eq(test.t3.b, test.t4.b)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t4 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 mpp[tiflash] inner join, equal:[eq(test.t1.a, test.t3.a)]", + " ├─ExchangeReceiver(Build) 9980.01 mpp[tiflash] ", + " │ └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t1.a)), not(isnull(test.t1.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo", + " └─Selection(Probe) 9980.01 mpp[tiflash] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t3 pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "TableReader 19492.21 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 19492.21 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 19492.21 mpp[tiflash] left outer join, equal:[eq(test.t2.b, test.t1.b)]", + " ├─ExchangeReceiver(Build) 10000.00 mpp[tiflash] ", + " │ └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t2.b, collate: binary]", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 15593.77 mpp[tiflash] ", + " └─ExchangeSender 15593.77 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.b, collate: binary]", + " └─Projection 15593.77 mpp[tiflash] test.t1.a, test.t1.b, test.t3.a, test.t3.b, test.t4.a, test.t4.b", + " └─HashJoin 15593.77 mpp[tiflash] inner join, equal:[eq(test.t3.a, test.t1.a)]", + " ├─ExchangeReceiver(Build) 9980.01 mpp[tiflash] ", + " │ └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t1.a)), not(isnull(test.t1.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 mpp[tiflash] inner join, equal:[eq(test.t3.b, test.t4.b)]", + " ├─ExchangeReceiver(Build) 9980.01 mpp[tiflash] ", + " │ └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t3 pushed down filter:empty, keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t4.b))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t4 pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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;", + "Plan": [ + "TableReader 30426.12 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 30426.12 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 30426.12 mpp[tiflash] test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b, test.t4.a, test.t4.b, test.t5.a, test.t5.b, test.t6.a, test.t6.b", + " └─HashJoin 30426.12 mpp[tiflash] left outer join, equal:[eq(test.t2.b, test.t1.b)]", + " ├─ExchangeReceiver(Build) 15593.77 mpp[tiflash] ", + " │ └─ExchangeSender 15593.77 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t2.b, collate: binary]", + " │ └─HashJoin 15593.77 mpp[tiflash] inner join, equal:[eq(test.t5.b, test.t6.b)]", + " │ ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t6.b))", + " │ │ └─TableFullScan 10000.00 mpp[tiflash] table:t6 pushed down filter:empty, keep order:false, stats:pseudo", + " │ └─HashJoin(Probe) 12475.01 mpp[tiflash] inner join, equal:[eq(test.t5.a, test.t2.a)]", + " │ ├─ExchangeReceiver(Build) 9980.01 mpp[tiflash] ", + " │ │ └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ │ └─TableFullScan 10000.00 mpp[tiflash] table:t5 pushed down filter:empty, keep order:false, stats:pseudo", + " │ └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 15593.77 mpp[tiflash] ", + " └─ExchangeSender 15593.77 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.b, collate: binary]", + " └─Projection 15593.77 mpp[tiflash] test.t1.a, test.t1.b, test.t3.a, test.t3.b, test.t4.a, test.t4.b", + " └─HashJoin 15593.77 mpp[tiflash] inner join, equal:[eq(test.t3.a, test.t1.a)]", + " ├─ExchangeReceiver(Build) 9980.01 mpp[tiflash] ", + " │ └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t1.a)), not(isnull(test.t1.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 mpp[tiflash] inner join, equal:[eq(test.t3.b, test.t4.b)]", + " ├─ExchangeReceiver(Build) 9980.01 mpp[tiflash] ", + " │ └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t3 pushed down filter:empty, keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t4.b))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t4 pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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;", + "Plan": [ + "TableReader 30426.12 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 30426.12 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 30426.12 mpp[tiflash] test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b, test.t4.a, test.t4.b, test.t5.a, test.t5.b, test.t6.a, test.t6.b", + " └─HashJoin 30426.12 mpp[tiflash] left outer join, equal:[eq(test.t2.b, test.t1.b)]", + " ├─ExchangeReceiver(Build) 15593.77 mpp[tiflash] ", + " │ └─ExchangeSender 15593.77 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t2.b, collate: binary]", + " │ └─HashJoin 15593.77 mpp[tiflash] inner join, equal:[eq(test.t5.b, test.t6.b)]", + " │ ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t6.b))", + " │ │ └─TableFullScan 10000.00 mpp[tiflash] table:t6 pushed down filter:empty, keep order:false, stats:pseudo", + " │ └─HashJoin(Probe) 12475.01 mpp[tiflash] inner join, equal:[eq(test.t5.a, test.t2.a)]", + " │ ├─ExchangeReceiver(Build) 9980.01 mpp[tiflash] ", + " │ │ └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ │ └─TableFullScan 10000.00 mpp[tiflash] table:t5 pushed down filter:empty, keep order:false, stats:pseudo", + " │ └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 15593.77 mpp[tiflash] ", + " └─ExchangeSender 15593.77 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.b, collate: binary]", + " └─HashJoin 15593.77 mpp[tiflash] inner join, equal:[eq(test.t3.b, test.t4.b)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t4 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 mpp[tiflash] inner join, equal:[eq(test.t1.a, test.t3.a)]", + " ├─ExchangeReceiver(Build) 9980.01 mpp[tiflash] ", + " │ └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t1.a)), not(isnull(test.t1.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo", + " └─Selection(Probe) 9980.01 mpp[tiflash] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t3 pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "TableReader 30426.12 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 30426.12 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 30426.12 mpp[tiflash] test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b, test.t4.a, test.t4.b, test.t5.a, test.t5.b, test.t6.a, test.t6.b", + " └─HashJoin 30426.12 mpp[tiflash] left outer join, equal:[eq(test.t2.b, test.t1.b)]", + " ├─ExchangeReceiver(Build) 15593.77 mpp[tiflash] ", + " │ └─ExchangeSender 15593.77 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t2.b, collate: binary]", + " │ └─HashJoin 15593.77 mpp[tiflash] inner join, equal:[eq(test.t5.b, test.t6.b)]", + " │ ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t6.b))", + " │ │ └─TableFullScan 10000.00 mpp[tiflash] table:t6 pushed down filter:empty, keep order:false, stats:pseudo", + " │ └─HashJoin(Probe) 12475.01 mpp[tiflash] inner join, equal:[eq(test.t5.a, test.t2.a)]", + " │ ├─ExchangeReceiver(Build) 9980.01 mpp[tiflash] ", + " │ │ └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ │ └─TableFullScan 10000.00 mpp[tiflash] table:t5 pushed down filter:empty, keep order:false, stats:pseudo", + " │ └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 15593.77 mpp[tiflash] ", + " └─ExchangeSender 15593.77 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.b, collate: binary]", + " └─HashJoin 15593.77 mpp[tiflash] inner join, equal:[eq(test.t3.b, test.t4.b)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t4 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 mpp[tiflash] inner join, equal:[eq(test.t1.a, test.t3.a)]", + " ├─ExchangeReceiver(Build) 9980.01 mpp[tiflash] ", + " │ └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t1.a)), not(isnull(test.t1.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo", + " └─Selection(Probe) 9980.01 mpp[tiflash] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t3 pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "TableReader 24389.65 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 24389.65 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 24389.65 mpp[tiflash] right outer join, equal:[eq(test.t.a, test.t1.a)]", + " ├─ExchangeReceiver(Build) 12487.50 mpp[tiflash] ", + " │ └─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t.a, collate: binary]", + " │ └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t4.a, test.t.a)]", + " │ ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t4.a))", + " │ │ └─TableFullScan 10000.00 mpp[tiflash] table:t4 pushed down filter:empty, keep order:false, stats:pseudo", + " │ └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t pushed down filter:empty, keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 15609.38 mpp[tiflash] ", + " └─ExchangeSender 15609.38 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.a, collate: binary]", + " └─HashJoin 15609.38 mpp[tiflash] inner join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t3 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 mpp[tiflash] inner join, equal:[eq(test.t1.b, test.t2.b)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t1.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t2.b))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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;", + "Plan": [ + "TableReader 24389.65 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 24389.65 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 24389.65 mpp[tiflash] test.t4.a, test.t4.b, test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + " └─HashJoin 24389.65 mpp[tiflash] right outer join, equal:[eq(test.t.a, test.t1.a)]", + " ├─ExchangeReceiver(Build) 12487.50 mpp[tiflash] ", + " │ └─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t.a, collate: binary]", + " │ └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t4.a, test.t.a)]", + " │ ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t4.a))", + " │ │ └─TableFullScan 10000.00 mpp[tiflash] table:t4 pushed down filter:empty, keep order:false, stats:pseudo", + " │ └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t pushed down filter:empty, keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 15609.38 mpp[tiflash] ", + " └─ExchangeSender 15609.38 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.a, collate: binary]", + " └─HashJoin 15609.38 mpp[tiflash] inner join, equal:[eq(test.t2.b, test.t1.b)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t1.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 mpp[tiflash] inner join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t3.b))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t3 pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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;", + "Plan": [ + "TableReader 24389.65 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 24389.65 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 24389.65 mpp[tiflash] right outer join, equal:[eq(test.t.a, test.t1.a)]", + " ├─ExchangeReceiver(Build) 12487.50 mpp[tiflash] ", + " │ └─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t.a, collate: binary]", + " │ └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t4.a, test.t.a)]", + " │ ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t4.a))", + " │ │ └─TableFullScan 10000.00 mpp[tiflash] table:t4 pushed down filter:empty, keep order:false, stats:pseudo", + " │ └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t pushed down filter:empty, keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 15609.38 mpp[tiflash] ", + " └─ExchangeSender 15609.38 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.a, collate: binary]", + " └─HashJoin 15609.38 mpp[tiflash] inner join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t3 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 mpp[tiflash] inner join, equal:[eq(test.t1.b, test.t2.b)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t1.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t2.b))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t3) */ * from t2 right join (t1 left join t3 on t1.a=t3.a) on t2.b=t1.b;", + "Plan": [ + "TableReader 15593.77 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 15593.77 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 15593.77 mpp[tiflash] right outer join, equal:[eq(test.t2.b, test.t1.b)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 mpp[tiflash] left outer join, equal:[eq(test.t1.a, test.t3.a)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t3 pushed down filter:empty, keep order:false, stats:pseudo", + " └─TableFullScan(Probe) 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2, t1, t3) */ * from t2 right join (t1 left join t3 on t1.a=t3.a) on t2.b=t1.b;", + "Plan": [ + "TableReader 15593.77 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 15593.77 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 15593.77 mpp[tiflash] left outer join, equal:[eq(test.t1.a, test.t3.a)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t3 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 mpp[tiflash] right outer join, equal:[eq(test.t2.b, test.t1.b)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false, stats:pseudo", + " └─TableFullScan(Probe) 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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;", + "Plan": [ + "TableReader 19492.21 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 19492.21 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 19492.21 mpp[tiflash] test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b, test.t4.a, test.t4.b", + " └─HashJoin 19492.21 mpp[tiflash] inner join, equal:[eq(test.t3.b, test.t4.b)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t4 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15593.77 mpp[tiflash] right outer join, equal:[eq(test.t2.b, test.t1.b)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 mpp[tiflash] inner join, equal:[eq(test.t3.a, test.t1.a)]", + " ├─ExchangeReceiver(Build) 9980.01 mpp[tiflash] ", + " │ └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t3 pushed down filter:empty, keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t1.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "TableReader 19492.21 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 19492.21 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 19492.21 mpp[tiflash] test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b, test.t4.a, test.t4.b", + " └─HashJoin 19492.21 mpp[tiflash] right outer join, equal:[eq(test.t2.b, test.t1.b)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15593.77 mpp[tiflash] inner join, equal:[eq(test.t3.a, test.t1.a)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 mpp[tiflash] inner join, equal:[eq(test.t3.b, test.t4.b)]", + " ├─ExchangeReceiver(Build) 9980.01 mpp[tiflash] ", + " │ └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t3 pushed down filter:empty, keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t4.b))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t4 pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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;", + "Plan": [ + "TableReader 30456.57 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 30456.57 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 30456.57 mpp[tiflash] test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b, test.t4.a, test.t4.b, test.t5.a, test.t5.b, test.t6.a, test.t6.b", + " └─HashJoin 30456.57 mpp[tiflash] inner join, equal:[eq(test.t5.b, test.t6.b)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t6.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t6 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 24365.26 mpp[tiflash] inner join, equal:[eq(test.t2.a, test.t5.a)]", + " ├─ExchangeReceiver(Build) 9980.01 mpp[tiflash] ", + " │ └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t5 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 19492.21 mpp[tiflash] inner join, equal:[eq(test.t1.b, test.t2.b)]", + " ├─ExchangeReceiver(Build) 9980.01 mpp[tiflash] ", + " │ └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15593.77 mpp[tiflash] inner join, equal:[eq(test.t3.a, test.t1.a)]", + " ├─ExchangeReceiver(Build) 9980.01 mpp[tiflash] ", + " │ └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t1.a)), not(isnull(test.t1.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 mpp[tiflash] inner join, equal:[eq(test.t3.b, test.t4.b)]", + " ├─ExchangeReceiver(Build) 9980.01 mpp[tiflash] ", + " │ └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t3 pushed down filter:empty, keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t4.b))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t4 pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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;", + "Plan": [ + "TableReader 30456.57 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 30456.57 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 30456.57 mpp[tiflash] test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b, test.t4.a, test.t4.b, test.t5.a, test.t5.b, test.t6.a, test.t6.b", + " └─HashJoin 30456.57 mpp[tiflash] inner join, equal:[eq(test.t5.b, test.t6.b)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t6.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t6 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 24365.26 mpp[tiflash] inner join, equal:[eq(test.t3.b, test.t4.b)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t4 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 19492.21 mpp[tiflash] inner join, equal:[eq(test.t2.a, test.t5.a)]", + " ├─ExchangeReceiver(Build) 9980.01 mpp[tiflash] ", + " │ └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t5 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15593.77 mpp[tiflash] inner join, equal:[eq(test.t1.a, test.t3.a)]", + " ├─ExchangeReceiver(Build) 9980.01 mpp[tiflash] ", + " │ └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t3 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 mpp[tiflash] inner join, equal:[eq(test.t2.b, test.t1.b)]", + " ├─ExchangeReceiver(Build) 9980.01 mpp[tiflash] ", + " │ └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false, stats:pseudo", + " └─Selection(Probe) 9980.01 mpp[tiflash] not(isnull(test.t1.a)), not(isnull(test.t1.b))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "TableReader 243165526.37 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 243165526.37 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 243165526.37 mpp[tiflash] test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b, test.t4.a, test.t4.b, test.t5.a, test.t5.b, test.t6.a, test.t6.b", + " └─HashJoin 243165526.37 mpp[tiflash] inner join, equal:[eq(test.t3.a, test.t1.a) eq(test.t2.b, test.t1.b)]", + " ├─ExchangeReceiver(Build) 9980.01 mpp[tiflash] ", + " │ └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t1.a)), not(isnull(test.t1.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 194532421.09 mpp[tiflash] inner join, equal:[eq(test.t5.a, test.t2.a)]", + " ├─ExchangeReceiver(Build) 9980.01 mpp[tiflash] ", + " │ └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 155625936.88 mpp[tiflash] inner join, equal:[eq(test.t5.b, test.t6.b)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t6.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t6 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 124500749.50 mpp[tiflash] CARTESIAN inner join", + " ├─ExchangeReceiver(Build) 9980.01 mpp[tiflash] ", + " │ └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t5 pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 mpp[tiflash] inner join, equal:[eq(test.t3.b, test.t4.b)]", + " ├─ExchangeReceiver(Build) 9980.01 mpp[tiflash] ", + " │ └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t3 pushed down filter:empty, keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t4.b))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t4 pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Warning": null + } + ] + }, + { + "Name": "TestJoinOrderHint4Subquery", + "Cases": [ + { + "SQL": "select /*+ straight_join() */ * from t1 join t2 on t1.a=t2.a where t1.a in (select t3.a from t3 where t1.b = t3.b)", + "Plan": [ + "HashJoin 9980.01 root semi join, equal:[eq(test.t1.b, test.t3.b) eq(test.t1.a, test.t3.a)]", + "├─TableReader(Build) 3.00 root data:Selection", + "│ └─Selection 3.00 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + "│ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "└─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ straight_join() */ * from t1 join t2 on t1.a=t2.a where t1.a not in (select t3.a from t3 where t1.b = t3.b)", + "Plan": [ + "HashJoin 9990.00 root anti semi join, equal:[eq(test.t1.b, test.t3.b)], other cond:eq(test.t1.a, test.t3.a)", + "├─TableReader(Build) 3.00 root data:TableFullScan", + "│ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ straight_join() */ * from t1 join t2 on t1.a=t2.a where exists (select t3.a from t3 where t1.b = t3.b)", + "Plan": [ + "HashJoin 9980.01 root semi join, equal:[eq(test.t1.b, test.t3.b)]", + "├─TableReader(Build) 3.00 root data:Selection", + "│ └─Selection 3.00 cop[tikv] not(isnull(test.t3.b))", + "│ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "└─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 9990.00 root anti semi join, equal:[eq(test.t1.b, test.t3.b)]", + "├─TableReader(Build) 3.00 root data:TableFullScan", + "│ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ straight_join() */ t1.a, (select min(t1.a) from t2 where t2.a > t1.a) from t1;", + "Plan": [ + "Projection 1.00 root test.t1.a, Column#11->Column#13", + "└─Apply 1.00 root CARTESIAN left outer join", + " ├─HashAgg(Build) 1.00 root funcs:min(test.t1.a)->Column#7, funcs:firstrow(test.t1.a)->test.t1.a", + " │ └─IndexReader 10000.00 root index:IndexFullScan", + " │ └─IndexFullScan 10000.00 cop[tikv] table:t1, index:a(a) keep order:false, stats:pseudo", + " └─MaxOneRow(Probe) 1.00 root ", + " └─Projection 2.00 root Column#7->Column#11", + " └─IndexReader 2.00 root index:Selection", + " └─Selection 2.00 cop[tikv] gt(test.t2.a, test.t1.a)", + " └─IndexFullScan 2.50 cop[tikv] table:t2, index:a(a) keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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;", + "Plan": [ + "Projection 1.00 root test.t1.a, Column#14->Column#16", + "└─Apply 1.00 root CARTESIAN left outer join", + " ├─StreamAgg(Build) 1.00 root funcs:min(test.t1.a)->Column#10, funcs:firstrow(test.t1.a)->test.t1.a", + " │ └─IndexHashJoin 3.75 root inner join, inner:IndexReader, outer key:test.t3.a, inner key:test.t1.a, equal cond:eq(test.t3.a, test.t1.a)", + " │ ├─IndexReader(Build) 3.00 root index:IndexFullScan", + " │ │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:false", + " │ └─IndexReader(Probe) 3.75 root index:Selection", + " │ └─Selection 3.75 cop[tikv] not(isnull(test.t1.a))", + " │ └─IndexRangeScan 3.75 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, test.t3.a)], keep order:false, stats:pseudo", + " └─MaxOneRow(Probe) 1.00 root ", + " └─Projection 2.00 root Column#10->Column#14", + " └─IndexReader 2.00 root index:Selection", + " └─Selection 2.00 cop[tikv] gt(test.t2.a, test.t1.a)", + " └─IndexFullScan 2.50 cop[tikv] table:t2, index:a(a) keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ straight_join() */ * from t1 join t2 on t1.a=t2.a where t1.a in (select t3.a from t3)", + "Plan": [ + "HashJoin 4.69 root inner join, equal:[eq(test.t1.a, test.t3.a)]", + "├─StreamAgg(Build) 3.00 root group by:test.t3.a, funcs:firstrow(test.t3.a)->test.t3.a", + "│ └─IndexReader 3.00 root index:StreamAgg", + "│ └─StreamAgg 3.00 cop[tikv] group by:test.t3.a, ", + "│ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:true", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ straight_join() */ * from t1 join t2 on t1.a=t2.a where t1.a not in (select t3.a from t3)", + "Plan": [ + "HashJoin 9990.00 root Null-aware anti semi join, equal:[eq(test.t1.a, test.t3.a)]", + "├─IndexReader(Build) 3.00 root index:IndexFullScan", + "│ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:false", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ straight_join() */ * from t1 join t2 on t1.a=t2.a where exists (select t3.a from t3)", + "Plan": [ + "HashJoin 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ straight_join() */ * from t1 join t2 on t1.a=t2.a where not exists (select t3.a from t3)", + "Plan": [ + "TableDual 0.00 root rows:0" + ], + "Warning": null + }, + { + "SQL": "select /*+ straight_join() */ * from t1 join (select * from t4) t2 on t1.a=t2.a join t3 on t2.b=t3.b;", + "Plan": [ + "HashJoin 4.69 root inner join, equal:[eq(test.t4.b, test.t3.b)]", + "├─TableReader(Build) 3.00 root data:Selection", + "│ └─Selection 3.00 cop[tikv] not(isnull(test.t3.b))", + "│ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "└─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t1.a, test.t4.a)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t4.a)), not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ straight_join() */ * from (select * from t4) t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b;", + "Plan": [ + "HashJoin 4.69 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + "├─TableReader(Build) 3.00 root data:Selection", + "│ └─Selection 3.00 cop[tikv] not(isnull(test.t3.b))", + "│ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "└─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t4.a, test.t2.a)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ straight_join() */ * from t1 join t2 on t1.a=t2.a join (select * from t4) t3 on t2.b=t3.b;", + "Plan": [ + "HashJoin 15593.77 root inner join, equal:[eq(test.t2.b, test.t4.b)]", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ straight_join() */ t1.a, (select min(t2.a) from t2) from t1;", + "Plan": [ + "Projection 10000.00 root test.t1.a, ->Column#11", + "└─IndexReader 10000.00 root index:IndexFullScan", + " └─IndexFullScan 10000.00 cop[tikv] table:t1, index:a(a) keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ straight_join() */ t1.a, (select min(t2.a) from t2) from t1 join t3 on t1.a = t3.a;", + "Plan": [ + "Projection 3.75 root test.t1.a, ->Column#14", + "└─IndexHashJoin 3.75 root inner join, inner:IndexReader, outer key:test.t3.a, inner key:test.t1.a, equal cond:eq(test.t3.a, test.t1.a)", + " ├─IndexReader(Build) 3.00 root index:IndexFullScan", + " │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:false", + " └─IndexReader(Probe) 3.75 root index:Selection", + " └─Selection 3.75 cop[tikv] not(isnull(test.t1.a))", + " └─IndexRangeScan 3.75 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, test.t3.a)], keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 12475.01 root semi join, equal:[eq(test.t1.b, test.t3.b) eq(test.t1.a, test.t3.a)]", + "├─TableReader(Build) 3.00 root data:Selection", + "│ └─Selection 3.00 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + "│ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "└─Projection(Probe) 15593.77 root test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t4.a, test.t4.b", + " └─HashJoin 15593.77 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t4.b, test.t1.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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 9980.01 root semi join, equal:[eq(test.t1.b, test.t3.b) eq(test.t1.a, test.t3.a)]", + "├─TableReader(Build) 3.00 root data:Selection", + "│ └─Selection 3.00 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + "│ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "└─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint" + ] + }, + { + "SQL": "select /*+ leading(t2, 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)", + "Plan": [ + "HashJoin 9980.01 root semi join, equal:[eq(test.t1.b, test.t3.b) eq(test.t1.a, test.t3.a)]", + "├─TableReader(Build) 3.00 root data:Selection", + "│ └─Selection 3.00 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + "│ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "└─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 9980.01 root semi join, equal:[eq(test.t1.b, test.t3.b) eq(test.t1.a, test.t3.a)]", + "├─TableReader(Build) 3.00 root data:Selection", + "│ └─Selection 3.00 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + "│ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "└─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 9980.01 root semi join, equal:[eq(test.t1.b, test.t3.b) eq(test.t1.a, test.t3.a)]", + "├─TableReader(Build) 3.00 root data:Selection", + "│ └─Selection 3.00 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + "│ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "└─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 9980.01 root semi join, equal:[eq(test.t1.b, test.t3.b) eq(test.t1.a, test.t3.a)]", + "├─TableReader(Build) 3.00 root data:Selection", + "│ └─Selection 3.00 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + "│ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "└─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 12475.01 root anti semi join, equal:[eq(test.t1.b, test.t3.b)], other cond:eq(test.t1.a, test.t3.a)", + "├─TableReader(Build) 3.00 root data:TableFullScan", + "│ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "└─Projection(Probe) 15593.77 root test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t4.a, test.t4.b", + " └─HashJoin 15593.77 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t4.b, test.t1.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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 9990.00 root anti semi join, equal:[eq(test.t1.b, test.t3.b)], other cond:eq(test.t1.a, test.t3.a)", + "├─TableReader(Build) 3.00 root data:TableFullScan", + "│ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 9990.00 root anti semi join, equal:[eq(test.t1.b, test.t3.b)], other cond:eq(test.t1.a, test.t3.a)", + "├─TableReader(Build) 3.00 root data:TableFullScan", + "│ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 9990.00 root anti semi join, equal:[eq(test.t1.b, test.t3.b)], other cond:eq(test.t1.a, test.t3.a)", + "├─TableReader(Build) 3.00 root data:TableFullScan", + "│ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 9990.00 root anti semi join, equal:[eq(test.t1.b, test.t3.b)], other cond:eq(test.t1.a, test.t3.a)", + "├─TableReader(Build) 3.00 root data:TableFullScan", + "│ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 9990.00 root anti semi join, equal:[eq(test.t1.b, test.t3.b)], other cond:eq(test.t1.a, test.t3.a)", + "├─TableReader(Build) 3.00 root data:TableFullScan", + "│ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 12475.01 root semi join, equal:[eq(test.t1.b, test.t3.b)]", + "├─TableReader(Build) 3.00 root data:Selection", + "│ └─Selection 3.00 cop[tikv] not(isnull(test.t3.b))", + "│ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "└─Projection(Probe) 15593.77 root test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t4.a, test.t4.b", + " └─HashJoin 15593.77 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t4.b, test.t1.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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 9980.01 root semi join, equal:[eq(test.t1.b, test.t3.b)]", + "├─TableReader(Build) 3.00 root data:Selection", + "│ └─Selection 3.00 cop[tikv] not(isnull(test.t3.b))", + "│ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "└─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 9980.01 root semi join, equal:[eq(test.t1.b, test.t3.b)]", + "├─TableReader(Build) 3.00 root data:Selection", + "│ └─Selection 3.00 cop[tikv] not(isnull(test.t3.b))", + "│ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "└─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 9980.01 root semi join, equal:[eq(test.t1.b, test.t3.b)]", + "├─TableReader(Build) 3.00 root data:Selection", + "│ └─Selection 3.00 cop[tikv] not(isnull(test.t3.b))", + "│ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "└─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 9980.01 root semi join, equal:[eq(test.t1.b, test.t3.b)]", + "├─TableReader(Build) 3.00 root data:Selection", + "│ └─Selection 3.00 cop[tikv] not(isnull(test.t3.b))", + "│ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "└─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 9980.01 root semi join, equal:[eq(test.t1.b, test.t3.b)]", + "├─TableReader(Build) 3.00 root data:Selection", + "│ └─Selection 3.00 cop[tikv] not(isnull(test.t3.b))", + "│ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "└─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 12475.01 root anti semi join, equal:[eq(test.t1.b, test.t3.b)]", + "├─TableReader(Build) 3.00 root data:TableFullScan", + "│ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "└─Projection(Probe) 15593.77 root test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t4.a, test.t4.b", + " └─HashJoin 15593.77 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t4.b, test.t1.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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 9990.00 root anti semi join, equal:[eq(test.t1.b, test.t3.b)]", + "├─TableReader(Build) 3.00 root data:TableFullScan", + "│ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 9990.00 root anti semi join, equal:[eq(test.t1.b, test.t3.b)]", + "├─TableReader(Build) 3.00 root data:TableFullScan", + "│ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 9990.00 root anti semi join, equal:[eq(test.t1.b, test.t3.b)]", + "├─TableReader(Build) 3.00 root data:TableFullScan", + "│ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 9990.00 root anti semi join, equal:[eq(test.t1.b, test.t3.b)]", + "├─TableReader(Build) 3.00 root data:TableFullScan", + "│ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 9990.00 root anti semi join, equal:[eq(test.t1.b, test.t3.b)]", + "├─TableReader(Build) 3.00 root data:TableFullScan", + "│ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "Projection 1.00 root test.t1.a, Column#14->Column#16", + "└─Apply 1.00 root CARTESIAN left outer join", + " ├─StreamAgg(Build) 1.00 root funcs:min(test.t1.a)->Column#10, funcs:firstrow(test.t1.a)->test.t1.a", + " │ └─IndexHashJoin 3.75 root inner join, inner:IndexReader, outer key:test.t3.a, inner key:test.t1.a, equal cond:eq(test.t3.a, test.t1.a)", + " │ ├─IndexReader(Build) 3.00 root index:IndexFullScan", + " │ │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:false", + " │ └─IndexReader(Probe) 3.75 root index:Selection", + " │ └─Selection 3.75 cop[tikv] not(isnull(test.t1.a))", + " │ └─IndexRangeScan 3.75 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, test.t3.a)], keep order:false, stats:pseudo", + " └─MaxOneRow(Probe) 1.00 root ", + " └─Projection 2.00 root Column#10->Column#14", + " └─IndexReader 2.00 root index:Selection", + " └─Selection 2.00 cop[tikv] gt(test.t2.a, test.t1.a)", + " └─IndexFullScan 2.50 cop[tikv] table:t2, index:a(a) keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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;", + "Plan": [ + "Projection 1.00 root test.t1.a, Column#14->Column#16", + "└─Apply 1.00 root CARTESIAN left outer join", + " ├─StreamAgg(Build) 1.00 root funcs:min(test.t1.a)->Column#10, funcs:firstrow(test.t1.a)->test.t1.a", + " │ └─IndexHashJoin 3.75 root inner join, inner:IndexReader, outer key:test.t3.a, inner key:test.t1.a, equal cond:eq(test.t3.a, test.t1.a)", + " │ ├─IndexReader(Build) 3.00 root index:IndexFullScan", + " │ │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:false", + " │ └─IndexReader(Probe) 3.75 root index:Selection", + " │ └─Selection 3.75 cop[tikv] not(isnull(test.t1.a))", + " │ └─IndexRangeScan 3.75 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, test.t3.a)], keep order:false, stats:pseudo", + " └─MaxOneRow(Probe) 1.00 root ", + " └─Projection 2.00 root Column#10->Column#14", + " └─IndexReader 2.00 root index:Selection", + " └─Selection 2.00 cop[tikv] gt(test.t2.a, test.t1.a)", + " └─IndexFullScan 2.50 cop[tikv] table:t2, index:a(a) keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t3) */. Maybe you can use the table alias name" + ] + }, + { + "SQL": "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;", + "Plan": [ + "Projection 1.00 root test.t1.a, Column#14->Column#16", + "└─Apply 1.00 root CARTESIAN left outer join", + " ├─StreamAgg(Build) 1.00 root funcs:min(test.t1.a)->Column#10, funcs:firstrow(test.t1.a)->test.t1.a", + " │ └─IndexHashJoin 3.75 root inner join, inner:IndexReader, outer key:test.t3.a, inner key:test.t1.a, equal cond:eq(test.t3.a, test.t1.a)", + " │ ├─IndexReader(Build) 3.00 root index:IndexFullScan", + " │ │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:false", + " │ └─IndexReader(Probe) 3.75 root index:Selection", + " │ └─Selection 3.75 cop[tikv] not(isnull(test.t1.a))", + " │ └─IndexRangeScan 3.75 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, test.t3.a)], keep order:false, stats:pseudo", + " └─MaxOneRow(Probe) 1.00 root ", + " └─Projection 2.00 root Column#10->Column#14", + " └─IndexReader 2.00 root index:Selection", + " └─Selection 2.00 cop[tikv] gt(test.t2.a, test.t1.a)", + " └─IndexFullScan 2.50 cop[tikv] table:t2, index:a(a) keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t2, t3) in optimizer hint /*+ LEADING(t2, t3) */. Maybe you can use the table alias name" + ] + }, + { + "SQL": "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;", + "Plan": [ + "Projection 1.00 root test.t1.a, Column#14->Column#16", + "└─Apply 1.00 root CARTESIAN left outer join", + " ├─StreamAgg(Build) 1.00 root funcs:min(test.t1.a)->Column#10, funcs:firstrow(test.t1.a)->test.t1.a", + " │ └─IndexHashJoin 3.75 root inner join, inner:IndexReader, outer key:test.t3.a, inner key:test.t1.a, equal cond:eq(test.t3.a, test.t1.a)", + " │ ├─IndexReader(Build) 3.00 root index:IndexFullScan", + " │ │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:false", + " │ └─IndexReader(Probe) 3.75 root index:Selection", + " │ └─Selection 3.75 cop[tikv] not(isnull(test.t1.a))", + " │ └─IndexRangeScan 3.75 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, test.t3.a)], keep order:false, stats:pseudo", + " └─MaxOneRow(Probe) 1.00 root ", + " └─Projection 2.00 root Column#10->Column#14", + " └─IndexReader 2.00 root index:Selection", + " └─Selection 2.00 cop[tikv] gt(test.t2.a, test.t1.a)", + " └─IndexFullScan 2.50 cop[tikv] table:t2, index:a(a) keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t1, t3) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "Projection 1.00 root test.t1.a, Column#14->Column#16", + "└─Apply 1.00 root CARTESIAN left outer join", + " ├─StreamAgg(Build) 1.00 root funcs:min(test.t1.a)->Column#10, funcs:firstrow(test.t1.a)->test.t1.a", + " │ └─IndexHashJoin 3.75 root inner join, inner:IndexReader, outer key:test.t3.a, inner key:test.t1.a, equal cond:eq(test.t3.a, test.t1.a)", + " │ ├─IndexReader(Build) 3.00 root index:IndexFullScan", + " │ │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:false", + " │ └─IndexReader(Probe) 3.75 root index:Selection", + " │ └─Selection 3.75 cop[tikv] not(isnull(test.t1.a))", + " │ └─IndexRangeScan 3.75 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, test.t3.a)], keep order:false, stats:pseudo", + " └─MaxOneRow(Probe) 1.00 root ", + " └─Projection 2.00 root Column#10->Column#14", + " └─IndexReader 2.00 root index:Selection", + " └─Selection 2.00 cop[tikv] gt(test.t2.a, test.t1.a)", + " └─IndexFullScan 2.50 cop[tikv] table:t2, index:a(a) keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t3, t2) in optimizer hint /*+ LEADING(t3, t2) */. Maybe you can use the table alias name" + ] + }, + { + "SQL": "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;", + "Plan": [ + "Projection 1.00 root test.t1.a, Column#14->Column#16", + "└─Apply 1.00 root CARTESIAN left outer join", + " ├─StreamAgg(Build) 1.00 root funcs:min(test.t1.a)->Column#10, funcs:firstrow(test.t1.a)->test.t1.a", + " │ └─IndexHashJoin 3.75 root inner join, inner:IndexReader, outer key:test.t3.a, inner key:test.t1.a, equal cond:eq(test.t3.a, test.t1.a)", + " │ ├─IndexReader(Build) 3.00 root index:IndexFullScan", + " │ │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:false", + " │ └─IndexReader(Probe) 3.75 root index:Selection", + " │ └─Selection 3.75 cop[tikv] not(isnull(test.t1.a))", + " │ └─IndexRangeScan 3.75 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, test.t3.a)], keep order:false, stats:pseudo", + " └─MaxOneRow(Probe) 1.00 root ", + " └─Projection 2.00 root Column#10->Column#14", + " └─IndexReader 2.00 root index:Selection", + " └─Selection 2.00 cop[tikv] gt(test.t2.a, test.t1.a)", + " └─IndexFullScan 2.50 cop[tikv] table:t2, index:a(a) keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t3, t1) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 4.69 root inner join, equal:[eq(test.t1.b, test.t4.b)]", + "├─IndexHashJoin(Build) 3.75 root inner join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a)", + "│ ├─IndexHashJoin(Build) 3.00 root inner join, inner:IndexLookUp, outer key:Column#13, inner key:test.t1.a, equal cond:eq(Column#13, test.t1.a), eq(test.t3.b, test.t1.b)", + "│ │ ├─Selection(Build) 2.40 root not(isnull(Column#13))", + "│ │ │ └─HashAgg 3.00 root group by:test.t3.b, funcs:max(test.t3.a)->Column#13, funcs:firstrow(test.t3.b)->test.t3.b", + "│ │ │ └─TableReader 3.00 root data:Selection", + "│ │ │ └─Selection 3.00 cop[tikv] not(isnull(test.t3.b))", + "│ │ │ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "│ │ └─IndexLookUp(Probe) 3.00 root ", + "│ │ ├─Selection(Build) 3.00 cop[tikv] not(isnull(test.t1.a))", + "│ │ │ └─IndexRangeScan 3.01 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, Column#13)], keep order:false, stats:pseudo", + "│ │ └─Selection(Probe) 3.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ └─TableRowIDScan 3.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ └─IndexLookUp(Probe) 3.75 root ", + "│ ├─Selection(Build) 3.75 cop[tikv] not(isnull(test.t2.a))", + "│ │ └─IndexRangeScan 3.75 cop[tikv] table:t2, index:a(a) range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo", + "│ └─TableRowIDScan(Probe) 3.75 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t4, t3) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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)", + "Plan": [ + "Projection 4.69 root test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t4.a, test.t4.b", + "└─IndexHashJoin 4.69 root inner join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a)", + " ├─HashJoin(Build) 3.75 root inner join, equal:[eq(test.t1.b, test.t3.b) eq(test.t1.a, Column#13)]", + " │ ├─Selection(Build) 2.40 root not(isnull(Column#13))", + " │ │ └─HashAgg 3.00 root group by:test.t3.b, funcs:max(test.t3.a)->Column#13, funcs:firstrow(test.t3.b)->test.t3.b", + " │ │ └─TableReader 3.00 root data:Selection", + " │ │ └─Selection 3.00 cop[tikv] not(isnull(test.t3.b))", + " │ │ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + " │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t4.b, test.t1.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", + " │ └─TableReader(Probe) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─IndexLookUp(Probe) 4.69 root ", + " ├─Selection(Build) 4.69 cop[tikv] not(isnull(test.t2.a))", + " │ └─IndexRangeScan 4.69 cop[tikv] table:t2, index:a(a) range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 4.69 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 4.69 root inner join, equal:[eq(test.t1.b, test.t4.b)]", + "├─IndexHashJoin(Build) 3.75 root inner join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a)", + "│ ├─IndexHashJoin(Build) 3.00 root inner join, inner:IndexLookUp, outer key:Column#13, inner key:test.t1.a, equal cond:eq(Column#13, test.t1.a), eq(test.t3.b, test.t1.b)", + "│ │ ├─Selection(Build) 2.40 root not(isnull(Column#13))", + "│ │ │ └─HashAgg 3.00 root group by:test.t3.b, funcs:max(test.t3.a)->Column#13, funcs:firstrow(test.t3.b)->test.t3.b", + "│ │ │ └─TableReader 3.00 root data:Selection", + "│ │ │ └─Selection 3.00 cop[tikv] not(isnull(test.t3.b))", + "│ │ │ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "│ │ └─IndexLookUp(Probe) 3.00 root ", + "│ │ ├─Selection(Build) 3.00 cop[tikv] not(isnull(test.t1.a))", + "│ │ │ └─IndexRangeScan 3.01 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, Column#13)], keep order:false, stats:pseudo", + "│ │ └─Selection(Probe) 3.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ └─TableRowIDScan 3.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ └─IndexLookUp(Probe) 3.75 root ", + "│ ├─Selection(Build) 3.75 cop[tikv] not(isnull(test.t2.a))", + "│ │ └─IndexRangeScan 3.75 cop[tikv] table:t2, index:a(a) range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo", + "│ └─TableRowIDScan(Probe) 3.75 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t3) */. Maybe you can use the table alias name" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 4.69 root inner join, equal:[eq(test.t1.b, test.t4.b)]", + "├─IndexHashJoin(Build) 3.75 root inner join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a)", + "│ ├─IndexHashJoin(Build) 3.00 root inner join, inner:IndexLookUp, outer key:Column#13, inner key:test.t1.a, equal cond:eq(Column#13, test.t1.a), eq(test.t3.b, test.t1.b)", + "│ │ ├─Selection(Build) 2.40 root not(isnull(Column#13))", + "│ │ │ └─HashAgg 3.00 root group by:test.t3.b, funcs:max(test.t3.a)->Column#13, funcs:firstrow(test.t3.b)->test.t3.b", + "│ │ │ └─TableReader 3.00 root data:Selection", + "│ │ │ └─Selection 3.00 cop[tikv] not(isnull(test.t3.b))", + "│ │ │ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "│ │ └─IndexLookUp(Probe) 3.00 root ", + "│ │ ├─Selection(Build) 3.00 cop[tikv] not(isnull(test.t1.a))", + "│ │ │ └─IndexRangeScan 3.01 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, Column#13)], keep order:false, stats:pseudo", + "│ │ └─Selection(Probe) 3.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ └─TableRowIDScan 3.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ └─IndexLookUp(Probe) 3.75 root ", + "│ ├─Selection(Build) 3.75 cop[tikv] not(isnull(test.t2.a))", + "│ │ └─IndexRangeScan 3.75 cop[tikv] table:t2, index:a(a) range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo", + "│ └─TableRowIDScan(Probe) 3.75 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t3, t2) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 4.69 root inner join, equal:[eq(test.t1.b, test.t4.b)]", + "├─IndexHashJoin(Build) 3.75 root inner join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a)", + "│ ├─HashJoin(Build) 3.00 root inner join, equal:[eq(test.t3.b, test.t1.b)], other cond:gt(test.t1.a, Column#13)", + "│ │ ├─Selection(Build) 2.40 root not(isnull(Column#13))", + "│ │ │ └─HashAgg 3.00 root group by:test.t3.b, funcs:min(test.t3.a)->Column#13, funcs:firstrow(test.t3.b)->test.t3.b", + "│ │ │ └─TableReader 3.00 root data:Selection", + "│ │ │ └─Selection 3.00 cop[tikv] not(isnull(test.t3.b))", + "│ │ │ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "│ │ └─TableReader(Probe) 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) 3.75 root ", + "│ ├─Selection(Build) 3.75 cop[tikv] not(isnull(test.t2.a))", + "│ │ └─IndexRangeScan 3.75 cop[tikv] table:t2, index:a(a) range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo", + "│ └─TableRowIDScan(Probe) 3.75 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t4, t3) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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)", + "Plan": [ + "Projection 4.69 root test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t4.a, test.t4.b", + "└─IndexHashJoin 4.69 root inner join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a)", + " ├─HashJoin(Build) 3.75 root inner join, equal:[eq(test.t1.b, test.t3.b)], other cond:gt(test.t1.a, Column#13)", + " │ ├─Selection(Build) 2.40 root not(isnull(Column#13))", + " │ │ └─HashAgg 3.00 root group by:test.t3.b, funcs:min(test.t3.a)->Column#13, funcs:firstrow(test.t3.b)->test.t3.b", + " │ │ └─TableReader 3.00 root data:Selection", + " │ │ └─Selection 3.00 cop[tikv] not(isnull(test.t3.b))", + " │ │ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + " │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t4.b, test.t1.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", + " │ └─TableReader(Probe) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─IndexLookUp(Probe) 4.69 root ", + " ├─Selection(Build) 4.69 cop[tikv] not(isnull(test.t2.a))", + " │ └─IndexRangeScan 4.69 cop[tikv] table:t2, index:a(a) range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 4.69 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 4.69 root inner join, equal:[eq(test.t1.b, test.t4.b)]", + "├─IndexHashJoin(Build) 3.75 root inner join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a)", + "│ ├─HashJoin(Build) 3.00 root inner join, equal:[eq(test.t3.b, test.t1.b)], other cond:gt(test.t1.a, Column#13)", + "│ │ ├─Selection(Build) 2.40 root not(isnull(Column#13))", + "│ │ │ └─HashAgg 3.00 root group by:test.t3.b, funcs:min(test.t3.a)->Column#13, funcs:firstrow(test.t3.b)->test.t3.b", + "│ │ │ └─TableReader 3.00 root data:Selection", + "│ │ │ └─Selection 3.00 cop[tikv] not(isnull(test.t3.b))", + "│ │ │ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "│ │ └─TableReader(Probe) 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) 3.75 root ", + "│ ├─Selection(Build) 3.75 cop[tikv] not(isnull(test.t2.a))", + "│ │ └─IndexRangeScan 3.75 cop[tikv] table:t2, index:a(a) range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo", + "│ └─TableRowIDScan(Probe) 3.75 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t3) */. Maybe you can use the table alias name" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 4.69 root inner join, equal:[eq(test.t1.b, test.t4.b)]", + "├─IndexHashJoin(Build) 3.75 root inner join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a)", + "│ ├─HashJoin(Build) 3.00 root inner join, equal:[eq(test.t3.b, test.t1.b)], other cond:gt(test.t1.a, Column#13)", + "│ │ ├─Selection(Build) 2.40 root not(isnull(Column#13))", + "│ │ │ └─HashAgg 3.00 root group by:test.t3.b, funcs:min(test.t3.a)->Column#13, funcs:firstrow(test.t3.b)->test.t3.b", + "│ │ │ └─TableReader 3.00 root data:Selection", + "│ │ │ └─Selection 3.00 cop[tikv] not(isnull(test.t3.b))", + "│ │ │ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "│ │ └─TableReader(Probe) 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) 3.75 root ", + "│ ├─Selection(Build) 3.75 cop[tikv] not(isnull(test.t2.a))", + "│ │ └─IndexRangeScan 3.75 cop[tikv] table:t2, index:a(a) range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo", + "│ └─TableRowIDScan(Probe) 3.75 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t3, t2) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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)", + "Plan": [ + "Projection 5.86 root test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t4.a, test.t4.b", + "└─IndexHashJoin 5.86 root inner join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a)", + " ├─HashJoin(Build) 4.69 root inner join, equal:[eq(test.t1.a, test.t3.a)]", + " │ ├─StreamAgg(Build) 3.00 root group by:test.t3.a, funcs:firstrow(test.t3.a)->test.t3.a", + " │ │ └─IndexReader 3.00 root index:StreamAgg", + " │ │ └─StreamAgg 3.00 cop[tikv] group by:test.t3.a, ", + " │ │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:true", + " │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t4.b, test.t1.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", + " │ └─TableReader(Probe) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─IndexLookUp(Probe) 5.86 root ", + " ├─Selection(Build) 5.86 cop[tikv] not(isnull(test.t2.a))", + " │ └─IndexRangeScan 5.87 cop[tikv] table:t2, index:a(a) range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 5.86 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where t1.a in (select t3.a from t3)", + "Plan": [ + "IndexHashJoin 4.69 root inner join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a)", + "├─IndexHashJoin(Build) 3.75 root inner join, inner:IndexLookUp, outer key:test.t3.a, inner key:test.t1.a, equal cond:eq(test.t3.a, test.t1.a)", + "│ ├─StreamAgg(Build) 3.00 root group by:test.t3.a, funcs:firstrow(test.t3.a)->test.t3.a", + "│ │ └─IndexReader 3.00 root index:StreamAgg", + "│ │ └─StreamAgg 3.00 cop[tikv] group by:test.t3.a, ", + "│ │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:true", + "│ └─IndexLookUp(Probe) 3.75 root ", + "│ ├─Selection(Build) 3.75 cop[tikv] not(isnull(test.t1.a))", + "│ │ └─IndexRangeScan 3.75 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, test.t3.a)], keep order:false, stats:pseudo", + "│ └─TableRowIDScan(Probe) 3.75 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─IndexLookUp(Probe) 4.69 root ", + " ├─Selection(Build) 4.69 cop[tikv] not(isnull(test.t2.a))", + " │ └─IndexRangeScan 4.69 cop[tikv] table:t2, index:a(a) range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 4.69 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2, t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where t1.a in (select t3.a from t3)", + "Plan": [ + "IndexHashJoin 4.69 root inner join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a)", + "├─IndexHashJoin(Build) 3.75 root inner join, inner:IndexLookUp, outer key:test.t3.a, inner key:test.t1.a, equal cond:eq(test.t3.a, test.t1.a)", + "│ ├─StreamAgg(Build) 3.00 root group by:test.t3.a, funcs:firstrow(test.t3.a)->test.t3.a", + "│ │ └─IndexReader 3.00 root index:StreamAgg", + "│ │ └─StreamAgg 3.00 cop[tikv] group by:test.t3.a, ", + "│ │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:true", + "│ └─IndexLookUp(Probe) 3.75 root ", + "│ ├─Selection(Build) 3.75 cop[tikv] not(isnull(test.t1.a))", + "│ │ └─IndexRangeScan 3.75 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, test.t3.a)], keep order:false, stats:pseudo", + "│ └─TableRowIDScan(Probe) 3.75 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─IndexLookUp(Probe) 4.69 root ", + " ├─Selection(Build) 4.69 cop[tikv] not(isnull(test.t2.a))", + " │ └─IndexRangeScan 4.69 cop[tikv] table:t2, index:a(a) range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 4.69 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid" + ] + }, + { + "SQL": "select /*+ leading(t1, t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where t1.a in (select t3.a from t3)", + "Plan": [ + "IndexHashJoin 4.69 root inner join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a)", + "├─IndexHashJoin(Build) 3.75 root inner join, inner:IndexLookUp, outer key:test.t3.a, inner key:test.t1.a, equal cond:eq(test.t3.a, test.t1.a)", + "│ ├─StreamAgg(Build) 3.00 root group by:test.t3.a, funcs:firstrow(test.t3.a)->test.t3.a", + "│ │ └─IndexReader 3.00 root index:StreamAgg", + "│ │ └─StreamAgg 3.00 cop[tikv] group by:test.t3.a, ", + "│ │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:true", + "│ └─IndexLookUp(Probe) 3.75 root ", + "│ ├─Selection(Build) 3.75 cop[tikv] not(isnull(test.t1.a))", + "│ │ └─IndexRangeScan 3.75 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, test.t3.a)], keep order:false, stats:pseudo", + "│ └─TableRowIDScan(Probe) 3.75 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─IndexLookUp(Probe) 4.69 root ", + " ├─Selection(Build) 4.69 cop[tikv] not(isnull(test.t2.a))", + " │ └─IndexRangeScan 4.69 cop[tikv] table:t2, index:a(a) range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 4.69 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid" + ] + }, + { + "SQL": "select /*+ leading(t3@sel_2, t2) */ * from t1 join t2 on t1.a=t2.a where t1.a in (select t3.a from t3)", + "Plan": [ + "IndexHashJoin 4.69 root inner join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a)", + "├─IndexHashJoin(Build) 3.75 root inner join, inner:IndexLookUp, outer key:test.t3.a, inner key:test.t1.a, equal cond:eq(test.t3.a, test.t1.a)", + "│ ├─StreamAgg(Build) 3.00 root group by:test.t3.a, funcs:firstrow(test.t3.a)->test.t3.a", + "│ │ └─IndexReader 3.00 root index:StreamAgg", + "│ │ └─StreamAgg 3.00 cop[tikv] group by:test.t3.a, ", + "│ │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:true", + "│ └─IndexLookUp(Probe) 3.75 root ", + "│ ├─Selection(Build) 3.75 cop[tikv] not(isnull(test.t1.a))", + "│ │ └─IndexRangeScan 3.75 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, test.t3.a)], keep order:false, stats:pseudo", + "│ └─TableRowIDScan(Probe) 3.75 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─IndexLookUp(Probe) 4.69 root ", + " ├─Selection(Build) 4.69 cop[tikv] not(isnull(test.t2.a))", + " │ └─IndexRangeScan 4.69 cop[tikv] table:t2, index:a(a) range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 4.69 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid" + ] + }, + { + "SQL": "select /*+ leading(t3@sel_2, t1) */ * from t1 join t2 on t1.a=t2.a where t1.a in (select t3.a from t3)", + "Plan": [ + "IndexHashJoin 4.69 root inner join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a)", + "├─IndexHashJoin(Build) 3.75 root inner join, inner:IndexLookUp, outer key:test.t3.a, inner key:test.t1.a, equal cond:eq(test.t3.a, test.t1.a)", + "│ ├─StreamAgg(Build) 3.00 root group by:test.t3.a, funcs:firstrow(test.t3.a)->test.t3.a", + "│ │ └─IndexReader 3.00 root index:StreamAgg", + "│ │ └─StreamAgg 3.00 cop[tikv] group by:test.t3.a, ", + "│ │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:true", + "│ └─IndexLookUp(Probe) 3.75 root ", + "│ ├─Selection(Build) 3.75 cop[tikv] not(isnull(test.t1.a))", + "│ │ └─IndexRangeScan 3.75 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, test.t3.a)], keep order:false, stats:pseudo", + "│ └─TableRowIDScan(Probe) 3.75 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─IndexLookUp(Probe) 4.69 root ", + " ├─Selection(Build) 4.69 cop[tikv] not(isnull(test.t2.a))", + " │ └─IndexRangeScan 4.69 cop[tikv] table:t2, index:a(a) range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 4.69 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 12475.01 root Null-aware anti semi join, equal:[eq(test.t1.a, test.t3.a)]", + "├─IndexReader(Build) 3.00 root index:IndexFullScan", + "│ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:false", + "└─Projection(Probe) 15593.77 root test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t4.a, test.t4.b", + " └─HashJoin 15593.77 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t4.b, test.t1.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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where t1.a not in (select t3.a from t3)", + "Plan": [ + "HashJoin 9990.00 root Null-aware anti semi join, equal:[eq(test.t1.a, test.t3.a)]", + "├─IndexReader(Build) 3.00 root index:IndexFullScan", + "│ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:false", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 9990.00 root Null-aware anti semi join, equal:[eq(test.t1.a, test.t3.a)]", + "├─IndexReader(Build) 3.00 root index:IndexFullScan", + "│ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:false", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 9990.00 root Null-aware anti semi join, equal:[eq(test.t1.a, test.t3.a)]", + "├─IndexReader(Build) 3.00 root index:IndexFullScan", + "│ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:false", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 9990.00 root Null-aware anti semi join, equal:[eq(test.t1.a, test.t3.a)]", + "├─IndexReader(Build) 3.00 root index:IndexFullScan", + "│ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:false", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 9990.00 root Null-aware anti semi join, equal:[eq(test.t1.a, test.t3.a)]", + "├─IndexReader(Build) 3.00 root index:IndexFullScan", + "│ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:false", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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)", + "Plan": [ + "Projection 15593.77 root test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t4.a, test.t4.b", + "└─HashJoin 15593.77 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t4.b, test.t1.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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where exists (select t3.a from t3)", + "Plan": [ + "HashJoin 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─TableReader(Probe) 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" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t3) */. Maybe you can use the table alias name" + ] + }, + { + "SQL": "select /*+ leading(t2, t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where exists (select t3.a from t3)", + "Plan": [ + "HashJoin 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─TableReader(Probe) 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" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t2, t3) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t1, t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where exists (select t3.a from t3)", + "Plan": [ + "HashJoin 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─TableReader(Probe) 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" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t1, t3) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t3@sel_2, t2) */ * from t1 join t2 on t1.a=t2.a where exists (select t3.a from t3)", + "Plan": [ + "HashJoin 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─TableReader(Probe) 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" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t3, t2) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t3@sel_2, t1) */ * from t1 join t2 on t1.a=t2.a where exists (select t3.a from t3)", + "Plan": [ + "HashJoin 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─TableReader(Probe) 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" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t3, t1) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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)", + "Plan": [ + "TableDual 0.00 root rows:0" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where not exists (select t3.a from t3)", + "Plan": [ + "TableDual 0.00 root rows:0" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t3) */. Maybe you can use the table alias name" + ] + }, + { + "SQL": "select /*+ leading(t2, t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where not exists (select t3.a from t3)", + "Plan": [ + "TableDual 0.00 root rows:0" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t2, t3) */. Maybe you can use the table alias name" + ] + }, + { + "SQL": "select /*+ leading(t1, t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where not exists (select t3.a from t3)", + "Plan": [ + "TableDual 0.00 root rows:0" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t1, t3) */. Maybe you can use the table alias name" + ] + }, + { + "SQL": "select /*+ leading(t3@sel_2, t2) */ * from t1 join t2 on t1.a=t2.a where not exists (select t3.a from t3)", + "Plan": [ + "TableDual 0.00 root rows:0" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t3, t2) */. Maybe you can use the table alias name" + ] + }, + { + "SQL": "select /*+ leading(t3@sel_2, t1) */ * from t1 join t2 on t1.a=t2.a where not exists (select t3.a from t3)", + "Plan": [ + "TableDual 0.00 root rows:0" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t3, t1) */. Maybe you can use the table alias name" + ] + }, + { + "SQL": "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;", + "Plan": [ + "Projection 4.69 root test.t1.a, test.t1.b, test.t4.a, test.t4.b, test.t3.a, test.t3.b", + "└─HashJoin 4.69 root inner join, equal:[eq(test.t4.b, test.t3.b)]", + " ├─TableReader(Build) 3.00 root data:Selection", + " │ └─Selection 3.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t4.a, test.t1.a)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t4.a)), not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t4) in optimizer hint /*+ LEADING(t4, t1) */. Maybe you can use the table alias name" + ] + }, + { + "SQL": "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;", + "Plan": [ + "Projection 4.69 root test.t4.a, test.t4.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 4.69 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─TableReader(Build) 3.00 root data:Selection", + " │ └─Selection 3.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.a, test.t4.a)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t4) in optimizer hint /*+ LEADING(t2, t4) */. Maybe you can use the table alias name" + ] + }, + { + "SQL": "select /*+ leading(t3) */ * from t1 join t2 on t1.a=t2.a join (select * from t4) t3 on t2.b=t3.b;", + "Plan": [ + "Projection 15593.77 root test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t4.a, test.t4.b", + "└─HashJoin 15593.77 root inner join, equal:[eq(test.t2.a, test.t1.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", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t4.b, test.t2.b)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t1) */ t1.a, (select min(t2.a) from t2) from t1 join t3 on t1.a = t3.a;", + "Plan": [ + "Projection 3.75 root test.t1.a, ->Column#14", + "└─IndexHashJoin 3.75 root inner join, inner:IndexReader, outer key:test.t3.a, inner key:test.t1.a, equal cond:eq(test.t3.a, test.t1.a)", + " ├─IndexReader(Build) 3.00 root index:IndexFullScan", + " │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:false", + " └─IndexReader(Probe) 3.75 root index:Selection", + " └─Selection 3.75 cop[tikv] not(isnull(test.t1.a))", + " └─IndexRangeScan 3.75 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, test.t3.a)], keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t1, t2@sel_2) */ t1.a, (select min(t2.a) from t2) from t1 join t3 on t1.a = t3.a;", + "Plan": [ + "Projection 3.75 root test.t1.a, ->Column#14", + "└─IndexHashJoin 3.75 root inner join, inner:IndexReader, outer key:test.t3.a, inner key:test.t1.a, equal cond:eq(test.t3.a, test.t1.a)", + " ├─IndexReader(Build) 3.00 root index:IndexFullScan", + " │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:false", + " └─IndexReader(Probe) 3.75 root index:Selection", + " └─Selection 3.75 cop[tikv] not(isnull(test.t1.a))", + " └─IndexRangeScan 3.75 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, test.t3.a)], keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t2) in optimizer hint /*+ LEADING(t1, t2) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t1, t3) */ t1.a, (select min(t2.a) from t2) from t1 join t3 on t1.a = t3.a;", + "Plan": [ + "Projection 3.75 root test.t1.a, ->Column#14", + "└─IndexHashJoin 3.75 root inner join, inner:IndexReader, outer key:test.t3.a, inner key:test.t1.a, equal cond:eq(test.t3.a, test.t1.a)", + " ├─IndexReader(Build) 3.00 root index:IndexFullScan", + " │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:false", + " └─IndexReader(Probe) 3.75 root index:Selection", + " └─Selection 3.75 cop[tikv] not(isnull(test.t1.a))", + " └─IndexRangeScan 3.75 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, test.t3.a)], keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2@sel_2, t1) */ t1.a, (select min(t2.a) from t2) from t1 join t3 on t1.a = t3.a;", + "Plan": [ + "Projection 3.75 root test.t1.a, ->Column#14", + "└─IndexHashJoin 3.75 root inner join, inner:IndexReader, outer key:test.t3.a, inner key:test.t1.a, equal cond:eq(test.t3.a, test.t1.a)", + " ├─IndexReader(Build) 3.00 root index:IndexFullScan", + " │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:false", + " └─IndexReader(Probe) 3.75 root index:Selection", + " └─Selection 3.75 cop[tikv] not(isnull(test.t1.a))", + " └─IndexRangeScan 3.75 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, test.t3.a)], keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t2) 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" + ] + }, + { + "SQL": "select /*+ leading(t2@sel_2, t3) */ t1.a, (select min(t2.a) from t2) from t1 join t3 on t1.a = t3.a;", + "Plan": [ + "Projection 3.75 root test.t1.a, ->Column#14", + "└─IndexHashJoin 3.75 root inner join, inner:IndexReader, outer key:test.t3.a, inner key:test.t1.a, equal cond:eq(test.t3.a, test.t1.a)", + " ├─IndexReader(Build) 3.00 root index:IndexFullScan", + " │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:false", + " └─IndexReader(Probe) 3.75 root index:Selection", + " └─Selection 3.75 cop[tikv] not(isnull(test.t1.a))", + " └─IndexRangeScan 3.75 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, test.t3.a)], keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t2) in optimizer hint /*+ LEADING(t2, t3) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t1, t2@sel_2) */ t1.a, (select min(t2.a) from t2) from t1 join t3 on t1.a = t3.a;", + "Plan": [ + "Projection 3.75 root test.t1.a, ->Column#14", + "└─IndexHashJoin 3.75 root inner join, inner:IndexReader, outer key:test.t3.a, inner key:test.t1.a, equal cond:eq(test.t3.a, test.t1.a)", + " ├─IndexReader(Build) 3.00 root index:IndexFullScan", + " │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:false", + " └─IndexReader(Probe) 3.75 root index:Selection", + " └─Selection 3.75 cop[tikv] not(isnull(test.t1.a))", + " └─IndexRangeScan 3.75 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, test.t3.a)], keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t2) in optimizer hint /*+ LEADING(t1, t2) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t3, t2@sel_2) */ t1.a, (select min(t2.a) from t2) from t1 join t3 on t1.a = t3.a;", + "Plan": [ + "Projection 3.75 root test.t1.a, ->Column#14", + "└─IndexHashJoin 3.75 root inner join, inner:IndexReader, outer key:test.t3.a, inner key:test.t1.a, equal cond:eq(test.t3.a, test.t1.a)", + " ├─IndexReader(Build) 3.00 root index:IndexFullScan", + " │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:false", + " └─IndexReader(Probe) 3.75 root index:Selection", + " └─Selection 3.75 cop[tikv] not(isnull(test.t1.a))", + " └─IndexRangeScan 3.75 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, test.t3.a)], keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t2) in optimizer hint /*+ LEADING(t3, t2) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + } + ] + }, + { + "Name": "TestLeadingJoinHint4OuterJoin", + "Cases": [ + { + "SQL": "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;", + "Plan": [ + "Projection 24365.26 root test.t4.a, test.t4.b, test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 24365.26 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 19492.21 root inner join, equal:[eq(test.t.b, test.t2.b)]", + " ├─TableReader(Build) 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", + " └─HashJoin(Probe) 15593.77 root left outer join, equal:[eq(test.t.a, test.t1.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", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.a, test.t4.a)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t2, t3) */ * 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;", + "Plan": [ + "Projection 24365.26 root test.t4.a, test.t4.b, test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 24365.26 root left outer join, equal:[eq(test.t.a, test.t1.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", + " └─HashJoin(Probe) 19492.21 root inner join, equal:[eq(test.t.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t2.b, test.t.b)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "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;", + "Plan": [ + "Projection 24365.26 root test.t4.a, test.t4.b, test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 24365.26 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 19492.21 root inner join, equal:[eq(test.t.b, test.t2.b)]", + " ├─TableReader(Build) 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", + " └─HashJoin(Probe) 15593.77 root left outer join, equal:[eq(test.t.a, test.t1.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", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.a, test.t4.a)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "Projection 24365.26 root test.t4.a, test.t4.b, test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 24365.26 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 19492.21 root inner join, equal:[eq(test.t.b, test.t2.b)]", + " ├─TableReader(Build) 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", + " └─HashJoin(Probe) 15593.77 root left outer join, equal:[eq(test.t.a, test.t1.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", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.a, test.t4.a)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "Projection 24365.26 root test.t4.a, test.t4.b, test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 24365.26 root left outer join, equal:[eq(test.t.a, test.t1.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", + " └─HashJoin(Probe) 19492.21 root inner join, equal:[eq(test.t.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t2.b, test.t.b)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t3.b, test.t2.b)]", + " ├─TableReader(Build) 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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.b=t1.b;", + "Plan": [ + "HashJoin 15609.38 root left outer join, equal:[eq(test.t2.b, test.t1.b)]", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t3.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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 /*+ leading(t2, t1, t3) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.b=t1.b;", + "Plan": [ + "HashJoin 15609.38 root left outer join, equal:[eq(test.t2.b, test.t1.b)]", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t3.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "HashJoin 19492.21 root left outer join, equal:[eq(test.t2.b, test.t1.b)]", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t3.b, test.t4.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t1.a, test.t3.a)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "HashJoin 19492.21 root left outer join, equal:[eq(test.t2.b, test.t1.b)]", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─Projection(Probe) 15593.77 root test.t1.a, test.t1.b, test.t3.a, test.t3.b, test.t4.a, test.t4.b", + " └─HashJoin 15593.77 root inner join, equal:[eq(test.t3.a, test.t1.a)]", + " ├─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", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t3.b, test.t4.b)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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;", + "Plan": [ + "Projection 30426.12 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b, test.t4.a, test.t4.b, test.t5.a, test.t5.b, test.t6.a, test.t6.b", + "└─HashJoin 30426.12 root left outer join, equal:[eq(test.t2.b, test.t1.b)]", + " ├─Projection(Build) 15593.77 root test.t1.a, test.t1.b, test.t3.a, test.t3.b, test.t4.a, test.t4.b", + " │ └─HashJoin 15593.77 root inner join, equal:[eq(test.t3.a, test.t1.a)]", + " │ ├─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", + " │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t3.b, test.t4.b)]", + " │ ├─TableReader(Build) 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " │ └─TableReader(Probe) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t5.b, test.t6.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t6.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t5.a, test.t2.a)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t5 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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;", + "Plan": [ + "Projection 30426.12 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b, test.t4.a, test.t4.b, test.t5.a, test.t5.b, test.t6.a, test.t6.b", + "└─HashJoin 30426.12 root left outer join, equal:[eq(test.t2.b, test.t1.b)]", + " ├─HashJoin(Build) 15593.77 root inner join, equal:[eq(test.t3.b, test.t4.b)]", + " │ ├─TableReader(Build) 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t1.a, test.t3.a)]", + " │ ├─TableReader(Build) 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " │ └─TableReader(Probe) 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", + " └─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t5.b, test.t6.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t6.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t5.a, test.t2.a)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t5 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "Projection 30426.12 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b, test.t4.a, test.t4.b, test.t5.a, test.t5.b, test.t6.a, test.t6.b", + "└─HashJoin 30426.12 root left outer join, equal:[eq(test.t2.b, test.t1.b)]", + " ├─HashJoin(Build) 15593.77 root inner join, equal:[eq(test.t3.b, test.t4.b)]", + " │ ├─TableReader(Build) 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t1.a, test.t3.a)]", + " │ ├─TableReader(Build) 9980.01 root data:Selection", + " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " │ └─TableReader(Probe) 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", + " └─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t5.b, test.t6.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t6.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t5.a, test.t2.a)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t5 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "HashJoin 24389.65 root right outer join, equal:[eq(test.t.a, test.t1.a)]", + "├─HashJoin(Build) 12487.50 root inner join, equal:[eq(test.t4.a, test.t.a)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 15609.38 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.b, test.t2.b)]", + " ├─TableReader(Build) 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", + " └─TableReader(Probe) 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 /*+ 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;", + "Plan": [ + "Projection 24389.65 root test.t4.a, test.t4.b, test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 24389.65 root right outer join, equal:[eq(test.t.a, test.t1.a)]", + " ├─HashJoin(Build) 12487.50 root inner join, equal:[eq(test.t4.a, test.t.a)]", + " │ ├─TableReader(Build) 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + " │ └─TableReader(Probe) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15609.38 root inner join, equal:[eq(test.t2.b, test.t1.b)]", + " ├─TableReader(Build) 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", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "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;", + "Plan": [ + "HashJoin 24389.65 root right outer join, equal:[eq(test.t.a, test.t1.a)]", + "├─HashJoin(Build) 12487.50 root inner join, equal:[eq(test.t4.a, test.t.a)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 15609.38 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.b, test.t2.b)]", + " ├─TableReader(Build) 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", + " └─TableReader(Probe) 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": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t3) */ * from t2 right join (t1 left join t3 on t1.a=t3.a) on t2.b=t1.b;", + "Plan": [ + "HashJoin 15593.77 root right outer join, equal:[eq(test.t2.b, test.t1.b)]", + "├─TableReader(Build) 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", + "└─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t3.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " │ └─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:t1 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2, t1, t3) */ * from t2 right join (t1 left join t3 on t1.a=t3.a) on t2.b=t1.b;", + "Plan": [ + "HashJoin 15593.77 root left outer join, equal:[eq(test.t1.a, test.t3.a)]", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root right outer join, equal:[eq(test.t2.b, test.t1.b)]", + " ├─TableReader(Build) 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", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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;", + "Plan": [ + "Projection 19492.21 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b, test.t4.a, test.t4.b", + "└─HashJoin 19492.21 root inner join, equal:[eq(test.t3.b, test.t4.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15593.77 root right outer join, equal:[eq(test.t2.b, test.t1.b)]", + " ├─TableReader(Build) 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", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t3.a, test.t1.a)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "Projection 19492.21 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b, test.t4.a, test.t4.b", + "└─HashJoin 19492.21 root right outer join, equal:[eq(test.t2.b, test.t1.b)]", + " ├─TableReader(Build) 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", + " └─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t3.a, test.t1.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", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t3.b, test.t4.b)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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;", + "Plan": [ + "Projection 30456.57 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b, test.t4.a, test.t4.b, test.t5.a, test.t5.b, test.t6.a, test.t6.b", + "└─HashJoin 30456.57 root inner join, equal:[eq(test.t5.b, test.t6.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t6.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 24365.26 root inner join, equal:[eq(test.t2.a, test.t5.a)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t5 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 19492.21 root inner join, equal:[eq(test.t1.b, test.t2.b)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t3.a, test.t1.a)]", + " ├─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", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t3.b, test.t4.b)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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;", + "Plan": [ + "Projection 30456.57 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b, test.t4.a, test.t4.b, test.t5.a, test.t5.b, test.t6.a, test.t6.b", + "└─HashJoin 30456.57 root inner join, equal:[eq(test.t5.b, test.t6.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t6.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 24365.26 root inner join, equal:[eq(test.t3.b, test.t4.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 19492.21 root inner join, equal:[eq(test.t2.a, test.t5.a)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t5 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t1.a, test.t3.a)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t1.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", + " └─TableReader(Probe) 9980.01 root data:Selection", + " └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "Projection 243165526.37 root test.t2.a, test.t2.b, test.t1.a, test.t1.b, test.t3.a, test.t3.b, test.t4.a, test.t4.b, test.t5.a, test.t5.b, test.t6.a, test.t6.b", + "└─HashJoin 243165526.37 root inner join, equal:[eq(test.t3.a, test.t1.a) eq(test.t2.b, test.t1.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", + " └─HashJoin(Probe) 194532421.09 root inner join, equal:[eq(test.t5.a, test.t2.a)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 155625936.88 root inner join, equal:[eq(test.t5.b, test.t6.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t6.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 124500749.50 root CARTESIAN inner join", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t5.a)), not(isnull(test.t5.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t5 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t3.b, test.t4.b)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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 ;", + "Plan": [ + "HashJoin 19492.21 root right outer join, equal:[eq(test.t1.b, test.t2.b)]", + "├─Projection(Build) 12475.01 root test.t.a, test.t.b, test.t1.a, test.t1.b", + "│ └─HashJoin 12475.01 root inner join, equal:[eq(test.t1.a, test.t.a)]", + "│ ├─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", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "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 ;", + "Plan": [ + "HashJoin 19492.21 root right outer join, equal:[eq(test.t1.b, test.t2.b)]", + "├─Projection(Build) 12475.01 root test.t.a, test.t.b, test.t1.a, test.t1.b", + "│ └─HashJoin 12475.01 root inner join, equal:[eq(test.t1.a, test.t.a)]", + "│ ├─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", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2) hash_join(t3) */ * 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 ;", + "Plan": [ + "HashJoin 19492.21 root right outer join, equal:[eq(test.t1.b, test.t2.b)]", + "├─Projection(Build) 12475.01 root test.t.a, test.t.b, test.t1.a, test.t1.b", + "│ └─HashJoin 12475.01 root inner join, equal:[eq(test.t1.a, test.t.a)]", + "│ ├─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", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t) hash_join(t2) */ * 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 ;", + "Plan": [ + "HashJoin 19492.21 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t1.b, test.t2.b)]", + " ├─TableReader(Build) 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", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.a, test.t1.a)]", + " ├─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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t) hash_join(t1) */ * 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 ;", + "Plan": [ + "HashJoin 19492.21 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t1.b, test.t2.b)]", + " ├─TableReader(Build) 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", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.a, test.t1.a)]", + " ├─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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t) hash_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 ;", + "Plan": [ + "HashJoin 19492.21 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t1.b, test.t2.b)]", + " ├─TableReader(Build) 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", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.a, test.t1.a)]", + " ├─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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2) INL_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 ;", + "Plan": [ + "HashJoin 19492.21 root right outer join, equal:[eq(test.t1.b, test.t2.b)]", + "├─Projection(Build) 12475.01 root test.t.a, test.t.b, test.t1.a, test.t1.b", + "│ └─IndexJoin 12475.01 root inner join, inner:IndexLookUp, outer key:test.t.a, inner key:test.t1.a, equal cond:eq(test.t.a, test.t1.a)", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "│ └─IndexLookUp(Probe) 12475.01 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.t.a)], keep order:false, stats:pseudo", + "│ └─Selection(Probe) 12475.01 cop[tikv] not(isnull(test.t1.b))", + "│ └─TableRowIDScan 12487.50 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2) INL_JOIN(t3) */ * 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 ;", + "Plan": [ + "HashJoin 19492.21 root right outer join, equal:[eq(test.t1.b, test.t2.b)]", + "├─Projection(Build) 12475.01 root test.t.a, test.t.b, test.t1.a, test.t1.b", + "│ └─HashJoin 12475.01 root inner join, equal:[eq(test.t1.a, test.t.a)]", + "│ ├─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", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": [ + "Warning 1815 Optimizer Hint /*+ INL_JOIN(t3) */ or /*+ TIDB_INLJ(t3) */ is inapplicable" + ] + }, + { + "SQL": "select /*+ leading(t3) INL_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 ;", + "Plan": [ + "Projection 19492.21 root test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 19492.21 root right outer join, equal:[eq(test.t1.b, test.t2.b)]", + " ├─Projection(Build) 12475.01 root test.t.a, test.t.b, test.t1.a, test.t1.b", + " │ └─IndexJoin 12475.01 root inner join, inner:IndexLookUp, outer key:test.t.a, inner key:test.t1.a, equal cond:eq(test.t.a, test.t1.a)", + " │ ├─TableReader(Build) 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + " │ └─IndexLookUp(Probe) 12475.01 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.t.a)], keep order:false, stats:pseudo", + " │ └─Selection(Probe) 12475.01 cop[tikv] not(isnull(test.t1.b))", + " │ └─TableRowIDScan 12487.50 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t3.b, test.t2.b)]", + " ├─TableReader(Build) 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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3) INL_JOIN(t3) */ * 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 ;", + "Plan": [ + "Projection 19492.21 root test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 19492.21 root right outer join, equal:[eq(test.t1.b, test.t2.b)]", + " ├─Projection(Build) 12475.01 root test.t.a, test.t.b, test.t1.a, test.t1.b", + " │ └─HashJoin 12475.01 root inner join, equal:[eq(test.t1.a, test.t.a)]", + " │ ├─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", + " │ └─TableReader(Probe) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t3.b, test.t2.b)]", + " ├─TableReader(Build) 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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 Optimizer Hint /*+ INL_JOIN(t3) */ or /*+ TIDB_INLJ(t3) */ is inapplicable" + ] + }, + { + "SQL": "select /*+ leading(t2) merge_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 ;", + "Plan": [ + "HashJoin 19492.21 root right outer join, equal:[eq(test.t1.b, test.t2.b)]", + "├─Projection(Build) 12475.01 root test.t.a, test.t.b, test.t1.a, test.t1.b", + "│ └─HashJoin 12475.01 root inner join, equal:[eq(test.t1.a, test.t.a)]", + "│ ├─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", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "└─MergeJoin(Probe) 12487.50 root inner join, left key:test.t2.b, right key:test.t3.b", + " ├─Sort(Build) 9990.00 root test.t3.b", + " │ └─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─Sort(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t1) merge_join(t2) */ * from t join t1 on t.a = t1.a left join t2 on t1.b = t2.b join t3 on t2.b = t3.b ;", + "Plan": [ + "Projection 19492.21 root test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 19492.21 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─MergeJoin(Probe) 15593.77 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) 12475.01 root test.t1.b", + " └─HashJoin 12475.01 root inner join, equal:[eq(test.t1.a, test.t.a)]", + " ├─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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2) merge_join(t3) */ * 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 ;", + "Plan": [ + "HashJoin 19492.21 root right outer join, equal:[eq(test.t1.b, test.t2.b)]", + "├─Projection(Build) 12475.01 root test.t.a, test.t.b, test.t1.a, test.t1.b", + "│ └─HashJoin 12475.01 root inner join, equal:[eq(test.t1.a, test.t.a)]", + "│ ├─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", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "└─MergeJoin(Probe) 12487.50 root inner join, left key:test.t2.b, right key:test.t3.b", + " ├─Sort(Build) 9990.00 root test.t3.b", + " │ └─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─Sort(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3) merge_join(t3) */ * from t join t1 on t.a = t1.a join t2 on t1.b = t2.b join t3 on t2.b = t3.b ;", + "Plan": [ + "Projection 19492.21 root test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 19492.21 root inner join, equal:[eq(test.t1.a, test.t.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t2.b, test.t1.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", + " └─MergeJoin(Probe) 12487.50 root inner join, left key:test.t3.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.t3.b", + " └─TableReader 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3) merge_join(t2) */ * from t right join t1 on t.a = t1.a join t2 on t1.b = t2.b join t3 on t2.b = t3.b ;", + "Plan": [ + "Projection 19511.72 root test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 19511.72 root right outer join, equal:[eq(test.t.a, test.t1.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15609.38 root inner join, equal:[eq(test.t2.b, test.t1.b)]", + " ├─TableReader(Build) 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", + " └─MergeJoin(Probe) 12487.50 root inner join, left key:test.t3.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.t3.b", + " └─TableReader 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3) merge_join(t2) */ * from t right join t1 on t.a = t1.a join t2 on t1.b = t2.b join t3 on t2.b = t3.b ;", + "Plan": [ + "Projection 19511.72 root test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 19511.72 root right outer join, equal:[eq(test.t.a, test.t1.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15609.38 root inner join, equal:[eq(test.t2.b, test.t1.b)]", + " ├─TableReader(Build) 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", + " └─MergeJoin(Probe) 12487.50 root inner join, left key:test.t3.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.t3.b", + " └─TableReader 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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 ;", + "Plan": [ + "Projection 19511.72 root test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 19511.72 root right outer join, equal:[eq(test.t.a, test.t1.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15609.38 root inner join, equal:[eq(test.t2.b, test.t1.b)]", + " ├─TableReader(Build) 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", + " └─MergeJoin(Probe) 12487.50 root inner join, left key:test.t3.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.t3.b", + " └─TableReader 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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 ;", + "Plan": [ + "Projection 19492.21 root test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 19492.21 root inner join, equal:[eq(test.t1.a, test.t.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t2.b, test.t1.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", + " └─MergeJoin(Probe) 12487.50 root inner join, left key:test.t3.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.t3.b", + " └─TableReader 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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 ;", + "Plan": [ + "Projection 19492.21 root test.t.a, test.t.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 19492.21 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t1.a, test.t.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t1.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", + " └─TableReader(Probe) 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" + ], + "Warning": [ + "Warning 1815 Optimizer Hint /*+ INL_JOIN(t1) */ or /*+ TIDB_INLJ(t1) */ is inapplicable", + "Warning 1815 Optimizer Hint /*+ INL_JOIN(t1) */ or /*+ TIDB_INLJ(t1) */ is inapplicable" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 12475.01 root semi join, equal:[eq(test.t1.b, test.t3.b) eq(test.t1.a, test.t3.a)]", + "├─TableReader(Build) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─Projection(Probe) 15593.77 root test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t4.a, test.t4.b", + " └─HashJoin 15593.77 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t4.b, test.t1.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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 9980.01 root semi join, equal:[eq(test.t1.b, test.t3.b) eq(test.t1.a, test.t3.a)]", + "├─TableReader(Build) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12475.01 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 9980.01 root semi join, equal:[eq(test.t1.b, test.t3.b) eq(test.t1.a, test.t3.a)]", + "├─TableReader(Build) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 12475.01 root anti semi join, equal:[eq(test.t1.b, test.t3.b)], other cond:eq(test.t1.a, test.t3.a)", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 15593.77 root right outer join, equal:[eq(test.t1.b, test.t4.b)]", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 9990.00 root anti semi join, equal:[eq(test.t1.b, test.t3.b)], other cond:eq(test.t1.a, test.t3.a)", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─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 leading hint is inapplicable, check the join type or the join algorithm hint" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 9990.00 root anti semi join, equal:[eq(test.t1.b, test.t3.b)], other cond:eq(test.t1.a, test.t3.a)", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root right outer join, equal:[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", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 12475.01 root semi join, equal:[eq(test.t1.b, test.t3.b)]", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─Projection(Probe) 15593.77 root test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t4.a, test.t4.b", + " └─HashJoin 15593.77 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t4.b, test.t1.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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 9990.00 root semi join, equal:[eq(test.t1.b, test.t3.b)]", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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": [ + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 9980.01 root semi join, equal:[eq(test.t1.b, test.t3.b)]", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 12475.01 root anti semi join, equal:[eq(test.t1.b, test.t3.b)]", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 15593.77 root right outer join, equal:[eq(test.t1.b, test.t4.b)]", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 9990.00 root anti semi join, equal:[eq(test.t1.b, test.t3.b)]", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─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 leading hint is inapplicable, check the join type or the join algorithm hint" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 9990.00 root anti semi join, equal:[eq(test.t1.b, test.t3.b)]", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root right outer join, equal:[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", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "Projection 1.00 root test.t1.a, Column#14->Column#16", + "└─Apply 1.00 root CARTESIAN left outer join", + " ├─HashAgg(Build) 1.00 root funcs:min(test.t1.a)->Column#10, funcs:firstrow(test.t1.a)->test.t1.a", + " │ └─MergeJoin 12487.50 root inner join, left key:test.t1.a, right key:test.t3.a", + " │ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + " │ │ └─IndexFullScan 9990.00 cop[tikv] table:t3, index:a(a) keep order:true, stats:pseudo", + " │ └─IndexReader(Probe) 9990.00 root index:IndexFullScan", + " │ └─IndexFullScan 9990.00 cop[tikv] table:t1, index:a(a) keep order:true, stats:pseudo", + " └─MaxOneRow(Probe) 1.00 root ", + " └─Projection 2.00 root Column#10->Column#14", + " └─IndexReader 2.00 root index:Selection", + " └─Selection 2.00 cop[tikv] gt(test.t2.a, test.t1.a)", + " └─IndexFullScan 2.50 cop[tikv] table:t2, index:a(a) keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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;", + "Plan": [ + "Projection 1.00 root test.t1.a, Column#14->Column#16", + "└─Apply 1.00 root CARTESIAN left outer join", + " ├─HashAgg(Build) 1.00 root funcs:min(test.t1.a)->Column#10, funcs:firstrow(test.t1.a)->test.t1.a", + " │ └─MergeJoin 12487.50 root inner join, left key:test.t1.a, right key:test.t3.a", + " │ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + " │ │ └─IndexFullScan 9990.00 cop[tikv] table:t3, index:a(a) keep order:true, stats:pseudo", + " │ └─IndexReader(Probe) 9990.00 root index:IndexFullScan", + " │ └─IndexFullScan 9990.00 cop[tikv] table:t1, index:a(a) keep order:true, stats:pseudo", + " └─MaxOneRow(Probe) 1.00 root ", + " └─Projection 2.00 root Column#10->Column#14", + " └─IndexReader 2.00 root index:Selection", + " └─Selection 2.00 cop[tikv] gt(test.t2.a, test.t1.a)", + " └─IndexFullScan 2.50 cop[tikv] table:t2, index:a(a) keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t3) */. Maybe you can use the table alias name" + ] + }, + { + "SQL": "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;", + "Plan": [ + "Projection 1.00 root test.t1.a, Column#14->Column#16", + "└─Apply 1.00 root CARTESIAN left outer join", + " ├─HashAgg(Build) 1.00 root funcs:min(test.t1.a)->Column#10, funcs:firstrow(test.t1.a)->test.t1.a", + " │ └─MergeJoin 12487.50 root inner join, left key:test.t1.a, right key:test.t3.a", + " │ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + " │ │ └─IndexFullScan 9990.00 cop[tikv] table:t3, index:a(a) keep order:true, stats:pseudo", + " │ └─IndexReader(Probe) 9990.00 root index:IndexFullScan", + " │ └─IndexFullScan 9990.00 cop[tikv] table:t1, index:a(a) keep order:true, stats:pseudo", + " └─MaxOneRow(Probe) 1.00 root ", + " └─Projection 2.00 root Column#10->Column#14", + " └─IndexReader 2.00 root index:Selection", + " └─Selection 2.00 cop[tikv] gt(test.t2.a, test.t1.a)", + " └─IndexFullScan 2.50 cop[tikv] table:t2, index:a(a) keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t2, t3) in optimizer hint /*+ LEADING(t2, t3) */. Maybe you can use the table alias name" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 12487.50 root inner join, equal:[eq(test.t1.b, test.t3.b) eq(test.t1.a, Column#13)]", + "├─Selection(Build) 6393.60 root not(isnull(Column#13))", + "│ └─HashAgg 7992.00 root group by:test.t3.b, funcs:max(Column#26)->Column#13, funcs:firstrow(test.t3.b)->test.t3.b", + "│ └─TableReader 7992.00 root data:HashAgg", + "│ └─HashAgg 7992.00 cop[tikv] group by:test.t3.b, funcs:max(test.t3.a)->Column#26", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t1.b, test.t4.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t4, t3) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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)", + "Plan": [ + "Projection 12487.50 root test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t4.a, test.t4.b", + "└─HashJoin 12487.50 root inner join, equal:[eq(test.t1.b, test.t3.b) eq(test.t1.a, Column#13)]", + " ├─Selection(Build) 6393.60 root not(isnull(Column#13))", + " │ └─HashAgg 7992.00 root group by:test.t3.b, funcs:max(Column#19)->Column#13, funcs:firstrow(test.t3.b)->test.t3.b", + " │ └─TableReader 7992.00 root data:HashAgg", + " │ └─HashAgg 7992.00 cop[tikv] group by:test.t3.b, funcs:max(test.t3.a)->Column#19", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t4.b, test.t1.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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 12487.50 root inner join, equal:[eq(test.t1.b, test.t3.b) eq(test.t1.a, Column#13)]", + "├─Selection(Build) 6393.60 root not(isnull(Column#13))", + "│ └─HashAgg 7992.00 root group by:test.t3.b, funcs:max(Column#23)->Column#13, funcs:firstrow(test.t3.b)->test.t3.b", + "│ └─TableReader 7992.00 root data:HashAgg", + "│ └─HashAgg 7992.00 cop[tikv] group by:test.t3.b, funcs:max(test.t3.a)->Column#23", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t1.b, test.t4.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t3) */. Maybe you can use the table alias name" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 12487.50 root inner join, equal:[eq(test.t1.b, test.t3.b) eq(test.t1.a, Column#13)]", + "├─Selection(Build) 6393.60 root not(isnull(Column#13))", + "│ └─HashAgg 7992.00 root group by:test.t3.b, funcs:max(Column#23)->Column#13, funcs:firstrow(test.t3.b)->test.t3.b", + "│ └─TableReader 7992.00 root data:HashAgg", + "│ └─HashAgg 7992.00 cop[tikv] group by:test.t3.b, funcs:max(test.t3.a)->Column#23", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t1.b, test.t4.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t3, t1) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 12487.50 root inner join, equal:[eq(test.t1.b, test.t3.b)], other cond:gt(test.t1.a, Column#13)", + "├─Selection(Build) 6393.60 root not(isnull(Column#13))", + "│ └─HashAgg 7992.00 root group by:test.t3.b, funcs:min(Column#23)->Column#13, funcs:firstrow(test.t3.b)->test.t3.b", + "│ └─TableReader 7992.00 root data:HashAgg", + "│ └─HashAgg 7992.00 cop[tikv] group by:test.t3.b, funcs:min(test.t3.a)->Column#23", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t1.b, test.t4.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t4, t3) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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)", + "Plan": [ + "Projection 12487.50 root test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t4.a, test.t4.b", + "└─HashJoin 12487.50 root inner join, equal:[eq(test.t1.b, test.t3.b)], other cond:gt(test.t1.a, Column#13)", + " ├─Selection(Build) 6393.60 root not(isnull(Column#13))", + " │ └─HashAgg 7992.00 root group by:test.t3.b, funcs:min(Column#19)->Column#13, funcs:firstrow(test.t3.b)->test.t3.b", + " │ └─TableReader 7992.00 root data:HashAgg", + " │ └─HashAgg 7992.00 cop[tikv] group by:test.t3.b, funcs:min(test.t3.a)->Column#19", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 15593.77 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t4.b, test.t1.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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 12487.50 root inner join, equal:[eq(test.t1.b, test.t3.b)], other cond:gt(test.t1.a, Column#13)", + "├─Selection(Build) 6393.60 root not(isnull(Column#13))", + "│ └─HashAgg 7992.00 root group by:test.t3.b, funcs:min(Column#23)->Column#13, funcs:firstrow(test.t3.b)->test.t3.b", + "│ └─TableReader 7992.00 root data:HashAgg", + "│ └─HashAgg 7992.00 cop[tikv] group by:test.t3.b, funcs:min(test.t3.a)->Column#23", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t1.b, test.t4.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t3) */. Maybe you can use the table alias name" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 12487.50 root inner join, equal:[eq(test.t1.b, test.t3.b)], other cond:gt(test.t1.a, Column#13)", + "├─Selection(Build) 6393.60 root not(isnull(Column#13))", + "│ └─HashAgg 7992.00 root group by:test.t3.b, funcs:min(Column#26)->Column#13, funcs:firstrow(test.t3.b)->test.t3.b", + "│ └─TableReader 7992.00 root data:HashAgg", + "│ └─HashAgg 7992.00 cop[tikv] group by:test.t3.b, funcs:min(test.t3.a)->Column#26", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t1.b, test.t4.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t3, t2) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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)", + "Plan": [ + "Projection 15593.77 root test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t4.a, test.t4.b", + "└─HashJoin 15593.77 root inner join, equal:[eq(test.t1.a, test.t3.a)]", + " ├─StreamAgg(Build) 7992.00 root group by:test.t3.a, funcs:firstrow(test.t3.a)->test.t3.a", + " │ └─IndexReader 7992.00 root index:StreamAgg", + " │ └─StreamAgg 7992.00 cop[tikv] group by:test.t3.a, ", + " │ └─IndexFullScan 9990.00 cop[tikv] table:t3, index:a(a) keep order:true, stats:pseudo", + " └─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t4.b, test.t1.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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3@sel_2) */ * from t1 left join t2 on t1.a=t2.a where t1.a in (select t3.a from t3)", + "Plan": [ + "HashJoin 12487.50 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + "├─HashJoin(Build) 9990.00 root inner join, equal:[eq(test.t3.a, test.t1.a)]", + "│ ├─StreamAgg(Build) 7992.00 root group by:test.t3.a, funcs:firstrow(test.t3.a)->test.t3.a", + "│ │ └─IndexReader 7992.00 root index:StreamAgg", + "│ │ └─StreamAgg 7992.00 cop[tikv] group by:test.t3.a, ", + "│ │ └─IndexFullScan 9990.00 cop[tikv] table:t3, index:a(a) keep order:true, stats:pseudo", + "│ └─TableReader(Probe) 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", + "└─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2, t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where t1.a in (select t3.a from t3)", + "Plan": [ + "HashJoin 12487.50 root inner join, equal:[eq(test.t1.a, test.t3.a)]", + "├─StreamAgg(Build) 7992.00 root group by:test.t3.a, funcs:firstrow(test.t3.a)->test.t3.a", + "│ └─IndexReader 7992.00 root index:StreamAgg", + "│ └─StreamAgg 7992.00 cop[tikv] group by:test.t3.a, ", + "│ └─IndexFullScan 9990.00 cop[tikv] table:t3, index:a(a) keep order:true, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": [ + "Warning 1815 We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 12487.50 root Null-aware anti semi join, equal:[eq(test.t1.a, test.t3.a)]", + "├─IndexReader(Build) 10000.00 root index:IndexFullScan", + "│ └─IndexFullScan 10000.00 cop[tikv] table:t3, index:a(a) keep order:false, stats:pseudo", + "└─HashJoin(Probe) 15609.38 root right outer join, equal:[eq(test.t1.b, test.t4.b)]", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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 /*+ leading(t3@sel_2) */ * from t1 left join t2 on t1.a=t2.a where t1.a not in (select t3.a from t3)", + "Plan": [ + "HashJoin 9990.00 root Null-aware anti semi join, equal:[eq(test.t1.a, test.t3.a)]", + "├─IndexReader(Build) 10000.00 root index:IndexFullScan", + "│ └─IndexFullScan 10000.00 cop[tikv] table:t3, index:a(a) keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─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 leading hint is inapplicable, check the join type or the join algorithm hint" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 9990.00 root Null-aware anti semi join, equal:[eq(test.t1.a, test.t3.a)]", + "├─IndexReader(Build) 10000.00 root index:IndexFullScan", + "│ └─IndexFullScan 10000.00 cop[tikv] table:t3, index:a(a) keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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)", + "Plan": [ + "TableDual 0.00 root rows:0" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3@sel_2) */ * from t1 left join t2 on t1.a=t2.a where exists (select t3.a from t3)", + "Plan": [ + "TableDual 0.00 root rows:0" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t3) */. Maybe you can use the table alias name" + ] + }, + { + "SQL": "select /*+ leading(t2, t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where exists (select t3.a from t3)", + "Plan": [ + "TableDual 0.00 root rows:0" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t2, t3) */. Maybe you can use the table alias name" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 15593.77 root right outer join, equal:[eq(test.t1.b, test.t4.b)]", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3@sel_2) */ * from t1 left join t2 on t1.a=t2.a where not exists (select t3.a from t3)", + "Plan": [ + "HashJoin 12487.50 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + "│ └─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 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t3) */. Maybe you can use the table alias name" + ] + }, + { + "SQL": "select /*+ leading(t2, t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where not exists (select t3.a from t3)", + "Plan": [ + "HashJoin 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─TableReader(Probe) 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" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t2, t3) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "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;", + "Plan": [ + "Projection 15593.77 root test.t1.a, test.t1.b, test.t4.a, test.t4.b, test.t3.a, test.t3.b", + "└─HashJoin 15593.77 root inner join, equal:[eq(test.t4.b, test.t3.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t4.a, test.t1.a)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t4.a)), not(isnull(test.t4.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t4) in optimizer hint /*+ LEADING(t4, t1) */. Maybe you can use the table alias name" + ] + }, + { + "SQL": "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;", + "Plan": [ + "HashJoin 15609.38 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root right outer join, equal:[eq(test.t4.a, test.t2.a)]", + " ├─TableReader(Build) 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", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t4) in optimizer hint /*+ LEADING(t2, t4) */. Maybe you can use the table alias name" + ] + }, + { + "SQL": "select /*+ leading(t3) */ * from t1 join t2 on t1.a=t2.a right join (select * from t4) t3 on t2.b=t3.b;", + "Plan": [ + "HashJoin 15593.77 root right outer join, equal:[eq(test.t2.b, test.t4.b)]", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + "└─Projection(Probe) 12475.01 root test.t1.a, test.t1.b, test.t2.a, test.t2.b", + " └─HashJoin 12475.01 root inner join, equal:[eq(test.t2.a, test.t1.a)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 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" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t1) */ t1.a, (select min(t2.a) from t2) from t1 join t3 on t1.a = t3.a;", + "Plan": [ + "Projection 12487.50 root test.t1.a, ->Column#14", + "└─MergeJoin 12487.50 root inner join, left key:test.t1.a, right key:test.t3.a", + " ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + " │ └─IndexFullScan 9990.00 cop[tikv] table:t3, index:a(a) keep order:true, stats:pseudo", + " └─IndexReader(Probe) 9990.00 root index:IndexFullScan", + " └─IndexFullScan 9990.00 cop[tikv] table:t1, index:a(a) keep order:true, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t1, t2@sel_2) */ t1.a, (select min(t2.a) from t2) from t1 join t3 on t1.a = t3.a;", + "Plan": [ + "Projection 12487.50 root test.t1.a, ->Column#14", + "└─MergeJoin 12487.50 root inner join, left key:test.t1.a, right key:test.t3.a", + " ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + " │ └─IndexFullScan 9990.00 cop[tikv] table:t3, index:a(a) keep order:true, stats:pseudo", + " └─IndexReader(Probe) 9990.00 root index:IndexFullScan", + " └─IndexFullScan 9990.00 cop[tikv] table:t1, index:a(a) keep order:true, stats:pseudo" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t2) in optimizer hint /*+ LEADING(t1, t2) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t1, t3) */ t1.a, (select min(t2.a) from t2) from t1 join t3 on t1.a = t3.a;", + "Plan": [ + "Projection 12487.50 root test.t1.a, ->Column#14", + "└─MergeJoin 12487.50 root inner join, left key:test.t1.a, right key:test.t3.a", + " ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + " │ └─IndexFullScan 9990.00 cop[tikv] table:t3, index:a(a) keep order:true, stats:pseudo", + " └─IndexReader(Probe) 9990.00 root index:IndexFullScan", + " └─IndexFullScan 9990.00 cop[tikv] table:t1, index:a(a) keep order:true, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2@sel_2, t1) */ t1.a, (select min(t2.a) from t2) from t1 left join t3 on t1.a = t3.a;", + "Plan": [ + "Projection 12487.50 root test.t1.a, ->Column#14", + "└─MergeJoin 12487.50 root left outer join, left key:test.t1.a, right key:test.t3.a", + " ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + " │ └─IndexFullScan 9990.00 cop[tikv] table:t3, index:a(a) keep order:true, stats:pseudo", + " └─IndexReader(Probe) 10000.00 root index:IndexFullScan", + " └─IndexFullScan 10000.00 cop[tikv] table:t1, index:a(a) keep order:true, stats:pseudo" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t2) 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" + ] + }, + { + "SQL": "select /*+ leading(t3, t2@sel_2) */ t1.a, (select min(t2.a) from t2) from t1 right join t3 on t1.a = t3.a;", + "Plan": [ + "Projection 12487.50 root test.t1.a, ->Column#14", + "└─MergeJoin 12487.50 root right outer join, left key:test.t1.a, right key:test.t3.a", + " ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + " │ └─IndexFullScan 9990.00 cop[tikv] table:t1, index:a(a) keep order:true, stats:pseudo", + " └─IndexReader(Probe) 10000.00 root index:IndexFullScan", + " └─IndexFullScan 10000.00 cop[tikv] table:t3, index:a(a) keep order:true, stats:pseudo" + ], + "Warning": [ + "Warning 1815 There are no matching table names for (t2) in optimizer hint /*+ LEADING(t3, t2) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + } + ] + } +] diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 028c83d180928..55e36462541bf 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1953,6 +1953,7 @@ func (p *LogicalJoin) tryToGetIndexJoin(prop *property.PhysicalProperty) (indexJ } } if supportRightOuter { +<<<<<<< HEAD allRightOuterJoins = p.getIndexJoinByOuterIdx(prop, 1) forcedRightOuterJoins = make([]PhysicalPlan, 0, len(allRightOuterJoins)) for _, j := range allRightOuterJoins { @@ -1970,6 +1971,64 @@ func (p *LogicalJoin) tryToGetIndexJoin(prop *property.PhysicalProperty) (indexJ forcedRightOuterJoins = append(forcedRightOuterJoins, j) } } +======= + 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 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, + preferLeftAsINLJInner, preferLeftAsINLHJInner, preferLeftAsINLMJInner) { + return candidates, false // no force index join hints + } + forced := make([]PhysicalPlan, 0, len(candidates)) + for _, candidate := range candidates { + innerSide, joinMethod, ok := p.getIndexJoinSideAndMethod(candidate) + if !ok { + continue +>>>>>>> c266a9791c5 (planner: support `no_index_join`, `no_index_hash_join`, `no_index_merge_join` hints (#45633)) } switch { case len(forcedRightOuterJoins) == 0 && !supportLeftOuter: diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index ff42798e77467..e58b01c4bec3d 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -93,6 +93,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. @@ -657,6 +663,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 @@ -3698,7 +3728,12 @@ func (b *PlanBuilder) pushTableHints(hints []*ast.TableOptimizerHint, currentLev hints = b.hintProcessor.GetCurrentStmtHints(hints, currentLevel) var ( sortMergeTables, inljTables, inlhjTables, inlmjTables, hashJoinTables, bcTables []hintTableInfo +<<<<<<< HEAD noHashJoinTables []hintTableInfo +======= + noIndexJoinTables, noIndexHashJoinTables, noIndexMergeJoinTables []hintTableInfo + noHashJoinTables, noMergeJoinTables []hintTableInfo +>>>>>>> c266a9791c5 (planner: support `no_index_join`, `no_index_hash_join`, `no_index_merge_join` hints (#45633)) shuffleJoinTables []hintTableInfo indexHintList, indexMergeHintList []indexHintInfo tiflashTables, tikvTables []hintTableInfo @@ -3738,6 +3773,17 @@ func (b *PlanBuilder) pushTableHints(hints []*ast.TableOptimizerHint, currentLev 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)...) +<<<<<<< HEAD +======= + 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)...) +>>>>>>> c266a9791c5 (planner: support `no_index_join`, `no_index_hash_join`, `no_index_merge_join` hints (#45633)) case HintMPP1PhaseAgg: aggHints.preferAggType |= preferMPP1PhaseAgg case HintMPP2PhaseAgg: @@ -3847,6 +3893,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, indexHintList: indexHintList, @@ -7064,6 +7111,13 @@ func getInnerFromParenthesesAndUnaryPlus(expr ast.ExprNode) ast.ExprNode { // join types. func containDifferentJoinTypes(preferJoinType uint) bool { preferJoinType &= ^preferNoHashJoin +<<<<<<< HEAD +======= + preferJoinType &= ^preferNoMergeJoin + preferJoinType &= ^preferNoIndexJoin + preferJoinType &= ^preferNoIndexHashJoin + preferJoinType &= ^preferNoIndexMergeJoin +>>>>>>> c266a9791c5 (planner: support `no_index_join`, `no_index_hash_join`, `no_index_merge_join` hints (#45633)) inlMask := preferRightAsINLJInner ^ preferLeftAsINLJInner inlhjMask := preferRightAsINLHJInner ^ preferLeftAsINLHJInner diff --git a/planner/core/logical_plans.go b/planner/core/logical_plans.go index 5156d519631dd..fa09a561d64a1 100644 --- a/planner/core/logical_plans.go +++ b/planner/core/logical_plans.go @@ -119,6 +119,13 @@ const ( preferHashJoin preferNoHashJoin preferMergeJoin +<<<<<<< HEAD +======= + preferNoMergeJoin + preferNoIndexJoin + preferNoIndexHashJoin + preferNoIndexMergeJoin +>>>>>>> c266a9791c5 (planner: support `no_index_join`, `no_index_hash_join`, `no_index_merge_join` hints (#45633)) preferBCJoin preferShuffleJoin preferRewriteSemiJoin diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 1257e82b3e91c..cfb183e2c34aa 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -89,6 +89,7 @@ type indexNestedLoopJoinTables struct { type tableHintInfo struct { indexNestedLoopJoinTables + noIndexJoinTables indexNestedLoopJoinTables sortMergeJoinTables []hintTableInfo broadcastJoinTables []hintTableInfo shuffleJoinTables []hintTableInfo @@ -262,6 +263,18 @@ func (info *tableHintInfo) ifPreferINLMJ(tableNames ...*hintTableInfo) bool { return info.matchTableName(tableNames, info.indexNestedLoopJoinTables.inlmjTables) } +func (info *tableHintInfo) ifPreferNoIndexJoin(tableNames ...*hintTableInfo) bool { + return info.matchTableName(tableNames, info.noIndexJoinTables.inljTables) +} + +func (info *tableHintInfo) ifPreferNoIndexHashJoin(tableNames ...*hintTableInfo) bool { + return info.matchTableName(tableNames, info.noIndexJoinTables.inlhjTables) +} + +func (info *tableHintInfo) ifPreferNoIndexMergeJoin(tableNames ...*hintTableInfo) bool { + return info.matchTableName(tableNames, info.noIndexJoinTables.inlmjTables) +} + func (info *tableHintInfo) ifPreferTiFlash(tableName *hintTableInfo) *hintTableInfo { if tableName == nil { return nil