Skip to content

Commit 1985662

Browse files
authored
table: add new option DupKeyCheckMode for table mutations (#55194)
ref #54397
1 parent 9fee330 commit 1985662

File tree

14 files changed

+288
-32
lines changed

14 files changed

+288
-32
lines changed

pkg/ddl/index.go

+6-3
Original file line numberDiff line numberDiff line change
@@ -1646,8 +1646,6 @@ func (w *addIndexTxnWorker) batchCheckUniqueKey(txn kv.Transaction, idxRecords [
16461646
}
16471647
idxRecords[w.recordIdx[i]].skip = found && idxRecords[w.recordIdx[i]].skip
16481648
}
1649-
// Constrains is already checked.
1650-
w.tblCtx.GetSessionVars().StmtCtx.BatchCheck = true
16511649
return nil
16521650
}
16531651

@@ -1841,7 +1839,12 @@ func (w *addIndexTxnWorker) BackfillData(handleRange reorgBackfillTask) (taskCtx
18411839
}
18421840

18431841
handle, err := w.indexes[i%len(w.indexes)].Create(
1844-
w.tblCtx, txn, idxRecord.vals, idxRecord.handle, idxRecord.rsData, table.WithIgnoreAssertion, table.FromBackfill)
1842+
w.tblCtx, txn, idxRecord.vals, idxRecord.handle, idxRecord.rsData,
1843+
table.WithIgnoreAssertion,
1844+
table.FromBackfill,
1845+
// Constrains is already checked in batchCheckUniqueKey
1846+
table.DupKeyCheckSkip,
1847+
)
18451848
if err != nil {
18461849
if kv.ErrKeyExists.Equal(err) && idxRecord.handle.Equal(handle) {
18471850
// Index already exists, skip it.

pkg/executor/admin.go

+5-3
Original file line numberDiff line numberDiff line change
@@ -513,8 +513,6 @@ func (e *RecoverIndexExec) backfillIndexInTxn(ctx context.Context, txn kv.Transa
513513
return result, err
514514
}
515515

516-
// Constrains is already checked.
517-
e.Ctx().GetSessionVars().StmtCtx.BatchCheck = true
518516
for _, row := range rows {
519517
if row.skip {
520518
continue
@@ -526,7 +524,11 @@ func (e *RecoverIndexExec) backfillIndexInTxn(ctx context.Context, txn kv.Transa
526524
return result, err
527525
}
528526

529-
_, err = e.index.Create(e.Ctx().GetTableCtx(), txn, row.idxVals, row.handle, row.rsData, table.WithIgnoreAssertion)
527+
_, err = e.index.Create(e.Ctx().GetTableCtx(), txn, row.idxVals, row.handle, row.rsData,
528+
table.WithIgnoreAssertion,
529+
// Constrains have already been checked.
530+
table.DupKeyCheckSkip,
531+
)
530532
if err != nil {
531533
return result, err
532534
}

pkg/executor/insert.go

+3-3
Original file line numberDiff line numberDiff line change
@@ -108,9 +108,9 @@ func (e *InsertExec) exec(ctx context.Context, rows [][]types.Datum) error {
108108
if sizeHint > remain {
109109
sizeHint = remain
110110
}
111-
err = e.addRecordWithAutoIDHint(ctx, row, sizeHint)
111+
err = e.addRecordWithAutoIDHint(ctx, row, sizeHint, table.DupKeyCheckDefault)
112112
} else {
113-
err = e.addRecord(ctx, row)
113+
err = e.addRecord(ctx, row, table.DupKeyCheckDefault)
114114
}
115115
if err != nil {
116116
return err
@@ -282,7 +282,7 @@ func (e *InsertExec) batchUpdateDupRows(ctx context.Context, newRows [][]types.D
282282
// and key-values should be filled back to dupOldRowValues for the further row check,
283283
// due to there may be duplicate keys inside the insert statement.
284284
if newRows[i] != nil {
285-
err := e.addRecord(ctx, newRows[i])
285+
err := e.addRecord(ctx, newRows[i], table.DupKeyCheckDefault)
286286
if err != nil {
287287
return err
288288
}

pkg/executor/insert_common.go

+9-10
Original file line numberDiff line numberDiff line change
@@ -1190,11 +1190,9 @@ func (e *InsertValues) handleDuplicateKey(ctx context.Context, txn kv.Transactio
11901190
// All duplicate rows will be ignored and appended as duplicate warnings.
11911191
func (e *InsertValues) batchCheckAndInsert(
11921192
ctx context.Context, rows [][]types.Datum,
1193-
addRecord func(ctx context.Context, row []types.Datum) error,
1193+
addRecord func(ctx context.Context, row []types.Datum, dupKeyCheck table.DupKeyCheckMode) error,
11941194
replace bool,
11951195
) error {
1196-
// all the rows will be checked, so it is safe to set BatchCheck = true
1197-
e.Ctx().GetSessionVars().StmtCtx.BatchCheck = true
11981196
defer tracing.StartRegion(ctx, "InsertValues.batchCheckAndInsert").End()
11991197
start := time.Now()
12001198
// Get keys need to be checked.
@@ -1307,7 +1305,8 @@ func (e *InsertValues) batchCheckAndInsert(
13071305
// it should be added to values map for the further row check.
13081306
// There may be duplicate keys inside the insert statement.
13091307
e.Ctx().GetSessionVars().StmtCtx.AddCopiedRows(1)
1310-
err = addRecord(ctx, rows[i])
1308+
// all the rows have been checked, so it is safe to use DupKeyCheckSkip
1309+
err = addRecord(ctx, rows[i], table.DupKeyCheckSkip)
13111310
if err != nil {
13121311
// throw warning when violate check constraint
13131312
if table.ErrCheckConstraintViolated.Equal(err) {
@@ -1405,21 +1404,21 @@ func (e *InsertValues) equalDatumsAsBinary(a []types.Datum, b []types.Datum) (bo
14051404
return true, nil
14061405
}
14071406

1408-
func (e *InsertValues) addRecord(ctx context.Context, row []types.Datum) error {
1409-
return e.addRecordWithAutoIDHint(ctx, row, 0)
1407+
func (e *InsertValues) addRecord(ctx context.Context, row []types.Datum, dupKeyCheck table.DupKeyCheckMode) error {
1408+
return e.addRecordWithAutoIDHint(ctx, row, 0, dupKeyCheck)
14101409
}
14111410

14121411
func (e *InsertValues) addRecordWithAutoIDHint(
1413-
ctx context.Context, row []types.Datum, reserveAutoIDCount int,
1412+
ctx context.Context, row []types.Datum, reserveAutoIDCount int, dupKeyCheck table.DupKeyCheckMode,
14141413
) (err error) {
14151414
vars := e.Ctx().GetSessionVars()
14161415
if !vars.ConstraintCheckInPlace {
14171416
vars.PresumeKeyNotExists = true
14181417
}
14191418
if reserveAutoIDCount > 0 {
1420-
_, err = e.Table.AddRecord(e.Ctx().GetTableCtx(), row, table.WithCtx(ctx), table.WithReserveAutoIDHint(reserveAutoIDCount))
1419+
_, err = e.Table.AddRecord(e.Ctx().GetTableCtx(), row, table.WithCtx(ctx), table.WithReserveAutoIDHint(reserveAutoIDCount), dupKeyCheck)
14211420
} else {
1422-
_, err = e.Table.AddRecord(e.Ctx().GetTableCtx(), row, table.WithCtx(ctx))
1421+
_, err = e.Table.AddRecord(e.Ctx().GetTableCtx(), row, table.WithCtx(ctx), dupKeyCheck)
14231422
}
14241423
vars.PresumeKeyNotExists = false
14251424
if err != nil {
@@ -1429,7 +1428,7 @@ func (e *InsertValues) addRecordWithAutoIDHint(
14291428
if e.lastInsertID != 0 {
14301429
vars.SetLastInsertID(e.lastInsertID)
14311430
}
1432-
if !vars.StmtCtx.BatchCheck {
1431+
if dupKeyCheck != table.DupKeyCheckSkip {
14331432
for _, fkc := range e.fkChecks {
14341433
err = fkc.insertRowNeedToCheck(vars.StmtCtx, row)
14351434
if err != nil {

pkg/executor/load_data.go

+4-4
Original file line numberDiff line numberDiff line change
@@ -655,9 +655,9 @@ func (w *commitWorker) checkAndInsertOneBatch(ctx context.Context, rows [][]type
655655
if sizeHint > remain {
656656
sizeHint = remain
657657
}
658-
err = w.addRecordWithAutoIDHint(ctx, row, sizeHint)
658+
err = w.addRecordWithAutoIDHint(ctx, row, sizeHint, table.DupKeyCheckDefault)
659659
} else {
660-
err = w.addRecord(ctx, row)
660+
err = w.addRecord(ctx, row, table.DupKeyCheckDefault)
661661
}
662662
if err != nil {
663663
return err
@@ -670,11 +670,11 @@ func (w *commitWorker) checkAndInsertOneBatch(ctx context.Context, rows [][]type
670670
}
671671
}
672672

673-
func (w *commitWorker) addRecordLD(ctx context.Context, row []types.Datum) error {
673+
func (w *commitWorker) addRecordLD(ctx context.Context, row []types.Datum, dupKeyCheck table.DupKeyCheckMode) error {
674674
if row == nil {
675675
return nil
676676
}
677-
return w.addRecord(ctx, row)
677+
return w.addRecord(ctx, row, dupKeyCheck)
678678
}
679679

680680
// GetInfilePath get infile path.

pkg/executor/replace.go

+2-1
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,7 @@ import (
2424
"github.com/pingcap/tidb/pkg/kv"
2525
"github.com/pingcap/tidb/pkg/meta/autoid"
2626
"github.com/pingcap/tidb/pkg/parser/mysql"
27+
"github.com/pingcap/tidb/pkg/table"
2728
"github.com/pingcap/tidb/pkg/table/tables"
2829
"github.com/pingcap/tidb/pkg/tablecodec"
2930
"github.com/pingcap/tidb/pkg/types"
@@ -105,7 +106,7 @@ func (e *ReplaceExec) replaceRow(ctx context.Context, r toBeCheckedRow) error {
105106
}
106107

107108
// No duplicated rows now, insert the row.
108-
err = e.addRecord(ctx, r.row)
109+
err = e.addRecord(ctx, r.row, table.DupKeyCheckDefault)
109110
if err != nil {
110111
return err
111112
}

pkg/lightning/backend/kv/base.go

+1-1
Original file line numberDiff line numberDiff line change
@@ -208,7 +208,7 @@ func (e *BaseKVEncoder) Record2KV(record, originalRow []types.Datum, rowID int64
208208

209209
// AddRecord adds a record into encoder
210210
func (e *BaseKVEncoder) AddRecord(record []types.Datum) (kv.Handle, error) {
211-
return e.table.AddRecord(e.SessionCtx.GetTableCtx(), record)
211+
return e.table.AddRecord(e.SessionCtx.GetTableCtx(), record, table.DupKeyCheckSkip)
212212
}
213213

214214
// TableAllocators returns the allocators of the table

pkg/lightning/backend/kv/session.go

-1
Original file line numberDiff line numberDiff line change
@@ -316,7 +316,6 @@ func NewSession(options *encode.SessionOptions, logger log.Logger) *Session {
316316
vars := variable.NewSessionVars(s)
317317
vars.SkipUTF8Check = true
318318
vars.StmtCtx.InInsertStmt = true
319-
vars.StmtCtx.BatchCheck = true
320319
vars.SQLMode = sqlMode
321320

322321
typeFlags := vars.StmtCtx.TypeFlags().

pkg/sessionctx/stmtctx/stmtctx.go

-1
Original file line numberDiff line numberDiff line change
@@ -194,7 +194,6 @@ type StatementContext struct {
194194
contextutil.PlanCacheTracker
195195
contextutil.RangeFallbackHandler
196196

197-
BatchCheck bool
198197
IgnoreExplainIDSuffix bool
199198
MultiSchemaInfo *model.MultiSchemaInfo
200199
// If the select statement was like 'select * from t as of timestamp ...' or in a stale read transaction

pkg/table/table.go

+33-1
Original file line numberDiff line numberDiff line change
@@ -121,7 +121,8 @@ type RecordIterFunc func(h kv.Handle, rec []types.Datum, cols []*Column) (more b
121121

122122
// commonMutateOpt is the common options for mutating a table.
123123
type commonMutateOpt struct {
124-
Ctx context.Context
124+
Ctx context.Context
125+
DupKeyCheck DupKeyCheckMode
125126
}
126127

127128
// AddRecordOpt contains the options will be used when adding a record.
@@ -222,6 +223,37 @@ func (i isUpdate) ApplyAddRecordOpt(opt *AddRecordOpt) {
222223
opt.IsUpdate = true
223224
}
224225

226+
// DupKeyCheckMode indicates how to check the duplicated key when adding/updating a record/index.
227+
type DupKeyCheckMode uint8
228+
229+
const (
230+
// DupKeyCheckDefault indicates using the default behavior.
231+
// Currently, this means to use the return value `ctx.LazyCheckKeyNotExists()`.
232+
// If the above method returns true, it will only check the duplicated key in the memory buffer,
233+
// otherwise, it will also check the duplicated key in the storage.
234+
// TODO: add `DupKeyCheckLazy` to indicate only checking the duplicated key in the memory buffer.
235+
// After `DupKeyCheckLazy` added, `DupKeyCheckDefault` will be renamed to `DupKeyCheckInPlace` to force check
236+
// the duplicated key in place.
237+
DupKeyCheckDefault DupKeyCheckMode = iota
238+
// DupKeyCheckSkip indicates skipping the duplicated key check.
239+
DupKeyCheckSkip
240+
)
241+
242+
// ApplyAddRecordOpt implements the AddRecordOption interface.
243+
func (m DupKeyCheckMode) ApplyAddRecordOpt(opt *AddRecordOpt) {
244+
opt.DupKeyCheck = m
245+
}
246+
247+
// ApplyUpdateRecordOpt implements the UpdateRecordOption interface.
248+
func (m DupKeyCheckMode) ApplyUpdateRecordOpt(opt *UpdateRecordOpt) {
249+
opt.DupKeyCheck = m
250+
}
251+
252+
// ApplyCreateIdxOpt implements the CreateIdxOption interface.
253+
func (m DupKeyCheckMode) ApplyCreateIdxOpt(opt *CreateIdxOpt) {
254+
opt.DupKeyCheck = m
255+
}
256+
225257
type columnAPI interface {
226258
// Cols returns the columns of the table which is used in select, including hidden columns.
227259
Cols() []*Column

pkg/table/tables/BUILD.bazel

+2-1
Original file line numberDiff line numberDiff line change
@@ -77,10 +77,11 @@ go_test(
7777
],
7878
embed = [":tables"],
7979
flaky = True,
80-
shard_count = 31,
80+
shard_count = 32,
8181
deps = [
8282
"//pkg/ddl",
8383
"//pkg/domain",
84+
"//pkg/errctx",
8485
"//pkg/infoschema",
8586
"//pkg/kv",
8687
"//pkg/lightning/backend/encode",

pkg/table/tables/index.go

+1-1
Original file line numberDiff line numberDiff line change
@@ -178,7 +178,7 @@ func (c *index) create(sctx table.MutateContext, txn kv.Transaction, indexedValu
178178
}
179179
vars := sctx.GetSessionVars()
180180
writeBufs := sctx.GetMutateBuffers().GetWriteStmtBufs()
181-
skipCheck := vars.StmtCtx.BatchCheck
181+
skipCheck := opt.DupKeyCheck == table.DupKeyCheckSkip
182182
evalCtx := sctx.GetExprCtx().GetEvalCtx()
183183
loc, ec := evalCtx.Location(), evalCtx.ErrCtx()
184184
for _, value := range indexedValues {

pkg/table/tables/tables.go

+2-2
Original file line numberDiff line numberDiff line change
@@ -893,7 +893,7 @@ func (t *TableCommon) addRecord(sctx table.MutateContext, r []types.Datum, opt *
893893
}
894894
key := t.RecordKey(recordID)
895895
var setPresume bool
896-
if !sctx.GetSessionVars().StmtCtx.BatchCheck {
896+
if opt.DupKeyCheck != table.DupKeyCheckSkip {
897897
if t.meta.TempTableType != model.TempTableNone {
898898
// Always check key for temporary table because it does not write to TiKV
899899
_, err = txn.Get(ctx, key)
@@ -1013,7 +1013,7 @@ func genIndexKeyStrs(colVals []types.Datum) ([]string, error) {
10131013
func (t *TableCommon) addIndices(sctx table.MutateContext, recordID kv.Handle, r []types.Datum, txn kv.Transaction, opt *table.CreateIdxOpt) (kv.Handle, error) {
10141014
writeBufs := sctx.GetMutateBuffers().GetWriteStmtBufs()
10151015
indexVals := writeBufs.IndexValsBuf
1016-
skipCheck := sctx.GetSessionVars().StmtCtx.BatchCheck
1016+
skipCheck := opt.DupKeyCheck == table.DupKeyCheckSkip
10171017
for _, v := range t.Indices() {
10181018
if !IsIndexWritable(v) {
10191019
continue

0 commit comments

Comments
 (0)