From 79bc9cdf7361803df5db646d1e1521f815d96e8f Mon Sep 17 00:00:00 2001 From: iskettaneh <173953022+iskettaneh@users.noreply.github.com> Date: Mon, 10 Nov 2025 19:51:55 -0500 Subject: [PATCH 1/3] kvserver/spanset: introduce forbidden spans matchers This commit introduces the concept of forbidden spans, which will allow us to assert our batches don't modify Raft's engine keys. --- pkg/kv/kvserver/batcheval/cmd_add_sstable.go | 2 +- pkg/kv/kvserver/batcheval/cmd_delete.go | 2 +- pkg/kv/kvserver/batcheval/cmd_delete_range.go | 2 +- .../kvserver/batcheval/cmd_end_transaction.go | 4 +- .../kvserver/batcheval/cmd_resolve_intent.go | 2 +- .../batcheval/cmd_resolve_intent_range.go | 2 +- pkg/kv/kvserver/spanset/batch.go | 54 ++++++++++++++++++- pkg/kv/kvserver/spanset/spanset.go | 51 ++++++++++++++++-- 8 files changed, 107 insertions(+), 12 deletions(-) diff --git a/pkg/kv/kvserver/batcheval/cmd_add_sstable.go b/pkg/kv/kvserver/batcheval/cmd_add_sstable.go index 3d399e727833..494158412478 100644 --- a/pkg/kv/kvserver/batcheval/cmd_add_sstable.go +++ b/pkg/kv/kvserver/batcheval/cmd_add_sstable.go @@ -337,7 +337,7 @@ func EvalAddSSTable( // addition, and instead just use this key-only iterator. If a caller actually // needs to know what data is there, it must issue its own real Scan. if args.ReturnFollowingLikelyNonEmptySpanStart { - existingIter, err := spanset.DisableReaderAssertions(readWriter).NewMVCCIterator( + existingIter, err := spanset.DisableUndeclaredSpanAssertions(readWriter).NewMVCCIterator( ctx, storage.MVCCKeyIterKind, // don't care if it is committed or not, just that it isn't empty. storage.IterOptions{ diff --git a/pkg/kv/kvserver/batcheval/cmd_delete.go b/pkg/kv/kvserver/batcheval/cmd_delete.go index 56a8525649d3..4eb9b68fa652 100644 --- a/pkg/kv/kvserver/batcheval/cmd_delete.go +++ b/pkg/kv/kvserver/batcheval/cmd_delete.go @@ -58,7 +58,7 @@ func Delete( // If requested, replace point tombstones with range tombstones. if cArgs.EvalCtx.EvalKnobs().UseRangeTombstonesForPointDeletes && h.Txn == nil { if err := storage.ReplacePointTombstonesWithRangeTombstones( - ctx, spanset.DisableReadWriterAssertions(readWriter), + ctx, spanset.DisableUndeclaredSpanAssertions(readWriter), cArgs.Stats, args.Key, args.EndKey); err != nil { return result.Result{}, err } diff --git a/pkg/kv/kvserver/batcheval/cmd_delete_range.go b/pkg/kv/kvserver/batcheval/cmd_delete_range.go index 8cc9cfea705f..3a33a05b4725 100644 --- a/pkg/kv/kvserver/batcheval/cmd_delete_range.go +++ b/pkg/kv/kvserver/batcheval/cmd_delete_range.go @@ -302,7 +302,7 @@ func deleteRangeTransactional( // If requested, replace point tombstones with range tombstones. if cArgs.EvalCtx.EvalKnobs().UseRangeTombstonesForPointDeletes && h.Txn == nil { if err := storage.ReplacePointTombstonesWithRangeTombstones( - ctx, spanset.DisableReadWriterAssertions(readWriter), + ctx, spanset.DisableUndeclaredSpanAssertions(readWriter), cArgs.Stats, args.Key, args.EndKey); err != nil { return result.Result{}, err } diff --git a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go index 43a0b2640654..a3667df1fffe 100644 --- a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go +++ b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go @@ -719,7 +719,7 @@ func resolveLocalLocksWithPagination( // If requested, replace point tombstones with range tombstones. if ok && evalCtx.EvalKnobs().UseRangeTombstonesForPointDeletes { if err := storage.ReplacePointTombstonesWithRangeTombstones( - ctx, spanset.DisableReadWriterAssertions(readWriter), + ctx, spanset.DisableUndeclaredSpanAssertions(readWriter), ms, update.Key, update.EndKey); err != nil { return 0, 0, 0, errors.Wrapf(err, "replacing point tombstones with range tombstones for write intent at %s on end transaction [%s]", @@ -757,7 +757,7 @@ func resolveLocalLocksWithPagination( // If requested, replace point tombstones with range tombstones. if evalCtx.EvalKnobs().UseRangeTombstonesForPointDeletes { if err := storage.ReplacePointTombstonesWithRangeTombstones( - ctx, spanset.DisableReadWriterAssertions(readWriter), + ctx, spanset.DisableUndeclaredSpanAssertions(readWriter), ms, update.Key, update.EndKey); err != nil { return 0, 0, 0, errors.Wrapf(err, "replacing point tombstones with range tombstones for write intent range at %s on end transaction [%s]", diff --git a/pkg/kv/kvserver/batcheval/cmd_resolve_intent.go b/pkg/kv/kvserver/batcheval/cmd_resolve_intent.go index 591a6d09a791..be5adb69a81f 100644 --- a/pkg/kv/kvserver/batcheval/cmd_resolve_intent.go +++ b/pkg/kv/kvserver/batcheval/cmd_resolve_intent.go @@ -129,7 +129,7 @@ func ResolveIntent( // If requested, replace point tombstones with range tombstones. if ok && cArgs.EvalCtx.EvalKnobs().UseRangeTombstonesForPointDeletes { if err := storage.ReplacePointTombstonesWithRangeTombstones(ctx, - spanset.DisableReadWriterAssertions(readWriter), ms, update.Key, update.EndKey); err != nil { + spanset.DisableUndeclaredSpanAssertions(readWriter), ms, update.Key, update.EndKey); err != nil { return result.Result{}, err } } diff --git a/pkg/kv/kvserver/batcheval/cmd_resolve_intent_range.go b/pkg/kv/kvserver/batcheval/cmd_resolve_intent_range.go index ab89b4c4ba71..4664a6baabb0 100644 --- a/pkg/kv/kvserver/batcheval/cmd_resolve_intent_range.go +++ b/pkg/kv/kvserver/batcheval/cmd_resolve_intent_range.go @@ -99,7 +99,7 @@ func ResolveIntentRange( // If requested, replace point tombstones with range tombstones. if cArgs.EvalCtx.EvalKnobs().UseRangeTombstonesForPointDeletes { if err := storage.ReplacePointTombstonesWithRangeTombstones(ctx, - spanset.DisableReadWriterAssertions(readWriter), ms, args.Key, args.EndKey); err != nil { + spanset.DisableUndeclaredSpanAssertions(readWriter), ms, args.Key, args.EndKey); err != nil { return result.Result{}, err } } diff --git a/pkg/kv/kvserver/spanset/batch.go b/pkg/kv/kvserver/spanset/batch.go index 10a9f294e814..6ef62e1493b9 100644 --- a/pkg/kv/kvserver/spanset/batch.go +++ b/pkg/kv/kvserver/spanset/batch.go @@ -766,7 +766,9 @@ func NewReadWriterAt(rw storage.ReadWriter, spans *SpanSet, ts hlc.Timestamp) st type spanSetBatch struct { ReadWriter - b storage.Batch + b storage.Batch + // TODO(ibrahim): The fields spans, spansOnly, and ts don't seem to be used. + // Consider removing them and performing the necessary clean ups. spans *SpanSet spansOnly bool @@ -838,6 +840,22 @@ func (s spanSetBatch) ClearRawEncodedRange(start, end []byte) error { return s.b.ClearRawEncodedRange(start, end) } +// clone returns a shallow copy of the spanSetBatch. The returned batch shares +// the same underlying storage.Batch but has its own spanSetBatch wrapper with +// a new copy of the SpanSet that uses a shallow copy of the underlying spans. +func (s spanSetBatch) clone() *spanSetBatch { + return &spanSetBatch{ + ReadWriter: ReadWriter{ + spanSetReader: spanSetReader{r: s.b, spans: s.spanSetReader.spans.ShallowCopy(), spansOnly: s.spansOnly, ts: s.ts}, + spanSetWriter: spanSetWriter{w: s.b, spans: s.spanSetWriter.spans.ShallowCopy(), spansOnly: s.spansOnly, ts: s.ts}, + }, + b: s.b, + spans: s.spans.ShallowCopy(), + spansOnly: s.spansOnly, + ts: s.ts, + } +} + // NewBatch returns a storage.Batch that asserts access of the underlying // Batch against the given SpanSet. We only consider span boundaries, associated // timestamps are not considered. @@ -888,6 +906,40 @@ func DisableReadWriterAssertions(rw storage.ReadWriter) storage.ReadWriter { } } +// DisableUndeclaredSpanAssertions returns a new batch wrapper with latch +// assertions disabled. It does not modify the original batch. The returned +// batch shares the same underlying storage.Batch but has its own SpanSet +// wrapper with the latch assertion disabled. +func DisableUndeclaredSpanAssertions(rw storage.ReadWriter) storage.ReadWriter { + switch v := rw.(type) { + case *spanSetBatch: + newSnapSetBatch := v.clone() + newSnapSetBatch.spanSetReader.spans.DisableUndeclaredAccessAssertions() + newSnapSetBatch.spanSetWriter.spans.DisableUndeclaredAccessAssertions() + return newSnapSetBatch + + default: + return rw + } +} + +// DisableForbiddenSpanAssertions returns a new batch wrapper with +// forbidden span assertions disabled. It does not modify the original batch. +// The returned batch shares the same underlying storage.Batch but has its own +// SpanSet wrapper with the forbidden span assertion disabled. +func DisableForbiddenSpanAssertions(rw storage.ReadWriter) storage.ReadWriter { + switch v := rw.(type) { + case *spanSetBatch: + newSnapSetBatch := v.clone() + newSnapSetBatch.spanSetReader.spans.DisableForbiddenSpansAssertions() + newSnapSetBatch.spanSetWriter.spans.DisableForbiddenSpansAssertions() + return newSnapSetBatch + + default: + return rw + } +} + // addLockTableSpans adds corresponding lock table spans for the declared // spans. This is to implicitly allow raw access to separated intents in the // lock table for any declared keys. Explicitly declaring lock table spans is diff --git a/pkg/kv/kvserver/spanset/spanset.go b/pkg/kv/kvserver/spanset/spanset.go index f9df171dd2ab..0a173961edfc 100644 --- a/pkg/kv/kvserver/spanset/spanset.go +++ b/pkg/kv/kvserver/spanset/spanset.go @@ -83,8 +83,14 @@ type Span struct { // The Span slice for a particular access and scope contains non-overlapping // spans in increasing key order after calls to SortAndDedup. type SpanSet struct { - spans [NumSpanAccess][NumSpanScope][]Span - allowUndeclared bool + spans [NumSpanAccess][NumSpanScope][]Span + // forbiddenSpansMatchers are functions that return an error if the given span + // shouldn't be accessed (forbidden). This allows for complex pattern matching + // like forbidding specific keys across all range IDs without enumerating them + // explicitly. + forbiddenSpansMatchers []func(roachpb.Span) error + allowUndeclared bool + allowForbidden bool } var spanSetPool = sync.Pool{ @@ -113,6 +119,8 @@ func (s *SpanSet) Release() { s.spans[sa][ss] = recycle } } + s.forbiddenSpansMatchers = nil + s.allowForbidden = false s.allowUndeclared = false spanSetPool.Put(s) } @@ -155,7 +163,19 @@ func (s *SpanSet) Copy() *SpanSet { n.spans[sa][ss] = append(n.spans[sa][ss], s.spans[sa][ss]...) } } + n.forbiddenSpansMatchers = append(n.forbiddenSpansMatchers, s.forbiddenSpansMatchers...) n.allowUndeclared = s.allowUndeclared + n.allowForbidden = s.allowForbidden + return n +} + +// ShallowCopy performs a shallow SpanSet copy. +func (s *SpanSet) ShallowCopy() *SpanSet { + n := New() + n.spans = s.spans + n.forbiddenSpansMatchers = s.forbiddenSpansMatchers + n.allowUndeclared = s.allowUndeclared + n.allowForbidden = s.allowForbidden return n } @@ -201,6 +221,12 @@ func (s *SpanSet) AddMVCC(access SpanAccess, span roachpb.Span, timestamp hlc.Ti s.spans[access][scope] = append(s.spans[access][scope], Span{Span: span, Timestamp: timestamp}) } +// AddForbiddenMatcher adds a forbidden span matcher. The matcher is a function +// that is called for each span access to check if it should be forbidden. +func (s *SpanSet) AddForbiddenMatcher(matcher func(roachpb.Span) error) { + s.forbiddenSpansMatchers = append(s.forbiddenSpansMatchers, matcher) +} + // Merge merges all spans in s2 into s. s2 is not modified. func (s *SpanSet) Merge(s2 *SpanSet) { for sa := SpanAccess(0); sa < NumSpanAccess; sa++ { @@ -208,7 +234,9 @@ func (s *SpanSet) Merge(s2 *SpanSet) { s.spans[sa][ss] = append(s.spans[sa][ss], s2.spans[sa][ss]...) } } + s.forbiddenSpansMatchers = append(s.forbiddenSpansMatchers, s2.forbiddenSpansMatchers...) s.allowUndeclared = s2.allowUndeclared + s.allowForbidden = s2.allowForbidden s.SortAndDedup() } @@ -331,9 +359,19 @@ func (s *SpanSet) CheckAllowedAt( func (s *SpanSet) checkAllowed( access SpanAccess, span roachpb.Span, check func(SpanAccess, Span) bool, ) error { + // Unless explicitly disabled, check if we access any forbidden spans. + if !s.allowForbidden { + // Check if the span is forbidden. + for _, matcher := range s.forbiddenSpansMatchers { + if err := matcher(span); err != nil { + return errors.Errorf("cannot %s span %s: matches forbidden pattern", + access, span) + } + } + } + + // Unless explicitly disabled, check if we access any undeclared spans. if s.allowUndeclared { - // If the request has specified that undeclared spans are allowed, do - // nothing. return nil } @@ -396,3 +434,8 @@ func (s *SpanSet) Validate() error { func (s *SpanSet) DisableUndeclaredAccessAssertions() { s.allowUndeclared = true } + +// DisableForbiddenSpansAssertions disables forbidden spans assertions. +func (s *SpanSet) DisableForbiddenSpansAssertions() { + s.allowForbidden = true +} From bb443121ff7c262e10f0c60b3360b24387655ff2 Mon Sep 17 00:00:00 2001 From: iskettaneh <173953022+iskettaneh@users.noreply.github.com> Date: Tue, 11 Nov 2025 15:46:25 -0500 Subject: [PATCH 2/3] spanset: Add Overlaps() helper function Analogous to how we have a spanset helper contains() that understands the special span representation: [x-eps,x). This commit adds Overlaps that expects the same span representation. Moreover, this commit makes this special span representation explicit by introducing a new type called `TrickySpan`. --- pkg/kv/kvserver/replica_eval_context_span.go | 14 +- pkg/kv/kvserver/replica_test.go | 2 +- pkg/kv/kvserver/spanset/batch.go | 80 ++++---- pkg/kv/kvserver/spanset/spanset.go | 65 ++++--- pkg/kv/kvserver/spanset/spanset_test.go | 185 ++++++++++++++----- 5 files changed, 231 insertions(+), 115 deletions(-) diff --git a/pkg/kv/kvserver/replica_eval_context_span.go b/pkg/kv/kvserver/replica_eval_context_span.go index e7d901448c0f..0a34b390df96 100644 --- a/pkg/kv/kvserver/replica_eval_context_span.go +++ b/pkg/kv/kvserver/replica_eval_context_span.go @@ -110,7 +110,7 @@ func (rec *SpanSetReplicaEvalContext) IsFirstRange() bool { func (rec SpanSetReplicaEvalContext) Desc() *roachpb.RangeDescriptor { desc := rec.i.Desc() rec.ss.AssertAllowed(spanset.SpanReadOnly, - roachpb.Span{Key: keys.RangeDescriptorKey(desc.StartKey)}, + spanset.TrickySpan{Key: keys.RangeDescriptorKey(desc.StartKey)}, ) return desc } @@ -151,7 +151,7 @@ func (rec SpanSetReplicaEvalContext) CanCreateTxnRecord( ctx context.Context, txnID uuid.UUID, txnKey []byte, txnMinTS hlc.Timestamp, ) (bool, kvpb.TransactionAbortedReason) { rec.ss.AssertAllowed(spanset.SpanReadOnly, - roachpb.Span{Key: keys.TransactionKey(txnKey, txnID)}, + spanset.TrickySpan{Key: keys.TransactionKey(txnKey, txnID)}, ) return rec.i.CanCreateTxnRecord(ctx, txnID, txnKey, txnMinTS) } @@ -163,7 +163,7 @@ func (rec SpanSetReplicaEvalContext) MinTxnCommitTS( ctx context.Context, txnID uuid.UUID, txnKey []byte, ) hlc.Timestamp { rec.ss.AssertAllowed(spanset.SpanReadOnly, - roachpb.Span{Key: keys.TransactionKey(txnKey, txnID)}, + spanset.TrickySpan{Key: keys.TransactionKey(txnKey, txnID)}, ) return rec.i.MinTxnCommitTS(ctx, txnID, txnKey) } @@ -173,7 +173,7 @@ func (rec SpanSetReplicaEvalContext) MinTxnCommitTS( // not be served. func (rec SpanSetReplicaEvalContext) GetGCThreshold() hlc.Timestamp { rec.ss.AssertAllowed(spanset.SpanReadOnly, - roachpb.Span{Key: keys.RangeGCThresholdKey(rec.GetRangeID())}, + spanset.TrickySpan{Key: keys.RangeGCThresholdKey(rec.GetRangeID())}, ) return rec.i.GetGCThreshold() } @@ -197,7 +197,7 @@ func (rec SpanSetReplicaEvalContext) GetLastReplicaGCTimestamp( ctx context.Context, ) (hlc.Timestamp, error) { if err := rec.ss.CheckAllowed(spanset.SpanReadOnly, - roachpb.Span{Key: keys.RangeLastReplicaGCTimestampKey(rec.GetRangeID())}, + spanset.TrickySpan{Key: keys.RangeLastReplicaGCTimestampKey(rec.GetRangeID())}, ); err != nil { return hlc.Timestamp{}, err } @@ -228,11 +228,11 @@ func (rec *SpanSetReplicaEvalContext) GetCurrentReadSummary(ctx context.Context) // To capture a read summary over the range, all keys must be latched for // writing to prevent any concurrent reads or writes. desc := rec.i.Desc() - rec.ss.AssertAllowed(spanset.SpanReadWrite, roachpb.Span{ + rec.ss.AssertAllowed(spanset.SpanReadWrite, spanset.TrickySpan{ Key: keys.MakeRangeKeyPrefix(desc.StartKey), EndKey: keys.MakeRangeKeyPrefix(desc.EndKey), }) - rec.ss.AssertAllowed(spanset.SpanReadWrite, roachpb.Span{ + rec.ss.AssertAllowed(spanset.SpanReadWrite, spanset.TrickySpan{ Key: desc.StartKey.AsRawKey(), EndKey: desc.EndKey.AsRawKey(), }) diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index f7f18e423720..cb2daf176b38 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -2732,7 +2732,7 @@ func TestReplicaLatchingSplitDeclaresWrites(t *testing.T) { {spanset.SpanReadWrite, roachpb.Key("b"), false}, {spanset.SpanReadWrite, roachpb.Key("d"), true}, } { - err := spans.CheckAllowed(tc.access, roachpb.Span{Key: tc.key}) + err := spans.CheckAllowed(tc.access, spanset.TrickySpan{Key: tc.key}) if tc.expectAccess { require.NoError(t, err) } else { diff --git a/pkg/kv/kvserver/spanset/batch.go b/pkg/kv/kvserver/spanset/batch.go index 6ef62e1493b9..540516f283e3 100644 --- a/pkg/kv/kvserver/spanset/batch.go +++ b/pkg/kv/kvserver/spanset/batch.go @@ -77,7 +77,7 @@ func (i *MVCCIterator) Valid() (bool, error) { // SeekGE is part of the storage.MVCCIterator interface. func (i *MVCCIterator) SeekGE(key storage.MVCCKey) { i.i.SeekGE(key) - i.checkAllowed(roachpb.Span{Key: key.Key}, true) + i.checkAllowed(TrickySpan{Key: key.Key}, true) } // SeekLT is part of the storage.MVCCIterator interface. @@ -85,7 +85,7 @@ func (i *MVCCIterator) SeekLT(key storage.MVCCKey) { i.i.SeekLT(key) // CheckAllowed{At} supports the span representation of [,key), which // corresponds to the span [key.Prev(),). - i.checkAllowed(roachpb.Span{EndKey: key.Key}, true) + i.checkAllowed(TrickySpan{EndKey: key.Key}, true) } // Next is part of the storage.MVCCIterator interface. @@ -117,12 +117,12 @@ func (i *MVCCIterator) checkAllowedCurrPosForward(errIfDisallowed bool) { // as long as the iterator itself is configured with proper boundaries. return } - i.checkAllowedValidPos(roachpb.Span{Key: i.UnsafeKey().Key}, errIfDisallowed) + i.checkAllowedValidPos(TrickySpan{Key: i.UnsafeKey().Key}, errIfDisallowed) } // checkAllowed checks the provided span if the current iterator position is // valid. -func (i *MVCCIterator) checkAllowed(span roachpb.Span, errIfDisallowed bool) { +func (i *MVCCIterator) checkAllowed(span TrickySpan, errIfDisallowed bool) { i.invalid = false i.err = nil if ok, _ := i.i.Valid(); !ok { @@ -134,7 +134,7 @@ func (i *MVCCIterator) checkAllowed(span roachpb.Span, errIfDisallowed bool) { i.checkAllowedValidPos(span, errIfDisallowed) } -func (i *MVCCIterator) checkAllowedValidPos(span roachpb.Span, errIfDisallowed bool) { +func (i *MVCCIterator) checkAllowedValidPos(span TrickySpan, errIfDisallowed bool) { var err error if i.spansOnly { err = i.spans.CheckAllowed(SpanReadOnly, span) @@ -213,11 +213,11 @@ func (i *MVCCIterator) FindSplitKey( start, end, minSplitKey roachpb.Key, targetSize int64, ) (storage.MVCCKey, error) { if i.spansOnly { - if err := i.spans.CheckAllowed(SpanReadOnly, roachpb.Span{Key: start, EndKey: end}); err != nil { + if err := i.spans.CheckAllowed(SpanReadOnly, TrickySpan{Key: start, EndKey: end}); err != nil { return storage.MVCCKey{}, err } } else { - if err := i.spans.CheckAllowedAt(SpanReadOnly, roachpb.Span{Key: start, EndKey: end}, i.ts); err != nil { + if err := i.spans.CheckAllowedAt(SpanReadOnly, TrickySpan{Key: start, EndKey: end}, i.ts); err != nil { return storage.MVCCKey{}, err } } @@ -261,10 +261,10 @@ func (i *EngineIterator) SeekEngineKeyGE(key storage.EngineKey) (valid bool, err } if key.IsMVCCKey() && !i.spansOnly { mvccKey, _ := key.ToMVCCKey() - if err := i.spans.CheckAllowedAt(SpanReadOnly, roachpb.Span{Key: mvccKey.Key}, i.ts); err != nil { + if err := i.spans.CheckAllowedAt(SpanReadOnly, TrickySpan{Key: mvccKey.Key}, i.ts); err != nil { return false, err } - } else if err = i.spans.CheckAllowed(SpanReadOnly, roachpb.Span{Key: key.Key}); err != nil { + } else if err = i.spans.CheckAllowed(SpanReadOnly, TrickySpan{Key: key.Key}); err != nil { return false, err } return valid, err @@ -278,10 +278,10 @@ func (i *EngineIterator) SeekEngineKeyLT(key storage.EngineKey) (valid bool, err } if key.IsMVCCKey() && !i.spansOnly { mvccKey, _ := key.ToMVCCKey() - if err := i.spans.CheckAllowedAt(SpanReadOnly, roachpb.Span{Key: mvccKey.Key}, i.ts); err != nil { + if err := i.spans.CheckAllowedAt(SpanReadOnly, TrickySpan{Key: mvccKey.Key}, i.ts); err != nil { return false, err } - } else if err = i.spans.CheckAllowed(SpanReadOnly, roachpb.Span{EndKey: key.Key}); err != nil { + } else if err = i.spans.CheckAllowed(SpanReadOnly, TrickySpan{EndKey: key.Key}); err != nil { return false, err } return valid, err @@ -313,7 +313,7 @@ func (i *EngineIterator) SeekEngineKeyGEWithLimit( if state != pebble.IterValid { return state, err } - if err = i.spans.CheckAllowed(SpanReadOnly, roachpb.Span{Key: key.Key}); err != nil { + if err = i.spans.CheckAllowed(SpanReadOnly, TrickySpan{Key: key.Key}); err != nil { return pebble.IterExhausted, err } return state, err @@ -327,7 +327,7 @@ func (i *EngineIterator) SeekEngineKeyLTWithLimit( if state != pebble.IterValid { return state, err } - if err = i.spans.CheckAllowed(SpanReadOnly, roachpb.Span{EndKey: key.Key}); err != nil { + if err = i.spans.CheckAllowed(SpanReadOnly, TrickySpan{EndKey: key.Key}); err != nil { return pebble.IterExhausted, err } return state, err @@ -354,11 +354,11 @@ func (i *EngineIterator) checkKeyAllowed() (valid bool, err error) { } if key.IsMVCCKey() && !i.spansOnly { mvccKey, _ := key.ToMVCCKey() - if err := i.spans.CheckAllowedAt(SpanReadOnly, roachpb.Span{Key: mvccKey.Key}, i.ts); err != nil { + if err := i.spans.CheckAllowedAt(SpanReadOnly, TrickySpan{Key: mvccKey.Key}, i.ts); err != nil { // Invalid, but no error. return false, nil // nolint:returnerrcheck } - } else if err = i.spans.CheckAllowed(SpanReadOnly, roachpb.Span{Key: key.Key}); err != nil { + } else if err = i.spans.CheckAllowed(SpanReadOnly, TrickySpan{Key: key.Key}); err != nil { // Invalid, but no error. return false, nil // nolint:returnerrcheck } @@ -469,11 +469,11 @@ func (s spanSetReader) MVCCIterate( f func(storage.MVCCKeyValue, storage.MVCCRangeKeyStack) error, ) error { if s.spansOnly { - if err := s.spans.CheckAllowed(SpanReadOnly, roachpb.Span{Key: start, EndKey: end}); err != nil { + if err := s.spans.CheckAllowed(SpanReadOnly, TrickySpan{Key: start, EndKey: end}); err != nil { return err } } else { - if err := s.spans.CheckAllowedAt(SpanReadOnly, roachpb.Span{Key: start, EndKey: end}, s.ts); err != nil { + if err := s.spans.CheckAllowedAt(SpanReadOnly, TrickySpan{Key: start, EndKey: end}, s.ts); err != nil { return err } } @@ -535,11 +535,11 @@ func (s spanSetWriter) ApplyBatchRepr(repr []byte, sync bool) error { func (s spanSetWriter) checkAllowed(key roachpb.Key) error { if s.spansOnly { - if err := s.spans.CheckAllowed(SpanReadWrite, roachpb.Span{Key: key}); err != nil { + if err := s.spans.CheckAllowed(SpanReadWrite, TrickySpan{Key: key}); err != nil { return err } } else { - if err := s.spans.CheckAllowedAt(SpanReadWrite, roachpb.Span{Key: key}, s.ts); err != nil { + if err := s.spans.CheckAllowedAt(SpanReadWrite, TrickySpan{Key: key}, s.ts); err != nil { return err } } @@ -561,7 +561,7 @@ func (s spanSetWriter) ClearUnversioned(key roachpb.Key, opts storage.ClearOptio } func (s spanSetWriter) ClearEngineKey(key storage.EngineKey, opts storage.ClearOptions) error { - if err := s.spans.CheckAllowed(SpanReadWrite, roachpb.Span{Key: key.Key}); err != nil { + if err := s.spans.CheckAllowed(SpanReadWrite, TrickySpan{Key: key.Key}); err != nil { return err } return s.w.ClearEngineKey(key, opts) @@ -575,11 +575,11 @@ func (s spanSetWriter) SingleClearEngineKey(key storage.EngineKey) error { func (s spanSetWriter) checkAllowedRange(start, end roachpb.Key) error { if s.spansOnly { - if err := s.spans.CheckAllowed(SpanReadWrite, roachpb.Span{Key: start, EndKey: end}); err != nil { + if err := s.spans.CheckAllowed(SpanReadWrite, TrickySpan{Key: start, EndKey: end}); err != nil { return err } } else { - if err := s.spans.CheckAllowedAt(SpanReadWrite, roachpb.Span{Key: start, EndKey: end}, s.ts); err != nil { + if err := s.spans.CheckAllowedAt(SpanReadWrite, TrickySpan{Key: start, EndKey: end}, s.ts); err != nil { return err } } @@ -661,11 +661,11 @@ func (s spanSetWriter) ClearMVCCRangeKey(rangeKey storage.MVCCRangeKey) error { func (s spanSetWriter) Merge(key storage.MVCCKey, value []byte) error { if s.spansOnly { - if err := s.spans.CheckAllowed(SpanReadWrite, roachpb.Span{Key: key.Key}); err != nil { + if err := s.spans.CheckAllowed(SpanReadWrite, TrickySpan{Key: key.Key}); err != nil { return err } } else { - if err := s.spans.CheckAllowedAt(SpanReadWrite, roachpb.Span{Key: key.Key}, s.ts); err != nil { + if err := s.spans.CheckAllowedAt(SpanReadWrite, TrickySpan{Key: key.Key}, s.ts); err != nil { return err } } @@ -697,7 +697,7 @@ func (s spanSetWriter) PutEngineKey(key storage.EngineKey, value []byte) error { if !s.spansOnly { panic("cannot do timestamp checking for putting EngineKey") } - if err := s.spans.CheckAllowed(SpanReadWrite, roachpb.Span{Key: key.Key}); err != nil { + if err := s.spans.CheckAllowed(SpanReadWrite, TrickySpan{Key: key.Key}); err != nil { return err } return s.w.PutEngineKey(key, value) @@ -768,7 +768,7 @@ type spanSetBatch struct { ReadWriter b storage.Batch // TODO(ibrahim): The fields spans, spansOnly, and ts don't seem to be used. - // Consider removing them and performing the necessary clean ups. + // Consider removing or marking them as intended. spans *SpanSet spansOnly bool @@ -840,20 +840,16 @@ func (s spanSetBatch) ClearRawEncodedRange(start, end []byte) error { return s.b.ClearRawEncodedRange(start, end) } -// clone returns a shallow copy of the spanSetBatch. The returned batch shares -// the same underlying storage.Batch but has its own spanSetBatch wrapper with -// a new copy of the SpanSet that uses a shallow copy of the underlying spans. -func (s spanSetBatch) clone() *spanSetBatch { - return &spanSetBatch{ - ReadWriter: ReadWriter{ - spanSetReader: spanSetReader{r: s.b, spans: s.spanSetReader.spans.ShallowCopy(), spansOnly: s.spansOnly, ts: s.ts}, - spanSetWriter: spanSetWriter{w: s.b, spans: s.spanSetWriter.spans.ShallowCopy(), spansOnly: s.spansOnly, ts: s.ts}, - }, - b: s.b, - spans: s.spans.ShallowCopy(), - spansOnly: s.spansOnly, - ts: s.ts, - } +// shallowCopy returns a shallow copy of the spanSetBatch. The returned batch +// shares the same underlying storage.Batch but has its own spanSetBatch wrapper +// with a new copy of the SpanSet that uses a shallow copy of the underlying +// spans. +func (s spanSetBatch) shallowCopy() *spanSetBatch { + b := s + b.spanSetReader.spans = b.spanSetReader.spans.ShallowCopy() + b.spanSetWriter.spans = b.spanSetWriter.spans.ShallowCopy() + b.spans = b.spans.ShallowCopy() + return &b } // NewBatch returns a storage.Batch that asserts access of the underlying @@ -913,7 +909,7 @@ func DisableReadWriterAssertions(rw storage.ReadWriter) storage.ReadWriter { func DisableUndeclaredSpanAssertions(rw storage.ReadWriter) storage.ReadWriter { switch v := rw.(type) { case *spanSetBatch: - newSnapSetBatch := v.clone() + newSnapSetBatch := v.shallowCopy() newSnapSetBatch.spanSetReader.spans.DisableUndeclaredAccessAssertions() newSnapSetBatch.spanSetWriter.spans.DisableUndeclaredAccessAssertions() return newSnapSetBatch @@ -930,7 +926,7 @@ func DisableUndeclaredSpanAssertions(rw storage.ReadWriter) storage.ReadWriter { func DisableForbiddenSpanAssertions(rw storage.ReadWriter) storage.ReadWriter { switch v := rw.(type) { case *spanSetBatch: - newSnapSetBatch := v.clone() + newSnapSetBatch := v.shallowCopy() newSnapSetBatch.spanSetReader.spans.DisableForbiddenSpansAssertions() newSnapSetBatch.spanSetWriter.spans.DisableForbiddenSpansAssertions() return newSnapSetBatch diff --git a/pkg/kv/kvserver/spanset/spanset.go b/pkg/kv/kvserver/spanset/spanset.go index 0a173961edfc..fc10ea864336 100644 --- a/pkg/kv/kvserver/spanset/spanset.go +++ b/pkg/kv/kvserver/spanset/spanset.go @@ -76,6 +76,11 @@ type Span struct { Timestamp hlc.Timestamp } +// TrickySpan represents a span that supports a special encoding where a nil +// start key with a non-nil end key represents the point span: +// [EndKey.Prev(), EndKey). +type TrickySpan roachpb.Span + // SpanSet tracks the set of key spans touched by a command, broken into MVCC // and non-MVCC accesses. The set is divided into subsets for access type // (read-only or read/write) and key scope (local or global; used to facilitate @@ -88,7 +93,7 @@ type SpanSet struct { // shouldn't be accessed (forbidden). This allows for complex pattern matching // like forbidding specific keys across all range IDs without enumerating them // explicitly. - forbiddenSpansMatchers []func(roachpb.Span) error + forbiddenSpansMatchers []func(TrickySpan) error allowUndeclared bool allowForbidden bool } @@ -172,10 +177,7 @@ func (s *SpanSet) Copy() *SpanSet { // ShallowCopy performs a shallow SpanSet copy. func (s *SpanSet) ShallowCopy() *SpanSet { n := New() - n.spans = s.spans - n.forbiddenSpansMatchers = s.forbiddenSpansMatchers - n.allowUndeclared = s.allowUndeclared - n.allowForbidden = s.allowForbidden + *n = *s return n } @@ -223,7 +225,7 @@ func (s *SpanSet) AddMVCC(access SpanAccess, span roachpb.Span, timestamp hlc.Ti // AddForbiddenMatcher adds a forbidden span matcher. The matcher is a function // that is called for each span access to check if it should be forbidden. -func (s *SpanSet) AddForbiddenMatcher(matcher func(roachpb.Span) error) { +func (s *SpanSet) AddForbiddenMatcher(matcher func(TrickySpan) error) { s.forbiddenSpansMatchers = append(s.forbiddenSpansMatchers, matcher) } @@ -235,6 +237,8 @@ func (s *SpanSet) Merge(s2 *SpanSet) { } } s.forbiddenSpansMatchers = append(s.forbiddenSpansMatchers, s2.forbiddenSpansMatchers...) + // TODO(ibrahim): Figure out if the merged `allowUndeclared` and + // `allowForbidden` should be true if it was true in either `s` or `s2`. s.allowUndeclared = s2.allowUndeclared s.allowForbidden = s2.allowForbidden s.SortAndDedup() @@ -270,7 +274,7 @@ func (s *SpanSet) Intersects(other *SpanSet) bool { otherSpans := other.GetSpans(sa, ss) for _, span := range otherSpans { // If access is allowed, we must have an overlap. - if err := s.CheckAllowed(sa, span.Span); err == nil { + if err := s.CheckAllowed(sa, TrickySpan(span.Span)); err == nil { return true } } @@ -282,7 +286,7 @@ func (s *SpanSet) Intersects(other *SpanSet) bool { // AssertAllowed calls CheckAllowed and fatals if the access is not allowed. // Timestamps associated with the spans in the spanset are not considered, // only the span boundaries are checked. -func (s *SpanSet) AssertAllowed(access SpanAccess, span roachpb.Span) { +func (s *SpanSet) AssertAllowed(access SpanAccess, span TrickySpan) { if err := s.CheckAllowed(access, span); err != nil { log.KvExec.Fatalf(context.TODO(), "%v", err) } @@ -303,7 +307,7 @@ func (s *SpanSet) AssertAllowed(access SpanAccess, span roachpb.Span) { // fail at checking if read only access over the span [a-d) was requested. This // is also a problem if the added spans were read only and the spanset wasn't // already SortAndDedup-ed. -func (s *SpanSet) CheckAllowed(access SpanAccess, span roachpb.Span) error { +func (s *SpanSet) CheckAllowed(access SpanAccess, span TrickySpan) error { return s.checkAllowed(access, span, func(_ SpanAccess, _ Span) bool { return true }) @@ -312,7 +316,7 @@ func (s *SpanSet) CheckAllowed(access SpanAccess, span roachpb.Span) error { // CheckAllowedAt is like CheckAllowed, except it returns an error if the access // is not allowed over the given keyspan at the given timestamp. func (s *SpanSet) CheckAllowedAt( - access SpanAccess, span roachpb.Span, timestamp hlc.Timestamp, + access SpanAccess, span TrickySpan, timestamp hlc.Timestamp, ) error { mvcc := !timestamp.IsEmpty() return s.checkAllowed(access, span, func(declAccess SpanAccess, declSpan Span) bool { @@ -357,7 +361,7 @@ func (s *SpanSet) CheckAllowedAt( } func (s *SpanSet) checkAllowed( - access SpanAccess, span roachpb.Span, check func(SpanAccess, Span) bool, + access SpanAccess, span TrickySpan, check func(SpanAccess, Span) bool, ) error { // Unless explicitly disabled, check if we access any forbidden spans. if !s.allowForbidden { @@ -383,7 +387,7 @@ func (s *SpanSet) checkAllowed( for ac := access; ac < NumSpanAccess; ac++ { for _, cur := range s.spans[ac][scope] { - if contains(cur.Span, span) && check(ac, cur) { + if Contains(cur.Span, span) && check(ac, cur) { return nil } } @@ -392,16 +396,10 @@ func (s *SpanSet) checkAllowed( return errors.Errorf("cannot %s undeclared span %s\ndeclared:\n%s\nstack:\n%s", access, span, s, debugutil.Stack()) } -// contains returns whether s1 contains s2. Unlike Span.Contains, this function -// supports spans with a nil start key and a non-nil end key (e.g. "[nil, c)"). -// In this form, s2.Key (inclusive) is considered to be the previous key to -// s2.EndKey (exclusive). -func contains(s1, s2 roachpb.Span) bool { - if s2.Key != nil { - // The common case. - return s1.Contains(s2) - } - +// doesNormalSpanContainPointTrickySpan takes a normal span (s1), and takes a +// tricky span s2, where the Key is nil, which represents: +// [EndKey.Prev(), EndKey), and returns whether s1 contains s2 or not. +func doesNormalSpanContainPointTrickySpan(s1 roachpb.Span, s2 TrickySpan) bool { // The following is equivalent to: // s1.Contains(roachpb.Span{Key: s2.EndKey.Prev()}) @@ -412,6 +410,29 @@ func contains(s1, s2 roachpb.Span) bool { return s1.Key.Compare(s2.EndKey) < 0 && s1.EndKey.Compare(s2.EndKey) >= 0 } +// Contains returns whether s1 contains s2, where s2 can be a TrickySpan. +func Contains(s1 roachpb.Span, s2 TrickySpan) bool { + if s2.Key != nil { + // The common case: s2 is a regular span with a non-nil start key. + return s1.Contains(roachpb.Span(s2)) + } + + // s2 is a TrickySpan with nil Key and non-nil EndKey. + return doesNormalSpanContainPointTrickySpan(s1, s2) +} + +// Overlaps returns whether s1 overlaps s2, where s2 can be a TrickySpan. +func Overlaps(s1 roachpb.Span, s2 TrickySpan) bool { + // The common case: both spans have non-nil start keys. + if s2.Key != nil { + return s1.Overlaps(roachpb.Span(s2)) + } + + // s2 is a TrickySpan with nil Key and non-nil EndKey. Since s2 is infinitely + // small, s1 and s2 overlap IFF s1 contains s2. + return doesNormalSpanContainPointTrickySpan(s1, s2) +} + // Validate returns an error if any spans that have been added to the set // are invalid. func (s *SpanSet) Validate() error { diff --git a/pkg/kv/kvserver/spanset/spanset_test.go b/pkg/kv/kvserver/spanset/spanset_test.go index f58c2d65b046..59a792e5396c 100644 --- a/pkg/kv/kvserver/spanset/spanset_test.go +++ b/pkg/kv/kvserver/spanset/spanset_test.go @@ -7,6 +7,7 @@ package spanset import ( "reflect" + "strings" "testing" "github.com/cockroachdb/cockroach/pkg/keys" @@ -140,7 +141,7 @@ func TestSpanSetCheckAllowedBoundaries(t *testing.T) { bdGkq.AddNonMVCC(SpanReadOnly, roachpb.Span{Key: roachpb.Key("g")}) bdGkq.AddNonMVCC(SpanReadOnly, roachpb.Span{Key: roachpb.Key("k"), EndKey: roachpb.Key("q")}) - allowed := []roachpb.Span{ + allowed := []TrickySpan{ // Exactly as declared. {Key: roachpb.Key("b"), EndKey: roachpb.Key("d")}, {Key: roachpb.Key("g")}, @@ -161,7 +162,7 @@ func TestSpanSetCheckAllowedBoundaries(t *testing.T) { } } - disallowed := []roachpb.Span{ + disallowed := []TrickySpan{ // Points outside the declared spans, and on the endpoints. {Key: roachpb.Key("a")}, {Key: roachpb.Key("d")}, @@ -203,25 +204,27 @@ func TestSpanSetCheckAllowedAtTimestamps(t *testing.T) { ss.AddNonMVCC(SpanReadWrite, roachpb.Span{Key: keys.RangeGCThresholdKey(1)}) var allowedRO = []struct { - span roachpb.Span + span TrickySpan ts hlc.Timestamp }{ // Read access allowed for a subspan or included point at a timestamp // equal to or below associated timestamp. - {roachpb.Span{Key: roachpb.Key("b"), EndKey: roachpb.Key("d")}, hlc.Timestamp{WallTime: 2}}, - {roachpb.Span{Key: roachpb.Key("b"), EndKey: roachpb.Key("d")}, hlc.Timestamp{WallTime: 1}}, - {roachpb.Span{Key: roachpb.Key("m"), EndKey: roachpb.Key("o")}, hlc.Timestamp{WallTime: 3}}, - {roachpb.Span{Key: roachpb.Key("m"), EndKey: roachpb.Key("o")}, hlc.Timestamp{WallTime: 2}}, - {roachpb.Span{Key: roachpb.Key("m"), EndKey: roachpb.Key("o")}, hlc.Timestamp{WallTime: 1}}, - {roachpb.Span{Key: roachpb.Key("g")}, hlc.Timestamp{WallTime: 2}}, - {roachpb.Span{Key: roachpb.Key("g")}, hlc.Timestamp{WallTime: 1}}, - {roachpb.Span{Key: roachpb.Key("s")}, hlc.Timestamp{WallTime: 3}}, - {roachpb.Span{Key: roachpb.Key("s")}, hlc.Timestamp{WallTime: 2}}, - {roachpb.Span{Key: roachpb.Key("s")}, hlc.Timestamp{WallTime: 1}}, + {TrickySpan{Key: roachpb.Key("b"), EndKey: roachpb.Key("d")}, hlc.Timestamp{WallTime: 2}}, + {TrickySpan{Key: roachpb.Key("b"), EndKey: roachpb.Key("d")}, hlc.Timestamp{WallTime: 1}}, + {TrickySpan{Key: roachpb.Key("m"), EndKey: roachpb.Key("o")}, hlc.Timestamp{WallTime: 3}}, + {TrickySpan{Key: roachpb.Key("m"), EndKey: roachpb.Key("o")}, hlc.Timestamp{WallTime: 2}}, + {TrickySpan{Key: roachpb.Key("m"), EndKey: roachpb.Key("o")}, hlc.Timestamp{WallTime: 1}}, + {TrickySpan{Key: roachpb.Key("g")}, hlc.Timestamp{WallTime: 2}}, + {TrickySpan{Key: roachpb.Key("g")}, hlc.Timestamp{WallTime: 1}}, + {TrickySpan{Key: roachpb.Key("s")}, hlc.Timestamp{WallTime: 3}}, + {TrickySpan{Key: roachpb.Key("s")}, hlc.Timestamp{WallTime: 2}}, + {TrickySpan{Key: roachpb.Key("s")}, hlc.Timestamp{WallTime: 1}}, // Local keys. - {roachpb.Span{Key: keys.RangeGCThresholdKey(1)}, hlc.Timestamp{}}, - {roachpb.Span{Key: keys.RangeGCThresholdKey(1)}, hlc.Timestamp{WallTime: 1}}, + {TrickySpan{Key: keys.RangeGCThresholdKey(1)}, hlc.Timestamp{}}, + {TrickySpan{Key: keys.RangeGCThresholdKey(1)}, hlc.Timestamp{WallTime: 1}}, + + // TODO(ibrahim): Add test cases with nil Key and not nil endKey. } for _, tc := range allowedRO { if err := ss.CheckAllowedAt(SpanReadOnly, tc.span, tc.ts); err != nil { @@ -230,29 +233,31 @@ func TestSpanSetCheckAllowedAtTimestamps(t *testing.T) { } var allowedRW = []struct { - span roachpb.Span + span TrickySpan ts hlc.Timestamp }{ // Write access allowed for a subspan or included point at exactly the // declared timestamp. - {roachpb.Span{Key: roachpb.Key("m"), EndKey: roachpb.Key("o")}, hlc.Timestamp{WallTime: 2}}, - {roachpb.Span{Key: roachpb.Key("m"), EndKey: roachpb.Key("o")}, hlc.Timestamp{WallTime: 3}}, - {roachpb.Span{Key: roachpb.Key("s")}, hlc.Timestamp{WallTime: 2}}, - {roachpb.Span{Key: roachpb.Key("s")}, hlc.Timestamp{WallTime: 3}}, + {TrickySpan{Key: roachpb.Key("m"), EndKey: roachpb.Key("o")}, hlc.Timestamp{WallTime: 2}}, + {TrickySpan{Key: roachpb.Key("m"), EndKey: roachpb.Key("o")}, hlc.Timestamp{WallTime: 3}}, + {TrickySpan{Key: roachpb.Key("s")}, hlc.Timestamp{WallTime: 2}}, + {TrickySpan{Key: roachpb.Key("s")}, hlc.Timestamp{WallTime: 3}}, // Points within the non-zero-length span. - {roachpb.Span{Key: roachpb.Key("n")}, hlc.Timestamp{WallTime: 2}}, + {TrickySpan{Key: roachpb.Key("n")}, hlc.Timestamp{WallTime: 2}}, // Points within the non-zero-length span at a timestamp higher than what's // declared. - {roachpb.Span{Key: roachpb.Key("n")}, hlc.Timestamp{WallTime: 3}}, + {TrickySpan{Key: roachpb.Key("n")}, hlc.Timestamp{WallTime: 3}}, // Sub span at and above the declared timestamp. - {roachpb.Span{Key: roachpb.Key("m"), EndKey: roachpb.Key("n")}, hlc.Timestamp{WallTime: 2}}, - {roachpb.Span{Key: roachpb.Key("m"), EndKey: roachpb.Key("n")}, hlc.Timestamp{WallTime: 3}}, + {TrickySpan{Key: roachpb.Key("m"), EndKey: roachpb.Key("n")}, hlc.Timestamp{WallTime: 2}}, + {TrickySpan{Key: roachpb.Key("m"), EndKey: roachpb.Key("n")}, hlc.Timestamp{WallTime: 3}}, // Local keys. - {roachpb.Span{Key: keys.RangeGCThresholdKey(1)}, hlc.Timestamp{}}, + {TrickySpan{Key: keys.RangeGCThresholdKey(1)}, hlc.Timestamp{}}, + + // TODO(ibrahim): Add test cases with nil Key and not nil endKey. } for _, tc := range allowedRW { if err := ss.CheckAllowedAt(SpanReadWrite, tc.span, tc.ts); err != nil { @@ -264,13 +269,13 @@ func TestSpanSetCheckAllowedAtTimestamps(t *testing.T) { writeErr := "cannot write undeclared span" var disallowedRO = []struct { - span roachpb.Span + span TrickySpan ts hlc.Timestamp }{ // Read access disallowed for subspan or included point at timestamp greater // than the associated timestamp. - {roachpb.Span{Key: roachpb.Key("b"), EndKey: roachpb.Key("d")}, hlc.Timestamp{WallTime: 3}}, - {roachpb.Span{Key: roachpb.Key("g")}, hlc.Timestamp{WallTime: 3}}, + {TrickySpan{Key: roachpb.Key("b"), EndKey: roachpb.Key("d")}, hlc.Timestamp{WallTime: 3}}, + {TrickySpan{Key: roachpb.Key("g")}, hlc.Timestamp{WallTime: 3}}, } for _, tc := range disallowedRO { if err := ss.CheckAllowedAt(SpanReadOnly, tc.span, tc.ts); !testutils.IsError(err, readErr) { @@ -279,24 +284,26 @@ func TestSpanSetCheckAllowedAtTimestamps(t *testing.T) { } var disallowedRW = []struct { - span roachpb.Span + span TrickySpan ts hlc.Timestamp }{ // Write access disallowed for subspan or included point at timestamp // less than the associated timestamp. - {roachpb.Span{Key: roachpb.Key("m"), EndKey: roachpb.Key("o")}, hlc.Timestamp{WallTime: 1}}, - {roachpb.Span{Key: roachpb.Key("s")}, hlc.Timestamp{WallTime: 1}}, + {TrickySpan{Key: roachpb.Key("m"), EndKey: roachpb.Key("o")}, hlc.Timestamp{WallTime: 1}}, + {TrickySpan{Key: roachpb.Key("s")}, hlc.Timestamp{WallTime: 1}}, // Read only spans. - {roachpb.Span{Key: roachpb.Key("b"), EndKey: roachpb.Key("d")}, hlc.Timestamp{WallTime: 2}}, - {roachpb.Span{Key: roachpb.Key("c")}, hlc.Timestamp{WallTime: 2}}, + {TrickySpan{Key: roachpb.Key("b"), EndKey: roachpb.Key("d")}, hlc.Timestamp{WallTime: 2}}, + {TrickySpan{Key: roachpb.Key("c")}, hlc.Timestamp{WallTime: 2}}, // Points within the non-zero-length span at a timestamp lower than what's // declared. - {roachpb.Span{Key: roachpb.Key("n")}, hlc.Timestamp{WallTime: 1}}, + {TrickySpan{Key: roachpb.Key("n")}, hlc.Timestamp{WallTime: 1}}, // Sub span below the declared timestamp. - {roachpb.Span{Key: roachpb.Key("m"), EndKey: roachpb.Key("n")}, hlc.Timestamp{WallTime: 1}}, + {TrickySpan{Key: roachpb.Key("m"), EndKey: roachpb.Key("n")}, hlc.Timestamp{WallTime: 1}}, + + // TODO(ibrahim): Add test cases with nil Key and not nil endKey. } for _, tc := range disallowedRW { if err := ss.CheckAllowedAt(SpanReadWrite, tc.span, tc.ts); !testutils.IsError(err, writeErr) { @@ -313,7 +320,7 @@ func TestSpanSetCheckAllowedReversed(t *testing.T) { bdGkq.AddNonMVCC(SpanReadOnly, roachpb.Span{Key: roachpb.Key("g")}) bdGkq.AddNonMVCC(SpanReadOnly, roachpb.Span{Key: roachpb.Key("k"), EndKey: roachpb.Key("q")}) - allowed := []roachpb.Span{ + allowed := []TrickySpan{ // Exactly as declared. {EndKey: roachpb.Key("d")}, {EndKey: roachpb.Key("q")}, @@ -324,7 +331,7 @@ func TestSpanSetCheckAllowedReversed(t *testing.T) { } } - disallowed := []roachpb.Span{ + disallowed := []TrickySpan{ // Points outside the declared spans, and on the endpoints. {EndKey: roachpb.Key("b")}, {EndKey: roachpb.Key("g")}, @@ -346,7 +353,7 @@ func TestSpanSetCheckAllowedAtReversed(t *testing.T) { bdGkq.AddMVCC(SpanReadOnly, roachpb.Span{Key: roachpb.Key("g")}, ts) bdGkq.AddMVCC(SpanReadOnly, roachpb.Span{Key: roachpb.Key("k"), EndKey: roachpb.Key("q")}, ts) - allowed := []roachpb.Span{ + allowed := []TrickySpan{ // Exactly as declared. {EndKey: roachpb.Key("d")}, {EndKey: roachpb.Key("q")}, @@ -357,7 +364,7 @@ func TestSpanSetCheckAllowedAtReversed(t *testing.T) { } } - disallowed := []roachpb.Span{ + disallowed := []TrickySpan{ // Points outside the declared spans, and on the endpoints. {EndKey: roachpb.Key("b")}, {EndKey: roachpb.Key("g")}, @@ -381,16 +388,108 @@ func TestSpanSetWriteImpliesRead(t *testing.T) { ss.AddNonMVCC(SpanReadOnly, roSpan) ss.AddNonMVCC(SpanReadWrite, rwSpan) - if err := ss.CheckAllowed(SpanReadOnly, roSpan); err != nil { + if err := ss.CheckAllowed(SpanReadOnly, TrickySpan(roSpan)); err != nil { t.Errorf("expected to be allowed to read roSpan, error: %+v", err) } - if err := ss.CheckAllowed(SpanReadWrite, roSpan); err == nil { + if err := ss.CheckAllowed(SpanReadWrite, TrickySpan(roSpan)); err == nil { t.Errorf("expected not to be allowed to write roSpan") } - if err := ss.CheckAllowed(SpanReadOnly, rwSpan); err != nil { + if err := ss.CheckAllowed(SpanReadOnly, TrickySpan(rwSpan)); err != nil { t.Errorf("expected to be allowed to read rwSpan, error: %+v", err) } - if err := ss.CheckAllowed(SpanReadWrite, rwSpan); err != nil { + if err := ss.CheckAllowed(SpanReadWrite, TrickySpan(rwSpan)); err != nil { t.Errorf("expected to be allowed to read rwSpan, error: %+v", err) } } + +// makeSpanHelper accepts strings like: "a-d", and returns a span with +// startKey = a, and endKey = d. It also accepts `X` which represents nil. For +// example, "X-d" returns a span with a nil startKey, and endKey = d. +func makeSpanHelper(t *testing.T, s string) roachpb.Span { + parts := strings.Split(s, "-") + require.Len(t, parts, 2) + + var start roachpb.Key + var end roachpb.Key + + if parts[0] != "X" { + start = roachpb.Key(parts[0]) + } + + if parts[1] != "X" { + end = roachpb.Key(parts[1]) + } + + return roachpb.Span{ + Key: start, + EndKey: end, + } +} + +// Test that Contains and Overlaps correctly determine whether one span +// contains, overlaps, or a combination of the two with another span. +func TestContainsAndOverlaps(t *testing.T) { + defer leaktest.AfterTest(t)() + + testCases := []struct { + s1 string + s2 string + contains bool + overlaps bool + }{ + // s1 is a full span, and s2 is a tricky span with nil StartKey. + {s1: "b-d", s2: "X-a", contains: false, overlaps: false}, + {s1: "b-d", s2: "X-b", contains: false, overlaps: false}, + {s1: "b-d", s2: "X-b\x00", contains: true, overlaps: true}, + {s1: "b-d", s2: "X-c", contains: true, overlaps: true}, + {s1: "b-d", s2: "X-d", contains: true, overlaps: true}, + {s1: "b-d", s2: "X-e", contains: false, overlaps: false}, + + // s1 is a full span, and s2 is a point span. + {s1: "b-d", s2: "a-X", contains: false, overlaps: false}, + {s1: "b-d", s2: "b-X", contains: true, overlaps: true}, + {s1: "b-d", s2: "c-X", contains: true, overlaps: true}, + {s1: "b-d", s2: "d-X", contains: false, overlaps: false}, + {s1: "b-d", s2: "e-X", contains: false, overlaps: false}, + + // s1 is a full span, and s2 is a full span. + {s1: "b-d", s2: "a-b", contains: false, overlaps: false}, + {s1: "b-d", s2: "a-b\x00", contains: false, overlaps: true}, + {s1: "b-d", s2: "a-c", contains: false, overlaps: true}, + {s1: "b-d", s2: "a-e", contains: false, overlaps: true}, + {s1: "b-d", s2: "b-d", contains: true, overlaps: true}, + {s1: "b-d", s2: "c-d", contains: true, overlaps: true}, + {s1: "b-d", s2: "c-e", contains: false, overlaps: true}, + {s1: "b-d", s2: "d-e", contains: false, overlaps: false}, + + // s1 is a point span, and s2 is a tricky span with nil StartKey. + {s1: "b-X", s2: "X-a", contains: false, overlaps: false}, + {s1: "b-X", s2: "X-b", contains: false, overlaps: false}, + {s1: "b-X", s2: "X-b\x00", contains: true, overlaps: true}, + {s1: "b-X", s2: "X-c", contains: false, overlaps: false}, + + // s1 is a point span, and s2 is a point span. + {s1: "b-X", s2: "a-X", contains: false, overlaps: false}, + {s1: "b-X", s2: "b-X", contains: true, overlaps: true}, + {s1: "b-X", s2: "c-X", contains: false, overlaps: false}, + + // s1 is a point span, and s2 is a full span. + {s1: "b-X", s2: "a-b", contains: false, overlaps: false}, + {s1: "b-X", s2: "a-c", contains: false, overlaps: true}, + {s1: "b-X", s2: "b-c", contains: false, overlaps: true}, + {s1: "b-X", s2: "c-d", contains: false, overlaps: false}, + } + + for _, tc := range testCases { + t.Run("", func(t *testing.T) { + span1 := makeSpanHelper(t, tc.s1) + span2 := makeSpanHelper(t, tc.s2) + // Sanity test assertion that if s1 contains s2, then it is expected to + // overlap with it. + require.True(t, !tc.contains || tc.overlaps) + + require.Equal(t, tc.contains, Contains(span1, TrickySpan(span2))) + require.Equal(t, tc.overlaps, Overlaps(span1, TrickySpan(span2))) + }) + } +} From 71bd03bc0263506bdaf76d7b42cb38421371d521 Mon Sep 17 00:00:00 2001 From: iskettaneh <173953022+iskettaneh@users.noreply.github.com> Date: Mon, 10 Nov 2025 19:59:34 -0500 Subject: [PATCH 3/3] spanset: assert that batches don't access store local and unreplicated RangeID local keys This commit 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 2 locations we know that we currently touch those keys. --- pkg/keys/constants.go | 1 + pkg/keys/keys.go | 11 ++ .../kvserver/batcheval/cmd_end_transaction.go | 7 +- pkg/kv/kvserver/replica_test.go | 125 ++++++++++++++++++ pkg/kv/kvserver/replica_write.go | 70 +++++++++- pkg/kv/kvserver/spanset/batch.go | 2 + 6 files changed, 213 insertions(+), 3 deletions(-) diff --git a/pkg/keys/constants.go b/pkg/keys/constants.go index 476c6443987c..2bc1d148adc3 100644 --- a/pkg/keys/constants.go +++ b/pkg/keys/constants.go @@ -163,6 +163,7 @@ var ( // // LocalStorePrefix is the prefix identifying per-store data. LocalStorePrefix = makeKey(LocalPrefix, roachpb.Key("s")) + LocalStoreMax = roachpb.Key(LocalStorePrefix).PrefixEnd() // localStoreClusterVersionSuffix stores the cluster-wide version // information for this store, updated any time the operator // updates the minimum cluster version. diff --git a/pkg/keys/keys.go b/pkg/keys/keys.go index 7a4249d490bc..89939f550072 100644 --- a/pkg/keys/keys.go +++ b/pkg/keys/keys.go @@ -308,6 +308,17 @@ func DecodeRangeIDKey( return roachpb.RangeID(rangeInt), infix, suffix, b, nil } +// DecodeRangeIDPrefix parses a local range ID prefix into range ID. +func DecodeRangeIDPrefix(key roachpb.Key) (roachpb.RangeID, error) { + if !bytes.HasPrefix(key, LocalRangeIDPrefix) { + return 0, errors.Errorf("key %s does not have %s prefix", key, LocalRangeIDPrefix) + } + // Cut the prefix, the Range ID, and the infix specifier. + b := key[len(LocalRangeIDPrefix):] + _, rangeInt, err := encoding.DecodeUvarintAscending(b) + return roachpb.RangeID(rangeInt), err +} + // AbortSpanKey returns a range-local key by Range ID for an // AbortSpan entry, with detail specified by encoding the // supplied transaction ID. diff --git a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go index a3667df1fffe..2bb5e311f6b7 100644 --- a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go +++ b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go @@ -1340,8 +1340,10 @@ func splitTriggerHelper( if err != nil { return enginepb.MVCCStats{}, result.Result{}, errors.Wrap(err, "unable to fetch last replica GC timestamp") } + if err := storage.MVCCPutProto( - ctx, batch, keys.RangeLastReplicaGCTimestampKey(split.RightDesc.RangeID), hlc.Timestamp{}, + ctx, spanset.DisableForbiddenSpanAssertions(batch), + keys.RangeLastReplicaGCTimestampKey(split.RightDesc.RangeID), hlc.Timestamp{}, &replicaGCTS, storage.MVCCWriteOptions{Category: fs.BatchEvalReadCategory}); err != nil { return enginepb.MVCCStats{}, result.Result{}, errors.Wrap(err, "unable to copy last replica GC timestamp") } @@ -1541,7 +1543,8 @@ func splitTriggerHelper( // as all replicas will be responsible for writing it locally before // applying the split. if !rec.ClusterSettings().Version.IsActive(ctx, clusterversion.V25_4_WriteInitialTruncStateBeforeSplitApplication) { - if err := kvstorage.WriteInitialTruncState(ctx, batch, split.RightDesc.RangeID); err != nil { + if err := kvstorage.WriteInitialTruncState(ctx, + spanset.DisableForbiddenSpanAssertions(batch), split.RightDesc.RangeID); err != nil { return enginepb.MVCCStats{}, result.Result{}, errors.Wrap(err, "unable to write initial Replica state") } } diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index cb2daf176b38..c0b334a93a43 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -15197,3 +15197,128 @@ func TestLeaderlessWatcherInit(t *testing.T) { t.Fatalf("expected LeaderlessWatcher channel to be closed") } } + +// TestOverlapsUnreplicatedRangeIDLocalKeys verifies that the function +// overlapsUnreplicatedRangeIDLocalKeys() successfully catches any overlap with +// unreplicated rangeID local keys. +func TestOverlapsUnreplicatedRangeIDLocalKeys(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + s := func(start, end roachpb.Key) roachpb.Span { + return roachpb.Span{Key: start, EndKey: end} + } + testCases := []struct { + span roachpb.Span + notOk bool + }{ + // Full spans not overlapping with unreplicated local RangeID spans. + {span: s(roachpb.KeyMin, keys.LocalRangeIDPrefix.AsRawKey())}, + {span: s(keys.RangeForceFlushKey(1), keys.RangeLeaseKey(1))}, + {span: s(keys.LocalRangeIDPrefix.AsRawKey().PrefixEnd(), roachpb.KeyMax)}, + + // Full spans overlapping with unreplicated local RangeID spans. + {span: s(roachpb.KeyMin, keys.MakeRangeIDUnreplicatedPrefix(1)), notOk: true}, // partial overlap + {span: s(roachpb.KeyMin, keys.RaftTruncatedStateKey(1)), notOk: true}, + {span: s(keys.LocalRangeIDPrefix.AsRawKey(), keys.LocalRangeIDPrefix.AsRawKey().PrefixEnd()), + notOk: true}, + {span: s(keys.RaftTruncatedStateKey(1), keys.RaftTruncatedStateKey(2)), notOk: true}, + {span: s(keys.MakeRangeIDUnreplicatedPrefix(1).PrefixEnd(), roachpb.KeyMax), notOk: true}, // partial overlap + {span: s(keys.MakeRangeIDUnreplicatedPrefix(1), + keys.MakeRangeIDUnreplicatedPrefix(1).PrefixEnd()), notOk: true}, + {span: s(keys.RaftTruncatedStateKey(1), roachpb.KeyMax), notOk: true}, + + // Point spans not overlapping with unreplicated local RangeID spans. + {span: s(roachpb.KeyMin, nil)}, + {span: s(keys.LocalRangeIDPrefix.AsRawKey().Prevish(1), nil)}, + {span: s(keys.MakeRangeIDUnreplicatedPrefix(1).Prevish(1), nil)}, + {span: s(keys.RangeForceFlushKey(1), nil)}, + {span: s(keys.MakeRangeIDUnreplicatedPrefix(1).PrefixEnd(), nil)}, + {span: s(keys.LocalRangeIDPrefix.AsRawKey().PrefixEnd(), nil)}, + {span: s(roachpb.KeyMax, nil)}, + + // Point spans overlapping with unreplicated local RangeID spans. + {span: s(keys.MakeRangeIDUnreplicatedPrefix(1), nil), notOk: true}, + {span: s(keys.RangeTombstoneKey(1), nil), notOk: true}, + {span: s(keys.RaftTruncatedStateKey(1), nil), notOk: true}, + {span: s(keys.RaftTruncatedStateKey(2), nil), notOk: true}, + {span: s(keys.MakeRangeIDUnreplicatedPrefix(1).PrefixEnd().Prevish(1), nil), notOk: true}, + + // Tricky spans not overlapping with unreplicated local RangeID spans. + {span: s(nil, keys.LocalRangeIDPrefix.AsRawKey())}, + {span: s(nil, keys.MakeRangeIDUnreplicatedPrefix(1))}, + {span: s(nil, keys.RangeForceFlushKey(1))}, + {span: s(nil, keys.MakeRangeIDUnreplicatedPrefix(1).PrefixEnd().Next())}, + {span: s(nil, keys.LocalRangeIDPrefix.AsRawKey().PrefixEnd().Next())}, + + // Tricky spans overlapping with unreplicated local RangeID spans. + {span: s(nil, keys.MakeRangeIDUnreplicatedPrefix(1).Next()), notOk: true}, + {span: s(nil, keys.RangeTombstoneKey(1).Next()), notOk: true}, + {span: s(nil, keys.RaftTruncatedStateKey(1).Next()), notOk: true}, + {span: s(nil, keys.RaftTruncatedStateKey(2).Next()), notOk: true}, + {span: s(nil, keys.MakeRangeIDUnreplicatedPrefix(1).PrefixEnd()), notOk: true}, + {span: s(nil, keys.LocalRangeIDPrefix.AsRawKey().PrefixEnd()), notOk: true}, // can't decode RangeID. + } + + for _, tc := range testCases { + t.Run("", func(t *testing.T) { + err := overlapsUnreplicatedRangeIDLocalKeys(spanset.TrickySpan(tc.span)) + require.Equal(t, tc.notOk, err != nil, tc.span) + }) + } +} + +// TestOverlapsStoreLocalKeys verifies that the function +// overlapsStoreLocalKeys() successfully catches any overlap with +// store local keys. +func TestOverlapsStoreLocalKeys(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + s := func(start, end roachpb.Key) roachpb.Span { + return roachpb.Span{Key: start, EndKey: end} + } + + testCases := []struct { + span roachpb.Span + notOK bool + }{ + // Full spans not overlapping with Store-local span. + {span: s(roachpb.KeyMin, keys.LocalStorePrefix)}, + {span: s(keys.LocalStoreMax, roachpb.KeyMax)}, + + // Full spans overlapping with Store-local span. + {span: s(roachpb.KeyMin, roachpb.Key(keys.LocalStorePrefix).Next()), notOK: true}, + {span: s(keys.LocalStorePrefix, keys.LocalStoreMax), notOK: true}, + {span: s(keys.StoreGossipKey(), keys.StoreIdentKey()), notOK: true}, + {span: s(keys.LocalStoreMax.Prevish(1), roachpb.KeyMax), notOK: true}, + + // Point spans not overlapping with Store-local span. + {span: s(roachpb.KeyMin, nil)}, + {span: s(roachpb.Key(keys.LocalStorePrefix).Prevish(1), nil)}, + {span: s(keys.LocalStoreMax, nil)}, + {span: s(keys.LocalStoreMax.Next(), nil)}, + {span: s(roachpb.KeyMax, nil)}, + + // Point spans overlapping with Store-local span. + {span: s(keys.LocalStorePrefix, nil), notOK: true}, + {span: s(keys.StoreGossipKey(), nil), notOK: true}, + {span: s(keys.LocalStoreMax.Prevish(1), nil), notOK: true}, + + // Tricky spans with nil StartKey not overlapping with Store-local span. + {span: s(nil, keys.LocalStorePrefix)}, + {span: s(nil, keys.LocalStoreMax.Next())}, + + // Tricky spans with nil StartKey overlapping with Store-local span. + {span: s(nil, roachpb.Key(keys.LocalStorePrefix).Next()), notOK: true}, + {span: s(nil, keys.StoreGossipKey()), notOK: true}, + {span: s(nil, keys.LocalStoreMax), notOK: true}, + } + + for _, tc := range testCases { + t.Run("", func(t *testing.T) { + err := overlapsStoreLocalKeys(spanset.TrickySpan(tc.span)) + require.Equal(t, tc.notOK, err != nil, tc.span) + }) + } +} diff --git a/pkg/kv/kvserver/replica_write.go b/pkg/kv/kvserver/replica_write.go index 3a8e1c167ae6..b830fd663008 100644 --- a/pkg/kv/kvserver/replica_write.go +++ b/pkg/kv/kvserver/replica_write.go @@ -10,6 +10,7 @@ import ( "sync" "time" + "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" @@ -814,8 +815,12 @@ func (r *Replica) newBatchedEngine(g *concurrency.Guard) (storage.Batch, *storag // safe as we're only ever writing at timestamps higher than the timestamp // any write latch would be declared at. But because of this, we don't // assert on access timestamps using spanset.NewBatchAt. - batch = spanset.NewBatch(batch, g.LatchSpans()) + spans := g.LatchSpans() + spans.AddForbiddenMatcher(overlapsUnreplicatedRangeIDLocalKeys) + spans.AddForbiddenMatcher(overlapsStoreLocalKeys) + batch = spanset.NewBatch(batch, spans) } + return batch, opLogger } @@ -884,3 +889,66 @@ func releaseMVCCStats(ms *enginepb.MVCCStats) { ms.Reset() mvccStatsPool.Put(ms) } + +// overlapsUnreplicatedRangeIDLocalKeys checks if the provided span overlaps +// with any unreplicated rangeID local keys. +// Note that we could receive the span with a nil startKey, which has a special +// meaning that the span represents: [endKey.Prev(), endKey). +func overlapsUnreplicatedRangeIDLocalKeys(span spanset.TrickySpan) error { + fullRangeIDLocalSpans := roachpb.Span{ + Key: keys.LocalRangeIDPrefix.AsRawKey(), + EndKey: keys.LocalRangeIDPrefix.AsRawKey().PrefixEnd(), + } + + // If the provided span is completely outside the rangeID local spans for any + // rangeID, then there is no overlap with any rangeID local keys. + if !spanset.Overlaps(fullRangeIDLocalSpans, span) { + return nil + } + + // At this point, we know that we overlap with fullRangeIDLocalSpans. If we + // are not completely within fullRangeIDLocalSpans, return an error as we + // make an assumption that spans should respect the local RangeID tree + // structure, and that spans that partially overlaps with + // fullRangeIDLocalSpans don't make logical sense. + if !spanset.Contains(fullRangeIDLocalSpans, span) { + return errors.Errorf("overlapping an unreplicated rangeID key") + } + + // If the span in inside fullRangeIDLocalSpans, we expect that both start and + // end keys should be in the same rangeID. + rangeIDKey := span.Key + if rangeIDKey == nil { + rangeIDKey = span.EndKey + } + rangeID, err := keys.DecodeRangeIDPrefix(rangeIDKey) + if err != nil { + return errors.NewAssertionErrorWithWrappedErrf(err, + "could not decode range ID for span: %s", span) + } + if spanset.Overlaps(roachpb.Span{ + Key: keys.MakeRangeIDUnreplicatedPrefix(rangeID), + EndKey: keys.MakeRangeIDUnreplicatedPrefix(rangeID).PrefixEnd(), + }, span) { + return errors.Errorf("overlapping an unreplicated rangeID span") + } + + return nil +} + +// overlapsStoreLocalKeys returns an error if the provided span overlaps +// with any store local keys. +// Note that we could receive the span with a nil startKey, which has a special +// meaning that the span represents: [endKey.Prev(), endKey). +func overlapsStoreLocalKeys(span spanset.TrickySpan) error { + localStoreSpan := roachpb.Span{ + Key: keys.LocalStorePrefix, + EndKey: keys.LocalStoreMax, + } + + if spanset.Overlaps(localStoreSpan, span) { + return errors.Errorf("overlaps with store local keys") + } + + return nil +} diff --git a/pkg/kv/kvserver/spanset/batch.go b/pkg/kv/kvserver/spanset/batch.go index 540516f283e3..1803858bfff7 100644 --- a/pkg/kv/kvserver/spanset/batch.go +++ b/pkg/kv/kvserver/spanset/batch.go @@ -923,6 +923,8 @@ func DisableUndeclaredSpanAssertions(rw storage.ReadWriter) storage.ReadWriter { // forbidden span assertions disabled. It does not modify the original batch. // The returned batch shares the same underlying storage.Batch but has its own // SpanSet wrapper with the forbidden span assertion disabled. +// TODO(ibrahim): We eventually want to eliminate all the users of this +// function. func DisableForbiddenSpanAssertions(rw storage.ReadWriter) storage.ReadWriter { switch v := rw.(type) { case *spanSetBatch: