*:check max/min timestamp bound with timezone aware (#6581)

- mytime: max/min configure with UTC tz, convert to local tz during process starting, convert it to session tz before checking bound.
- time: so `check()`/`add()` need force pass `StatementContext` parameter.
- builtin_time: pass `StatementContext` from `b.ctx.sessionVars` into check/add.
- dump: also set timezone for StatementContext with UTC.
- *_test/mock: fix test case, make mock context with stmt TimeZone.
- add testcase for checkTimestamp and mytime convert timezone.
This commit is contained in:
lysu
2018-05-24 15:27:38 +08:00
committed by tiancaiamao
parent e705c18d19
commit 87ce884b2e
11 changed files with 173 additions and 48 deletions

View File

@ -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

View File

@ -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)

View File

@ -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)
}

View File

@ -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()

View File

@ -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)

View File

@ -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))

View File

@ -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")
}

17
types/export_test.go Normal file
View File

@ -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

View File

@ -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))
}

View File

@ -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))
}
}
}

View File

@ -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
}