block

package
v2.1.4 Latest Latest
Warning

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

Go to latest
Published: Jan 21, 2026 License: BSD-3-Clause Imports: 32 Imported by: 1

Documentation

Index

Constants

View Source
const CategoryMax = 30

CategoryMax is the maximum value of a category, and is also the maximum number of categories that can be registered.

View Source
const MetadataSize = 336

MetadataSize is the size of the metadata. The value is chosen to fit a colblk.DataBlockDecoder and a CockroachDB colblk.KeySeeker.

View Source
const TrailerLen = 5

TrailerLen is the length of the trailer at the end of a block.

Variables

View Source
var (
	NoCompression     = simpleCompressionProfile("NoCompression", compression.None)
	SnappyCompression = simpleCompressionProfile("Snappy", compression.Snappy)
	ZstdCompression   = simpleCompressionProfile("ZSTD", compression.ZstdLevel3)
	MinLZCompression  = simpleCompressionProfile("MinLZ", compression.MinLZFastest)

	DefaultCompression = SnappyCompression
	FastestCompression = simpleCompressionProfile("Fastest", fastestCompression)

	FastCompression = registerCompressionProfile(CompressionProfile{
		Name:                           "Fast",
		DataBlocks:                     fastestCompression,
		ValueBlocks:                    compression.ZstdLevel1,
		OtherBlocks:                    fastestCompression,
		MinReductionPercent:            10,
		AdaptiveReductionCutoffPercent: 30,
	})

	BalancedCompression = registerCompressionProfile(CompressionProfile{
		Name:                           "Balanced",
		DataBlocks:                     compression.ZstdLevel1,
		ValueBlocks:                    compression.ZstdLevel1,
		OtherBlocks:                    fastestCompression,
		MinReductionPercent:            5,
		AdaptiveReductionCutoffPercent: 20,
	})

	GoodCompression = registerCompressionProfile(CompressionProfile{
		Name:                           "Good",
		DataBlocks:                     compression.ZstdLevel3,
		ValueBlocks:                    compression.ZstdLevel3,
		OtherBlocks:                    fastestCompression,
		MinReductionPercent:            5,
		AdaptiveReductionCutoffPercent: 10,
	})
)
View Source
var NoFragmentTransforms = FragmentIterTransforms{}

NoFragmentTransforms is the default value for IterTransforms.

View Source
var NoReadEnv = ReadEnv{}

NoReadEnv is the empty ReadEnv which reports no stats and does not use a buffer pool.

View Source
var NoTransforms = IterTransforms{}

NoTransforms is the default value for IterTransforms.

Functions

func CastMetadata added in v2.1.0

func CastMetadata[T any](md *Metadata) *T

CastMetadata casts the provided metadata to the type parameter T. If the Metadata has not already been initialized, callers should use CastMetadataZero.

func CastMetadataZero added in v2.1.0

func CastMetadataZero[T any](md *Metadata) *T

CastMetadataZero casts the provided metadata to the type parameter T, zeroing the memory backing the metadata first. This zeroing is necessary when first initializing the data structure to ensure that the Go garbage collector doesn't misinterpret any of T's pointer fields, falsely detecting them as invalid pointers.

func CompressAndChecksumToTempBuffer added in v2.1.0

func CompressAndChecksumToTempBuffer(
	blockData []byte, blockKind Kind, compressor *Compressor, checksummer *Checksummer,
) (PhysicalBlock, *TempBuffer)

CompressAndChecksumToTempBuffer compresses and checksums the provided block into a TempBuffer. The caller should Release() the TempBuffer once it is no longer necessary.

func CopyAndChecksumToTempBuffer added in v2.1.0

func CopyAndChecksumToTempBuffer(
	blockData []byte, blockKind Kind, compressor *Compressor, checksummer *Checksummer,
) (PhysicalBlock, *TempBuffer)

CopyAndChecksumToTempBuffer copies (without compressing) and checksums the provided block into a TempBuffer. The caller should Release() the TempBuffer once it is no longer necessary.

func DecompressInto

func DecompressInto(ci CompressionIndicator, compressed []byte, buf []byte) error

DecompressInto decompresses compressed into buf. The buf slice must have the exact size as the decompressed value. Callers may use DecompressedLen to determine the correct size.

func DecompressedLen

func DecompressedLen(ci CompressionIndicator, b []byte) (decompressedLen int, err error)

DecompressedLen returns the length of the provided block once decompressed, allowing the caller to allocate a buffer exactly sized to the decompressed payload.

func DeterministicReadBlockDurationForTesting added in v2.1.0

func DeterministicReadBlockDurationForTesting() func()

DeterministicReadBlockDurationForTesting is for tests that want a deterministic value of the time to read a block (that is not in the cache). The return value is a function that must be called before the test exits.

func ReadRaw added in v2.1.0

func ReadRaw(
	ctx context.Context,
	f objstorage.Readable,
	readHandle objstorage.ReadHandle,
	logger base.LoggerAndTracer,
	fileNum base.DiskFileNum,
	buf []byte,
	off int64,
) ([]byte, error)

ReadRaw reads len(buf) bytes from the provided Readable at the given offset into buf. It's used to read the footer of a table.

func ValidateChecksum added in v2.1.0

func ValidateChecksum(checksumType ChecksumType, b []byte, bh Handle) error

ValidateChecksum validates the checksum of a block.

Types

type AllocedBuffer

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

AllocedBuffer is an allocated memory buffer.

type Buf

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

A Buf holds a reference to a manually-managed, pooled byte buffer.

func (*Buf) Release

func (b *Buf) Release()

Release releases the buffer back to the pool.

func (Buf) Valid

func (b Buf) Valid() bool

Valid returns true if the buf holds a valid buffer.

type BufferHandle

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

