Skip to content

Commit

Permalink
keyspan: add Assert iterator
Browse files Browse the repository at this point in the history
This iterator is used in `invariants` mode to verify that the results
of the iterator operations are sane, and to verify that the ranges
inside sstables (physical or virtual) conform to the smallest/largest
keys in the metadata.

We use this iterator to wrap the RangeDel and RangeKey iterators
returned by sstable readers.
  • Loading branch information
RaduBerinde committed Dec 19, 2023
1 parent bbf7dc4 commit 5c5ad7e
Show file tree
Hide file tree
Showing 9 changed files with 318 additions and 4 deletions.
173 changes: 173 additions & 0 deletions internal/keyspan/assert_iter.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,173 @@
// Copyright 2023 The LevelDB-Go and Pebble Authors. All rights reserved. Use
// of this source code is governed by a BSD-style license that can be found in
// the LICENSE file.

package keyspan

import (
"fmt"

"github.com/cockroachdb/errors"
"github.com/cockroachdb/pebble/internal/base"
"github.com/cockroachdb/pebble/internal/invariants"
)

// Assert wraps an iterator and asserts that operations return sane results.
func Assert(iter FragmentIterator, cmp base.Compare) FragmentIterator {
return &assertIter{
iter: iter,
cmp: cmp,
}
}

// MaybeAssert wraps an iterator and asserts that operations return sane
// results if we are in testing mode.
func MaybeAssert(iter FragmentIterator, cmp base.Compare) FragmentIterator {
if invariants.Enabled && iter != nil {
// Don't wrap an assertIter.
if _, ok := iter.(*assertIter); !ok {
return Assert(iter, cmp)
}
}
return iter
}

// AssertUserKeyBounds wraps an iterator and asserts that all spans are within
// the given bounds [lower, upper).
func AssertUserKeyBounds(
iter FragmentIterator, lower, upper []byte, cmp base.Compare,
) FragmentIterator {
return AssertBounds(iter, base.MakeSearchKey(lower), upper, cmp)
}

// AssertBounds wraps an iterator and asserts that all spans are within the
// given bounds [lower.UserKey, upper), and that all keys in a span that starts
// exactly at lower.UserKey are >= lower.
//
// The asymmetry here is due to fragment spans having exclusive end user keys.
func AssertBounds(
iter FragmentIterator, lower base.InternalKey, upper []byte, cmp base.Compare,
) FragmentIterator {
i := &assertIter{
iter: iter,
cmp: cmp,
}
i.checkBounds.enabled = true
i.checkBounds.lower = lower
i.checkBounds.upper = upper
return i
}

// assertIter is a pass-through FragmentIterator wrapper which performs checks
// on what the wrapped iterator returns.
//
// It verifies that results for various operations are sane, and it optionally
// verifies that spans are within given bounds.
type assertIter struct {
iter FragmentIterator
cmp base.Compare
checkBounds struct {
enabled bool
lower base.InternalKey
upper []byte
}
lastSpanStart []byte
lastSpanEnd []byte
}

var _ FragmentIterator = (*assertIter)(nil)

func (i *assertIter) panicf(format string, args ...interface{}) {
str := fmt.Sprintf(format, args...)
panic(errors.AssertionFailedf("%s; wraps %T", str, i.iter))
}

func (i *assertIter) check(span *Span) {
i.lastSpanStart = i.lastSpanStart[:0]
i.lastSpanEnd = i.lastSpanEnd[:0]
if span == nil {
return
}
if i.checkBounds.enabled {
lower := i.checkBounds.lower
switch startCmp := i.cmp(span.Start, lower.UserKey); {
case startCmp < 0:
i.panicf("lower bound %q violated by span %s", lower.UserKey, span)
case startCmp == 0:
// Note: trailers are in descending order.
if len(span.Keys) > 0 && span.SmallestKey().Trailer > lower.Trailer {
i.panicf("lower bound %s violated by key %s", lower, span.SmallestKey())
}
}
if i.cmp(span.End, i.checkBounds.upper) > 0 {
i.panicf("upper bound %q violated by span %s", i.checkBounds.upper, span)
}
}
// Save the span to check Next/Prev operations.
i.lastSpanStart = append(i.lastSpanStart, span.Start...)
i.lastSpanEnd = append(i.lastSpanEnd, span.End...)
}

// SeekGE implements FragmentIterator.
func (i *assertIter) SeekGE(key []byte) *Span {
span := i.iter.SeekGE(key)
if span != nil && i.cmp(span.End, key) <= 0 {
i.panicf("incorrect SeekGE(%q) span %s", key, span)
}
i.check(span)
return span
}

// SeekLT implements FragmentIterator.
func (i *assertIter) SeekLT(key []byte) *Span {
span := i.iter.SeekLT(key)
if span != nil && i.cmp(span.Start, key) >= 0 {
i.panicf("incorrect SeekLT(%q) span %s", key, span)
}
i.check(span)
return span
}

// First implements FragmentIterator.
func (i *assertIter) First() *Span {
span := i.iter.First()
i.check(span)
return span
}

