Skip to content

Commit

Permalink
colblk: cache IndexBlockDecoder in block metadata
Browse files Browse the repository at this point in the history
```
RandSeekInSST/v4/single-level-10  1.21µs ± 1%  1.25µs ± 5%   +3.40%  (p=0.005 n=8+8)
RandSeekInSST/v4/two-level-10     2.06µs ± 4%  2.06µs ± 1%     ~     (p=0.959 n=8+8)
RandSeekInSST/v5/single-level-10  1.11µs ± 1%  1.06µs ± 1%   -4.03%  (p=0.000 n=8+8)
RandSeekInSST/v5/two-level-10     1.79µs ± 5%  1.59µs ± 1%  -11.45%  (p=0.000 n=7+8)
```
  • Loading branch information
RaduBerinde committed Oct 18, 2024
1 parent dff8179 commit 70a98bc
Show file tree
Hide file tree
Showing 7 changed files with 60 additions and 32 deletions.
6 changes: 3 additions & 3 deletions sstable/block/buffer_pool.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,13 +38,13 @@ func (b Value) getInternalBuf() []byte {
return b.v.Buf()
}

// Get returns the byte slice for the block data.
func (b Value) Get() []byte {
// BlockData returns the byte slice for the block data.
func (b Value) BlockData() []byte {
return b.getInternalBuf()[MetadataSize:]
}

// GetMetadata returns the block metadata.
func (b Value) GetMetadata() *Metadata {
func (b Value) BlockMetadata() *Metadata {
return (*Metadata)(b.getInternalBuf())
}

Expand Down
29 changes: 22 additions & 7 deletions sstable/colblk/data_block.go
Original file line number Diff line number Diff line change
Expand Up @@ -737,13 +737,11 @@ func (rw *DataBlockRewriter) RewriteSuffixes(
return start, end, rewritten, nil
}

// DataBlockDecoderSize is the size of a DataBlockDecoder struct. If allocating
// memory for a data block, the caller may want to additionally allocate memory
// for the corresponding DataBlockDecoder.
const DataBlockDecoderSize = unsafe.Sizeof(DataBlockDecoder{})

// Assert that a DataBlockDecoder can fit inside block.Metadata.
const _ uint = block.MetadataSize - uint(DataBlockDecoderSize)
const _ uint = block.MetadataSize - uint(unsafe.Sizeof(DataBlockDecoder{}))

// Assert that an IndexBlockDecoder can fit inside block.Metadata.
const _ uint = block.MetadataSize - uint(unsafe.Sizeof(IndexBlockDecoder{}))

// InitDataBlockMetadata initializes the metadata for a data block.
func InitDataBlockMetadata(schema KeySchema, md *block.Metadata, data []byte) (err error) {
Expand All @@ -763,6 +761,23 @@ func InitDataBlockMetadata(schema KeySchema, md *block.Metadata, data []byte) (e
return nil
}

// InitIndexBlockMetadata initializes the metadata for an index block.
func InitIndexBlockMetadata(md *block.Metadata, data []byte) (err error) {
if uintptr(unsafe.Pointer(md))%8 != 0 {
return errors.AssertionFailedf("metadata is not 8-byte aligned")
}
d := (*IndexBlockDecoder)(unsafe.Pointer(md))
// Initialization can panic; convert panics to corruption errors (so higher
// layers can add file number and offset information).
defer func() {
if r := recover(); r != nil {
err = base.CorruptionErrorf("error initializing data block metadata: %v", r)
}
}()
d.Init(data)
return nil
}

// A DataBlockDecoder holds state for interpreting a columnar data block. It may
// be shared among multiple DataBlockIters.
type DataBlockDecoder struct {
Expand Down Expand Up @@ -899,7 +914,7 @@ func (i *DataBlockIter) InitOnce(
}

// Init initializes the data block iterator, configuring it to read from the
// provided reader.
// provided decoder.
func (i *DataBlockIter) Init(d *DataBlockDecoder, transforms block.IterTransforms) error {
i.d = d
// Leave i.h unchanged.
Expand Down
17 changes: 8 additions & 9 deletions sstable/colblk/index_block.go
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,7 @@ package colblk

import (
"slices"
"unsafe"

"github.com/cockroachdb/errors"
"github.com/cockroachdb/pebble/internal/base"
Expand Down Expand Up @@ -192,15 +193,17 @@ type IndexIter struct {
syntheticSuffix block.SyntheticSuffix
noTransforms bool

h block.BufferHandle
h block.BufferHandle
// TODO(radu): remove allocDecoder and require any Init callers to provide the
// decoder.
allocDecoder IndexBlockDecoder
keyBuf []byte
}

// Assert that IndexIter satisfies the block.IndexBlockIterator interface.
var _ block.IndexBlockIterator = (*IndexIter)(nil)

// InitWithDecoder initializes an index iterator from the provided reader.
// InitWithDecoder initializes an index iterator from the provided decoder.
func (i *IndexIter) InitWithDecoder(
compare base.Compare, split base.Split, d *IndexBlockDecoder, transforms block.IterTransforms,
) {
Expand All @@ -209,6 +212,7 @@ func (i *IndexIter) InitWithDecoder(
split: split,
d: d,
n: int(d.bd.header.Rows),
row: -1,
h: i.h,
allocDecoder: i.allocDecoder,
keyBuf: i.keyBuf,
Expand All @@ -233,15 +237,10 @@ func (i *IndexIter) Init(
func (i *IndexIter) InitHandle(
cmp base.Compare, split base.Split, blk block.BufferHandle, transforms block.IterTransforms,
) error {
// TODO(jackson): If block.h != nil, use a *IndexBlockDecoder that's allocated
// when the block is loaded into the block cache. On cache hits, this will
// reduce the amount of setup necessary to use an iterator. (It's relatively
// common to open an iterator and perform just a few seeks, so avoiding the
// overhead can be material.)
i.h.Release()
i.h = blk
i.allocDecoder.Init(i.h.BlockData())
i.InitWithDecoder(cmp, split, &i.allocDecoder, transforms)
d := (*IndexBlockDecoder)(unsafe.Pointer(blk.BlockMetadata()))
i.InitWithDecoder(cmp, split, d, transforms)
return nil
}

Expand Down
11 changes: 8 additions & 3 deletions sstable/colblk/index_block_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ import (
"strings"
"sync"
"testing"
"unsafe"

"github.com/cockroachdb/datadriven"
"github.com/cockroachdb/pebble/internal/base"
Expand Down Expand Up @@ -115,12 +116,16 @@ func TestIndexIterInitHandle(t *testing.T) {
bh2 := block.Handle{Offset: 2008, Length: 1000}
w.AddBlockHandle([]byte("a"), bh1, nil)
w.AddBlockHandle([]byte("b"), bh2, nil)
b := w.Finish(w.Rows())
blockData := w.Finish(w.Rows())

c := cache.New(10 << 10)
defer c.Unref()
v := block.Alloc(len(b), nil)
copy(v.Get(), b)

v := block.Alloc(block.MetadataSize+len(blockData), nil)
copy(v.BlockData(), blockData)
d := (*IndexBlockDecoder)(unsafe.Pointer(v.BlockMetadata()))
d.Init(blockData)

v.MakeHandle(c, cache.ID(1), base.DiskFileNum(1), 0).Release()

getBlockAndIterate := func(it *IndexIter) {
Expand Down
2 changes: 1 addition & 1 deletion sstable/colblk/keyspan_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -85,7 +85,7 @@ func TestKeyspanBlockPooling(t *testing.T) {
c := cache.New(10 << 10)
defer c.Unref()
v := block.Alloc(len(b), nil)
copy(v.Get(), b)
copy(v.BlockData(), b)
v.MakeHandle(c, cache.ID(1), base.DiskFileNum(1), 0).Release()

getBlockAndIterate := func() {
Expand Down
2 changes: 1 addition & 1 deletion sstable/layout.go
Original file line number Diff line number Diff line change
Expand Up @@ -321,7 +321,7 @@ var (
)

func formatColblkIndexBlock(tp treeprinter.Node, r *Reader, b NamedBlockHandle, data []byte) error {
iter := new(colblk.IndexIter)
var iter colblk.IndexIter
if err := iter.Init(r.Compare, r.Split, data, NoTransforms); err != nil {
return err
}
Expand Down
25 changes: 17 additions & 8 deletions sstable/reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -393,7 +393,16 @@ func (r *Reader) readIndexBlock(
ctx context.Context, env readBlockEnv, readHandle objstorage.ReadHandle, bh block.Handle,
) (block.BufferHandle, error) {
ctx = objiotracing.WithBlockType(ctx, objiotracing.MetadataBlock)
return r.readBlockInternal(ctx, env, readHandle, bh, noInitBlockMetadataFn)
return r.readBlockInternal(ctx, env, readHandle, bh, r.initIndexBlockMetadata)
}

// initIndexBlockMetadata initializes the Metadata for a data block. This will
// later be used (and reused) when reading from the block.
func (r *Reader) initIndexBlockMetadata(metadata *block.Metadata, data []byte) error {
if r.tableFormat.BlockColumnar() {
return colblk.InitIndexBlockMetadata(metadata, data)
}
return nil
}

func (r *Reader) readDataBlock(
Expand Down Expand Up @@ -507,9 +516,9 @@ func (r *Reader) readBlockInternal(
readStopwatch := makeStopwatch()
var err error
if readHandle != nil {
err = readHandle.ReadAt(ctx, compressed.Get(), int64(bh.Offset))
err = readHandle.ReadAt(ctx, compressed.BlockData(), int64(bh.Offset))
} else {
err = r.readable.ReadAt(ctx, compressed.Get(), int64(bh.Offset))
err = r.readable.ReadAt(ctx, compressed.BlockData(), int64(bh.Offset))
}
readDuration := readStopwatch.stop()
// Call IsTracingEnabled to avoid the allocations of boxing integers into an
Expand All @@ -528,34 +537,34 @@ func (r *Reader) readBlockInternal(
return block.BufferHandle{}, err
}
env.BlockRead(bh.Length, readDuration)
if err := checkChecksum(r.checksumType, compressed.Get(), bh, r.cacheOpts.FileNum); err != nil {
if err := checkChecksum(r.checksumType, compressed.BlockData(), bh, r.cacheOpts.FileNum); err != nil {
compressed.Release()
return block.BufferHandle{}, err
}

typ := block.CompressionIndicator(compressed.Get()[bh.Length])
typ := block.CompressionIndicator(compressed.BlockData()[bh.Length])
compressed.Truncate(int(bh.Length))

var decompressed block.Value
if typ == block.NoCompressionIndicator {
decompressed = compressed
} else {
// Decode the length of the decompressed value.
decodedLen, prefixLen, err := block.DecompressedLen(typ, compressed.Get())
decodedLen, prefixLen, err := block.DecompressedLen(typ, compressed.BlockData())
if err != nil {
compressed.Release()
return block.BufferHandle{}, err
}

decompressed = block.Alloc(decodedLen, env.BufferPool)
err = block.DecompressInto(typ, compressed.Get()[prefixLen:], decompressed.Get())
err = block.DecompressInto(typ, compressed.BlockData()[prefixLen:], decompressed.BlockData())
compressed.Release()
if err != nil {
decompressed.Release()
return block.BufferHandle{}, err
}
}
if err := initBlockMetadataFn(decompressed.GetMetadata(), decompressed.Get()); err != nil {
if err := initBlockMetadataFn(decompressed.BlockMetadata(), decompressed.BlockData()); err != nil {
decompressed.Release()
return block.BufferHandle{}, err
}
Expand Down

0 comments on commit 70a98bc

Please sign in to comment.