Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

executor: fix insert ignore into not exists partition (#21904) #21971

Merged
merged 2 commits into from
Jan 26, 2021
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
11 changes: 10 additions & 1 deletion executor/batch_checker.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,10 @@ import (
"context"
"strconv"

"github.com/pingcap/errors"
"github.com/pingcap/parser/model"
"github.com/pingcap/parser/terror"
"github.com/pingcap/tidb/errno"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/sessionctx"
Expand All @@ -38,7 +41,8 @@ type toBeCheckedRow struct {
handleKey *keyValueWithDupInfo
uniqueKeys []*keyValueWithDupInfo
// t is the table or partition this row belongs to.
t table.Table
t table.Table
ignored bool
}

// encodeNewRow encodes a new row to value.
Expand Down Expand Up @@ -96,6 +100,11 @@ func getKeysNeedCheckOneRow(ctx sessionctx.Context, t table.Table, row []types.D
if p, ok := t.(table.PartitionedTable); ok {
t, err = p.GetPartitionByRow(ctx, row)
if err != nil {
if terr, ok := errors.Cause(err).(*terror.Error); ctx.GetSessionVars().StmtCtx.IgnoreNoPartition && ok && terr.Code() == errno.ErrNoPartitionForGivenValue {
ctx.GetSessionVars().StmtCtx.AppendWarning(err)
result = append(result, toBeCheckedRow{ignored: true})
return result, nil
}
return nil, err
}
}
Expand Down
1 change: 1 addition & 0 deletions executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -1691,6 +1691,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) {
// but should not make DupKeyAsWarning or BadNullAsWarning,
sc.DupKeyAsWarning = stmt.IgnoreErr
sc.BadNullAsWarning = stmt.IgnoreErr
sc.IgnoreNoPartition = stmt.IgnoreErr
sc.TruncateAsWarning = !vars.StrictSQLMode || stmt.IgnoreErr
sc.DividedByZeroAsWarning = !vars.StrictSQLMode || stmt.IgnoreErr
sc.AllowInvalidDate = vars.SQLMode.HasAllowInvalidDatesMode()
Expand Down
6 changes: 6 additions & 0 deletions executor/insert.go
Original file line number Diff line number Diff line change
Expand Up @@ -113,13 +113,19 @@ func prefetchUniqueIndices(ctx context.Context, txn kv.Transaction, rows []toBeC

nKeys := 0
for _, r := range rows {
if r.ignored {
continue
}
if r.handleKey != nil {
nKeys++
}
nKeys += len(r.uniqueKeys)
}
batchKeys := make([]kv.Key, 0, nKeys)
for _, r := range rows {
if r.ignored {
continue
}
if r.handleKey != nil {
batchKeys = append(batchKeys, r.handleKey.newKey)
}
Expand Down
3 changes: 3 additions & 0 deletions executor/insert_common.go
Original file line number Diff line number Diff line change
Expand Up @@ -1028,6 +1028,9 @@ func (e *InsertValues) batchCheckAndInsert(ctx context.Context, rows [][]types.D

// append warnings and get no duplicated error rows
for i, r := range toBeCheckedRows {
if r.ignored {
continue
}
skip := false
if r.handleKey != nil {
_, err := txn.Get(ctx, r.handleKey.newKey)
Expand Down
4 changes: 4 additions & 0 deletions executor/write_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -569,6 +569,10 @@ commit;`
tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1048 Column 'i' cannot be null"))
testSQL = `select * from badnull`
tk.MustQuery(testSQL).Check(testkit.Rows("0"))

tk.MustExec("create table tp (id int) partition by range (id) (partition p0 values less than (1), partition p1 values less than(2))")
tk.MustExec("insert ignore into tp values (1), (3)")
tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1526 Table has no partition for value 3"))
}

func (s *testSuite8) TestInsertOnDup(c *C) {
Expand Down
1 change: 1 addition & 0 deletions sessionctx/stmtctx/stmtctx.go
Original file line number Diff line number Diff line change
Expand Up @@ -80,6 +80,7 @@ type StatementContext struct {
InNullRejectCheck bool
AllowInvalidDate bool
OptimDependOnMutableConst bool
IgnoreNoPartition bool

// mu struct holds variables that change during execution.
mu struct {
Expand Down