A BufferHandle is a handle to manually-managed memory. The handle may point to a block in the block cache (h.cv != nil), or a buffer that exists outside the block cache allocated from a BufferPool (b.Valid()).

func CacheBufferHandle

func CacheBufferHandle(cv *cache.Value) BufferHandle

CacheBufferHandle constructs a BufferHandle from a block cache Handle.

func (BufferHandle) BlockData

func (bh BufferHandle) BlockData() []byte

BlockData retrieves the buffer for the block data.

func (BufferHandle) BlockMetadata

func (bh BufferHandle) BlockMetadata() *Metadata

BlockMetadata returns the buffer for the block metadata.

func (BufferHandle) Release

func (bh BufferHandle) Release()

Release releases the buffer, either back to the block cache or BufferPool. It is okay to call Release on a zero-value BufferHandle (to no effect).

func (BufferHandle) Valid

func (bh BufferHandle) Valid() bool

Valid returns true if the BufferHandle holds a value.

type BufferPool

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

A BufferPool holds a pool of buffers for holding sstable blocks. An initial size of the pool is provided on Init, but a BufferPool will grow to meet the largest working set size. It'll never shrink. When a buffer is released, the BufferPool recycles the buffer for future allocations.

A BufferPool should only be used for short-lived allocations with well-understood working set sizes to avoid excessive memory consumption.

BufferPool is not thread-safe.

func (*BufferPool) Alloc

func (p *BufferPool) Alloc(n int) Buf

Alloc allocates a new buffer of size n. If the pool already holds a buffer at least as large as n, the pooled buffer is used instead.

Alloc is O(MAX(N,M)) where N is the largest number of concurrently in-use buffers allocated and M is the initialSize passed to Init.

func (*BufferPool) Init

func (p *BufferPool) Init(initialSize int)

Init initializes the pool with an initial working set buffer size of `initialSize`.

func (*BufferPool) InitPreallocated

func (p *BufferPool) InitPreallocated(pool []AllocedBuffer)

InitPreallocated is like Init but for internal sstable package use in instances where a pre-allocated slice of []allocedBuffer already exists. It's used to avoid an extra allocation initializing BufferPool.pool.

func (*BufferPool) Release

func (p *BufferPool) Release()

Release releases all buffers held by the pool and resets the pool to an uninitialized state.

type Category added in v2.1.0

type Category uint8

Category is a user-understandable string, where stats are aggregated for each category. The cardinality of this should be low, say < 20. The prefix "pebble-" is reserved for internal Pebble categories.

Examples of categories that can be useful in the CockroachDB context are: sql-user, sql-stats, raft, rangefeed, mvcc-gc, range-snapshot.

const CategoryUnknown Category = 0

CategoryUnknown is the unknown category. It has the latency-sensitive QoS level.

func Categories added in v2.1.0

func Categories() []Category

Categories returns all registered categories, including CategoryUnknown.

Can only be called after all categories have been registered. Calling RegisterCategory() after Categories() will result in a panic.

func RegisterCategory added in v2.1.0

func RegisterCategory(name string, qosLevel QoSLevel) Category

RegisterCategory registers a new category. Each category has a name and an associated QoS level. The category name must be unique.

Only CategoryMax categories can be registered in total.

func StringToCategoryForTesting added in v2.1.0

func StringToCategoryForTesting(s string) Category

StringToCategoryForTesting returns the Category for the string, or panics if the string is not known.

func (Category) QoSLevel added in v2.1.0

func (c Category) QoSLevel() QoSLevel

QoSLevel returns the QoSLevel associated with this Category.

func (Category) SafeFormat added in v2.1.0

func (c Category) SafeFormat(p redact.SafePrinter, verb rune)

SafeFormat implements the redact.SafeFormatter interface.

func (Category) String added in v2.1.0

func (c Category) String() string

type CategoryStats added in v2.1.0

type CategoryStats struct {
	// BlockBytes is the bytes in the loaded blocks. If the block was
	// compressed, this is the compressed bytes. Currently, only the index
	// blocks, data blocks containing points, and filter blocks are included.
	// Additionally, value blocks read after the corresponding iterator is
	// closed are not included.
	BlockBytes uint64
	// BlockBytesInCache is the subset of BlockBytes that were in the block
	// cache.
	BlockBytesInCache uint64
	// BlockReadDuration is the total duration to read the bytes not in the
	// cache, i.e., BlockBytes-BlockBytesInCache.
	BlockReadDuration time.Duration
}

CategoryStats provides stats about a category of reads.

type CategoryStatsAggregate added in v2.1.0

type CategoryStatsAggregate struct {
	Category      Category
	CategoryStats CategoryStats
}

CategoryStatsAggregate is the aggregate for the given category.

type CategoryStatsCollector added in v2.1.0

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

CategoryStatsCollector collects and aggregates the stats per category.

func (*CategoryStatsCollector) Accumulator added in v2.1.0

func (c *CategoryStatsCollector) Accumulator(p uint64, category Category) *CategoryStatsShard

Accumulator returns a stats accumulator for the given category. The provided p is used to detrmine which shard to write stats to.

func (*CategoryStatsCollector) GetStats added in v2.1.0

GetStats returns the aggregated stats.

type CategoryStatsShard added in v2.1.0

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

CategoryStatsShard holds CategoryStats with a mutex to ensure safe access.

func (*CategoryStatsShard) Accumulate added in v2.1.0

func (c *CategoryStatsShard) Accumulate(
	blockBytes, blockBytesInCache uint64, blockReadDuration time.Duration,
)

Accumulate implements the IterStatsAccumulator interface.

type ChecksumType

type ChecksumType byte

ChecksumType specifies the checksum used for blocks.

