Skip to content

Commit

Permalink
plan, executor: avoid children type assertion. (pingcap#5616)
Browse files Browse the repository at this point in the history
  • Loading branch information
hanfei1991 authored Jan 12, 2018
1 parent 18ba347 commit 3cff21e
Show file tree
Hide file tree
Showing 26 changed files with 179 additions and 135 deletions.
2 changes: 1 addition & 1 deletion executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -995,7 +995,7 @@ func (b *executorBuilder) buildAnalyzeColumnsPushdown(task plan.AnalyzeColumnsTa
func (b *executorBuilder) buildAnalyze(v *plan.Analyze) Executor {
e := &AnalyzeExec{
baseExecutor: newBaseExecutor(v.Schema(), b.ctx),
tasks: make([]*analyzeTask, 0, len(v.Children())),
tasks: make([]*analyzeTask, 0, len(v.ColTasks)+len(v.IdxTasks)),
}
for _, task := range v.ColTasks {
e.tasks = append(e.tasks, &analyzeTask{
Expand Down
8 changes: 5 additions & 3 deletions executor/metrics.go
Original file line number Diff line number Diff line change
Expand Up @@ -279,9 +279,11 @@ func (pa *stmtAttributes) fromPlan(p plan.Plan) {
}
pa.hasIndexDouble = true
}
children := p.Children()
for _, child := range children {
pa.fromPlan(child)
if pp, ok := p.(plan.PhysicalPlan); ok {
children := pp.Children()
for _, child := range children {
pa.fromPlan(child)
}
}
}

Expand Down
4 changes: 2 additions & 2 deletions plan/aggregation_push_down.go
Original file line number Diff line number Diff line change
Expand Up @@ -374,7 +374,7 @@ func (a *aggregationOptimizer) aggPushDown(p LogicalPlan) LogicalPlan {
var gbyCols []*expression.Column
gbyCols = expression.ExtractColumnsFromExpressions(gbyCols, agg.GroupByItems, nil)
pushedAgg := a.makeNewAgg(agg.AggFuncs, gbyCols)
newChildren := make([]Plan, 0, len(union.children))
newChildren := make([]LogicalPlan, 0, len(union.children))
for _, child := range union.children {
newChild := a.pushAggCrossUnion(pushedAgg, union.Schema(), child.(LogicalPlan))
newChildren = append(newChildren, newChild)
Expand All @@ -383,7 +383,7 @@ func (a *aggregationOptimizer) aggPushDown(p LogicalPlan) LogicalPlan {
}
}
}
newChildren := make([]Plan, 0, len(p.Children()))
newChildren := make([]LogicalPlan, 0, len(p.Children()))
for _, child := range p.Children() {
newChild := a.aggPushDown(child.(LogicalPlan))
newChildren = append(newChildren, newChild)
Expand Down
33 changes: 16 additions & 17 deletions plan/column_pruning.go
Original file line number Diff line number Diff line change
Expand Up @@ -62,7 +62,7 @@ func exprHasSetVar(expr expression.Expression) bool {
// PruneColumns implements LogicalPlan interface.
// If any expression has SetVar functions, we do not prune it.
func (p *LogicalProjection) PruneColumns(parentUsedCols []*expression.Column) {
child := p.children[0].(LogicalPlan)
child := p.children[0]
used := getUsedList(parentUsedCols, p.schema)
for i := len(used) - 1; i >= 0; i-- {
if !used[i] && !exprHasSetVar(p.Exprs[i]) {
Expand All @@ -77,14 +77,14 @@ func (p *LogicalProjection) PruneColumns(parentUsedCols []*expression.Column) {

// PruneColumns implements LogicalPlan interface.
func (p *LogicalSelection) PruneColumns(parentUsedCols []*expression.Column) {
child := p.children[0].(LogicalPlan)
child := p.children[0]
parentUsedCols = expression.ExtractColumnsFromExpressions(parentUsedCols, p.Conditions, nil)
child.PruneColumns(parentUsedCols)
}

// PruneColumns implements LogicalPlan interface.
func (la *LogicalAggregation) PruneColumns(parentUsedCols []*expression.Column) {
child := la.children[0].(LogicalPlan)
child := la.children[0]
used := getUsedList(parentUsedCols, la.Schema())
for i := len(used) - 1; i >= 0; i-- {
if !used[i] {
Expand Down Expand Up @@ -116,7 +116,7 @@ func (la *LogicalAggregation) PruneColumns(parentUsedCols []*expression.Column)

// PruneColumns implements LogicalPlan interface.
func (ls *LogicalSort) PruneColumns(parentUsedCols []*expression.Column) {
child := ls.children[0].(LogicalPlan)
child := ls.children[0]
for i := len(ls.ByItems) - 1; i >= 0; i-- {
cols := expression.ExtractColumns(ls.ByItems[i].Expr)
if len(cols) == 0 {
Expand All @@ -130,8 +130,7 @@ func (ls *LogicalSort) PruneColumns(parentUsedCols []*expression.Column) {

// PruneColumns implements LogicalPlan interface.
func (p *LogicalUnionAll) PruneColumns(parentUsedCols []*expression.Column) {
for _, c := range p.Children() {
child := c.(LogicalPlan)
for _, child := range p.Children() {
child.PruneColumns(parentUsedCols)
}
}
Expand All @@ -141,7 +140,7 @@ func (p *LogicalUnionScan) PruneColumns(parentUsedCols []*expression.Column) {
for _, col := range p.Schema().TblID2Handle {
parentUsedCols = append(parentUsedCols, col[0])
}
p.children[0].(LogicalPlan).PruneColumns(parentUsedCols)
p.children[0].PruneColumns(parentUsedCols)
}

// PruneColumns implements LogicalPlan interface.
Expand Down Expand Up @@ -172,7 +171,7 @@ func (p *LogicalTableDual) PruneColumns(_ []*expression.Column) {

// PruneColumns implements LogicalPlan interface.
func (p *LogicalExists) PruneColumns(parentUsedCols []*expression.Column) {
p.children[0].(LogicalPlan).PruneColumns(nil)
p.children[0].PruneColumns(nil)
}

func (p *LogicalJoin) extractUsedCols(parentUsedCols []*expression.Column) (leftCols []*expression.Column, rightCols []*expression.Column) {
Expand All @@ -188,8 +187,8 @@ func (p *LogicalJoin) extractUsedCols(parentUsedCols []*expression.Column) (left
for _, otherCond := range p.OtherConditions {
parentUsedCols = append(parentUsedCols, expression.ExtractColumns(otherCond)...)
}
lChild := p.children[0].(LogicalPlan)
rChild := p.children[1].(LogicalPlan)
lChild := p.children[0]
rChild := p.children[1]
for _, col := range parentUsedCols {
if lChild.Schema().Contains(col) {
leftCols = append(leftCols, col)
Expand All @@ -201,8 +200,8 @@ func (p *LogicalJoin) extractUsedCols(parentUsedCols []*expression.Column) (left
}

func (p *LogicalJoin) mergeSchema() {
lChild := p.children[0].(LogicalPlan)
rChild := p.children[1].(LogicalPlan)
lChild := p.children[0]
rChild := p.children[1]
composedSchema := expression.MergeSchema(lChild.Schema(), rChild.Schema())
if p.JoinType == SemiJoin || p.JoinType == AntiSemiJoin {
p.schema = lChild.Schema().Clone()
Expand All @@ -218,17 +217,17 @@ func (p *LogicalJoin) mergeSchema() {
// PruneColumns implements LogicalPlan interface.
func (p *LogicalJoin) PruneColumns(parentUsedCols []*expression.Column) {
leftCols, rightCols := p.extractUsedCols(parentUsedCols)
lChild := p.children[0].(LogicalPlan)
rChild := p.children[1].(LogicalPlan)
lChild := p.children[0]
rChild := p.children[1]
lChild.PruneColumns(leftCols)
rChild.PruneColumns(rightCols)
p.mergeSchema()
}

// PruneColumns implements LogicalPlan interface.
func (la *LogicalApply) PruneColumns(parentUsedCols []*expression.Column) {
lChild := la.children[0].(LogicalPlan)
rChild := la.children[1].(LogicalPlan)
lChild := la.children[0]
rChild := la.children[1]
leftCols, rightCols := la.extractUsedCols(parentUsedCols)
rChild.PruneColumns(rightCols)
la.extractCorColumnsBySchema()
Expand All @@ -247,6 +246,6 @@ func (p *LogicalLock) PruneColumns(parentUsedCols []*expression.Column) {
for _, cols := range p.children[0].Schema().TblID2Handle {
parentUsedCols = append(parentUsedCols, cols...)
}
p.children[0].(LogicalPlan).PruneColumns(parentUsedCols)
p.children[0].PruneColumns(parentUsedCols)
}
}
6 changes: 3 additions & 3 deletions plan/plans.go → plan/common_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -188,7 +188,7 @@ func (e *Execute) rebuildRange(p Plan) error {
if err != nil {
return errors.Trace(err)
}
default:
case PhysicalPlan:
var err error
for _, child := range x.Children() {
err = e.rebuildRange(child)
Expand Down Expand Up @@ -422,10 +422,10 @@ func (e *Explain) prepareTaskDot(p PhysicalPlan, taskTp string, buffer *bytes.Bu
return
}

var copTasks []Plan
var copTasks []PhysicalPlan
var pipelines []string

for planQueue := []Plan{p}; len(planQueue) > 0; planQueue = planQueue[1:] {
for planQueue := []PhysicalPlan{p}; len(planQueue) > 0; planQueue = planQueue[1:] {
curPlan := planQueue[0]
switch copPlan := curPlan.(type) {
case *PhysicalTableReader:
Expand Down
4 changes: 2 additions & 2 deletions plan/decorrelate.go
Original file line number Diff line number Diff line change
Expand Up @@ -154,9 +154,9 @@ func (s *decorrelateSolver) optimize(p LogicalPlan, _ context.Context) (LogicalP
}
}
}
newChildren := make([]Plan, 0, len(p.Children()))
newChildren := make([]LogicalPlan, 0, len(p.Children()))
for _, child := range p.Children() {
np, err := s.optimize(child.(LogicalPlan), nil)
np, err := s.optimize(child, nil)
if err != nil {
return nil, errors.Trace(err)
}
Expand Down
4 changes: 2 additions & 2 deletions plan/eliminate_projection.go
Original file line number Diff line number Diff line change
Expand Up @@ -120,9 +120,9 @@ func (pe *projectionEliminater) eliminate(p LogicalPlan, replace map[string]*exp

switch x := p.(type) {
case *LogicalJoin:
x.schema = buildJoinSchema(x.JoinType, x)
x.schema = buildLogicalJoinSchema(x.JoinType, x)
case *LogicalApply:
x.schema = buildJoinSchema(x.JoinType, x)
x.schema = buildLogicalJoinSchema(x.JoinType, x)
default:
for _, dst := range p.Schema().Columns {
resolveColumnAndReplace(dst, replace)
Expand Down
2 changes: 1 addition & 1 deletion plan/gen_physical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -244,7 +244,7 @@ func (p *LogicalJoin) constructIndexJoin(prop *requiredProp, innerJoinKeys, oute
// Then, we will extract the join keys of p's equal conditions. Then check whether all of them are just the primary key
// or match some part of on index. If so we will choose the best one and construct a index join.
func (p *LogicalJoin) getIndexJoinByOuterIdx(prop *requiredProp, outerIdx int) []PhysicalPlan {
innerChild := p.children[1-outerIdx].(LogicalPlan)
innerChild := p.children[1-outerIdx]
var (
innerJoinKeys []*expression.Column
outerJoinKeys []*expression.Column
Expand Down
4 changes: 2 additions & 2 deletions plan/join_reorder.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,8 +32,8 @@ func tryToGetJoinGroup(j *LogicalJoin) ([]LogicalPlan, bool) {
if j.reordered || !j.cartesianJoin || bits.OnesCount(j.preferJoinType) > 0 {
return nil, false
}
lChild := j.children[0].(LogicalPlan)
rChild := j.children[1].(LogicalPlan)
lChild := j.children[0]
rChild := j.children[1]
if nj, ok := lChild.(*LogicalJoin); ok {
plans, valid := tryToGetJoinGroup(nj)
return append(plans, rChild), valid
Expand Down
5 changes: 3 additions & 2 deletions plan/logical_plan_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -639,7 +639,7 @@ func (b *planBuilder) buildProjection4Union(u *LogicalUnionAll) {

func (b *planBuilder) buildUnion(union *ast.UnionStmt) LogicalPlan {
u := LogicalUnionAll{}.init(b.ctx)
u.children = make([]Plan, len(union.SelectList.Selects))
u.children = make([]LogicalPlan, len(union.SelectList.Selects))
for i, sel := range union.SelectList.Selects {
u.children[i] = b.buildSelect(sel)
if b.err != nil {
Expand Down Expand Up @@ -1859,6 +1859,7 @@ func (b *planBuilder) buildUpdate(update *ast.UpdateStmt) Plan {
}.init(b.ctx)
updt.SetSchema(p.Schema())
updt.SelectPlan, b.err = doOptimize(b.optFlag, p, b.ctx)
updt.ResolveIndices()
return updt
}

Expand Down Expand Up @@ -1945,7 +1946,7 @@ func (b *planBuilder) buildUpdateLists(tableList []*ast.TableName, list []*ast.A
}

// extractTableAsNameForUpdate extracts tables' alias names for update.
func extractTableAsNameForUpdate(p Plan, asNames map[*model.TableInfo][]*model.CIStr) {
func extractTableAsNameForUpdate(p LogicalPlan, asNames map[*model.TableInfo][]*model.CIStr) {
switch x := p.(type) {
case *DataSource:
alias := extractTableAlias(p.(LogicalPlan))
Expand Down
12 changes: 6 additions & 6 deletions plan/logical_plan_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -952,9 +952,9 @@ func (s *testPlanSuite) TestColumnPruning(c *C) {

p, err := BuildLogicalPlan(s.ctx, stmt, s.is)
c.Assert(err, IsNil)
p, err = logicalOptimize(flagPredicatePushDown|flagPrunColumns, p.(LogicalPlan), s.ctx)
lp, err := logicalOptimize(flagPredicatePushDown|flagPrunColumns, p.(LogicalPlan), s.ctx)
c.Assert(err, IsNil)
checkDataSourceCols(p, c, tt.ans, comment)
checkDataSourceCols(lp, c, tt.ans, comment)
}
}

Expand All @@ -965,7 +965,7 @@ func (s *testPlanSuite) TestAllocID(c *C) {
c.Assert(pA.id+1, Equals, pB.id)
}

func checkDataSourceCols(p Plan, c *C, ans map[int][]string, comment CommentInterface) {
func checkDataSourceCols(p LogicalPlan, c *C, ans map[int][]string, comment CommentInterface) {
switch p.(type) {
case *DataSource:
colList, ok := ans[p.ID()]
Expand Down Expand Up @@ -1118,7 +1118,7 @@ func (s *testPlanSuite) TestValidate(c *C) {
}
}

func checkUniqueKeys(p Plan, c *C, ans map[int][][]string, sql string) {
func checkUniqueKeys(p LogicalPlan, c *C, ans map[int][][]string, sql string) {
keyList, ok := ans[p.ID()]
c.Assert(ok, IsTrue, Commentf("for %s, %v not found", sql, p.ID()))
c.Assert(len(p.Schema().Keys), Equals, len(keyList), Commentf("for %s, %v, the number of key doesn't match, the schema is %s", sql, p.ID(), p.Schema()))
Expand Down Expand Up @@ -1207,9 +1207,9 @@ func (s *testPlanSuite) TestUniqueKeyInfo(c *C) {

p, err := BuildLogicalPlan(s.ctx, stmt, s.is)
c.Assert(err, IsNil)
p, err = logicalOptimize(flagPredicatePushDown|flagPrunColumns|flagBuildKeyInfo, p.(LogicalPlan), s.ctx)
lp, err := logicalOptimize(flagPredicatePushDown|flagPrunColumns|flagBuildKeyInfo, p.(LogicalPlan), s.ctx)
c.Assert(err, IsNil)
checkUniqueKeys(p, c, tt.ans, tt.sql)
checkUniqueKeys(lp, c, tt.ans, tt.sql)
}
}

Expand Down
2 changes: 1 addition & 1 deletion plan/max_min_eliminate.go
Original file line number Diff line number Diff line change
Expand Up @@ -81,6 +81,6 @@ func (a *maxMinEliminator) eliminateMaxMin(p LogicalPlan) {
}

for _, child := range p.Children() {
a.eliminateMaxMin(child.(LogicalPlan))
a.eliminateMaxMin(child)
}
}
1 change: 0 additions & 1 deletion plan/optimizer.go
Original file line number Diff line number Diff line change
Expand Up @@ -81,7 +81,6 @@ func Optimize(ctx context.Context, node ast.Node, is infoschema.InfoSchema) (Pla
if logic, ok := p.(LogicalPlan); ok {
return doOptimize(builder.optFlag, logic, ctx)
}
p.ResolveIndices()
if execPlan, ok := p.(*Execute); ok {
err := execPlan.optimizePreparedPlan(ctx, is)
return p, errors.Trace(err)
Expand Down
6 changes: 3 additions & 3 deletions plan/physical_plan_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -102,9 +102,9 @@ func (p *baseLogicalPlan) convert2PhysicalPlan(prop *requiredProp) (t task, err
}

func (p *baseLogicalPlan) getBestTask(bestTask task, pp PhysicalPlan) (task, error) {
tasks := make([]task, 0, len(p.basePlan.children))
for i, child := range p.basePlan.children {
childTask, err := child.(LogicalPlan).convert2PhysicalPlan(pp.getChildReqProps(i))
tasks := make([]task, 0, len(p.children))
for i, child := range p.children {
childTask, err := child.convert2PhysicalPlan(pp.getChildReqProps(i))
if err != nil {
return nil, errors.Trace(err)
}
Expand Down
Loading

0 comments on commit 3cff21e

Please sign in to comment.