diff --git a/expression/builtin_time.go b/expression/builtin_time.go index 7269e6034d..e9d191c472 100644 --- a/expression/builtin_time.go +++ b/expression/builtin_time.go @@ -3558,7 +3558,7 @@ func (b *builtinTimestamp2ArgsSig) evalTime(row types.Row) (types.Time, bool, er if err != nil { return types.Time{}, true, errors.Trace(handleInvalidTimeError(b.ctx, err)) } - tmp, err := tm.Add(duration) + tmp, err := tm.Add(b.ctx.GetSessionVars().StmtCtx, duration) if err != nil { return types.Time{}, true, errors.Trace(err) } @@ -3682,7 +3682,7 @@ func strDatetimeAddDuration(sc *stmtctx.StatementContext, d string, arg1 types.D if err != nil { return "", errors.Trace(err) } - ret, err := arg0.Add(arg1) + ret, err := arg0.Add(sc, arg1) if err != nil { return "", errors.Trace(err) } @@ -3838,7 +3838,7 @@ func (b *builtinAddDatetimeAndDurationSig) evalTime(row types.Row) (types.Time, if isNull || err != nil { return types.ZeroDatetime, isNull, errors.Trace(err) } - result, err := arg0.Add(arg1) + result, err := arg0.Add(b.ctx.GetSessionVars().StmtCtx, arg1) return result, err != nil, errors.Trace(err) } @@ -3870,7 +3870,7 @@ func (b *builtinAddDatetimeAndStringSig) evalTime(row types.Row) (types.Time, bo if err != nil { return types.ZeroDatetime, true, errors.Trace(err) } - result, err := arg0.Add(arg1) + result, err := arg0.Add(b.ctx.GetSessionVars().StmtCtx, arg1) return result, err != nil, errors.Trace(err) } @@ -5146,7 +5146,7 @@ func (b *builtinTimestampAddSig) evalString(row types.Row) (string, bool, error) return "", true, errors.Trace(types.ErrInvalidTimeFormat.GenByArgs(unit)) } r := types.Time{Time: types.FromGoTime(tb), Type: mysql.TypeDatetime, Fsp: fsp} - if err = r.Check(); err != nil { + if err = r.Check(b.ctx.GetSessionVars().StmtCtx); err != nil { return "", true, errors.Trace(handleInvalidTimeError(b.ctx, err)) } return r.String(), false, nil diff --git a/server/server_test.go b/server/server_test.go index 4d15180b87..c90878a3f6 100644 --- a/server/server_test.go +++ b/server/server_test.go @@ -326,6 +326,7 @@ func runTestPreparedString(t *C) { func runTestPreparedTimestamp(t *C) { runTestsOnNewDB(t, nil, "prepared_timestamp", func(dbt *DBTest) { dbt.mustExec("create table test (a timestamp, b time)") + dbt.mustExec("set time_zone='+00:00'") insertStmt := dbt.mustPrepare("insert test values (?, ?)") defer insertStmt.Close() vts := time.Unix(1, 1) diff --git a/statistics/dump.go b/statistics/dump.go index 8e855ae196..f97a9d2e1f 100644 --- a/statistics/dump.go +++ b/statistics/dump.go @@ -14,6 +14,8 @@ package statistics import ( + "time" + "github.com/juju/errors" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/model" @@ -71,8 +73,10 @@ func (h *Handle) DumpStatsToJSON(dbName string, tableInfo *model.TableInfo) (*JS Count: tbl.Count, ModifyCount: tbl.ModifyCount, } + for _, col := range tbl.Columns { - hist, err := col.ConvertTo(new(stmtctx.StatementContext), types.NewFieldType(mysql.TypeBlob)) + sc := &stmtctx.StatementContext{TimeZone: time.UTC} + hist, err := col.ConvertTo(sc, types.NewFieldType(mysql.TypeBlob)) if err != nil { return nil, errors.Trace(err) } @@ -177,6 +181,7 @@ func (h *Handle) LoadStatsFromJSONToTable(tableInfo *model.TableInfo, jsonTbl *J tbl.Indices[idx.ID] = idx } } + for id, jsonCol := range jsonTbl.Columns { for _, colInfo := range tableInfo.Columns { if colInfo.Name.L != id { @@ -184,7 +189,8 @@ func (h *Handle) LoadStatsFromJSONToTable(tableInfo *model.TableInfo, jsonTbl *J } hist := HistogramFromProto(jsonCol.Histogram) count := int64(hist.totalRowCount()) - hist, err := hist.ConvertTo(new(stmtctx.StatementContext), &colInfo.FieldType) + sc := &stmtctx.StatementContext{TimeZone: time.UTC} + hist, err := hist.ConvertTo(sc, &colInfo.FieldType) if err != nil { return nil, errors.Trace(err) } diff --git a/statistics/scalar.go b/statistics/scalar.go index 62e46524e5..e59b672807 100644 --- a/statistics/scalar.go +++ b/statistics/scalar.go @@ -16,7 +16,6 @@ package statistics import ( "encoding/binary" "math" - "time" "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" @@ -67,13 +66,9 @@ func convertDatumToScalar(value *types.Datum, commonPfxLen int) float64 { Fsp: types.DefaultFsp, } case mysql.TypeTimestamp: - minTime = types.Time{ - Time: types.MinTimestamp, - Type: mysql.TypeTimestamp, - Fsp: types.DefaultFsp, - } + minTime = types.MinTimestamp } - sc := &stmtctx.StatementContext{TimeZone: time.UTC} + sc := &stmtctx.StatementContext{TimeZone: types.BoundTimezone} return float64(valueTime.Sub(sc, &minTime).Duration) case types.KindString, types.KindBytes: bytes := value.GetBytes() diff --git a/tablecodec/tablecodec_test.go b/tablecodec/tablecodec_test.go index 706d0d0610..32ad85e59a 100644 --- a/tablecodec/tablecodec_test.go +++ b/tablecodec/tablecodec_test.go @@ -73,7 +73,7 @@ func (s *testTableCodecSuite) TestRowCodec(c *C) { for _, col := range cols { colIDs = append(colIDs, col.id) } - sc := &stmtctx.StatementContext{TimeZone: time.Local} + sc := &stmtctx.StatementContext{TimeZone: time.UTC} bs, err := EncodeRow(sc, row, colIDs, nil, nil) c.Assert(err, IsNil) c.Assert(bs, NotNil) @@ -83,7 +83,7 @@ func (s *testTableCodecSuite) TestRowCodec(c *C) { for _, col := range cols { colMap[col.id] = col.tp } - r, err := DecodeRow(bs, colMap, time.Local) + r, err := DecodeRow(bs, colMap, time.UTC) c.Assert(err, IsNil) c.Assert(r, NotNil) c.Assert(r, HasLen, 3) @@ -98,7 +98,7 @@ func (s *testTableCodecSuite) TestRowCodec(c *C) { // colMap may contains more columns than encoded row. colMap[4] = types.NewFieldType(mysql.TypeFloat) - r, err = DecodeRow(bs, colMap, time.Local) + r, err = DecodeRow(bs, colMap, time.UTC) c.Assert(err, IsNil) c.Assert(r, NotNil) c.Assert(r, HasLen, 3) @@ -113,7 +113,7 @@ func (s *testTableCodecSuite) TestRowCodec(c *C) { // colMap may contains less columns than encoded row. delete(colMap, 3) delete(colMap, 4) - r, err = DecodeRow(bs, colMap, time.Local) + r, err = DecodeRow(bs, colMap, time.UTC) c.Assert(err, IsNil) c.Assert(r, NotNil) c.Assert(r, HasLen, 2) @@ -133,7 +133,7 @@ func (s *testTableCodecSuite) TestRowCodec(c *C) { c.Assert(err, IsNil) c.Assert(bs, HasLen, 1) - r, err = DecodeRow(bs, colMap, time.Local) + r, err = DecodeRow(bs, colMap, time.UTC) c.Assert(err, IsNil) c.Assert(r, IsNil) } @@ -151,7 +151,8 @@ func (s *testTableCodecSuite) TestTimeCodec(c *C) { row := make([]types.Datum, colLen) row[0] = types.NewIntDatum(100) row[1] = types.NewBytesDatum([]byte("abc")) - ts, err := types.ParseTimestamp(nil, "2016-06-23 11:30:45") + ts, err := types.ParseTimestamp(&stmtctx.StatementContext{TimeZone: time.UTC}, + "2016-06-23 11:30:45") c.Assert(err, IsNil) row[2] = types.NewDatum(ts) du, err := types.ParseDuration("12:59:59.999999", 6) @@ -163,7 +164,7 @@ func (s *testTableCodecSuite) TestTimeCodec(c *C) { for _, col := range cols { colIDs = append(colIDs, col.id) } - sc := &stmtctx.StatementContext{TimeZone: time.Local} + sc := &stmtctx.StatementContext{TimeZone: time.UTC} bs, err := EncodeRow(sc, row, colIDs, nil, nil) c.Assert(err, IsNil) c.Assert(bs, NotNil) @@ -173,7 +174,7 @@ func (s *testTableCodecSuite) TestTimeCodec(c *C) { for _, col := range cols { colMap[col.id] = col.tp } - r, err := DecodeRow(bs, colMap, time.Local) + r, err := DecodeRow(bs, colMap, time.UTC) c.Assert(err, IsNil) c.Assert(r, NotNil) c.Assert(r, HasLen, colLen) @@ -201,7 +202,7 @@ func (s *testTableCodecSuite) TestCutRow(c *C) { row[1] = types.NewBytesDatum([]byte("abc")) row[2] = types.NewDecimalDatum(types.NewDecFromInt(1)) - sc := &stmtctx.StatementContext{TimeZone: time.Local} + sc := &stmtctx.StatementContext{TimeZone: time.UTC} data := make([][]byte, 3) data[0], err = EncodeValue(sc, row[0]) c.Assert(err, IsNil) @@ -237,7 +238,7 @@ func (s *testTableCodecSuite) TestCutKeyNew(c *C) { values := []types.Datum{types.NewIntDatum(1), types.NewBytesDatum([]byte("abc")), types.NewFloat64Datum(5.5)} handle := types.NewIntDatum(100) values = append(values, handle) - sc := &stmtctx.StatementContext{TimeZone: time.Local} + sc := &stmtctx.StatementContext{TimeZone: time.UTC} encodedValue, err := codec.EncodeKey(sc, nil, values...) c.Assert(err, IsNil) tableID := int64(4) @@ -260,7 +261,7 @@ func (s *testTableCodecSuite) TestCutKey(c *C) { values := []types.Datum{types.NewIntDatum(1), types.NewBytesDatum([]byte("abc")), types.NewFloat64Datum(5.5)} handle := types.NewIntDatum(100) values = append(values, handle) - sc := &stmtctx.StatementContext{TimeZone: time.Local} + sc := &stmtctx.StatementContext{TimeZone: time.UTC} encodedValue, err := codec.EncodeKey(sc, nil, values...) c.Assert(err, IsNil) tableID := int64(4) @@ -347,7 +348,7 @@ func (s *testTableCodecSuite) TestDecodeIndexKey(c *C) { } valueStrs = append(valueStrs, str) } - sc := &stmtctx.StatementContext{TimeZone: time.Local} + sc := &stmtctx.StatementContext{TimeZone: time.UTC} encodedValue, err := codec.EncodeKey(sc, nil, values...) c.Assert(err, IsNil) indexKey := EncodeIndexSeekKey(tableID, indexID, encodedValue) diff --git a/types/convert_test.go b/types/convert_test.go index c92d546425..659962177b 100644 --- a/types/convert_test.go +++ b/types/convert_test.go @@ -156,7 +156,14 @@ func (s *testTypeConvertSuite) TestConvertType(c *C) { c.Assert(err, IsNil) c.Assert(vv.(Duration).String(), Equals, "10:11:12.1") - vt, err := ParseTime(nil, "2010-10-10 10:11:11.12345", mysql.TypeTimestamp, 2) + vd, err := ParseTime(nil, "2010-10-10 10:11:11.12345", mysql.TypeDatetime, 2) + c.Assert(vd.String(), Equals, "2010-10-10 10:11:11.12") + c.Assert(err, IsNil) + v, err = Convert(vd, ft) + c.Assert(err, IsNil) + c.Assert(v.(Duration).String(), Equals, "10:11:11.1") + + vt, err := ParseTime(&stmtctx.StatementContext{TimeZone: time.UTC}, "2010-10-10 10:11:11.12345", mysql.TypeTimestamp, 2) c.Assert(vt.String(), Equals, "2010-10-10 10:11:11.12") c.Assert(err, IsNil) v, err = Convert(vt, ft) @@ -323,7 +330,8 @@ func (s *testTypeConvertSuite) TestConvertToString(c *C) { testToString(c, Enum{Name: "a", Value: 1}, "a") testToString(c, Set{Name: "a", Value: 1}, "a") - t, err := ParseTime(nil, "2011-11-10 11:11:11.999999", mysql.TypeTimestamp, 6) + t, err := ParseTime(&stmtctx.StatementContext{TimeZone: time.UTC}, + "2011-11-10 11:11:11.999999", mysql.TypeTimestamp, 6) c.Assert(err, IsNil) testToString(c, t, "2011-11-10 11:11:11.999999") @@ -470,6 +478,7 @@ func accept(c *C, tp byte, value interface{}, unsigned bool, expected string) { } d := NewDatum(value) sc := new(stmtctx.StatementContext) + sc.TimeZone = time.UTC sc.IgnoreTruncate = true casted, err := d.ConvertTo(sc, ft) c.Assert(err, IsNil, Commentf("%v", ft)) diff --git a/types/datum_test.go b/types/datum_test.go index b2fd3f202d..a5b106f857 100644 --- a/types/datum_test.go +++ b/types/datum_test.go @@ -68,7 +68,7 @@ func (ts *testDatumSuite) TestToBool(c *C) { testDatumToBool(c, Enum{Name: "a", Value: 1}, 1) testDatumToBool(c, Set{Name: "a", Value: 1}, 1) - t, err := ParseTime(nil, "2011-11-10 11:11:11.999999", mysql.TypeTimestamp, 6) + t, err := ParseTime(&stmtctx.StatementContext{TimeZone: time.UTC}, "2011-11-10 11:11:11.999999", mysql.TypeTimestamp, 6) c.Assert(err, IsNil) testDatumToBool(c, t, 1) @@ -145,7 +145,9 @@ func (ts *testTypeConvertSuite) TestToInt64(c *C) { testDatumToInt64(c, Set{Name: "a", Value: 1}, int64(1)) testDatumToInt64(c, json.CreateBinary(int64(3)), int64(3)) - t, err := ParseTime(nil, "2011-11-10 11:11:11.999999", mysql.TypeTimestamp, 0) + t, err := ParseTime(&stmtctx.StatementContext{ + TimeZone: time.UTC, + }, "2011-11-10 11:11:11.999999", mysql.TypeTimestamp, 0) c.Assert(err, IsNil) testDatumToInt64(c, t, int64(20111110111112)) @@ -192,7 +194,7 @@ func (ts *testTypeConvertSuite) TestToFloat32(c *C) { // mustParseTimeIntoDatum is similar to ParseTime but panic if any error occurs. func mustParseTimeIntoDatum(s string, tp byte, fsp int) (d Datum) { - t, err := ParseTime(nil, s, tp, fsp) + t, err := ParseTime(&stmtctx.StatementContext{TimeZone: time.UTC}, s, tp, fsp) if err != nil { panic("ParseTime fail") } diff --git a/types/export_test.go b/types/export_test.go new file mode 100644 index 0000000000..ab09c4de70 --- /dev/null +++ b/types/export_test.go @@ -0,0 +1,17 @@ +// Copyright 2018 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package types + +// CheckTimestampTypeForTest export CheckTimestampType for test. +var CheckTimestampTypeForTest = checkTimestampType diff --git a/types/time.go b/types/time.go index 6a2ba778dc..6aa9eb2b96 100644 --- a/types/time.go +++ b/types/time.go @@ -110,10 +110,20 @@ var ( // MaxDatetime is the maximum for mysql datetime type. MaxDatetime = FromDate(9999, 12, 31, 23, 59, 59, 999999) + // BoundTimezone is the timezone for min and max timestamp. + BoundTimezone = gotime.UTC // MinTimestamp is the minimum for mysql timestamp type. - MinTimestamp = FromDate(1970, 1, 1, 0, 0, 1, 0) + MinTimestamp = Time{ + Time: FromDate(1970, 1, 1, 0, 0, 1, 0), + Type: mysql.TypeTimestamp, + Fsp: DefaultFsp, + } // maxTimestamp is the maximum for mysql timestamp type. - maxTimestamp = FromDate(2038, 1, 19, 3, 14, 7, 999999) + maxTimestamp = Time{ + Time: FromDate(2038, 1, 19, 3, 14, 7, 999999), + Type: mysql.TypeTimestamp, + Fsp: DefaultFsp, + } // WeekdayNames lists names of weekdays, which are used in builtin time function `dayname`. WeekdayNames = []string{ @@ -298,6 +308,11 @@ func (t Time) Compare(o Time) int { return compareTime(t.Time, o.Time) } +// compareTime compare two MysqlTime. +// return: +// 0: if a == b +// 1: if a > b +// -1: if a < b func compareTime(a, b MysqlTime) int { ta := datetimeToUint64(a) tb := datetimeToUint64(b) @@ -463,7 +478,7 @@ func (t *Time) check(sc *stmtctx.StatementContext) error { var err error switch t.Type { case mysql.TypeTimestamp: - err = checkTimestampType(t.Time) + err = checkTimestampType(sc, t.Time) case mysql.TypeDatetime: err = checkDatetimeType(t.Time, allowZeroInDate) case mysql.TypeDate: @@ -473,8 +488,8 @@ func (t *Time) check(sc *stmtctx.StatementContext) error { } // Check if 't' is valid -func (t *Time) Check() error { - return t.check(nil) +func (t *Time) Check(sc *stmtctx.StatementContext) error { + return t.check(sc) } // Sub subtracts t1 from t, returns a duration value. @@ -506,7 +521,7 @@ func (t *Time) Sub(sc *stmtctx.StatementContext, t1 *Time) Duration { } // Add adds d to t, returns the result time value. -func (t *Time) Add(d Duration) (Time, error) { +func (t *Time) Add(sc *stmtctx.StatementContext, d Duration) (Time, error) { sign, hh, mm, ss, micro := splitDuration(d.Duration) seconds, microseconds, _ := calcTimeDiff(t.Time, FromDate(0, 0, 0, hh, mm, ss, micro), -sign) days := seconds / secondsIn24Hour @@ -529,7 +544,7 @@ func (t *Time) Add(d Duration) (Time, error) { Type: t.Type, Fsp: fsp, } - return ret, ret.Check() + return ret, ret.Check(sc) } // TimestampDiff returns t2 - t1 where t1 and t2 are date or datetime expressions. @@ -854,7 +869,7 @@ func (d Duration) ConvertToTime(sc *stmtctx.StatementContext, tp uint8) (Time, e Type: mysql.TypeDatetime, Fsp: d.Fsp, } - return t.Convert(nil, tp) + return t.Convert(sc, tp) } // RoundFrac rounds fractional seconds precision with new fsp and returns a new one. @@ -1345,12 +1360,27 @@ func checkMonthDay(year, month, day int) error { return nil } -func checkTimestampType(t MysqlTime) error { +func checkTimestampType(sc *stmtctx.StatementContext, t MysqlTime) error { if compareTime(t, ZeroTime) == 0 { return nil } - if compareTime(t, maxTimestamp) > 0 || compareTime(t, MinTimestamp) < 0 { + if sc == nil { + return errors.New("statementContext is required during checkTimestampType") + } + + var checkTime MysqlTime + if sc.TimeZone != BoundTimezone { + convertTime := Time{Time: t, Type: mysql.TypeTimestamp} + err := convertTime.ConvertTimeZone(sc.TimeZone, BoundTimezone) + if err != nil { + return err + } + checkTime = convertTime.Time + } else { + checkTime = t + } + if compareTime(checkTime, maxTimestamp.Time) > 0 || compareTime(checkTime, MinTimestamp.Time) < 0 { return errors.Trace(ErrInvalidTimeFormat.GenByArgs(t)) } diff --git a/types/time_test.go b/types/time_test.go index 43323d3f58..a46ce09e28 100644 --- a/types/time_test.go +++ b/types/time_test.go @@ -19,6 +19,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/tidb/mysql" + "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testleak" @@ -114,7 +115,7 @@ func (s *testTimeSuite) TestTimestamp(c *C) { } for _, test := range table { - t, err := types.ParseTimestamp(nil, test.Input) + t, err := types.ParseTimestamp(&stmtctx.StatementContext{TimeZone: time.UTC}, test.Input) c.Assert(err, IsNil) c.Assert(t.String(), Equals, test.Expect) } @@ -125,7 +126,7 @@ func (s *testTimeSuite) TestTimestamp(c *C) { } for _, test := range errTable { - _, err := types.ParseTimestamp(nil, test) + _, err := types.ParseTimestamp(&stmtctx.StatementContext{TimeZone: time.UTC}, test) c.Assert(err, NotNil) } } @@ -407,11 +408,14 @@ func (s *testTimeSuite) getLocation(c *C) *time.Location { func (s *testTimeSuite) TestCodec(c *C) { defer testleak.AfterTest(c)() + + sc := &stmtctx.StatementContext{TimeZone: time.UTC} + // MySQL timestamp value doesn't allow month=0 or day=0. - t, err := types.ParseTimestamp(nil, "2016-12-00 00:00:00") + t, err := types.ParseTimestamp(sc, "2016-12-00 00:00:00") c.Assert(err, NotNil) - t, err = types.ParseTimestamp(nil, "2010-10-10 10:11:11") + t, err = types.ParseTimestamp(sc, "2010-10-10 10:11:11") c.Assert(err, IsNil) packed, err := t.ToPackedUint() c.Assert(err, IsNil) @@ -518,7 +522,9 @@ func (s *testTimeSuite) TestParseTimeFromNum(c *C) { c.Assert(t.String(), Equals, test.ExpectDateTimeValue) // testtypes.ParseTimestampFromNum - t, err = types.ParseTimestampFromNum(nil, test.Input) + t, err = types.ParseTimestampFromNum(&stmtctx.StatementContext{ + TimeZone: time.UTC, + }, test.Input) if test.ExpectTimeStampError { c.Assert(err, NotNil) } else { @@ -652,7 +658,7 @@ func (s *testTimeSuite) TestParseFrac(c *C) { func (s *testTimeSuite) TestRoundFrac(c *C) { sc := mock.NewContext().GetSessionVars().StmtCtx sc.IgnoreZeroInDate = true - sc.TimeZone = time.Local + sc.TimeZone = time.UTC defer testleak.AfterTest(c)() tbl := []struct { Input string @@ -923,6 +929,9 @@ func (s *testTimeSuite) TestTimeAdd(c *C) { {"2017-08-21", "01:01:01.001", "2017-08-21 01:01:01.001"}, } + sc := &stmtctx.StatementContext{ + TimeZone: time.UTC, + } for _, t := range tbl { v1, err := types.ParseTime(nil, t.Arg1, mysql.TypeDatetime, types.MaxFsp) c.Assert(err, IsNil) @@ -930,7 +939,7 @@ func (s *testTimeSuite) TestTimeAdd(c *C) { c.Assert(err, IsNil) result, err := types.ParseTime(nil, t.Ret, mysql.TypeDatetime, types.MaxFsp) c.Assert(err, IsNil) - v2, err := v1.Add(dur) + v2, err := v1.Add(sc, dur) c.Assert(err, IsNil) c.Assert(v2.Compare(result), Equals, 0, Commentf("%v %v", v2.Time, result.Time)) } @@ -967,3 +976,56 @@ func (s *testTimeSuite) TestTruncateOverflowMySQLTime(c *C) { c.Assert(err, IsNil) c.Assert(res, Equals, types.MinTime+1) } + +func (s *testTimeSuite) TestCheckTimestamp(c *C) { + + shanghaiTz, _ := time.LoadLocation("Asia/Shanghai") + + tests := []struct { + tz *time.Location + input types.MysqlTime + expectRetError bool + }{{ + tz: shanghaiTz, + input: types.FromDate(2038, 1, 19, 11, 14, 7, 0), + expectRetError: false, + }, { + tz: shanghaiTz, + input: types.FromDate(1970, 1, 1, 8, 1, 1, 0), + expectRetError: false, + }, { + tz: shanghaiTz, + input: types.FromDate(2038, 1, 19, 12, 14, 7, 0), + expectRetError: true, + }, { + tz: shanghaiTz, + input: types.FromDate(1970, 1, 1, 7, 1, 1, 0), + expectRetError: true, + }, { + tz: time.UTC, + input: types.FromDate(2038, 1, 19, 3, 14, 7, 0), + expectRetError: false, + }, { + tz: time.UTC, + input: types.FromDate(1970, 1, 1, 0, 1, 1, 0), + expectRetError: false, + }, { + tz: time.UTC, + input: types.FromDate(2038, 1, 19, 4, 14, 7, 0), + expectRetError: true, + }, { + tz: time.UTC, + input: types.FromDate(1969, 1, 1, 0, 0, 0, 0), + expectRetError: true, + }, + } + + for _, t := range tests { + validTimestamp := types.CheckTimestampTypeForTest(&stmtctx.StatementContext{TimeZone: t.tz}, t.input) + if t.expectRetError { + c.Assert(validTimestamp, NotNil, Commentf("For %s", t.input)) + } else { + c.Assert(validTimestamp, IsNil, Commentf("For %s", t.input)) + } + } +} diff --git a/util/mock/context.go b/util/mock/context.go index cc2fef68b7..048f7a04c7 100644 --- a/util/mock/context.go +++ b/util/mock/context.go @@ -17,6 +17,7 @@ package mock import ( "fmt" "sync" + "time" "github.com/juju/errors" "github.com/pingcap/tidb/kv" @@ -217,6 +218,7 @@ func NewContext() *Context { cancel: cancel, } sctx.sessionVars.MaxChunkSize = 2 + sctx.sessionVars.StmtCtx.TimeZone = time.UTC return sctx }