Skip to content

Commit

Permalink
plan, expression: remove useless error return (pingcap#5085)
Browse files Browse the repository at this point in the history
  • Loading branch information
hanfei1991 authored Nov 13, 2017
1 parent 66c19de commit 5c76863
Show file tree
Hide file tree
Showing 12 changed files with 105 additions and 213 deletions.
7 changes: 1 addition & 6 deletions expression/aggregation/count.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,6 @@
package aggregation

import (
log "github.com/Sirupsen/logrus"
"github.com/juju/errors"
"github.com/pingcap/tidb/context"
"github.com/pingcap/tidb/expression"
Expand All @@ -39,11 +38,7 @@ func (cf *countFunction) Clone() Aggregation {
// CalculateDefaultValue implements Aggregation interface.
func (cf *countFunction) CalculateDefaultValue(schema *expression.Schema, ctx context.Context) (d types.Datum, valid bool) {
for _, arg := range cf.Args {
result, err := expression.EvaluateExprWithNull(ctx, schema, arg)
if err != nil {
log.Warnf("Evaluate expr with null failed in function %s, err msg is %s", cf, err.Error())
return d, false
}
result := expression.EvaluateExprWithNull(ctx, schema, arg)
if con, ok := result.(*expression.Constant); ok {
if con.Value.IsNull() {
return types.NewDatum(0), true
Expand Down
7 changes: 1 addition & 6 deletions expression/aggregation/first_row.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,6 @@
package aggregation

import (
log "github.com/Sirupsen/logrus"
"github.com/juju/errors"
"github.com/pingcap/tidb/context"
"github.com/pingcap/tidb/expression"
Expand Down Expand Up @@ -70,11 +69,7 @@ func (ff *firstRowFunction) GetPartialResult(ctx *AggEvaluateContext) []types.Da
// CalculateDefaultValue implements Aggregation interface.
func (ff *firstRowFunction) CalculateDefaultValue(schema *expression.Schema, ctx context.Context) (d types.Datum, valid bool) {
arg := ff.Args[0]
result, err := expression.EvaluateExprWithNull(ctx, schema, arg)
if err != nil {
log.Warnf("Evaluate expr with null failed in function %s, err msg is %s", ff, err.Error())
return d, false
}
result := expression.EvaluateExprWithNull(ctx, schema, arg)
if con, ok := result.(*expression.Constant); ok {
return con.Value, true
}
Expand Down
7 changes: 1 addition & 6 deletions expression/aggregation/max_min.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,6 @@
package aggregation

import (
log "github.com/Sirupsen/logrus"
"github.com/juju/errors"
"github.com/pingcap/tidb/context"
"github.com/pingcap/tidb/expression"
Expand All @@ -39,11 +38,7 @@ func (mmf *maxMinFunction) Clone() Aggregation {
// CalculateDefaultValue implements Aggregation interface.
func (mmf *maxMinFunction) CalculateDefaultValue(schema *expression.Schema, ctx context.Context) (d types.Datum, valid bool) {
arg := mmf.Args[0]
result, err := expression.EvaluateExprWithNull(ctx, schema, arg)
if err != nil {
log.Warnf("Evaluate expr with null failed in function %s, err msg is %s", mmf, err.Error())
return d, false
}
result := expression.EvaluateExprWithNull(ctx, schema, arg)
if con, ok := result.(*expression.Constant); ok {
return con.Value, true
}
Expand Down
7 changes: 2 additions & 5 deletions expression/aggregation/sum.go
Original file line number Diff line number Diff line change
Expand Up @@ -62,12 +62,9 @@ func (sf *sumFunction) GetPartialResult(ctx *AggEvaluateContext) []types.Datum {
// CalculateDefaultValue implements Aggregation interface.
func (sf *sumFunction) CalculateDefaultValue(schema *expression.Schema, ctx context.Context) (d types.Datum, valid bool) {
arg := sf.Args[0]
result, err := expression.EvaluateExprWithNull(ctx, schema, arg)
if err != nil {
log.Warnf("Evaluate expr with null failed in function %s, err msg is %s", sf, err.Error())
return d, false
}
result := expression.EvaluateExprWithNull(ctx, schema, arg)
if con, ok := result.(*expression.Constant); ok {
var err error
d, err = calculateSum(ctx.GetSessionVars().StmtCtx, d, con.Value)
if err != nil {
log.Warnf("CalculateSum failed in function %s, err msg is %s", sf, err.Error())
Expand Down
24 changes: 7 additions & 17 deletions expression/expression.go
Original file line number Diff line number Diff line change
Expand Up @@ -215,35 +215,25 @@ func SplitDNFItems(onExpr Expression) []Expression {

// EvaluateExprWithNull sets columns in schema as null and calculate the final result of the scalar function.
// If the Expression is a non-constant value, it means the result is unknown.
func EvaluateExprWithNull(ctx context.Context, schema *Schema, expr Expression) (Expression, error) {
func EvaluateExprWithNull(ctx context.Context, schema *Schema, expr Expression) Expression {
switch x := expr.(type) {
case *ScalarFunction:
var err error
args := make([]Expression, len(x.GetArgs()))
for i, arg := range x.GetArgs() {
args[i], err = EvaluateExprWithNull(ctx, schema, arg)
if err != nil {
return nil, errors.Trace(err)
}
}
newFunc, err := NewFunction(ctx, x.FuncName.L, types.NewFieldType(mysql.TypeTiny), args...)
if err != nil {
return nil, errors.Trace(err)
args[i] = EvaluateExprWithNull(ctx, schema, arg)
}
return newFunc, nil
return NewFunctionInternal(ctx, x.FuncName.L, types.NewFieldType(mysql.TypeTiny), args...)
case *Column:
if !schema.Contains(x) {
return x, nil
return x
}
constant := &Constant{Value: types.Datum{}, RetType: types.NewFieldType(mysql.TypeNull)}
return constant, nil
return &Constant{Value: types.Datum{}, RetType: types.NewFieldType(mysql.TypeNull)}
case *Constant:
if x.DeferredExpr != nil {
newConst := FoldConstant(x)
return newConst, nil
return FoldConstant(x)
}
}
return expr.Clone(), nil
return expr.Clone()
}

// TableInfo2Schema converts table info to schema with empty DBName.
Expand Down
6 changes: 2 additions & 4 deletions expression/expression_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,14 +42,12 @@ func (s *testEvaluatorSuite) TestEvaluateExprWithNull(c *C) {

// ifnull(null, ifnull(col1, 1))
schema := &Schema{Columns: []*Column{col0}}
res, err := EvaluateExprWithNull(s.ctx, schema, ifnullOuter)
c.Assert(err, IsNil)
res := EvaluateExprWithNull(s.ctx, schema, ifnullOuter)
c.Assert(res.String(), Equals, "ifnull(<nil>, ifnull(col1, 1))")

schema.Columns = append(schema.Columns, col1)
// ifnull(null, ifnull(null, 1))
res, err = EvaluateExprWithNull(s.ctx, schema, ifnullOuter)
c.Assert(err, IsNil)
res = EvaluateExprWithNull(s.ctx, schema, ifnullOuter)
c.Assert(res.Equal(One, s.ctx), IsTrue)
}

Expand Down
5 changes: 1 addition & 4 deletions plan/decorrelate.go
Original file line number Diff line number Diff line change
Expand Up @@ -70,10 +70,7 @@ func (a *LogicalAggregation) canPullUp() bool {
}
for _, f := range a.AggFuncs {
for _, arg := range f.GetArgs() {
expr, err := expression.EvaluateExprWithNull(a.ctx, a.children[0].Schema(), arg)
if err != nil {
return false
}
expr := expression.EvaluateExprWithNull(a.ctx, a.children[0].Schema(), arg)
if con, ok := expr.(*expression.Constant); !ok || !con.Value.IsNull() {
return false
}
Expand Down
8 changes: 2 additions & 6 deletions plan/eliminate_projection.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,10 +14,8 @@
package plan

import (
"github.com/juju/errors"
"github.com/pingcap/tidb/context"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/terror"
)

// canProjectionBeEliminatedLoose checks whether a projection can be eliminated, returns true if
Expand Down Expand Up @@ -83,8 +81,7 @@ func doPhysicalProjectionElimination(p PhysicalPlan) PhysicalPlan {
return p
}
child := p.Children()[0]
err := RemovePlan(p)
terror.Log(errors.Trace(err))
removePlan(p)
return child.(PhysicalPlan)
}

Expand Down Expand Up @@ -166,8 +163,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)
}
err := RemovePlan(p)
terror.Log(errors.Trace(err))
removePlan(p)
return child.(LogicalPlan)
}

Expand Down
33 changes: 0 additions & 33 deletions plan/logical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,6 @@
package plan

import (
"github.com/juju/errors"
"github.com/pingcap/tidb/ast"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/expression/aggregation"
Expand Down Expand Up @@ -407,35 +406,3 @@ type Delete struct {
Tables []*ast.TableName
IsMultiTable bool
}

// setParentAndChildren sets parent and children relationship.
func setParentAndChildren(parent Plan, children ...Plan) {
if children == nil || parent == nil {
return
}
for _, child := range children {
child.SetParents(parent)
}
parent.SetChildren(children...)
}

// RemovePlan means removing a plan.
func RemovePlan(p Plan) error {
parents := p.Parents()
children := p.Children()
if len(parents) > 1 || len(children) != 1 {
return SystemInternalErrorType.Gen("can't remove this plan")
}
if len(parents) == 0 {
child := children[0]
child.SetParents()
return nil
}
parent, child := parents[0], children[0]
err := parent.ReplaceChild(p, child)
if err != nil {
return errors.Trace(err)
}
err = child.ReplaceParent(p, parent)
return errors.Trace(err)
}
44 changes: 1 addition & 43 deletions plan/plan.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,10 +30,6 @@ import (
// It is created from ast.Node first, then optimized by the optimizer,
// finally used by the executor to create a Cursor which executes the statement.
type Plan interface {
// ReplaceParent means replacing a parent with another one.
ReplaceParent(parent, newPar Plan) error
// ReplaceChild means replacing a child with another one.
ReplaceChild(children, newChild Plan) error
// Get all the parents.
Parents() []Plan
// Get all the children.
Expand Down Expand Up @@ -156,7 +152,7 @@ type LogicalPlan interface {
// PredicatePushDown pushes down the predicates in the where/on/having clauses as deeply as possible.
// It will accept a predicate that is an expression slice, and return the expressions that can't be pushed.
// Because it might change the root if the having clause exists, we need to return a plan that represents a new root.
PredicatePushDown([]expression.Expression) ([]expression.Expression, LogicalPlan, error)
PredicatePushDown([]expression.Expression) ([]expression.Expression, LogicalPlan)

// PruneColumns prunes the unused columns.
PruneColumns([]*expression.Column)
Expand Down Expand Up @@ -275,22 +271,6 @@ func newBasePhysicalPlan(basePlan *basePlan) basePhysicalPlan {
}
}

// PredicatePushDown implements LogicalPlan interface.
func (p *baseLogicalPlan) PredicatePushDown(predicates []expression.Expression) ([]expression.Expression, LogicalPlan, error) {
if len(p.basePlan.children) == 0 {
return predicates, p.basePlan.self.(LogicalPlan), nil
}
child := p.basePlan.children[0].(LogicalPlan)
rest, _, err := child.PredicatePushDown(predicates)
if err != nil {
return nil, nil, errors.Trace(err)
}
if len(rest) > 0 {
addSelection(p.basePlan.self, child, rest, p.basePlan.allocator)
}
return nil, p.basePlan.self.(LogicalPlan), nil
}

func (p *baseLogicalPlan) extractCorrelatedCols() []*expression.CorrelatedColumn {
var corCols []*expression.CorrelatedColumn
for _, child := range p.basePlan.children {
Expand Down Expand Up @@ -373,28 +353,6 @@ func (p *basePlan) Schema() *expression.Schema {
return p.schema
}

// ReplaceParent means replace a parent for another one.
func (p *basePlan) ReplaceParent(parent, newPar Plan) error {
for i, par := range p.parents {
if par.ID() == parent.ID() {
p.parents[i] = newPar
return nil
}
}
return SystemInternalErrorType.Gen("ReplaceParent Failed: parent \"%s\" not found", parent.ExplainID())
}

// ReplaceChild means replace a child with another one.
func (p *basePlan) ReplaceChild(child, newChild Plan) error {
for i, ch := range p.children {
if ch.ID() == child.ID() {
p.children[i] = newChild
return nil
}
}
return SystemInternalErrorType.Gen("ReplaceChildren Failed: child \"%s\" not found", child.ExplainID())
}

// Parents implements Plan Parents interface.
func (p *basePlan) Parents() []Plan {
return p.parents
Expand Down
Loading

0 comments on commit 5c76863

Please sign in to comment.