Skip to content

Commit 89e0848

Browse files
craig[bot]iskettaneh
andcommitted
Merge #157153
157153: spanset: assert that batches don't access store local and unreplicated RangeID local keys r=iskettaneh a=iskettaneh This PR adds the following test-only assertions: 1) Generated batches don't touch store-local keys. 2) Generated batches don't touch unreplicated RangeID local keys. We disable the check in exactly 3 locations we know that we currently touch those keys. Fixes: #156537 Release note: None Co-authored-by: iskettaneh <[email protected]>
2 parents ab5107e + 29dc524 commit 89e0848

14 files changed

+528
-107
lines changed

pkg/keys/constants.go

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -163,6 +163,7 @@ var (
163163
//
164164
// LocalStorePrefix is the prefix identifying per-store data.
165165
LocalStorePrefix = makeKey(LocalPrefix, roachpb.Key("s"))
166+
LocalStoreMax = roachpb.Key(LocalStorePrefix).PrefixEnd()
166167
// localStoreClusterVersionSuffix stores the cluster-wide version
167168
// information for this store, updated any time the operator
168169
// updates the minimum cluster version.

pkg/keys/keys.go

Lines changed: 11 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -308,6 +308,17 @@ func DecodeRangeIDKey(
308308
return roachpb.RangeID(rangeInt), infix, suffix, b, nil
309309
}
310310

311+
// DecodeRangeIDPrefix parses a local range ID prefix into range ID.
312+
func DecodeRangeIDPrefix(key roachpb.Key) (roachpb.RangeID, error) {
313+
if !bytes.HasPrefix(key, LocalRangeIDPrefix) {
314+
return 0, errors.Errorf("key %s does not have %s prefix", key, LocalRangeIDPrefix)
315+
}
316+
// Cut the prefix, the Range ID, and the infix specifier.
317+
b := key[len(LocalRangeIDPrefix):]
318+
_, rangeInt, err := encoding.DecodeUvarintAscending(b)
319+
return roachpb.RangeID(rangeInt), err
320+
}
321+
311322
// AbortSpanKey returns a range-local key by Range ID for an
312323
// AbortSpan entry, with detail specified by encoding the
313324
// supplied transaction ID.

pkg/kv/kvserver/batcheval/cmd_add_sstable.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -337,7 +337,7 @@ func EvalAddSSTable(
337337
// addition, and instead just use this key-only iterator. If a caller actually
338338
// needs to know what data is there, it must issue its own real Scan.
339339
if args.ReturnFollowingLikelyNonEmptySpanStart {
340-
existingIter, err := spanset.DisableReaderAssertions(readWriter).NewMVCCIterator(
340+
existingIter, err := spanset.DisableUndeclaredSpanAssertions(readWriter).NewMVCCIterator(
341341
ctx,
342342
storage.MVCCKeyIterKind, // don't care if it is committed or not, just that it isn't empty.
343343
storage.IterOptions{

pkg/kv/kvserver/batcheval/cmd_delete.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -58,7 +58,7 @@ func Delete(
5858
// If requested, replace point tombstones with range tombstones.
5959
if cArgs.EvalCtx.EvalKnobs().UseRangeTombstonesForPointDeletes && h.Txn == nil {
6060
if err := storage.ReplacePointTombstonesWithRangeTombstones(
61-
ctx, spanset.DisableReadWriterAssertions(readWriter),
61+
ctx, spanset.DisableUndeclaredSpanAssertions(readWriter),
6262
cArgs.Stats, args.Key, args.EndKey); err != nil {
6363
return result.Result{}, err
6464
}

pkg/kv/kvserver/batcheval/cmd_delete_range.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -302,7 +302,7 @@ func deleteRangeTransactional(
302302
// If requested, replace point tombstones with range tombstones.
303303
if cArgs.EvalCtx.EvalKnobs().UseRangeTombstonesForPointDeletes && h.Txn == nil {
304304
if err := storage.ReplacePointTombstonesWithRangeTombstones(
305-
ctx, spanset.DisableReadWriterAssertions(readWriter),
305+
ctx, spanset.DisableUndeclaredSpanAssertions(readWriter),
306306
cArgs.Stats, args.Key, args.EndKey); err != nil {
307307
return result.Result{}, err
308308
}

pkg/kv/kvserver/batcheval/cmd_end_transaction.go

Lines changed: 7 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -719,7 +719,7 @@ func resolveLocalLocksWithPagination(
719719
// If requested, replace point tombstones with range tombstones.
720720
if ok && evalCtx.EvalKnobs().UseRangeTombstonesForPointDeletes {
721721
if err := storage.ReplacePointTombstonesWithRangeTombstones(
722-
ctx, spanset.DisableReadWriterAssertions(readWriter),
722+
ctx, spanset.DisableUndeclaredSpanAssertions(readWriter),
723723
ms, update.Key, update.EndKey); err != nil {
724724
return 0, 0, 0, errors.Wrapf(err,
725725
"replacing point tombstones with range tombstones for write intent at %s on end transaction [%s]",
@@ -757,7 +757,7 @@ func resolveLocalLocksWithPagination(
757757
// If requested, replace point tombstones with range tombstones.
758758
if evalCtx.EvalKnobs().UseRangeTombstonesForPointDeletes {
759759
if err := storage.ReplacePointTombstonesWithRangeTombstones(
760-
ctx, spanset.DisableReadWriterAssertions(readWriter),
760+
ctx, spanset.DisableUndeclaredSpanAssertions(readWriter),
761761
ms, update.Key, update.EndKey); err != nil {
762762
return 0, 0, 0, errors.Wrapf(err,
763763
"replacing point tombstones with range tombstones for write intent range at %s on end transaction [%s]",
@@ -1340,8 +1340,10 @@ func splitTriggerHelper(
13401340
if err != nil {
13411341
return enginepb.MVCCStats{}, result.Result{}, errors.Wrap(err, "unable to fetch last replica GC timestamp")
13421342
}
1343+
13431344
if err := storage.MVCCPutProto(
1344-
ctx, batch, keys.RangeLastReplicaGCTimestampKey(split.RightDesc.RangeID), hlc.Timestamp{},
1345+
ctx, spanset.DisableForbiddenSpanAssertions(batch),
1346+
keys.RangeLastReplicaGCTimestampKey(split.RightDesc.RangeID), hlc.Timestamp{},
13451347
&replicaGCTS, storage.MVCCWriteOptions{Category: fs.BatchEvalReadCategory}); err != nil {
13461348
return enginepb.MVCCStats{}, result.Result{}, errors.Wrap(err, "unable to copy last replica GC timestamp")
13471349
}
@@ -1541,7 +1543,8 @@ func splitTriggerHelper(
15411543
// as all replicas will be responsible for writing it locally before
15421544
// applying the split.
15431545
if !rec.ClusterSettings().Version.IsActive(ctx, clusterversion.V25_4_WriteInitialTruncStateBeforeSplitApplication) {
1544-
if err := kvstorage.WriteInitialTruncState(ctx, batch, split.RightDesc.RangeID); err != nil {
1546+
if err := kvstorage.WriteInitialTruncState(ctx,
1547+
spanset.DisableForbiddenSpanAssertions(batch), split.RightDesc.RangeID); err != nil {
15451548
return enginepb.MVCCStats{}, result.Result{}, errors.Wrap(err, "unable to write initial Replica state")
15461549
}
15471550
}

pkg/kv/kvserver/batcheval/cmd_resolve_intent.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -129,7 +129,7 @@ func ResolveIntent(
129129
// If requested, replace point tombstones with range tombstones.
130130
if ok && cArgs.EvalCtx.EvalKnobs().UseRangeTombstonesForPointDeletes {
131131
if err := storage.ReplacePointTombstonesWithRangeTombstones(ctx,
132-
spanset.DisableReadWriterAssertions(readWriter), ms, update.Key, update.EndKey); err != nil {
132+
spanset.DisableUndeclaredSpanAssertions(readWriter), ms, update.Key, update.EndKey); err != nil {
133133
return result.Result{}, err
134134
}
135135
}

pkg/kv/kvserver/batcheval/cmd_resolve_intent_range.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -99,7 +99,7 @@ func ResolveIntentRange(
9999
// If requested, replace point tombstones with range tombstones.
100100
if cArgs.EvalCtx.EvalKnobs().UseRangeTombstonesForPointDeletes {
101101
if err := storage.ReplacePointTombstonesWithRangeTombstones(ctx,
102-
spanset.DisableReadWriterAssertions(readWriter), ms, args.Key, args.EndKey); err != nil {
102+
spanset.DisableUndeclaredSpanAssertions(readWriter), ms, args.Key, args.EndKey); err != nil {
103103
return result.Result{}, err
104104
}
105105
}

pkg/kv/kvserver/replica_eval_context_span.go

Lines changed: 7 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -110,7 +110,7 @@ func (rec *SpanSetReplicaEvalContext) IsFirstRange() bool {
110110
func (rec SpanSetReplicaEvalContext) Desc() *roachpb.RangeDescriptor {
111111
desc := rec.i.Desc()
112112
rec.ss.AssertAllowed(spanset.SpanReadOnly,
113-
roachpb.Span{Key: keys.RangeDescriptorKey(desc.StartKey)},
113+
spanset.TrickySpan{Key: keys.RangeDescriptorKey(desc.StartKey)},
114114
)
115115
return desc
116116
}
@@ -151,7 +151,7 @@ func (rec SpanSetReplicaEvalContext) CanCreateTxnRecord(
151151
ctx context.Context, txnID uuid.UUID, txnKey []byte, txnMinTS hlc.Timestamp,
152152
) (bool, kvpb.TransactionAbortedReason) {
153153
rec.ss.AssertAllowed(spanset.SpanReadOnly,
154-
roachpb.Span{Key: keys.TransactionKey(txnKey, txnID)},
154+
spanset.TrickySpan{Key: keys.TransactionKey(txnKey, txnID)},
155155
)
156156
return rec.i.CanCreateTxnRecord(ctx, txnID, txnKey, txnMinTS)
157157
}
@@ -163,7 +163,7 @@ func (rec SpanSetReplicaEvalContext) MinTxnCommitTS(
163163
ctx context.Context, txnID uuid.UUID, txnKey []byte,
164164
) hlc.Timestamp {
165165
rec.ss.AssertAllowed(spanset.SpanReadOnly,
166-
roachpb.Span{Key: keys.TransactionKey(txnKey, txnID)},
166+
spanset.TrickySpan{Key: keys.TransactionKey(txnKey, txnID)},
167167
)
168168
return rec.i.MinTxnCommitTS(ctx, txnID, txnKey)
169169
}
@@ -173,7 +173,7 @@ func (rec SpanSetReplicaEvalContext) MinTxnCommitTS(
173173
// not be served.
174174
func (rec SpanSetReplicaEvalContext) GetGCThreshold() hlc.Timestamp {
175175
rec.ss.AssertAllowed(spanset.SpanReadOnly,
176-
roachpb.Span{Key: keys.RangeGCThresholdKey(rec.GetRangeID())},
176+
spanset.TrickySpan{Key: keys.RangeGCThresholdKey(rec.GetRangeID())},
177177
)
178178
return rec.i.GetGCThreshold()
179179
}
@@ -197,7 +197,7 @@ func (rec SpanSetReplicaEvalContext) GetLastReplicaGCTimestamp(
197197
ctx context.Context,
198198
) (hlc.Timestamp, error) {
199199
if err := rec.ss.CheckAllowed(spanset.SpanReadOnly,
200-
roachpb.Span{Key: keys.RangeLastReplicaGCTimestampKey(rec.GetRangeID())},
200+
spanset.TrickySpan{Key: keys.RangeLastReplicaGCTimestampKey(rec.GetRangeID())},
201201
); err != nil {
202202
return hlc.Timestamp{}, err
203203
}
@@ -228,11 +228,11 @@ func (rec *SpanSetReplicaEvalContext) GetCurrentReadSummary(ctx context.Context)
228228
// To capture a read summary over the range, all keys must be latched for
229229
// writing to prevent any concurrent reads or writes.
230230
desc := rec.i.Desc()
231-
rec.ss.AssertAllowed(spanset.SpanReadWrite, roachpb.Span{
231+
rec.ss.AssertAllowed(spanset.SpanReadWrite, spanset.TrickySpan{
232232
Key: keys.MakeRangeKeyPrefix(desc.StartKey),
233233
EndKey: keys.MakeRangeKeyPrefix(desc.EndKey),
234234
})
235-
rec.ss.AssertAllowed(spanset.SpanReadWrite, roachpb.Span{
235+
rec.ss.AssertAllowed(spanset.SpanReadWrite, spanset.TrickySpan{
236236
Key: desc.StartKey.AsRawKey(),
237237
EndKey: desc.EndKey.AsRawKey(),
238238
})

pkg/kv/kvserver/replica_test.go

Lines changed: 126 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -2732,7 +2732,7 @@ func TestReplicaLatchingSplitDeclaresWrites(t *testing.T) {
27322732
{spanset.SpanReadWrite, roachpb.Key("b"), false},
27332733
{spanset.SpanReadWrite, roachpb.Key("d"), true},
27342734
} {
2735-
err := spans.CheckAllowed(tc.access, roachpb.Span{Key: tc.key})
2735+
err := spans.CheckAllowed(tc.access, spanset.TrickySpan{Key: tc.key})
27362736
if tc.expectAccess {
27372737
require.NoError(t, err)
27382738
} else {
@@ -15197,3 +15197,128 @@ func TestLeaderlessWatcherInit(t *testing.T) {
1519715197
t.Fatalf("expected LeaderlessWatcher channel to be closed")
1519815198
}
1519915199
}
15200+
15201+
// TestOverlapsUnreplicatedRangeIDLocalKeys verifies that the function
15202+
// overlapsUnreplicatedRangeIDLocalKeys() successfully catches any overlap with
15203+
// unreplicated rangeID local keys.
15204+
func TestOverlapsUnreplicatedRangeIDLocalKeys(t *testing.T) {
15205+
defer leaktest.AfterTest(t)()
15206+
defer log.Scope(t).Close(t)
15207+
15208+
s := func(start, end roachpb.Key) roachpb.Span {
15209+
return roachpb.Span{Key: start, EndKey: end}
15210+
}
15211+
testCases := []struct {
15212+
span roachpb.Span
15213+
notOk bool
15214+
}{
15215+
// Full spans not overlapping with unreplicated local RangeID spans.
15216+
{span: s(roachpb.KeyMin, keys.LocalRangeIDPrefix.AsRawKey())},
15217+
{span: s(keys.RangeForceFlushKey(1), keys.RangeLeaseKey(1))},
15218+
{span: s(keys.LocalRangeIDPrefix.AsRawKey().PrefixEnd(), roachpb.KeyMax)},
15219+
15220+
// Full spans overlapping with unreplicated local RangeID spans.
15221+
{span: s(roachpb.KeyMin, keys.MakeRangeIDUnreplicatedPrefix(1)), notOk: true}, // partial overlap
15222+
{span: s(roachpb.KeyMin, keys.RaftTruncatedStateKey(1)), notOk: true},
15223+
{span: s(keys.LocalRangeIDPrefix.AsRawKey(), keys.LocalRangeIDPrefix.AsRawKey().PrefixEnd()),
15224+
notOk: true},
15225+
{span: s(keys.RaftTruncatedStateKey(1), keys.RaftTruncatedStateKey(2)), notOk: true},
15226+
{span: s(keys.MakeRangeIDUnreplicatedPrefix(1).PrefixEnd(), roachpb.KeyMax), notOk: true}, // partial overlap
15227+
{span: s(keys.MakeRangeIDUnreplicatedPrefix(1),
15228+
keys.MakeRangeIDUnreplicatedPrefix(1).PrefixEnd()), notOk: true},
15229+
{span: s(keys.RaftTruncatedStateKey(1), roachpb.KeyMax), notOk: true},
15230+
15231+
// Point spans not overlapping with unreplicated local RangeID spans.
15232+
{span: s(roachpb.KeyMin, nil)},
15233+
{span: s(keys.LocalRangeIDPrefix.AsRawKey().Prevish(1), nil)},
15234+
{span: s(keys.MakeRangeIDUnreplicatedPrefix(1).Prevish(1), nil)},
15235+
{span: s(keys.RangeForceFlushKey(1), nil)},
15236+
{span: s(keys.MakeRangeIDUnreplicatedPrefix(1).PrefixEnd(), nil)},
15237+
{span: s(keys.LocalRangeIDPrefix.AsRawKey().PrefixEnd(), nil)},
15238+
{span: s(roachpb.KeyMax, nil)},
15239+
15240+
// Point spans overlapping with unreplicated local RangeID spans.
15241+
{span: s(keys.MakeRangeIDUnreplicatedPrefix(1), nil), notOk: true},
15242+
{span: s(keys.RangeTombstoneKey(1), nil), notOk: true},
15243+
{span: s(keys.RaftTruncatedStateKey(1), nil), notOk: true},
15244+
{span: s(keys.RaftTruncatedStateKey(2), nil), notOk: true},
15245+
{span: s(keys.MakeRangeIDUnreplicatedPrefix(1).PrefixEnd().Prevish(1), nil), notOk: true},
15246+
15247+
// Tricky spans not overlapping with unreplicated local RangeID spans.
15248+
{span: s(nil, keys.LocalRangeIDPrefix.AsRawKey())},
15249+
{span: s(nil, keys.MakeRangeIDUnreplicatedPrefix(1))},
15250+
{span: s(nil, keys.RangeForceFlushKey(1))},
15251+
{span: s(nil, keys.MakeRangeIDUnreplicatedPrefix(1).PrefixEnd().Next())},
15252+
{span: s(nil, keys.LocalRangeIDPrefix.AsRawKey().PrefixEnd().Next())},
15253+
15254+
// Tricky spans overlapping with unreplicated local RangeID spans.
15255+
{span: s(nil, keys.MakeRangeIDUnreplicatedPrefix(1).Next()), notOk: true},
15256+
{span: s(nil, keys.RangeTombstoneKey(1).Next()), notOk: true},
15257+
{span: s(nil, keys.RaftTruncatedStateKey(1).Next()), notOk: true},
15258+
{span: s(nil, keys.RaftTruncatedStateKey(2).Next()), notOk: true},
15259+
{span: s(nil, keys.MakeRangeIDUnreplicatedPrefix(1).PrefixEnd()), notOk: true},
15260+
{span: s(nil, keys.LocalRangeIDPrefix.AsRawKey().PrefixEnd()), notOk: true}, // can't decode RangeID.
15261+
}
15262+
15263+
for _, tc := range testCases {
15264+
t.Run("", func(t *testing.T) {
15265+
err := overlapsUnreplicatedRangeIDLocalKeys(spanset.TrickySpan(tc.span))
15266+
require.Equal(t, tc.notOk, err != nil, tc.span)
15267+
})
15268+
}
15269+
}
15270+
15271+
// TestOverlapsStoreLocalKeys verifies that the function
15272+
// overlapsStoreLocalKeys() successfully catches any overlap with
15273+
// store local keys.
15274+
func TestOverlapsStoreLocalKeys(t *testing.T) {
15275+
defer leaktest.AfterTest(t)()
15276+
defer log.Scope(t).Close(t)
15277+
15278+
s := func(start, end roachpb.Key) roachpb.Span {
15279+
return roachpb.Span{Key: start, EndKey: end}
15280+
}
15281+
15282+
testCases := []struct {
15283+
span roachpb.Span
15284+
notOK bool
15285+
}{
15286+
// Full spans not overlapping with Store-local span.
15287+
{span: s(roachpb.KeyMin, keys.LocalStorePrefix)},
15288+
{span: s(keys.LocalStoreMax, roachpb.KeyMax)},
15289+
15290+
// Full spans overlapping with Store-local span.
15291+
{span: s(roachpb.KeyMin, roachpb.Key(keys.LocalStorePrefix).Next()), notOK: true},
15292+
{span: s(keys.LocalStorePrefix, keys.LocalStoreMax), notOK: true},
15293+
{span: s(keys.StoreGossipKey(), keys.StoreIdentKey()), notOK: true},
15294+
{span: s(keys.LocalStoreMax.Prevish(1), roachpb.KeyMax), notOK: true},
15295+
15296+
// Point spans not overlapping with Store-local span.
15297+
{span: s(roachpb.KeyMin, nil)},
15298+
{span: s(roachpb.Key(keys.LocalStorePrefix).Prevish(1), nil)},
15299+
{span: s(keys.LocalStoreMax, nil)},
15300+
{span: s(keys.LocalStoreMax.Next(), nil)},
15301+
{span: s(roachpb.KeyMax, nil)},
15302+
15303+
// Point spans overlapping with Store-local span.
15304+
{span: s(keys.LocalStorePrefix, nil), notOK: true},
15305+
{span: s(keys.StoreGossipKey(), nil), notOK: true},
15306+
{span: s(keys.LocalStoreMax.Prevish(1), nil), notOK: true},
15307+
15308+
// Tricky spans with nil StartKey not overlapping with Store-local span.
15309+
{span: s(nil, keys.LocalStorePrefix)},
15310+
{span: s(nil, keys.LocalStoreMax.Next())},
15311+
15312+
// Tricky spans with nil StartKey overlapping with Store-local span.
15313+
{span: s(nil, roachpb.Key(keys.LocalStorePrefix).Next()), notOK: true},
15314+
{span: s(nil, keys.StoreGossipKey()), notOK: true},
15315+
{span: s(nil, keys.LocalStoreMax), notOK: true},
15316+
}
15317+
15318+
for _, tc := range testCases {
15319+
t.Run("", func(t *testing.T) {
15320+
err := overlapsStoreLocalKeys(spanset.TrickySpan(tc.span))
15321+
require.Equal(t, tc.notOK, err != nil, tc.span)
15322+
})
15323+
}
15324+
}

0 commit comments

Comments
 (0)