Skip to content

Commit 291f14e

Browse files
committed
kvserver: add overlapsStateEngineKeys
This commit adds the function overlapsStateEngineKeys that checks if a specific span overlaps with any StateEngine keys. This will be used later to enforce RaftEngine assertions and ensure that we do not touch and StateEngine keys. References: #158281 Release note: None
1 parent a7f95a7 commit 291f14e

File tree

6 files changed

+160
-93
lines changed

6 files changed

+160
-93
lines changed

pkg/keys/keys.go

Lines changed: 18 additions & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -1142,18 +1142,18 @@ func MakeRangeIDPrefixBuf(rangeID roachpb.RangeID) RangeIDPrefixBuf {
11421142
return RangeIDPrefixBuf(MakeRangeIDPrefix(rangeID))
11431143
}
11441144

1145-
func (b RangeIDPrefixBuf) replicatedPrefix() roachpb.Key {
1145+
func (b RangeIDPrefixBuf) ReplicatedPrefix() roachpb.Key {
11461146
return append(roachpb.Key(b), LocalRangeIDReplicatedInfix...)
11471147
}
11481148

1149-
func (b RangeIDPrefixBuf) unreplicatedPrefix() roachpb.Key {
1149+
func (b RangeIDPrefixBuf) UnreplicatedPrefix() roachpb.Key {
11501150
return append(roachpb.Key(b), localRangeIDUnreplicatedInfix...)
11511151
}
11521152

11531153
// AbortSpanKey returns a range-local key by Range ID for an AbortSpan
11541154
// entry, with detail specified by encoding the supplied transaction ID.
11551155
func (b RangeIDPrefixBuf) AbortSpanKey(txnID uuid.UUID) roachpb.Key {
1156-
key := append(b.replicatedPrefix(), LocalAbortSpanSuffix...)
1156+
key := append(b.ReplicatedPrefix(), LocalAbortSpanSuffix...)
11571157
return encoding.EncodeBytesAscending(key, txnID.GetBytes())
11581158
}
11591159

@@ -1162,67 +1162,67 @@ func (b RangeIDPrefixBuf) AbortSpanKey(txnID uuid.UUID) roachpb.Key {
11621162
// specific transaction should serialize on latches. The per-transaction bit is
11631163
// achieved by encoding the supplied transaction ID into the key.
11641164
func (b RangeIDPrefixBuf) ReplicatedSharedLocksTransactionLatchingKey(txnID uuid.UUID) roachpb.Key {
1165-
key := append(b.replicatedPrefix(), LocalReplicatedSharedLocksTransactionLatchingKeySuffix...)
1165+
key := append(b.ReplicatedPrefix(), LocalReplicatedSharedLocksTransactionLatchingKeySuffix...)
11661166
return encoding.EncodeBytesAscending(key, txnID.GetBytes())
11671167
}
11681168

11691169
// RangeAppliedStateKey returns a system-local key for the range applied state key.
11701170
// See comment on RangeAppliedStateKey function.
11711171
func (b RangeIDPrefixBuf) RangeAppliedStateKey() roachpb.Key {
1172-
return append(b.replicatedPrefix(), LocalRangeAppliedStateSuffix...)
1172+
return append(b.ReplicatedPrefix(), LocalRangeAppliedStateSuffix...)
11731173
}
11741174

11751175
// RangeForceFlushKey returns a system-local key for the range force flush
11761176
// key.
11771177
func (b RangeIDPrefixBuf) RangeForceFlushKey() roachpb.Key {
1178-
return append(b.replicatedPrefix(), LocalRangeForceFlushSuffix...)
1178+
return append(b.ReplicatedPrefix(), LocalRangeForceFlushSuffix...)
11791179
}
11801180

11811181
// RangeLeaseKey returns a system-local key for a range lease.
11821182
func (b RangeIDPrefixBuf) RangeLeaseKey() roachpb.Key {
1183-
return append(b.replicatedPrefix(), LocalRangeLeaseSuffix...)
1183+
return append(b.ReplicatedPrefix(), LocalRangeLeaseSuffix...)
11841184
}
11851185

11861186
// RangePriorReadSummaryKey returns a system-local key for a range's prior read
11871187
// summary.
11881188
func (b RangeIDPrefixBuf) RangePriorReadSummaryKey() roachpb.Key {
1189-
return append(b.replicatedPrefix(), LocalRangePriorReadSummarySuffix...)
1189+
return append(b.ReplicatedPrefix(), LocalRangePriorReadSummarySuffix...)
11901190
}
11911191

11921192
// RangeGCThresholdKey returns a system-local key for the GC threshold.
11931193
func (b RangeIDPrefixBuf) RangeGCThresholdKey() roachpb.Key {
1194-
return append(b.replicatedPrefix(), LocalRangeGCThresholdSuffix...)
1194+
return append(b.ReplicatedPrefix(), LocalRangeGCThresholdSuffix...)
11951195
}
11961196

11971197
// RangeGCHintKey returns a range-local key for the GC hint data.
11981198
func (b RangeIDPrefixBuf) RangeGCHintKey() roachpb.Key {
1199-
return append(b.replicatedPrefix(), LocalRangeGCHintSuffix...)
1199+
return append(b.ReplicatedPrefix(), LocalRangeGCHintSuffix...)
12001200
}
12011201

12021202
// RangeVersionKey returns a system-local key for the range version.
12031203
func (b RangeIDPrefixBuf) RangeVersionKey() roachpb.Key {
1204-
return append(b.replicatedPrefix(), LocalRangeVersionSuffix...)
1204+
return append(b.ReplicatedPrefix(), LocalRangeVersionSuffix...)
12051205
}
12061206

12071207
// RangeTombstoneKey returns a system-local key for a range tombstone.
12081208
func (b RangeIDPrefixBuf) RangeTombstoneKey() roachpb.Key {
1209-
return append(b.unreplicatedPrefix(), LocalRangeTombstoneSuffix...)
1209+
return append(b.UnreplicatedPrefix(), LocalRangeTombstoneSuffix...)
12101210
}
12111211

12121212
// RaftTruncatedStateKey returns a system-local key for a RaftTruncatedState.
12131213
func (b RangeIDPrefixBuf) RaftTruncatedStateKey() roachpb.Key {
1214-
return append(b.unreplicatedPrefix(), LocalRaftTruncatedStateSuffix...)
1214+
return append(b.UnreplicatedPrefix(), LocalRaftTruncatedStateSuffix...)
12151215
}
12161216

12171217
// RaftHardStateKey returns a system-local key for a Raft HardState.
12181218
func (b RangeIDPrefixBuf) RaftHardStateKey() roachpb.Key {
1219-
return append(b.unreplicatedPrefix(), LocalRaftHardStateSuffix...)
1219+
return append(b.UnreplicatedPrefix(), LocalRaftHardStateSuffix...)
12201220
}
12211221

12221222
// RaftLogPrefix returns the system-local prefix shared by all Entries
12231223
// in a Raft log.
12241224
func (b RangeIDPrefixBuf) RaftLogPrefix() roachpb.Key {
1225-
return append(b.unreplicatedPrefix(), LocalRaftLogSuffix...)
1225+
return append(b.UnreplicatedPrefix(), LocalRaftLogSuffix...)
12261226
}
12271227

12281228
// RaftLogKey returns a system-local key for a Raft log entry.
@@ -1232,17 +1232,17 @@ func (b RangeIDPrefixBuf) RaftLogKey(logIndex kvpb.RaftIndex) roachpb.Key {
12321232

12331233
// RaftReplicaIDKey returns a system-local key for a RaftReplicaID.
12341234
func (b RangeIDPrefixBuf) RaftReplicaIDKey() roachpb.Key {
1235-
return append(b.unreplicatedPrefix(), LocalRaftReplicaIDSuffix...)
1235+
return append(b.UnreplicatedPrefix(), LocalRaftReplicaIDSuffix...)
12361236
}
12371237

12381238
// RangeLastReplicaGCTimestampKey returns a range-local key for
12391239
// the range's last replica GC timestamp.
12401240
func (b RangeIDPrefixBuf) RangeLastReplicaGCTimestampKey() roachpb.Key {
1241-
return append(b.unreplicatedPrefix(), LocalRangeLastReplicaGCTimestampSuffix...)
1241+
return append(b.UnreplicatedPrefix(), LocalRangeLastReplicaGCTimestampSuffix...)
12421242
}
12431243

12441244
// MVCCRangeKeyGCKey returns a range local key protecting range
12451245
// tombstone mvcc stats calculations during range tombstone GC.
12461246
func (b RangeIDPrefixBuf) MVCCRangeKeyGCKey() roachpb.Key {
1247-
return append(b.unreplicatedPrefix(), LocalRangeMVCCRangeKeyGCLockSuffix...)
1247+
return append(b.UnreplicatedPrefix(), LocalRangeMVCCRangeKeyGCLockSuffix...)
12481248
}

pkg/kv/kvserver/replica_raftstorage.go

Lines changed: 65 additions & 20 deletions
Original file line numberDiff line numberDiff line change
@@ -880,18 +880,26 @@ func (r *Replica) destroyInfoRaftMuLocked() kvstorage.DestroyReplicaInfo {
880880
}
881881
}
882882

883-
// overlapsLocalRaftKeys returns an error if the provided span overlaps with any
884-
// unreplicated rangeID local raft keys.
883+
// validateIsStateEngineSpan asserts that the provided span only overlaps with
884+
// keys in the State engine and returns an error if not.
885885
// Note that we could receive the span with a nil startKey, which has a special
886886
// meaning that the span represents: [endKey.Prev(), endKey).
887-
func overlapsLocalRaftKeys(span spanset.TrickySpan) error {
888-
fullRangeIDLocalSpans := roachpb.Span{
889-
Key: keys.LocalRangeIDPrefix.AsRawKey(),
890-
EndKey: keys.LocalRangeIDPrefix.AsRawKey().PrefixEnd(),
887+
func validateIsStateEngineSpan(span spanset.TrickySpan) error {
888+
// If the provided span overlaps with local store span, it cannot be a
889+
// StateEngine span because Store-local keys belong to the LogEngine.
890+
if spanset.Overlaps(roachpb.Span{
891+
Key: keys.LocalStorePrefix,
892+
EndKey: keys.LocalStoreMax,
893+
}, span) {
894+
return errors.Errorf("overlaps with store local keys")
891895
}
892896

893897
// If the provided span is completely outside the rangeID local spans for any
894898
// rangeID, then there is no overlap with any rangeID local keys.
899+
fullRangeIDLocalSpans := roachpb.Span{
900+
Key: keys.LocalRangeIDPrefix.AsRawKey(),
901+
EndKey: keys.LocalRangeIDPrefix.AsRawKey().PrefixEnd(),
902+
}
895903
if !spanset.Overlaps(fullRangeIDLocalSpans, span) {
896904
return nil
897905
}
@@ -919,42 +927,79 @@ func overlapsLocalRaftKeys(span spanset.TrickySpan) error {
919927

920928
// If the span is inside RangeIDLocalSpans but outside RangeIDUnreplicated,
921929
// it cannot overlap local raft keys.
930+
rangeIDPrefixBuf := keys.MakeRangeIDPrefixBuf(rangeID)
922931
if !spanset.Overlaps(roachpb.Span{
923-
Key: keys.MakeRangeIDUnreplicatedPrefix(rangeID),
924-
EndKey: keys.MakeRangeIDUnreplicatedPrefix(rangeID).PrefixEnd(),
932+
Key: rangeIDPrefixBuf.UnreplicatedPrefix(),
933+
EndKey: rangeIDPrefixBuf.UnreplicatedPrefix().PrefixEnd(),
925934
}, span) {
926935
return nil
927936
}
928937

929-
// RangeTombstoneKey and RaftReplicaIDKey are considered part of the
930-
// StateEngine keys. We need to exclude them from the check below.
931-
if span.Key != nil && roachpb.Span(span).Equal(roachpb.Span{
932-
Key: keys.RangeTombstoneKey(rangeID),
938+
// RangeTombstoneKey and RaftReplicaIDKey belong to the StateEngine, and can
939+
// be accessed as point keys.
940+
if roachpb.Span(span).Equal(roachpb.Span{
941+
Key: rangeIDPrefixBuf.RangeTombstoneKey(),
933942
}) {
934943
return nil
935944
}
936945

937-
if span.Key != nil && roachpb.Span(span).Equal(roachpb.Span{
938-
Key: keys.RaftReplicaIDKey(rangeID),
946+
if roachpb.Span(span).Equal(roachpb.Span{
947+
Key: rangeIDPrefixBuf.RaftReplicaIDKey(),
939948
}) {
940949
return nil
941950
}
942951

943952
return errors.Errorf("overlapping an unreplicated rangeID span")
944953
}
945954

946-
// overlapsStoreLocalKeys returns an error if the provided span overlaps
947-
// with any store local keys.
955+
// validateIsRaftEngineSpan asserts that the provided span only overlaps with
956+
// keys in the Raft engine and returns an error if not.
948957
// Note that we could receive the span with a nil startKey, which has a special
949958
// meaning that the span represents: [endKey.Prev(), endKey).
950-
func overlapsStoreLocalKeys(span spanset.TrickySpan) error {
951-
localStoreSpan := roachpb.Span{
959+
func validateIsRaftEngineSpan(span spanset.TrickySpan) error {
960+
// The LogEngine owns only Store-local and RangeID-local raft keys. A span
961+
// inside Store-local is correct. If it's only partially inside, an error is
962+
// returned below, as part of checking RangeID-local spans.
963+
if spanset.Contains(roachpb.Span{
952964
Key: keys.LocalStorePrefix,
953965
EndKey: keys.LocalStoreMax,
966+
}, span) {
967+
return nil
954968
}
955969

956-
if spanset.Overlaps(localStoreSpan, span) {
957-
return errors.Errorf("overlaps with store local keys")
970+
// At this point, the remaining possible LogEngine keys are inside
971+
// LocalRangeID spans. If the span is not completely inside it, it must
972+
// overlap with some StateEngine keys.
973+
if !spanset.Contains(roachpb.Span{
974+
Key: keys.LocalRangeIDPrefix.AsRawKey(),
975+
EndKey: keys.LocalRangeIDPrefix.AsRawKey().PrefixEnd(),
976+
}, span) {
977+
return errors.Errorf("overlaps with state engine keys")
978+
}
979+
980+
// If the span in inside LocalRangeID, we assume that both start and
981+
// end keys should be in the same rangeID.
982+
rangeIDKey := span.Key
983+
if rangeIDKey == nil {
984+
rangeIDKey = span.EndKey
985+
}
986+
rangeID, err := keys.DecodeRangeIDPrefix(rangeIDKey)
987+
if err != nil {
988+
return errors.NewAssertionErrorWithWrappedErrf(err,
989+
"could not decode range ID for span: %s", span)
990+
}
991+
rangeIDPrefixBuf := keys.MakeRangeIDPrefixBuf(rangeID)
992+
if !spanset.Contains(roachpb.Span{
993+
Key: rangeIDPrefixBuf.UnreplicatedPrefix(),
994+
EndKey: rangeIDPrefixBuf.UnreplicatedPrefix().PrefixEnd(),
995+
}, span) {
996+
return errors.Errorf("overlaps with state engine keys")
997+
}
998+
if spanset.Overlaps(roachpb.Span{Key: rangeIDPrefixBuf.RangeTombstoneKey()}, span) {
999+
return errors.Errorf("overlaps with state engine keys")
1000+
}
1001+
if spanset.Overlaps(roachpb.Span{Key: rangeIDPrefixBuf.RaftReplicaIDKey()}, span) {
1002+
return errors.Errorf("overlaps with state engine keys")
9581003
}
9591004

9601005
return nil

pkg/kv/kvserver/replica_read.go

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -94,8 +94,7 @@ func (r *Replica) executeReadOnlyBatch(
9494
}
9595
if util.RaceEnabled {
9696
spans := g.LatchSpans()
97-
spans.AddForbiddenMatcher(overlapsLocalRaftKeys)
98-
spans.AddForbiddenMatcher(overlapsStoreLocalKeys)
97+
spans.AddForbiddenMatcher(validateIsStateEngineSpan)
9998
rw = spanset.NewReadWriterAt(rw, spans, ba.Timestamp)
10099
}
101100
defer rw.Close()

0 commit comments

Comments
 (0)