diff --git a/bindinfo/bind_test.go b/bindinfo/bind_test.go index 8a9cbf32b1eec..f632e5402b376 100644 --- a/bindinfo/bind_test.go +++ b/bindinfo/bind_test.go @@ -165,7 +165,7 @@ func (s *testSuite) TestGlobalBinding(c *C) { metrics.BindTotalGauge.WithLabelValues(metrics.ScopeGlobal, bindinfo.Using).Write(pb) c.Assert(pb.GetGauge().GetValue(), Equals, float64(1)) metrics.BindMemoryUsage.WithLabelValues(metrics.ScopeGlobal, bindinfo.Using).Write(pb) - c.Assert(pb.GetGauge().GetValue(), Equals, float64(161)) + c.Assert(pb.GetGauge().GetValue(), Equals, float64(121)) sql, hash := parser.NormalizeDigest("select * from t where i > ?") @@ -221,7 +221,7 @@ func (s *testSuite) TestGlobalBinding(c *C) { c.Assert(pb.GetGauge().GetValue(), Equals, float64(0)) metrics.BindMemoryUsage.WithLabelValues(metrics.ScopeGlobal, bindinfo.Using).Write(pb) // From newly created global bind handle. - c.Assert(pb.GetGauge().GetValue(), Equals, float64(161)) + c.Assert(pb.GetGauge().GetValue(), Equals, float64(121)) bindHandle = bindinfo.NewBindHandle(tk.Se) err = bindHandle.Update(true) @@ -268,7 +268,7 @@ func (s *testSuite) TestSessionBinding(c *C) { metrics.BindTotalGauge.WithLabelValues(metrics.ScopeSession, bindinfo.Using).Write(pb) c.Assert(pb.GetGauge().GetValue(), Equals, float64(1)) metrics.BindMemoryUsage.WithLabelValues(metrics.ScopeSession, bindinfo.Using).Write(pb) - c.Assert(pb.GetGauge().GetValue(), Equals, float64(161)) + c.Assert(pb.GetGauge().GetValue(), Equals, float64(121)) handle := tk.Se.Value(bindinfo.SessionBindInfoKeyType).(*bindinfo.SessionHandle) bindData := handle.GetBindRecord("select * from t where i > ?", "test") diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index d9b84cf3d7c2a..f0b5d236e57c8 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -437,7 +437,7 @@ func checkColumnDefaultValue(ctx sessionctx.Context, col *table.Column, value in if value != nil && ctx.GetSessionVars().SQLMode.HasNoZeroDateMode() && ctx.GetSessionVars().SQLMode.HasStrictMode() && types.IsTypeTime(col.Tp) { if vv, ok := value.(string); ok { - timeValue, err := expression.GetTimeValue(ctx, vv, col.Tp, col.Decimal) + timeValue, err := expression.GetTimeValue(ctx, vv, col.Tp, int8(col.Decimal)) if err != nil { return hasDefaultValue, value, errors.Trace(err) } @@ -455,7 +455,7 @@ func convertTimestampDefaultValToUTC(ctx sessionctx.Context, defaultVal interfac } if vv, ok := defaultVal.(string); ok { if vv != types.ZeroDatetimeStr && strings.ToUpper(vv) != strings.ToUpper(ast.CurrentTimestamp) { - t, err := types.ParseTime(ctx.GetSessionVars().StmtCtx, vv, col.Tp, col.Decimal) + t, err := types.ParseTime(ctx.GetSessionVars().StmtCtx, vv, col.Tp, int8(col.Decimal)) if err != nil { return defaultVal, errors.Trace(err) } @@ -636,7 +636,7 @@ func getDefaultValue(ctx sessionctx.Context, colName string, c *ast.ColumnOption } } } - vd, err := expression.GetTimeValue(ctx, c.Expr, tp, fsp) + vd, err := expression.GetTimeValue(ctx, c.Expr, tp, int8(fsp)) value := vd.GetValue() if err != nil { return nil, ErrInvalidDefaultValue.GenWithStackByArgs(colName) @@ -951,7 +951,7 @@ func checkColumnAttributes(colName string, tp *types.FieldType) error { return types.ErrMBiggerThanD.GenWithStackByArgs(colName) } case mysql.TypeDatetime, mysql.TypeDuration, mysql.TypeTimestamp: - if tp.Decimal != types.UnspecifiedFsp && (tp.Decimal < types.MinFsp || tp.Decimal > types.MaxFsp) { + if tp.Decimal != int(types.UnspecifiedFsp) && (tp.Decimal < int(types.MinFsp) || tp.Decimal > int(types.MaxFsp)) { return types.ErrTooBigPrecision.GenWithStackByArgs(tp.Decimal, colName, types.MaxFsp) } } diff --git a/executor/write.go b/executor/write.go index a09fb33849bc8..cc2b05409385c 100644 --- a/executor/write.go +++ b/executor/write.go @@ -129,7 +129,7 @@ func updateRecord(ctx context.Context, sctx sessionctx.Context, h int64, oldData // 4. Fill values into on-update-now fields, only if they are really changed. for i, col := range t.Cols() { if mysql.HasOnUpdateNowFlag(col.Flag) && !modified[i] && !onUpdateSpecified[i] { - if v, err := expression.GetTimeValue(sctx, strings.ToUpper(ast.CurrentTimestamp), col.Tp, col.Decimal); err == nil { + if v, err := expression.GetTimeValue(sctx, strings.ToUpper(ast.CurrentTimestamp), col.Tp, int8(col.Decimal)); err == nil { newData[i] = v modified[i] = true } else { diff --git a/expression/builtin.go b/expression/builtin.go index dc1c1d40bd683..6bf88833338a4 100644 --- a/expression/builtin.go +++ b/expression/builtin.go @@ -129,21 +129,21 @@ func newBaseBuiltinFuncWithTp(ctx sessionctx.Context, args []Expression, retType fieldType = &types.FieldType{ Tp: mysql.TypeDatetime, Flen: mysql.MaxDatetimeWidthWithFsp, - Decimal: types.MaxFsp, + Decimal: int(types.MaxFsp), Flag: mysql.BinaryFlag, } case types.ETTimestamp: fieldType = &types.FieldType{ Tp: mysql.TypeTimestamp, Flen: mysql.MaxDatetimeWidthWithFsp, - Decimal: types.MaxFsp, + Decimal: int(types.MaxFsp), Flag: mysql.BinaryFlag, } case types.ETDuration: fieldType = &types.FieldType{ Tp: mysql.TypeDuration, Flen: mysql.MaxDurationWidthWithFsp, - Decimal: types.MaxFsp, + Decimal: int(types.MaxFsp), Flag: mysql.BinaryFlag, } case types.ETJson: diff --git a/expression/builtin_arithmetic.go b/expression/builtin_arithmetic.go index 85b1d757767d8..091e7e99541f9 100644 --- a/expression/builtin_arithmetic.go +++ b/expression/builtin_arithmetic.go @@ -118,10 +118,10 @@ func setFlenDecimal4RealOrDecimal(retTp, a, b *types.FieldType, isReal bool) { func (c *arithmeticDivideFunctionClass) setType4DivDecimal(retTp, a, b *types.FieldType) { var deca, decb = a.Decimal, b.Decimal - if deca == types.UnspecifiedFsp { + if deca == int(types.UnspecifiedFsp) { deca = 0 } - if decb == types.UnspecifiedFsp { + if decb == int(types.UnspecifiedFsp) { decb = 0 } retTp.Decimal = deca + precIncrement diff --git a/expression/builtin_cast.go b/expression/builtin_cast.go index 0b8d0abd4977e..be3b5273330b3 100644 --- a/expression/builtin_cast.go +++ b/expression/builtin_cast.go @@ -540,7 +540,7 @@ func (b *builtinCastIntAsTimeSig) evalTime(row chunk.Row) (res types.Time, isNul if isNull || err != nil { return res, isNull, err } - res, err = types.ParseTimeFromNum(b.ctx.GetSessionVars().StmtCtx, val, b.tp.Tp, b.tp.Decimal) + res, err = types.ParseTimeFromNum(b.ctx.GetSessionVars().StmtCtx, val, b.tp.Tp, int8(b.tp.Decimal)) if err != nil { return types.Time{}, true, handleInvalidTimeError(b.ctx, err) } @@ -566,7 +566,7 @@ func (b *builtinCastIntAsDurationSig) evalDuration(row chunk.Row) (res types.Dur if isNull || err != nil { return res, isNull, err } - dur, err := types.NumberToDuration(val, b.tp.Decimal) + dur, err := types.NumberToDuration(val, int8(b.tp.Decimal)) if err != nil { if types.ErrOverflow.Equal(err) { err = b.ctx.GetSessionVars().StmtCtx.HandleOverflow(err, err) @@ -821,7 +821,7 @@ func (b *builtinCastRealAsTimeSig) evalTime(row chunk.Row) (types.Time, bool, er return types.Time{}, true, err } sc := b.ctx.GetSessionVars().StmtCtx - res, err := types.ParseTime(sc, strconv.FormatFloat(val, 'f', -1, 64), b.tp.Tp, b.tp.Decimal) + res, err := types.ParseTime(sc, strconv.FormatFloat(val, 'f', -1, 64), b.tp.Tp, int8(b.tp.Decimal)) if err != nil { return types.Time{}, true, handleInvalidTimeError(b.ctx, err) } @@ -847,7 +847,7 @@ func (b *builtinCastRealAsDurationSig) evalDuration(row chunk.Row) (res types.Du if isNull || err != nil { return res, isNull, err } - res, err = types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, strconv.FormatFloat(val, 'f', -1, 64), b.tp.Decimal) + res, err = types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, strconv.FormatFloat(val, 'f', -1, 64), int8(b.tp.Decimal)) return res, false, err } @@ -978,7 +978,7 @@ func (b *builtinCastDecimalAsTimeSig) evalTime(row chunk.Row) (res types.Time, i return res, isNull, err } sc := b.ctx.GetSessionVars().StmtCtx - res, err = types.ParseTimeFromFloatString(sc, string(val.ToString()), b.tp.Tp, b.tp.Decimal) + res, err = types.ParseTimeFromFloatString(sc, string(val.ToString()), b.tp.Tp, int8(b.tp.Decimal)) if err != nil { return types.Time{}, true, handleInvalidTimeError(b.ctx, err) } @@ -1004,7 +1004,7 @@ func (b *builtinCastDecimalAsDurationSig) evalDuration(row chunk.Row) (res types if isNull || err != nil { return res, true, err } - res, err = types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, string(val.ToString()), b.tp.Decimal) + res, err = types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, string(val.ToString()), int8(b.tp.Decimal)) if types.ErrTruncatedWrongVal.Equal(err) { err = b.ctx.GetSessionVars().StmtCtx.HandleTruncate(err) // ZeroDuration of error ErrTruncatedWrongVal needs to be considered NULL. @@ -1186,7 +1186,7 @@ func (b *builtinCastStringAsTimeSig) evalTime(row chunk.Row) (res types.Time, is return res, isNull, err } sc := b.ctx.GetSessionVars().StmtCtx - res, err = types.ParseTime(sc, val, b.tp.Tp, b.tp.Decimal) + res, err = types.ParseTime(sc, val, b.tp.Tp, int8(b.tp.Decimal)) if err != nil { return types.Time{}, true, handleInvalidTimeError(b.ctx, err) } @@ -1212,7 +1212,7 @@ func (b *builtinCastStringAsDurationSig) evalDuration(row chunk.Row) (res types. if isNull || err != nil { return res, isNull, err } - res, err = types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, val, b.tp.Decimal) + res, err = types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, val, int8(b.tp.Decimal)) if types.ErrTruncatedWrongVal.Equal(err) { sc := b.ctx.GetSessionVars().StmtCtx err = sc.HandleTruncate(err) @@ -1244,7 +1244,7 @@ func (b *builtinCastTimeAsTimeSig) evalTime(row chunk.Row) (res types.Time, isNu if res, err = res.Convert(sc, b.tp.Tp); err != nil { return types.Time{}, true, handleInvalidTimeError(b.ctx, err) } - res, err = res.RoundFrac(sc, b.tp.Decimal) + res, err = res.RoundFrac(sc, int8(b.tp.Decimal)) if b.tp.Tp == mysql.TypeDate { // Truncate hh:mm:ss part if the type is Date. res.Time = types.FromDate(res.Time.Year(), res.Time.Month(), res.Time.Day(), 0, 0, 0, 0) @@ -1358,7 +1358,7 @@ func (b *builtinCastTimeAsDurationSig) evalDuration(row chunk.Row) (res types.Du if err != nil { return res, false, err } - res, err = res.RoundFrac(b.tp.Decimal) + res, err = res.RoundFrac(int8(b.tp.Decimal)) return res, false, err } @@ -1377,7 +1377,7 @@ func (b *builtinCastDurationAsDurationSig) evalDuration(row chunk.Row) (res type if isNull || err != nil { return res, isNull, err } - res, err = res.RoundFrac(b.tp.Decimal) + res, err = res.RoundFrac(int8(b.tp.Decimal)) return res, false, err } @@ -1505,7 +1505,7 @@ func (b *builtinCastDurationAsTimeSig) evalTime(row chunk.Row) (res types.Time, if err != nil { return types.Time{}, true, handleInvalidTimeError(b.ctx, err) } - res, err = res.RoundFrac(sc, b.tp.Decimal) + res, err = res.RoundFrac(sc, int8(b.tp.Decimal)) return res, false, err } @@ -1625,7 +1625,7 @@ func (b *builtinCastJSONAsTimeSig) evalTime(row chunk.Row) (res types.Time, isNu return res, false, err } sc := b.ctx.GetSessionVars().StmtCtx - res, err = types.ParseTime(sc, s, b.tp.Tp, b.tp.Decimal) + res, err = types.ParseTime(sc, s, b.tp.Tp, int8(b.tp.Decimal)) if err != nil { return types.Time{}, true, handleInvalidTimeError(b.ctx, err) } @@ -1655,7 +1655,7 @@ func (b *builtinCastJSONAsDurationSig) evalDuration(row chunk.Row) (res types.Du if err != nil { return res, false, err } - res, err = types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, s, b.tp.Decimal) + res, err = types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, s, int8(b.tp.Decimal)) if types.ErrTruncatedWrongVal.Equal(err) { sc := b.ctx.GetSessionVars().StmtCtx err = sc.HandleTruncate(err) @@ -1793,7 +1793,7 @@ func WrapWithCastAsString(ctx sessionctx.Context, expr Expression) Expression { // into consideration, so we set `expr.GetType().Flen + 2` as the `argLen`. // Since the length of float and double is not accurate, we do not handle // them. - if exprTp.Tp == mysql.TypeNewDecimal && argLen != types.UnspecifiedFsp { + if exprTp.Tp == mysql.TypeNewDecimal && argLen != int(types.UnspecifiedFsp) { argLen += 2 } if exprTp.EvalType() == types.ETInt { @@ -1818,7 +1818,7 @@ func WrapWithCastAsTime(ctx sessionctx.Context, expr Expression, tp *types.Field case mysql.TypeDatetime, mysql.TypeTimestamp, mysql.TypeDate, mysql.TypeDuration: tp.Decimal = x.Decimal default: - tp.Decimal = types.MaxFsp + tp.Decimal = int(types.MaxFsp) } switch tp.Tp { case mysql.TypeDate: @@ -1844,7 +1844,7 @@ func WrapWithCastAsDuration(ctx sessionctx.Context, expr Expression) Expression case mysql.TypeDatetime, mysql.TypeTimestamp, mysql.TypeDate: tp.Decimal = x.Decimal default: - tp.Decimal = types.MaxFsp + tp.Decimal = int(types.MaxFsp) } tp.Flen = mysql.MaxDurationWidthNoFsp if tp.Decimal > 0 { diff --git a/expression/builtin_cast_test.go b/expression/builtin_cast_test.go index c0636e4b34a04..eb2ca59062599 100644 --- a/expression/builtin_cast_test.go +++ b/expression/builtin_cast_test.go @@ -321,7 +321,7 @@ func (s *testEvaluatorSuite) TestCastFuncSig(c *C) { var sig builtinFunc durationColumn := &Column{RetType: types.NewFieldType(mysql.TypeDuration), Index: 0} - durationColumn.RetType.Decimal = types.DefaultFsp + durationColumn.RetType.Decimal = int(types.DefaultFsp) // Test cast as Decimal. castToDecCases := []struct { before *Column @@ -805,7 +805,7 @@ func (s *testEvaluatorSuite) TestCastFuncSig(c *C) { for i, t := range castToTimeCases { args := []Expression{t.before} tp := types.NewFieldType(mysql.TypeDatetime) - tp.Decimal = types.DefaultFsp + tp.Decimal = int(types.DefaultFsp) timeFunc := newBaseBuiltinFunc(ctx, args) timeFunc.tp = tp switch i { @@ -834,7 +834,7 @@ func (s *testEvaluatorSuite) TestCastFuncSig(c *C) { before *Column after types.Time row chunk.MutRow - fsp int + fsp int8 tp byte }{ // cast real as Time(0). @@ -889,7 +889,7 @@ func (s *testEvaluatorSuite) TestCastFuncSig(c *C) { for i, t := range castToTimeCases2 { args := []Expression{t.before} tp := types.NewFieldType(t.tp) - tp.Decimal = t.fsp + tp.Decimal = int(t.fsp) timeFunc := newBaseBuiltinFunc(ctx, args) timeFunc.tp = tp switch i { @@ -912,7 +912,7 @@ func (s *testEvaluatorSuite) TestCastFuncSig(c *C) { resAfter := t.after.String() if t.fsp > 0 { resAfter += "." - for i := 0; i < t.fsp; i++ { + for i := 0; i < int(t.fsp); i++ { resAfter += "0" } } @@ -970,7 +970,7 @@ func (s *testEvaluatorSuite) TestCastFuncSig(c *C) { for i, t := range castToDurationCases { args := []Expression{t.before} tp := types.NewFieldType(mysql.TypeDuration) - tp.Decimal = types.DefaultFsp + tp.Decimal = int(types.DefaultFsp) durationFunc := newBaseBuiltinFunc(ctx, args) durationFunc.tp = tp switch i { @@ -1144,7 +1144,7 @@ func (s *testEvaluatorSuite) TestWrapWithCastAsTypesClasses(c *C) { ctx := s.ctx durationColumn0 := &Column{RetType: types.NewFieldType(mysql.TypeDuration), Index: 0} - durationColumn0.RetType.Decimal = types.DefaultFsp + durationColumn0.RetType.Decimal = int(types.DefaultFsp) durationColumn3 := &Column{RetType: types.NewFieldType(mysql.TypeDuration), Index: 0} durationColumn3.RetType.Decimal = 3 cases := []struct { diff --git a/expression/builtin_time.go b/expression/builtin_time.go index d93eccbffc5f7..df5768069a55a 100644 --- a/expression/builtin_time.go +++ b/expression/builtin_time.go @@ -240,7 +240,7 @@ var ( _ builtinFunc = &builtinSubDateDatetimeDecimalSig{} ) -func convertTimeToMysqlTime(t time.Time, fsp int, roundMode types.RoundMode) (types.Time, error) { +func convertTimeToMysqlTime(t time.Time, fsp int8, roundMode types.RoundMode) (types.Time, error) { var tr time.Time var err error if roundMode == types.ModeTruncate { @@ -548,7 +548,7 @@ func (b *builtinDurationStringTimeDiffSig) evalDuration(row chunk.Row) (d types. } sc := b.ctx.GetSessionVars().StmtCtx - rhs, _, isDuration, err := convertStringToDuration(sc, rhsStr, b.tp.Decimal) + rhs, _, isDuration, err := convertStringToDuration(sc, rhsStr, int8(b.tp.Decimal)) if err != nil || !isDuration { return d, true, err } @@ -581,7 +581,7 @@ func (b *builtinStringDurationTimeDiffSig) evalDuration(row chunk.Row) (d types. } sc := b.ctx.GetSessionVars().StmtCtx - lhs, _, isDuration, err := convertStringToDuration(sc, lhsStr, b.tp.Decimal) + lhs, _, isDuration, err := convertStringToDuration(sc, lhsStr, int8(b.tp.Decimal)) if err != nil || !isDuration { return d, true, err } @@ -639,7 +639,7 @@ func (b *builtinTimeStringTimeDiffSig) evalDuration(row chunk.Row) (d types.Dura } sc := b.ctx.GetSessionVars().StmtCtx - _, rhs, isDuration, err := convertStringToDuration(sc, rhsStr, b.tp.Decimal) + _, rhs, isDuration, err := convertStringToDuration(sc, rhsStr, int8(b.tp.Decimal)) if err != nil || isDuration { return d, true, err } @@ -672,7 +672,7 @@ func (b *builtinStringTimeTimeDiffSig) evalDuration(row chunk.Row) (d types.Dura } sc := b.ctx.GetSessionVars().StmtCtx - _, lhs, isDuration, err := convertStringToDuration(sc, lhsStr, b.tp.Decimal) + _, lhs, isDuration, err := convertStringToDuration(sc, lhsStr, int8(b.tp.Decimal)) if err != nil || isDuration { return d, true, err } @@ -705,7 +705,7 @@ func (b *builtinStringStringTimeDiffSig) evalDuration(row chunk.Row) (d types.Du } sc := b.ctx.GetSessionVars().StmtCtx - fsp := b.tp.Decimal + fsp := int8(b.tp.Decimal) lhsDur, lhsTime, lhsIsDuration, err := convertStringToDuration(sc, lhs, fsp) if err != nil { return d, true, err @@ -747,12 +747,12 @@ func (b *builtinNullTimeDiffSig) evalDuration(row chunk.Row) (d types.Duration, // convertStringToDuration converts string to duration, it return types.Time because in some case // it will converts string to datetime. -func convertStringToDuration(sc *stmtctx.StatementContext, str string, fsp int) (d types.Duration, t types.Time, +func convertStringToDuration(sc *stmtctx.StatementContext, str string, fsp int8) (d types.Duration, t types.Time, isDuration bool, err error) { if n := strings.IndexByte(str, '.'); n >= 0 { lenStrFsp := len(str[n+1:]) - if lenStrFsp <= types.MaxFsp { - fsp = mathutil.Max(lenStrFsp, fsp) + if lenStrFsp <= int(types.MaxFsp) { + fsp = mathutil.MaxInt8(int8(lenStrFsp), fsp) } } return types.StrToDuration(sc, str, fsp) @@ -1573,15 +1573,15 @@ func (c *fromUnixTimeFunctionClass) getFunction(ctx sessionctx.Context, args []E bf := newBaseBuiltinFuncWithTp(ctx, args, retTp, argTps...) if len(args) == 1 { if isArg0Str { - bf.tp.Decimal = types.MaxFsp + bf.tp.Decimal = int(types.MaxFsp) } else if isArg0Con { arg0, _, err1 := args[0].EvalDecimal(ctx, chunk.Row{}) if err1 != nil { return sig, err1 } fsp := int(arg0.GetDigitsFrac()) - if fsp > types.MaxFsp { - fsp = types.MaxFsp + if fsp > int(types.MaxFsp) { + fsp = int(types.MaxFsp) } bf.tp.Decimal = fsp } @@ -1593,7 +1593,7 @@ func (c *fromUnixTimeFunctionClass) getFunction(ctx sessionctx.Context, args []E return sig, nil } -func evalFromUnixTime(ctx sessionctx.Context, fsp int, row chunk.Row, arg Expression) (res types.Time, isNull bool, err error) { +func evalFromUnixTime(ctx sessionctx.Context, fsp int8, row chunk.Row, arg Expression) (res types.Time, isNull bool, err error) { unixTimeStamp, isNull, err := arg.EvalDecimal(ctx, row) if err != nil || isNull { return res, isNull, err @@ -1629,11 +1629,11 @@ func evalFromUnixTime(ctx sessionctx.Context, fsp int, row chunk.Row, arg Expres if err != nil && !terror.ErrorEqual(err, types.ErrTruncated) { return res, true, err } - fracDigitsNumber := int(unixTimeStamp.GetDigitsFrac()) + fracDigitsNumber := unixTimeStamp.GetDigitsFrac() if fsp < 0 { fsp = types.MaxFsp } - fsp = mathutil.Max(fracDigitsNumber, fsp) + fsp = mathutil.MaxInt8(fracDigitsNumber, fsp) if fsp > types.MaxFsp { fsp = types.MaxFsp } @@ -1660,7 +1660,7 @@ func (b *builtinFromUnixTime1ArgSig) Clone() builtinFunc { // evalTime evals a builtinFromUnixTime1ArgSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_from-unixtime func (b *builtinFromUnixTime1ArgSig) evalTime(row chunk.Row) (res types.Time, isNull bool, err error) { - return evalFromUnixTime(b.ctx, b.tp.Decimal, row, b.args[0]) + return evalFromUnixTime(b.ctx, int8(b.tp.Decimal), row, b.args[0]) } type builtinFromUnixTime2ArgSig struct { @@ -1680,7 +1680,7 @@ func (b *builtinFromUnixTime2ArgSig) evalString(row chunk.Row) (res string, isNu if isNull || err != nil { return "", true, err } - t, isNull, err := evalFromUnixTime(b.ctx, b.tp.Decimal, row, b.args[0]) + t, isNull, err := evalFromUnixTime(b.ctx, int8(b.tp.Decimal), row, b.args[0]) if isNull || err != nil { return "", isNull, err } @@ -1774,7 +1774,7 @@ type strToDateFunctionClass struct { baseFunctionClass } -func (c *strToDateFunctionClass) getRetTp(ctx sessionctx.Context, arg Expression) (tp byte, fsp int) { +func (c *strToDateFunctionClass) getRetTp(ctx sessionctx.Context, arg Expression) (tp byte, fsp int8) { tp = mysql.TypeDatetime if _, ok := arg.(*Constant); !ok { return tp, types.MaxFsp @@ -1805,22 +1805,22 @@ func (c *strToDateFunctionClass) getFunction(ctx sessionctx.Context, args []Expr switch retTp { case mysql.TypeDate: bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETDatetime, types.ETString, types.ETString) - bf.tp.Tp, bf.tp.Flen, bf.tp.Decimal = mysql.TypeDate, mysql.MaxDateWidth, types.MinFsp + bf.tp.Tp, bf.tp.Flen, bf.tp.Decimal = mysql.TypeDate, mysql.MaxDateWidth, int(types.MinFsp) sig = &builtinStrToDateDateSig{bf} case mysql.TypeDatetime: bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETDatetime, types.ETString, types.ETString) if fsp == types.MinFsp { - bf.tp.Flen, bf.tp.Decimal = mysql.MaxDatetimeWidthNoFsp, types.MinFsp + bf.tp.Flen, bf.tp.Decimal = mysql.MaxDatetimeWidthNoFsp, int(types.MinFsp) } else { - bf.tp.Flen, bf.tp.Decimal = mysql.MaxDatetimeWidthWithFsp, types.MaxFsp + bf.tp.Flen, bf.tp.Decimal = mysql.MaxDatetimeWidthWithFsp, int(types.MaxFsp) } sig = &builtinStrToDateDatetimeSig{bf} case mysql.TypeDuration: bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETDuration, types.ETString, types.ETString) if fsp == types.MinFsp { - bf.tp.Flen, bf.tp.Decimal = mysql.MaxDurationWidthNoFsp, types.MinFsp + bf.tp.Flen, bf.tp.Decimal = mysql.MaxDurationWidthNoFsp, int(types.MinFsp) } else { - bf.tp.Flen, bf.tp.Decimal = mysql.MaxDurationWidthWithFsp, types.MaxFsp + bf.tp.Flen, bf.tp.Decimal = mysql.MaxDurationWidthWithFsp, int(types.MaxFsp) } sig = &builtinStrToDateDurationSig{bf} } @@ -1887,7 +1887,7 @@ func (b *builtinStrToDateDatetimeSig) evalTime(row chunk.Row) (types.Time, bool, if b.ctx.GetSessionVars().SQLMode.HasNoZeroDateMode() && (t.Time.Year() == 0 || t.Time.Month() == 0 || t.Time.Day() == 0) { return types.Time{}, true, handleInvalidTimeError(b.ctx, types.ErrIncorrectDatetimeValue.GenWithStackByArgs(t.String())) } - t.Type, t.Fsp = mysql.TypeDatetime, b.tp.Decimal + t.Type, t.Fsp = mysql.TypeDatetime, int8(b.tp.Decimal) return t, false, nil } @@ -1922,7 +1922,7 @@ func (b *builtinStrToDateDurationSig) evalDuration(row chunk.Row) (types.Duratio if b.ctx.GetSessionVars().SQLMode.HasNoZeroDateMode() && (t.Time.Year() == 0 || t.Time.Month() == 0 || t.Time.Day() == 0) { return types.Duration{}, true, handleInvalidTimeError(b.ctx, types.ErrIncorrectDatetimeValue.GenWithStackByArgs(t.String())) } - t.Fsp = b.tp.Decimal + t.Fsp = int8(b.tp.Decimal) dur, err := t.ConvertToDuration() return dur, err != nil, err } @@ -1971,7 +1971,7 @@ func (b *builtinSysDateWithFspSig) evalTime(row chunk.Row) (d types.Time, isNull loc := b.ctx.GetSessionVars().Location() now := time.Now().In(loc) - result, err := convertTimeToMysqlTime(now, int(fsp), types.ModeHalfEven) + result, err := convertTimeToMysqlTime(now, int8(fsp), types.ModeHalfEven) if err != nil { return types.Time{}, true, err } @@ -2051,7 +2051,7 @@ func (c *currentTimeFunctionClass) getFunction(ctx sessionctx.Context, args []Ex if len(args) == 0 { bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETDuration) - bf.tp.Flen, bf.tp.Decimal = mysql.MaxDurationWidthNoFsp, types.MinFsp + bf.tp.Flen, bf.tp.Decimal = mysql.MaxDurationWidthNoFsp, int(types.MinFsp) sig = &builtinCurrentTime0ArgSig{bf} return sig, nil } @@ -2120,7 +2120,7 @@ func (b *builtinCurrentTime1ArgSig) evalDuration(row chunk.Row) (types.Duration, return types.Duration{}, true, err } dur := nowTs.In(tz).Format(types.TimeFSPFormat) - res, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, dur, int(fsp)) + res, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, dur, int8(fsp)) if err != nil { return types.Duration{}, true, err } @@ -2168,12 +2168,14 @@ func (b *builtinTimeSig) evalDuration(row chunk.Row) (res types.Duration, isNull fsp = len(expr) - idx - 1 } - if fsp, err = types.CheckFsp(fsp); err != nil { + tmpFsp := int8(0) + if tmpFsp, err = types.CheckFsp(fsp); err != nil { return res, isNull, err } + fsp = int(tmpFsp) sc := b.ctx.GetSessionVars().StmtCtx - res, err = types.ParseDuration(sc, expr, fsp) + res, err = types.ParseDuration(sc, expr, int8(fsp)) if types.ErrTruncatedWrongVal.Equal(err) { err = sc.HandleTruncate(err) } @@ -2205,9 +2207,9 @@ func (c *timeLiteralFunctionClass) getFunction(ctx sessionctx.Context, args []Ex return nil, err } bf := newBaseBuiltinFuncWithTp(ctx, []Expression{}, types.ETDuration) - bf.tp.Flen, bf.tp.Decimal = 10, duration.Fsp - if duration.Fsp > 0 { - bf.tp.Flen += 1 + duration.Fsp + bf.tp.Flen, bf.tp.Decimal = 10, int(duration.Fsp) + if int(duration.Fsp) > 0 { + bf.tp.Flen += 1 + int(duration.Fsp) } sig := &builtinTimeLiteralSig{bf, duration} return sig, nil @@ -2277,9 +2279,9 @@ func getFlenAndDecimal4UTCTimestampAndNow(ctx sessionctx.Context, arg Expression if constant, ok := arg.(*Constant); ok { fsp, isNull, err := constant.EvalInt(ctx, chunk.Row{}) if isNull || err != nil || fsp > int64(types.MaxFsp) { - decimal = types.MaxFsp + decimal = int(types.MaxFsp) } else if fsp < int64(types.MinFsp) { - decimal = types.MinFsp + decimal = int(types.MinFsp) } else { decimal = int(fsp) } @@ -2317,7 +2319,7 @@ func (c *utcTimestampFunctionClass) getFunction(ctx sessionctx.Context, args []E return sig, nil } -func evalUTCTimestampWithFsp(ctx sessionctx.Context, fsp int) (types.Time, bool, error) { +func evalUTCTimestampWithFsp(ctx sessionctx.Context, fsp int8) (types.Time, bool, error) { nowTs, err := getStmtTimestamp(ctx) if err != nil { return types.Time{}, true, err @@ -2354,7 +2356,7 @@ func (b *builtinUTCTimestampWithArgSig) evalTime(row chunk.Row) (types.Time, boo return types.Time{}, true, errors.Errorf("Invalid negative %d specified, must in [0, 6].", num) } - result, isNull, err := evalUTCTimestampWithFsp(b.ctx, int(num)) + result, isNull, err := evalUTCTimestampWithFsp(b.ctx, int8(num)) return result, isNull, err } @@ -2371,7 +2373,7 @@ func (b *builtinUTCTimestampWithoutArgSig) Clone() builtinFunc { // evalTime evals UTC_TIMESTAMP(). // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_utc-timestamp func (b *builtinUTCTimestampWithoutArgSig) evalTime(row chunk.Row) (types.Time, bool, error) { - result, isNull, err := evalUTCTimestampWithFsp(b.ctx, 0) + result, isNull, err := evalUTCTimestampWithFsp(b.ctx, int8(0)) return result, isNull, err } @@ -2404,7 +2406,7 @@ func (c *nowFunctionClass) getFunction(ctx sessionctx.Context, args []Expression return sig, nil } -func evalNowWithFsp(ctx sessionctx.Context, fsp int) (types.Time, bool, error) { +func evalNowWithFsp(ctx sessionctx.Context, fsp int8) (types.Time, bool, error) { nowTs, err := getStmtTimestamp(ctx) if err != nil { return types.Time{}, true, err @@ -2458,7 +2460,7 @@ func (b *builtinNowWithArgSig) evalTime(row chunk.Row) (types.Time, bool, error) return types.Time{}, true, errors.Errorf("Invalid negative %d specified, must in [0, 6].", fsp) } - result, isNull, err := evalNowWithFsp(b.ctx, int(fsp)) + result, isNull, err := evalNowWithFsp(b.ctx, int8(fsp)) return result, isNull, err } @@ -2475,7 +2477,7 @@ func (b *builtinNowWithoutArgSig) Clone() builtinFunc { // evalTime evals NOW() // see: https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_now func (b *builtinNowWithoutArgSig) evalTime(row chunk.Row) (types.Time, bool, error) { - result, isNull, err := evalNowWithFsp(b.ctx, 0) + result, isNull, err := evalNowWithFsp(b.ctx, int8(0)) return result, isNull, err } @@ -2851,12 +2853,12 @@ func (c *addDateFunctionClass) getFunction(ctx sessionctx.Context, args []Expres switch unit { // If the unit has micro second, then the fsp must be the MaxFsp. case "MICROSECOND", "SECOND_MICROSECOND", "MINUTE_MICROSECOND", "HOUR_MICROSECOND", "DAY_MICROSECOND": - internalFsp = types.MaxFsp + internalFsp = int(types.MaxFsp) // If the unit is second, the fsp is related with the arg[1]'s. case "SECOND": - internalFsp = types.MaxFsp + internalFsp = int(types.MaxFsp) if intervalEvalTp != types.ETString { - internalFsp = mathutil.Min(args[1].GetType().Decimal, types.MaxFsp) + internalFsp = mathutil.Min(args[1].GetType().Decimal, int(types.MaxFsp)) } // Otherwise, the fsp should be 0. } @@ -3503,12 +3505,12 @@ func (c *subDateFunctionClass) getFunction(ctx sessionctx.Context, args []Expres switch unit { // If the unit has micro second, then the fsp must be the MaxFsp. case "MICROSECOND", "SECOND_MICROSECOND", "MINUTE_MICROSECOND", "HOUR_MICROSECOND", "DAY_MICROSECOND": - internalFsp = types.MaxFsp + internalFsp = int(types.MaxFsp) // If the unit is second, the fsp is related with the arg[1]'s. case "SECOND": - internalFsp = types.MaxFsp + internalFsp = int(types.MaxFsp) if intervalEvalTp != types.ETString { - internalFsp = mathutil.Min(args[1].GetType().Decimal, types.MaxFsp) + internalFsp = mathutil.Min(args[1].GetType().Decimal, int(types.MaxFsp)) } // Otherwise, the fsp should be 0. } @@ -4360,10 +4362,10 @@ type timestampFunctionClass struct { baseFunctionClass } -func (c *timestampFunctionClass) getDefaultFsp(tp *types.FieldType) int { +func (c *timestampFunctionClass) getDefaultFsp(tp *types.FieldType) int8 { if tp.Tp == mysql.TypeDatetime || tp.Tp == mysql.TypeDate || tp.Tp == mysql.TypeDuration || tp.Tp == mysql.TypeTimestamp { - return tp.Decimal + return int8(tp.Decimal) } switch cls := tp.EvalType(); cls { case types.ETInt: @@ -4371,8 +4373,8 @@ func (c *timestampFunctionClass) getDefaultFsp(tp *types.FieldType) int { case types.ETReal, types.ETDatetime, types.ETTimestamp, types.ETDuration, types.ETJson, types.ETString: return types.MaxFsp case types.ETDecimal: - if tp.Decimal < types.MaxFsp { - return tp.Decimal + if tp.Decimal < int(types.MaxFsp) { + return int8(tp.Decimal) } return types.MaxFsp } @@ -4389,7 +4391,7 @@ func (c *timestampFunctionClass) getFunction(ctx sessionctx.Context, args []Expr } fsp := c.getDefaultFsp(args[0].GetType()) if argLen == 2 { - fsp = mathutil.Max(fsp, c.getDefaultFsp(args[1].GetType())) + fsp = mathutil.MaxInt8(fsp, c.getDefaultFsp(args[1].GetType())) } isFloat := false switch args[0].GetType().Tp { @@ -4397,9 +4399,9 @@ func (c *timestampFunctionClass) getFunction(ctx sessionctx.Context, args []Expr isFloat = true } bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETDatetime, evalTps...) - bf.tp.Decimal, bf.tp.Flen = fsp, 19 + bf.tp.Decimal, bf.tp.Flen = int(fsp), 19 if fsp != 0 { - bf.tp.Flen += 1 + fsp + bf.tp.Flen += 1 + int(fsp) } var sig builtinFunc if argLen == 2 { @@ -4517,9 +4519,9 @@ func (c *timestampLiteralFunctionClass) getFunction(ctx sessionctx.Context, args return nil, err } bf := newBaseBuiltinFuncWithTp(ctx, []Expression{}, types.ETDatetime) - bf.tp.Flen, bf.tp.Decimal = mysql.MaxDatetimeWidthNoFsp, tm.Fsp + bf.tp.Flen, bf.tp.Decimal = mysql.MaxDatetimeWidthNoFsp, int(tm.Fsp) if tm.Fsp > 0 { - bf.tp.Flen += tm.Fsp + 1 + bf.tp.Flen += int(tm.Fsp) + 1 } sig := &builtinTimestampLiteralSig{bf, tm} return sig, nil @@ -4545,7 +4547,7 @@ func (b *builtinTimestampLiteralSig) evalTime(row chunk.Row) (types.Time, bool, // getFsp4TimeAddSub is used to in function 'ADDTIME' and 'SUBTIME' to evaluate `fsp` for the // second parameter. It's used only if the second parameter is of string type. It's different // from getFsp in that the result of getFsp4TimeAddSub is either 6 or 0. -func getFsp4TimeAddSub(s string) int { +func getFsp4TimeAddSub(s string) int8 { if len(s)-strings.Index(s, ".")-1 == len(s) { return types.MinFsp } @@ -4588,7 +4590,7 @@ func getBf4TimeAddSub(ctx sessionctx.Context, args []Expression) (tp1, tp2 *type } bf = newBaseBuiltinFuncWithTp(ctx, args, retTp, argTp1, argTp2) - bf.tp.Decimal = mathutil.Min(mathutil.Max(arg0Dec, arg1Dec), types.MaxFsp) + bf.tp.Decimal = mathutil.Min(mathutil.Max(arg0Dec, arg1Dec), int(types.MaxFsp)) if retTp == types.ETString { bf.tp.Tp, bf.tp.Flen, bf.tp.Decimal = mysql.TypeString, mysql.MaxDatetimeWidthWithFsp, types.UnspecifiedLength } @@ -5076,7 +5078,7 @@ type convertTzFunctionClass struct { } func (c *convertTzFunctionClass) getDecimal(ctx sessionctx.Context, arg Expression) int { - decimal := types.MaxFsp + decimal := int(types.MaxFsp) if dt, isConstant := arg.(*Constant); isConstant { switch arg.GetType().EvalType() { case types.ETInt: @@ -5090,11 +5092,11 @@ func (c *convertTzFunctionClass) getDecimal(ctx sessionctx.Context, arg Expressi } } } - if decimal > types.MaxFsp { - return types.MaxFsp + if decimal > int(types.MaxFsp) { + return int(types.MaxFsp) } - if decimal < types.MinFsp { - return types.MinFsp + if decimal < int(types.MinFsp) { + return int(types.MinFsp) } return decimal } @@ -5171,7 +5173,7 @@ func (b *builtinConvertTzSig) evalTime(row chunk.Row) (types.Time, bool, error) return types.Time{ Time: types.FromGoTime(t.In(toTz)), Type: mysql.TypeDatetime, - Fsp: b.tp.Decimal, + Fsp: int8(b.tp.Decimal), }, false, nil } if fromTzMatched && toTzMatched { @@ -5183,7 +5185,7 @@ func (b *builtinConvertTzSig) evalTime(row chunk.Row) (types.Time, bool, error) return types.Time{ Time: types.FromGoTime(t.Add(timeZone2Duration(toTzStr) - timeZone2Duration(fromTzStr))), Type: mysql.TypeDatetime, - Fsp: b.tp.Decimal, + Fsp: int8(b.tp.Decimal), }, false, nil } return types.Time{}, true, nil @@ -5358,7 +5360,7 @@ func (b *builtinMakeTimeSig) evalDuration(row chunk.Row) (types.Duration, bool, second = 59 } fsp := b.tp.Decimal - dur, err = types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, fmt.Sprintf("%02d:%02d:%v", hour, minute, second), fsp) + dur, err = types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, fmt.Sprintf("%02d:%02d:%v", hour, minute, second), int8(fsp)) if err != nil { return dur, true, err } @@ -5555,10 +5557,10 @@ func (c *secToTimeFunctionClass) getFunction(ctx sessionctx.Context, args []Expr } else { retFsp = argType.Decimal } - if retFsp > types.MaxFsp || retFsp == types.UnspecifiedLength { - retFsp = types.MaxFsp - } else if retFsp < types.MinFsp { - retFsp = types.MinFsp + if retFsp > int(types.MaxFsp) || retFsp == int(types.UnspecifiedFsp) { + retFsp = int(types.MaxFsp) + } else if retFsp < int(types.MinFsp) { + retFsp = int(types.MinFsp) } retFlen = 10 if retFsp > 0 { @@ -5615,7 +5617,7 @@ func (b *builtinSecToTimeSig) evalDuration(row chunk.Row) (types.Duration, bool, secondDemical = float64(second) + demical var dur types.Duration - dur, err = types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, fmt.Sprintf("%s%02d:%02d:%v", negative, hour, minute, secondDemical), b.tp.Decimal) + dur, err = types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, fmt.Sprintf("%s%02d:%02d:%v", negative, hour, minute, secondDemical), int8(b.tp.Decimal)) if err != nil { return types.Duration{}, err != nil, err } @@ -6297,9 +6299,9 @@ func (c *utcTimeFunctionClass) getFlenAndDecimal4UTCTime(ctx sessionctx.Context, if constant, ok := args[0].(*Constant); ok { fsp, isNull, err := constant.EvalInt(ctx, chunk.Row{}) if isNull || err != nil || fsp > int64(types.MaxFsp) { - decimal = types.MaxFsp + decimal = int(types.MaxFsp) } else if fsp < int64(types.MinFsp) { - decimal = types.MinFsp + decimal = int(types.MinFsp) } else { decimal = int(fsp) } @@ -6380,7 +6382,7 @@ func (b *builtinUTCTimeWithArgSig) evalDuration(row chunk.Row) (types.Duration, if err != nil { return types.Duration{}, true, err } - v, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, nowTs.UTC().Format(types.TimeFSPFormat), int(fsp)) + v, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, nowTs.UTC().Format(types.TimeFSPFormat), int8(fsp)) return v, false, err } @@ -6393,7 +6395,7 @@ func (c *lastDayFunctionClass) getFunction(ctx sessionctx.Context, args []Expres return nil, err } bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETDatetime, types.ETDatetime) - bf.tp.Tp, bf.tp.Flen, bf.tp.Decimal = mysql.TypeDate, mysql.MaxDateWidth, types.DefaultFsp + bf.tp.Tp, bf.tp.Flen, bf.tp.Decimal = mysql.TypeDate, mysql.MaxDateWidth, int(types.DefaultFsp) sig := &builtinLastDaySig{bf} return sig, nil } @@ -6437,9 +6439,9 @@ func getExpressionFsp(ctx sessionctx.Context, expression Expression) (int, error if isNil || err != nil { return 0, err } - return types.GetFsp(str), nil + return int(types.GetFsp(str)), nil } - return mathutil.Min(expression.GetType().Decimal, types.MaxFsp), nil + return mathutil.Min(expression.GetType().Decimal, int(types.MaxFsp)), nil } // tidbParseTsoFunctionClass extracts physical time from a tso @@ -6454,7 +6456,7 @@ func (c *tidbParseTsoFunctionClass) getFunction(ctx sessionctx.Context, args []E argTp := args[0].GetType().EvalType() bf := newBaseBuiltinFuncWithTp(ctx, args, argTp, types.ETInt) - bf.tp.Tp, bf.tp.Flen, bf.tp.Decimal = mysql.TypeDate, mysql.MaxDateWidth, types.DefaultFsp + bf.tp.Tp, bf.tp.Flen, bf.tp.Decimal = mysql.TypeDate, mysql.MaxDateWidth, int(types.DefaultFsp) sig := &builtinTidbParseTsoSig{bf} return sig, nil } diff --git a/expression/builtin_time_test.go b/expression/builtin_time_test.go index f37af43e18618..5141c684af658 100644 --- a/expression/builtin_time_test.go +++ b/expression/builtin_time_test.go @@ -892,7 +892,7 @@ func (s *testEvaluatorSuite) TestAddTimeSig(c *C) { c.Assert(err, IsNil) res, _, err := du.add(s.ctx, now, "1", "MICROSECOND") c.Assert(err, IsNil) - c.Assert(res.Fsp, Equals, 6) + c.Assert(res.Fsp, Equals, int8(6)) tbl = []struct { Input string @@ -1094,13 +1094,13 @@ func (s *testEvaluatorSuite) TestSysDate(c *C) { c.Assert(err, NotNil) } -func convertToTimeWithFsp(sc *stmtctx.StatementContext, arg types.Datum, tp byte, fsp int) (d types.Datum, err error) { +func convertToTimeWithFsp(sc *stmtctx.StatementContext, arg types.Datum, tp byte, fsp int8) (d types.Datum, err error) { if fsp > types.MaxFsp { fsp = types.MaxFsp } f := types.NewFieldType(tp) - f.Decimal = fsp + f.Decimal = int(fsp) d, err = arg.ConvertTo(sc, f) if err != nil { @@ -1475,7 +1475,7 @@ func (s *testEvaluatorSuite) TestTimeDiff(c *C) { args []interface{} expectStr string isNil bool - fsp int + fsp int8 getErr bool }{ {[]interface{}{"2000:01:01 00:00:00", "2000:01:01 00:00:00.000001"}, "-00:00:00.000001", false, 6, false}, @@ -1858,7 +1858,7 @@ func (s *testEvaluatorSuite) TestDateArithFuncs(c *C) { testDurations := []struct { fc functionClass dur string - fsp int + fsp int8 unit string format interface{} expected string diff --git a/expression/distsql_builtin.go b/expression/distsql_builtin.go index a603858d89591..42045b1dde823 100644 --- a/expression/distsql_builtin.go +++ b/expression/distsql_builtin.go @@ -554,7 +554,7 @@ func convertTime(data []byte, ftPB *tipb.FieldType, tz *time.Location) (*Constan } var t types.Time t.Type = ft.Tp - t.Fsp = ft.Decimal + t.Fsp = int8(ft.Decimal) err = t.FromPackedUint(v) if err != nil { return nil, err diff --git a/expression/distsql_builtin_test.go b/expression/distsql_builtin_test.go index a9ab97f2711c1..47f3f3aa704f7 100644 --- a/expression/distsql_builtin_test.go +++ b/expression/distsql_builtin_test.go @@ -920,8 +920,8 @@ func newIntFieldType() *types.FieldType { func newDurFieldType() *types.FieldType { return &types.FieldType{ - Tp: mysql.TypeDuration, - Flag: types.DefaultFsp, + Tp: mysql.TypeDuration, + Decimal: int(types.DefaultFsp), } } diff --git a/expression/helper.go b/expression/helper.go index f5268a6619107..e1f6980689abc 100644 --- a/expression/helper.go +++ b/expression/helper.go @@ -55,7 +55,7 @@ func IsValidCurrentTimestampExpr(exprNode ast.ExprNode, fieldType *types.FieldTy } // GetTimeValue gets the time value with type tp. -func GetTimeValue(ctx sessionctx.Context, v interface{}, tp byte, fsp int) (d types.Datum, err error) { +func GetTimeValue(ctx sessionctx.Context, v interface{}, tp byte, fsp int8) (d types.Datum, err error) { value := types.Time{ Type: tp, Fsp: fsp, @@ -70,7 +70,7 @@ func GetTimeValue(ctx sessionctx.Context, v interface{}, tp byte, fsp int) (d ty if err != nil { return d, err } - value.Time = types.FromGoTime(defaultTime.Truncate(time.Duration(math.Pow10(9-fsp)) * time.Nanosecond)) + value.Time = types.FromGoTime(defaultTime.Truncate(time.Duration(math.Pow10(9-int(fsp))) * time.Nanosecond)) if tp == mysql.TypeTimestamp || tp == mysql.TypeDatetime { err = value.ConvertTimeZone(time.Local, ctx.GetSessionVars().Location()) if err != nil { diff --git a/expression/typeinfer_test.go b/expression/typeinfer_test.go index dcd7571813b97..9acb4e7aa5ad8 100644 --- a/expression/typeinfer_test.go +++ b/expression/typeinfer_test.go @@ -1184,7 +1184,7 @@ func (s *testInferTypeSuite) createTestCase4OtherFuncs() []typeInferTestCase { {"bit_count(c_set )", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 2, 0}, {"bit_count(c_enum )", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 2, 0}, - {`@varname`, mysql.TypeVarString, charset.CharsetUTF8MB4, 0, mysql.MaxFieldVarCharLength, types.UnspecifiedFsp}, + {`@varname`, mysql.TypeVarString, charset.CharsetUTF8MB4, 0, mysql.MaxFieldVarCharLength, int(types.UnspecifiedFsp)}, } } @@ -1763,9 +1763,9 @@ func (s *testInferTypeSuite) createTestCase4TimeFuncs() []typeInferTestCase { {"quarter(c_set )", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 1, 0}, {"quarter(c_enum )", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 1, 0}, - {"current_time()", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxDurationWidthNoFsp, types.MinFsp}, - {"current_time(0)", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxDurationWidthWithFsp, types.MinFsp}, - {"current_time(6)", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxDurationWidthWithFsp, types.MaxFsp}, + {"current_time()", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxDurationWidthNoFsp, int(types.MinFsp)}, + {"current_time(0)", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxDurationWidthWithFsp, int(types.MinFsp)}, + {"current_time(6)", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxDurationWidthWithFsp, int(types.MaxFsp)}, {"sec_to_time(c_int_d )", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, 10, 0}, {"sec_to_time(c_bigint_d )", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, 10, 0}, @@ -1807,11 +1807,11 @@ func (s *testInferTypeSuite) createTestCase4TimeFuncs() []typeInferTestCase { {"time_to_sec(c_set )", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 10, 0}, {"time_to_sec(c_enum )", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 10, 0}, - {"str_to_date(c_varchar, '%Y:%m:%d')", mysql.TypeDate, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxDateWidth, types.MinFsp}, - {"str_to_date(c_varchar, '%Y:%m:%d %H:%i:%s')", mysql.TypeDatetime, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxDatetimeWidthNoFsp, types.MinFsp}, - {"str_to_date(c_varchar, '%Y:%m:%d %H:%i:%s.%f')", mysql.TypeDatetime, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxDatetimeWidthWithFsp, types.MaxFsp}, - {"str_to_date(c_varchar, '%H:%i:%s')", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxDurationWidthNoFsp, types.MinFsp}, - {"str_to_date(c_varchar, '%H:%i:%s.%f')", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxDurationWidthWithFsp, types.MaxFsp}, + {"str_to_date(c_varchar, '%Y:%m:%d')", mysql.TypeDate, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxDateWidth, int(types.MinFsp)}, + {"str_to_date(c_varchar, '%Y:%m:%d %H:%i:%s')", mysql.TypeDatetime, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxDatetimeWidthNoFsp, int(types.MinFsp)}, + {"str_to_date(c_varchar, '%Y:%m:%d %H:%i:%s.%f')", mysql.TypeDatetime, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxDatetimeWidthWithFsp, int(types.MaxFsp)}, + {"str_to_date(c_varchar, '%H:%i:%s')", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxDurationWidthNoFsp, int(types.MinFsp)}, + {"str_to_date(c_varchar, '%H:%i:%s.%f')", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxDurationWidthWithFsp, int(types.MaxFsp)}, {"period_add(c_int_d , c_int_d)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 6, 0}, {"period_add(c_bigint_d , c_int_d)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 6, 0}, @@ -1856,11 +1856,11 @@ func (s *testInferTypeSuite) createTestCase4TimeFuncs() []typeInferTestCase { {"get_format(DATE, 'USA')", mysql.TypeVarString, charset.CharsetUTF8MB4, 0, 17, types.UnspecifiedLength}, - {"convert_tz(c_time_d, c_text_d, c_text_d)", mysql.TypeDatetime, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxDatetimeWidthWithFsp, types.MaxFsp}, + {"convert_tz(c_time_d, c_text_d, c_text_d)", mysql.TypeDatetime, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxDatetimeWidthWithFsp, int(types.MaxFsp)}, {"from_unixtime(20170101.999)", mysql.TypeDatetime, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxDatetimeWidthWithFsp, 3}, - {"from_unixtime(20170101.1234567)", mysql.TypeDatetime, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxDatetimeWidthWithFsp, types.MaxFsp}, - {"from_unixtime('20170101.999')", mysql.TypeDatetime, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxDatetimeWidthWithFsp, types.MaxFsp}, + {"from_unixtime(20170101.1234567)", mysql.TypeDatetime, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxDatetimeWidthWithFsp, int(types.MaxFsp)}, + {"from_unixtime('20170101.999')", mysql.TypeDatetime, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxDatetimeWidthWithFsp, int(types.MaxFsp)}, {"from_unixtime(20170101.123, '%H')", mysql.TypeVarString, charset.CharsetUTF8MB4, 0, 2, types.UnspecifiedLength}, {"extract(day from c_char)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxIntWidth, 0}, diff --git a/planner/core/expression_rewriter.go b/planner/core/expression_rewriter.go index af1be06afc031..5cd66c40b97b8 100644 --- a/planner/core/expression_rewriter.go +++ b/planner/core/expression_rewriter.go @@ -939,7 +939,7 @@ func (er *expressionRewriter) newFunction(funcName string, retType *types.FieldT } func (er *expressionRewriter) checkTimePrecision(ft *types.FieldType) error { - if ft.EvalType() == types.ETDuration && ft.Decimal > types.MaxFsp { + if ft.EvalType() == types.ETDuration && ft.Decimal > int(types.MaxFsp) { return errTooBigPrecision.GenWithStackByArgs(ft.Decimal, "CAST", types.MaxFsp) } return nil @@ -1527,7 +1527,7 @@ func (er *expressionRewriter) evalDefaultExpr(v *ast.DefaultExpr) { zero := types.Time{ Time: types.ZeroTime, Type: mysql.TypeTimestamp, - Fsp: col.Decimal, + Fsp: int8(col.Decimal), } val = &expression.Constant{ Value: types.NewDatum(zero), diff --git a/server/driver_tidb.go b/server/driver_tidb.go index c5c9324a83f49..8bcdb84b52dac 100644 --- a/server/driver_tidb.go +++ b/server/driver_tidb.go @@ -415,7 +415,7 @@ func convertColumnInfo(fld *ast.ResultField) (ci *ColumnInfo) { if fld.Column.Tp == mysql.TypeNewDecimal { // Consider the negative sign. ci.ColumnLength++ - if fld.Column.Decimal > types.DefaultFsp { + if fld.Column.Decimal > int(types.DefaultFsp) { // Consider the decimal point. ci.ColumnLength++ } @@ -442,7 +442,7 @@ func convertColumnInfo(fld *ast.ResultField) (ci *ColumnInfo) { if fld.Column.Decimal == types.UnspecifiedLength { if fld.Column.Tp == mysql.TypeDuration { - ci.Decimal = types.DefaultFsp + ci.Decimal = uint8(types.DefaultFsp) } else { ci.Decimal = mysql.NotFixedDec } diff --git a/statistics/scalar.go b/statistics/scalar.go index 53073e16967ce..dd54b0e6043a5 100644 --- a/statistics/scalar.go +++ b/statistics/scalar.go @@ -237,8 +237,8 @@ func enumRangeValues(low, high types.Datum, lowExclude, highExclude bool) []type return values case types.KindMysqlDuration: lowDur, highDur := low.GetMysqlDuration(), high.GetMysqlDuration() - fsp := mathutil.Max(lowDur.Fsp, highDur.Fsp) - stepSize := int64(math.Pow10(types.MaxFsp-fsp)) * int64(time.Microsecond) + fsp := mathutil.MaxInt8(lowDur.Fsp, highDur.Fsp) + stepSize := int64(math.Pow10(int(types.MaxFsp-fsp))) * int64(time.Microsecond) lowDur.Duration = lowDur.Duration.Round(time.Duration(stepSize)) remaining := int64(highDur.Duration-lowDur.Duration)/stepSize + 1 - int64(exclude) if remaining >= maxNumStep { @@ -258,7 +258,7 @@ func enumRangeValues(low, high types.Datum, lowExclude, highExclude bool) []type if lowTime.Type != highTime.Type { return nil } - fsp := mathutil.Max(lowTime.Fsp, highTime.Fsp) + fsp := mathutil.MaxInt8(lowTime.Fsp, highTime.Fsp) var stepSize int64 sc := &stmtctx.StatementContext{TimeZone: time.UTC} if lowTime.Type == mysql.TypeDate { @@ -270,7 +270,7 @@ func enumRangeValues(low, high types.Datum, lowExclude, highExclude bool) []type if err != nil { return nil } - stepSize = int64(math.Pow10(types.MaxFsp-fsp)) * int64(time.Microsecond) + stepSize = int64(math.Pow10(int(types.MaxFsp-fsp))) * int64(time.Microsecond) } remaining := int64(highTime.Sub(sc, &lowTime).Duration)/stepSize + 1 - int64(exclude) if remaining >= maxNumStep { diff --git a/table/column.go b/table/column.go index 21c961dbbfc0a..412dcce6c50aa 100644 --- a/table/column.go +++ b/table/column.go @@ -399,7 +399,7 @@ func getColDefaultValue(ctx sessionctx.Context, col *model.ColumnInfo, defaultVa defer func() { sc.TimeZone = originalTZ }() } } - value, err := expression.GetTimeValue(ctx, defaultVal, col.Tp, col.Decimal) + value, err := expression.GetTimeValue(ctx, defaultVal, col.Tp, int8(col.Decimal)) if err != nil { return types.Datum{}, errGetDefaultFailed.GenWithStack("Field '%s' get default value fail - %s", col.Name, err) diff --git a/tablecodec/tablecodec.go b/tablecodec/tablecodec.go index 568cc51f3f529..99e8edd7e5247 100644 --- a/tablecodec/tablecodec.go +++ b/tablecodec/tablecodec.go @@ -426,7 +426,7 @@ func unflatten(datum types.Datum, ft *types.FieldType, loc *time.Location) (type case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeTimestamp: var t types.Time t.Type = ft.Tp - t.Fsp = ft.Decimal + t.Fsp = int8(ft.Decimal) var err error err = t.FromPackedUint(datum.GetUint64()) if err != nil { @@ -442,7 +442,7 @@ func unflatten(datum types.Datum, ft *types.FieldType, loc *time.Location) (type datum.SetMysqlTime(t) return datum, nil case mysql.TypeDuration: //duration should read fsp from column meta data - dur := types.Duration{Duration: time.Duration(datum.GetInt64()), Fsp: ft.Decimal} + dur := types.Duration{Duration: time.Duration(datum.GetInt64()), Fsp: int8(ft.Decimal)} datum.SetValue(dur) return datum, nil case mysql.TypeEnum: diff --git a/types/convert.go b/types/convert.go index 1f733d50354d1..db9c9dfdb5a4a 100644 --- a/types/convert.go +++ b/types/convert.go @@ -293,7 +293,7 @@ func StrToUint(sc *stmtctx.StatementContext, str string) (uint64, error) { } // StrToDateTime converts str to MySQL DateTime. -func StrToDateTime(sc *stmtctx.StatementContext, str string, fsp int) (Time, error) { +func StrToDateTime(sc *stmtctx.StatementContext, str string, fsp int8) (Time, error) { return ParseTime(sc, str, mysql.TypeDatetime, fsp) } @@ -301,7 +301,7 @@ func StrToDateTime(sc *stmtctx.StatementContext, str string, fsp int) (Time, err // and returns Time when str is in datetime format. // when isDuration is true, the d is returned, when it is false, the t is returned. // See https://dev.mysql.com/doc/refman/5.5/en/date-and-time-literals.html. -func StrToDuration(sc *stmtctx.StatementContext, str string, fsp int) (d Duration, t Time, isDuration bool, err error) { +func StrToDuration(sc *stmtctx.StatementContext, str string, fsp int8) (d Duration, t Time, isDuration bool, err error) { str = strings.TrimSpace(str) length := len(str) if length > 0 && str[0] == '-' { @@ -324,7 +324,7 @@ func StrToDuration(sc *stmtctx.StatementContext, str string, fsp int) (d Duratio } // NumberToDuration converts number to Duration. -func NumberToDuration(number int64, fsp int) (Duration, error) { +func NumberToDuration(number int64, fsp int8) (Duration, error) { if number > TimeMaxValue { // Try to parse DATETIME. if number >= 10000000000 { // '2001-00-00 00-00-00' diff --git a/types/convert_test.go b/types/convert_test.go index de86c398f7195..5246f9bf6d751 100644 --- a/types/convert_test.go +++ b/types/convert_test.go @@ -1008,7 +1008,7 @@ func (s *testTypeConvertSuite) TestNumberToDuration(c *C) { } for _, tc := range testCases { - dur, err := NumberToDuration(tc.number, tc.fsp) + dur, err := NumberToDuration(tc.number, int8(tc.fsp)) if tc.hasErr { c.Assert(err, NotNil) continue @@ -1038,7 +1038,7 @@ func (s *testTypeConvertSuite) TestStrToDuration(c *C) { sc := new(stmtctx.StatementContext) var tests = []struct { str string - fsp int + fsp int8 isDuration bool }{ {"20190412120000", 4, false}, diff --git a/types/datum.go b/types/datum.go index c4f7edb276467..5e448d77652df 100644 --- a/types/datum.go +++ b/types/datum.go @@ -260,7 +260,7 @@ func (d *Datum) SetMysqlDecimal(b *MyDecimal) { // GetMysqlDuration gets Duration value func (d *Datum) GetMysqlDuration() Duration { - return Duration{Duration: time.Duration(d.i), Fsp: int(d.decimal)} + return Duration{Duration: time.Duration(d.i), Fsp: int8(d.decimal)} } // SetMysqlDuration sets Duration value @@ -939,7 +939,7 @@ func (d *Datum) convertToMysqlTimestamp(sc *stmtctx.StatementContext, target *Fi ) fsp := DefaultFsp if target.Decimal != UnspecifiedLength { - fsp = target.Decimal + fsp = int8(target.Decimal) } switch d.k { case KindMysqlTime: @@ -973,7 +973,7 @@ func (d *Datum) convertToMysqlTime(sc *stmtctx.StatementContext, target *FieldTy tp := target.Tp fsp := DefaultFsp if target.Decimal != UnspecifiedLength { - fsp = target.Decimal + fsp = int8(target.Decimal) } var ( ret Datum @@ -1019,7 +1019,7 @@ func (d *Datum) convertToMysqlDuration(sc *stmtctx.StatementContext, target *Fie tp := target.Tp fsp := DefaultFsp if target.Decimal != UnspecifiedLength { - fsp = target.Decimal + fsp = int8(target.Decimal) } var ret Datum switch d.k { diff --git a/types/datum_test.go b/types/datum_test.go index a1e8eb468983b..9b983ad496921 100644 --- a/types/datum_test.go +++ b/types/datum_test.go @@ -227,7 +227,7 @@ func (ts *testTypeConvertSuite) TestToFloat64(c *C) { } // mustParseTimeIntoDatum is similar to ParseTime but panic if any error occurs. -func mustParseTimeIntoDatum(s string, tp byte, fsp int) (d Datum) { +func mustParseTimeIntoDatum(s string, tp byte, fsp int8) (d Datum) { t, err := ParseTime(&stmtctx.StatementContext{TimeZone: time.UTC}, s, tp, fsp) if err != nil { panic("ParseTime fail") @@ -316,14 +316,6 @@ func (ts *testDatumSuite) TestToBytes(c *C) { } } -func mustParseDurationDatum(str string, fsp int) Datum { - dur, err := ParseDuration(nil, str, fsp) - if err != nil { - panic(err) - } - return NewDurationDatum(dur) -} - func (ts *testDatumSuite) TestComputePlusAndMinus(c *C) { sc := &stmtctx.StatementContext{TimeZone: time.UTC} tests := []struct { diff --git a/types/field_type.go b/types/field_type.go index ddb836f3b3752..b50f8e4e996d1 100644 --- a/types/field_type.go +++ b/types/field_type.go @@ -221,18 +221,18 @@ func DefaultTypeForValue(value interface{}, tp *FieldType) { case mysql.TypeDatetime, mysql.TypeTimestamp: tp.Flen = mysql.MaxDatetimeWidthNoFsp if x.Fsp > DefaultFsp { // consider point('.') and the fractional part. - tp.Flen += x.Fsp + 1 + tp.Flen += int(x.Fsp) + 1 } - tp.Decimal = x.Fsp + tp.Decimal = int(x.Fsp) } SetBinChsClnFlag(tp) case Duration: tp.Tp = mysql.TypeDuration tp.Flen = len(x.String()) if x.Fsp > DefaultFsp { // consider point('.') and the fractional part. - tp.Flen = x.Fsp + 1 + tp.Flen = int(x.Fsp) + 1 } - tp.Decimal = x.Fsp + tp.Decimal = int(x.Fsp) SetBinChsClnFlag(tp) case *MyDecimal: tp.Tp = mysql.TypeNewDecimal diff --git a/types/fsp.go b/types/fsp.go index c9709822c6453..1059c9d5dd789 100644 --- a/types/fsp.go +++ b/types/fsp.go @@ -23,46 +23,46 @@ import ( const ( // UnspecifiedFsp is the unspecified fractional seconds part. - UnspecifiedFsp = -1 + UnspecifiedFsp = int8(-1) // MaxFsp is the maximum digit of fractional seconds part. - MaxFsp = 6 + MaxFsp = int8(6) // MinFsp is the minimum digit of fractional seconds part. - MinFsp = 0 + MinFsp = int8(0) // DefaultFsp is the default digit of fractional seconds part. // MySQL use 0 as the default Fsp. - DefaultFsp = 0 + DefaultFsp = int8(0) ) // CheckFsp checks whether fsp is in valid range. -func CheckFsp(fsp int) (int, error) { - if fsp == UnspecifiedFsp { +func CheckFsp(fsp int) (int8, error) { + if fsp == int(UnspecifiedFsp) { return DefaultFsp, nil } - if fsp < MinFsp || fsp > MaxFsp { + if fsp < int(MinFsp) || fsp > int(MaxFsp) { return DefaultFsp, errors.Errorf("Invalid fsp %d", fsp) } - return fsp, nil + return int8(fsp), nil } // ParseFrac parses the input string according to fsp, returns the microsecond, // and also a bool value to indice overflow. eg: // "999" fsp=2 will overflow. -func ParseFrac(s string, fsp int) (v int, overflow bool, err error) { +func ParseFrac(s string, fsp int8) (v int, overflow bool, err error) { if len(s) == 0 { return 0, false, nil } - fsp, err = CheckFsp(fsp) + fsp, err = CheckFsp(int(fsp)) if err != nil { return 0, false, errors.Trace(err) } - if fsp >= len(s) { + if int(fsp) >= len(s) { tmp, e := strconv.ParseInt(s, 10, 64) if e != nil { return 0, false, errors.Trace(e) } - v = int(float64(tmp) * math.Pow10(MaxFsp-len(s))) + v = int(float64(tmp) * math.Pow10(int(MaxFsp)-len(s))) return } @@ -73,7 +73,7 @@ func ParseFrac(s string, fsp int) (v int, overflow bool, err error) { } tmp = (tmp + 5) / 10 - if float64(tmp) >= math.Pow10(fsp) { + if float64(tmp) >= math.Pow10(int(fsp)) { // overflow return 0, true, nil } @@ -82,7 +82,7 @@ func ParseFrac(s string, fsp int) (v int, overflow bool, err error) { // 1236 round 3 -> 124 -> 124000 // 0312 round 2 -> 3 -> 30000 // 999 round 2 -> 100 -> overflow - v = int(float64(tmp) * math.Pow10(MaxFsp-fsp)) + v = int(float64(tmp) * math.Pow10(int(MaxFsp-fsp))) return } diff --git a/types/fsp_test.go b/types/fsp_test.go index b8f29cd4077d7..1dc02cc3da089 100644 --- a/types/fsp_test.go +++ b/types/fsp_test.go @@ -25,7 +25,7 @@ type FspTest struct{} func (s *FspTest) TestCheckFsp(c *C) { c.Parallel() - obtained, err := CheckFsp(UnspecifiedFsp) + obtained, err := CheckFsp(int(UnspecifiedFsp)) c.Assert(obtained, Equals, DefaultFsp) c.Assert(err, IsNil) @@ -33,33 +33,33 @@ func (s *FspTest) TestCheckFsp(c *C) { c.Assert(obtained, Equals, DefaultFsp) c.Assert(err, ErrorMatches, "Invalid fsp -2019") - obtained, err = CheckFsp(MinFsp - 4294967296) + obtained, err = CheckFsp(int(MinFsp) - 4294967296) c.Assert(obtained, Equals, DefaultFsp) - c.Assert(err, ErrorMatches, "Invalid fsp "+strconv.Itoa(MinFsp-4294967296)) + c.Assert(err, ErrorMatches, "Invalid fsp "+strconv.Itoa(int(MinFsp)-4294967296)) // UnspecifiedFsp obtained, err = CheckFsp(-1) c.Assert(obtained, Equals, DefaultFsp) c.Assert(err, IsNil) - obtained, err = CheckFsp(MaxFsp + 1) + obtained, err = CheckFsp(int(MaxFsp) + 1) c.Assert(obtained, Equals, DefaultFsp) - c.Assert(err, ErrorMatches, "Invalid fsp "+strconv.Itoa(MaxFsp+1)) + c.Assert(err, ErrorMatches, "Invalid fsp "+strconv.Itoa(int(MaxFsp)+1)) - obtained, err = CheckFsp(MaxFsp + 2019) + obtained, err = CheckFsp(int(MaxFsp) + 2019) c.Assert(obtained, Equals, DefaultFsp) - c.Assert(err, ErrorMatches, "Invalid fsp "+strconv.Itoa(MaxFsp+2019)) + c.Assert(err, ErrorMatches, "Invalid fsp "+strconv.Itoa(int(MaxFsp)+2019)) - obtained, err = CheckFsp(MaxFsp + 4294967296) + obtained, err = CheckFsp(int(MaxFsp) + 4294967296) c.Assert(obtained, Equals, DefaultFsp) - c.Assert(err, ErrorMatches, "Invalid fsp "+strconv.Itoa(MaxFsp+4294967296)) + c.Assert(err, ErrorMatches, "Invalid fsp "+strconv.Itoa(int(MaxFsp)+4294967296)) - obtained, err = CheckFsp((MaxFsp + MinFsp) / 2) + obtained, err = CheckFsp(int(MaxFsp+MinFsp) / 2) c.Assert(obtained, Equals, (MaxFsp+MinFsp)/2) c.Assert(err, IsNil) obtained, err = CheckFsp(5) - c.Assert(obtained, Equals, 5) + c.Assert(obtained, Equals, int8(5)) c.Assert(err, IsNil) } @@ -70,7 +70,8 @@ func (s *FspTest) TestParseFrac(c *C) { c.Assert(overflow, Equals, false) c.Assert(err, IsNil) - obtained, overflow, err = ParseFrac("999", 200) + a := 200 + obtained, overflow, err = ParseFrac("999", int8(a)) c.Assert(obtained, Equals, 0) c.Assert(overflow, Equals, false) c.Assert(err, ErrorMatches, "Invalid fsp .*") diff --git a/types/mytime.go b/types/mytime.go index 733571f0239a1..b5923eec36b0a 100644 --- a/types/mytime.go +++ b/types/mytime.go @@ -16,19 +16,27 @@ package types import ( gotime "time" + "fmt" "github.com/pingcap/errors" ) // MysqlTime is the internal struct type for Time. +// The order of the attributes is refined to reduce the memory overhead +// considering memory alignment. type MysqlTime struct { - year uint16 // year <= 9999 - month uint8 // month <= 12 - day uint8 // day <= 31 - // When it's type is Time, HH:MM:SS may be 839:59:59, so use int to avoid overflow. - hour int // hour <= 23 - minute uint8 // minute <= 59 - second uint8 // second <= 59 + // When it's type is Time, HH:MM:SS may be 839:59:59, so use uint32 to avoid overflow. + hour uint32 // hour <= 23 microsecond uint32 + year uint16 // year <= 9999 + month uint8 // month <= 12 + day uint8 // day <= 31 + minute uint8 // minute <= 59 + second uint8 // second <= 59 +} + +// String implements fmt.Stringer. +func (t MysqlTime) String() string { + return fmt.Sprintf("{%d %d %d %d %d %d %d}", t.year, t.month, t.day, t.hour, t.minute, t.second, t.microsecond) } // Year returns the year value. @@ -173,7 +181,7 @@ func AddDate(year, month, day int64, ot gotime.Time) (nt gotime.Time) { } func calcTimeFromSec(to *MysqlTime, seconds, microseconds int) { - to.hour = seconds / 3600 + to.hour = uint32(seconds / 3600) seconds = seconds % 3600 to.minute = uint8(seconds / 60) to.second = uint8(seconds % 60) diff --git a/types/mytime_test.go b/types/mytime_test.go index 76a32f4b40a94..748bb02e8aed9 100644 --- a/types/mytime_test.go +++ b/types/mytime_test.go @@ -15,6 +15,7 @@ package types import ( "time" + "unsafe" . "github.com/pingcap/check" ) @@ -39,9 +40,9 @@ func (s *testMyTimeSuite) TestWeek(c *C) { Mode int Expect int }{ - {MysqlTime{2008, 2, 20, 0, 0, 0, 0}, 0, 7}, - {MysqlTime{2008, 2, 20, 0, 0, 0, 0}, 1, 8}, - {MysqlTime{2008, 12, 31, 0, 0, 0, 0}, 1, 53}, + {MysqlTime{year: 2008, month: 2, day: 20, hour: 0, minute: 0, second: 0, microsecond: 0}, 0, 7}, + {MysqlTime{year: 2008, month: 2, day: 20, hour: 0, minute: 0, second: 0, microsecond: 0}, 1, 8}, + {MysqlTime{year: 2008, month: 12, day: 31, hour: 0, minute: 0, second: 0, microsecond: 0}, 1, 53}, } for ith, tt := range tests { @@ -68,22 +69,22 @@ func (s *testMyTimeSuite) TestCalcTimeDiff(c *C) { }{ // calcTimeDiff can be used for month = 0. { - MysqlTime{2006, 0, 1, 12, 23, 21, 0}, - MysqlTime{2006, 0, 3, 21, 23, 22, 0}, + MysqlTime{year: 2006, month: 0, day: 1, hour: 12, minute: 23, second: 21, microsecond: 0}, + MysqlTime{year: 2006, month: 0, day: 3, hour: 21, minute: 23, second: 22, microsecond: 0}, 1, - MysqlTime{0, 0, 0, 57, 0, 1, 0}, + MysqlTime{year: 0, month: 0, day: 0, hour: 57, minute: 0, second: 1, microsecond: 0}, }, { - MysqlTime{0, 0, 0, 21, 23, 24, 0}, - MysqlTime{0, 0, 0, 11, 23, 22, 0}, + MysqlTime{year: 0, month: 0, day: 0, hour: 21, minute: 23, second: 24, microsecond: 0}, + MysqlTime{year: 0, month: 0, day: 0, hour: 11, minute: 23, second: 22, microsecond: 0}, 1, - MysqlTime{0, 0, 0, 10, 0, 2, 0}, + MysqlTime{year: 0, month: 0, day: 0, hour: 10, minute: 0, second: 2, microsecond: 0}, }, { - MysqlTime{0, 0, 0, 1, 2, 3, 0}, - MysqlTime{0, 0, 0, 5, 2, 0, 0}, + MysqlTime{year: 0, month: 0, day: 0, hour: 1, minute: 2, second: 3, microsecond: 0}, + MysqlTime{year: 0, month: 0, day: 0, hour: 5, minute: 2, second: 0, microsecond: 0}, -1, - MysqlTime{0, 0, 0, 6, 4, 3, 0}, + MysqlTime{year: 0, month: 0, day: 0, hour: 6, minute: 4, second: 3, microsecond: 0}, }, } @@ -101,11 +102,11 @@ func (s *testMyTimeSuite) TestCompareTime(c *C) { T2 MysqlTime Expect int }{ - {MysqlTime{0, 0, 0, 0, 0, 0, 0}, MysqlTime{0, 0, 0, 0, 0, 0, 0}, 0}, - {MysqlTime{0, 0, 0, 0, 1, 0, 0}, MysqlTime{0, 0, 0, 0, 0, 0, 0}, 1}, - {MysqlTime{2006, 1, 2, 3, 4, 5, 6}, MysqlTime{2016, 1, 2, 3, 4, 5, 0}, -1}, - {MysqlTime{0, 0, 0, 11, 22, 33, 0}, MysqlTime{0, 0, 0, 12, 21, 33, 0}, -1}, - {MysqlTime{9999, 12, 30, 23, 59, 59, 999999}, MysqlTime{0, 1, 2, 3, 4, 5, 6}, 1}, + {MysqlTime{year: 0, month: 0, day: 0, hour: 0, minute: 0, second: 0, microsecond: 0}, MysqlTime{year: 0, month: 0, day: 0, hour: 0, minute: 0, second: 0, microsecond: 0}, 0}, + {MysqlTime{year: 0, month: 0, day: 0, hour: 0, minute: 1, second: 0, microsecond: 0}, MysqlTime{year: 0, month: 0, day: 0, hour: 0, minute: 0, second: 0, microsecond: 0}, 1}, + {MysqlTime{year: 2006, month: 1, day: 2, hour: 3, minute: 4, second: 5, microsecond: 6}, MysqlTime{year: 2016, month: 1, day: 2, hour: 3, minute: 4, second: 5, microsecond: 0}, -1}, + {MysqlTime{year: 0, month: 0, day: 0, hour: 11, minute: 22, second: 33, microsecond: 0}, MysqlTime{year: 0, month: 0, day: 0, hour: 12, minute: 21, second: 33, microsecond: 0}, -1}, + {MysqlTime{year: 9999, month: 12, day: 30, hour: 23, minute: 59, second: 59, microsecond: 999999}, MysqlTime{year: 0, month: 1, day: 2, hour: 3, minute: 4, second: 5, microsecond: 6}, 1}, } for _, tt := range tests { @@ -151,34 +152,34 @@ func (s *testMyTimeSuite) TestMixDateAndTime(c *C) { expect MysqlTime }{ { - date: MysqlTime{1896, 3, 4, 0, 0, 0, 0}, - time: MysqlTime{0, 0, 0, 12, 23, 24, 5}, + date: MysqlTime{year: 1896, month: 3, day: 4, hour: 0, minute: 0, second: 0, microsecond: 0}, + time: MysqlTime{year: 0, month: 0, day: 0, hour: 12, minute: 23, second: 24, microsecond: 5}, neg: false, - expect: MysqlTime{1896, 3, 4, 12, 23, 24, 5}, + expect: MysqlTime{year: 1896, month: 3, day: 4, hour: 12, minute: 23, second: 24, microsecond: 5}, }, { - date: MysqlTime{1896, 3, 4, 0, 0, 0, 0}, - time: MysqlTime{0, 0, 0, 24, 23, 24, 5}, + date: MysqlTime{year: 1896, month: 3, day: 4, hour: 0, minute: 0, second: 0, microsecond: 0}, + time: MysqlTime{year: 0, month: 0, day: 0, hour: 24, minute: 23, second: 24, microsecond: 5}, neg: false, - expect: MysqlTime{1896, 3, 5, 0, 23, 24, 5}, + expect: MysqlTime{year: 1896, month: 3, day: 5, hour: 0, minute: 23, second: 24, microsecond: 5}, }, { - date: MysqlTime{2016, 12, 31, 0, 0, 0, 0}, - time: MysqlTime{0, 0, 0, 24, 0, 0, 0}, + date: MysqlTime{year: 2016, month: 12, day: 31, hour: 0, minute: 0, second: 0, microsecond: 0}, + time: MysqlTime{year: 0, month: 0, day: 0, hour: 24, minute: 0, second: 0, microsecond: 0}, neg: false, - expect: MysqlTime{2017, 1, 1, 0, 0, 0, 0}, + expect: MysqlTime{year: 2017, month: 1, day: 1, hour: 0, minute: 0, second: 0, microsecond: 0}, }, { - date: MysqlTime{2016, 12, 0, 0, 0, 0, 0}, - time: MysqlTime{0, 0, 0, 24, 0, 0, 0}, + date: MysqlTime{year: 2016, month: 12, day: 0, hour: 0, minute: 0, second: 0, microsecond: 0}, + time: MysqlTime{year: 0, month: 0, day: 0, hour: 24, minute: 0, second: 0, microsecond: 0}, neg: false, - expect: MysqlTime{2016, 12, 1, 0, 0, 0, 0}, + expect: MysqlTime{year: 2016, month: 12, day: 1, hour: 0, minute: 0, second: 0, microsecond: 0}, }, { - date: MysqlTime{2017, 1, 12, 3, 23, 15, 0}, - time: MysqlTime{0, 0, 0, 2, 21, 10, 0}, + date: MysqlTime{year: 2017, month: 1, day: 12, hour: 3, minute: 23, second: 15, microsecond: 0}, + time: MysqlTime{year: 0, month: 0, day: 0, hour: 2, minute: 21, second: 10, microsecond: 0}, neg: true, - expect: MysqlTime{2017, 1, 12, 1, 2, 5, 0}, + expect: MysqlTime{year: 2017, month: 1, day: 12, hour: 1, minute: 2, second: 5, microsecond: 0}, }, } @@ -193,19 +194,19 @@ func (s *testMyTimeSuite) TestIsLeapYear(c *C) { T MysqlTime Expect bool }{ - {MysqlTime{1960, 1, 1, 0, 0, 0, 0}, true}, - {MysqlTime{1963, 2, 21, 0, 0, 0, 0}, false}, - {MysqlTime{2008, 11, 25, 0, 0, 0, 0}, true}, - {MysqlTime{2017, 4, 24, 0, 0, 0, 0}, false}, - {MysqlTime{1988, 2, 29, 0, 0, 0, 0}, true}, - {MysqlTime{2000, 3, 15, 0, 0, 0, 0}, true}, - {MysqlTime{1992, 5, 3, 0, 0, 0, 0}, true}, - {MysqlTime{2024, 10, 1, 0, 0, 0, 0}, true}, - {MysqlTime{2016, 6, 29, 0, 0, 0, 0}, true}, - {MysqlTime{2015, 6, 29, 0, 0, 0, 0}, false}, - {MysqlTime{2014, 9, 31, 0, 0, 0, 0}, false}, - {MysqlTime{2001, 12, 7, 0, 0, 0, 0}, false}, - {MysqlTime{1989, 7, 6, 0, 0, 0, 0}, false}, + {MysqlTime{year: 1960, month: 1, day: 1, hour: 0, minute: 0, second: 0, microsecond: 0}, true}, + {MysqlTime{year: 1963, month: 2, day: 21, hour: 0, minute: 0, second: 0, microsecond: 0}, false}, + {MysqlTime{year: 2008, month: 11, day: 25, hour: 0, minute: 0, second: 0, microsecond: 0}, true}, + {MysqlTime{year: 2017, month: 4, day: 24, hour: 0, minute: 0, second: 0, microsecond: 0}, false}, + {MysqlTime{year: 1988, month: 2, day: 29, hour: 0, minute: 0, second: 0, microsecond: 0}, true}, + {MysqlTime{year: 2000, month: 3, day: 15, hour: 0, minute: 0, second: 0, microsecond: 0}, true}, + {MysqlTime{year: 1992, month: 5, day: 3, hour: 0, minute: 0, second: 0, microsecond: 0}, true}, + {MysqlTime{year: 2024, month: 10, day: 1, hour: 0, minute: 0, second: 0, microsecond: 0}, true}, + {MysqlTime{year: 2016, month: 6, day: 29, hour: 0, minute: 0, second: 0, microsecond: 0}, true}, + {MysqlTime{year: 2015, month: 6, day: 29, hour: 0, minute: 0, second: 0, microsecond: 0}, false}, + {MysqlTime{year: 2014, month: 9, day: 31, hour: 0, minute: 0, second: 0, microsecond: 0}, false}, + {MysqlTime{year: 2001, month: 12, day: 7, hour: 0, minute: 0, second: 0, microsecond: 0}, false}, + {MysqlTime{year: 1989, month: 7, day: 6, hour: 0, minute: 0, second: 0, microsecond: 0}, false}, } for _, tt := range tests { @@ -277,9 +278,9 @@ func (s *testMyTimeSuite) TestWeekday(c *C) { Input MysqlTime Expect string }{ - {MysqlTime{2019, 01, 01, 0, 0, 0, 0}, "Tuesday"}, - {MysqlTime{2019, 02, 31, 0, 0, 0, 0}, "Sunday"}, - {MysqlTime{2019, 04, 31, 0, 0, 0, 0}, "Wednesday"}, + {MysqlTime{year: 2019, month: 01, day: 01, hour: 0, minute: 0, second: 0, microsecond: 0}, "Tuesday"}, + {MysqlTime{year: 2019, month: 02, day: 31, hour: 0, minute: 0, second: 0, microsecond: 0}, "Sunday"}, + {MysqlTime{year: 2019, month: 04, day: 31, hour: 0, minute: 0, second: 0, microsecond: 0}, "Wednesday"}, } for _, tt := range tests { @@ -287,3 +288,8 @@ func (s *testMyTimeSuite) TestWeekday(c *C) { c.Check(weekday.String(), Equals, tt.Expect) } } + +func (s *testMyTimeSuite) TestTimeStructSize(c *C) { + c.Assert(unsafe.Sizeof(MysqlTime{}), Equals, uintptr(0x10)) + c.Assert(unsafe.Sizeof(Time{}), Equals, uintptr(0x14)) +} diff --git a/types/time.go b/types/time.go index 73889fccefc46..e14149ba14eff 100644 --- a/types/time.go +++ b/types/time.go @@ -203,13 +203,13 @@ func FromGoTime(t gotime.Time) MysqlTime { // FromDate makes a internal time representation from the given date. func FromDate(year int, month int, day int, hour int, minute int, second int, microsecond int) MysqlTime { return MysqlTime{ - uint16(year), - uint8(month), - uint8(day), - hour, - uint8(minute), - uint8(second), - uint32(microsecond), + year: uint16(year), + month: uint8(month), + day: uint8(day), + hour: uint32(hour), + minute: uint8(minute), + second: uint8(second), + microsecond: uint32(microsecond), } } @@ -225,11 +225,11 @@ type Time struct { Type uint8 // Fsp is short for Fractional Seconds Precision. // See http://dev.mysql.com/doc/refman/5.7/en/fractional-seconds.html - Fsp int + Fsp int8 } // MaxMySQLTime returns Time with maximum mysql time type. -func MaxMySQLTime(fsp int) Time { +func MaxMySQLTime(fsp int8) Time { return Time{Time: FromDate(0, 0, 0, TimeMaxHour, TimeMaxMinute, TimeMaxSecond, 0), Type: mysql.TypeDuration, Fsp: fsp} } @@ -309,7 +309,7 @@ func (t Time) ToNumber() *MyDecimal { if t.Fsp > 0 { s1 := fmt.Sprintf("%s.%06d", s, t.Time.Microsecond()) - s = s1[:len(s)+t.Fsp+1] + s = s1[:len(s)+int(t.Fsp)+1] } // We skip checking error here because time formatted string can be parsed certainly. @@ -392,19 +392,19 @@ func (t Time) CompareString(sc *stmtctx.StatementContext, str string) (int, erro } // roundTime rounds the time value according to digits count specified by fsp. -func roundTime(t gotime.Time, fsp int) gotime.Time { - d := gotime.Duration(math.Pow10(9 - fsp)) +func roundTime(t gotime.Time, fsp int8) gotime.Time { + d := gotime.Duration(math.Pow10(9 - int(fsp))) return t.Round(d) } // RoundFrac rounds the fraction part of a time-type value according to `fsp`. -func (t Time) RoundFrac(sc *stmtctx.StatementContext, fsp int) (Time, error) { +func (t Time) RoundFrac(sc *stmtctx.StatementContext, fsp int8) (Time, error) { if t.Type == mysql.TypeDate || t.IsZero() { // date type has no fsp return t, nil } - fsp, err := CheckFsp(fsp) + fsp, err := CheckFsp(int(fsp)) if err != nil { return t, errors.Trace(err) } @@ -438,8 +438,9 @@ func (t Time) RoundFrac(sc *stmtctx.StatementContext, fsp int) (Time, error) { } // GetFsp gets the fsp of a string. -func GetFsp(s string) (fsp int) { +func GetFsp(s string) int8 { index := GetFracIndex(s) + var fsp int if index < 0 { fsp = 0 } else { @@ -451,7 +452,7 @@ func GetFsp(s string) (fsp int) { } else if fsp > 6 { fsp = 6 } - return + return int8(fsp) } // GetFracIndex finds the last '.' for get fracStr, index = -1 means fracStr not found. @@ -474,22 +475,22 @@ func GetFracIndex(s string) (index int) { // We will use the “round half up” rule, e.g, >= 0.5 -> 1, < 0.5 -> 0, // so 2011:11:11 10:10:10.888888 round 0 -> 2011:11:11 10:10:11 // and 2011:11:11 10:10:10.111111 round 0 -> 2011:11:11 10:10:10 -func RoundFrac(t gotime.Time, fsp int) (gotime.Time, error) { - _, err := CheckFsp(fsp) +func RoundFrac(t gotime.Time, fsp int8) (gotime.Time, error) { + _, err := CheckFsp(int(fsp)) if err != nil { return t, errors.Trace(err) } - return t.Round(gotime.Duration(math.Pow10(9-fsp)) * gotime.Nanosecond), nil + return t.Round(gotime.Duration(math.Pow10(9-int(fsp))) * gotime.Nanosecond), nil } // TruncateFrac truncates fractional seconds precision with new fsp and returns a new one. // 2011:11:11 10:10:10.888888 round 0 -> 2011:11:11 10:10:10 // 2011:11:11 10:10:10.111111 round 0 -> 2011:11:11 10:10:10 -func TruncateFrac(t gotime.Time, fsp int) (gotime.Time, error) { - if _, err := CheckFsp(fsp); err != nil { +func TruncateFrac(t gotime.Time, fsp int8) (gotime.Time, error) { + if _, err := CheckFsp(int(fsp)); err != nil { return t, err } - return t.Truncate(gotime.Duration(math.Pow10(9-fsp)) * gotime.Nanosecond), nil + return t.Truncate(gotime.Duration(math.Pow10(9-int(fsp))) * gotime.Nanosecond), nil } // ToPackedUint encodes Time to a packed uint64 value. @@ -683,7 +684,7 @@ func splitDateTime(format string) (seps []string, fracStr string) { } // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-literals.html. -func parseDatetime(sc *stmtctx.StatementContext, str string, fsp int, isFloat bool) (Time, error) { +func parseDatetime(sc *stmtctx.StatementContext, str string, fsp int8, isFloat bool) (Time, error) { // Try to split str with delimiter. // TODO: only punctuation can be the delimiter for date parts or time parts. // But only space and T can be the delimiter between the date and time part. @@ -896,7 +897,7 @@ type Duration struct { gotime.Duration // Fsp is short for Fractional Seconds Precision. // See http://dev.mysql.com/doc/refman/5.7/en/fractional-seconds.html - Fsp int + Fsp int8 } //Add adds d to d, returns a duration value. @@ -1002,8 +1003,8 @@ func (d Duration) ConvertToTime(sc *stmtctx.StatementContext, tp uint8) (Time, e // We will use the “round half up” rule, e.g, >= 0.5 -> 1, < 0.5 -> 0, // so 10:10:10.999999 round 0 -> 10:10:11 // and 10:10:10.000000 round 0 -> 10:10:10 -func (d Duration) RoundFrac(fsp int) (Duration, error) { - fsp, err := CheckFsp(fsp) +func (d Duration) RoundFrac(fsp int8) (Duration, error) { + fsp, err := CheckFsp(int(fsp)) if err != nil { return d, errors.Trace(err) } @@ -1013,7 +1014,7 @@ func (d Duration) RoundFrac(fsp int) (Duration, error) { } n := gotime.Date(0, 0, 0, 0, 0, 0, 0, gotime.Local) - nd := n.Add(d.Duration).Round(gotime.Duration(math.Pow10(9-fsp)) * gotime.Nanosecond).Sub(n) + nd := n.Add(d.Duration).Round(gotime.Duration(math.Pow10(9-int(fsp))) * gotime.Nanosecond).Sub(n) return Duration{Duration: nd, Fsp: fsp}, nil } @@ -1072,7 +1073,7 @@ func (d Duration) MicroSecond() int { // ParseDuration parses the time form a formatted string with a fractional seconds part, // returns the duration type Time value. // See http://dev.mysql.com/doc/refman/5.7/en/fractional-seconds.html -func ParseDuration(sc *stmtctx.StatementContext, str string, fsp int) (Duration, error) { +func ParseDuration(sc *stmtctx.StatementContext, str string, fsp int8) (Duration, error) { var ( day, hour, minute, second int err error @@ -1081,7 +1082,7 @@ func ParseDuration(sc *stmtctx.StatementContext, str string, fsp int) (Duration, origStr = str ) - fsp, err = CheckFsp(fsp) + fsp, err = CheckFsp(int(fsp)) if err != nil { return ZeroDuration, errors.Trace(err) } @@ -1336,17 +1337,17 @@ func parseDateTimeFromNum(sc *stmtctx.StatementContext, num int64) (Time, error) // The valid datetime range is from '1000-01-01 00:00:00.000000' to '9999-12-31 23:59:59.999999'. // The valid timestamp range is from '1970-01-01 00:00:01.000000' to '2038-01-19 03:14:07.999999'. // The valid date range is from '1000-01-01' to '9999-12-31' -func ParseTime(sc *stmtctx.StatementContext, str string, tp byte, fsp int) (Time, error) { +func ParseTime(sc *stmtctx.StatementContext, str string, tp byte, fsp int8) (Time, error) { return parseTime(sc, str, tp, fsp, false) } // ParseTimeFromFloatString is similar to ParseTime, except that it's used to parse a float converted string. -func ParseTimeFromFloatString(sc *stmtctx.StatementContext, str string, tp byte, fsp int) (Time, error) { +func ParseTimeFromFloatString(sc *stmtctx.StatementContext, str string, tp byte, fsp int8) (Time, error) { return parseTime(sc, str, tp, fsp, true) } -func parseTime(sc *stmtctx.StatementContext, str string, tp byte, fsp int, isFloat bool) (Time, error) { - fsp, err := CheckFsp(fsp) +func parseTime(sc *stmtctx.StatementContext, str string, tp byte, fsp int8, isFloat bool) (Time, error) { + fsp, err := CheckFsp(int(fsp)) if err != nil { return Time{Time: ZeroTime, Type: tp}, errors.Trace(err) } @@ -1381,8 +1382,8 @@ func ParseDate(sc *stmtctx.StatementContext, str string) (Time, error) { // ParseTimeFromNum parses a formatted int64, // returns the value which type is tp. -func ParseTimeFromNum(sc *stmtctx.StatementContext, num int64, tp byte, fsp int) (Time, error) { - fsp, err := CheckFsp(fsp) +func ParseTimeFromNum(sc *stmtctx.StatementContext, num int64, tp byte, fsp int8) (Time, error) { + fsp, err := CheckFsp(int(fsp)) if err != nil { return Time{Time: ZeroTime, Type: tp}, errors.Trace(err) } @@ -1764,7 +1765,7 @@ func parseTimeValue(format string, index, cnt int) (int64, int64, int64, int64, if err != nil { return 0, 0, 0, 0, ErrIncorrectDatetimeValue.GenWithStackByArgs(originalFmt) } - microseconds, err := strconv.ParseInt(alignFrac(fields[MicrosecondIndex], MaxFsp), 10, 64) + microseconds, err := strconv.ParseInt(alignFrac(fields[MicrosecondIndex], int(MaxFsp)), 10, 64) if err != nil { return 0, 0, 0, 0, ErrIncorrectDatetimeValue.GenWithStackByArgs(originalFmt) } @@ -2353,7 +2354,7 @@ func hour24TwoDigits(t *MysqlTime, input string, ctx map[string]int) (string, bo if !succ || v >= 24 { return input, false } - t.hour = v + t.hour = uint32(v) return input[2:], true } @@ -2406,9 +2407,9 @@ func time12Hour(t *MysqlTime, input string, ctx map[string]int) (string, bool) { remain := skipWhiteSpace(input[8:]) switch { case strings.HasPrefix(remain, "AM"): - t.hour = hour + t.hour = uint32(hour) case strings.HasPrefix(remain, "PM"): - t.hour = hour + 12 + t.hour = uint32(hour + 12) default: return input, false } @@ -2441,7 +2442,7 @@ func time24Hour(t *MysqlTime, input string, ctx map[string]int) (string, bool) { return input, false } - t.hour = hour + t.hour = uint32(hour) t.minute = uint8(minute) t.second = uint8(second) return input[8:], true @@ -2522,7 +2523,7 @@ func hour24Numeric(t *MysqlTime, input string, ctx map[string]int) (string, bool if !ok || v > 23 { return input, false } - t.hour = v + t.hour = uint32(v) ctx["%H"] = v return input[length:], true } @@ -2536,7 +2537,7 @@ func hour12Numeric(t *MysqlTime, input string, ctx map[string]int) (string, bool if !ok || v > 12 || v == 0 { return input, false } - t.hour = v + t.hour = uint32(v) return input[length:], true } diff --git a/types/time_test.go b/types/time_test.go index eb3283054dd8e..f1e5ebfc4e5bb 100644 --- a/types/time_test.go +++ b/types/time_test.go @@ -80,7 +80,7 @@ func (s *testTimeSuite) TestDateTime(c *C) { fspTbl := []struct { Input string - Fsp int + Fsp int8 Expect string }{ {"20170118.123", 6, "2017-01-18 12:03:00.000000"}, @@ -279,9 +279,9 @@ func (s *testTimeSuite) TestDurationAdd(c *C) { defer testleak.AfterTest(c)() table := []struct { Input string - Fsp int + Fsp int8 InputAdd string - FspAdd int + FspAdd int8 Expect string }{ {"00:00:00.1", 1, "00:00:00.1", 1, "00:00:00.2"}, @@ -318,9 +318,9 @@ func (s *testTimeSuite) TestDurationSub(c *C) { defer testleak.AfterTest(c)() table := []struct { Input string - Fsp int + Fsp int8 InputAdd string - FspAdd int + FspAdd int8 Expect string }{ {"00:00:00.1", 1, "00:00:00.1", 1, "00:00:00.0"}, @@ -343,7 +343,7 @@ func (s *testTimeSuite) TestTimeFsp(c *C) { defer testleak.AfterTest(c)() table := []struct { Input string - Fsp int + Fsp int8 Expect string }{ {"00:00:00.1", 0, "00:00:00"}, @@ -366,7 +366,7 @@ func (s *testTimeSuite) TestTimeFsp(c *C) { errTable := []struct { Input string - Fsp int + Fsp int8 }{ {"00:00:00.1", -2}, {"00:00:00.1", 7}, @@ -605,7 +605,7 @@ func (s *testTimeSuite) TestToNumber(c *C) { defer testleak.AfterTest(c)() tblDateTime := []struct { Input string - Fsp int + Fsp int8 Expect string }{ {"12-12-31 11:30:45", 0, "20121231113045"}, @@ -628,7 +628,7 @@ func (s *testTimeSuite) TestToNumber(c *C) { // Fix issue #1046 tblDate := []struct { Input string - Fsp int + Fsp int8 Expect string }{ {"12-12-31 11:30:45", 0, "20121231"}, @@ -650,7 +650,7 @@ func (s *testTimeSuite) TestToNumber(c *C) { tblDuration := []struct { Input string - Fsp int + Fsp int8 Expect string }{ {"11:30:45", 0, "113045"}, @@ -677,7 +677,7 @@ func (s *testTimeSuite) TestParseFrac(c *C) { defer testleak.AfterTest(c)() tbl := []struct { S string - Fsp int + Fsp int8 Ret int Overflow bool }{ @@ -717,7 +717,7 @@ func (s *testTimeSuite) TestRoundFrac(c *C) { defer testleak.AfterTest(c)() tbl := []struct { Input string - Fsp int + Fsp int8 Except string }{ {"2012-12-31 11:30:45.123456", 4, "2012-12-31 11:30:45.1235"}, @@ -743,7 +743,7 @@ func (s *testTimeSuite) TestRoundFrac(c *C) { tbl = []struct { Input string - Fsp int + Fsp int8 Except string }{ {"11:30:45.123456", 4, "11:30:45.1235"}, @@ -764,7 +764,7 @@ func (s *testTimeSuite) TestRoundFrac(c *C) { cols := []struct { input time.Time - fsp int + fsp int8 output time.Time }{ {time.Date(2011, 11, 11, 10, 10, 10, 888888, time.UTC), 0, time.Date(2011, 11, 11, 10, 10, 10, 11, time.UTC)}, @@ -782,7 +782,7 @@ func (s *testTimeSuite) TestConvert(c *C) { defer testleak.AfterTest(c)() tbl := []struct { Input string - Fsp int + Fsp int8 Except string }{ {"2012-12-31 11:30:45.123456", 4, "11:30:45.1235"}, @@ -804,7 +804,7 @@ func (s *testTimeSuite) TestConvert(c *C) { tblDuration := []struct { Input string - Fsp int + Fsp int8 }{ {"11:30:45.123456", 4}, {"11:30:45.123456", 6}, @@ -1247,7 +1247,7 @@ func (s *testTimeSuite) TestCurrentTime(c *C) { res := types.CurrentTime(mysql.TypeTimestamp) c.Assert(res.Time, NotNil) c.Assert(res.Type, Equals, mysql.TypeTimestamp) - c.Assert(res.Fsp, Equals, 0) + c.Assert(res.Fsp, Equals, int8(0)) } func (s *testTimeSuite) TestInvalidZero(c *C) { @@ -1265,16 +1265,16 @@ func (s *testTimeSuite) TestInvalidZero(c *C) { func (s *testTimeSuite) TestGetFsp(c *C) { res := types.GetFsp("2019:04:12 14:00:00.123456") - c.Assert(res, Equals, 6) + c.Assert(res, Equals, int8(6)) res = types.GetFsp("2019:04:12 14:00:00.1234567890") - c.Assert(res, Equals, 6) + c.Assert(res, Equals, int8(6)) res = types.GetFsp("2019:04:12 14:00:00.1") - c.Assert(res, Equals, 1) + c.Assert(res, Equals, int8(1)) res = types.GetFsp("2019:04:12 14:00:00") - c.Assert(res, Equals, 0) + c.Assert(res, Equals, int8(0)) } func (s *testTimeSuite) TestExtractDatetimeNum(c *C) { @@ -1563,7 +1563,7 @@ func (s *testTimeSuite) TestTimeOverflow(c *C) { func (s *testTimeSuite) TestTruncateFrac(c *C) { cols := []struct { input time.Time - fsp int + fsp int8 output time.Time }{ {time.Date(2011, 11, 11, 10, 10, 10, 888888, time.UTC), 0, time.Date(2011, 11, 11, 10, 10, 10, 11, time.UTC)}, diff --git a/util/chunk/chunk.go b/util/chunk/chunk.go index 453b57c71b85a..1449e8d83db23 100644 --- a/util/chunk/chunk.go +++ b/util/chunk/chunk.go @@ -651,6 +651,6 @@ func readTime(buf []byte) types.Time { return types.Time{ Time: types.FromDate(year, month, day, hour, minute, second, microseconds), Type: tp, - Fsp: fsp, + Fsp: int8(fsp), } } diff --git a/util/chunk/column.go b/util/chunk/column.go index 9609e2e6eede1..e6e5e0413a050 100644 --- a/util/chunk/column.go +++ b/util/chunk/column.go @@ -502,7 +502,7 @@ func (c *Column) GetTime(rowID int) types.Time { // GetDuration returns the Duration in the specific row. func (c *Column) GetDuration(rowID int, fillFsp int) types.Duration { dur := *(*int64)(unsafe.Pointer(&c.data[rowID*8])) - return types.Duration{Duration: time.Duration(dur), Fsp: fillFsp} + return types.Duration{Duration: time.Duration(dur), Fsp: int8(fillFsp)} } func (c *Column) getNameValue(rowID int) (string, uint64) { diff --git a/util/codec/codec.go b/util/codec/codec.go index c9d3ffae0f135..8d6d733feb48d 100644 --- a/util/codec/codec.go +++ b/util/codec/codec.go @@ -729,7 +729,7 @@ func (decoder *Decoder) DecodeOne(b []byte, colIdx int, ft *types.FieldType) (re if err != nil { return nil, errors.Trace(err) } - v := types.Duration{Duration: time.Duration(r), Fsp: ft.Decimal} + v := types.Duration{Duration: time.Duration(r), Fsp: int8(ft.Decimal)} chk.AppendDuration(colIdx, v) case jsonFlag: var size int @@ -753,7 +753,7 @@ func (decoder *Decoder) DecodeOne(b []byte, colIdx int, ft *types.FieldType) (re func appendIntToChunk(val int64, chk *chunk.Chunk, colIdx int, ft *types.FieldType) { switch ft.Tp { case mysql.TypeDuration: - v := types.Duration{Duration: time.Duration(val), Fsp: ft.Decimal} + v := types.Duration{Duration: time.Duration(val), Fsp: int8(ft.Decimal)} chk.AppendDuration(colIdx, v) default: chk.AppendInt64(colIdx, val) @@ -765,7 +765,7 @@ func appendUintToChunk(val uint64, chk *chunk.Chunk, colIdx int, ft *types.Field case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeTimestamp: var t types.Time t.Type = ft.Tp - t.Fsp = ft.Decimal + t.Fsp = int8(ft.Decimal) var err error err = t.FromPackedUint(val) if err != nil { diff --git a/util/rowcodec/decoder.go b/util/rowcodec/decoder.go index f452b0deb2f63..46b091d0f49bb 100644 --- a/util/rowcodec/decoder.go +++ b/util/rowcodec/decoder.go @@ -168,7 +168,7 @@ func (decoder *Decoder) decodeColData(colIdx int, colData []byte, chk *chunk.Chu case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeTimestamp: var t types.Time t.Type = ft.Tp - t.Fsp = ft.Decimal + t.Fsp = int8(ft.Decimal) err := t.FromPackedUint(decodeUint(colData)) if err != nil { return err @@ -183,7 +183,7 @@ func (decoder *Decoder) decodeColData(colIdx int, colData []byte, chk *chunk.Chu case mysql.TypeDuration: var dur types.Duration dur.Duration = time.Duration(decodeInt(colData)) - dur.Fsp = ft.Decimal + dur.Fsp = int8(ft.Decimal) chk.AppendDuration(colIdx, dur) case mysql.TypeEnum: // ignore error deliberately, to read empty enum value.