rowblk

package
v2.0.2 Latest Latest
Warning

This package is not in the latest version of its module.

Go to latest
Published: Jan 22, 2025 License: BSD-3-Clause Imports: 20 Imported by: 0

Documentation

Overview

Package rowblk defines facilities for row-oriented sstable blocks.

Index

Constants

View Source
const (
	// MaximumSize is an extremely generous maximum block size of 256MiB. We
	// explicitly place this limit to reserve a few bits in the restart for internal
	// use.
	MaximumSize = 1 << 28
	// EmptySize holds the size of an empty block. Every block ends in a uint32
	// trailer encoding the number of restart points within the block.
	EmptySize = 4
)
View Source
const (
	// TrailerObsoleteBit is a bit within the internal key trailer that's used
	// by the row-oriented block format to signify when a key is obsolete. It's
	// internal to the row-oriented block format, set when writing a block and
	// unset by blockIter, so no code outside block writing/reading code ever
	// sees it.
	//
	// TODO(jackson): Unexport once the blockIter is also in this package.
	TrailerObsoleteBit = base.InternalKeyTrailer(base.InternalKeyKindSSTableInternalObsoleteBit)
	// TrailerObsoleteMask defines a mask for the obsolete bit in the internal
	// key trailer.
	TrailerObsoleteMask = (base.InternalKeyTrailer(base.SeqNumMax) << 8) | base.InternalKeyTrailer(base.InternalKeyKindSSTableInternalObsoleteMask)
)

Variables

This section is empty.

Functions

func NewFragmentIter

func NewFragmentIter(
	fileNum base.DiskFileNum,
	comparer *base.Comparer,
	blockHandle block.BufferHandle,
	transforms block.FragmentIterTransforms,
) (keyspan.FragmentIterator, error)

NewFragmentIter returns a new keyspan iterator that iterates over a block's spans.

Types

type DescribeKV

type DescribeKV func(w io.Writer, key *base.InternalKey, val []byte, enc KVEncoding)

DescribeKV is a function that formats a key-value pair, writing the description to w.

type IndexIter

type IndexIter struct {
	// contains filtered or unexported fields
}

IndexIter is a lightweight adapter that implements block.IndexIterator for a row-based index block.

func (*IndexIter) BlockHandleWithProperties

func (i *IndexIter) BlockHandleWithProperties() (block.HandleWithProperties, error)

BlockHandleWithProperties decodes the block handle with any encoded properties at the iterator's current position.

func (*IndexIter) Close

func (i *IndexIter) Close() error

Close closes the iterator, releasing any resources it holds.

func (*IndexIter) First

func (i *IndexIter) First() bool

First seeks index iterator to the first block entry. It returns false if the index block is empty.

func (*IndexIter) Handle

func (i *IndexIter) Handle() block.BufferHandle

Handle returns the underlying block buffer handle, if the iterator was initialized with one.

func (*IndexIter) Init

func (i *IndexIter) Init(c *base.Comparer, blk []byte, transforms block.IterTransforms) error

Init initializes an iterator from the provided block data slice.

func (*IndexIter) InitHandle

func (i *IndexIter) InitHandle(
	comparer *base.Comparer, block block.BufferHandle, transforms block.IterTransforms,
) error

InitHandle initializes an iterator from the provided block handle.

func (*IndexIter) Invalidate

func (i *IndexIter) Invalidate()

Invalidate invalidates the block iterator, removing references to the block it was initialized with.

func (*IndexIter) IsDataInvalidated

func (i *IndexIter) IsDataInvalidated() bool

IsDataInvalidated returns true when the blockIter has been invalidated using an invalidate call. NB: this is different from blockIter.Valid which is part of the InternalIterator implementation.

func (*IndexIter) Last

func (i *IndexIter) Last() bool

Last seeks index iterator to the last block entry. It returns false if the index block is empty.

func (*IndexIter) Next

func (i *IndexIter) Next() bool

Next steps the index iterator to the next block entry. It returns false if the index block is exhausted.

func (*IndexIter) Prev

func (i *IndexIter) Prev() bool

Prev steps the index iterator to the previous block entry. It returns false if the index block is exhausted.

func (*IndexIter) SeekGE

func (i *IndexIter) SeekGE(key []byte) bool

SeekGE seeks the index iterator to the first block entry with a separator key greater or equal to the given key. If it returns true, the iterator is positioned over the first block that might contain the key [key], and following blocks have keys ≥ Separator(). It returns false if the seek key is greater than all index block separators.

func (*IndexIter) Separator

func (i *IndexIter) Separator() []byte

Separator returns the separator at the iterator's current position. The iterator must be positioned at a valid row. A Separator is a user key guaranteed to be greater than or equal to every key contained within the referenced block(s).

func (*IndexIter) SeparatorGT

