Skip to content

Commit

Permalink
plan: remove the rest useless type assertion. (pingcap#5693)
Browse files Browse the repository at this point in the history
  • Loading branch information
hanfei1991 authored Jan 22, 2018
1 parent 236e50a commit c6e1466
Show file tree
Hide file tree
Showing 9 changed files with 25 additions and 26 deletions.
10 changes: 5 additions & 5 deletions plan/aggregation_push_down.go
Original file line number Diff line number Diff line change
Expand Up @@ -198,7 +198,7 @@ func (a *aggregationOptimizer) allFirstRow(aggFuncs []*aggregation.AggFuncDesc)
// process it temporarily. If not, We will add additional group by columns and first row functions. We make a new aggregation operator.
// If the pushed aggregation is grouped by unique key, it's no need to push it down.
func (a *aggregationOptimizer) tryToPushDownAgg(aggFuncs []*aggregation.AggFuncDesc, gbyCols []*expression.Column, join *LogicalJoin, childIdx int) LogicalPlan {
child := join.children[childIdx].(LogicalPlan)
child := join.children[childIdx]
if a.allFirstRow(aggFuncs) {
return child
}
Expand Down Expand Up @@ -339,12 +339,12 @@ func (a *aggregationOptimizer) aggPushDown(p LogicalPlan) LogicalPlan {
rightInvalid := a.checkAnyCountAndSum(leftAggFuncs)
leftInvalid := a.checkAnyCountAndSum(rightAggFuncs)
if rightInvalid {
rChild = join.children[1].(LogicalPlan)
rChild = join.children[1]
} else {
rChild = a.tryToPushDownAgg(rightAggFuncs, rightGbyCols, join, 1)
}
if leftInvalid {
lChild = join.children[0].(LogicalPlan)
lChild = join.children[0]
} else {
lChild = a.tryToPushDownAgg(leftAggFuncs, leftGbyCols, join, 0)
}
Expand Down Expand Up @@ -378,7 +378,7 @@ func (a *aggregationOptimizer) aggPushDown(p LogicalPlan) LogicalPlan {
pushedAgg := a.makeNewAgg(agg.ctx, agg.AggFuncs, gbyCols)
newChildren := make([]LogicalPlan, 0, len(union.children))
for _, child := range union.children {
newChild := a.pushAggCrossUnion(pushedAgg, union.Schema(), child.(LogicalPlan))
newChild := a.pushAggCrossUnion(pushedAgg, union.Schema(), child)
newChildren = append(newChildren, newChild)
}
union.SetChildren(newChildren...)
Expand All @@ -387,7 +387,7 @@ func (a *aggregationOptimizer) aggPushDown(p LogicalPlan) LogicalPlan {
}
newChildren := make([]LogicalPlan, 0, len(p.Children()))
for _, child := range p.Children() {
newChild := a.aggPushDown(child.(LogicalPlan))
newChild := a.aggPushDown(child)
newChildren = append(newChildren, newChild)
}
p.SetChildren(newChildren...)
Expand Down
2 changes: 1 addition & 1 deletion plan/build_key_info.go
Original file line number Diff line number Diff line change
Expand Up @@ -128,7 +128,7 @@ func (p *LogicalProjection) buildKeyInfo() {
func (p *LogicalJoin) buildKeyInfo() {
p.schema.Keys = nil
p.baseLogicalPlan.buildKeyInfo()
p.maxOneRow = p.children[0].(LogicalPlan).MaxOneRow() && p.children[1].(LogicalPlan).MaxOneRow()
p.maxOneRow = p.children[0].MaxOneRow() && p.children[1].MaxOneRow()
switch p.JoinType {
case SemiJoin, LeftOuterSemiJoin, AntiSemiJoin, AntiLeftOuterSemiJoin:
p.schema.Keys = p.children[0].Schema().Clone().Keys
Expand Down
14 changes: 7 additions & 7 deletions plan/decorrelate.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@ import (
// only [t2.a] is treated as this apply's correlated column.
func (la *LogicalApply) extractCorColumnsBySchema() {
schema := la.children[0].Schema()
corCols := la.children[1].(LogicalPlan).extractCorrelatedCols()
corCols := la.children[1].extractCorrelatedCols()
resultCorCols := make([]*expression.CorrelatedColumn, schema.Len())
for _, corCol := range corCols {
idx := schema.ColumnIndex(&corCol.Column)
Expand Down Expand Up @@ -85,7 +85,7 @@ type decorrelateSolver struct{}
func (s *decorrelateSolver) optimize(p LogicalPlan) (LogicalPlan, error) {
if apply, ok := p.(*LogicalApply); ok {
outerPlan := apply.children[0]
innerPlan := apply.children[1].(LogicalPlan)
innerPlan := apply.children[1]
apply.extractCorColumnsBySchema()
if len(apply.corCols) == 0 {
// If the inner plan is non-correlated, the apply will be simplified to join.
Expand All @@ -100,12 +100,12 @@ func (s *decorrelateSolver) optimize(p LogicalPlan) (LogicalPlan, error) {
newConds = append(newConds, cond.Decorrelate(outerPlan.Schema()))
}
apply.attachOnConds(newConds)
innerPlan = sel.children[0].(LogicalPlan)
innerPlan = sel.children[0]
apply.SetChildren(outerPlan, innerPlan)
return s.optimize(p)
} else if m, ok := innerPlan.(*LogicalMaxOneRow); ok {
if m.children[0].(LogicalPlan).MaxOneRow() {
innerPlan = m.children[0].(LogicalPlan)
if m.children[0].MaxOneRow() {
innerPlan = m.children[0]
apply.SetChildren(outerPlan, innerPlan)
return s.optimize(p)
}
Expand All @@ -114,7 +114,7 @@ func (s *decorrelateSolver) optimize(p LogicalPlan) (LogicalPlan, error) {
proj.Exprs[i] = expr.Decorrelate(outerPlan.Schema())
}
apply.columnSubstitute(proj.Schema(), proj.Exprs)
innerPlan = proj.children[0].(LogicalPlan)
innerPlan = proj.children[0]
apply.SetChildren(outerPlan, innerPlan)
if apply.JoinType != SemiJoin && apply.JoinType != LeftOuterSemiJoin && apply.JoinType != AntiSemiJoin && apply.JoinType != AntiLeftOuterSemiJoin {
proj.SetSchema(apply.Schema())
Expand All @@ -130,7 +130,7 @@ func (s *decorrelateSolver) optimize(p LogicalPlan) (LogicalPlan, error) {
return s.optimize(p)
} else if agg, ok := innerPlan.(*LogicalAggregation); ok {
if apply.canPullUpAgg() && agg.canPullUp() {
innerPlan = agg.children[0].(LogicalPlan)
innerPlan = agg.children[0]
apply.JoinType = LeftOuterJoin
apply.SetChildren(outerPlan, innerPlan)
agg.SetSchema(apply.Schema())
Expand Down
10 changes: 5 additions & 5 deletions plan/eliminate_projection.go
Original file line number Diff line number Diff line change
Expand Up @@ -69,15 +69,15 @@ func resolveExprAndReplace(origin expression.Expression, replace map[string]*exp

func doPhysicalProjectionElimination(p PhysicalPlan) PhysicalPlan {
for i, child := range p.Children() {
p.Children()[i] = doPhysicalProjectionElimination(child.(PhysicalPlan))
p.Children()[i] = doPhysicalProjectionElimination(child)
}

proj, isProj := p.(*PhysicalProjection)
if !isProj || !canProjectionBeEliminatedStrict(proj) {
return p
}
child := p.Children()[0]
return child.(PhysicalPlan)
return child
}

// eliminatePhysicalProjection should be called after physical optimization to eliminate the redundant projection
Expand All @@ -101,7 +101,7 @@ type projectionEliminater struct {
// optimize implements the logicalOptRule interface.
func (pe *projectionEliminater) optimize(lp LogicalPlan) (LogicalPlan, error) {
root := pe.eliminate(lp, make(map[string]*expression.Column), false)
return root.(LogicalPlan), nil
return root, nil
}

// eliminate eliminates the redundant projection in a logical plan.
Expand All @@ -114,7 +114,7 @@ func (pe *projectionEliminater) eliminate(p LogicalPlan, replace map[string]*exp
childFlag = true
}
for i, child := range p.Children() {
p.Children()[i] = pe.eliminate(child.(LogicalPlan), replace, childFlag)
p.Children()[i] = pe.eliminate(child, replace, childFlag)
}

switch x := p.(type) {
Expand All @@ -136,7 +136,7 @@ func (pe *projectionEliminater) eliminate(p LogicalPlan, replace map[string]*exp
for i, col := range proj.Schema().Columns {
replace[string(col.HashCode())] = exprs[i].(*expression.Column)
}
return p.Children()[0].(LogicalPlan)
return p.Children()[0]
}

func (p *LogicalJoin) replaceExprColumns(replace map[string]*expression.Column) {
Expand Down
2 changes: 1 addition & 1 deletion plan/expression_rewriter.go
Original file line number Diff line number Diff line change
Expand Up @@ -529,7 +529,7 @@ func (er *expressionRewriter) handleExistSubquery(v *ast.ExistsSubqueryExpr) (as
}
np = er.b.buildExists(np)
if len(np.extractCorrelatedCols()) > 0 {
er.p = er.b.buildSemiApply(er.p, np.Children()[0].(LogicalPlan), nil, er.asScalar, false)
er.p = er.b.buildSemiApply(er.p, np.Children()[0], nil, er.asScalar, false)
if !er.asScalar {
return v, true
}
Expand Down
2 changes: 1 addition & 1 deletion plan/initialize.go
Original file line number Diff line number Diff line change
Expand Up @@ -365,7 +365,7 @@ func flattenPushDownPlan(p PhysicalPlan) []PhysicalPlan {
if len(p.Children()) == 0 {
break
}
p = p.Children()[0].(PhysicalPlan)
p = p.Children()[0]
}
for i := 0; i < len(plans)/2; i++ {
j := len(plans) - i - 1
Expand Down
6 changes: 3 additions & 3 deletions plan/logical_plan_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -1740,13 +1740,13 @@ out:
// This can be removed when in exists clause,
// e.g. exists(select count(*) from t order by a) is equal to exists t.
case *LogicalProjection, *LogicalSort:
p = p.Children()[0].(LogicalPlan)
p = p.Children()[0]
case *LogicalAggregation:
if len(plan.GroupByItems) == 0 {
p = b.buildTableDual()
break out
}
p = p.Children()[0].(LogicalPlan)
p = p.Children()[0]
default:
break out
}
Expand Down Expand Up @@ -1957,7 +1957,7 @@ func (b *planBuilder) buildUpdateLists(tableList []*ast.TableName, list []*ast.A
func extractTableAsNameForUpdate(p LogicalPlan, asNames map[*model.TableInfo][]*model.CIStr) {
switch x := p.(type) {
case *DataSource:
alias := extractTableAlias(p.(LogicalPlan))
alias := extractTableAlias(p)
if alias != nil {
if _, ok := asNames[x.tableInfo]; !ok {
asNames[x.tableInfo] = make([]*model.CIStr, 0, 1)
Expand Down
2 changes: 1 addition & 1 deletion plan/optimizer.go
Original file line number Diff line number Diff line change
Expand Up @@ -162,7 +162,7 @@ func existsCartesianProduct(p LogicalPlan) bool {
return join.JoinType == InnerJoin || join.JoinType == LeftOuterJoin || join.JoinType == RightOuterJoin
}
for _, child := range p.Children() {
if existsCartesianProduct(child.(LogicalPlan)) {
if existsCartesianProduct(child) {
return true
}
}
Expand Down
3 changes: 1 addition & 2 deletions plan/plan.go
Original file line number Diff line number Diff line change
Expand Up @@ -306,8 +306,7 @@ func (p *baseLogicalPlan) PruneColumns(parentUsedCols []*expression.Column) {
if len(p.children) == 0 {
return
}
child := p.children[0].(LogicalPlan)
child.PruneColumns(parentUsedCols)
p.children[0].PruneColumns(parentUsedCols)
}

// basePlan implements base Plan interface.
Expand Down

0 comments on commit c6e1466

Please sign in to comment.