*: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:
@ -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
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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)
|
||||
}
|
||||
|
||||
@ -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()
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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))
|
||||
|
||||
@ -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
17
types/export_test.go
Normal 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
|
||||
@ -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))
|
||||
}
|
||||
|
||||
|
||||
@ -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))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user