Documentation ¶
Overview ¶
Package rowblk defines facilities for row-oriented sstable blocks.
Index ¶
- Constants
- func NewFragmentIter(fileNum base.DiskFileNum, comparer *base.Comparer, ...) (keyspan.FragmentIterator, error)
- type DescribeKV
- type IndexIter
- func (i *IndexIter) BlockHandleWithProperties() (block.HandleWithProperties, error)
- func (i *IndexIter) Close() error
- func (i *IndexIter) First() bool
- func (i *IndexIter) Handle() block.BufferHandle
- func (i *IndexIter) Init(c *base.Comparer, blk []byte, transforms block.IterTransforms) error
- func (i *IndexIter) InitHandle(comparer *base.Comparer, block block.BufferHandle, ...) error
- func (i *IndexIter) Invalidate()
- func (i *IndexIter) IsDataInvalidated() bool
- func (i *IndexIter) Last() bool
- func (i *IndexIter) Next() bool
- func (i *IndexIter) Prev() bool
- func (i *IndexIter) SeekGE(key []byte) bool
- func (i *IndexIter) Separator() []byte
- func (i *IndexIter) SeparatorGT(key []byte, inclusively bool) bool
- func (i *IndexIter) SeparatorLT(key []byte) bool
- func (i *IndexIter) Valid() bool
- type Iter
- func (i *Iter) Close() error
- func (i *Iter) DebugTree(tp treeprinter.Node)
- func (i *Iter) Describe(tp treeprinter.Node, fmtKV DescribeKV)
- func (i *Iter) Error() error
- func (i *Iter) First() *base.InternalKV
- func (i *Iter) Handle() block.BufferHandle
- func (i *Iter) Init(cmp base.Compare, suffixCmp base.ComparePointSuffixes, split base.Split, ...) error
- func (i *Iter) InitHandle(comparer *base.Comparer, block block.BufferHandle, ...) error
- func (i *Iter) Invalidate()
- func (i *Iter) IsDataInvalidated() bool
- func (i *Iter) IsLowerBound(k []byte) bool
- func (i *Iter) KV() *base.InternalKV
- func (i *Iter) Key() *base.InternalKey
- func (i *Iter) Last() *base.InternalKV
- func (i *Iter) Next() *base.InternalKV
- func (i *Iter) NextPrefix(succKey []byte) *base.InternalKV
- func (i *Iter) Prev() *base.InternalKV
- func (i *Iter) SeekGE(key []byte, flags base.SeekGEFlags) *base.InternalKV
- func (i *Iter) SeekLT(key []byte, flags base.SeekLTFlags) *base.InternalKV
- func (i *Iter) SeekPrefixGE(prefix, key []byte, flags base.SeekGEFlags) *base.InternalKV
- func (i *Iter) SetBounds(lower, upper []byte)
- func (i *Iter) SetContext(_ context.Context)
- func (i *Iter) SetGetLazyValuer(g block.GetLazyValueForPrefixAndValueHandler)
- func (i *Iter) SetHasValuePrefix(hasValuePrefix bool)
- func (i *Iter) String() string
- func (i *Iter) Valid() bool
- func (i *Iter) Value() base.LazyValue
- type KVEncoding
- type RawIter
- func (i *RawIter) Close() error
- func (i *RawIter) DebugTree(tp treeprinter.Node)
- func (i *RawIter) Describe(tp treeprinter.Node, fmtKV DescribeKV)
- func (i *RawIter) Error() error
- func (i *RawIter) First() bool
- func (i *RawIter) Init(cmp base.Compare, blk []byte) error
- func (i *RawIter) Key() base.InternalKey
- func (i *RawIter) Last() bool
- func (i *RawIter) Next() bool
- func (i *RawIter) Prev() bool
- func (i *RawIter) SeekGE(key []byte) bool
- func (i *RawIter) Valid() bool
- func (i *RawIter) Value() []byte
- type Rewriter
- type Writer
- func (w *Writer) Add(key base.InternalKey, value []byte)
- func (w *Writer) AddRaw(key, value []byte)
- func (w *Writer) AddRawString(key string, value []byte)
- func (w *Writer) AddWithOptionalValuePrefix(key base.InternalKey, isObsolete bool, value []byte, maxSharedKeyLen int, ...)
- func (w *Writer) CurKey() base.InternalKey
- func (w *Writer) CurUserKey() []byte
- func (w *Writer) CurValue() []byte
- func (w *Writer) EntryCount() int
- func (w *Writer) EstimatedSize() int
- func (w *Writer) Finish() []byte
- func (w *Writer) Reset()
Constants ¶
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 )
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) First ¶
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) 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 ¶
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 ¶
Last seeks index iterator to the last block entry. It returns false if the index block is empty.
func (*IndexIter) Next ¶
Next steps the index iterator to the next block entry. It returns false if the index block is exhausted.
func (*IndexIter) Prev ¶
Prev steps the index iterator to the previous block entry. It returns false if the index block is exhausted.
func (*IndexIter) SeekGE ¶
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 ¶
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 ¶
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 ¶
SeparatorLT returns true if the separator at the iterator's current position is strictly less than the provided key.
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) 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) 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 ¶
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 ¶
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 ¶
SetBounds implements base.InternalIterator. It panics, as bounds should always be handled the by the parent sstable iterator.
func (*Iter) SetContext ¶
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 ¶
SetHasValuePrefix sets whether or not the block iterator should expect values corresponding to Set keys to have a prefix byte.
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 uint32 // 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 ¶
NewRawIter constructs a new raw block iterator.
func (*RawIter) Close ¶
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 ¶
Error implements internalIterator.Error, as documented in the pebble package.
func (*RawIter) First ¶
First implements internalIterator.First, as documented in the pebble package.
func (*RawIter) Key ¶
func (i *RawIter) Key() base.InternalKey
Key implements internalIterator.Key, as documented in the pebble package.
func (*RawIter) SeekGE ¶
SeekGE implements internalIterator.SeekGE, 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 ¶
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) AddRawString ¶
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 ¶
CurUserKey returns the most recently written user key.
func (*Writer) EntryCount ¶
EntryCount returns the count of entries written to the writer.
func (*Writer) EstimatedSize ¶
EstimatedSize returns the estimated size of the block in bytes.