Skip to content

Commit

Permalink
rowblk: add IndexIter type
Browse files Browse the repository at this point in the history
Add an IndexIter type that abstracts some of the details of a rowblk index
block away from the sstable iterator. Future work will define a constraint and
make single-level and two-level iterators generic with respect to the type of
the index block, allowing the addition of a colblk implementation.
  • Loading branch information
jbowens committed Aug 20, 2024
1 parent aa32f0b commit de53a98
Show file tree
Hide file tree
Showing 7 changed files with 278 additions and 123 deletions.
61 changes: 61 additions & 0 deletions sstable/block/block.go
Original file line number Diff line number Diff line change
Expand Up @@ -183,6 +183,67 @@ type DataBlockIterator[D any] interface {
*D // non-interface type constraint element
}

// IndexBlockIterator is a type constraint for implementations of block
// iterators over index blocks. It's currently satisifed by the
// *rowblk.IndexIter type.
//
// IndexBlockIterator requires that the type be a pointer to its type parameter,
// I, to allow sstable iterators embed the block iterator within its struct. See
// this example from the Go generics proposal:
// https://go.googlesource.com/proposal/+/refs/heads/master/design/43651-type-parameters.md#pointer-method-example
type IndexBlockIterator[I any] interface {
// InitHandle initializes an iterator from the provided block handle.
InitHandle(base.Compare, base.Split, BufferHandle, IterTransforms) error
// ResetForReuse resets the index iterator for reuse, retaining buffers to
// avoid future allocations.
ResetForReuse() I
// Valid returns true if the iterator is currently positioned at a valid
// block handle.
Valid() bool
// IsDataInvalidated returns true when the iterator has been invalidated
// using an Invalidate call.
//
// NB: this is different from Valid which indicates whether the iterator is
// currently positioned over a valid block entry.
IsDataInvalidated() bool
// Invalidate invalidates the block iterator, removing references to the
// block it was initialized with.
Invalidate()
// Handle returns the underlying block buffer handle, if the iterator was
// initialized with one.
Handle() BufferHandle
// Separator returns the separator at the iterator's current position. The
// iterator must be positioned at a valid row. A Separator is a user key
// guaranteed to be greater than or equal to every key contained within the
// referenced block(s).
Separator() []byte
// BlockHandleWithProperties decodes the block handle with any encoded
// properties at the iterator's current position.
BlockHandleWithProperties() (HandleWithProperties, error)
// SeekGE seeks the index iterator to the first block entry with a separator
// key greater or equal to the given key. If it returns true, the iterator
// is positioned over the first block that might contain the key [key], and
// following blocks have keys ≥ Separator(). It returns false if the seek
// key is greater than all index block separators.
SeekGE(key []byte) bool
// First seeks index iterator to the first block entry. It returns false if
// the index block is empty.
First() bool
// Last seeks index iterator to the last block entry. It returns false if
// the index block is empty.
Last() bool
// Next steps the index iterator to the next block entry. It returns false
// if the index block is exhausted.
Next() bool
// Prev steps the index iterator to the previous block entry. It returns
// false if the index block is exhausted.
Prev() bool
// Close closes the iterator, releasing any resources it holds.
Close() error

*I // non-interface type constraint element
}

