Skip to content

Commit ddf92ba

Browse files
craig[bot]stevendannakev-cao
committed
148819: batcheval: allow sequence number regression on lock acquire r=arulajmani a=stevendanna This was an oversight in 39d519b. When flushing the txn write buffer, we may also be regressing a lock that was previously flushed added to the replicated lock table via an unreplicated lock table flush. Fixes: #148789 Release note: None 148823: backup: increase pool size for backup sink tests r=dt a=kev-cao A test failure was discovered that was likely caused by an OOM. This commit increases the pool size for the test. Epic: CRDB-51482 Fixes: #148630 Co-authored-by: Steven Danna <[email protected]> Co-authored-by: Kevin Cao <[email protected]>
3 parents 421a257 + 88d6f6f + f488371 commit ddf92ba

File tree

3 files changed

+12
-4
lines changed

3 files changed

+12
-4
lines changed

pkg/backup/backupsink/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -37,6 +37,7 @@ go_test(
3737
"sst_sink_key_writer_test.go",
3838
],
3939
embed = [":backupsink"],
40+
exec_properties = {"test.Pool": "large"},
4041
deps = [
4142
"//pkg/backup/backuppb",
4243
"//pkg/ccl/storageccl",

pkg/kv/kvserver/batcheval/cmd_get.go

Lines changed: 6 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -120,8 +120,13 @@ func Get(
120120
shouldLockKey := getRes.Value != nil || args.LockNonExisting
121121
var res result.Result
122122
if args.KeyLockingStrength != lock.None && shouldLockKey {
123+
// ExpectExclusionSince is used by callers (namely, txnWriteBuffers) that
124+
// are likely to be sending replicated, locking Get requests at sequence
125+
// numbers corresponding to unreplicated locks taken earlier in the
126+
// transaction. In this case, sequence number regression is not unexpected.
127+
allowSequenceNumberRegression := args.ExpectExclusionSince.IsSet()
123128
acq, err := acquireLockOnKey(ctx, readWriter, h.Txn, args.KeyLockingStrength,
124-
args.KeyLockingDurability, args.Key, cArgs.Stats, cArgs.EvalCtx.ClusterSettings())
129+
args.KeyLockingDurability, args.Key, cArgs.Stats, cArgs.EvalCtx.ClusterSettings(), allowSequenceNumberRegression)
125130
if err != nil {
126131
return result.Result{}, err
127132
}

pkg/kv/kvserver/batcheval/lock.go

Lines changed: 5 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -137,7 +137,7 @@ func acquireLocksOnKeys(
137137
case kvpb.BATCH_RESPONSE:
138138
err := storage.MVCCScanDecodeKeyValues(scanRes.KVData, func(key storage.MVCCKey, _ []byte) error {
139139
k := copyKey(key.Key)
140-
acq, err := acquireLockOnKey(ctx, readWriter, txn, str, dur, k, ms, settings)
140+
acq, err := acquireLockOnKey(ctx, readWriter, txn, str, dur, k, ms, settings, false /* allowSequenceNumberRegression */)
141141
if err != nil {
142142
return err
143143
}
@@ -153,7 +153,7 @@ func acquireLocksOnKeys(
153153
case kvpb.KEY_VALUES:
154154
for _, row := range scanRes.KVs {
155155
k := copyKey(row.Key)
156-
acq, err := acquireLockOnKey(ctx, readWriter, txn, str, dur, k, ms, settings)
156+
acq, err := acquireLockOnKey(ctx, readWriter, txn, str, dur, k, ms, settings, false /* allowSequenceNumberRegression */)
157157
if err != nil {
158158
return nil, err
159159
}
@@ -198,6 +198,7 @@ func acquireLockOnKey(
198198
key roachpb.Key,
199199
ms *enginepb.MVCCStats,
200200
settings *cluster.Settings,
201+
allowSequenceNumberRegression bool,
201202
) (roachpb.LockAcquisition, error) {
202203
maxLockConflicts := storage.MaxConflictsPerLockConflictError.Get(&settings.SV)
203204
targetLockConflictBytes := storage.TargetBytesPerLockConflictError.Get(&settings.SV)
@@ -233,7 +234,8 @@ func acquireLockOnKey(
233234
// conflicts with un-contended replicated locks -- we need to do so before
234235
// we can acquire our own replicated lock; do that now, and also acquire
235236
// the replicated lock if no conflicts are found.
236-
if err := storage.MVCCAcquireLock(ctx, readWriter, &txn.TxnMeta, txn.IgnoredSeqNums, str, key, ms, maxLockConflicts, targetLockConflictBytes, false); err != nil {
237+
if err := storage.MVCCAcquireLock(ctx, readWriter, &txn.TxnMeta, txn.IgnoredSeqNums, str,
238+
key, ms, maxLockConflicts, targetLockConflictBytes, allowSequenceNumberRegression); err != nil {
237239
return roachpb.LockAcquisition{}, err
238240
}
239241
default:

0 commit comments

Comments
 (0)