Skip to content

Commit

Permalink
executor, optimizer: new executor and optimizer implementation
Browse files Browse the repository at this point in the history
  • Loading branch information
coocood committed Nov 13, 2015
1 parent 7010679 commit c01af56
Show file tree
Hide file tree
Showing 26 changed files with 1,376 additions and 183 deletions.
17 changes: 14 additions & 3 deletions ast/ast.go
Original file line number Diff line number Diff line change
Expand Up @@ -80,17 +80,28 @@ type DMLNode interface {
dmlStatement()
}

// ResultField is computed from a parsed select statement.
// ResultField represents a result field which can be a column from a table,
// or an expression in select field. It is a generated property during
// binding process. ResultField is the key element to evaluate an ColumnNameExpr.
// After binding process, every ColumnNameExpr will be resolved to a ResultField.
// During execution, every row retrieved from table will set the row value to
// ResultField, so ColumnNameExpr resolved to that ResultField can be easily evaluated.
type ResultField struct {
Column *model.ColumnInfo
ColumnAsName model.CIStr
Table *model.TableInfo
TableAsName model.CIStr
DBName model.CIStr

// The expression for the result field. If it is generated from a select field, it would
// be the expression of that select field, otherwise the type would be ValueExpr and value
// will be set for every retrieved row.
Expr ExprNode
}

