Skip to content

Commit 29dc524

Browse files
committed
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.
1 parent 0930e79 commit 29dc524

File tree

6 files changed

+213
-3
lines changed

6 files changed

+213
-3
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_end_transaction.go

Lines changed: 5 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -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/replica_test.go

Lines changed: 125 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -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+
}

pkg/kv/kvserver/replica_write.go

Lines changed: 69 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -10,6 +10,7 @@ import (
1010
"sync"
1111
"time"
1212

13+
"github.com/cockroachdb/cockroach/pkg/keys"
1314
"github.com/cockroachdb/cockroach/pkg/kv"
1415
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
1516
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval"
@@ -814,8 +815,12 @@ func (r *Replica) newBatchedEngine(g *concurrency.Guard) (storage.Batch, *storag
814815
// safe as we're only ever writing at timestamps higher than the timestamp
815816
// any write latch would be declared at. But because of this, we don't
816817
// assert on access timestamps using spanset.NewBatchAt.
817-
batch = spanset.NewBatch(batch, g.LatchSpans())
818+
spans := g.LatchSpans()
819+
spans.AddForbiddenMatcher(overlapsUnreplicatedRangeIDLocalKeys)
820+
spans.AddForbiddenMatcher(overlapsStoreLocalKeys)
821+
batch = spanset.NewBatch(batch, spans)
818822
}
823+
819824
return batch, opLogger
820825
}
821826

@@ -884,3 +889,66 @@ func releaseMVCCStats(ms *enginepb.MVCCStats) {
884889
ms.Reset()
885890
mvccStatsPool.Put(ms)
886891
}
892+
893+
// overlapsUnreplicatedRangeIDLocalKeys checks if the provided span overlaps
894+
// with any unreplicated rangeID local keys.
895+
// Note that we could receive the span with a nil startKey, which has a special
896+
// meaning that the span represents: [endKey.Prev(), endKey).
897+
func overlapsUnreplicatedRangeIDLocalKeys(span spanset.TrickySpan) error {
898+
fullRangeIDLocalSpans := roachpb.Span{
899+
Key: keys.LocalRangeIDPrefix.AsRawKey(),
900+
EndKey: keys.LocalRangeIDPrefix.AsRawKey().PrefixEnd(),
901+
}
902+
903+
// If the provided span is completely outside the rangeID local spans for any
904+
// rangeID, then there is no overlap with any rangeID local keys.
905+
if !spanset.Overlaps(fullRangeIDLocalSpans, span) {
906+
return nil
907+
}
908+
909+
// At this point, we know that we overlap with fullRangeIDLocalSpans. If we
910+
// are not completely within fullRangeIDLocalSpans, return an error as we
911+
// make an assumption that spans should respect the local RangeID tree
912+
// structure, and that spans that partially overlaps with
913+
// fullRangeIDLocalSpans don't make logical sense.
914+
if !spanset.Contains(fullRangeIDLocalSpans, span) {
915+
return errors.Errorf("overlapping an unreplicated rangeID key")
916+
}
917+
918+
// If the span in inside fullRangeIDLocalSpans, we expect that both start and
919+
// end keys should be in the same rangeID.
920+
rangeIDKey := span.Key
921+
if rangeIDKey == nil {
922+
rangeIDKey = span.EndKey
923+
}
924+
rangeID, err := keys.DecodeRangeIDPrefix(rangeIDKey)
925+
if err != nil {
926+
return errors.NewAssertionErrorWithWrappedErrf(err,
927+
"could not decode range ID for span: %s", span)
928+
}
929+
if spanset.Overlaps(roachpb.Span{
930+
Key: keys.MakeRangeIDUnreplicatedPrefix(rangeID),
931+
EndKey: keys.MakeRangeIDUnreplicatedPrefix(rangeID).PrefixEnd(),
932+
}, span) {
933+
return errors.Errorf("overlapping an unreplicated rangeID span")
934+
}
935+
936+
return nil
937+
}
938+
939+
// overlapsStoreLocalKeys returns an error if the provided span overlaps
940+
// with any store local keys.
941+
// Note that we could receive the span with a nil startKey, which has a special
942+
// meaning that the span represents: [endKey.Prev(), endKey).
943+
func overlapsStoreLocalKeys(span spanset.TrickySpan) error {
944+
localStoreSpan := roachpb.Span{
945+
Key: keys.LocalStorePrefix,
946+
EndKey: keys.LocalStoreMax,
947+
}
948+
949+
if spanset.Overlaps(localStoreSpan, span) {
950+
return errors.Errorf("overlaps with store local keys")
951+
}
952+
953+
return nil
954+
}

pkg/kv/kvserver/spanset/batch.go

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -923,6 +923,8 @@ func DisableUndeclaredSpanAssertions(rw storage.ReadWriter) storage.ReadWriter {
923923
// forbidden span assertions disabled. It does not modify the original batch.
924924
// The returned batch shares the same underlying storage.Batch but has its own
925925
// SpanSet wrapper with the forbidden span assertion disabled.
926+
// TODO(ibrahim): We eventually want to eliminate all the users of this
927+
// function.
926928
func DisableForbiddenSpanAssertions(rw storage.ReadWriter) storage.ReadWriter {
927929
switch v := rw.(type) {
928930
case *spanSetBatch:

0 commit comments

Comments
 (0)