From 0fc7469c4187fa29e1001e3a5f6ac61dc09ca1d4 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Tue, 25 Dec 2018 17:40:26 +0800 Subject: [PATCH 01/12] planner/core: make join reorder by dp work --- planner/core/cbo_test.go | 2 +- planner/core/logical_plan_test.go | 8 +- planner/core/optimizer.go | 2 +- planner/core/physical_plan_test.go | 6 +- planner/core/rule_join_reorder_dp.go | 135 ++++++++++++++++++---- planner/core/rule_join_reorder_dp_test.go | 6 +- planner/core/rule_join_reorder_greedy.go | 38 +++--- 7 files changed, 150 insertions(+), 47 deletions(-) diff --git a/planner/core/cbo_test.go b/planner/core/cbo_test.go index 459f5ae20317f..6e3a8e35bca17 100644 --- a/planner/core/cbo_test.go +++ b/planner/core/cbo_test.go @@ -409,7 +409,7 @@ func (s *testAnalyzeSuite) TestEmptyTable(c *C) { }, { sql: "select * from t where c1 in (select c1 from t1)", - best: "RightHashJoin{TableReader(Table(t1)->HashAgg)->HashAgg->TableReader(Table(t))}(test.t1.c1,test.t.c1)->Projection", + best: "LeftHashJoin{TableReader(Table(t))->TableReader(Table(t1)->HashAgg)->HashAgg}(test.t.c1,test.t1.c1)->Projection", }, { sql: "select * from t, t1 where t.c1 = t1.c1", diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index f9a3497fbae12..3e23e176bb934 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -767,7 +767,7 @@ func (s *testPlanSuite) TestJoinReOrder(c *C) { }{ { sql: "select * from t t1, t t2, t t3, t t4, t t5, t t6 where t1.a = t2.b and t2.a = t3.b and t3.c = t4.a and t4.d = t2.c and t5.d = t6.d", - best: "Join{Join{Join{Join{DataScan(t1)->DataScan(t2)}(t1.a,t2.b)->DataScan(t3)}(t2.a,t3.b)->DataScan(t4)}(t3.c,t4.a)(t2.c,t4.d)->Join{DataScan(t5)->DataScan(t6)}(t5.d,t6.d)}->Projection", + best: "Join{Join{Join{DataScan(t1)->DataScan(t2)}(t1.a,t2.b)->Join{DataScan(t4)->DataScan(t3)}(t4.a,t3.c)}(t2.c,t4.d)(t2.a,t3.b)->Join{DataScan(t5)->DataScan(t6)}(t5.d,t6.d)(t2.b,t1.a)}->Projection", }, { sql: "select * from t t1, t t2, t t3, t t4, t t5, t t6, t t7, t t8 where t1.a = t8.a", @@ -775,11 +775,11 @@ func (s *testPlanSuite) TestJoinReOrder(c *C) { }, { sql: "select * from t t1, t t2, t t3, t t4, t t5 where t1.a = t5.a and t5.a = t4.a and t4.a = t3.a and t3.a = t2.a and t2.a = t1.a and t1.a = t3.a and t2.a = t4.a and t5.b < 8", - best: "Join{Join{Join{Join{DataScan(t5)->DataScan(t1)}(t5.a,t1.a)->DataScan(t2)}(t1.a,t2.a)->DataScan(t3)}(t2.a,t3.a)(t1.a,t3.a)->DataScan(t4)}(t5.a,t4.a)(t3.a,t4.a)(t2.a,t4.a)->Projection", + best: "Join{Join{Join{Join{DataScan(t1)->DataScan(t5)}(t1.a,t5.a)->DataScan(t3)}(t1.a,t3.a)->DataScan(t2)}(t3.a,t2.a)(t1.a,t2.a)->DataScan(t4)}(t5.a,t4.a)(t3.a,t4.a)(t2.a,t4.a)->Projection", }, { sql: "select * from t t1, t t2, t t3, t t4, t t5 where t1.a = t5.a and t5.a = t4.a and t4.a = t3.a and t3.a = t2.a and t2.a = t1.a and t1.a = t3.a and t2.a = t4.a and t3.b = 1 and t4.a = 1", - best: "Join{Join{Join{DataScan(t3)->DataScan(t1)}->Join{DataScan(t2)->DataScan(t4)}}->DataScan(t5)}->Projection", + best: "Join{Join{Join{DataScan(t1)->DataScan(t2)}->Join{DataScan(t3)->DataScan(t4)}}->DataScan(t5)}->Projection", }, { sql: "select * from t o where o.b in (select t3.c from t t1, t t2, t t3 where t1.a = t3.a and t2.a = t3.a and t2.a = o.a)", @@ -787,7 +787,7 @@ func (s *testPlanSuite) TestJoinReOrder(c *C) { }, { sql: "select * from t o where o.b in (select t3.c from t t1, t t2, t t3 where t1.a = t3.a and t2.a = t3.a and t2.a = o.a and t1.a = 1)", - best: "Apply{DataScan(o)->Join{Join{DataScan(t3)->DataScan(t1)}->DataScan(t2)}->Projection}->Projection", + best: "Apply{DataScan(o)->Join{Join{DataScan(t1)->DataScan(t2)}->DataScan(t3)}->Projection}->Projection", }, } for _, tt := range tests { diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index b7698ea3dd9dc..4137b1a1329a4 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -58,7 +58,7 @@ var optRuleList = []logicalOptRule{ &partitionProcessor{}, &aggregationPushDownSolver{}, &pushDownTopNOptimizer{}, - &joinReOrderGreedySolver{}, + &joinReOrderSolver{}, } // logicalOptRule means a logical optimizing rule, which contains decorrelate, ppd, column pruning, etc. diff --git a/planner/core/physical_plan_test.go b/planner/core/physical_plan_test.go index bd5e2d7388908..629d12e638a8f 100644 --- a/planner/core/physical_plan_test.go +++ b/planner/core/physical_plan_test.go @@ -240,11 +240,11 @@ func (s *testPlanSuite) TestDAGPlanBuilderJoin(c *C) { }, { sql: "select * from t t1 join t t2 on t1.a = t2.a join t t3 on t1.a = t3.a", - best: "MergeInnerJoin{MergeInnerJoin{TableReader(Table(t))->TableReader(Table(t))}(t1.a,t2.a)->TableReader(Table(t))}(t1.a,t3.a)", + best: "MergeInnerJoin{MergeInnerJoin{TableReader(Table(t))->TableReader(Table(t))}(t1.a,t3.a)->TableReader(Table(t))}(t1.a,t2.a)->Projection", }, { sql: "select * from t t1 join t t2 on t1.a = t2.a join t t3 on t1.b = t3.a", - best: "LeftHashJoin{MergeInnerJoin{TableReader(Table(t))->TableReader(Table(t))}(t1.a,t2.a)->TableReader(Table(t))}(t1.b,t3.a)", + best: "LeftHashJoin{LeftHashJoin{TableReader(Table(t))->TableReader(Table(t))}(t1.b,t3.a)->TableReader(Table(t))}(t1.a,t2.a)->Projection", }, { sql: "select * from t t1 join t t2 on t1.b = t2.a order by t1.a", @@ -269,7 +269,7 @@ func (s *testPlanSuite) TestDAGPlanBuilderJoin(c *C) { }, { sql: "select * from t t1 join t t2 on t1.b = t2.b join t t3 on t1.b = t3.b", - best: "LeftHashJoin{LeftHashJoin{TableReader(Table(t))->TableReader(Table(t))}(t1.b,t2.b)->TableReader(Table(t))}(t1.b,t3.b)", + best: "LeftHashJoin{LeftHashJoin{TableReader(Table(t))->TableReader(Table(t))}(t1.b,t3.b)->TableReader(Table(t))}(t1.b,t2.b)->Projection", }, { sql: "select * from t t1 join t t2 on t1.a = t2.a order by t1.a", diff --git a/planner/core/rule_join_reorder_dp.go b/planner/core/rule_join_reorder_dp.go index a7a70731bf4d1..a47e38877a112 100644 --- a/planner/core/rule_join_reorder_dp.go +++ b/planner/core/rule_join_reorder_dp.go @@ -23,19 +23,25 @@ import ( type joinReorderDPSolver struct { ctx sessionctx.Context - newJoin func(lChild, rChild LogicalPlan, eqConds []*expression.ScalarFunction) LogicalPlan + newJoin func(lChild, rChild LogicalPlan, eqConds []*expression.ScalarFunction, otherConds []expression.Expression) LogicalPlan } -type joinGroupEdge struct { +type joinGroupEqEdge struct { nodeIDs []int edge *expression.ScalarFunction } -func (s *joinReorderDPSolver) solve(joinGroup []LogicalPlan, conds []expression.Expression) (LogicalPlan, error) { +type joinGroupNonEqEdge struct { + nodeIDs []int + idMask uint + expr expression.Expression +} + +func (s *joinReorderDPSolver) solve(joinGroup []LogicalPlan, eqConds, otherConds []expression.Expression) (LogicalPlan, error) { adjacents := make([][]int, len(joinGroup)) - totalEdges := make([]joinGroupEdge, 0, len(conds)) - addEdge := func(node1, node2 int, edgeContent *expression.ScalarFunction) { - totalEdges = append(totalEdges, joinGroupEdge{ + totalEqEdges := make([]joinGroupEqEdge, 0, len(eqConds)) + addEqEdge := func(node1, node2 int, edgeContent *expression.ScalarFunction) { + totalEqEdges = append(totalEqEdges, joinGroupEqEdge{ nodeIDs: []int{node1, node2}, edge: edgeContent, }) @@ -43,7 +49,7 @@ func (s *joinReorderDPSolver) solve(joinGroup []LogicalPlan, conds []expression. adjacents[node2] = append(adjacents[node2], node1) } // Build Graph for join group - for _, cond := range conds { + for _, cond := range eqConds { sf := cond.(*expression.ScalarFunction) lCol := sf.GetArgs()[0].(*expression.Column) rCol := sf.GetArgs()[1].(*expression.Column) @@ -55,7 +61,23 @@ func (s *joinReorderDPSolver) solve(joinGroup []LogicalPlan, conds []expression. if err != nil { return nil, err } - addEdge(lIdx, rIdx, sf) + addEqEdge(lIdx, rIdx, sf) + } + totalNonEqEdges := make([]joinGroupNonEqEdge, 0, len(otherConds)) + for _, cond := range otherConds { + cols := expression.ExtractColumns(cond) + mask := uint(0) + for _, col := range cols { + idx, err := findNodeIndexInGroup(joinGroup, col) + if err != nil { + return nil, err + } + mask |= 1 << uint(idx) + } + totalNonEqEdges = append(totalNonEqEdges, joinGroupNonEqEdge{ + idMask: mask, + expr: cond, + }) } visited := make([]bool, len(joinGroup)) nodeID2VisitID := make([]int, len(joinGroup)) @@ -66,15 +88,37 @@ func (s *joinReorderDPSolver) solve(joinGroup []LogicalPlan, conds []expression. continue } visitID2NodeID := s.bfsGraph(i, visited, adjacents, nodeID2VisitID) + nodeIDMask := uint(0) + for _, nodeID := range visitID2NodeID { + nodeIDMask |= uint(nodeID) + } + var subNonEqEdges []joinGroupNonEqEdge + for i := len(totalNonEqEdges) - 1; i >= 0; i-- { + // If this edge is not the subset of the current sub graph. + if totalNonEqEdges[i].idMask&nodeIDMask != nodeIDMask { + continue + } + newMask := uint(0) + for _, nodeID := range totalNonEqEdges[i].nodeIDs { + newMask |= uint(nodeID) + } + totalNonEqEdges[i].idMask = newMask + subNonEqEdges = append(subNonEqEdges, totalNonEqEdges[i]) + totalNonEqEdges = append(totalNonEqEdges[:i], totalNonEqEdges[i+1:]...) + } // Do DP on each sub graph. - join, err := s.dpGraph(visitID2NodeID, nodeID2VisitID, joinGroup, totalEdges) + join, err := s.dpGraph(visitID2NodeID, nodeID2VisitID, joinGroup, totalEqEdges, subNonEqEdges) if err != nil { return nil, err } joins = append(joins, join) } + remainedOtherConds := make([]expression.Expression, 0, len(totalNonEqEdges)) + for _, edge := range totalNonEqEdges { + remainedOtherConds = append(remainedOtherConds, edge.expr) + } // Build bushy tree for cartesian joins. - return s.makeBushyJoin(joins), nil + return s.makeBushyJoin(joins, remainedOtherConds), nil } // bfsGraph bfs a sub graph starting at startPos. And relabel its label for future use. @@ -98,7 +142,8 @@ func (s *joinReorderDPSolver) bfsGraph(startNode int, visited []bool, adjacents return visitID2NodeID } -func (s *joinReorderDPSolver) dpGraph(newPos2OldPos, oldPos2NewPos []int, joinGroup []LogicalPlan, totalEdges []joinGroupEdge) (LogicalPlan, error) { +func (s *joinReorderDPSolver) dpGraph(newPos2OldPos, oldPos2NewPos []int, joinGroup []LogicalPlan, + totalEqEdges []joinGroupEqEdge, totalNonEqEdges []joinGroupNonEqEdge) (LogicalPlan, error) { nodeCnt := uint(len(newPos2OldPos)) bestPlan := make([]LogicalPlan, 1< 0 && (rightMask&(1< 0 { - usedEdges = append(usedEdges, edge) + usedEqEdges = append(usedEqEdges, edge) } else if (leftMask&(1< 0 && (rightMask&(1< 0 { - usedEdges = append(usedEdges, edge) + usedEqEdges = append(usedEqEdges, edge) + } + } + for _, edge := range totalNonEqEdges { + // If the result is false, means that the current group hasn't covered the columns involved in the expression. + if edge.idMask&(leftMask|rightMask) != edge.idMask { + continue + } + // Check whether this expression is only built from one side of the join. + if edge.idMask&leftMask == 0 || edge.idMask&rightMask == 0 { + continue } + otherConds = append(otherConds, edge.expr) } - return usedEdges + return usedEqEdges, otherConds } -func (s *joinReorderDPSolver) newJoinWithEdge(leftPlan, rightPlan LogicalPlan, edges []joinGroupEdge) (LogicalPlan, error) { +func (s *joinReorderDPSolver) newJoinWithEdge(leftPlan, rightPlan LogicalPlan, edges []joinGroupEqEdge, otherConds []expression.Expression) (LogicalPlan, error) { var eqConds []*expression.ScalarFunction for _, edge := range edges { lCol := edge.edge.GetArgs()[0].(*expression.Column) @@ -165,13 +226,13 @@ func (s *joinReorderDPSolver) newJoinWithEdge(leftPlan, rightPlan LogicalPlan, e eqConds = append(eqConds, newSf) } } - join := s.newJoin(leftPlan, rightPlan, eqConds) + join := s.newJoin(leftPlan, rightPlan, eqConds, otherConds) _, err := join.deriveStats() return join, err } // Make cartesian join as bushy tree. -func (s *joinReorderDPSolver) makeBushyJoin(cartesianJoinGroup []LogicalPlan) LogicalPlan { +func (s *joinReorderDPSolver) makeBushyJoin(cartesianJoinGroup []LogicalPlan, otherConds []expression.Expression) LogicalPlan { for len(cartesianJoinGroup) > 1 { resultJoinGroup := make([]LogicalPlan, 0, len(cartesianJoinGroup)) for i := 0; i < len(cartesianJoinGroup); i += 2 { @@ -179,7 +240,16 @@ func (s *joinReorderDPSolver) makeBushyJoin(cartesianJoinGroup []LogicalPlan) Lo resultJoinGroup = append(resultJoinGroup, cartesianJoinGroup[i]) break } - resultJoinGroup = append(resultJoinGroup, s.newJoin(cartesianJoinGroup[i], cartesianJoinGroup[i+1], nil)) + mergedSchema := expression.MergeSchema(cartesianJoinGroup[i].Schema(), cartesianJoinGroup[i+1].Schema()) + var usedOtherConds []expression.Expression + for i := len(otherConds) - 1; i >= 0; i-- { + cols := expression.ExtractColumns(otherConds[i]) + if mergedSchema.ColumnsIndices(cols) != nil { + usedOtherConds = append(usedOtherConds, otherConds[i]) + otherConds = append(otherConds[:i], otherConds[i+1:]...) + } + } + resultJoinGroup = append(resultJoinGroup, s.newJoin(cartesianJoinGroup[i], cartesianJoinGroup[i+1], nil, usedOtherConds)) } cartesianJoinGroup = resultJoinGroup } @@ -194,3 +264,24 @@ func findNodeIndexInGroup(group []LogicalPlan, col *expression.Column) (int, err } return -1, ErrUnknownColumn.GenWithStackByArgs(col, "JOIN REORDER RULE") } + +func (s *joinReorderDPSolver) newJoinWithConds(leftPlan, rightPlan LogicalPlan, eqConds []*expression.ScalarFunction, otherConds []expression.Expression) LogicalPlan { + join := s.newCartesianJoin(leftPlan, rightPlan) + join.EqualConditions = eqConds + join.OtherConditions = otherConds + for _, eqCond := range join.EqualConditions { + join.LeftJoinKeys = append(join.LeftJoinKeys, eqCond.GetArgs()[0].(*expression.Column)) + join.RightJoinKeys = append(join.RightJoinKeys, eqCond.GetArgs()[1].(*expression.Column)) + } + return join +} + +func (s *joinReorderDPSolver) newCartesianJoin(lChild, rChild LogicalPlan) *LogicalJoin { + join := LogicalJoin{ + JoinType: InnerJoin, + reordered: true, + }.Init(s.ctx) + join.SetSchema(expression.MergeSchema(lChild.Schema(), rChild.Schema())) + join.SetChildren(lChild, rChild) + return join +} diff --git a/planner/core/rule_join_reorder_dp_test.go b/planner/core/rule_join_reorder_dp_test.go index 12da2ddae6260..d50df370f2f05 100644 --- a/planner/core/rule_join_reorder_dp_test.go +++ b/planner/core/rule_join_reorder_dp_test.go @@ -56,7 +56,7 @@ func (mj *mockLogicalJoin) deriveStats() (*property.StatsInfo, error) { return mj.statsMap[mj.involvedNodeSet], nil } -func (s *testJoinReorderDPSuite) newMockJoin(lChild, rChild LogicalPlan, eqConds []*expression.ScalarFunction) LogicalPlan { +func (s *testJoinReorderDPSuite) newMockJoin(lChild, rChild LogicalPlan, eqConds []*expression.ScalarFunction, _ []expression.Expression) LogicalPlan { retJoin := mockLogicalJoin{}.init(s.ctx) retJoin.schema = expression.MergeSchema(lChild.Schema(), rChild.Schema()) retJoin.statsMap = s.statsMap @@ -192,7 +192,7 @@ func (s *testJoinReorderDPSuite) TestDPReorderTPCHQ5(c *C) { ctx: s.ctx, newJoin: s.newMockJoin, } - result, err := solver.solve(joinGroups, eqConds) + result, err := solver.solve(joinGroups, eqConds, nil) c.Assert(err, IsNil) c.Assert(s.planToString(result), Equals, "MockJoin{supplier, MockJoin{lineitem, MockJoin{orders, MockJoin{customer, MockJoin{nation, region}}}}}") } @@ -207,7 +207,7 @@ func (s *testJoinReorderDPSuite) TestDPReorderAllCartesian(c *C) { ctx: s.ctx, newJoin: s.newMockJoin, } - result, err := solver.solve(joinGroup, nil) + result, err := solver.solve(joinGroup, nil, nil) c.Assert(err, IsNil) c.Assert(s.planToString(result), Equals, "MockJoin{MockJoin{a, b}, MockJoin{c, d}}") } diff --git a/planner/core/rule_join_reorder_greedy.go b/planner/core/rule_join_reorder_greedy.go index f56401083360d..bb16efddc66ee 100644 --- a/planner/core/rule_join_reorder_greedy.go +++ b/planner/core/rule_join_reorder_greedy.go @@ -48,7 +48,7 @@ func extractJoinGroup(p LogicalPlan) (group []LogicalPlan, eqEdges []*expression return group, eqEdges, otherConds } -type joinReOrderGreedySolver struct { +type joinReOrderSolver struct { } type joinReorderGreedySingleGroupSolver struct { @@ -196,11 +196,11 @@ func (s *joinReorderGreedySingleGroupSolver) newCartesianJoin(lChild, rChild Log return join } -func (s *joinReOrderGreedySolver) optimize(p LogicalPlan) (LogicalPlan, error) { +func (s *joinReOrderSolver) optimize(p LogicalPlan) (LogicalPlan, error) { return s.optimizeRecursive(p.context(), p) } -func (s *joinReOrderGreedySolver) optimizeRecursive(ctx sessionctx.Context, p LogicalPlan) (LogicalPlan, error) { +func (s *joinReOrderSolver) optimizeRecursive(ctx sessionctx.Context, p LogicalPlan) (LogicalPlan, error) { var err error curJoinGroup, eqEdges, otherConds := extractJoinGroup(p) if len(curJoinGroup) > 1 { @@ -210,17 +210,29 @@ func (s *joinReOrderGreedySolver) optimizeRecursive(ctx sessionctx.Context, p Lo return nil, err } } - groupSolver := &joinReorderGreedySingleGroupSolver{ - ctx: ctx, - curJoinGroup: curJoinGroup, - eqEdges: eqEdges, - otherConds: otherConds, - } - p, err = groupSolver.solve() - if err != nil { - return nil, err + if len(curJoinGroup) > 10 { + greedySolver := &joinReorderGreedySingleGroupSolver{ + ctx: ctx, + curJoinGroup: curJoinGroup, + eqEdges: eqEdges, + otherConds: otherConds, + } + p, err = greedySolver.solve() + if err != nil { + return nil, err + } + return p, nil + } else { + dpSolver := &joinReorderDPSolver{ + ctx: ctx, + } + dpSolver.newJoin = dpSolver.newJoinWithConds + p, err = dpSolver.solve(curJoinGroup, expression.ScalarFuncs2Exprs(eqEdges), otherConds) + if err != nil { + return nil, err + } + return p, nil } - return p, nil } newChildren := make([]LogicalPlan, 0, len(p.Children())) for _, child := range p.Children() { From d38fc887bb3f077afcb1be49e12675ca686a2036 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Tue, 25 Dec 2018 19:01:54 +0800 Subject: [PATCH 02/12] fix explain test --- cmd/explaintest/r/explain_complex.result | 20 +- .../r/explain_complex_stats.result | 4 +- cmd/explaintest/r/explain_easy.result | 73 ++- cmd/explaintest/r/explain_easy_stats.result | 24 +- cmd/explaintest/r/tpch.result | 552 +++++++++--------- 5 files changed, 337 insertions(+), 336 deletions(-) diff --git a/cmd/explaintest/r/explain_complex.result b/cmd/explaintest/r/explain_complex.result index 2b43ae3a2357a..99366a5f23718 100644 --- a/cmd/explaintest/r/explain_complex.result +++ b/cmd/explaintest/r/explain_complex.result @@ -118,20 +118,20 @@ id count task operator info Projection_13 1.00 root gad.id, test.dd.id, gad.aid, gad.cm, test.dd.dic, test.dd.ip, test.dd.t, gad.p1, gad.p2, gad.p3, gad.p4, gad.p5, gad.p6_md5, gad.p7_md5, gad.ext, gad.t └─Limit_16 1.00 root offset:0, count:2500 └─HashAgg_19 1.00 root group by:gad.aid, test.dd.dic, funcs:firstrow(gad.id), firstrow(gad.aid), firstrow(gad.cm), firstrow(gad.p1), firstrow(gad.p2), firstrow(gad.p3), firstrow(gad.p4), firstrow(gad.p5), firstrow(gad.p6_md5), firstrow(gad.p7_md5), firstrow(gad.ext), firstrow(gad.t), firstrow(test.dd.id), firstrow(test.dd.dic), firstrow(test.dd.ip), firstrow(test.dd.t) - └─IndexJoin_24 0.00 root inner join, inner:IndexLookUp_23, outer key:gad.aid, inner key:test.dd.aid, other cond:eq(test.dd.ip, gad.ip), gt(test.dd.t, gad.t) - ├─IndexLookUp_23 0.00 root - │ ├─IndexScan_20 10.00 cop table:dd, index:aid, dic, range: decided by [gad.aid gad.ip], keep order:false, stats:pseudo - │ └─Selection_22 0.00 cop eq(test.dd.bm, 0), eq(test.dd.pt, "android"), gt(test.dd.t, 1478143908) - │ └─TableScan_21 10.00 cop table:dd, keep order:false, stats:pseudo - └─IndexLookUp_33 3.33 root - ├─IndexScan_30 3333.33 cop table:gad, index:t, range:(1478143908,+inf], keep order:false, stats:pseudo - └─Selection_32 3.33 cop eq(gad.pt, "android") - └─TableScan_31 3333.33 cop table:st, keep order:false, stats:pseudo + └─IndexJoin_24 0.00 root inner join, inner:IndexLookUp_23, outer key:gad.aid, inner key:test.dd.aid, other cond:eq(gad.ip, test.dd.ip) + ├─IndexLookUp_33 3.33 root + │ ├─IndexScan_30 3333.33 cop table:gad, index:t, range:(1478143908,+inf], keep order:false, stats:pseudo + │ └─Selection_32 3.33 cop eq(gad.pt, "android") + │ └─TableScan_31 3333.33 cop table:st, keep order:false, stats:pseudo + └─IndexLookUp_23 0.00 root + ├─IndexScan_20 10.00 cop table:dd, index:aid, dic, range: decided by [gad.aid gad.ip], keep order:false, stats:pseudo + └─Selection_22 0.00 cop eq(test.dd.bm, 0), eq(test.dd.pt, "android"), gt(test.dd.t, 1478143908) + └─TableScan_21 10.00 cop table:dd, keep order:false, stats:pseudo explain select gad.id as gid,sdk.id as sid,gad.aid as aid,gad.cm as cm,sdk.dic as dic,sdk.ip as ip, sdk.t as t, gad.p1 as p1, gad.p2 as p2, gad.p3 as p3, gad.p4 as p4, gad.p5 as p5, gad.p6_md5 as p6, gad.p7_md5 as p7, gad.ext as ext from st gad join dd sdk on gad.aid = sdk.aid and gad.dic = sdk.mac and gad.t < sdk.t where gad.t > 1477971479 and gad.bm = 0 and gad.pt = 'ios' and gad.dit = 'mac' and sdk.t > 1477971479 and sdk.bm = 0 and sdk.pt = 'ios' limit 3000; id count task operator info Projection_10 0.00 root gad.id, sdk.id, gad.aid, gad.cm, sdk.dic, sdk.ip, sdk.t, gad.p1, gad.p2, gad.p3, gad.p4, gad.p5, gad.p6_md5, gad.p7_md5, gad.ext └─Limit_13 0.00 root offset:0, count:3000 - └─IndexJoin_18 0.00 root inner join, inner:IndexLookUp_17, outer key:gad.aid, inner key:sdk.aid, other cond:eq(gad.dic, sdk.mac), lt(gad.t, sdk.t) + └─IndexJoin_18 0.00 root inner join, inner:IndexLookUp_17, outer key:gad.aid, inner key:sdk.aid, other cond:eq(gad.dic, sdk.mac) ├─IndexLookUp_27 0.00 root │ ├─IndexScan_24 3333.33 cop table:gad, index:t, range:(1477971479,+inf], keep order:false, stats:pseudo │ └─Selection_26 0.00 cop eq(gad.bm, 0), eq(gad.dit, "mac"), eq(gad.pt, "ios") diff --git a/cmd/explaintest/r/explain_complex_stats.result b/cmd/explaintest/r/explain_complex_stats.result index 1569a2d3a850c..a2f3ff20c1af5 100644 --- a/cmd/explaintest/r/explain_complex_stats.result +++ b/cmd/explaintest/r/explain_complex_stats.result @@ -128,7 +128,7 @@ id count task operator info Projection_13 424.00 root gad.id, test.dd.id, gad.aid, gad.cm, test.dd.dic, test.dd.ip, test.dd.t, gad.p1, gad.p2, gad.p3, gad.p4, gad.p5, gad.p6_md5, gad.p7_md5, gad.ext, gad.t └─Limit_16 424.00 root offset:0, count:2500 └─HashAgg_19 424.00 root group by:gad.aid, test.dd.dic, funcs:firstrow(gad.id), firstrow(gad.aid), firstrow(gad.cm), firstrow(gad.p1), firstrow(gad.p2), firstrow(gad.p3), firstrow(gad.p4), firstrow(gad.p5), firstrow(gad.p6_md5), firstrow(gad.p7_md5), firstrow(gad.ext), firstrow(gad.t), firstrow(test.dd.id), firstrow(test.dd.dic), firstrow(test.dd.ip), firstrow(test.dd.t) - └─IndexJoin_24 424.00 root inner join, inner:IndexLookUp_23, outer key:gad.aid, inner key:test.dd.aid, other cond:eq(gad.ip, test.dd.ip), gt(test.dd.t, gad.t) + └─IndexJoin_24 424.00 root inner join, inner:IndexLookUp_23, outer key:gad.aid, inner key:test.dd.aid, other cond:eq(gad.ip, test.dd.ip) ├─TableReader_29 424.00 root data:Selection_28 │ └─Selection_28 424.00 cop eq(gad.bm, 0), eq(gad.pt, "android"), gt(gad.t, 1478143908) │ └─TableScan_27 1999.00 cop table:gad, range:[0,+inf], keep order:false @@ -140,7 +140,7 @@ explain select gad.id as gid,sdk.id as sid,gad.aid as aid,gad.cm as cm,sdk.dic a id count task operator info Projection_10 170.34 root gad.id, sdk.id, gad.aid, gad.cm, sdk.dic, sdk.ip, sdk.t, gad.p1, gad.p2, gad.p3, gad.p4, gad.p5, gad.p6_md5, gad.p7_md5, gad.ext └─Limit_13 170.34 root offset:0, count:3000 - └─IndexJoin_18 170.34 root inner join, inner:IndexLookUp_17, outer key:gad.aid, inner key:sdk.aid, other cond:eq(gad.dic, sdk.mac), lt(gad.t, sdk.t) + └─IndexJoin_18 170.34 root inner join, inner:IndexLookUp_17, outer key:gad.aid, inner key:sdk.aid, other cond:eq(gad.dic, sdk.mac) ├─TableReader_23 170.34 root data:Selection_22 │ └─Selection_22 170.34 cop eq(gad.bm, 0), eq(gad.dit, "mac"), eq(gad.pt, "ios"), gt(gad.t, 1477971479) │ └─TableScan_21 1999.00 cop table:gad, range:[0,+inf], keep order:false diff --git a/cmd/explaintest/r/explain_easy.result b/cmd/explaintest/r/explain_easy.result index f6631ebd4056b..245a8bd252f2b 100644 --- a/cmd/explaintest/r/explain_easy.result +++ b/cmd/explaintest/r/explain_easy.result @@ -60,12 +60,12 @@ explain select count(b.c2) from t1 a, t2 b where a.c1 = b.c2 group by a.c1; id count task operator info Projection_11 10000.00 root cast(join_agg_0) └─IndexJoin_14 10000.00 root inner join, inner:TableReader_13, outer key:b.c2, inner key:a.c1 - ├─HashAgg_21 8000.00 root group by:col_2, funcs:count(col_0), firstrow(col_1) - │ └─TableReader_22 8000.00 root data:HashAgg_17 - │ └─HashAgg_17 8000.00 cop group by:b.c2, funcs:count(b.c2), firstrow(b.c2) - │ └─TableScan_20 10000.00 cop table:b, range:[-inf,+inf], keep order:false, stats:pseudo - └─TableReader_13 10.00 root data:TableScan_12 - └─TableScan_12 10.00 cop table:a, range: decided by [b.c2], keep order:false, stats:pseudo + ├─TableReader_13 10.00 root data:TableScan_12 + │ └─TableScan_12 10.00 cop table:a, range: decided by [b.c2], keep order:false, stats:pseudo + └─HashAgg_21 8000.00 root group by:col_2, funcs:count(col_0), firstrow(col_1) + └─TableReader_22 8000.00 root data:HashAgg_17 + └─HashAgg_17 8000.00 cop group by:b.c2, funcs:count(b.c2), firstrow(b.c2) + └─TableScan_20 10000.00 cop table:b, range:[-inf,+inf], keep order:false, stats:pseudo explain select * from t2 order by t2.c2 limit 0, 1; id count task operator info TopN_7 1.00 root test.t2.c2:asc, offset:0, count:1 @@ -96,12 +96,12 @@ explain select c1 from t1 where c1 in (select c2 from t2); id count task operator info Projection_9 10000.00 root test.t1.c1 └─IndexJoin_12 10000.00 root inner join, inner:TableReader_11, outer key:test.t2.c2, inner key:test.t1.c1 - ├─HashAgg_19 8000.00 root group by:col_1, funcs:firstrow(col_0) - │ └─TableReader_20 8000.00 root data:HashAgg_15 - │ └─HashAgg_15 8000.00 cop group by:test.t2.c2, funcs:firstrow(test.t2.c2) - │ └─TableScan_18 10000.00 cop table:t2, range:[-inf,+inf], keep order:false, stats:pseudo - └─TableReader_11 10.00 root data:TableScan_10 - └─TableScan_10 10.00 cop table:t1, range: decided by [test.t2.c2], keep order:false, stats:pseudo + ├─TableReader_11 10.00 root data:TableScan_10 + │ └─TableScan_10 10.00 cop table:t1, range: decided by [test.t2.c2], keep order:false, stats:pseudo + └─HashAgg_19 8000.00 root group by:col_1, funcs:firstrow(col_0) + └─TableReader_20 8000.00 root data:HashAgg_15 + └─HashAgg_15 8000.00 cop group by:test.t2.c2, funcs:firstrow(test.t2.c2) + └─TableScan_18 10000.00 cop table:t2, range:[-inf,+inf], keep order:false, stats:pseudo explain select (select count(1) k from t1 s where s.c1 = t1.c1 having k != 0) from t1; id count task operator info Projection_12 10000.00 root k @@ -356,13 +356,12 @@ id count task operator info TableDual_5 0.00 root rows:0 explain select * from t t1 join t t2 where t1.b = t2.b and t2.b is null; id count task operator info -Projection_7 12.50 root t1.a, t1.b, t2.a, t2.b -└─HashRightJoin_9 12.50 root inner join, inner:TableReader_12, equal:[eq(t2.b, t1.b)] - ├─TableReader_12 10.00 root data:Selection_11 - │ └─Selection_11 10.00 cop isnull(t2.b) - │ └─TableScan_10 10000.00 cop table:t2, range:[-inf,+inf], keep order:false, stats:pseudo - └─TableReader_14 10000.00 root data:TableScan_13 - └─TableScan_13 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo +HashLeftJoin_8 12.50 root inner join, inner:TableReader_14, equal:[eq(t1.b, t2.b)] +├─TableReader_11 10000.00 root data:TableScan_10 +│ └─TableScan_10 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo +└─TableReader_14 10.00 root data:Selection_13 + └─Selection_13 10.00 cop isnull(t2.b) + └─TableScan_12 10000.00 cop table:t2, range:[-inf,+inf], keep order:false, stats:pseudo explain select * from t t1 where not exists (select * from t t2 where t1.b = t2.b); id count task operator info HashLeftJoin_9 8000.00 root anti semi join, inner:TableReader_13, equal:[eq(t1.b, t2.b)] @@ -480,15 +479,15 @@ Projection_12 10000.00 root 9_aux_0 ├─TableReader_16 10000.00 root data:TableScan_15 │ └─TableScan_15 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo └─HashAgg_19 1.00 root funcs:count(join_agg_0) - └─HashRightJoin_21 8000.00 root inner join, inner:HashAgg_27, equal:[eq(t1.a, s.a)] - ├─HashAgg_27 6400.00 root group by:col_2, funcs:count(col_0), firstrow(col_1) - │ └─TableReader_28 6400.00 root data:HashAgg_22 - │ └─HashAgg_22 6400.00 cop group by:t1.a, funcs:count(1), firstrow(t1.a) - │ └─Selection_26 8000.00 cop eq(t1.a, test.t.a) - │ └─TableScan_25 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo - └─TableReader_34 8000.00 root data:Selection_33 - └─Selection_33 8000.00 cop eq(s.a, test.t.a) - └─TableScan_32 10000.00 cop table:s, range:[-inf,+inf], keep order:false, stats:pseudo + └─HashLeftJoin_20 8000.00 root inner join, inner:HashAgg_30, equal:[eq(s.a, t1.a)] + ├─TableReader_24 8000.00 root data:Selection_23 + │ └─Selection_23 8000.00 cop eq(s.a, test.t.a) + │ └─TableScan_22 10000.00 cop table:s, range:[-inf,+inf], keep order:false, stats:pseudo + └─HashAgg_30 6400.00 root group by:col_2, funcs:count(col_0), firstrow(col_1) + └─TableReader_31 6400.00 root data:HashAgg_25 + └─HashAgg_25 6400.00 cop group by:t1.a, funcs:count(1), firstrow(t1.a) + └─Selection_29 8000.00 cop eq(t1.a, test.t.a) + └─TableScan_28 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo explain select * from t ta left outer join t tb on ta.nb = tb.nb and ta.a > 1 where ifnull(tb.a, 1) or tb.a is null; id count task operator info Selection_7 10000.00 root or(ifnull(tb.a, 1), isnull(tb.a)) @@ -514,15 +513,15 @@ Projection_14 10000.00 root 9_aux_0 │ └─TableReader_19 10000.00 root data:TableScan_18 │ └─TableScan_18 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo └─HashAgg_23 1.00 root funcs:count(join_agg_0) - └─HashRightJoin_25 8000.00 root inner join, inner:HashAgg_31, equal:[eq(t1.a, s.a)] - ├─HashAgg_31 6400.00 root group by:col_2, funcs:count(col_0), firstrow(col_1) - │ └─TableReader_32 6400.00 root data:HashAgg_26 - │ └─HashAgg_26 6400.00 cop group by:t1.a, funcs:count(1), firstrow(t1.a) - │ └─Selection_30 8000.00 cop eq(t1.a, test.t.a) - │ └─TableScan_29 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo - └─TableReader_38 8000.00 root data:Selection_37 - └─Selection_37 8000.00 cop eq(s.a, test.t.a) - └─TableScan_36 10000.00 cop table:s, range:[-inf,+inf], keep order:false, stats:pseudo + └─HashLeftJoin_24 8000.00 root inner join, inner:HashAgg_34, equal:[eq(s.a, t1.a)] + ├─TableReader_28 8000.00 root data:Selection_27 + │ └─Selection_27 8000.00 cop eq(s.a, test.t.a) + │ └─TableScan_26 10000.00 cop table:s, range:[-inf,+inf], keep order:false, stats:pseudo + └─HashAgg_34 6400.00 root group by:col_2, funcs:count(col_0), firstrow(col_1) + └─TableReader_35 6400.00 root data:HashAgg_29 + └─HashAgg_29 6400.00 cop group by:t1.a, funcs:count(1), firstrow(t1.a) + └─Selection_33 8000.00 cop eq(t1.a, test.t.a) + └─TableScan_32 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo drop table if exists t; create table t(a int); explain select * from t where _tidb_rowid = 0; diff --git a/cmd/explaintest/r/explain_easy_stats.result b/cmd/explaintest/r/explain_easy_stats.result index 12cdb96b29f03..f6c3203ac42a6 100644 --- a/cmd/explaintest/r/explain_easy_stats.result +++ b/cmd/explaintest/r/explain_easy_stats.result @@ -66,12 +66,12 @@ explain select count(b.c2) from t1 a, t2 b where a.c1 = b.c2 group by a.c1; id count task operator info Projection_11 1985.00 root cast(join_agg_0) └─IndexJoin_14 1985.00 root inner join, inner:TableReader_13, outer key:b.c2, inner key:a.c1 - ├─HashAgg_21 1985.00 root group by:col_2, funcs:count(col_0), firstrow(col_1) - │ └─TableReader_22 1985.00 root data:HashAgg_17 - │ └─HashAgg_17 1985.00 cop group by:b.c2, funcs:count(b.c2), firstrow(b.c2) - │ └─TableScan_20 1985.00 cop table:b, range:[-inf,+inf], keep order:false - └─TableReader_13 1.00 root data:TableScan_12 - └─TableScan_12 1.00 cop table:a, range: decided by [b.c2], keep order:false + ├─TableReader_13 1.00 root data:TableScan_12 + │ └─TableScan_12 1.00 cop table:a, range: decided by [b.c2], keep order:false + └─HashAgg_21 1985.00 root group by:col_2, funcs:count(col_0), firstrow(col_1) + └─TableReader_22 1985.00 root data:HashAgg_17 + └─HashAgg_17 1985.00 cop group by:b.c2, funcs:count(b.c2), firstrow(b.c2) + └─TableScan_20 1985.00 cop table:b, range:[-inf,+inf], keep order:false explain select * from t2 order by t2.c2 limit 0, 1; id count task operator info TopN_7 1.00 root test.t2.c2:asc, offset:0, count:1 @@ -94,12 +94,12 @@ explain select c1 from t1 where c1 in (select c2 from t2); id count task operator info Projection_9 1985.00 root test.t1.c1 └─IndexJoin_12 1985.00 root inner join, inner:TableReader_11, outer key:test.t2.c2, inner key:test.t1.c1 - ├─HashAgg_19 1985.00 root group by:col_1, funcs:firstrow(col_0) - │ └─TableReader_20 1985.00 root data:HashAgg_15 - │ └─HashAgg_15 1985.00 cop group by:test.t2.c2, funcs:firstrow(test.t2.c2) - │ └─TableScan_18 1985.00 cop table:t2, range:[-inf,+inf], keep order:false - └─TableReader_11 1.00 root data:TableScan_10 - └─TableScan_10 1.00 cop table:t1, range: decided by [test.t2.c2], keep order:false + ├─TableReader_11 1.00 root data:TableScan_10 + │ └─TableScan_10 1.00 cop table:t1, range: decided by [test.t2.c2], keep order:false + └─HashAgg_19 1985.00 root group by:col_1, funcs:firstrow(col_0) + └─TableReader_20 1985.00 root data:HashAgg_15 + └─HashAgg_15 1985.00 cop group by:test.t2.c2, funcs:firstrow(test.t2.c2) + └─TableScan_18 1985.00 cop table:t2, range:[-inf,+inf], keep order:false explain select * from information_schema.columns; id count task operator info MemTableScan_4 10000.00 root diff --git a/cmd/explaintest/r/tpch.result b/cmd/explaintest/r/tpch.result index 5a9b2b50fd2f8..fd41424ee84be 100644 --- a/cmd/explaintest/r/tpch.result +++ b/cmd/explaintest/r/tpch.result @@ -182,38 +182,39 @@ s_name, p_partkey limit 100; id count task operator info -Projection_36 100.00 root tpch.supplier.s_acctbal, tpch.supplier.s_name, tpch.nation.n_name, tpch.part.p_partkey, tpch.part.p_mfgr, tpch.supplier.s_address, tpch.supplier.s_phone, tpch.supplier.s_comment -└─TopN_39 100.00 root tpch.supplier.s_acctbal:desc, tpch.nation.n_name:asc, tpch.supplier.s_name:asc, tpch.part.p_partkey:asc, offset:0, count:100 - └─HashRightJoin_44 155496.00 root inner join, inner:HashLeftJoin_50, equal:[eq(tpch.part.p_partkey, tpch.partsupp.ps_partkey) eq(tpch.partsupp.ps_supplycost, min(ps_supplycost))] - ├─HashLeftJoin_50 155496.00 root inner join, inner:TableReader_73, equal:[eq(tpch.partsupp.ps_partkey, tpch.part.p_partkey)] - │ ├─HashRightJoin_53 8155010.44 root inner join, inner:HashRightJoin_55, equal:[eq(tpch.supplier.s_suppkey, tpch.partsupp.ps_suppkey)] - │ │ ├─HashRightJoin_55 100000.00 root inner join, inner:HashRightJoin_61, equal:[eq(tpch.nation.n_nationkey, tpch.supplier.s_nationkey)] - │ │ │ ├─HashRightJoin_61 5.00 root inner join, inner:TableReader_66, equal:[eq(tpch.region.r_regionkey, tpch.nation.n_regionkey)] - │ │ │ │ ├─TableReader_66 1.00 root data:Selection_65 - │ │ │ │ │ └─Selection_65 1.00 cop eq(tpch.region.r_name, "ASIA") - │ │ │ │ │ └─TableScan_64 5.00 cop table:region, range:[-inf,+inf], keep order:false - │ │ │ │ └─TableReader_63 25.00 root data:TableScan_62 - │ │ │ │ └─TableScan_62 25.00 cop table:nation, range:[-inf,+inf], keep order:false - │ │ │ └─TableReader_68 500000.00 root data:TableScan_67 - │ │ │ └─TableScan_67 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false - │ │ └─TableReader_70 40000000.00 root data:TableScan_69 - │ │ └─TableScan_69 40000000.00 cop table:partsupp, range:[-inf,+inf], keep order:false - │ └─TableReader_73 155496.00 root data:Selection_72 - │ └─Selection_72 155496.00 cop eq(tpch.part.p_size, 30), like(tpch.part.p_type, "%STEEL", 92) - │ └─TableScan_71 10000000.00 cop table:part, range:[-inf,+inf], keep order:false - └─HashAgg_76 8155010.44 root group by:tpch.partsupp.ps_partkey, funcs:min(tpch.partsupp.ps_supplycost), firstrow(tpch.partsupp.ps_partkey) - └─HashRightJoin_80 8155010.44 root inner join, inner:HashRightJoin_82, equal:[eq(tpch.supplier.s_suppkey, tpch.partsupp.ps_suppkey)] - ├─HashRightJoin_82 100000.00 root inner join, inner:HashRightJoin_88, equal:[eq(tpch.nation.n_nationkey, tpch.supplier.s_nationkey)] - │ ├─HashRightJoin_88 5.00 root inner join, inner:TableReader_93, equal:[eq(tpch.region.r_regionkey, tpch.nation.n_regionkey)] - │ │ ├─TableReader_93 1.00 root data:Selection_92 - │ │ │ └─Selection_92 1.00 cop eq(tpch.region.r_name, "ASIA") - │ │ │ └─TableScan_91 5.00 cop table:region, range:[-inf,+inf], keep order:false - │ │ └─TableReader_90 25.00 root data:TableScan_89 - │ │ └─TableScan_89 25.00 cop table:nation, range:[-inf,+inf], keep order:false - │ └─TableReader_95 500000.00 root data:TableScan_94 - │ └─TableScan_94 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false - └─TableReader_97 40000000.00 root data:TableScan_96 - └─TableScan_96 40000000.00 cop table:partsupp, range:[-inf,+inf], keep order:false +Projection_86 100.00 root tpch.supplier.s_acctbal, tpch.supplier.s_name, tpch.nation.n_name, tpch.part.p_partkey, tpch.part.p_mfgr, tpch.supplier.s_address, tpch.supplier.s_phone, tpch.supplier.s_comment +└─TopN_89 100.00 root tpch.supplier.s_acctbal:desc, tpch.nation.n_name:asc, tpch.supplier.s_name:asc, tpch.part.p_partkey:asc, offset:0, count:100 + └─HashLeftJoin_93 127533.58 root inner join, inner:HashLeftJoin_119, equal:[eq(tpch.partsupp.ps_partkey, tpch.part.p_partkey) eq(min(ps_supplycost), tpch.partsupp.ps_supplycost)] + ├─HashAgg_97 8155010.44 root group by:tpch.partsupp.ps_partkey, funcs:min(tpch.partsupp.ps_supplycost), firstrow(tpch.partsupp.ps_partkey) + │ └─HashLeftJoin_100 8155010.44 root inner join, inner:HashLeftJoin_104, equal:[eq(tpch.partsupp.ps_suppkey, tpch.supplier.s_suppkey)] + │ ├─TableReader_103 40000000.00 root data:TableScan_102 + │ │ └─TableScan_102 40000000.00 cop table:partsupp, range:[-inf,+inf], keep order:false + │ └─HashLeftJoin_104 100000.00 root inner join, inner:HashLeftJoin_112, equal:[eq(tpch.supplier.s_nationkey, tpch.nation.n_nationkey)] + │ ├─TableReader_107 500000.00 root data:TableScan_106 + │ │ └─TableScan_106 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false + │ └─HashLeftJoin_112 5.00 root inner join, inner:TableReader_118, equal:[eq(tpch.nation.n_regionkey, tpch.region.r_regionkey)] + │ ├─TableReader_115 25.00 root data:TableScan_114 + │ │ └─TableScan_114 25.00 cop table:nation, range:[-inf,+inf], keep order:false + │ └─TableReader_118 1.00 root data:Selection_117 + │ └─Selection_117 1.00 cop eq(tpch.region.r_name, "ASIA") + │ └─TableScan_116 5.00 cop table:region, range:[-inf,+inf], keep order:false + └─HashLeftJoin_119 127533.58 root inner join, inner:HashLeftJoin_144, equal:[eq(tpch.partsupp.ps_suppkey, tpch.supplier.s_suppkey)] + ├─IndexJoin_125 625547.12 root inner join, inner:IndexLookUp_124, outer key:tpch.part.p_partkey, inner key:tpch.partsupp.ps_partkey + │ ├─TableReader_141 155496.00 root data:Selection_140 + │ │ └─Selection_140 155496.00 cop eq(tpch.part.p_size, 30), like(tpch.part.p_type, "%STEEL", 92) + │ │ └─TableScan_139 10000000.00 cop table:part, range:[-inf,+inf], keep order:false + │ └─IndexLookUp_124 1.00 root + │ ├─IndexScan_122 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [tpch.part.p_partkey], keep order:false + │ └─TableScan_123 1.00 cop table:partsupp, keep order:false + └─HashLeftJoin_144 100000.00 root inner join, inner:HashLeftJoin_152, equal:[eq(tpch.supplier.s_nationkey, tpch.nation.n_nationkey)] + ├─TableReader_147 500000.00 root data:TableScan_146 + │ └─TableScan_146 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false + └─HashLeftJoin_152 5.00 root inner join, inner:TableReader_158, equal:[eq(tpch.nation.n_regionkey, tpch.region.r_regionkey)] + ├─TableReader_155 25.00 root data:TableScan_154 + │ └─TableScan_154 25.00 cop table:nation, range:[-inf,+inf], keep order:false + └─TableReader_158 1.00 root data:Selection_157 + └─Selection_157 1.00 cop eq(tpch.region.r_name, "ASIA") + └─TableScan_156 5.00 cop table:region, range:[-inf,+inf], keep order:false /* Q3 Shipping Priority Query This query retrieves the 10 unshipped orders with the highest value. @@ -248,21 +249,21 @@ revenue desc, o_orderdate limit 10; id count task operator info -Projection_14 10.00 root tpch.lineitem.l_orderkey, 7_col_0, tpch.orders.o_orderdate, tpch.orders.o_shippriority -└─TopN_17 10.00 root 7_col_0:desc, tpch.orders.o_orderdate:asc, offset:0, count:10 - └─HashAgg_23 40227041.09 root group by:tpch.lineitem.l_orderkey, tpch.orders.o_orderdate, tpch.orders.o_shippriority, funcs:sum(mul(tpch.lineitem.l_extendedprice, minus(1, tpch.lineitem.l_discount))), firstrow(tpch.orders.o_orderdate), firstrow(tpch.orders.o_shippriority), firstrow(tpch.lineitem.l_orderkey) - └─IndexJoin_29 91515927.49 root inner join, inner:IndexLookUp_28, outer key:tpch.orders.o_orderkey, inner key:tpch.lineitem.l_orderkey - ├─HashRightJoin_49 22592975.51 root inner join, inner:TableReader_55, equal:[eq(tpch.customer.c_custkey, tpch.orders.o_custkey)] - │ ├─TableReader_55 1498236.00 root data:Selection_54 - │ │ └─Selection_54 1498236.00 cop eq(tpch.customer.c_mktsegment, "AUTOMOBILE") - │ │ └─TableScan_53 7500000.00 cop table:customer, range:[-inf,+inf], keep order:false - │ └─TableReader_52 36870000.00 root data:Selection_51 - │ └─Selection_51 36870000.00 cop lt(tpch.orders.o_orderdate, 1995-03-13 00:00:00.000000) - │ └─TableScan_50 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false - └─IndexLookUp_28 162945114.27 root - ├─IndexScan_25 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [tpch.orders.o_orderkey], keep order:false - └─Selection_27 162945114.27 cop gt(tpch.lineitem.l_shipdate, 1995-03-13 00:00:00.000000) - └─TableScan_26 1.00 cop table:lineitem, keep order:false +Projection_16 10.00 root tpch.lineitem.l_orderkey, 7_col_0, tpch.orders.o_orderdate, tpch.orders.o_shippriority +└─TopN_19 10.00 root 7_col_0:desc, tpch.orders.o_orderdate:asc, offset:0, count:10 + └─HashAgg_25 40227041.09 root group by:tpch.lineitem.l_orderkey, tpch.orders.o_orderdate, tpch.orders.o_shippriority, funcs:sum(mul(tpch.lineitem.l_extendedprice, minus(1, tpch.lineitem.l_discount))), firstrow(tpch.orders.o_orderdate), firstrow(tpch.orders.o_shippriority), firstrow(tpch.lineitem.l_orderkey) + └─IndexJoin_31 91515927.49 root inner join, inner:IndexLookUp_30, outer key:tpch.orders.o_orderkey, inner key:tpch.lineitem.l_orderkey + ├─HashRightJoin_51 22592975.51 root inner join, inner:TableReader_57, equal:[eq(tpch.customer.c_custkey, tpch.orders.o_custkey)] + │ ├─TableReader_57 1498236.00 root data:Selection_56 + │ │ └─Selection_56 1498236.00 cop eq(tpch.customer.c_mktsegment, "AUTOMOBILE") + │ │ └─TableScan_55 7500000.00 cop table:customer, range:[-inf,+inf], keep order:false + │ └─TableReader_54 36870000.00 root data:Selection_53 + │ └─Selection_53 36870000.00 cop lt(tpch.orders.o_orderdate, 1995-03-13 00:00:00.000000) + │ └─TableScan_52 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false + └─IndexLookUp_30 162945114.27 root + ├─IndexScan_27 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [tpch.orders.o_orderkey], keep order:false + └─Selection_29 162945114.27 cop gt(tpch.lineitem.l_shipdate, 1995-03-13 00:00:00.000000) + └─TableScan_28 1.00 cop table:lineitem, keep order:false /* Q4 Order Priority Checking Query This query determines how well the order priority system is working and gives an assessment of customer satisfaction. @@ -341,28 +342,29 @@ n_name order by revenue desc; id count task operator info -Sort_23 5.00 root revenue:desc -└─Projection_25 5.00 root tpch.nation.n_name, 13_col_0 - └─HashAgg_28 5.00 root group by:tpch.nation.n_name, funcs:sum(mul(tpch.lineitem.l_extendedprice, minus(1, tpch.lineitem.l_discount))), firstrow(tpch.nation.n_name) - └─IndexJoin_31 11822812.50 root inner join, inner:TableReader_30, outer key:tpch.orders.o_custkey, inner key:tpch.customer.c_custkey, other cond:eq(tpch.supplier.s_nationkey, tpch.customer.c_nationkey) - ├─IndexJoin_38 11822812.50 root inner join, inner:TableReader_37, outer key:tpch.lineitem.l_orderkey, inner key:tpch.orders.o_orderkey - │ ├─HashRightJoin_42 61163763.01 root inner join, inner:HashRightJoin_44, equal:[eq(tpch.supplier.s_suppkey, tpch.lineitem.l_suppkey)] - │ │ ├─HashRightJoin_44 100000.00 root inner join, inner:HashRightJoin_50, equal:[eq(tpch.nation.n_nationkey, tpch.supplier.s_nationkey)] - │ │ │ ├─HashRightJoin_50 5.00 root inner join, inner:TableReader_55, equal:[eq(tpch.region.r_regionkey, tpch.nation.n_regionkey)] - │ │ │ │ ├─TableReader_55 1.00 root data:Selection_54 - │ │ │ │ │ └─Selection_54 1.00 cop eq(tpch.region.r_name, "MIDDLE EAST") - │ │ │ │ │ └─TableScan_53 5.00 cop table:region, range:[-inf,+inf], keep order:false - │ │ │ │ └─TableReader_52 25.00 root data:TableScan_51 - │ │ │ │ └─TableScan_51 25.00 cop table:nation, range:[-inf,+inf], keep order:false - │ │ │ └─TableReader_57 500000.00 root data:TableScan_56 - │ │ │ └─TableScan_56 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false - │ │ └─TableReader_59 300005811.00 root data:TableScan_58 - │ │ └─TableScan_58 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false - │ └─TableReader_37 11822812.50 root data:Selection_36 - │ └─Selection_36 11822812.50 cop ge(tpch.orders.o_orderdate, 1994-01-01 00:00:00.000000), lt(tpch.orders.o_orderdate, 1995-01-01) - │ └─TableScan_35 1.00 cop table:orders, range: decided by [tpch.lineitem.l_orderkey], keep order:false - └─TableReader_30 1.00 root data:TableScan_29 - └─TableScan_29 1.00 cop table:customer, range: decided by [tpch.supplier.s_nationkey tpch.orders.o_custkey], keep order:false +Sort_84 5.00 root revenue:desc +└─Projection_86 5.00 root tpch.nation.n_name, 13_col_0 + └─HashAgg_89 5.00 root group by:tpch.nation.n_name, funcs:sum(mul(tpch.lineitem.l_extendedprice, minus(1, tpch.lineitem.l_discount))), firstrow(tpch.nation.n_name) + └─IndexJoin_92 9763568.69 root inner join, inner:TableReader_91, outer key:tpch.orders.o_custkey, inner key:tpch.customer.c_custkey, other cond:eq(tpch.customer.c_nationkey, tpch.supplier.s_nationkey) + ├─TableReader_91 1.00 root data:TableScan_90 + │ └─TableScan_90 1.00 cop table:customer, range: decided by [tpch.supplier.s_nationkey tpch.orders.o_custkey], keep order:false + └─HashLeftJoin_95 9763568.69 root inner join, inner:HashLeftJoin_120, equal:[eq(tpch.lineitem.l_suppkey, tpch.supplier.s_suppkey)] + ├─IndexJoin_101 47889913.89 root inner join, inner:IndexLookUp_100, outer key:tpch.orders.o_orderkey, inner key:tpch.lineitem.l_orderkey + │ ├─TableReader_117 11822812.50 root data:Selection_116 + │ │ └─Selection_116 11822812.50 cop ge(tpch.orders.o_orderdate, 1994-01-01 00:00:00.000000), lt(tpch.orders.o_orderdate, 1995-01-01) + │ │ └─TableScan_115 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false + │ └─IndexLookUp_100 1.00 root + │ ├─IndexScan_98 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [tpch.orders.o_orderkey], keep order:false + │ └─TableScan_99 1.00 cop table:lineitem, keep order:false + └─HashLeftJoin_120 100000.00 root inner join, inner:HashLeftJoin_128, equal:[eq(tpch.supplier.s_nationkey, tpch.nation.n_nationkey)] + ├─TableReader_123 500000.00 root data:TableScan_122 + │ └─TableScan_122 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false + └─HashLeftJoin_128 5.00 root inner join, inner:TableReader_134, equal:[eq(tpch.nation.n_regionkey, tpch.region.r_regionkey)] + ├─TableReader_131 25.00 root data:TableScan_130 + │ └─TableScan_130 25.00 cop table:nation, range:[-inf,+inf], keep order:false + └─TableReader_134 1.00 root data:Selection_133 + └─Selection_133 1.00 cop eq(tpch.region.r_name, "MIDDLE EAST") + └─TableScan_132 5.00 cop table:region, range:[-inf,+inf], keep order:false /* Q6 Forecasting Revenue Change Query This query quantifies the amount of revenue increase that would have resulted from eliminating certain companywide @@ -441,30 +443,30 @@ supp_nation, cust_nation, l_year; id count task operator info -Sort_22 768.91 root shipping.supp_nation:asc, shipping.cust_nation:asc, shipping.l_year:asc -└─Projection_24 768.91 root shipping.supp_nation, shipping.cust_nation, shipping.l_year, 14_col_0 - └─HashAgg_27 768.91 root group by:shipping.cust_nation, shipping.l_year, shipping.supp_nation, funcs:sum(shipping.volume), firstrow(shipping.supp_nation), firstrow(shipping.cust_nation), firstrow(shipping.l_year) - └─Projection_28 1957240.42 root n1.n_name, n2.n_name, extract("YEAR", tpch.lineitem.l_shipdate), mul(tpch.lineitem.l_extendedprice, minus(1, tpch.lineitem.l_discount)) - └─HashLeftJoin_33 1957240.42 root inner join, inner:TableReader_68, equal:[eq(tpch.customer.c_nationkey, n2.n_nationkey)], other cond:or(and(eq(n1.n_name, "JAPAN"), eq(n2.n_name, "INDIA")), and(eq(n1.n_name, "INDIA"), eq(n2.n_name, "JAPAN"))) - ├─IndexJoin_37 24465505.20 root inner join, inner:TableReader_36, outer key:tpch.orders.o_custkey, inner key:tpch.customer.c_custkey - │ ├─IndexJoin_43 24465505.20 root inner join, inner:TableReader_42, outer key:tpch.lineitem.l_orderkey, inner key:tpch.orders.o_orderkey - │ │ ├─HashRightJoin_47 24465505.20 root inner join, inner:HashRightJoin_53, equal:[eq(tpch.supplier.s_suppkey, tpch.lineitem.l_suppkey)] - │ │ │ ├─HashRightJoin_53 40000.00 root inner join, inner:TableReader_58, equal:[eq(n1.n_nationkey, tpch.supplier.s_nationkey)] - │ │ │ │ ├─TableReader_58 2.00 root data:Selection_57 - │ │ │ │ │ └─Selection_57 2.00 cop or(eq(n1.n_name, "JAPAN"), eq(n1.n_name, "INDIA")) - │ │ │ │ │ └─TableScan_56 25.00 cop table:n1, range:[-inf,+inf], keep order:false - │ │ │ │ └─TableReader_55 500000.00 root data:TableScan_54 - │ │ │ │ └─TableScan_54 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false - │ │ │ └─TableReader_61 91321768.29 root data:Selection_60 - │ │ │ └─Selection_60 91321768.29 cop ge(tpch.lineitem.l_shipdate, 1995-01-01 00:00:00.000000), le(tpch.lineitem.l_shipdate, 1996-12-31 00:00:00.000000) - │ │ │ └─TableScan_59 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false - │ │ └─TableReader_42 1.00 root data:TableScan_41 - │ │ └─TableScan_41 1.00 cop table:orders, range: decided by [tpch.lineitem.l_orderkey], keep order:false - │ └─TableReader_36 1.00 root data:TableScan_35 - │ └─TableScan_35 1.00 cop table:customer, range: decided by [tpch.orders.o_custkey], keep order:false - └─TableReader_68 2.00 root data:Selection_67 - └─Selection_67 2.00 cop or(eq(n2.n_name, "INDIA"), eq(n2.n_name, "JAPAN")) - └─TableScan_66 25.00 cop table:n2, range:[-inf,+inf], keep order:false +Sort_52 768.91 root shipping.supp_nation:asc, shipping.cust_nation:asc, shipping.l_year:asc +└─Projection_54 768.91 root shipping.supp_nation, shipping.cust_nation, shipping.l_year, 14_col_0 + └─HashAgg_57 768.91 root group by:shipping.cust_nation, shipping.l_year, shipping.supp_nation, funcs:sum(shipping.volume), firstrow(shipping.supp_nation), firstrow(shipping.cust_nation), firstrow(shipping.l_year) + └─Projection_58 9818555.15 root n1.n_name, n2.n_name, extract("YEAR", tpch.lineitem.l_shipdate), mul(tpch.lineitem.l_extendedprice, minus(1, tpch.lineitem.l_discount)) + └─HashLeftJoin_60 9818555.15 root inner join, inner:HashLeftJoin_78, equal:[eq(tpch.lineitem.l_orderkey, tpch.orders.o_orderkey)] + ├─HashRightJoin_63 24465505.20 root inner join, inner:HashLeftJoin_68, equal:[eq(tpch.supplier.s_suppkey, tpch.lineitem.l_suppkey)] + │ ├─HashLeftJoin_68 40000.00 root inner join, inner:TableReader_74, equal:[eq(tpch.supplier.s_nationkey, n1.n_nationkey)] + │ │ ├─TableReader_71 500000.00 root data:TableScan_70 + │ │ │ └─TableScan_70 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false + │ │ └─TableReader_74 2.00 root data:Selection_73 + │ │ └─Selection_73 2.00 cop or(eq(n1.n_name, "JAPAN"), eq(n1.n_name, "INDIA")) + │ │ └─TableScan_72 25.00 cop table:n1, range:[-inf,+inf], keep order:false + │ └─TableReader_77 91321768.29 root data:Selection_76 + │ └─Selection_76 91321768.29 cop ge(tpch.lineitem.l_shipdate, 1995-01-01 00:00:00.000000), le(tpch.lineitem.l_shipdate, 1996-12-31 00:00:00.000000) + │ └─TableScan_75 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false + └─HashLeftJoin_78 9047830.45 root inner join, inner:HashLeftJoin_86, equal:[eq(tpch.orders.o_custkey, tpch.customer.c_custkey)] + ├─TableReader_81 75000000.00 root data:TableScan_80 + │ └─TableScan_80 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false + └─HashLeftJoin_86 600000.00 root inner join, inner:TableReader_92, equal:[eq(tpch.customer.c_nationkey, n2.n_nationkey)] + ├─TableReader_89 7500000.00 root data:TableScan_88 + │ └─TableScan_88 7500000.00 cop table:customer, range:[-inf,+inf], keep order:false + └─TableReader_92 2.00 root data:Selection_91 + └─Selection_91 2.00 cop or(eq(n2.n_name, "INDIA"), eq(n2.n_name, "JAPAN")) + └─TableScan_90 25.00 cop table:n2, range:[-inf,+inf], keep order:false /* Q8 National Market Share Query This query determines how the market share of a given nation within a given region has changed over two years for @@ -513,37 +515,36 @@ o_year order by o_year; id count task operator info -Sort_29 718.01 root all_nations.o_year:asc -└─Projection_31 718.01 root all_nations.o_year, div(18_col_0, 18_col_1) - └─HashAgg_34 718.01 root group by:all_nations.o_year, funcs:sum(case(eq(all_nations.nation, "INDIA"), all_nations.volume, 0)), sum(all_nations.volume), firstrow(all_nations.o_year) - └─Projection_35 562348.12 root extract("YEAR", tpch.orders.o_orderdate), mul(tpch.lineitem.l_extendedprice, minus(1, tpch.lineitem.l_discount)), n2.n_name - └─HashLeftJoin_39 562348.12 root inner join, inner:TableReader_87, equal:[eq(tpch.supplier.s_nationkey, n2.n_nationkey)] - ├─IndexJoin_43 562348.12 root inner join, inner:TableReader_42, outer key:tpch.lineitem.l_suppkey, inner key:tpch.supplier.s_suppkey - │ ├─HashLeftJoin_50 562348.12 root inner join, inner:TableReader_83, equal:[eq(tpch.lineitem.l_partkey, tpch.part.p_partkey)] - │ │ ├─IndexJoin_56 90661378.61 root inner join, inner:IndexLookUp_55, outer key:tpch.orders.o_orderkey, inner key:tpch.lineitem.l_orderkey - │ │ │ ├─HashRightJoin_60 22382008.93 root inner join, inner:HashRightJoin_62, equal:[eq(tpch.customer.c_custkey, tpch.orders.o_custkey)] - │ │ │ │ ├─HashRightJoin_62 1500000.00 root inner join, inner:HashRightJoin_68, equal:[eq(n1.n_nationkey, tpch.customer.c_nationkey)] - │ │ │ │ │ ├─HashRightJoin_68 5.00 root inner join, inner:TableReader_73, equal:[eq(tpch.region.r_regionkey, n1.n_regionkey)] - │ │ │ │ │ │ ├─TableReader_73 1.00 root data:Selection_72 - │ │ │ │ │ │ │ └─Selection_72 1.00 cop eq(tpch.region.r_name, "ASIA") - │ │ │ │ │ │ │ └─TableScan_71 5.00 cop table:region, range:[-inf,+inf], keep order:false - │ │ │ │ │ │ └─TableReader_70 25.00 root data:TableScan_69 - │ │ │ │ │ │ └─TableScan_69 25.00 cop table:n1, range:[-inf,+inf], keep order:false - │ │ │ │ │ └─TableReader_75 7500000.00 root data:TableScan_74 - │ │ │ │ │ └─TableScan_74 7500000.00 cop table:customer, range:[-inf,+inf], keep order:false - │ │ │ │ └─TableReader_78 22382008.93 root data:Selection_77 - │ │ │ │ └─Selection_77 22382008.93 cop ge(tpch.orders.o_orderdate, 1995-01-01 00:00:00.000000), le(tpch.orders.o_orderdate, 1996-12-31 00:00:00.000000) - │ │ │ │ └─TableScan_76 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false - │ │ │ └─IndexLookUp_55 1.00 root - │ │ │ ├─IndexScan_53 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [tpch.orders.o_orderkey], keep order:false - │ │ │ └─TableScan_54 1.00 cop table:lineitem, keep order:false - │ │ └─TableReader_83 61674.00 root data:Selection_82 - │ │ └─Selection_82 61674.00 cop eq(tpch.part.p_type, "SMALL PLATED COPPER") - │ │ └─TableScan_81 10000000.00 cop table:part, range:[-inf,+inf], keep order:false - │ └─TableReader_42 1.00 root data:TableScan_41 - │ └─TableScan_41 1.00 cop table:supplier, range: decided by [tpch.lineitem.l_suppkey], keep order:false - └─TableReader_87 25.00 root data:TableScan_86 - └─TableScan_86 25.00 cop table:n2, range:[-inf,+inf], keep order:false +Sort_137 718.01 root all_nations.o_year:asc +└─Projection_139 718.01 root all_nations.o_year, div(18_col_0, 18_col_1) + └─HashAgg_142 718.01 root group by:all_nations.o_year, funcs:sum(case(eq(all_nations.nation, "INDIA"), all_nations.volume, 0)), sum(all_nations.volume), firstrow(all_nations.o_year) + └─Projection_143 372171.05 root extract("YEAR", tpch.orders.o_orderdate), mul(tpch.lineitem.l_extendedprice, minus(1, tpch.lineitem.l_discount)), n2.n_name + └─HashRightJoin_148 372171.05 root inner join, inner:TableReader_209, equal:[eq(n2.n_nationkey, tpch.supplier.s_nationkey)] + ├─TableReader_209 25.00 root data:TableScan_208 + │ └─TableScan_208 25.00 cop table:n2, range:[-inf,+inf], keep order:false + └─IndexJoin_151 372171.05 root inner join, inner:TableReader_150, outer key:tpch.lineitem.l_suppkey, inner key:tpch.supplier.s_suppkey + ├─TableReader_150 1.00 root data:TableScan_149 + │ └─TableScan_149 1.00 cop table:supplier, range: decided by [tpch.lineitem.l_suppkey], keep order:false + └─HashLeftJoin_154 372171.05 root inner join, inner:HashLeftJoin_199, equal:[eq(tpch.customer.c_nationkey, n1.n_nationkey)] + ├─IndexJoin_158 1860855.27 root inner join, inner:TableReader_157, outer key:tpch.orders.o_custkey, inner key:tpch.customer.c_custkey + │ ├─IndexJoin_165 1860855.27 root inner join, inner:TableReader_164, outer key:tpch.lineitem.l_orderkey, inner key:tpch.orders.o_orderkey + │ │ ├─HashRightJoin_184 1860855.27 root inner join, inner:TableReader_189, equal:[eq(tpch.part.p_partkey, tpch.lineitem.l_partkey)] + │ │ │ ├─TableReader_189 61674.00 root data:Selection_188 + │ │ │ │ └─Selection_188 61674.00 cop eq(tpch.part.p_type, "SMALL PLATED COPPER") + │ │ │ │ └─TableScan_187 10000000.00 cop table:part, range:[-inf,+inf], keep order:false + │ │ │ └─TableReader_186 300005811.00 root data:TableScan_185 + │ │ │ └─TableScan_185 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false + │ │ └─TableReader_164 22382008.93 root data:Selection_163 + │ │ └─Selection_163 22382008.93 cop ge(tpch.orders.o_orderdate, 1995-01-01 00:00:00.000000), le(tpch.orders.o_orderdate, 1996-12-31 00:00:00.000000) + │ │ └─TableScan_162 1.00 cop table:orders, range: decided by [tpch.lineitem.l_orderkey], keep order:false + │ └─TableReader_157 1.00 root data:TableScan_156 + │ └─TableScan_156 1.00 cop table:customer, range: decided by [tpch.orders.o_custkey], keep order:false + └─HashLeftJoin_199 5.00 root inner join, inner:TableReader_205, equal:[eq(n1.n_regionkey, tpch.region.r_regionkey)] + ├─TableReader_202 25.00 root data:TableScan_201 + │ └─TableScan_201 25.00 cop table:n1, range:[-inf,+inf], keep order:false + └─TableReader_205 1.00 root data:Selection_204 + └─Selection_204 1.00 cop eq(tpch.region.r_name, "ASIA") + └─TableScan_203 5.00 cop table:region, range:[-inf,+inf], keep order:false /* Q9 Product Type Profit Measure Query This query determines how much profit is made on a given line of parts, broken out by supplier nation and year. @@ -588,29 +589,30 @@ order by nation, o_year desc; id count task operator info -Sort_25 2406.00 root profit.nation:asc, profit.o_year:desc -└─Projection_27 2406.00 root profit.nation, profit.o_year, 14_col_0 - └─HashAgg_30 2406.00 root group by:profit.nation, profit.o_year, funcs:sum(profit.amount), firstrow(profit.nation), firstrow(profit.o_year) - └─Projection_31 971049283.51 root tpch.nation.n_name, extract("YEAR", tpch.orders.o_orderdate), minus(mul(tpch.lineitem.l_extendedprice, minus(1, tpch.lineitem.l_discount)), mul(tpch.partsupp.ps_supplycost, tpch.lineitem.l_quantity)) - └─IndexJoin_35 971049283.51 root inner join, inner:IndexLookUp_34, outer key:tpch.lineitem.l_suppkey, tpch.lineitem.l_partkey, inner key:tpch.partsupp.ps_suppkey, tpch.partsupp.ps_partkey - ├─IndexJoin_41 241379546.70 root inner join, inner:TableReader_40, outer key:tpch.lineitem.l_orderkey, inner key:tpch.orders.o_orderkey - │ ├─HashLeftJoin_52 241379546.70 root inner join, inner:TableReader_69, equal:[eq(tpch.lineitem.l_partkey, tpch.part.p_partkey)] - │ │ ├─HashRightJoin_55 300005811.00 root inner join, inner:HashRightJoin_60, equal:[eq(tpch.supplier.s_suppkey, tpch.lineitem.l_suppkey)] - │ │ │ ├─HashRightJoin_60 500000.00 root inner join, inner:TableReader_64, equal:[eq(tpch.nation.n_nationkey, tpch.supplier.s_nationkey)] - │ │ │ │ ├─TableReader_64 25.00 root data:TableScan_63 - │ │ │ │ │ └─TableScan_63 25.00 cop table:nation, range:[-inf,+inf], keep order:false - │ │ │ │ └─TableReader_62 500000.00 root data:TableScan_61 - │ │ │ │ └─TableScan_61 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false - │ │ │ └─TableReader_66 300005811.00 root data:TableScan_65 - │ │ │ └─TableScan_65 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false - │ │ └─TableReader_69 8000000.00 root data:Selection_68 - │ │ └─Selection_68 8000000.00 cop like(tpch.part.p_name, "%dim%", 92) - │ │ └─TableScan_67 10000000.00 cop table:part, range:[-inf,+inf], keep order:false - │ └─TableReader_40 1.00 root data:TableScan_39 - │ └─TableScan_39 1.00 cop table:orders, range: decided by [tpch.lineitem.l_orderkey], keep order:false - └─IndexLookUp_34 1.00 root - ├─IndexScan_32 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [tpch.lineitem.l_suppkey tpch.lineitem.l_partkey], keep order:false - └─TableScan_33 1.00 cop table:partsupp, keep order:false +Sort_78 2406.00 root profit.nation:asc, profit.o_year:desc +└─Projection_80 2406.00 root profit.nation, profit.o_year, 14_col_0 + └─HashAgg_83 2406.00 root group by:profit.nation, profit.o_year, funcs:sum(profit.amount), firstrow(profit.nation), firstrow(profit.o_year) + └─Projection_84 971049283.51 root tpch.nation.n_name, extract("YEAR", tpch.orders.o_orderdate), minus(mul(tpch.lineitem.l_extendedprice, minus(1, tpch.lineitem.l_discount)), mul(tpch.partsupp.ps_supplycost, tpch.lineitem.l_quantity)) + └─IndexJoin_88 971049283.51 root inner join, inner:IndexLookUp_87, outer key:tpch.lineitem.l_suppkey, tpch.lineitem.l_partkey, inner key:tpch.partsupp.ps_suppkey, tpch.partsupp.ps_partkey + ├─IndexLookUp_87 1.00 root + │ ├─IndexScan_85 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [tpch.lineitem.l_suppkey tpch.lineitem.l_partkey], keep order:false + │ └─TableScan_86 1.00 cop table:partsupp, keep order:false + └─HashLeftJoin_91 241379546.70 root inner join, inner:HashLeftJoin_125, equal:[eq(tpch.lineitem.l_suppkey, tpch.supplier.s_suppkey)] + ├─MergeJoin_93 241379546.70 root inner join, left key:tpch.lineitem.l_orderkey, right key:tpch.orders.o_orderkey + │ ├─IndexJoin_102 241379546.70 root inner join, inner:TableReader_101, outer key:tpch.lineitem.l_partkey, inner key:tpch.part.p_partkey + │ │ ├─TableReader_101 8000000.00 root data:Selection_100 + │ │ │ └─Selection_100 8000000.00 cop like(tpch.part.p_name, "%dim%", 92) + │ │ │ └─TableScan_99 1.00 cop table:part, range: decided by [tpch.lineitem.l_partkey], keep order:false + │ │ └─IndexLookUp_106 300005811.00 root + │ │ ├─IndexScan_104 300005811.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range:[NULL,+inf], keep order:true + │ │ └─TableScan_105 300005811.00 cop table:lineitem, keep order:false + │ └─TableReader_108 75000000.00 root data:TableScan_107 + │ └─TableScan_107 75000000.00 cop table:orders, range:[-inf,+inf], keep order:true + └─HashLeftJoin_125 500000.00 root inner join, inner:TableReader_130, equal:[eq(tpch.supplier.s_nationkey, tpch.nation.n_nationkey)] + ├─TableReader_128 500000.00 root data:TableScan_127 + │ └─TableScan_127 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false + └─TableReader_130 25.00 root data:TableScan_129 + └─TableScan_129 25.00 cop table:nation, range:[-inf,+inf], keep order:false /* Q10 Returned Item Reporting Query The query identifies customers who might be having problems with the parts that are shipped to them. @@ -655,23 +657,23 @@ order by revenue desc limit 20; id count task operator info -Projection_17 20.00 root tpch.customer.c_custkey, tpch.customer.c_name, 9_col_0, tpch.customer.c_acctbal, tpch.nation.n_name, tpch.customer.c_address, tpch.customer.c_phone, tpch.customer.c_comment -└─TopN_20 20.00 root 9_col_0:desc, offset:0, count:20 - └─HashAgg_26 3017307.69 root group by:tpch.customer.c_acctbal, tpch.customer.c_address, tpch.customer.c_comment, tpch.customer.c_custkey, tpch.customer.c_name, tpch.customer.c_phone, tpch.nation.n_name, funcs:sum(mul(tpch.lineitem.l_extendedprice, minus(1, tpch.lineitem.l_discount))), firstrow(tpch.customer.c_custkey), firstrow(tpch.customer.c_name), firstrow(tpch.customer.c_address), firstrow(tpch.customer.c_phone), firstrow(tpch.customer.c_acctbal), firstrow(tpch.customer.c_comment), firstrow(tpch.nation.n_name) - └─IndexJoin_32 12222016.17 root inner join, inner:IndexLookUp_31, outer key:tpch.orders.o_orderkey, inner key:tpch.lineitem.l_orderkey - ├─HashLeftJoin_35 3017307.69 root inner join, inner:TableReader_48, equal:[eq(tpch.customer.c_custkey, tpch.orders.o_custkey)] - │ ├─HashRightJoin_41 7500000.00 root inner join, inner:TableReader_45, equal:[eq(tpch.nation.n_nationkey, tpch.customer.c_nationkey)] - │ │ ├─TableReader_45 25.00 root data:TableScan_44 - │ │ │ └─TableScan_44 25.00 cop table:nation, range:[-inf,+inf], keep order:false - │ │ └─TableReader_43 7500000.00 root data:TableScan_42 - │ │ └─TableScan_42 7500000.00 cop table:customer, range:[-inf,+inf], keep order:false - │ └─TableReader_48 3017307.69 root data:Selection_47 - │ └─Selection_47 3017307.69 cop ge(tpch.orders.o_orderdate, 1993-08-01 00:00:00.000000), lt(tpch.orders.o_orderdate, 1993-11-01) - │ └─TableScan_46 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false - └─IndexLookUp_31 73916005.00 root - ├─IndexScan_28 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [tpch.orders.o_orderkey], keep order:false - └─Selection_30 73916005.00 cop eq(tpch.lineitem.l_returnflag, "R") - └─TableScan_29 1.00 cop table:lineitem, keep order:false +Projection_24 20.00 root tpch.customer.c_custkey, tpch.customer.c_name, 9_col_0, tpch.customer.c_acctbal, tpch.nation.n_name, tpch.customer.c_address, tpch.customer.c_phone, tpch.customer.c_comment +└─TopN_27 20.00 root 9_col_0:desc, offset:0, count:20 + └─HashAgg_33 3017307.69 root group by:tpch.customer.c_acctbal, tpch.customer.c_address, tpch.customer.c_comment, tpch.customer.c_custkey, tpch.customer.c_name, tpch.customer.c_phone, tpch.nation.n_name, funcs:sum(mul(tpch.lineitem.l_extendedprice, minus(1, tpch.lineitem.l_discount))), firstrow(tpch.customer.c_custkey), firstrow(tpch.customer.c_name), firstrow(tpch.customer.c_address), firstrow(tpch.customer.c_phone), firstrow(tpch.customer.c_acctbal), firstrow(tpch.customer.c_comment), firstrow(tpch.nation.n_name) + └─IndexJoin_39 12222016.17 root inner join, inner:IndexLookUp_38, outer key:tpch.orders.o_orderkey, inner key:tpch.lineitem.l_orderkey + ├─HashRightJoin_60 3017307.69 root inner join, inner:TableReader_72, equal:[eq(tpch.nation.n_nationkey, tpch.customer.c_nationkey)] + │ ├─TableReader_72 25.00 root data:TableScan_71 + │ │ └─TableScan_71 25.00 cop table:nation, range:[-inf,+inf], keep order:false + │ └─IndexJoin_63 3017307.69 root inner join, inner:TableReader_62, outer key:tpch.orders.o_custkey, inner key:tpch.customer.c_custkey + │ ├─TableReader_62 1.00 root data:TableScan_61 + │ │ └─TableScan_61 1.00 cop table:customer, range: decided by [tpch.orders.o_custkey], keep order:false + │ └─TableReader_68 3017307.69 root data:Selection_67 + │ └─Selection_67 3017307.69 cop ge(tpch.orders.o_orderdate, 1993-08-01 00:00:00.000000), lt(tpch.orders.o_orderdate, 1993-11-01) + │ └─TableScan_66 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false + └─IndexLookUp_38 73916005.00 root + ├─IndexScan_35 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [tpch.orders.o_orderkey], keep order:false + └─Selection_37 73916005.00 cop eq(tpch.lineitem.l_returnflag, "R") + └─TableScan_36 1.00 cop table:lineitem, keep order:false /* Q11 Important Stock Identification Query This query finds the most important subset of suppliers' stock in a given nation. @@ -708,19 +710,19 @@ and n_name = 'MOZAMBIQUE' order by value desc; id count task operator info -Projection_63 1304801.67 root tpch.partsupp.ps_partkey, value -└─Sort_64 1304801.67 root value:desc - └─Selection_66 1304801.67 root gt(sel_agg_4, NULL) - └─HashAgg_69 1631002.09 root group by:tpch.partsupp.ps_partkey, funcs:sum(mul(tpch.partsupp.ps_supplycost, cast(tpch.partsupp.ps_availqty))), firstrow(tpch.partsupp.ps_partkey) - └─HashRightJoin_73 1631002.09 root inner join, inner:HashRightJoin_79, equal:[eq(tpch.supplier.s_suppkey, tpch.partsupp.ps_suppkey)] - ├─HashRightJoin_79 20000.00 root inner join, inner:TableReader_84, equal:[eq(tpch.nation.n_nationkey, tpch.supplier.s_nationkey)] - │ ├─TableReader_84 1.00 root data:Selection_83 - │ │ └─Selection_83 1.00 cop eq(tpch.nation.n_name, "MOZAMBIQUE") - │ │ └─TableScan_82 25.00 cop table:nation, range:[-inf,+inf], keep order:false - │ └─TableReader_81 500000.00 root data:TableScan_80 - │ └─TableScan_80 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false - └─TableReader_86 40000000.00 root data:TableScan_85 - └─TableScan_85 40000000.00 cop table:partsupp, range:[-inf,+inf], keep order:false +Projection_67 1304801.67 root tpch.partsupp.ps_partkey, value +└─Sort_68 1304801.67 root value:desc + └─Selection_70 1304801.67 root gt(sel_agg_4, NULL) + └─HashAgg_73 1631002.09 root group by:tpch.partsupp.ps_partkey, funcs:sum(mul(tpch.partsupp.ps_supplycost, cast(tpch.partsupp.ps_availqty))), firstrow(tpch.partsupp.ps_partkey) + └─HashLeftJoin_76 1631002.09 root inner join, inner:HashLeftJoin_84, equal:[eq(tpch.partsupp.ps_suppkey, tpch.supplier.s_suppkey)] + ├─TableReader_79 40000000.00 root data:TableScan_78 + │ └─TableScan_78 40000000.00 cop table:partsupp, range:[-inf,+inf], keep order:false + └─HashLeftJoin_84 20000.00 root inner join, inner:TableReader_90, equal:[eq(tpch.supplier.s_nationkey, tpch.nation.n_nationkey)] + ├─TableReader_87 500000.00 root data:TableScan_86 + │ └─TableScan_86 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false + └─TableReader_90 1.00 root data:Selection_89 + └─Selection_89 1.00 cop eq(tpch.nation.n_name, "MOZAMBIQUE") + └─TableScan_88 25.00 cop table:nation, range:[-inf,+inf], keep order:false /* Q12 Shipping Modes and Order Priority Query This query determines whether selecting less expensive modes of shipping is negatively affecting the critical-priority @@ -764,12 +766,12 @@ id count task operator info Sort_9 1.00 root tpch.lineitem.l_shipmode:asc └─Projection_11 1.00 root tpch.lineitem.l_shipmode, 5_col_0, 5_col_1 └─HashAgg_14 1.00 root group by:tpch.lineitem.l_shipmode, funcs:sum(case(or(eq(tpch.orders.o_orderpriority, "1-URGENT"), eq(tpch.orders.o_orderpriority, "2-HIGH")), 1, 0)), sum(case(and(ne(tpch.orders.o_orderpriority, "1-URGENT"), ne(tpch.orders.o_orderpriority, "2-HIGH")), 1, 0)), firstrow(tpch.lineitem.l_shipmode) - └─IndexJoin_18 10023369.01 root inner join, inner:TableReader_17, outer key:tpch.lineitem.l_orderkey, inner key:tpch.orders.o_orderkey - ├─TableReader_35 10023369.01 root data:Selection_34 - │ └─Selection_34 10023369.01 cop ge(tpch.lineitem.l_receiptdate, 1997-01-01 00:00:00.000000), in(tpch.lineitem.l_shipmode, "RAIL", "FOB"), lt(tpch.lineitem.l_commitdate, tpch.lineitem.l_receiptdate), lt(tpch.lineitem.l_receiptdate, 1998-01-01), lt(tpch.lineitem.l_shipdate, tpch.lineitem.l_commitdate) - │ └─TableScan_33 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false - └─TableReader_17 1.00 root data:TableScan_16 - └─TableScan_16 1.00 cop table:orders, range: decided by [tpch.lineitem.l_orderkey], keep order:false + └─IndexJoin_23 10023369.01 root inner join, inner:TableReader_22, outer key:tpch.lineitem.l_orderkey, inner key:tpch.orders.o_orderkey + ├─TableReader_22 1.00 root data:TableScan_21 + │ └─TableScan_21 1.00 cop table:orders, range: decided by [tpch.lineitem.l_orderkey], keep order:false + └─TableReader_35 10023369.01 root data:Selection_34 + └─Selection_34 10023369.01 cop ge(tpch.lineitem.l_receiptdate, 1997-01-01 00:00:00.000000), in(tpch.lineitem.l_shipmode, "RAIL", "FOB"), lt(tpch.lineitem.l_commitdate, tpch.lineitem.l_receiptdate), lt(tpch.lineitem.l_receiptdate, 1998-01-01), lt(tpch.lineitem.l_shipdate, tpch.lineitem.l_commitdate) + └─TableScan_33 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false /* Q13 Customer Distribution Query This query seeks relationships between customers and the size of their orders. @@ -920,16 +922,16 @@ p_brand, p_type, p_size; id count task operator info -Sort_13 3863988.24 root supplier_cnt:desc, tpch.part.p_brand:asc, tpch.part.p_type:asc, tpch.part.p_size:asc -└─Projection_15 3863988.24 root tpch.part.p_brand, tpch.part.p_type, tpch.part.p_size, 9_col_0 - └─HashAgg_18 3863988.24 root group by:tpch.part.p_brand, tpch.part.p_size, tpch.part.p_type, funcs:count(distinct tpch.partsupp.ps_suppkey), firstrow(tpch.part.p_brand), firstrow(tpch.part.p_type), firstrow(tpch.part.p_size) +Sort_13 14.41 root supplier_cnt:desc, tpch.part.p_brand:asc, tpch.part.p_type:asc, tpch.part.p_size:asc +└─Projection_15 14.41 root tpch.part.p_brand, tpch.part.p_type, tpch.part.p_size, 9_col_0 + └─HashAgg_18 14.41 root group by:tpch.part.p_brand, tpch.part.p_size, tpch.part.p_type, funcs:count(distinct tpch.partsupp.ps_suppkey), firstrow(tpch.part.p_brand), firstrow(tpch.part.p_type), firstrow(tpch.part.p_size) └─HashLeftJoin_23 3863988.24 root anti semi join, inner:TableReader_47, equal:[eq(tpch.partsupp.ps_suppkey, tpch.supplier.s_suppkey)] - ├─IndexJoin_27 4829985.30 root inner join, inner:IndexReader_26, outer key:tpch.part.p_partkey, inner key:tpch.partsupp.ps_partkey - │ ├─TableReader_42 1200618.43 root data:Selection_41 - │ │ └─Selection_41 1200618.43 cop in(tpch.part.p_size, 48, 19, 12, 4, 41, 7, 21, 39), ne(tpch.part.p_brand, "Brand#34"), not(like(tpch.part.p_type, "LARGE BRUSHED%", 92)) - │ │ └─TableScan_40 10000000.00 cop table:part, range:[-inf,+inf], keep order:false - │ └─IndexReader_26 1.00 root index:IndexScan_25 - │ └─IndexScan_25 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [tpch.part.p_partkey], keep order:false + ├─IndexJoin_31 4829985.30 root inner join, inner:IndexReader_30, outer key:tpch.part.p_partkey, inner key:tpch.partsupp.ps_partkey + │ ├─IndexReader_30 1.00 root index:IndexScan_29 + │ │ └─IndexScan_29 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [tpch.part.p_partkey], keep order:false + │ └─TableReader_42 1200618.43 root data:Selection_41 + │ └─Selection_41 1200618.43 cop in(tpch.part.p_size, 48, 19, 12, 4, 41, 7, 21, 39), ne(tpch.part.p_brand, "Brand#34"), not(like(tpch.part.p_type, "LARGE BRUSHED%", 92)) + │ └─TableScan_40 10000000.00 cop table:part, range:[-inf,+inf], keep order:false └─TableReader_47 400000.00 root data:Selection_46 └─Selection_46 400000.00 cop like(tpch.supplier.s_comment, "%Customer%Complaints%", 92) └─TableScan_45 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false @@ -962,20 +964,20 @@ where l_partkey = p_partkey ); id count task operator info -Projection_16 1.00 root div(11_col_0, 7.0) -└─StreamAgg_21 1.00 root funcs:sum(tpch.lineitem.l_extendedprice) - └─Projection_44 293773.83 root tpch.lineitem.l_partkey, tpch.lineitem.l_quantity, tpch.lineitem.l_extendedprice, tpch.part.p_partkey, tpch.part.p_brand, tpch.part.p_container, mul(0.2, 7_col_0) - └─HashRightJoin_46 293773.83 root inner join, inner:HashRightJoin_30, equal:[eq(tpch.part.p_partkey, tpch.lineitem.l_partkey)], other cond:lt(tpch.lineitem.l_quantity, mul(0.2, 7_col_0)) - ├─HashRightJoin_30 9736.49 root inner join, inner:TableReader_41, equal:[eq(tpch.part.p_partkey, tpch.lineitem.l_partkey)] - │ ├─TableReader_41 9736.49 root data:Selection_40 - │ │ └─Selection_40 9736.49 cop eq(tpch.part.p_brand, "Brand#44"), eq(tpch.part.p_container, "WRAP PKG") - │ │ └─TableScan_39 10000000.00 cop table:part, range:[-inf,+inf], keep order:false - │ └─HashAgg_35 9943040.00 root group by:col_3, funcs:avg(col_0, col_1), firstrow(col_2) - │ └─TableReader_36 9943040.00 root data:HashAgg_31 - │ └─HashAgg_31 9943040.00 cop group by:tpch.lineitem.l_partkey, funcs:avg(tpch.lineitem.l_quantity), firstrow(tpch.lineitem.l_partkey) - │ └─TableScan_34 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false - └─TableReader_62 300005811.00 root data:TableScan_61 - └─TableScan_61 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false +Projection_17 1.00 root div(11_col_0, 7.0) +└─StreamAgg_22 1.00 root funcs:sum(tpch.lineitem.l_extendedprice) + └─Projection_45 293773.83 root tpch.lineitem.l_partkey, tpch.lineitem.l_quantity, tpch.lineitem.l_extendedprice, tpch.part.p_partkey, tpch.part.p_brand, tpch.part.p_container, mul(0.2, 7_col_0) + └─HashLeftJoin_46 293773.83 root inner join, inner:HashRightJoin_33, equal:[eq(tpch.lineitem.l_partkey, tpch.part.p_partkey)] + ├─TableReader_49 300005811.00 root data:TableScan_48 + │ └─TableScan_48 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false + └─HashRightJoin_33 9736.49 root inner join, inner:TableReader_44, equal:[eq(tpch.part.p_partkey, tpch.lineitem.l_partkey)] + ├─TableReader_44 9736.49 root data:Selection_43 + │ └─Selection_43 9736.49 cop eq(tpch.part.p_brand, "Brand#44"), eq(tpch.part.p_container, "WRAP PKG") + │ └─TableScan_42 10000000.00 cop table:part, range:[-inf,+inf], keep order:false + └─HashAgg_38 9943040.00 root group by:col_3, funcs:avg(col_0, col_1), firstrow(col_2) + └─TableReader_39 9943040.00 root data:HashAgg_34 + └─HashAgg_34 9943040.00 cop group by:tpch.lineitem.l_partkey, funcs:avg(tpch.lineitem.l_quantity), firstrow(tpch.lineitem.l_partkey) + └─TableScan_37 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false /* Q18 Large Volume Customer Query The Large Volume Customer Query ranks customers based on their having placed a large quantity order. Large @@ -1019,24 +1021,24 @@ o_totalprice desc, o_orderdate limit 100; id count task operator info -Projection_24 100.00 root tpch.customer.c_name, tpch.customer.c_custkey, tpch.orders.o_orderkey, tpch.orders.o_orderdate, tpch.orders.o_totalprice, 14_col_0 -└─TopN_27 100.00 root tpch.orders.o_totalprice:desc, tpch.orders.o_orderdate:asc, offset:0, count:100 - └─HashAgg_33 59251097.60 root group by:tpch.customer.c_custkey, tpch.customer.c_name, tpch.orders.o_orderdate, tpch.orders.o_orderkey, tpch.orders.o_totalprice, funcs:sum(tpch.lineitem.l_quantity), firstrow(tpch.customer.c_custkey), firstrow(tpch.customer.c_name), firstrow(tpch.orders.o_orderkey), firstrow(tpch.orders.o_totalprice), firstrow(tpch.orders.o_orderdate) - └─IndexJoin_38 240004648.80 root inner join, inner:IndexLookUp_37, outer key:tpch.orders.o_orderkey, inner key:tpch.lineitem.l_orderkey - ├─HashLeftJoin_41 59251097.60 root inner join, inner:Selection_52, equal:[eq(tpch.orders.o_orderkey, tpch.lineitem.l_orderkey)] - │ ├─HashRightJoin_47 75000000.00 root inner join, inner:TableReader_51, equal:[eq(tpch.customer.c_custkey, tpch.orders.o_custkey)] - │ │ ├─TableReader_51 7500000.00 root data:TableScan_50 - │ │ │ └─TableScan_50 7500000.00 cop table:customer, range:[-inf,+inf], keep order:false - │ │ └─TableReader_49 75000000.00 root data:TableScan_48 - │ │ └─TableScan_48 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false - │ └─Selection_52 59251097.60 root gt(sel_agg_2, 314) - │ └─HashAgg_59 74063872.00 root group by:col_2, funcs:sum(col_0), firstrow(col_1) - │ └─TableReader_60 74063872.00 root data:HashAgg_53 - │ └─HashAgg_53 74063872.00 cop group by:tpch.lineitem.l_orderkey, funcs:sum(tpch.lineitem.l_quantity), firstrow(tpch.lineitem.l_orderkey) - │ └─TableScan_58 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false - └─IndexLookUp_37 1.00 root - ├─IndexScan_35 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [tpch.orders.o_orderkey], keep order:false - └─TableScan_36 1.00 cop table:lineitem, keep order:false +Projection_32 100.00 root tpch.customer.c_name, tpch.customer.c_custkey, tpch.orders.o_orderkey, tpch.orders.o_orderdate, tpch.orders.o_totalprice, 14_col_0 +└─TopN_35 100.00 root tpch.orders.o_totalprice:desc, tpch.orders.o_orderdate:asc, offset:0, count:100 + └─HashAgg_41 59251097.60 root group by:tpch.customer.c_custkey, tpch.customer.c_name, tpch.orders.o_orderdate, tpch.orders.o_orderkey, tpch.orders.o_totalprice, funcs:sum(tpch.lineitem.l_quantity), firstrow(tpch.customer.c_custkey), firstrow(tpch.customer.c_name), firstrow(tpch.orders.o_orderkey), firstrow(tpch.orders.o_totalprice), firstrow(tpch.orders.o_orderdate) + └─IndexJoin_46 240004648.80 root inner join, inner:IndexLookUp_45, outer key:tpch.orders.o_orderkey, inner key:tpch.lineitem.l_orderkey + ├─HashRightJoin_58 59251097.60 root inner join, inner:TableReader_84, equal:[eq(tpch.customer.c_custkey, tpch.orders.o_custkey)] + │ ├─TableReader_84 7500000.00 root data:TableScan_83 + │ │ └─TableScan_83 7500000.00 cop table:customer, range:[-inf,+inf], keep order:false + │ └─IndexJoin_61 59251097.60 root inner join, inner:TableReader_60, outer key:tpch.lineitem.l_orderkey, inner key:tpch.orders.o_orderkey + │ ├─TableReader_60 1.00 root data:TableScan_59 + │ │ └─TableScan_59 1.00 cop table:orders, range: decided by [tpch.lineitem.l_orderkey], keep order:false + │ └─Selection_64 59251097.60 root gt(sel_agg_2, 314) + │ └─HashAgg_71 74063872.00 root group by:col_2, funcs:sum(col_0), firstrow(col_1) + │ └─TableReader_72 74063872.00 root data:HashAgg_65 + │ └─HashAgg_65 74063872.00 cop group by:tpch.lineitem.l_orderkey, funcs:sum(tpch.lineitem.l_quantity), firstrow(tpch.lineitem.l_orderkey) + │ └─TableScan_70 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false + └─IndexLookUp_45 1.00 root + ├─IndexScan_43 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [tpch.orders.o_orderkey], keep order:false + └─TableScan_44 1.00 cop table:lineitem, keep order:false /* Q19 Discounted Revenue Query The Discounted Revenue Query reports the gross discounted revenue attributed to the sale of selected parts handled @@ -1084,7 +1086,7 @@ and l_shipinstruct = 'DELIVER IN PERSON' ); id count task operator info StreamAgg_13 1.00 root funcs:sum(mul(tpch.lineitem.l_extendedprice, minus(1, tpch.lineitem.l_discount))) -└─IndexJoin_29 6286493.79 root inner join, inner:TableReader_28, outer key:tpch.lineitem.l_partkey, inner key:tpch.part.p_partkey, other cond:or(and(and(eq(tpch.part.p_brand, "Brand#52"), in(tpch.part.p_container, "SM CASE", "SM BOX", "SM PACK", "SM PKG")), and(ge(tpch.lineitem.l_quantity, 4), and(le(tpch.lineitem.l_quantity, 14), le(tpch.part.p_size, 5)))), or(and(and(eq(tpch.part.p_brand, "Brand#11"), in(tpch.part.p_container, "MED BAG", "MED BOX", "MED PKG", "MED PACK")), and(ge(tpch.lineitem.l_quantity, 18), and(le(tpch.lineitem.l_quantity, 28), le(tpch.part.p_size, 10)))), and(and(eq(tpch.part.p_brand, "Brand#51"), in(tpch.part.p_container, "LG CASE", "LG BOX", "LG PACK", "LG PKG")), and(ge(tpch.lineitem.l_quantity, 29), and(le(tpch.lineitem.l_quantity, 39), le(tpch.part.p_size, 15)))))) +└─IndexJoin_29 6286493.79 root inner join, inner:TableReader_28, outer key:tpch.lineitem.l_partkey, inner key:tpch.part.p_partkey ├─TableReader_34 6286493.79 root data:Selection_33 │ └─Selection_33 6286493.79 cop eq(tpch.lineitem.l_shipinstruct, "DELIVER IN PERSON"), in(tpch.lineitem.l_shipmode, "AIR", "AIR REG"), or(and(ge(tpch.lineitem.l_quantity, 4), le(tpch.lineitem.l_quantity, 14)), or(and(ge(tpch.lineitem.l_quantity, 18), le(tpch.lineitem.l_quantity, 28)), and(ge(tpch.lineitem.l_quantity, 29), le(tpch.lineitem.l_quantity, 39)))) │ └─TableScan_32 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false @@ -1138,30 +1140,30 @@ and n_name = 'ALGERIA' order by s_name; id count task operator info -Sort_28 20000.00 root tpch.supplier.s_name:asc -└─Projection_30 20000.00 root tpch.supplier.s_name, tpch.supplier.s_address - └─HashRightJoin_32 20000.00 root inner join, inner:HashRightJoin_38, equal:[eq(tpch.supplier.s_suppkey, tpch.partsupp.ps_suppkey)] - ├─HashRightJoin_38 20000.00 root inner join, inner:TableReader_43, equal:[eq(tpch.nation.n_nationkey, tpch.supplier.s_nationkey)] - │ ├─TableReader_43 1.00 root data:Selection_42 - │ │ └─Selection_42 1.00 cop eq(tpch.nation.n_name, "ALGERIA") - │ │ └─TableScan_41 25.00 cop table:nation, range:[-inf,+inf], keep order:false - │ └─TableReader_40 500000.00 root data:TableScan_39 - │ └─TableScan_39 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false - └─HashAgg_46 64006.34 root group by:tpch.partsupp.ps_suppkey, funcs:firstrow(tpch.partsupp.ps_suppkey) - └─Projection_47 64006.34 root tpch.partsupp.ps_partkey, tpch.partsupp.ps_suppkey, tpch.partsupp.ps_availqty, tpch.part.p_partkey, mul(0.5, 14_col_0) - └─Selection_48 64006.34 root gt(cast(tpch.partsupp.ps_availqty), mul(0.5, 14_col_0)) - └─HashAgg_51 80007.93 root group by:tpch.partsupp.ps_partkey, tpch.partsupp.ps_suppkey, funcs:firstrow(tpch.partsupp.ps_partkey), firstrow(tpch.partsupp.ps_suppkey), firstrow(tpch.partsupp.ps_availqty), firstrow(tpch.part.p_partkey), sum(tpch.lineitem.l_quantity) - └─HashLeftJoin_54 9711455.06 root left outer join, inner:TableReader_80, equal:[eq(tpch.partsupp.ps_partkey, tpch.lineitem.l_partkey) eq(tpch.partsupp.ps_suppkey, tpch.lineitem.l_suppkey)] - ├─IndexJoin_59 321865.05 root inner join, inner:IndexLookUp_58, outer key:tpch.part.p_partkey, inner key:tpch.partsupp.ps_partkey - │ ├─TableReader_75 80007.93 root data:Selection_74 - │ │ └─Selection_74 80007.93 cop like(tpch.part.p_name, "green%", 92) - │ │ └─TableScan_73 10000000.00 cop table:part, range:[-inf,+inf], keep order:false - │ └─IndexLookUp_58 1.00 root - │ ├─IndexScan_56 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [tpch.part.p_partkey], keep order:false - │ └─TableScan_57 1.00 cop table:partsupp, keep order:false - └─TableReader_80 44189356.65 root data:Selection_79 - └─Selection_79 44189356.65 cop ge(tpch.lineitem.l_shipdate, 1993-01-01 00:00:00.000000), lt(tpch.lineitem.l_shipdate, 1994-01-01) - └─TableScan_78 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false +Sort_30 20000.00 root tpch.supplier.s_name:asc +└─Projection_32 20000.00 root tpch.supplier.s_name, tpch.supplier.s_address + └─HashLeftJoin_33 20000.00 root inner join, inner:HashLeftJoin_76, equal:[eq(tpch.partsupp.ps_suppkey, tpch.supplier.s_suppkey)] + ├─HashAgg_37 257492.04 root group by:tpch.partsupp.ps_suppkey, funcs:firstrow(tpch.partsupp.ps_suppkey) + │ └─Projection_38 257492.04 root tpch.partsupp.ps_partkey, tpch.partsupp.ps_suppkey, tpch.partsupp.ps_availqty, tpch.part.p_partkey, mul(0.5, 14_col_0) + │ └─Selection_39 257492.04 root gt(cast(tpch.partsupp.ps_availqty), mul(0.5, 14_col_0)) + │ └─HashAgg_42 321865.05 root group by:tpch.partsupp.ps_partkey, tpch.partsupp.ps_suppkey, funcs:firstrow(tpch.partsupp.ps_partkey), firstrow(tpch.partsupp.ps_suppkey), firstrow(tpch.partsupp.ps_availqty), firstrow(tpch.part.p_partkey), sum(tpch.lineitem.l_quantity) + │ └─HashLeftJoin_45 9711455.06 root left outer join, inner:TableReader_71, equal:[eq(tpch.partsupp.ps_partkey, tpch.lineitem.l_partkey) eq(tpch.partsupp.ps_suppkey, tpch.lineitem.l_suppkey)] + │ ├─IndexJoin_54 321865.05 root inner join, inner:IndexLookUp_53, outer key:tpch.part.p_partkey, inner key:tpch.partsupp.ps_partkey + │ │ ├─IndexLookUp_53 1.00 root + │ │ │ ├─IndexScan_51 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [tpch.part.p_partkey], keep order:false + │ │ │ └─TableScan_52 1.00 cop table:partsupp, keep order:false + │ │ └─TableReader_66 80007.93 root data:Selection_65 + │ │ └─Selection_65 80007.93 cop like(tpch.part.p_name, "green%", 92) + │ │ └─TableScan_64 10000000.00 cop table:part, range:[-inf,+inf], keep order:false + │ └─TableReader_71 44189356.65 root data:Selection_70 + │ └─Selection_70 44189356.65 cop ge(tpch.lineitem.l_shipdate, 1993-01-01 00:00:00.000000), lt(tpch.lineitem.l_shipdate, 1994-01-01) + │ └─TableScan_69 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false + └─HashLeftJoin_76 20000.00 root inner join, inner:TableReader_82, equal:[eq(tpch.supplier.s_nationkey, tpch.nation.n_nationkey)] + ├─TableReader_79 500000.00 root data:TableScan_78 + │ └─TableScan_78 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false + └─TableReader_82 1.00 root data:Selection_81 + └─Selection_81 1.00 cop eq(tpch.nation.n_name, "ALGERIA") + └─TableScan_80 25.00 cop table:nation, range:[-inf,+inf], keep order:false /* Q21 Suppliers Who Kept Orders Waiting Query This query identifies certain suppliers who were not able to ship required parts in a timely manner. @@ -1211,32 +1213,32 @@ numwait desc, s_name limit 100; id count task operator info -Projection_25 1.00 root tpch.supplier.s_name, 17_col_0 -└─TopN_28 1.00 root 17_col_0:desc, tpch.supplier.s_name:asc, offset:0, count:100 - └─HashAgg_34 1.00 root group by:tpch.supplier.s_name, funcs:count(1), firstrow(tpch.supplier.s_name) - └─IndexJoin_40 7828961.66 root anti semi join, inner:IndexLookUp_39, outer key:l1.l_orderkey, inner key:l3.l_orderkey, other cond:ne(l3.l_suppkey, l1.l_suppkey), ne(l3.l_suppkey, tpch.supplier.s_suppkey) - ├─IndexJoin_56 9786202.08 root semi join, inner:IndexLookUp_55, outer key:l1.l_orderkey, inner key:l2.l_orderkey, other cond:ne(l2.l_suppkey, l1.l_suppkey), ne(l2.l_suppkey, tpch.supplier.s_suppkey) - │ ├─IndexJoin_62 12232752.60 root inner join, inner:TableReader_61, outer key:l1.l_orderkey, inner key:tpch.orders.o_orderkey - │ │ ├─HashRightJoin_66 12232752.60 root inner join, inner:HashRightJoin_72, equal:[eq(tpch.supplier.s_suppkey, l1.l_suppkey)] - │ │ │ ├─HashRightJoin_72 20000.00 root inner join, inner:TableReader_77, equal:[eq(tpch.nation.n_nationkey, tpch.supplier.s_nationkey)] - │ │ │ │ ├─TableReader_77 1.00 root data:Selection_76 - │ │ │ │ │ └─Selection_76 1.00 cop eq(tpch.nation.n_name, "EGYPT") - │ │ │ │ │ └─TableScan_75 25.00 cop table:nation, range:[-inf,+inf], keep order:false - │ │ │ │ └─TableReader_74 500000.00 root data:TableScan_73 - │ │ │ │ └─TableScan_73 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false - │ │ │ └─TableReader_80 240004648.80 root data:Selection_79 - │ │ │ └─Selection_79 240004648.80 cop gt(l1.l_receiptdate, l1.l_commitdate) - │ │ │ └─TableScan_78 300005811.00 cop table:l1, range:[-inf,+inf], keep order:false - │ │ └─TableReader_61 36517371.00 root data:Selection_60 - │ │ └─Selection_60 36517371.00 cop eq(tpch.orders.o_orderstatus, "F") - │ │ └─TableScan_59 1.00 cop table:orders, range: decided by [l1.l_orderkey], keep order:false - │ └─IndexLookUp_55 1.00 root - │ ├─IndexScan_53 1.00 cop table:l2, index:L_ORDERKEY, L_LINENUMBER, range: decided by [l1.l_orderkey], keep order:false - │ └─TableScan_54 1.00 cop table:lineitem, keep order:false - └─IndexLookUp_39 240004648.80 root - ├─IndexScan_36 1.00 cop table:l3, index:L_ORDERKEY, L_LINENUMBER, range: decided by [l1.l_orderkey], keep order:false - └─Selection_38 240004648.80 cop gt(l3.l_receiptdate, l3.l_commitdate) - └─TableScan_37 1.00 cop table:lineitem, keep order:false +Projection_32 100.00 root tpch.supplier.s_name, 17_col_0 +└─TopN_35 100.00 root 17_col_0:desc, tpch.supplier.s_name:asc, offset:0, count:100 + └─HashAgg_41 12800.00 root group by:tpch.supplier.s_name, funcs:count(1), firstrow(tpch.supplier.s_name) + └─IndexJoin_47 7828961.66 root anti semi join, inner:IndexLookUp_46, outer key:l1.l_orderkey, inner key:l3.l_orderkey, other cond:ne(l3.l_suppkey, l1.l_suppkey), ne(l3.l_suppkey, tpch.supplier.s_suppkey) + ├─IndexJoin_63 9786202.08 root semi join, inner:IndexLookUp_62, outer key:l1.l_orderkey, inner key:l2.l_orderkey, other cond:ne(l2.l_suppkey, l1.l_suppkey), ne(l2.l_suppkey, tpch.supplier.s_suppkey) + │ ├─IndexJoin_69 12232752.60 root inner join, inner:TableReader_68, outer key:l1.l_orderkey, inner key:tpch.orders.o_orderkey + │ │ ├─HashRightJoin_73 12232752.60 root inner join, inner:HashLeftJoin_78, equal:[eq(tpch.supplier.s_suppkey, l1.l_suppkey)] + │ │ │ ├─HashLeftJoin_78 20000.00 root inner join, inner:TableReader_84, equal:[eq(tpch.supplier.s_nationkey, tpch.nation.n_nationkey)] + │ │ │ │ ├─TableReader_81 500000.00 root data:TableScan_80 + │ │ │ │ │ └─TableScan_80 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false + │ │ │ │ └─TableReader_84 1.00 root data:Selection_83 + │ │ │ │ └─Selection_83 1.00 cop eq(tpch.nation.n_name, "EGYPT") + │ │ │ │ └─TableScan_82 25.00 cop table:nation, range:[-inf,+inf], keep order:false + │ │ │ └─TableReader_87 240004648.80 root data:Selection_86 + │ │ │ └─Selection_86 240004648.80 cop gt(l1.l_receiptdate, l1.l_commitdate) + │ │ │ └─TableScan_85 300005811.00 cop table:l1, range:[-inf,+inf], keep order:false + │ │ └─TableReader_68 36517371.00 root data:Selection_67 + │ │ └─Selection_67 36517371.00 cop eq(tpch.orders.o_orderstatus, "F") + │ │ └─TableScan_66 1.00 cop table:orders, range: decided by [l1.l_orderkey], keep order:false + │ └─IndexLookUp_62 1.00 root + │ ├─IndexScan_60 1.00 cop table:l2, index:L_ORDERKEY, L_LINENUMBER, range: decided by [l1.l_orderkey], keep order:false + │ └─TableScan_61 1.00 cop table:lineitem, keep order:false + └─IndexLookUp_46 240004648.80 root + ├─IndexScan_43 1.00 cop table:l3, index:L_ORDERKEY, L_LINENUMBER, range: decided by [l1.l_orderkey], keep order:false + └─Selection_45 240004648.80 cop gt(l3.l_receiptdate, l3.l_commitdate) + └─TableScan_44 1.00 cop table:lineitem, keep order:false /* Q22 Global Sales Opportunity Query The Global Sales Opportunity Query identifies geographies where there are customers who may be likely to make a From 0c1cb35948a2804312ac84cfbbcb9678e6728722 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Tue, 25 Dec 2018 19:06:39 +0800 Subject: [PATCH 03/12] fix lint --- planner/core/rule_join_reorder_greedy.go | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/planner/core/rule_join_reorder_greedy.go b/planner/core/rule_join_reorder_greedy.go index bb16efddc66ee..2f35423dfd983 100644 --- a/planner/core/rule_join_reorder_greedy.go +++ b/planner/core/rule_join_reorder_greedy.go @@ -222,17 +222,16 @@ func (s *joinReOrderSolver) optimizeRecursive(ctx sessionctx.Context, p LogicalP return nil, err } return p, nil - } else { - dpSolver := &joinReorderDPSolver{ - ctx: ctx, - } - dpSolver.newJoin = dpSolver.newJoinWithConds - p, err = dpSolver.solve(curJoinGroup, expression.ScalarFuncs2Exprs(eqEdges), otherConds) - if err != nil { - return nil, err - } - return p, nil } + dpSolver := &joinReorderDPSolver{ + ctx: ctx, + } + dpSolver.newJoin = dpSolver.newJoinWithConds + p, err = dpSolver.solve(curJoinGroup, expression.ScalarFuncs2Exprs(eqEdges), otherConds) + if err != nil { + return nil, err + } + return p, nil } newChildren := make([]LogicalPlan, 0, len(p.Children())) for _, child := range p.Children() { From 868a44d59d756f1f31f4458543b3811e4ec5954d Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Wed, 26 Dec 2018 14:00:13 +0800 Subject: [PATCH 04/12] fix unit-test and integration test --- cmd/explaintest/r/explain_complex.result | 4 ++-- cmd/explaintest/r/explain_complex_stats.result | 4 ++-- cmd/explaintest/r/tpch.result | 6 +++--- planner/core/rule_join_reorder_dp.go | 16 +++++++++++----- 4 files changed, 18 insertions(+), 12 deletions(-) diff --git a/cmd/explaintest/r/explain_complex.result b/cmd/explaintest/r/explain_complex.result index 99366a5f23718..77074e6f8195d 100644 --- a/cmd/explaintest/r/explain_complex.result +++ b/cmd/explaintest/r/explain_complex.result @@ -118,7 +118,7 @@ id count task operator info Projection_13 1.00 root gad.id, test.dd.id, gad.aid, gad.cm, test.dd.dic, test.dd.ip, test.dd.t, gad.p1, gad.p2, gad.p3, gad.p4, gad.p5, gad.p6_md5, gad.p7_md5, gad.ext, gad.t └─Limit_16 1.00 root offset:0, count:2500 └─HashAgg_19 1.00 root group by:gad.aid, test.dd.dic, funcs:firstrow(gad.id), firstrow(gad.aid), firstrow(gad.cm), firstrow(gad.p1), firstrow(gad.p2), firstrow(gad.p3), firstrow(gad.p4), firstrow(gad.p5), firstrow(gad.p6_md5), firstrow(gad.p7_md5), firstrow(gad.ext), firstrow(gad.t), firstrow(test.dd.id), firstrow(test.dd.dic), firstrow(test.dd.ip), firstrow(test.dd.t) - └─IndexJoin_24 0.00 root inner join, inner:IndexLookUp_23, outer key:gad.aid, inner key:test.dd.aid, other cond:eq(gad.ip, test.dd.ip) + └─IndexJoin_24 0.00 root inner join, inner:IndexLookUp_23, outer key:gad.aid, inner key:test.dd.aid, other cond:eq(gad.ip, test.dd.ip), gt(test.dd.t, gad.t) ├─IndexLookUp_33 3.33 root │ ├─IndexScan_30 3333.33 cop table:gad, index:t, range:(1478143908,+inf], keep order:false, stats:pseudo │ └─Selection_32 3.33 cop eq(gad.pt, "android") @@ -131,7 +131,7 @@ explain select gad.id as gid,sdk.id as sid,gad.aid as aid,gad.cm as cm,sdk.dic a id count task operator info Projection_10 0.00 root gad.id, sdk.id, gad.aid, gad.cm, sdk.dic, sdk.ip, sdk.t, gad.p1, gad.p2, gad.p3, gad.p4, gad.p5, gad.p6_md5, gad.p7_md5, gad.ext └─Limit_13 0.00 root offset:0, count:3000 - └─IndexJoin_18 0.00 root inner join, inner:IndexLookUp_17, outer key:gad.aid, inner key:sdk.aid, other cond:eq(gad.dic, sdk.mac) + └─IndexJoin_18 0.00 root inner join, inner:IndexLookUp_17, outer key:gad.aid, inner key:sdk.aid, other cond:eq(gad.dic, sdk.mac), lt(gad.t, sdk.t) ├─IndexLookUp_27 0.00 root │ ├─IndexScan_24 3333.33 cop table:gad, index:t, range:(1477971479,+inf], keep order:false, stats:pseudo │ └─Selection_26 0.00 cop eq(gad.bm, 0), eq(gad.dit, "mac"), eq(gad.pt, "ios") diff --git a/cmd/explaintest/r/explain_complex_stats.result b/cmd/explaintest/r/explain_complex_stats.result index a2f3ff20c1af5..1569a2d3a850c 100644 --- a/cmd/explaintest/r/explain_complex_stats.result +++ b/cmd/explaintest/r/explain_complex_stats.result @@ -128,7 +128,7 @@ id count task operator info Projection_13 424.00 root gad.id, test.dd.id, gad.aid, gad.cm, test.dd.dic, test.dd.ip, test.dd.t, gad.p1, gad.p2, gad.p3, gad.p4, gad.p5, gad.p6_md5, gad.p7_md5, gad.ext, gad.t └─Limit_16 424.00 root offset:0, count:2500 └─HashAgg_19 424.00 root group by:gad.aid, test.dd.dic, funcs:firstrow(gad.id), firstrow(gad.aid), firstrow(gad.cm), firstrow(gad.p1), firstrow(gad.p2), firstrow(gad.p3), firstrow(gad.p4), firstrow(gad.p5), firstrow(gad.p6_md5), firstrow(gad.p7_md5), firstrow(gad.ext), firstrow(gad.t), firstrow(test.dd.id), firstrow(test.dd.dic), firstrow(test.dd.ip), firstrow(test.dd.t) - └─IndexJoin_24 424.00 root inner join, inner:IndexLookUp_23, outer key:gad.aid, inner key:test.dd.aid, other cond:eq(gad.ip, test.dd.ip) + └─IndexJoin_24 424.00 root inner join, inner:IndexLookUp_23, outer key:gad.aid, inner key:test.dd.aid, other cond:eq(gad.ip, test.dd.ip), gt(test.dd.t, gad.t) ├─TableReader_29 424.00 root data:Selection_28 │ └─Selection_28 424.00 cop eq(gad.bm, 0), eq(gad.pt, "android"), gt(gad.t, 1478143908) │ └─TableScan_27 1999.00 cop table:gad, range:[0,+inf], keep order:false @@ -140,7 +140,7 @@ explain select gad.id as gid,sdk.id as sid,gad.aid as aid,gad.cm as cm,sdk.dic a id count task operator info Projection_10 170.34 root gad.id, sdk.id, gad.aid, gad.cm, sdk.dic, sdk.ip, sdk.t, gad.p1, gad.p2, gad.p3, gad.p4, gad.p5, gad.p6_md5, gad.p7_md5, gad.ext └─Limit_13 170.34 root offset:0, count:3000 - └─IndexJoin_18 170.34 root inner join, inner:IndexLookUp_17, outer key:gad.aid, inner key:sdk.aid, other cond:eq(gad.dic, sdk.mac) + └─IndexJoin_18 170.34 root inner join, inner:IndexLookUp_17, outer key:gad.aid, inner key:sdk.aid, other cond:eq(gad.dic, sdk.mac), lt(gad.t, sdk.t) ├─TableReader_23 170.34 root data:Selection_22 │ └─Selection_22 170.34 cop eq(gad.bm, 0), eq(gad.dit, "mac"), eq(gad.pt, "ios"), gt(gad.t, 1477971479) │ └─TableScan_21 1999.00 cop table:gad, range:[0,+inf], keep order:false diff --git a/cmd/explaintest/r/tpch.result b/cmd/explaintest/r/tpch.result index fd41424ee84be..e06783071377e 100644 --- a/cmd/explaintest/r/tpch.result +++ b/cmd/explaintest/r/tpch.result @@ -447,7 +447,7 @@ Sort_52 768.91 root shipping.supp_nation:asc, shipping.cust_nation:asc, shipping └─Projection_54 768.91 root shipping.supp_nation, shipping.cust_nation, shipping.l_year, 14_col_0 └─HashAgg_57 768.91 root group by:shipping.cust_nation, shipping.l_year, shipping.supp_nation, funcs:sum(shipping.volume), firstrow(shipping.supp_nation), firstrow(shipping.cust_nation), firstrow(shipping.l_year) └─Projection_58 9818555.15 root n1.n_name, n2.n_name, extract("YEAR", tpch.lineitem.l_shipdate), mul(tpch.lineitem.l_extendedprice, minus(1, tpch.lineitem.l_discount)) - └─HashLeftJoin_60 9818555.15 root inner join, inner:HashLeftJoin_78, equal:[eq(tpch.lineitem.l_orderkey, tpch.orders.o_orderkey)] + └─HashLeftJoin_60 9818555.15 root inner join, inner:HashLeftJoin_78, equal:[eq(tpch.lineitem.l_orderkey, tpch.orders.o_orderkey)], other cond:or(and(eq(n1.n_name, "JAPAN"), eq(n2.n_name, "INDIA")), and(eq(n1.n_name, "INDIA"), eq(n2.n_name, "JAPAN"))) ├─HashRightJoin_63 24465505.20 root inner join, inner:HashLeftJoin_68, equal:[eq(tpch.supplier.s_suppkey, tpch.lineitem.l_suppkey)] │ ├─HashLeftJoin_68 40000.00 root inner join, inner:TableReader_74, equal:[eq(tpch.supplier.s_nationkey, n1.n_nationkey)] │ │ ├─TableReader_71 500000.00 root data:TableScan_70 @@ -967,7 +967,7 @@ id count task operator info Projection_17 1.00 root div(11_col_0, 7.0) └─StreamAgg_22 1.00 root funcs:sum(tpch.lineitem.l_extendedprice) └─Projection_45 293773.83 root tpch.lineitem.l_partkey, tpch.lineitem.l_quantity, tpch.lineitem.l_extendedprice, tpch.part.p_partkey, tpch.part.p_brand, tpch.part.p_container, mul(0.2, 7_col_0) - └─HashLeftJoin_46 293773.83 root inner join, inner:HashRightJoin_33, equal:[eq(tpch.lineitem.l_partkey, tpch.part.p_partkey)] + └─HashLeftJoin_46 293773.83 root inner join, inner:HashRightJoin_33, equal:[eq(tpch.lineitem.l_partkey, tpch.part.p_partkey)], other cond:lt(tpch.lineitem.l_quantity, mul(0.2, 7_col_0)) ├─TableReader_49 300005811.00 root data:TableScan_48 │ └─TableScan_48 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false └─HashRightJoin_33 9736.49 root inner join, inner:TableReader_44, equal:[eq(tpch.part.p_partkey, tpch.lineitem.l_partkey)] @@ -1086,7 +1086,7 @@ and l_shipinstruct = 'DELIVER IN PERSON' ); id count task operator info StreamAgg_13 1.00 root funcs:sum(mul(tpch.lineitem.l_extendedprice, minus(1, tpch.lineitem.l_discount))) -└─IndexJoin_29 6286493.79 root inner join, inner:TableReader_28, outer key:tpch.lineitem.l_partkey, inner key:tpch.part.p_partkey +└─IndexJoin_29 6286493.79 root inner join, inner:TableReader_28, outer key:tpch.lineitem.l_partkey, inner key:tpch.part.p_partkey, other cond:or(and(and(eq(tpch.part.p_brand, "Brand#52"), in(tpch.part.p_container, "SM CASE", "SM BOX", "SM PACK", "SM PKG")), and(ge(tpch.lineitem.l_quantity, 4), and(le(tpch.lineitem.l_quantity, 14), le(tpch.part.p_size, 5)))), or(and(and(eq(tpch.part.p_brand, "Brand#11"), in(tpch.part.p_container, "MED BAG", "MED BOX", "MED PKG", "MED PACK")), and(ge(tpch.lineitem.l_quantity, 18), and(le(tpch.lineitem.l_quantity, 28), le(tpch.part.p_size, 10)))), and(and(eq(tpch.part.p_brand, "Brand#51"), in(tpch.part.p_container, "LG CASE", "LG BOX", "LG PACK", "LG PKG")), and(ge(tpch.lineitem.l_quantity, 29), and(le(tpch.lineitem.l_quantity, 39), le(tpch.part.p_size, 15)))))) ├─TableReader_34 6286493.79 root data:Selection_33 │ └─Selection_33 6286493.79 cop eq(tpch.lineitem.l_shipinstruct, "DELIVER IN PERSON"), in(tpch.lineitem.l_shipmode, "AIR", "AIR REG"), or(and(ge(tpch.lineitem.l_quantity, 4), le(tpch.lineitem.l_quantity, 14)), or(and(ge(tpch.lineitem.l_quantity, 18), le(tpch.lineitem.l_quantity, 28)), and(ge(tpch.lineitem.l_quantity, 29), le(tpch.lineitem.l_quantity, 39)))) │ └─TableScan_32 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false diff --git a/planner/core/rule_join_reorder_dp.go b/planner/core/rule_join_reorder_dp.go index a47e38877a112..7403aa9b093a1 100644 --- a/planner/core/rule_join_reorder_dp.go +++ b/planner/core/rule_join_reorder_dp.go @@ -67,16 +67,19 @@ func (s *joinReorderDPSolver) solve(joinGroup []LogicalPlan, eqConds, otherConds for _, cond := range otherConds { cols := expression.ExtractColumns(cond) mask := uint(0) + ids := make([]int, 0, len(cols)) for _, col := range cols { idx, err := findNodeIndexInGroup(joinGroup, col) if err != nil { return nil, err } + ids = append(ids, idx) mask |= 1 << uint(idx) } totalNonEqEdges = append(totalNonEqEdges, joinGroupNonEqEdge{ - idMask: mask, - expr: cond, + nodeIDs: ids, + idMask: mask, + expr: cond, }) } visited := make([]bool, len(joinGroup)) @@ -90,17 +93,17 @@ func (s *joinReorderDPSolver) solve(joinGroup []LogicalPlan, eqConds, otherConds visitID2NodeID := s.bfsGraph(i, visited, adjacents, nodeID2VisitID) nodeIDMask := uint(0) for _, nodeID := range visitID2NodeID { - nodeIDMask |= uint(nodeID) + nodeIDMask |= 1 << uint(nodeID) } var subNonEqEdges []joinGroupNonEqEdge for i := len(totalNonEqEdges) - 1; i >= 0; i-- { // If this edge is not the subset of the current sub graph. - if totalNonEqEdges[i].idMask&nodeIDMask != nodeIDMask { + if totalNonEqEdges[i].idMask&nodeIDMask != totalNonEqEdges[i].idMask { continue } newMask := uint(0) for _, nodeID := range totalNonEqEdges[i].nodeIDs { - newMask |= uint(nodeID) + newMask |= 1 << uint(nodeID2VisitID[nodeID]) } totalNonEqEdges[i].idMask = newMask subNonEqEdges = append(subNonEqEdges, totalNonEqEdges[i]) @@ -240,6 +243,9 @@ func (s *joinReorderDPSolver) makeBushyJoin(cartesianJoinGroup []LogicalPlan, ot resultJoinGroup = append(resultJoinGroup, cartesianJoinGroup[i]) break } + // TODO:Since the other condition may involve no less than two tables, e.g. t1.a = t2.b+t3.c. + // So We'll need a extra stage to deal with it. + // Currently, we just add it when building cartesianJoinGroup. mergedSchema := expression.MergeSchema(cartesianJoinGroup[i].Schema(), cartesianJoinGroup[i+1].Schema()) var usedOtherConds []expression.Expression for i := len(otherConds) - 1; i >= 0; i-- { From dd7cd1aa1373b5927a3777256d7d94a861280eed Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Wed, 2 Jan 2019 19:38:54 +0800 Subject: [PATCH 05/12] address comments --- planner/core/expression_rewriter.go | 2 +- planner/core/logical_plan_builder.go | 2 +- planner/core/logical_plan_test.go | 2 +- planner/core/optimizer.go | 2 +- planner/core/rule_join_reorder_greedy.go | 2 +- sessionctx/variable/session.go | 47 ++++++++++++++---------- sessionctx/variable/sysvar.go | 1 + sessionctx/variable/tidb_vars.go | 5 +++ sessionctx/variable/varsutil.go | 2 +- sessionctx/variable/varsutil_test.go | 9 +++++ 10 files changed, 48 insertions(+), 26 deletions(-) diff --git a/planner/core/expression_rewriter.go b/planner/core/expression_rewriter.go index 2a4af05983ace..0fa0fdc466329 100644 --- a/planner/core/expression_rewriter.go +++ b/planner/core/expression_rewriter.go @@ -642,7 +642,7 @@ func (er *expressionRewriter) handleInSubquery(v *ast.PatternInExpr) (ast.Node, // We need to try to eliminate the agg and the projection produced by this operation. er.b.optFlag |= flagEliminateAgg er.b.optFlag |= flagEliminateProjection - er.b.optFlag |= flagJoinReOrderGreedy + er.b.optFlag |= flagJoinReOrder // Build distinct for the inner query. agg := er.b.buildDistinct(np, np.Schema().Len()) for _, col := range agg.schema.Columns { diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 72aa76dddb3e4..364f1b1486822 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -318,7 +318,7 @@ func (b *PlanBuilder) buildJoin(joinNode *ast.Join) (LogicalPlan, error) { joinPlan.JoinType = RightOuterJoin resetNotNullFlag(joinPlan.schema, 0, leftPlan.Schema().Len()) default: - b.optFlag = b.optFlag | flagJoinReOrderGreedy + b.optFlag = b.optFlag | flagJoinReOrder joinPlan.JoinType = InnerJoin } diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index 58b15745826be..7baf751ccaa70 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -797,7 +797,7 @@ func (s *testPlanSuite) TestJoinReOrder(c *C) { p, err := BuildLogicalPlan(s.ctx, stmt, s.is) c.Assert(err, IsNil) - p, err = logicalOptimize(flagPredicatePushDown|flagJoinReOrderGreedy, p.(LogicalPlan)) + p, err = logicalOptimize(flagPredicatePushDown|flagJoinReOrder, p.(LogicalPlan)) c.Assert(err, IsNil) c.Assert(ToString(p), Equals, tt.best, Commentf("for %s", tt.sql)) } diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index 0b94a17f9b6dd..d669a1631c6b1 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -43,7 +43,7 @@ const ( flagPartitionProcessor flagPushDownAgg flagPushDownTopN - flagJoinReOrderGreedy + flagJoinReOrder ) var optRuleList = []logicalOptRule{ diff --git a/planner/core/rule_join_reorder_greedy.go b/planner/core/rule_join_reorder_greedy.go index 6e36b3ffbe005..7744fb0323591 100644 --- a/planner/core/rule_join_reorder_greedy.go +++ b/planner/core/rule_join_reorder_greedy.go @@ -210,7 +210,7 @@ func (s *joinReOrderSolver) optimizeRecursive(ctx sessionctx.Context, p LogicalP return nil, err } } - if len(curJoinGroup) > 10 { + if len(curJoinGroup) > ctx.GetSessionVars().TiDBOptJoinOrderAlgoThreshold { greedySolver := &joinReorderGreedySingleGroupSolver{ ctx: ctx, curJoinGroup: curJoinGroup, diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 86fd36b5837d2..4a11a2739bfd0 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -332,31 +332,36 @@ type SessionVars struct { // CommandValue indicates which command current session is doing. CommandValue uint32 + + // TIDBOptJoinOrderAlgoThreshold defines the threshold less than which + // we'll choose a rather time consuming algorithm to calculate the join order. + TiDBOptJoinOrderAlgoThreshold int } // NewSessionVars creates a session vars object. func NewSessionVars() *SessionVars { vars := &SessionVars{ - Users: make(map[string]string), - systems: make(map[string]string), - PreparedStmts: make(map[uint32]*ast.Prepared), - PreparedStmtNameToID: make(map[string]uint32), - PreparedParams: make([]types.Datum, 0, 10), - TxnCtx: &TransactionContext{}, - KVVars: kv.NewVariables(), - RetryInfo: &RetryInfo{}, - StrictSQLMode: true, - Status: mysql.ServerStatusAutocommit, - StmtCtx: new(stmtctx.StatementContext), - AllowAggPushDown: false, - OptimizerSelectivityLevel: DefTiDBOptimizerSelectivityLevel, - RetryLimit: DefTiDBRetryLimit, - DisableTxnAutoRetry: DefTiDBDisableTxnAutoRetry, - DDLReorgPriority: kv.PriorityLow, - AllowInSubqToJoinAndAgg: DefOptInSubqToJoinAndAgg, - EnableRadixJoin: false, - L2CacheSize: cpuid.CPU.Cache.L2, - CommandValue: uint32(mysql.ComSleep), + Users: make(map[string]string), + systems: make(map[string]string), + PreparedStmts: make(map[uint32]*ast.Prepared), + PreparedStmtNameToID: make(map[string]uint32), + PreparedParams: make([]types.Datum, 0, 10), + TxnCtx: &TransactionContext{}, + KVVars: kv.NewVariables(), + RetryInfo: &RetryInfo{}, + StrictSQLMode: true, + Status: mysql.ServerStatusAutocommit, + StmtCtx: new(stmtctx.StatementContext), + AllowAggPushDown: false, + OptimizerSelectivityLevel: DefTiDBOptimizerSelectivityLevel, + RetryLimit: DefTiDBRetryLimit, + DisableTxnAutoRetry: DefTiDBDisableTxnAutoRetry, + DDLReorgPriority: kv.PriorityLow, + AllowInSubqToJoinAndAgg: DefOptInSubqToJoinAndAgg, + EnableRadixJoin: false, + L2CacheSize: cpuid.CPU.Cache.L2, + CommandValue: uint32(mysql.ComSleep), + TiDBOptJoinOrderAlgoThreshold: DefTiDBOptJoinOrderAlgoThreshold, } vars.Concurrency = Concurrency{ IndexLookupConcurrency: DefIndexLookupConcurrency, @@ -690,6 +695,8 @@ func (s *SessionVars) SetSystemVar(name string, val string) error { s.EnableRadixJoin = TiDBOptOn(val) case TiDBEnableWindowFunction: s.EnableWindowFunction = TiDBOptOn(val) + case TiDBOptJoinOrderAlgoThreshold: + s.TiDBOptJoinOrderAlgoThreshold = tidbOptPositiveInt32(val, DefTiDBOptJoinOrderAlgoThreshold) } s.systems[name] = val return nil diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 1e7437931ae78..0c1cd67fe844f 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -676,6 +676,7 @@ var defaultSysVars = []*SysVar{ {ScopeSession, TiDBDDLReorgPriority, "PRIORITY_LOW"}, {ScopeSession, TiDBForcePriority, mysql.Priority2Str[DefTiDBForcePriority]}, {ScopeSession, TiDBEnableRadixJoin, boolToIntStr(DefTiDBUseRadixJoin)}, + {ScopeGlobal | ScopeSession, TiDBOptJoinOrderAlgoThreshold, strconv.Itoa(DefTiDBOptJoinOrderAlgoThreshold)}, } // SynonymsSysVariables is synonyms of system variables. diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 4b04b48e1707d..9e9ca1d384f1c 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -225,6 +225,10 @@ const ( // tidb_enable_window_function is used to control whether to enable the window function. TiDBEnableWindowFunction = "tidb_enable_window_function" + + // TIDBOptJoinOrderAlgoThreshold defines the threshold less than which + // we'll choose a rather time consuming algorithm to calculate the join order. + TiDBOptJoinOrderAlgoThreshold = "tidb_opt_join_order_algo_threshold" ) // Default TiDB system variable values. @@ -274,6 +278,7 @@ const ( DefTiDBForcePriority = mysql.NoPriority DefTiDBUseRadixJoin = false DefEnableWindowFunction = false + DefTiDBOptJoinOrderAlgoThreshold = 10 ) // Process global variables. diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index f25df3a3938d5..d70759c1b1764 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -356,7 +356,7 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string) (string, TiDBDistSQLScanConcurrency, TiDBIndexSerialScanConcurrency, TiDBDDLReorgWorkerCount, TiDBBackoffLockFast, TiDBMaxChunkSize, - TiDBDMLBatchSize, TiDBOptimizerSelectivityLevel: + TiDBDMLBatchSize, TiDBOptimizerSelectivityLevel, TiDBOptJoinOrderAlgoThreshold: v, err := strconv.Atoi(value) if err != nil { return value, ErrWrongTypeForVar.GenWithStackByArgs(name) diff --git a/sessionctx/variable/varsutil_test.go b/sessionctx/variable/varsutil_test.go index 8fcf3c4d15f92..82735c6c2898e 100644 --- a/sessionctx/variable/varsutil_test.go +++ b/sessionctx/variable/varsutil_test.go @@ -79,6 +79,7 @@ func (s *testVarsutilSuite) TestNewSessionVars(c *C) { c.Assert(vars.MemQuotaNestedLoopApply, Equals, int64(DefTiDBMemQuotaNestedLoopApply)) c.Assert(vars.EnableRadixJoin, Equals, DefTiDBUseRadixJoin) c.Assert(vars.AllowWriteRowID, Equals, DefOptWriteRowID) + c.Assert(vars.TiDBOptJoinOrderAlgoThreshold, Equals, DefTiDBOptJoinOrderAlgoThreshold) assertFieldsGreaterThanZero(c, reflect.ValueOf(vars.Concurrency)) assertFieldsGreaterThanZero(c, reflect.ValueOf(vars.MemQuota)) @@ -237,4 +238,12 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { c.Assert(err, IsNil) c.Assert(val, Equals, "on") c.Assert(v.EnableTablePartition, Equals, "on") + + c.Assert(v.TiDBOptJoinOrderAlgoThreshold, Equals, DefTiDBOptJoinOrderAlgoThreshold) + err = SetSessionSystemVar(v, TiDBOptJoinOrderAlgoThreshold, types.NewIntDatum(5)) + c.Assert(err, IsNil) + val, err = GetSessionSystemVar(v, TiDBOptJoinOrderAlgoThreshold) + c.Assert(err, IsNil) + c.Assert(val, Equals, "5") + c.Assert(v.TiDBOptJoinOrderAlgoThreshold, Equals, 5) } From db4039f698b01ea9c73dd693556f971091fc1aca Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Mon, 7 Jan 2019 13:09:18 +0800 Subject: [PATCH 06/12] address commnets --- planner/core/rule_join_reorder_dp.go | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/planner/core/rule_join_reorder_dp.go b/planner/core/rule_join_reorder_dp.go index 27849dbe38eab..58a48b59d23c9 100644 --- a/planner/core/rule_join_reorder_dp.go +++ b/planner/core/rule_join_reorder_dp.go @@ -189,17 +189,15 @@ func (s *joinReorderDPSolver) dpGraph(newPos2OldPos, oldPos2NewPos []int, joinGr } func (s *joinReorderDPSolver) nodesAreConnected(leftMask, rightMask uint, oldPos2NewPos []int, - totalEdges []joinGroupEqEdge, totalNonEqEdges []joinGroupNonEqEdge) ([]joinGroupEqEdge, []expression.Expression) { + totalEqEdges []joinGroupEqEdge, totalNonEqEdges []joinGroupNonEqEdge) ([]joinGroupEqEdge, []expression.Expression) { var ( usedEqEdges []joinGroupEqEdge otherConds []expression.Expression ) - for _, edge := range totalEdges { + for _, edge := range totalEqEdges { lIdx := uint(oldPos2NewPos[edge.nodeIDs[0]]) rIdx := uint(oldPos2NewPos[edge.nodeIDs[1]]) - if (leftMask&(1< 0 && (rightMask&(1< 0 { - usedEqEdges = append(usedEqEdges, edge) - } else if (leftMask&(1< 0 && (rightMask&(1< 0 { + if ((leftMask&(1< 0 && (rightMask&(1< 0) || ((leftMask&(1< 0 && (rightMask&(1< 0) { usedEqEdges = append(usedEqEdges, edge) } } @@ -249,8 +247,7 @@ func (s *joinReorderDPSolver) makeBushyJoin(cartesianJoinGroup []LogicalPlan, ot mergedSchema := expression.MergeSchema(cartesianJoinGroup[i].Schema(), cartesianJoinGroup[i+1].Schema()) var usedOtherConds []expression.Expression for i := len(otherConds) - 1; i >= 0; i-- { - cols := expression.ExtractColumns(otherConds[i]) - if mergedSchema.ColumnsIndices(cols) != nil { + if expression.ExprFromSchema(otherConds[i], mergedSchema) { usedOtherConds = append(usedOtherConds, otherConds[i]) otherConds = append(otherConds[:i], otherConds[i+1:]...) } From 4fef276db13bad72b9e81326d637a744f5926a7e Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Mon, 4 Mar 2019 16:00:29 +0800 Subject: [PATCH 07/12] refactor code and disable dp by default --- cmd/explaintest/r/explain_complex.result | 18 +- cmd/explaintest/r/explain_easy.result | 15 +- cmd/explaintest/r/tpch.result | 561 +++++++++++----------- planner/core/rule_join_reorder.go | 19 +- planner/core/rule_join_reorder_dp.go | 52 +- planner/core/rule_join_reorder_dp_test.go | 37 +- planner/core/rule_join_reorder_greedy.go | 14 +- sessionctx/variable/tidb_vars.go | 2 +- 8 files changed, 366 insertions(+), 352 deletions(-) diff --git a/cmd/explaintest/r/explain_complex.result b/cmd/explaintest/r/explain_complex.result index 2e3efce496b46..66d38378b0f33 100644 --- a/cmd/explaintest/r/explain_complex.result +++ b/cmd/explaintest/r/explain_complex.result @@ -118,15 +118,15 @@ id count task operator info Projection_13 1.00 root gad.id, test.dd.id, gad.aid, gad.cm, test.dd.dic, test.dd.ip, test.dd.t, gad.p1, gad.p2, gad.p3, gad.p4, gad.p5, gad.p6_md5, gad.p7_md5, gad.ext, gad.t └─Limit_16 1.00 root offset:0, count:2500 └─HashAgg_19 1.00 root group by:gad.aid, test.dd.dic, funcs:firstrow(gad.id), firstrow(gad.aid), firstrow(gad.cm), firstrow(gad.p1), firstrow(gad.p2), firstrow(gad.p3), firstrow(gad.p4), firstrow(gad.p5), firstrow(gad.p6_md5), firstrow(gad.p7_md5), firstrow(gad.ext), firstrow(gad.t), firstrow(test.dd.id), firstrow(test.dd.dic), firstrow(test.dd.ip), firstrow(test.dd.t) - └─IndexJoin_24 0.00 root inner join, inner:IndexLookUp_23, outer key:gad.aid, inner key:test.dd.aid, other cond:eq(gad.ip, test.dd.ip), gt(test.dd.t, gad.t) - ├─IndexLookUp_33 3.33 root - │ ├─IndexScan_30 3333.33 cop table:gad, index:t, range:(1478143908,+inf], keep order:false, stats:pseudo - │ └─Selection_32 3.33 cop eq(gad.pt, "android"), not(isnull(gad.ip)) - │ └─TableScan_31 3333.33 cop table:st, keep order:false, stats:pseudo - └─IndexLookUp_23 0.00 root - ├─IndexScan_20 10.00 cop table:dd, index:aid, dic, range: decided by [gad.aid gad.ip], keep order:false, stats:pseudo - └─Selection_22 0.00 cop eq(test.dd.bm, 0), eq(test.dd.pt, "android"), gt(test.dd.t, 1478143908), not(isnull(test.dd.ip)), not(isnull(test.dd.t)) - └─TableScan_21 10.00 cop table:dd, keep order:false, stats:pseudo + └─IndexJoin_24 0.00 root inner join, inner:IndexLookUp_23, outer key:gad.aid, inner key:test.dd.aid, other cond:eq(test.dd.ip, gad.ip), gt(test.dd.t, gad.t) + ├─IndexLookUp_23 0.00 root + │ ├─IndexScan_20 10.00 cop table:dd, index:aid, dic, range: decided by [gad.aid gad.ip], keep order:false, stats:pseudo + │ └─Selection_22 0.00 cop eq(test.dd.bm, 0), eq(test.dd.pt, "android"), gt(test.dd.t, 1478143908), not(isnull(test.dd.ip)), not(isnull(test.dd.t)) + │ └─TableScan_21 10.00 cop table:dd, keep order:false, stats:pseudo + └─IndexLookUp_33 3.33 root + ├─IndexScan_30 3333.33 cop table:gad, index:t, range:(1478143908,+inf], keep order:false, stats:pseudo + └─Selection_32 3.33 cop eq(gad.pt, "android"), not(isnull(gad.ip)) + └─TableScan_31 3333.33 cop table:st, keep order:false, stats:pseudo explain select gad.id as gid,sdk.id as sid,gad.aid as aid,gad.cm as cm,sdk.dic as dic,sdk.ip as ip, sdk.t as t, gad.p1 as p1, gad.p2 as p2, gad.p3 as p3, gad.p4 as p4, gad.p5 as p5, gad.p6_md5 as p6, gad.p7_md5 as p7, gad.ext as ext from st gad join dd sdk on gad.aid = sdk.aid and gad.dic = sdk.mac and gad.t < sdk.t where gad.t > 1477971479 and gad.bm = 0 and gad.pt = 'ios' and gad.dit = 'mac' and sdk.t > 1477971479 and sdk.bm = 0 and sdk.pt = 'ios' limit 3000; id count task operator info Projection_10 0.00 root gad.id, sdk.id, gad.aid, gad.cm, sdk.dic, sdk.ip, sdk.t, gad.p1, gad.p2, gad.p3, gad.p4, gad.p5, gad.p6_md5, gad.p7_md5, gad.ext diff --git a/cmd/explaintest/r/explain_easy.result b/cmd/explaintest/r/explain_easy.result index bdceaf6a0d992..d313e457a670d 100644 --- a/cmd/explaintest/r/explain_easy.result +++ b/cmd/explaintest/r/explain_easy.result @@ -365,13 +365,14 @@ id count task operator info TableDual_5 0.00 root rows:0 explain select * from t t1 join t t2 where t1.b = t2.b and t2.b is null; id count task operator info -HashLeftJoin_8 0.00 root inner join, inner:TableReader_15, equal:[eq(t1.b, t2.b)] -├─TableReader_12 9990.00 root data:Selection_11 -│ └─Selection_11 9990.00 cop not(isnull(t1.b)) -│ └─TableScan_10 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo -└─TableReader_15 0.00 root data:Selection_14 - └─Selection_14 0.00 cop isnull(t2.b), not(isnull(t2.b)) - └─TableScan_13 10000.00 cop table:t2, range:[-inf,+inf], keep order:false, stats:pseudo +Projection_7 0.00 root t1.a, t1.b, t2.a, t2.b +└─HashRightJoin_9 0.00 root inner join, inner:TableReader_12, equal:[eq(t2.b, t1.b)] + ├─TableReader_12 0.00 root data:Selection_11 + │ └─Selection_11 0.00 cop isnull(t2.b), not(isnull(t2.b)) + │ └─TableScan_10 10000.00 cop table:t2, range:[-inf,+inf], keep order:false, stats:pseudo + └─TableReader_15 9990.00 root data:Selection_14 + └─Selection_14 9990.00 cop not(isnull(t1.b)) + └─TableScan_13 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo explain select * from t t1 where not exists (select * from t t2 where t1.b = t2.b); id count task operator info HashLeftJoin_9 8000.00 root anti semi join, inner:TableReader_13, equal:[eq(t1.b, t2.b)] diff --git a/cmd/explaintest/r/tpch.result b/cmd/explaintest/r/tpch.result index 8482e754f5d74..8bdb543a3814a 100644 --- a/cmd/explaintest/r/tpch.result +++ b/cmd/explaintest/r/tpch.result @@ -182,39 +182,38 @@ s_name, p_partkey limit 100; id count task operator info -Projection_86 100.00 root tpch.supplier.s_acctbal, tpch.supplier.s_name, tpch.nation.n_name, tpch.part.p_partkey, tpch.part.p_mfgr, tpch.supplier.s_address, tpch.supplier.s_phone, tpch.supplier.s_comment -└─TopN_89 100.00 root tpch.supplier.s_acctbal:desc, tpch.nation.n_name:asc, tpch.supplier.s_name:asc, tpch.part.p_partkey:asc, offset:0, count:100 - └─HashLeftJoin_93 127533.58 root inner join, inner:HashLeftJoin_119, equal:[eq(tpch.partsupp.ps_partkey, tpch.part.p_partkey) eq(min(ps_supplycost), tpch.partsupp.ps_supplycost)] - ├─HashAgg_97 8155010.44 root group by:tpch.partsupp.ps_partkey, funcs:min(tpch.partsupp.ps_supplycost), firstrow(tpch.partsupp.ps_partkey) - │ └─HashLeftJoin_100 8155010.44 root inner join, inner:HashLeftJoin_104, equal:[eq(tpch.partsupp.ps_suppkey, tpch.supplier.s_suppkey)] - │ ├─TableReader_103 40000000.00 root data:TableScan_102 - │ │ └─TableScan_102 40000000.00 cop table:partsupp, range:[-inf,+inf], keep order:false - │ └─HashLeftJoin_104 100000.00 root inner join, inner:HashLeftJoin_112, equal:[eq(tpch.supplier.s_nationkey, tpch.nation.n_nationkey)] - │ ├─TableReader_107 500000.00 root data:TableScan_106 - │ │ └─TableScan_106 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false - │ └─HashLeftJoin_112 5.00 root inner join, inner:TableReader_118, equal:[eq(tpch.nation.n_regionkey, tpch.region.r_regionkey)] - │ ├─TableReader_115 25.00 root data:TableScan_114 - │ │ └─TableScan_114 25.00 cop table:nation, range:[-inf,+inf], keep order:false - │ └─TableReader_118 1.00 root data:Selection_117 - │ └─Selection_117 1.00 cop eq(tpch.region.r_name, "ASIA") - │ └─TableScan_116 5.00 cop table:region, range:[-inf,+inf], keep order:false - └─HashLeftJoin_119 127533.58 root inner join, inner:HashLeftJoin_145, equal:[eq(tpch.partsupp.ps_suppkey, tpch.supplier.s_suppkey)] - ├─IndexJoin_125 625547.12 root inner join, inner:IndexLookUp_124, outer key:tpch.part.p_partkey, inner key:tpch.partsupp.ps_partkey - │ ├─TableReader_142 155496.00 root data:Selection_141 - │ │ └─Selection_141 155496.00 cop eq(tpch.part.p_size, 30), like(tpch.part.p_type, "%STEEL", 92) - │ │ └─TableScan_140 10000000.00 cop table:part, range:[-inf,+inf], keep order:false - │ └─IndexLookUp_124 1.00 root - │ ├─IndexScan_122 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [tpch.part.p_partkey], keep order:false - │ └─TableScan_123 1.00 cop table:partsupp, keep order:false - └─HashLeftJoin_145 100000.00 root inner join, inner:HashLeftJoin_153, equal:[eq(tpch.supplier.s_nationkey, tpch.nation.n_nationkey)] - ├─TableReader_148 500000.00 root data:TableScan_147 - │ └─TableScan_147 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false - └─HashLeftJoin_153 5.00 root inner join, inner:TableReader_159, equal:[eq(tpch.nation.n_regionkey, tpch.region.r_regionkey)] - ├─TableReader_156 25.00 root data:TableScan_155 - │ └─TableScan_155 25.00 cop table:nation, range:[-inf,+inf], keep order:false - └─TableReader_159 1.00 root data:Selection_158 - └─Selection_158 1.00 cop eq(tpch.region.r_name, "ASIA") - └─TableScan_157 5.00 cop table:region, range:[-inf,+inf], keep order:false +Projection_36 100.00 root tpch.supplier.s_acctbal, tpch.supplier.s_name, tpch.nation.n_name, tpch.part.p_partkey, tpch.part.p_mfgr, tpch.supplier.s_address, tpch.supplier.s_phone, tpch.supplier.s_comment +└─TopN_39 100.00 root tpch.supplier.s_acctbal:desc, tpch.nation.n_name:asc, tpch.supplier.s_name:asc, tpch.part.p_partkey:asc, offset:0, count:100 + └─HashRightJoin_44 155496.00 root inner join, inner:HashLeftJoin_50, equal:[eq(tpch.part.p_partkey, tpch.partsupp.ps_partkey) eq(tpch.partsupp.ps_supplycost, min(ps_supplycost))] + ├─HashLeftJoin_50 155496.00 root inner join, inner:TableReader_73, equal:[eq(tpch.partsupp.ps_partkey, tpch.part.p_partkey)] + │ ├─HashRightJoin_53 8155010.44 root inner join, inner:HashRightJoin_55, equal:[eq(tpch.supplier.s_suppkey, tpch.partsupp.ps_suppkey)] + │ │ ├─HashRightJoin_55 100000.00 root inner join, inner:HashRightJoin_61, equal:[eq(tpch.nation.n_nationkey, tpch.supplier.s_nationkey)] + │ │ │ ├─HashRightJoin_61 5.00 root inner join, inner:TableReader_66, equal:[eq(tpch.region.r_regionkey, tpch.nation.n_regionkey)] + │ │ │ │ ├─TableReader_66 1.00 root data:Selection_65 + │ │ │ │ │ └─Selection_65 1.00 cop eq(tpch.region.r_name, "ASIA") + │ │ │ │ │ └─TableScan_64 5.00 cop table:region, range:[-inf,+inf], keep order:false + │ │ │ │ └─TableReader_63 25.00 root data:TableScan_62 + │ │ │ │ └─TableScan_62 25.00 cop table:nation, range:[-inf,+inf], keep order:false + │ │ │ └─TableReader_68 500000.00 root data:TableScan_67 + │ │ │ └─TableScan_67 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false + │ │ └─TableReader_70 40000000.00 root data:TableScan_69 + │ │ └─TableScan_69 40000000.00 cop table:partsupp, range:[-inf,+inf], keep order:false + │ └─TableReader_73 155496.00 root data:Selection_72 + │ └─Selection_72 155496.00 cop eq(tpch.part.p_size, 30), like(tpch.part.p_type, "%STEEL", 92) + │ └─TableScan_71 10000000.00 cop table:part, range:[-inf,+inf], keep order:false + └─HashAgg_76 8155010.44 root group by:tpch.partsupp.ps_partkey, funcs:min(tpch.partsupp.ps_supplycost), firstrow(tpch.partsupp.ps_partkey) + └─HashRightJoin_80 8155010.44 root inner join, inner:HashRightJoin_82, equal:[eq(tpch.supplier.s_suppkey, tpch.partsupp.ps_suppkey)] + ├─HashRightJoin_82 100000.00 root inner join, inner:HashRightJoin_88, equal:[eq(tpch.nation.n_nationkey, tpch.supplier.s_nationkey)] + │ ├─HashRightJoin_88 5.00 root inner join, inner:TableReader_93, equal:[eq(tpch.region.r_regionkey, tpch.nation.n_regionkey)] + │ │ ├─TableReader_93 1.00 root data:Selection_92 + │ │ │ └─Selection_92 1.00 cop eq(tpch.region.r_name, "ASIA") + │ │ │ └─TableScan_91 5.00 cop table:region, range:[-inf,+inf], keep order:false + │ │ └─TableReader_90 25.00 root data:TableScan_89 + │ │ └─TableScan_89 25.00 cop table:nation, range:[-inf,+inf], keep order:false + │ └─TableReader_95 500000.00 root data:TableScan_94 + │ └─TableScan_94 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false + └─TableReader_97 40000000.00 root data:TableScan_96 + └─TableScan_96 40000000.00 cop table:partsupp, range:[-inf,+inf], keep order:false /* Q3 Shipping Priority Query This query retrieves the 10 unshipped orders with the highest value. @@ -249,22 +248,22 @@ revenue desc, o_orderdate limit 10; id count task operator info -Projection_16 10.00 root tpch.lineitem.l_orderkey, 7_col_0, tpch.orders.o_orderdate, tpch.orders.o_shippriority -└─TopN_19 10.00 root 7_col_0:desc, tpch.orders.o_orderdate:asc, offset:0, count:10 - └─HashAgg_25 40227041.09 root group by:col_4, col_5, col_6, funcs:sum(col_0), firstrow(col_1), firstrow(col_2), firstrow(col_3) - └─Projection_62 91515927.49 root mul(tpch.lineitem.l_extendedprice, minus(1, tpch.lineitem.l_discount)), tpch.orders.o_orderdate, tpch.orders.o_shippriority, tpch.lineitem.l_orderkey, tpch.lineitem.l_orderkey, tpch.orders.o_orderdate, tpch.orders.o_shippriority - └─IndexJoin_31 91515927.49 root inner join, inner:IndexLookUp_30, outer key:tpch.orders.o_orderkey, inner key:tpch.lineitem.l_orderkey - ├─HashRightJoin_52 22592975.51 root inner join, inner:TableReader_58, equal:[eq(tpch.customer.c_custkey, tpch.orders.o_custkey)] - │ ├─TableReader_58 1498236.00 root data:Selection_57 - │ │ └─Selection_57 1498236.00 cop eq(tpch.customer.c_mktsegment, "AUTOMOBILE") - │ │ └─TableScan_56 7500000.00 cop table:customer, range:[-inf,+inf], keep order:false - │ └─TableReader_55 36870000.00 root data:Selection_54 - │ └─Selection_54 36870000.00 cop lt(tpch.orders.o_orderdate, 1995-03-13 00:00:00.000000) - │ └─TableScan_53 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false - └─IndexLookUp_30 162945114.27 root - ├─IndexScan_27 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [tpch.orders.o_orderkey], keep order:false - └─Selection_29 162945114.27 cop gt(tpch.lineitem.l_shipdate, 1995-03-13 00:00:00.000000) - └─TableScan_28 1.00 cop table:lineitem, keep order:false +Projection_14 10.00 root tpch.lineitem.l_orderkey, 7_col_0, tpch.orders.o_orderdate, tpch.orders.o_shippriority +└─TopN_17 10.00 root 7_col_0:desc, tpch.orders.o_orderdate:asc, offset:0, count:10 + └─HashAgg_23 40227041.09 root group by:col_4, col_5, col_6, funcs:sum(col_0), firstrow(col_1), firstrow(col_2), firstrow(col_3) + └─Projection_60 91515927.49 root mul(tpch.lineitem.l_extendedprice, minus(1, tpch.lineitem.l_discount)), tpch.orders.o_orderdate, tpch.orders.o_shippriority, tpch.lineitem.l_orderkey, tpch.lineitem.l_orderkey, tpch.orders.o_orderdate, tpch.orders.o_shippriority + └─IndexJoin_29 91515927.49 root inner join, inner:IndexLookUp_28, outer key:tpch.orders.o_orderkey, inner key:tpch.lineitem.l_orderkey + ├─HashRightJoin_50 22592975.51 root inner join, inner:TableReader_56, equal:[eq(tpch.customer.c_custkey, tpch.orders.o_custkey)] + │ ├─TableReader_56 1498236.00 root data:Selection_55 + │ │ └─Selection_55 1498236.00 cop eq(tpch.customer.c_mktsegment, "AUTOMOBILE") + │ │ └─TableScan_54 7500000.00 cop table:customer, range:[-inf,+inf], keep order:false + │ └─TableReader_53 36870000.00 root data:Selection_52 + │ └─Selection_52 36870000.00 cop lt(tpch.orders.o_orderdate, 1995-03-13 00:00:00.000000) + │ └─TableScan_51 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false + └─IndexLookUp_28 162945114.27 root + ├─IndexScan_25 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [tpch.orders.o_orderkey], keep order:false + └─Selection_27 162945114.27 cop gt(tpch.lineitem.l_shipdate, 1995-03-13 00:00:00.000000) + └─TableScan_26 1.00 cop table:lineitem, keep order:false /* Q4 Order Priority Checking Query This query determines how well the order priority system is working and gives an assessment of customer satisfaction. @@ -343,30 +342,29 @@ n_name order by revenue desc; id count task operator info -Sort_84 5.00 root revenue:desc -└─Projection_86 5.00 root tpch.nation.n_name, 13_col_0 - └─HashAgg_89 5.00 root group by:col_2, funcs:sum(col_0), firstrow(col_1) - └─Projection_139 9763568.69 root mul(tpch.lineitem.l_extendedprice, minus(1, tpch.lineitem.l_discount)), tpch.nation.n_name, tpch.nation.n_name - └─IndexJoin_92 9763568.69 root inner join, inner:TableReader_91, outer key:tpch.orders.o_custkey, inner key:tpch.customer.c_custkey, other cond:eq(tpch.customer.c_nationkey, tpch.supplier.s_nationkey) - ├─TableReader_91 1.00 root data:TableScan_90 - │ └─TableScan_90 1.00 cop table:customer, range: decided by [tpch.supplier.s_nationkey tpch.orders.o_custkey], keep order:false - └─HashLeftJoin_95 9763568.69 root inner join, inner:HashLeftJoin_121, equal:[eq(tpch.lineitem.l_suppkey, tpch.supplier.s_suppkey)] - ├─IndexJoin_101 47889913.89 root inner join, inner:IndexLookUp_100, outer key:tpch.orders.o_orderkey, inner key:tpch.lineitem.l_orderkey - │ ├─TableReader_118 11822812.50 root data:Selection_117 - │ │ └─Selection_117 11822812.50 cop ge(tpch.orders.o_orderdate, 1994-01-01 00:00:00.000000), lt(tpch.orders.o_orderdate, 1995-01-01) - │ │ └─TableScan_116 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false - │ └─IndexLookUp_100 1.00 root - │ ├─IndexScan_98 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [tpch.orders.o_orderkey], keep order:false - │ └─TableScan_99 1.00 cop table:lineitem, keep order:false - └─HashLeftJoin_121 100000.00 root inner join, inner:HashLeftJoin_129, equal:[eq(tpch.supplier.s_nationkey, tpch.nation.n_nationkey)] - ├─TableReader_124 500000.00 root data:TableScan_123 - │ └─TableScan_123 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false - └─HashLeftJoin_129 5.00 root inner join, inner:TableReader_135, equal:[eq(tpch.nation.n_regionkey, tpch.region.r_regionkey)] - ├─TableReader_132 25.00 root data:TableScan_131 - │ └─TableScan_131 25.00 cop table:nation, range:[-inf,+inf], keep order:false - └─TableReader_135 1.00 root data:Selection_134 - └─Selection_134 1.00 cop eq(tpch.region.r_name, "MIDDLE EAST") - └─TableScan_133 5.00 cop table:region, range:[-inf,+inf], keep order:false +Sort_23 5.00 root revenue:desc +└─Projection_25 5.00 root tpch.nation.n_name, 13_col_0 + └─HashAgg_28 5.00 root group by:col_2, funcs:sum(col_0), firstrow(col_1) + └─Projection_66 11822812.50 root mul(tpch.lineitem.l_extendedprice, minus(1, tpch.lineitem.l_discount)), tpch.nation.n_name, tpch.nation.n_name + └─IndexJoin_31 11822812.50 root inner join, inner:TableReader_30, outer key:tpch.orders.o_custkey, inner key:tpch.customer.c_custkey, other cond:eq(tpch.supplier.s_nationkey, tpch.customer.c_nationkey) + ├─IndexJoin_38 11822812.50 root inner join, inner:TableReader_37, outer key:tpch.lineitem.l_orderkey, inner key:tpch.orders.o_orderkey + │ ├─HashRightJoin_42 61163763.01 root inner join, inner:HashRightJoin_44, equal:[eq(tpch.supplier.s_suppkey, tpch.lineitem.l_suppkey)] + │ │ ├─HashRightJoin_44 100000.00 root inner join, inner:HashRightJoin_50, equal:[eq(tpch.nation.n_nationkey, tpch.supplier.s_nationkey)] + │ │ │ ├─HashRightJoin_50 5.00 root inner join, inner:TableReader_55, equal:[eq(tpch.region.r_regionkey, tpch.nation.n_regionkey)] + │ │ │ │ ├─TableReader_55 1.00 root data:Selection_54 + │ │ │ │ │ └─Selection_54 1.00 cop eq(tpch.region.r_name, "MIDDLE EAST") + │ │ │ │ │ └─TableScan_53 5.00 cop table:region, range:[-inf,+inf], keep order:false + │ │ │ │ └─TableReader_52 25.00 root data:TableScan_51 + │ │ │ │ └─TableScan_51 25.00 cop table:nation, range:[-inf,+inf], keep order:false + │ │ │ └─TableReader_57 500000.00 root data:TableScan_56 + │ │ │ └─TableScan_56 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false + │ │ └─TableReader_59 300005811.00 root data:TableScan_58 + │ │ └─TableScan_58 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false + │ └─TableReader_37 11822812.50 root data:Selection_36 + │ └─Selection_36 11822812.50 cop ge(tpch.orders.o_orderdate, 1994-01-01 00:00:00.000000), lt(tpch.orders.o_orderdate, 1995-01-01) + │ └─TableScan_35 1.00 cop table:orders, range: decided by [tpch.lineitem.l_orderkey], keep order:false + └─TableReader_30 1.00 root data:TableScan_29 + └─TableScan_29 1.00 cop table:customer, range: decided by [tpch.supplier.s_nationkey tpch.orders.o_custkey], keep order:false /* Q6 Forecasting Revenue Change Query This query quantifies the amount of revenue increase that would have resulted from eliminating certain companywide @@ -445,30 +443,30 @@ supp_nation, cust_nation, l_year; id count task operator info -Sort_52 768.91 root shipping.supp_nation:asc, shipping.cust_nation:asc, shipping.l_year:asc -└─Projection_54 768.91 root shipping.supp_nation, shipping.cust_nation, shipping.l_year, 14_col_0 - └─HashAgg_57 768.91 root group by:shipping.cust_nation, shipping.l_year, shipping.supp_nation, funcs:sum(shipping.volume), firstrow(shipping.supp_nation), firstrow(shipping.cust_nation), firstrow(shipping.l_year) - └─Projection_58 9818555.15 root n1.n_name, n2.n_name, extract("YEAR", tpch.lineitem.l_shipdate), mul(tpch.lineitem.l_extendedprice, minus(1, tpch.lineitem.l_discount)) - └─HashLeftJoin_60 9818555.15 root inner join, inner:HashLeftJoin_78, equal:[eq(tpch.lineitem.l_orderkey, tpch.orders.o_orderkey)], other cond:or(and(eq(n1.n_name, "JAPAN"), eq(n2.n_name, "INDIA")), and(eq(n1.n_name, "INDIA"), eq(n2.n_name, "JAPAN"))) - ├─HashRightJoin_63 24465505.20 root inner join, inner:HashLeftJoin_68, equal:[eq(tpch.supplier.s_suppkey, tpch.lineitem.l_suppkey)] - │ ├─HashLeftJoin_68 40000.00 root inner join, inner:TableReader_74, equal:[eq(tpch.supplier.s_nationkey, n1.n_nationkey)] - │ │ ├─TableReader_71 500000.00 root data:TableScan_70 - │ │ │ └─TableScan_70 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false - │ │ └─TableReader_74 2.00 root data:Selection_73 - │ │ └─Selection_73 2.00 cop or(eq(n1.n_name, "JAPAN"), eq(n1.n_name, "INDIA")) - │ │ └─TableScan_72 25.00 cop table:n1, range:[-inf,+inf], keep order:false - │ └─TableReader_77 91321768.29 root data:Selection_76 - │ └─Selection_76 91321768.29 cop ge(tpch.lineitem.l_shipdate, 1995-01-01 00:00:00.000000), le(tpch.lineitem.l_shipdate, 1996-12-31 00:00:00.000000) - │ └─TableScan_75 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false - └─HashLeftJoin_78 9047830.45 root inner join, inner:HashLeftJoin_86, equal:[eq(tpch.orders.o_custkey, tpch.customer.c_custkey)] - ├─TableReader_81 75000000.00 root data:TableScan_80 - │ └─TableScan_80 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false - └─HashLeftJoin_86 600000.00 root inner join, inner:TableReader_92, equal:[eq(tpch.customer.c_nationkey, n2.n_nationkey)] - ├─TableReader_89 7500000.00 root data:TableScan_88 - │ └─TableScan_88 7500000.00 cop table:customer, range:[-inf,+inf], keep order:false - └─TableReader_92 2.00 root data:Selection_91 - └─Selection_91 2.00 cop or(eq(n2.n_name, "INDIA"), eq(n2.n_name, "JAPAN")) - └─TableScan_90 25.00 cop table:n2, range:[-inf,+inf], keep order:false +Sort_22 768.91 root shipping.supp_nation:asc, shipping.cust_nation:asc, shipping.l_year:asc +└─Projection_24 768.91 root shipping.supp_nation, shipping.cust_nation, shipping.l_year, 14_col_0 + └─HashAgg_27 768.91 root group by:shipping.cust_nation, shipping.l_year, shipping.supp_nation, funcs:sum(shipping.volume), firstrow(shipping.supp_nation), firstrow(shipping.cust_nation), firstrow(shipping.l_year) + └─Projection_28 1957240.42 root n1.n_name, n2.n_name, extract("YEAR", tpch.lineitem.l_shipdate), mul(tpch.lineitem.l_extendedprice, minus(1, tpch.lineitem.l_discount)) + └─HashLeftJoin_33 1957240.42 root inner join, inner:TableReader_68, equal:[eq(tpch.customer.c_nationkey, n2.n_nationkey)], other cond:or(and(eq(n1.n_name, "JAPAN"), eq(n2.n_name, "INDIA")), and(eq(n1.n_name, "INDIA"), eq(n2.n_name, "JAPAN"))) + ├─IndexJoin_37 24465505.20 root inner join, inner:TableReader_36, outer key:tpch.orders.o_custkey, inner key:tpch.customer.c_custkey + │ ├─IndexJoin_43 24465505.20 root inner join, inner:TableReader_42, outer key:tpch.lineitem.l_orderkey, inner key:tpch.orders.o_orderkey + │ │ ├─HashRightJoin_47 24465505.20 root inner join, inner:HashRightJoin_53, equal:[eq(tpch.supplier.s_suppkey, tpch.lineitem.l_suppkey)] + │ │ │ ├─HashRightJoin_53 40000.00 root inner join, inner:TableReader_58, equal:[eq(n1.n_nationkey, tpch.supplier.s_nationkey)] + │ │ │ │ ├─TableReader_58 2.00 root data:Selection_57 + │ │ │ │ │ └─Selection_57 2.00 cop or(eq(n1.n_name, "JAPAN"), eq(n1.n_name, "INDIA")) + │ │ │ │ │ └─TableScan_56 25.00 cop table:n1, range:[-inf,+inf], keep order:false + │ │ │ │ └─TableReader_55 500000.00 root data:TableScan_54 + │ │ │ │ └─TableScan_54 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false + │ │ │ └─TableReader_61 91321768.29 root data:Selection_60 + │ │ │ └─Selection_60 91321768.29 cop ge(tpch.lineitem.l_shipdate, 1995-01-01 00:00:00.000000), le(tpch.lineitem.l_shipdate, 1996-12-31 00:00:00.000000) + │ │ │ └─TableScan_59 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false + │ │ └─TableReader_42 1.00 root data:TableScan_41 + │ │ └─TableScan_41 1.00 cop table:orders, range: decided by [tpch.lineitem.l_orderkey], keep order:false + │ └─TableReader_36 1.00 root data:TableScan_35 + │ └─TableScan_35 1.00 cop table:customer, range: decided by [tpch.orders.o_custkey], keep order:false + └─TableReader_68 2.00 root data:Selection_67 + └─Selection_67 2.00 cop or(eq(n2.n_name, "INDIA"), eq(n2.n_name, "JAPAN")) + └─TableScan_66 25.00 cop table:n2, range:[-inf,+inf], keep order:false /* Q8 National Market Share Query This query determines how the market share of a given nation within a given region has changed over two years for @@ -517,37 +515,38 @@ o_year order by o_year; id count task operator info -Sort_137 718.01 root all_nations.o_year:asc -└─Projection_139 718.01 root all_nations.o_year, div(18_col_0, 18_col_1) - └─HashAgg_142 718.01 root group by:col_3, funcs:sum(col_0), sum(col_1), firstrow(col_2) - └─Projection_212 372171.05 root case(eq(all_nations.nation, "INDIA"), all_nations.volume, 0), all_nations.volume, all_nations.o_year, all_nations.o_year - └─Projection_143 372171.05 root extract("YEAR", tpch.orders.o_orderdate), mul(tpch.lineitem.l_extendedprice, minus(1, tpch.lineitem.l_discount)), n2.n_name - └─HashRightJoin_148 372171.05 root inner join, inner:TableReader_210, equal:[eq(n2.n_nationkey, tpch.supplier.s_nationkey)] - ├─TableReader_210 25.00 root data:TableScan_209 - │ └─TableScan_209 25.00 cop table:n2, range:[-inf,+inf], keep order:false - └─IndexJoin_151 372171.05 root inner join, inner:TableReader_150, outer key:tpch.lineitem.l_suppkey, inner key:tpch.supplier.s_suppkey - ├─TableReader_150 1.00 root data:TableScan_149 - │ └─TableScan_149 1.00 cop table:supplier, range: decided by [tpch.lineitem.l_suppkey], keep order:false - └─HashLeftJoin_154 372171.05 root inner join, inner:HashLeftJoin_200, equal:[eq(tpch.customer.c_nationkey, n1.n_nationkey)] - ├─IndexJoin_158 1860855.27 root inner join, inner:TableReader_157, outer key:tpch.orders.o_custkey, inner key:tpch.customer.c_custkey - │ ├─IndexJoin_165 1860855.27 root inner join, inner:TableReader_164, outer key:tpch.lineitem.l_orderkey, inner key:tpch.orders.o_orderkey - │ │ ├─HashRightJoin_185 1860855.27 root inner join, inner:TableReader_190, equal:[eq(tpch.part.p_partkey, tpch.lineitem.l_partkey)] - │ │ │ ├─TableReader_190 61674.00 root data:Selection_189 - │ │ │ │ └─Selection_189 61674.00 cop eq(tpch.part.p_type, "SMALL PLATED COPPER") - │ │ │ │ └─TableScan_188 10000000.00 cop table:part, range:[-inf,+inf], keep order:false - │ │ │ └─TableReader_187 300005811.00 root data:TableScan_186 - │ │ │ └─TableScan_186 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false - │ │ └─TableReader_164 22382008.93 root data:Selection_163 - │ │ └─Selection_163 22382008.93 cop ge(tpch.orders.o_orderdate, 1995-01-01 00:00:00.000000), le(tpch.orders.o_orderdate, 1996-12-31 00:00:00.000000) - │ │ └─TableScan_162 1.00 cop table:orders, range: decided by [tpch.lineitem.l_orderkey], keep order:false - │ └─TableReader_157 1.00 root data:TableScan_156 - │ └─TableScan_156 1.00 cop table:customer, range: decided by [tpch.orders.o_custkey], keep order:false - └─HashLeftJoin_200 5.00 root inner join, inner:TableReader_206, equal:[eq(n1.n_regionkey, tpch.region.r_regionkey)] - ├─TableReader_203 25.00 root data:TableScan_202 - │ └─TableScan_202 25.00 cop table:n1, range:[-inf,+inf], keep order:false - └─TableReader_206 1.00 root data:Selection_205 - └─Selection_205 1.00 cop eq(tpch.region.r_name, "ASIA") - └─TableScan_204 5.00 cop table:region, range:[-inf,+inf], keep order:false +Sort_29 718.01 root all_nations.o_year:asc +└─Projection_31 718.01 root all_nations.o_year, div(18_col_0, 18_col_1) + └─HashAgg_34 718.01 root group by:col_3, funcs:sum(col_0), sum(col_1), firstrow(col_2) + └─Projection_89 562348.12 root case(eq(all_nations.nation, "INDIA"), all_nations.volume, 0), all_nations.volume, all_nations.o_year, all_nations.o_year + └─Projection_35 562348.12 root extract("YEAR", tpch.orders.o_orderdate), mul(tpch.lineitem.l_extendedprice, minus(1, tpch.lineitem.l_discount)), n2.n_name + └─HashLeftJoin_39 562348.12 root inner join, inner:TableReader_87, equal:[eq(tpch.supplier.s_nationkey, n2.n_nationkey)] + ├─IndexJoin_43 562348.12 root inner join, inner:TableReader_42, outer key:tpch.lineitem.l_suppkey, inner key:tpch.supplier.s_suppkey + │ ├─HashLeftJoin_50 562348.12 root inner join, inner:TableReader_83, equal:[eq(tpch.lineitem.l_partkey, tpch.part.p_partkey)] + │ │ ├─IndexJoin_56 90661378.61 root inner join, inner:IndexLookUp_55, outer key:tpch.orders.o_orderkey, inner key:tpch.lineitem.l_orderkey + │ │ │ ├─HashRightJoin_60 22382008.93 root inner join, inner:HashRightJoin_62, equal:[eq(tpch.customer.c_custkey, tpch.orders.o_custkey)] + │ │ │ │ ├─HashRightJoin_62 1500000.00 root inner join, inner:HashRightJoin_68, equal:[eq(n1.n_nationkey, tpch.customer.c_nationkey)] + │ │ │ │ │ ├─HashRightJoin_68 5.00 root inner join, inner:TableReader_73, equal:[eq(tpch.region.r_regionkey, n1.n_regionkey)] + │ │ │ │ │ │ ├─TableReader_73 1.00 root data:Selection_72 + │ │ │ │ │ │ │ └─Selection_72 1.00 cop eq(tpch.region.r_name, "ASIA") + │ │ │ │ │ │ │ └─TableScan_71 5.00 cop table:region, range:[-inf,+inf], keep order:false + │ │ │ │ │ │ └─TableReader_70 25.00 root data:TableScan_69 + │ │ │ │ │ │ └─TableScan_69 25.00 cop table:n1, range:[-inf,+inf], keep order:false + │ │ │ │ │ └─TableReader_75 7500000.00 root data:TableScan_74 + │ │ │ │ │ └─TableScan_74 7500000.00 cop table:customer, range:[-inf,+inf], keep order:false + │ │ │ │ └─TableReader_78 22382008.93 root data:Selection_77 + │ │ │ │ └─Selection_77 22382008.93 cop ge(tpch.orders.o_orderdate, 1995-01-01 00:00:00.000000), le(tpch.orders.o_orderdate, 1996-12-31 00:00:00.000000) + │ │ │ │ └─TableScan_76 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false + │ │ │ └─IndexLookUp_55 1.00 root + │ │ │ ├─IndexScan_53 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [tpch.orders.o_orderkey], keep order:false + │ │ │ └─TableScan_54 1.00 cop table:lineitem, keep order:false + │ │ └─TableReader_83 61674.00 root data:Selection_82 + │ │ └─Selection_82 61674.00 cop eq(tpch.part.p_type, "SMALL PLATED COPPER") + │ │ └─TableScan_81 10000000.00 cop table:part, range:[-inf,+inf], keep order:false + │ └─TableReader_42 1.00 root data:TableScan_41 + │ └─TableScan_41 1.00 cop table:supplier, range: decided by [tpch.lineitem.l_suppkey], keep order:false + └─TableReader_87 25.00 root data:TableScan_86 + └─TableScan_86 25.00 cop table:n2, range:[-inf,+inf], keep order:false /* Q9 Product Type Profit Measure Query This query determines how much profit is made on a given line of parts, broken out by supplier nation and year. @@ -592,31 +591,29 @@ order by nation, o_year desc; id count task operator info -Sort_78 2406.00 root profit.nation:asc, profit.o_year:desc -└─Projection_80 2406.00 root profit.nation, profit.o_year, 14_col_0 - └─HashAgg_83 2406.00 root group by:profit.nation, profit.o_year, funcs:sum(profit.amount), firstrow(profit.nation), firstrow(profit.o_year) - └─Projection_84 971049283.51 root tpch.nation.n_name, extract("YEAR", tpch.orders.o_orderdate), minus(mul(tpch.lineitem.l_extendedprice, minus(1, tpch.lineitem.l_discount)), mul(tpch.partsupp.ps_supplycost, tpch.lineitem.l_quantity)) - └─IndexJoin_88 971049283.51 root inner join, inner:IndexLookUp_87, outer key:tpch.lineitem.l_suppkey, tpch.lineitem.l_partkey, inner key:tpch.partsupp.ps_suppkey, tpch.partsupp.ps_partkey - ├─IndexLookUp_87 1.00 root - │ ├─IndexScan_85 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [tpch.lineitem.l_suppkey tpch.lineitem.l_partkey], keep order:false - │ └─TableScan_86 1.00 cop table:partsupp, keep order:false - └─HashLeftJoin_91 241379546.70 root inner join, inner:HashLeftJoin_126, equal:[eq(tpch.lineitem.l_suppkey, tpch.supplier.s_suppkey)] - ├─MergeJoin_93 241379546.70 root inner join, left key:tpch.lineitem.l_orderkey, right key:tpch.orders.o_orderkey - │ ├─IndexJoin_102 241379546.70 root inner join, inner:TableReader_101, outer key:tpch.lineitem.l_partkey, inner key:tpch.part.p_partkey - │ │ ├─TableReader_101 8000000.00 root data:Selection_100 - │ │ │ └─Selection_100 8000000.00 cop like(tpch.part.p_name, "%dim%", 92) - │ │ │ └─TableScan_99 1.00 cop table:part, range: decided by [tpch.lineitem.l_partkey], keep order:false - │ │ └─Projection_107 300005811.00 root tpch.lineitem.l_orderkey, tpch.lineitem.l_partkey, tpch.lineitem.l_suppkey, tpch.lineitem.l_quantity, tpch.lineitem.l_extendedprice, tpch.lineitem.l_discount - │ │ └─IndexLookUp_106 300005811.00 root - │ │ ├─IndexScan_104 300005811.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range:[NULL,+inf], keep order:true - │ │ └─TableScan_105 300005811.00 cop table:lineitem, keep order:false - │ └─TableReader_109 75000000.00 root data:TableScan_108 - │ └─TableScan_108 75000000.00 cop table:orders, range:[-inf,+inf], keep order:true - └─HashLeftJoin_126 500000.00 root inner join, inner:TableReader_131, equal:[eq(tpch.supplier.s_nationkey, tpch.nation.n_nationkey)] - ├─TableReader_129 500000.00 root data:TableScan_128 - │ └─TableScan_128 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false - └─TableReader_131 25.00 root data:TableScan_130 - └─TableScan_130 25.00 cop table:nation, range:[-inf,+inf], keep order:false +Sort_25 2406.00 root profit.nation:asc, profit.o_year:desc +└─Projection_27 2406.00 root profit.nation, profit.o_year, 14_col_0 + └─HashAgg_30 2406.00 root group by:profit.nation, profit.o_year, funcs:sum(profit.amount), firstrow(profit.nation), firstrow(profit.o_year) + └─Projection_31 971049283.51 root tpch.nation.n_name, extract("YEAR", tpch.orders.o_orderdate), minus(mul(tpch.lineitem.l_extendedprice, minus(1, tpch.lineitem.l_discount)), mul(tpch.partsupp.ps_supplycost, tpch.lineitem.l_quantity)) + └─IndexJoin_35 971049283.51 root inner join, inner:IndexLookUp_34, outer key:tpch.lineitem.l_suppkey, tpch.lineitem.l_partkey, inner key:tpch.partsupp.ps_suppkey, tpch.partsupp.ps_partkey + ├─IndexJoin_41 241379546.70 root inner join, inner:TableReader_40, outer key:tpch.lineitem.l_orderkey, inner key:tpch.orders.o_orderkey + │ ├─HashLeftJoin_52 241379546.70 root inner join, inner:TableReader_69, equal:[eq(tpch.lineitem.l_partkey, tpch.part.p_partkey)] + │ │ ├─HashRightJoin_55 300005811.00 root inner join, inner:HashRightJoin_60, equal:[eq(tpch.supplier.s_suppkey, tpch.lineitem.l_suppkey)] + │ │ │ ├─HashRightJoin_60 500000.00 root inner join, inner:TableReader_64, equal:[eq(tpch.nation.n_nationkey, tpch.supplier.s_nationkey)] + │ │ │ │ ├─TableReader_64 25.00 root data:TableScan_63 + │ │ │ │ │ └─TableScan_63 25.00 cop table:nation, range:[-inf,+inf], keep order:false + │ │ │ │ └─TableReader_62 500000.00 root data:TableScan_61 + │ │ │ │ └─TableScan_61 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false + │ │ │ └─TableReader_66 300005811.00 root data:TableScan_65 + │ │ │ └─TableScan_65 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false + │ │ └─TableReader_69 8000000.00 root data:Selection_68 + │ │ └─Selection_68 8000000.00 cop like(tpch.part.p_name, "%dim%", 92) + │ │ └─TableScan_67 10000000.00 cop table:part, range:[-inf,+inf], keep order:false + │ └─TableReader_40 1.00 root data:TableScan_39 + │ └─TableScan_39 1.00 cop table:orders, range: decided by [tpch.lineitem.l_orderkey], keep order:false + └─IndexLookUp_34 1.00 root + ├─IndexScan_32 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [tpch.lineitem.l_suppkey tpch.lineitem.l_partkey], keep order:false + └─TableScan_33 1.00 cop table:partsupp, keep order:false /* Q10 Returned Item Reporting Query The query identifies customers who might be having problems with the parts that are shipped to them. @@ -661,24 +658,24 @@ order by revenue desc limit 20; id count task operator info -Projection_24 20.00 root tpch.customer.c_custkey, tpch.customer.c_name, 9_col_0, tpch.customer.c_acctbal, tpch.nation.n_name, tpch.customer.c_address, tpch.customer.c_phone, tpch.customer.c_comment -└─TopN_27 20.00 root 9_col_0:desc, offset:0, count:20 - └─HashAgg_33 3017307.69 root group by:col_10, col_11, col_12, col_13, col_14, col_8, col_9, funcs:sum(col_0), firstrow(col_1), firstrow(col_2), firstrow(col_3), firstrow(col_4), firstrow(col_5), firstrow(col_6), firstrow(col_7) - └─Projection_77 12222016.17 root mul(tpch.lineitem.l_extendedprice, minus(1, tpch.lineitem.l_discount)), tpch.customer.c_custkey, tpch.customer.c_name, tpch.customer.c_address, tpch.customer.c_phone, tpch.customer.c_acctbal, tpch.customer.c_comment, tpch.nation.n_name, tpch.customer.c_custkey, tpch.customer.c_name, tpch.customer.c_acctbal, tpch.customer.c_phone, tpch.nation.n_name, tpch.customer.c_address, tpch.customer.c_comment - └─IndexJoin_39 12222016.17 root inner join, inner:IndexLookUp_38, outer key:tpch.orders.o_orderkey, inner key:tpch.lineitem.l_orderkey - ├─HashRightJoin_61 3017307.69 root inner join, inner:TableReader_73, equal:[eq(tpch.nation.n_nationkey, tpch.customer.c_nationkey)] - │ ├─TableReader_73 25.00 root data:TableScan_72 - │ │ └─TableScan_72 25.00 cop table:nation, range:[-inf,+inf], keep order:false - │ └─IndexJoin_64 3017307.69 root inner join, inner:TableReader_63, outer key:tpch.orders.o_custkey, inner key:tpch.customer.c_custkey - │ ├─TableReader_63 1.00 root data:TableScan_62 - │ │ └─TableScan_62 1.00 cop table:customer, range: decided by [tpch.orders.o_custkey], keep order:false - │ └─TableReader_69 3017307.69 root data:Selection_68 - │ └─Selection_68 3017307.69 cop ge(tpch.orders.o_orderdate, 1993-08-01 00:00:00.000000), lt(tpch.orders.o_orderdate, 1993-11-01) - │ └─TableScan_67 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false - └─IndexLookUp_38 73916005.00 root - ├─IndexScan_35 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [tpch.orders.o_orderkey], keep order:false - └─Selection_37 73916005.00 cop eq(tpch.lineitem.l_returnflag, "R") - └─TableScan_36 1.00 cop table:lineitem, keep order:false +Projection_17 20.00 root tpch.customer.c_custkey, tpch.customer.c_name, 9_col_0, tpch.customer.c_acctbal, tpch.nation.n_name, tpch.customer.c_address, tpch.customer.c_phone, tpch.customer.c_comment +└─TopN_20 20.00 root 9_col_0:desc, offset:0, count:20 + └─HashAgg_26 3017307.69 root group by:col_10, col_11, col_12, col_13, col_14, col_8, col_9, funcs:sum(col_0), firstrow(col_1), firstrow(col_2), firstrow(col_3), firstrow(col_4), firstrow(col_5), firstrow(col_6), firstrow(col_7) + └─Projection_52 12222016.17 root mul(tpch.lineitem.l_extendedprice, minus(1, tpch.lineitem.l_discount)), tpch.customer.c_custkey, tpch.customer.c_name, tpch.customer.c_address, tpch.customer.c_phone, tpch.customer.c_acctbal, tpch.customer.c_comment, tpch.nation.n_name, tpch.customer.c_custkey, tpch.customer.c_name, tpch.customer.c_acctbal, tpch.customer.c_phone, tpch.nation.n_name, tpch.customer.c_address, tpch.customer.c_comment + └─IndexJoin_32 12222016.17 root inner join, inner:IndexLookUp_31, outer key:tpch.orders.o_orderkey, inner key:tpch.lineitem.l_orderkey + ├─HashLeftJoin_35 3017307.69 root inner join, inner:TableReader_48, equal:[eq(tpch.customer.c_custkey, tpch.orders.o_custkey)] + │ ├─HashRightJoin_41 7500000.00 root inner join, inner:TableReader_45, equal:[eq(tpch.nation.n_nationkey, tpch.customer.c_nationkey)] + │ │ ├─TableReader_45 25.00 root data:TableScan_44 + │ │ │ └─TableScan_44 25.00 cop table:nation, range:[-inf,+inf], keep order:false + │ │ └─TableReader_43 7500000.00 root data:TableScan_42 + │ │ └─TableScan_42 7500000.00 cop table:customer, range:[-inf,+inf], keep order:false + │ └─TableReader_48 3017307.69 root data:Selection_47 + │ └─Selection_47 3017307.69 cop ge(tpch.orders.o_orderdate, 1993-08-01 00:00:00.000000), lt(tpch.orders.o_orderdate, 1993-11-01) + │ └─TableScan_46 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false + └─IndexLookUp_31 73916005.00 root + ├─IndexScan_28 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [tpch.orders.o_orderkey], keep order:false + └─Selection_30 73916005.00 cop eq(tpch.lineitem.l_returnflag, "R") + └─TableScan_29 1.00 cop table:lineitem, keep order:false /* Q11 Important Stock Identification Query This query finds the most important subset of suppliers' stock in a given nation. @@ -715,20 +712,20 @@ and n_name = 'MOZAMBIQUE' order by value desc; id count task operator info -Projection_67 1304801.67 root tpch.partsupp.ps_partkey, value -└─Sort_68 1304801.67 root value:desc - └─Selection_70 1304801.67 root gt(sel_agg_4, NULL) - └─HashAgg_73 1631002.09 root group by:col_2, funcs:sum(col_0), firstrow(col_1) - └─Projection_92 1631002.09 root mul(tpch.partsupp.ps_supplycost, cast(tpch.partsupp.ps_availqty)), tpch.partsupp.ps_partkey, tpch.partsupp.ps_partkey - └─HashLeftJoin_76 1631002.09 root inner join, inner:HashLeftJoin_84, equal:[eq(tpch.partsupp.ps_suppkey, tpch.supplier.s_suppkey)] - ├─TableReader_79 40000000.00 root data:TableScan_78 - │ └─TableScan_78 40000000.00 cop table:partsupp, range:[-inf,+inf], keep order:false - └─HashLeftJoin_84 20000.00 root inner join, inner:TableReader_90, equal:[eq(tpch.supplier.s_nationkey, tpch.nation.n_nationkey)] - ├─TableReader_87 500000.00 root data:TableScan_86 - │ └─TableScan_86 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false - └─TableReader_90 1.00 root data:Selection_89 - └─Selection_89 1.00 cop eq(tpch.nation.n_name, "MOZAMBIQUE") - └─TableScan_88 25.00 cop table:nation, range:[-inf,+inf], keep order:false +Projection_63 1304801.67 root tpch.partsupp.ps_partkey, value +└─Sort_64 1304801.67 root value:desc + └─Selection_66 1304801.67 root gt(sel_agg_4, NULL) + └─HashAgg_69 1631002.09 root group by:col_2, funcs:sum(col_0), firstrow(col_1) + └─Projection_88 1631002.09 root mul(tpch.partsupp.ps_supplycost, cast(tpch.partsupp.ps_availqty)), tpch.partsupp.ps_partkey, tpch.partsupp.ps_partkey + └─HashRightJoin_73 1631002.09 root inner join, inner:HashRightJoin_79, equal:[eq(tpch.supplier.s_suppkey, tpch.partsupp.ps_suppkey)] + ├─HashRightJoin_79 20000.00 root inner join, inner:TableReader_84, equal:[eq(tpch.nation.n_nationkey, tpch.supplier.s_nationkey)] + │ ├─TableReader_84 1.00 root data:Selection_83 + │ │ └─Selection_83 1.00 cop eq(tpch.nation.n_name, "MOZAMBIQUE") + │ │ └─TableScan_82 25.00 cop table:nation, range:[-inf,+inf], keep order:false + │ └─TableReader_81 500000.00 root data:TableScan_80 + │ └─TableScan_80 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false + └─TableReader_86 40000000.00 root data:TableScan_85 + └─TableScan_85 40000000.00 cop table:partsupp, range:[-inf,+inf], keep order:false /* Q12 Shipping Modes and Order Priority Query This query determines whether selecting less expensive modes of shipping is negatively affecting the critical-priority @@ -773,12 +770,12 @@ Sort_9 1.00 root tpch.lineitem.l_shipmode:asc └─Projection_11 1.00 root tpch.lineitem.l_shipmode, 5_col_0, 5_col_1 └─HashAgg_14 1.00 root group by:col_3, funcs:sum(col_0), sum(col_1), firstrow(col_2) └─Projection_40 10023369.01 root cast(case(or(eq(tpch.orders.o_orderpriority, "1-URGENT"), eq(tpch.orders.o_orderpriority, "2-HIGH")), 1, 0)), cast(case(and(ne(tpch.orders.o_orderpriority, "1-URGENT"), ne(tpch.orders.o_orderpriority, "2-HIGH")), 1, 0)), tpch.lineitem.l_shipmode, tpch.lineitem.l_shipmode - └─IndexJoin_23 10023369.01 root inner join, inner:TableReader_22, outer key:tpch.lineitem.l_orderkey, inner key:tpch.orders.o_orderkey - ├─TableReader_22 1.00 root data:TableScan_21 - │ └─TableScan_21 1.00 cop table:orders, range: decided by [tpch.lineitem.l_orderkey], keep order:false - └─TableReader_36 10023369.01 root data:Selection_35 - └─Selection_35 10023369.01 cop ge(tpch.lineitem.l_receiptdate, 1997-01-01 00:00:00.000000), in(tpch.lineitem.l_shipmode, "RAIL", "FOB"), lt(tpch.lineitem.l_commitdate, tpch.lineitem.l_receiptdate), lt(tpch.lineitem.l_receiptdate, 1998-01-01), lt(tpch.lineitem.l_shipdate, tpch.lineitem.l_commitdate) - └─TableScan_34 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false + └─IndexJoin_18 10023369.01 root inner join, inner:TableReader_17, outer key:tpch.lineitem.l_orderkey, inner key:tpch.orders.o_orderkey + ├─TableReader_36 10023369.01 root data:Selection_35 + │ └─Selection_35 10023369.01 cop ge(tpch.lineitem.l_receiptdate, 1997-01-01 00:00:00.000000), in(tpch.lineitem.l_shipmode, "RAIL", "FOB"), lt(tpch.lineitem.l_commitdate, tpch.lineitem.l_receiptdate), lt(tpch.lineitem.l_receiptdate, 1998-01-01), lt(tpch.lineitem.l_shipdate, tpch.lineitem.l_commitdate) + │ └─TableScan_34 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false + └─TableReader_17 1.00 root data:TableScan_16 + └─TableScan_16 1.00 cop table:orders, range: decided by [tpch.lineitem.l_orderkey], keep order:false /* Q13 Customer Distribution Query This query seeks relationships between customers and the size of their orders. @@ -930,16 +927,16 @@ p_brand, p_type, p_size; id count task operator info -Sort_13 14.41 root supplier_cnt:desc, tpch.part.p_brand:asc, tpch.part.p_type:asc, tpch.part.p_size:asc -└─Projection_15 14.41 root tpch.part.p_brand, tpch.part.p_type, tpch.part.p_size, 9_col_0 - └─HashAgg_18 14.41 root group by:tpch.part.p_brand, tpch.part.p_size, tpch.part.p_type, funcs:count(distinct tpch.partsupp.ps_suppkey), firstrow(tpch.part.p_brand), firstrow(tpch.part.p_type), firstrow(tpch.part.p_size) +Sort_13 3863988.24 root supplier_cnt:desc, tpch.part.p_brand:asc, tpch.part.p_type:asc, tpch.part.p_size:asc +└─Projection_15 3863988.24 root tpch.part.p_brand, tpch.part.p_type, tpch.part.p_size, 9_col_0 + └─HashAgg_18 3863988.24 root group by:tpch.part.p_brand, tpch.part.p_size, tpch.part.p_type, funcs:count(distinct tpch.partsupp.ps_suppkey), firstrow(tpch.part.p_brand), firstrow(tpch.part.p_type), firstrow(tpch.part.p_size) └─HashLeftJoin_23 3863988.24 root anti semi join, inner:TableReader_46, equal:[eq(tpch.partsupp.ps_suppkey, tpch.supplier.s_suppkey)] - ├─IndexJoin_31 4829985.30 root inner join, inner:IndexReader_30, outer key:tpch.part.p_partkey, inner key:tpch.partsupp.ps_partkey - │ ├─IndexReader_30 1.00 root index:IndexScan_29 - │ │ └─IndexScan_29 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [tpch.part.p_partkey], keep order:false - │ └─TableReader_41 1200618.43 root data:Selection_40 - │ └─Selection_40 1200618.43 cop in(tpch.part.p_size, 48, 19, 12, 4, 41, 7, 21, 39), ne(tpch.part.p_brand, "Brand#34"), not(like(tpch.part.p_type, "LARGE BRUSHED%", 92)) - │ └─TableScan_39 10000000.00 cop table:part, range:[-inf,+inf], keep order:false + ├─IndexJoin_27 4829985.30 root inner join, inner:IndexReader_26, outer key:tpch.part.p_partkey, inner key:tpch.partsupp.ps_partkey + │ ├─TableReader_41 1200618.43 root data:Selection_40 + │ │ └─Selection_40 1200618.43 cop in(tpch.part.p_size, 48, 19, 12, 4, 41, 7, 21, 39), ne(tpch.part.p_brand, "Brand#34"), not(like(tpch.part.p_type, "LARGE BRUSHED%", 92)) + │ │ └─TableScan_39 10000000.00 cop table:part, range:[-inf,+inf], keep order:false + │ └─IndexReader_26 1.00 root index:IndexScan_25 + │ └─IndexScan_25 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [tpch.part.p_partkey], keep order:false └─TableReader_46 400000.00 root data:Selection_45 └─Selection_45 400000.00 cop like(tpch.supplier.s_comment, "%Customer%Complaints%", 92) └─TableScan_44 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false @@ -972,20 +969,20 @@ where l_partkey = p_partkey ); id count task operator info -Projection_17 1.00 root div(11_col_0, 7.0) -└─StreamAgg_22 1.00 root funcs:sum(tpch.lineitem.l_extendedprice) - └─Projection_45 293773.83 root tpch.lineitem.l_partkey, tpch.lineitem.l_quantity, tpch.lineitem.l_extendedprice, tpch.part.p_partkey, tpch.part.p_brand, tpch.part.p_container, mul(0.2, 7_col_0) - └─HashLeftJoin_46 293773.83 root inner join, inner:HashRightJoin_33, equal:[eq(tpch.lineitem.l_partkey, tpch.part.p_partkey)], other cond:lt(tpch.lineitem.l_quantity, mul(0.2, 7_col_0)) - ├─TableReader_49 300005811.00 root data:TableScan_48 - │ └─TableScan_48 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false - └─HashRightJoin_33 9736.49 root inner join, inner:TableReader_44, equal:[eq(tpch.part.p_partkey, tpch.lineitem.l_partkey)] - ├─TableReader_44 9736.49 root data:Selection_43 - │ └─Selection_43 9736.49 cop eq(tpch.part.p_brand, "Brand#44"), eq(tpch.part.p_container, "WRAP PKG") - │ └─TableScan_42 10000000.00 cop table:part, range:[-inf,+inf], keep order:false - └─HashAgg_38 9943040.00 root group by:col_3, funcs:avg(col_0, col_1), firstrow(col_2) - └─TableReader_39 9943040.00 root data:HashAgg_34 - └─HashAgg_34 9943040.00 cop group by:tpch.lineitem.l_partkey, funcs:avg(tpch.lineitem.l_quantity), firstrow(tpch.lineitem.l_partkey) - └─TableScan_37 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false +Projection_16 1.00 root div(11_col_0, 7.0) +└─StreamAgg_21 1.00 root funcs:sum(tpch.lineitem.l_extendedprice) + └─Projection_44 293773.83 root tpch.lineitem.l_partkey, tpch.lineitem.l_quantity, tpch.lineitem.l_extendedprice, tpch.part.p_partkey, tpch.part.p_brand, tpch.part.p_container, mul(0.2, 7_col_0) + └─HashRightJoin_46 293773.83 root inner join, inner:HashRightJoin_30, equal:[eq(tpch.part.p_partkey, tpch.lineitem.l_partkey)], other cond:lt(tpch.lineitem.l_quantity, mul(0.2, 7_col_0)) + ├─HashRightJoin_30 293773.83 root inner join, inner:TableReader_35, equal:[eq(tpch.part.p_partkey, tpch.lineitem.l_partkey)] + │ ├─TableReader_35 9736.49 root data:Selection_34 + │ │ └─Selection_34 9736.49 cop eq(tpch.part.p_brand, "Brand#44"), eq(tpch.part.p_container, "WRAP PKG") + │ │ └─TableScan_33 10000000.00 cop table:part, range:[-inf,+inf], keep order:false + │ └─TableReader_32 300005811.00 root data:TableScan_31 + │ └─TableScan_31 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false + └─HashAgg_61 9943040.00 root group by:col_3, funcs:avg(col_0, col_1), firstrow(col_2) + └─TableReader_62 9943040.00 root data:HashAgg_58 + └─HashAgg_58 9943040.00 cop group by:tpch.lineitem.l_partkey, funcs:avg(tpch.lineitem.l_quantity), firstrow(tpch.lineitem.l_partkey) + └─TableScan_39 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false /* Q18 Large Volume Customer Query The Large Volume Customer Query ranks customers based on their having placed a large quantity order. Large @@ -1029,24 +1026,24 @@ o_totalprice desc, o_orderdate limit 100; id count task operator info -Projection_32 100.00 root tpch.customer.c_name, tpch.customer.c_custkey, tpch.orders.o_orderkey, tpch.orders.o_orderdate, tpch.orders.o_totalprice, 14_col_0 -└─TopN_35 100.00 root tpch.orders.o_totalprice:desc, tpch.orders.o_orderdate:asc, offset:0, count:100 - └─HashAgg_41 59251097.60 root group by:tpch.customer.c_custkey, tpch.customer.c_name, tpch.orders.o_orderdate, tpch.orders.o_orderkey, tpch.orders.o_totalprice, funcs:sum(tpch.lineitem.l_quantity), firstrow(tpch.customer.c_custkey), firstrow(tpch.customer.c_name), firstrow(tpch.orders.o_orderkey), firstrow(tpch.orders.o_totalprice), firstrow(tpch.orders.o_orderdate) - └─IndexJoin_46 240004648.80 root inner join, inner:IndexLookUp_45, outer key:tpch.orders.o_orderkey, inner key:tpch.lineitem.l_orderkey - ├─HashRightJoin_58 59251097.60 root inner join, inner:TableReader_85, equal:[eq(tpch.customer.c_custkey, tpch.orders.o_custkey)] - │ ├─TableReader_85 7500000.00 root data:TableScan_84 - │ │ └─TableScan_84 7500000.00 cop table:customer, range:[-inf,+inf], keep order:false - │ └─IndexJoin_61 59251097.60 root inner join, inner:TableReader_60, outer key:tpch.lineitem.l_orderkey, inner key:tpch.orders.o_orderkey - │ ├─TableReader_60 1.00 root data:TableScan_59 - │ │ └─TableScan_59 1.00 cop table:orders, range: decided by [tpch.lineitem.l_orderkey], keep order:false - │ └─Selection_64 59251097.60 root gt(sel_agg_2, 314) - │ └─HashAgg_71 74063872.00 root group by:col_2, funcs:sum(col_0), firstrow(col_1) - │ └─TableReader_72 74063872.00 root data:HashAgg_65 - │ └─HashAgg_65 74063872.00 cop group by:tpch.lineitem.l_orderkey, funcs:sum(tpch.lineitem.l_quantity), firstrow(tpch.lineitem.l_orderkey) - │ └─TableScan_70 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false - └─IndexLookUp_45 1.00 root - ├─IndexScan_43 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [tpch.orders.o_orderkey], keep order:false - └─TableScan_44 1.00 cop table:lineitem, keep order:false +Projection_24 100.00 root tpch.customer.c_name, tpch.customer.c_custkey, tpch.orders.o_orderkey, tpch.orders.o_orderdate, tpch.orders.o_totalprice, 14_col_0 +└─TopN_27 100.00 root tpch.orders.o_totalprice:desc, tpch.orders.o_orderdate:asc, offset:0, count:100 + └─HashAgg_33 59251097.60 root group by:tpch.customer.c_custkey, tpch.customer.c_name, tpch.orders.o_orderdate, tpch.orders.o_orderkey, tpch.orders.o_totalprice, funcs:sum(tpch.lineitem.l_quantity), firstrow(tpch.customer.c_custkey), firstrow(tpch.customer.c_name), firstrow(tpch.orders.o_orderkey), firstrow(tpch.orders.o_totalprice), firstrow(tpch.orders.o_orderdate) + └─IndexJoin_38 240004648.80 root inner join, inner:IndexLookUp_37, outer key:tpch.orders.o_orderkey, inner key:tpch.lineitem.l_orderkey + ├─HashLeftJoin_41 59251097.60 root inner join, inner:Selection_52, equal:[eq(tpch.orders.o_orderkey, tpch.lineitem.l_orderkey)] + │ ├─HashRightJoin_47 75000000.00 root inner join, inner:TableReader_51, equal:[eq(tpch.customer.c_custkey, tpch.orders.o_custkey)] + │ │ ├─TableReader_51 7500000.00 root data:TableScan_50 + │ │ │ └─TableScan_50 7500000.00 cop table:customer, range:[-inf,+inf], keep order:false + │ │ └─TableReader_49 75000000.00 root data:TableScan_48 + │ │ └─TableScan_48 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false + │ └─Selection_52 59251097.60 root gt(sel_agg_2, 314) + │ └─HashAgg_59 74063872.00 root group by:col_2, funcs:sum(col_0), firstrow(col_1) + │ └─TableReader_60 74063872.00 root data:HashAgg_53 + │ └─HashAgg_53 74063872.00 cop group by:tpch.lineitem.l_orderkey, funcs:sum(tpch.lineitem.l_quantity), firstrow(tpch.lineitem.l_orderkey) + │ └─TableScan_58 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false + └─IndexLookUp_37 1.00 root + ├─IndexScan_35 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [tpch.orders.o_orderkey], keep order:false + └─TableScan_36 1.00 cop table:lineitem, keep order:false /* Q19 Discounted Revenue Query The Discounted Revenue Query reports the gross discounted revenue attributed to the sale of selected parts handled @@ -1149,30 +1146,30 @@ and n_name = 'ALGERIA' order by s_name; id count task operator info -Sort_30 20000.00 root tpch.supplier.s_name:asc -└─Projection_32 20000.00 root tpch.supplier.s_name, tpch.supplier.s_address - └─HashLeftJoin_33 20000.00 root inner join, inner:HashLeftJoin_77, equal:[eq(tpch.partsupp.ps_suppkey, tpch.supplier.s_suppkey)] - ├─HashAgg_37 257492.04 root group by:tpch.partsupp.ps_suppkey, funcs:firstrow(tpch.partsupp.ps_suppkey) - │ └─Projection_38 257492.04 root tpch.partsupp.ps_partkey, tpch.partsupp.ps_suppkey, tpch.partsupp.ps_availqty, tpch.part.p_partkey, mul(0.5, 14_col_0) - │ └─Selection_39 257492.04 root gt(cast(tpch.partsupp.ps_availqty), mul(0.5, 14_col_0)) - │ └─HashAgg_42 321865.05 root group by:tpch.partsupp.ps_partkey, tpch.partsupp.ps_suppkey, funcs:firstrow(tpch.partsupp.ps_partkey), firstrow(tpch.partsupp.ps_suppkey), firstrow(tpch.partsupp.ps_availqty), firstrow(tpch.part.p_partkey), sum(tpch.lineitem.l_quantity) - │ └─HashLeftJoin_45 9711455.06 root left outer join, inner:TableReader_72, equal:[eq(tpch.partsupp.ps_partkey, tpch.lineitem.l_partkey) eq(tpch.partsupp.ps_suppkey, tpch.lineitem.l_suppkey)] - │ ├─IndexJoin_54 321865.05 root inner join, inner:IndexLookUp_53, outer key:tpch.part.p_partkey, inner key:tpch.partsupp.ps_partkey - │ │ ├─IndexLookUp_53 1.00 root - │ │ │ ├─IndexScan_51 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [tpch.part.p_partkey], keep order:false - │ │ │ └─TableScan_52 1.00 cop table:partsupp, keep order:false - │ │ └─TableReader_67 80007.93 root data:Selection_66 - │ │ └─Selection_66 80007.93 cop like(tpch.part.p_name, "green%", 92) - │ │ └─TableScan_65 10000000.00 cop table:part, range:[-inf,+inf], keep order:false - │ └─TableReader_72 44189356.65 root data:Selection_71 - │ └─Selection_71 44189356.65 cop ge(tpch.lineitem.l_shipdate, 1993-01-01 00:00:00.000000), lt(tpch.lineitem.l_shipdate, 1994-01-01) - │ └─TableScan_70 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false - └─HashLeftJoin_77 20000.00 root inner join, inner:TableReader_83, equal:[eq(tpch.supplier.s_nationkey, tpch.nation.n_nationkey)] - ├─TableReader_80 500000.00 root data:TableScan_79 - │ └─TableScan_79 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false - └─TableReader_83 1.00 root data:Selection_82 - └─Selection_82 1.00 cop eq(tpch.nation.n_name, "ALGERIA") - └─TableScan_81 25.00 cop table:nation, range:[-inf,+inf], keep order:false +Sort_28 20000.00 root tpch.supplier.s_name:asc +└─Projection_30 20000.00 root tpch.supplier.s_name, tpch.supplier.s_address + └─HashRightJoin_32 20000.00 root inner join, inner:HashRightJoin_38, equal:[eq(tpch.supplier.s_suppkey, tpch.partsupp.ps_suppkey)] + ├─HashRightJoin_38 20000.00 root inner join, inner:TableReader_43, equal:[eq(tpch.nation.n_nationkey, tpch.supplier.s_nationkey)] + │ ├─TableReader_43 1.00 root data:Selection_42 + │ │ └─Selection_42 1.00 cop eq(tpch.nation.n_name, "ALGERIA") + │ │ └─TableScan_41 25.00 cop table:nation, range:[-inf,+inf], keep order:false + │ └─TableReader_40 500000.00 root data:TableScan_39 + │ └─TableScan_39 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false + └─HashAgg_46 64006.34 root group by:tpch.partsupp.ps_suppkey, funcs:firstrow(tpch.partsupp.ps_suppkey) + └─Projection_47 64006.34 root tpch.partsupp.ps_partkey, tpch.partsupp.ps_suppkey, tpch.partsupp.ps_availqty, tpch.part.p_partkey, mul(0.5, 14_col_0) + └─Selection_48 64006.34 root gt(cast(tpch.partsupp.ps_availqty), mul(0.5, 14_col_0)) + └─HashAgg_51 80007.93 root group by:tpch.partsupp.ps_partkey, tpch.partsupp.ps_suppkey, funcs:firstrow(tpch.partsupp.ps_partkey), firstrow(tpch.partsupp.ps_suppkey), firstrow(tpch.partsupp.ps_availqty), firstrow(tpch.part.p_partkey), sum(tpch.lineitem.l_quantity) + └─HashLeftJoin_54 9711455.06 root left outer join, inner:TableReader_81, equal:[eq(tpch.partsupp.ps_partkey, tpch.lineitem.l_partkey) eq(tpch.partsupp.ps_suppkey, tpch.lineitem.l_suppkey)] + ├─IndexJoin_59 321865.05 root inner join, inner:IndexLookUp_58, outer key:tpch.part.p_partkey, inner key:tpch.partsupp.ps_partkey + │ ├─TableReader_76 80007.93 root data:Selection_75 + │ │ └─Selection_75 80007.93 cop like(tpch.part.p_name, "green%", 92) + │ │ └─TableScan_74 10000000.00 cop table:part, range:[-inf,+inf], keep order:false + │ └─IndexLookUp_58 1.00 root + │ ├─IndexScan_56 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [tpch.part.p_partkey], keep order:false + │ └─TableScan_57 1.00 cop table:partsupp, keep order:false + └─TableReader_81 44189356.65 root data:Selection_80 + └─Selection_80 44189356.65 cop ge(tpch.lineitem.l_shipdate, 1993-01-01 00:00:00.000000), lt(tpch.lineitem.l_shipdate, 1994-01-01) + └─TableScan_79 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false /* Q21 Suppliers Who Kept Orders Waiting Query This query identifies certain suppliers who were not able to ship required parts in a timely manner. @@ -1222,32 +1219,32 @@ numwait desc, s_name limit 100; id count task operator info -Projection_32 100.00 root tpch.supplier.s_name, 17_col_0 -└─TopN_35 100.00 root 17_col_0:desc, tpch.supplier.s_name:asc, offset:0, count:100 - └─HashAgg_41 12800.00 root group by:tpch.supplier.s_name, funcs:count(1), firstrow(tpch.supplier.s_name) - └─IndexJoin_47 7828961.66 root anti semi join, inner:IndexLookUp_46, outer key:l1.l_orderkey, inner key:l3.l_orderkey, other cond:ne(l3.l_suppkey, l1.l_suppkey), ne(l3.l_suppkey, tpch.supplier.s_suppkey) - ├─IndexJoin_63 9786202.08 root semi join, inner:IndexLookUp_62, outer key:l1.l_orderkey, inner key:l2.l_orderkey, other cond:ne(l2.l_suppkey, l1.l_suppkey), ne(l2.l_suppkey, tpch.supplier.s_suppkey) - │ ├─IndexJoin_69 12232752.60 root inner join, inner:TableReader_68, outer key:l1.l_orderkey, inner key:tpch.orders.o_orderkey - │ │ ├─HashRightJoin_73 12232752.60 root inner join, inner:HashLeftJoin_78, equal:[eq(tpch.supplier.s_suppkey, l1.l_suppkey)] - │ │ │ ├─HashLeftJoin_78 20000.00 root inner join, inner:TableReader_84, equal:[eq(tpch.supplier.s_nationkey, tpch.nation.n_nationkey)] - │ │ │ │ ├─TableReader_81 500000.00 root data:TableScan_80 - │ │ │ │ │ └─TableScan_80 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false - │ │ │ │ └─TableReader_84 1.00 root data:Selection_83 - │ │ │ │ └─Selection_83 1.00 cop eq(tpch.nation.n_name, "EGYPT") - │ │ │ │ └─TableScan_82 25.00 cop table:nation, range:[-inf,+inf], keep order:false - │ │ │ └─TableReader_87 240004648.80 root data:Selection_86 - │ │ │ └─Selection_86 240004648.80 cop gt(l1.l_receiptdate, l1.l_commitdate) - │ │ │ └─TableScan_85 300005811.00 cop table:l1, range:[-inf,+inf], keep order:false - │ │ └─TableReader_68 36517371.00 root data:Selection_67 - │ │ └─Selection_67 36517371.00 cop eq(tpch.orders.o_orderstatus, "F") - │ │ └─TableScan_66 1.00 cop table:orders, range: decided by [l1.l_orderkey], keep order:false - │ └─IndexLookUp_62 1.00 root - │ ├─IndexScan_60 1.00 cop table:l2, index:L_ORDERKEY, L_LINENUMBER, range: decided by [l1.l_orderkey], keep order:false - │ └─TableScan_61 1.00 cop table:lineitem, keep order:false - └─IndexLookUp_46 240004648.80 root - ├─IndexScan_43 1.00 cop table:l3, index:L_ORDERKEY, L_LINENUMBER, range: decided by [l1.l_orderkey], keep order:false - └─Selection_45 240004648.80 cop gt(l3.l_receiptdate, l3.l_commitdate) - └─TableScan_44 1.00 cop table:lineitem, keep order:false +Projection_25 1.00 root tpch.supplier.s_name, 17_col_0 +└─TopN_28 1.00 root 17_col_0:desc, tpch.supplier.s_name:asc, offset:0, count:100 + └─HashAgg_34 1.00 root group by:tpch.supplier.s_name, funcs:count(1), firstrow(tpch.supplier.s_name) + └─IndexJoin_40 7828961.66 root anti semi join, inner:IndexLookUp_39, outer key:l1.l_orderkey, inner key:l3.l_orderkey, other cond:ne(l3.l_suppkey, l1.l_suppkey), ne(l3.l_suppkey, tpch.supplier.s_suppkey) + ├─IndexJoin_56 9786202.08 root semi join, inner:IndexLookUp_55, outer key:l1.l_orderkey, inner key:l2.l_orderkey, other cond:ne(l2.l_suppkey, l1.l_suppkey), ne(l2.l_suppkey, tpch.supplier.s_suppkey) + │ ├─IndexJoin_62 12232752.60 root inner join, inner:TableReader_61, outer key:l1.l_orderkey, inner key:tpch.orders.o_orderkey + │ │ ├─HashRightJoin_66 12232752.60 root inner join, inner:HashRightJoin_72, equal:[eq(tpch.supplier.s_suppkey, l1.l_suppkey)] + │ │ │ ├─HashRightJoin_72 20000.00 root inner join, inner:TableReader_77, equal:[eq(tpch.nation.n_nationkey, tpch.supplier.s_nationkey)] + │ │ │ │ ├─TableReader_77 1.00 root data:Selection_76 + │ │ │ │ │ └─Selection_76 1.00 cop eq(tpch.nation.n_name, "EGYPT") + │ │ │ │ │ └─TableScan_75 25.00 cop table:nation, range:[-inf,+inf], keep order:false + │ │ │ │ └─TableReader_74 500000.00 root data:TableScan_73 + │ │ │ │ └─TableScan_73 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false + │ │ │ └─TableReader_80 240004648.80 root data:Selection_79 + │ │ │ └─Selection_79 240004648.80 cop gt(l1.l_receiptdate, l1.l_commitdate) + │ │ │ └─TableScan_78 300005811.00 cop table:l1, range:[-inf,+inf], keep order:false + │ │ └─TableReader_61 36517371.00 root data:Selection_60 + │ │ └─Selection_60 36517371.00 cop eq(tpch.orders.o_orderstatus, "F") + │ │ └─TableScan_59 1.00 cop table:orders, range: decided by [l1.l_orderkey], keep order:false + │ └─IndexLookUp_55 1.00 root + │ ├─IndexScan_53 1.00 cop table:l2, index:L_ORDERKEY, L_LINENUMBER, range: decided by [l1.l_orderkey], keep order:false + │ └─TableScan_54 1.00 cop table:lineitem, keep order:false + └─IndexLookUp_39 240004648.80 root + ├─IndexScan_36 1.00 cop table:l3, index:L_ORDERKEY, L_LINENUMBER, range: decided by [l1.l_orderkey], keep order:false + └─Selection_38 240004648.80 cop gt(l3.l_receiptdate, l3.l_commitdate) + └─TableScan_37 1.00 cop table:lineitem, keep order:false /* Q22 Global Sales Opportunity Query The Global Sales Opportunity Query identifies geographies where there are customers who may be likely to make a diff --git a/planner/core/rule_join_reorder.go b/planner/core/rule_join_reorder.go index 3dfee2e196e57..90401c82397c1 100644 --- a/planner/core/rule_join_reorder.go +++ b/planner/core/rule_join_reorder.go @@ -79,10 +79,10 @@ func (s *joinReOrderSolver) optimizeRecursive(ctx sessionctx.Context, p LogicalP p, err = groupSolver.solve(curJoinGroup) } else { dpSolver := &joinReorderDPSolver{ - ctx: ctx, + baseSingleGroupJoinOrderSolver: baseGroupSolver, } - dpSolver.newJoin = dpSolver.newJoinWithConds - p, err = dpSolver.solve(curJoinGroup, expression.ScalarFuncs2Exprs(eqEdges), otherConds) + dpSolver.newJoin = dpSolver.newJoinWithEdges + p, err = dpSolver.solve(curJoinGroup, expression.ScalarFuncs2Exprs(eqEdges)) } if err != nil { return nil, err @@ -151,22 +151,15 @@ func (s *baseSingleGroupJoinOrderSolver) newCartesianJoin(lChild, rChild Logical return join } -func (s *baseSingleGroupJoinOrderSolver) newJoinWithEdges(eqEdges []*expression.ScalarFunction, remainedOtherConds []expression.Expression, - lChild, rChild LogicalPlan) (*LogicalJoin, []expression.Expression) { +func (s *baseSingleGroupJoinOrderSolver) newJoinWithEdges(lChild, rChild LogicalPlan, eqEdges []*expression.ScalarFunction, otherConds []expression.Expression) LogicalPlan { newJoin := s.newCartesianJoin(lChild, rChild) newJoin.EqualConditions = eqEdges + newJoin.OtherConditions = otherConds for _, eqCond := range newJoin.EqualConditions { newJoin.LeftJoinKeys = append(newJoin.LeftJoinKeys, eqCond.GetArgs()[0].(*expression.Column)) newJoin.RightJoinKeys = append(newJoin.RightJoinKeys, eqCond.GetArgs()[1].(*expression.Column)) } - for i := len(remainedOtherConds) - 1; i >= 0; i-- { - cols := expression.ExtractColumns(remainedOtherConds[i]) - if newJoin.schema.ColumnsIndices(cols) != nil { - newJoin.OtherConditions = append(newJoin.OtherConditions, remainedOtherConds[i]) - remainedOtherConds = append(remainedOtherConds[:i], remainedOtherConds[i+1:]...) - } - } - return newJoin, remainedOtherConds + return newJoin } // calcJoinCumCost calculates the cumulative cost of the join node. diff --git a/planner/core/rule_join_reorder_dp.go b/planner/core/rule_join_reorder_dp.go index 58a48b59d23c9..ab93e57e4dcd9 100644 --- a/planner/core/rule_join_reorder_dp.go +++ b/planner/core/rule_join_reorder_dp.go @@ -18,11 +18,10 @@ import ( "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/sessionctx" ) type joinReorderDPSolver struct { - ctx sessionctx.Context + *baseSingleGroupJoinOrderSolver newJoin func(lChild, rChild LogicalPlan, eqConds []*expression.ScalarFunction, otherConds []expression.Expression) LogicalPlan } @@ -37,8 +36,18 @@ type joinGroupNonEqEdge struct { expr expression.Expression } -func (s *joinReorderDPSolver) solve(joinGroup []LogicalPlan, eqConds, otherConds []expression.Expression) (LogicalPlan, error) { - adjacents := make([][]int, len(joinGroup)) +func (s *joinReorderDPSolver) solve(joinGroup []LogicalPlan, eqConds []expression.Expression) (LogicalPlan, error) { + for _, node := range joinGroup { + _, err := node.recursiveDeriveStats() + if err != nil { + return nil, err + } + s.curJoinGroup = append(s.curJoinGroup, &jrNode{ + p: node, + cumCost: s.baseNodeCumCost(node), + }) + } + adjacents := make([][]int, len(s.curJoinGroup)) totalEqEdges := make([]joinGroupEqEdge, 0, len(eqConds)) addEqEdge := func(node1, node2 int, edgeContent *expression.ScalarFunction) { totalEqEdges = append(totalEqEdges, joinGroupEqEdge{ @@ -63,8 +72,8 @@ func (s *joinReorderDPSolver) solve(joinGroup []LogicalPlan, eqConds, otherConds } addEqEdge(lIdx, rIdx, sf) } - totalNonEqEdges := make([]joinGroupNonEqEdge, 0, len(otherConds)) - for _, cond := range otherConds { + totalNonEqEdges := make([]joinGroupNonEqEdge, 0, len(s.otherConds)) + for _, cond := range s.otherConds { cols := expression.ExtractColumns(cond) mask := uint(0) ids := make([]int, 0, len(cols)) @@ -148,11 +157,10 @@ func (s *joinReorderDPSolver) bfsGraph(startNode int, visited []bool, adjacents func (s *joinReorderDPSolver) dpGraph(newPos2OldPos, oldPos2NewPos []int, joinGroup []LogicalPlan, totalEqEdges []joinGroupEqEdge, totalNonEqEdges []joinGroupNonEqEdge) (LogicalPlan, error) { nodeCnt := uint(len(newPos2OldPos)) - bestPlan := make([]LogicalPlan, 1< join.statsInfo().Count()+bestCost[remain]+bestCost[sub] { - bestPlan[nodeBitmap] = join - bestCost[nodeBitmap] = join.statsInfo().Count() + bestCost[remain] + bestCost[sub] + curCost := s.calcJoinCumCost(join, bestPlan[sub], bestPlan[remain]) + if bestPlan[nodeBitmap] == nil { + bestPlan[nodeBitmap] = &jrNode{ + p: join, + cumCost: curCost, + } + } else if bestPlan[nodeBitmap].cumCost > curCost { + bestPlan[nodeBitmap].p = join + bestPlan[nodeBitmap].cumCost = curCost } } } - return bestPlan[(1<= 0; i-- { + cols := expression.ExtractColumns(remainOtherConds[i]) + for _, col := range cols { + if !leftNode.Schema().Contains(col) && !rightNode.Schema().Contains(col) { + continue loopOtherCond + } + } + otherConds = append(otherConds, remainOtherConds[i]) + remainOtherConds = append(remainOtherConds[:i], remainOtherConds[i+1:]...) + } + return s.newJoinWithEdges(leftNode, rightNode, usedEdges, otherConds), remainOtherConds } diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index c82b3f97efa7b..e6414f92d690b 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -289,7 +289,7 @@ const ( DefTiDBForcePriority = mysql.NoPriority DefTiDBUseRadixJoin = false DefEnableWindowFunction = false - DefTiDBOptJoinOrderAlgoThreshold = 10 + DefTiDBOptJoinOrderAlgoThreshold = 0 ) // Process global variables. From b28a496aeb0e83d9f5b70672007d1ff344ad5305 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Mon, 4 Mar 2019 17:11:32 +0800 Subject: [PATCH 08/12] fix unit-test --- planner/core/logical_plan_test.go | 8 ++++---- planner/core/physical_plan_test.go | 6 +++--- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index 1362557061fad..bcdda3117817c 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -886,7 +886,7 @@ func (s *testPlanSuite) TestJoinReOrder(c *C) { }{ { sql: "select * from t t1, t t2, t t3, t t4, t t5, t t6 where t1.a = t2.b and t2.a = t3.b and t3.c = t4.a and t4.d = t2.c and t5.d = t6.d", - best: "Join{Join{Join{DataScan(t1)->DataScan(t2)}(t1.a,t2.b)->Join{DataScan(t4)->DataScan(t3)}(t4.a,t3.c)}(t2.c,t4.d)(t2.a,t3.b)->Join{DataScan(t5)->DataScan(t6)}(t5.d,t6.d)(t2.b,t1.a)}->Projection", + best: "Join{Join{Join{Join{DataScan(t1)->DataScan(t2)}(t1.a,t2.b)->DataScan(t3)}(t2.a,t3.b)->DataScan(t4)}(t3.c,t4.a)(t2.c,t4.d)->Join{DataScan(t5)->DataScan(t6)}(t5.d,t6.d)}->Projection", }, { sql: "select * from t t1, t t2, t t3, t t4, t t5, t t6, t t7, t t8 where t1.a = t8.a", @@ -894,11 +894,11 @@ func (s *testPlanSuite) TestJoinReOrder(c *C) { }, { sql: "select * from t t1, t t2, t t3, t t4, t t5 where t1.a = t5.a and t5.a = t4.a and t4.a = t3.a and t3.a = t2.a and t2.a = t1.a and t1.a = t3.a and t2.a = t4.a and t5.b < 8", - best: "Join{Join{Join{Join{DataScan(t1)->DataScan(t5)}(t1.a,t5.a)->DataScan(t3)}(t1.a,t3.a)->DataScan(t2)}(t3.a,t2.a)(t1.a,t2.a)->DataScan(t4)}(t5.a,t4.a)(t3.a,t4.a)(t2.a,t4.a)->Projection", + best: "Join{Join{Join{Join{DataScan(t5)->DataScan(t1)}(t5.a,t1.a)->DataScan(t2)}(t1.a,t2.a)->DataScan(t3)}(t2.a,t3.a)(t1.a,t3.a)->DataScan(t4)}(t5.a,t4.a)(t3.a,t4.a)(t2.a,t4.a)->Projection", }, { sql: "select * from t t1, t t2, t t3, t t4, t t5 where t1.a = t5.a and t5.a = t4.a and t4.a = t3.a and t3.a = t2.a and t2.a = t1.a and t1.a = t3.a and t2.a = t4.a and t3.b = 1 and t4.a = 1", - best: "Join{Join{Join{DataScan(t1)->DataScan(t2)}->Join{DataScan(t3)->DataScan(t4)}}->DataScan(t5)}->Projection", + best: "Join{Join{Join{DataScan(t3)->DataScan(t1)}->Join{DataScan(t2)->DataScan(t4)}}->DataScan(t5)}->Projection", }, { sql: "select * from t o where o.b in (select t3.c from t t1, t t2, t t3 where t1.a = t3.a and t2.a = t3.a and t2.a = o.a)", @@ -906,7 +906,7 @@ func (s *testPlanSuite) TestJoinReOrder(c *C) { }, { sql: "select * from t o where o.b in (select t3.c from t t1, t t2, t t3 where t1.a = t3.a and t2.a = t3.a and t2.a = o.a and t1.a = 1)", - best: "Apply{DataScan(o)->Join{Join{DataScan(t1)->DataScan(t2)}->DataScan(t3)}->Projection}->Projection", + best: "Apply{DataScan(o)->Join{Join{DataScan(t3)->DataScan(t1)}->DataScan(t2)}->Projection}->Projection", }, } for _, tt := range tests { diff --git a/planner/core/physical_plan_test.go b/planner/core/physical_plan_test.go index b0096a9ec1795..3bd71a09d0614 100644 --- a/planner/core/physical_plan_test.go +++ b/planner/core/physical_plan_test.go @@ -241,11 +241,11 @@ func (s *testPlanSuite) TestDAGPlanBuilderJoin(c *C) { }, { sql: "select * from t t1 join t t2 on t1.a = t2.a join t t3 on t1.a = t3.a", - best: "MergeInnerJoin{MergeInnerJoin{TableReader(Table(t))->TableReader(Table(t))}(t1.a,t3.a)->TableReader(Table(t))}(t1.a,t2.a)->Projection", + best: "MergeInnerJoin{MergeInnerJoin{TableReader(Table(t))->TableReader(Table(t))}(t1.a,t2.a)->TableReader(Table(t))}(t1.a,t3.a)", }, { sql: "select * from t t1 join t t2 on t1.a = t2.a join t t3 on t1.b = t3.a", - best: "LeftHashJoin{LeftHashJoin{TableReader(Table(t))->TableReader(Table(t))}(t1.b,t3.a)->TableReader(Table(t))}(t1.a,t2.a)->Projection", + best: "LeftHashJoin{MergeInnerJoin{TableReader(Table(t))->TableReader(Table(t))}(t1.a,t2.a)->TableReader(Table(t))}(t1.b,t3.a)", }, { sql: "select * from t t1 join t t2 on t1.b = t2.a order by t1.a", @@ -270,7 +270,7 @@ func (s *testPlanSuite) TestDAGPlanBuilderJoin(c *C) { }, { sql: "select * from t t1 join t t2 on t1.b = t2.b join t t3 on t1.b = t3.b", - best: "LeftHashJoin{LeftHashJoin{TableReader(Table(t))->TableReader(Table(t))}(t1.b,t3.b)->TableReader(Table(t))}(t1.b,t2.b)->Projection", + best: "LeftHashJoin{LeftHashJoin{TableReader(Table(t))->TableReader(Table(t))}(t1.b,t2.b)->TableReader(Table(t))}(t1.b,t3.b)", }, { sql: "select * from t t1 join t t2 on t1.a = t2.a order by t1.a", From 1eb891f5ab9ee27810d5288ef110ffdad35a92ea Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Wed, 6 Mar 2019 13:43:27 +0800 Subject: [PATCH 09/12] address comments --- planner/core/rule_join_reorder.go | 4 +- planner/core/rule_join_reorder_dp.go | 2 +- planner/core/rule_join_reorder_greedy.go | 20 ++++------ sessionctx/variable/session.go | 48 ++++++++++++------------ sessionctx/variable/sysvar.go | 2 +- sessionctx/variable/tidb_vars.go | 6 +-- sessionctx/variable/varsutil.go | 10 ++++- sessionctx/variable/varsutil_test.go | 10 ++--- 8 files changed, 53 insertions(+), 49 deletions(-) diff --git a/planner/core/rule_join_reorder.go b/planner/core/rule_join_reorder.go index 90401c82397c1..fac63d725cbb5 100644 --- a/planner/core/rule_join_reorder.go +++ b/planner/core/rule_join_reorder.go @@ -71,8 +71,8 @@ func (s *joinReOrderSolver) optimizeRecursive(ctx sessionctx.Context, p LogicalP ctx: ctx, otherConds: otherConds, } - if len(curJoinGroup) > ctx.GetSessionVars().TiDBOptJoinOrderAlgoThreshold { - groupSolver := &joinReorderGreedySingleGroupSolver{ + if len(curJoinGroup) > ctx.GetSessionVars().TiDBOptJoinReorderThreshold { + groupSolver := &joinReorderGreedySolver{ baseSingleGroupJoinOrderSolver: baseGroupSolver, eqEdges: eqEdges, } diff --git a/planner/core/rule_join_reorder_dp.go b/planner/core/rule_join_reorder_dp.go index ab93e57e4dcd9..5994e6bc14132 100644 --- a/planner/core/rule_join_reorder_dp.go +++ b/planner/core/rule_join_reorder_dp.go @@ -255,7 +255,7 @@ func (s *joinReorderDPSolver) makeBushyJoin(cartesianJoinGroup []LogicalPlan, ot resultJoinGroup = append(resultJoinGroup, cartesianJoinGroup[i]) break } - // TODO:Since the other condition may involve no less than two tables, e.g. t1.a = t2.b+t3.c. + // TODO:Since the other condition may involve more than two tables, e.g. t1.a = t2.b+t3.c. // So We'll need a extra stage to deal with it. // Currently, we just add it when building cartesianJoinGroup. mergedSchema := expression.MergeSchema(cartesianJoinGroup[i].Schema(), cartesianJoinGroup[i+1].Schema()) diff --git a/planner/core/rule_join_reorder_greedy.go b/planner/core/rule_join_reorder_greedy.go index 0cad645968ce8..e169845104ba4 100644 --- a/planner/core/rule_join_reorder_greedy.go +++ b/planner/core/rule_join_reorder_greedy.go @@ -21,7 +21,7 @@ import ( "github.com/pingcap/tidb/expression" ) -type joinReorderGreedySingleGroupSolver struct { +type joinReorderGreedySolver struct { *baseSingleGroupJoinOrderSolver eqEdges []*expression.ScalarFunction } @@ -40,7 +40,7 @@ type joinReorderGreedySingleGroupSolver 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 *joinReorderGreedySingleGroupSolver) solve(joinNodePlans []LogicalPlan) (LogicalPlan, error) { +func (s *joinReorderGreedySolver) solve(joinNodePlans []LogicalPlan) (LogicalPlan, error) { for _, node := range joinNodePlans { _, err := node.recursiveDeriveStats() if err != nil { @@ -67,7 +67,7 @@ func (s *joinReorderGreedySingleGroupSolver) solve(joinNodePlans []LogicalPlan) return s.makeBushyJoin(cartesianGroup), nil } -func (s *joinReorderGreedySingleGroupSolver) constructConnectedJoinTree() (*jrNode, error) { +func (s *joinReorderGreedySolver) constructConnectedJoinTree() (*jrNode, error) { curJoinTree := s.curJoinGroup[0] s.curJoinGroup = s.curJoinGroup[1:] for { @@ -106,7 +106,7 @@ func (s *joinReorderGreedySingleGroupSolver) constructConnectedJoinTree() (*jrNo return curJoinTree, nil } -func (s *joinReorderGreedySingleGroupSolver) checkConnectionAndMakeJoin(leftNode, rightNode LogicalPlan) (LogicalPlan, []expression.Expression) { +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) @@ -124,16 +124,12 @@ func (s *joinReorderGreedySingleGroupSolver) checkConnectionAndMakeJoin(leftNode return nil, nil } var otherConds []expression.Expression -loopOtherCond: + mergedSchema := expression.MergeSchema(leftNode.Schema(), rightNode.Schema()) for i := len(remainOtherConds) - 1; i >= 0; i-- { - cols := expression.ExtractColumns(remainOtherConds[i]) - for _, col := range cols { - if !leftNode.Schema().Contains(col) && !rightNode.Schema().Contains(col) { - continue loopOtherCond - } + if expression.ExprFromSchema(remainOtherConds[i], mergedSchema) { + otherConds = append(otherConds, remainOtherConds[i]) + remainOtherConds = append(remainOtherConds[:i], remainOtherConds[i+1:]...) } - otherConds = append(otherConds, remainOtherConds[i]) - remainOtherConds = append(remainOtherConds[:i], remainOtherConds[i+1:]...) } return s.newJoinWithEdges(leftNode, rightNode, usedEdges, otherConds), remainOtherConds } diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 8f0c8025057c4..a22ab3a5cb7a9 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -335,33 +335,33 @@ type SessionVars struct { // TIDBOptJoinOrderAlgoThreshold defines the threshold less than which // we'll choose a rather time consuming algorithm to calculate the join order. - TiDBOptJoinOrderAlgoThreshold int + TiDBOptJoinReorderThreshold int } // NewSessionVars creates a session vars object. func NewSessionVars() *SessionVars { vars := &SessionVars{ - Users: make(map[string]string), - systems: make(map[string]string), - PreparedStmts: make(map[uint32]*ast.Prepared), - PreparedStmtNameToID: make(map[string]uint32), - PreparedParams: make([]types.Datum, 0, 10), - TxnCtx: &TransactionContext{}, - KVVars: kv.NewVariables(), - RetryInfo: &RetryInfo{}, - StrictSQLMode: true, - Status: mysql.ServerStatusAutocommit, - StmtCtx: new(stmtctx.StatementContext), - AllowAggPushDown: false, - OptimizerSelectivityLevel: DefTiDBOptimizerSelectivityLevel, - RetryLimit: DefTiDBRetryLimit, - DisableTxnAutoRetry: DefTiDBDisableTxnAutoRetry, - DDLReorgPriority: kv.PriorityLow, - AllowInSubqToJoinAndAgg: DefOptInSubqToJoinAndAgg, - EnableRadixJoin: false, - L2CacheSize: cpuid.CPU.Cache.L2, - CommandValue: uint32(mysql.ComSleep), - TiDBOptJoinOrderAlgoThreshold: DefTiDBOptJoinOrderAlgoThreshold, + Users: make(map[string]string), + systems: make(map[string]string), + PreparedStmts: make(map[uint32]*ast.Prepared), + PreparedStmtNameToID: make(map[string]uint32), + PreparedParams: make([]types.Datum, 0, 10), + TxnCtx: &TransactionContext{}, + KVVars: kv.NewVariables(), + RetryInfo: &RetryInfo{}, + StrictSQLMode: true, + Status: mysql.ServerStatusAutocommit, + StmtCtx: new(stmtctx.StatementContext), + AllowAggPushDown: false, + OptimizerSelectivityLevel: DefTiDBOptimizerSelectivityLevel, + RetryLimit: DefTiDBRetryLimit, + DisableTxnAutoRetry: DefTiDBDisableTxnAutoRetry, + DDLReorgPriority: kv.PriorityLow, + AllowInSubqToJoinAndAgg: DefOptInSubqToJoinAndAgg, + EnableRadixJoin: false, + L2CacheSize: cpuid.CPU.Cache.L2, + CommandValue: uint32(mysql.ComSleep), + TiDBOptJoinReorderThreshold: DefTiDBOptJoinReorderThreshold, } vars.Concurrency = Concurrency{ IndexLookupConcurrency: DefIndexLookupConcurrency, @@ -694,8 +694,8 @@ func (s *SessionVars) SetSystemVar(name string, val string) error { s.EnableRadixJoin = TiDBOptOn(val) case TiDBEnableWindowFunction: s.EnableWindowFunction = TiDBOptOn(val) - case TiDBOptJoinOrderAlgoThreshold: - s.TiDBOptJoinOrderAlgoThreshold = tidbOptPositiveInt32(val, DefTiDBOptJoinOrderAlgoThreshold) + case TiDBOptJoinReorderThreshold: + s.TiDBOptJoinReorderThreshold = tidbOptPositiveInt32(val, DefTiDBOptJoinReorderThreshold) case TiDBCheckMb4ValueInUtf8: config.GetGlobalConfig().CheckMb4ValueInUtf8 = TiDBOptOn(val) } diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index b549e71abc2ae..bf0819aac6438 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -683,7 +683,7 @@ var defaultSysVars = []*SysVar{ {ScopeSession, TiDBDDLReorgPriority, "PRIORITY_LOW"}, {ScopeSession, TiDBForcePriority, mysql.Priority2Str[DefTiDBForcePriority]}, {ScopeSession, TiDBEnableRadixJoin, BoolToIntStr(DefTiDBUseRadixJoin)}, - {ScopeGlobal | ScopeSession, TiDBOptJoinOrderAlgoThreshold, strconv.Itoa(DefTiDBOptJoinOrderAlgoThreshold)}, + {ScopeGlobal | ScopeSession, TiDBOptJoinReorderThreshold, strconv.Itoa(DefTiDBOptJoinReorderThreshold)}, {ScopeSession, TiDBCheckMb4ValueInUtf8, BoolToIntStr(config.GetGlobalConfig().CheckMb4ValueInUtf8)}, } diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index e6414f92d690b..525316975b1d8 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -235,9 +235,9 @@ const ( // tidb_enable_window_function is used to control whether to enable the window function. TiDBEnableWindowFunction = "tidb_enable_window_function" - // TIDBOptJoinOrderAlgoThreshold defines the threshold less than which + // TIDBOptJoinReorderThreshold defines the threshold less than which // we'll choose a rather time consuming algorithm to calculate the join order. - TiDBOptJoinOrderAlgoThreshold = "tidb_opt_join_order_algo_threshold" + TiDBOptJoinReorderThreshold = "tidb_opt_join_reorder_threshold" ) // Default TiDB system variable values. @@ -289,7 +289,7 @@ const ( DefTiDBForcePriority = mysql.NoPriority DefTiDBUseRadixJoin = false DefEnableWindowFunction = false - DefTiDBOptJoinOrderAlgoThreshold = 0 + DefTiDBOptJoinReorderThreshold = 0 ) // Process global variables. diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index 3b3f6d341b56b..310f0ad46f3ef 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -386,7 +386,7 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string) (string, TiDBDistSQLScanConcurrency, TiDBIndexSerialScanConcurrency, TiDBDDLReorgWorkerCount, TiDBBackoffLockFast, - TiDBDMLBatchSize, TiDBOptimizerSelectivityLevel, TiDBOptJoinOrderAlgoThreshold: + TiDBDMLBatchSize, TiDBOptimizerSelectivityLevel: v, err := strconv.Atoi(value) if err != nil { return value, ErrWrongTypeForVar.GenWithStackByArgs(name) @@ -450,6 +450,14 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string) (string, return value, errors.Errorf("tidb_max_chunk_size(%d) cannot be smaller than %d", v, maxChunkSizeLowerBound) } return value, nil + case TiDBOptJoinReorderThreshold: + v, err := strconv.Atoi(value) + if err != nil { + return value, ErrWrongTypeForVar.GenWithStackByArgs(name) + } + if v < 0 || v > 64 { + return value, errors.Errorf("tidb_join_order_algo_threshold cannot be smaller than 0 and larger than 63") + } } return value, nil } diff --git a/sessionctx/variable/varsutil_test.go b/sessionctx/variable/varsutil_test.go index c5e89da52dce7..ded206b1ec1bd 100644 --- a/sessionctx/variable/varsutil_test.go +++ b/sessionctx/variable/varsutil_test.go @@ -79,7 +79,7 @@ func (s *testVarsutilSuite) TestNewSessionVars(c *C) { c.Assert(vars.MemQuotaNestedLoopApply, Equals, int64(DefTiDBMemQuotaNestedLoopApply)) c.Assert(vars.EnableRadixJoin, Equals, DefTiDBUseRadixJoin) c.Assert(vars.AllowWriteRowID, Equals, DefOptWriteRowID) - c.Assert(vars.TiDBOptJoinOrderAlgoThreshold, Equals, DefTiDBOptJoinOrderAlgoThreshold) + c.Assert(vars.TiDBOptJoinReorderThreshold, Equals, DefTiDBOptJoinReorderThreshold) assertFieldsGreaterThanZero(c, reflect.ValueOf(vars.Concurrency)) assertFieldsGreaterThanZero(c, reflect.ValueOf(vars.MemQuota)) @@ -238,13 +238,13 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { c.Assert(val, Equals, "on") c.Assert(v.EnableTablePartition, Equals, "on") - c.Assert(v.TiDBOptJoinOrderAlgoThreshold, Equals, DefTiDBOptJoinOrderAlgoThreshold) - err = SetSessionSystemVar(v, TiDBOptJoinOrderAlgoThreshold, types.NewIntDatum(5)) + c.Assert(v.TiDBOptJoinReorderThreshold, Equals, DefTiDBOptJoinReorderThreshold) + err = SetSessionSystemVar(v, TiDBOptJoinReorderThreshold, types.NewIntDatum(5)) c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBOptJoinOrderAlgoThreshold) + val, err = GetSessionSystemVar(v, TiDBOptJoinReorderThreshold) c.Assert(err, IsNil) c.Assert(val, Equals, "5") - c.Assert(v.TiDBOptJoinOrderAlgoThreshold, Equals, 5) + c.Assert(v.TiDBOptJoinReorderThreshold, Equals, 5) SetSessionSystemVar(v, TiDBCheckMb4ValueInUtf8, types.NewStringDatum("1")) val, err = GetSessionSystemVar(v, TiDBCheckMb4ValueInUtf8) From a0b366122430c45952318ba142a499ed912a5050 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Mon, 11 Mar 2019 17:38:20 +0800 Subject: [PATCH 10/12] address comments --- planner/core/rule_join_reorder_dp.go | 31 +++++++++++++++------------- sessionctx/variable/session.go | 4 ++-- sessionctx/variable/varsutil.go | 4 ++-- 3 files changed, 21 insertions(+), 18 deletions(-) diff --git a/planner/core/rule_join_reorder_dp.go b/planner/core/rule_join_reorder_dp.go index 5994e6bc14132..6574d9810bdae 100644 --- a/planner/core/rule_join_reorder_dp.go +++ b/planner/core/rule_join_reorder_dp.go @@ -31,9 +31,9 @@ type joinGroupEqEdge struct { } type joinGroupNonEqEdge struct { - nodeIDs []int - idMask uint - expr expression.Expression + nodeIDs []int + nodeIDMask uint + expr expression.Expression } func (s *joinReorderDPSolver) solve(joinGroup []LogicalPlan, eqConds []expression.Expression) (LogicalPlan, error) { @@ -86,9 +86,9 @@ func (s *joinReorderDPSolver) solve(joinGroup []LogicalPlan, eqConds []expressio mask |= 1 << uint(idx) } totalNonEqEdges = append(totalNonEqEdges, joinGroupNonEqEdge{ - nodeIDs: ids, - idMask: mask, - expr: cond, + nodeIDs: ids, + nodeIDMask: mask, + expr: cond, }) } visited := make([]bool, len(joinGroup)) @@ -107,14 +107,14 @@ func (s *joinReorderDPSolver) solve(joinGroup []LogicalPlan, eqConds []expressio var subNonEqEdges []joinGroupNonEqEdge for i := len(totalNonEqEdges) - 1; i >= 0; i-- { // If this edge is not the subset of the current sub graph. - if totalNonEqEdges[i].idMask&nodeIDMask != totalNonEqEdges[i].idMask { + if totalNonEqEdges[i].nodeIDMask&nodeIDMask != totalNonEqEdges[i].nodeIDMask { continue } newMask := uint(0) for _, nodeID := range totalNonEqEdges[i].nodeIDs { newMask |= 1 << uint(nodeID2VisitID[nodeID]) } - totalNonEqEdges[i].idMask = newMask + totalNonEqEdges[i].nodeIDMask = newMask subNonEqEdges = append(subNonEqEdges, totalNonEqEdges[i]) totalNonEqEdges = append(totalNonEqEdges[:i], totalNonEqEdges[i+1:]...) } @@ -154,13 +154,16 @@ func (s *joinReorderDPSolver) bfsGraph(startNode int, visited []bool, adjacents return visitID2NodeID } -func (s *joinReorderDPSolver) dpGraph(newPos2OldPos, oldPos2NewPos []int, joinGroup []LogicalPlan, +// dpGraph is the core part of this algorithm. +// It implements the traditional join reorder algorithm: DP by subset using the following formula: +// bestPlan[S:set of node] = the best one among Join(bestPlan[S1:subset of S], bestPlan[S2: S/S1]) +func (s *joinReorderDPSolver) dpGraph(visitID2NodeID, nodeID2VisitID []int, joinGroup []LogicalPlan, totalEqEdges []joinGroupEqEdge, totalNonEqEdges []joinGroupNonEqEdge) (LogicalPlan, error) { - nodeCnt := uint(len(newPos2OldPos)) + nodeCnt := uint(len(visitID2NodeID)) bestPlan := make([]*jrNode, 1< 64 { - return value, errors.Errorf("tidb_join_order_algo_threshold cannot be smaller than 0 and larger than 63") + if v < 0 || v >= 64 { + return value, errors.Errorf("tidb_join_order_algo_threshold(%d) cannot be smaller than 0 or larger than 63", v) } } return value, nil From 3f116ee9bc8ba636e1a1c23015974f740c43daed Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Mon, 11 Mar 2019 20:31:06 +0800 Subject: [PATCH 11/12] address comments --- expression/util.go | 13 +++++++++++++ expression/util_test.go | 14 ++++++++++++++ planner/core/rule_join_reorder_dp.go | 9 +++------ planner/core/rule_join_reorder_greedy.go | 9 +++------ 4 files changed, 33 insertions(+), 12 deletions(-) diff --git a/expression/util.go b/expression/util.go index 98220b9275fb6..081a1215ccede 100644 --- a/expression/util.go +++ b/expression/util.go @@ -41,6 +41,19 @@ func Filter(result []Expression, input []Expression, filter func(Expression) boo return result } +// FilterOutInPlace do the filtering out in place. +// The remained are the ones who doesn't match the filter, storing in the original slice. +// The filteredOut are the ones match the filter, storing in a new slice. +func FilterOutInPlace(input []Expression, filter func(Expression) bool) (remained, filteredOut []Expression) { + for i := len(input) - 1; i >= 0; i-- { + if filter(input[i]) { + filteredOut = append(filteredOut, input[i]) + input = append(input[:i], input[i+1:]...) + } + } + return input, filteredOut +} + // ExtractColumns extracts all columns from an expression. func ExtractColumns(expr Expression) (cols []*Column) { // Pre-allocate a slice to reduce allocation, 8 doesn't have special meaning. diff --git a/expression/util_test.go b/expression/util_test.go index f1e03ec971226..1fa987418cd94 100644 --- a/expression/util_test.go +++ b/expression/util_test.go @@ -83,6 +83,20 @@ func (s *testUtilSuite) TestFilter(c *check.C) { c.Assert(result, check.HasLen, 1) } +func (s *testUtilSuite) TestFilterOutInPlace(c *check.C) { + conditions := []Expression{ + newFunction(ast.EQ, newColumn(0), newColumn(1)), + newFunction(ast.EQ, newColumn(1), newColumn(2)), + newFunction(ast.LogicOr, newLonglong(1), newColumn(0)), + } + remained, filtered := FilterOutInPlace(conditions, isLogicOrFunction) + c.Assert(len(remained), check.Equals, 2) + c.Assert(remained[0].(*ScalarFunction).FuncName.L, check.Equals, "eq") + c.Assert(remained[1].(*ScalarFunction).FuncName.L, check.Equals, "eq") + c.Assert(len(filtered), check.Equals, 1) + c.Assert(filtered[0].(*ScalarFunction).FuncName.L, check.Equals, "or") +} + func isLogicOrFunction(e Expression) bool { if f, ok := e.(*ScalarFunction); ok { return f.FuncName.L == ast.LogicOr diff --git a/planner/core/rule_join_reorder_dp.go b/planner/core/rule_join_reorder_dp.go index 6574d9810bdae..18b549d7813c8 100644 --- a/planner/core/rule_join_reorder_dp.go +++ b/planner/core/rule_join_reorder_dp.go @@ -263,12 +263,9 @@ func (s *joinReorderDPSolver) makeBushyJoin(cartesianJoinGroup []LogicalPlan, ot // Currently, we just add it when building cartesianJoinGroup. mergedSchema := expression.MergeSchema(cartesianJoinGroup[i].Schema(), cartesianJoinGroup[i+1].Schema()) var usedOtherConds []expression.Expression - for i := len(otherConds) - 1; i >= 0; i-- { - if expression.ExprFromSchema(otherConds[i], mergedSchema) { - usedOtherConds = append(usedOtherConds, otherConds[i]) - otherConds = append(otherConds[:i], otherConds[i+1:]...) - } - } + otherConds, usedOtherConds = expression.FilterOutInPlace(otherConds, func(expr expression.Expression) bool { + return expression.ExprFromSchema(expr, mergedSchema) + }) resultJoinGroup = append(resultJoinGroup, s.newJoin(cartesianJoinGroup[i], cartesianJoinGroup[i+1], nil, usedOtherConds)) } cartesianJoinGroup = resultJoinGroup diff --git a/planner/core/rule_join_reorder_greedy.go b/planner/core/rule_join_reorder_greedy.go index e169845104ba4..349a8437caa9f 100644 --- a/planner/core/rule_join_reorder_greedy.go +++ b/planner/core/rule_join_reorder_greedy.go @@ -125,11 +125,8 @@ func (s *joinReorderGreedySolver) checkConnectionAndMakeJoin(leftNode, rightNode } var otherConds []expression.Expression mergedSchema := expression.MergeSchema(leftNode.Schema(), rightNode.Schema()) - for i := len(remainOtherConds) - 1; i >= 0; i-- { - if expression.ExprFromSchema(remainOtherConds[i], mergedSchema) { - otherConds = append(otherConds, remainOtherConds[i]) - remainOtherConds = append(remainOtherConds[:i], remainOtherConds[i+1:]...) - } - } + remainOtherConds, otherConds = expression.FilterOutInPlace(otherConds, func(expr expression.Expression) bool { + return expression.ExprFromSchema(expr, mergedSchema) + }) return s.newJoinWithEdges(leftNode, rightNode, usedEdges, otherConds), remainOtherConds } From 99c45c2db02271711b66858b0da9dc16df8b1488 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Fri, 15 Mar 2019 15:37:44 +0800 Subject: [PATCH 12/12] fix small bug --- 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 349a8437caa9f..6bdf993ec54ba 100644 --- a/planner/core/rule_join_reorder_greedy.go +++ b/planner/core/rule_join_reorder_greedy.go @@ -125,7 +125,7 @@ func (s *joinReorderGreedySolver) checkConnectionAndMakeJoin(leftNode, rightNode } var otherConds []expression.Expression mergedSchema := expression.MergeSchema(leftNode.Schema(), rightNode.Schema()) - remainOtherConds, otherConds = expression.FilterOutInPlace(otherConds, func(expr expression.Expression) bool { + remainOtherConds, otherConds = expression.FilterOutInPlace(remainOtherConds, func(expr expression.Expression) bool { return expression.ExprFromSchema(expr, mergedSchema) }) return s.newJoinWithEdges(leftNode, rightNode, usedEdges, otherConds), remainOtherConds