Skip to content

Commit

Permalink
kvserver: avoid spanset assertion in MVCCGarbageCollect
Browse files Browse the repository at this point in the history
See
cockroachdb#80619 (comment).

My understanding is that in `MVCCGarbageCollect`, we look at each `key`.
First, we seek the iterator to construct the `MVCCMetadata` (if there is
one), so we're here:

```
key @ ts=100   <-- iter
key @ ts=099
...
key @ ts=007
key @ ts=006
key @ ts=005
```

If we want to GC at timestamp, say, 6, we need to iterate forwards. To
avoid calling `Next()` too many times, eventually we'll instead
`SeekLT(key @ 6)`.

But we only hold the latch for `key`, so we can't seek into `[/Min,
key)`.

Disable the spanset assertions here. We use the result of `SeekLT` only
if it puts us into keyspace covered by the latch. An import cycle
(spanset - storage - spanset) made this require a small refactor.

Release note: None
  • Loading branch information
tbg committed Jun 8, 2022
1 parent 4c4a32e commit 065d18f
Show file tree
Hide file tree
Showing 6 changed files with 57 additions and 26 deletions.
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/batcheval/cmd_add_sstable.go
Original file line number Diff line number Diff line change
Expand Up @@ -338,7 +338,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 := spanset.DisableReaderAssertions(readWriter).NewMVCCIterator(
existingIter := storage.DisableReaderAssertions(readWriter).NewMVCCIterator(
storage.MVCCKeyIterKind, // don't care if it is committed or not, just that it isn't empty.
storage.IterOptions{UpperBound: reply.RangeSpan.EndKey},
)
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/batcheval/cmd_recompute_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -71,7 +71,7 @@ func RecomputeStats(

// Disable the assertions which check that all reads were previously declared.
// See the comment in `declareKeysRecomputeStats` for details on this.
reader = spanset.DisableReaderAssertions(reader)
reader = storage.DisableReaderAssertions(reader)

actualMS, err := rditer.ComputeStatsForRange(desc, reader, cArgs.Header.Timestamp.WallTime)
if err != nil {
Expand Down
46 changes: 23 additions & 23 deletions pkg/kv/kvserver/spanset/batch.go
Original file line number Diff line number Diff line change
Expand Up @@ -719,6 +719,13 @@ func NewReadWriterAt(rw storage.ReadWriter, spans *SpanSet, ts hlc.Timestamp) st
return makeSpanSetReadWriterAt(rw, spans, ts)
}

var _ storage.DisableReaderAssertionsI = (*ReadWriter)(nil)

// DisableReaderAssertions implements storage.DisableReaderAssertionsI.
func (r *ReadWriter) DisableReaderAssertions() (next storage.Reader) {
return r.r
}

type spanSetBatch struct {
ReadWriter
b storage.Batch
Expand All @@ -729,25 +736,31 @@ type spanSetBatch struct {
}

var _ storage.Batch = spanSetBatch{}
var _ storage.DisableReaderAssertionsI = (*spanSetBatch)(nil)

// DisableReaderAssertions implements storage.DisableReaderAssertionsI.
func (b *spanSetBatch) DisableReaderAssertions() (next storage.Reader) {
return b.r
}

func (s spanSetBatch) Commit(sync bool) error {
return s.b.Commit(sync)
func (b spanSetBatch) Commit(sync bool) error {
return b.b.Commit(sync)
}

func (s spanSetBatch) Empty() bool {
return s.b.Empty()
func (b spanSetBatch) Empty() bool {
return b.b.Empty()
}

func (s spanSetBatch) Count() uint32 {
return s.b.Count()
func (b spanSetBatch) Count() uint32 {
return b.b.Count()
}

func (s spanSetBatch) Len() int {
return s.b.Len()
func (b spanSetBatch) Len() int {
return b.b.Len()
}

func (s spanSetBatch) Repr() []byte {
return s.b.Repr()
func (b spanSetBatch) Repr() []byte {
return b.b.Repr()
}

// NewBatch returns a storage.Batch that asserts access of the underlying
Expand All @@ -774,19 +787,6 @@ func NewBatchAt(b storage.Batch, spans *SpanSet, ts hlc.Timestamp) storage.Batch
}
}

// DisableReaderAssertions unwraps any storage.Reader implementations that may
// assert access against a given SpanSet.
func DisableReaderAssertions(reader storage.Reader) storage.Reader {
switch v := reader.(type) {
case ReadWriter:
return DisableReaderAssertions(v.r)
case *spanSetBatch:
return DisableReaderAssertions(v.r)
default:
return reader
}
}

// 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
Expand Down
1 change: 1 addition & 0 deletions pkg/storage/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ go_library(
"pebble_merge.go",
"pebble_mvcc_scanner.go",
"read_as_of_iterator.go",
"reader_assertions.go",
"replicas_storage.go",
"resource_limiter.go",
"row_counter.go",
Expand Down
4 changes: 3 additions & 1 deletion pkg/storage/mvcc.go
Original file line number Diff line number Diff line change
Expand Up @@ -3546,7 +3546,9 @@ func MVCCGarbageCollect(

// Bound the iterator appropriately for the set of keys we'll be garbage
// collecting.
iter := rw.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{
// Disable spanset assertions as we intentionally (but safely) call SeekLT(key)
// where we only own a point latch for `key`.
iter := DisableReaderAssertions(rw).NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{
LowerBound: keys[0].Key,
UpperBound: keys[len(keys)-1].Key.Next(),
})
Expand Down
28 changes: 28 additions & 0 deletions pkg/storage/reader_assertions.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,28 @@
// Copyright 2022 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
//

package storage

// DisableReaderAssertionsI is optionally implemented by arguments to DisableReaderAssertions.
type DisableReaderAssertionsI interface {
DisableReaderAssertions() (wrapped Reader)
}

// DisableReaderAssertions unwraps any storage.Reader implementations that may
// assert access against a given SpanSet.
func DisableReaderAssertions(reader Reader) Reader {
switch v := reader.(type) {
case DisableReaderAssertionsI:
return DisableReaderAssertions(v.DisableReaderAssertions())
default:
return reader
}
}

0 comments on commit 065d18f

Please sign in to comment.