Skip to content

Commit

Permalink
txn: support skipping locking of some INSERT in pessimistic transacti…
Browse files Browse the repository at this point in the history
…ons (#36473)

close #36579
  • Loading branch information
ekexium committed Sep 1, 2022
1 parent c372adb commit 835922a
Show file tree
Hide file tree
Showing 30 changed files with 402 additions and 47 deletions.
8 changes: 4 additions & 4 deletions DEPS.bzl
Original file line number Diff line number Diff line change
Expand Up @@ -2797,8 +2797,8 @@ def go_deps():
name = "com_github_pingcap_kvproto",
build_file_proto_mode = "disable_global",
importpath = "github.com/pingcap/kvproto",
sum = "h1:4UQdx1acoUrQD0Q5Etz1ABd31duzSgp3XwEnb/cvV9I=",
version = "v0.0.0-20220804022843-f006036b1277",
sum = "h1:s1al2ci3MEj5VnNuUCGAfeqpbCxcMeZibOyxw8ClHLE=",
version = "v0.0.0-20220818023518-a0f02b6efcee",
)
go_repository(
name = "com_github_pingcap_log",
Expand Down Expand Up @@ -3401,8 +3401,8 @@ def go_deps():
name = "com_github_tikv_client_go_v2",
build_file_proto_mode = "disable_global",
importpath = "github.com/tikv/client-go/v2",
sum = "h1:/nr7P8uzJQ7u3wPEBHCokrsVmuDvi/1x/zI/ydk5n8U=",
version = "v2.0.1-0.20220818084834-0d0ae0dcfb1f",
sum = "h1:pZoPlKWCecxJKL8oRq/se71RTljYDrQlZQ2NzKkMYi0=",
version = "v2.0.1-0.20220830073839-0130f767386c",
)
go_repository(
name = "com_github_tikv_pd_client",
Expand Down
11 changes: 10 additions & 1 deletion docs/design/2022-08-04-pessimistic-lazy-constraint-check.md
Original file line number Diff line number Diff line change
Expand Up @@ -101,7 +101,7 @@ Consider the following scenario (from @cfzjywxk):

The `INSERT` statement puts the row with `id = 1` in the transaction write buffer of TiDB without checking the constraint. The later `SELECT FOR UPDATE` will read and lock the row with `id = 1` in TiKV. If the `SELECT FOR UPDATE` succeeded, it would be difficult to decide the result set. Returning `(1, 1), (1, 2)` breaks the unique constraint, while returning `(1, 1)` or `(1, 2)` may be all strange semantically. Using the wrong result set for following operations may even cause data inconsistency.

So, we choose to do the missing constraint check and locking whenever a key that skipped constraint check before is read from the transaction buffer. In union scan or point get executors, if we involve any key in the buffer with a `NeedConflictCheckInPrewrite` flag, we will reset the flag and add it again to the staging buffer. Later, we can acquire locks and check the constraints for these keys. In this way, the result set will not break any constraint.
So, we choose to do the missing constraint check and locking whenever a key that skipped constraint check before is read from the transaction buffer. In union scan or point get executors, if we involve any key in the buffer with a `NeedConstraintCheckInPrewrite` flag, we will reset the flag and add it again to the staging buffer. Later, we can acquire locks and check the constraints for these keys. In this way, the result set will not break any constraint.

This means the read-only statements like the `SELECT FOR UPDATE` above will throw a "duplicate entry" error in the case above. It may be strange that a read-only statement raises errors like this. We should make the user aware of the behavior.

Expand Down Expand Up @@ -157,3 +157,12 @@ First, atomicity is not affected. Although we skip the locking phase for some of
The uniqueness constraints are also preserved. For all the keys with a `PresumeKeyNotExists` flag, we check the constraint either when prewriting them, or when acquiring the pessimistic locks like in the case of [Locking Lazy Checked Keys](#behavior-of-locking-lazy-checked-keys) above. So we can guarantee no duplicated entry exists after committing the transaction. In the case of "rollback to savepoint", some keys that need constraint checks may be unchanged in the end, but we will still check the constraints for them in prewrite to make sure the client does not miss any errors.

Due to the "read committed" semantics of DMLs in pessimistic transactions, the late locking could succeed even if duplicated entries exist at the time of `INSERT` because other transactions remove the duplicated entry after that. From the view of our transaction, it's equivalent to the case when other transactions remove the duplicated entry before our `INSERT`. There will be no data corruption after the transaction commits.

#### Safety with multiple operations in one statement

In current TiDB(<=6.2) implementation, the locking phase of pessimistic DML (except SELECT FOR UPDATE) begins after executors. If there are multiple operations on one key in the execution phase, they may not behave like what we expect. For example in the same statement there are operations:

1. write a key without the NeedConstraintCheckInPrewrite flag. This may be a normal locking request or because of compensating a deferred lock.
2. write the same key with the NeedConstraintCheckInPrewrite flag

In the locking phase we will not acquire pessimistic lock for the key because the NeedConstraintCheckInPrewrite flag is set. This should not happen in the current TiDB implementation, but is noteworthy.
1 change: 1 addition & 0 deletions errno/errcode.go
Original file line number Diff line number Diff line change
Expand Up @@ -1028,6 +1028,7 @@ const (
ErrSettingNoopVariable = 8144
ErrGettingNoopVariable = 8145
ErrCannotMigrateSession = 8146
ErrLazyUniquenessCheckFailure = 8147

// Error codes used by TiDB ddl package
ErrUnsupportedDDLOperation = 8200
Expand Down
1 change: 1 addition & 0 deletions errno/errname.go
Original file line number Diff line number Diff line change
Expand Up @@ -1023,6 +1023,7 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{
ErrSettingNoopVariable: mysql.Message("setting %s has no effect in TiDB", nil),
ErrGettingNoopVariable: mysql.Message("variable %s has no effect in TiDB", nil),
ErrCannotMigrateSession: mysql.Message("cannot migrate the current session: %s", nil),
ErrLazyUniquenessCheckFailure: mysql.Message("transaction aborted because lazy uniqueness check is enabled and an error occurred: %s", nil),

ErrWarnOptimizerHintInvalidInteger: mysql.Message("integer value is out of range in '%s'", nil),
ErrWarnOptimizerHintUnsupportedHint: mysql.Message("Optimizer hint %s is not supported by TiDB and is ignored", nil),
Expand Down
5 changes: 5 additions & 0 deletions errors.toml
Original file line number Diff line number Diff line change
Expand Up @@ -1511,6 +1511,11 @@ error = '''
setting %s has no effect in TiDB
'''

["executor:8147"]
error = '''
transaction aborted because lazy uniqueness check is enabled and an error occurred: %s
'''

["executor:8212"]
error = '''
Failed to split region ranges: %s
Expand Down
27 changes: 24 additions & 3 deletions executor/adapter.go
Original file line number Diff line number Diff line change
Expand Up @@ -247,7 +247,7 @@ type ExecStmt struct {
}

// GetStmtNode returns the stmtNode inside Statement
func (a ExecStmt) GetStmtNode() ast.StmtNode {
func (a *ExecStmt) GetStmtNode() ast.StmtNode {
return a.StmtNode
}

Expand Down Expand Up @@ -720,16 +720,37 @@ func (a *ExecStmt) handleNoDelayExecutor(ctx context.Context, e Executor) (sqlex
return nil, err
}

func (a *ExecStmt) handlePessimisticDML(ctx context.Context, e Executor) error {
func (a *ExecStmt) handlePessimisticDML(ctx context.Context, e Executor) (err error) {
sctx := a.Ctx
// Do not active the transaction here.
// Do not activate the transaction here.
// When autocommit = 0 and transaction in pessimistic mode,
// statements like set xxx = xxx; should not active the transaction.
txn, err := sctx.Txn(false)
if err != nil {
return err
}
txnCtx := sctx.GetSessionVars().TxnCtx
defer func() {
if err != nil && !sctx.GetSessionVars().ConstraintCheckInPlacePessimistic {
// If it's not a retryable error, rollback current transaction instead of rolling back current statement like
// in normal transactions, because we cannot locate and rollback the statement that leads to the lock error.
// This is too strict, but since the feature is not for everyone, it's the easiest way to guarantee safety.
stmtText := a.OriginText()
if sctx.GetSessionVars().EnableRedactLog {
stmtText = parser.Normalize(stmtText)
}
logutil.Logger(ctx).Info("Transaction abort for the safety of lazy uniqueness check. "+
"Note this may not be a uniqueness violation.",
zap.Error(err),
zap.String("statement", stmtText),
zap.Uint64("conn", sctx.GetSessionVars().ConnectionID),
zap.Uint64("txnStartTS", txnCtx.StartTS),
zap.Uint64("forUpdateTS", txnCtx.GetForUpdateTS()),
)
sctx.GetSessionVars().SetInTxn(false)
err = ErrLazyUniquenessCheckFailure.GenWithStackByArgs(err.Error())
}
}()
for {
startPointGetLocking := time.Now()
_, err = a.handleNoDelayExecutor(ctx, e)
Expand Down
1 change: 1 addition & 0 deletions executor/errors.go
Original file line number Diff line number Diff line change
Expand Up @@ -56,6 +56,7 @@ var (
ErrViewInvalid = dbterror.ClassExecutor.NewStd(mysql.ErrViewInvalid)
ErrInstanceScope = dbterror.ClassExecutor.NewStd(mysql.ErrInstanceScope)
ErrSettingNoopVariable = dbterror.ClassExecutor.NewStd(mysql.ErrSettingNoopVariable)
ErrLazyUniquenessCheckFailure = dbterror.ClassExecutor.NewStd(mysql.ErrLazyUniquenessCheckFailure)

ErrBRIEBackupFailed = dbterror.ClassExecutor.NewStd(mysql.ErrBRIEBackupFailed)
ErrBRIERestoreFailed = dbterror.ClassExecutor.NewStd(mysql.ErrBRIERestoreFailed)
Expand Down
4 changes: 2 additions & 2 deletions go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -63,7 +63,7 @@ require (
github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c
github.com/pingcap/failpoint v0.0.0-20220423142525-ae43b7f4e5c3
github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059
github.com/pingcap/kvproto v0.0.0-20220804022843-f006036b1277
github.com/pingcap/kvproto v0.0.0-20220818023518-a0f02b6efcee
github.com/pingcap/log v1.1.0
github.com/pingcap/sysutil v0.0.0-20220114020952-ea68d2dbf5b4
github.com/pingcap/tidb/parser v0.0.0-20211011031125-9b13dc409c5e
Expand All @@ -80,7 +80,7 @@ require (
github.com/stretchr/testify v1.8.0
github.com/tdakkota/asciicheck v0.1.1
github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2
github.com/tikv/client-go/v2 v2.0.1-0.20220818084834-0d0ae0dcfb1f
github.com/tikv/client-go/v2 v2.0.1-0.20220830073839-0130f767386c
github.com/tikv/pd/client v0.0.0-20220725055910-7187a7ab72db
github.com/timakin/bodyclose v0.0.0-20210704033933-f49887972144
github.com/twmb/murmur3 v1.1.3
Expand Down
8 changes: 4 additions & 4 deletions go.sum
Original file line number Diff line number Diff line change
Expand Up @@ -757,8 +757,8 @@ github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 h1:surzm05a8C9dN
github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989/go.mod h1:O17XtbryoCJhkKGbT62+L2OlrniwqiGLSqrmdHCMzZw=
github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w=
github.com/pingcap/kvproto v0.0.0-20220510035547-0e2f26c0a46a/go.mod h1:OYtxs0786qojVTmkVeufx93xe+jUgm56GUYRIKnmaGI=
github.com/pingcap/kvproto v0.0.0-20220804022843-f006036b1277 h1:4UQdx1acoUrQD0Q5Etz1ABd31duzSgp3XwEnb/cvV9I=
github.com/pingcap/kvproto v0.0.0-20220804022843-f006036b1277/go.mod h1:OYtxs0786qojVTmkVeufx93xe+jUgm56GUYRIKnmaGI=
github.com/pingcap/kvproto v0.0.0-20220818023518-a0f02b6efcee h1:s1al2ci3MEj5VnNuUCGAfeqpbCxcMeZibOyxw8ClHLE=
github.com/pingcap/kvproto v0.0.0-20220818023518-a0f02b6efcee/go.mod h1:OYtxs0786qojVTmkVeufx93xe+jUgm56GUYRIKnmaGI=
github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8=
github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8=
github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM=
Expand Down Expand Up @@ -908,8 +908,8 @@ github.com/tenntenn/text/transform v0.0.0-20200319021203-7eef512accb3 h1:f+jULpR
github.com/tenntenn/text/transform v0.0.0-20200319021203-7eef512accb3/go.mod h1:ON8b8w4BN/kE1EOhwT0o+d62W65a6aPw1nouo9LMgyY=
github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 h1:mbAskLJ0oJfDRtkanvQPiooDH8HvJ2FBh+iKT/OmiQQ=
github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2/go.mod h1:2PfKggNGDuadAa0LElHrByyrz4JPZ9fFx6Gs7nx7ZZU=
github.com/tikv/client-go/v2 v2.0.1-0.20220818084834-0d0ae0dcfb1f h1:/nr7P8uzJQ7u3wPEBHCokrsVmuDvi/1x/zI/ydk5n8U=
github.com/tikv/client-go/v2 v2.0.1-0.20220818084834-0d0ae0dcfb1f/go.mod h1:v3DEt8LS9olI6D6El17pYBWq7B28hw3NnDFTxQHDLpY=
github.com/tikv/client-go/v2 v2.0.1-0.20220830073839-0130f767386c h1:pZoPlKWCecxJKL8oRq/se71RTljYDrQlZQ2NzKkMYi0=
github.com/tikv/client-go/v2 v2.0.1-0.20220830073839-0130f767386c/go.mod h1:DqgQZKxPtMRazyELFyj7Ic2iOo+5XGTetxDp2KYnODs=
github.com/tikv/pd/client v0.0.0-20220725055910-7187a7ab72db h1:r1eMh9Rny3hfWuBuxOnbsCRrR4FhthiNxLQ5rAUtaww=
github.com/tikv/pd/client v0.0.0-20220725055910-7187a7ab72db/go.mod h1:ew8kS0yIcEaSetuuywkTLIUBR+sz3J5XvAYRae11qwc=
github.com/timakin/bodyclose v0.0.0-20210704033933-f49887972144 h1:kl4KhGNsJIbDHS9/4U9yQo1UcPQM0kOMJHn29EoH/Ro=
Expand Down
4 changes: 4 additions & 0 deletions kv/interface_mock_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -157,6 +157,10 @@ func (t *mockTxn) ClearDiskFullOpt() {
// TODO nothing
}

func (t *mockTxn) UpdateMemBufferFlags(_ []byte, _ ...FlagsOp) {

}

// newMockTxn new a mockTxn.
func newMockTxn() Transaction {
return &mockTxn{
Expand Down
12 changes: 12 additions & 0 deletions kv/keyflags.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,9 @@ const (
// it's expected to be unchangeable within the current transaction.
flagAssertExists
flagAssertNotExists
// the flag indicates the conflict and constraint check of the key should be postponed
// to the next pessimistic lock or prewrite request.
flagNeedConstraintCheckInPrewrite
)

// HasPresumeKeyNotExists returns whether the associated key use lazy check.
Expand Down Expand Up @@ -63,6 +66,11 @@ func (f KeyFlags) HasAssertionFlags() bool {
return f&flagAssertExists != 0 || f&flagAssertNotExists != 0
}

// HasNeedConstraintCheckInPrewrite returns whether the key needs to do constraint and conflict check in prewrite.
func (f KeyFlags) HasNeedConstraintCheckInPrewrite() bool {
return f&flagNeedConstraintCheckInPrewrite != 0
}

// FlagsOp describes KeyFlags modify operation.
type FlagsOp uint16

Expand All @@ -79,6 +87,8 @@ const (
SetAssertUnknown
// SetAssertNone marks the associated key without any assert.
SetAssertNone
// SetNeedConstraintCheckInPrewrite sets the flag flagNeedConstraintCheckInPrewrite
SetNeedConstraintCheckInPrewrite
)

// ApplyFlagsOps applys flagspos to origin.
Expand All @@ -98,6 +108,8 @@ func ApplyFlagsOps(origin KeyFlags, ops ...FlagsOp) KeyFlags {
case SetAssertUnknown:
origin |= flagAssertExists
origin |= flagAssertNotExists
case SetNeedConstraintCheckInPrewrite:
origin |= flagNeedConstraintCheckInPrewrite
}
}
return origin
Expand Down
3 changes: 3 additions & 0 deletions kv/kv.go
Original file line number Diff line number Diff line change
Expand Up @@ -245,6 +245,9 @@ type Transaction interface {

// RollbackMemDBToCheckpoint rollbacks the transaction's memDB to the specified checkpoint.
RollbackMemDBToCheckpoint(*tikv.MemDBCheckpoint)

// UpdateMemBufferFlags updates the flags of a node in the mem buffer.
UpdateMemBufferFlags(key []byte, flags ...FlagsOp)
}

// AssertionProto is an interface defined for the assertion protocol.
Expand Down
8 changes: 8 additions & 0 deletions metrics/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -151,6 +151,14 @@ var (
Help: "Bucketed histogram of different states of a transaction.",
Buckets: prometheus.ExponentialBuckets(0.0005, 2, 29), // 0.5ms ~ 1.5days
}, []string{LblType, LblHasLock})
LazyPessimisticUniqueCheckSetCount = prometheus.NewCounter(
prometheus.CounterOpts{
Namespace: "tidb",
Subsystem: "session",
Name: "lazy_pessimistic_unique_check_set_count",
Help: "Counter of setting tidb_constraint_check_in_place to false",
},
)
)

// Label constants.
Expand Down
5 changes: 5 additions & 0 deletions metrics/telemetry.go
Original file line number Diff line number Diff line change
Expand Up @@ -217,3 +217,8 @@ func GetNonTransactionalStmtCounter() NonTransactionalStmtCounter {
func GetSavepointStmtCounter() int64 {
return readCounter(StmtNodeCounter.With(prometheus.Labels{LblType: "Savepoint"}))
}

// GetLazyPessimisticUniqueCheckSetCounter returns the counter of setting tidb_constraint_check_in_place_pessimistic to false.
func GetLazyPessimisticUniqueCheckSetCounter() int64 {
return readCounter(LazyPessimisticUniqueCheckSetCount)
}
4 changes: 2 additions & 2 deletions session/schema_amender.go
Original file line number Diff line number Diff line change
Expand Up @@ -393,7 +393,7 @@ func (a *amendOperationAddIndex) genMutations(ctx context.Context, sctx sessionc
key := deletedMutations.GetKeys()[i]
if _, ok := a.insertedNewIndexKeys[string(key)]; !ok {
resAddMutations.Push(deletedMutations.GetOps()[i], key, deletedMutations.GetValues()[i], deletedMutations.IsPessimisticLock(i),
deletedMutations.IsAssertExists(i), deletedMutations.IsAssertNotExist(i))
deletedMutations.IsAssertExists(i), deletedMutations.IsAssertNotExist(i), deletedMutations.NeedConstraintCheckInPrewrite(i))
}
}
for i := 0; i < len(insertedMutations.GetKeys()); i++ {
Expand All @@ -403,7 +403,7 @@ func (a *amendOperationAddIndex) genMutations(ctx context.Context, sctx sessionc
destKeyOp = kvrpcpb.Op_Put
}
resAddMutations.Push(destKeyOp, key, insertedMutations.GetValues()[i], insertedMutations.IsPessimisticLock(i),
insertedMutations.IsAssertExists(i), insertedMutations.IsAssertNotExist(i))
insertedMutations.IsAssertExists(i), insertedMutations.IsAssertNotExist(i), insertedMutations.NeedConstraintCheckInPrewrite(i))
}
} else {
resAddMutations.MergeMutations(deletedMutations)
Expand Down
12 changes: 6 additions & 6 deletions session/schema_amender_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -143,7 +143,7 @@ func prepareTestData(
oldData.ops = append(oldData.ops, keyOp)
oldData.rowValue = append(oldData.rowValue, thisRowValue)
if keyOp == kvrpcpb.Op_Del {
mutations.Push(keyOp, rowKey, []byte{}, true, false, false)
mutations.Push(keyOp, rowKey, []byte{}, true, false, false, false)
}
}
oldRowValues[i] = thisRowValue
Expand Down Expand Up @@ -171,9 +171,9 @@ func prepareTestData(
}
require.NoError(t, err)
if keyOp == kvrpcpb.Op_Put || keyOp == kvrpcpb.Op_Insert {
mutations.Push(keyOp, rowKey, rowValue, true, false, false)
mutations.Push(keyOp, rowKey, rowValue, true, false, false, false)
} else if keyOp == kvrpcpb.Op_Lock {
mutations.Push(keyOp, rowKey, []byte{}, true, false, false)
mutations.Push(keyOp, rowKey, []byte{}, true, false, false, false)
}
newRowValues[i] = thisRowValue
newRowKvMap[string(rowKey)] = thisRowValue
Expand Down Expand Up @@ -212,7 +212,7 @@ func prepareTestData(
if info.indexInfoAtCommit.Meta().Unique {
isPessimisticLock = true
}
oldIdxKeyMutation.Push(kvrpcpb.Op_Del, idxKey, []byte{}, isPessimisticLock, false, false)
oldIdxKeyMutation.Push(kvrpcpb.Op_Del, idxKey, []byte{}, isPessimisticLock, false, false, false)
}
}
if addIndexNeedAddOp(info.AmendOpType) && mayGenPutIndexRowKeyOp(keyOp) {
Expand All @@ -224,7 +224,7 @@ func prepareTestData(
mutOp = kvrpcpb.Op_Insert
isPessimisticLock = true
}
newIdxKeyMutation.Push(mutOp, idxKey, idxVal, isPessimisticLock, false, false)
newIdxKeyMutation.Push(mutOp, idxKey, idxVal, isPessimisticLock, false, false, false)
}
skipMerge := false
if info.AmendOpType == AmendNeedAddDeleteAndInsert {
Expand Down Expand Up @@ -440,7 +440,7 @@ func TestAmendCollectAndGenMutations(t *testing.T) {
idxKey := tablecodec.EncodeIndexSeekKey(oldTbInfo.Meta().ID, oldTbInfo.Indices()[i].Meta().ID, idxValue)
err = txn.Set(idxKey, idxValue)
require.NoError(t, err)
mutations.Push(kvrpcpb.Op_Put, idxKey, idxValue, false, false, false)
mutations.Push(kvrpcpb.Op_Put, idxKey, idxValue, false, false, false, false)
}

res, err := schemaAmender.genAllAmendMutations(ctx, &mutations, collector)
Expand Down
8 changes: 8 additions & 0 deletions session/txn.go
Original file line number Diff line number Diff line change
Expand Up @@ -450,6 +450,7 @@ func (txn *LazyTxn) KeysNeedToLock() ([]kv.Key, error) {
}
keys = append(keys, k)
})

return keys, nil
}

Expand Down Expand Up @@ -483,6 +484,13 @@ func keyNeedToLock(k, v []byte, flags kv.KeyFlags) bool {
// meta key always need to lock.
return true
}

// a pessimistic locking is skipped, perform the conflict check and
// constraint check (more accurately, PresumeKeyNotExist) in prewrite (or later pessimistic locking)
if flags.HasNeedConstraintCheckInPrewrite() {
return false
}

if flags.HasPresumeKeyNotExists() {
return true
}
Expand Down
4 changes: 4 additions & 0 deletions sessionctx/variable/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -1230,6 +1230,10 @@ type SessionVars struct {

// GeneralPlanCacheSize controls the size of general plan cache.
GeneralPlanCacheSize uint64

// ConstraintCheckInPlacePessimistic controls whether to skip the locking of some keys in pessimistic transactions.
// Postpone the conflict check and constraint check to prewrite or later pessimistic locking requests.
ConstraintCheckInPlacePessimistic bool
}

// GetPreparedStmtByName returns the prepared statement specified by stmtName.
Expand Down
Loading

0 comments on commit 835922a

Please sign in to comment.