Skip to content

Commit

Permalink
*: disable insert null to not-null column for single-row insertion in…
Browse files Browse the repository at this point in the history
… non-strict mode (#55477)

close #55457, close #56381
  • Loading branch information
joechenrh authored Nov 19, 2024
1 parent 4e47aad commit 91beef4
Show file tree
Hide file tree
Showing 23 changed files with 152 additions and 23 deletions.
5 changes: 3 additions & 2 deletions pkg/ddl/reorg.go
Original file line number Diff line number Diff line change
Expand Up @@ -257,8 +257,9 @@ func reorgTypeFlagsWithSQLMode(mode mysql.SQLMode) types.Flags {

func reorgErrLevelsWithSQLMode(mode mysql.SQLMode) errctx.LevelMap {
return errctx.LevelMap{
errctx.ErrGroupTruncate: errctx.ResolveErrLevel(false, !mode.HasStrictMode()),
errctx.ErrGroupBadNull: errctx.ResolveErrLevel(false, !mode.HasStrictMode()),
errctx.ErrGroupTruncate: errctx.ResolveErrLevel(false, !mode.HasStrictMode()),
errctx.ErrGroupBadNull: errctx.ResolveErrLevel(false, !mode.HasStrictMode()),
errctx.ErrGroupNoDefault: errctx.ResolveErrLevel(false, !mode.HasStrictMode()),
errctx.ErrGroupDividedByZero: errctx.ResolveErrLevel(
!mode.HasErrorForDivisionByZeroMode(),
!mode.HasStrictMode(),
Expand Down
4 changes: 4 additions & 0 deletions pkg/errctx/context.go
Original file line number Diff line number Diff line change
Expand Up @@ -196,6 +196,8 @@ const (
ErrGroupDupKey
// ErrGroupBadNull is the group of bad null errors
ErrGroupBadNull
// ErrGroupNoDefault is the group of no default value errors
ErrGroupNoDefault
// ErrGroupDividedByZero is the group of divided by zero errors
ErrGroupDividedByZero
// ErrGroupAutoIncReadFailed is the group of auto increment read failed errors
Expand Down Expand Up @@ -223,6 +225,8 @@ func init() {
ErrGroupBadNull: {
errno.ErrBadNull,
errno.ErrWarnNullToNotnull,
},
ErrGroupNoDefault: {
errno.ErrNoDefaultForField,
},
ErrGroupDividedByZero: {
Expand Down
1 change: 1 addition & 0 deletions pkg/executor/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -441,6 +441,7 @@ go_test(
"//pkg/store/helper",
"//pkg/store/mockstore",
"//pkg/store/mockstore/unistore",
"//pkg/table",
"//pkg/table/tables",
"//pkg/tablecodec",
"//pkg/testkit",
Expand Down
10 changes: 10 additions & 0 deletions pkg/executor/executor_pkg_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -348,6 +348,7 @@ func TestErrLevelsForResetStmtContext(t *testing.T) {
l[errctx.ErrGroupTruncate] = errctx.LevelError
l[errctx.ErrGroupDupKey] = errctx.LevelError
l[errctx.ErrGroupBadNull] = errctx.LevelError
l[errctx.ErrGroupNoDefault] = errctx.LevelError
l[errctx.ErrGroupDividedByZero] = errctx.LevelError
l[errctx.ErrGroupAutoIncReadFailed] = errctx.LevelError
l[errctx.ErrGroupNoMatchedPartition] = errctx.LevelError
Expand All @@ -362,6 +363,7 @@ func TestErrLevelsForResetStmtContext(t *testing.T) {
l[errctx.ErrGroupTruncate] = errctx.LevelWarn
l[errctx.ErrGroupDupKey] = errctx.LevelError
l[errctx.ErrGroupBadNull] = errctx.LevelWarn
l[errctx.ErrGroupNoDefault] = errctx.LevelWarn
l[errctx.ErrGroupDividedByZero] = errctx.LevelWarn
l[errctx.ErrGroupAutoIncReadFailed] = errctx.LevelError
l[errctx.ErrGroupNoMatchedPartition] = errctx.LevelError
Expand All @@ -376,6 +378,7 @@ func TestErrLevelsForResetStmtContext(t *testing.T) {
l[errctx.ErrGroupTruncate] = errctx.LevelWarn
l[errctx.ErrGroupDupKey] = errctx.LevelWarn
l[errctx.ErrGroupBadNull] = errctx.LevelWarn
l[errctx.ErrGroupNoDefault] = errctx.LevelWarn
l[errctx.ErrGroupDividedByZero] = errctx.LevelWarn
l[errctx.ErrGroupAutoIncReadFailed] = errctx.LevelWarn
l[errctx.ErrGroupNoMatchedPartition] = errctx.LevelWarn
Expand All @@ -390,6 +393,7 @@ func TestErrLevelsForResetStmtContext(t *testing.T) {
l[errctx.ErrGroupTruncate] = errctx.LevelWarn
l[errctx.ErrGroupDupKey] = errctx.LevelWarn
l[errctx.ErrGroupBadNull] = errctx.LevelWarn
l[errctx.ErrGroupNoDefault] = errctx.LevelWarn
l[errctx.ErrGroupDividedByZero] = errctx.LevelWarn
l[errctx.ErrGroupAutoIncReadFailed] = errctx.LevelError
l[errctx.ErrGroupNoMatchedPartition] = errctx.LevelWarn
Expand All @@ -404,6 +408,7 @@ func TestErrLevelsForResetStmtContext(t *testing.T) {
l[errctx.ErrGroupTruncate] = errctx.LevelWarn
l[errctx.ErrGroupDupKey] = errctx.LevelWarn
l[errctx.ErrGroupBadNull] = errctx.LevelWarn
l[errctx.ErrGroupNoDefault] = errctx.LevelWarn
l[errctx.ErrGroupDividedByZero] = errctx.LevelWarn
l[errctx.ErrGroupAutoIncReadFailed] = errctx.LevelError
l[errctx.ErrGroupNoMatchedPartition] = errctx.LevelError
Expand All @@ -418,6 +423,7 @@ func TestErrLevelsForResetStmtContext(t *testing.T) {
l[errctx.ErrGroupTruncate] = errctx.LevelError
l[errctx.ErrGroupDupKey] = errctx.LevelError
l[errctx.ErrGroupBadNull] = errctx.LevelError
l[errctx.ErrGroupNoDefault] = errctx.LevelError
l[errctx.ErrGroupDividedByZero] = errctx.LevelIgnore
l[errctx.ErrGroupAutoIncReadFailed] = errctx.LevelError
l[errctx.ErrGroupNoMatchedPartition] = errctx.LevelError
Expand All @@ -432,6 +438,7 @@ func TestErrLevelsForResetStmtContext(t *testing.T) {
l[errctx.ErrGroupTruncate] = errctx.LevelWarn
l[errctx.ErrGroupDupKey] = errctx.LevelError
l[errctx.ErrGroupBadNull] = errctx.LevelError
l[errctx.ErrGroupNoDefault] = errctx.LevelError
l[errctx.ErrGroupDividedByZero] = errctx.LevelWarn
l[errctx.ErrGroupAutoIncReadFailed] = errctx.LevelError
l[errctx.ErrGroupNoMatchedPartition] = errctx.LevelError
Expand All @@ -446,6 +453,7 @@ func TestErrLevelsForResetStmtContext(t *testing.T) {
l[errctx.ErrGroupTruncate] = errctx.LevelWarn
l[errctx.ErrGroupDupKey] = errctx.LevelError
l[errctx.ErrGroupBadNull] = errctx.LevelError
l[errctx.ErrGroupNoDefault] = errctx.LevelError
l[errctx.ErrGroupDividedByZero] = errctx.LevelWarn
l[errctx.ErrGroupAutoIncReadFailed] = errctx.LevelError
l[errctx.ErrGroupNoMatchedPartition] = errctx.LevelError
Expand All @@ -460,6 +468,7 @@ func TestErrLevelsForResetStmtContext(t *testing.T) {
l[errctx.ErrGroupTruncate] = errctx.LevelError
l[errctx.ErrGroupDupKey] = errctx.LevelError
l[errctx.ErrGroupBadNull] = errctx.LevelError
l[errctx.ErrGroupNoDefault] = errctx.LevelError
l[errctx.ErrGroupDividedByZero] = errctx.LevelWarn
l[errctx.ErrGroupAutoIncReadFailed] = errctx.LevelError
l[errctx.ErrGroupNoMatchedPartition] = errctx.LevelWarn
Expand All @@ -474,6 +483,7 @@ func TestErrLevelsForResetStmtContext(t *testing.T) {
l[errctx.ErrGroupTruncate] = errctx.LevelError
l[errctx.ErrGroupDupKey] = errctx.LevelError
l[errctx.ErrGroupBadNull] = errctx.LevelError
l[errctx.ErrGroupNoDefault] = errctx.LevelError
l[errctx.ErrGroupDividedByZero] = errctx.LevelWarn
l[errctx.ErrGroupAutoIncReadFailed] = errctx.LevelError
l[errctx.ErrGroupNoMatchedPartition] = errctx.LevelWarn
Expand Down
28 changes: 28 additions & 0 deletions pkg/executor/insert_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
"github.com/pingcap/tidb/pkg/executor"
"github.com/pingcap/tidb/pkg/meta/autoid"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
"github.com/pingcap/tidb/pkg/table"
"github.com/pingcap/tidb/pkg/testkit"
"github.com/pingcap/tidb/pkg/util"
"github.com/pingcap/tidb/pkg/util/execdetails"
Expand Down Expand Up @@ -657,3 +658,30 @@ func TestMySQLInsertID(t *testing.T) {
tk.MustExec("insert into tb(a, b) values(1,2) on duplicate key update b = 2;")
require.Equal(t, tk.Session().LastInsertID(), uint64(0))
}

func TestInsertNullInNonStrictMode(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("create table t1 (id int primary key, col1 varchar(10) not null default '')")
tk.MustExec("create table t2 (id int primary key, col1 varchar(10))")
tk.MustExec("insert into t2 values (1, null)")
tk.MustExec("insert ignore into t1 values(5, null)")

tk.MustExec("set session sql_mode = ''")

err := tk.ExecToErr("insert into t1 values(1, null)")
require.EqualError(t, err, table.ErrColumnCantNull.GenWithStackByArgs("col1").Error())

err = tk.ExecToErr("insert into t1 set id = 1, col1 = null")
require.EqualError(t, err, table.ErrColumnCantNull.GenWithStackByArgs("col1").Error())

err = tk.ExecToErr("insert t1 VALUES (5, 5) ON DUPLICATE KEY UPDATE col1 = null")
require.EqualError(t, err, table.ErrColumnCantNull.GenWithStackByArgs("col1").Error())

tk.MustExec("insert into t1 select * from t2")
tk.MustExec("insert into t1 values(2, null), (3, 3), (4, 4)")
tk.MustExec("update t1 set col1 = null where id = 3")
tk.MustExec("insert ignore t1 VALUES (4, 4) ON DUPLICATE KEY UPDATE col1 = null")
tk.MustQuery("select * from t1").Check(testkit.RowsWithSep("|", "1|", "2|", "3|", "4|", "5|"))
}
1 change: 1 addition & 0 deletions pkg/executor/load_data.go
Original file line number Diff line number Diff line change
Expand Up @@ -153,6 +153,7 @@ func setNonRestrictiveFlags(stmtCtx *stmtctx.StatementContext) {
levels := stmtCtx.ErrLevels()
levels[errctx.ErrGroupDupKey] = errctx.LevelWarn
levels[errctx.ErrGroupBadNull] = errctx.LevelWarn
levels[errctx.ErrGroupNoDefault] = errctx.LevelWarn
stmtCtx.SetErrLevels(levels)
stmtCtx.SetTypeFlags(stmtCtx.TypeFlags().WithTruncateAsWarning(true))
}
Expand Down
8 changes: 7 additions & 1 deletion pkg/executor/select.go
Original file line number Diff line number Diff line change
Expand Up @@ -1067,7 +1067,11 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) {
errLevels[errctx.ErrGroupAutoIncReadFailed] = errctx.LevelWarn
errLevels[errctx.ErrGroupNoMatchedPartition] = errctx.LevelWarn
}
errLevels[errctx.ErrGroupBadNull] = errctx.ResolveErrLevel(false, !strictSQLMode || stmt.IgnoreErr)
// For single-row INSERT statements, ignore non-strict mode
// See https://dev.mysql.com/doc/refman/5.7/en/constraint-invalid-data.html
isSingleInsert := len(stmt.Lists) == 1
errLevels[errctx.ErrGroupBadNull] = errctx.ResolveErrLevel(false, (!strictSQLMode && !isSingleInsert) || stmt.IgnoreErr)
errLevels[errctx.ErrGroupNoDefault] = errctx.ResolveErrLevel(false, !strictSQLMode || stmt.IgnoreErr)
errLevels[errctx.ErrGroupDividedByZero] = errctx.ResolveErrLevel(
!vars.SQLMode.HasErrorForDivisionByZeroMode(),
!strictSQLMode || stmt.IgnoreErr,
Expand Down Expand Up @@ -1212,6 +1216,7 @@ func ResetUpdateStmtCtx(sc *stmtctx.StatementContext, stmt *ast.UpdateStmt, vars
errLevels := sc.ErrLevels()
errLevels[errctx.ErrGroupDupKey] = errctx.ResolveErrLevel(false, stmt.IgnoreErr)
errLevels[errctx.ErrGroupBadNull] = errctx.ResolveErrLevel(false, !strictSQLMode || stmt.IgnoreErr)
errLevels[errctx.ErrGroupNoDefault] = errLevels[errctx.ErrGroupBadNull]
errLevels[errctx.ErrGroupDividedByZero] = errctx.ResolveErrLevel(
!vars.SQLMode.HasErrorForDivisionByZeroMode(),
!strictSQLMode || stmt.IgnoreErr,
Expand All @@ -1233,6 +1238,7 @@ func ResetDeleteStmtCtx(sc *stmtctx.StatementContext, stmt *ast.DeleteStmt, vars
errLevels := sc.ErrLevels()
errLevels[errctx.ErrGroupDupKey] = errctx.ResolveErrLevel(false, stmt.IgnoreErr)
errLevels[errctx.ErrGroupBadNull] = errctx.ResolveErrLevel(false, !strictSQLMode || stmt.IgnoreErr)
errLevels[errctx.ErrGroupNoDefault] = errLevels[errctx.ErrGroupBadNull]
errLevels[errctx.ErrGroupDividedByZero] = errctx.ResolveErrLevel(
!vars.SQLMode.HasErrorForDivisionByZeroMode(),
!strictSQLMode || stmt.IgnoreErr,
Expand Down
1 change: 1 addition & 0 deletions pkg/executor/test/writetest/write_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -240,6 +240,7 @@ func TestIssue18681(t *testing.T) {
levels := ctx.GetSessionVars().StmtCtx.ErrLevels()
levels[errctx.ErrGroupDupKey] = errctx.LevelWarn
levels[errctx.ErrGroupBadNull] = errctx.LevelWarn
levels[errctx.ErrGroupNoDefault] = errctx.LevelWarn

sc := ctx.GetSessionVars().StmtCtx
oldTypeFlags := sc.TypeFlags()
Expand Down
2 changes: 2 additions & 0 deletions pkg/expression/builtin_miscellaneous_vec_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -154,6 +154,7 @@ func TestSleepVectorized(t *testing.T) {
// non-strict model
var levels errctx.LevelMap
levels[errctx.ErrGroupBadNull] = errctx.LevelWarn
levels[errctx.ErrGroupNoDefault] = errctx.LevelWarn
sessVars.StmtCtx.SetErrLevels(levels)
input.AppendFloat64(0, 1)
err = vecEvalType(ctx, f, types.ETInt, input, result)
Expand Down Expand Up @@ -188,6 +189,7 @@ func TestSleepVectorized(t *testing.T) {

// for error case under the strict model
levels[errctx.ErrGroupBadNull] = errctx.LevelError
levels[errctx.ErrGroupNoDefault] = errctx.LevelError
sessVars.StmtCtx.SetErrLevels(levels)
input.Reset()
input.AppendNull(0)
Expand Down
2 changes: 2 additions & 0 deletions pkg/expression/evaluator_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -108,6 +108,7 @@ func TestSleep(t *testing.T) {
// non-strict model
var levels errctx.LevelMap
levels[errctx.ErrGroupBadNull] = errctx.LevelWarn
levels[errctx.ErrGroupNoDefault] = errctx.LevelWarn
sessVars.StmtCtx.SetErrLevels(levels)
d := make([]types.Datum, 1)
f, err := fc.getFunction(ctx, datumsToConstants(d))
Expand All @@ -128,6 +129,7 @@ func TestSleep(t *testing.T) {

// for error case under the strict model
levels[errctx.ErrGroupBadNull] = errctx.LevelError
levels[errctx.ErrGroupNoDefault] = errctx.LevelError
sessVars.StmtCtx.SetErrLevels(levels)
d[0].SetNull()
_, err = fc.getFunction(ctx, datumsToConstants(d))
Expand Down
2 changes: 2 additions & 0 deletions pkg/expression/exprstatic/evalctx_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -110,6 +110,7 @@ func getEvalCtxOptionsForTest(t *testing.T) ([]EvalCtxOption, *evalCtxOptionsTes
WithTypeFlags(types.FlagAllowNegativeToUnsigned | types.FlagSkipASCIICheck),
WithErrLevelMap(errctx.LevelMap{
errctx.ErrGroupBadNull: errctx.LevelError,
errctx.ErrGroupNoDefault: errctx.LevelError,
errctx.ErrGroupDividedByZero: errctx.LevelWarn,
}),
WithLocation(loc),
Expand All @@ -134,6 +135,7 @@ func checkOptionsStaticEvalCtx(t *testing.T, ctx *EvalContext, s *evalCtxOptions
)
require.Equal(t, errctx.NewContextWithLevels(errctx.LevelMap{
errctx.ErrGroupBadNull: errctx.LevelError,
errctx.ErrGroupNoDefault: errctx.LevelError,
errctx.ErrGroupDividedByZero: errctx.LevelWarn,
}, ctx), ctx.ErrCtx())
require.Same(t, s.loc, ctx.Location())
Expand Down
6 changes: 5 additions & 1 deletion pkg/expression/sessionexpr/sessionctx_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -54,7 +54,11 @@ func TestSessionEvalContextBasic(t *testing.T) {
ctx.ResetSessionAndStmtTimeZone(time.FixedZone("UTC+11", 11*3600))
vars.SQLMode = mysql.ModeStrictTransTables | mysql.ModeNoZeroDate
sc.SetTypeFlags(types.FlagIgnoreInvalidDateErr | types.FlagSkipUTF8Check)
sc.SetErrLevels(errctx.LevelMap{errctx.ErrGroupDupKey: errctx.LevelWarn, errctx.ErrGroupBadNull: errctx.LevelIgnore})
sc.SetErrLevels(errctx.LevelMap{
errctx.ErrGroupDupKey: errctx.LevelWarn,
errctx.ErrGroupBadNull: errctx.LevelIgnore,
errctx.ErrGroupNoDefault: errctx.LevelIgnore,
})
vars.CurrentDB = "db1"
vars.MaxAllowedPacket = 123456

Expand Down
1 change: 1 addition & 0 deletions pkg/lightning/backend/kv/context.go
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,7 @@ func newLitExprContext(sqlMode mysql.SQLMode, sysVars map[string]string, timesta
errLevels := stmtctx.DefaultStmtErrLevels
errLevels[errctx.ErrGroupTruncate] = errctx.ResolveErrLevel(flags.IgnoreTruncateErr(), flags.TruncateAsWarning())
errLevels[errctx.ErrGroupBadNull] = errctx.ResolveErrLevel(false, !sqlMode.HasStrictMode())
errLevels[errctx.ErrGroupNoDefault] = errctx.ResolveErrLevel(false, !sqlMode.HasStrictMode())
errLevels[errctx.ErrGroupDividedByZero] =
errctx.ResolveErrLevel(!sqlMode.HasErrorForDivisionByZeroMode(), !sqlMode.HasStrictMode())

Expand Down
6 changes: 6 additions & 0 deletions pkg/lightning/backend/kv/context_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,7 @@ func TestLitExprContext(t *testing.T) {
m := stmtctx.DefaultStmtErrLevels
m[errctx.ErrGroupTruncate] = errctx.LevelWarn
m[errctx.ErrGroupBadNull] = errctx.LevelWarn
m[errctx.ErrGroupNoDefault] = errctx.LevelWarn
m[errctx.ErrGroupDividedByZero] = errctx.LevelIgnore
return m
}(),
Expand All @@ -72,6 +73,7 @@ func TestLitExprContext(t *testing.T) {
m := stmtctx.DefaultStmtErrLevels
m[errctx.ErrGroupTruncate] = errctx.LevelError
m[errctx.ErrGroupBadNull] = errctx.LevelError
m[errctx.ErrGroupNoDefault] = errctx.LevelError
m[errctx.ErrGroupDividedByZero] = errctx.LevelError
return m
}(),
Expand All @@ -83,6 +85,7 @@ func TestLitExprContext(t *testing.T) {
m := stmtctx.DefaultStmtErrLevels
m[errctx.ErrGroupTruncate] = errctx.LevelWarn
m[errctx.ErrGroupBadNull] = errctx.LevelWarn
m[errctx.ErrGroupNoDefault] = errctx.LevelWarn
m[errctx.ErrGroupDividedByZero] = errctx.LevelWarn
return m
}(),
Expand All @@ -94,6 +97,7 @@ func TestLitExprContext(t *testing.T) {
m := stmtctx.DefaultStmtErrLevels
m[errctx.ErrGroupTruncate] = errctx.LevelError
m[errctx.ErrGroupBadNull] = errctx.LevelError
m[errctx.ErrGroupNoDefault] = errctx.LevelError
m[errctx.ErrGroupDividedByZero] = errctx.LevelIgnore
return m
}(),
Expand All @@ -105,6 +109,7 @@ func TestLitExprContext(t *testing.T) {
m := stmtctx.DefaultStmtErrLevels
m[errctx.ErrGroupTruncate] = errctx.LevelError
m[errctx.ErrGroupBadNull] = errctx.LevelError
m[errctx.ErrGroupNoDefault] = errctx.LevelError
m[errctx.ErrGroupDividedByZero] = errctx.LevelIgnore
return m
}(),
Expand All @@ -116,6 +121,7 @@ func TestLitExprContext(t *testing.T) {
m := stmtctx.DefaultStmtErrLevels
m[errctx.ErrGroupTruncate] = errctx.LevelError
m[errctx.ErrGroupBadNull] = errctx.LevelError
m[errctx.ErrGroupNoDefault] = errctx.LevelError
m[errctx.ErrGroupDividedByZero] = errctx.LevelIgnore
return m
}(),
Expand Down
2 changes: 1 addition & 1 deletion pkg/lightning/errormanager/errormanager.go
Original file line number Diff line number Diff line change
Expand Up @@ -150,7 +150,7 @@ const (
`

insertIntoConflictErrorData = `
INSERT INTO %s.` + ConflictErrorTableName + `
INSERT IGNORE INTO %s.` + ConflictErrorTableName + `
(task_id, table_name, index_name, key_data, row_data, raw_key, raw_value, raw_handle, raw_row, kv_type)
VALUES
`
Expand Down
2 changes: 1 addition & 1 deletion pkg/lightning/errormanager/errormanager_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -497,7 +497,7 @@ func TestReplaceConflictOneUniqueKey(t *testing.T) {
AddRow(3, data3IndexKey, "uni_b", data3IndexValue, data3RowKey).
AddRow(4, data3IndexKey, "uni_b", data4IndexValue, data4RowKey))
mockDB.ExpectBegin()
mockDB.ExpectExec("INSERT INTO `lightning_task_info`\\.conflict_error_v3.*").
mockDB.ExpectExec("INSERT IGNORE INTO `lightning_task_info`\\.conflict_error_v3.*").
WithArgs(0, "test", nil, nil, data2RowKey, data2RowValue, 2,
0, "test", nil, nil, data4RowKey, data4RowValue, 2).
WillReturnResult(driver.ResultNoRows)
Expand Down
8 changes: 4 additions & 4 deletions pkg/lightning/errormanager/resolveconflict_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -178,7 +178,7 @@ func TestReplaceConflictMultipleKeysNonclusteredPk(t *testing.T) {
AddRow(3, data6RowKey, "PRIMARY", data6RowValue, data5RowKey).
AddRow(4, data6RowKey, "PRIMARY", data7NonclusteredValue, data6NonclusteredKey))
mockDB.ExpectBegin()
mockDB.ExpectExec("INSERT INTO `lightning_task_info`\\.conflict_error_v3.*").
mockDB.ExpectExec("INSERT IGNORE INTO `lightning_task_info`\\.conflict_error_v3.*").
WithArgs(0, "a", nil, nil, data2NonclusteredKey, data2NonclusteredValue, 2,
0, "a", nil, nil, data6NonclusteredKey, data6NonclusteredValue, 2).
WillReturnResult(driver.ResultNoRows)
Expand Down Expand Up @@ -361,7 +361,7 @@ func TestReplaceConflictOneKeyNonclusteredPk(t *testing.T) {
AddRow(1, data3IndexKey, "PRIMARY", data3IndexValue, data3RowKey).
AddRow(2, data3IndexKey, "PRIMARY", data4IndexValue, data4RowKey))
mockDB.ExpectBegin()
mockDB.ExpectExec("INSERT INTO `lightning_task_info`\\.conflict_error_v3.*").
mockDB.ExpectExec("INSERT IGNORE INTO `lightning_task_info`\\.conflict_error_v3.*").
WithArgs(0, "a", nil, nil, data4RowKey, data4RowValue, 2).
WillReturnResult(driver.ResultNoRows)
mockDB.ExpectCommit()
Expand Down Expand Up @@ -547,7 +547,7 @@ func TestReplaceConflictOneUniqueKeyNonclusteredPk(t *testing.T) {
AddRow(5, data3IndexKey, "PRIMARY", data3IndexValue, data3RowKey).
AddRow(6, data3IndexKey, "PRIMARY", data4NonclusteredValue, data4RowKey))
mockDB.ExpectBegin()
mockDB.ExpectExec("INSERT INTO `lightning_task_info`\\.conflict_error_v3.*").
mockDB.ExpectExec("INSERT IGNORE INTO `lightning_task_info`\\.conflict_error_v3.*").
WithArgs(0, "a", nil, nil, data5RowKey, data5RowValue, 2,
0, "a", nil, nil, data2RowKey, data2RowValue, 2,
0, "a", nil, nil, data4RowKey, data4RowValue, 2).
Expand Down Expand Up @@ -754,7 +754,7 @@ func TestReplaceConflictOneUniqueKeyNonclusteredVarcharPk(t *testing.T) {
AddRow(5, data3IndexKey, "PRIMARY", data3IndexValue, data3RowKey).
AddRow(6, data3IndexKey, "PRIMARY", data4IndexValue, data4RowKey))
mockDB.ExpectBegin()
mockDB.ExpectExec("INSERT INTO `lightning_task_info`\\.conflict_error_v3.*").
mockDB.ExpectExec("INSERT IGNORE INTO `lightning_task_info`\\.conflict_error_v3.*").
WithArgs(0, "a", nil, nil, data5RowKey, data5RowValue, 2,
0, "a", nil, nil, data2RowKey, data2RowValue, 2,
0, "a", nil, nil, data4RowKey, data4RowValue, 2).
Expand Down
7 changes: 5 additions & 2 deletions pkg/table/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -511,7 +511,10 @@ func (c *Column) CheckNotNull(data *types.Datum, rowCntInLoadData uint64) error
// error is ErrWarnNullToNotnull.
// Otherwise, the error is ErrColumnCantNull.
// If BadNullAsWarning is true, it will append the error as a warning, else return the error.
func (c *Column) HandleBadNull(ec errctx.Context, d *types.Datum, rowCntInLoadData uint64) error {
func (c *Column) HandleBadNull(
ec errctx.Context,
d *types.Datum,
rowCntInLoadData uint64) error {
if err := c.CheckNotNull(d, rowCntInLoadData); err != nil {
if ec.HandleError(err) == nil {
*d = GetZeroValue(c.ToInfo())
Expand Down Expand Up @@ -554,7 +557,7 @@ func GetColOriginDefaultValueWithoutStrictSQLMode(ctx expression.BuildContext, c
// But CheckNoDefaultValueForInsert logic should only check before insert.
func CheckNoDefaultValueForInsert(sc *stmtctx.StatementContext, col *model.ColumnInfo) error {
if mysql.HasNoDefaultValueFlag(col.GetFlag()) && !col.DefaultIsExpr && col.GetDefaultValue() == nil && col.GetType() != mysql.TypeEnum {
ignoreErr := sc.ErrGroupLevel(errctx.ErrGroupBadNull) != errctx.LevelError
ignoreErr := sc.ErrGroupLevel(errctx.ErrGroupNoDefault) != errctx.LevelError
if !ignoreErr {
return ErrNoDefaultValue.GenWithStackByArgs(col.Name)
}
Expand Down
Loading

0 comments on commit 91beef4

Please sign in to comment.