Skip to content

Commit

Permalink
revert hint scope & remove test (#11740)
Browse files Browse the repository at this point in the history
  • Loading branch information
foreyes committed Aug 15, 2019
1 parent 1b13a2a commit f866bb4
Show file tree
Hide file tree
Showing 2 changed files with 23 additions and 104 deletions.
39 changes: 23 additions & 16 deletions planner/core/logical_plan_builder.go
Expand Up @@ -1942,7 +1942,7 @@ func (b *PlanBuilder) unfoldWildStar(p LogicalPlan, selectFields []*ast.SelectFi
return resultList, nil
}

func (b *PlanBuilder) pushTableHints(hints []*ast.TableOptimizerHint) {
func (b *PlanBuilder) pushTableHints(hints []*ast.TableOptimizerHint) bool {
var sortMergeTables, INLJTables, hashJoinTables []hintTableInfo
var preferAggType uint
for _, hint := range hints {
Expand All @@ -1961,12 +1961,16 @@ func (b *PlanBuilder) pushTableHints(hints []*ast.TableOptimizerHint) {
// ignore hints that not implemented
}
}
b.tableHintInfo = append(b.tableHintInfo, tableHintInfo{
sortMergeJoinTables: sortMergeTables,
indexNestedLoopJoinTables: INLJTables,
hashJoinTables: hashJoinTables,
preferAggType: preferAggType,
})
if len(sortMergeTables)+len(INLJTables)+len(hashJoinTables) > 0 || preferAggType != 0 {
b.tableHintInfo = append(b.tableHintInfo, tableHintInfo{
sortMergeJoinTables: sortMergeTables,
indexNestedLoopJoinTables: INLJTables,
hashJoinTables: hashJoinTables,
preferAggType: preferAggType,
})
return true
}
return false
}

func (b *PlanBuilder) popTableHints() {
Expand Down Expand Up @@ -1996,9 +2000,10 @@ func (b *PlanBuilder) TableHints() *tableHintInfo {
}

func (b *PlanBuilder) buildSelect(ctx context.Context, sel *ast.SelectStmt) (p LogicalPlan, err error) {
b.pushTableHints(sel.TableHints)
// table hints are only visible in the current SELECT statement.
defer b.popTableHints()
if b.pushTableHints(sel.TableHints) {
// table hints are only visible in the current SELECT statement.
defer b.popTableHints()
}

if sel.SelectStmtOpts != nil {
origin := b.inStraightJoin
Expand Down Expand Up @@ -2622,9 +2627,10 @@ func buildColumns2Handle(
}

func (b *PlanBuilder) buildUpdate(ctx context.Context, update *ast.UpdateStmt) (Plan, error) {
b.pushTableHints(update.TableHints)
// table hints are only visible in the current UPDATE statement.
defer b.popTableHints()
if b.pushTableHints(update.TableHints) {
// table hints are only visible in the current UPDATE statement.
defer b.popTableHints()
}

// update subquery table should be forbidden
var asNameList []string
Expand Down Expand Up @@ -2842,9 +2848,10 @@ func extractTableAsNameForUpdate(p LogicalPlan, asNames map[*model.TableInfo][]*
}

func (b *PlanBuilder) buildDelete(ctx context.Context, delete *ast.DeleteStmt) (Plan, error) {
b.pushTableHints(delete.TableHints)
// table hints are only visible in the current DELETE statement.
defer b.popTableHints()
if b.pushTableHints(delete.TableHints) {
// table hints are only visible in the current DELETE statement.
defer b.popTableHints()
}

p, err := b.buildResultSetNode(ctx, delete.TableRefs.TableRefs)
if err != nil {
Expand Down
88 changes: 0 additions & 88 deletions planner/core/physical_plan_test.go
Expand Up @@ -1653,94 +1653,6 @@ func (s *testPlanSuite) TestAggregationHints(c *C) {
}
}

func (s *testPlanSuite) TestHintScope(c *C) {
defer testleak.AfterTest(c)()
store, dom, err := newStoreWithBootstrap()
c.Assert(err, IsNil)
defer func() {
dom.Close()
store.Close()
}()
se, err := session.CreateSession4Test(store)
c.Assert(err, IsNil)
_, err = se.Execute(context.Background(), "use test")
c.Assert(err, IsNil)

tests := []struct {
sql string
best string
}{
// join hints
{
sql: "select /*+ TIDB_SMJ(t1) */ t1.a, t1.b from t t1, (select /*+ TIDB_INLJ(t3) */ t2.a from t t2, t t3 where t2.a = t3.c) s where t1.a=s.a",
best: "MergeInnerJoin{TableReader(Table(t))->IndexJoin{TableReader(Table(t))->IndexReader(Index(t.c_d_e)[[NULL,+inf]])}(test.t2.a,test.t3.c)}(test.t1.a,test.t2.a)->Projection",
},
{
sql: "select /*+ TIDB_SMJ(t1) */ t1.a, t1.b from t t1, (select /*+ TIDB_HJ(t2) */ t2.a from t t2, t t3 where t2.a = t3.c) s where t1.a=s.a",
best: "MergeInnerJoin{TableReader(Table(t))->LeftHashJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t2.a,test.t3.c)->Sort}(test.t1.a,test.t2.a)->Projection",
},
{
sql: "select /*+ TIDB_INLJ(t1) */ t1.a, t1.b from t t1, (select /*+ TIDB_HJ(t2) */ t2.a from t t2, t t3 where t2.a = t3.c) s where t1.a=s.a",
best: "IndexJoin{TableReader(Table(t))->LeftHashJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t2.a,test.t3.c)}(test.t2.a,test.t1.a)->Projection",
},
{
sql: "select /*+ TIDB_INLJ(t1) */ t1.a, t1.b from t t1, (select /*+ TIDB_SMJ(t2) */ t2.a from t t2, t t3 where t2.a = t3.c) s where t1.a=s.a",
best: "IndexJoin{TableReader(Table(t))->MergeInnerJoin{TableReader(Table(t))->IndexReader(Index(t.c_d_e)[[NULL,+inf]])}(test.t2.a,test.t3.c)}(test.t2.a,test.t1.a)->Projection",
},
{
sql: "select /*+ TIDB_HJ(t1) */ t1.a, t1.b from t t1, (select /*+ TIDB_SMJ(t2) */ t2.a from t t2, t t3 where t2.a = t3.c) s where t1.a=s.a",
best: "RightHashJoin{TableReader(Table(t))->MergeInnerJoin{TableReader(Table(t))->IndexReader(Index(t.c_d_e)[[NULL,+inf]])}(test.t2.a,test.t3.c)}(test.t1.a,test.t2.a)->Projection",
},
{
sql: "select /*+ TIDB_HJ(t1) */ t1.a, t1.b from t t1, (select /*+ TIDB_INLJ(t2) */ t2.a from t t2, t t3 where t2.a = t3.c) s where t1.a=s.a",
best: "RightHashJoin{TableReader(Table(t))->IndexJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t3.c,test.t2.a)}(test.t1.a,test.t2.a)->Projection",
},
{
sql: "select /*+ TIDB_SMJ(t1) */ t1.a, t1.b from t t1, (select t2.a from t t2, t t3 where t2.a = t3.c) s where t1.a=s.a",
best: "MergeInnerJoin{TableReader(Table(t))->MergeInnerJoin{TableReader(Table(t))->IndexReader(Index(t.c_d_e)[[NULL,+inf]])}(test.t2.a,test.t3.c)}(test.t1.a,test.t2.a)->Projection",
},
{
sql: "select /*+ TIDB_INLJ(t1) */ t1.a, t1.b from t t1, (select t2.a from t t2, t t3 where t2.a = t3.c) s where t1.a=s.a",
best: "IndexJoin{TableReader(Table(t))->MergeInnerJoin{TableReader(Table(t))->IndexReader(Index(t.c_d_e)[[NULL,+inf]])}(test.t2.a,test.t3.c)}(test.t2.a,test.t1.a)->Projection",
},
{
sql: "select /*+ TIDB_HJ(t1) */ t1.a, t1.b from t t1, (select t2.a from t t2, t t3 where t2.a = t3.c) s where t1.a=s.a",
best: "RightHashJoin{TableReader(Table(t))->MergeInnerJoin{TableReader(Table(t))->IndexReader(Index(t.c_d_e)[[NULL,+inf]])}(test.t2.a,test.t3.c)}(test.t1.a,test.t2.a)->Projection",
},
// aggregation hints
{
sql: "select /*+ STREAM_AGG */ s, count(s) from (select /*+ HASH_AGG */ sum(t1.a) as s from t t1, t t2 where t1.a = t2.b group by t1.a) p group by s",
best: "LeftHashJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t1.a,test.t2.b)->Projection->HashAgg->Sort->StreamAgg->Projection",
},
{
sql: "select /*+ HASH_AGG */ s, count(s) from (select /*+ STREAM_AGG */ sum(t1.a) as s from t t1, t t2 where t1.a = t2.b group by t1.a) p group by s",
best: "LeftHashJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t1.a,test.t2.b)->Sort->Projection->StreamAgg->HashAgg->Projection",
},
{
sql: "select /*+ HASH_AGG */ s, count(s) from (select sum(t1.a) as s from t t1, t t2 where t1.a = t2.b group by t1.a) p group by s",
best: "LeftHashJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t1.a,test.t2.b)->Projection->HashAgg->HashAgg->Projection",
},
{
sql: "select /*+ STREAM_AGG */ s, count(s) from (select sum(t1.a) as s from t t1, t t2 where t1.a = t2.b group by t1.a) p group by s",
best: "LeftHashJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t1.a,test.t2.b)->Projection->HashAgg->Sort->StreamAgg->Projection",
},
}

ctx := context.Background()
for i, test := range tests {
comment := Commentf("case:%v sql:%s", i, test)
stmt, err := s.ParseOneStmt(test.sql, "", "")
c.Assert(err, IsNil, comment)

p, err := planner.Optimize(ctx, se, stmt, s.is)
c.Assert(err, IsNil)
c.Assert(core.ToString(p), Equals, test.best)

warnings := se.GetSessionVars().StmtCtx.GetWarnings()
c.Assert(warnings, HasLen, 0, comment)
}
}

func (s *testPlanSuite) TestHintAlias(c *C) {
defer testleak.AfterTest(c)()
store, dom, err := newStoreWithBootstrap()
Expand Down

0 comments on commit f866bb4

Please sign in to comment.