const (
	ChecksumTypeNone     ChecksumType = 0
	ChecksumTypeCRC32c   ChecksumType = 1
	ChecksumTypeXXHash   ChecksumType = 2
	ChecksumTypeXXHash64 ChecksumType = 3
)

The available checksum types. These values are part of the durable format and should not be changed.

func (ChecksumType) String

func (t ChecksumType) String() string

String implements fmt.Stringer.

type Checksummer

type Checksummer struct {
	Type ChecksumType
	// contains filtered or unexported fields
}

A Checksummer calculates checksums for blocks.

func (*Checksummer) Checksum

func (c *Checksummer) Checksum(block []byte, blockType byte) (checksum uint32)

Checksum computes a checksum over the provided block and block type.

func (*Checksummer) Init added in v2.1.0

func (c *Checksummer) Init(typ ChecksumType)

type CompressionIndicator

type CompressionIndicator byte

CompressionIndicator is the byte stored physically within the block.Trailer to indicate the compression type.

TODO(jackson): Avoid exporting once all compression and decompression is delegated to the block package.

const (
	NoCompressionIndicator     CompressionIndicator = 0
	SnappyCompressionIndicator CompressionIndicator = 1
	ZlibCompressionIndicator   CompressionIndicator = 2
	Bzip2CompressionIndicator  CompressionIndicator = 3
	Lz4CompressionIndicator    CompressionIndicator = 4
	Lz4hcCompressionIndicator  CompressionIndicator = 5
	XpressCompressionIndicator CompressionIndicator = 6
	ZstdCompressionIndicator   CompressionIndicator = 7
	MinLZCompressionIndicator  CompressionIndicator = 8
)

The block type gives the per-block compression format. These constants are part of the file format and should not be changed. They are different from the Compression constants because the latter are designed so that the zero value of the Compression type means to use the default compression (which is snappy). Not all compression types listed here are supported.

func (CompressionIndicator) Algorithm added in v2.1.0

func (CompressionIndicator) String

func (i CompressionIndicator) String() string

String implements fmt.Stringer.

type CompressionProfile added in v2.1.0

type CompressionProfile struct {
	Name string

	// DataBlocks applies to sstable data blocks.
	// ValueBlocks applies to sstable value blocks and blob file value blocks.
	// OtherBlocks applies to all other blocks (such as index, filter, metadata
	// blocks).
	//
	// Some blocks (like rangedel) never use compression; this is at the
	// discretion of the sstable or blob file writer.
	//
	// Note that MinLZ is only supported with table formats v6+. Older formats
	// fall back to Snappy.
	DataBlocks  compression.Setting
	ValueBlocks compression.Setting
	OtherBlocks compression.Setting

	// Blocks that are reduced by less than this percentage are stored
	// uncompressed.
	MinReductionPercent uint8

	// AdaptiveReductionCutoffPercent (when set to a non-zero value) enables
	// adaptive compressors for data and value blocks which fall back to the
	// OtherBlocks setting. The OtherBlocks setting is used when the
	// DataBlocks/ValueBlocks setting cannot achieve a further data reduction of
	// at least AdaptiveReductionCutoffPercent%.
	AdaptiveReductionCutoffPercent uint8
}

CompressionProfile contains the parameters for compressing blocks in an sstable or blob file.

CompressionProfile is a more advanced successor to Compression.

func CompressionProfileByName added in v2.1.0

func CompressionProfileByName(name string) *CompressionProfile

CompressionProfileByName returns the built-in compression profile with the given name, or nil if there is no such profile. It is case-insensitive.

The caller must gracefully handle the nil return case as an unknown (user-defined or deprecated) profile.

func (*CompressionProfile) UsesMinLZ added in v2.1.0

func (p *CompressionProfile) UsesMinLZ() bool

UsesMinLZ returns true if the profile uses the MinLZ compression algorithm (for any block kind).

type CompressionStats added in v2.1.0

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

CompressionStats collects compression statistics for a single file - the total compressed and uncompressed sizes for each distinct compression.Setting used.

func (CompressionStats) All added in v2.1.0

All returns an iterator over the collected stats, in arbitrary order.

func (*CompressionStats) MergeWith added in v2.1.0

func (c *CompressionStats) MergeWith(other *CompressionStats)

MergeWith updates the receiver stats to include the other stats.

func (CompressionStats) String added in v2.1.0

func (c CompressionStats) String() string

String returns a string representation of the stats, in the format: "<setting1>:<compressed1>/<uncompressed1>,<setting2>:<compressed2>/<uncompressed2>,..."

type CompressionStatsForSetting added in v2.1.0

type CompressionStatsForSetting struct {
	Setting           compression.Setting
	UncompressedBytes uint64
	CompressedBytes   uint64
}

type Compressor added in v2.1.0

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

Compressor is used to compress blocks. Typical usage:

c := MakeCompressor(profile)
.. = c.Compress(..)
.. = c.Compress(..)
c.Close()

func MakeCompressor added in v2.1.0

func MakeCompressor(profile *CompressionProfile) Compressor

MakeCompressor returns a Compressor that applies the given compression profile. Close must be called when the compressor is no longer needed.

func (*Compressor) Close added in v2.1.0

func (c *Compressor) Close()

Close must be called when the Compressor is no longer needed. After Close is called, the Compressor must not be used again.

func (*Compressor) Compress added in v2.1.0

func (c *Compressor) Compress(dst, src []byte, kind Kind) (CompressionIndicator, []byte)

Compress a block, appending the compressed data to dst[:0].

In addition to the buffer, returns the algorithm that was used.

func (*Compressor) Stats added in v2.1.0

func (c *Compressor) Stats() *CompressionStats

Stats returns the compression stats. The result can only be used until the next call to the Compressor.

func (*Compressor) UncompressedBlock added in v2.1.0

func (c *Compressor) UncompressedBlock(size int, kind Kind)