func (i *IndexIter) SeparatorGT(key []byte, inclusively bool) bool

SeparatorGT returns true if the separator at the iterator's current position is strictly greater than (or equal, if orEqual=true) the provided key.

func (*IndexIter) SeparatorLT

func (i *IndexIter) SeparatorLT(key []byte) bool

SeparatorLT returns true if the separator at the iterator's current position is strictly less than the provided key.

func (*IndexIter) Valid

func (i *IndexIter) Valid() bool

Valid returns true if the iterator is currently positioned at a valid block handle.

type Iter

type Iter struct {
	// contains filtered or unexported fields
}

Iter is an iterator over a single block of data.

An Iter provides an additional guarantee around key stability when a block has a restart interval of 1 (i.e. when there is no prefix compression). Key stability refers to whether the InternalKey.UserKey bytes returned by a positioning call will remain stable after a subsequent positioning call. The normal case is that a positioning call will invalidate any previously returned InternalKey.UserKey. If a block has a restart interval of 1 (no prefix compression), Iter guarantees that InternalKey.UserKey will point to the key as stored in the block itself which will remain valid until the Iter is closed. The key stability guarantee is used by the range tombstone and range key code, which knows that the respective blocks are always encoded with a restart interval of 1. This per-block key stability guarantee is sufficient for range tombstones and range deletes as they are always encoded in a single block. Note: this stability guarantee no longer holds for a block iter with synthetic prefix/suffix replacement, but we don't use the synthetic suffix/prefix functionality of Iter for range keys.

An Iter also provides a value stability guarantee for range deletions and range keys since there is only a single range deletion and range key block per sstable and the Iter will not release the bytes for the block until it is closed.

Note on why Iter knows about lazyValueHandling:

Iter's positioning functions (that return a LazyValue), are too complex to inline even prior to lazyValueHandling. Iter.Next and Iter.First were by far the cheapest and had costs 195 and 180 respectively, which exceeds the budget of 80. We initially tried to keep the lazyValueHandling logic out of Iter by wrapping it with a lazyValueDataBlockIter. singleLevelIter and twoLevelIter would use this wrapped iter. The functions in lazyValueDataBlockIter were simple, in that they called the corresponding Iter func and then decided whether the value was in fact in-place (so return immediately) or needed further handling. But these also turned out too costly for mid-stack inlining since simple calls like the following have a high cost that is barely under the budget of 80

k, v := i.data.SeekGE(key, flags)  // cost 74
k, v := i.data.Next()              // cost 72

We have 2 options for minimizing performance regressions:

  • Include the lazyValueHandling logic in the already non-inlineable Iter functions: Since most of the time is spent in data block iters, it is acceptable to take the small hit of unnecessary branching (which hopefully branch prediction will predict correctly) for other kinds of blocks.
  • Duplicate the logic of singleLevelIterator and twoLevelIterator for the v3 sstable and only use the aforementioned lazyValueDataBlockIter for a v3 sstable. We would want to manage these copies via code generation.

We have picked the first option here.

func NewIter

func NewIter(
	cmp base.Compare,
	suffixCmp base.ComparePointSuffixes,
	split base.Split,
	block []byte,
	transforms block.IterTransforms,
) (*Iter, error)

NewIter constructs a new row-oriented block iterator over the provided serialized block.

func (*Iter) Close

func (i *Iter) Close() error

Close implements internalIterator.Close, as documented in the pebble package.

func (*Iter) DebugTree

func (i *Iter) DebugTree(tp treeprinter.Node)

DebugTree is part of the InternalIterator interface.

func (*Iter) Describe

func (i *Iter) Describe(tp treeprinter.Node, fmtKV DescribeKV)

Describe describes the contents of a block, writing the description to w. It invokes fmtKV to describe each key-value pair.

func (*Iter) Error

func (i *Iter) Error() error

Error implements internalIterator.Error, as documented in the pebble package.

func (*Iter) First

func (i *Iter) First() *base.InternalKV

First implements internalIterator.First, as documented in the pebble package.

func (*Iter) Handle

func (i *Iter) Handle() block.BufferHandle

Handle returns the underlying block buffer handle, if the iterator was initialized with one.

func (*Iter) Init

func (i *Iter) Init(
	cmp base.Compare,
	suffixCmp base.ComparePointSuffixes,
	split base.Split,
	blk []byte,
	transforms block.IterTransforms,
) error

Init initializes the block iterator from the provided block.

func (*Iter) InitHandle

func (i *Iter) InitHandle(
	comparer *base.Comparer, block block.BufferHandle, transforms block.IterTransforms,
) error

InitHandle initializes an iterator from the provided block handle. NB: two cases of hideObsoletePoints:

  • Local sstable iteration: syntheticSeqNum will be set iff the sstable was ingested.
  • Foreign sstable iteration: syntheticSeqNum is always set.

