Skip to content

Commit 1b9e9be

Browse files
committed
kvserver,storage: handle ExpectExclusionSince during eval
Epic: none Release note: None
1 parent 4fbc6ec commit 1b9e9be

File tree

6 files changed

+39
-81
lines changed

6 files changed

+39
-81
lines changed

pkg/kv/kvserver/batcheval/cmd_get.go

Lines changed: 17 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -15,6 +15,7 @@ import (
1515
"github.com/cockroachdb/cockroach/pkg/storage"
1616
"github.com/cockroachdb/cockroach/pkg/storage/fs"
1717
"github.com/cockroachdb/cockroach/pkg/util/log"
18+
"github.com/cockroachdb/errors"
1819
)
1920

2021
func init() {
@@ -42,24 +43,23 @@ func Get(
4243
}
4344

4445
readTimestamp := h.Timestamp
45-
moreRecentPolicy := storage.IgnoreMoreRecent
46+
failOnMoreRecent := args.KeyLockingStrength != lock.None
47+
expectExclusionSince := !args.ExpectExclusionSince.IsEmpty()
4648

4749
// If ExpectExclusionSince is set, use it as the read timestamp and set the
4850
// MoreRecentPolicy to ExclusionViolationErrorOnMoreRecent to ensure that an
4951
// exclusion violation error is returned if a write has occurred since the
5052
// exclusion timestamp.
51-
if !args.ExpectExclusionSince.IsEmpty() {
53+
if expectExclusionSince {
5254
readTimestamp = args.ExpectExclusionSince
53-
moreRecentPolicy = storage.ExclusionViolationErrorOnMoreRecent
54-
} else if args.KeyLockingStrength != lock.None {
55-
moreRecentPolicy = storage.WriteTooOldErrorOnMoreRecent
55+
failOnMoreRecent = true
5656
}
5757

5858
getRes, err := storage.MVCCGet(ctx, readWriter, args.Key, readTimestamp, storage.MVCCGetOptions{
5959
Inconsistent: h.ReadConsistency != kvpb.CONSISTENT,
6060
SkipLocked: h.WaitPolicy == lock.WaitPolicy_SkipLocked,
6161
Txn: h.Txn,
62-
MoreRecentPolicy: moreRecentPolicy,
62+
FailOnMoreRecent: failOnMoreRecent,
6363
ScanStats: cArgs.ScanStats,
6464
Uncertainty: cArgs.Uncertainty,
6565
MemoryAccount: cArgs.EvalCtx.GetResponseMemoryAccount(),
@@ -72,6 +72,17 @@ func Get(
7272
ReturnRawMVCCValues: args.ReturnRawMVCCValues,
7373
})
7474
if err != nil {
75+
// If the user has set ExpectExclusionSince, transform any WriteTooOld error
76+
// into an ExclusionViolationError.
77+
if expectExclusionSince {
78+
if wtoErr := (*kvpb.WriteTooOldError)(nil); errors.As(err, &wtoErr) {
79+
err = kvpb.NewExclusionViolationError(
80+
readTimestamp,
81+
wtoErr.ActualTimestamp.Prev(),
82+
args.Key,
83+
)
84+
}
85+
}
7586
return result.Result{}, err
7687
}
7788
reply.ResumeSpan = getRes.ResumeSpan

pkg/storage/mvcc.go

Lines changed: 4 additions & 23 deletions
Original file line numberDiff line numberDiff line change
@@ -1230,28 +1230,13 @@ type LockTableView interface {
12301230
Close()
12311231
}
12321232

1233-
// MVCCGetMoreRecentPolicy determines how MVCCGet should handle more recent
1234-
// versions of a key.
1235-
type MVCCGetMoreRecentPolicy int
1236-
1237-
const (
1238-
// IgnoreMoreRecent indicates that MVCCGet should ignore more recent versions of a key.
1239-
IgnoreMoreRecent MVCCGetMoreRecentPolicy = iota
1240-
// WriteTooOldErrorOnMoreRecent indicates that MVCCGet should return a WriteTooOldErrorOnMoreRecent
1241-
// if a more recent version of a key is found.
1242-
WriteTooOldErrorOnMoreRecent
1243-
// ExclusionViolationErrorOnMoreRecent indicates that MVCCGet should return an ExclusionLostError
1244-
// if a more recent version of a key is found.
1245-
ExclusionViolationErrorOnMoreRecent
1246-
)
1247-
12481233
// MVCCGetOptions bundles options for the MVCCGet family of functions.
12491234
type MVCCGetOptions struct {
12501235
// See the documentation for MVCCGet for information on these parameters.
12511236
Inconsistent bool
12521237
SkipLocked bool
12531238
Tombstones bool
1254-
MoreRecentPolicy MVCCGetMoreRecentPolicy
1239+
FailOnMoreRecent bool
12551240
Txn *roachpb.Transaction
12561241
ScanStats *kvpb.ScanStats
12571242
Uncertainty uncertainty.Interval
@@ -1317,7 +1302,7 @@ func (opts *MVCCGetOptions) validate() error {
13171302
if opts.Inconsistent && opts.SkipLocked {
13181303
return errors.Errorf("cannot allow inconsistent reads with skip locked option")
13191304
}
1320-
if opts.Inconsistent && opts.MoreRecentPolicy != IgnoreMoreRecent {
1305+
if opts.Inconsistent && opts.FailOnMoreRecent {
13211306
return errors.Errorf("cannot allow inconsistent reads with fail on more recent option")
13221307
}
13231308
if opts.DontInterleaveIntents && opts.SkipLocked {
@@ -1587,7 +1572,7 @@ func mvccGet(
15871572
skipLocked: opts.SkipLocked,
15881573
tombstones: opts.Tombstones,
15891574
rawMVCCValues: opts.ReturnRawMVCCValues,
1590-
moreRecentPolicy: opts.MoreRecentPolicy,
1575+
failOnMoreRecent: opts.FailOnMoreRecent,
15911576
keyBuf: mvccScanner.keyBuf,
15921577
decodeMVCCHeaders: true,
15931578
}
@@ -4466,10 +4451,6 @@ func mvccScanInit(
44664451
if opts.MemoryAccount != nil {
44674452
memAccount = opts.MemoryAccount
44684453
}
4469-
moreRecentPolicy := IgnoreMoreRecent
4470-
if opts.FailOnMoreRecent {
4471-
moreRecentPolicy = WriteTooOldErrorOnMoreRecent
4472-
}
44734454
*mvccScanner = pebbleMVCCScanner{
44744455
parent: iter,
44754456
memAccount: memAccount,
@@ -4488,7 +4469,7 @@ func mvccScanInit(
44884469
inconsistent: opts.Inconsistent,
44894470
skipLocked: opts.SkipLocked,
44904471
tombstones: opts.Tombstones,
4491-
moreRecentPolicy: moreRecentPolicy,
4472+
failOnMoreRecent: opts.FailOnMoreRecent,
44924473
keyBuf: mvccScanner.keyBuf,
44934474
// NB: If the `results` argument passed to this function is a pointer to
44944475
// mvccScanner.alloc.pebbleResults, we don't want to overwrite any

pkg/storage/mvcc_history_test.go

Lines changed: 2 additions & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -94,7 +94,7 @@ var (
9494
// increment [t=<name>] [ts=<int>[,<int>]] [localTs=<int>[,<int>]] [resolve [status=<txnstatus>]] [ambiguousReplay] [maxLockConflicts=<int>] [targetLockConflictBytes=<int>] k=<key> [inc=<val>]
9595
// put [t=<name>] [ts=<int>[,<int>]] [localTs=<int>[,<int>]] [resolve [status=<txnstatus>]] [ambiguousReplay] [maxLockConflicts=<int>] k=<key> v=<string> [raw]
9696
// put_rangekey ts=<int>[,<int>] [localTs=<int>[,<int>]] k=<key> end=<key> [syntheticBit]
97-
// put_blind_inline k=<key> v=<string> [prev=<string>]
97+
// put_blind_inline k=<key> v=<string> [prev=<string>]
9898
// get [t=<name>] [ts=<int>[,<int>]] [resolve [status=<txnstatus>]] k=<key> [inconsistent] [skipLocked] [tombstones] [failOnMoreRecent] [localUncertaintyLimit=<int>[,<int>]] [globalUncertaintyLimit=<int>[,<int>]] [maxKeys=<int>] [targetBytes=<int>] [allowEmpty]
9999
// scan [t=<name>] [ts=<int>[,<int>]] [resolve [status=<txnstatus>]] k=<key> [end=<key>] [inconsistent] [skipLocked] [tombstones] [reverse] [failOnMoreRecent] [localUncertaintyLimit=<int>[,<int>]] [globalUncertaintyLimit=<int>[,<int>]] [max=<max>] [targetbytes=<target>] [wholeRows[=<int>]] [allowEmpty]
100100
// export [k=<key>] [end=<key>] [ts=<int>[,<int>]] [kTs=<int>[,<int>]] [startTs=<int>[,<int>]] [maxLockConflicts=<int>] [targetLockConflictBytes=<int>] [allRevisions] [targetSize=<int>] [maxSize=<int>] [stopMidKey] [fingerprint]
@@ -1396,24 +1396,8 @@ func cmdGet(e *evalCtx) error {
13961396
opts.Tombstones = true
13971397
}
13981398
if e.hasArg("failOnMoreRecent") {
1399-
opts.MoreRecentPolicy = storage.WriteTooOldErrorOnMoreRecent
1400-
}
1401-
1402-
if e.hasArg("moreRecentPolicy") {
1403-
var moreRecentPolicyStr string
1404-
e.scanArg("moreRecentPolicy", &moreRecentPolicyStr)
1405-
switch moreRecentPolicyStr {
1406-
case "ignoreMoreRecent":
1407-
opts.MoreRecentPolicy = storage.IgnoreMoreRecent
1408-
case "writeTooOldErrorOnMoreRecent":
1409-
opts.MoreRecentPolicy = storage.WriteTooOldErrorOnMoreRecent
1410-
case "exclusionViolationErrorOnMoreRecent":
1411-
opts.MoreRecentPolicy = storage.ExclusionViolationErrorOnMoreRecent
1412-
default:
1413-
e.Fatalf("unknown moreRecentPolicy: %s", moreRecentPolicyStr)
1414-
}
1399+
opts.FailOnMoreRecent = true
14151400
}
1416-
14171401
opts.Uncertainty = uncertainty.Interval{
14181402
GlobalLimit: e.getTsWithName("globalUncertaintyLimit"),
14191403
LocalLimit: hlc.ClockTimestamp(e.getTsWithName("localUncertaintyLimit")),

pkg/storage/pebble_mvcc_scanner.go

Lines changed: 13 additions & 19 deletions
Original file line numberDiff line numberDiff line change
@@ -444,7 +444,7 @@ type pebbleMVCCScanner struct {
444444
// extended encoding of any returned value. This includes the
445445
// MVCCValueHeader.
446446
rawMVCCValues bool
447-
moreRecentPolicy MVCCGetMoreRecentPolicy
447+
failOnMoreRecent bool
448448
keyBuf []byte
449449
savedBuf []byte
450450
lazyFetcherBuf pebble.LazyFetcher
@@ -571,12 +571,12 @@ func (p *pebbleMVCCScanner) init(
571571
// future-time timestamps with earlier local timestamps. We are only able
572572
// to skip uncertainty checks if p.ts >= global_uncertainty_limit.
573573
//
574-
// We disable checkUncertainty when the scanner is configured with a
575-
// moreRecentPolicy other than IgnoreMoreRecent. This avoids cases in which a
576-
// scan would have failed with a WriteTooOldError or ExclusionViolationError
577-
// but instead gets an unexpected ReadWithinUncertaintyIntervalError See:
574+
// We disable checkUncertainty when the scanner is configured with failOnMoreRecent.
575+
// This avoids cases in which a scan would have failed with a WriteTooOldError
576+
// but instead gets an unexpected ReadWithinUncertaintyIntervalError
577+
// See:
578578
// https://github.com/cockroachdb/cockroach/issues/119681
579-
p.checkUncertainty = p.ts.Less(p.uncertainty.GlobalLimit) && p.moreRecentPolicy == IgnoreMoreRecent
579+
p.checkUncertainty = p.ts.Less(p.uncertainty.GlobalLimit) && !p.failOnMoreRecent
580580
}
581581

582582
// get seeks to the start key exactly once and adds one KV to the result set.
@@ -793,15 +793,9 @@ func (p *pebbleMVCCScanner) maybeFailOnMoreRecent() {
793793
if p.err != nil || p.mostRecentTS.IsEmpty() {
794794
return
795795
}
796-
797-
switch p.moreRecentPolicy {
798-
case WriteTooOldErrorOnMoreRecent:
799-
// The txn can't write at the existing timestamp, so we provide the error
800-
// with the timestamp immediately after it.
801-
p.err = kvpb.NewWriteTooOldError(p.ts, p.mostRecentTS.Next(), p.mostRecentKey)
802-
case ExclusionViolationErrorOnMoreRecent:
803-
p.err = kvpb.NewExclusionViolationError(p.ts, p.mostRecentTS, p.mostRecentKey)
804-
}
796+
// The txn can't write at the existing timestamp, so we provide the error
797+
// with the timestamp immediately after it.
798+
p.err = kvpb.NewWriteTooOldError(p.ts, p.mostRecentTS.Next(), p.mostRecentKey)
805799
p.results.clear()
806800
p.intents.Reset()
807801
}
@@ -867,7 +861,7 @@ func (p *pebbleMVCCScanner) getOne(ctx context.Context) (ok, added bool) {
867861

868862
// ts == read_ts
869863
if p.curUnsafeKey.Timestamp == p.ts {
870-
if p.moreRecentPolicy != IgnoreMoreRecent {
864+
if p.failOnMoreRecent {
871865
// 2. Our txn's read timestamp is equal to the most recent
872866
// version's timestamp and the scanner has been configured to
873867
// throw a write too old error on equal or more recent versions.
@@ -899,7 +893,7 @@ func (p *pebbleMVCCScanner) getOne(ctx context.Context) (ok, added bool) {
899893
}
900894

901895
// ts > read_ts
902-
if p.moreRecentPolicy != IgnoreMoreRecent {
896+
if p.failOnMoreRecent {
903897
// 4. Our txn's read timestamp is less than the most recent
904898
// version's timestamp and the scanner has been configured to
905899
// throw a write too old error on equal or more recent versions.
@@ -982,7 +976,7 @@ func (p *pebbleMVCCScanner) getOne(ctx context.Context) (ok, added bool) {
982976

983977
ownIntent := p.txn != nil && p.meta.Txn.ID.Equal(p.txn.ID)
984978
if !ownIntent {
985-
conflictingIntent := metaTS.LessEq(p.ts) || (p.moreRecentPolicy != IgnoreMoreRecent)
979+
conflictingIntent := metaTS.LessEq(p.ts) || p.failOnMoreRecent
986980
if !conflictingIntent {
987981
// 8. The key contains an intent, but we're reading below the intent.
988982
// Seek to the desired version, checking for uncertainty if necessary.
@@ -1578,7 +1572,7 @@ func (p *pebbleMVCCScanner) processRangeKeys(seeked bool, reverse bool) bool {
15781572
// Check for conflicts with range keys at or above the read timestamp.
15791573
// We don't need to handle e.g. skipLocked, because range keys don't
15801574
// currently have intents.
1581-
if p.moreRecentPolicy != IgnoreMoreRecent {
1575+
if p.failOnMoreRecent {
15821576
if key := p.parent.UnsafeKey(); !hasPoint || !key.Timestamp.IsEmpty() {
15831577
if newest := p.curRangeKeys.Newest(); p.ts.LessEq(newest) {
15841578
if p.mostRecentTS.Forward(newest) {

pkg/storage/pebble_mvcc_scanner_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -95,7 +95,7 @@ func TestMVCCScanWithManyVersionsAndSeparatedIntents(t *testing.T) {
9595
ts: ts,
9696
inconsistent: false,
9797
tombstones: false,
98-
moreRecentPolicy: IgnoreMoreRecent,
98+
failOnMoreRecent: false,
9999
}
100100
var results pebbleResults
101101
mvccScanner.init(nil /* txn */, uncertainty.Interval{}, &results)

pkg/storage/testdata/mvcc_histories/read_fail_on_more_recent

Lines changed: 2 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,7 @@ meta: "k2"/0,0 -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=
2121
data: "k2"/10.000000000,0 -> /BYTES/v
2222

2323
# Test cases:
24-
#
24+
#
2525
# for k in (k1, k2):
2626
# for op in (get, scan):
2727
# for ts in (9, 10, 11):
@@ -51,12 +51,6 @@ get k=k1 ts=10,0 failOnMoreRecent
5151
get: "k1" -> <no data>
5252
error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k1" at timestamp 10.000000000,0 too old; must write at or above 10.000000000,1
5353

54-
run error
55-
get k=k1 ts=10,0 moreRecentPolicy=exclusionViolationErrorOnMoreRecent
56-
----
57-
get: "k1" -> <no data>
58-
error: (*kvpb.ExclusionViolationError:) write exclusion on key "k1" expected since 10.000000000,0 but found write at 10.000000000,0
59-
6054
run ok
6155
get k=k1 ts=11,0
6256
----
@@ -170,7 +164,7 @@ scan: "k2"-"k3" -> <no data>
170164
error: (*kvpb.LockConflictError:) conflicting locks on "k2"
171165

172166
# More test cases:
173-
#
167+
#
174168
# span = [k1, k3)
175169
# op = scan
176170
# for ts in (9, 10, 11):
@@ -221,12 +215,6 @@ get k=k1 ts=9,0 inconsistent failOnMoreRecent
221215
get: "k1" -> <no data>
222216
error: (*withstack.withStack:) cannot allow inconsistent reads with fail on more recent option
223217

224-
run error
225-
get k=k1 ts=9,0 inconsistent moreRecentPolicy=exclusionViolationErrorOnMoreRecent
226-
----
227-
get: "k1" -> <no data>
228-
error: (*withstack.withStack:) cannot allow inconsistent reads with fail on more recent option
229-
230218
run error
231219
scan k=k1 end=k2 ts=9,0 inconsistent failOnMoreRecent
232220
----

0 commit comments

Comments
 (0)