UncompressedBlock informs the compressor that a block of the given size and kind was written uncompressed. This is used so that the final statistics are complete.

type DataBlockIterator

type DataBlockIterator interface {
	base.InternalIterator

	// Handle returns the handle to the block.
	Handle() BufferHandle
	// InitHandle initializes the block from the provided buffer handle.
	//
	// The iterator takes ownership of the BufferHandle and releases it when it is
	// closed (or re-initialized with another handle). This happens even in error
	// cases.
	InitHandle(*base.Comparer, BufferHandle, IterTransforms) error
	// Valid returns true if the iterator is currently positioned at a valid KV.
	Valid() bool
	// KV returns the key-value pair at the current iterator position. The
	// iterator must be Valid().
	KV() *base.InternalKV
	// IsLowerBound returns true if all keys produced by this iterator are >= the
	// given key. The function is best effort; false negatives are allowed.
	//
	// If IsLowerBound is true then Compare(First().UserKey, k) >= 0.
	//
	// If the iterator produces no keys (i.e. First() is nil), IsLowerBound can
	// return true for any key.
	IsLowerBound(k []byte) bool
	// Invalidate invalidates the block iterator, removing references to the
	// block it was initialized with. The iterator may continue to be used after
	// a call to Invalidate, but all positioning methods should return false.
	// Valid() must also return false.
	Invalidate()
	// IsDataInvalidated returns true when the iterator has been invalidated
	// using an Invalidate call.
	//
	// NB: this is different from Valid which indicates whether the current *KV*
	// is valid.
	IsDataInvalidated() bool
}

DataBlockIterator is a type constraint for implementations of block iterators over data blocks. It's implemented by *rowblk.Iter and *colblk.DataBlockIter.

type Decompressor added in v2.1.0

type Decompressor = compression.Decompressor

func GetDecompressor added in v2.1.0

func GetDecompressor(c CompressionIndicator) Decompressor

type FlushGovernor

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

FlushGovernor is used to decide when to flush a block. It takes into consideration a target block size and (optionally) allocation size classes.

When allocation size classes are used, we use the allocation class that is closest to the target block size. We also take into account the next allocation class and use it if it reduces internal fragmentation.

func MakeFlushGovernor

func MakeFlushGovernor(
	targetBlockSize int,
	blockSizeThreshold int,
	sizeClassAwareThreshold int,
	allocatorSizeClasses []int,
) FlushGovernor

MakeFlushGovernor initializes a flush controller.

There are two cases:

1. No allocation classes. If we don't have any allocatorSizeClasses, or targetBlockSize doesn't fit between two allocation classes, then we flush right before the block would exceed targetBlockSize (except if the block size would be smaller than blockSizeThreshold percent of the target, in which case we flush right after the target block size is exceeded).

2. With allocation classes. We take into account allocation size classes no smaller than sizeClassAwareThreshold percent of the target block size and up to the first class that fits the target block size. We flush near allocation class boundaries to minimize wasted memory space in the block cache (internal fragmentation).

The FlushGovernor is immutable and can be copied by value.

func (*FlushGovernor) LowWatermark

func (fg *FlushGovernor) LowWatermark() int

LowWatermark returns the minimum size of a block that could be flushed. ShouldFlush will never return true if sizeBefore is below the low watermark.

This can be used in a "fast path" check that uses an easy-to-compute overestimation of the block size.

func (*FlushGovernor) ShouldFlush

func (fg *FlushGovernor) ShouldFlush(sizeBefore int, sizeAfter int) bool

ShouldFlush returns true if we should flush the current block of sizeBefore instead of adding another KV that would increase the block to sizeAfter.

func (FlushGovernor) String

func (fg FlushGovernor) String() string

type FragmentIterTransforms

type FragmentIterTransforms struct {
	SyntheticSeqNum          SyntheticSeqNum
	SyntheticPrefixAndSuffix SyntheticPrefixAndSuffix
}

FragmentIterTransforms allow on-the-fly transformation of range deletion or range key data at iteration time.

func (*FragmentIterTransforms) HasSyntheticPrefix

func (t *FragmentIterTransforms) HasSyntheticPrefix() bool

func (*FragmentIterTransforms) HasSyntheticSuffix

func (t *FragmentIterTransforms) HasSyntheticSuffix() bool

func (*FragmentIterTransforms) NoTransforms

func (t *FragmentIterTransforms) NoTransforms() bool

NoTransforms returns true if there are no transforms enabled.

func (*FragmentIterTransforms) SyntheticPrefix

func (t *FragmentIterTransforms) SyntheticPrefix() []byte

func (*FragmentIterTransforms) SyntheticSuffix

func (t *FragmentIterTransforms) SyntheticSuffix() []byte

type GetInternalValueForPrefixAndValueHandler added in v2.1.0

type GetInternalValueForPrefixAndValueHandler interface {
	// GetInternalValueForPrefixAndValueHandle returns a InternalValue for the
	// given value prefix and value.
	//
	// The result is only valid until the next call to
	// GetInternalValueForPrefixAndValueHandle. Use InternalValue.Clone if the
	// lifetime of the InternalValue needs to be extended. For more details, see
	// the "memory management" comment where LazyValue is declared.
	GetInternalValueForPrefixAndValueHandle(handle []byte) base.InternalValue
}

GetInternalValueForPrefixAndValueHandler is an interface for getting an InternalValue from a value prefix and value.

type Handle

type Handle struct {
	// Offset identifies the offset of the block within the file.
	Offset uint64
	// Length is the length of the block data (excludes the trailer).
	Length uint64
}

Handle is the file offset and length of a block.

func DecodeHandle

func DecodeHandle(src []byte) (Handle, int)

