Skip to content

Commit

Permalink
plan: support subquery in Do statement (#8343) (#9879)
Browse files Browse the repository at this point in the history
  • Loading branch information
eurekaka authored and zz-jason committed Mar 27, 2019
1 parent 9ff5ee0 commit 4f6eef9
Show file tree
Hide file tree
Showing 5 changed files with 64 additions and 12 deletions.
13 changes: 13 additions & 0 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -3067,3 +3067,16 @@ func (s *testSuite) TestCurrentTimestampValueSelection(c *C) {
c.Assert(strings.Split(b, ".")[1], Equals, "00")
c.Assert(len(strings.Split(d, ".")[1]), Equals, 3)
}

func (s *testSuite) TestDoSubquery(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec(`use test`)
tk.MustExec(`drop table if exists t`)
tk.MustExec(`create table t(a int)`)
_, err := tk.Exec(`do 1 in (select * from t)`)
c.Assert(err, IsNil, Commentf("err %v", err))
tk.MustExec(`insert into t values(1)`)
r, err := tk.Exec(`do 1 in (select * from t)`)
c.Assert(err, IsNil, Commentf("err %v", err))
c.Assert(r, IsNil, Commentf("result of Do not empty"))
}
5 changes: 3 additions & 2 deletions plan/column_pruning.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,11 +14,12 @@
package plan

import (
"fmt"

"github.com/pingcap/tidb/ast"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/model"
log "github.com/sirupsen/logrus"
)

type columnPruner struct {
Expand All @@ -34,7 +35,7 @@ func getUsedList(usedCols []*expression.Column, schema *expression.Schema) []boo
for _, col := range usedCols {
idx := schema.ColumnIndex(col)
if idx == -1 {
log.Errorf("Can't find column %s from schema %s.", col, schema)
panic(fmt.Sprintf("Can't find column %s from schema %s.", col, schema))
}
used[idx] = true
}
Expand Down
4 changes: 4 additions & 0 deletions plan/eliminate_projection.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,10 @@ func canProjectionBeEliminatedLoose(p *LogicalProjection) bool {
// canProjectionBeEliminatedStrict checks whether a projection can be eliminated, returns true if
// the projection just copy its child's output.
func canProjectionBeEliminatedStrict(p *PhysicalProjection) bool {
// If this projection is specially added for `DO`, we keep it.
if p.CalculateNoDelay == true {
return false
}
if p.Schema().Len() == 0 {
return true
}
Expand Down
31 changes: 31 additions & 0 deletions plan/physical_plan_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1273,3 +1273,34 @@ func (s *testPlanSuite) TestIndexLookupCartesianJoin(c *C) {
err = plan.ErrInternal.GenByArgs("TIDB_INLJ hint is inapplicable without column equal ON condition")
c.Assert(terror.ErrorEqual(err, lastWarn.Err), IsTrue)
}

func (s *testPlanSuite) TestDoSubquery(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
}{
{
sql: "do 1 in (select a from t)",
best: "LeftHashJoin{Dual->TableReader(Table(t))}->Projection",
},
}
for _, tt := range tests {
comment := Commentf("for %s", tt.sql)
stmt, err := s.ParseOneStmt(tt.sql, "", "")
c.Assert(err, IsNil, comment)
p, err := plan.Optimize(se, stmt, s.is)
c.Assert(err, IsNil)
c.Assert(plan.ToString(p), Equals, tt.best, comment)
}
}
23 changes: 13 additions & 10 deletions plan/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -192,28 +192,31 @@ func (b *planBuilder) buildExecute(v *ast.ExecuteStmt) Plan {
}

func (b *planBuilder) buildDo(v *ast.DoStmt) Plan {
var p LogicalPlan
dual := LogicalTableDual{RowCount: 1}.init(b.ctx)

p := LogicalProjection{Exprs: make([]expression.Expression, 0, len(v.Exprs))}.init(b.ctx)
dual.SetSchema(expression.NewSchema())
p = dual
proj := LogicalProjection{Exprs: make([]expression.Expression, 0, len(v.Exprs))}.init(b.ctx)
schema := expression.NewSchema(make([]*expression.Column, 0, len(v.Exprs))...)
for _, astExpr := range v.Exprs {
expr, _, err := b.rewrite(astExpr, dual, nil, true)
expr, np, err := b.rewrite(astExpr, p, nil, true)
if err != nil {
b.err = errors.Trace(err)
return nil
}
p.Exprs = append(p.Exprs, expr)
p = np
proj.Exprs = append(proj.Exprs, expr)
schema.Append(&expression.Column{
FromID: p.id,
FromID: p.ID(),
Position: schema.Len() + 1,
RetType: expr.GetType(),
})
}
p.SetChildren(dual)
p.self = p
p.SetSchema(schema)
p.calculateNoDelay = true
return p
proj.SetChildren(p)
proj.self = proj
proj.SetSchema(schema)
proj.calculateNoDelay = true
return proj
}

func (b *planBuilder) buildSet(v *ast.SetStmt) Plan {
Expand Down

0 comments on commit 4f6eef9

Please sign in to comment.