Skip to content

Commit

Permalink
executor: insert ignore return errAutoincReadFailed warning (#38486)
Browse files Browse the repository at this point in the history
close #38483
  • Loading branch information
hawkingrei authored Nov 7, 2022
1 parent 54ff6e3 commit f79e67f
Show file tree
Hide file tree
Showing 4 changed files with 56 additions and 32 deletions.
1 change: 1 addition & 0 deletions executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -2042,6 +2042,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) {
sc.DupKeyAsWarning = stmt.IgnoreErr
sc.BadNullAsWarning = !vars.StrictSQLMode || stmt.IgnoreErr
sc.IgnoreNoPartition = stmt.IgnoreErr
sc.ErrAutoincReadFailedAsWarning = stmt.IgnoreErr
sc.TruncateAsWarning = !vars.StrictSQLMode || stmt.IgnoreErr
sc.DividedByZeroAsWarning = !vars.StrictSQLMode || stmt.IgnoreErr
sc.AllowInvalidDate = vars.SQLMode.HasAllowInvalidDatesMode()
Expand Down
16 changes: 15 additions & 1 deletion executor/insert.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,11 +22,14 @@ import (
"time"

"github.com/opentracing/opentracing-go"
"github.com/pingcap/errors"
"github.com/pingcap/tidb/errno"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/meta/autoid"
"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/parser/mysql"
"github.com/pingcap/tidb/parser/terror"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/types"
Expand Down Expand Up @@ -310,7 +313,18 @@ func (e *InsertExec) Next(ctx context.Context, req *chunk.Chunk) error {
if len(e.children) > 0 && e.children[0] != nil {
return insertRowsFromSelect(ctx, e)
}
return insertRows(ctx, e)
err := insertRows(ctx, e)
if err != nil {
terr, ok := errors.Cause(err).(*terror.Error)
if ok && len(e.OnDuplicate) == 0 &&
e.ctx.GetSessionVars().StmtCtx.ErrAutoincReadFailedAsWarning &&
terr.Code() == errno.ErrAutoincReadFailed {
e.ctx.GetSessionVars().StmtCtx.AppendWarning(err)
return nil
}
return err
}
return nil
}

// Close implements the Executor Close interface.
Expand Down
8 changes: 8 additions & 0 deletions executor/insert_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -398,6 +398,14 @@ func TestInsertWrongValueForField(t *testing.T) {
tk.MustExec(`SET @@sql_mode='STRICT_TRANS_TABLES'`)
tk.MustGetErrMsg(`INSERT INTO ts (id, time1) VALUES (2, TIMESTAMP '1018-12-24 00:00:00')`, `[table:1292]Incorrect timestamp value: '1018-12-24 00:00:00' for column 'time1' at row 1`)
tk.MustExec(`DROP TABLE ts`)

tk.MustExec(`CREATE TABLE t0(c0 SMALLINT AUTO_INCREMENT PRIMARY KEY);`)
tk.MustExec(`INSERT IGNORE INTO t0(c0) VALUES (194626268);`)
tk.MustExec(`INSERT IGNORE INTO t0(c0) VALUES ('*')`)
tk.MustQuery(`SHOW WARNINGS`).Check(testkit.Rows(
`Warning 1366 Incorrect smallint value: '*' for column 'c0' at row 1`,
`Warning 1690 constant 32768 overflows smallint`,
`Warning 1467 Failed to read auto-increment value from storage engine`))
}

func TestInsertValueForCastDecimalField(t *testing.T) {
Expand Down
63 changes: 32 additions & 31 deletions sessionctx/stmtctx/stmtctx.go
Original file line number Diff line number Diff line change
Expand Up @@ -109,37 +109,38 @@ 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
DDLJobID int64
InInsertStmt bool
InUpdateStmt bool
InDeleteStmt bool
InSelectStmt bool
InLoadDataStmt bool
InExplainStmt bool
InCreateOrAlterStmt bool
InSetSessionStatesStmt bool
InPreparedPlanBuilding bool
IgnoreTruncate bool
IgnoreZeroInDate bool
NoZeroDate bool
DupKeyAsWarning bool
BadNullAsWarning bool
DividedByZeroAsWarning bool
TruncateAsWarning bool
OverflowAsWarning bool
InShowWarning bool
UseCache bool
BatchCheck bool
InNullRejectCheck bool
AllowInvalidDate bool
IgnoreNoPartition bool
SkipPlanCache bool
IgnoreExplainIDSuffix bool
SkipUTF8Check bool
SkipASCIICheck bool
SkipUTF8MB4Check bool
MultiSchemaInfo *model.MultiSchemaInfo
IsDDLJobInQueue bool
DDLJobID int64
InInsertStmt bool
InUpdateStmt bool
InDeleteStmt bool
InSelectStmt bool
InLoadDataStmt bool
InExplainStmt bool
InCreateOrAlterStmt bool
InSetSessionStatesStmt bool
InPreparedPlanBuilding bool
IgnoreTruncate bool
IgnoreZeroInDate bool
NoZeroDate bool
DupKeyAsWarning bool
BadNullAsWarning bool
DividedByZeroAsWarning bool
TruncateAsWarning bool
OverflowAsWarning bool
ErrAutoincReadFailedAsWarning bool
InShowWarning bool
UseCache bool
BatchCheck bool
InNullRejectCheck bool
AllowInvalidDate bool
IgnoreNoPartition bool
SkipPlanCache bool
IgnoreExplainIDSuffix bool
SkipUTF8Check bool
SkipASCIICheck bool
SkipUTF8MB4Check bool
MultiSchemaInfo *model.MultiSchemaInfo
// If the select statement was like 'select * from t as of timestamp ...' or in a stale read transaction
// or is affected by the tidb_read_staleness session variable, then the statement will be makred as isStaleness
// in stmtCtx
Expand Down

0 comments on commit f79e67f

Please sign in to comment.