func (*Iter) Invalidate

func (i *Iter) Invalidate()

Invalidate invalidates the block iterator, removing references to the block it was initialized with.

func (*Iter) IsDataInvalidated

func (i *Iter) IsDataInvalidated() bool

IsDataInvalidated returns true when the blockIter has been invalidated using an invalidate call. NB: this is different from blockIter.Valid which is part of the InternalIterator implementation.

func (*Iter) IsLowerBound

func (i *Iter) IsLowerBound(k []byte) bool

IsLowerBound implements the block.DataBlockIterator interface.

func (*Iter) KV

func (i *Iter) KV() *base.InternalKV

KV returns the internal KV at the current iterator position.

func (*Iter) Key

func (i *Iter) Key() *base.InternalKey

Key returns the internal key at the current iterator position.

func (*Iter) Last

func (i *Iter) Last() *base.InternalKV

Last implements internalIterator.Last, as documented in the pebble package.

func (*Iter) Next

func (i *Iter) Next() *base.InternalKV

Next implements internalIterator.Next, as documented in the pebble package.

func (*Iter) NextPrefix

func (i *Iter) NextPrefix(succKey []byte) *base.InternalKV

NextPrefix implements (base.InternalIterator).NextPrefix.

func (*Iter) Prev

func (i *Iter) Prev() *base.InternalKV

Prev implements internalIterator.Prev, as documented in the pebble package.

func (*Iter) SeekGE

func (i *Iter) SeekGE(key []byte, flags base.SeekGEFlags) *base.InternalKV

SeekGE implements internalIterator.SeekGE, as documented in the pebble package.

func (*Iter) SeekLT

func (i *Iter) SeekLT(key []byte, flags base.SeekLTFlags) *base.InternalKV

SeekLT implements internalIterator.SeekLT, as documented in the pebble package.

func (*Iter) SeekPrefixGE

func (i *Iter) SeekPrefixGE(prefix, key []byte, flags base.SeekGEFlags) *base.InternalKV

SeekPrefixGE implements internalIterator.SeekPrefixGE, as documented in the pebble package.

func (*Iter) SetBounds

func (i *Iter) SetBounds(lower, upper []byte)

SetBounds implements base.InternalIterator. It panics, as bounds should always be handled the by the parent sstable iterator.

func (*Iter) SetContext

func (i *Iter) SetContext(_ context.Context)

SetContext implements base.InternalIterator.

func (*Iter) SetGetLazyValuer

func (i *Iter) SetGetLazyValuer(g block.GetLazyValueForPrefixAndValueHandler)

SetGetLazyValuer sets the value block reader the iterator should use to get lazy values when the value encodes a value prefix.

func (*Iter) SetHasValuePrefix

func (i *Iter) SetHasValuePrefix(hasValuePrefix bool)

SetHasValuePrefix sets whether or not the block iterator should expect values corresponding to Set keys to have a prefix byte.

func (*Iter) String

func (i *Iter) String() string

String implements fmt.Stringer.

func (*Iter) Valid

func (i *Iter) Valid() bool

Valid returns true if the iterator is currently positioned at a valid KV.

func (*Iter) Value

func (i *Iter) Value() base.LazyValue

Value returns the value at the current iterator position.

type KVEncoding

type KVEncoding struct {
	// IsRestart is true if the key is a restart point.
	IsRestart bool
	// Offset is the position within the block at which the key-value pair is
	// encoded.
	Offset offsetInBlock
	// Length is the total length of the KV pair as it is encoded in the block
	// format.
	Length int32
	// KeyShared is the number of bytes this KV's user key shared with its predecessor.
	KeyShared uint32
	// KeyUnshared is the number of bytes this KV's user key did not share with
	// its predecessor.
	KeyUnshared uint32
	// ValueLen is the length of the internal value.
	ValueLen uint32
}

KVEncoding describes the encoding of a key-value pair within the block.

type RawIter

type RawIter struct {
	// contains filtered or unexported fields
}

RawIter is an iterator over a single block of data. Unlike blockIter, keys are stored in "raw" format (i.e. not as internal keys). Note that there is significant similarity between this code and the code in blockIter. Yet reducing duplication is difficult due to the blockIter being performance critical. RawIter must only be used for blocks where the value is stored together with the key.

func NewRawIter

func NewRawIter(cmp base.Compare, block []byte) (*RawIter, error)

NewRawIter constructs a new raw block iterator.

func (*RawIter) Close

func (i *RawIter) Close() error

Close implements internalIterator.Close, as documented in the pebble package.

func (*RawIter) DebugTree

func (i *RawIter) DebugTree(tp treeprinter.Node)

DebugTree is part of the InternalIterator interface.

func (*RawIter) Describe

func (i *RawIter) Describe(tp treeprinter.Node, fmtKV DescribeKV)

