Skip to content

Commit

Permalink
planner: lazy the subquery-more-than-1-row error after the insert dup…
Browse files Browse the repository at this point in the history
…licate is actually triggered (pingcap#29270)
  • Loading branch information
AilinKid authored Nov 11, 2021
1 parent 58c02d5 commit 1eec131
Show file tree
Hide file tree
Showing 7 changed files with 51 additions and 4 deletions.
5 changes: 5 additions & 0 deletions errors.toml
Original file line number Diff line number Diff line change
Expand Up @@ -1231,6 +1231,11 @@ error = '''
Operand should contain %d column(s)
'''

["planner:1242"]
error = '''
Subquery returns more than 1 row
'''

["planner:1247"]
error = '''
Reference '%-.64s' not supported (%s)
Expand Down
3 changes: 3 additions & 0 deletions executor/insert.go
Original file line number Diff line number Diff line change
Expand Up @@ -389,6 +389,9 @@ func (e *InsertExec) doDupRowUpdate(ctx context.Context, handle kv.Handle, oldRo
// Update old row when the key is duplicated.
e.evalBuffer4Dup.SetDatums(e.row4Update...)
for _, col := range cols {
if col.LazyErr != nil {
return col.LazyErr
}
val, err1 := col.Expr.Eval(e.evalBuffer4Dup.ToRow())
if err1 != nil {
return err1
Expand Down
3 changes: 3 additions & 0 deletions expression/expression.go
Original file line number Diff line number Diff line change
Expand Up @@ -748,6 +748,9 @@ type Assignment struct {
// ColName indicates its original column name in table schema. It's used for outputting helping message when executing meets some errors.
ColName model.CIStr
Expr Expression
// LazyErr is used in statement like `INSERT INTO t1 (a) VALUES (1) ON DUPLICATE KEY UPDATE a= (SELECT b FROM source);`, ErrSubqueryMoreThan1Row
// should be evaluated after the duplicate situation is detected in the executing procedure.
LazyErr error
}

// VarAssignment represents a variable assignment in Set, such as set global a = 1.
Expand Down
1 change: 1 addition & 0 deletions planner/core/errors.go
Original file line number Diff line number Diff line change
Expand Up @@ -107,4 +107,5 @@ var (
// ErrPartitionNoTemporary returns when partition at temporary mode
ErrPartitionNoTemporary = dbterror.ClassOptimizer.NewStd(mysql.ErrPartitionNoTemporary)
ErrViewSelectTemporaryTable = dbterror.ClassOptimizer.NewStd(mysql.ErrViewSelectTmptable)
ErrSubqueryMoreThan1Row = dbterror.ClassOptimizer.NewStd(mysql.ErrSubqueryNo1Row)
)
26 changes: 26 additions & 0 deletions planner/core/expression_rewriter_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ package core_test

import (
. "github.com/pingcap/check"
"github.com/pingcap/tidb/parser/mysql"
"github.com/pingcap/tidb/parser/terror"
"github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/util/collate"
Expand Down Expand Up @@ -466,6 +467,31 @@ func (s *testExpressionRewriterSuiteSerial) TestBetweenExprCollation(c *C) {
tk.MustGetErrMsg("select * from t1 where a between 'B' collate utf8mb4_general_ci and c collate utf8mb4_unicode_ci;", "[expression:1270]Illegal mix of collations (latin1_bin,IMPLICIT), (utf8mb4_general_ci,EXPLICIT), (utf8mb4_unicode_ci,EXPLICIT) for operation 'BETWEEN'")
}

func (s *testExpressionRewriterSuite) TestInsertOnDuplicateLazyMoreThan1Row(c *C) {
defer testleak.AfterTest(c)()
store, dom, err := newStoreWithBootstrap()
c.Assert(err, IsNil)
tk := testkit.NewTestKit(c, store)
defer func() {
dom.Close()
store.Close()
}()

tk.MustExec("use test")
tk.MustExec("DROP TABLE if exists t1, t2, source;")
tk.MustExec("CREATE TABLE t1(a INTEGER PRIMARY KEY);")
tk.MustExec("CREATE TABLE t2(a INTEGER);")
tk.MustExec("CREATE TABLE source (b INTEGER);")
tk.MustExec("INSERT INTO t1 VALUES (1);")
tk.MustExec("INSERT INTO t2 VALUES (1);")
tk.MustExec("INSERT INTO source VALUES (1),(1);")
// the on duplicate is not triggered by t1's primary key.
tk.MustGetErrCode("INSERT INTO t1 (a) VALUES (1) ON DUPLICATE KEY UPDATE a= (SELECT b FROM source);", mysql.ErrSubqueryNo1Row)
// the on duplicate is not triggered.
tk.MustExec("INSERT INTO t2 (a) VALUES (1) ON DUPLICATE KEY UPDATE a= (SELECT b FROM source);")
tk.MustExec("DROP TABLE if exists t1, t2, source;")
}

func (s *testExpressionRewriterSuite) TestMultiColInExpression(c *C) {
store, dom, err := newStoreWithBootstrap()
c.Assert(err, IsNil)
Expand Down
8 changes: 7 additions & 1 deletion planner/core/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@ import (
"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/parser/mysql"
"github.com/pingcap/tidb/parser/opcode"
"github.com/pingcap/tidb/parser/terror"
"github.com/pingcap/tidb/planner/property"
"github.com/pingcap/tidb/planner/util"
"github.com/pingcap/tidb/privilege"
Expand Down Expand Up @@ -3138,13 +3139,18 @@ func (p *Insert) resolveOnDuplicate(onDup []*ast.Assignment, tblInfo *model.Tabl

expr, err := yield(assign.Expr)
if err != nil {
return nil, err
// Throw other error as soon as possible except ErrSubqueryMoreThan1Row which need duplicate in insert in triggered.
// Refer to https://github.com/pingcap/tidb/issues/29260 for more information.
if terr, ok := errors.Cause(err).(*terror.Error); !(ok && ErrSubqueryMoreThan1Row.Code() == terr.Code()) {
return nil, err
}
}

p.OnDuplicate = append(p.OnDuplicate, &expression.Assignment{
Col: p.tableSchema.Columns[idx],
ColName: p.tableColNames[idx].ColName,
Expr: expr,
LazyErr: err,
})
}
return onDupColSet, nil
Expand Down
9 changes: 6 additions & 3 deletions planner/core/resolve_indices.go
Original file line number Diff line number Diff line change
Expand Up @@ -600,9 +600,12 @@ func (p *Insert) ResolveIndices() (err error) {
return err
}
asgn.Col = newCol.(*expression.Column)
asgn.Expr, err = asgn.Expr.ResolveIndices(p.Schema4OnDuplicate)
if err != nil {
return err
// Once the asgn.lazyErr exists, asgn.Expr here is nil.
if asgn.Expr != nil {
asgn.Expr, err = asgn.Expr.ResolveIndices(p.Schema4OnDuplicate)
if err != nil {
return err
}
}
}
for _, set := range p.SetList {
Expand Down

0 comments on commit 1eec131

Please sign in to comment.