// Last implements FragmentIterator.
func (i *assertIter) Last() *Span {
span := i.iter.Last()
i.check(span)
return span
}

// Next implements FragmentIterator.
func (i *assertIter) Next() *Span {
span := i.iter.Next()
if span != nil && len(i.lastSpanEnd) > 0 && i.cmp(i.lastSpanEnd, span.Start) > 0 {
i.panicf("Next span %s not after last span end %q", span, i.lastSpanEnd)
}
i.check(span)
return span
}

// Prev implements FragmentIterator.
func (i *assertIter) Prev() *Span {
span := i.iter.Prev()
if span != nil && len(i.lastSpanStart) > 0 && i.cmp(i.lastSpanStart, span.End) < 0 {
i.panicf("Prev span %s not before last span start %q", span, i.lastSpanStart)
}
i.check(span)
return span
}

// Error implements FragmentIterator.
func (i *assertIter) Error() error {
return i.iter.Error()
}

// Close implements FragmentIterator.
func (i *assertIter) Close() error {
return i.iter.Close()
}
61 changes: 61 additions & 0 deletions internal/keyspan/assert_iter_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,61 @@
// Copyright 2023 The LevelDB-Go and Pebble Authors. All rights reserved. Use
// of this source code is governed by a BSD-style license that can be found in
// the LICENSE file.

package keyspan

import (
"fmt"
"strings"
"testing"

"github.com/cockroachdb/datadriven"
"github.com/cockroachdb/pebble/internal/base"
"github.com/cockroachdb/pebble/internal/testkeys"
"github.com/stretchr/testify/require"
)

func TestAssertBoundsIter(t *testing.T) {
cmp := testkeys.Comparer.Compare
var spans []Span
datadriven.RunTest(t, "testdata/assert_iter", func(t *testing.T, td *datadriven.TestData) string {
switch cmd := td.Cmd; cmd {
case "define":
spans = spans[:0]
lines := strings.Split(strings.TrimSpace(td.Input), "\n")
for _, line := range lines {
spans = append(spans, ParseSpan(line))
}
return ""

case "assert-bounds", "assert-userkey-bounds":
lines := strings.Split(td.Input, "\n")
require.Equal(t, 2, len(lines))
upper := []byte(lines[1])
innerIter := NewIter(cmp, spans)
var iter FragmentIterator
if cmd == "assert-bounds" {
lower := base.ParseInternalKey(lines[0])
iter = AssertBounds(innerIter, lower, upper, cmp)
} else {
lower := []byte(lines[0])
iter = AssertUserKeyBounds(innerIter, lower, upper, cmp)
}
defer iter.Close()

return func() (res string) {
defer func() {
if r := recover(); r != nil {
res = fmt.Sprintf("%v", r)
}
}()
for span := iter.First(); span != nil; span = iter.Next() {
}
return "OK"
}()

default:
return fmt.Sprintf("unknown command: %s", cmd)
}
})
}
2 changes: 1 addition & 1 deletion internal/keyspan/filter.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@ var _ FragmentIterator = (*filteringIter)(nil)
// Filter returns a new filteringIter that will filter the Spans from the
// provided child iterator using the provided FilterFunc.
func Filter(iter FragmentIterator, filter FilterFunc, cmp base.Compare) FragmentIterator {
return &filteringIter{iter: iter, filterFn: filter, cmp: cmp}
return MaybeAssert(&filteringIter{iter: iter, filterFn: filter, cmp: cmp}, cmp)
}

