diff --git a/executor/merge_join.go b/executor/merge_join.go index 7939d128d9948..46bf9bf0b5e53 100644 --- a/executor/merge_join.go +++ b/executor/merge_join.go @@ -88,7 +88,7 @@ type mergeJoinInnerTable struct { } func (t *mergeJoinInnerTable) init(ctx context.Context, chk4Reader *chunk.Chunk) (err error) { - if t.reader == nil || t.joinKeys == nil || len(t.joinKeys) == 0 || ctx == nil { + if t.reader == nil || ctx == nil { return errors.Errorf("Invalid arguments: Empty arguments detected.") } t.ctx = ctx diff --git a/executor/merge_join_test.go b/executor/merge_join_test.go index 899164094630c..fce712be0dbc1 100644 --- a/executor/merge_join_test.go +++ b/executor/merge_join_test.go @@ -236,7 +236,6 @@ func checkPlanAndRun(tk *testkit.TestKit, c *C, plan string, sql string) *testki } func (s *testSuite) TestMergeJoin(c *C) { - // FIXME: the TIDB_SMJ hint does not really work when there is no index on join onCondition. tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -331,6 +330,25 @@ func (s *testSuite) TestMergeJoin(c *C) { tk.MustExec("create table s(a int, primary key(a))") tk.MustExec("insert into s value(1)") tk.MustQuery("select /*+ TIDB_SMJ(t, s) */ count(*) from t join s on t.a = s.a").Check(testkit.Rows("4")) + + // Test TIDB_SMJ for cartesian product. + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int)") + tk.MustExec("insert into t value(1),(2)") + tk.MustQuery("explain select /*+ TIDB_SMJ(t1, t2) */ * from t t1 join t t2 order by t1.a, t2.a").Check(testkit.Rows( + "Sort_6 100000000.00 root t1.a:asc, t2.a:asc", + "└─MergeJoin_9 100000000.00 root inner join", + " ├─TableReader_11 10000.00 root data:TableScan_10", + " │ └─TableScan_10 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", + " └─TableReader_13 10000.00 root data:TableScan_12", + " └─TableScan_12 10000.00 cop table:t2, range:[-inf,+inf], keep order:false, stats:pseudo", + )) + tk.MustQuery("select /*+ TIDB_SMJ(t1, t2) */ * from t t1 join t t2 order by t1.a, t2.a").Check(testkit.Rows( + "1 1", + "1 2", + "2 1", + "2 2", + )) } func (s *testSuite) Test3WaysMergeJoin(c *C) { diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index dd076d3f9ce52..5f9b181369772 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -588,12 +588,16 @@ func (p *LogicalJoin) buildFakeEqCondsForIndexJoin(keys, idxCols []*expression.C // tryToGetIndexJoin will get index join by hints. If we can generate a valid index join by hint, the second return value // will be true, which means we force to choose this index join. Otherwise we will select a join algorithm with min-cost. func (p *LogicalJoin) tryToGetIndexJoin(prop *property.PhysicalProperty) ([]PhysicalPlan, bool) { - if len(p.EqualConditions) == 0 { - return nil, false - } plans := make([]PhysicalPlan, 0, 2) rightOuter := (p.preferJoinType & preferLeftAsIndexInner) > 0 leftOuter := (p.preferJoinType & preferRightAsIndexInner) > 0 + if len(p.EqualConditions) == 0 { + if leftOuter || rightOuter { + warning := ErrInternal.GenWithStack("TIDB_INLJ hint is inapplicable without column equal ON condition") + p.ctx.GetSessionVars().StmtCtx.AppendWarning(warning) + } + return nil, false + } switch p.JoinType { case SemiJoin, AntiSemiJoin, LeftOuterSemiJoin, AntiLeftOuterSemiJoin, LeftOuterJoin: join := p.getIndexJoinByOuterIdx(prop, 0) diff --git a/planner/core/physical_plan_test.go b/planner/core/physical_plan_test.go index eb21d01215066..b5d0246388ac6 100644 --- a/planner/core/physical_plan_test.go +++ b/planner/core/physical_plan_test.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/util/testleak" "golang.org/x/net/context" ) @@ -1280,3 +1281,27 @@ func (s *testPlanSuite) TestRequestTypeSupportedOff(c *C) { c.Assert(err, IsNil) c.Assert(core.ToString(p), Equals, expect, Commentf("for %s", sql)) } + +func (s *testPlanSuite) TestIndexLookupCartesianJoin(c *C) { + defer testleak.AfterTest(c)() + store, dom, err := newStoreWithBootstrap() + c.Assert(err, IsNil) + defer func() { + dom.Close() + store.Close() + }() + se, err := session.CreateSession4Test(store) + c.Assert(err, IsNil) + _, err = se.Execute(context.Background(), "use test") + c.Assert(err, IsNil) + sql := "select /*+ TIDB_INLJ(t1, t2) */ * from t t1 join t t2" + stmt, err := s.ParseOneStmt(sql, "", "") + c.Assert(err, IsNil) + p, err := core.Optimize(se, stmt, s.is) + c.Assert(err, IsNil) + c.Assert(core.ToString(p), Equals, "LeftHashJoin{TableReader(Table(t))->TableReader(Table(t))}") + warnings := se.GetSessionVars().StmtCtx.GetWarnings() + lastWarn := warnings[len(warnings)-1] + err = core.ErrInternal.GenWithStack("TIDB_INLJ hint is inapplicable without column equal ON condition") + c.Assert(terror.ErrorEqual(err, lastWarn.Err), IsTrue) +}