Skip to content

Commit

Permalink
sstable: clean up UpdateKeySuffixes
Browse files Browse the repository at this point in the history
This change cleans up the design of `SuffixReplaceableBlockCollector`:
instead of having a separate optional interface, we fold this
functionality in the `BlockPropertyCollector` and
`DataBlockIntervalCollector` interfaces. This makes things much easier
to follow.

Note that the old design was especially problematic for
`DataBlockIntervalCollector`: we were passing the encoded properties
to the implementation but the encoding of intervals into properties is
implemented by Pebble. This works only because the Cockroach
implementation (`pebbleDataBlockMVCCTimeIntervalCollector`) does not
need to look at the old properties at all.
  • Loading branch information
RaduBerinde committed Apr 9, 2024
1 parent 278b6a6 commit 6cdb88d
Show file tree
Hide file tree
Showing 7 changed files with 178 additions and 71 deletions.
22 changes: 22 additions & 0 deletions iterator_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -485,6 +485,16 @@ func (c *minSeqNumPropertyCollector) FinishTable(buf []byte) ([]byte, error) {
return binary.AppendUvarint(buf, c.minSeqNum), nil
}

func (c *minSeqNumPropertyCollector) AddCollectedWithSuffixReplacement(
oldProp []byte, oldSuffix, newSuffix []byte,
) error {
return errors.Errorf("not implemented")
}

func (c *minSeqNumPropertyCollector) SupportsSuffixReplacement() bool {
return false
}

