*: disable insert null to not-null column for single-row insertion in non-strict mode (#55477)

close pingcap/tidb#55457, close pingcap/tidb#56381
This commit is contained in:
Ruihao Chen
2024-11-19 20:46:11 +08:00
committed by GitHub
parent 4e47aad9a1
commit 91beef4bb1
23 changed files with 152 additions and 23 deletions

View File

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

View File

@ -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
@ -223,6 +225,8 @@ func init() {
ErrGroupBadNull: {
errno.ErrBadNull,
errno.ErrWarnNullToNotnull,
},
ErrGroupNoDefault: {
errno.ErrNoDefaultForField,
},
ErrGroupDividedByZero: {

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -485,7 +485,7 @@ func TestGetDefaultValue(t *testing.T) {
ctx.GetSessionVars().SQLMode = mysql.DelSQLMode(defaultMode, mysql.ModeStrictAllTables|mysql.ModeStrictTransTables)
}
levels := sc.ErrLevels()
levels[errctx.ErrGroupBadNull] = errctx.ResolveErrLevel(false, !tt.strict)
levels[errctx.ErrGroupNoDefault] = errctx.ResolveErrLevel(false, !tt.strict)
sc.SetErrLevels(levels)
val, err := GetColDefaultValue(ctx, tt.colInfo)
if err != nil {
@ -507,7 +507,7 @@ func TestGetDefaultValue(t *testing.T) {
ctx.GetSessionVars().SQLMode = mysql.DelSQLMode(defaultMode, mysql.ModeStrictAllTables|mysql.ModeStrictTransTables)
}
levels := sc.ErrLevels()
levels[errctx.ErrGroupBadNull] = errctx.ResolveErrLevel(false, !tt.strict)
levels[errctx.ErrGroupNoDefault] = errctx.ResolveErrLevel(false, !tt.strict)
sc.SetErrLevels(levels)
val, err := GetColOriginDefaultValue(ctx, tt.colInfo)
if err != nil {

View File

@ -3821,9 +3821,7 @@ show warnings;
Level Code Message
Warning 1364 Field 'a' doesn't have a default value
insert t values (null);
show warnings;
Level Code Message
Warning 1048 Column 'a' cannot be null
Error 1048 (23000): Column 'a' cannot be null
insert ignore t values (null);
show warnings;
Level Code Message
@ -3838,7 +3836,6 @@ a
0
0
0
0
127
insert tdouble values (10.23);
select * from tdouble;

View File

@ -1866,9 +1866,7 @@ f1 f2
SELECT * FROM t1 order by f1;
f1 f2
1 0
2 2
SET sql_mode='';
INSERT t1 VALUES (1, 1) ON DUPLICATE KEY UPDATE f2 = null;
2 2
SET sql_mode='';
INSERT t1 VALUES (1, 1) ON DUPLICATE KEY UPDATE f2 = null;
Error 1048 (23000): Column 'f2' cannot be null
@ -2142,3 +2140,41 @@ Warning 1452 Cannot add or update a child row: a foreign key constraint fails (`
insert ignore into child values (1, 3) on duplicate key update ref = 3;
Level Code Message
Warning 1452 Cannot add or update a child row: a foreign key constraint fails (`executor__insert`.`child`, CONSTRAINT `fk_1` FOREIGN KEY (`ref`) REFERENCES `parent` (`ref`))
insert into parent values (2, 3) on duplicate key update ref = 3;
Error 1451 (23000): Cannot delete or update a parent row: a foreign key constraint fails (`executor__insert`.`child`, CONSTRAINT `fk_1` FOREIGN KEY (`ref`) REFERENCES `parent` (`ref`))
insert ignore into parent values (2, 3) on duplicate key update ref = 3;
drop table if exists t1, t2;
create table t1 (id int primary key, col1 varchar(10) not null default '');
create table t2 (id int primary key, col1 varchar(10));
insert into t2 values (1, null);
insert ignore into t1 values(5, null);
set session sql_mode = '';
insert into t1 values(1, null);
Error 1048 (23000): Column 'col1' cannot be null
insert into t1 set id = 1, col1 = null;
Error 1048 (23000): Column 'col1' cannot be null
insert t1 VALUES (5, 5) ON DUPLICATE KEY UPDATE col1 = null;
Error 1048 (23000): Column 'col1' cannot be null
insert t1 VALUES (5, 5), (6, null) ON DUPLICATE KEY UPDATE col1 = null;
select * from t1;
id col1
5
6
insert into t1 select * from t2;
show warnings;
Level Code Message
Warning 1048 Column 'col1' cannot be null
insert into t1 values(2, null), (3, 3), (4, 4);
show warnings;
Level Code Message
Warning 1048 Column 'col1' cannot be null
update t1 set col1 = null where id = 3;
show warnings;
Level Code Message
Warning 1048 Column 'col1' cannot be null
insert ignore t1 VALUES (4, 4) ON DUPLICATE KEY UPDATE col1 = null;
select * from t1;
id col1
1
2
3

View File

@ -2104,8 +2104,8 @@ insert tdouble values (10.23);
set sql_mode = '';
insert t values ();
show warnings;
-- error 1048
insert t values (null);
show warnings;
insert ignore t values (null);
show warnings;
insert t select null;

View File

@ -1407,8 +1407,8 @@ show warnings;
SELECT * FROM t1 order by f1;
SET sql_mode='';
-- error 1048
INSERT t1 VALUES (1, 1) ON DUPLICATE KEY UPDATE f2 = null;
show warnings;
SELECT * FROM t1 order by f1;
set sql_mode=default;
@ -1620,3 +1620,27 @@ insert into parent values (2, 3) on duplicate key update ref = 3;
--enable_warnings
insert ignore into parent values (2, 3) on duplicate key update ref = 3;
--disable_warnings
# TestIssue55457
drop table if exists t1, t2;
create table t1 (id int primary key, col1 varchar(10) not null default '');
create table t2 (id int primary key, col1 varchar(10));
insert into t2 values (1, null);
insert ignore into t1 values(5, null);
set session sql_mode = '';
-- error 1048
insert into t1 values(1, null);
-- error 1048
insert into t1 set id = 1, col1 = null;
-- error 1048
insert t1 VALUES (5, 5) ON DUPLICATE KEY UPDATE col1 = null;
insert t1 VALUES (5, 5), (6, null) ON DUPLICATE KEY UPDATE col1 = null;
select * from t1;
insert into t1 select * from t2;
show warnings;
insert into t1 values(2, null), (3, 3), (4, 4);
show warnings;
update t1 set col1 = null where id = 3;
show warnings;
insert ignore t1 VALUES (4, 4) ON DUPLICATE KEY UPDATE col1 = null;
select * from t1;