From e5ad056a6a57fc5c967d4018cd7aeecc6443e5eb Mon Sep 17 00:00:00 2001 From: iskettaneh <173953022+iskettaneh@users.noreply.github.com> Date: Fri, 21 Nov 2025 12:09:50 -0500 Subject: [PATCH] spanset: assert read-only batches don't access store local and unreplicated RangeID local keys Similar to what we did in #157153, this commit does the same but for read-only batches. Testing showed no failures. --- pkg/kv/kvserver/replica.go | 64 ++++++++++++++++++++++++++++++++ pkg/kv/kvserver/replica_read.go | 5 ++- pkg/kv/kvserver/replica_write.go | 64 -------------------------------- 3 files changed, 68 insertions(+), 65 deletions(-) diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index e25b55002d78..4b9d11bb63a0 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -36,6 +36,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/logstore" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/rafttrace" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/rangefeed" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/split" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/tenantrate" "github.com/cockroachdb/cockroach/pkg/raft" @@ -2984,3 +2985,66 @@ func (r *Replica) SendStreamStats(stats *rac2.RangeSendStreamStats) { r.flowControlV2.SendStreamStats(stats) } } + +// 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/replica_read.go b/pkg/kv/kvserver/replica_read.go index f659693cca58..867ddd78c6e1 100644 --- a/pkg/kv/kvserver/replica_read.go +++ b/pkg/kv/kvserver/replica_read.go @@ -93,7 +93,10 @@ func (r *Replica) executeReadOnlyBatch( return nil, g, nil, kvpb.NewError(err) } if util.RaceEnabled { - rw = spanset.NewReadWriterAt(rw, g.LatchSpans(), ba.Timestamp) + spans := g.LatchSpans() + spans.AddForbiddenMatcher(overlapsUnreplicatedRangeIDLocalKeys) + spans.AddForbiddenMatcher(overlapsStoreLocalKeys) + rw = spanset.NewReadWriterAt(rw, spans, ba.Timestamp) } defer rw.Close() diff --git a/pkg/kv/kvserver/replica_write.go b/pkg/kv/kvserver/replica_write.go index b830fd663008..7cae1f23e8b9 100644 --- a/pkg/kv/kvserver/replica_write.go +++ b/pkg/kv/kvserver/replica_write.go @@ -10,7 +10,6 @@ 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" @@ -889,66 +888,3 @@ 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 -}