// SeekGE implements FragmentIterator.
Expand Down
2 changes: 1 addition & 1 deletion internal/keyspan/interleaving_iter.go
Original file line number Diff line number Diff line change
Expand Up @@ -211,7 +211,7 @@ func (i *InterleavingIter) Init(
cmp: comparer.Compare,
comparer: comparer,
pointIter: pointIter,
keyspanIter: keyspanIter,
keyspanIter: MaybeAssert(keyspanIter, comparer.Compare),
mask: opts.Mask,
lower: opts.LowerBound,
upper: opts.UpperBound,
Expand Down
1 change: 1 addition & 0 deletions internal/keyspan/level_iter.go
Original file line number Diff line number Diff line change
Expand Up @@ -180,6 +180,7 @@ func (l *LevelIter) loadFile(file *manifest.FileMetadata, dir int) loadFileRetur
}
if indicator != fileAlreadyLoaded {
l.iter, l.err = l.newIter(file, l.tableOpts)
l.iter = MaybeAssert(l.iter, l.cmp)
indicator = newFileLoaded
}
if l.err != nil {
Expand Down
47 changes: 47 additions & 0 deletions internal/keyspan/testdata/assert_iter
Original file line number Diff line number Diff line change
@@ -0,0 +1,47 @@
define
a-c:{(#3,RANGEKEYUNSET,@5) (#2,RANGEKEYSET,@5,apples) (#1,RANGEKEYSET,@3,bananas)}
c-d:{(#4,RANGEKEYSET,@3,bananas) (#3,RANGEKEYDEL)}
d-e:{(#4,RANGEKEYSET,@3,bananas) (#4,RANGEKEYSET,@1,pineapple)}
----

assert-userkey-bounds
a
z
----
OK

assert-userkey-bounds
b
z
----
lower bound "b" violated by span a-c:{(#3,RANGEKEYUNSET,@5) (#2,RANGEKEYSET,@5,apples) (#1,RANGEKEYSET,@3,bananas)}; wraps *keyspan.Iter

assert-bounds
a.SET.1
z
----
lower bound a#1,1 violated by key a#3,20; wraps *keyspan.Iter

assert-bounds
a.SET.5
z
----
OK

assert-bounds
b.SET.1
z
----
lower bound "b" violated by span a-c:{(#3,RANGEKEYUNSET,@5) (#2,RANGEKEYSET,@5,apples) (#1,RANGEKEYSET,@3,bananas)}; wraps *keyspan.Iter

assert-userkey-bounds
a
d
----
upper bound "d" violated by span d-e:{(#4,RANGEKEYSET,@3,bananas) (#4,RANGEKEYSET,@1,pineapple)}; wraps *keyspan.Iter

assert-userkey-bounds
a
e
----
OK
13 changes: 13 additions & 0 deletions table_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -496,6 +496,19 @@ func (c *tableCacheShard) newIters(
return nil, nil, err
}

// Assert expected bounds in tests.
if invariants.Enabled && rangeDelIter != nil {
cmp := base.DefaultComparer.Compare
if dbOpts.opts.Comparer != nil {
cmp = dbOpts.opts.Comparer.Compare
}
// TODO(radu): we should be using AssertBounds, but it currently fails in
// some cases (#3167).
rangeDelIter = keyspan.AssertUserKeyBounds(
rangeDelIter, file.SmallestPointKey.UserKey, file.LargestPointKey.UserKey, cmp,
)
}

if !ok {
c.unrefValue(v)
// Return an empty iterator. This iterator has no mutable state, so
Expand Down
6 changes: 4 additions & 2 deletions table_cache_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -261,7 +261,6 @@ func TestVirtualReadsWiring(t *testing.T) {
DisableAutomaticCompactions: true,
})
require.NoError(t, err)
defer d.Close()

b := newBatch(d)
// Some combination of sets, range deletes, and range key sets/unsets, so
Expand Down Expand Up @@ -311,7 +310,7 @@ func TestVirtualReadsWiring(t *testing.T) {
Size: parentFile.Size / 2,
SmallestSeqNum: parentFile.SmallestSeqNum,
LargestSeqNum: parentFile.LargestSeqNum,
Smallest: base.MakeInternalKey([]byte{'d'}, parentFile.Smallest.SeqNum()+1, InternalKeyKindSet),
Smallest: base.MakeInternalKey([]byte{'d'}, parentFile.Smallest.SeqNum()+2, InternalKeyKindRangeDelete),
Largest: base.MakeInternalKey([]byte{'z'}, parentFile.Largest.SeqNum(), InternalKeyKindSet),
HasPointKeys: true,
Virtual: true,
Expand Down Expand Up @@ -387,6 +386,9 @@ func TestVirtualReadsWiring(t *testing.T) {
require.Equal(t, []byte{expected[i]}, iter.Value())
}
iter.Close()

// We don't defer this Close in case we get a panic while holding d.mu.
d.Close()
}

// The table cache shouldn't be usable after all the dbs close.
Expand Down
17 changes: 17 additions & 0 deletions table_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,7 @@ import (

"github.com/cockroachdb/errors"
"github.com/cockroachdb/pebble/internal/base"
"github.com/cockroachdb/pebble/internal/invariants"
"github.com/cockroachdb/pebble/internal/keyspan"
"github.com/cockroachdb/pebble/internal/manifest"
"github.com/cockroachdb/pebble/sstable"
Expand Down Expand Up @@ -905,6 +906,14 @@ func newCombinedDeletionKeyspanIter(
return nil, err
}
if iter != nil {
// Assert expected bounds in tests.
if invariants.Enabled {
// TODO(radu): we should be using AssertBounds, but it currently fails in
// some cases (#3167).
iter = keyspan.AssertUserKeyBounds(
iter, m.SmallestPointKey.UserKey, m.LargestPointKey.UserKey, comparer.Compare,
)
}
dIter := &keyspan.DefragmentingIter{}
dIter.Init(comparer, iter, equal, reducer, new(keyspan.DefragmentingBuffers))
iter = dIter
Expand All @@ -922,6 +931,14 @@ func newCombinedDeletionKeyspanIter(
return nil, err
}
if iter != nil {
// Assert expected bounds in tests.
if invariants.Enabled {
// TODO(radu): we should be using AssertBounds, but it currently fails in
// some cases (#3167).
iter = keyspan.AssertUserKeyBounds(
iter, m.SmallestRangeKey.UserKey, m.LargestRangeKey.UserKey, comparer.Compare,
)
}
// Wrap the range key iterator in a filter that elides keys other than range
// key deletions.
iter = keyspan.Filter(iter, func(in *keyspan.Span, out *keyspan.Span) (keep bool) {
Expand Down

0 comments on commit 5c5ad7e

Please sign in to comment.