// minSeqNumFilter is a BlockPropertyFilter that uses the
// minSeqNumPropertyCollector data to filter out entire tables.
type minSeqNumFilter struct {
Expand Down Expand Up @@ -1121,6 +1131,8 @@ type testBlockIntervalCollector struct {
lower, upper uint64
}

var _ sstable.DataBlockIntervalCollector = (*testBlockIntervalCollector)(nil)

func (bi *testBlockIntervalCollector) Add(key InternalKey, value []byte) error {
k := key.UserKey
if len(k) < bi.numLength+bi.offsetFromEnd {
Expand Down Expand Up @@ -1156,6 +1168,16 @@ func (bi *testBlockIntervalCollector) FinishDataBlock() (lower uint64, upper uin
return l, u, nil
}

func (bi *testBlockIntervalCollector) AddCollectedWithSuffixReplacement(
oldLower, oldUpper uint64, oldSuffix, newSuffix []byte,
) error {
return errors.Errorf("not implemented")
}

func (bi *testBlockIntervalCollector) SupportsSuffixReplacement() bool {
return false
}

func TestIteratorBlockIntervalFilter(t *testing.T) {
var mem vfs.FS
var d *DB
Expand Down
114 changes: 69 additions & 45 deletions sstable/block_property.go
Original file line number Diff line number Diff line change
Expand Up @@ -98,51 +98,55 @@ import (
type BlockPropertyCollector interface {
// Name returns the name of the block property collector.
Name() string

// Add is called with each new entry added to a data block in the sstable.
// The callee can assume that these are in sorted order.
Add(key InternalKey, value []byte) error

// AddCollectedWithSuffixReplacement adds previously collected property data
// and updates it to reflect a change of suffix on all keys: the old property
// data is assumed to be constructed from keys that all have the same
// oldSuffix and is recalculated to reflect the same keys but with newSuffix.
//
// A collector which supports this method must be able to derive its updated
// value from its old value and the change being made to the suffix, without
// needing to be passed each updated K/V.
//
// For example, a collector that only inspects values can simply copy its
// previously computed property as-is, since key-suffix replacement does not
// change values, while a collector that depends only on key suffixes, like
// one which collected mvcc-timestamp bounds from timestamp-suffixed keys, can
// just set its new bounds from the new suffix, as it is common to all keys,
// without needing to recompute it from every key.
//
// This method is optional (if it is not implemented, it always returns an
// error). SupportsSuffixReplacement() can be used to check if this method is
// implemented.
AddCollectedWithSuffixReplacement(oldProp []byte, oldSuffix, newSuffix []byte) error

// SupportsSuffixReplacement returns whether the collector supports the
// AddCollectedWithSuffixReplacement method.
SupportsSuffixReplacement() bool

// FinishDataBlock is called when all the entries have been added to a
// data block. Subsequent Add calls will be for the next data block. It
// returns the property value for the finished block.
FinishDataBlock(buf []byte) ([]byte, error)

// AddPrevDataBlockToIndexBlock adds the entry corresponding to the
// previous FinishDataBlock to the current index block.
AddPrevDataBlockToIndexBlock()

// FinishIndexBlock is called when an index block, containing all the
// key-value pairs since the last FinishIndexBlock, will no longer see new
// entries. It returns the property value for the index block.
FinishIndexBlock(buf []byte) ([]byte, error)

// FinishTable is called when the sstable is finished, and returns the
// property value for the sstable.
FinishTable(buf []byte) ([]byte, error)
}

// SuffixReplaceableBlockCollector is an extension to the BlockPropertyCollector
// interface that allows a block property collector to indicate that it supports
// being *updated* during suffix replacement, i.e. when an existing SST in which
// all keys have the same key suffix is updated to have a new suffix.
//
// A collector which supports being updated in such cases must be able to derive
// its updated value from its old value and the change being made to the suffix,
// without needing to be passed each updated K/V.
//
// For example, a collector that only inspects values would can simply copy its
// previously computed property as-is, since key-suffix replacement does not
// change values, while a collector that depends only on key suffixes, like one
// which collected mvcc-timestamp bounds from timestamp-suffixed keys, can just
// set its new bounds from the new suffix, as it is common to all keys, without
// needing to recompute it from every key.
//
// An implementation of DataBlockIntervalCollector can also implement this
// interface, in which case the BlockPropertyCollector returned by passing it to
// NewBlockIntervalCollector will also implement this interface automatically.
type SuffixReplaceableBlockCollector interface {
// UpdateKeySuffixes is called when a block is updated to change the suffix of
// all keys in the block, and is passed the old value for that prop, if any,
// for that block as well as the old and new suffix.
UpdateKeySuffixes(oldProp []byte, oldSuffix, newSuffix []byte) error
}

// BlockPropertyFilter is used in an Iterator to filter sstables and blocks
// within the sstable. It should not maintain any per-sstable state, and must
// be thread-safe.
Expand Down Expand Up @@ -237,6 +241,25 @@ type DataBlockIntervalCollector interface {
// Add is called with each new entry added to a data block in the sstable.
// The callee can assume that these are in sorted order.
Add(key InternalKey, value []byte) error

// AddCollectedWithSuffixReplacement extends the interval with a given
// interval after updating it to reflect a change of suffix on all keys: the
// [oldLower, oldUpper) interval is assumed to be constructed from keys that
// all have the same oldSuffix and is recalculated to reflect the same keys but
// with newSuffix.
//
// This method is optional (if it is not implemented, it always returns an
// error). SupportsSuffixReplacement() can be used to check if this method is
// implemented.
//
// See BlockPropertyCollector.AddCollectedWithSuffixReplacement for more
// information.
AddCollectedWithSuffixReplacement(oldLower, oldUpper uint64, oldSuffix, newSuffix []byte) error

// SupportsSuffixReplacement returns whether the collector supports the
// AddCollectedWithSuffixReplacement method.
SupportsSuffixReplacement() bool

// FinishDataBlock is called when all the entries have been added to a
// data block. Subsequent Add calls will be for the next data block. It
// returns the [lower, upper) for the finished block.
Expand All @@ -262,23 +285,19 @@ func NewBlockIntervalCollector(
if pointCollector == nil && rangeCollector == nil {
panic("sstable: at least one interval collector must be provided")
}
bic := BlockIntervalCollector{
return &BlockIntervalCollector{
name: name,
points: pointCollector,
ranges: rangeCollector,
}
if _, ok := pointCollector.(SuffixReplaceableBlockCollector); ok {
return &suffixReplacementBlockCollectorWrapper{bic}
}
return &bic
}

// Name implements the BlockPropertyCollector interface.
// Name is part of the BlockPropertyCollector interface.
func (b *BlockIntervalCollector) Name() string {
return b.name
}

// Add implements the BlockPropertyCollector interface.
// Add is part of the BlockPropertyCollector interface.
func (b *BlockIntervalCollector) Add(key InternalKey, value []byte) error {
if rangekey.IsRangeKey(key.Kind()) {
if b.ranges != nil {
Expand All @@ -290,7 +309,23 @@ func (b *BlockIntervalCollector) Add(key InternalKey, value []byte) error {
return nil
}

// FinishDataBlock implements the BlockPropertyCollector interface.
// AddCollectedWithSuffixReplacement is part of the BlockPropertyCollector interface.
func (b *BlockIntervalCollector) AddCollectedWithSuffixReplacement(
oldProp []byte, oldSuffix, newSuffix []byte,
) error {
var i interval
if err := i.decode(oldProp); err != nil {
return err
}
return b.points.AddCollectedWithSuffixReplacement(i.lower, i.upper, oldSuffix, newSuffix)
}

// SupportsSuffixReplacement is part of the BlockPropertyCollector interface.
func (b *BlockIntervalCollector) SupportsSuffixReplacement() bool {
return b.points.SupportsSuffixReplacement()
}

// FinishDataBlock is part of the BlockPropertyCollector interface.
func (b *BlockIntervalCollector) FinishDataBlock(buf []byte) ([]byte, error) {
if b.points == nil {
return buf, nil
Expand Down Expand Up @@ -402,17 +437,6 @@ func (i interval) intersects(x interval) bool {
return i.upper > x.lower && i.lower < x.upper
}

type suffixReplacementBlockCollectorWrapper struct {
BlockIntervalCollector
}

// UpdateKeySuffixes implements the SuffixReplaceableBlockCollector interface.
func (w *suffixReplacementBlockCollectorWrapper) UpdateKeySuffixes(
oldProp []byte, from, to []byte,
) error {
return w.BlockIntervalCollector.points.(SuffixReplaceableBlockCollector).UpdateKeySuffixes(oldProp, from, to)
}

// BlockIntervalSyntheticReplacer provides methods to conduct just in time
// adjustments of a passed in block prop interval before filtering.
type BlockIntervalSyntheticReplacer interface {
Expand Down
9 changes: 7 additions & 2 deletions sstable/block_property_obsolete.go
Original file line number Diff line number Diff line change
Expand Up @@ -63,8 +63,8 @@ func (o *obsoleteKeyBlockPropertyCollector) FinishTable(buf []byte) ([]byte, err
return obsoleteKeyBlockPropertyEncode(!o.tableIsNonObsolete, buf), nil
}

// UpdateKeySuffixes is part of the BlockPropertyCollector interface.
func (o *obsoleteKeyBlockPropertyCollector) UpdateKeySuffixes(
// AddCollectedWithSuffixReplacement is part of the BlockPropertyCollector interface.
func (o *obsoleteKeyBlockPropertyCollector) AddCollectedWithSuffixReplacement(
oldProp []byte, oldSuffix, newSuffix []byte,
) error {
// Verify the property is valid.
Expand All @@ -77,6 +77,11 @@ func (o *obsoleteKeyBlockPropertyCollector) UpdateKeySuffixes(
return nil
}

// SupportsSuffixReplacement is part of the BlockPropertyCollector interface.
func (o *obsoleteKeyBlockPropertyCollector) SupportsSuffixReplacement() bool {
return true
}

// obsoleteKeyBlockPropertyFilter implements the filter that excludes blocks
// that only contain obsolete keys. It pairs with
// obsoleteKeyBlockPropertyCollector.
Expand Down
58 changes: 52 additions & 6 deletions sstable/block_property_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -174,10 +174,22 @@ type testDataBlockIntervalCollector struct {
i interval
}

var _ DataBlockIntervalCollector = &testDataBlockIntervalCollector{}

func (c *testDataBlockIntervalCollector) Add(key InternalKey, value []byte) error {
return nil
}

func (c *testDataBlockIntervalCollector) AddCollectedWithSuffixReplacement(
oldLower, oldUpper uint64, oldSuffix, newSuffix []byte,
) error {
return errors.Errorf("not supported")
}

func (c *testDataBlockIntervalCollector) SupportsSuffixReplacement() bool {
return false
}

func (c *testDataBlockIntervalCollector) FinishDataBlock() (lower uint64, upper uint64, err error) {
return c.i.lower, c.i.upper, nil
}
Expand Down Expand Up @@ -809,13 +821,25 @@ func (c *valueCharBlockIntervalCollector) FinishDataBlock() (lower, upper uint64
return l, u, nil
}

func (c *valueCharBlockIntervalCollector) AddCollectedWithSuffixReplacement(
oldLower, oldUpper uint64, oldSuffix, newSuffix []byte,
) error {
return errors.Errorf("not supported")
}

func (c *valueCharBlockIntervalCollector) SupportsSuffixReplacement() bool {
return false
}

// testKeysSuffixIntervalCollector maintains an interval over the timestamps in
// MVCC-like suffixes for keys (e.g. foo@123).
type suffixIntervalCollector struct {
initialized bool
lower, upper uint64
}

var _ DataBlockIntervalCollector = &suffixIntervalCollector{}

// Add implements DataBlockIntervalCollector by adding the timestamp(s) in the
// suffix(es) of this record to the current interval.
//
Expand Down Expand Up @@ -872,6 +896,16 @@ func (c *suffixIntervalCollector) FinishDataBlock() (lower, upper uint64, err er
return l, u, nil
}

func (c *suffixIntervalCollector) AddCollectedWithSuffixReplacement(
oldLower, oldUpper uint64, oldSuffix, newSuffix []byte,
) error {
return errors.Errorf("not implemented")
}

func (c *suffixIntervalCollector) SupportsSuffixReplacement() bool {
return false
}

func TestBlockProperties(t *testing.T) {
var r *Reader
defer func() {
Expand Down Expand Up @@ -1385,7 +1419,6 @@ type keyCountCollector struct {
}

var _ BlockPropertyCollector = &keyCountCollector{}
var _ SuffixReplaceableBlockCollector = &keyCountCollector{}

func keyCountCollectorFn(name string) func() BlockPropertyCollector {
return func() BlockPropertyCollector { return &keyCountCollector{name: name} }
Expand Down Expand Up @@ -1425,15 +1458,21 @@ func (p *keyCountCollector) FinishTable(buf []byte) ([]byte, error) {
return buf, nil
}

func (p *keyCountCollector) UpdateKeySuffixes(old []byte, _, _ []byte) error {
n, err := strconv.Atoi(string(old))
func (p *keyCountCollector) AddCollectedWithSuffixReplacement(
oldProp []byte, oldSuffix, newSuffix []byte,
) error {
n, err := strconv.Atoi(string(oldProp))
if err != nil {
return err
}
p.block = n
return nil
}

func (p *keyCountCollector) SupportsSuffixReplacement() bool {
return true
}

// intSuffixCollector is testing prop collector that collects the min and
// max value of numeric suffix of keys (interpreting suffixLen bytes as ascii
// for conversion with atoi).
Expand Down Expand Up @@ -1481,19 +1520,26 @@ type intSuffixIntervalCollector struct {
intSuffixCollector
}

var _ DataBlockIntervalCollector = &intSuffixIntervalCollector{}

func intSuffixIntervalCollectorFn(name string, length int) func() BlockPropertyCollector {
return func() BlockPropertyCollector {
return NewBlockIntervalCollector(name, &intSuffixIntervalCollector{makeIntSuffixCollector(length)}, nil)
}
}

var _ DataBlockIntervalCollector = &intSuffixIntervalCollector{}
var _ SuffixReplaceableBlockCollector = &intSuffixIntervalCollector{}

func (p *intSuffixIntervalCollector) FinishDataBlock() (lower uint64, upper uint64, err error) {
return p.min, p.max + 1, nil
}

func (p *intSuffixIntervalCollector) UpdateKeySuffixes(oldProp []byte, from, to []byte) error {
return p.setFromSuffix(to)
func (p *intSuffixIntervalCollector) AddCollectedWithSuffixReplacement(
oldLower, oldUpper uint64, oldSuffix, newSuffix []byte,
) error {
return p.setFromSuffix(newSuffix)
}

func (p *intSuffixIntervalCollector) SupportsSuffixReplacement() bool {
return true
}
13 changes: 13 additions & 0 deletions sstable/block_property_test_utils.go
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,7 @@ import (
"fmt"
"math"

"github.com/cockroachdb/errors"
"github.com/cockroachdb/pebble/internal/base"
"github.com/cockroachdb/pebble/internal/testkeys"
)
Expand Down Expand Up @@ -140,3 +141,15 @@ func (c *testKeysSuffixIntervalCollector) FinishDataBlock() (lower, upper uint64
c.initialized = false
return l, u, nil
}

// AddCollectedWithSuffixReplacement is part of the DataBlockIntervalCollector interface.
func (c *testKeysSuffixIntervalCollector) AddCollectedWithSuffixReplacement(
oldLower, oldUpper uint64, oldSuffix, newSuffix []byte,
) error {
return errors.Errorf("not implemented")
}

// SupportsSuffixReplacement part of the DataBlockIntervalCollector interface.
func (c *testKeysSuffixIntervalCollector) SupportsSuffixReplacement() bool {
return false
}
Loading

0 comments on commit 6cdb88d

Please sign in to comment.