Describe describes the contents of a block, writing the description to w. It invokes fmtKV to describe each key-value pair.

func (*RawIter) Error

func (i *RawIter) Error() error

Error implements internalIterator.Error, as documented in the pebble package.

func (*RawIter) First

func (i *RawIter) First() bool

First implements internalIterator.First, as documented in the pebble package.

func (*RawIter) Init

func (i *RawIter) Init(cmp base.Compare, blk []byte) error

Init initializes the raw block iterator.

func (*RawIter) Key

func (i *RawIter) Key() base.InternalKey

Key implements internalIterator.Key, as documented in the pebble package.

func (*RawIter) Last

func (i *RawIter) Last() bool

Last implements internalIterator.Last, as documented in the pebble package.

func (*RawIter) Next

func (i *RawIter) Next() bool

Next implements internalIterator.Next, as documented in the pebble package.

func (*RawIter) Prev

func (i *RawIter) Prev() bool

Prev implements internalIterator.Prev, as documented in the pebble package.

func (*RawIter) SeekGE

func (i *RawIter) SeekGE(key []byte) bool

SeekGE implements internalIterator.SeekGE, as documented in the pebble package.

func (*RawIter) Valid

func (i *RawIter) Valid() bool

Valid implements internalIterator.Valid, as documented in the pebble package.

func (*RawIter) Value

func (i *RawIter) Value() []byte

Value implements internalIterator.Value, as documented in the pebble package.

type Rewriter

type Rewriter struct {
	// contains filtered or unexported fields
}

Rewriter may be used to rewrite row-based blocks.

func NewRewriter

func NewRewriter(comparer *base.Comparer, restartInterval int) *Rewriter

NewRewriter constructs a new rewriter.

func (*Rewriter) RewriteSuffixes

func (r *Rewriter) RewriteSuffixes(
	input []byte, from []byte, to []byte,
) (start, end base.InternalKey, rewritten []byte, err error)

RewriteSuffixes rewrites the input block. It expects the input block to only contain keys with the suffix `from`. It rewrites the block to contain the same keys with the suffix `to`.

RewriteSuffixes returns the start and end keys of the rewritten block, and the finished rewritten block.

type Writer

type Writer struct {
	// RestartInterval configures the interval at which the writer will write a
	// full key without prefix compression, and encode a corresponding restart
	// point.
	RestartInterval int
	// contains filtered or unexported fields
}

Writer buffers and serializes key/value pairs into a row-oriented block.

func (*Writer) Add

func (w *Writer) Add(key base.InternalKey, value []byte)

Add adds a key value pair to the block without a value prefix.

func (*Writer) AddRaw

func (w *Writer) AddRaw(key, value []byte)

AddRaw adds a key value pair to the block.

func (*Writer) AddRawString

func (w *Writer) AddRawString(key string, value []byte)

AddRawString is AddRaw but with a string key.

func (*Writer) AddWithOptionalValuePrefix

func (w *Writer) AddWithOptionalValuePrefix(
	key base.InternalKey,
	isObsolete bool,
	value []byte,
	maxSharedKeyLen int,
	addValuePrefix bool,
	valuePrefix block.ValuePrefix,
	setHasSameKeyPrefix bool,
)

AddWithOptionalValuePrefix adds a key value pair to the block, optionally including a value prefix.

Callers that always set addValuePrefix to false should use add() instead.

isObsolete indicates whether this key-value pair is obsolete in this sstable (only applicable when writing data blocks) -- see the comment in table.go and the longer one in format.go. addValuePrefix adds a 1 byte prefix to the value, specified in valuePrefix -- this is used for data blocks in TableFormatPebblev3 onwards for SETs (see the comment in format.go, with more details in value_block.go). setHasSameKeyPrefix is also used in TableFormatPebblev3 onwards for SETs.

func (*Writer) CurKey

func (w *Writer) CurKey() base.InternalKey

CurKey returns the most recently written key.

func (*Writer) CurUserKey

func (w *Writer) CurUserKey() []byte

CurUserKey returns the most recently written user key.

func (*Writer) CurValue

func (w *Writer) CurValue() []byte

CurValue returns the most recently written value.

func (*Writer) EntryCount

func (w *Writer) EntryCount() int

EntryCount returns the count of entries written to the writer.

func (*Writer) EstimatedSize

func (w *Writer) EstimatedSize() int

EstimatedSize returns the estimated size of the block in bytes.

func (*Writer) Finish

func (w *Writer) Finish() []byte

Finish finalizes the block, serializes it and returns the serialized data.

func (*Writer) Reset

func (w *Writer) Reset()

Reset resets the block writer to empty, preserving buffers for reuse.

Jump to

Keyboard shortcuts

? : This menu
/ : Search site
f or F : Jump to
y or Y : Canonical URL