Skip to content

Commit

Permalink
expression: fix tidb can't alter table from other-type with null valu…
Browse files Browse the repository at this point in the history
…e to timestamp with NOT NULL attribute (pingcap#29664)
  • Loading branch information
AilinKid authored Dec 3, 2021
1 parent 70d2973 commit 2844660
Show file tree
Hide file tree
Showing 9 changed files with 126 additions and 20 deletions.
2 changes: 2 additions & 0 deletions ddl/backfilling.go
Original file line number Diff line number Diff line change
Expand Up @@ -606,6 +606,8 @@ func (w *worker) writePhysicalTableRecord(t table.PhysicalTable, bfWorkerType ba
// Simulate the sql mode environment in the worker sessionCtx.
sqlMode := reorgInfo.ReorgMeta.SQLMode
sessCtx.GetSessionVars().SQLMode = sqlMode
// TODO: skip set the timezone, it will cause data inconsistency when add index, since some reorg place using the timeUtil.SystemLocation() to do the time conversion. (need a more systemic plan)
// sessCtx.GetSessionVars().TimeZone = reorgInfo.ReorgMeta.Location
sessCtx.GetSessionVars().StmtCtx.BadNullAsWarning = !sqlMode.HasStrictMode()
sessCtx.GetSessionVars().StmtCtx.TruncateAsWarning = !sqlMode.HasStrictMode()
sessCtx.GetSessionVars().StmtCtx.OverflowAsWarning = !sqlMode.HasStrictMode()
Expand Down
37 changes: 29 additions & 8 deletions ddl/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ import (
"github.com/pingcap/failpoint"
"github.com/pingcap/tidb/config"
ddlutil "github.com/pingcap/tidb/ddl/util"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/meta"
Expand Down Expand Up @@ -969,12 +970,14 @@ func (w *worker) doModifyColumnTypeWithData(
colName model.CIStr, pos *ast.ColumnPosition, changingIdxs []*model.IndexInfo) (ver int64, _ error) {
var err error
originalState := changingCol.State
targetCol := changingCol.Clone()
targetCol.Name = colName
switch changingCol.State {
case model.StateNone:
// Column from null to not null.
if !mysql.HasNotNullFlag(oldCol.Flag) && mysql.HasNotNullFlag(changingCol.Flag) {
// Introduce the `mysql.PreventNullInsertFlag` flag to prevent users from inserting or updating null values.
err := modifyColsFromNull2NotNull(w, dbInfo, tblInfo, []*model.ColumnInfo{oldCol}, oldCol.Name, oldCol.Tp != changingCol.Tp)
err := modifyColsFromNull2NotNull(w, dbInfo, tblInfo, []*model.ColumnInfo{oldCol}, targetCol, oldCol.Tp != changingCol.Tp)
if err != nil {
if ErrWarnDataTruncated.Equal(err) || errInvalidUseOfNull.Equal(err) {
job.State = model.JobStateRollingback
Expand Down Expand Up @@ -1018,7 +1021,7 @@ func (w *worker) doModifyColumnTypeWithData(
// Column from null to not null.
if !mysql.HasNotNullFlag(oldCol.Flag) && mysql.HasNotNullFlag(changingCol.Flag) {
// Introduce the `mysql.PreventNullInsertFlag` flag to prevent users from inserting or updating null values.
err := modifyColsFromNull2NotNull(w, dbInfo, tblInfo, []*model.ColumnInfo{oldCol}, oldCol.Name, oldCol.Tp != changingCol.Tp)
err := modifyColsFromNull2NotNull(w, dbInfo, tblInfo, []*model.ColumnInfo{oldCol}, targetCol, oldCol.Tp != changingCol.Tp)
if err != nil {
if ErrWarnDataTruncated.Equal(err) || errInvalidUseOfNull.Equal(err) {
job.State = model.JobStateRollingback
Expand Down Expand Up @@ -1348,6 +1351,14 @@ func (w *updateColumnWorker) getRowRecord(handle kv.Handle, recordKey []byte, ra
oldWarn = oldWarn[:0]
}
w.sessCtx.GetSessionVars().StmtCtx.SetWarnings(oldWarn)
val := w.rowMap[w.oldColInfo.ID]
col := w.newColInfo
if val.Kind() == types.KindNull && col.FieldType.Tp == mysql.TypeTimestamp && mysql.HasNotNullFlag(col.Flag) {
if v, err := expression.GetTimeCurrentTimestamp(w.sessCtx, col.Tp, int8(col.Decimal)); err == nil {
// convert null value to timestamp should be substituted with current timestamp if NOT_NULL flag is set.
w.rowMap[w.oldColInfo.ID] = v
}
}
newColVal, err := table.CastValue(w.sessCtx, w.rowMap[w.oldColInfo.ID], w.newColInfo, false, false)
if err != nil {
return w.reformatErrors(err)
Expand Down Expand Up @@ -1483,7 +1494,7 @@ func (w *worker) doModifyColumn(
}

// Introduce the `mysql.PreventNullInsertFlag` flag to prevent users from inserting or updating null values.
err := modifyColsFromNull2NotNull(w, dbInfo, tblInfo, []*model.ColumnInfo{oldCol}, newCol.Name, oldCol.Tp != newCol.Tp)
err := modifyColsFromNull2NotNull(w, dbInfo, tblInfo, []*model.ColumnInfo{oldCol}, newCol, oldCol.Tp != newCol.Tp)
if err != nil {
if ErrWarnDataTruncated.Equal(err) || errInvalidUseOfNull.Equal(err) {
job.State = model.JobStateRollingback
Expand Down Expand Up @@ -1666,7 +1677,18 @@ func applyNewAutoRandomBits(d *ddlCtx, m *meta.Meta, dbInfo *model.DBInfo,

// checkForNullValue ensure there are no null values of the column of this table.
// `isDataTruncated` indicates whether the new field and the old field type are the same, in order to be compatible with mysql.
func checkForNullValue(ctx context.Context, sctx sessionctx.Context, isDataTruncated bool, schema, table, newCol model.CIStr, oldCols ...*model.ColumnInfo) error {
func checkForNullValue(ctx context.Context, sctx sessionctx.Context, isDataTruncated bool, schema, table model.CIStr, newCol *model.ColumnInfo, oldCols ...*model.ColumnInfo) error {
needCheckNullValue := false
for _, oldCol := range oldCols {
if oldCol.Tp != mysql.TypeTimestamp && newCol.Tp == mysql.TypeTimestamp {
// special case for convert null value of non-timestamp type to timestamp type, null value will be substituted with current timestamp.
continue
}
needCheckNullValue = true
}
if !needCheckNullValue {
return nil
}
var buf strings.Builder
buf.WriteString("select 1 from %n.%n where ")
paramsList := make([]interface{}, 0, 2+len(oldCols))
Expand All @@ -1692,7 +1714,7 @@ func checkForNullValue(ctx context.Context, sctx sessionctx.Context, isDataTrunc
rowCount := len(rows)
if rowCount != 0 {
if isDataTruncated {
return ErrWarnDataTruncated.GenWithStackByArgs(newCol.L, rowCount)
return ErrWarnDataTruncated.GenWithStackByArgs(newCol.Name.L, rowCount)
}
return errInvalidUseOfNull
}
Expand Down Expand Up @@ -1802,8 +1824,7 @@ func rollbackModifyColumnJob(t *meta.Meta, tblInfo *model.TableInfo, job *model.

// modifyColsFromNull2NotNull modifies the type definitions of 'null' to 'not null'.
// Introduce the `mysql.PreventNullInsertFlag` flag to prevent users from inserting or updating null values.
func modifyColsFromNull2NotNull(w *worker, dbInfo *model.DBInfo, tblInfo *model.TableInfo, cols []*model.ColumnInfo,
newColName model.CIStr, isDataTruncated bool) error {
func modifyColsFromNull2NotNull(w *worker, dbInfo *model.DBInfo, tblInfo *model.TableInfo, cols []*model.ColumnInfo, newCol *model.ColumnInfo, isDataTruncated bool) error {
// Get sessionctx from context resource pool.
var sctx sessionctx.Context
sctx, err := w.sessPool.get()
Expand All @@ -1820,7 +1841,7 @@ func modifyColsFromNull2NotNull(w *worker, dbInfo *model.DBInfo, tblInfo *model.
})
if !skipCheck {
// If there is a null value inserted, it cannot be modified and needs to be rollback.
err = checkForNullValue(w.ddlJobCtx, sctx, isDataTruncated, dbInfo.Name, tblInfo.Name, newColName, cols...)
err = checkForNullValue(w.ddlJobCtx, sctx, isDataTruncated, dbInfo.Name, tblInfo.Name, newCol, cols...)
if err != nil {
return errors.Trace(err)
}
Expand Down
46 changes: 46 additions & 0 deletions ddl/column_type_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2305,3 +2305,49 @@ func (s *testColumnTypeChangeSuite) TestForIssue24621(c *C) {
errMsg := "[types:1265]Data truncated for column 'a', value is '0123456789abc'"
tk.MustGetErrMsg("alter table t modify a char(12) null;", errMsg)
}

func (s *testColumnTypeChangeSuite) TestChangeNullValueFromOtherTypeToTimestamp(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")

// Some ddl cases.
prepare := func() {
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a int null)")
tk.MustExec("insert into t values()")
tk.MustQuery("select * from t").Check(testkit.Rows("<nil>"))
}

prepare()
tk.MustExec("alter table t modify column a timestamp NOT NULL")
tk.MustQuery("select count(*) from t where a = null").Check(testkit.Rows("0"))

prepare()
// only from other type NULL to timestamp type NOT NULL, it should be successful.
_, err := tk.Exec("alter table t change column a a1 time NOT NULL")
c.Assert(err, NotNil)
c.Assert(err.Error(), Equals, "[ddl:1265]Data truncated for column 'a1' at row 1")

prepare2 := func() {
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a timestamp null)")
tk.MustExec("insert into t values()")
tk.MustQuery("select * from t").Check(testkit.Rows("<nil>"))
}

prepare2()
// only from other type NULL to timestamp type NOT NULL, it should be successful. (timestamp to timestamp excluded)
_, err = tk.Exec("alter table t modify column a timestamp NOT NULL")
c.Assert(err, NotNil)
c.Assert(err.Error(), Equals, "[ddl:1265]Data truncated for column 'a' at row 1")

// Some dml cases.
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a timestamp NOT NULL)")
_, err = tk.Exec("insert into t values()")
c.Assert(err, NotNil)
c.Assert(err.Error(), Equals, "[table:1364]Field 'a' doesn't have a default value")

_, err = tk.Exec("insert into t values(null)")
c.Assert(err.Error(), Equals, "[table:1048]Column 'a' cannot be null")
}
6 changes: 5 additions & 1 deletion ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -4211,7 +4211,7 @@ func (d *ddl) getModifiableColumnJob(ctx context.Context, sctx sessionctx.Contex
// We support modifying the type definitions of 'null' to 'not null' now.
var modifyColumnTp byte
if !mysql.HasNotNullFlag(col.Flag) && mysql.HasNotNullFlag(newCol.Flag) {
if err = checkForNullValue(ctx, sctx, true, ident.Schema, ident.Name, newCol.Name, col.ColumnInfo); err != nil {
if err = checkForNullValue(ctx, sctx, true, ident.Schema, ident.Name, newCol.ColumnInfo, col.ColumnInfo); err != nil {
return nil, errors.Trace(err)
}
// `modifyColumnTp` indicates that there is a type modification.
Expand Down Expand Up @@ -4242,6 +4242,7 @@ func (d *ddl) getModifiableColumnJob(ctx context.Context, sctx sessionctx.Contex
SQLMode: sctx.GetSessionVars().SQLMode,
Warnings: make(map[errors.ErrorID]*terror.Error),
WarningsCount: make(map[errors.ErrorID]int64),
Location: sctx.GetSessionVars().Location(),
},
Args: []interface{}{&newCol, originalColName, spec.Position, modifyColumnTp, newAutoRandBits},
}
Expand Down Expand Up @@ -4481,6 +4482,7 @@ func (d *ddl) RenameColumn(ctx sessionctx.Context, ident ast.Ident, spec *ast.Al
SQLMode: ctx.GetSessionVars().SQLMode,
Warnings: make(map[errors.ErrorID]*terror.Error),
WarningsCount: make(map[errors.ErrorID]int64),
Location: ctx.GetSessionVars().Location(),
},
Args: []interface{}{&newCol, oldColName, spec.Position, 0},
}
Expand Down Expand Up @@ -5292,6 +5294,7 @@ func (d *ddl) CreatePrimaryKey(ctx sessionctx.Context, ti ast.Ident, indexName m
SQLMode: ctx.GetSessionVars().SQLMode,
Warnings: make(map[errors.ErrorID]*terror.Error),
WarningsCount: make(map[errors.ErrorID]int64),
Location: ctx.GetSessionVars().Location(),
},
Args: []interface{}{unique, indexName, indexPartSpecifications, indexOption, sqlMode, nil, global},
Priority: ctx.GetSessionVars().DDLReorgPriority,
Expand Down Expand Up @@ -5474,6 +5477,7 @@ func (d *ddl) CreateIndex(ctx sessionctx.Context, ti ast.Ident, keyType ast.Inde
SQLMode: ctx.GetSessionVars().SQLMode,
Warnings: make(map[errors.ErrorID]*terror.Error),
WarningsCount: make(map[errors.ErrorID]int64),
Location: ctx.GetSessionVars().Location(),
},
Args: []interface{}{unique, indexName, indexPartSpecifications, indexOption, hiddenCols, global},
Priority: ctx.GetSessionVars().DDLReorgPriority,
Expand Down
2 changes: 1 addition & 1 deletion ddl/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -371,7 +371,7 @@ func checkPrimaryKeyNotNull(w *worker, sqlMode mysql.SQLMode, t *meta.Meta, job
return nil, nil
}

err = modifyColsFromNull2NotNull(w, dbInfo, tblInfo, nullCols, model.NewCIStr(""), false)
err = modifyColsFromNull2NotNull(w, dbInfo, tblInfo, nullCols, &model.ColumnInfo{Name: model.NewCIStr("")}, false)
if err == nil {
return nil, nil
}
Expand Down
1 change: 1 addition & 0 deletions ddl/reorg.go
Original file line number Diff line number Diff line change
Expand Up @@ -199,6 +199,7 @@ func (w *worker) runReorgJob(t *meta.Meta, reorgInfo *reorgInfo, tblInfo *model.
SQLMode: mysql.ModeNone,
Warnings: make(map[errors.ErrorID]*terror.Error),
WarningsCount: make(map[errors.ErrorID]int64),
Location: time.Local,
}
}
if w.reorgCtx.doneCh == nil {
Expand Down
39 changes: 29 additions & 10 deletions expression/helper.go
Original file line number Diff line number Diff line change
Expand Up @@ -56,26 +56,45 @@ func IsValidCurrentTimestampExpr(exprNode ast.ExprNode, fieldType *types.FieldTy
return (containsArg && isConsistent) || (!containsArg && !containsFsp)
}

// GetTimeCurrentTimestamp is used for generating a timestamp for some special cases: cast null value to timestamp type with not null flag.
func GetTimeCurrentTimestamp(ctx sessionctx.Context, tp byte, fsp int8) (d types.Datum, err error) {
var t types.Time
t, err = getTimeCurrentTimeStamp(ctx, tp, fsp)
if err != nil {
return d, err
}
d.SetMysqlTime(t)
return d, nil
}

func getTimeCurrentTimeStamp(ctx sessionctx.Context, tp byte, fsp int8) (t types.Time, err error) {
value := types.NewTime(types.ZeroCoreTime, tp, fsp)
defaultTime, err := getStmtTimestamp(ctx)
if err != nil {
return value, err
}
value.SetCoreTime(types.FromGoTime(defaultTime.Truncate(time.Duration(math.Pow10(9-int(fsp))) * time.Nanosecond)))
if tp == mysql.TypeTimestamp || tp == mysql.TypeDatetime {
err = value.ConvertTimeZone(time.Local, ctx.GetSessionVars().Location())
if err != nil {
return value, err
}
}
return value, nil
}

// GetTimeValue gets the time value with type tp.
func GetTimeValue(ctx sessionctx.Context, v interface{}, tp byte, fsp int8) (d types.Datum, err error) {
value := types.NewTime(types.ZeroCoreTime, tp, fsp)
var value types.Time

sc := ctx.GetSessionVars().StmtCtx
switch x := v.(type) {
case string:
upperX := strings.ToUpper(x)
if upperX == strings.ToUpper(ast.CurrentTimestamp) {
defaultTime, err := getStmtTimestamp(ctx)
if err != nil {
if value, err = getTimeCurrentTimeStamp(ctx, tp, fsp); err != nil {
return d, err
}
value.SetCoreTime(types.FromGoTime(defaultTime.Truncate(time.Duration(math.Pow10(9-int(fsp))) * time.Nanosecond)))
if tp == mysql.TypeTimestamp || tp == mysql.TypeDatetime {
err = value.ConvertTimeZone(time.Local, ctx.GetSessionVars().Location())
if err != nil {
return d, err
}
}
} else if upperX == types.ZeroDatetimeStr {
value, err = types.ParseTimeFromNum(sc, 0, tp, fsp)
terror.Log(err)
Expand Down
12 changes: 12 additions & 0 deletions expression/helper_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,18 @@ func TestGetTimeValue(t *testing.T) {
timeValue = v.GetMysqlTime()
require.Equal(t, "2012-12-12 00:00:00", timeValue.String())

// trigger the stmt context cache.
err = variable.SetSessionSystemVar(sessionVars, "timestamp", "0")
require.NoError(t, err)

v1, err := GetTimeCurrentTimestamp(ctx, mysql.TypeTimestamp, types.MinFsp)
require.NoError(t, err)

v2, err := GetTimeCurrentTimestamp(ctx, mysql.TypeTimestamp, types.MinFsp)
require.NoError(t, err)

require.Equal(t, v1, v2)

err = variable.SetSessionSystemVar(sessionVars, "timestamp", "1234")
require.NoError(t, err)

Expand Down
1 change: 1 addition & 0 deletions parser/model/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -207,6 +207,7 @@ type DDLReorgMeta struct {
SQLMode mysql.SQLMode `json:"sql_mode"`
Warnings map[errors.ErrorID]*terror.Error `json:"warnings"`
WarningsCount map[errors.ErrorID]int64 `json:"warnings_count"`
Location *time.Location `json:"time_location"`
}

// NewDDLReorgMeta new a DDLReorgMeta.
Expand Down

0 comments on commit 2844660

Please sign in to comment.