Skip to content

Commit

Permalink
refactor show clause. (pingcap#1940)
Browse files Browse the repository at this point in the history
  • Loading branch information
hanfei1991 authored Nov 10, 2016
1 parent 2776455 commit e176447
Show file tree
Hide file tree
Showing 10 changed files with 142 additions and 135 deletions.
35 changes: 2 additions & 33 deletions executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@ import (
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/model"
"github.com/pingcap/tidb/parser/opcode"
"github.com/pingcap/tidb/plan"
"github.com/pingcap/tidb/sessionctx/autocommit"
"github.com/pingcap/tidb/sessionctx/variable"
Expand Down Expand Up @@ -64,9 +63,6 @@ func (b *executorBuilder) build(p plan.Plan) Executor {
return b.buildExecute(v)
case *plan.Explain:
return b.buildExplain(v)
case *plan.Filter:
src := b.build(v.GetChildByIndex(0))
return b.buildFilter(src, v.Conditions)
case *plan.Insert:
return b.buildInsert(v)
case *plan.LoadData:
Expand Down Expand Up @@ -123,21 +119,10 @@ func (b *executorBuilder) build(p plan.Plan) Executor {
}
}

func (b *executorBuilder) buildFilter(src Executor, conditions []ast.ExprNode) Executor {
if len(conditions) == 0 {
return src
}
return &FilterExec{
Src: src,
Condition: b.joinConditions(conditions),
ctx: b.ctx,
}
}

func (b *executorBuilder) buildShowDDL(v *plan.ShowDDL) Executor {
return &ShowDDLExec{
fields: v.Fields(),
ctx: b.ctx,
schema: v.GetSchema(),
}
}

Expand All @@ -155,22 +140,6 @@ func (b *executorBuilder) buildDeallocate(v *plan.Deallocate) Executor {
}
}

func (b *executorBuilder) joinConditions(conditions []ast.ExprNode) ast.ExprNode {
if len(conditions) == 0 {
return nil
}
if len(conditions) == 1 {
return conditions[0]
}
condition := &ast.BinaryOperationExpr{
Op: opcode.AndAnd,
L: conditions[0],
R: b.joinConditions(conditions[1:]),
}
ast.MergeChildrenFlags(condition, condition.L, condition.R)
return condition
}

func (b *executorBuilder) buildSelectLock(v *plan.SelectLock) Executor {
src := b.build(v.GetChildByIndex(0))
if autocommit.ShouldAutocommit(b.ctx) {
Expand Down Expand Up @@ -235,7 +204,7 @@ func (b *executorBuilder) buildShow(v *plan.Show) Executor {
GlobalScope: v.GlobalScope,
ctx: b.ctx,
is: b.is,
fields: v.Fields(),
schema: v.GetSchema(),
}
if e.Tp == ast.ShowGrants && len(e.User) == 0 {
e.User = variable.GetSessionVars(e.ctx).User
Expand Down
9 changes: 3 additions & 6 deletions executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -118,19 +118,19 @@ type Executor interface {

// ShowDDLExec represents a show DDL executor.
type ShowDDLExec struct {
fields []*ast.ResultField
schema expression.Schema
ctx context.Context
done bool
}

// Schema implements the Executor Schema interface.
func (e *ShowDDLExec) Schema() expression.Schema {
return nil
return e.schema
}

// Fields implements the Executor Fields interface.
func (e *ShowDDLExec) Fields() []*ast.ResultField {
return e.fields
return nil
}

// Next implements the Executor Next interface.
Expand Down Expand Up @@ -178,9 +178,6 @@ func (e *ShowDDLExec) Next() (*Row, error) {
bgOwner,
bgJob,
)
for i, f := range e.fields {
f.Expr.SetValue(row.Data[i].GetValue())
}
e.done = true

return row, nil
Expand Down
9 changes: 3 additions & 6 deletions executor/show.go
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,7 @@ type ShowExec struct {
// Used by show variables
GlobalScope bool

fields []*ast.ResultField
schema expression.Schema
ctx context.Context
is infoschema.InfoSchema

Expand All @@ -57,12 +57,12 @@ type ShowExec struct {

// Schema implements the Executor Schema interface.
func (e *ShowExec) Schema() expression.Schema {
return nil
return e.schema
}

// Fields implements the Executor Fields interface.
func (e *ShowExec) Fields() []*ast.ResultField {
return e.fields
return nil
}

// Next implements Execution Next interface.
Expand All @@ -77,9 +77,6 @@ func (e *ShowExec) Next() (*Row, error) {
return nil, nil
}
row := e.rows[e.cursor]
for i, field := range e.fields {
field.Expr.SetValue(row.Data[i].GetValue())
}
e.cursor++
return row, nil
}
Expand Down
24 changes: 24 additions & 0 deletions expression/expression.go
Original file line number Diff line number Diff line change
Expand Up @@ -513,3 +513,27 @@ func EvaluateExprWithNull(schema Schema, expr Expression) (Expression, error) {
return x.Clone(), nil
}
}

// ResultFieldsToSchema converts slice of result fields to schema.
func ResultFieldsToSchema(fields []*ast.ResultField) Schema {
schema := make(Schema, 0, len(fields))
for i, field := range fields {
colName := field.ColumnAsName
if colName.L == "" {
colName = field.Column.Name
}
tblName := field.TableAsName
if tblName.L == "" {
tblName = field.Table.Name
}
col := &Column{
ColName: colName,
TblName: tblName,
DBName: field.DBName,
RetType: &field.Column.FieldType,
Position: i,
}
schema = append(schema, col)
}
return schema
}
5 changes: 5 additions & 0 deletions plan/match_property.go
Original file line number Diff line number Diff line change
Expand Up @@ -311,3 +311,8 @@ func (p *PhysicalDummyScan) matchProperty(_ *requiredProperty, _ ...*physicalPla
func (p *Delete) matchProperty(_ *requiredProperty, _ ...*physicalPlanInfo) *physicalPlanInfo {
panic("You can't call this function!")
}

// matchProperty implements PhysicalPlan matchProperty interface.
func (p *Show) matchProperty(_ *requiredProperty, _ ...*physicalPlanInfo) *physicalPlanInfo {
panic("You can't call this function!")
}
6 changes: 6 additions & 0 deletions plan/physical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -699,6 +699,12 @@ func (p *Delete) Copy() PhysicalPlan {
return &np
}

// Copy implements the PhysicalPlan Copy interface.
func (p *Show) Copy() PhysicalPlan {
np := *p
return &np
}

// Copy implements the PhysicalPlan Copy interface.
func (p *PhysicalUnionScan) Copy() PhysicalPlan {
np := *p
Expand Down
12 changes: 8 additions & 4 deletions plan/plan.go
Original file line number Diff line number Diff line change
Expand Up @@ -238,24 +238,28 @@ func newBaseLogicalPlan(tp string, a *idAllocator) baseLogicalPlan {
// PredicatePushDown implements LogicalPlan PredicatePushDown interface.
func (p *baseLogicalPlan) PredicatePushDown(predicates []expression.Expression) ([]expression.Expression, LogicalPlan, error) {
if len(p.GetChildren()) == 0 {
return predicates, nil, nil
return predicates, p, nil
}
child := p.GetChildByIndex(0).(LogicalPlan)
rest, _, err := child.PredicatePushDown(predicates)
if err != nil {
return nil, nil, errors.Trace(err)
return nil, p, errors.Trace(err)
}
if len(rest) > 0 {
err = addSelection(p, child, rest, p.allocator)
if err != nil {
return nil, nil, errors.Trace(err)
return nil, p, errors.Trace(err)
}
}
return nil, nil, nil
return nil, p, nil
}

// PruneColumnsAndResolveIndices implements LogicalPlan PruneColumnsAndResolveIndices interface.
func (p *baseLogicalPlan) PruneColumnsAndResolveIndices(parentUsedCols []*expression.Column) ([]*expression.Column, error) {
if len(p.children) == 0 {
p.schema.InitIndices()
return nil, nil
}
outer, err := p.GetChildByIndex(0).(LogicalPlan).PruneColumnsAndResolveIndices(parentUsedCols)
p.SetSchema(p.GetChildByIndex(0).GetSchema())
return outer, errors.Trace(err)
Expand Down
Loading

0 comments on commit e176447

Please sign in to comment.