// IterTransforms allow on-the-fly transformation of data at iteration time.
//
// These transformations could in principle be implemented as block transforms
Expand Down
19 changes: 8 additions & 11 deletions sstable/data_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@ import (
"github.com/cockroachdb/pebble/internal/testkeys"
"github.com/cockroachdb/pebble/objstorage"
"github.com/cockroachdb/pebble/objstorage/objstorageprovider"
"github.com/cockroachdb/pebble/sstable/block"
"github.com/cockroachdb/pebble/sstable/rowblk"
"github.com/cockroachdb/pebble/vfs"
)
Expand Down Expand Up @@ -433,13 +432,12 @@ func runIterCmd(
if twoLevelIter, ok := origIter.(*twoLevelIterator[rowblk.Iter, *rowblk.Iter]); ok {
si = &twoLevelIter.secondLevel
if twoLevelIter.topLevelIndex.Valid() {
fmt.Fprintf(&b, "| topLevelIndex.Key() = %q\n", twoLevelIter.topLevelIndex.Key())
v := twoLevelIter.topLevelIndex.Value()
bhp, err := block.DecodeHandleWithProperties(v.InPlaceValue())
fmt.Fprintf(&b, "| topLevelIndex.Key() = %q\n", twoLevelIter.topLevelIndex.Separator())
bhp, err := twoLevelIter.topLevelIndex.BlockHandleWithProperties()
if err != nil {
fmt.Fprintf(&b, "| topLevelIndex.InPlaceValue() failed to decode as BHP: %s\n", err)
fmt.Fprintf(&b, "| topLevelIndex entry failed to decode as BHP: %s\n", err)
} else {
fmt.Fprintf(&b, "| topLevelIndex.InPlaceValue() = (Offset: %d, Length: %d, Props: %x)\n",
fmt.Fprintf(&b, "| topLevelIndex.BlockHandleWithProperties() = (Offset: %d, Length: %d, Props: %x)\n",
bhp.Offset, bhp.Length, bhp.Props)
}
} else {
Expand All @@ -448,13 +446,12 @@ func runIterCmd(
fmt.Fprintf(&b, "| topLevelIndex.isDataInvalidated()=%t\n", twoLevelIter.topLevelIndex.IsDataInvalidated())
}
if si.index.Valid() {
fmt.Fprintf(&b, "| index.Key() = %q\n", si.index.Key())
v := si.index.Value()
bhp, err := block.DecodeHandleWithProperties(v.InPlaceValue())
fmt.Fprintf(&b, "| index.Separator() = %q\n", si.index.Separator())
bhp, err := si.index.BlockHandleWithProperties()
if err != nil {
fmt.Fprintf(&b, "| index.InPlaceValue() failed to decode as BHP: %s\n", err)
fmt.Fprintf(&b, "| index entry failed to decode as BHP: %s\n", err)
} else {
fmt.Fprintf(&b, "| index.InPlaceValue() = (Offset: %d, Length: %d, Props: %x)\n",
fmt.Fprintf(&b, "| index.BlockHandleWithProperties() = (Offset: %d, Length: %d, Props: %x)\n",
bhp.Offset, bhp.Length, bhp.Props)
}
} else {
Expand Down
14 changes: 7 additions & 7 deletions sstable/reader_iter.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,10 +52,10 @@ type Iterator interface {
// The data-exhausted property is tracked in a more subtle manner. We define
// two predicates:
// - partial-local-data-exhausted (PLDE):
// i.data.isDataInvalidated() || !i.data.valid()
// i.data.IsDataInvalidated() || !i.data.Valid()
// - partial-global-data-exhausted (PGDE):
// i.index.isDataInvalidated() || !i.index.valid() || i.data.isDataInvalidated() ||
// !i.data.valid()
// i.index.IsDataInvalidated() || !i.index.Valid() || i.data.IsDataInvalidated() ||
// !i.data.Valid()
//
// PLDE is defined for a singleLevelIterator. PGDE is defined for a
// twoLevelIterator. Oddly, in our code below the singleLevelIterator does not
Expand Down Expand Up @@ -96,13 +96,13 @@ type Iterator interface {
// state.
//
// Implementation detail: In the code PLDE only checks that
// i.data.isDataInvalidated(). This narrower check is safe, since this is a
// i.data.IsDataInvalidated(). This narrower check is safe, since this is a
// subset of the set expressed by the OR expression. Also, it is not a
// de-optimization since whenever we exhaust the iterator we explicitly call
// i.data.invalidate(). PGDE checks i.index.isDataInvalidated() &&
// i.data.isDataInvalidated(). Again, this narrower check is safe, and not a
// i.data.Invalidate(). PGDE checks i.index.IsDataInvalidated() &&
// i.data.IsDataInvalidated(). Again, this narrower check is safe, and not a
// de-optimization since whenever we exhaust the iterator we explicitly call
// i.index.invalidate() and i.data.invalidate(). The && is questionable -- for
// i.index.Invalidate() and i.data.Invalidate(). The && is questionable -- for
// now this is a bit of defensive code. We should seriously consider removing
// it, since defensive code suggests we are not confident about our invariants
// (and if we are not confident, we need more invariant assertions, not
Expand Down
Loading

0 comments on commit de53a98

Please sign in to comment.