DecodeHandle returns the block handle encoded in a variable-width encoding at the start of src, as well as the number of bytes it occupies. It returns zero if given invalid input. A block handle for a data block or a first/lower level index block should not be decoded using DecodeHandle since the caller may validate that the number of bytes decoded is equal to the length of src, which will be false if the properties are not decoded. In those cases the caller should use DecodeHandleWithProperties.

func (Handle) EncodeVarints

func (h Handle) EncodeVarints(dst []byte) int

EncodeVarints encodes the block handle into dst using a variable-width encoding and returns the number of bytes written.

func (Handle) String added in v2.1.0

func (h Handle) String() string

String implements fmt.Stringer.

type HandleWithProperties

type HandleWithProperties struct {
	Handle
	Props []byte
}

HandleWithProperties is used for data blocks and first/lower level index blocks, since they can be annotated using BlockPropertyCollectors.

func DecodeHandleWithProperties

func DecodeHandleWithProperties(src []byte) (HandleWithProperties, error)

DecodeHandleWithProperties returns the block handle and properties encoded in a variable-width encoding at the start of src. src needs to be exactly the length that was encoded. This method must be used for data block and first/lower level index blocks. The properties in the block handle point to the bytes in src.

func (HandleWithProperties) EncodeVarints

func (h HandleWithProperties) EncodeVarints(dst []byte) []byte

EncodeVarints encodes the block handle and properties into dst using a variable-width encoding and returns the number of bytes written.

type IndexBlockIterator

type IndexBlockIterator interface {
	// Init initializes the block iterator from the provided block.
	Init(*base.Comparer, []byte, IterTransforms) error
	// InitHandle initializes an iterator from the provided block handle.
	//
	// The iterator takes ownership of the BufferHandle and releases it when it is
	// closed (or re-initialized with another handle). This happens even in error
	// cases.
	InitHandle(*base.Comparer, BufferHandle, IterTransforms) error
	// Valid returns true if the iterator is currently positioned at a valid
	// block handle.
	Valid() bool
	// IsDataInvalidated returns true when the iterator has been invalidated
	// using an Invalidate call.
	//
	// NB: this is different from Valid which indicates whether the iterator is
	// currently positioned over a valid block entry.
	IsDataInvalidated() bool
	// Invalidate invalidates the block iterator, removing references to the
	// block it was initialized with. The iterator may continue to be used after
	// a call to Invalidate, but all positioning methods should return false.
	// Valid() must also return false.
	Invalidate()
	// Handle returns the underlying block buffer handle, if the iterator was
	// initialized with one.
	Handle() BufferHandle
	// 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).
	Separator() []byte
	// SeparatorLT returns true if the separator at the iterator's current
	// position is strictly less than the provided key. For some
	// implementations, it may be more performant to call SeparatorLT rather
	// than explicitly performing Compare(Separator(), key) < 0.
	SeparatorLT(key []byte) 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. For some implementations, it may be more performant to call
	// SeparatorGT rather than explicitly performing a comparison using the key
	// returned by Separator.
	SeparatorGT(key []byte, orEqual bool) bool
	// BlockHandleWithProperties decodes the block handle with any encoded
	// properties at the iterator's current position.
	BlockHandleWithProperties() (HandleWithProperties, error)
	// 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.
	SeekGE(key []byte) bool
	// First seeks index iterator to the first block entry. It returns false if
	// the index block is empty.
	First() bool
	// Last seeks index iterator to the last block entry. It returns false if
	// the index block is empty.
	Last() bool
	// Next steps the index iterator to the next block entry. It returns false
	// if the index block is exhausted in the forward direction. A call to Next
	// while already exhausted in the forward direction is a no-op.
	Next() bool
	// Prev steps the index iterator to the previous block entry. It returns
	// false if the index block is exhausted in the reverse direction. A call to
	// Prev while already exhausted in the reverse direction is a no-op.
	Prev() bool
	// Close closes the iterator, releasing any resources it holds. After Close,
	// the iterator must be reset such that it could be reused after a call to
	// Init or InitHandle.
	Close() error
}

IndexBlockIterator is an interface for implementations of block iterators over index blocks. It's implemented by *rowblk.IndexIter and *colblk.IndexBlockIter.

type IterTransforms

type IterTransforms struct {
	// SyntheticSeqNum, if set, overrides the sequence number in all keys. It is
	// set if the sstable was ingested or it is foreign.
	SyntheticSeqNum SyntheticSeqNum
	// HideObsoletePoints, if true, skips over obsolete points during iteration.
	// This is the norm when the sstable is foreign or the largest sequence number
	// of the sstable is below the one we are reading.
	HideObsoletePoints bool

	SyntheticPrefixAndSuffix SyntheticPrefixAndSuffix
}

IterTransforms allow on-the-fly transformation of data at iteration time.

These transformations could in principle be implemented as block transforms (at least for non-virtual sstables), but applying them during iteration is preferable.

func (*IterTransforms) HasSyntheticPrefix

func (t *IterTransforms) HasSyntheticPrefix() bool

func (*IterTransforms) HasSyntheticSuffix

func (t *IterTransforms) HasSyntheticSuffix() bool

func (*IterTransforms) NoTransforms

func (t *IterTransforms) NoTransforms() bool

NoTransforms returns true if there are no transforms enabled.

func (*IterTransforms) SyntheticPrefix

func (t *IterTransforms) SyntheticPrefix() []byte

func (*IterTransforms) SyntheticSuffix

func (t *IterTransforms) SyntheticSuffix() []byte

type Kind added in v2.1.0

type Kind = blockkind.Kind

Kind is a convenience alias.

type Metadata

type Metadata [MetadataSize]byte

Metadata is an in-memory buffer that stores metadata for a block. It is allocated together with the buffer storing the block and is initialized once when the block is read from disk.

