Skip to content

Commit

Permalink
plan, executor: clean code. (pingcap#4994)
Browse files Browse the repository at this point in the history
  • Loading branch information
hanfei1991 authored and zz-jason committed Nov 2, 2017
1 parent 0616a2f commit c70ed24
Show file tree
Hide file tree
Showing 7 changed files with 12 additions and 121 deletions.
2 changes: 1 addition & 1 deletion executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -484,7 +484,7 @@ func (b *executorBuilder) buildMergeJoin(v *plan.PhysicalMergeJoin) Executor {
joinType: v.JoinType,
defaultValues: v.DefaultValues,
}
exec, err := joinBuilder.BuildMergeJoin(v.Desc)
exec, err := joinBuilder.BuildMergeJoin()
if err != nil {
b.err = err
return nil
Expand Down
3 changes: 1 addition & 2 deletions executor/join_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,7 @@ type joinBuilder struct {
defaultValues []types.Datum
}

func (b *joinBuilder) BuildMergeJoin(assumeSortedDesc bool) (*MergeJoinExec, error) {
func (b *joinBuilder) BuildMergeJoin() (*MergeJoinExec, error) {
var leftJoinKeys, rightJoinKeys []*expression.Column
for _, eqCond := range b.eqConditions {
if len(eqCond.GetArgs()) != 2 {
Expand Down Expand Up @@ -74,7 +74,6 @@ func (b *joinBuilder) BuildMergeJoin(assumeSortedDesc bool) (*MergeJoinExec, err
outerIter: leftRowBlock,
innerIter: rightRowBlock,
schema: b.schema,
desc: assumeSortedDesc,
resultGenerator: newJoinResultGenerator(b.context, b.joinType, b.defaultValues, b.otherFilter),
}

Expand Down
4 changes: 0 additions & 4 deletions executor/merge_join.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,6 @@ type MergeJoinExec struct {
stmtCtx *variable.StatementContext
schema *expression.Schema
prepared bool
desc bool

outerKeys []*expression.Column
innerKeys []*expression.Column
Expand Down Expand Up @@ -229,9 +228,6 @@ func (e *MergeJoinExec) computeJoin() (bool, error) {
if err != nil {
return false, errors.Trace(err)
}
if e.desc {
compareResult = -compareResult
}
}

if compareResult > 0 {
Expand Down
5 changes: 0 additions & 5 deletions plan/explain.go
Original file line number Diff line number Diff line change
Expand Up @@ -294,11 +294,6 @@ func (p *PhysicalMergeJoin) ExplainInfo() string {
buffer.WriteString(fmt.Sprintf(", other cond:%s",
expression.ExplainExpressionList(p.OtherConditions)))
}
if p.Desc {
buffer.WriteString(", desc")
} else {
buffer.WriteString(", asc")
}
if len(p.leftKeys) > 0 {
buffer.WriteString(fmt.Sprintf(", left key:%s",
expression.ExplainColumnList(p.leftKeys)))
Expand Down
10 changes: 5 additions & 5 deletions plan/logical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -143,7 +143,7 @@ func (p *LogicalJoin) attachOnConds(onConds []expression.Expression) {
}

func (p *LogicalJoin) extractCorrelatedCols() []*expression.CorrelatedColumn {
corCols := p.basePlan.extractCorrelatedCols()
corCols := p.baseLogicalPlan.extractCorrelatedCols()
for _, fun := range p.EqualConditions {
corCols = append(corCols, extractCorColumns(fun)...)
}
Expand Down Expand Up @@ -173,7 +173,7 @@ type Projection struct {
}

func (p *Projection) extractCorrelatedCols() []*expression.CorrelatedColumn {
corCols := p.basePlan.extractCorrelatedCols()
corCols := p.baseLogicalPlan.extractCorrelatedCols()
for _, expr := range p.Exprs {
corCols = append(corCols, extractCorColumns(expr)...)
}
Expand All @@ -195,7 +195,7 @@ type LogicalAggregation struct {
}

func (p *LogicalAggregation) extractCorrelatedCols() []*expression.CorrelatedColumn {
corCols := p.basePlan.extractCorrelatedCols()
corCols := p.baseLogicalPlan.extractCorrelatedCols()
for _, expr := range p.GroupByItems {
corCols = append(corCols, extractCorColumns(expr)...)
}
Expand Down Expand Up @@ -231,7 +231,7 @@ type Selection struct {
}

func (p *Selection) extractCorrelatedCols() []*expression.CorrelatedColumn {
corCols := p.basePlan.extractCorrelatedCols()
corCols := p.baseLogicalPlan.extractCorrelatedCols()
for _, cond := range p.Conditions {
corCols = append(corCols, extractCorColumns(cond)...)
}
Expand Down Expand Up @@ -347,7 +347,7 @@ type Sort struct {
}

func (p *Sort) extractCorrelatedCols() []*expression.CorrelatedColumn {
corCols := p.basePlan.extractCorrelatedCols()
corCols := p.baseLogicalPlan.extractCorrelatedCols()
for _, item := range p.ByItems {
corCols = append(corCols, extractCorColumns(item.Expr)...)
}
Expand Down
99 changes: 0 additions & 99 deletions plan/physical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,11 +17,9 @@ import (
"github.com/pingcap/tidb/ast"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/expression/aggregation"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/model"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/util/types"
"github.com/pingcap/tipb/go-tipb"
)

var (
Expand Down Expand Up @@ -175,32 +173,9 @@ type physicalTableSource struct {
*basePlan
basePhysicalPlan

client kv.Client

Aggregated bool
readOnly bool
AggFuncsPB []*tipb.Expr
GbyItemsPB []*tipb.ByItem

// TableConditionPBExpr is the pb structure of conditions that used in the table scan.
TableConditionPBExpr *tipb.Expr
// IndexConditionPBExpr is the pb structure of conditions that used in the index scan.
IndexConditionPBExpr *tipb.Expr

// AccessCondition is used to calculate range.
AccessCondition []expression.Expression

LimitCount *int64
SortItemsPB []*tipb.ByItem

// The following fields are used for explaining and testing. Because pb structures are not human-readable.

aggFuncs []aggregation.Aggregation
gbyItems []expression.Expression
sortItems []*ByItems
indexFilterConditions []expression.Expression
tableFilterConditions []expression.Expression

// filterCondition is only used by new planner.
filterCondition []expression.Expression

Expand All @@ -220,21 +195,6 @@ func needValue(af aggregation.Aggregation) bool {
af.GetName() == ast.AggFuncMax || af.GetName() == ast.AggFuncMin || af.GetName() == ast.AggFuncGroupConcat
}

func (p *physicalTableSource) tryToAddUnionScan(resultPlan PhysicalPlan, s *expression.Schema) PhysicalPlan {
if p.readOnly {
return resultPlan
}
conditions := append(p.indexFilterConditions, p.tableFilterConditions...)
us := PhysicalUnionScan{
Conditions: append(conditions, p.AccessCondition...),
NeedColHandle: p.NeedColHandle,
}.init(p.allocator, p.ctx)
us.SetChildren(resultPlan)
us.SetSchema(s)
p.NeedColHandle = true
return us
}

// PhysicalTableScan represents a table scan plan.
type PhysicalTableScan struct {
physicalTableSource
Expand Down Expand Up @@ -312,7 +272,6 @@ type PhysicalMergeJoin struct {
OtherConditions []expression.Expression

DefaultValues []types.Datum
Desc bool

leftKeys []*expression.Column
rightKeys []*expression.Column
Expand Down Expand Up @@ -382,64 +341,6 @@ type PhysicalUnionScan struct {
Conditions []expression.Expression
}

func (p *PhysicalHashJoin) extractCorrelatedCols() []*expression.CorrelatedColumn {
corCols := p.basePlan.extractCorrelatedCols()
for _, fun := range p.EqualConditions {
corCols = append(corCols, extractCorColumns(fun)...)
}
for _, fun := range p.LeftConditions {
corCols = append(corCols, extractCorColumns(fun)...)
}
for _, fun := range p.RightConditions {
corCols = append(corCols, extractCorColumns(fun)...)
}
for _, fun := range p.OtherConditions {
corCols = append(corCols, extractCorColumns(fun)...)
}
return corCols
}

func (p *PhysicalHashSemiJoin) extractCorrelatedCols() []*expression.CorrelatedColumn {
corCols := p.basePlan.extractCorrelatedCols()
for _, fun := range p.EqualConditions {
corCols = append(corCols, extractCorColumns(fun)...)
}
for _, fun := range p.LeftConditions {
corCols = append(corCols, extractCorColumns(fun)...)
}
for _, fun := range p.RightConditions {
corCols = append(corCols, extractCorColumns(fun)...)
}
for _, fun := range p.OtherConditions {
corCols = append(corCols, extractCorColumns(fun)...)
}
return corCols
}

func (p *PhysicalApply) extractCorrelatedCols() []*expression.CorrelatedColumn {
corCols := p.basePlan.extractCorrelatedCols()
corCols = append(corCols, p.PhysicalJoin.extractCorrelatedCols()...)
for i := len(corCols) - 1; i >= 0; i-- {
if p.PhysicalJoin.Children()[0].Schema().Contains(&corCols[i].Column) {
corCols = append(corCols[:i], corCols[i+1:]...)
}
}
return corCols
}

func (p *PhysicalAggregation) extractCorrelatedCols() []*expression.CorrelatedColumn {
corCols := p.basePlan.extractCorrelatedCols()
for _, expr := range p.GroupByItems {
corCols = append(corCols, extractCorColumns(expr)...)
}
for _, fun := range p.AggFuncs {
for _, arg := range fun.GetArgs() {
corCols = append(corCols, extractCorColumns(arg)...)
}
}
return corCols
}

// Copy implements the PhysicalPlan Copy interface.
func (p *PhysicalIndexScan) Copy() PhysicalPlan {
np := *p
Expand Down
10 changes: 5 additions & 5 deletions plan/plan.go
Original file line number Diff line number Diff line change
Expand Up @@ -61,8 +61,6 @@ type Plan interface {

context() context.Context

extractCorrelatedCols() []*expression.CorrelatedColumn

// ResolveIndices resolves the indices for columns. After doing this, the columns can evaluate the rows by their indices.
ResolveIndices()

Expand Down Expand Up @@ -197,6 +195,8 @@ type LogicalPlan interface {

// generatePhysicalPlans generates all possible plans.
generatePhysicalPlans() []PhysicalPlan

extractCorrelatedCols() []*expression.CorrelatedColumn
}

// PhysicalPlan is a tree of the physical operators.
Expand Down Expand Up @@ -307,10 +307,10 @@ func (p *baseLogicalPlan) PredicatePushDown(predicates []expression.Expression)
return nil, p.basePlan.self.(LogicalPlan), nil
}

func (p *basePlan) extractCorrelatedCols() []*expression.CorrelatedColumn {
func (p *baseLogicalPlan) extractCorrelatedCols() []*expression.CorrelatedColumn {
var corCols []*expression.CorrelatedColumn
for _, child := range p.children {
corCols = append(corCols, child.extractCorrelatedCols()...)
for _, child := range p.basePlan.children {
corCols = append(corCols, child.(LogicalPlan).extractCorrelatedCols()...)
}
return corCols
}
Expand Down

0 comments on commit c70ed24

Please sign in to comment.