Skip to content

Commit

Permalink
*: cut off duration.fsp in chunk (#7043)
Browse files Browse the repository at this point in the history
  • Loading branch information
lysu authored and coocood committed Jul 17, 2018
1 parent 44e34bd commit 9cf670a
Show file tree
Hide file tree
Showing 20 changed files with 121 additions and 43 deletions.
2 changes: 1 addition & 1 deletion executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -772,7 +772,7 @@ func (s *testSuite) TestIssue2612(c *C) {
chk := rs.NewChunk()
err = rs.Next(context.Background(), chk)
c.Assert(err, IsNil)
c.Assert(chk.GetRow(0).GetDuration(0).String(), Equals, "-46:09:02")
c.Assert(chk.GetRow(0).GetDuration(0, 0).String(), Equals, "-46:09:02")
}

// TestIssue345 is related with https://github.com/pingcap/tidb/issues/345
Expand Down
2 changes: 1 addition & 1 deletion executor/write_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -230,7 +230,7 @@ func (s *testSuite) TestInsert(c *C) {
tk.MustExec("insert into t value(20070219173709.055870), (20070219173709.055), (20070219173709.055870123)")
tk.MustQuery("select * from t").Check(testkit.Rows("17:37:09.055870", "17:37:09.055000", "17:37:09.055870"))
_, err = tk.Exec("insert into t value(-20070219173709.055870)")
c.Assert(err.Error(), Equals, "[types:1292]Incorrect time value '-20070219173709.055870'")
c.Assert(err.Error(), Equals, "[types:1292]Incorrect time value: '-20070219173709.055870'")

tk.MustExec("drop table if exists t")
tk.MustExec("set @@sql_mode=''")
Expand Down
4 changes: 4 additions & 0 deletions expression/builtin_compare.go
Original file line number Diff line number Diff line change
Expand Up @@ -195,6 +195,10 @@ func (c *coalesceFunctionClass) getFunction(ctx sessionctx.Context, args []Expre
sig = &builtinCoalesceTimeSig{bf}
sig.setPbCode(tipb.ScalarFuncSig_CoalesceTime)
case types.ETDuration:
bf.tp.Decimal, err = getExpressionFsp(ctx, args[0])
if err != nil {
return nil, errors.Trace(err)
}
sig = &builtinCoalesceDurationSig{bf}
sig.setPbCode(tipb.ScalarFuncSig_CoalesceDuration)
}
Expand Down
3 changes: 2 additions & 1 deletion expression/builtin_other.go
Original file line number Diff line number Diff line change
Expand Up @@ -665,7 +665,8 @@ func (b *builtinValuesDurationSig) evalDuration(_ types.Row) (types.Duration, bo
if row.IsNull(b.offset) {
return types.Duration{}, true, nil
}
return row.GetDuration(b.offset), false, nil
duration := row.GetDuration(b.offset, b.getRetTp().Decimal)
return duration, false, nil
}
return types.Duration{}, true, errors.Errorf("Session current insert values len %d and column's offset %v don't match", row.Len(), b.offset)
}
Expand Down
62 changes: 53 additions & 9 deletions expression/builtin_time.go
Original file line number Diff line number Diff line change
Expand Up @@ -401,7 +401,16 @@ func (c *timeDiffFunctionClass) getFunction(ctx sessionctx.Context, args []Expre
arg0FieldTp, arg1FieldTp := args[0].GetType(), args[1].GetType()
arg0Tp, arg1Tp := c.getArgEvalTp(arg0FieldTp), c.getArgEvalTp(arg1FieldTp)
bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETDuration, arg0Tp, arg1Tp)
bf.tp.Decimal = mathutil.Max(arg0FieldTp.Decimal, arg1FieldTp.Decimal)

arg0Dec, err := getExpressionFsp(ctx, args[0])
if err != nil {
return nil, errors.Trace(err)
}
arg1Dec, err := getExpressionFsp(ctx, args[1])
if err != nil {
return nil, errors.Trace(err)
}
bf.tp.Decimal = mathutil.Max(arg0Dec, arg1Dec)

var sig builtinFunc
// arg0 and arg1 must be the same time type(compatible), or timediff will return NULL.
Expand Down Expand Up @@ -2040,10 +2049,15 @@ type timeFunctionClass struct {
}

func (c *timeFunctionClass) getFunction(ctx sessionctx.Context, args []Expression) (builtinFunc, error) {
if err := c.verifyArgs(args); err != nil {
err := c.verifyArgs(args)
if err != nil {
return nil, errors.Trace(err)
}
bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETDuration, types.ETString)
bf.tp.Decimal, err = getExpressionFsp(ctx, args[0])
if err != nil {
return nil, errors.Trace(err)
}
sig := &builtinTimeSig{bf}
return sig, nil
}
Expand Down Expand Up @@ -3274,9 +3288,11 @@ func (c *unixTimestampFunctionClass) getFunction(ctx sessionctx.Context, args []
if err := c.verifyArgs(args); err != nil {
return nil, errors.Trace(err)
}
var argTps []types.EvalType
var retTp types.EvalType
var retFLen, retDecimal int
var (
argTps []types.EvalType
retTp types.EvalType
retFLen, retDecimal int
)

if len(args) == 0 {
retTp, retDecimal = types.ETInt, 0
Expand Down Expand Up @@ -3635,7 +3651,7 @@ func getFsp4TimeAddSub(s string) int {

// getBf4TimeAddSub parses input types, generates baseBuiltinFunc and set related attributes for
// builtin function 'ADDTIME' and 'SUBTIME'
func getBf4TimeAddSub(ctx sessionctx.Context, args []Expression) (tp1, tp2 *types.FieldType, bf baseBuiltinFunc) {
func getBf4TimeAddSub(ctx sessionctx.Context, args []Expression) (tp1, tp2 *types.FieldType, bf baseBuiltinFunc, err error) {
tp1, tp2 = args[0].GetType(), args[1].GetType()
var argTp1, argTp2, retTp types.EvalType
switch tp1.Tp {
Expand All @@ -3654,8 +3670,17 @@ func getBf4TimeAddSub(ctx sessionctx.Context, args []Expression) (tp1, tp2 *type
default:
argTp2 = types.ETString
}
arg0Dec, err := getExpressionFsp(ctx, args[0])
if err != nil {
return
}
arg1Dec, err := getExpressionFsp(ctx, args[1])
if err != nil {
return
}

bf = newBaseBuiltinFuncWithTp(ctx, args, retTp, argTp1, argTp2)
bf.tp.Decimal = tp1.Decimal
bf.tp.Decimal = mathutil.Min(mathutil.Max(arg0Dec, arg1Dec), types.MaxFsp)
if retTp == types.ETString {
bf.tp.Tp, bf.tp.Flen, bf.tp.Decimal = mysql.TypeString, mysql.MaxDatetimeWidthWithFsp, types.UnspecifiedLength
}
Expand Down Expand Up @@ -3759,7 +3784,10 @@ func (c *addTimeFunctionClass) getFunction(ctx sessionctx.Context, args []Expres
if err = c.verifyArgs(args); err != nil {
return nil, errors.Trace(err)
}
tp1, tp2, bf := getBf4TimeAddSub(ctx, args)
tp1, tp2, bf, err := getBf4TimeAddSub(ctx, args)
if err != nil {
return nil, err
}
switch tp1.Tp {
case mysql.TypeDatetime, mysql.TypeTimestamp:
switch tp2.Tp {
Expand Down Expand Up @@ -4622,7 +4650,10 @@ func (c *subTimeFunctionClass) getFunction(ctx sessionctx.Context, args []Expres
if err = c.verifyArgs(args); err != nil {
return nil, errors.Trace(err)
}
tp1, tp2, bf := getBf4TimeAddSub(ctx, args)
tp1, tp2, bf, err := getBf4TimeAddSub(ctx, args)
if err != nil {
return nil, err
}
switch tp1.Tp {
case mysql.TypeDatetime, mysql.TypeTimestamp:
switch tp2.Tp {
Expand Down Expand Up @@ -5370,3 +5401,16 @@ func (b *builtinLastDaySig) evalTime(row types.Row) (types.Time, bool, error) {
}
return ret, false, nil
}

// getExpressionFsp calculates the fsp from given expression.
func getExpressionFsp(ctx sessionctx.Context, expression Expression) (int, error) {
constExp, isConstant := expression.(*Constant)
if isConstant && types.IsString(expression.GetType()) && !isTemporalColumn(expression) {
str, isNil, err := constExp.EvalString(ctx, nil)
if isNil || err != nil {
return 0, errors.Trace(err)
}
return types.GetFsp(str), nil
}
return mathutil.Min(expression.GetType().Decimal, types.MaxFsp), nil
}
3 changes: 2 additions & 1 deletion expression/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -272,7 +272,8 @@ func (col *Column) EvalDuration(ctx sessionctx.Context, row types.Row) (types.Du
if row.IsNull(col.Index) {
return types.Duration{}, true, nil
}
return row.GetDuration(col.Index), false, nil
duration := row.GetDuration(col.Index, col.RetType.Decimal)
return duration, false, nil
}

// EvalJSON returns JSON representation of Column.
Expand Down
8 changes: 4 additions & 4 deletions expression/typeinfer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1249,7 +1249,7 @@ func (s *testInferTypeSuite) createTestCase4TimeFuncs() []typeInferTestCase {
{"addtime(c_timestamp, c_time_d)", mysql.TypeDatetime, charset.CharsetBin, mysql.BinaryFlag, 26, 4},
{"addtime(c_timestamp_d, c_time_d)", mysql.TypeDatetime, charset.CharsetBin, mysql.BinaryFlag, 26, 0},
{"addtime(c_time, c_time)", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, 15, 3},
{"addtime(c_time_d, c_time)", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, 15, 0},
{"addtime(c_time_d, c_time)", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, 15, 3},
{"addtime(c_char, c_time_d)", mysql.TypeString, charset.CharsetUTF8, 0, 26, types.UnspecifiedLength},
{"addtime(c_char, c_datetime)", mysql.TypeString, charset.CharsetUTF8, 0, 26, types.UnspecifiedLength},
{"addtime(c_char, c_int_d)", mysql.TypeString, charset.CharsetUTF8, 0, 26, types.UnspecifiedLength},
Expand All @@ -1263,7 +1263,7 @@ func (s *testInferTypeSuite) createTestCase4TimeFuncs() []typeInferTestCase {
{"subtime(c_timestamp, c_time_d)", mysql.TypeDatetime, charset.CharsetBin, mysql.BinaryFlag, 26, 4},
{"subtime(c_timestamp_d, c_time_d)", mysql.TypeDatetime, charset.CharsetBin, mysql.BinaryFlag, 26, 0},
{"subtime(c_time, c_time)", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, 15, 3},
{"subtime(c_time_d, c_time)", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, 15, 0},
{"subtime(c_time_d, c_time)", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, 15, 3},
{"subtime(c_char, c_time_d)", mysql.TypeString, charset.CharsetUTF8, 0, 26, types.UnspecifiedLength},
{"subtime(c_char, c_datetime)", mysql.TypeString, charset.CharsetUTF8, 0, 26, types.UnspecifiedLength},
{"subtime(c_char, c_int_d)", mysql.TypeString, charset.CharsetUTF8, 0, 26, types.UnspecifiedLength},
Expand Down Expand Up @@ -1303,7 +1303,7 @@ func (s *testInferTypeSuite) createTestCase4TimeFuncs() []typeInferTestCase {
{"addtime(c_timestamp, c_time_d)", mysql.TypeDatetime, charset.CharsetBin, mysql.BinaryFlag, 26, 4},
{"addtime(c_timestamp_d, c_time_d)", mysql.TypeDatetime, charset.CharsetBin, mysql.BinaryFlag, 26, 0},
{"addtime(c_time, c_time)", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, 15, 3},
{"addtime(c_time_d, c_time)", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, 15, 0},
{"addtime(c_time_d, c_time)", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, 15, 3},
{"addtime(c_char, c_time_d)", mysql.TypeString, charset.CharsetUTF8, 0, 26, types.UnspecifiedLength},
{"addtime(c_char, c_datetime)", mysql.TypeString, charset.CharsetUTF8, 0, 26, types.UnspecifiedLength},
{"addtime(c_char, c_int_d)", mysql.TypeString, charset.CharsetUTF8, 0, 26, types.UnspecifiedLength},
Expand All @@ -1317,7 +1317,7 @@ func (s *testInferTypeSuite) createTestCase4TimeFuncs() []typeInferTestCase {
{"subtime(c_timestamp, c_time_d)", mysql.TypeDatetime, charset.CharsetBin, mysql.BinaryFlag, 26, 4},
{"subtime(c_timestamp_d, c_time_d)", mysql.TypeDatetime, charset.CharsetBin, mysql.BinaryFlag, 26, 0},
{"subtime(c_time, c_time)", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, 15, 3},
{"subtime(c_time_d, c_time)", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, 15, 0},
{"subtime(c_time_d, c_time)", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, 15, 3},
{"subtime(c_char, c_time_d)", mysql.TypeString, charset.CharsetUTF8, 0, 26, types.UnspecifiedLength},
{"subtime(c_char, c_datetime)", mysql.TypeString, charset.CharsetUTF8, 0, 26, types.UnspecifiedLength},
{"subtime(c_char, c_int_d)", mysql.TypeString, charset.CharsetUTF8, 0, 26, types.UnspecifiedLength},
Expand Down
6 changes: 5 additions & 1 deletion server/driver_tidb.go
Original file line number Diff line number Diff line change
Expand Up @@ -402,7 +402,11 @@ func convertColumnInfo(fld *ast.ResultField) (ci *ColumnInfo) {
}

if fld.Column.Decimal == types.UnspecifiedLength {
ci.Decimal = mysql.NotFixedDec
if fld.Column.Tp == mysql.TypeDuration {
ci.Decimal = types.DefaultFsp
} else {
ci.Decimal = mysql.NotFixedDec
}
} else {
ci.Decimal = uint8(fld.Column.Decimal)
}
Expand Down
5 changes: 3 additions & 2 deletions server/util.go
Original file line number Diff line number Diff line change
Expand Up @@ -265,7 +265,7 @@ func dumpBinaryRow(buffer []byte, columns []*ColumnInfo, row types.Row) ([]byte,
return buffer, errors.Trace(err)
}
case mysql.TypeDuration:
buffer = append(buffer, dumpBinaryTime(row.GetDuration(i).Duration)...)
buffer = append(buffer, dumpBinaryTime(row.GetDuration(i, 0).Duration)...)
case mysql.TypeEnum:
buffer = dumpLengthEncodedString(buffer, hack.Slice(row.GetEnum(i).String()))
case mysql.TypeSet:
Expand Down Expand Up @@ -319,7 +319,8 @@ func dumpTextRow(buffer []byte, columns []*ColumnInfo, row types.Row) ([]byte, e
case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeTimestamp:
buffer = dumpLengthEncodedString(buffer, hack.Slice(row.GetTime(i).String()))
case mysql.TypeDuration:
buffer = dumpLengthEncodedString(buffer, hack.Slice(row.GetDuration(i).String()))
dur := row.GetDuration(i, int(col.Decimal))
buffer = dumpLengthEncodedString(buffer, hack.Slice(dur.String()))
case mysql.TypeEnum:
buffer = dumpLengthEncodedString(buffer, hack.Slice(row.GetEnum(i).String()))
case mysql.TypeSet:
Expand Down
2 changes: 1 addition & 1 deletion statistics/scalar.go
Original file line number Diff line number Diff line change
Expand Up @@ -126,7 +126,7 @@ func (hg *Histogram) calcFraction(index int, value *types.Datum) float64 {
case types.KindUint64:
return calcFraction(float64(lower.GetUint64(0)), float64(upper.GetUint64(0)), float64(value.GetUint64()))
case types.KindMysqlDuration:
return calcFraction(float64(lower.GetDuration(0).Duration), float64(upper.GetDuration(0).Duration), float64(value.GetMysqlDuration().Duration))
return calcFraction(float64(lower.GetDuration(0, 0).Duration), float64(upper.GetDuration(0, 0).Duration), float64(value.GetMysqlDuration().Duration))
case types.KindMysqlDecimal, types.KindMysqlTime:
return calcFraction(hg.scalars[index].lower, hg.scalars[index].upper, convertDatumToScalar(value, 0))
case types.KindBytes, types.KindString:
Expand Down
15 changes: 13 additions & 2 deletions types/datum.go
Original file line number Diff line number Diff line change
Expand Up @@ -1019,8 +1019,19 @@ func (d *Datum) convertToMysqlDuration(sc *stmtctx.StatementContext, target *Fie
timeStr, err := d.ToString()
if err != nil {
return ret, errors.Trace(err)
} else if timeStr[0] == '-' {
return ret, ErrInvalidTimeFormat.Gen("Incorrect time value '%s'", timeStr)
}
timeNum, err := d.ToInt64(sc)
if err != nil {
return ret, errors.Trace(err)
}
// For huge numbers(>'0001-00-00 00-00-00') try full DATETIME in ParseDuration.
if timeNum > MaxDuration && timeNum < 10000000000 {
// mysql return max in no strict sql mode.
ret.SetValue(Duration{Duration: MaxTime, Fsp: 0})
return ret, ErrInvalidTimeFormat.Gen("Incorrect time value: '%s'", timeStr)
}
if timeNum < -MaxDuration {
return ret, ErrInvalidTimeFormat.Gen("Incorrect time value: '%s'", timeStr)
}
t, err := ParseDuration(timeStr, fsp)
ret.SetValue(t)
Expand Down
10 changes: 8 additions & 2 deletions types/etc.go
Original file line number Diff line number Diff line change
Expand Up @@ -87,7 +87,7 @@ func IsTemporalWithDate(tp byte) bool {
// IsBinaryStr returns a boolean indicating
// whether the field type is a binary string type.
func IsBinaryStr(ft *FieldType) bool {
if ft.Collate == charset.CollationBin && (IsTypeChar(ft.Tp) || IsTypeBlob(ft.Tp) || IsTypeVarchar(ft.Tp) || IsTypeUnspecified(ft.Tp)) {
if ft.Collate == charset.CollationBin && IsString(ft) {
return true
}
return false
Expand All @@ -96,12 +96,18 @@ func IsBinaryStr(ft *FieldType) bool {
// IsNonBinaryStr returns a boolean indicating
// whether the field type is a non-binary string type.
func IsNonBinaryStr(ft *FieldType) bool {
if ft.Collate != charset.CollationBin && (IsTypeChar(ft.Tp) || IsTypeBlob(ft.Tp) || IsTypeVarchar(ft.Tp) || IsTypeUnspecified(ft.Tp)) {
if ft.Collate != charset.CollationBin && IsString(ft) {
return true
}
return false
}

// IsString returns a boolean indicating
// whether the field type is a string type.
func IsString(ft *FieldType) bool {
return IsTypeChar(ft.Tp) || IsTypeBlob(ft.Tp) || IsTypeVarchar(ft.Tp) || IsTypeUnspecified(ft.Tp)
}

var type2Str = map[byte]string{
mysql.TypeBit: "bit",
mysql.TypeBlob: "text",
Expand Down
6 changes: 4 additions & 2 deletions types/row.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,9 @@ type Row interface {
GetTime(colIdx int) Time

// GetDuration returns the Duration value with the colIdx.
GetDuration(colIdx int) Duration
// fillFsp is needed for refill fsp info if duration came from chunk.Row which is no longer store fsp info.
// If caller make sure that data from Datum or only use Duration.Duration properties can pass 0 as fillFsp.
GetDuration(colIdx int, fillFsp int) Duration

// GetEnum returns the Enum value with the colIdx.
GetEnum(colIdx int) Enum
Expand Down Expand Up @@ -117,7 +119,7 @@ func (dr DatumRow) GetTime(colIdx int) Time {
}

// GetDuration implements Row interface.
func (dr DatumRow) GetDuration(colIdx int) Duration {
func (dr DatumRow) GetDuration(colIdx int, fillFsp int) Duration {
return dr[colIdx].GetMysqlDuration()
}

Expand Down
5 changes: 3 additions & 2 deletions types/time.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,8 @@ const (
MinYear int16 = 1901
// MaxYear is the maximum for mysql year type.
MaxYear int16 = 2155

// MaxDuration is the maximum for duration.
MaxDuration int64 = 838*10000 + 59*100 + 59
// MinTime is the minimum for mysql time type.
MinTime = -gotime.Duration(838*3600+59*60+59) * gotime.Second
// MaxTime is the maximum for mysql time type.
Expand Down Expand Up @@ -764,7 +765,7 @@ func AdjustYear(y int64) (int64, error) {
return y, nil
}

// Duration is the type for MySQL time type.
// Duration is the type for MySQL TIME type.
type Duration struct {
gotime.Duration
// Fsp is short for Fractional Seconds Precision.
Expand Down
5 changes: 3 additions & 2 deletions util/chunk/chunk_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -96,7 +96,8 @@ func (s *testChunkSuite) TestChunk(c *check.C) {
row := chk.GetRow(0)
c.Assert(row.GetFloat32(0), check.Equals, f32Val)
c.Assert(row.GetTime(2).Compare(tVal), check.Equals, 0)
c.Assert(row.GetDuration(3), check.DeepEquals, durVal)
// fsp no longer maintain in arrow
c.Assert(row.GetDuration(3, 0).Duration, check.DeepEquals, durVal.Duration)
c.Assert(row.GetEnum(4), check.DeepEquals, enumVal)
c.Assert(row.GetSet(5), check.DeepEquals, setVal)

Expand Down Expand Up @@ -412,7 +413,7 @@ func (s *testChunkSuite) TestChunkMemoryUsage(c *check.C) {
colUsage[1] = initCap>>3 + (initCap+1)*4 + initCap*4 + 0
colUsage[2] = initCap>>3 + (initCap+1)*4 + initCap*4 + 0
colUsage[3] = initCap>>3 + 0 + initCap*16 + 16
colUsage[4] = initCap>>3 + 0 + initCap*16 + 16
colUsage[4] = initCap>>3 + 0 + initCap*8 + 8

expectedUsage := 0
for i := range colUsage {
Expand Down
4 changes: 2 additions & 2 deletions util/chunk/codec.go
Original file line number Diff line number Diff line change
Expand Up @@ -168,9 +168,9 @@ func getFixedLen(colType *types.FieldType) int {
case mysql.TypeFloat:
return 4
case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong,
mysql.TypeLonglong, mysql.TypeDouble, mysql.TypeYear:
mysql.TypeLonglong, mysql.TypeDouble, mysql.TypeYear, mysql.TypeDuration:
return 8
case mysql.TypeDuration, mysql.TypeDate, mysql.TypeDatetime, mysql.TypeTimestamp:
case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeTimestamp:
return 16
case mysql.TypeNewDecimal:
return types.MyDecimalStructSize
Expand Down
3 changes: 1 addition & 2 deletions util/chunk/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,8 +21,7 @@ import (
)

func (c *column) appendDuration(dur types.Duration) {
*(*types.Duration)(unsafe.Pointer(&c.elemBuf[0])) = dur
c.finishAppendFixed()
c.appendInt64(int64(dur.Duration))
}

func (c *column) appendMyDecimal(dec *types.MyDecimal) {
Expand Down
8 changes: 4 additions & 4 deletions util/chunk/compare.go
Original file line number Diff line number Diff line change
Expand Up @@ -128,8 +128,8 @@ func cmpDuration(l Row, lCol int, r Row, rCol int) int {
if lNull || rNull {
return cmpNull(lNull, rNull)
}
lDur, rDur := l.GetDuration(lCol), r.GetDuration(rCol)
return types.CompareInt64(int64(lDur.Duration), int64(rDur.Duration))
lDur, rDur := l.GetDuration(lCol, 0).Duration, r.GetDuration(rCol, 0).Duration
return types.CompareInt64(int64(lDur), int64(rDur))
}

func cmpNameValue(l Row, lCol int, r Row, rCol int) int {
Expand Down Expand Up @@ -190,8 +190,8 @@ func Compare(row Row, colIdx int, ad *types.Datum) int {
l, r := row.GetMyDecimal(colIdx), ad.GetMysqlDecimal()
return l.Compare(r)
case types.KindMysqlDuration:
l, r := row.GetDuration(colIdx), ad.GetMysqlDuration()
return types.CompareInt64(int64(l.Duration), int64(r.Duration))
l, r := row.GetDuration(colIdx, 0).Duration, ad.GetMysqlDuration().Duration
return types.CompareInt64(int64(l), int64(r))
case types.KindMysqlEnum:
l, r := row.GetEnum(colIdx).Value, ad.GetMysqlEnum().Value
return types.CompareUint64(l, r)
Expand Down
Loading

0 comments on commit 9cf670a

Please sign in to comment.