Skip to content

Commit

Permalink
*: insert of invalid timestamp succeeded (#26584)
Browse files Browse the repository at this point in the history
  • Loading branch information
mjonss authored Aug 5, 2021
1 parent 8ebf6c3 commit 3cda7d0
Show file tree
Hide file tree
Showing 6 changed files with 34 additions and 26 deletions.
4 changes: 2 additions & 2 deletions ddl/column_type_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2175,8 +2175,8 @@ func (s *testColumnTypeChangeSuite) TestCastDateToTimestampInReorgAttribute(c *C
s.dom.DDL().(ddl.DDLForTest).SetHook(hook)

tk.MustExec("alter table t modify column a TIMESTAMP NULL DEFAULT '2021-04-28 03:35:11' FIRST")
c.Assert(checkErr1.Error(), Equals, "[types:1292]Incorrect datetime value: '3977-02-22 00:00:00'")
c.Assert(checkErr2.Error(), Equals, "[types:1292]Incorrect datetime value: '3977-02-22 00:00:00'")
c.Assert(checkErr1.Error(), Equals, "[types:1292]Incorrect timestamp value: '3977-02-22'")
c.Assert(checkErr2.Error(), Equals, "[types:1292]Incorrect timestamp value: '3977-02-22'")
tk.MustExec("drop table if exists t")
}

Expand Down
5 changes: 5 additions & 0 deletions executor/index_lookup_join.go
Original file line number Diff line number Diff line change
Expand Up @@ -517,6 +517,11 @@ func (iw *innerWorker) constructLookupContent(task *lookUpJoinTask) ([]*indexJoi
for rowIdx := 0; rowIdx < numRows; rowIdx++ {
dLookUpKey, dHashKey, err := iw.constructDatumLookupKey(task, chkIdx, rowIdx)
if err != nil {
if terror.ErrorEqual(err, types.ErrWrongValue) {
// We ignore rows with invalid datetime.
task.encodedLookUpKeys[chkIdx].AppendNull(0)
continue
}
return nil, err
}
if dHashKey == nil {
Expand Down
12 changes: 12 additions & 0 deletions executor/insert_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -336,6 +336,18 @@ func (s *testSuite3) TestInsertWrongValueForField(c *C) {
tk.MustExec(`create table t (a year);`)
_, err = tk.Exec(`insert into t values(2156);`)
c.Assert(err.Error(), Equals, `[types:8033]invalid year`)

tk.MustExec(`DROP TABLE IF EXISTS ts`)
tk.MustExec(`CREATE TABLE ts (id int DEFAULT NULL, time1 TIMESTAMP NULL DEFAULT NULL)`)
tk.MustExec(`SET @@sql_mode=''`)
tk.MustExec(`INSERT INTO ts (id, time1) VALUES (1, TIMESTAMP '1018-12-23 00:00:00')`)
tk.MustQuery(`SHOW WARNINGS`).Check(testkit.Rows(`Warning 1292 Incorrect timestamp value: '1018-12-23 00:00:00'`))
tk.MustQuery(`SELECT * FROM ts ORDER BY id`).Check(testkit.Rows(`1 0000-00-00 00:00:00`))

tk.MustExec(`SET @@sql_mode='STRICT_TRANS_TABLES'`)
_, err = tk.Exec(`INSERT INTO ts (id, time1) VALUES (2, TIMESTAMP '1018-12-24 00:00:00')`)
c.Assert(err.Error(), Equals, `[table:1292]Incorrect timestamp value: '1018-12-24 00:00:00' for column 'time1' at row 1`)
tk.MustExec(`DROP TABLE ts`)
}

func (s *testSuite3) TestInsertValueForCastDecimalField(c *C) {
Expand Down
4 changes: 1 addition & 3 deletions sessionctx/stmtctx/stmtctx.go
Original file line number Diff line number Diff line change
Expand Up @@ -63,9 +63,7 @@ type StatementContext struct {

// IsDDLJobInQueue is used to mark whether the DDL job is put into the queue.
// If IsDDLJobInQueue is true, it means the DDL job is in the queue of storage, and it can be handled by the DDL worker.
IsDDLJobInQueue bool
// InReorgAttribute is indicated for cast function that the transition is a kind of reorg process.
InReorgAttribute bool
IsDDLJobInQueue bool
InInsertStmt bool
InUpdateStmt bool
InDeleteStmt bool
Expand Down
21 changes: 10 additions & 11 deletions table/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -224,11 +224,13 @@ func handleZeroDatetime(ctx sessionctx.Context, col *model.ColumnInfo, casted ty

ignoreErr := sc.DupKeyAsWarning

// Timestamp in MySQL is since EPOCH 1970-01-01 00:00:00 UTC and can by definition not have invalid dates!
// Zero date is special for MySQL timestamp and *NOT* 1970-01-01 00:00:00, but 0000-00-00 00:00:00!
// in MySQL 8.0, the Timestamp's case is different to Datetime/Date, as shown below:
//
// | | NZD | NZD|ST | ELSE | ELSE|ST |
// | ------------ | ----------------- | ------- | ----------------- | -------- |
// | `0000-00-01` | Success + Warning | Error | Success + Warning | Error |
// | `0000-00-01` | Truncate + Warning| Error | Truncate + Warning| Error |
// | `0000-00-00` | Success + Warning | Error | Success | Success |
//
// * **NZD**: NO_ZERO_DATE_MODE
Expand Down Expand Up @@ -273,21 +275,13 @@ func handleZeroDatetime(ctx sessionctx.Context, col *model.ColumnInfo, casted ty

// CastValue casts a value based on column type.
// If forceIgnoreTruncate is true, truncated errors will be ignored.
// If returnOverflow is true, don't handle overflow errors in this function.
// If returnErr is true, directly return any conversion errors.
// It's safe now and it's the same as the behavior of select statement.
// Set it to true only in FillVirtualColumnValue and UnionScanExec.Next()
// If the handle of err is changed latter, the behavior of forceIgnoreTruncate also need to change.
// TODO: change the third arg to TypeField. Not pass ColumnInfo.
func CastValue(ctx sessionctx.Context, val types.Datum, col *model.ColumnInfo, returnErr, forceIgnoreTruncate bool) (casted types.Datum, err error) {
sc := ctx.GetSessionVars().StmtCtx
// Set the reorg attribute for cast value functionality.
if col.ChangeStateInfo != nil {
origin := ctx.GetSessionVars().StmtCtx.InReorgAttribute
ctx.GetSessionVars().StmtCtx.InReorgAttribute = true
defer func() {
ctx.GetSessionVars().StmtCtx.InReorgAttribute = origin
}()
}
casted, err = val.ConvertTo(sc, &col.FieldType)
// TODO: make sure all truncate errors are handled by ConvertTo.
if returnErr && err != nil {
Expand All @@ -302,7 +296,12 @@ func CastValue(ctx sessionctx.Context, val types.Datum, col *model.ColumnInfo, r
} else if (sc.InInsertStmt || sc.InUpdateStmt) && !casted.IsNull() &&
(val.Kind() != types.KindMysqlTime || !val.GetMysqlTime().IsZero()) &&
(col.Tp == mysql.TypeDate || col.Tp == mysql.TypeDatetime || col.Tp == mysql.TypeTimestamp) {
if innCasted, exit, innErr := handleZeroDatetime(ctx, col, casted, val.GetString(), types.ErrWrongValue.Equal(err)); exit {
str, err1 := val.ToString()
if err1 != nil {
logutil.BgLogger().Warn("Datum ToString failed", zap.Stringer("Datum", val), zap.Error(err1))
str = val.GetString()
}
if innCasted, exit, innErr := handleZeroDatetime(ctx, col, casted, str, types.ErrWrongValue.Equal(err)); exit {
return innCasted, innErr
}
}
Expand Down
14 changes: 4 additions & 10 deletions types/datum.go
Original file line number Diff line number Diff line change
Expand Up @@ -1128,16 +1128,10 @@ func (d *Datum) convertToMysqlTimestamp(sc *stmtctx.StatementContext, target *Fi
}
switch d.k {
case KindMysqlTime:
// `select timestamp(cast("1000-01-02 23:59:59" as date)); ` casts usage will succeed.
// Alter datetime("1000-01-02 23:59:59") to timestamp will error.
if sc.InReorgAttribute {
t, err = d.GetMysqlTime().Convert(sc, target.Tp)
if err != nil {
ret.SetMysqlTime(t)
return ret, errors.Trace(ErrWrongValue.GenWithStackByArgs(DateTimeStr, t.String()))
}
} else {
t = d.GetMysqlTime()
t, err = d.GetMysqlTime().Convert(sc, target.Tp)
if err != nil {
ret.SetMysqlTime(ZeroTimestamp)
return ret, errors.Trace(ErrWrongValue.GenWithStackByArgs(TimestampStr, t.String()))
}
t, err = t.RoundFrac(sc, fsp)
case KindMysqlDuration:
Expand Down

0 comments on commit 3cda7d0

Please sign in to comment.