Skip to content

Commit

Permalink
*: Add Row interface (pingcap#4859)
Browse files Browse the repository at this point in the history
  • Loading branch information
coocood authored and winoros committed Oct 23, 2017
1 parent befb615 commit ed7f68c
Show file tree
Hide file tree
Showing 36 changed files with 745 additions and 585 deletions.
2 changes: 1 addition & 1 deletion executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -95,7 +95,7 @@ const (
// If there is sort need in the double read, then the table scan of the double read must store the handle.
// If there is a select for update. then we need to store the handle until the lock plan. But if there is aggregation, the handle info can be removed.
// Otherwise the executor's returned rows don't need to store the handle information.
type Row []types.Datum
type Row = types.DatumRow

type baseExecutor struct {
children []Executor
Expand Down
2 changes: 1 addition & 1 deletion executor/union_scan.go
Original file line number Diff line number Diff line change
Expand Up @@ -251,7 +251,7 @@ func (us *UnionScanExec) compare(a, b Row) (int, error) {
func (us *UnionScanExec) buildAndSortAddedRows(t table.Table) error {
us.addedRows = make([]Row, 0, len(us.dirty.addedRows))
for h, data := range us.dirty.addedRows {
newData := make([]types.Datum, 0, us.schema.Len())
newData := make(types.DatumRow, 0, us.schema.Len())
for _, col := range us.columns {
if col.ID == model.ExtraHandleID {
newData = append(newData, types.NewIntDatum(h))
Expand Down
8 changes: 4 additions & 4 deletions executor/write.go
Original file line number Diff line number Diff line change
Expand Up @@ -910,7 +910,7 @@ func (e *InsertValues) getRows(cols []*table.Column, ignoreErr bool) (rows [][]t
// getRow eval the insert statement. Because the value of column may calculated based on other column,
// it use fillDefaultValues to init the empty row before eval expressions when needFillDefaultValues is true.
func (e *InsertValues) getRow(cols []*table.Column, list []expression.Expression, ignoreErr bool) ([]types.Datum, error) {
row := make([]types.Datum, len(e.Table.Cols()))
row := make(types.DatumRow, len(e.Table.Cols()))
hasValue := make([]bool, len(e.Table.Cols()))

if e.needFillDefaultValues {
Expand Down Expand Up @@ -1000,7 +1000,7 @@ func (e *InsertValues) fillRowData(cols []*table.Column, vals []types.Datum, ign
return e.fillGenColData(cols, len(vals), hasValue, row, ignoreErr)
}

func (e *InsertValues) fillGenColData(cols []*table.Column, valLen int, hasValue []bool, row []types.Datum, ignoreErr bool) ([]types.Datum, error) {
func (e *InsertValues) fillGenColData(cols []*table.Column, valLen int, hasValue []bool, row types.DatumRow, ignoreErr bool) ([]types.Datum, error) {
err := e.initDefaultValues(row, hasValue, ignoreErr)
if err != nil {
return nil, errors.Trace(err)
Expand Down Expand Up @@ -1155,7 +1155,7 @@ func (e *InsertExec) onDuplicateUpdate(row []types.Datum, h int64, cols []*expre

// evaluate assignment
assignFlag := make([]bool, len(e.Table.WritableCols()))
newData := make([]types.Datum, len(data))
newData := make(types.DatumRow, len(data))
copy(newData, data)
for _, col := range cols {
val, err1 := col.Expr.Eval(newData)
Expand Down Expand Up @@ -1387,7 +1387,7 @@ func (e *UpdateExec) fetchRows() error {
if row == nil {
return nil
}
newRowData := make([]types.Datum, len(row))
newRowData := make(types.DatumRow, len(row))
copy(newRowData, row)
for _, assign := range e.OrderedList {
val, err := assign.Expr.Eval(newRowData)
Expand Down
4 changes: 2 additions & 2 deletions expression/aggregation/aggregation.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@ type Aggregation interface {
json.Marshaler

// Update during executing.
Update(ctx *AggEvaluateContext, sc *variable.StatementContext, row []types.Datum) error
Update(ctx *AggEvaluateContext, sc *variable.StatementContext, row types.Row) error

// GetPartialResult will called by coprocessor to get partial results. For avg function, partial results will return
// sum and count values at the same time.
Expand Down Expand Up @@ -245,7 +245,7 @@ func (af *aggFunction) CreateContext() *AggEvaluateContext {
return ctx
}

func (af *aggFunction) updateSum(ctx *AggEvaluateContext, sc *variable.StatementContext, row []types.Datum) error {
func (af *aggFunction) updateSum(ctx *AggEvaluateContext, sc *variable.StatementContext, row types.Row) error {
a := af.Args[0]
value, err := a.Eval(row)
if err != nil {
Expand Down
4 changes: 2 additions & 2 deletions expression/aggregation/avg.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@ func (af *avgFunction) GetType() *types.FieldType {
return ft
}

func (af *avgFunction) updateAvg(ctx *AggEvaluateContext, sc *variable.StatementContext, row []types.Datum) error {
func (af *avgFunction) updateAvg(ctx *AggEvaluateContext, sc *variable.StatementContext, row types.Row) error {
a := af.Args[1]
value, err := a.Eval(row)
if err != nil {
Expand Down Expand Up @@ -73,7 +73,7 @@ func (af *avgFunction) updateAvg(ctx *AggEvaluateContext, sc *variable.Statement
}

// Update implements Aggregation interface.
func (af *avgFunction) Update(ctx *AggEvaluateContext, sc *variable.StatementContext, row []types.Datum) error {
func (af *avgFunction) Update(ctx *AggEvaluateContext, sc *variable.StatementContext, row types.Row) error {
if af.mode == FinalMode {
return af.updateAvg(ctx, sc, row)
}
Expand Down
2 changes: 1 addition & 1 deletion expression/aggregation/concat.go
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,7 @@ func (cf *concatFunction) writeValue(ctx *AggEvaluateContext, val types.Datum) {
}

// Update implements Aggregation interface.
func (cf *concatFunction) Update(ctx *AggEvaluateContext, sc *variable.StatementContext, row []types.Datum) error {
func (cf *concatFunction) Update(ctx *AggEvaluateContext, sc *variable.StatementContext, row types.Row) error {
datumBuf := make([]types.Datum, 0, len(cf.Args))
for _, a := range cf.Args {
value, err := a.Eval(row)
Expand Down
2 changes: 1 addition & 1 deletion expression/aggregation/count.go
Original file line number Diff line number Diff line change
Expand Up @@ -64,7 +64,7 @@ func (cf *countFunction) GetType() *types.FieldType {
}

// Update implements Aggregation interface.
func (cf *countFunction) Update(ctx *AggEvaluateContext, sc *variable.StatementContext, row []types.Datum) error {
func (cf *countFunction) Update(ctx *AggEvaluateContext, sc *variable.StatementContext, row types.Row) error {
var datumBuf []types.Datum
if cf.Distinct {
datumBuf = make([]types.Datum, 0, len(cf.Args))
Expand Down
2 changes: 1 addition & 1 deletion expression/aggregation/first_row.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,7 @@ func (ff *firstRowFunction) GetType() *types.FieldType {
}

// Update implements Aggregation interface.
func (ff *firstRowFunction) Update(ctx *AggEvaluateContext, sc *variable.StatementContext, row []types.Datum) error {
func (ff *firstRowFunction) Update(ctx *AggEvaluateContext, sc *variable.StatementContext, row types.Row) error {
if ctx.GotFirstRow {
return nil
}
Expand Down
2 changes: 1 addition & 1 deletion expression/aggregation/max_min.go
Original file line number Diff line number Diff line change
Expand Up @@ -66,7 +66,7 @@ func (mmf *maxMinFunction) GetPartialResult(ctx *AggEvaluateContext) []types.Dat
}

// Update implements Aggregation interface.
func (mmf *maxMinFunction) Update(ctx *AggEvaluateContext, sc *variable.StatementContext, row []types.Datum) error {
func (mmf *maxMinFunction) Update(ctx *AggEvaluateContext, sc *variable.StatementContext, row types.Row) error {
if len(mmf.Args) != 1 {
return errors.New("Wrong number of args for AggFuncMaxMin")
}
Expand Down
2 changes: 1 addition & 1 deletion expression/aggregation/sum.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@ func (sf *sumFunction) Clone() Aggregation {
}

// Update implements Aggregation interface.
func (sf *sumFunction) Update(ctx *AggEvaluateContext, sc *variable.StatementContext, row []types.Datum) error {
func (sf *sumFunction) Update(ctx *AggEvaluateContext, sc *variable.StatementContext, row types.Row) error {
return sf.updateSum(ctx, sc, row)
}

Expand Down
28 changes: 14 additions & 14 deletions expression/builtin.go
Original file line number Diff line number Diff line change
Expand Up @@ -155,31 +155,31 @@ func (b *baseBuiltinFunc) getArgs() []Expression {
return b.args
}

func (b *baseBuiltinFunc) evalInt(row []types.Datum) (int64, bool, error) {
func (b *baseBuiltinFunc) evalInt(row types.Row) (int64, bool, error) {
panic("baseBuiltinFunc.evalInt() should never be called.")
}

func (b *baseBuiltinFunc) evalReal(row []types.Datum) (float64, bool, error) {
func (b *baseBuiltinFunc) evalReal(row types.Row) (float64, bool, error) {
panic("baseBuiltinFunc.evalReal() should never be called.")
}

func (b *baseBuiltinFunc) evalString(row []types.Datum) (string, bool, error) {
func (b *baseBuiltinFunc) evalString(row types.Row) (string, bool, error) {
panic("baseBuiltinFunc.evalString() should never be called.")
}

func (b *baseBuiltinFunc) evalDecimal(row []types.Datum) (*types.MyDecimal, bool, error) {
func (b *baseBuiltinFunc) evalDecimal(row types.Row) (*types.MyDecimal, bool, error) {
panic("baseBuiltinFunc.evalDecimal() should never be called.")
}

func (b *baseBuiltinFunc) evalTime(row []types.Datum) (types.Time, bool, error) {
func (b *baseBuiltinFunc) evalTime(row types.Row) (types.Time, bool, error) {
panic("baseBuiltinFunc.evalTime() should never be called.")
}

func (b *baseBuiltinFunc) evalDuration(row []types.Datum) (types.Duration, bool, error) {
func (b *baseBuiltinFunc) evalDuration(row types.Row) (types.Duration, bool, error) {
panic("baseBuiltinFunc.evalDuration() should never be called.")
}

func (b *baseBuiltinFunc) evalJSON(row []types.Datum) (json.JSON, bool, error) {
func (b *baseBuiltinFunc) evalJSON(row types.Row) (json.JSON, bool, error) {
panic("baseBuiltinFunc.evalJSON() should never be called.")
}

Expand Down Expand Up @@ -218,19 +218,19 @@ func (b *baseBuiltinFunc) getCtx() context.Context {
// builtinFunc stands for a particular function signature.
type builtinFunc interface {
// evalInt evaluates int result of builtinFunc by given row.
evalInt(row []types.Datum) (val int64, isNull bool, err error)
evalInt(row types.Row) (val int64, isNull bool, err error)
// evalReal evaluates real representation of builtinFunc by given row.
evalReal(row []types.Datum) (val float64, isNull bool, err error)
evalReal(row types.Row) (val float64, isNull bool, err error)
// evalString evaluates string representation of builtinFunc by given row.
evalString(row []types.Datum) (val string, isNull bool, err error)
evalString(row types.Row) (val string, isNull bool, err error)
// evalDecimal evaluates decimal representation of builtinFunc by given row.
evalDecimal(row []types.Datum) (val *types.MyDecimal, isNull bool, err error)
evalDecimal(row types.Row) (val *types.MyDecimal, isNull bool, err error)
// evalTime evaluates DATE/DATETIME/TIMESTAMP representation of builtinFunc by given row.
evalTime(row []types.Datum) (val types.Time, isNull bool, err error)
evalTime(row types.Row) (val types.Time, isNull bool, err error)
// evalDuration evaluates duration representation of builtinFunc by given row.
evalDuration(row []types.Datum) (val types.Duration, isNull bool, err error)
evalDuration(row types.Row) (val types.Duration, isNull bool, err error)
// evalJSON evaluates JSON representation of builtinFunc by given row.
evalJSON(row []types.Datum) (val json.JSON, isNull bool, err error)
evalJSON(row types.Row) (val json.JSON, isNull bool, err error)
// getArgs returns the arguments expressions.
getArgs() []Expression
// equal check if this function equals to another function.
Expand Down
34 changes: 17 additions & 17 deletions expression/builtin_arithmetic.go
Original file line number Diff line number Diff line change
Expand Up @@ -168,7 +168,7 @@ type builtinArithmeticPlusIntSig struct {
baseBuiltinFunc
}

func (s *builtinArithmeticPlusIntSig) evalInt(row []types.Datum) (val int64, isNull bool, err error) {
func (s *builtinArithmeticPlusIntSig) evalInt(row types.Row) (val int64, isNull bool, err error) {
sc := s.ctx.GetSessionVars().StmtCtx

a, isNull, err := s.args[0].EvalInt(row, sc)
Expand Down Expand Up @@ -216,7 +216,7 @@ type builtinArithmeticPlusDecimalSig struct {
baseBuiltinFunc
}

func (s *builtinArithmeticPlusDecimalSig) evalDecimal(row []types.Datum) (*types.MyDecimal, bool, error) {
func (s *builtinArithmeticPlusDecimalSig) evalDecimal(row types.Row) (*types.MyDecimal, bool, error) {
sc := s.ctx.GetSessionVars().StmtCtx
a, isNull, err := s.args[0].EvalDecimal(row, sc)
if isNull || err != nil {
Expand All @@ -238,7 +238,7 @@ type builtinArithmeticPlusRealSig struct {
baseBuiltinFunc
}

func (s *builtinArithmeticPlusRealSig) evalReal(row []types.Datum) (float64, bool, error) {
func (s *builtinArithmeticPlusRealSig) evalReal(row types.Row) (float64, bool, error) {
sc := s.ctx.GetSessionVars().StmtCtx
a, isNull, err := s.args[0].EvalReal(row, sc)
if isNull || err != nil {
Expand Down Expand Up @@ -292,7 +292,7 @@ type builtinArithmeticMinusRealSig struct {
baseBuiltinFunc
}

func (s *builtinArithmeticMinusRealSig) evalReal(row []types.Datum) (float64, bool, error) {
func (s *builtinArithmeticMinusRealSig) evalReal(row types.Row) (float64, bool, error) {
sc := s.ctx.GetSessionVars().StmtCtx
a, isNull, err := s.args[0].EvalReal(row, sc)
if isNull || err != nil {
Expand All @@ -312,7 +312,7 @@ type builtinArithmeticMinusDecimalSig struct {
baseBuiltinFunc
}

func (s *builtinArithmeticMinusDecimalSig) evalDecimal(row []types.Datum) (*types.MyDecimal, bool, error) {
func (s *builtinArithmeticMinusDecimalSig) evalDecimal(row types.Row) (*types.MyDecimal, bool, error) {
sc := s.ctx.GetSessionVars().StmtCtx
a, isNull, err := s.args[0].EvalDecimal(row, sc)
if isNull || err != nil {
Expand All @@ -334,7 +334,7 @@ type builtinArithmeticMinusIntSig struct {
baseBuiltinFunc
}

func (s *builtinArithmeticMinusIntSig) evalInt(row []types.Datum) (val int64, isNull bool, err error) {
func (s *builtinArithmeticMinusIntSig) evalInt(row types.Row) (val int64, isNull bool, err error) {
sc := s.ctx.GetSessionVars().StmtCtx

a, isNull, err := s.args[0].EvalInt(row, sc)
Expand Down Expand Up @@ -418,7 +418,7 @@ type builtinArithmeticMultiplyDecimalSig struct{ baseBuiltinFunc }
type builtinArithmeticMultiplyIntUnsignedSig struct{ baseBuiltinFunc }
type builtinArithmeticMultiplyIntSig struct{ baseBuiltinFunc }

func (s *builtinArithmeticMultiplyRealSig) evalReal(row []types.Datum) (float64, bool, error) {
func (s *builtinArithmeticMultiplyRealSig) evalReal(row types.Row) (float64, bool, error) {
sc := s.ctx.GetSessionVars().StmtCtx
a, isNull, err := s.args[0].EvalReal(row, sc)
if isNull || err != nil {
Expand All @@ -435,7 +435,7 @@ func (s *builtinArithmeticMultiplyRealSig) evalReal(row []types.Datum) (float64,
return result, false, nil
}

func (s *builtinArithmeticMultiplyDecimalSig) evalDecimal(row []types.Datum) (*types.MyDecimal, bool, error) {
func (s *builtinArithmeticMultiplyDecimalSig) evalDecimal(row types.Row) (*types.MyDecimal, bool, error) {
sc := s.ctx.GetSessionVars().StmtCtx
a, isNull, err := s.args[0].EvalDecimal(row, sc)
if isNull || err != nil {
Expand All @@ -453,7 +453,7 @@ func (s *builtinArithmeticMultiplyDecimalSig) evalDecimal(row []types.Datum) (*t
return c, false, nil
}

func (s *builtinArithmeticMultiplyIntUnsignedSig) evalInt(row []types.Datum) (val int64, isNull bool, err error) {
func (s *builtinArithmeticMultiplyIntUnsignedSig) evalInt(row types.Row) (val int64, isNull bool, err error) {
sc := s.ctx.GetSessionVars().StmtCtx
a, isNull, err := s.args[0].EvalInt(row, sc)
if isNull || err != nil {
Expand All @@ -472,7 +472,7 @@ func (s *builtinArithmeticMultiplyIntUnsignedSig) evalInt(row []types.Datum) (va
return int64(result), false, nil
}

func (s *builtinArithmeticMultiplyIntSig) evalInt(row []types.Datum) (val int64, isNull bool, err error) {
func (s *builtinArithmeticMultiplyIntSig) evalInt(row types.Row) (val int64, isNull bool, err error) {
sc := s.ctx.GetSessionVars().StmtCtx
a, isNull, err := s.args[0].EvalInt(row, sc)
if isNull || err != nil {
Expand Down Expand Up @@ -515,7 +515,7 @@ func (c *arithmeticDivideFunctionClass) getFunction(ctx context.Context, args []
type builtinArithmeticDivideRealSig struct{ baseBuiltinFunc }
type builtinArithmeticDivideDecimalSig struct{ baseBuiltinFunc }

func (s *builtinArithmeticDivideRealSig) evalReal(row []types.Datum) (float64, bool, error) {
func (s *builtinArithmeticDivideRealSig) evalReal(row types.Row) (float64, bool, error) {
sc := s.ctx.GetSessionVars().StmtCtx
a, isNull, err := s.args[0].EvalReal(row, sc)
if isNull || err != nil {
Expand All @@ -535,7 +535,7 @@ func (s *builtinArithmeticDivideRealSig) evalReal(row []types.Datum) (float64, b
return result, false, nil
}

func (s *builtinArithmeticDivideDecimalSig) evalDecimal(row []types.Datum) (*types.MyDecimal, bool, error) {
func (s *builtinArithmeticDivideDecimalSig) evalDecimal(row types.Row) (*types.MyDecimal, bool, error) {
sc := s.ctx.GetSessionVars().StmtCtx
a, isNull, err := s.args[0].EvalDecimal(row, sc)
if isNull || err != nil {
Expand Down Expand Up @@ -585,7 +585,7 @@ func (c *arithmeticIntDivideFunctionClass) getFunction(ctx context.Context, args
type builtinArithmeticIntDivideIntSig struct{ baseBuiltinFunc }
type builtinArithmeticIntDivideDecimalSig struct{ baseBuiltinFunc }

func (s *builtinArithmeticIntDivideIntSig) evalInt(row []types.Datum) (int64, bool, error) {
func (s *builtinArithmeticIntDivideIntSig) evalInt(row types.Row) (int64, bool, error) {
sc := s.ctx.GetSessionVars().StmtCtx
b, isNull, err := s.args[1].EvalInt(row, sc)
if isNull || err != nil {
Expand Down Expand Up @@ -624,7 +624,7 @@ func (s *builtinArithmeticIntDivideIntSig) evalInt(row []types.Datum) (int64, bo
return ret, err != nil, errors.Trace(err)
}

func (s *builtinArithmeticIntDivideDecimalSig) evalInt(row []types.Datum) (int64, bool, error) {
func (s *builtinArithmeticIntDivideDecimalSig) evalInt(row types.Row) (int64, bool, error) {
sc := s.ctx.GetSessionVars().StmtCtx
a, isNull, err := s.args[0].EvalDecimal(row, sc)
if isNull || err != nil {
Expand Down Expand Up @@ -715,7 +715,7 @@ type builtinArithmeticModRealSig struct {
baseBuiltinFunc
}

func (s *builtinArithmeticModRealSig) evalReal(row []types.Datum) (float64, bool, error) {
func (s *builtinArithmeticModRealSig) evalReal(row types.Row) (float64, bool, error) {
sc := s.ctx.GetSessionVars().StmtCtx
b, isNull, err := s.args[1].EvalReal(row, sc)
if isNull || err != nil {
Expand All @@ -738,7 +738,7 @@ type builtinArithmeticModDecimalSig struct {
baseBuiltinFunc
}

func (s *builtinArithmeticModDecimalSig) evalDecimal(row []types.Datum) (*types.MyDecimal, bool, error) {
func (s *builtinArithmeticModDecimalSig) evalDecimal(row types.Row) (*types.MyDecimal, bool, error) {
sc := s.ctx.GetSessionVars().StmtCtx
a, isNull, err := s.args[0].EvalDecimal(row, sc)
if isNull || err != nil {
Expand All @@ -760,7 +760,7 @@ type builtinArithmeticModIntSig struct {
baseBuiltinFunc
}

func (s *builtinArithmeticModIntSig) evalInt(row []types.Datum) (val int64, isNull bool, err error) {
func (s *builtinArithmeticModIntSig) evalInt(row types.Row) (val int64, isNull bool, err error) {
sc := s.ctx.GetSessionVars().StmtCtx

b, isNull, err := s.args[1].EvalInt(row, sc)
Expand Down
Loading

0 comments on commit ed7f68c

Please sign in to comment.