Portions of this buffer can be cast to the structures we need (through CastMetadata[Zero]), but note that any pointers in these structures should be considered invisible to the GC for the purpose of preserving lifetime. Pointers to the block's data buffer are ok, since the metadata and the data have the same lifetime (sharing the underlying allocation).

type PhysicalBlock

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

PhysicalBlock represents a block (possibly compressed) as it is stored physically on disk, including its trailer.

func CompressAndChecksum

func CompressAndChecksum(
	dst *[]byte, blockData []byte, blockKind Kind, compressor *Compressor, checksummer *Checksummer,
) PhysicalBlock

CompressAndChecksum compresses and checksums the provided block, returning the compressed block and its trailer. The result is appended to the dst argument.

func CopyAndChecksum added in v2.1.0

func CopyAndChecksum(
	dst *[]byte, blockData []byte, blockKind Kind, compressor *Compressor, checksummer *Checksummer,
) PhysicalBlock

CopyAndChecksum copies the provided block (without compressing it) and checksums it, returning the physical block. The result is appended to the dst argument.

Note that we still need to provide a Compressor so we can inform it of the uncompressed block (for statistics).

func NewPhysicalBlock

func NewPhysicalBlock(data []byte) PhysicalBlock

NewPhysicalBlock returns a new PhysicalBlock with the provided block data. The trailer is set from the last TrailerLen bytes of the block. The data could be compressed.

func (PhysicalBlock) Clone

func (b PhysicalBlock) Clone() PhysicalBlock

Clone returns a deep copy of the block.

func (*PhysicalBlock) CloneWithByteAlloc

func (b *PhysicalBlock) CloneWithByteAlloc(a *bytealloc.A) PhysicalBlock

CloneWithByteAlloc returns a deep copy of the block, using the provided bytealloc.A to allocate memory for the new copy.

func (*PhysicalBlock) LengthWithTrailer

func (b *PhysicalBlock) LengthWithTrailer() int

LengthWithTrailer returns the length of the data block, including the trailer.

func (*PhysicalBlock) LengthWithoutTrailer

func (b *PhysicalBlock) LengthWithoutTrailer() int

LengthWithoutTrailer returns the length of the data block, excluding the trailer.

func (*PhysicalBlock) WriteTo

func (b *PhysicalBlock) WriteTo(w objstorage.Writable) (n int, err error)

WriteTo writes the block (including its trailer) to the provided Writable. If err == nil, n is the number of bytes successfully written to the Writable.

WriteTo might mangle the block data.

type QoSLevel added in v2.1.0

type QoSLevel uint8

QoSLevel describes whether the read is latency-sensitive or not. Each category must map to a single QoSLevel. While category strings are opaque to Pebble, the QoSLevel may be internally utilized in Pebble to better optimize future reads.

const (
	// LatencySensitiveQoSLevel is the default when QoSLevel is not specified,
	// and represents reads that are latency-sensitive.
	LatencySensitiveQoSLevel QoSLevel = iota
	// NonLatencySensitiveQoSLevel represents reads that are not
	// latency-sensitive.
	NonLatencySensitiveQoSLevel
)

func StringToQoSForTesting added in v2.1.0

func StringToQoSForTesting(s string) QoSLevel

StringToQoSForTesting returns the QoSLevel for the string, or panics if the string is not known.

func (QoSLevel) SafeFormat added in v2.1.0

func (q QoSLevel) SafeFormat(p redact.SafePrinter, verb rune)

SafeFormat implements the redact.SafeFormatter interface.

type ReadEnv added in v2.1.0

type ReadEnv struct {
	// stats and iterStats are slightly different. stats is a shared struct
	// supplied from the outside, and represents stats for the whole iterator
	// tree and can be reset from the outside (e.g. when the pebble.Iterator is
	// being reused). It is currently only provided when the iterator tree is
	// rooted at pebble.Iterator. iterStats contains an sstable iterator's
	// private stats that are reported to a CategoryStatsCollector when this
	// iterator is closed. In the important code paths, the CategoryStatsCollector
	// is managed by the fileCacheContainer.
	Stats     *base.InternalIteratorStats
	IterStats *CategoryStatsShard

	// BufferPool is not-nil if we read blocks into a buffer pool and not into the
	// cache. This is used during compactions.
	BufferPool *BufferPool

	// ReportCorruptionFn is called with ReportCorruptionArg and the error
	// whenever an SSTable corruption is detected. The argument is used to avoid
	// allocating a separate function for each object. It returns an error with
	// more details.
	ReportCorruptionFn  func(opaque any, err error) error
	ReportCorruptionArg any
}

ReadEnv contains arguments used when reading a block which apply to all the block reads performed by a higher-level operation.

func (*ReadEnv) BlockRead added in v2.1.0

func (env *ReadEnv) BlockRead(blockLength uint64, readDuration time.Duration)

BlockRead updates the stats when a block had to be read.

func (*ReadEnv) BlockServedFromCache added in v2.1.0

func (env *ReadEnv) BlockServedFromCache(blockLength uint64)

BlockServedFromCache updates the stats when a block was found in the cache.

type Reader added in v2.1.0

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

A Reader reads blocks from a single file, handling caching, checksum validation and decompression.

func (*Reader) ChecksumType added in v2.1.0

func (r *Reader) ChecksumType() ChecksumType

ChecksumType returns the checksum type used by the reader.

func (*Reader) Close added in v2.1.0

func (r *Reader) Close() error

Close releases resources associated with the Reader.

func (*Reader) FileNum added in v2.1.0

func (r *Reader) FileNum() base.DiskFileNum

FileNum returns the file number of the file being read.

func (*Reader) GetFromCache added in v2.1.0

func (r *Reader) GetFromCache(bh Handle) *cache.Value

