Skip to content

Commit

Permalink
type: encapsulate Time API (pingcap#14288)
Browse files Browse the repository at this point in the history
  • Loading branch information
TennyZhuang authored and sre-bot committed Dec 31, 2019
1 parent 81a6eb0 commit d006443
Show file tree
Hide file tree
Showing 81 changed files with 783 additions and 867 deletions.
2 changes: 1 addition & 1 deletion bindinfo/cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -67,7 +67,7 @@ func (b *Binding) isSame(rb *Binding) bool {

// SinceUpdateTime returns the duration since last update time. Export for test.
func (b *Binding) SinceUpdateTime() (time.Duration, error) {
updateTime, err := b.UpdateTime.Time.GoTime(time.Local)
updateTime, err := b.UpdateTime.GoTime(time.Local)
if err != nil {
return 0, err
}
Expand Down
12 changes: 2 additions & 10 deletions bindinfo/handle.go
Original file line number Diff line number Diff line change
Expand Up @@ -231,11 +231,7 @@ func (h *BindHandle) AddBindRecord(sctx sessionctx.Context, is infoschema.InfoSc
return err1
}
for i := range record.Bindings {
record.Bindings[i].CreateTime = types.Time{
Time: types.FromGoTime(oracle.GetTimeFromTS(txn.StartTS())),
Type: mysql.TypeTimestamp,
Fsp: 3,
}
record.Bindings[i].CreateTime = types.NewTime(types.FromGoTime(oracle.GetTimeFromTS(txn.StartTS())), mysql.TypeTimestamp, 3)
record.Bindings[i].UpdateTime = record.Bindings[0].CreateTime

// insert the BindRecord to the storage.
Expand Down Expand Up @@ -284,11 +280,7 @@ func (h *BindHandle) DropBindRecord(originalSQL, db string, binding *Binding) (e
return err1
}

updateTs := types.Time{
Time: types.FromGoTime(oracle.GetTimeFromTS(txn.StartTS())),
Type: mysql.TypeTimestamp,
Fsp: 3,
}
updateTs := types.NewTime(types.FromGoTime(oracle.GetTimeFromTS(txn.StartTS())), mysql.TypeTimestamp, 3)

bindSQL := ""
if binding != nil {
Expand Down
6 changes: 1 addition & 5 deletions bindinfo/session_handle.go
Original file line number Diff line number Diff line change
Expand Up @@ -50,11 +50,7 @@ func (h *SessionHandle) appendBindRecord(hash string, meta *BindRecord) {
// AddBindRecord new a BindRecord with BindMeta, add it to the cache.
func (h *SessionHandle) AddBindRecord(sctx sessionctx.Context, is infoschema.InfoSchema, record *BindRecord) error {
for i := range record.Bindings {
record.Bindings[i].CreateTime = types.Time{
Time: types.FromGoTime(time.Now()),
Type: mysql.TypeTimestamp,
Fsp: 3,
}
record.Bindings[i].CreateTime = types.NewTime(types.FromGoTime(time.Now()), mysql.TypeTimestamp, 3)
record.Bindings[i].UpdateTime = record.Bindings[i].CreateTime
}

Expand Down
2 changes: 1 addition & 1 deletion cmd/importer/stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -136,7 +136,7 @@ func (h *histogram) randDate(unit string, mysqlFmt string, dateFmt string) strin
return str
}
delta := randInt(0, int(diff)-1)
l, err := lower.Time.GoTime(time.Local)
l, err := lower.GoTime(time.Local)
if err != nil {
log.Fatal(err.Error())
}
Expand Down
4 changes: 2 additions & 2 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,7 @@ import (
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/table/tables"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/types/parser_driver"
driver "github.com/pingcap/tidb/types/parser_driver"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/domainutil"
Expand Down Expand Up @@ -451,7 +451,7 @@ func checkColumnDefaultValue(ctx sessionctx.Context, col *table.Column, value in
if err != nil {
return hasDefaultValue, value, errors.Trace(err)
}
if timeValue.GetMysqlTime().Time == types.ZeroTime {
if timeValue.GetMysqlTime().CoreTime() == types.ZeroCoreTime {
return hasDefaultValue, value, types.ErrInvalidDefault.GenWithStackByArgs(col.Name.O)
}
}
Expand Down
2 changes: 1 addition & 1 deletion ddl/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -775,7 +775,7 @@ func (w *addIndexWorker) getIndexRecord(handle int64, recordKey []byte, rawRecor

if idxColumnVal.Kind() == types.KindMysqlTime {
t := idxColumnVal.GetMysqlTime()
if t.Type == mysql.TypeTimestamp && sysZone != time.UTC {
if t.Type() == mysql.TypeTimestamp && sysZone != time.UTC {
err := t.ConvertTimeZone(sysZone, time.UTC)
if err != nil {
return nil, errors.Trace(err)
Expand Down
18 changes: 9 additions & 9 deletions executor/aggfuncs/func_count.go
Original file line number Diff line number Diff line change
Expand Up @@ -359,15 +359,15 @@ func appendDecimal(encodedBytes []byte, val *types.MyDecimal) ([]byte, error) {
}

func writeTime(buf []byte, t types.Time) {
binary.BigEndian.PutUint16(buf, uint16(t.Time.Year()))
buf[2] = uint8(t.Time.Month())
buf[3] = uint8(t.Time.Day())
buf[4] = uint8(t.Time.Hour())
buf[5] = uint8(t.Time.Minute())
buf[6] = uint8(t.Time.Second())
binary.BigEndian.PutUint32(buf[8:], uint32(t.Time.Microsecond()))
buf[12] = t.Type
buf[13] = uint8(t.Fsp)
binary.BigEndian.PutUint16(buf, uint16(t.Year()))
buf[2] = uint8(t.Month())
buf[3] = uint8(t.Day())
buf[4] = uint8(t.Hour())
buf[5] = uint8(t.Minute())
buf[6] = uint8(t.Second())
binary.BigEndian.PutUint32(buf[8:], uint32(t.Microsecond()))
buf[12] = t.Type()
buf[13] = uint8(t.Fsp())
}

func appendTime(encodedBytes, buf []byte, val types.Time) []byte {
Expand Down
6 changes: 1 addition & 5 deletions executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -737,11 +737,7 @@ func (e *ShowSlowExec) Next(ctx context.Context, req *chunk.Chunk) error {
for e.cursor < len(e.result) && req.NumRows() < e.maxChunkSize {
slow := e.result[e.cursor]
req.AppendString(0, slow.SQL)
req.AppendTime(1, types.Time{
Time: types.FromGoTime(slow.Start),
Type: mysql.TypeTimestamp,
Fsp: types.MaxFsp,
})
req.AppendTime(1, types.NewTime(types.FromGoTime(slow.Start), mysql.TypeTimestamp, types.MaxFsp))
req.AppendDuration(2, types.Duration{Duration: slow.Duration, Fsp: types.MaxFsp})
req.AppendString(3, slow.Detail.String())
if slow.Succ {
Expand Down
10 changes: 5 additions & 5 deletions executor/metric_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -136,11 +136,11 @@ func (e *MetricRetriever) genRows(value pmodel.Value, r promQLQueryRange, quanti
func (e *MetricRetriever) genRecord(metric pmodel.Metric, pair pmodel.SamplePair, r promQLQueryRange, quantile float64) []types.Datum {
record := make([]types.Datum, 0, 2+len(e.tblDef.Labels)+1)
// Record order should keep same with genColumnInfos.
record = append(record, types.NewTimeDatum(types.Time{
Time: types.FromGoTime(time.Unix(int64(pair.Timestamp/1000), int64(pair.Timestamp%1000)*1e6)),
Type: mysql.TypeDatetime,
Fsp: types.MaxFsp,
}))
record = append(record, types.NewTimeDatum(types.NewTime(
types.FromGoTime(time.Unix(int64(pair.Timestamp/1000), int64(pair.Timestamp%1000)*1e6)),
mysql.TypeDatetime,
types.MaxFsp,
)))
record = append(record, types.NewFloat64Datum(float64(pair.Value)))
for _, label := range e.tblDef.Labels {
v := ""
Expand Down
2 changes: 1 addition & 1 deletion executor/show_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -110,7 +110,7 @@ func (e *ShowExec) histogramToRow(dbName, tblName, partitionName, colName string

func (e *ShowExec) versionToTime(version uint64) types.Time {
t := time.Unix(0, oracle.ExtractPhysical(version)*int64(time.Millisecond))
return types.Time{Time: types.FromGoTime(t), Type: mysql.TypeDatetime}
return types.NewTime(types.FromGoTime(t), mysql.TypeDatetime, 0)
}

func (e *ShowExec) fetchShowStatsBuckets() error {
Expand Down
15 changes: 3 additions & 12 deletions executor/split_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -228,14 +228,8 @@ func (s *testSplitIndex) TestSplitIndex(c *C) {
// .
// .
// region10: [2019-01-01 00:00:00 ~ +inf)
lowerTime := types.Time{
Time: types.FromDate(2010, 1, 1, 0, 0, 0, 0),
Type: mysql.TypeTimestamp,
}
upperTime := types.Time{
Time: types.FromDate(2020, 1, 1, 0, 0, 0, 0),
Type: mysql.TypeTimestamp,
}
lowerTime := types.NewTime(types.FromDate(2010, 1, 1, 0, 0, 0, 0), mysql.TypeTimestamp, types.DefaultFsp)
upperTime := types.NewTime(types.FromDate(2020, 1, 1, 0, 0, 0, 0), mysql.TypeTimestamp, types.DefaultFsp)
e.lower = []types.Datum{types.NewDatum(lowerTime)}
e.upper = []types.Datum{types.NewDatum(upperTime)}
e.num = 10
Expand Down Expand Up @@ -268,10 +262,7 @@ func (s *testSplitIndex) TestSplitIndex(c *C) {
}

for _, ca := range cases3 {
value := types.Time{
Time: ca.value,
Type: mysql.TypeTimestamp,
}
value := types.NewTime(ca.value, mysql.TypeTimestamp, types.DefaultFsp)
// test for min int64 handle
idxValue, _, err := index.GenIndexKey(ctx.GetSessionVars().StmtCtx, []types.Datum{types.NewDatum(value)}, math.MinInt64, nil)
c.Assert(err, IsNil)
Expand Down
4 changes: 2 additions & 2 deletions executor/trace.go
Original file line number Diff line number Diff line change
Expand Up @@ -221,7 +221,7 @@ func generateLogResult(allSpans []basictracer.RawSpan, chk *chunk.Chunk) {
for rIdx := range allSpans {
span := &allSpans[rIdx]

chk.AppendTime(0, types.Time{Time: types.FromGoTime(span.Start), Type: mysql.TypeTimestamp, Fsp: 6})
chk.AppendTime(0, types.NewTime(types.FromGoTime(span.Start), mysql.TypeTimestamp, 6))
chk.AppendString(1, "--- start span "+span.Operation+" ----")
chk.AppendString(2, "")
chk.AppendString(3, span.Operation)
Expand All @@ -233,7 +233,7 @@ func generateLogResult(allSpans []basictracer.RawSpan, chk *chunk.Chunk) {
for _, l := range span.Logs {
for _, field := range l.Fields {
if field.Key() == logutil.TraceEventKey {
chk.AppendTime(0, types.Time{Time: types.FromGoTime(l.Timestamp), Type: mysql.TypeTimestamp, Fsp: 6})
chk.AppendTime(0, types.NewTime(types.FromGoTime(l.Timestamp), mysql.TypeTimestamp, 6))
chk.AppendString(1, field.Value().(string))
chk.AppendString(2, tags)
chk.AppendString(3, span.Operation)
Expand Down
10 changes: 5 additions & 5 deletions expression/bench_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -245,7 +245,7 @@ func (g *defaultGener) gen() interface{} {
return d
case types.ETDatetime, types.ETTimestamp:
gt := getRandomTime()
t := types.Time{Time: gt, Type: convertETType(g.eType)}
t := types.NewTime(gt, convertETType(g.eType), 0)
return t
case types.ETDuration:
d := types.Duration{
Expand Down Expand Up @@ -362,7 +362,7 @@ func (g *realStringGener) gen() interface{} {
type jsonTimeGener struct{}

func (g *jsonTimeGener) gen() interface{} {
tm := types.Time{Time: getRandomTime(), Type: mysql.TypeDatetime, Fsp: types.DefaultFsp}
tm := types.NewTime(getRandomTime(), mysql.TypeDatetime, types.DefaultFsp)
return json.CreateBinary(tm.String())
}

Expand Down Expand Up @@ -611,7 +611,7 @@ func (g *dateTimeGener) gen() interface{} {
} else {
gt = types.FromDate(g.Year, g.Month, g.Day, rand.Intn(12), rand.Intn(60), rand.Intn(60), 0)
}
t := types.Time{Time: gt, Type: mysql.TypeDatetime}
t := types.NewTime(gt, mysql.TypeDatetime, types.DefaultFsp)
return t
}

Expand Down Expand Up @@ -1136,7 +1136,7 @@ func testVectorizedBuiltinFunc(c *C, vecExprCases vecExprBenchCases) {
ctx.GetSessionVars().PreparedParams = []types.Datum{
types.NewIntDatum(1),
types.NewDecimalDatum(types.NewDecFromStringForTest("20170118123950.123")),
types.NewTimeDatum(types.Time{Time: types.FromGoTime(testTime), Fsp: 6, Type: mysql.TypeTimestamp}),
types.NewTimeDatum(types.NewTime(types.FromGoTime(testTime), mysql.TypeTimestamp, 6)),
types.NewDurationDatum(types.ZeroDuration),
types.NewStringDatum("{}"),
types.NewBinaryLiteralDatum(types.BinaryLiteral([]byte{1})),
Expand Down Expand Up @@ -1356,7 +1356,7 @@ func benchmarkVectorizedBuiltinFunc(b *testing.B, vecExprCases vecExprBenchCases
ctx.GetSessionVars().PreparedParams = []types.Datum{
types.NewIntDatum(1),
types.NewDecimalDatum(types.NewDecFromStringForTest("20170118123950.123")),
types.NewTimeDatum(types.Time{Time: types.FromGoTime(testTime), Fsp: 6, Type: mysql.TypeTimestamp}),
types.NewTimeDatum(types.NewTime(types.FromGoTime(testTime), mysql.TypeTimestamp, 6)),
types.NewDurationDatum(types.ZeroDuration),
types.NewStringDatum("{}"),
types.NewBinaryLiteralDatum(types.BinaryLiteral([]byte{1})),
Expand Down
2 changes: 1 addition & 1 deletion expression/builtin.go
Original file line number Diff line number Diff line change
Expand Up @@ -244,7 +244,7 @@ func (b *baseBuiltinFunc) evalDecimal(row chunk.Row) (*types.MyDecimal, bool, er
}

func (b *baseBuiltinFunc) evalTime(row chunk.Row) (types.Time, bool, error) {
return types.Time{}, false, errors.Errorf("baseBuiltinFunc.evalTime() should never be called, please contact the TiDB team for help")
return types.ZeroTime, false, errors.Errorf("baseBuiltinFunc.evalTime() should never be called, please contact the TiDB team for help")
}

func (b *baseBuiltinFunc) evalDuration(row chunk.Row) (types.Duration, bool, error) {
Expand Down
36 changes: 18 additions & 18 deletions expression/builtin_cast.go
Original file line number Diff line number Diff line change
Expand Up @@ -551,11 +551,11 @@ func (b *builtinCastIntAsTimeSig) evalTime(row chunk.Row) (res types.Time, isNul
}
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)
return types.ZeroTime, true, handleInvalidTimeError(b.ctx, err)
}
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)
res.SetCoreTime(types.FromDate(res.Year(), res.Month(), res.Day(), 0, 0, 0, 0))
}
return res, false, nil
}
Expand Down Expand Up @@ -706,8 +706,8 @@ func (b *builtinCastTimeAsJSONSig) evalJSON(row chunk.Row) (res json.BinaryJSON,
if isNull || err != nil {
return res, isNull, err
}
if val.Type == mysql.TypeDatetime || val.Type == mysql.TypeTimestamp {
val.Fsp = types.MaxFsp
if val.Type() == mysql.TypeDatetime || val.Type() == mysql.TypeTimestamp {
val.SetFsp(types.MaxFsp)
}
return json.CreateBinary(val.String()), false, nil
}
Expand Down Expand Up @@ -830,21 +830,21 @@ func (b *builtinCastRealAsTimeSig) Clone() builtinFunc {
func (b *builtinCastRealAsTimeSig) evalTime(row chunk.Row) (types.Time, bool, error) {
val, isNull, err := b.args[0].EvalReal(b.ctx, row)
if isNull || err != nil {
return types.Time{}, true, err
return types.ZeroTime, true, err
}
// MySQL compatibility: 0 should not be converted to null, see #11203
fv := strconv.FormatFloat(val, 'f', -1, 64)
if fv == "0" {
return types.Time{}, false, nil
return types.ZeroTime, false, nil
}
sc := b.ctx.GetSessionVars().StmtCtx
res, err := types.ParseTime(sc, fv, b.tp.Tp, int8(b.tp.Decimal))
if err != nil {
return types.Time{}, true, handleInvalidTimeError(b.ctx, err)
return types.ZeroTime, true, handleInvalidTimeError(b.ctx, err)
}
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)
res.SetCoreTime(types.FromDate(res.Year(), res.Month(), res.Day(), 0, 0, 0, 0))
}
return res, false, nil
}
Expand Down Expand Up @@ -997,11 +997,11 @@ func (b *builtinCastDecimalAsTimeSig) evalTime(row chunk.Row) (res types.Time, i
sc := b.ctx.GetSessionVars().StmtCtx
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)
return types.ZeroTime, true, handleInvalidTimeError(b.ctx, err)
}
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)
res.SetCoreTime(types.FromDate(res.Year(), res.Month(), res.Day(), 0, 0, 0, 0))
}
return res, false, err
}
Expand Down Expand Up @@ -1213,11 +1213,11 @@ func (b *builtinCastStringAsTimeSig) evalTime(row chunk.Row) (res types.Time, is
sc := b.ctx.GetSessionVars().StmtCtx
res, err = types.ParseTime(sc, val, b.tp.Tp, int8(b.tp.Decimal))
if err != nil {
return types.Time{}, true, handleInvalidTimeError(b.ctx, err)
return types.ZeroTime, true, handleInvalidTimeError(b.ctx, err)
}
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)
res.SetCoreTime(types.FromDate(res.Year(), res.Month(), res.Day(), 0, 0, 0, 0))
}
return res, false, nil
}
Expand Down Expand Up @@ -1267,13 +1267,13 @@ func (b *builtinCastTimeAsTimeSig) evalTime(row chunk.Row) (res types.Time, isNu

sc := b.ctx.GetSessionVars().StmtCtx
if res, err = res.Convert(sc, b.tp.Tp); err != nil {
return types.Time{}, true, handleInvalidTimeError(b.ctx, err)
return types.ZeroTime, true, handleInvalidTimeError(b.ctx, err)
}
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)
res.Type = b.tp.Tp
res.SetCoreTime(types.FromDate(res.Year(), res.Month(), res.Day(), 0, 0, 0, 0))
res.SetType(b.tp.Tp)
}
return res, false, err
}
Expand Down Expand Up @@ -1534,7 +1534,7 @@ func (b *builtinCastDurationAsTimeSig) evalTime(row chunk.Row) (res types.Time,
sc := b.ctx.GetSessionVars().StmtCtx
res, err = val.ConvertToTime(sc, b.tp.Tp)
if err != nil {
return types.Time{}, true, handleInvalidTimeError(b.ctx, err)
return types.ZeroTime, true, handleInvalidTimeError(b.ctx, err)
}
res, err = res.RoundFrac(sc, int8(b.tp.Decimal))
return res, false, err
Expand Down Expand Up @@ -1658,11 +1658,11 @@ func (b *builtinCastJSONAsTimeSig) evalTime(row chunk.Row) (res types.Time, isNu
sc := b.ctx.GetSessionVars().StmtCtx
res, err = types.ParseTime(sc, s, b.tp.Tp, int8(b.tp.Decimal))
if err != nil {
return types.Time{}, true, handleInvalidTimeError(b.ctx, err)
return types.ZeroTime, true, handleInvalidTimeError(b.ctx, err)
}
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)
res.SetCoreTime(types.FromDate(res.Year(), res.Month(), res.Day(), 0, 0, 0, 0))
}
return
}
Expand Down
Loading

0 comments on commit d006443

Please sign in to comment.