Skip to content

Commit 587a810

Browse files
authored
executor: change the evaluation order of columns in Update and Insert statements (#57123) (#58708)
ref #56829
1 parent 1d198e6 commit 587a810

File tree

5 files changed

+418
-158
lines changed

5 files changed

+418
-158
lines changed

pkg/executor/insert.go

Lines changed: 74 additions & 33 deletions
Original file line numberDiff line numberDiff line change
@@ -30,6 +30,7 @@ import (
3030
"github.com/pingcap/tidb/pkg/meta/model"
3131
"github.com/pingcap/tidb/pkg/parser/mysql"
3232
"github.com/pingcap/tidb/pkg/parser/terror"
33+
"github.com/pingcap/tidb/pkg/sessionctx"
3334
"github.com/pingcap/tidb/pkg/sessionctx/variable"
3435
"github.com/pingcap/tidb/pkg/table"
3536
"github.com/pingcap/tidb/pkg/table/tables"
@@ -193,7 +194,16 @@ func (e *InsertValues) prefetchDataCache(ctx context.Context, txn kv.Transaction
193194
}
194195

195196
// updateDupRow updates a duplicate row to a new row.
196-
func (e *InsertExec) updateDupRow(ctx context.Context, idxInBatch int, txn kv.Transaction, row toBeCheckedRow, handle kv.Handle, _ []*expression.Assignment, dupKeyCheck table.DupKeyCheckMode, autoColIdx int) error {
197+
func (e *InsertExec) updateDupRow(
198+
ctx context.Context,
199+
idxInBatch int,
200+
txn kv.Transaction,
201+
row toBeCheckedRow,
202+
handle kv.Handle,
203+
_ []*expression.Assignment,
204+
dupKeyCheck table.DupKeyCheckMode,
205+
autoColIdx int,
206+
) error {
197207
oldRow, err := getOldRow(ctx, e.Ctx(), txn, row.t, handle, e.GenExprs)
198208
if err != nil {
199209
return err
@@ -430,8 +440,15 @@ func (e *InsertExec) initEvalBuffer4Dup() {
430440
}
431441

432442
// doDupRowUpdate updates the duplicate row.
433-
func (e *InsertExec) doDupRowUpdate(ctx context.Context, handle kv.Handle, oldRow []types.Datum, newRow []types.Datum,
434-
extraCols []types.Datum, cols []*expression.Assignment, idxInBatch int, dupKeyMode table.DupKeyCheckMode, autoColIdx int) error {
443+
func (e *InsertExec) doDupRowUpdate(
444+
ctx context.Context,
445+
handle kv.Handle,
446+
oldRow, newRow, extraCols []types.Datum,
447+
assigns []*expression.Assignment,
448+
idxInBatch int,
449+
dupKeyMode table.DupKeyCheckMode,
450+
autoColIdx int,
451+
) error {
435452
assignFlag := make([]bool, len(e.Table.WritableCols()))
436453
// See http://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_values
437454
e.curInsertVals.SetDatums(newRow...)
@@ -445,53 +462,77 @@ func (e *InsertExec) doDupRowUpdate(ctx context.Context, handle kv.Handle, oldRo
445462
e.row4Update = append(e.row4Update, extraCols...)
446463
e.row4Update = append(e.row4Update, newRow...)
447464

448-
// Update old row when the key is duplicated.
449-
e.evalBuffer4Dup.SetDatums(e.row4Update...)
450-
sctx := e.Ctx()
451-
evalCtx := sctx.GetExprCtx().GetEvalCtx()
452-
sc := sctx.GetSessionVars().StmtCtx
453-
warnCnt := int(sc.WarningCount())
454-
for _, col := range cols {
455-
if col.LazyErr != nil {
456-
return col.LazyErr
457-
}
458-
val, err1 := col.Expr.Eval(evalCtx, e.evalBuffer4Dup.ToRow())
459-
if err1 != nil {
460-
return err1
461-
}
462-
c := col.Col.ToInfo()
463-
c.Name = col.ColName
464-
e.row4Update[col.Col.Index], err1 = table.CastValue(sctx, val, c, false, false)
465-
if err1 != nil {
466-
return err1
465+
// Only evaluate non-generated columns here,
466+
// other fields will be evaluated in updateRecord.
467+
var generated, nonGenerated []*expression.Assignment
468+
cols := e.Table.Cols()
469+
for _, assign := range assigns {
470+
if cols[assign.Col.Index].IsGenerated() {
471+
generated = append(generated, assign)
472+
} else {
473+
nonGenerated = append(nonGenerated, assign)
467474
}
475+
}
476+
477+
warnCnt := int(e.Ctx().GetSessionVars().StmtCtx.WarningCount())
478+
errorHandler := func(sctx sessionctx.Context, assign *expression.Assignment, val *types.Datum, err error) error {
479+
c := assign.Col.ToInfo()
480+
c.Name = assign.ColName
481+
sc := sctx.GetSessionVars().StmtCtx
482+
468483
if newWarnings := sc.TruncateWarnings(warnCnt); len(newWarnings) > 0 {
469484
for k := range newWarnings {
470485
// Use `idxInBatch` here for simplicity, since the offset of the batch is unknown under the current context.
471-
newWarnings[k].Err = completeInsertErr(c, &val, idxInBatch, newWarnings[k].Err)
486+
newWarnings[k].Err = completeInsertErr(c, val, idxInBatch, newWarnings[k].Err)
472487
}
473488
sc.AppendWarnings(newWarnings)
474489
warnCnt += len(newWarnings)
475490
}
476-
e.evalBuffer4Dup.SetDatum(col.Col.Index, e.row4Update[col.Col.Index])
477-
assignFlag[col.Col.Index] = true
491+
return err
478492
}
479493

480-
newData := e.row4Update[:len(oldRow)]
481-
if e.ignoreErr {
482-
ignored, err := checkFKIgnoreErr(ctx, e.Ctx(), e.fkChecks, newData)
494+
// Update old row when the key is duplicated.
495+
e.evalBuffer4Dup.SetDatums(e.row4Update...)
496+
sctx := e.Ctx()
497+
evalCtx := sctx.GetExprCtx().GetEvalCtx()
498+
for _, assign := range nonGenerated {
499+
var val types.Datum
500+
if assign.LazyErr != nil {
501+
return assign.LazyErr
502+
}
503+
val, err := assign.Expr.Eval(evalCtx, e.evalBuffer4Dup.ToRow())
483504
if err != nil {
484505
return err
485506
}
486507

487-
// meets an error, skip this row.
488-
if ignored {
489-
return nil
508+
c := assign.Col.ToInfo()
509+
idx := assign.Col.Index
510+
c.Name = assign.ColName
511+
val, err = table.CastValue(sctx, val, c, false, false)
512+
if err != nil {
513+
return err
490514
}
515+
516+
_ = errorHandler(sctx, assign, &val, nil)
517+
e.evalBuffer4Dup.SetDatum(idx, val)
518+
e.row4Update[assign.Col.Index] = val
519+
assignFlag[assign.Col.Index] = true
491520
}
492-
_, err := updateRecord(ctx, e.Ctx(), handle, oldRow, newData, assignFlag, e.Table, true, e.memTracker, e.fkChecks, e.fkCascades, dupKeyMode, e.ignoreErr)
521+
522+
newData := e.row4Update[:len(oldRow)]
523+
_, ignored, err := updateRecord(
524+
ctx, e.Ctx(),
525+
handle, oldRow, newData,
526+
0, generated, e.evalBuffer4Dup, errorHandler,
527+
assignFlag, e.Table,
528+
true, e.memTracker, e.fkChecks, e.fkCascades, dupKeyMode, e.ignoreErr)
529+
530+
if ignored {
531+
return nil
532+
}
533+
493534
if err != nil {
494-
return err
535+
return errors.Trace(err)
495536
}
496537

497538
if autoColIdx >= 0 {

pkg/executor/insert_test.go

Lines changed: 18 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -240,6 +240,24 @@ func testInsertOnDuplicateKey(t *testing.T, tk *testkit.TestKit) {
240240
"<nil>/<nil>/x/1.2",
241241
"<nil>/<nil>/x/1.2"))
242242

243+
// Test issue 56829
244+
tk.MustExec(`
245+
CREATE TABLE cache (
246+
cache_key varchar(512) NOT NULL,
247+
updated_at datetime NOT NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP,
248+
expired_at datetime GENERATED ALWAYS AS (if(expires > 0, date_add(updated_at, interval expires second), date_add(updated_at, interval 99 year))) VIRTUAL,
249+
expires int(11),
250+
PRIMARY KEY (cache_key) /*T![clustered_index] CLUSTERED */,
251+
KEY idx_c_on_expired_at (expired_at)
252+
)`)
253+
tk.MustExec("INSERT INTO cache(cache_key, expires) VALUES ('2001-01-01 11:11:11', 60) ON DUPLICATE KEY UPDATE expires = expires + 1")
254+
tk.MustExec("select sleep(1)")
255+
tk.MustExec("INSERT INTO cache(cache_key, expires) VALUES ('2001-01-01 11:11:11', 60) ON DUPLICATE KEY UPDATE expires = expires + 1")
256+
tk.MustExec("admin check table cache")
257+
rs1 := tk.MustQuery("select cache_key, expired_at from cache use index() order by cache_key")
258+
rs2 := tk.MustQuery("select cache_key, expired_at from cache use index(idx_c_on_expired_at) order by cache_key")
259+
require.True(t, rs1.Equal(rs2.Rows()))
260+
243261
// reproduce insert on duplicate key update bug under new row format.
244262
tk.MustExec(`drop table if exists t1`)
245263
tk.MustExec(`create table t1(c1 decimal(6,4), primary key(c1))`)

0 commit comments

Comments
 (0)