GetFromCache retrieves the block from the cache, if it is present.

Users should prefer using Read, which handles reading from object storage on a cache miss.

func (*Reader) Init added in v2.1.0

func (r *Reader) Init(readable objstorage.Readable, ro ReaderOptions, checksumType ChecksumType)

Init initializes the Reader to read blocks from the provided Readable.

func (*Reader) Read added in v2.1.0

func (r *Reader) Read(
	ctx context.Context,
	env ReadEnv,
	readHandle objstorage.ReadHandle,
	bh Handle,
	kind Kind,
	initBlockMetadataFn func(*Metadata, []byte) error,
) (handle BufferHandle, _ error)

Read reads the block referenced by the provided handle. The readHandle is optional.

func (*Reader) Readable added in v2.1.0

func (r *Reader) Readable() objstorage.Readable

Readable returns the underlying objstorage.Readable.

Users should avoid accessing the underlying Readable if it can be avoided.

func (*Reader) UsePreallocatedReadHandle added in v2.1.0

func (r *Reader) UsePreallocatedReadHandle(
	readBeforeSize objstorage.ReadBeforeSize, rh *objstorageprovider.PreallocatedReadHandle,
) objstorage.ReadHandle

UsePreallocatedReadHandle returns a ReadHandle that reads from the reader and uses the provided preallocated read handle to back the read handle, avoiding an unnecessary allocation.

type ReaderOptions added in v2.1.0

type ReaderOptions struct {
	// CacheOpts contains the information needed to interact with the block
	// cache.
	CacheOpts sstableinternal.CacheOptions
	// LoadBlockSema, if set, is used to limit the number of blocks that can be
	// loaded (i.e. read from the filesystem) in parallel. Each load acquires
	// one unit from the semaphore for the duration of the read.
	LoadBlockSema *fifo.Semaphore
	// LoggerAndTracer is an optional logger and tracer.
	LoggerAndTracer base.LoggerAndTracer
}

ReaderOptions configures a block reader.

type SyntheticPrefix

type SyntheticPrefix []byte

SyntheticPrefix represents a byte slice that is implicitly prepended to every key in a file being read or accessed by a reader. Note that since the byte slice is prepended to every KV rather than replacing a byte prefix, the result of prepending the synthetic prefix must be a full, valid key while the partial key physically stored within the sstable need not be a valid key according to user key semantics.

Note that elsewhere we use the language of 'prefix' to describe the user key portion of a MVCC key, as defined by the Comparer's base.Split method. The SyntheticPrefix is related only in that it's a byte prefix that is incorporated into the logical MVCC prefix.

The table's bloom filters are constructed only on the partial keys physically stored in the table, but interactions with the file including seeks and reads will all behave as if the file had been constructed from keys that include the synthetic prefix. Note that all Compare operations will act on a partial key (before any prepending), so the Comparer must support comparing these partial keys.

The synthetic prefix will never modify key metadata stored in the key suffix.

NB: Since this transformation currently only applies to point keys, a block with range keys cannot be iterated over with a synthetic prefix.

func (SyntheticPrefix) Apply

func (sp SyntheticPrefix) Apply(key []byte) []byte

Apply prepends the synthetic prefix to a key.

func (SyntheticPrefix) Invert

func (sp SyntheticPrefix) Invert(key []byte) []byte

Invert removes the synthetic prefix from a key.

func (SyntheticPrefix) IsSet

func (sp SyntheticPrefix) IsSet() bool

IsSet returns true if the synthetic prefix is not enpty.

type SyntheticPrefixAndSuffix

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

SyntheticPrefixAndSuffix is a more compact way of representing both a synthetic prefix and a synthetic suffix. See SyntheticPrefix and SyntheticSuffix.

The zero value is valid, representing no synthetic prefix or suffix.

func MakeSyntheticPrefixAndSuffix

func MakeSyntheticPrefixAndSuffix(
	prefix SyntheticPrefix, suffix SyntheticSuffix,
) SyntheticPrefixAndSuffix

MakeSyntheticPrefixAndSuffix returns a SyntheticPrefixAndSuffix with the given prefix and suffix.

func (SyntheticPrefixAndSuffix) HasPrefix

func (ps SyntheticPrefixAndSuffix) HasPrefix() bool

HasPrefix returns true if ps contains a non-empty synthetic prefix.

func (SyntheticPrefixAndSuffix) HasSuffix

func (ps SyntheticPrefixAndSuffix) HasSuffix() bool

HasSuffix returns true if ps contains a non-empty synthetic suffix.

func (SyntheticPrefixAndSuffix) IsUnset

func (ps SyntheticPrefixAndSuffix) IsUnset() bool

IsUnset returns true if HasPrefix() and HasSuffix() both return false.

func (SyntheticPrefixAndSuffix) Prefix

Prefix returns the synthetic prefix.

func (SyntheticPrefixAndSuffix) PrefixLen

func (ps SyntheticPrefixAndSuffix) PrefixLen() uint32

PrefixLen returns the length of the synthetic prefix, or 0 if it is not set.

func (SyntheticPrefixAndSuffix) RemoveSuffix

RemoveSuffix returns a SyntheticPrefixAndSuffix that has the same prefix as the receiver but no suffix.

func (SyntheticPrefixAndSuffix) Suffix

Suffix returns the synthetic suffix.

func (SyntheticPrefixAndSuffix) SuffixLen

func (ps SyntheticPrefixAndSuffix) SuffixLen() uint32

SuffixLen returns the length of the synthetic prefix, or 0 if it is not set.

type SyntheticSeqNum

type SyntheticSeqNum base.SeqNum

SyntheticSeqNum is used to override all sequence numbers in a table. It is set to a non-zero value when the table was created externally and ingested whole.

const NoSyntheticSeqNum SyntheticSeqNum = 0

