Skip to content

Commit ad834d9

Browse files
committed
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`.
1 parent d54cde3 commit ad834d9

File tree

5 files changed

+229
-115
lines changed

5 files changed

+229
-115
lines changed

pkg/kv/kvserver/replica_eval_context_span.go

Lines changed: 7 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -104,7 +104,7 @@ func (rec *SpanSetReplicaEvalContext) IsFirstRange() bool {
104104
func (rec SpanSetReplicaEvalContext) Desc() *roachpb.RangeDescriptor {
105105
desc := rec.i.Desc()
106106
rec.ss.AssertAllowed(spanset.SpanReadOnly,
107-
roachpb.Span{Key: keys.RangeDescriptorKey(desc.StartKey)},
107+
spanset.TrickySpan{Key: keys.RangeDescriptorKey(desc.StartKey)},
108108
)
109109
return desc
110110
}
@@ -145,7 +145,7 @@ func (rec SpanSetReplicaEvalContext) CanCreateTxnRecord(
145145
ctx context.Context, txnID uuid.UUID, txnKey []byte, txnMinTS hlc.Timestamp,
146146
) (bool, kvpb.TransactionAbortedReason) {
147147
rec.ss.AssertAllowed(spanset.SpanReadOnly,
148-
roachpb.Span{Key: keys.TransactionKey(txnKey, txnID)},
148+
spanset.TrickySpan{Key: keys.TransactionKey(txnKey, txnID)},
149149
)
150150
return rec.i.CanCreateTxnRecord(ctx, txnID, txnKey, txnMinTS)
151151
}
@@ -157,7 +157,7 @@ func (rec SpanSetReplicaEvalContext) MinTxnCommitTS(
157157
ctx context.Context, txnID uuid.UUID, txnKey []byte,
158158
) hlc.Timestamp {
159159
rec.ss.AssertAllowed(spanset.SpanReadOnly,
160-
roachpb.Span{Key: keys.TransactionKey(txnKey, txnID)},
160+
spanset.TrickySpan{Key: keys.TransactionKey(txnKey, txnID)},
161161
)
162162
return rec.i.MinTxnCommitTS(ctx, txnID, txnKey)
163163
}
@@ -167,7 +167,7 @@ func (rec SpanSetReplicaEvalContext) MinTxnCommitTS(
167167
// not be served.
168168
func (rec SpanSetReplicaEvalContext) GetGCThreshold() hlc.Timestamp {
169169
rec.ss.AssertAllowed(spanset.SpanReadOnly,
170-
roachpb.Span{Key: keys.RangeGCThresholdKey(rec.GetRangeID())},
170+
spanset.TrickySpan{Key: keys.RangeGCThresholdKey(rec.GetRangeID())},
171171
)
172172
return rec.i.GetGCThreshold()
173173
}
@@ -191,7 +191,7 @@ func (rec SpanSetReplicaEvalContext) GetLastReplicaGCTimestamp(
191191
ctx context.Context,
192192
) (hlc.Timestamp, error) {
193193
if err := rec.ss.CheckAllowed(spanset.SpanReadOnly,
194-
roachpb.Span{Key: keys.RangeLastReplicaGCTimestampKey(rec.GetRangeID())},
194+
spanset.TrickySpan{Key: keys.RangeLastReplicaGCTimestampKey(rec.GetRangeID())},
195195
); err != nil {
196196
return hlc.Timestamp{}, err
197197
}
@@ -222,11 +222,11 @@ func (rec *SpanSetReplicaEvalContext) GetCurrentReadSummary(ctx context.Context)
222222
// To capture a read summary over the range, all keys must be latched for
223223
// writing to prevent any concurrent reads or writes.
224224
desc := rec.i.Desc()
225-
rec.ss.AssertAllowed(spanset.SpanReadWrite, roachpb.Span{
225+
rec.ss.AssertAllowed(spanset.SpanReadWrite, spanset.TrickySpan{
226226
Key: keys.MakeRangeKeyPrefix(desc.StartKey),
227227
EndKey: keys.MakeRangeKeyPrefix(desc.EndKey),
228228
})
229-
rec.ss.AssertAllowed(spanset.SpanReadWrite, roachpb.Span{
229+
rec.ss.AssertAllowed(spanset.SpanReadWrite, spanset.TrickySpan{
230230
Key: desc.StartKey.AsRawKey(),
231231
EndKey: desc.EndKey.AsRawKey(),
232232
})

pkg/kv/kvserver/replica_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -2732,7 +2732,7 @@ func TestReplicaLatchingSplitDeclaresWrites(t *testing.T) {
27322732
{spanset.SpanReadWrite, roachpb.Key("b"), false},
27332733
{spanset.SpanReadWrite, roachpb.Key("d"), true},
27342734
} {
2735-
err := spans.CheckAllowed(tc.access, roachpb.Span{Key: tc.key})
2735+
err := spans.CheckAllowed(tc.access, spanset.TrickySpan{Key: tc.key})
27362736
if tc.expectAccess {
27372737
require.NoError(t, err)
27382738
} else {

pkg/kv/kvserver/spanset/batch.go

Lines changed: 38 additions & 42 deletions
Original file line numberDiff line numberDiff line change
@@ -77,15 +77,15 @@ func (i *MVCCIterator) Valid() (bool, error) {
7777
// SeekGE is part of the storage.MVCCIterator interface.
7878
func (i *MVCCIterator) SeekGE(key storage.MVCCKey) {
7979
i.i.SeekGE(key)
80-
i.checkAllowed(roachpb.Span{Key: key.Key}, true)
80+
i.checkAllowed(TrickySpan{Key: key.Key}, true)
8181
}
8282

8383
// SeekLT is part of the storage.MVCCIterator interface.
8484
func (i *MVCCIterator) SeekLT(key storage.MVCCKey) {
8585
i.i.SeekLT(key)
8686
// CheckAllowed{At} supports the span representation of [,key), which
8787
// corresponds to the span [key.Prev(),).
88-
i.checkAllowed(roachpb.Span{EndKey: key.Key}, true)
88+
i.checkAllowed(TrickySpan{EndKey: key.Key}, true)
8989
}
9090

9191
// Next is part of the storage.MVCCIterator interface.
@@ -117,12 +117,12 @@ func (i *MVCCIterator) checkAllowedCurrPosForward(errIfDisallowed bool) {
117117
// as long as the iterator itself is configured with proper boundaries.
118118
return
119119
}
120-
i.checkAllowedValidPos(roachpb.Span{Key: i.UnsafeKey().Key}, errIfDisallowed)
120+
i.checkAllowedValidPos(TrickySpan{Key: i.UnsafeKey().Key}, errIfDisallowed)
121121
}
122122

123123
// checkAllowed checks the provided span if the current iterator position is
124124
// valid.
125-
func (i *MVCCIterator) checkAllowed(span roachpb.Span, errIfDisallowed bool) {
125+
func (i *MVCCIterator) checkAllowed(span TrickySpan, errIfDisallowed bool) {
126126
i.invalid = false
127127
i.err = nil
128128
if ok, _ := i.i.Valid(); !ok {
@@ -134,7 +134,7 @@ func (i *MVCCIterator) checkAllowed(span roachpb.Span, errIfDisallowed bool) {
134134
i.checkAllowedValidPos(span, errIfDisallowed)
135135
}
136136

137-
func (i *MVCCIterator) checkAllowedValidPos(span roachpb.Span, errIfDisallowed bool) {
137+
func (i *MVCCIterator) checkAllowedValidPos(span TrickySpan, errIfDisallowed bool) {
138138
var err error
139139
if i.spansOnly {
140140
err = i.spans.CheckAllowed(SpanReadOnly, span)
@@ -213,11 +213,11 @@ func (i *MVCCIterator) FindSplitKey(
213213
start, end, minSplitKey roachpb.Key, targetSize int64,
214214
) (storage.MVCCKey, error) {
215215
if i.spansOnly {
216-
if err := i.spans.CheckAllowed(SpanReadOnly, roachpb.Span{Key: start, EndKey: end}); err != nil {
216+
if err := i.spans.CheckAllowed(SpanReadOnly, TrickySpan{Key: start, EndKey: end}); err != nil {
217217
return storage.MVCCKey{}, err
218218
}
219219
} else {
220-
if err := i.spans.CheckAllowedAt(SpanReadOnly, roachpb.Span{Key: start, EndKey: end}, i.ts); err != nil {
220+
if err := i.spans.CheckAllowedAt(SpanReadOnly, TrickySpan{Key: start, EndKey: end}, i.ts); err != nil {
221221
return storage.MVCCKey{}, err
222222
}
223223
}
@@ -261,10 +261,10 @@ func (i *EngineIterator) SeekEngineKeyGE(key storage.EngineKey) (valid bool, err
261261
}
262262
if key.IsMVCCKey() && !i.spansOnly {
263263
mvccKey, _ := key.ToMVCCKey()
264-
if err := i.spans.CheckAllowedAt(SpanReadOnly, roachpb.Span{Key: mvccKey.Key}, i.ts); err != nil {
264+
if err := i.spans.CheckAllowedAt(SpanReadOnly, TrickySpan{Key: mvccKey.Key}, i.ts); err != nil {
265265
return false, err
266266
}
267-
} else if err = i.spans.CheckAllowed(SpanReadOnly, roachpb.Span{Key: key.Key}); err != nil {
267+
} else if err = i.spans.CheckAllowed(SpanReadOnly, TrickySpan{Key: key.Key}); err != nil {
268268
return false, err
269269
}
270270
return valid, err
@@ -278,10 +278,10 @@ func (i *EngineIterator) SeekEngineKeyLT(key storage.EngineKey) (valid bool, err
278278
}
279279
if key.IsMVCCKey() && !i.spansOnly {
280280
mvccKey, _ := key.ToMVCCKey()
281-
if err := i.spans.CheckAllowedAt(SpanReadOnly, roachpb.Span{Key: mvccKey.Key}, i.ts); err != nil {
281+
if err := i.spans.CheckAllowedAt(SpanReadOnly, TrickySpan{Key: mvccKey.Key}, i.ts); err != nil {
282282
return false, err
283283
}
284-
} else if err = i.spans.CheckAllowed(SpanReadOnly, roachpb.Span{EndKey: key.Key}); err != nil {
284+
} else if err = i.spans.CheckAllowed(SpanReadOnly, TrickySpan{EndKey: key.Key}); err != nil {
285285
return false, err
286286
}
287287
return valid, err
@@ -313,7 +313,7 @@ func (i *EngineIterator) SeekEngineKeyGEWithLimit(
313313
if state != pebble.IterValid {
314314
return state, err
315315
}
316-
if err = i.spans.CheckAllowed(SpanReadOnly, roachpb.Span{Key: key.Key}); err != nil {
316+
if err = i.spans.CheckAllowed(SpanReadOnly, TrickySpan{Key: key.Key}); err != nil {
317317
return pebble.IterExhausted, err
318318
}
319319
return state, err
@@ -327,7 +327,7 @@ func (i *EngineIterator) SeekEngineKeyLTWithLimit(
327327
if state != pebble.IterValid {
328328
return state, err
329329
}
330-
if err = i.spans.CheckAllowed(SpanReadOnly, roachpb.Span{EndKey: key.Key}); err != nil {
330+
if err = i.spans.CheckAllowed(SpanReadOnly, TrickySpan{EndKey: key.Key}); err != nil {
331331
return pebble.IterExhausted, err
332332
}
333333
return state, err
@@ -354,11 +354,11 @@ func (i *EngineIterator) checkKeyAllowed() (valid bool, err error) {
354354
}
355355
if key.IsMVCCKey() && !i.spansOnly {
356356
mvccKey, _ := key.ToMVCCKey()
357-
if err := i.spans.CheckAllowedAt(SpanReadOnly, roachpb.Span{Key: mvccKey.Key}, i.ts); err != nil {
357+
if err := i.spans.CheckAllowedAt(SpanReadOnly, TrickySpan{Key: mvccKey.Key}, i.ts); err != nil {
358358
// Invalid, but no error.
359359
return false, nil // nolint:returnerrcheck
360360
}
361-
} else if err = i.spans.CheckAllowed(SpanReadOnly, roachpb.Span{Key: key.Key}); err != nil {
361+
} else if err = i.spans.CheckAllowed(SpanReadOnly, TrickySpan{Key: key.Key}); err != nil {
362362
// Invalid, but no error.
363363
return false, nil // nolint:returnerrcheck
364364
}
@@ -469,11 +469,11 @@ func (s spanSetReader) MVCCIterate(
469469
f func(storage.MVCCKeyValue, storage.MVCCRangeKeyStack) error,
470470
) error {
471471
if s.spansOnly {
472-
if err := s.spans.CheckAllowed(SpanReadOnly, roachpb.Span{Key: start, EndKey: end}); err != nil {
472+
if err := s.spans.CheckAllowed(SpanReadOnly, TrickySpan{Key: start, EndKey: end}); err != nil {
473473
return err
474474
}
475475
} else {
476-
if err := s.spans.CheckAllowedAt(SpanReadOnly, roachpb.Span{Key: start, EndKey: end}, s.ts); err != nil {
476+
if err := s.spans.CheckAllowedAt(SpanReadOnly, TrickySpan{Key: start, EndKey: end}, s.ts); err != nil {
477477
return err
478478
}
479479
}
@@ -535,11 +535,11 @@ func (s spanSetWriter) ApplyBatchRepr(repr []byte, sync bool) error {
535535

536536
func (s spanSetWriter) checkAllowed(key roachpb.Key) error {
537537
if s.spansOnly {
538-
if err := s.spans.CheckAllowed(SpanReadWrite, roachpb.Span{Key: key}); err != nil {
538+
if err := s.spans.CheckAllowed(SpanReadWrite, TrickySpan{Key: key}); err != nil {
539539
return err
540540
}
541541
} else {
542-
if err := s.spans.CheckAllowedAt(SpanReadWrite, roachpb.Span{Key: key}, s.ts); err != nil {
542+
if err := s.spans.CheckAllowedAt(SpanReadWrite, TrickySpan{Key: key}, s.ts); err != nil {
543543
return err
544544
}
545545
}
@@ -561,7 +561,7 @@ func (s spanSetWriter) ClearUnversioned(key roachpb.Key, opts storage.ClearOptio
561561
}
562562

563563
func (s spanSetWriter) ClearEngineKey(key storage.EngineKey, opts storage.ClearOptions) error {
564-
if err := s.spans.CheckAllowed(SpanReadWrite, roachpb.Span{Key: key.Key}); err != nil {
564+
if err := s.spans.CheckAllowed(SpanReadWrite, TrickySpan{Key: key.Key}); err != nil {
565565
return err
566566
}
567567
return s.w.ClearEngineKey(key, opts)
@@ -575,11 +575,11 @@ func (s spanSetWriter) SingleClearEngineKey(key storage.EngineKey) error {
575575

576576
func (s spanSetWriter) checkAllowedRange(start, end roachpb.Key) error {
577577
if s.spansOnly {
578-
if err := s.spans.CheckAllowed(SpanReadWrite, roachpb.Span{Key: start, EndKey: end}); err != nil {
578+
if err := s.spans.CheckAllowed(SpanReadWrite, TrickySpan{Key: start, EndKey: end}); err != nil {
579579
return err
580580
}
581581
} else {
582-
if err := s.spans.CheckAllowedAt(SpanReadWrite, roachpb.Span{Key: start, EndKey: end}, s.ts); err != nil {
582+
if err := s.spans.CheckAllowedAt(SpanReadWrite, TrickySpan{Key: start, EndKey: end}, s.ts); err != nil {
583583
return err
584584
}
585585
}
@@ -661,11 +661,11 @@ func (s spanSetWriter) ClearMVCCRangeKey(rangeKey storage.MVCCRangeKey) error {
661661

662662
func (s spanSetWriter) Merge(key storage.MVCCKey, value []byte) error {
663663
if s.spansOnly {
664-
if err := s.spans.CheckAllowed(SpanReadWrite, roachpb.Span{Key: key.Key}); err != nil {
664+
if err := s.spans.CheckAllowed(SpanReadWrite, TrickySpan{Key: key.Key}); err != nil {
665665
return err
666666
}
667667
} else {
668-
if err := s.spans.CheckAllowedAt(SpanReadWrite, roachpb.Span{Key: key.Key}, s.ts); err != nil {
668+
if err := s.spans.CheckAllowedAt(SpanReadWrite, TrickySpan{Key: key.Key}, s.ts); err != nil {
669669
return err
670670
}
671671
}
@@ -697,7 +697,7 @@ func (s spanSetWriter) PutEngineKey(key storage.EngineKey, value []byte) error {
697697
if !s.spansOnly {
698698
panic("cannot do timestamp checking for putting EngineKey")
699699
}
700-
if err := s.spans.CheckAllowed(SpanReadWrite, roachpb.Span{Key: key.Key}); err != nil {
700+
if err := s.spans.CheckAllowed(SpanReadWrite, TrickySpan{Key: key.Key}); err != nil {
701701
return err
702702
}
703703
return s.w.PutEngineKey(key, value)
@@ -768,7 +768,7 @@ type spanSetBatch struct {
768768
ReadWriter
769769
b storage.Batch
770770
// TODO(ibrahim): The fields spans, spansOnly, and ts don't seem to be used.
771-
// Consider removing them and performing the necessary clean ups.
771+
// Consider removing or marking them as intended.
772772
spans *SpanSet
773773

774774
spansOnly bool
@@ -840,20 +840,16 @@ func (s spanSetBatch) ClearRawEncodedRange(start, end []byte) error {
840840
return s.b.ClearRawEncodedRange(start, end)
841841
}
842842

843-
// clone returns a shallow copy of the spanSetBatch. The returned batch shares
844-
// the same underlying storage.Batch but has its own spanSetBatch wrapper with
845-
// a new copy of the SpanSet that uses a shallow copy of the underlying spans.
846-
func (s spanSetBatch) clone() *spanSetBatch {
847-
return &spanSetBatch{
848-
ReadWriter: ReadWriter{
849-
spanSetReader: spanSetReader{r: s.b, spans: s.spanSetReader.spans.ShallowCopy(), spansOnly: s.spansOnly, ts: s.ts},
850-
spanSetWriter: spanSetWriter{w: s.b, spans: s.spanSetWriter.spans.ShallowCopy(), spansOnly: s.spansOnly, ts: s.ts},
851-
},
852-
b: s.b,
853-
spans: s.spans.ShallowCopy(),
854-
spansOnly: s.spansOnly,
855-
ts: s.ts,
856-
}
843+
// shallowCopy returns a shallow copy of the spanSetBatch. The returned batch
844+
// shares the same underlying storage.Batch but has its own spanSetBatch wrapper
845+
// with a new copy of the SpanSet that uses a shallow copy of the underlying
846+
// spans.
847+
func (s spanSetBatch) shallowCopy() *spanSetBatch {
848+
b := s
849+
b.spanSetReader.spans = b.spanSetReader.spans.ShallowCopy()
850+
b.spanSetWriter.spans = b.spanSetWriter.spans.ShallowCopy()
851+
b.spans = b.spans.ShallowCopy()
852+
return &b
857853
}
858854

859855
// NewBatch returns a storage.Batch that asserts access of the underlying
@@ -913,7 +909,7 @@ func DisableReadWriterAssertions(rw storage.ReadWriter) storage.ReadWriter {
913909
func DisableLatchAssertions(rw storage.ReadWriter) storage.ReadWriter {
914910
switch v := rw.(type) {
915911
case *spanSetBatch:
916-
newSnapSetBatch := v.clone()
912+
newSnapSetBatch := v.shallowCopy()
917913
newSnapSetBatch.spanSetReader.spans.DisableUndeclaredAccessAssertions()
918914
newSnapSetBatch.spanSetWriter.spans.DisableUndeclaredAccessAssertions()
919915
return newSnapSetBatch
@@ -930,7 +926,7 @@ func DisableLatchAssertions(rw storage.ReadWriter) storage.ReadWriter {
930926
func DisableForbiddenSpanAssertionsOnBatch(rw storage.ReadWriter) storage.ReadWriter {
931927
switch v := rw.(type) {
932928
case *spanSetBatch:
933-
newSnapSetBatch := v.clone()
929+
newSnapSetBatch := v.shallowCopy()
934930
newSnapSetBatch.spanSetReader.spans.DisableForbiddenSpansAssertions()
935931
newSnapSetBatch.spanSetWriter.spans.DisableForbiddenSpansAssertions()
936932
return newSnapSetBatch

0 commit comments

Comments
 (0)