*: 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:
@ -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(),
|
||||
|
||||
@ -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: {
|
||||
|
||||
@ -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",
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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|"))
|
||||
}
|
||||
|
||||
@ -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))
|
||||
}
|
||||
|
||||
@ -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,
|
||||
|
||||
@ -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()
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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))
|
||||
|
||||
@ -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())
|
||||
|
||||
@ -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
|
||||
|
||||
|
||||
@ -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())
|
||||
|
||||
|
||||
@ -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
|
||||
}(),
|
||||
|
||||
@ -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
|
||||
`
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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).
|
||||
|
||||
@ -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)
|
||||
}
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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;
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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;
|
||||
|
||||
@ -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;
|
||||
|
||||
Reference in New Issue
Block a user