Skip to content

Commit

Permalink
sstable: genericize sstable iterators by data block iterator
Browse files Browse the repository at this point in the history
In preparation for columnar blocks, this commit introduces type parameters to
the {single,two}LevelIterator types. Parameterizing the sstable iterator
ensures we pay the cost of dynamic dispatch only when entering the sstable
iterator (as we do today).

In order to continue to allocate the data block iterator struct within the
single-level iterator, this commit introduces two type paramters: one for the
non-pointer iterator type, and one for the pointer type on which iteration
methods are defined. This adds some slightly cumbersome ergonomics to data
block iterator usage sites.
  • Loading branch information
jbowens committed Aug 12, 2024
1 parent 57ff76d commit 3ff865b
Show file tree
Hide file tree
Showing 10 changed files with 318 additions and 198 deletions.
39 changes: 39 additions & 0 deletions sstable/block/block.go
Original file line number Diff line number Diff line change
Expand Up @@ -89,6 +89,45 @@ func (c *Checksummer) Checksum(block []byte, blockType []byte) (checksum uint32)
return checksum
}

// DataBlockIterator is a type constraint for implementations of block iterators
// over data blocks. It's currently satisifed by the *rowblk.Iter type.
//
// DataBlockIterator requires that the type be a pointer to its type parameter,
// D, 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 DataBlockIterator[D any] interface {
base.InternalIterator

// Handle returns the handle to the block.
Handle() BufferHandle
// InitHandle initializes the block from the provided buffer handle.
InitHandle(base.Compare, base.Split, BufferHandle, IterTransforms) error
// Valid returns true if the iterator is currently positioned at a valid KV.
Valid() bool
// KV returns the key-value pair at the current iterator position. The
// iterator must be Valid().
KV() *base.InternalKV
// ResetForReuse resets the iterator so that it may be used for iteration
// over a new block. It returns the non-pointer D type to allow resetting
// while initializing the containing struct, eg::
// iter = sstableIter{dataBlockIter: iter.dataBlockIter.ResetForReuse()}
ResetForReuse() D
// FirstUserKey returns the first user key contained within the data block.
FirstUserKey() []byte
// Invalidate invalidates the block iterator, removing references to the block
// it was initialized with.
Invalidate()
// IsDataInvalidated returns true when the iterator has been invalidated
// using an Invalidate call.
//
// NB: this is different from Valid which indicates whether the current *KV*
// is valid.
IsDataInvalidated() bool

*D // 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
5 changes: 3 additions & 2 deletions sstable/data_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (
"github.com/cockroachdb/pebble/internal/testkeys"
"github.com/cockroachdb/pebble/objstorage"
"github.com/cockroachdb/pebble/objstorage/objstorageprovider"
"github.com/cockroachdb/pebble/sstable/rowblk"
"github.com/cockroachdb/pebble/vfs"
)