NoSyntheticSeqNum is the default zero value for SyntheticSeqNum, which disables overriding the sequence number.

type SyntheticSuffix

type SyntheticSuffix []byte

SyntheticSuffix will replace every suffix of every point key surfaced during block iteration. A synthetic suffix can be used if:

  1. no two keys in the sst share the same prefix; and
  2. pebble.Compare(prefix + replacementSuffix, prefix + originalSuffix) < 0, for all keys in the backing sst which have a suffix (i.e. originalSuffix is not empty).

Range dels are not supported when synthetic suffix is used.

For range keys, the synthetic suffix applies to the suffix that is part of RangeKeySet - if it is non-empty, it is replaced with the SyntheticSuffix. RangeKeyUnset keys are not supported when a synthetic suffix is used.

func (SyntheticSuffix) IsSet

func (ss SyntheticSuffix) IsSet() bool

IsSet returns true if the synthetic suffix is not empty.

type TempBuffer added in v2.1.0

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

TempBuffer is a buffer that is used temporarily and is released back to a pool for reuse.

func NewTempBuffer added in v2.1.0

func NewTempBuffer() *TempBuffer

NewTempBuffer returns a TempBuffer from the pool. The buffer will have zero size and length and arbitrary capacity.

func (*TempBuffer) Append added in v2.1.0

func (tb *TempBuffer) Append(v []byte) (startOffset int)

Append appends the contents of v to the buffer, growing the buffer if necessary. Returns the offset at which it was appended.

func (*TempBuffer) Data added in v2.1.0

func (tb *TempBuffer) Data() []byte

Data returns the byte slice currently backing the Buffer.

func (*TempBuffer) Release added in v2.1.0

func (tb *TempBuffer) Release()

Release releases the buffer back to the pool for reuse.

func (*TempBuffer) Reset added in v2.1.0

func (tb *TempBuffer) Reset()

Reset is equivalent to Resize(0).

func (*TempBuffer) Resize added in v2.1.0

func (tb *TempBuffer) Resize(length int)

Resize resizes the buffer to the specified length, allocating if necessary. If the length is longer than the current length, the values of the new bytes are arbitrary.

func (*TempBuffer) Size added in v2.1.0

func (tb *TempBuffer) Size() int

Size returns the current size of the buffer.

type Trailer

type Trailer = [TrailerLen]byte

Trailer is the trailer at the end of a block, encoding the block type (compression) and a checksum.

func MakeTrailer

func MakeTrailer(blockType byte, checksum uint32) (t Trailer)

MakeTrailer constructs a trailer from a block type and a checksum.

type Value

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

Value is a block buffer, either backed by the block cache or a BufferPool.

func Alloc

func Alloc(n int, p *BufferPool) Value

Alloc allocates a new Value for a block of length n (excluding the block trailer, but including an associated Metadata). If bufferPool is non-nil, Alloc allocates the buffer from the pool. Otherwise it allocates it from the block cache.

func (Value) BlockData

func (b Value) BlockData() []byte

BlockData returns the byte slice for the block data.

func (Value) BlockMetadata

func (b Value) BlockMetadata() *Metadata

BlockMetadata returns the block metadata.

func (Value) MakeHandle

func (b Value) MakeHandle() BufferHandle

MakeHandle constructs a BufferHandle from the Value.

func (Value) Release

func (b Value) Release()

Release releases the handle.

func (*Value) SetInCacheForTesting added in v2.1.0

func (b *Value) SetInCacheForTesting(h *cache.Handle, fileNum base.DiskFileNum, offset uint64)

func (Value) Truncate

func (b Value) Truncate(n int)

Truncate truncates the block to n bytes.

type ValuePrefix

type ValuePrefix byte

ValuePrefix is the single byte prefix in values indicating either an in-place value or a value encoding a valueHandle. It encodes multiple kinds of information (see below).

func BlobValueHandlePrefix added in v2.1.0

func BlobValueHandlePrefix(setHasSameKeyPrefix bool, attr base.ShortAttribute) ValuePrefix

BlobValueHandlePrefix returns the ValuePrefix for a blob.

func InPlaceValuePrefix

func InPlaceValuePrefix(setHasSameKeyPrefix bool) ValuePrefix

InPlaceValuePrefix returns the ValuePrefix for an in-place value.

func ValueBlockHandlePrefix added in v2.1.0

func ValueBlockHandlePrefix(setHasSameKeyPrefix bool, attribute base.ShortAttribute) ValuePrefix

ValueBlockHandlePrefix returns the ValuePrefix for a valblk.Handle.

func (ValuePrefix) IsBlobValueHandle added in v2.1.0

func (vp ValuePrefix) IsBlobValueHandle() bool

IsBlobValueHandle returns true if the ValuePrefix is for a blob.

func (ValuePrefix) IsInPlaceValue added in v2.1.0

func (vp ValuePrefix) IsInPlaceValue() bool

IsInPlaceValue returns true if the ValuePrefix is for an in-place value.

func (ValuePrefix) IsValueBlockHandle added in v2.1.0

func (vp ValuePrefix) IsValueBlockHandle() bool

IsValueBlockHandle returns true if the ValuePrefix is for a valblk.Handle.

func (ValuePrefix) SetHasSamePrefix

func (vp ValuePrefix) SetHasSamePrefix() bool

SetHasSamePrefix returns true if the ValuePrefix encodes that the key is a set with the same prefix as the preceding key which also is a set.

func (ValuePrefix) ShortAttribute

func (vp ValuePrefix) ShortAttribute() base.ShortAttribute

ShortAttribute returns the user-defined base.ShortAttribute encoded in the ValuePrefix.

REQUIRES: !IsInPlaceValue()

Directories

Path Synopsis

Jump to

Keyboard shortcuts

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