Skip to content

Commit

Permalink
optimizer, executor: remove panic and Fatal, returns error instead.
Browse files Browse the repository at this point in the history
  • Loading branch information
coocood committed Dec 9, 2015
1 parent d5f8228 commit 4f56515
Show file tree
Hide file tree
Showing 13 changed files with 150 additions and 62 deletions.
3 changes: 3 additions & 0 deletions executor/adapter.go
Original file line number Diff line number Diff line change
Expand Up @@ -138,6 +138,9 @@ func (a *statementAdapter) IsDDL() bool {
func (a *statementAdapter) Exec(ctx context.Context) (rset.Recordset, error) {
b := newExecutorBuilder(ctx, a.is)
e := b.build(a.plan)
if b.err != nil {
return nil, errors.Trace(b.err)
}
fields := make([]*field.ResultField, 0, len(e.Fields()))
for _, v := range e.Fields() {
f := &field.ResultField{
Expand Down
7 changes: 4 additions & 3 deletions executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,6 @@
package executor

import (
"github.com/ngaut/log"
"github.com/pingcap/tidb/ast"
"github.com/pingcap/tidb/column"
"github.com/pingcap/tidb/context"
Expand All @@ -29,6 +28,7 @@ import (
type executorBuilder struct {
ctx context.Context
is infoschema.InfoSchema
err error
}

func newExecutorBuilder(ctx context.Context, is infoschema.InfoSchema) *executorBuilder {
Expand Down Expand Up @@ -56,9 +56,10 @@ func (b *executorBuilder) build(p plan.Plan) Executor {
return b.buildSort(v)
case *plan.Limit:
return b.buildLimit(v)
default:
b.err = ErrUnknownPlan.Gen("Unknown Plan %T", p)
return nil
}
log.Fatalf("Unknown Plan %T", p)
return nil
}

func (b *executorBuilder) buildTableScan(v *plan.TableScan) Executor {
Expand Down
49 changes: 32 additions & 17 deletions executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,14 +18,14 @@ import (
"strings"

"github.com/juju/errors"
"github.com/ngaut/log"
"github.com/pingcap/tidb/ast"
"github.com/pingcap/tidb/context"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/optimizer/evaluator"
"github.com/pingcap/tidb/optimizer/plan"
"github.com/pingcap/tidb/sessionctx/forupdate"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/terror"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/types"
)
Expand All @@ -40,6 +40,16 @@ var (
_ Executor = &SortExec{}
)

// Error instances.
var (
ErrUnknownPlan = terror.ClassExecutor.New(CodeUnknownPlan, "Unknown plan")
)

// Error codes.
const (
CodeUnknownPlan terror.ErrCode = iota + 1
)

// Row represents a record row.
type Row struct {
// Data is the output record data for current Plan.
Expand Down Expand Up @@ -195,13 +205,20 @@ func (e *IndexRangeExec) Next() (*Row, error) {
}
val := idxKey[0]
if !e.skipLowCmp {
cmp := indexCompare(val, e.lowVal)
var cmp int
cmp, err = indexCompare(val, e.lowVal)
if err != nil {
return nil, errors.Trace(err)
}
if cmp < 0 || (cmp == 0 && e.lowExclude) {
continue
}
e.skipLowCmp = true
}
cmp := indexCompare(val, e.highVal)
cmp, err := indexCompare(val, e.highVal)
if err != nil {
return nil, errors.Trace(err)
}
if cmp > 0 || (cmp == 0 && e.highExclude) {
// This span has finished iteration.
e.finished = true
Expand All @@ -217,40 +234,38 @@ func (e *IndexRangeExec) Next() (*Row, error) {
}

// comparison function that takes minNotNullVal and maxVal into account.
func indexCompare(a interface{}, b interface{}) int {
func indexCompare(a interface{}, b interface{}) (int, error) {
if a == nil && b == nil {
return 0
return 0, nil
} else if b == nil {
return 1
return 1, nil
} else if a == nil {
return -1
return -1, nil
}

// a and b both not nil
if a == plan.MinNotNullVal && b == plan.MinNotNullVal {
return 0
return 0, nil
} else if b == plan.MinNotNullVal {
return 1
return 1, nil
} else if a == plan.MinNotNullVal {
return -1
return -1, nil
}

// a and b both not min value
if a == plan.MaxVal && b == plan.MaxVal {
return 0
return 0, nil
} else if a == plan.MaxVal {
return 1
return 1, nil
} else if b == plan.MaxVal {
return -1
return -1, nil
}

n, err := types.Compare(a, b)
if err != nil {
// Old compare panics if err, so here we do the same thing now.
// TODO: return err instead of panic.
log.Fatalf("should never happend %v", err)
return 0, errors.Trace(err)
}
return n
return n, nil
}

func (e *IndexRangeExec) lookupRow(h int64) (*Row, error) {
Expand Down
11 changes: 5 additions & 6 deletions optimizer/evaluator/evaluator_binop.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@ import (
"math"

"github.com/juju/errors"
"github.com/ngaut/log"
"github.com/pingcap/tidb/ast"
"github.com/pingcap/tidb/mysql"
"github.com/pingcap/tidb/parser/opcode"
Expand All @@ -41,7 +40,7 @@ func (e *Evaluator) binaryOperation(o *ast.BinaryOperationExpr) bool {
case opcode.Plus, opcode.Minus, opcode.Mod, opcode.Div, opcode.Mul, opcode.IntDiv:
return e.handleArithmeticOp(o)
default:
e.err = errors.New("invalid operation")
e.err = ErrInvalidOperation
return false
}
}
Expand All @@ -55,7 +54,7 @@ func (e *Evaluator) handleLogicOperation(o *ast.BinaryOperationExpr) bool {
case opcode.LogicXor:
return e.handleXor(o)
default:
log.Fatalf("unkown operator %v", o.Op)
e.err = ErrInvalidOperation.Gen("unkown operator %s", o.Op)
return false
}
}
Expand Down Expand Up @@ -203,7 +202,7 @@ func getCompResult(op opcode.Op, value int) (bool, error) {
case opcode.NullEQ:
return value == 0, nil
default:
return false, errors.Errorf("invalid op %v in comparision operation", op)
return false, ErrInvalidOperation.Gen("invalid op %v in comparision operation", op)
}
}

Expand Down Expand Up @@ -240,7 +239,7 @@ func (e *Evaluator) handleBitOp(o *ast.BinaryOperationExpr) bool {
case opcode.LeftShift:
o.SetValue(uint64(x) << uint64(y))
default:
e.err = errors.Errorf("invalid op %v in bit operation", o.Op)
e.err = ErrInvalidOperation.Gen("invalid op %v in bit operation", o.Op)
return false
}
return true
Expand Down Expand Up @@ -282,7 +281,7 @@ func (e *Evaluator) handleArithmeticOp(o *ast.BinaryOperationExpr) bool {
case opcode.IntDiv:
result, e.err = computeIntDiv(a, b)
default:
e.err = errors.Errorf("invalid op %v in arithmetic operation", o.Op)
e.err = ErrInvalidOperation.Gen("invalid op %v in arithmetic operation", o.Op)
return false
}
o.SetValue(result)
Expand Down
10 changes: 8 additions & 2 deletions optimizer/optimizer.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,11 +39,17 @@ func Optimize(is infoschema.InfoSchema, ctx context.Context, node ast.Node) (pla
if err := rewriteStatic(ctx, node); err != nil {
return nil, errors.Trace(err)
}
p := plan.BuildPlan(node)
p, err := plan.BuildPlan(node)
if err != nil {
return nil, errors.Trace(err)
}
bestCost := plan.EstimateCost(p)
bestPlan := p

alts := plan.Alternatives(p)
alts, err := plan.Alternatives(p)
if err != nil {
return nil, errors.Trace(err)
}
for _, alt := range alts {
cost := plan.EstimateCost(alt)
if cost < bestCost {
Expand Down
30 changes: 19 additions & 11 deletions optimizer/plan/alternatives.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,27 +13,32 @@

package plan

import (
"github.com/ngaut/log"
)
import "github.com/juju/errors"

// Alternatives returns multiple alternative plans that
// can be picked base on their cost.
func Alternatives(p Plan) []Plan {
func Alternatives(p Plan) ([]Plan, error) {
var plans []Plan
switch x := p.(type) {
case nil:
case *TableScan:
plans = tableScanAlternatives(x)
case WithSrcPlan:
plans = planWithSrcAlternatives(x)
var err error
plans, err = planWithSrcAlternatives(x)
if err != nil {
return nil, errors.Trace(err)
}
default:
log.Fatalf("unknown plan %T", p)
return nil, ErrUnsupportedType.Gen("Unknown plan %T", p)
}
for _, val := range plans {
refine(val)
err := refine(val)
if err != nil {
return nil, errors.Trace(err)
}
}
return plans
return plans, nil
}

// tableScanAlternatives returns all index plans from the same table.
Expand All @@ -57,14 +62,17 @@ func tableScanAlternatives(p *TableScan) []Plan {

// planWithSrcAlternatives shallow copies the WithSrcPlan,
// and set its src to src alternatives.
func planWithSrcAlternatives(p WithSrcPlan) []Plan {
srcs := Alternatives(p.Src())
func planWithSrcAlternatives(p WithSrcPlan) ([]Plan, error) {
srcs, err := Alternatives(p.Src())
if err != nil {
return nil, errors.Trace(err)
}
for i, val := range srcs {
alt := shallowCopy(p)
alt.SetSrc(val)
srcs[i] = alt
}
return srcs
return srcs, nil
}

func shallowCopy(p WithSrcPlan) WithSrcPlan {
Expand Down
10 changes: 5 additions & 5 deletions optimizer/plan/explainer.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,19 +16,18 @@ package plan
import (
"fmt"
"strings"

"github.com/ngaut/log"
)

// Explain explains a Plan, returns description string.
func Explain(p Plan) string {
func Explain(p Plan) (string, error) {
var e explainer
p.Accept(&e)
return strings.Join(e.strs, "->")
return strings.Join(e.strs, "->"), e.err
}

type explainer struct {
strs []string
err error
}

func (e *explainer) Enter(in Plan) (Plan, bool) {
Expand Down Expand Up @@ -56,7 +55,8 @@ func (e *explainer) Leave(in Plan) (Plan, bool) {
case *Limit:
str = "Limit"
default:
log.Fatalf("Unknown plan type %T", in)
e.err = ErrUnsupportedType.Gen("Unknown plan type %T", in)
return in, false
}
e.strs = append(e.strs, str)
return in, true
Expand Down
18 changes: 13 additions & 5 deletions optimizer/plan/plan_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -194,6 +194,7 @@ func (s *testPlanSuite) TestRangeBuilder(c *C) {
c.Assert(rc, Equals, 0, Commentf("error %v, for expr %", lexer.Errors(), ca.exprStr))
stmt := lexer.Stmts()[0].(*ast.SelectStmt)
result := rb.build(stmt.Where)
c.Assert(rb.err, IsNil)
got := fmt.Sprintf("%v", result)
c.Assert(got, Equals, ca.resultStr, Commentf("differen for expr %s", ca.exprStr))
}
Expand Down Expand Up @@ -240,8 +241,11 @@ func (s *testPlanSuite) TestBuilder(c *C) {
c.Assert(rc, Equals, 0, Commentf("error %v for expr %s", lexer.Errors(), ca.sqlStr))
stmt := lexer.Stmts()[0].(*ast.SelectStmt)
mockResolve(stmt)
p := BuildPlan(stmt)
c.Assert(ca.planStr, Equals, Explain(p))
p, err := BuildPlan(stmt)
c.Assert(err, IsNil)
explainStr, err := Explain(p)
c.Assert(err, IsNil)
c.Assert(ca.planStr, Equals, explainStr)
}
}

Expand Down Expand Up @@ -298,18 +302,22 @@ func (s *testPlanSuite) TestBestPlan(c *C) {
c.Assert(rc, Equals, 0, Commentf("error %v for sql %s", lexer.Errors(), ca.sql))
stmt := lexer.Stmts()[0].(*ast.SelectStmt)
mockResolve(stmt)
p := BuildPlan(stmt)
p, err := BuildPlan(stmt)
c.Assert(err, IsNil)
bestCost := EstimateCost(p)
bestPlan := p
alts := Alternatives(p)
alts, err := Alternatives(p)
c.Assert(err, IsNil)
for _, alt := range alts {
cost := EstimateCost(alt)
if cost < bestCost {
bestCost = cost
bestPlan = alt
}
}
c.Assert(Explain(bestPlan), Equals, ca.best, Commentf("for %s cost %v", ca.sql, bestCost))
explainStr, err := Explain(bestPlan)
c.Assert(err, IsNil)
c.Assert(explainStr, Equals, ca.best, Commentf("for %s cost %v", ca.sql, bestCost))
}
}

Expand Down
Loading

0 comments on commit 4f56515

Please sign in to comment.