Skip to content

Commit

Permalink
colblk: rename DataBlockWriter/Reader to Encoder/Decoder
Browse files Browse the repository at this point in the history
These names are more appropriate because these types don't actually
perform any reading or writing, and there are other associated reader
and writer types.
  • Loading branch information
RaduBerinde committed Oct 7, 2024
1 parent d320ff0 commit 3f7527f
Show file tree
Hide file tree
Showing 6 changed files with 44 additions and 45 deletions.
12 changes: 6 additions & 6 deletions sstable/colblk/cockroach_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -183,7 +183,7 @@ var cockroachKeySeekerPool = sync.Pool{
}

type cockroachKeySeeker struct {
reader *DataBlockReader
reader *DataBlockDecoder
roachKeys PrefixBytes
mvccWallTimes UnsafeUints
mvccLogical UnsafeUints
Expand All @@ -194,7 +194,7 @@ type cockroachKeySeeker struct {
var _ KeySeeker = (*cockroachKeySeeker)(nil)

// Init is part of the KeySeeker interface.
func (ks *cockroachKeySeeker) Init(r *DataBlockReader) error {
func (ks *cockroachKeySeeker) Init(r *DataBlockDecoder) error {
ks.reader = r
ks.roachKeys = r.r.PrefixBytes(cockroachColRoachKey)
ks.mvccWallTimes = r.r.Uints(cockroachColMVCCWallTime)
Expand Down Expand Up @@ -403,7 +403,7 @@ func TestCockroachDataBlock(t *testing.T) {
BaseWallTime: seed,
}, valueLen)

var reader DataBlockReader
var reader DataBlockDecoder
var it DataBlockIter
it.InitOnce(cockroachKeySchema, crdbtest.Compare, crdbtest.Split, getLazyValuer(func([]byte) base.LazyValue {
return base.LazyValue{ValueOrHandle: []byte("mock external value")}
Expand Down Expand Up @@ -450,7 +450,7 @@ func generateDataBlock(
) (data []byte, keys [][]byte, values [][]byte) {
keys, values = crdbtest.RandomKVs(rng, targetBlockSize/valueLen, cfg, valueLen)

var w DataBlockWriter
var w DataBlockEncoder
w.Init(cockroachKeySchema)
count := 0
for w.Size() < targetBlockSize {
Expand Down Expand Up @@ -492,7 +492,7 @@ func benchmarkCockroachDataBlockWriter(b *testing.B, keyConfig crdbtest.KeyConfi
rng := rand.New(rand.NewSource(seed))
_, keys, values := generateDataBlock(rng, targetBlockSize, keyConfig, valueLen)

var w DataBlockWriter
var w DataBlockEncoder
w.Init(cockroachKeySchema)

b.ResetTimer()
Expand Down Expand Up @@ -629,7 +629,7 @@ func benchmarkCockroachDataBlockIter(

serializedBlock, keys, _ := generateDataBlock(rng, targetBlockSize, cfg.KeyConfig, cfg.ValueLen)

var reader DataBlockReader
var reader DataBlockDecoder
var it DataBlockIter
it.InitOnce(cockroachKeySchema, crdbtest.Compare, crdbtest.Split, getLazyValuer(func([]byte) base.LazyValue {
return base.LazyValue{ValueOrHandle: []byte("mock external value")}
Expand Down
65 changes: 32 additions & 33 deletions sstable/colblk/data_block.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@ import (
//
// TODO(jackson): Consider making this KVSchema. It feels like there's an
// opportunity to generalize the ShortAttribute so that when a value is stored
// out-of-band, the DataBlockWriter calls user-provided code to store the short
// out-of-band, the DataBlockEncoder calls user-provided code to store the short
// attributes inlined within the data block. For inlined-values, the
// user-defined value columns would be implicitly null.
type KeySchema struct {
Expand Down Expand Up @@ -98,8 +98,8 @@ func (kcmp KeyComparison) PrefixEqual() bool { return kcmp.PrefixLen == kcmp.Com
// goroutines. In practice, multiple DataBlockIterators may use the same
// KeySeeker.
type KeySeeker interface {
// Init initializes the iterator to read from the provided DataBlockReader.
Init(b *DataBlockReader) error
// Init initializes the iterator to read from the provided DataBlockDecoder.
Init(b *DataBlockDecoder) error
// IsLowerBound returns true if all keys in the data block (after suffix
// replacement if syntheticSuffix is not empty) are >= the given key. If the
// data block contains no keys, returns true.
Expand Down Expand Up @@ -296,13 +296,13 @@ var _ KeySeeker = (*defaultKeySeeker)(nil)

type defaultKeySeeker struct {
comparer *base.Comparer
reader *DataBlockReader
reader *DataBlockDecoder
prefixes PrefixBytes
suffixes RawBytes
sharedPrefix []byte
}

func (ks *defaultKeySeeker) Init(r *DataBlockReader) error {
func (ks *defaultKeySeeker) Init(r *DataBlockDecoder) error {
ks.reader = r
ks.prefixes = r.r.PrefixBytes(defaultKeySchemaColumnPrefix)
ks.suffixes = r.r.RawBytes(defaultKeySchemaColumnSuffix)
Expand Down Expand Up @@ -404,9 +404,8 @@ func (ks *defaultKeySeeker) Release() {
defaultKeySeekerPool.Put(ks)
}

// DataBlockWriter writes columnar data blocks, encoding keys using a
// user-defined schema.
type DataBlockWriter struct {
// DataBlockEncoder encodes columnar data blocks using a user-defined schema.
type DataBlockEncoder struct {
Schema KeySchema
KeyWriter KeyWriter
// trailers is the column writer for InternalKey uint64 trailers.
Expand Down Expand Up @@ -451,7 +450,7 @@ const (
const dataBlockCustomHeaderSize = 4

// Init initializes the data block writer.
func (w *DataBlockWriter) Init(schema KeySchema) {
func (w *DataBlockEncoder) Init(schema KeySchema) {
w.Schema = schema
w.KeyWriter = schema.NewKeyWriter()
w.trailers.Init()
Expand All @@ -466,7 +465,7 @@ func (w *DataBlockWriter) Init(schema KeySchema) {
}

// Reset resets the data block writer to its initial state, retaining buffers.
func (w *DataBlockWriter) Reset() {
func (w *DataBlockEncoder) Reset() {
w.KeyWriter.Reset()
w.trailers.Reset()
w.prefixSame.Reset()
Expand All @@ -479,8 +478,8 @@ func (w *DataBlockWriter) Reset() {
w.enc.reset()
}

// String outputs a human-readable summary of internal DataBlockWriter state.
func (w *DataBlockWriter) String() string {
// String outputs a human-readable summary of internal DataBlockEncoder state.
func (w *DataBlockEncoder) String() string {
var buf bytes.Buffer
size := uint32(w.Size())
fmt.Fprintf(&buf, "size=%d:\n", size)
Expand Down Expand Up @@ -517,7 +516,7 @@ func (w *DataBlockWriter) String() string {
//
// The caller is required to pass this in because in expected use cases, the
// caller will also require the same information.
func (w *DataBlockWriter) Add(
func (w *DataBlockEncoder) Add(
ikey base.InternalKey,
value []byte,
valuePrefix block.ValuePrefix,
Expand Down Expand Up @@ -549,12 +548,12 @@ func (w *DataBlockWriter) Add(
}

// Rows returns the number of rows in the current pending data block.
func (w *DataBlockWriter) Rows() int {
func (w *DataBlockEncoder) Rows() int {
return w.rows
}

// Size returns the size of the current pending data block.
func (w *DataBlockWriter) Size() int {
func (w *DataBlockEncoder) Size() int {
off := blockHeaderSize(len(w.Schema.ColumnTypes)+dataBlockColumnMax, dataBlockCustomHeaderSize)
off = w.KeyWriter.Size(w.rows, off)
off = w.trailers.Size(w.rows, off)
Expand All @@ -569,13 +568,13 @@ func (w *DataBlockWriter) Size() int {
// Finish serializes the pending data block, including the first [rows] rows.
// The value of [rows] must be Rows() or Rows()-1. The provided size must be the
// size of the data block with the provided row count (i.e., the return value of
// [Size] when DataBlockWriter.Rows() = [rows]).
// [Size] when DataBlockEncoder.Rows() = [rows]).
//
// Finish the returns the serialized, uncompressed data block and the
// InternalKey of the last key contained within the data block. The memory of
// the lastKey's UserKey is owned by the DataBlockWriter. The caller must
// the lastKey's UserKey is owned by the DataBlockEncoder. The caller must
// copy it if they require it to outlive a Reset of the writer.
func (w *DataBlockWriter) Finish(rows, size int) (finished []byte, lastKey base.InternalKey) {
func (w *DataBlockEncoder) Finish(rows, size int) (finished []byte, lastKey base.InternalKey) {
if invariants.Enabled && rows != w.rows && rows != w.rows-1 {
panic(errors.AssertionFailedf("data block has %d rows; asked to finish %d", w.rows, rows))
}
Expand Down Expand Up @@ -617,8 +616,8 @@ func (w *DataBlockWriter) Finish(rows, size int) (finished []byte, lastKey base.
type DataBlockRewriter struct {
KeySchema KeySchema

writer DataBlockWriter
reader DataBlockReader
writer DataBlockEncoder
reader DataBlockDecoder
iter DataBlockIter
keySeeker KeySeeker
compare base.Compare
Expand Down Expand Up @@ -734,14 +733,14 @@ func (rw *DataBlockRewriter) RewriteSuffixes(
return start, end, rewritten, nil
}

// DataBlockReaderSize is the size of a DataBlockReader struct. If allocating
// 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 DataBlockReader.
const DataBlockReaderSize = unsafe.Sizeof(DataBlockReader{})
// for the corresponding DataBlockDecoder.
const DataBlockDecoderSize = unsafe.Sizeof(DataBlockDecoder{})

// A DataBlockReader holds state for reading a columnar data block. It may be
// shared among multiple DataBlockIters.
type DataBlockReader struct {
// A DataBlockDecoder holds state for interpreting a columnar data block. It may
// be shared among multiple DataBlockIters.
type DataBlockDecoder struct {
r BlockReader
// trailers holds an array of the InternalKey trailers, encoding the key
// kind and sequence number of each key.
Expand Down Expand Up @@ -772,12 +771,12 @@ type DataBlockReader struct {
}

// BlockReader returns a pointer to the underlying BlockReader.
func (r *DataBlockReader) BlockReader() *BlockReader {
func (r *DataBlockDecoder) BlockReader() *BlockReader {
return &r.r
}

// Init initializes the data block reader with the given serialized data block.
func (r *DataBlockReader) Init(schema KeySchema, data []byte) {
func (r *DataBlockDecoder) Init(schema KeySchema, data []byte) {
r.r.Init(data, dataBlockCustomHeaderSize)
r.trailers = r.r.Uints(len(schema.ColumnTypes) + dataBlockColumnTrailer)
r.prefixChanged = r.r.Bitmap(len(schema.ColumnTypes) + dataBlockColumnPrefixChanged)
Expand All @@ -789,7 +788,7 @@ func (r *DataBlockReader) Init(schema KeySchema, data []byte) {

// Describe descirbes the binary format of the data block, assuming f.Offset()
// is positioned at the beginning of the same data block described by r.
func (r *DataBlockReader) Describe(f *binfmt.Formatter) {
func (r *DataBlockDecoder) Describe(f *binfmt.Formatter) {
// Set the relative offset. When loaded into memory, the beginning of blocks
// are aligned. Padding that ensures alignment is done relative to the
// current offset. Setting the relative offset ensures that if we're
Expand Down Expand Up @@ -828,7 +827,7 @@ type DataBlockIter struct {
// -- Fields that are initialized for each block --
// For any changes to these fields, InitHandle should be updated.

r *DataBlockReader
r *DataBlockDecoder
h block.BufferHandle
maxRow int
transforms block.IterTransforms
Expand All @@ -849,7 +848,7 @@ type DataBlockIter struct {
// iteration.
nextObsoletePoint int

readerAlloc DataBlockReader
readerAlloc DataBlockDecoder
}

// InitOnce configures the data block iterator's key schema and lazy value
Expand All @@ -869,7 +868,7 @@ func (i *DataBlockIter) InitOnce(

// Init initializes the data block iterator, configuring it to read from the
// provided reader.
func (i *DataBlockIter) Init(r *DataBlockReader, transforms block.IterTransforms) error {
func (i *DataBlockIter) Init(r *DataBlockDecoder, transforms block.IterTransforms) error {
i.r = r
// Leave i.h unchanged.
numRows := int(r.r.header.Rows)
Expand Down Expand Up @@ -1364,6 +1363,6 @@ func (i *DataBlockIter) Close() error {
i.h = block.BufferHandle{}
i.transforms = block.IterTransforms{}
i.kv = base.InternalKV{}
i.readerAlloc = DataBlockReader{}
i.readerAlloc = DataBlockDecoder{}
return nil
}
6 changes: 3 additions & 3 deletions sstable/colblk/data_block_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,8 +25,8 @@ var testKeysSchema = DefaultKeySchema(testkeys.Comparer, 16)

func TestDataBlock(t *testing.T) {
var buf bytes.Buffer
var w DataBlockWriter
var r DataBlockReader
var w DataBlockEncoder
var r DataBlockDecoder
var it DataBlockIter
rw := NewDataBlockRewriter(testKeysSchema, testkeys.Comparer.Compare, testkeys.Comparer.Split)
var sizes []int
Expand Down Expand Up @@ -149,7 +149,7 @@ func benchmarkDataBlockWriter(b *testing.B, prefixSize, valueSize int) {
rng := rand.New(rand.NewSource(seed))
keys, values := makeTestKeyRandomKVs(rng, prefixSize, valueSize, targetBlockSize)

var w DataBlockWriter
var w DataBlockEncoder
w.Init(testKeysSchema)
b.ResetTimer()

Expand Down
2 changes: 1 addition & 1 deletion sstable/colblk_writer.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@ type RawColumnWriter struct {
props Properties
// block writers buffering unflushed data.
dataBlock struct {
colblk.DataBlockWriter
colblk.DataBlockEncoder
// numDeletions stores the count of point tombstones in this data block.
// It's used to determine if this data block is considered
// tombstone-dense for the purposes of compaction.
Expand Down
2 changes: 1 addition & 1 deletion sstable/colblk_writer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -115,7 +115,7 @@ func describeSSTableBinary(f *binfmt.Formatter, schema colblk.KeySchema) error {
f.HexBytesln(block.TrailerLen, "%s block trailer", bh.Name)
continue
case "data":
var r colblk.DataBlockReader
var r colblk.DataBlockDecoder
// NB: The byte slice used to Init must be aligned (like an
// allocated block would be in practice).
r.Init(schema, aligned.Copy(f.Data()[bh.Offset:bh.Offset+bh.Length]))
Expand Down
2 changes: 1 addition & 1 deletion sstable/layout.go
Original file line number Diff line number Diff line change
Expand Up @@ -322,7 +322,7 @@ func formatColblkDataBlock(
data []byte,
fmtRecord func(key *base.InternalKey, value []byte),
) error {
var reader colblk.DataBlockReader
var reader colblk.DataBlockDecoder
reader.Init(r.keySchema, data)
f := binfmt.New(data)
f.SetLinePrefix(" ")
Expand Down

0 comments on commit 3f7527f

Please sign in to comment.