Expand Down Expand Up @@ -427,8 +428,8 @@ func runIterCmd(
continue
case "internal-iter-state":
fmt.Fprintf(&b, "| %T:\n", origIter)
si, _ := origIter.(*singleLevelIterator)
if twoLevelIter, ok := origIter.(*twoLevelIterator); ok {
si, _ := origIter.(*singleLevelIterator[rowblk.Iter, *rowblk.Iter])
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())
Expand Down
9 changes: 5 additions & 4 deletions sstable/reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -207,10 +207,11 @@ func (r *Reader) newPointIter(
var res Iterator
var err error
if r.Properties.IndexType == twoLevelIndex {
res, err = newTwoLevelIterator(ctx, r, vState, transforms, lower, upper, filterer, filterBlockSizeLimit,
res, err = newRowBlockTwoLevelIterator(
ctx, r, vState, transforms, lower, upper, filterer, filterBlockSizeLimit,
stats, categoryAndQoS, statsCollector, rp, nil /* bufferPool */)
} else {
res, err = newSingleLevelIterator(
res, err = newRowBlockSingleLevelIterator(
ctx, r, vState, transforms, lower, upper, filterer, filterBlockSizeLimit,
stats, categoryAndQoS, statsCollector, rp, nil /* bufferPool */)
}
Expand Down Expand Up @@ -261,7 +262,7 @@ func (r *Reader) newCompactionIter(
transforms.HideObsoletePoints = true
}
if r.Properties.IndexType == twoLevelIndex {
i, err := newTwoLevelIterator(
i, err := newRowBlockTwoLevelIterator(
context.Background(),
r, vState, transforms, nil /* lower */, nil /* upper */, nil,
NeverUseFilterBlock, nil /* stats */, categoryAndQoS, statsCollector, rp, bufferPool,
Expand All @@ -272,7 +273,7 @@ func (r *Reader) newCompactionIter(
i.SetupForCompaction()
return i, nil
}
i, err := newSingleLevelIterator(
i, err := newRowBlockSingleLevelIterator(
context.Background(), r, vState, transforms, nil /* lower */, nil, /* upper */
nil, NeverUseFilterBlock, nil /* stats */, categoryAndQoS, statsCollector, rp, bufferPool,
)
Expand Down
50 changes: 29 additions & 21 deletions sstable/reader_iter.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,8 @@ import (

"github.com/cockroachdb/pebble/internal/base"
"github.com/cockroachdb/pebble/internal/invariants"
"github.com/cockroachdb/pebble/sstable/block"
"github.com/cockroachdb/pebble/sstable/rowblk"
)

// Iterator iterates over an entire table of data.
Expand Down Expand Up @@ -108,27 +110,33 @@ type Iterator interface {
//
// TODO(sumeer): remove the aforementioned defensive code.

var singleLevelIterPool = sync.Pool{
New: func() interface{} {
i := &singleLevelIterator{}
// Note: this is a no-op if invariants are disabled or race is enabled.
invariants.SetFinalizer(i, checkSingleLevelIterator)
return i
},
}
var (
singleLevelIterRowBlockPool sync.Pool // *singleLevelIterator[rowblk.Iter, *rowblk.Iter]
twoLevelIterRowBlockPool sync.Pool // *twoLevelIterator[rowblk.Iter, *rowblk.Iter]
)

var twoLevelIterPool = sync.Pool{
New: func() interface{} {
i := &twoLevelIterator{}
// Note: this is a no-op if invariants are disabled or race is enabled.
invariants.SetFinalizer(i, checkTwoLevelIterator)
return i
},
func init() {
singleLevelIterRowBlockPool = sync.Pool{
New: func() interface{} {
i := &singleLevelIterator[rowblk.Iter, *rowblk.Iter]{pool: &singleLevelIterRowBlockPool}
// Note: this is a no-op if invariants are disabled or race is enabled.
invariants.SetFinalizer(i, checkSingleLevelIterator[rowblk.Iter, *rowblk.Iter])
return i
},
}
twoLevelIterRowBlockPool = sync.Pool{
New: func() interface{} {
i := &twoLevelIterator[rowblk.Iter, *rowblk.Iter]{pool: &twoLevelIterRowBlockPool}
// Note: this is a no-op if invariants are disabled or race is enabled.
invariants.SetFinalizer(i, checkTwoLevelIterator[rowblk.Iter, *rowblk.Iter])
return i
},
}
}

func checkSingleLevelIterator(obj interface{}) {
i := obj.(*singleLevelIterator)
if p := i.data.Handle().Get(); p != nil {
func checkSingleLevelIterator[D any, PD block.DataBlockIterator[D]](obj interface{}) {
i := obj.(*singleLevelIterator[D, PD])
if p := PD(&i.data).Handle().Get(); p != nil {
fmt.Fprintf(os.Stderr, "singleLevelIterator.data.handle is not nil: %p\n", p)
os.Exit(1)
}
Expand All @@ -138,9 +146,9 @@ func checkSingleLevelIterator(obj interface{}) {
}
}

func checkTwoLevelIterator(obj interface{}) {
i := obj.(*twoLevelIterator)
if p := i.secondLevel.data.Handle().Get(); p != nil {
func checkTwoLevelIterator[D any, PD block.DataBlockIterator[D]](obj interface{}) {
i := obj.(*twoLevelIterator[D, PD])
if p := PD(&i.secondLevel.data).Handle().Get(); p != nil {
fmt.Fprintf(os.Stderr, "singleLevelIterator.data.handle is not nil: %p\n", p)
os.Exit(1)
}
Expand Down
Loading

0 comments on commit 3ff865b

Please sign in to comment.