From 83a054c00be7e499f83ddcc2db3c6b7fdcc948b1 Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Thu, 5 May 2022 15:42:17 +0800 Subject: [PATCH 01/36] planner: refactor the join reorder codes --- planner/core/rule_join_reorder.go | 68 +++++++++++++++++++----- planner/core/rule_join_reorder_dp.go | 15 +----- planner/core/rule_join_reorder_greedy.go | 37 ++----------- 3 files changed, 61 insertions(+), 59 deletions(-) diff --git a/planner/core/rule_join_reorder.go b/planner/core/rule_join_reorder.go index 3f2065650ff33..c10d171d19fcb 100644 --- a/planner/core/rule_join_reorder.go +++ b/planner/core/rule_join_reorder.go @@ -21,6 +21,7 @@ import ( "sort" "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/util/plancodec" "github.com/pingcap/tidb/util/tracing" @@ -80,23 +81,29 @@ func (s *joinReOrderSolver) optimizeRecursive(ctx sessionctx.Context, p LogicalP return nil, err } } + originalSchema := p.Schema() + baseGroupSolver := &baseSingleGroupJoinOrderSolver{ ctx: ctx, otherConds: otherConds, + eqEdges: eqEdges, } - originalSchema := p.Schema() + err = baseGroupSolver.deriveStatsAndGenerateJRNodeGroup(curJoinGroup, tracer) + if err != nil { + return nil, err + } + if len(curJoinGroup) > ctx.GetSessionVars().TiDBOptJoinReorderThreshold { groupSolver := &joinReorderGreedySolver{ baseSingleGroupJoinOrderSolver: baseGroupSolver, - eqEdges: eqEdges, } - p, err = groupSolver.solve(curJoinGroup, tracer) + p, err = groupSolver.solve(tracer) } else { dpSolver := &joinReorderDPSolver{ baseSingleGroupJoinOrderSolver: baseGroupSolver, } dpSolver.newJoin = dpSolver.newJoinWithEdges - p, err = dpSolver.solve(curJoinGroup, expression.ScalarFuncs2Exprs(eqEdges), tracer) + p, err = dpSolver.solve(curJoinGroup, tracer) } if err != nil { return nil, err @@ -139,6 +146,24 @@ type baseSingleGroupJoinOrderSolver struct { ctx sessionctx.Context curJoinGroup []*jrNode otherConds []expression.Expression + eqEdges []*expression.ScalarFunction +} + +func (s *baseSingleGroupJoinOrderSolver) deriveStatsAndGenerateJRNodeGroup(joinNodePlans []LogicalPlan, tracer *joinReorderTrace) error { + s.curJoinGroup = make([]*jrNode, 0, len(joinNodePlans)) + for _, node := range joinNodePlans { + _, err := node.recursiveDeriveStats(nil) + if err != nil { + return err + } + cost := s.baseNodeCumCost(node) + s.curJoinGroup = append(s.curJoinGroup, &jrNode{ + p: node, + cumCost: cost, + }) + tracer.appendLogicalJoinCost(node, cost) + } + return nil } // baseNodeCumCost calculate the cumulative cost of the node in the join group. @@ -150,6 +175,31 @@ func (s *baseSingleGroupJoinOrderSolver) baseNodeCumCost(groupNode LogicalPlan) return cost } +func (s *baseSingleGroupJoinOrderSolver) checkConnection(lChild, rChild LogicalPlan) (usedEdges []*expression.ScalarFunction) { + for _, edge := range s.eqEdges { + lCol := edge.GetArgs()[0].(*expression.Column) + rCol := edge.GetArgs()[1].(*expression.Column) + if lChild.Schema().Contains(lCol) && rChild.Schema().Contains(rCol) { + usedEdges = append(usedEdges, edge) + } else if rChild.Schema().Contains(lCol) && lChild.Schema().Contains(rCol) { + newSf := expression.NewFunctionInternal(s.ctx, ast.EQ, edge.GetType(), rCol, lCol).(*expression.ScalarFunction) + usedEdges = append(usedEdges, newSf) + } + } + return +} + +func (s *baseSingleGroupJoinOrderSolver) makeJoin(lChild, rChild LogicalPlan, eqEdges []*expression.ScalarFunction) (LogicalPlan, []expression.Expression) { + remainOtherConds := make([]expression.Expression, len(s.otherConds)) + copy(remainOtherConds, s.otherConds) + var otherConds []expression.Expression + mergedSchema := expression.MergeSchema(lChild.Schema(), rChild.Schema()) + remainOtherConds, otherConds = expression.FilterOutInPlace(remainOtherConds, func(expr expression.Expression) bool { + return expression.ExprFromSchema(expr, mergedSchema) + }) + return s.newJoinWithEdges(lChild, rChild, eqEdges, otherConds), remainOtherConds +} + // makeBushyJoin build bushy tree for the nodes which have no equal condition to connect them. func (s *baseSingleGroupJoinOrderSolver) makeBushyJoin(cartesianJoinGroup []LogicalPlan) LogicalPlan { resultJoinGroup := make([]LogicalPlan, 0, (len(cartesianJoinGroup)+1)/2) @@ -160,14 +210,8 @@ func (s *baseSingleGroupJoinOrderSolver) makeBushyJoin(cartesianJoinGroup []Logi resultJoinGroup = append(resultJoinGroup, cartesianJoinGroup[i]) break } - newJoin := s.newCartesianJoin(cartesianJoinGroup[i], cartesianJoinGroup[i+1]) - for i := len(s.otherConds) - 1; i >= 0; i-- { - cols := expression.ExtractColumns(s.otherConds[i]) - if newJoin.schema.ColumnsIndices(cols) != nil { - newJoin.OtherConditions = append(newJoin.OtherConditions, s.otherConds[i]) - s.otherConds = append(s.otherConds[:i], s.otherConds[i+1:]...) - } - } + newJoin, remainOtherConds := s.makeJoin(cartesianJoinGroup[i], cartesianJoinGroup[i+1], nil) + s.otherConds = remainOtherConds resultJoinGroup = append(resultJoinGroup, newJoin) } cartesianJoinGroup, resultJoinGroup = resultJoinGroup, cartesianJoinGroup diff --git a/planner/core/rule_join_reorder_dp.go b/planner/core/rule_join_reorder_dp.go index 459755de2e2ec..c1ba7daa7bc61 100644 --- a/planner/core/rule_join_reorder_dp.go +++ b/planner/core/rule_join_reorder_dp.go @@ -37,19 +37,8 @@ type joinGroupNonEqEdge struct { expr expression.Expression } -func (s *joinReorderDPSolver) solve(joinGroup []LogicalPlan, eqConds []expression.Expression, tracer *joinReorderTrace) (LogicalPlan, error) { - for _, node := range joinGroup { - _, err := node.recursiveDeriveStats(nil) - if err != nil { - return nil, err - } - cost := s.baseNodeCumCost(node) - s.curJoinGroup = append(s.curJoinGroup, &jrNode{ - p: node, - cumCost: cost, - }) - tracer.appendLogicalJoinCost(node, cost) - } +func (s *joinReorderDPSolver) solve(joinGroup []LogicalPlan, tracer *joinReorderTrace) (LogicalPlan, error) { + eqConds := expression.ScalarFuncs2Exprs(s.eqEdges) adjacents := make([][]int, len(s.curJoinGroup)) totalEqEdges := make([]joinGroupEqEdge, 0, len(eqConds)) addEqEdge := func(node1, node2 int, edgeContent *expression.ScalarFunction) { diff --git a/planner/core/rule_join_reorder_greedy.go b/planner/core/rule_join_reorder_greedy.go index 70b287125e3b9..83ae7864bb686 100644 --- a/planner/core/rule_join_reorder_greedy.go +++ b/planner/core/rule_join_reorder_greedy.go @@ -19,12 +19,10 @@ import ( "sort" "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/parser/ast" ) type joinReorderGreedySolver struct { *baseSingleGroupJoinOrderSolver - eqEdges []*expression.ScalarFunction } // solve reorders the join nodes in the group based on a greedy algorithm. @@ -41,19 +39,7 @@ type joinReorderGreedySolver struct { // // For the nodes and join trees which don't have a join equal condition to // connect them, we make a bushy join tree to do the cartesian joins finally. -func (s *joinReorderGreedySolver) solve(joinNodePlans []LogicalPlan, tracer *joinReorderTrace) (LogicalPlan, error) { - for _, node := range joinNodePlans { - _, err := node.recursiveDeriveStats(nil) - if err != nil { - return nil, err - } - cost := s.baseNodeCumCost(node) - s.curJoinGroup = append(s.curJoinGroup, &jrNode{ - p: node, - cumCost: cost, - }) - tracer.appendLogicalJoinCost(node, cost) - } +func (s *joinReorderGreedySolver) solve(tracer *joinReorderTrace) (LogicalPlan, error) { sort.SliceStable(s.curJoinGroup, func(i, j int) bool { return s.curJoinGroup[i].cumCost < s.curJoinGroup[j].cumCost }) @@ -111,26 +97,9 @@ func (s *joinReorderGreedySolver) constructConnectedJoinTree(tracer *joinReorder } func (s *joinReorderGreedySolver) checkConnectionAndMakeJoin(leftNode, rightNode LogicalPlan) (LogicalPlan, []expression.Expression) { - var usedEdges []*expression.ScalarFunction - remainOtherConds := make([]expression.Expression, len(s.otherConds)) - copy(remainOtherConds, s.otherConds) - for _, edge := range s.eqEdges { - lCol := edge.GetArgs()[0].(*expression.Column) - rCol := edge.GetArgs()[1].(*expression.Column) - if leftNode.Schema().Contains(lCol) && rightNode.Schema().Contains(rCol) { - usedEdges = append(usedEdges, edge) - } else if rightNode.Schema().Contains(lCol) && leftNode.Schema().Contains(rCol) { - newSf := expression.NewFunctionInternal(s.ctx, ast.EQ, edge.GetType(), rCol, lCol).(*expression.ScalarFunction) - usedEdges = append(usedEdges, newSf) - } - } + usedEdges := s.checkConnection(leftNode, rightNode) if len(usedEdges) == 0 { return nil, nil } - var otherConds []expression.Expression - mergedSchema := expression.MergeSchema(leftNode.Schema(), rightNode.Schema()) - remainOtherConds, otherConds = expression.FilterOutInPlace(remainOtherConds, func(expr expression.Expression) bool { - return expression.ExprFromSchema(expr, mergedSchema) - }) - return s.newJoinWithEdges(leftNode, rightNode, usedEdges, otherConds), remainOtherConds + return s.makeJoin(leftNode, rightNode, usedEdges) } From b872695ce0e7c3376b8f8962acc68744ba875b02 Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Thu, 5 May 2022 16:23:38 +0800 Subject: [PATCH 02/36] fix ut --- planner/core/rule_join_reorder_dp_test.go | 33 ++++++++++++++++------- 1 file changed, 23 insertions(+), 10 deletions(-) diff --git a/planner/core/rule_join_reorder_dp_test.go b/planner/core/rule_join_reorder_dp_test.go index ed41d55c43520..5d8079f6503b1 100644 --- a/planner/core/rule_join_reorder_dp_test.go +++ b/planner/core/rule_join_reorder_dp_test.go @@ -178,13 +178,23 @@ func TestDPReorderTPCHQ5(t *testing.T) { eqConds = append(eqConds, expression.NewFunctionInternal(ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), joinGroups[2].Schema().Columns[0], joinGroups[4].Schema().Columns[0])) eqConds = append(eqConds, expression.NewFunctionInternal(ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), joinGroups[3].Schema().Columns[0], joinGroups[4].Schema().Columns[0])) eqConds = append(eqConds, expression.NewFunctionInternal(ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), joinGroups[4].Schema().Columns[0], joinGroups[5].Schema().Columns[0])) + eqEdges := make([]*expression.ScalarFunction, 0, len(eqConds)) + for _, cond := range eqConds { + sf, isSF := cond.(*expression.ScalarFunction) + require.True(t, isSF) + eqEdges = append(eqEdges, sf) + } + baseGroupSolver := &baseSingleGroupJoinOrderSolver{ + ctx: ctx, + eqEdges: eqEdges, + } + err := baseGroupSolver.deriveStatsAndGenerateJRNodeGroup(joinGroups, nil) + require.NoError(t, err) solver := &joinReorderDPSolver{ - baseSingleGroupJoinOrderSolver: &baseSingleGroupJoinOrderSolver{ - ctx: ctx, - }, - newJoin: newMockJoin(ctx, statsMap), + baseSingleGroupJoinOrderSolver: baseGroupSolver, + newJoin: newMockJoin(ctx, statsMap), } - result, err := solver.solve(joinGroups, eqConds, nil) + result, err := solver.solve(joinGroups, nil) require.NoError(t, err) expected := "MockJoin{supplier, MockJoin{lineitem, MockJoin{orders, MockJoin{customer, MockJoin{nation, region}}}}}" @@ -202,13 +212,16 @@ func TestDPReorderAllCartesian(t *testing.T) { joinGroup = append(joinGroup, newDataSource(ctx, "b", 100)) joinGroup = append(joinGroup, newDataSource(ctx, "c", 100)) joinGroup = append(joinGroup, newDataSource(ctx, "d", 100)) + baseGroupSolver := &baseSingleGroupJoinOrderSolver{ + ctx: ctx, + } + err := baseGroupSolver.deriveStatsAndGenerateJRNodeGroup(joinGroup, nil) + require.NoError(t, err) solver := &joinReorderDPSolver{ - baseSingleGroupJoinOrderSolver: &baseSingleGroupJoinOrderSolver{ - ctx: ctx, - }, - newJoin: newMockJoin(ctx, statsMap), + baseSingleGroupJoinOrderSolver: baseGroupSolver, + newJoin: newMockJoin(ctx, statsMap), } - result, err := solver.solve(joinGroup, nil, nil) + result, err := solver.solve(joinGroup, nil) require.NoError(t, err) expected := "MockJoin{MockJoin{a, b}, MockJoin{c, d}}" From 5bf862b6753cfa52522cac0b4ba16a913c1e9c90 Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Wed, 11 May 2022 11:52:16 +0800 Subject: [PATCH 03/36] resolve conflicts --- planner/core/rule_join_reorder.go | 77 ++++++++++++++++++++++-- planner/core/rule_join_reorder_greedy.go | 56 +---------------- 2 files changed, 75 insertions(+), 58 deletions(-) diff --git a/planner/core/rule_join_reorder.go b/planner/core/rule_join_reorder.go index d060c708f954d..1f4fed9a0f4f2 100644 --- a/planner/core/rule_join_reorder.go +++ b/planner/core/rule_join_reorder.go @@ -21,6 +21,7 @@ import ( "sort" "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/util/plancodec" "github.com/pingcap/tidb/util/tracing" @@ -102,7 +103,14 @@ func (s *joinReOrderSolver) optimizeRecursive(ctx sessionctx.Context, p LogicalP baseGroupSolver := &baseSingleGroupJoinOrderSolver{ ctx: ctx, otherConds: otherConds, + eqEdges: eqEdges, + joinTypes: joinTypes, } + err = baseGroupSolver.deriveStatsAndGenerateJRNodeGroup(curJoinGroup, tracer) + if err != nil { + return nil, err + } + originalSchema := p.Schema() // Not support outer join reorder when using the DP algorithm @@ -116,16 +124,14 @@ func (s *joinReOrderSolver) optimizeRecursive(ctx sessionctx.Context, p LogicalP if len(curJoinGroup) > ctx.GetSessionVars().TiDBOptJoinReorderThreshold || !isSupportDP { groupSolver := &joinReorderGreedySolver{ baseSingleGroupJoinOrderSolver: baseGroupSolver, - eqEdges: eqEdges, - joinTypes: joinTypes, } - p, err = groupSolver.solve(curJoinGroup, tracer) + p, err = groupSolver.solve(tracer) } else { dpSolver := &joinReorderDPSolver{ baseSingleGroupJoinOrderSolver: baseGroupSolver, } dpSolver.newJoin = dpSolver.newJoinWithEdges - p, err = dpSolver.solve(curJoinGroup, expression.ScalarFuncs2Exprs(eqEdges), tracer) + p, err = dpSolver.solve(curJoinGroup, tracer) } if err != nil { return nil, err @@ -168,6 +174,25 @@ type baseSingleGroupJoinOrderSolver struct { ctx sessionctx.Context curJoinGroup []*jrNode otherConds []expression.Expression + eqEdges []*expression.ScalarFunction + joinTypes []JoinType +} + +func (s *baseSingleGroupJoinOrderSolver) deriveStatsAndGenerateJRNodeGroup(joinNodePlans []LogicalPlan, tracer *joinReorderTrace) error { + s.curJoinGroup = make([]*jrNode, 0, len(joinNodePlans)) + for _, node := range joinNodePlans { + _, err := node.recursiveDeriveStats(nil) + if err != nil { + return err + } + cost := s.baseNodeCumCost(node) + s.curJoinGroup = append(s.curJoinGroup, &jrNode{ + p: node, + cumCost: cost, + }) + tracer.appendLogicalJoinCost(node, cost) + } + return nil } // baseNodeCumCost calculate the cumulative cost of the node in the join group. @@ -179,7 +204,49 @@ func (s *baseSingleGroupJoinOrderSolver) baseNodeCumCost(groupNode LogicalPlan) return cost } -// makeBushyJoin build bushy tree for the nodes which have no equal condition to connect them. +func (s *baseSingleGroupJoinOrderSolver) checkConnection(leftPlan, rightPlan LogicalPlan) (leftNode, rightNode LogicalPlan, usedEdges []*expression.ScalarFunction, joinType JoinType) { + joinType = InnerJoin + leftNode, rightNode = leftPlan, rightPlan + for idx, edge := range s.eqEdges { + lCol := edge.GetArgs()[0].(*expression.Column) + rCol := edge.GetArgs()[1].(*expression.Column) + if leftPlan.Schema().Contains(lCol) && rightPlan.Schema().Contains(rCol) { + joinType = s.joinTypes[idx] + usedEdges = append(usedEdges, edge) + } else if rightPlan.Schema().Contains(lCol) && leftPlan.Schema().Contains(rCol) { + joinType = s.joinTypes[idx] + if joinType != InnerJoin { + rightNode, leftNode = leftPlan, rightPlan + usedEdges = append(usedEdges, edge) + } else { + newSf := expression.NewFunctionInternal(s.ctx, ast.EQ, edge.GetType(), rCol, lCol).(*expression.ScalarFunction) + usedEdges = append(usedEdges, newSf) + } + } + } + return +} + +func (s *baseSingleGroupJoinOrderSolver) makeJoin(leftPlan, rightPlan LogicalPlan, eqEdges []*expression.ScalarFunction, joinType JoinType) (LogicalPlan, []expression.Expression) { + remainOtherConds := make([]expression.Expression, len(s.otherConds)) + copy(remainOtherConds, s.otherConds) + var otherConds []expression.Expression + var leftConds []expression.Expression + var rightConds []expression.Expression + mergedSchema := expression.MergeSchema(leftPlan.Schema(), rightPlan.Schema()) + + remainOtherConds, leftConds = expression.FilterOutInPlace(remainOtherConds, func(expr expression.Expression) bool { + return expression.ExprFromSchema(expr, leftPlan.Schema()) && !expression.ExprFromSchema(expr, rightPlan.Schema()) + }) + remainOtherConds, rightConds = expression.FilterOutInPlace(remainOtherConds, func(expr expression.Expression) bool { + return expression.ExprFromSchema(expr, rightPlan.Schema()) && !expression.ExprFromSchema(expr, leftPlan.Schema()) + }) + remainOtherConds, otherConds = expression.FilterOutInPlace(remainOtherConds, func(expr expression.Expression) bool { + return expression.ExprFromSchema(expr, mergedSchema) + }) + return s.newJoinWithEdges(leftPlan, rightPlan, eqEdges, otherConds, leftConds, rightConds, joinType), remainOtherConds +} + func (s *baseSingleGroupJoinOrderSolver) makeBushyJoin(cartesianJoinGroup []LogicalPlan) LogicalPlan { resultJoinGroup := make([]LogicalPlan, 0, (len(cartesianJoinGroup)+1)/2) for len(cartesianJoinGroup) > 1 { diff --git a/planner/core/rule_join_reorder_greedy.go b/planner/core/rule_join_reorder_greedy.go index 525fb76a80d83..85516240b63e4 100644 --- a/planner/core/rule_join_reorder_greedy.go +++ b/planner/core/rule_join_reorder_greedy.go @@ -19,13 +19,10 @@ import ( "sort" "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/parser/ast" ) type joinReorderGreedySolver struct { *baseSingleGroupJoinOrderSolver - eqEdges []*expression.ScalarFunction - joinTypes []JoinType } // solve reorders the join nodes in the group based on a greedy algorithm. @@ -42,20 +39,7 @@ type joinReorderGreedySolver struct { // // For the nodes and join trees which don't have a join equal condition to // connect them, we make a bushy join tree to do the cartesian joins finally. -func (s *joinReorderGreedySolver) solve(joinNodePlans []LogicalPlan, tracer *joinReorderTrace) (LogicalPlan, error) { - for _, node := range joinNodePlans { - _, err := node.recursiveDeriveStats(nil) - if err != nil { - return nil, err - } - cost := s.baseNodeCumCost(node) - s.curJoinGroup = append(s.curJoinGroup, &jrNode{ - p: node, - cumCost: cost, - }) - tracer.appendLogicalJoinCost(node, cost) - } - +func (s *joinReorderGreedySolver) solve(tracer *joinReorderTrace) (LogicalPlan, error) { // Sort plans by cost sort.SliceStable(s.curJoinGroup, func(i, j int) bool { return s.curJoinGroup[i].cumCost < s.curJoinGroup[j].cumCost @@ -114,43 +98,9 @@ func (s *joinReorderGreedySolver) constructConnectedJoinTree(tracer *joinReorder } func (s *joinReorderGreedySolver) checkConnectionAndMakeJoin(leftPlan, rightPlan LogicalPlan) (LogicalPlan, []expression.Expression) { - var usedEdges []*expression.ScalarFunction - remainOtherConds := make([]expression.Expression, len(s.otherConds)) - copy(remainOtherConds, s.otherConds) - joinType := InnerJoin - for idx, edge := range s.eqEdges { - lCol := edge.GetArgs()[0].(*expression.Column) - rCol := edge.GetArgs()[1].(*expression.Column) - if leftPlan.Schema().Contains(lCol) && rightPlan.Schema().Contains(rCol) { - joinType = s.joinTypes[idx] - usedEdges = append(usedEdges, edge) - } else if rightPlan.Schema().Contains(lCol) && leftPlan.Schema().Contains(rCol) { - joinType = s.joinTypes[idx] - if joinType != InnerJoin { - rightPlan, leftPlan = leftPlan, rightPlan - usedEdges = append(usedEdges, edge) - } else { - newSf := expression.NewFunctionInternal(s.ctx, ast.EQ, edge.GetType(), rCol, lCol).(*expression.ScalarFunction) - usedEdges = append(usedEdges, newSf) - } - } - } + leftPlan, rightPlan, usedEdges, joinType := s.checkConnection(leftPlan, rightPlan) if len(usedEdges) == 0 { return nil, nil } - var otherConds []expression.Expression - var leftConds []expression.Expression - var rightConds []expression.Expression - mergedSchema := expression.MergeSchema(leftPlan.Schema(), rightPlan.Schema()) - - remainOtherConds, leftConds = expression.FilterOutInPlace(remainOtherConds, func(expr expression.Expression) bool { - return expression.ExprFromSchema(expr, leftPlan.Schema()) && !expression.ExprFromSchema(expr, rightPlan.Schema()) - }) - remainOtherConds, rightConds = expression.FilterOutInPlace(remainOtherConds, func(expr expression.Expression) bool { - return expression.ExprFromSchema(expr, rightPlan.Schema()) && !expression.ExprFromSchema(expr, leftPlan.Schema()) - }) - remainOtherConds, otherConds = expression.FilterOutInPlace(remainOtherConds, func(expr expression.Expression) bool { - return expression.ExprFromSchema(expr, mergedSchema) - }) - return s.newJoinWithEdges(leftPlan, rightPlan, usedEdges, otherConds, leftConds, rightConds, joinType), remainOtherConds + return s.makeJoin(leftPlan, rightPlan, usedEdges, joinType) } From e7ded101e5aba3caae6b8f1a8bf7e96d74fee6f1 Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Wed, 11 May 2022 13:56:16 +0800 Subject: [PATCH 04/36] add more comments --- planner/core/rule_join_reorder.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/planner/core/rule_join_reorder.go b/planner/core/rule_join_reorder.go index 1f4fed9a0f4f2..d06c8be47f62f 100644 --- a/planner/core/rule_join_reorder.go +++ b/planner/core/rule_join_reorder.go @@ -178,6 +178,7 @@ type baseSingleGroupJoinOrderSolver struct { joinTypes []JoinType } +// deriveStatsAndGenerateJRNodeGroup used to derive the stats for the joinNodePlans and generate the jrNode groups based on the cost. func (s *baseSingleGroupJoinOrderSolver) deriveStatsAndGenerateJRNodeGroup(joinNodePlans []LogicalPlan, tracer *joinReorderTrace) error { s.curJoinGroup = make([]*jrNode, 0, len(joinNodePlans)) for _, node := range joinNodePlans { @@ -204,6 +205,7 @@ func (s *baseSingleGroupJoinOrderSolver) baseNodeCumCost(groupNode LogicalPlan) return cost } +// checkConnection used to check whether two nodes have equal conditions or not. func (s *baseSingleGroupJoinOrderSolver) checkConnection(leftPlan, rightPlan LogicalPlan) (leftNode, rightNode LogicalPlan, usedEdges []*expression.ScalarFunction, joinType JoinType) { joinType = InnerJoin leftNode, rightNode = leftPlan, rightPlan @@ -227,6 +229,7 @@ func (s *baseSingleGroupJoinOrderSolver) checkConnection(leftPlan, rightPlan Log return } +// makeJoin build join tree for the nodes which have equal conditions to connect them. func (s *baseSingleGroupJoinOrderSolver) makeJoin(leftPlan, rightPlan LogicalPlan, eqEdges []*expression.ScalarFunction, joinType JoinType) (LogicalPlan, []expression.Expression) { remainOtherConds := make([]expression.Expression, len(s.otherConds)) copy(remainOtherConds, s.otherConds) @@ -247,6 +250,7 @@ func (s *baseSingleGroupJoinOrderSolver) makeJoin(leftPlan, rightPlan LogicalPla return s.newJoinWithEdges(leftPlan, rightPlan, eqEdges, otherConds, leftConds, rightConds, joinType), remainOtherConds } +// makeBushyJoin build bushy tree for the nodes which have no equal condition to connect them. func (s *baseSingleGroupJoinOrderSolver) makeBushyJoin(cartesianJoinGroup []LogicalPlan) LogicalPlan { resultJoinGroup := make([]LogicalPlan, 0, (len(cartesianJoinGroup)+1)/2) for len(cartesianJoinGroup) > 1 { From 86313872f075371cfa07b244ca7dc73cce656092 Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Wed, 11 May 2022 16:19:53 +0800 Subject: [PATCH 05/36] planner: support leading hints in join reorder optimization --- planner/core/exhaust_physical_plans.go | 2 +- planner/core/expression_rewriter.go | 2 +- planner/core/logical_plan_builder.go | 30 +++++++- planner/core/logical_plans.go | 5 +- planner/core/planbuilder.go | 3 +- planner/core/rule_join_reorder.go | 92 +++++++++++++++++------ planner/core/rule_join_reorder_dp.go | 12 +++ planner/core/rule_join_reorder_dp_test.go | 31 +++----- planner/core/rule_join_reorder_greedy.go | 18 ++++- 9 files changed, 140 insertions(+), 55 deletions(-) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index ef8507470cf2e..5fd3aa18286f1 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1657,7 +1657,7 @@ func (p *LogicalJoin) tryToGetIndexJoin(prop *property.PhysicalProperty) (indexJ case hasINLMJHint: errMsg = "Optimizer Hint INL_MERGE_JOIN is inapplicable" } - if p.hintInfo != nil { + if p.hintInfo != nil && p.preferJoinType > 0 { t := p.hintInfo.indexNestedLoopJoinTables switch { case len(t.inljTables) != 0: diff --git a/planner/core/expression_rewriter.go b/planner/core/expression_rewriter.go index ee5669a45e81f..76ca82e3b1b6b 100644 --- a/planner/core/expression_rewriter.go +++ b/planner/core/expression_rewriter.go @@ -953,7 +953,7 @@ func (er *expressionRewriter) handleInSubquery(ctx context.Context, v *ast.Patte join.AttachOnConds(expression.SplitCNFItems(checkCondition)) // Set join hint for this join. if er.b.TableHints() != nil { - join.setPreferredJoinType(er.b.TableHints()) + join.setPreferredJoinTypeAndOrder(er.b.TableHints()) } er.p = join } else { diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 69773f62d9535..64af57446008c 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -76,6 +76,8 @@ const ( // HintStraightJoin causes TiDB to join tables in the order in which they appear in the FROM clause. HintStraightJoin = "straight_join" + // HintLeading specifies the set of tables to be used as the prefix in the execution plan. + HintLeading = "leading" // TiDBIndexNestedLoopJoin is hint enforce index nested loop join. TiDBIndexNestedLoopJoin = "tidb_inlj" @@ -554,7 +556,7 @@ func extractTableAlias(p Plan, parentOffset int) *hintTableInfo { return nil } -func (p *LogicalJoin) setPreferredJoinType(hintInfo *tableHintInfo) { +func (p *LogicalJoin) setPreferredJoinTypeAndOrder(hintInfo *tableHintInfo) { if hintInfo == nil { return } @@ -594,8 +596,12 @@ func (p *LogicalJoin) setPreferredJoinType(hintInfo *tableHintInfo) { p.ctx.GetSessionVars().StmtCtx.AppendWarning(warning) p.preferJoinType = 0 } + // set the join order + if hintInfo.leadingJoinOrder != nil { + p.preferJoinOrder = hintInfo.matchTableName([]*hintTableInfo{lhsAlias, rhsAlias}, hintInfo.leadingJoinOrder) + } // set hintInfo for further usage if this hint info can be used. - if p.preferJoinType != 0 { + if p.preferJoinType != 0 || p.preferJoinOrder { p.hintInfo = hintInfo } } @@ -767,7 +773,7 @@ func (b *PlanBuilder) buildJoin(ctx context.Context, joinNode *ast.Join) (Logica } // Set preferred join algorithm if some join hints is specified by user. - joinPlan.setPreferredJoinType(b.TableHints()) + joinPlan.setPreferredJoinTypeAndOrder(b.TableHints()) // "NATURAL JOIN" doesn't have "ON" or "USING" conditions. // @@ -3511,12 +3517,14 @@ func (b *PlanBuilder) pushTableHints(hints []*ast.TableOptimizerHint, currentLev aggHints aggHintInfo timeRangeHint ast.HintTimeRange limitHints limitHintInfo + leadingJoinOrder []hintTableInfo + leadingHintCnt int ) for _, hint := range hints { // Set warning for the hint that requires the table name. switch hint.HintName.L { case TiDBMergeJoin, HintSMJ, TiDBIndexNestedLoopJoin, HintINLJ, HintINLHJ, HintINLMJ, - TiDBHashJoin, HintHJ, HintUseIndex, HintIgnoreIndex, HintForceIndex, HintIndexMerge: + TiDBHashJoin, HintHJ, HintUseIndex, HintIgnoreIndex, HintForceIndex, HintIndexMerge, HintLeading: if len(hint.Tables) == 0 { b.pushHintWithoutTableWarning(hint) continue @@ -3613,10 +3621,22 @@ func (b *PlanBuilder) pushTableHints(hints []*ast.TableOptimizerHint, currentLev timeRangeHint = hint.HintData.(ast.HintTimeRange) case HintLimitToCop: limitHints.preferLimitToCop = true + case HintLeading: + if leadingHintCnt == 0 { + leadingJoinOrder = append(leadingJoinOrder, tableNames2HintTableInfo(b.ctx, hint.HintName.L, hint.Tables, b.hintProcessor, currentLevel)...) + } + leadingHintCnt++ default: // ignore hints that not implemented } } + if leadingHintCnt > 1 { + // If there are more leading hints, all leading hints will be invalid. + leadingJoinOrder = leadingJoinOrder[:0] + // Append warning if there are invalid index names. + errMsg := fmt.Sprintf("We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid") + b.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack(errMsg)) + } b.tableHintInfo = append(b.tableHintInfo, tableHintInfo{ sortMergeJoinTables: sortMergeTables, broadcastJoinTables: BCTables, @@ -3629,6 +3649,7 @@ func (b *PlanBuilder) pushTableHints(hints []*ast.TableOptimizerHint, currentLev indexMergeHintList: indexMergeHintList, timeRangeHint: timeRangeHint, limitHints: limitHints, + leadingJoinOrder: leadingJoinOrder, }) } @@ -3649,6 +3670,7 @@ func (b *PlanBuilder) popTableHints() { b.appendUnmatchedJoinHintWarning(HintSMJ, TiDBMergeJoin, hintInfo.sortMergeJoinTables) b.appendUnmatchedJoinHintWarning(HintBCJ, TiDBBroadCastJoin, hintInfo.broadcastJoinTables) b.appendUnmatchedJoinHintWarning(HintHJ, TiDBHashJoin, hintInfo.hashJoinTables) + b.appendUnmatchedJoinHintWarning(HintLeading, "", hintInfo.leadingJoinOrder) b.appendUnmatchedStorageHintWarning(hintInfo.tiflashTables, hintInfo.tikvTables) b.tableHintInfo = b.tableHintInfo[:len(b.tableHintInfo)-1] } diff --git a/planner/core/logical_plans.go b/planner/core/logical_plans.go index 7f89a8269393d..6ad8ce006bba9 100644 --- a/planner/core/logical_plans.go +++ b/planner/core/logical_plans.go @@ -136,8 +136,9 @@ type LogicalJoin struct { StraightJoin bool // hintInfo stores the join algorithm hint information specified by client. - hintInfo *tableHintInfo - preferJoinType uint + hintInfo *tableHintInfo + preferJoinType uint + preferJoinOrder bool EqualConditions []*expression.ScalarFunction LeftConditions expression.CNFExprs diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index f2005a20c40be..c50d780d3fd5d 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -97,6 +97,7 @@ type tableHintInfo struct { indexMergeHintList []indexHintInfo timeRangeHint ast.HintTimeRange limitHints limitHintInfo + leadingJoinOrder []hintTableInfo } type limitHintInfo struct { @@ -182,7 +183,7 @@ func tableNames2HintTableInfo(ctx sessionctx.Context, hintName string, hintTable tableInfo.dbName = defaultDBName } switch hintName { - case TiDBMergeJoin, HintSMJ, TiDBIndexNestedLoopJoin, HintINLJ, HintINLHJ, HintINLMJ, TiDBHashJoin, HintHJ: + case TiDBMergeJoin, HintSMJ, TiDBIndexNestedLoopJoin, HintINLJ, HintINLHJ, HintINLMJ, TiDBHashJoin, HintHJ, HintLeading: if len(tableInfo.partitions) > 0 { isInapplicable = true } diff --git a/planner/core/rule_join_reorder.go b/planner/core/rule_join_reorder.go index d06c8be47f62f..2445063fbdd03 100644 --- a/planner/core/rule_join_reorder.go +++ b/planner/core/rule_join_reorder.go @@ -34,29 +34,38 @@ import ( // For example: "InnerJoin(InnerJoin(a, b), LeftJoin(c, d))" // results in a join group {a, b, c, d}. func extractJoinGroup(p LogicalPlan) (group []LogicalPlan, eqEdges []*expression.ScalarFunction, - otherConds []expression.Expression, joinTypes []JoinType) { + otherConds []expression.Expression, joinTypes []JoinType, hintInfo *tableHintInfo) { join, isJoin := p.(*LogicalJoin) if !isJoin || join.preferJoinType > uint(0) || join.StraightJoin || (join.JoinType != InnerJoin && join.JoinType != LeftOuterJoin && join.JoinType != RightOuterJoin) || ((join.JoinType == LeftOuterJoin || join.JoinType == RightOuterJoin) && join.EqualConditions == nil) { - return []LogicalPlan{p}, nil, nil, nil + return []LogicalPlan{p}, nil, nil, nil, nil + } + if join.preferJoinOrder { + hintInfo = join.hintInfo } if join.JoinType != RightOuterJoin { - lhsGroup, lhsEqualConds, lhsOtherConds, lhsJoinTypes := extractJoinGroup(join.children[0]) + lhsGroup, lhsEqualConds, lhsOtherConds, lhsJoinTypes, lhsHintInfo := extractJoinGroup(join.children[0]) group = append(group, lhsGroup...) eqEdges = append(eqEdges, lhsEqualConds...) otherConds = append(otherConds, lhsOtherConds...) joinTypes = append(joinTypes, lhsJoinTypes...) + if hintInfo == nil && lhsHintInfo != nil { + hintInfo = lhsHintInfo + } } else { group = append(group, join.children[0]) } if join.JoinType != LeftOuterJoin { - rhsGroup, rhsEqualConds, rhsOtherConds, rhsJoinTypes := extractJoinGroup(join.children[1]) + rhsGroup, rhsEqualConds, rhsOtherConds, rhsJoinTypes, rhsHintInfo := extractJoinGroup(join.children[1]) group = append(group, rhsGroup...) eqEdges = append(eqEdges, rhsEqualConds...) otherConds = append(otherConds, rhsOtherConds...) joinTypes = append(joinTypes, rhsJoinTypes...) + if hintInfo == nil && rhsHintInfo != nil { + hintInfo = rhsHintInfo + } } else { group = append(group, join.children[1]) } @@ -68,7 +77,7 @@ func extractJoinGroup(p LogicalPlan) (group []LogicalPlan, eqEdges []*expression for range join.EqualConditions { joinTypes = append(joinTypes, join.JoinType) } - return group, eqEdges, otherConds, joinTypes + return group, eqEdges, otherConds, joinTypes, hintInfo } type joinReOrderSolver struct { @@ -92,7 +101,7 @@ func (s *joinReOrderSolver) optimize(ctx context.Context, p LogicalPlan, opt *lo func (s *joinReOrderSolver) optimizeRecursive(ctx sessionctx.Context, p LogicalPlan, tracer *joinReorderTrace) (LogicalPlan, error) { var err error - curJoinGroup, eqEdges, otherConds, joinTypes := extractJoinGroup(p) + curJoinGroup, eqEdges, otherConds, joinTypes, hintInfo := extractJoinGroup(p) if len(curJoinGroup) > 1 { for i := range curJoinGroup { curJoinGroup[i], err = s.optimizeRecursive(ctx, curJoinGroup[i], tracer) @@ -106,11 +115,47 @@ func (s *joinReOrderSolver) optimizeRecursive(ctx sessionctx.Context, p LogicalP eqEdges: eqEdges, joinTypes: joinTypes, } - err = baseGroupSolver.deriveStatsAndGenerateJRNodeGroup(curJoinGroup, tracer) - if err != nil { - return nil, err - } + joinGroupNum := len(curJoinGroup) + var leadingJoinGroup []LogicalPlan + if hintInfo != nil { + for _, hintTbl := range hintInfo.leadingJoinOrder { + for i, joinGroup := range curJoinGroup { + tableAlias := extractTableAlias(joinGroup, joinGroup.SelectBlockOffset()) + if tableAlias == nil { + continue + } + if hintTbl.dbName.L == tableAlias.dbName.L && hintTbl.tblName.L == tableAlias.tblName.L && hintTbl.selectOffset == tableAlias.selectOffset { + leadingJoinGroup = append(leadingJoinGroup, joinGroup) + curJoinGroup = append(curJoinGroup[:i], curJoinGroup[i+1:]...) + break + } + } + } + var errMsg string + if len(leadingJoinGroup) != len(hintInfo.leadingJoinOrder) { + errMsg = fmt.Sprint("leading hint is inapplicable, check if the leading hint table is valid") + } else if joinGroupNum <= ctx.GetSessionVars().TiDBOptJoinReorderThreshold { + errMsg = fmt.Sprint("leading hint is inapplicable for the DP join reorder algorithm") + } + if len(errMsg) > 0 { + curJoinGroup = append(curJoinGroup, leadingJoinGroup...) + leadingJoinGroup = nil + ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack(errMsg)) + } + if leadingJoinGroup != nil { + leadingJoin := leadingJoinGroup[0] + leadingJoinGroup = leadingJoinGroup[1:] + for len(leadingJoinGroup) > 0 { + var usedEdges []*expression.ScalarFunction + var joinType JoinType + leadingJoin, leadingJoinGroup[0], usedEdges, joinType = baseGroupSolver.checkConnection(leadingJoin, leadingJoinGroup[0]) + leadingJoin, baseGroupSolver.otherConds = baseGroupSolver.makeJoin(leadingJoin, leadingJoinGroup[0], usedEdges, joinType) + leadingJoinGroup = leadingJoinGroup[1:] + } + baseGroupSolver.leadingJoinGroup = leadingJoin + } + } originalSchema := p.Schema() // Not support outer join reorder when using the DP algorithm @@ -121,11 +166,11 @@ func (s *joinReOrderSolver) optimizeRecursive(ctx sessionctx.Context, p LogicalP break } } - if len(curJoinGroup) > ctx.GetSessionVars().TiDBOptJoinReorderThreshold || !isSupportDP { + if joinGroupNum > ctx.GetSessionVars().TiDBOptJoinReorderThreshold || !isSupportDP { groupSolver := &joinReorderGreedySolver{ baseSingleGroupJoinOrderSolver: baseGroupSolver, } - p, err = groupSolver.solve(tracer) + p, err = groupSolver.solve(curJoinGroup, tracer) } else { dpSolver := &joinReorderDPSolver{ baseSingleGroupJoinOrderSolver: baseGroupSolver, @@ -171,29 +216,30 @@ func (s *joinReOrderSolver) optimizeRecursive(ctx sessionctx.Context, p LogicalP // nolint:structcheck type baseSingleGroupJoinOrderSolver struct { - ctx sessionctx.Context - curJoinGroup []*jrNode - otherConds []expression.Expression - eqEdges []*expression.ScalarFunction - joinTypes []JoinType + ctx sessionctx.Context + curJoinGroup []*jrNode + otherConds []expression.Expression + eqEdges []*expression.ScalarFunction + joinTypes []JoinType + leadingJoinGroup LogicalPlan } -// deriveStatsAndGenerateJRNodeGroup used to derive the stats for the joinNodePlans and generate the jrNode groups based on the cost. -func (s *baseSingleGroupJoinOrderSolver) deriveStatsAndGenerateJRNodeGroup(joinNodePlans []LogicalPlan, tracer *joinReorderTrace) error { - s.curJoinGroup = make([]*jrNode, 0, len(joinNodePlans)) +// generateJoinOrderNode used to derive the stats for the joinNodePlans and generate the jrNode groups based on the cost. +func (s *baseSingleGroupJoinOrderSolver) generateJoinOrderNode(joinNodePlans []LogicalPlan, tracer *joinReorderTrace) ([]*jrNode, error) { + joinGroup := make([]*jrNode, 0, len(joinNodePlans)) for _, node := range joinNodePlans { _, err := node.recursiveDeriveStats(nil) if err != nil { - return err + return nil, err } cost := s.baseNodeCumCost(node) - s.curJoinGroup = append(s.curJoinGroup, &jrNode{ + joinGroup = append(joinGroup, &jrNode{ p: node, cumCost: cost, }) tracer.appendLogicalJoinCost(node, cost) } - return nil + return joinGroup, nil } // baseNodeCumCost calculate the cumulative cost of the node in the join group. diff --git a/planner/core/rule_join_reorder_dp.go b/planner/core/rule_join_reorder_dp.go index 1cafdf6dc1e5b..c91d74e1b7c28 100644 --- a/planner/core/rule_join_reorder_dp.go +++ b/planner/core/rule_join_reorder_dp.go @@ -39,6 +39,18 @@ type joinGroupNonEqEdge struct { func (s *joinReorderDPSolver) solve(joinGroup []LogicalPlan, tracer *joinReorderTrace) (LogicalPlan, error) { eqConds := expression.ScalarFuncs2Exprs(s.eqEdges) + for _, node := range joinGroup { + _, err := node.recursiveDeriveStats(nil) + if err != nil { + return nil, err + } + cost := s.baseNodeCumCost(node) + s.curJoinGroup = append(s.curJoinGroup, &jrNode{ + p: node, + cumCost: cost, + }) + tracer.appendLogicalJoinCost(node, cost) + } adjacents := make([][]int, len(s.curJoinGroup)) totalEqEdges := make([]joinGroupEqEdge, 0, len(eqConds)) addEqEdge := func(node1, node2 int, edgeContent *expression.ScalarFunction) { diff --git a/planner/core/rule_join_reorder_dp_test.go b/planner/core/rule_join_reorder_dp_test.go index 6b93733b1401b..451c7f18059f6 100644 --- a/planner/core/rule_join_reorder_dp_test.go +++ b/planner/core/rule_join_reorder_dp_test.go @@ -180,23 +180,13 @@ func TestDPReorderTPCHQ5(t *testing.T) { eqConds = append(eqConds, expression.NewFunctionInternal(ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), joinGroups[2].Schema().Columns[0], joinGroups[4].Schema().Columns[0])) eqConds = append(eqConds, expression.NewFunctionInternal(ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), joinGroups[3].Schema().Columns[0], joinGroups[4].Schema().Columns[0])) eqConds = append(eqConds, expression.NewFunctionInternal(ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), joinGroups[4].Schema().Columns[0], joinGroups[5].Schema().Columns[0])) - eqEdges := make([]*expression.ScalarFunction, 0, len(eqConds)) - for _, cond := range eqConds { - sf, isSF := cond.(*expression.ScalarFunction) - require.True(t, isSF) - eqEdges = append(eqEdges, sf) - } - baseGroupSolver := &baseSingleGroupJoinOrderSolver{ - ctx: ctx, - eqEdges: eqEdges, - } - err := baseGroupSolver.deriveStatsAndGenerateJRNodeGroup(joinGroups, nil) - require.NoError(t, err) solver := &joinReorderDPSolver{ - baseSingleGroupJoinOrderSolver: baseGroupSolver, - newJoin: newMockJoin(ctx, statsMap), + baseSingleGroupJoinOrderSolver: &baseSingleGroupJoinOrderSolver{ + ctx: ctx, + }, + newJoin: newMockJoin(ctx, statsMap), } - result, err := solver.solve(joinGroups, nil) + result, err := solver.solve(joinGroups, eqConds, nil) require.NoError(t, err) expected := "MockJoin{supplier, MockJoin{lineitem, MockJoin{orders, MockJoin{customer, MockJoin{nation, region}}}}}" @@ -214,14 +204,11 @@ func TestDPReorderAllCartesian(t *testing.T) { joinGroup = append(joinGroup, newDataSource(ctx, "b", 100)) joinGroup = append(joinGroup, newDataSource(ctx, "c", 100)) joinGroup = append(joinGroup, newDataSource(ctx, "d", 100)) - baseGroupSolver := &baseSingleGroupJoinOrderSolver{ - ctx: ctx, - } - err := baseGroupSolver.deriveStatsAndGenerateJRNodeGroup(joinGroup, nil) - require.NoError(t, err) solver := &joinReorderDPSolver{ - baseSingleGroupJoinOrderSolver: baseGroupSolver, - newJoin: newMockJoin(ctx, statsMap), + baseSingleGroupJoinOrderSolver: &baseSingleGroupJoinOrderSolver{ + ctx: ctx, + }, + newJoin: newMockJoin(ctx, statsMap), } result, err := solver.solve(joinGroup, nil) require.NoError(t, err) diff --git a/planner/core/rule_join_reorder_greedy.go b/planner/core/rule_join_reorder_greedy.go index 85516240b63e4..6401d9a677c8c 100644 --- a/planner/core/rule_join_reorder_greedy.go +++ b/planner/core/rule_join_reorder_greedy.go @@ -39,12 +39,28 @@ type joinReorderGreedySolver struct { // // For the nodes and join trees which don't have a join equal condition to // connect them, we make a bushy join tree to do the cartesian joins finally. -func (s *joinReorderGreedySolver) solve(tracer *joinReorderTrace) (LogicalPlan, error) { +func (s *joinReorderGreedySolver) solve(joinNodePlans []LogicalPlan, tracer *joinReorderTrace) (LogicalPlan, error) { + var err error + s.curJoinGroup, err = s.generateJoinOrderNode(joinNodePlans, tracer) + if err != nil { + return nil, err + } + var leadingJoinNodes []*jrNode + if s.leadingJoinGroup != nil { + leadingJoinNodes, err = s.generateJoinOrderNode([]LogicalPlan{s.leadingJoinGroup}, tracer) + if err != nil { + return nil, err + } + } // Sort plans by cost sort.SliceStable(s.curJoinGroup, func(i, j int) bool { return s.curJoinGroup[i].cumCost < s.curJoinGroup[j].cumCost }) + if leadingJoinNodes != nil { + leadingJoinNodes := append(leadingJoinNodes, s.curJoinGroup...) + s.curJoinGroup = leadingJoinNodes + } var cartesianGroup []LogicalPlan for len(s.curJoinGroup) > 0 { newNode, err := s.constructConnectedJoinTree(tracer) From cdb86390fe65f3bb7b4093c8bd4f91da8525a757 Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Wed, 11 May 2022 16:49:09 +0800 Subject: [PATCH 06/36] fix ut --- planner/core/rule_join_reorder_dp_test.go | 18 +++++++++++++----- 1 file changed, 13 insertions(+), 5 deletions(-) diff --git a/planner/core/rule_join_reorder_dp_test.go b/planner/core/rule_join_reorder_dp_test.go index 451c7f18059f6..d62fcbce0e746 100644 --- a/planner/core/rule_join_reorder_dp_test.go +++ b/planner/core/rule_join_reorder_dp_test.go @@ -180,13 +180,21 @@ func TestDPReorderTPCHQ5(t *testing.T) { eqConds = append(eqConds, expression.NewFunctionInternal(ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), joinGroups[2].Schema().Columns[0], joinGroups[4].Schema().Columns[0])) eqConds = append(eqConds, expression.NewFunctionInternal(ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), joinGroups[3].Schema().Columns[0], joinGroups[4].Schema().Columns[0])) eqConds = append(eqConds, expression.NewFunctionInternal(ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), joinGroups[4].Schema().Columns[0], joinGroups[5].Schema().Columns[0])) + eqEdges := make([]*expression.ScalarFunction, 0, len(eqConds)) + for _, cond := range eqConds { + sf, isSF := cond.(*expression.ScalarFunction) + require.True(t, isSF) + eqEdges = append(eqEdges, sf) + } + baseGroupSolver := &baseSingleGroupJoinOrderSolver{ + ctx: ctx, + eqEdges: eqEdges, + } solver := &joinReorderDPSolver{ - baseSingleGroupJoinOrderSolver: &baseSingleGroupJoinOrderSolver{ - ctx: ctx, - }, - newJoin: newMockJoin(ctx, statsMap), + baseSingleGroupJoinOrderSolver: baseGroupSolver, + newJoin: newMockJoin(ctx, statsMap), } - result, err := solver.solve(joinGroups, eqConds, nil) + result, err := solver.solve(joinGroups, nil) require.NoError(t, err) expected := "MockJoin{supplier, MockJoin{lineitem, MockJoin{orders, MockJoin{customer, MockJoin{nation, region}}}}}" From 88a2bb1c466d1062baf226f8721e3e5cbeb77723 Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Thu, 12 May 2022 16:15:49 +0800 Subject: [PATCH 07/36] fix check_dev --- planner/core/logical_plan_builder.go | 2 +- planner/core/rule_join_reorder.go | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 64af57446008c..9afd664b0850d 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -3634,7 +3634,7 @@ func (b *PlanBuilder) pushTableHints(hints []*ast.TableOptimizerHint, currentLev // If there are more leading hints, all leading hints will be invalid. leadingJoinOrder = leadingJoinOrder[:0] // Append warning if there are invalid index names. - errMsg := fmt.Sprintf("We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid") + errMsg := "We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid" b.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack(errMsg)) } b.tableHintInfo = append(b.tableHintInfo, tableHintInfo{ diff --git a/planner/core/rule_join_reorder.go b/planner/core/rule_join_reorder.go index 657fd21975585..c52e52b1c3ab8 100644 --- a/planner/core/rule_join_reorder.go +++ b/planner/core/rule_join_reorder.go @@ -134,9 +134,9 @@ func (s *joinReOrderSolver) optimizeRecursive(ctx sessionctx.Context, p LogicalP } var errMsg string if len(leadingJoinGroup) != len(hintInfo.leadingJoinOrder) { - errMsg = fmt.Sprint("leading hint is inapplicable, check if the leading hint table is valid") + errMsg = "leading hint is inapplicable, check if the leading hint table is valid" } else if joinGroupNum <= ctx.GetSessionVars().TiDBOptJoinReorderThreshold { - errMsg = fmt.Sprint("leading hint is inapplicable for the DP join reorder algorithm") + errMsg = "leading hint is inapplicable for the DP join reorder algorithm" } if len(errMsg) > 0 { curJoinGroup = append(curJoinGroup, leadingJoinGroup...) From 919628c97c7e4d0c1f7679984ccaa770dc1518fe Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Thu, 12 May 2022 16:55:27 +0800 Subject: [PATCH 08/36] add more test cases --- planner/core/rule_join_reorder_test.go | 15 + .../core/testdata/join_reorder_suite_in.json | 70 + .../core/testdata/join_reorder_suite_out.json | 1521 +++++++++++++++-- 3 files changed, 1459 insertions(+), 147 deletions(-) diff --git a/planner/core/rule_join_reorder_test.go b/planner/core/rule_join_reorder_test.go index 40bbf12c8a623..8ebb62cc31502 100644 --- a/planner/core/rule_join_reorder_test.go +++ b/planner/core/rule_join_reorder_test.go @@ -55,3 +55,18 @@ func TestStraightJoinHint(t *testing.T) { tk.MustExec("create table t4(a int, b int, key(a));") runJoinReorderTestData(t, tk, "TestStraightJoinHint") } + +func TestLeadingJoinHint(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + 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));") + runJoinReorderTestData(t, tk, "TestLeadingJoinHint") +} diff --git a/planner/core/testdata/join_reorder_suite_in.json b/planner/core/testdata/join_reorder_suite_in.json index 02b41c4355c4f..fd66617e30ca0 100644 --- a/planner/core/testdata/join_reorder_suite_in.json +++ b/planner/core/testdata/join_reorder_suite_in.json @@ -29,5 +29,75 @@ "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": "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 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 /*+ leading(t2) */ * 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 /*+ leading(t3) */ * 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 /*+ leading(t4) */ * 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 /*+ leading(t2, t1) */ * 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 /*+ leading(t2, t3) */ * 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 /*+ leading(t4, t1) */ * 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 /*+ leading(t3, t1) */ * 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 * from ((select /*+ leading(t1) */ 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;", + "select * from ((select /*+ leading(t2) */ 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;", + "select * from ((select /*+ leading(t1, t3) */ 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;", + "select * from ((select /*+ leading(t2, t4, t3) */ 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;", + "select /*+ leading(t3) */ * from ((select /*+ leading(t1) */ 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;", + "select /*+ leading(t3, t1) */ * from ((select /*+ leading(t2) */ 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;", + "select /*+ leading(t3, t1, t2) */ * from ((select /*+ leading(t1, t3) */ 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;", + "select /*+ leading(t3, t4) */ * from ((select /*+ leading(t2, t4, t3) */ 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;" + ] } ] diff --git a/planner/core/testdata/join_reorder_suite_out.json b/planner/core/testdata/join_reorder_suite_out.json index a0af850e8a9d8..aab188b30a3ff 100644 --- a/planner/core/testdata/join_reorder_suite_out.json +++ b/planner/core/testdata/join_reorder_suite_out.json @@ -1,7 +1,7 @@ [ { "Name": "TestStraightJoinHint", - "Cases": [ + "Cases": [ { "SQL": "select /*+ straight_join() */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", "Plan": [ @@ -368,191 +368,1418 @@ " └─IndexReader(Probe) 9990.00 root index:IndexFullScan", " └─IndexFullScan 9990.00 cop[tikv] table:t, index:a(a) keep order:false, stats:pseudo" ] + }, + { + "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" + ] }, { - "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;", + "SQL": "select /*+ leading(t2, t1) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", "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)]", + "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" + ] + }, + { + "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.t4.a))", - " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", - " └─HashJoin(Probe) 155781718.59 root CARTESIAN inner join", + " │ └─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.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)]", - " ├─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", - " └─IndexReader(Probe) 9990.00 root index:IndexFullScan", - " └─IndexFullScan 9990.00 cop[tikv] table:t, index:a(a) keep order:false, stats:pseudo" + " │ └─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" ] }, { - "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;", + "SQL": "select /*+ leading(t2, t3) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", "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)]", + "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.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)]", + " │ └─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", - " └─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" + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" ] }, { - "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;", + "SQL": "select /*+ leading(t, t1, t2) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", "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)]", + "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.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)]", + " │ └─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" + ] + }, + { + "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", - " └─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" + " └─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" ] }, { - "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;", + "SQL": "select /*+ leading(t3, t1, t) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", "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)]", + "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.t4.a))", - " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", - " └─HashJoin(Probe) 155781718.59 root CARTESIAN inner join", + " │ └─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.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) 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" + " │ └─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" ] }, { - "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;", + "SQL": "select /*+ leading(t2, t3, t) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", "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", + "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.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) 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" + " │ └─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" ] }, { - "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;", + "SQL": "select /*+ leading(t2, t3, t, t4) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", "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)]", + "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.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", - " └─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" + " └─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" + ] + }, + { + "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" + ] + }, + { + "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" + ] + }, + { + "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" + ] + }, + { + "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" + ] + }, + { + "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" + ] + }, + { + "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" + ] + }, + { + "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" + ] + }, + { + "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" + ] + }, + { + "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" + ] + }, + { + "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" + ] + }, + { + "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" + ] + }, + { + "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" + ] + }, + { + "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" + ] + }, + { + "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" + ] + }, + { + "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" + ] + }, + { + "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" + ] + }, + { + "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" + ] + }, + { + "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" + ] + }, + { + "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" + ] + }, + { + "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" + ] + }, + { + "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" + ] + }, + { + "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" + ] + }, + { + "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" + ] + }, + { + "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" + ] + }, + { + "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" + ] + }, + { + "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" + ] + }, + { + "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" + ] + }, + { + "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" + ] + }, + { + "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" + ] + }, + { + "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" + ] + }, + { + "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" + ] + }, + { + "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" + ] + }, + { + "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" + ] + }, + { + "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" + ] + }, + { + "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" + ] + }, + { + "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" + ] + }, + { + "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" + ] + }, + { + "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" + ] + }, + { + "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" + ] + }, + { + "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" + ] + }, + { + "SQL": "select /*+ leading(t1) */ * 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": [ + "Projection 304261169.13 root test.t3.a, test.t3.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.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", + " │ └─IndexReader(Probe) 9990.00 root index:IndexFullScan", + " │ └─IndexFullScan 9990.00 cop[tikv] table:t, index:a(a) 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.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.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" + ] + }, + { + "SQL": "select /*+ leading(t2) */ * 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": [ + "Projection 304261169.13 root test.t3.a, test.t3.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.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", + " │ └─IndexReader(Probe) 9990.00 root index:IndexFullScan", + " │ └─IndexFullScan 9990.00 cop[tikv] table:t, index:a(a) keep order:false, stats:pseudo", + " └─HashJoin(Probe) 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.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" + ] + }, + { + "SQL": "select /*+ leading(t3) */ * 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 CARTESIAN inner join", + "├─HashJoin(Build) 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", + "│ └─IndexReader(Probe) 9990.00 root index:IndexFullScan", + "│ └─IndexFullScan 9990.00 cop[tikv] table:t, index:a(a) 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.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" + ] + }, + { + "SQL": "select /*+ leading(t4) */ * 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": [ + "Projection 304261169.13 root test.t3.a, test.t3.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.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", + " │ └─IndexReader(Probe) 9990.00 root index:IndexFullScan", + " │ └─IndexFullScan 9990.00 cop[tikv] table:t, index:a(a) 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.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.t4.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ leading(t2, t1) */ * 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": [ + "Projection 304261169.13 root test.t3.a, test.t3.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.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", + " │ └─IndexReader(Probe) 9990.00 root index:IndexFullScan", + " │ └─IndexFullScan 9990.00 cop[tikv] table:t, index:a(a) keep order:false, stats:pseudo", + " └─HashJoin(Probe) 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.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" + ] + }, + { + "SQL": "select /*+ leading(t2, t3) */ * 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 CARTESIAN inner join", + "├─HashJoin(Build) 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", + "│ └─IndexReader(Probe) 9990.00 root index:IndexFullScan", + "│ └─IndexFullScan 9990.00 cop[tikv] table:t, index:a(a) 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.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" + ] + }, + { + "SQL": "select /*+ leading(t4, t1) */ * 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": [ + "Projection 304261169.13 root test.t3.a, test.t3.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.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", + " │ └─IndexReader(Probe) 9990.00 root index:IndexFullScan", + " │ └─IndexFullScan 9990.00 cop[tikv] table:t, index:a(a) 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.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.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" + ] + }, + { + "SQL": "select /*+ leading(t3, t1) */ * 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 CARTESIAN inner join", + "├─HashJoin(Build) 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", + "│ └─IndexReader(Probe) 9990.00 root index:IndexFullScan", + "│ └─IndexFullScan 9990.00 cop[tikv] table:t, index:a(a) 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.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" + ] + }, + { + "SQL": "select * from ((select /*+ leading(t1) */ 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 CARTESIAN inner join", + "├─HashJoin(Build) 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) 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", + "└─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.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" + ] + }, + { + "SQL": "select * from ((select /*+ leading(t2) */ 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 CARTESIAN inner join", + "├─HashJoin(Build) 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) 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", + "└─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.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" + ] + }, + { + "SQL": "select * from ((select /*+ leading(t1, t3) */ 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.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.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" + ] + }, + { + "SQL": "select * from ((select /*+ leading(t2, t4, t3) */ 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": [ + "Projection 304261169.13 root test.t3.a, test.t3.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.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", + " │ └─IndexReader(Probe) 9990.00 root index:IndexFullScan", + " │ └─IndexFullScan 9990.00 cop[tikv] table:t, index:a(a) keep order:false, stats:pseudo", + " └─HashJoin(Probe) 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.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) 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" + ] + }, + { + "SQL": "select /*+ leading(t3) */ * from ((select /*+ leading(t1) */ 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 CARTESIAN inner join", + "├─HashJoin(Build) 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) 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", + "└─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.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" + ] + }, + { + "SQL": "select /*+ leading(t3, t1) */ * from ((select /*+ leading(t2) */ 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 CARTESIAN inner join", + "├─HashJoin(Build) 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) 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", + "└─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.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" + ] + }, + { + "SQL": "select /*+ leading(t3, t1, t2) */ * from ((select /*+ leading(t1, t3) */ 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 CARTESIAN inner join", + "├─HashJoin(Build) 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) 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", + "└─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.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" + ] + }, + { + "SQL": "select /*+ leading(t3, t4) */ * from ((select /*+ leading(t2, t4, t3) */ 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 CARTESIAN inner join", + "├─HashJoin(Build) 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) 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", + "└─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.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" ] } ] From 3226deba64880d83997b715c532a53f95618450e Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Thu, 12 May 2022 17:00:56 +0800 Subject: [PATCH 09/36] fix ut --- .../core/testdata/join_reorder_suite_out.json | 190 +++++++++++++++++- 1 file changed, 189 insertions(+), 1 deletion(-) diff --git a/planner/core/testdata/join_reorder_suite_out.json b/planner/core/testdata/join_reorder_suite_out.json index aab188b30a3ff..810e92d4c3af6 100644 --- a/planner/core/testdata/join_reorder_suite_out.json +++ b/planner/core/testdata/join_reorder_suite_out.json @@ -1,7 +1,7 @@ [ { "Name": "TestStraightJoinHint", - "Cases": [ + "Cases": [ { "SQL": "select /*+ straight_join() */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", "Plan": [ @@ -368,6 +368,194 @@ " └─IndexReader(Probe) 9990.00 root index:IndexFullScan", " └─IndexFullScan 9990.00 cop[tikv] table:t, index:a(a) keep order:false, stats:pseudo" ] + }, + { + "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)]", + " ├─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", + " └─IndexReader(Probe) 9990.00 root index:IndexFullScan", + " └─IndexFullScan 9990.00 cop[tikv] table:t, index:a(a) keep order:false, stats:pseudo" + ] + }, + { + "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" + ] + }, + { + "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" + ] + }, + { + "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) 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" + ] + }, + { + "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) 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" + ] + }, + { + "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" + ] + } + ] }, { "Name": "TestLeadingJoinHint", From 098ef46975df12129ce2c84387f592e2f5be2524 Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Thu, 12 May 2022 17:43:37 +0800 Subject: [PATCH 10/36] fix ut --- planner/core/rule_join_reorder_test.go | 8 +- .../core/testdata/join_reorder_suite_in.json | 32 +- .../core/testdata/join_reorder_suite_out.json | 439 +++++++++--------- 3 files changed, 241 insertions(+), 238 deletions(-) diff --git a/planner/core/rule_join_reorder_test.go b/planner/core/rule_join_reorder_test.go index 8ebb62cc31502..61e521acff762 100644 --- a/planner/core/rule_join_reorder_test.go +++ b/planner/core/rule_join_reorder_test.go @@ -47,7 +47,7 @@ func TestStraightJoinHint(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - tk.MustExec("drop table if exists t, t1, t2, t3;") + 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));") @@ -62,11 +62,15 @@ func TestLeadingJoinHint(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - tk.MustExec("drop table if exists t, t1, t2, t3;") + 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") } diff --git a/planner/core/testdata/join_reorder_suite_in.json b/planner/core/testdata/join_reorder_suite_in.json index fd66617e30ca0..5821f26519638 100644 --- a/planner/core/testdata/join_reorder_suite_in.json +++ b/planner/core/testdata/join_reorder_suite_in.json @@ -82,22 +82,22 @@ "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 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 /*+ leading(t2) */ * 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 /*+ leading(t3) */ * 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 /*+ leading(t4) */ * 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 /*+ leading(t2, t1) */ * 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 /*+ leading(t2, t3) */ * 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 /*+ leading(t4, t1) */ * 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 /*+ leading(t3, t1) */ * 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 * from ((select /*+ leading(t1) */ 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;", - "select * from ((select /*+ leading(t2) */ 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;", - "select * from ((select /*+ leading(t1, t3) */ 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;", - "select * from ((select /*+ leading(t2, t4, t3) */ 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;", - "select /*+ leading(t3) */ * from ((select /*+ leading(t1) */ 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;", - "select /*+ leading(t3, t1) */ * from ((select /*+ leading(t2) */ 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;", - "select /*+ leading(t3, t1, t2) */ * from ((select /*+ leading(t1, t3) */ 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;", - "select /*+ leading(t3, t4) */ * from ((select /*+ leading(t2, t4, t3) */ 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;" + "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;" ] } ] diff --git a/planner/core/testdata/join_reorder_suite_out.json b/planner/core/testdata/join_reorder_suite_out.json index 810e92d4c3af6..9e730cbc88f01 100644 --- a/planner/core/testdata/join_reorder_suite_out.json +++ b/planner/core/testdata/join_reorder_suite_out.json @@ -1469,28 +1469,28 @@ ] }, { - "SQL": "select /*+ leading(t1) */ * 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;", + "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.t3.a, test.t3.b, test.t4.a, test.t4.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b", + "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.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)]", + " ├─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) 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", - " │ └─IndexReader(Probe) 9990.00 root index:IndexFullScan", - " │ └─IndexFullScan 9990.00 cop[tikv] table:t, index:a(a) keep order:false, stats:pseudo", + " │ │ └─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.t7.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t7 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.t3.a)]", + " └─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.t3.a))", - " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " │ └─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))", @@ -1501,24 +1501,24 @@ ] }, { - "SQL": "select /*+ leading(t2) */ * 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;", + "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.t3.a, test.t3.b, test.t4.a, test.t4.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b", + "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.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)]", + " ├─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) 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", - " │ └─IndexReader(Probe) 9990.00 root index:IndexFullScan", - " │ └─IndexFullScan 9990.00 cop[tikv] table:t, index:a(a) keep order:false, stats:pseudo", - " └─HashJoin(Probe) 19511.72 root inner join, equal:[eq(test.t4.a, test.t3.a)]", + " │ │ └─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.t7.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t7 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.t3.a))", - " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " │ └─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))", @@ -1533,19 +1533,19 @@ ] }, { - "SQL": "select /*+ leading(t3) */ * 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;", + "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.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)]", + "├─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) 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", - "│ └─IndexReader(Probe) 9990.00 root index:IndexFullScan", - "│ └─IndexFullScan 9990.00 cop[tikv] table:t, index:a(a) keep order:false, stats:pseudo", + "│ │ └─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.t7.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t7 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))", @@ -1554,30 +1554,30 @@ " ├─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.t4.a)]", + " └─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.t3.a))", - " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + " └─Selection 9990.00 cop[tikv] not(isnull(test.t8.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" ] }, { - "SQL": "select /*+ leading(t4) */ * 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;", + "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.t3.a, test.t3.b, test.t4.a, test.t4.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b", + "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.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)]", + " ├─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) 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", - " │ └─IndexReader(Probe) 9990.00 root index:IndexFullScan", - " │ └─IndexFullScan 9990.00 cop[tikv] table:t, index:a(a) keep order:false, stats:pseudo", + " │ │ └─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.t7.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t7 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))", @@ -1586,34 +1586,34 @@ " ├─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.t3.a)]", + " └─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.t3.a))", - " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " │ └─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" ] }, { - "SQL": "select /*+ leading(t2, t1) */ * 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;", + "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.t3.a, test.t3.b, test.t4.a, test.t4.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b", + "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.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)]", + " ├─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) 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", - " │ └─IndexReader(Probe) 9990.00 root index:IndexFullScan", - " │ └─IndexFullScan 9990.00 cop[tikv] table:t, index:a(a) keep order:false, stats:pseudo", - " └─HashJoin(Probe) 19511.72 root inner join, equal:[eq(test.t4.a, test.t3.a)]", + " │ │ └─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.t7.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t7 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.t3.a))", - " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " │ └─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))", @@ -1628,19 +1628,19 @@ ] }, { - "SQL": "select /*+ leading(t2, t3) */ * 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;", + "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.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)]", + "├─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) 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", - "│ └─IndexReader(Probe) 9990.00 root index:IndexFullScan", - "│ └─IndexFullScan 9990.00 cop[tikv] table:t, index:a(a) keep order:false, stats:pseudo", + "│ │ └─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.t7.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t7 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))", @@ -1649,38 +1649,38 @@ " ├─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.t4.a)]", + " └─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.t3.a))", - " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + " └─Selection 9990.00 cop[tikv] not(isnull(test.t8.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" ] }, { - "SQL": "select /*+ leading(t4, t1) */ * 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;", + "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.t3.a, test.t3.b, test.t4.a, test.t4.b, test.t1.a, test.t1.b, test.t2.a, test.t2.b", + "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.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)]", + " ├─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) 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", - " │ └─IndexReader(Probe) 9990.00 root index:IndexFullScan", - " │ └─IndexFullScan 9990.00 cop[tikv] table:t, index:a(a) keep order:false, stats:pseudo", + " │ │ └─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.t7.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t7 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.t3.a)]", + " └─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.t3.a))", - " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " │ └─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))", @@ -1691,19 +1691,19 @@ ] }, { - "SQL": "select /*+ leading(t3, t1) */ * 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;", + "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.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)]", + "├─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) 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", - "│ └─IndexReader(Probe) 9990.00 root index:IndexFullScan", - "│ └─IndexFullScan 9990.00 cop[tikv] table:t, index:a(a) keep order:false, stats:pseudo", + "│ │ └─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.t7.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t7 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))", @@ -1712,29 +1712,29 @@ " ├─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.t4.a)]", + " └─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.t3.a))", - " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + " └─Selection 9990.00 cop[tikv] not(isnull(test.t8.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" ] }, { - "SQL": "select * from ((select /*+ leading(t1) */ 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;", + "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.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)]", + "├─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)]", "│ ├─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", + "│ │ └─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", + "│ └─IndexReader(Probe) 9990.00 root index:IndexFullScan", + "│ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) 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))", @@ -1743,29 +1743,29 @@ " ├─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.t4.a)]", + " └─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.t3.a))", - " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + " └─Selection 9990.00 cop[tikv] not(isnull(test.t8.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" ] }, { - "SQL": "select * from ((select /*+ leading(t2) */ 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;", + "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.t1.a, test.t.a)]", + "├─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:t, index:a(a) keep order:false, stats:pseudo", - "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t1.b)]", + "│ │ └─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) 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", + "│ │ └─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.t2.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t7 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))", @@ -1774,17 +1774,48 @@ " ├─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.t4.a)]", + " └─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.t3.a))", - " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + " └─Selection 9990.00 cop[tikv] not(isnull(test.t8.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" + ] + }, + { + "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" ] }, { - "SQL": "select * from ((select /*+ leading(t1, t3) */ 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;", + "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", @@ -1794,72 +1825,40 @@ " ├─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.t3.a, test.t4.a)]", + " └─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.t1.a, test.t.a)]", + " └─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:t, index:a(a) keep order:false, stats:pseudo", - " └─HashJoin(Probe) 124625374.88 root inner join, equal:[eq(test.t1.b, test.t2.b)]", + " │ └─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.t2.b))", - " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " │ └─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.t1.a)), not(isnull(test.t1.b))", - " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " │ └─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.t3.a))", - " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select * from ((select /*+ leading(t2, t4, t3) */ 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": [ - "Projection 304261169.13 root test.t3.a, test.t3.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.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", - " │ └─IndexReader(Probe) 9990.00 root index:IndexFullScan", - " │ └─IndexFullScan 9990.00 cop[tikv] table:t, index:a(a) keep order:false, stats:pseudo", - " └─HashJoin(Probe) 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.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) 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" + " └─Selection 9990.00 cop[tikv] not(isnull(test.t8.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" ] }, { - "SQL": "select /*+ leading(t3) */ * from ((select /*+ leading(t1) */ 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;", + "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.t1.a, test.t.a)]", + "├─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:t, index:a(a) keep order:false, stats:pseudo", - "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t1.b, test.t2.b)]", + "│ │ └─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) 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", + "│ │ └─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.t2.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t7 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))", @@ -1868,29 +1867,29 @@ " ├─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.t4.a)]", + " └─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.t3.a))", - " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + " └─Selection 9990.00 cop[tikv] not(isnull(test.t8.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" ] }, { - "SQL": "select /*+ leading(t3, t1) */ * from ((select /*+ leading(t2) */ 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;", + "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.t1.a, test.t.a)]", + "├─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:t, index:a(a) keep order:false, stats:pseudo", - "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t1.b, test.t2.b)]", + "│ │ └─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) 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", + "│ │ └─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.t2.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t7 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))", @@ -1899,29 +1898,29 @@ " ├─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.t4.a)]", + " └─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.t3.a))", - " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + " └─Selection 9990.00 cop[tikv] not(isnull(test.t8.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" ] }, { - "SQL": "select /*+ leading(t3, t1, t2) */ * from ((select /*+ leading(t1, t3) */ 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;", + "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.t1.a, test.t.a)]", + "├─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:t, index:a(a) keep order:false, stats:pseudo", - "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t1.b, test.t2.b)]", + "│ │ └─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) 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", + "│ │ └─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.t2.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t7 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))", @@ -1930,29 +1929,29 @@ " ├─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.t4.a)]", + " └─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.t3.a))", - " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + " └─Selection 9990.00 cop[tikv] not(isnull(test.t8.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" ] }, { - "SQL": "select /*+ leading(t3, t4) */ * from ((select /*+ leading(t2, t4, t3) */ 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;", + "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.t1.a, test.t.a)]", + "├─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:t, index:a(a) keep order:false, stats:pseudo", - "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t1.b, test.t2.b)]", + "│ │ └─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) 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", + "│ │ └─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.t2.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t7 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))", @@ -1961,13 +1960,13 @@ " ├─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.t4.a)]", + " └─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.t3.a))", - " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + " └─Selection 9990.00 cop[tikv] not(isnull(test.t8.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" ] } ] From 1bd70df8c6537b27f75e4151454d8ea7ac025630 Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Fri, 13 May 2022 11:00:07 +0800 Subject: [PATCH 11/36] add the warning messages for the test cases --- planner/core/rule_join_reorder_test.go | 7 +- .../core/testdata/join_reorder_suite_out.json | 274 ++++++++++++------ 2 files changed, 196 insertions(+), 85 deletions(-) diff --git a/planner/core/rule_join_reorder_test.go b/planner/core/rule_join_reorder_test.go index 61e521acff762..fb51e9b18f702 100644 --- a/planner/core/rule_join_reorder_test.go +++ b/planner/core/rule_join_reorder_test.go @@ -26,8 +26,9 @@ import ( func runJoinReorderTestData(t *testing.T, tk *testkit.TestKit, name string) { var input []string var output []struct { - SQL string - Plan []string + SQL string + Plan []string + Warning []string } joinReorderSuiteData := plannercore.GetJoinReorderSuiteData() joinReorderSuiteData.GetTestCasesByName(name, t, &input, &output) @@ -36,8 +37,10 @@ func runJoinReorderTestData(t *testing.T, tk *testkit.TestKit, name string) { 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...)) } } diff --git a/planner/core/testdata/join_reorder_suite_out.json b/planner/core/testdata/join_reorder_suite_out.json index 9e730cbc88f01..2939233cf5c10 100644 --- a/planner/core/testdata/join_reorder_suite_out.json +++ b/planner/core/testdata/join_reorder_suite_out.json @@ -19,7 +19,8 @@ " └─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;", @@ -38,7 +39,8 @@ " └─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;", @@ -57,7 +59,8 @@ " │ └─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;", @@ -76,7 +79,8 @@ " │ └─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;", @@ -92,7 +96,8 @@ " └─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;", @@ -108,7 +113,8 @@ " └─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;", @@ -124,7 +130,8 @@ " └─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;", @@ -140,7 +147,8 @@ " └─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;", @@ -156,7 +164,8 @@ " └─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;", @@ -172,7 +181,8 @@ " └─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;", @@ -188,7 +198,8 @@ " └─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;", @@ -204,7 +215,8 @@ " └─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;", @@ -220,7 +232,8 @@ " └─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;", @@ -236,7 +249,8 @@ " └─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;", @@ -256,7 +270,8 @@ " └─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;", @@ -276,7 +291,8 @@ " └─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;", @@ -296,7 +312,8 @@ " └─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;", @@ -316,7 +333,8 @@ " └─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;", @@ -336,7 +354,8 @@ " └─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;", @@ -367,7 +386,8 @@ " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", " └─IndexReader(Probe) 9990.00 root index:IndexFullScan", " └─IndexFullScan 9990.00 cop[tikv] table:t, index:a(a) 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;", @@ -398,7 +418,8 @@ " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", " └─IndexReader(Probe) 9990.00 root index:IndexFullScan", " └─IndexFullScan 9990.00 cop[tikv] table:t, index:a(a) 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;", @@ -429,7 +450,8 @@ " └─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;", @@ -460,7 +482,8 @@ " └─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;", @@ -491,7 +514,8 @@ " └─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 ((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;", @@ -522,7 +546,8 @@ " └─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 ((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;", @@ -553,7 +578,8 @@ " └─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 } ] }, @@ -577,7 +603,8 @@ " └─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;", @@ -597,7 +624,8 @@ " └─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;", @@ -617,7 +645,8 @@ " │ └─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;", @@ -637,7 +666,8 @@ " │ └─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;", @@ -656,7 +686,8 @@ " └─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, t3) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", @@ -676,7 +707,8 @@ " └─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;", @@ -696,7 +728,8 @@ " │ └─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;", @@ -716,7 +749,8 @@ " │ └─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;", @@ -736,6 +770,10 @@ " └─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 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" ] }, { @@ -753,7 +791,8 @@ " └─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;", @@ -770,7 +809,8 @@ " └─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;", @@ -787,7 +827,8 @@ " └─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;", @@ -804,7 +845,8 @@ " └─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;", @@ -820,7 +862,8 @@ " └─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;", @@ -837,7 +880,8 @@ " └─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;", @@ -854,7 +898,8 @@ " └─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;", @@ -871,7 +916,8 @@ " └─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;", @@ -888,7 +934,8 @@ " └─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;", @@ -904,7 +951,8 @@ " └─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 t2 join (t1 join t3 on t1.a=t3.a) on t2.a=1;", @@ -921,7 +969,8 @@ " └─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;", @@ -938,7 +987,8 @@ " └─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;", @@ -954,7 +1004,8 @@ " └─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;", @@ -971,7 +1022,8 @@ " └─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;", @@ -988,7 +1040,8 @@ " └─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, t3) */ * from t2 join (t1 join t3 on t1.a=t3.a) on t2.a=1;", @@ -1004,7 +1057,8 @@ " └─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;", @@ -1021,7 +1075,8 @@ " └─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;", @@ -1038,7 +1093,8 @@ " └─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;", @@ -1055,7 +1111,8 @@ " └─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;", @@ -1072,7 +1129,8 @@ " └─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;", @@ -1089,7 +1147,8 @@ " └─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;", @@ -1106,7 +1165,8 @@ " └─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;", @@ -1123,7 +1183,8 @@ " └─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;", @@ -1140,7 +1201,8 @@ " └─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;", @@ -1157,7 +1219,8 @@ " └─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;", @@ -1174,7 +1237,8 @@ " └─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;", @@ -1195,7 +1259,8 @@ " └─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;", @@ -1216,7 +1281,8 @@ " └─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;", @@ -1237,7 +1303,8 @@ " └─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;", @@ -1258,7 +1325,8 @@ " └─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;", @@ -1279,7 +1347,8 @@ " └─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;", @@ -1300,7 +1369,8 @@ " └─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;", @@ -1321,7 +1391,8 @@ " └─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;", @@ -1342,7 +1413,8 @@ " └─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;", @@ -1363,7 +1435,8 @@ " └─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;", @@ -1384,7 +1457,8 @@ " └─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;", @@ -1404,7 +1478,8 @@ " └─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;", @@ -1425,7 +1500,8 @@ " └─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;", @@ -1446,7 +1522,8 @@ " └─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;", @@ -1466,7 +1543,8 @@ " └─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;", @@ -1498,7 +1576,8 @@ " └─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;", @@ -1530,7 +1609,8 @@ " └─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;", @@ -1561,6 +1641,9 @@ " └─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" ] }, { @@ -1593,7 +1676,8 @@ " └─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;", @@ -1625,7 +1709,8 @@ " └─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;", @@ -1656,6 +1741,9 @@ " └─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" ] }, { @@ -1688,7 +1776,8 @@ " └─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;", @@ -1719,6 +1808,9 @@ " └─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" ] }, { @@ -1750,7 +1842,8 @@ " └─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;", @@ -1781,7 +1874,8 @@ " └─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;", @@ -1812,7 +1906,8 @@ " │ └─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;", @@ -1843,7 +1938,8 @@ " └─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;", @@ -1874,6 +1970,9 @@ " └─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" ] }, { @@ -1905,6 +2004,9 @@ " └─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" ] }, { @@ -1936,6 +2038,9 @@ " └─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" ] }, { @@ -1967,6 +2072,9 @@ " └─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" ] } ] From 2ec1a7ce14f7b1a63bd933a69ebcf39c3ebd72fe Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Fri, 13 May 2022 11:59:55 +0800 Subject: [PATCH 12/36] fix ut and address comments --- planner/core/rule_join_reorder.go | 104 ++++++++++-------- .../core/testdata/join_reorder_suite_out.json | 12 +- 2 files changed, 63 insertions(+), 53 deletions(-) diff --git a/planner/core/rule_join_reorder.go b/planner/core/rule_join_reorder.go index c52e52b1c3ab8..c263dd9fb1f6b 100644 --- a/planner/core/rule_join_reorder.go +++ b/planner/core/rule_join_reorder.go @@ -109,6 +109,17 @@ func (s *joinReOrderSolver) optimizeRecursive(ctx sessionctx.Context, p LogicalP return nil, err } } + originalSchema := p.Schema() + + // Not support outer join reorder when using the DP algorithm + isSupportDP := true + for _, joinType := range joinTypes { + if joinType != InnerJoin { + isSupportDP = false + break + } + } + baseGroupSolver := &baseSingleGroupJoinOrderSolver{ ctx: ctx, otherConds: otherConds, @@ -117,57 +128,21 @@ func (s *joinReOrderSolver) optimizeRecursive(ctx sessionctx.Context, p LogicalP } joinGroupNum := len(curJoinGroup) - var leadingJoinGroup []LogicalPlan - if hintInfo != nil { - for _, hintTbl := range hintInfo.leadingJoinOrder { - for i, joinGroup := range curJoinGroup { - tableAlias := extractTableAlias(joinGroup, joinGroup.SelectBlockOffset()) - if tableAlias == nil { - continue - } - if hintTbl.dbName.L == tableAlias.dbName.L && hintTbl.tblName.L == tableAlias.tblName.L && hintTbl.selectOffset == tableAlias.selectOffset { - leadingJoinGroup = append(leadingJoinGroup, joinGroup) - curJoinGroup = append(curJoinGroup[:i], curJoinGroup[i+1:]...) - break - } + useGreedy := joinGroupNum > ctx.GetSessionVars().TiDBOptJoinReorderThreshold || !isSupportDP + if hintInfo != nil && hintInfo.leadingJoinOrder != nil { + if useGreedy { + ok, leftJoinGroup := baseGroupSolver.generateLeadingJoinGroup(curJoinGroup, hintInfo) + if !ok { + ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("leading hint is inapplicable, check if the leading hint table is valid")) + } else { + curJoinGroup = leftJoinGroup } - } - var errMsg string - if len(leadingJoinGroup) != len(hintInfo.leadingJoinOrder) { - errMsg = "leading hint is inapplicable, check if the leading hint table is valid" - } else if joinGroupNum <= ctx.GetSessionVars().TiDBOptJoinReorderThreshold { - errMsg = "leading hint is inapplicable for the DP join reorder algorithm" - } - if len(errMsg) > 0 { - curJoinGroup = append(curJoinGroup, leadingJoinGroup...) - leadingJoinGroup = nil - ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack(errMsg)) - } - if leadingJoinGroup != nil { - leadingJoin := leadingJoinGroup[0] - leadingJoinGroup = leadingJoinGroup[1:] - for len(leadingJoinGroup) > 0 { - var usedEdges []*expression.ScalarFunction - var joinType JoinType - leadingJoin, leadingJoinGroup[0], usedEdges, joinType = baseGroupSolver.checkConnection(leadingJoin, leadingJoinGroup[0]) - leadingJoin, baseGroupSolver.otherConds = baseGroupSolver.makeJoin(leadingJoin, leadingJoinGroup[0], usedEdges, joinType) - leadingJoinGroup = leadingJoinGroup[1:] - } - baseGroupSolver.leadingJoinGroup = leadingJoin + } else { + ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("leading hint is inapplicable for the DP join reorder algorithm")) } } - originalSchema := p.Schema() - - // Not support outer join reorder when using the DP algorithm - isSupportDP := true - for _, joinType := range joinTypes { - if joinType != InnerJoin { - isSupportDP = false - break - } - } - if joinGroupNum > ctx.GetSessionVars().TiDBOptJoinReorderThreshold || !isSupportDP { + if useGreedy { groupSolver := &joinReorderGreedySolver{ baseSingleGroupJoinOrderSolver: baseGroupSolver, } @@ -225,6 +200,41 @@ type baseSingleGroupJoinOrderSolver struct { leadingJoinGroup LogicalPlan } +func (s *baseSingleGroupJoinOrderSolver) generateLeadingJoinGroup(curJoinGroup []LogicalPlan, hintInfo *tableHintInfo) (bool, []LogicalPlan) { + var leadingJoinGroup []LogicalPlan + leftJoinGroup := make([]LogicalPlan, len(curJoinGroup)) + for i := range curJoinGroup { + leftJoinGroup[i] = curJoinGroup[i] + } + for _, hintTbl := range hintInfo.leadingJoinOrder { + for i, joinGroup := range leftJoinGroup { + tableAlias := extractTableAlias(joinGroup, joinGroup.SelectBlockOffset()) + if tableAlias == nil { + continue + } + if hintTbl.dbName.L == tableAlias.dbName.L && hintTbl.tblName.L == tableAlias.tblName.L && hintTbl.selectOffset == tableAlias.selectOffset { + leadingJoinGroup = append(leadingJoinGroup, joinGroup) + leftJoinGroup = append(leftJoinGroup[:i], leftJoinGroup[i+1:]...) + break + } + } + } + if len(leadingJoinGroup) != len(hintInfo.leadingJoinOrder) || leadingJoinGroup == nil { + return false, nil + } + leadingJoin := leadingJoinGroup[0] + leadingJoinGroup = leadingJoinGroup[1:] + for len(leadingJoinGroup) > 0 { + var usedEdges []*expression.ScalarFunction + var joinType JoinType + leadingJoin, leadingJoinGroup[0], usedEdges, joinType = s.checkConnection(leadingJoin, leadingJoinGroup[0]) + leadingJoin, s.otherConds = s.makeJoin(leadingJoin, leadingJoinGroup[0], usedEdges, joinType) + leadingJoinGroup = leadingJoinGroup[1:] + } + s.leadingJoinGroup = leadingJoin + return true, leftJoinGroup +} + // generateJoinOrderNode used to derive the stats for the joinNodePlans and generate the jrNode groups based on the cost. func (s *baseSingleGroupJoinOrderSolver) generateJoinOrderNode(joinNodePlans []LogicalPlan, tracer *joinReorderTrace) ([]*jrNode, error) { joinGroup := make([]*jrNode, 0, len(joinNodePlans)) diff --git a/planner/core/testdata/join_reorder_suite_out.json b/planner/core/testdata/join_reorder_suite_out.json index 2939233cf5c10..6c85caf9f275d 100644 --- a/planner/core/testdata/join_reorder_suite_out.json +++ b/planner/core/testdata/join_reorder_suite_out.json @@ -759,17 +759,17 @@ "└─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)]", + " └─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.t.a))", - " │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", - " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t1.b, test.t2.b)]", + " │ └─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.t2.b))", - " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + " └─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", From bae868181be6ce59cdd4aa93a7c1c89f70eba64c Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Fri, 13 May 2022 13:06:46 +0800 Subject: [PATCH 13/36] fix test --- planner/core/rule_join_reorder.go | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/planner/core/rule_join_reorder.go b/planner/core/rule_join_reorder.go index c263dd9fb1f6b..05da2cc6bda21 100644 --- a/planner/core/rule_join_reorder.go +++ b/planner/core/rule_join_reorder.go @@ -203,9 +203,7 @@ type baseSingleGroupJoinOrderSolver struct { func (s *baseSingleGroupJoinOrderSolver) generateLeadingJoinGroup(curJoinGroup []LogicalPlan, hintInfo *tableHintInfo) (bool, []LogicalPlan) { var leadingJoinGroup []LogicalPlan leftJoinGroup := make([]LogicalPlan, len(curJoinGroup)) - for i := range curJoinGroup { - leftJoinGroup[i] = curJoinGroup[i] - } + copy(leftJoinGroup, curJoinGroup) for _, hintTbl := range hintInfo.leadingJoinOrder { for i, joinGroup := range leftJoinGroup { tableAlias := extractTableAlias(joinGroup, joinGroup.SelectBlockOffset()) From 8bf25486bf0cd3c16713fe2574344d67072a9010 Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Fri, 13 May 2022 18:24:27 +0800 Subject: [PATCH 14/36] fix ut --- planner/core/rule_join_reorder.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/planner/core/rule_join_reorder.go b/planner/core/rule_join_reorder.go index 09a92504ef66d..78550e51e555f 100644 --- a/planner/core/rule_join_reorder.go +++ b/planner/core/rule_join_reorder.go @@ -68,7 +68,7 @@ func extractJoinGroup(p LogicalPlan) (group []LogicalPlan, eqEdges []*expression } } if noExpand { - return []LogicalPlan{p}, nil, nil, nil + return []LogicalPlan{p}, nil, nil, nil, nil } group = append(group, lhsGroup...) eqEdges = append(eqEdges, lhsEqualConds...) @@ -105,7 +105,7 @@ func extractJoinGroup(p LogicalPlan) (group []LogicalPlan, eqEdges []*expression } } if noExpand { - return []LogicalPlan{p}, nil, nil, nil + return []LogicalPlan{p}, nil, nil, nil, nil } group = append(group, rhsGroup...) eqEdges = append(eqEdges, rhsEqualConds...) From 1fa1c8c1b94e2cd642e2c99f957da9b4d8a342e5 Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Mon, 16 May 2022 11:34:28 +0800 Subject: [PATCH 15/36] forbid outer join --- planner/core/rule_join_reorder.go | 39 +++++++++++++++----------- planner/core/rule_join_reorder_test.go | 22 +++++++++++++++ 2 files changed, 45 insertions(+), 16 deletions(-) diff --git a/planner/core/rule_join_reorder.go b/planner/core/rule_join_reorder.go index 78550e51e555f..7112e55e075bb 100644 --- a/planner/core/rule_join_reorder.go +++ b/planner/core/rule_join_reorder.go @@ -34,18 +34,19 @@ import ( // For example: "InnerJoin(InnerJoin(a, b), LeftJoin(c, d))" // results in a join group {a, b, c, d}. func extractJoinGroup(p LogicalPlan) (group []LogicalPlan, eqEdges []*expression.ScalarFunction, - otherConds []expression.Expression, joinTypes []JoinType, hintInfo *tableHintInfo) { + otherConds []expression.Expression, joinTypes []JoinType, hintInfo *tableHintInfo, hasOuterJoin bool) { join, isJoin := p.(*LogicalJoin) if !isJoin || join.preferJoinType > uint(0) || join.StraightJoin || (join.JoinType != InnerJoin && join.JoinType != LeftOuterJoin && join.JoinType != RightOuterJoin) || ((join.JoinType == LeftOuterJoin || join.JoinType == RightOuterJoin) && join.EqualConditions == nil) { - return []LogicalPlan{p}, nil, nil, nil, nil + return []LogicalPlan{p}, nil, nil, nil, nil, false } if join.preferJoinOrder { hintInfo = join.hintInfo } + hasOuterJoin = hasOuterJoin || (join.JoinType != InnerJoin) if join.JoinType != RightOuterJoin { - lhsGroup, lhsEqualConds, lhsOtherConds, lhsJoinTypes, lhsHintInfo := extractJoinGroup(join.children[0]) + lhsGroup, lhsEqualConds, lhsOtherConds, lhsJoinTypes, lhsHintInfo, lhsHasOuterJoin := extractJoinGroup(join.children[0]) noExpand := false // If the filters of the outer join is related with multiple leaves of the outer join side. We don't reorder it for now. if join.JoinType == LeftOuterJoin { @@ -68,7 +69,7 @@ func extractJoinGroup(p LogicalPlan) (group []LogicalPlan, eqEdges []*expression } } if noExpand { - return []LogicalPlan{p}, nil, nil, nil, nil + return []LogicalPlan{p}, nil, nil, nil, nil, false } group = append(group, lhsGroup...) eqEdges = append(eqEdges, lhsEqualConds...) @@ -77,12 +78,13 @@ func extractJoinGroup(p LogicalPlan) (group []LogicalPlan, eqEdges []*expression if hintInfo == nil && lhsHintInfo != nil { hintInfo = lhsHintInfo } + hasOuterJoin = hasOuterJoin || lhsHasOuterJoin } else { group = append(group, join.children[0]) } if join.JoinType != LeftOuterJoin { - rhsGroup, rhsEqualConds, rhsOtherConds, rhsJoinTypes, rhsHintInfo := extractJoinGroup(join.children[1]) + rhsGroup, rhsEqualConds, rhsOtherConds, rhsJoinTypes, rhsHintInfo, rhsHasOuterJoin := extractJoinGroup(join.children[1]) noExpand := false // If the filters of the outer join is related with multiple leaves of the outer join side. We don't reorder it for now. if join.JoinType == RightOuterJoin { @@ -105,7 +107,7 @@ func extractJoinGroup(p LogicalPlan) (group []LogicalPlan, eqEdges []*expression } } if noExpand { - return []LogicalPlan{p}, nil, nil, nil, nil + return []LogicalPlan{p}, nil, nil, nil, nil, false } group = append(group, rhsGroup...) eqEdges = append(eqEdges, rhsEqualConds...) @@ -114,6 +116,7 @@ func extractJoinGroup(p LogicalPlan) (group []LogicalPlan, eqEdges []*expression if hintInfo == nil && rhsHintInfo != nil { hintInfo = rhsHintInfo } + hasOuterJoin = hasOuterJoin || rhsHasOuterJoin } else { group = append(group, join.children[1]) } @@ -125,7 +128,7 @@ func extractJoinGroup(p LogicalPlan) (group []LogicalPlan, eqEdges []*expression for range join.EqualConditions { joinTypes = append(joinTypes, join.JoinType) } - return group, eqEdges, otherConds, joinTypes, hintInfo + return group, eqEdges, otherConds, joinTypes, hintInfo, hasOuterJoin } type joinReOrderSolver struct { @@ -149,7 +152,7 @@ func (s *joinReOrderSolver) optimize(ctx context.Context, p LogicalPlan, opt *lo func (s *joinReOrderSolver) optimizeRecursive(ctx sessionctx.Context, p LogicalPlan, tracer *joinReorderTrace) (LogicalPlan, error) { var err error - curJoinGroup, eqEdges, otherConds, joinTypes, hintInfo := extractJoinGroup(p) + curJoinGroup, eqEdges, otherConds, joinTypes, hintInfo, hasOuterJoin := extractJoinGroup(p) if len(curJoinGroup) > 1 { for i := range curJoinGroup { curJoinGroup[i], err = s.optimizeRecursive(ctx, curJoinGroup[i], tracer) @@ -177,16 +180,20 @@ func (s *joinReOrderSolver) optimizeRecursive(ctx sessionctx.Context, p LogicalP joinGroupNum := len(curJoinGroup) useGreedy := joinGroupNum > ctx.GetSessionVars().TiDBOptJoinReorderThreshold || !isSupportDP - if hintInfo != nil && hintInfo.leadingJoinOrder != nil { - if useGreedy { - ok, leftJoinGroup := baseGroupSolver.generateLeadingJoinGroup(curJoinGroup, hintInfo) - if !ok { - ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("leading hint is inapplicable, check if the leading hint table is valid")) + if hasOuterJoin { + ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("leading hint is inapplicable when we have outer join")) + } else { + if hintInfo != nil && hintInfo.leadingJoinOrder != nil { + if useGreedy { + ok, leftJoinGroup := baseGroupSolver.generateLeadingJoinGroup(curJoinGroup, hintInfo) + if !ok { + ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("leading hint is inapplicable, check if the leading hint table is valid")) + } else { + curJoinGroup = leftJoinGroup + } } else { - curJoinGroup = leftJoinGroup + ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("leading hint is inapplicable for the DP join reorder algorithm")) } - } else { - ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("leading hint is inapplicable for the DP join reorder algorithm")) } } diff --git a/planner/core/rule_join_reorder_test.go b/planner/core/rule_join_reorder_test.go index fb51e9b18f702..39f200cb27362 100644 --- a/planner/core/rule_join_reorder_test.go +++ b/planner/core/rule_join_reorder_test.go @@ -76,4 +76,26 @@ func TestLeadingJoinHint(t *testing.T) { 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") + + tk.MustExec("select /*+ leading(t1, t3) */ * from t1 left join t2 on t1.a=t2.a left join t3 on t2.b=t3.b") + res := tk.MustQuery("show warnings").Rows() + require.True(t, len(res) > 0) + tk.MustExec("select /*+ leading(t2) */ * from t1 left join t2 on t1.a=t2.a left join t3 on t2.b=t3.b") + res = tk.MustQuery("show warnings").Rows() + require.True(t, len(res) > 0) + tk.MustExec("select /*+ leading(t2, t3) */ * from t1 left join t2 on t1.a=t2.a left join t3 on t2.b=t3.b") + res = tk.MustQuery("show warnings").Rows() + require.True(t, len(res) > 0) + tk.MustExec("select /*+ leading(t1, t2, t3) */ * from t1 left join t2 on t1.a=t2.a left join t3 on t2.b=t3.b") + res = tk.MustQuery("show warnings").Rows() + require.True(t, len(res) > 0) + tk.MustExec("select /*+ leading(t1, t3) */ * from t1 join t2 on t1.a=t2.a left join t3 on t2.b=t3.b") + res = tk.MustQuery("show warnings").Rows() + require.True(t, len(res) > 0) + tk.MustExec("select /*+ leading(t1, t2) */ * from t1 join t2 on t1.a=t2.a left join t3 on t2.b=t3.b") + res = tk.MustQuery("show warnings").Rows() + require.True(t, len(res) > 0) + tk.MustExec("select /*+ leading(t3) */ * from t1 join t2 on t1.a=t2.a left join t3 on t2.b=t3.b") + res = tk.MustQuery("show warnings").Rows() + require.True(t, len(res) > 0) } From 2ad7cf9d7e063ea2660ac5ed742f36b1e5abb0b1 Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Mon, 16 May 2022 16:11:16 +0800 Subject: [PATCH 16/36] address comments --- planner/core/rule_join_reorder.go | 9 +++++---- planner/core/rule_join_reorder_greedy.go | 4 ++++ planner/core/rule_join_reorder_test.go | 5 +++++ 3 files changed, 14 insertions(+), 4 deletions(-) diff --git a/planner/core/rule_join_reorder.go b/planner/core/rule_join_reorder.go index 7112e55e075bb..5cf2397c8d5a6 100644 --- a/planner/core/rule_join_reorder.go +++ b/planner/core/rule_join_reorder.go @@ -180,10 +180,11 @@ func (s *joinReOrderSolver) optimizeRecursive(ctx sessionctx.Context, p LogicalP joinGroupNum := len(curJoinGroup) useGreedy := joinGroupNum > ctx.GetSessionVars().TiDBOptJoinReorderThreshold || !isSupportDP - if hasOuterJoin { - ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("leading hint is inapplicable when we have outer join")) - } else { - if hintInfo != nil && hintInfo.leadingJoinOrder != nil { + + if hintInfo != nil && hintInfo.leadingJoinOrder != nil { + if hasOuterJoin { + ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("leading hint is inapplicable when we have outer join")) + } else { if useGreedy { ok, leftJoinGroup := baseGroupSolver.generateLeadingJoinGroup(curJoinGroup, hintInfo) if !ok { diff --git a/planner/core/rule_join_reorder_greedy.go b/planner/core/rule_join_reorder_greedy.go index 6401d9a677c8c..905acbf251bb6 100644 --- a/planner/core/rule_join_reorder_greedy.go +++ b/planner/core/rule_join_reorder_greedy.go @@ -47,6 +47,8 @@ func (s *joinReorderGreedySolver) solve(joinNodePlans []LogicalPlan, tracer *joi } var leadingJoinNodes []*jrNode if s.leadingJoinGroup != nil { + // We have a leading hint to let some tables join first. The result is stored in the s.leadingJoinGroup. + // We generate jrNode separately for it. leadingJoinNodes, err = s.generateJoinOrderNode([]LogicalPlan{s.leadingJoinGroup}, tracer) if err != nil { return nil, err @@ -58,6 +60,8 @@ func (s *joinReorderGreedySolver) solve(joinNodePlans []LogicalPlan, tracer *joi }) if leadingJoinNodes != nil { + // The leadingJoinNodes should be the first element in the s.curJoinGroup. + // So it can be joined first. leadingJoinNodes := append(leadingJoinNodes, s.curJoinGroup...) s.curJoinGroup = leadingJoinNodes } diff --git a/planner/core/rule_join_reorder_test.go b/planner/core/rule_join_reorder_test.go index 39f200cb27362..7cbfb229d42c0 100644 --- a/planner/core/rule_join_reorder_test.go +++ b/planner/core/rule_join_reorder_test.go @@ -77,6 +77,7 @@ func TestLeadingJoinHint(t *testing.T) { tk.MustExec("create table t8(a int, b int, key(a));") runJoinReorderTestData(t, tk, "TestLeadingJoinHint") + // test cases for outer join tk.MustExec("select /*+ leading(t1, t3) */ * from t1 left join t2 on t1.a=t2.a left join t3 on t2.b=t3.b") res := tk.MustQuery("show warnings").Rows() require.True(t, len(res) > 0) @@ -98,4 +99,8 @@ func TestLeadingJoinHint(t *testing.T) { tk.MustExec("select /*+ leading(t3) */ * from t1 join t2 on t1.a=t2.a left join t3 on t2.b=t3.b") res = tk.MustQuery("show warnings").Rows() require.True(t, len(res) > 0) + + // 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")) } From 38fdc63b22205bf212182458190ee908487fb08e Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Mon, 16 May 2022 17:30:26 +0800 Subject: [PATCH 17/36] planner: add more test cases for join order hint --- planner/core/logical_plan_builder.go | 12 +++-- planner/core/rule_join_reorder_test.go | 69 ++++++++++++++++++++++++++ 2 files changed, 76 insertions(+), 5 deletions(-) diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 9666321410bca..b905f973d39fe 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -3630,12 +3630,14 @@ func (b *PlanBuilder) pushTableHints(hints []*ast.TableOptimizerHint, currentLev // ignore hints that not implemented } } - if leadingHintCnt > 1 { - // If there are more leading hints, all leading hints will be invalid. + if leadingHintCnt > 1 || (leadingHintCnt > 0 && b.ctx.GetSessionVars().StmtCtx.StraightJoinOrder) { + // If there are more leading hints or the straight_join hint existes, all leading hints will be invalid. leadingJoinOrder = leadingJoinOrder[:0] - // Append warning if there are invalid index names. - errMsg := "We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid" - b.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack(errMsg)) + if leadingHintCnt > 1 { + b.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid")) + } else if b.ctx.GetSessionVars().StmtCtx.StraightJoinOrder { + b.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("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")) + } } b.tableHintInfo = append(b.tableHintInfo, tableHintInfo{ sortMergeJoinTables: sortMergeTables, diff --git a/planner/core/rule_join_reorder_test.go b/planner/core/rule_join_reorder_test.go index 7cbfb229d42c0..d0b625ea6cdc1 100644 --- a/planner/core/rule_join_reorder_test.go +++ b/planner/core/rule_join_reorder_test.go @@ -104,3 +104,72 @@ func TestLeadingJoinHint(t *testing.T) { 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, clean := testkit.CreateMockStore(t) + defer clean() + + 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")) + + // test cases for using the join order hint and join algorithm hint + tk.MustExec("select /*+ 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")) + + runJoinReorderTestData(t, tk, "TestJoinOrderHint") +} + +func TestJoinOrderHintWithBinding(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + 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("select * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b") + tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("0")) + tk.MustExec("create global binding for select * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b using select /*+ straight_join() */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b") + tk.MustExec("select * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b") + tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("1")) + res := tk.MustQuery("show global bindings").Rows() + require.Equal(t, res[0][0], "select * from ( `test` . `t1` join `test` . `t2` on `t1` . `a` = `t2` . `a` ) join `test` . `t3` on `t2` . `b` = `t3` . `b`", "SELECT /*+ straight_join()*/ * FROM (`test`.`t1` JOIN `test`.`t2` ON `t1`.`a` = `t2`.`a`) JOIN `test`.`t3` ON `t2`.`b` = `t3`.`b`") + + tk.MustExec("drop global binding for select * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b") + tk.MustExec("select * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b") + tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("0")) + res = tk.MustQuery("show global bindings").Rows() + require.Equal(t, len(res), 0) + + tk.MustExec("create global binding for select * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b using select /*+ leading(t3) */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b") + tk.MustExec("select * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b") + tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("1")) + res = tk.MustQuery("show global bindings").Rows() + require.Equal(t, res[0][0], "select * from ( `test` . `t1` join `test` . `t2` on `t1` . `a` = `t2` . `a` ) join `test` . `t3` on `t2` . `b` = `t3` . `b`") + + tk.MustExec("drop global binding for select * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b") +} From 6b3793f327b3c5880dbc6811c56b7607653281ab Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Tue, 17 May 2022 17:44:28 +0800 Subject: [PATCH 18/36] planner: add more test cases for join order hint --- planner/core/rule_join_reorder.go | 34 +- planner/core/rule_join_reorder_test.go | 120 ++- .../core/testdata/join_reorder_suite_in.json | 67 ++ .../core/testdata/join_reorder_suite_out.json | 886 ++++++++++++++++++ 4 files changed, 1092 insertions(+), 15 deletions(-) diff --git a/planner/core/rule_join_reorder.go b/planner/core/rule_join_reorder.go index 5cf2397c8d5a6..49b35146f9dd1 100644 --- a/planner/core/rule_join_reorder.go +++ b/planner/core/rule_join_reorder.go @@ -34,7 +34,7 @@ import ( // For example: "InnerJoin(InnerJoin(a, b), LeftJoin(c, d))" // results in a join group {a, b, c, d}. func extractJoinGroup(p LogicalPlan) (group []LogicalPlan, eqEdges []*expression.ScalarFunction, - otherConds []expression.Expression, joinTypes []JoinType, hintInfo *tableHintInfo, hasOuterJoin bool) { + otherConds []expression.Expression, joinTypes []JoinType, hintInfo []*tableHintInfo, hasOuterJoin bool) { join, isJoin := p.(*LogicalJoin) if !isJoin || join.preferJoinType > uint(0) || join.StraightJoin || (join.JoinType != InnerJoin && join.JoinType != LeftOuterJoin && join.JoinType != RightOuterJoin) || @@ -42,7 +42,7 @@ func extractJoinGroup(p LogicalPlan) (group []LogicalPlan, eqEdges []*expression return []LogicalPlan{p}, nil, nil, nil, nil, false } if join.preferJoinOrder { - hintInfo = join.hintInfo + hintInfo = append(hintInfo, join.hintInfo) } hasOuterJoin = hasOuterJoin || (join.JoinType != InnerJoin) if join.JoinType != RightOuterJoin { @@ -75,9 +75,7 @@ func extractJoinGroup(p LogicalPlan) (group []LogicalPlan, eqEdges []*expression eqEdges = append(eqEdges, lhsEqualConds...) otherConds = append(otherConds, lhsOtherConds...) joinTypes = append(joinTypes, lhsJoinTypes...) - if hintInfo == nil && lhsHintInfo != nil { - hintInfo = lhsHintInfo - } + hintInfo = append(hintInfo, lhsHintInfo...) hasOuterJoin = hasOuterJoin || lhsHasOuterJoin } else { group = append(group, join.children[0]) @@ -113,9 +111,7 @@ func extractJoinGroup(p LogicalPlan) (group []LogicalPlan, eqEdges []*expression eqEdges = append(eqEdges, rhsEqualConds...) otherConds = append(otherConds, rhsOtherConds...) joinTypes = append(joinTypes, rhsJoinTypes...) - if hintInfo == nil && rhsHintInfo != nil { - hintInfo = rhsHintInfo - } + hintInfo = append(hintInfo, rhsHintInfo...) hasOuterJoin = hasOuterJoin || rhsHasOuterJoin } else { group = append(group, join.children[1]) @@ -181,12 +177,30 @@ func (s *joinReOrderSolver) optimizeRecursive(ctx sessionctx.Context, p LogicalP joinGroupNum := len(curJoinGroup) useGreedy := joinGroupNum > ctx.GetSessionVars().TiDBOptJoinReorderThreshold || !isSupportDP - if hintInfo != nil && hintInfo.leadingJoinOrder != nil { + hintInfoNum := len(hintInfo) + var leadingHintInfo *tableHintInfo + if hintInfoNum > 0 { + hasDiffLeadingHint := false + leadingHintInfo = hintInfo[0] + // One join group has one leading hint at most. Check whether there are different join order hints. + for i := 1; i < hintInfoNum; i++ { + if hintInfo[i] != hintInfo[i-1] { + hasDiffLeadingHint = true + break + } + } + if hasDiffLeadingHint { + ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid")) + leadingHintInfo = nil + } + } + + if leadingHintInfo != nil && leadingHintInfo.leadingJoinOrder != nil { if hasOuterJoin { ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("leading hint is inapplicable when we have outer join")) } else { if useGreedy { - ok, leftJoinGroup := baseGroupSolver.generateLeadingJoinGroup(curJoinGroup, hintInfo) + ok, leftJoinGroup := baseGroupSolver.generateLeadingJoinGroup(curJoinGroup, leadingHintInfo) if !ok { ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("leading hint is inapplicable, check if the leading hint table is valid")) } else { diff --git a/planner/core/rule_join_reorder_test.go b/planner/core/rule_join_reorder_test.go index d0b625ea6cdc1..b56a966007293 100644 --- a/planner/core/rule_join_reorder_test.go +++ b/planner/core/rule_join_reorder_test.go @@ -129,12 +129,69 @@ func TestJoinOrderHint(t *testing.T) { 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")) - // test cases for using the join order hint and join algorithm hint - tk.MustExec("select /*+ 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")) + // 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 /*+ 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")) + 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")) + + // conflict between table names + tk.MustExec("select /*+ leading(t3) */ * from t1 join t2 on t1.a=t2.a left join t3 on t2.b=t3.b") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 leading hint is inapplicable when we have outer join")) + + tk.MustExec("select /*+ leading(t1, t3) */ * from t1 join t2 on t1.a=t2.a left join t3 on t2.b=t3.b") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 leading hint is inapplicable when we have outer join")) + + // 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") } @@ -173,3 +230,56 @@ func TestJoinOrderHintWithBinding(t *testing.T) { tk.MustExec("drop global binding for select * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b") } + +func TestJoinOrderHint4StaticPartitionTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + 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(`set @@tidb_partition_prune_mode="static"`) + runJoinReorderTestData(t, tk, "TestJoinOrderHint4StaticPartitionTable") +} + +func TestJoinOrderHint4DynamicPartitionTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + 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(`set @@tidb_partition_prune_mode="dynamic"`) + runJoinReorderTestData(t, tk, "TestJoinOrderHint4DynamicPartitionTable") +} + +func TestJoinOrderHint4DifferentJoinType(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + 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));") + + // inner join and outer join have already been tested. + runJoinReorderTestData(t, tk, "TestJoinOrderHint4DifferentJoinType") +} diff --git a/planner/core/testdata/join_reorder_suite_in.json b/planner/core/testdata/join_reorder_suite_in.json index 5821f26519638..0873613a72f98 100644 --- a/planner/core/testdata/join_reorder_suite_in.json +++ b/planner/core/testdata/join_reorder_suite_in.json @@ -99,5 +99,72 @@ "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;" ] + }, + { + "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 /*+ leading(t1) hash_join(t1) */ * 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(t1) merge_join(t1) */ * 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;" + ] + }, + { + "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;" + ] + }, + { + "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;" + ] + }, + { + "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 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", + + // 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" + ] } ] diff --git a/planner/core/testdata/join_reorder_suite_out.json b/planner/core/testdata/join_reorder_suite_out.json index 6c85caf9f275d..cb9357579d6b9 100644 --- a/planner/core/testdata/join_reorder_suite_out.json +++ b/planner/core/testdata/join_reorder_suite_out.json @@ -2078,5 +2078,891 @@ ] } ] + }, + { + "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 /*+ leading(t1) 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.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.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) 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.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.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(t1) merge_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.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", + " └─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 /*+ 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", + " └─Projection 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" + ], + "Warning": null + } + ] + }, + { + "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" + ] + } + ] + }, + { + "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 + } + ] + }, + { + "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 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 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 when we have outer join" + ] + }, + { + "SQL": "select /*+ leading(t3) */ * 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 when we have outer join" + ] + }, + { + "SQL": "select /*+ leading(t2, t3) */ * 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 when we have outer join" + ] + }, + { + "SQL": "select /*+ leading(t3, t2) */ * 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 when we have outer join" + ] + }, + { + "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 when we have outer join" + ] + }, + { + "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": [ + "Warning 1815 leading hint is inapplicable when we have outer join" + ] + }, + { + "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 when we have outer join" + ] + }, + { + "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 when we have outer join" + ] + }, + { + "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 when we have outer join" + ] + }, + { + "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": null + }, + { + "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 + } + ] } ] From cf00fbb987a095191c079417de9d04a819af4406 Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Tue, 17 May 2022 17:55:35 +0800 Subject: [PATCH 19/36] revert --- planner/core/rule_join_reorder_test.go | 21 +++++++-------------- 1 file changed, 7 insertions(+), 14 deletions(-) diff --git a/planner/core/rule_join_reorder_test.go b/planner/core/rule_join_reorder_test.go index b56a966007293..668c2ee9b2e9c 100644 --- a/planner/core/rule_join_reorder_test.go +++ b/planner/core/rule_join_reorder_test.go @@ -79,26 +79,19 @@ func TestLeadingJoinHint(t *testing.T) { // test cases for outer join tk.MustExec("select /*+ leading(t1, t3) */ * from t1 left join t2 on t1.a=t2.a left join t3 on t2.b=t3.b") - res := tk.MustQuery("show warnings").Rows() - require.True(t, len(res) > 0) + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 leading hint is inapplicable when we have outer join")) tk.MustExec("select /*+ leading(t2) */ * from t1 left join t2 on t1.a=t2.a left join t3 on t2.b=t3.b") - res = tk.MustQuery("show warnings").Rows() - require.True(t, len(res) > 0) + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 leading hint is inapplicable when we have outer join")) tk.MustExec("select /*+ leading(t2, t3) */ * from t1 left join t2 on t1.a=t2.a left join t3 on t2.b=t3.b") - res = tk.MustQuery("show warnings").Rows() - require.True(t, len(res) > 0) + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 leading hint is inapplicable when we have outer join")) tk.MustExec("select /*+ leading(t1, t2, t3) */ * from t1 left join t2 on t1.a=t2.a left join t3 on t2.b=t3.b") - res = tk.MustQuery("show warnings").Rows() - require.True(t, len(res) > 0) + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 leading hint is inapplicable when we have outer join")) tk.MustExec("select /*+ leading(t1, t3) */ * from t1 join t2 on t1.a=t2.a left join t3 on t2.b=t3.b") - res = tk.MustQuery("show warnings").Rows() - require.True(t, len(res) > 0) + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 leading hint is inapplicable when we have outer join")) tk.MustExec("select /*+ leading(t1, t2) */ * from t1 join t2 on t1.a=t2.a left join t3 on t2.b=t3.b") - res = tk.MustQuery("show warnings").Rows() - require.True(t, len(res) > 0) + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 leading hint is inapplicable when we have outer join")) tk.MustExec("select /*+ leading(t3) */ * from t1 join t2 on t1.a=t2.a left join t3 on t2.b=t3.b") - res = tk.MustQuery("show warnings").Rows() - require.True(t, len(res) > 0) + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 leading hint is inapplicable when we have outer join")) // 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") From e880d70e39f1c2c31eea0ab3ca543dd2d6eeb93d Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Tue, 17 May 2022 18:05:07 +0800 Subject: [PATCH 20/36] rename --- planner/core/rule_join_reorder.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/planner/core/rule_join_reorder.go b/planner/core/rule_join_reorder.go index 49b35146f9dd1..5d35669f1ed52 100644 --- a/planner/core/rule_join_reorder.go +++ b/planner/core/rule_join_reorder.go @@ -177,13 +177,13 @@ func (s *joinReOrderSolver) optimizeRecursive(ctx sessionctx.Context, p LogicalP joinGroupNum := len(curJoinGroup) useGreedy := joinGroupNum > ctx.GetSessionVars().TiDBOptJoinReorderThreshold || !isSupportDP - hintInfoNum := len(hintInfo) + leadingHintNum := len(hintInfo) var leadingHintInfo *tableHintInfo - if hintInfoNum > 0 { + if leadingHintNum > 0 { hasDiffLeadingHint := false leadingHintInfo = hintInfo[0] // One join group has one leading hint at most. Check whether there are different join order hints. - for i := 1; i < hintInfoNum; i++ { + for i := 1; i < leadingHintNum; i++ { if hintInfo[i] != hintInfo[i-1] { hasDiffLeadingHint = true break From a54c4c9e5a3d988711d0a071e5ed63567b4013af Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Tue, 17 May 2022 18:16:16 +0800 Subject: [PATCH 21/36] add test cases for tiflash --- planner/core/rule_join_reorder_test.go | 33 ++++- .../core/testdata/join_reorder_suite_in.json | 11 ++ .../core/testdata/join_reorder_suite_out.json | 130 ++++++++++++++++++ 3 files changed, 173 insertions(+), 1 deletion(-) diff --git a/planner/core/rule_join_reorder_test.go b/planner/core/rule_join_reorder_test.go index 668c2ee9b2e9c..1db95604c2173 100644 --- a/planner/core/rule_join_reorder_test.go +++ b/planner/core/rule_join_reorder_test.go @@ -15,6 +15,8 @@ package core_test import ( + "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/parser/model" "testing" plannercore "github.com/pingcap/tidb/planner/core" @@ -273,6 +275,35 @@ func TestJoinOrderHint4DifferentJoinType(t *testing.T) { tk.MustExec("create table t7(a int, b int, key(a));") tk.MustExec("create table t8(a int, b int, key(a));") - // inner join and outer join have already been tested. runJoinReorderTestData(t, tk, "TestJoinOrderHint4DifferentJoinType") } + +func TestJoinOrderHint4TiFlash(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + 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));") + + // 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" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1;") + runJoinReorderTestData(t, tk, "TestJoinOrderHint4TiFlash") +} diff --git a/planner/core/testdata/join_reorder_suite_in.json b/planner/core/testdata/join_reorder_suite_in.json index 0873613a72f98..411adca102913 100644 --- a/planner/core/testdata/join_reorder_suite_in.json +++ b/planner/core/testdata/join_reorder_suite_in.json @@ -166,5 +166,16 @@ "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" ] + }, + { + "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" + ] } ] diff --git a/planner/core/testdata/join_reorder_suite_out.json b/planner/core/testdata/join_reorder_suite_out.json index cb9357579d6b9..3b4c6226e1b14 100644 --- a/planner/core/testdata/join_reorder_suite_out.json +++ b/planner/core/testdata/join_reorder_suite_out.json @@ -2964,5 +2964,135 @@ "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 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", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t3 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", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t1.a))", + " └─TableFullScan 10000.00 mpp[tiflash] 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": [ + "TableReader 15593.77 root 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", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t3 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", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t1.a))", + " └─TableFullScan 10000.00 mpp[tiflash] 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": [ + "TableReader 15593.77 root 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", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 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", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t3.b))", + " └─TableFullScan 10000.00 mpp[tiflash] 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": [ + "TableReader 15593.77 root 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", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 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", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t3.b))", + " └─TableFullScan 10000.00 mpp[tiflash] 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": [ + "TableReader 15593.77 root 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", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 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", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t3.b))", + " └─TableFullScan 10000.00 mpp[tiflash] 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": [ + "TableReader 124625374.88 root 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", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 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", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t3 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t1.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ], + "Warning": null + } + ] } ] From cf8fa62f70666539bc7dc19f09373e2be8deeaf5 Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Wed, 18 May 2022 10:14:47 +0800 Subject: [PATCH 22/36] format --- planner/core/rule_join_reorder_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/planner/core/rule_join_reorder_test.go b/planner/core/rule_join_reorder_test.go index 1db95604c2173..f972df6151ebe 100644 --- a/planner/core/rule_join_reorder_test.go +++ b/planner/core/rule_join_reorder_test.go @@ -15,10 +15,10 @@ package core_test import ( - "github.com/pingcap/tidb/domain" - "github.com/pingcap/tidb/parser/model" "testing" + "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/parser/model" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/testkit/testdata" From 544d1236d9fb991bd62e84bbe1bf5d3c7eada9b5 Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Wed, 18 May 2022 11:38:37 +0800 Subject: [PATCH 23/36] add test cases for subquery --- planner/core/rule_join_reorder_test.go | 20 + .../core/testdata/join_reorder_suite_in.json | 100 ++ .../core/testdata/join_reorder_suite_out.json | 1332 +++++++++++++++++ 3 files changed, 1452 insertions(+) diff --git a/planner/core/rule_join_reorder_test.go b/planner/core/rule_join_reorder_test.go index f972df6151ebe..27d679e0f50d8 100644 --- a/planner/core/rule_join_reorder_test.go +++ b/planner/core/rule_join_reorder_test.go @@ -307,3 +307,23 @@ func TestJoinOrderHint4TiFlash(t *testing.T) { tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1;") runJoinReorderTestData(t, tk, "TestJoinOrderHint4TiFlash") } + +func TestJoinOrderHint4Subquery(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + 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));") + + runJoinReorderTestData(t, tk, "TestJoinOrderHint4Subquery") +} diff --git a/planner/core/testdata/join_reorder_suite_in.json b/planner/core/testdata/join_reorder_suite_in.json index 411adca102913..3d9f43ef1bd12 100644 --- a/planner/core/testdata/join_reorder_suite_in.json +++ b/planner/core/testdata/join_reorder_suite_in.json @@ -177,5 +177,105 @@ "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" ] + }, + { + "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(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(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(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(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(t1) */ * 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) */ * 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(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(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;", + + // leading hint + uncorrelated subquery + "select /*+ leading(t1) */ * from t1 join t2 on t1.a=t2.a 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(t1) */ * from t1 join t2 on t1.a=t2.a 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(t1) */ * from t1 join t2 on t1.a=t2.a 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(t1) */ * from t1 join t2 on t1.a=t2.a 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;" + ] } ] diff --git a/planner/core/testdata/join_reorder_suite_out.json b/planner/core/testdata/join_reorder_suite_out.json index 3b4c6226e1b14..0209b12b48bae 100644 --- a/planner/core/testdata/join_reorder_suite_out.json +++ b/planner/core/testdata/join_reorder_suite_out.json @@ -3094,5 +3094,1337 @@ "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) 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": 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) 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 /*+ 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) 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 /*+ 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) 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 /*+ 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", + "└─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", + " └─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", + "└─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", + " └─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 12487.50 root inner join, equal:[eq(test.t1.a, test.t3.a)]", + "├─HashAgg(Build) 7992.00 root group by:test.t3.a, funcs:firstrow(test.t3.a)->test.t3.a", + "│ └─IndexReader 9990.00 root index:IndexFullScan", + "│ └─IndexFullScan 9990.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": 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 CARTESIAN anti semi join, other cond: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": null + }, + { + "SQL": "select /*+ straight_join() */ * from t1 join t2 on t1.a=t2.a where exists (select t3.a from t3)", + "Plan": [ + "TableDual 0.00 root rows:0" + ], + "Warning": null + }, + { + "SQL": "select /*+ straight_join() */ * 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": 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 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.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 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.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 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) */ * 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) 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": 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) 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 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 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 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 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 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 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 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 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 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(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) 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(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) 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 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 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 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 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 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 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 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 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 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(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) 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@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) 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 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 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 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 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 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 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 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 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 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(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) 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(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) 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 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 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 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 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 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 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 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 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 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(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", + "└─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", + " └─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", + "└─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", + " └─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", + "└─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", + " └─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", + "└─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", + " └─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", + "└─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", + " └─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", + "└─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", + " └─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(t1) */ * 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)]", + "├─HashAgg(Build) 7992.00 root group by:test.t3.a, funcs:firstrow(test.t3.a)->test.t3.a", + "│ └─IndexReader 9990.00 root index:IndexFullScan", + "│ └─IndexFullScan 9990.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": 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": [ + "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", + "└─HashJoin(Probe) 9990.00 root inner join, equal:[eq(test.t3.a, test.t1.a)]", + " ├─HashAgg(Build) 7992.00 root group by:test.t3.a, funcs:firstrow(test.t3.a)->test.t3.a", + " │ └─IndexReader 9990.00 root index:IndexFullScan", + " │ └─IndexFullScan 9990.00 cop[tikv] table:t3, index:a(a) 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@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)]", + "├─HashAgg(Build) 7992.00 root group by:test.t3.a, funcs:firstrow(test.t3.a)->test.t3.a", + "│ └─IndexReader 9990.00 root index:IndexFullScan", + "│ └─IndexFullScan 9990.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 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": [ + "HashJoin 12487.50 root inner join, equal:[eq(test.t1.a, test.t3.a)]", + "├─HashAgg(Build) 7992.00 root group by:test.t3.a, funcs:firstrow(test.t3.a)->test.t3.a", + "│ └─IndexReader 9990.00 root index:IndexFullScan", + "│ └─IndexFullScan 9990.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 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": [ + "HashJoin 12487.50 root inner join, equal:[eq(test.t1.a, test.t3.a)]", + "├─HashAgg(Build) 7992.00 root group by:test.t3.a, funcs:firstrow(test.t3.a)->test.t3.a", + "│ └─IndexReader 9990.00 root index:IndexFullScan", + "│ └─IndexFullScan 9990.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 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": [ + "HashJoin 12487.50 root inner join, equal:[eq(test.t1.a, test.t3.a)]", + "├─HashAgg(Build) 7992.00 root group by:test.t3.a, funcs:firstrow(test.t3.a)->test.t3.a", + "│ └─IndexReader 9990.00 root index:IndexFullScan", + "│ └─IndexFullScan 9990.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 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) */ * from t1 join t2 on t1.a=t2.a where t1.a not in (select t3.a from t3)", + "Plan": [ + "HashJoin 9990.00 root CARTESIAN anti semi join, other cond: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": 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 CARTESIAN anti semi join, other cond: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 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 t1.a not in (select t3.a from t3)", + "Plan": [ + "HashJoin 9990.00 root CARTESIAN anti semi join, other cond: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 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 t1.a not in (select t3.a from t3)", + "Plan": [ + "HashJoin 9990.00 root CARTESIAN anti semi join, other cond: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 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 t1.a not in (select t3.a from t3)", + "Plan": [ + "HashJoin 9990.00 root CARTESIAN anti semi join, other cond: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 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 t1.a not in (select t3.a from t3)", + "Plan": [ + "HashJoin 9990.00 root CARTESIAN anti semi join, other cond: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 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(t1) */ * from t1 join t2 on t1.a=t2.a where 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 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(t1, 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(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 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 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(t1) */ * 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": 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": [ + "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 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(t1, 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(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 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(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 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(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@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 join t2 on t1.a=t2.a join t3 on t2.b=t3.b;", + "Plan": [ + "Projection 15593.77 root test.t4.a, test.t4.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.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.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.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" + ] + }, + { + "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 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(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 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(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 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(t3, 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(t3, t2) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + } + ] } ] From be2e7735c322527959038e204832243eb1031ac2 Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Wed, 18 May 2022 11:53:33 +0800 Subject: [PATCH 24/36] refactor the code --- planner/core/rule_join_reorder.go | 42 +++++++++++++++++++------------ 1 file changed, 26 insertions(+), 16 deletions(-) diff --git a/planner/core/rule_join_reorder.go b/planner/core/rule_join_reorder.go index 5d35669f1ed52..5ac207a7cd815 100644 --- a/planner/core/rule_join_reorder.go +++ b/planner/core/rule_join_reorder.go @@ -177,22 +177,9 @@ func (s *joinReOrderSolver) optimizeRecursive(ctx sessionctx.Context, p LogicalP joinGroupNum := len(curJoinGroup) useGreedy := joinGroupNum > ctx.GetSessionVars().TiDBOptJoinReorderThreshold || !isSupportDP - leadingHintNum := len(hintInfo) - var leadingHintInfo *tableHintInfo - if leadingHintNum > 0 { - hasDiffLeadingHint := false - leadingHintInfo = hintInfo[0] - // One join group has one leading hint at most. Check whether there are different join order hints. - for i := 1; i < leadingHintNum; i++ { - if hintInfo[i] != hintInfo[i-1] { - hasDiffLeadingHint = true - break - } - } - if hasDiffLeadingHint { - ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid")) - leadingHintInfo = nil - } + leadingHintInfo, hasDiffLeadingHint := checkAndGenerateLeadingHint(hintInfo) + if hasDiffLeadingHint { + ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid")) } if leadingHintInfo != nil && leadingHintInfo.leadingJoinOrder != nil { @@ -260,6 +247,29 @@ func (s *joinReOrderSolver) optimizeRecursive(ctx sessionctx.Context, p LogicalP return p, nil } +// checkAndGenerateLeadingHint used to check and generate the valid leading hint. +// We are allowed to use at most one leading hint in a join group. When more than one, +// all leading hints in the current join group will be invalid. +func checkAndGenerateLeadingHint(hintInfo []*tableHintInfo) (*tableHintInfo, bool) { + leadingHintNum := len(hintInfo) + var leadingHintInfo *tableHintInfo + hasDiffLeadingHint := false + if leadingHintNum > 0 { + leadingHintInfo = hintInfo[0] + // One join group has one leading hint at most. Check whether there are different join order hints. + for i := 1; i < leadingHintNum; i++ { + if hintInfo[i] != hintInfo[i-1] { + hasDiffLeadingHint = true + break + } + } + if hasDiffLeadingHint { + leadingHintInfo = nil + } + } + return leadingHintInfo, hasDiffLeadingHint +} + // nolint:structcheck type baseSingleGroupJoinOrderSolver struct { ctx sessionctx.Context From 8aa6eee28ed6a1f50afd804ce135a9de7afd4964 Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Wed, 18 May 2022 15:19:08 +0800 Subject: [PATCH 25/36] add more comments --- planner/core/rule_join_reorder.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/planner/core/rule_join_reorder.go b/planner/core/rule_join_reorder.go index 5ac207a7cd815..6fab84068b5e3 100644 --- a/planner/core/rule_join_reorder.go +++ b/planner/core/rule_join_reorder.go @@ -250,6 +250,10 @@ func (s *joinReOrderSolver) optimizeRecursive(ctx sessionctx.Context, p LogicalP // checkAndGenerateLeadingHint used to check and generate the valid leading hint. // We are allowed to use at most one leading hint in a join group. When more than one, // all leading hints in the current join group will be invalid. +// For example: 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 +// The Join Group {t1, t2, t3} contains two leading hints includes leading(t3) and leading(t1). +// Although they are in different query blocks, they are conflicting. +// In addition, the table alias 't4' cannot be recognized because of the join group. func checkAndGenerateLeadingHint(hintInfo []*tableHintInfo) (*tableHintInfo, bool) { leadingHintNum := len(hintInfo) var leadingHintInfo *tableHintInfo From 3af51166529d8c9e6f7219fe8635b7497859a506 Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Wed, 18 May 2022 15:24:41 +0800 Subject: [PATCH 26/36] add more test cases for join algorithm --- .../core/testdata/join_reorder_suite_in.json | 5 +- .../core/testdata/join_reorder_suite_out.json | 65 +++++++++++++++++++ 2 files changed, 69 insertions(+), 1 deletion(-) diff --git a/planner/core/testdata/join_reorder_suite_in.json b/planner/core/testdata/join_reorder_suite_in.json index 3d9f43ef1bd12..db22d64d310d5 100644 --- a/planner/core/testdata/join_reorder_suite_in.json +++ b/planner/core/testdata/join_reorder_suite_in.json @@ -106,10 +106,13 @@ // 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(t1) hash_join(t1) */ * 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(t1) merge_join(t1) */ * 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(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;" ] }, { diff --git a/planner/core/testdata/join_reorder_suite_out.json b/planner/core/testdata/join_reorder_suite_out.json index 0209b12b48bae..a70825ccd8148 100644 --- a/planner/core/testdata/join_reorder_suite_out.json +++ b/planner/core/testdata/join_reorder_suite_out.json @@ -2119,6 +2119,25 @@ ], "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) 1.25 root ", + " ├─Selection(Build) 1.25 cop[tikv] not(isnull(test.t1.a))", + " │ └─IndexRangeScan 1.25 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, test.t.a)], keep order:false, stats:pseudo", + " └─Selection(Probe) 1.25 cop[tikv] not(isnull(test.t1.b))", + " └─TableRowIDScan 1.25 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": null + }, { "SQL": "select /*+ leading(t1) hash_join(t1) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", "Plan": [ @@ -2161,6 +2180,29 @@ ], "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.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", + " └─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) 1.25 root ", + " ├─Selection(Build) 1.25 cop[tikv] not(isnull(test.t1.a))", + " │ └─IndexRangeScan 1.25 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, test.t.a)], keep order:false, stats:pseudo", + " └─Selection(Probe) 1.25 cop[tikv] not(isnull(test.t1.b))", + " └─TableRowIDScan 1.25 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": null + }, { "SQL": "select /*+ leading(t1) merge_join(t1) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", "Plan": [ @@ -2208,6 +2250,29 @@ " └─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.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", + " └─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) 1.25 root ", + " ├─Selection(Build) 1.25 cop[tikv] not(isnull(test.t1.a))", + " │ └─IndexRangeScan 1.25 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, test.t.a)], keep order:false, stats:pseudo", + " └─Selection(Probe) 1.25 cop[tikv] not(isnull(test.t1.b))", + " └─TableRowIDScan 1.25 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": null } ] }, From 8799ae7fdb413473761d9dcff848bf44ff602da0 Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Wed, 18 May 2022 15:51:27 +0800 Subject: [PATCH 27/36] add more test cases for different join type --- .../core/testdata/join_reorder_suite_in.json | 20 + .../core/testdata/join_reorder_suite_out.json | 500 +++++------------- 2 files changed, 165 insertions(+), 355 deletions(-) diff --git a/planner/core/testdata/join_reorder_suite_in.json b/planner/core/testdata/join_reorder_suite_in.json index db22d64d310d5..82011a7ca01fa 100644 --- a/planner/core/testdata/join_reorder_suite_in.json +++ b/planner/core/testdata/join_reorder_suite_in.json @@ -137,8 +137,11 @@ "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 left full join t3 on t2.b=t3.b", + "select /*+ straight_join() */ * from t1 join t2 on t1.a=t2.a right full join t3 on t2.b=t3.b", "select /*+ straight_join() */ * from t1 join t2 on t1.a=t2.a cross join t3", // inner join @@ -148,12 +151,24 @@ "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", + "select /*+ leading(t2) */ * from t1 join t2 on t1.a=t2.a left full join t3 on t2.b=t3.b", + "select /*+ leading(t3) */ * from t1 join t2 on t1.a=t2.a left full join t3 on t2.b=t3.b", + "select /*+ leading(t2, t3) */ * from t1 join t2 on t1.a=t2.a left full join t3 on t2.b=t3.b", + "select /*+ leading(t3, t2) */ * from t1 join t2 on t1.a=t2.a left full join t3 on t2.b=t3.b", + "select /*+ leading(t3, t1) */ * from t1 join t2 on t1.a=t2.a left full 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", @@ -161,6 +176,11 @@ "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", + "select /*+ leading(t2) */ * from t1 join t2 on t1.a=t2.a right full join t3 on t2.b=t3.b", + "select /*+ leading(t3) */ * from t1 join t2 on t1.a=t2.a right full join t3 on t2.b=t3.b", + "select /*+ leading(t2, t3) */ * from t1 join t2 on t1.a=t2.a right full join t3 on t2.b=t3.b", + "select /*+ leading(t3, t2) */ * from t1 join t2 on t1.a=t2.a right full join t3 on t2.b=t3.b", + "select /*+ leading(t3, t1) */ * from t1 join t2 on t1.a=t2.a right full join t3 on t2.b=t3.b", // cross join "select /*+ leading(t2) */ * from t1 join t2 on t1.a=t2.a cross join t3", diff --git a/planner/core/testdata/join_reorder_suite_out.json b/planner/core/testdata/join_reorder_suite_out.json index a70825ccd8148..87e1ba6fb7ab7 100644 --- a/planner/core/testdata/join_reorder_suite_out.json +++ b/planner/core/testdata/join_reorder_suite_out.json @@ -2614,418 +2614,208 @@ "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" - ], + "SQL": "select /*+ straight_join() */ * from t1 straight join t2 straight join t3", + "Plan": null, "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" - ], + "SQL": "", + "Plan": null, "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" - ], + "SQL": "", + "Plan": null, "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" - ], + "SQL": "", + "Plan": null, "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" - ], + "SQL": "", + "Plan": null, "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" - ], + "SQL": "", + "Plan": null, "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" - ], + "SQL": "", + "Plan": null, "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" - ], + "SQL": "", + "Plan": null, "Warning": null }, { - "SQL": "select /*+ leading(t2) */ * 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 when we have outer join" - ] + "SQL": "", + "Plan": null, + "Warning": null }, { - "SQL": "select /*+ leading(t3) */ * 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 when we have outer join" - ] + "SQL": "", + "Plan": null, + "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": [ - "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 when we have outer join" - ] + "SQL": "", + "Plan": null, + "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": [ - "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 when we have outer join" - ] + "SQL": "", + "Plan": null, + "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 when we have outer join" - ] + "SQL": "", + "Plan": null, + "Warning": null }, { - "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" - ], + "SQL": "", + "Plan": null, "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": [ - "Warning 1815 leading hint is inapplicable when we have outer join" - ] + "SQL": "", + "Plan": null, + "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 when we have outer join" - ] + "SQL": "", + "Plan": null, + "Warning": null }, { - "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 when we have outer join" - ] + "SQL": "", + "Plan": null, + "Warning": null }, { - "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 when we have outer join" - ] + "SQL": "", + "Plan": null, + "Warning": null }, { - "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" - ], + "SQL": "", + "Plan": null, "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" - ], + "SQL": "", + "Plan": null, "Warning": null }, { - "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" - ], + "SQL": "", + "Plan": null, "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" - ], + "SQL": "", + "Plan": null, "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" - ], + "SQL": "", + "Plan": null, + "Warning": null + }, + { + "SQL": "", + "Plan": null, + "Warning": null + }, + { + "SQL": "", + "Plan": null, + "Warning": null + }, + { + "SQL": "", + "Plan": null, + "Warning": null + }, + { + "SQL": "", + "Plan": null, + "Warning": null + }, + { + "SQL": "", + "Plan": null, + "Warning": null + }, + { + "SQL": "", + "Plan": null, + "Warning": null + }, + { + "SQL": "", + "Plan": null, + "Warning": null + }, + { + "SQL": "", + "Plan": null, + "Warning": null + }, + { + "SQL": "", + "Plan": null, + "Warning": null + }, + { + "SQL": "", + "Plan": null, + "Warning": null + }, + { + "SQL": "", + "Plan": null, + "Warning": null + }, + { + "SQL": "", + "Plan": null, + "Warning": null + }, + { + "SQL": "", + "Plan": null, + "Warning": null + }, + { + "SQL": "", + "Plan": null, + "Warning": null + }, + { + "SQL": "", + "Plan": null, + "Warning": null + }, + { + "SQL": "", + "Plan": null, + "Warning": null + }, + { + "SQL": "", + "Plan": null, + "Warning": null + }, + { + "SQL": "", + "Plan": null, "Warning": null } ] From ac191a649bf9fc46dc57e1b19cf0000ad03a8dc9 Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Wed, 18 May 2022 16:11:09 +0800 Subject: [PATCH 28/36] add more test cases for different join types --- .../core/testdata/join_reorder_suite_in.json | 24 +- .../core/testdata/join_reorder_suite_out.json | 548 ++++++++++++++---- 2 files changed, 427 insertions(+), 145 deletions(-) diff --git a/planner/core/testdata/join_reorder_suite_in.json b/planner/core/testdata/join_reorder_suite_in.json index 82011a7ca01fa..bb2ad55ce9ab1 100644 --- a/planner/core/testdata/join_reorder_suite_in.json +++ b/planner/core/testdata/join_reorder_suite_in.json @@ -137,11 +137,9 @@ "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 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 left full join t3 on t2.b=t3.b", - "select /*+ straight_join() */ * from t1 join t2 on t1.a=t2.a right full join t3 on t2.b=t3.b", "select /*+ straight_join() */ * from t1 join t2 on t1.a=t2.a cross join t3", // inner join @@ -152,11 +150,11 @@ "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", + "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", @@ -164,11 +162,6 @@ "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", - "select /*+ leading(t2) */ * from t1 join t2 on t1.a=t2.a left full join t3 on t2.b=t3.b", - "select /*+ leading(t3) */ * from t1 join t2 on t1.a=t2.a left full join t3 on t2.b=t3.b", - "select /*+ leading(t2, t3) */ * from t1 join t2 on t1.a=t2.a left full join t3 on t2.b=t3.b", - "select /*+ leading(t3, t2) */ * from t1 join t2 on t1.a=t2.a left full join t3 on t2.b=t3.b", - "select /*+ leading(t3, t1) */ * from t1 join t2 on t1.a=t2.a left full 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", @@ -176,11 +169,6 @@ "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", - "select /*+ leading(t2) */ * from t1 join t2 on t1.a=t2.a right full join t3 on t2.b=t3.b", - "select /*+ leading(t3) */ * from t1 join t2 on t1.a=t2.a right full join t3 on t2.b=t3.b", - "select /*+ leading(t2, t3) */ * from t1 join t2 on t1.a=t2.a right full join t3 on t2.b=t3.b", - "select /*+ leading(t3, t2) */ * from t1 join t2 on t1.a=t2.a right full join t3 on t2.b=t3.b", - "select /*+ leading(t3, t1) */ * from t1 join t2 on t1.a=t2.a right full join t3 on t2.b=t3.b", // cross join "select /*+ leading(t2) */ * from t1 join t2 on t1.a=t2.a cross join t3", diff --git a/planner/core/testdata/join_reorder_suite_out.json b/planner/core/testdata/join_reorder_suite_out.json index 87e1ba6fb7ab7..8768b3b53dba5 100644 --- a/planner/core/testdata/join_reorder_suite_out.json +++ b/planner/core/testdata/join_reorder_suite_out.json @@ -2614,208 +2614,502 @@ "Warning": null }, { - "SQL": "select /*+ straight_join() */ * from t1 straight join t2 straight join t3", - "Plan": null, - "Warning": null - }, - { - "SQL": "", - "Plan": null, - "Warning": null - }, - { - "SQL": "", - "Plan": null, - "Warning": null - }, - { - "SQL": "", - "Plan": null, - "Warning": null - }, - { - "SQL": "", - "Plan": null, - "Warning": null - }, - { - "SQL": "", - "Plan": null, - "Warning": null - }, - { - "SQL": "", - "Plan": null, - "Warning": null - }, - { - "SQL": "", - "Plan": null, - "Warning": null - }, - { - "SQL": "", - "Plan": null, - "Warning": null - }, - { - "SQL": "", - "Plan": null, - "Warning": null - }, - { - "SQL": "", - "Plan": null, - "Warning": null - }, - { - "SQL": "", - "Plan": null, - "Warning": null - }, - { - "SQL": "", - "Plan": 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": "", - "Plan": 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": "", - "Plan": 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": "", - "Plan": 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": "", - "Plan": 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": "", - "Plan": 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": "", - "Plan": 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": "", - "Plan": 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": "", - "Plan": 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": "", - "Plan": 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": null }, { - "SQL": "", - "Plan": null, + "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": null }, { - "SQL": "", - "Plan": null, + "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": null }, { - "SQL": "", - "Plan": null, + "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": null }, { - "SQL": "", - "Plan": null, + "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": null }, { - "SQL": "", - "Plan": null, - "Warning": null + "SQL": "select /*+ leading(t2) */ * 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 when we have outer join" + ] }, { - "SQL": "", - "Plan": null, - "Warning": null + "SQL": "select /*+ leading(t3) */ * 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 when we have outer join" + ] }, { - "SQL": "", - "Plan": null, - "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": [ + "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 when we have outer join" + ] }, { - "SQL": "", - "Plan": null, - "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": [ + "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 when we have outer join" + ] }, { - "SQL": "", - "Plan": null, - "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 when we have outer join" + ] }, { - "SQL": "", - "Plan": null, + "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": "", - "Plan": null, - "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": [ + "Warning 1815 leading hint is inapplicable when we have outer join" + ] }, { - "SQL": "", - "Plan": null, - "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 when we have outer join" + ] }, { - "SQL": "", - "Plan": null, - "Warning": null + "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 when we have outer join" + ] }, { - "SQL": "", - "Plan": null, - "Warning": null + "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 when we have outer join" + ] }, { - "SQL": "", - "Plan": null, + "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": "", - "Plan": 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": null }, { - "SQL": "", - "Plan": null, + "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": "", - "Plan": 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": "", - "Plan": 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 } ] From 56b83ada06f6b5374a2d8c939bd04ceded23391f Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Wed, 18 May 2022 16:20:24 +0800 Subject: [PATCH 29/36] fix ut --- planner/core/testdata/join_reorder_suite_out.json | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/planner/core/testdata/join_reorder_suite_out.json b/planner/core/testdata/join_reorder_suite_out.json index 8768b3b53dba5..4f712f93ee12b 100644 --- a/planner/core/testdata/join_reorder_suite_out.json +++ b/planner/core/testdata/join_reorder_suite_out.json @@ -1972,7 +1972,7 @@ " └─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" + "Warning 1815 We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid" ] }, { @@ -2006,7 +2006,7 @@ " └─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" + "Warning 1815 We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid" ] }, { @@ -2040,7 +2040,7 @@ " └─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" + "Warning 1815 We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid" ] }, { @@ -2074,7 +2074,7 @@ " └─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" + "Warning 1815 We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid" ] } ] From 327da4d779455a6e7a99d8e22d21c04d2495d4da Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Wed, 18 May 2022 22:43:49 +0800 Subject: [PATCH 30/36] update the test results --- planner/core/rule_join_reorder.go | 18 ++++-- .../core/testdata/join_reorder_suite_in.json | 4 +- .../core/testdata/join_reorder_suite_out.json | 59 +++++++++++-------- 3 files changed, 49 insertions(+), 32 deletions(-) diff --git a/planner/core/rule_join_reorder.go b/planner/core/rule_join_reorder.go index 6fab84068b5e3..0aae4f646d939 100644 --- a/planner/core/rule_join_reorder.go +++ b/planner/core/rule_join_reorder.go @@ -36,13 +36,20 @@ import ( func extractJoinGroup(p LogicalPlan) (group []LogicalPlan, eqEdges []*expression.ScalarFunction, otherConds []expression.Expression, joinTypes []JoinType, hintInfo []*tableHintInfo, hasOuterJoin bool) { join, isJoin := p.(*LogicalJoin) + if isJoin && join.preferJoinOrder { + // When there is a leading hint, the hint may not take effect for other reasons. + // For example, the join type is cross join or straight join, or exists the join algorithm hint, etc. + // We need to return the hint information to warn + hintInfo = append(hintInfo, join.hintInfo) + } if !isJoin || join.preferJoinType > uint(0) || join.StraightJoin || (join.JoinType != InnerJoin && join.JoinType != LeftOuterJoin && join.JoinType != RightOuterJoin) || ((join.JoinType == LeftOuterJoin || join.JoinType == RightOuterJoin) && join.EqualConditions == nil) { - return []LogicalPlan{p}, nil, nil, nil, nil, false - } - if join.preferJoinOrder { - hintInfo = append(hintInfo, join.hintInfo) + if hintInfo != nil { + // The leading hint can not work for some reasons. So clear it in the join node. + join.hintInfo = nil + } + return []LogicalPlan{p}, nil, nil, nil, hintInfo, false } hasOuterJoin = hasOuterJoin || (join.JoinType != InnerJoin) if join.JoinType != RightOuterJoin { @@ -235,6 +242,9 @@ func (s *joinReOrderSolver) optimizeRecursive(ctx sessionctx.Context, p LogicalP } return p, nil } + if len(curJoinGroup) == 1 && hintInfo != nil { + ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("leading hint is inapplicable, check the join type or the join algorithm hint")) + } newChildren := make([]LogicalPlan, 0, len(p.Children())) for _, child := range p.Children() { newChild, err := s.optimizeRecursive(ctx, child, tracer) diff --git a/planner/core/testdata/join_reorder_suite_in.json b/planner/core/testdata/join_reorder_suite_in.json index bb2ad55ce9ab1..346d62dcd62c8 100644 --- a/planner/core/testdata/join_reorder_suite_in.json +++ b/planner/core/testdata/join_reorder_suite_in.json @@ -107,10 +107,10 @@ "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(t1) hash_join(t1) */ * from t, t1, t2, t3 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(t1) merge_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;" ] diff --git a/planner/core/testdata/join_reorder_suite_out.json b/planner/core/testdata/join_reorder_suite_out.json index 4f712f93ee12b..563c4423b0bb3 100644 --- a/planner/core/testdata/join_reorder_suite_out.json +++ b/planner/core/testdata/join_reorder_suite_out.json @@ -2139,25 +2139,29 @@ "Warning": null }, { - "SQL": "select /*+ leading(t1) hash_join(t1) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", + "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.t2.b, test.t1.b)]", + " └─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" + " └─Projection(Probe) 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" ], - "Warning": null + "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(t2) hash_join(t1) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", @@ -2204,28 +2208,31 @@ "Warning": null }, { - "SQL": "select /*+ leading(t1) merge_join(t1) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", + "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.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", - " └─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" + " └─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", + " └─Projection 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" ], - "Warning": null + "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) merge_join(t2) */ * from t, t1, t2, t3 where t.a = t1.a and t1.b=t2.b;", From 1fb7e1dfd80feab0f4fc75118003393327add32f Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Wed, 18 May 2022 22:46:49 +0800 Subject: [PATCH 31/36] update the test results --- .../core/testdata/join_reorder_suite_out.json | 23 +++++++++++++++---- 1 file changed, 18 insertions(+), 5 deletions(-) diff --git a/planner/core/testdata/join_reorder_suite_out.json b/planner/core/testdata/join_reorder_suite_out.json index 563c4423b0bb3..8d25b88cb97e3 100644 --- a/planner/core/testdata/join_reorder_suite_out.json +++ b/planner/core/testdata/join_reorder_suite_out.json @@ -2785,7 +2785,9 @@ " └─TableReader(Probe) 10000.00 root data:TableFullScan", " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], - "Warning": null + "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", @@ -2799,7 +2801,9 @@ " └─TableReader(Probe) 10000.00 root data:TableFullScan", " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], - "Warning": null + "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", @@ -2813,7 +2817,10 @@ " └─TableReader(Probe) 10000.00 root data:TableFullScan", " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], - "Warning": null + "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", @@ -2827,7 +2834,10 @@ " └─TableReader(Probe) 10000.00 root data:TableFullScan", " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], - "Warning": null + "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", @@ -2841,7 +2851,10 @@ " └─TableReader(Probe) 10000.00 root data:TableFullScan", " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], - "Warning": null + "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", From ed2f08336366fdc1c5ae67f416498725376daf1f Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Wed, 18 May 2022 23:13:12 +0800 Subject: [PATCH 32/36] update the test cases --- planner/core/rule_join_reorder_greedy.go | 8 ++++++++ planner/core/testdata/join_reorder_suite_out.json | 4 +++- 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/planner/core/rule_join_reorder_greedy.go b/planner/core/rule_join_reorder_greedy.go index 905acbf251bb6..37f245b7b5818 100644 --- a/planner/core/rule_join_reorder_greedy.go +++ b/planner/core/rule_join_reorder_greedy.go @@ -59,6 +59,8 @@ func (s *joinReorderGreedySolver) solve(joinNodePlans []LogicalPlan, tracer *joi return s.curJoinGroup[i].cumCost < s.curJoinGroup[j].cumCost }) + // joinNodeNum indicates the number of join nodes except leading join nodes in the current join group + joinNodeNum := len(s.curJoinGroup) if leadingJoinNodes != nil { // The leadingJoinNodes should be the first element in the s.curJoinGroup. // So it can be joined first. @@ -71,6 +73,12 @@ func (s *joinReorderGreedySolver) solve(joinNodePlans []LogicalPlan, tracer *joi if err != nil { return nil, err } + if len(s.curJoinGroup) == joinNodeNum { + // Getting here means that there is no join condition between the table used in the leading hint and other tables + // For example: select /*+ leading(t3) */ * from t1 join t2 on t1.a=t2.a cross join t3 + // We can not let table t3 join first. + s.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("leading hint is inapplicable, check if the leading hint table has join conditions with other tables")) + } cartesianGroup = append(cartesianGroup, newNode.p) } diff --git a/planner/core/testdata/join_reorder_suite_out.json b/planner/core/testdata/join_reorder_suite_out.json index 8d25b88cb97e3..b04c3bd3b4e85 100644 --- a/planner/core/testdata/join_reorder_suite_out.json +++ b/planner/core/testdata/join_reorder_suite_out.json @@ -3079,7 +3079,9 @@ " └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], - "Warning": null + "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", From c46c495945e06347f932da7968aaed8797bbdf7a Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Thu, 19 May 2022 10:03:17 +0800 Subject: [PATCH 33/36] fix ut --- planner/core/rule_join_reorder_greedy.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/planner/core/rule_join_reorder_greedy.go b/planner/core/rule_join_reorder_greedy.go index 37f245b7b5818..309f4ac116bc4 100644 --- a/planner/core/rule_join_reorder_greedy.go +++ b/planner/core/rule_join_reorder_greedy.go @@ -73,7 +73,7 @@ func (s *joinReorderGreedySolver) solve(joinNodePlans []LogicalPlan, tracer *joi if err != nil { return nil, err } - if len(s.curJoinGroup) == joinNodeNum { + if joinNodeNum > 0 && len(s.curJoinGroup) == joinNodeNum { // Getting here means that there is no join condition between the table used in the leading hint and other tables // For example: select /*+ leading(t3) */ * from t1 join t2 on t1.a=t2.a cross join t3 // We can not let table t3 join first. From 1a2b0d1bc9add83a6d252cfb5d6113f7091d1993 Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Thu, 19 May 2022 10:06:45 +0800 Subject: [PATCH 34/36] update the test results --- planner/core/rule_join_reorder_test.go | 2 + .../core/testdata/join_reorder_suite_out.json | 679 +++++++++--------- 2 files changed, 355 insertions(+), 326 deletions(-) diff --git a/planner/core/rule_join_reorder_test.go b/planner/core/rule_join_reorder_test.go index 27d679e0f50d8..59a53c41adb04 100644 --- a/planner/core/rule_join_reorder_test.go +++ b/planner/core/rule_join_reorder_test.go @@ -324,6 +324,8 @@ func TestJoinOrderHint4Subquery(t *testing.T) { 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") } diff --git a/planner/core/testdata/join_reorder_suite_out.json b/planner/core/testdata/join_reorder_suite_out.json index b04c3bd3b4e85..295991010ccfc 100644 --- a/planner/core/testdata/join_reorder_suite_out.json +++ b/planner/core/testdata/join_reorder_suite_out.json @@ -3273,9 +3273,9 @@ "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) 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(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))", @@ -3290,8 +3290,8 @@ "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) 10000.00 root data:TableFullScan", - "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "├─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))", @@ -3306,9 +3306,9 @@ "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) 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(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))", @@ -3323,8 +3323,8 @@ "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) 10000.00 root data:TableFullScan", - "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "├─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))", @@ -3356,12 +3356,13 @@ "Plan": [ "Projection 1.00 root test.t1.a, Column#14", "└─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", + " ├─StreamAgg(Build) 1.00 root funcs:min(test.t1.a)->Column#10, funcs:firstrow(test.t1.a)->test.t1.a", + " │ └─IndexJoin 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) 1.25 root index:Selection", + " │ └─Selection 1.25 cop[tikv] not(isnull(test.t1.a))", + " │ └─IndexRangeScan 1.25 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", " └─IndexReader 2.00 root index:Selection", @@ -3373,10 +3374,10 @@ { "SQL": "select /*+ straight_join() */ * 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)]", - "├─HashAgg(Build) 7992.00 root group by:test.t3.a, funcs:firstrow(test.t3.a)->test.t3.a", - "│ └─IndexReader 9990.00 root index:IndexFullScan", - "│ └─IndexFullScan 9990.00 cop[tikv] table:t3, index:a(a) keep order:false, stats:pseudo", + "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:IndexFullScan", + "│ └─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))", @@ -3391,8 +3392,8 @@ "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 CARTESIAN anti semi join, other cond: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", + "├─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))", @@ -3405,13 +3406,6 @@ }, { "SQL": "select /*+ straight_join() */ * from t1 join t2 on t1.a=t2.a where exists (select t3.a from t3)", - "Plan": [ - "TableDual 0.00 root rows:0" - ], - "Warning": null - }, - { - "SQL": "select /*+ straight_join() */ * 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", @@ -3423,13 +3417,20 @@ ], "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 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 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))", @@ -3443,10 +3444,10 @@ { "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 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 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))", @@ -3486,12 +3487,13 @@ { "SQL": "select /*+ straight_join() */ 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" + "Projection 3.75 root test.t1.a, ->Column#14", + "└─IndexJoin 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) 1.25 root index:Selection", + " └─Selection 1.25 cop[tikv] not(isnull(test.t1.a))", + " └─IndexRangeScan 1.25 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, test.t3.a)], keep order:false, stats:pseudo" ], "Warning": null }, @@ -3499,9 +3501,9 @@ "SQL": "select /*+ leading(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) 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(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))", @@ -3516,9 +3518,9 @@ "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) 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(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))", @@ -3535,9 +3537,9 @@ "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) 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(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))", @@ -3555,9 +3557,9 @@ "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) 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(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))", @@ -3575,9 +3577,9 @@ "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) 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(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))", @@ -3595,9 +3597,9 @@ "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) 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(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))", @@ -3615,8 +3617,8 @@ "SQL": "select /*+ leading(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) 10000.00 root data:TableFullScan", - "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "├─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))", @@ -3631,8 +3633,8 @@ "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) 10000.00 root data:TableFullScan", - "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "├─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))", @@ -3649,8 +3651,8 @@ "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) 10000.00 root data:TableFullScan", - "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "├─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))", @@ -3668,8 +3670,8 @@ "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) 10000.00 root data:TableFullScan", - "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "├─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))", @@ -3687,8 +3689,8 @@ "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) 10000.00 root data:TableFullScan", - "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "├─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))", @@ -3706,8 +3708,8 @@ "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) 10000.00 root data:TableFullScan", - "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "├─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))", @@ -3725,9 +3727,9 @@ "SQL": "select /*+ leading(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) 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(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))", @@ -3742,9 +3744,9 @@ "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) 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(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))", @@ -3761,9 +3763,9 @@ "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) 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(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))", @@ -3781,9 +3783,9 @@ "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) 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(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))", @@ -3801,9 +3803,9 @@ "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) 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(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))", @@ -3821,9 +3823,9 @@ "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) 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(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))", @@ -3841,8 +3843,8 @@ "SQL": "select /*+ leading(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) 10000.00 root data:TableFullScan", - "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "├─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))", @@ -3857,8 +3859,8 @@ "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) 10000.00 root data:TableFullScan", - "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "├─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))", @@ -3875,8 +3877,8 @@ "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) 10000.00 root data:TableFullScan", - "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "├─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))", @@ -3894,8 +3896,8 @@ "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) 10000.00 root data:TableFullScan", - "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "├─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))", @@ -3913,8 +3915,8 @@ "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) 10000.00 root data:TableFullScan", - "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "├─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))", @@ -3932,8 +3934,8 @@ "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) 10000.00 root data:TableFullScan", - "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "├─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))", @@ -3952,12 +3954,13 @@ "Plan": [ "Projection 1.00 root test.t1.a, Column#14", "└─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", + " ├─StreamAgg(Build) 1.00 root funcs:min(test.t1.a)->Column#10, funcs:firstrow(test.t1.a)->test.t1.a", + " │ └─IndexJoin 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) 1.25 root index:Selection", + " │ └─Selection 1.25 cop[tikv] not(isnull(test.t1.a))", + " │ └─IndexRangeScan 1.25 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", " └─IndexReader 2.00 root index:Selection", @@ -3971,12 +3974,13 @@ "Plan": [ "Projection 1.00 root test.t1.a, Column#14", "└─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", + " ├─StreamAgg(Build) 1.00 root funcs:min(test.t1.a)->Column#10, funcs:firstrow(test.t1.a)->test.t1.a", + " │ └─IndexJoin 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) 1.25 root index:Selection", + " │ └─Selection 1.25 cop[tikv] not(isnull(test.t1.a))", + " │ └─IndexRangeScan 1.25 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", " └─IndexReader 2.00 root index:Selection", @@ -3992,12 +3996,13 @@ "Plan": [ "Projection 1.00 root test.t1.a, Column#14", "└─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", + " ├─StreamAgg(Build) 1.00 root funcs:min(test.t1.a)->Column#10, funcs:firstrow(test.t1.a)->test.t1.a", + " │ └─IndexJoin 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) 1.25 root index:Selection", + " │ └─Selection 1.25 cop[tikv] not(isnull(test.t1.a))", + " │ └─IndexRangeScan 1.25 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", " └─IndexReader 2.00 root index:Selection", @@ -4013,12 +4018,13 @@ "Plan": [ "Projection 1.00 root test.t1.a, Column#14", "└─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", + " ├─StreamAgg(Build) 1.00 root funcs:min(test.t1.a)->Column#10, funcs:firstrow(test.t1.a)->test.t1.a", + " │ └─IndexJoin 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) 1.25 root index:Selection", + " │ └─Selection 1.25 cop[tikv] not(isnull(test.t1.a))", + " │ └─IndexRangeScan 1.25 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", " └─IndexReader 2.00 root index:Selection", @@ -4035,12 +4041,13 @@ "Plan": [ "Projection 1.00 root test.t1.a, Column#14", "└─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", + " ├─StreamAgg(Build) 1.00 root funcs:min(test.t1.a)->Column#10, funcs:firstrow(test.t1.a)->test.t1.a", + " │ └─IndexJoin 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) 1.25 root index:Selection", + " │ └─Selection 1.25 cop[tikv] not(isnull(test.t1.a))", + " │ └─IndexRangeScan 1.25 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", " └─IndexReader 2.00 root index:Selection", @@ -4056,12 +4063,13 @@ "Plan": [ "Projection 1.00 root test.t1.a, Column#14", "└─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", + " ├─StreamAgg(Build) 1.00 root funcs:min(test.t1.a)->Column#10, funcs:firstrow(test.t1.a)->test.t1.a", + " │ └─IndexJoin 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) 1.25 root index:Selection", + " │ └─Selection 1.25 cop[tikv] not(isnull(test.t1.a))", + " │ └─IndexRangeScan 1.25 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", " └─IndexReader 2.00 root index:Selection", @@ -4076,51 +4084,57 @@ { "SQL": "select /*+ leading(t1) */ * 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)]", - "├─HashAgg(Build) 7992.00 root group by:test.t3.a, funcs:firstrow(test.t3.a)->test.t3.a", - "│ └─IndexReader 9990.00 root index:IndexFullScan", - "│ └─IndexFullScan 9990.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" + "IndexJoin 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)", + "├─IndexJoin(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:IndexFullScan", + "│ │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:true", + "│ └─IndexLookUp(Probe) 1.25 root ", + "│ ├─Selection(Build) 1.25 cop[tikv] not(isnull(test.t1.a))", + "│ │ └─IndexRangeScan 1.25 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, test.t3.a)], keep order:false, stats:pseudo", + "│ └─TableRowIDScan(Probe) 1.25 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─IndexLookUp(Probe) 1.25 root ", + " ├─Selection(Build) 1.25 cop[tikv] not(isnull(test.t2.a))", + " │ └─IndexRangeScan 1.25 cop[tikv] table:t2, index:a(a) range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 1.25 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": [ - "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", - "└─HashJoin(Probe) 9990.00 root inner join, equal:[eq(test.t3.a, test.t1.a)]", - " ├─HashAgg(Build) 7992.00 root group by:test.t3.a, funcs:firstrow(test.t3.a)->test.t3.a", - " │ └─IndexReader 9990.00 root index:IndexFullScan", - " │ └─IndexFullScan 9990.00 cop[tikv] table:t3, index:a(a) 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" + "IndexJoin 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)", + "├─IndexJoin(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:IndexFullScan", + "│ │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:true", + "│ └─IndexLookUp(Probe) 1.25 root ", + "│ ├─Selection(Build) 1.25 cop[tikv] not(isnull(test.t1.a))", + "│ │ └─IndexRangeScan 1.25 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, test.t3.a)], keep order:false, stats:pseudo", + "│ └─TableRowIDScan(Probe) 1.25 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─IndexLookUp(Probe) 1.25 root ", + " ├─Selection(Build) 1.25 cop[tikv] not(isnull(test.t2.a))", + " │ └─IndexRangeScan 1.25 cop[tikv] table:t2, index:a(a) range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 1.25 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)]", - "├─HashAgg(Build) 7992.00 root group by:test.t3.a, funcs:firstrow(test.t3.a)->test.t3.a", - "│ └─IndexReader 9990.00 root index:IndexFullScan", - "│ └─IndexFullScan 9990.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" + "IndexJoin 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)", + "├─IndexJoin(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:IndexFullScan", + "│ │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:true", + "│ └─IndexLookUp(Probe) 1.25 root ", + "│ ├─Selection(Build) 1.25 cop[tikv] not(isnull(test.t1.a))", + "│ │ └─IndexRangeScan 1.25 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, test.t3.a)], keep order:false, stats:pseudo", + "│ └─TableRowIDScan(Probe) 1.25 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─IndexLookUp(Probe) 1.25 root ", + " ├─Selection(Build) 1.25 cop[tikv] not(isnull(test.t2.a))", + " │ └─IndexRangeScan 1.25 cop[tikv] table:t2, index:a(a) range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 1.25 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" @@ -4129,17 +4143,19 @@ { "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": [ - "HashJoin 12487.50 root inner join, equal:[eq(test.t1.a, test.t3.a)]", - "├─HashAgg(Build) 7992.00 root group by:test.t3.a, funcs:firstrow(test.t3.a)->test.t3.a", - "│ └─IndexReader 9990.00 root index:IndexFullScan", - "│ └─IndexFullScan 9990.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" + "IndexJoin 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)", + "├─IndexJoin(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:IndexFullScan", + "│ │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:true", + "│ └─IndexLookUp(Probe) 1.25 root ", + "│ ├─Selection(Build) 1.25 cop[tikv] not(isnull(test.t1.a))", + "│ │ └─IndexRangeScan 1.25 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, test.t3.a)], keep order:false, stats:pseudo", + "│ └─TableRowIDScan(Probe) 1.25 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─IndexLookUp(Probe) 1.25 root ", + " ├─Selection(Build) 1.25 cop[tikv] not(isnull(test.t2.a))", + " │ └─IndexRangeScan 1.25 cop[tikv] table:t2, index:a(a) range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 1.25 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" @@ -4148,17 +4164,19 @@ { "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": [ - "HashJoin 12487.50 root inner join, equal:[eq(test.t1.a, test.t3.a)]", - "├─HashAgg(Build) 7992.00 root group by:test.t3.a, funcs:firstrow(test.t3.a)->test.t3.a", - "│ └─IndexReader 9990.00 root index:IndexFullScan", - "│ └─IndexFullScan 9990.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" + "IndexJoin 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)", + "├─IndexJoin(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:IndexFullScan", + "│ │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:true", + "│ └─IndexLookUp(Probe) 1.25 root ", + "│ ├─Selection(Build) 1.25 cop[tikv] not(isnull(test.t1.a))", + "│ │ └─IndexRangeScan 1.25 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, test.t3.a)], keep order:false, stats:pseudo", + "│ └─TableRowIDScan(Probe) 1.25 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─IndexLookUp(Probe) 1.25 root ", + " ├─Selection(Build) 1.25 cop[tikv] not(isnull(test.t2.a))", + " │ └─IndexRangeScan 1.25 cop[tikv] table:t2, index:a(a) range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 1.25 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" @@ -4167,17 +4185,19 @@ { "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": [ - "HashJoin 12487.50 root inner join, equal:[eq(test.t1.a, test.t3.a)]", - "├─HashAgg(Build) 7992.00 root group by:test.t3.a, funcs:firstrow(test.t3.a)->test.t3.a", - "│ └─IndexReader 9990.00 root index:IndexFullScan", - "│ └─IndexFullScan 9990.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" + "IndexJoin 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)", + "├─IndexJoin(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:IndexFullScan", + "│ │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:true", + "│ └─IndexLookUp(Probe) 1.25 root ", + "│ ├─Selection(Build) 1.25 cop[tikv] not(isnull(test.t1.a))", + "│ │ └─IndexRangeScan 1.25 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, test.t3.a)], keep order:false, stats:pseudo", + "│ └─TableRowIDScan(Probe) 1.25 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─IndexLookUp(Probe) 1.25 root ", + " ├─Selection(Build) 1.25 cop[tikv] not(isnull(test.t2.a))", + " │ └─IndexRangeScan 1.25 cop[tikv] table:t2, index:a(a) range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 1.25 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" @@ -4187,8 +4207,8 @@ "SQL": "select /*+ leading(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 CARTESIAN anti semi join, other cond: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", + "├─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))", @@ -4203,8 +4223,8 @@ "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 CARTESIAN anti semi join, other cond: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", + "├─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))", @@ -4221,8 +4241,8 @@ "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 CARTESIAN anti semi join, other cond: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", + "├─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))", @@ -4240,8 +4260,8 @@ "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 CARTESIAN anti semi join, other cond: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", + "├─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))", @@ -4259,8 +4279,8 @@ "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 CARTESIAN anti semi join, other cond: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", + "├─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))", @@ -4278,8 +4298,8 @@ "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 CARTESIAN anti semi join, other cond: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", + "├─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))", @@ -4295,58 +4315,6 @@ }, { "SQL": "select /*+ leading(t1) */ * from t1 join t2 on t1.a=t2.a where 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 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(t1, 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(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 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 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(t1) */ * 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", @@ -4359,7 +4327,7 @@ "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)", + "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", @@ -4374,7 +4342,7 @@ ] }, { - "SQL": "select /*+ leading(t2, t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where not exists (select t3.a from t3)", + "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", @@ -4390,7 +4358,7 @@ ] }, { - "SQL": "select /*+ leading(t1, t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where not exists (select t3.a from t3)", + "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", @@ -4406,7 +4374,7 @@ ] }, { - "SQL": "select /*+ leading(t3@sel_2, t2) */ * from t1 join t2 on t1.a=t2.a where not exists (select t3.a from t3)", + "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", @@ -4422,7 +4390,7 @@ ] }, { - "SQL": "select /*+ leading(t3@sel_2, t1) */ * from t1 join t2 on t1.a=t2.a where not exists (select t3.a from t3)", + "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", @@ -4437,14 +4405,66 @@ "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" ] }, + { + "SQL": "select /*+ leading(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": 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 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", + "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))", @@ -4460,11 +4480,11 @@ { "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 15593.77 root test.t4.a, test.t4.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", + "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))", @@ -4500,24 +4520,26 @@ { "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" + "Projection 3.75 root test.t1.a, ->Column#14", + "└─IndexJoin 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) 1.25 root index:Selection", + " └─Selection 1.25 cop[tikv] not(isnull(test.t1.a))", + " └─IndexRangeScan 1.25 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 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" + "Projection 3.75 root test.t1.a, ->Column#14", + "└─IndexJoin 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) 1.25 root index:Selection", + " └─Selection 1.25 cop[tikv] not(isnull(test.t1.a))", + " └─IndexRangeScan 1.25 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", @@ -4527,24 +4549,26 @@ { "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" + "Projection 3.75 root test.t1.a, ->Column#14", + "└─IndexJoin 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) 1.25 root index:Selection", + " └─Selection 1.25 cop[tikv] not(isnull(test.t1.a))", + " └─IndexRangeScan 1.25 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 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" + "Projection 3.75 root test.t1.a, ->Column#14", + "└─IndexJoin 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) 1.25 root index:Selection", + " └─Selection 1.25 cop[tikv] not(isnull(test.t1.a))", + " └─IndexRangeScan 1.25 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", @@ -4554,12 +4578,13 @@ { "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 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" + "Projection 3.75 root test.t1.a, ->Column#14", + "└─IndexJoin 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) 1.25 root index:Selection", + " └─Selection 1.25 cop[tikv] not(isnull(test.t1.a))", + " └─IndexRangeScan 1.25 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", @@ -4569,12 +4594,13 @@ { "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" + "Projection 3.75 root test.t1.a, ->Column#14", + "└─IndexJoin 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) 1.25 root index:Selection", + " └─Selection 1.25 cop[tikv] not(isnull(test.t1.a))", + " └─IndexRangeScan 1.25 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", @@ -4584,12 +4610,13 @@ { "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 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" + "Projection 3.75 root test.t1.a, ->Column#14", + "└─IndexJoin 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) 1.25 root index:Selection", + " └─Selection 1.25 cop[tikv] not(isnull(test.t1.a))", + " └─IndexRangeScan 1.25 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", From 5636cc6b0e86f7c886692b5d8824bcbdf02ba05c Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Thu, 19 May 2022 10:49:50 +0800 Subject: [PATCH 35/36] update test cases --- .../core/testdata/join_reorder_suite_in.json | 26 +- .../core/testdata/join_reorder_suite_out.json | 384 +++++++++++++++--- 2 files changed, 335 insertions(+), 75 deletions(-) diff --git a/planner/core/testdata/join_reorder_suite_in.json b/planner/core/testdata/join_reorder_suite_in.json index 346d62dcd62c8..8b0e77e742422 100644 --- a/planner/core/testdata/join_reorder_suite_in.json +++ b/planner/core/testdata/join_reorder_suite_in.json @@ -212,28 +212,28 @@ "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(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 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(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 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(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 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(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(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)", @@ -247,29 +247,39 @@ "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(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 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(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 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(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 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(t1) */ * from t1 join t2 on t1.a=t2.a where not 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)", diff --git a/planner/core/testdata/join_reorder_suite_out.json b/planner/core/testdata/join_reorder_suite_out.json index 295991010ccfc..30d2bb9441989 100644 --- a/planner/core/testdata/join_reorder_suite_out.json +++ b/planner/core/testdata/join_reorder_suite_out.json @@ -3498,19 +3498,24 @@ "Warning": null }, { - "SQL": "select /*+ leading(t1) */ * from t1 join t2 on t1.a=t2.a where t1.a in (select t3.a from t3 where t1.b = t3.b)", + "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 9980.01 root semi join, equal:[eq(test.t1.b, test.t3.b) eq(test.t1.a, test.t3.a)]", + "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", - "└─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" + "└─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 }, @@ -3614,18 +3619,23 @@ ] }, { - "SQL": "select /*+ leading(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)", + "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 9990.00 root anti semi join, equal:[eq(test.t1.b, test.t3.b)], other cond:eq(test.t1.a, test.t3.a)", + "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", - "└─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" + "└─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 }, @@ -3724,19 +3734,24 @@ ] }, { - "SQL": "select /*+ leading(t1) */ * from t1 join t2 on t1.a=t2.a where exists (select t3.a from t3 where t1.b = t3.b)", + "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 9980.01 root semi join, equal:[eq(test.t1.b, test.t3.b)]", + "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", - "└─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" + "└─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 }, @@ -3840,18 +3855,23 @@ ] }, { - "SQL": "select /*+ leading(t1) */ * from t1 join t2 on t1.a=t2.a where not exists (select t3.a from t3 where t1.b = t3.b)", + "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 9990.00 root anti semi join, equal:[eq(test.t1.b, test.t3.b)]", + "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", - "└─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" + "└─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 }, @@ -4082,21 +4102,241 @@ ] }, { - "SQL": "select /*+ leading(t1) */ * from t1 join t2 on t1.a=t2.a where t1.a in (select t3.a from t3)", + "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)]", + "├─IndexJoin(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)", + "│ ├─IndexJoin(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) 1.25 root ", + "│ │ ├─Selection(Build) 1.25 cop[tikv] not(isnull(test.t1.a))", + "│ │ │ └─IndexRangeScan 1.25 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, Column#13)], keep order:false, stats:pseudo", + "│ │ └─Selection(Probe) 1.25 cop[tikv] not(isnull(test.t1.b))", + "│ │ └─TableRowIDScan 1.25 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ └─IndexLookUp(Probe) 1.25 root ", + "│ ├─Selection(Build) 1.25 cop[tikv] not(isnull(test.t2.a))", + "│ │ └─IndexRangeScan 1.25 cop[tikv] table:t2, index:a(a) range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo", + "│ └─TableRowIDScan(Probe) 1.25 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": [ - "IndexJoin 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)", - "├─IndexJoin(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:IndexFullScan", - "│ │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:true", + "Projection 4.69 root test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t4.a, test.t4.b", + "└─IndexJoin 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) 1.25 root ", + " ├─Selection(Build) 1.25 cop[tikv] not(isnull(test.t2.a))", + " │ └─IndexRangeScan 1.25 cop[tikv] table:t2, index:a(a) range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 1.25 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)]", + "├─IndexJoin(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)", + "│ ├─IndexJoin(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) 1.25 root ", + "│ │ ├─Selection(Build) 1.25 cop[tikv] not(isnull(test.t1.a))", + "│ │ │ └─IndexRangeScan 1.25 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, Column#13)], keep order:false, stats:pseudo", + "│ │ └─Selection(Probe) 1.25 cop[tikv] not(isnull(test.t1.b))", + "│ │ └─TableRowIDScan 1.25 cop[tikv] table:t1 keep order:false, stats:pseudo", "│ └─IndexLookUp(Probe) 1.25 root ", - "│ ├─Selection(Build) 1.25 cop[tikv] not(isnull(test.t1.a))", - "│ │ └─IndexRangeScan 1.25 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, test.t3.a)], keep order:false, stats:pseudo", - "│ └─TableRowIDScan(Probe) 1.25 cop[tikv] table:t1 keep order:false, stats:pseudo", - "└─IndexLookUp(Probe) 1.25 root ", - " ├─Selection(Build) 1.25 cop[tikv] not(isnull(test.t2.a))", - " │ └─IndexRangeScan 1.25 cop[tikv] table:t2, index:a(a) range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 1.25 cop[tikv] table:t2 keep order:false, stats:pseudo" + "│ ├─Selection(Build) 1.25 cop[tikv] not(isnull(test.t2.a))", + "│ │ └─IndexRangeScan 1.25 cop[tikv] table:t2, index:a(a) range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo", + "│ └─TableRowIDScan(Probe) 1.25 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)]", + "├─IndexJoin(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)", + "│ ├─IndexJoin(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) 1.25 root ", + "│ │ ├─Selection(Build) 1.25 cop[tikv] not(isnull(test.t1.a))", + "│ │ │ └─IndexRangeScan 1.25 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, Column#13)], keep order:false, stats:pseudo", + "│ │ └─Selection(Probe) 1.25 cop[tikv] not(isnull(test.t1.b))", + "│ │ └─TableRowIDScan 1.25 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ └─IndexLookUp(Probe) 1.25 root ", + "│ ├─Selection(Build) 1.25 cop[tikv] not(isnull(test.t2.a))", + "│ │ └─IndexRangeScan 1.25 cop[tikv] table:t2, index:a(a) range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo", + "│ └─TableRowIDScan(Probe) 1.25 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)]", + "├─IndexJoin(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) 1.25 root ", + "│ ├─Selection(Build) 1.25 cop[tikv] not(isnull(test.t2.a))", + "│ │ └─IndexRangeScan 1.25 cop[tikv] table:t2, index:a(a) range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo", + "│ └─TableRowIDScan(Probe) 1.25 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", + "└─IndexJoin 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) 1.25 root ", + " ├─Selection(Build) 1.25 cop[tikv] not(isnull(test.t2.a))", + " │ └─IndexRangeScan 1.25 cop[tikv] table:t2, index:a(a) range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 1.25 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)]", + "├─IndexJoin(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) 1.25 root ", + "│ ├─Selection(Build) 1.25 cop[tikv] not(isnull(test.t2.a))", + "│ │ └─IndexRangeScan 1.25 cop[tikv] table:t2, index:a(a) range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo", + "│ └─TableRowIDScan(Probe) 1.25 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)]", + "├─IndexJoin(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) 1.25 root ", + "│ ├─Selection(Build) 1.25 cop[tikv] not(isnull(test.t2.a))", + "│ │ └─IndexRangeScan 1.25 cop[tikv] table:t2, index:a(a) range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo", + "│ └─TableRowIDScan(Probe) 1.25 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", + "└─IndexJoin 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:IndexFullScan", + " │ │ └─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) 1.25 root ", + " ├─Selection(Build) 1.25 cop[tikv] not(isnull(test.t2.a))", + " │ └─IndexRangeScan 1.25 cop[tikv] table:t2, index:a(a) range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 1.25 cop[tikv] table:t2 keep order:false, stats:pseudo" ], "Warning": null }, @@ -4204,18 +4444,23 @@ ] }, { - "SQL": "select /*+ leading(t1) */ * from t1 join t2 on t1.a=t2.a where t1.a not in (select t3.a from t3)", + "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 9990.00 root CARTESIAN anti semi join, other cond:eq(test.t1.a, test.t3.a)", + "HashJoin 12475.01 root CARTESIAN anti semi join, 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" + "└─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 }, @@ -4314,15 +4559,20 @@ ] }, { - "SQL": "select /*+ leading(t1) */ * from t1 join t2 on t1.a=t2.a where exists (select t3.a from t3)", + "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": [ - "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" + "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 }, @@ -4406,7 +4656,7 @@ ] }, { - "SQL": "select /*+ leading(t1) */ * from t1 join t2 on t1.a=t2.a where not exists (select t3.a from t3)", + "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" ], From ed7f2c1c708a5dfac4adfec6496c49ed3f1888df Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Thu, 19 May 2022 10:51:24 +0800 Subject: [PATCH 36/36] update the test results --- planner/core/testdata/join_reorder_suite_out.json | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/planner/core/testdata/join_reorder_suite_out.json b/planner/core/testdata/join_reorder_suite_out.json index 30d2bb9441989..4fa0ac13bc667 100644 --- a/planner/core/testdata/join_reorder_suite_out.json +++ b/planner/core/testdata/join_reorder_suite_out.json @@ -687,7 +687,9 @@ " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" ], - "Warning": null + "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;", @@ -952,7 +954,9 @@ " └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], - "Warning": null + "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;", @@ -1041,7 +1045,9 @@ " └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" ], - "Warning": null + "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;",