// ResultSetNode interface has ResultFields property which is computed and set by visitor.
// Implementations include SelectStmt, SubqueryExpr, TableSource, TableName and Join.
// ResultSetNode interface has ResultFields property which is computed and set by
// optimizer.InfoBinder during binding process. Implementations include SelectStmt,
// SubqueryExpr, TableSource, TableName and Join.
type ResultSetNode interface {
Node
// GetResultFields gets result fields of the result set node.
Expand Down
3 changes: 0 additions & 3 deletions ast/cloner.go
Original file line number Diff line number Diff line change
Expand Up @@ -65,9 +65,6 @@ func copyStruct(in Node) (out Node) {
case *DefaultExpr:
nv := *v
out = &nv
case *IdentifierExpr:
nv := *v
out = &nv
case *ExistsSubqueryExpr:
nv := *v
out = &nv
Expand Down
26 changes: 4 additions & 22 deletions ast/expressions.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,6 @@ var (
_ Node = &ColumnName{}
_ ExprNode = &ColumnNameExpr{}
_ ExprNode = &DefaultExpr{}
_ ExprNode = &IdentifierExpr{}
_ ExprNode = &ExistsSubqueryExpr{}
_ ExprNode = &PatternInExpr{}
_ ExprNode = &IsNullExpr{}
Expand Down Expand Up @@ -358,10 +357,6 @@ type ColumnName struct {
Schema model.CIStr
Table model.CIStr
Name model.CIStr

DBInfo *model.DBInfo
TableInfo *model.TableInfo
ColumnInfo *model.ColumnInfo
}

// Accept implements Node Accept interface.
Expand All @@ -380,6 +375,10 @@ type ColumnNameExpr struct {

// Name is the referenced column name.
Name *ColumnName

// Refer is the result field the column name refers to.
// The value of Refer.Expr is used as the value of the expression.
Refer *ResultField
}

// Accept implements Node Accept interface.
Expand Down Expand Up @@ -421,23 +420,6 @@ func (n *DefaultExpr) Accept(v Visitor) (Node, bool) {
return v.Leave(n)
}

// IdentifierExpr represents an identifier expression.
type IdentifierExpr struct {
exprNode
// Name is the identifier name.
Name model.CIStr
}

// Accept implements Node Accept interface.
func (n *IdentifierExpr) Accept(v Visitor) (Node, bool) {
newNod, skipChildren := v.Enter(n)
if skipChildren {
return v.Leave(newNod)
}
n = newNod.(*IdentifierExpr)
return v.Leave(n)
}

// ExistsSubqueryExpr is the expression for "exists (select ...)".
// https://dev.mysql.com/doc/refman/5.7/en/exists-and-not-exists-subqueries.html
type ExistsSubqueryExpr struct {
Expand Down
28 changes: 14 additions & 14 deletions ddl/ddl_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,10 +21,10 @@ import (
"github.com/pingcap/tidb"
"github.com/pingcap/tidb/context"
"github.com/pingcap/tidb/ddl"
"github.com/pingcap/tidb/executor"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/model"
"github.com/pingcap/tidb/mysql"
"github.com/pingcap/tidb/optimizer"
"github.com/pingcap/tidb/parser"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/stmt"
Expand Down Expand Up @@ -65,7 +65,7 @@ func (ts *testSuite) TestT(c *C) {
err = sessionctx.GetDomain(ctx).DDL().CreateSchema(ctx, tbIdent.Schema)
c.Assert(terror.ErrorEqual(err, ddl.ErrExists), IsTrue)

tbStmt := statement("create table t (a int primary key not null, b varchar(255), key idx_b (b), c int, d int unique)").(*stmts.CreateTableStmt)
tbStmt := statement(ctx, "create table t (a int primary key not null, b varchar(255), key idx_b (b), c int, d int unique)").(*stmts.CreateTableStmt)

err = sessionctx.GetDomain(ctx).DDL().CreateTable(ctx, table.Ident{Schema: noExist, Name: tbIdent.Name}, tbStmt.Cols, tbStmt.Constraints)
c.Assert(terror.DatabaseNotExists.Equal(err), IsTrue)
Expand All @@ -76,7 +76,7 @@ func (ts *testSuite) TestT(c *C) {

tbIdent2 := tbIdent
tbIdent2.Name = model.NewCIStr("t2")
tbStmt = statement("create table t2 (a int unique not null)").(*stmts.CreateTableStmt)
tbStmt = statement(ctx, "create table t2 (a int unique not null)").(*stmts.CreateTableStmt)
err = sessionctx.GetDomain(ctx).DDL().CreateTable(ctx, tbIdent2, tbStmt.Cols, tbStmt.Constraints)
c.Assert(err, IsNil)
tb, err := sessionctx.GetDomain(ctx).InfoSchema().TableByName(tbIdent2.Schema, tbIdent2.Name)
Expand All @@ -86,15 +86,15 @@ func (ts *testSuite) TestT(c *C) {
c.Assert(err, IsNil)
rid1, err := tb.AddRecord(ctx, []interface{}{2})
c.Assert(err, IsNil)
alterStmt := statement(`alter table t2 add b enum("bb") first`).(*stmts.AlterTableStmt)
alterStmt := statement(ctx, `alter table t2 add b enum("bb") first`).(*stmts.AlterTableStmt)
sessionctx.GetDomain(ctx).DDL().AlterTable(ctx, tbIdent2, alterStmt.Specs)
c.Assert(alterStmt.Specs[0].String(), Not(Equals), "")
cols, err := tb.Row(ctx, rid0)
c.Assert(err, IsNil)
c.Assert(len(cols), Equals, 2)
c.Assert(cols[0], Equals, nil)
c.Assert(cols[1], Equals, int64(1))
alterStmt = statement("alter table t2 add c varchar(255) after b").(*stmts.AlterTableStmt)
alterStmt = statement(ctx, "alter table t2 add c varchar(255) after b").(*stmts.AlterTableStmt)
sessionctx.GetDomain(ctx).DDL().AlterTable(ctx, tbIdent2, alterStmt.Specs)
c.Assert(alterStmt.Specs[0].String(), Not(Equals), "")
tb, err = sessionctx.GetDomain(ctx).InfoSchema().TableByName(tbIdent2.Schema, tbIdent2.Name)
Expand All @@ -121,7 +121,7 @@ func (ts *testSuite) TestT(c *C) {
_, err = tb.AddRecord(ctx, []interface{}{1, "b", 2, 4})
c.Assert(err, IsNil)

alterStmt = statement("alter table t add column aa int first").(*stmts.AlterTableStmt)
alterStmt = statement(ctx, "alter table t add column aa int first").(*stmts.AlterTableStmt)
sessionctx.GetDomain(ctx).DDL().AlterTable(ctx, tbIdent, alterStmt.Specs)
c.Assert(alterStmt.Specs[0].String(), Not(Equals), "")
// Check indices info
Expand All @@ -139,16 +139,16 @@ func (ts *testSuite) TestT(c *C) {
c.Assert(col.Offset, Equals, o)
}
}
alterStmt = statement("alter table t add column bb int after b").(*stmts.AlterTableStmt)
alterStmt = statement(ctx, "alter table t add column bb int after b").(*stmts.AlterTableStmt)
err = sessionctx.GetDomain(ctx).DDL().AlterTable(ctx, tbIdent, alterStmt.Specs)
c.Assert(err, IsNil)
c.Assert(alterStmt.Specs[0].String(), Not(Equals), "")
// Inserting a duplicated column will cause error.
alterStmt = statement("alter table t add column bb int after b").(*stmts.AlterTableStmt)
alterStmt = statement(ctx, "alter table t add column bb int after b").(*stmts.AlterTableStmt)
err = sessionctx.GetDomain(ctx).DDL().AlterTable(ctx, tbIdent, alterStmt.Specs)
c.Assert(err, NotNil)

idxStmt := statement("CREATE INDEX idx_c ON t (c)").(*stmts.CreateIndexStmt)
idxStmt := statement(ctx, "CREATE INDEX idx_c ON t (c)").(*stmts.CreateIndexStmt)
idxName := model.NewCIStr(idxStmt.IndexName)
err = sessionctx.GetDomain(ctx).DDL().CreateIndex(ctx, tbIdent, idxStmt.Unique, idxName, idxStmt.IndexColNames)
c.Assert(err, IsNil)
Expand Down Expand Up @@ -177,11 +177,11 @@ func (ts *testSuite) TestConstraintNames(c *C) {
Name: tblName,
}

tbStmt := statement("create table t (a int, b int, index a (a, b), index a (a))").(*stmts.CreateTableStmt)
tbStmt := statement(ctx, "create table t (a int, b int, index a (a, b), index a (a))").(*stmts.CreateTableStmt)
err := sessionctx.GetDomain(ctx).DDL().CreateTable(ctx, tbIdent, tbStmt.Cols, tbStmt.Constraints)
c.Assert(err, NotNil)

tbStmt = statement("create table t (a int, b int, index A (a, b), index (a))").(*stmts.CreateTableStmt)
tbStmt = statement(ctx, "create table t (a int, b int, index A (a, b), index (a))").(*stmts.CreateTableStmt)
err = sessionctx.GetDomain(ctx).DDL().CreateTable(ctx, tbIdent, tbStmt.Cols, tbStmt.Constraints)
c.Assert(err, IsNil)
tbl, err := sessionctx.GetDomain(ctx).InfoSchema().TableByName(schemaName, tblName)
Expand All @@ -194,11 +194,11 @@ func (ts *testSuite) TestConstraintNames(c *C) {
c.Assert(err, IsNil)
}

func statement(sql string) stmt.Statement {
func statement(ctx context.Context, sql string) stmt.Statement {
log.Debug("Compile", sql)
lexer := parser.NewLexer(sql)
parser.YYParse(lexer)
compiler := &optimizer.Compiler{}
stm, _ := compiler.Compile(lexer.Stmts()[0])
compiler := &executor.Compiler{}
stm, _ := compiler.Compile(ctx, lexer.Stmts()[0])
return stm
}
83 changes: 83 additions & 0 deletions executor/adapter.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,83 @@
// Copyright 2015 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and
// limitations under the License.

package executor

import (
"github.com/pingcap/tidb/context"
"github.com/pingcap/tidb/field"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/optimizer/plan"
oplan "github.com/pingcap/tidb/plan"
"github.com/pingcap/tidb/rset"
"github.com/pingcap/tidb/util/format"
)

// adapter wraps a executor, implements rset.Recordset interface
type recordsetAdapter struct {
fields []*field.ResultField
executor Executor
}

func (a *recordsetAdapter) Do(f func(data []interface{}) (bool, error)) error {
return nil
}

func (a *recordsetAdapter) Fields() ([]*field.ResultField, error) {
return nil, nil
}

func (a *recordsetAdapter) FirstRow() ([]interface{}, error) {
return nil, nil
}

func (a *recordsetAdapter) Rows(limit, offset int) ([][]interface{}, error) {
return nil, nil
}

func (a *recordsetAdapter) Next() (*oplan.Row, error) {
return nil, nil
}

func (a *recordsetAdapter) Close() error {
return nil
}

type statementAdapter struct {
is infoschema.InfoSchema
plan plan.Plan
}

func (a *statementAdapter) Explain(ctx context.Context, w format.Formatter) {
return
}

func (a *statementAdapter) OriginText() string {
return ""
}

func (a *statementAdapter) SetText(text string) {
return
}

func (a *statementAdapter) IsDDL() bool {
return false
}

func (a *statementAdapter) Exec(ctx context.Context) (rset.Recordset, error) {
b := newExecutorBuilder(ctx, a.is)
e := b.build(a.plan)
return &recordsetAdapter{
executor: e,
}, nil
}
59 changes: 59 additions & 0 deletions executor/builder.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,59 @@
// Copyright 2015 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and
// limitations under the License.

package executor

import (
"github.com/pingcap/tidb/context"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/optimizer/plan"
)

// executorBuilder builds an Executor from a Plan.
// the InfoSchema must be the same one used in InfoBinder.
type executorBuilder struct {
ctx context.Context
is infoschema.InfoSchema
}

func newExecutorBuilder(ctx context.Context, is infoschema.InfoSchema) *executorBuilder {
return &executorBuilder{
ctx: ctx,
is: is,
}
}

func (b *executorBuilder) build(p plan.Plan) Executor {
switch v := p.(type) {
case *plan.TableScan:
return b.buildTableScan(v)
case *plan.IndexScan:
return b.buildIndexScan(v)
}
return nil
}

func (b *executorBuilder) buildTableScan(v *plan.TableScan) Executor {
table, _ := b.is.TableByID(v.Table.ID)
return &TableScanExec{
t: table,
}
}

func (b *executorBuilder) buildIndexScan(v *plan.IndexScan) Executor {
table, _ := b.is.TableByID(v.Table.ID)
e := &IndexScanExec{
Table: table,
}
return e
}
Loading

0 comments on commit c01af56

Please sign in to comment.