Skip to content

Commit e34e953

Browse files
authored
executor: change the evaluation order of columns in Update and Insert statements (#57123) (#58494)
ref #56829
1 parent 3fbf496 commit e34e953

File tree

5 files changed

+376
-123
lines changed

5 files changed

+376
-123
lines changed

pkg/executor/insert.go

Lines changed: 72 additions & 25 deletions
Original file line numberDiff line numberDiff line change
@@ -29,6 +29,7 @@ import (
2929
"github.com/pingcap/tidb/pkg/parser/model"
3030
"github.com/pingcap/tidb/pkg/parser/mysql"
3131
"github.com/pingcap/tidb/pkg/parser/terror"
32+
"github.com/pingcap/tidb/pkg/sessionctx"
3233
"github.com/pingcap/tidb/pkg/table"
3334
"github.com/pingcap/tidb/pkg/table/tables"
3435
"github.com/pingcap/tidb/pkg/tablecodec"
@@ -186,7 +187,15 @@ func (e *InsertValues) prefetchDataCache(ctx context.Context, txn kv.Transaction
186187
}
187188

188189
// updateDupRow updates a duplicate row to a new row.
189-
func (e *InsertExec) updateDupRow(ctx context.Context, idxInBatch int, txn kv.Transaction, row toBeCheckedRow, handle kv.Handle, _ []*expression.Assignment, autoColIdx int) error {
190+
func (e *InsertExec) updateDupRow(
191+
ctx context.Context,
192+
idxInBatch int,
193+
txn kv.Transaction,
194+
row toBeCheckedRow,
195+
handle kv.Handle,
196+
_ []*expression.Assignment,
197+
autoColIdx int,
198+
) error {
190199
oldRow, err := getOldRow(ctx, e.Ctx(), txn, row.t, handle, e.GenExprs)
191200
if err != nil {
192201
return err
@@ -385,8 +394,14 @@ func (e *InsertExec) initEvalBuffer4Dup() {
385394
}
386395

387396
// doDupRowUpdate updates the duplicate row.
388-
func (e *InsertExec) doDupRowUpdate(ctx context.Context, handle kv.Handle, oldRow []types.Datum, newRow []types.Datum,
389-
extraCols []types.Datum, cols []*expression.Assignment, idxInBatch int, autoColIdx int) error {
397+
func (e *InsertExec) doDupRowUpdate(
398+
ctx context.Context,
399+
handle kv.Handle,
400+
oldRow, newRow, extraCols []types.Datum,
401+
assigns []*expression.Assignment,
402+
idxInBatch int,
403+
autoColIdx int,
404+
) error {
390405
assignFlag := make([]bool, len(e.Table.WritableCols()))
391406
// See http://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_values
392407
e.curInsertVals.SetDatums(newRow...)
@@ -400,40 +415,72 @@ func (e *InsertExec) doDupRowUpdate(ctx context.Context, handle kv.Handle, oldRo
400415
e.row4Update = append(e.row4Update, extraCols...)
401416
e.row4Update = append(e.row4Update, newRow...)
402417

403-
// Update old row when the key is duplicated.
404-
e.evalBuffer4Dup.SetDatums(e.row4Update...)
405-
sc := e.Ctx().GetSessionVars().StmtCtx
406-
warnCnt := int(sc.WarningCount())
407-
for _, col := range cols {
408-
if col.LazyErr != nil {
409-
return col.LazyErr
410-
}
411-
val, err1 := col.Expr.Eval(e.evalBuffer4Dup.ToRow())
412-
if err1 != nil {
413-
return err1
414-
}
415-
c := col.Col.ToInfo()
416-
c.Name = col.ColName
417-
e.row4Update[col.Col.Index], err1 = table.CastValue(e.Ctx(), val, c, false, false)
418-
if err1 != nil {
419-
return err1
418+
// Only evaluate non-generated columns here,
419+
// other fields will be evaluated in updateRecord.
420+
var generated, nonGenerated []*expression.Assignment
421+
cols := e.Table.Cols()
422+
for _, assign := range assigns {
423+
if cols[assign.Col.Index].IsGenerated() {
424+
generated = append(generated, assign)
425+
} else {
426+
nonGenerated = append(nonGenerated, assign)
420427
}
428+
}
429+
430+
warnCnt := int(e.Ctx().GetSessionVars().StmtCtx.WarningCount())
431+
errorHandler := func(sctx sessionctx.Context, assign *expression.Assignment, val *types.Datum, err error) error {
432+
c := assign.Col.ToInfo()
433+
c.Name = assign.ColName
434+
sc := sctx.GetSessionVars().StmtCtx
435+
421436
if newWarnings := sc.TruncateWarnings(warnCnt); len(newWarnings) > 0 {
422437
for k := range newWarnings {
423438
// Use `idxInBatch` here for simplicity, since the offset of the batch is unknown under the current context.
424-
newWarnings[k].Err = completeInsertErr(c, &val, idxInBatch, newWarnings[k].Err)
439+
newWarnings[k].Err = completeInsertErr(c, val, idxInBatch, newWarnings[k].Err)
425440
}
426441
sc.AppendWarnings(newWarnings)
427442
warnCnt += len(newWarnings)
428443
}
429-
e.evalBuffer4Dup.SetDatum(col.Col.Index, e.row4Update[col.Col.Index])
430-
assignFlag[col.Col.Index] = true
444+
return err
445+
}
446+
447+
// Update old row when the key is duplicated.
448+
e.evalBuffer4Dup.SetDatums(e.row4Update...)
449+
sctx := e.Ctx()
450+
for _, assign := range nonGenerated {
451+
var val types.Datum
452+
if assign.LazyErr != nil {
453+
return assign.LazyErr
454+
}
455+
val, err := assign.Expr.Eval(e.evalBuffer4Dup.ToRow())
456+
if err != nil {
457+
return err
458+
}
459+
460+
c := assign.Col.ToInfo()
461+
idx := assign.Col.Index
462+
c.Name = assign.ColName
463+
val, err = table.CastValue(sctx, val, c, false, false)
464+
if err != nil {
465+
return err
466+
}
467+
468+
_ = errorHandler(sctx, assign, &val, nil)
469+
e.evalBuffer4Dup.SetDatum(idx, val)
470+
e.row4Update[assign.Col.Index] = val
471+
assignFlag[assign.Col.Index] = true
431472
}
432473

433474
newData := e.row4Update[:len(oldRow)]
434-
_, err := updateRecord(ctx, e.Ctx(), handle, oldRow, newData, assignFlag, e.Table, true, e.memTracker, e.fkChecks, e.fkCascades)
475+
_, err := updateRecord(
476+
ctx, e.Ctx(),
477+
handle, oldRow, newData,
478+
0, generated, e.evalBuffer4Dup, errorHandler,
479+
assignFlag, e.Table,
480+
true, e.memTracker, e.fkChecks, e.fkCascades)
481+
435482
if err != nil {
436-
return err
483+
return errors.Trace(err)
437484
}
438485

439486
if autoColIdx >= 0 {

pkg/executor/insert_test.go

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

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

0 commit comments

Comments
 (0)