Skip to content

Commit

Permalink
sstable: allow range key property collection
Browse files Browse the repository at this point in the history
Currently, when point keys are added to an sstable, the key is run
through each of the block property collectors configured on the
`sstable.Writer`. Range keys are not included.

Allow range keys to run through the same block property collector
pipeline by looping through all configured collectors on the writer and
passing the range key. Any struct that implements
`sstable.BlockPropertyCollector` is allowed to accept any type of key
kind. It is up to the implementation to filter out keys that are not
applicable. This allows for flexibility in crafting collectors that can
take specific types of keys, or combinations of key kinds (e.g. point-
and range-key specific collectors, or table-wide "global" collectors).

Rename the existing `DataBlockIntervalCollector` interface to
`BlockIntervalCollector`, and rename the `FinishDataBlock` method to
`FinishBlock`. Both of these change are intended to make the interface
applicable to keys other than point keys that may have dedicated blocks
in the sstable, as is the case for range keys.

Rename the existing `BlockIntervalCollector` struct to
`DataBlockIntervalCollector`. To make the struct specific to point keys,
ignore range key kinds in calls to `Add.

Add a new `BlockPropertyCollector` helper implementation,
`RangeKeyBlockIntervalCollector`, that operates exclusively on range
keys. All other keys kinds are ignored. The collector is intended to
support maintaining an upper and lower bound on the MVCC timestamps
present in a range key block in an sstable.

Add a test implementation of a `BlockIntervalCollector` and a
data-driven that demonstrates maintaining upper and lower bounds on
point and range keys with integer suffixes (e.g. `[email protected]:foo`,
`[email protected]:bar [(@100=baz)]`, etc.).

One downside with this implementation is that the
`BlockPropertyCollector` interface contains methods such as
`FinishDataBlock` and `FinishIndexBlock` that are not applicable to
range keys (range keys are all in a single block, and do not have an
index block, respectively). However, retaining these specific methods
allows for implementations to be created that could support both point
and range keys (for example, if creating a collector whose properties
are intended to be applicable to all key kinds).

Alternatives approaches considered, with relative downsides:

- adding a new member field to `BlockIntervalCollector` for tracking the
  range key interval. The downside with this approach is that it
  requires that both the point and range key intervals be encoded into
  the properties for the table and block with the same name. This make
  it difficult to disambiguate the intervals on the read path.

- have member fields for both point and range key collectors in
  `sstable.Writer`. The downside with this approach is that it requires
  more intrusive changes to the `Writer` to call the specific type of
  property collector on the write path, which does not scale nicely to
  support various block types that require property collection (i.e.
  potentially supporting range-dels in the future). There is also some
  nuance to managing separate collections of collectors with the
  `shortID` mapping that is used when encoding the properties in the
  properties and index blocks (i.e. need to be careful not to re-use the
  same ordinals, etc.).
  • Loading branch information
nicktrav committed Dec 22, 2021
1 parent 06e42cf commit e9d6878
Show file tree
Hide file tree
Showing 5 changed files with 302 additions and 54 deletions.
8 changes: 4 additions & 4 deletions iterator_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1088,7 +1088,7 @@ func (bi *testBlockIntervalCollector) Add(key InternalKey, value []byte) error {
return nil
}

func (bi *testBlockIntervalCollector) FinishDataBlock() (lower uint64, upper uint64, err error) {
func (bi *testBlockIntervalCollector) FinishBlock() (lower uint64, upper uint64, err error) {
bi.initialized = false
l, u := bi.lower, bi.upper
bi.lower, bi.upper = 0, 0
Expand Down Expand Up @@ -1118,7 +1118,7 @@ func TestIteratorBlockIntervalFilter(t *testing.T) {
for _, c := range collectors {
coll := c
bpCollectors = append(bpCollectors, func() BlockPropertyCollector {
return sstable.NewBlockIntervalCollector(
return sstable.NewDataBlockIntervalCollector(
fmt.Sprintf("%d", coll.id),
&testBlockIntervalCollector{numLength: 2, offsetFromEnd: coll.offset})
})
Expand Down Expand Up @@ -1240,7 +1240,7 @@ func TestIteratorRandomizedBlockIntervalFilter(t *testing.T) {
FormatMajorVersion: FormatNewest,
BlockPropertyCollectors: []func() BlockPropertyCollector{
func() BlockPropertyCollector {
return sstable.NewBlockIntervalCollector("0", &testBlockIntervalCollector{
return sstable.NewDataBlockIntervalCollector("0", &testBlockIntervalCollector{
numLength: 2,
})
},
Expand Down Expand Up @@ -1553,7 +1553,7 @@ func BenchmarkBlockPropertyFilter(b *testing.B) {
FormatMajorVersion: FormatNewest,
BlockPropertyCollectors: []func() BlockPropertyCollector{
func() BlockPropertyCollector {
return sstable.NewBlockIntervalCollector("0", &testBlockIntervalCollector{
return sstable.NewDataBlockIntervalCollector("0", &testBlockIntervalCollector{
numLength: 3,
})
},
Expand Down
145 changes: 111 additions & 34 deletions sstable/block_property.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ import (
"sync"

"github.com/cockroachdb/pebble/internal/base"
"github.com/cockroachdb/pebble/internal/rangekey"
)

// Block properties are an optional user-facing feature that can be used to
Expand Down Expand Up @@ -91,63 +92,69 @@ type BlockPropertyFilter interface {
Intersects(prop []byte) (bool, error)
}

// BlockIntervalCollector is a helper implementation of BlockPropertyCollector
// for users who want to represent a set of the form [lower,upper) where both
// lower and upper are uint64, and lower <= upper.
// BlockIntervalCollector is the interface used by BlockIntervalCollector
// that contains the actual logic pertaining to the property. It only
// maintains state for the current data block, and resets that state in
// FinishDataBlock. This interface can be used to reduce parsing costs.
type BlockIntervalCollector interface {
// Add is called with each new entry added to a block in the sstable. The
// callee can assume that these are in sorted order.
Add(key InternalKey, value []byte) error
// FinishBlock is called when all the entries have been added to a block.
// Subsequent calls to Add will be for the next block, if applicable. It
// returns the [lower, upper) for the finished block.
FinishBlock() (lower uint64, upper uint64, err error)
}

// DataBlockIntervalCollector is a helper implementation of
// BlockPropertyCollector for users who want to represent a set of the form
// [lower,upper) where both lower and upper are uint64, and lower <= upper.
//
// The set is encoded as:
// - Two varint integers, (lower,upper-lower), when upper-lower > 0
// - Nil, when upper-lower=0
//
// Users must not expect this to preserve differences between empty sets --
// they will all get turned into the semantically equivalent [0,0).
type BlockIntervalCollector struct {
type DataBlockIntervalCollector struct {
name string
dbic DataBlockIntervalCollector
bic BlockIntervalCollector

blockInterval interval
indexInterval interval
tableInterval interval
}

var _ BlockPropertyCollector = &BlockIntervalCollector{}

// DataBlockIntervalCollector is the interface used by BlockIntervalCollector
// that contains the actual logic pertaining to the property. It only
// maintains state for the current data block, and resets that state in
// FinishDataBlock. This interface can be used to reduce parsing costs.
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
// 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.
FinishDataBlock() (lower uint64, upper uint64, err error)
}
var _ BlockPropertyCollector = &DataBlockIntervalCollector{}

// NewBlockIntervalCollector constructs a BlockIntervalCollector, with the
// given name and data block collector.
func NewBlockIntervalCollector(
name string, blockAttributeCollector DataBlockIntervalCollector) *BlockIntervalCollector {
return &BlockIntervalCollector{
name: name, dbic: blockAttributeCollector}
// NewDataBlockIntervalCollector constructs a DataBlockIntervalCollector, with
// the given name and BlockIntervalCollector.
func NewDataBlockIntervalCollector(
name string, collector BlockIntervalCollector) *DataBlockIntervalCollector {
return &DataBlockIntervalCollector{
name: name,
bic: collector,
}
}

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

// Add implements the BlockPropertyCollector interface.
func (b *BlockIntervalCollector) Add(key InternalKey, value []byte) error {
return b.dbic.Add(key, value)
func (b *DataBlockIntervalCollector) Add(key InternalKey, value []byte) error {
// Skip range keys.
if rangekey.IsRangeKey(key.Kind()) {
return nil
}
return b.bic.Add(key, value)
}

// FinishDataBlock implements the BlockPropertyCollector interface.
func (b *BlockIntervalCollector) FinishDataBlock(buf []byte) ([]byte, error) {
func (b *DataBlockIntervalCollector) FinishDataBlock(buf []byte) ([]byte, error) {
var err error
b.blockInterval.lower, b.blockInterval.upper, err = b.dbic.FinishDataBlock()
b.blockInterval.lower, b.blockInterval.upper, err = b.bic.FinishBlock()
if err != nil {
return buf, err
}
Expand All @@ -158,23 +165,93 @@ func (b *BlockIntervalCollector) FinishDataBlock(buf []byte) ([]byte, error) {

// AddPrevDataBlockToIndexBlock implements the BlockPropertyCollector
// interface.
func (b *BlockIntervalCollector) AddPrevDataBlockToIndexBlock() {
func (b *DataBlockIntervalCollector) AddPrevDataBlockToIndexBlock() {
b.indexInterval.union(b.blockInterval)
b.blockInterval = interval{}
}

// FinishIndexBlock implements the BlockPropertyCollector interface.
func (b *BlockIntervalCollector) FinishIndexBlock(buf []byte) ([]byte, error) {
func (b *DataBlockIntervalCollector) FinishIndexBlock(buf []byte) ([]byte, error) {
buf = b.indexInterval.encode(buf)
b.indexInterval = interval{}
return buf, nil
}

// FinishTable implements the BlockPropertyCollector interface.
func (b *BlockIntervalCollector) FinishTable(buf []byte) ([]byte, error) {
func (b *DataBlockIntervalCollector) FinishTable(buf []byte) ([]byte, error) {
return b.tableInterval.encode(buf), nil
}

// RangeBlockIntervalCollector is a helper implementation of
// BlockPropertyCollector for users who want to represent a set of the form
// [lower,upper) where both lower and upper are uint64, and lower <= upper.
//
// This BlockPropertyCollector is specific to range keys. Keys of kinds other
// than range keys will be ignored.
type RangeBlockIntervalCollector struct {
name string
bic BlockIntervalCollector
}

var _ BlockPropertyCollector = &RangeBlockIntervalCollector{}

// NewRangeBlockIntervalCollector constructs a RangeBlockIntervalCollector, with
// the given name and BlockIntervalCollector.
func NewRangeBlockIntervalCollector(
name string, collector BlockIntervalCollector) *RangeBlockIntervalCollector {
return &RangeBlockIntervalCollector{
name: name,
bic: collector,
}
}

// Name implements the BlockPropertyCollector interface.
func (b *RangeBlockIntervalCollector) Name() string {
return b.name
}

// Add implements the BlockPropertyCollector interface.
func (b *RangeBlockIntervalCollector) Add(key InternalKey, value []byte) error {
// Skip non-range keys.
if !rangekey.IsRangeKey(key.Kind()) {
return nil
}
return b.bic.Add(key, value)
}

// FinishDataBlock implements the BlockPropertyCollector interface.
//
// As range keys are confined to their own data block, this method is a no-op.
func (b *RangeBlockIntervalCollector) FinishDataBlock(_ []byte) ([]byte, error) {
return nil, nil // no-op
}

// AddPrevDataBlockToIndexBlock implements the BlockPropertyCollector
// interface.
//
// As range keys do not have an associated index block, this method is a no-op.
func (b *RangeBlockIntervalCollector) AddPrevDataBlockToIndexBlock() {
// no-op
}

// FinishIndexBlock implements the BlockPropertyCollector interface.
//
// As range keys do not have an associated index block, this method is a no-op.
func (b *RangeBlockIntervalCollector) FinishIndexBlock(_ []byte) ([]byte, error) {
return nil, nil // no-op
}

// FinishTable implements the BlockPropertyCollector interface, encoding the
// lower and upper bound as delta-encoded interval.
func (b *RangeBlockIntervalCollector) FinishTable(buf []byte) ([]byte, error) {
lower, upper, err := b.bic.FinishBlock()
if err != nil {
return nil, err
}
i := interval{lower: lower, upper: upper}
return i.encode(buf), nil
}

type interval struct {
lower uint64
upper uint64
Expand Down
Loading

0 comments on commit e9d6878

Please sign in to comment.