manifest

package
v2.1.1 Latest Latest
Warning

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

Go to latest
Published: Oct 23, 2025 License: BSD-3-Clause Imports: 29 Imported by: 0

Documentation

Index

Constants

View Source
const NumLevels = 7

NumLevels is the number of levels a Version contains.

Variables

View Source
var NumFilesAnnotator = SumAnnotator(func(f *TableMetadata) (uint64, bool) {
	return 1, true
})

NumFilesAnnotator is an Annotator which computes an annotation value equal to the number of files included in the annotation. Particularly, it can be used to efficiently calculate the number of files in a given key range using range annotations.

Functions

func CheckOrdering

func CheckOrdering(comparer *base.Comparer, level Layer, files LevelIterator) error

CheckOrdering checks that the files are consistent with respect to seqnums (for level 0 files -- see detailed comment below) and increasing and non- overlapping internal key ranges (for non-level 0 files).

func ExtendKeyRange added in v2.1.0

func ExtendKeyRange(
	ucmp Compare, bounds base.UserKeyBounds, iters ...iter.Seq[*TableMetadata],
) base.UserKeyBounds

ExtendKeyRange returns the narrowest UserKeyBounds that encompass the provided bounds and the bounds of all the TableMetadata in iters.

func KeyRange

func KeyRange(ucmp Compare, iters ...iter.Seq[*TableMetadata]) base.UserKeyBounds

KeyRange returns the narrowest UserKeyBounds that encompass the bounds of all the TableMetadata in iters.

func SortBySmallest

func SortBySmallest(files []*TableMetadata, cmp Compare)

SortBySmallest sorts the specified files by smallest key using the supplied comparison function to order user keys.

Types

type AggregateBlobFileStats added in v2.1.0

type AggregateBlobFileStats struct {
	// Count is the number of blob files in the set.
	Count uint64
	// PhysicalSize is the sum of the size of all blob files in the set.  This
	// is the size of the blob files on physical storage. Data within blob files
	// is compressed, so this value may be less than ValueSize.
	PhysicalSize uint64
	// ValueSize is the sum of the length of the uncompressed values in all blob
	// files in the set.
	ValueSize uint64
	// ReferencedValueSize is the sum of the length of the uncompressed values
	// in all blob files in the set that are still referenced by live tables
	// (i.e., in the latest version).
	ReferencedValueSize uint64
	// ReferencesCount is the total number of tracked references in live tables
	// (i.e., in the latest version). When virtual sstables are present, this
	// count is per-virtual sstable (not per backing physical sstable).
	ReferencesCount uint64
}

AggregateBlobFileStats records cumulative stats across blob files.

func (AggregateBlobFileStats) String added in v2.1.0

func (s AggregateBlobFileStats) String() string

String implements fmt.Stringer.

type AnnotationAggregator

type AnnotationAggregator[T any] interface {
	// Zero returns the zero value of an annotation. This value is returned
	// when a LevelMetadata is empty. The dst argument, if non-nil, is an
	// obsolete value previously returned by this Annotator and may be
	// overwritten and reused to avoid a memory allocation.
	Zero(dst *T) *T

	// Accumulate computes the annotation for a single file in a level's
	// metadata. It merges the file's value into dst and returns a bool flag
	// indicating whether or not the value is stable and okay to cache as an
	// annotation. If the file's value may change over the life of the file,
	// the annotator must return false.
	//
	// Implementations may modify dst and return it to avoid an allocation.
	Accumulate(f *TableMetadata, dst *T) (v *T, cacheOK bool)

	// Merge combines two values src and dst, returning the result.
	// Implementations may modify dst and return it to avoid an allocation.
	Merge(src *T, dst *T) *T
}

An AnnotationAggregator defines how an annotation should be accumulated from a single TableMetadata and merged with other annotated values.

type Annotator

type Annotator[T any] struct {
	Aggregator AnnotationAggregator[T]
}

An Annotator defines a computation over a level's TableMetadata. If the computation is stable and uses inputs that are fixed for the lifetime of a TableMetadata, the LevelMetadata's internal data structures are annotated with the intermediary computations. This allows the computation to be computed incrementally as edits are applied to a level.

func SumAnnotator

func SumAnnotator(accumulate func(f *TableMetadata) (v uint64, cacheOK bool)) *Annotator[uint64]

SumAnnotator takes a function that computes a uint64 value from a single TableMetadata and returns an Annotator that sums together the values across files.

func (*Annotator[T]) InvalidateLevelAnnotation

func (a *Annotator[T]) InvalidateLevelAnnotation(lm LevelMetadata)

InvalidateLevelAnnotation clears any cached annotations defined by Annotator. A pointer to the Annotator is used as the key for pre-calculated values, so the same Annotator must be used to clear the appropriate cached annotation. Calls to InvalidateLevelAnnotation are *not* concurrent-safe with any other calls to Annotator methods for the same Annotator (concurrent calls from other annotators are fine). Any calls to this function must have some externally-guaranteed mutual exclusion.

func (*Annotator[T]) LevelAnnotation

func (a *Annotator[T]) LevelAnnotation(lm LevelMetadata) *T

LevelAnnotation calculates the annotation defined by this Annotator for all files in the given LevelMetadata. A pointer to the Annotator is used as the key for pre-calculated values, so the same Annotator must be used to avoid duplicate computation.

func (*Annotator[T]) LevelRangeAnnotation

func (a *Annotator[T]) LevelRangeAnnotation(
	cmp base.Compare, lm LevelMetadata, bounds base.UserKeyBounds,
) *T

LevelRangeAnnotation calculates the annotation defined by this Annotator for the files within LevelMetadata which are within the range [lowerBound, upperBound). A pointer to the Annotator is used as the key for pre-calculated values, so the same Annotator must be used to avoid duplicate computation.

func (*Annotator[T]) MultiLevelAnnotation

func (a *Annotator[T]) MultiLevelAnnotation(lms []LevelMetadata) *T

MultiLevelAnnotation calculates the annotation defined by this Annotator for all files across the given levels. A pointer to the Annotator is used as the key for pre-calculated values, so the same Annotator must be used to avoid duplicate computation.

func (*Annotator[T]) VersionRangeAnnotation

func (a *Annotator[T]) VersionRangeAnnotation(v *Version, bounds base.UserKeyBounds) *T

VersionRangeAnnotation calculates the annotation defined by this Annotator for all files within the given Version which are within the range defined by bounds.

type BlobFileMetadata added in v2.1.0

type BlobFileMetadata struct {
	// FileID is a stable identifier for referencing a blob file containing
	// values. It is the same domain as the BlobReference.FileID. Blob
	// references use the FileID to look up the physical blob file containing
	// referenced values.
	FileID base.BlobFileID
	// Physical is the metadata for the physical blob file.
	//
	// If the blob file has been replaced, Physical.FileNum ≠ FileID. Physical
	// is always non-nil.
	Physical *PhysicalBlobFile
}

BlobFileMetadata encapsulates a blob file ID used to identify a particular blob file, and a reference-counted physical blob file. Different Versions may contain different BlobFileMetadata with the same FileID, but if so they necessarily point to different PhysicalBlobFiles.

See the BlobFileSet documentation for more details.

func ParseBlobFileMetadataDebug added in v2.1.0

func ParseBlobFileMetadataDebug(s string) (_ BlobFileMetadata, err error)

ParseBlobFileMetadataDebug parses a BlobFileMetadata from its string representation. This function is intended for use in tests. It's the inverse of BlobFileMetadata.String().

func (BlobFileMetadata) Ref added in v2.1.0

func (m BlobFileMetadata) Ref()

Ref increments the reference count for the physical blob file.

func (BlobFileMetadata) SafeFormat added in v2.1.0

func (m BlobFileMetadata) SafeFormat(w redact.SafePrinter, _ rune)

SafeFormat implements redact.SafeFormatter.

func (BlobFileMetadata) String added in v2.1.0

func (m BlobFileMetadata) String() string

String implements fmt.Stringer.

func (BlobFileMetadata) Unref added in v2.1.0

func (m BlobFileMetadata) Unref(of ObsoleteFilesSet)

Unref decrements the reference count for the physical blob file. If the reference count reaches zero, the blob file is added to the provided obsolete files set.

type BlobFileSet added in v2.1.0

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

BlobFileSet contains a set of blob files that are referenced by a version. It's used to maintain reference counts on blob files still in-use by some referenced version.

It's backed by a copy-on-write B-Tree of BlobFileMetadata keyed by FileID. A version edit that adds or deletes m blob files updates m⋅log(n) nodes in the B-Tree.

Initially a BlobFileMetadata has a FileID that matches the DiskFileNum of the backing physical blob file. However a blob file may be replaced without replacing the referencing TableMetadatas, which is recorded in the BlobFileSet by replacing the old BlobFileMetadata with a different PhysicalBlobFile.

func MakeBlobFileSet added in v2.1.0

func MakeBlobFileSet(entries []BlobFileMetadata) BlobFileSet

MakeBlobFileSet creates a BlobFileSet from the given blob files.

func (*BlobFileSet) All added in v2.1.0

All returns an iterator over all the blob files in the set.

func (*BlobFileSet) Count added in v2.1.0

func (s *BlobFileSet) Count() int

Count returns the number of blob files in the set.

func (*BlobFileSet) Lookup added in v2.1.0

func (s *BlobFileSet) Lookup(fileID base.BlobFileID) (base.DiskFileNum, bool)

Lookup returns the file number of the physical blob file backing the given file ID. It returns false for the second return value if the FileID is not present in the set.

func (*BlobFileSet) LookupPhysical added in v2.1.0

func (s *BlobFileSet) LookupPhysical(fileID base.BlobFileID) (*PhysicalBlobFile, bool)

LookupPhysical returns the *PhysicalBlobFile backing the given file ID. It returns false for the second return value if the FileID is not present in the set.

type BlobReference added in v2.1.0

type BlobReference struct {
	// FileID identifies the referenced blob file. FileID is stable. If a blob
	// file is rewritten and a blob reference is preserved during a compaction,
	// the new sstable's BlobReference will preserve the same FileID.
	FileID base.BlobFileID
	// ValueSize is the sum of the lengths of the uncompressed values within the
	// blob file for which there exists a reference in the sstable. Note that if
	// any of the referencing tables are virtualized tables, the ValueSize may
	// be approximate.
	ValueSize uint64
	// EstimatedPhysicalSize is an estimate of the physical size of the blob
	// reference, in bytes. It's calculated by scaling the blob file's physical
	// size according to the ValueSize of the blob reference relative to the
	// total ValueSize of the blob file.
	EstimatedPhysicalSize uint64
}

A BlobReference describes a sstable's reference to a blob value file. A BlobReference is immutable.

func MakeBlobReference added in v2.1.0

func MakeBlobReference(
	fileID base.BlobFileID, valueSize uint64, phys *PhysicalBlobFile,
) BlobReference

MakeBlobReference creates a BlobReference from the given file ID, value size, and physical blob file.

type BlobReferenceDepth added in v2.1.0

type BlobReferenceDepth int

BlobReferenceDepth is a statistic maintained per-sstable, indicating an upper bound on the number of blob files that a reader scanning the table would need to keep open if they only open and close referenced blob files once. In other words, it's the stack depth of blob files referenced by a sstable. If a flush or compaction rewrites an sstable's values to a new blob file, the resulting sstable has a blob reference depth of 1. When a compaction reuses blob references, the max blob reference depth of the files in each level is used, and then the depth is summed, and assigned to the output. This is a loose upper bound (assuming worst case distribution of keys in all inputs) but avoids tracking key spans for references and using key comparisons.

Because the blob reference depth is the size of the working set of blob files referenced by the table, it cannot exceed the count of distinct blob file references.

Example: Consider a compaction of file f0 from L0 and files f1, f2, f3 from L1, where the former has blob reference depth of 1 and files f1, f2, f3 all happen to have a blob-reference-depth of 1. Say we produce many output files, one of which is f4. We are assuming here that the blobs referenced by f0 whose keys happened to be written to f4 are spread all across the key span of f4. Say keys from f1 and f2 also made their way to f4. Then we will first have keys that refer to blobs referenced by f1,f0 and at some point once we move past the keys of f1, we will have keys that refer to blobs referenced by f2,f0. In some sense, we have a working set of 2 blob files at any point in time, and this is similar to the idea of level stack depth for reads -- hence we adopt the depth terminology. We want to keep this stack depth in check, since locality is important, while allowing it to be higher than 1, since otherwise we will need to rewrite blob files in every compaction (defeating the write amp benefit we are looking for). Similar to the level depth, this simplistic analysis does not take into account distribution of keys involved in the compaction and which of them have blob references. Also the locality is actually better than in this analysis because more of the keys will be from the lower level.

type BlobReferences added in v2.1.0

type BlobReferences []BlobReference

BlobReferences is a slice of BlobReference. The order of the slice is significant and should be maintained. In practice, a sstable's BlobReferences are ordered by earliest appearance within the sstable. The ordering is persisted to the manifest.

func (*BlobReferences) BlobFileIDByID added in v2.1.0

func (br *BlobReferences) BlobFileIDByID(i blob.ReferenceID) base.BlobFileID

BlobFileIDByID returns the BlobFileID for the identified BlobReference.

func (*BlobReferences) IDByBlobFileID added in v2.1.0

func (br *BlobReferences) IDByBlobFileID(fileID base.BlobFileID) (blob.ReferenceID, bool)

IDByBlobFileID returns the reference ID for the given BlobFileID. If the blob file ID is not found, the second return value is false. IDByBlobFileID is linear in the length of the BlobReferences slice.

type BlobRewriteHeuristic added in v2.1.0

type BlobRewriteHeuristic struct {
	// CurrentTime returns the current time.
	CurrentTime func() time.Time
	// MinimumAge is the minimum age of a blob file that is considered for
	// rewrite and replacement.
	//
	// TODO(jackson): Support updating this at runtime. Lowering the value is
	// simple: pop from the recentlyCreated heap and push to the candidates
	// heap. Raising the value is more complex: we would need to iterate over
	// all the blob files in the candidates heap.
	MinimumAge time.Duration
}

BlobRewriteHeuristic configures the heuristic used to determine which blob files should be rewritten and replaced in order to reduce value-separation induced space amplification.

The heuristic divides blob files into three categories:

1. Fully referenced: Blob files that are fully referenced by live tables. 2. Recently created: Blob files with garbage that were recently created. 3. Eligible: Blob files with garbage that are old.

Files in the first category (fully referenced) should never be rewritten, because rewriting them has no impact on space amplification.

Among files that are not fully referenced, the heuristic separates files into files that were recently created (less than MinimumAgeSecs seconds old) and files that are old and eligible for rewrite. We defer rewriting recently created files under the assumption that their references may be removed through ordinary compactions. The threshold for what is considered recent is the MinimumAgeSecs field.

type BlobRewriteHeuristicStats added in v2.1.0

type BlobRewriteHeuristicStats struct {
	CountFilesFullyReferenced int
	CountFilesTooRecent       int
	CountFilesEligible        int
	NextEligible              BlobFileMetadata
	NextEligibleLivePct       float64
	NextRewrite               BlobFileMetadata
	NextRewriteLivePct        float64
}

BlobRewriteHeuristicStats records statistics about the blob rewrite heuristic.

func (BlobRewriteHeuristicStats) String added in v2.1.0

func (s BlobRewriteHeuristicStats) String() string

String implements fmt.Stringer.

type BulkVersionEdit

type BulkVersionEdit struct {
	AddedTables   [NumLevels]map[base.FileNum]*TableMetadata
	DeletedTables [NumLevels]map[base.FileNum]*TableMetadata

	BlobFiles struct {
		// Added holds the metadata of all new blob files introduced within the
		// aggregated version edit, keyed by file number.
		Added map[base.BlobFileID]*PhysicalBlobFile
		// Deleted holds a list of all blob files that became unreferenced by
		// any sstables, making them obsolete within the resulting version (a
		// zombie if still referenced by previous versions). Deleted file
		// numbers must not exist in Added.
		//
		// Deleted is keyed by blob file ID and points to the physical blob file.
		Deleted map[base.BlobFileID]*PhysicalBlobFile
	}

	// AddedFileBacking is a map to support lookup so that we can populate the
	// TableBacking of virtual sstables during manifest replay.
	AddedFileBacking   map[base.DiskFileNum]*TableBacking
	RemovedFileBacking []base.DiskFileNum

	// AllAddedTables maps table number to table metadata for all added sstables
	// from accumulated version edits. AllAddedTables is only populated if set to
	// non-nil by a caller. It must be set to non-nil when replaying version edits
	// read from a MANIFEST (as opposed to VersionEdits constructed in-memory).
	// While replaying a MANIFEST file, VersionEdit.DeletedFiles map entries have
	// nil values, because the on-disk deletion record encodes only the file
	// number. Accumulate uses AllAddedTables to correctly populate the
	// BulkVersionEdit's Deleted field with non-nil *TableMetadata.
	AllAddedTables map[base.FileNum]*TableMetadata

	// MarkedForCompactionCountDiff holds the aggregated count of files
	// marked for compaction added or removed.
	MarkedForCompactionCountDiff int
}

BulkVersionEdit summarizes the files added and deleted from a set of version edits.

INVARIANTS: No file can be added to a level more than once. This is true globally, and also true for all of the calls to Accumulate for a single bulk version edit.

No file can be removed from a level more than once. This is true globally, and also true for all of the calls to Accumulate for a single bulk version edit.

A sstable file must not be added and removed from a given level in the same version edit, and a blob file must not be both added and deleted in the same version edit.

A file that is being removed from a level must have been added to that level before (in a prior version edit). Note that a given file can be deleted from a level and added to another level in a single version edit

func (*BulkVersionEdit) Accumulate

func (b *BulkVersionEdit) Accumulate(ve *VersionEdit) error

Accumulate adds the file addition and deletions in the specified version edit to the bulk edit's internal state.

INVARIANTS: (1) If a table is added to a given level in a call to Accumulate and then removed from that level in a subsequent call, the file will not be present in the resulting BulkVersionEdit.Deleted for that level. (2) If a new table is added and it includes a reference to a blob file, that blob file must either appear in BlobFiles.Added, or the blob file must be referenced by a table deleted in the same bulk version edit.

After accumulation of version edits, the bulk version edit may have information about a file which has been deleted from a level, but it may not have information about the same file added to the same level. The add could've occurred as part of a previous bulk version edit. In this case, the deleted file must be present in BulkVersionEdit.Deleted, at the end of the accumulation, because we need to decrease the refcount of the deleted file in Apply.

func (*BulkVersionEdit) Apply

func (b *BulkVersionEdit) Apply(curr *Version, readCompactionRate int64) (*Version, error)

Apply applies the delta b to the current version to produce a new version. The ordering of tables within the new version is consistent with respect to the comparer.

Apply updates the backing refcounts (Ref/Unref) as files are installed into the levels.

curr may be nil, which is equivalent to a pointer to a zero version.

Not that L0SublevelFiles is not initialized in the returned version; it is the caller's responsibility to set it using L0Organizer.PerformUpdate().

type CompactionState

type CompactionState uint8

CompactionState is the compaction state of a file.

The following shows the valid state transitions:

NotCompacting --> Compacting --> Compacted
      ^               |
      |               |
      +-------<-------+

Input files to a compaction transition to Compacting when a compaction is picked. A file that has finished compacting typically transitions into the Compacted state, at which point it is effectively obsolete ("zombied") and will eventually be removed from the LSM. A file that has been move-compacted will transition from Compacting back into the NotCompacting state, signaling that the file may be selected for a subsequent compaction. A failed compaction will result in all input tables transitioning from Compacting to NotCompacting.

This state is in-memory only. It is not persisted to the manifest.

const (
	CompactionStateNotCompacting CompactionState = iota
	CompactionStateCompacting
	CompactionStateCompacted
)

CompactionStates.

func (CompactionState) String

func (s CompactionState) String() string

String implements fmt.Stringer.

type Compare

type Compare = base.Compare

Compare exports the base.Compare type.

type CurrentBlobFileSet added in v2.1.0

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

CurrentBlobFileSet describes the set of blob files that are currently live in the latest Version. CurrentBlobFileSet is not thread-safe. In practice its use is protected by the versionSet logLock.

func (*CurrentBlobFileSet) ApplyAndUpdateVersionEdit added in v2.1.0

func (s *CurrentBlobFileSet) ApplyAndUpdateVersionEdit(ve *VersionEdit) error

ApplyAndUpdateVersionEdit applies a version edit to the current blob file set, updating its internal tracking of extant blob file references. If after applying the version edit a blob file has no more references, the version edit is modified to record the blob file removal.

func (*CurrentBlobFileSet) Init added in v2.1.0

Init initializes the CurrentBlobFileSet with the state of the provided BulkVersionEdit. This is used after replaying a manifest.

func (*CurrentBlobFileSet) Metadatas added in v2.1.0

func (s *CurrentBlobFileSet) Metadatas() []BlobFileMetadata

Metadatas returns a slice of all blob file metadata in the set, sorted by file number for determinism.

func (*CurrentBlobFileSet) ReferencingTables added in v2.1.0

func (s *CurrentBlobFileSet) ReferencingTables(fileID base.BlobFileID) []*TableMetadata

ReferencingTables returns a slice containing the set of tables that reference the blob file with the provided file ID. The returned slice is sorted by table number.

func (*CurrentBlobFileSet) ReplacementCandidate added in v2.1.0

func (s *CurrentBlobFileSet) ReplacementCandidate() (BlobFileMetadata, bool)

ReplacementCandidate returns the next blob file that should be rewritten. If there are no candidates, the second return value is false. Successive calls to ReplacementCandidate may (but are not guaranteed to) return the same blob file until the blob file is replaced.

func (*CurrentBlobFileSet) Stats added in v2.1.0

Stats returns the cumulative stats across all blob files in the set and the stats for the rewrite heaps.

func (*CurrentBlobFileSet) String added in v2.1.0

func (s *CurrentBlobFileSet) String() string

String implements fmt.Stringer.

type DeletedBlobFileEntry added in v2.1.0

type DeletedBlobFileEntry struct {
	FileID  base.BlobFileID
	FileNum base.DiskFileNum
}

DeletedBlobFileEntry holds the state for a blob file deletion. The blob file ID may still be in-use with a different physical blob file.

type DeletedTableEntry added in v2.1.0

type DeletedTableEntry struct {
	Level   int
	FileNum base.FileNum
}

DeletedTableEntry holds the state for a sstable deletion from a level. The table itself might still be referenced by another level.

type InternalKey

type InternalKey = base.InternalKey

InternalKey exports the base.InternalKey type.

type InternalKeyBounds added in v2.1.0

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

InternalKeyBounds represents set of keys (smallest, largest) used for the in-memory and on-disk partial DBs that make up a pebble DB.

It consists of the smallest, largest keys and their respective trailers. The keys are represented as a single string; their individual representations are given by the userKeySeparatorIdx as:

  • smallest: [0, userKeySeparatorIdx)
  • largest: [userKeySeparatorIdx, len(userKeyData))

This format allows us to save a couple of bytes that will add up proportionally to the amount of sstables we have.

func (*InternalKeyBounds) Largest added in v2.1.0

func (ikr *InternalKeyBounds) Largest() InternalKey

func (*InternalKeyBounds) LargestUserKey added in v2.1.0

func (ikr *InternalKeyBounds) LargestUserKey() []byte

func (*InternalKeyBounds) SetInternalKeyBounds added in v2.1.0

func (ikr *InternalKeyBounds) SetInternalKeyBounds(smallest, largest InternalKey)

func (*InternalKeyBounds) SetLargest added in v2.1.0

func (ikr *InternalKeyBounds) SetLargest(ik InternalKey)

func (*InternalKeyBounds) SetSmallest added in v2.1.0

func (ikr *InternalKeyBounds) SetSmallest(ik InternalKey)

func (*InternalKeyBounds) Smallest added in v2.1.0

func (ikr *InternalKeyBounds) Smallest() InternalKey

func (*InternalKeyBounds) SmallestUserKey added in v2.1.0

func (ikr *InternalKeyBounds) SmallestUserKey() []byte

type KeyType

type KeyType int8

KeyType is used to specify the type of keys we're looking for in LevelIterator positioning operations. Files not containing any keys of the desired type are skipped.

const (
	// KeyTypePointAndRange denotes a search among the entire keyspace, including
	// both point keys and range keys. No sstables are skipped.
	KeyTypePointAndRange KeyType = iota
	// KeyTypePoint denotes a search among the point keyspace. SSTables with no
	// point keys will be skipped. Note that the point keyspace includes rangedels.
	KeyTypePoint
	// KeyTypeRange denotes a search among the range keyspace. SSTables with no
	// range keys will be skipped.
	KeyTypeRange
)

type L0Compaction

type L0Compaction struct {
	Bounds    base.UserKeyBounds
	IsIntraL0 bool
}

L0Compaction describes an active compaction with inputs from L0.

type L0CompactionFiles

type L0CompactionFiles struct {
	Files []*TableMetadata

	FilesIncluded bitSet
	// contains filtered or unexported fields
}

L0CompactionFiles represents a candidate set of L0 files for compaction. Also referred to as "lcf". Contains state information useful for generating the compaction (such as Files), as well as for picking between candidate compactions (eg. fileBytes and seedIntervalStackDepthReduction).

func (*L0CompactionFiles) Clone

Clone allocates a new L0CompactionFiles, with the same underlying data. Note that the two fileMetadata slices contain values that point to the same underlying fileMetadata object. This is safe because these objects are read only.

func (*L0CompactionFiles) String

func (l *L0CompactionFiles) String() string

String merely prints the starting address of the first file, if it exists.

type L0Organizer added in v2.1.0

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

L0Organizer keeps track of L0 state, including the subdivision into sublevels.

It is designed to be used as a singleton (per DB) which gets updated as the version changes. It is used to initialize L0-related Version fields.

The level 0 sstables are organized in a series of sublevels. Similar to the seqnum invariant in normal levels, there is no internal key in a lower sublevel table that has both the same user key and a higher sequence number. Within a sublevel, tables are sorted by their internal key range and any two tables at the same sublevel do not overlap. Unlike the normal levels, sublevel n contains older tables (lower sequence numbers) than sublevel n+1 (this is because the number of sublevels is variable).

func NewL0Organizer added in v2.1.0

func NewL0Organizer(comparer *base.Comparer, flushSplitBytes int64) *L0Organizer

NewL0Organizer creates the L0 organizer. The L0 organizer is responsible for maintaining the current L0 state and is kept in-sync with the current Version.

flushSplitBytes denotes the target number of bytes per sublevel in each flush split interval (i.e. range between two flush split keys) in L0 sstables. When set to zero, only a single sstable is generated by each flush. When set to a non-zero value, flushes are split at points to meet L0's TargetFileSize, any grandparent-related overlap options, and at boundary keys of L0 flush split intervals (which are targeted to contain around FlushSplitBytes bytes in each sublevel between pairs of boundary keys). Splitting sstables during flush allows increased compaction flexibility and concurrency when those tables are compacted to lower levels.

func (L0Organizer) Check added in v2.1.0

func (s L0Organizer) Check()

Check performs sanity checks on l0Sublevels in invariants mode.

func (L0Organizer) ExtendL0ForBaseCompactionTo added in v2.1.0

func (s L0Organizer) ExtendL0ForBaseCompactionTo(
	smallest, largest InternalKey, candidate *L0CompactionFiles,
) bool

ExtendL0ForBaseCompactionTo extends the specified base compaction candidate L0CompactionFiles to optionally cover more files in L0 without "touching" any of the passed-in keys (i.e. the smallest/largest bounds are exclusive), as including any user keys for those internal keys could require choosing more files in LBase which is undesirable. Unbounded start/end keys are indicated by passing in the InvalidInternalKey.

func (L0Organizer) FlushSplitKeys added in v2.1.0

func (s L0Organizer) FlushSplitKeys() [][]byte

FlushSplitKeys returns a slice of user keys to split flushes at. Used by flushes to avoid writing sstables that straddle these split keys. These should be interpreted as the keys to start the next sstable (not the last key to include in the prev sstable). These are user keys so that range tombstones can be properly truncated (untruncated range tombstones are not permitted for L0 files).

func (L0Organizer) InUseKeyRanges added in v2.1.0

func (s L0Organizer) InUseKeyRanges(smallest, largest []byte) []base.UserKeyBounds

InUseKeyRanges returns the merged table bounds of L0 files overlapping the provided user key range. The returned key ranges are sorted and nonoverlapping.

func (L0Organizer) InitCompactingFileInfo added in v2.1.0

func (s L0Organizer) InitCompactingFileInfo(inProgress []L0Compaction)

InitCompactingFileInfo initializes internal flags relating to compacting files. Must be called after sublevel initialization.

Requires DB.mu *and* the manifest lock to be held.

func (L0Organizer) MaxDepthAfterOngoingCompactions added in v2.1.0

func (s L0Organizer) MaxDepthAfterOngoingCompactions() int

MaxDepthAfterOngoingCompactions returns an estimate of maximum depth of sublevels after all ongoing compactions run to completion. Used by compaction picker to decide compaction score for L0. There is no scoring for intra-L0 compactions -- they only run if L0 score is high but we're unable to pick an L0 -> Lbase compaction.

func (*L0Organizer) PerformUpdate added in v2.1.0

func (o *L0Organizer) PerformUpdate(prepared L0PreparedUpdate, newVersion *Version)

PerformUpdate applies an update the L0 organizer which was previously prepared using PrepareUpdate.

Sets newVersion.L0SublevelFiles (which is immutable once set).

This method cannot be called concurrently with any other methods.

func (L0Organizer) PickBaseCompaction added in v2.1.0

func (s L0Organizer) PickBaseCompaction(
	logger base.Logger,
	minCompactionDepth int,
	baseFiles LevelSlice,
	baseLevel int,
	problemSpans *problemspans.ByLevel,
) *L0CompactionFiles

PickBaseCompaction picks a base compaction based on the above specified heuristics, for the specified Lbase files and a minimum depth of overlapping files that can be selected for compaction. Returns nil if no compaction is possible.

func (L0Organizer) PickIntraL0Compaction added in v2.1.0

func (s L0Organizer) PickIntraL0Compaction(
	earliestUnflushedSeqNum base.SeqNum, minCompactionDepth int, problemSpans *problemspans.ByLevel,
) *L0CompactionFiles

PickIntraL0Compaction picks an intra-L0 compaction for files in this sublevel. This method is only called when a base compaction cannot be chosen. See comment above [PickBaseCompaction] for heuristics involved in this selection.

func (*L0Organizer) PrepareUpdate added in v2.1.0

func (o *L0Organizer) PrepareUpdate(bve *BulkVersionEdit, newVersion *Version) L0PreparedUpdate

PrepareUpdate is the first step in the two-step process to update the L0Organizer. This first step performs as much work as it can without modifying the L0Organizer.

This method can be called concurrently with other methods (other than PerformUpdate). It allows doing most of the update work outside an important lock.

func (L0Organizer) ReadAmplification added in v2.1.0

func (s L0Organizer) ReadAmplification() int

ReadAmplification returns the contribution of l0Sublevels to the read amplification for any particular point key. It is the maximum height of any tracked fileInterval. This is always less than or equal to the number of sublevels.

func (*L0Organizer) ResetForTesting added in v2.1.0

func (o *L0Organizer) ResetForTesting(v *Version)

ResetForTesting reinitializes the L0Organizer to reflect the given version. Sets v.L0SublevelFiles.

func (L0Organizer) String added in v2.1.0

func (s L0Organizer) String() string

String produces a string containing useful debug information. Useful in test code and debugging.

func (L0Organizer) UpdateStateForStartedCompaction added in v2.1.0

func (s L0Organizer) UpdateStateForStartedCompaction(inputs []LevelSlice, isBase bool) error

UpdateStateForStartedCompaction updates internal l0Sublevels state for a recently started compaction. isBase specifies if this is a base compaction; if false, this is assumed to be an intra-L0 compaction. The specified compaction must be involving L0 SSTables. It's assumed that the Compacting and IsIntraL0Compacting fields are already set on all [TableMetadata]s passed in.

type L0PreparedUpdate added in v2.1.0

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

L0PreparedUpdate is returned by L0Organizer.PrepareUpdate(), to be passed to PerformUpdate().

type Layer

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

Layer represents a section of the logical sstable hierarchy. It can represent:

  • a level L1 through L6, or
  • the entire L0 level, or
  • a specific L0 sublevel, or
  • the layer of flushable ingests (which is conceptually above the LSM).

func FlushableIngestsLayer

func FlushableIngestsLayer() Layer

FlushableIngestsLayer returns a Layer that represents the flushable ingests layer (which is logically above L0).

func L0Sublevel

func L0Sublevel(sublevel int) Layer

L0Sublevel returns a Layer that represents a specific L0 sublevel.

func Level

func Level(level int) Layer

Level returns a Layer that represents an entire level (L0 through L6).

func (Layer) IsFlushableIngests

func (l Layer) IsFlushableIngests() bool

IsFlushableIngests returns true if the layer represents flushable ingests.

func (Layer) IsL0Sublevel

func (l Layer) IsL0Sublevel() bool

IsL0Sublevel returns true if the layer represents an L0 sublevel.

func (Layer) IsSet

func (l Layer) IsSet() bool

IsSet returns true if l has been initialized.

func (Layer) Level

func (l Layer) Level() int

Level returns the level for the layer. Must not be called if the layer represents flushable ingests.

func (Layer) SafeFormat

func (l Layer) SafeFormat(s redact.SafePrinter, verb rune)

SafeFormat implements redact.SafeFormatter.

func (Layer) String

func (l Layer) String() string

func (Layer) Sublevel

func (l Layer) Sublevel() int

Sublevel returns the L0 sublevel. Can only be called if the layer represents an L0 sublevel.

type LevelFile

type LevelFile struct {
	*TableMetadata
	// contains filtered or unexported fields
}

LevelFile holds a file's metadata along with its position within a level of the LSM.

func (LevelFile) Slice

func (lf LevelFile) Slice() LevelSlice

Slice constructs a LevelSlice containing only this file.

type LevelIterator

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

LevelIterator iterates over a set of files' metadata. Its zero value is an empty iterator.

func (*LevelIterator) Clone

func (i *LevelIterator) Clone() LevelIterator

Clone copies the iterator, returning an independent iterator at the same position.

func (*LevelIterator) Filter

func (i *LevelIterator) Filter(keyType KeyType) LevelIterator

Filter clones the iterator and sets the desired KeyType as the key to filter files on.

func (*LevelIterator) First

func (i *LevelIterator) First() *TableMetadata

First seeks to the first file in the iterator and returns it.

func (*LevelIterator) Last

func (i *LevelIterator) Last() *TableMetadata

Last seeks to the last file in the iterator and returns it.

func (*LevelIterator) Next

func (i *LevelIterator) Next() *TableMetadata

Next advances the iterator to the next file and returns it.

func (*LevelIterator) Prev

func (i *LevelIterator) Prev() *TableMetadata

Prev moves the iterator the previous file and returns it.

func (*LevelIterator) SeekGE

func (i *LevelIterator) SeekGE(cmp Compare, userKey []byte) *TableMetadata

SeekGE seeks to the first file with a largest key (of the desired type) that is an upper bound for the given user key. This is the first file that could contain a user key that is greater than or equal to userKey.

More specifically, userKey is less than the file's largest.UserKey or they are equal and largest is not an exclusive sentinel.

The iterator must have been constructed from L1+ or from a single sublevel of L0, because it requires the underlying files to be sorted by user keys and non-overlapping.

func (*LevelIterator) SeekLT

func (i *LevelIterator) SeekLT(cmp Compare, userKey []byte) *TableMetadata

SeekLT seeks to the last file with a smallest key (of the desired type) that is less than the given user key. This is the last file that could contain a key less than userKey.

The iterator must have been constructed from L1+ or from a single sublevel of L0, because it requires the underlying files to be sorted by user keys and non-overlapping.

func (LevelIterator) String

func (i LevelIterator) String() string

func (*LevelIterator) Take

func (i *LevelIterator) Take() LevelFile

Take constructs a LevelFile containing the file at the iterator's current position. Take panics if the iterator is not currently positioned over a file.

type LevelMetadata

type LevelMetadata struct {

	// NumVirtual is the number of virtual sstables in the level.
	NumVirtual uint64
	// VirtualTableSize is the size of the virtual sstables in the level.
	VirtualTableSize uint64
	// contains filtered or unexported fields
}

LevelMetadata contains metadata for all of the files within a level of the LSM.

func MakeLevelMetadata

func MakeLevelMetadata(cmp Compare, level int, files []*TableMetadata) LevelMetadata

MakeLevelMetadata creates a LevelMetadata with the given files.

func (*LevelMetadata) AggregateSize added in v2.1.0

func (lm *LevelMetadata) AggregateSize() uint64

AggregateSize returns the aggregate size estimate of all sstables within the level, plus an estimate of the physical size of values stored externally in blob files. This quantity is equal to TableSize() + EstimatedReferenceSize().

func (*LevelMetadata) All added in v2.1.0

func (lm *LevelMetadata) All() iter.Seq[*TableMetadata]

All returns an iterator over all files in the level.

func (*LevelMetadata) Empty

func (lm *LevelMetadata) Empty() bool

Empty indicates whether there are any files in the level.

func (*LevelMetadata) EstimatedReferenceSize added in v2.1.0

func (lm *LevelMetadata) EstimatedReferenceSize() uint64

EstimatedReferenceSize returns an estimate of the physical size of all the file's blob references in the table. This sum, added to TableSize(), yields AggregateSize().

func (*LevelMetadata) Find

Find finds the provided file in the level. If it exists, returns a LevelSlice that contains just that file; otherwise, returns an empty LevelSlice.

func (*LevelMetadata) Iter

func (lm *LevelMetadata) Iter() LevelIterator

Iter constructs a LevelIterator over the entire level.

func (*LevelMetadata) Len

func (lm *LevelMetadata) Len() int

Len returns the number of files within the level.

func (*LevelMetadata) Slice

func (lm *LevelMetadata) Slice() LevelSlice

Slice constructs a slice containing the entire level.

func (*LevelMetadata) TableSize added in v2.1.0

func (lm *LevelMetadata) TableSize() uint64

TableSize returns the cumulative size of all sstables within the level. This quantity does NOT include the size of values stored externally in blob files.

type LevelSlice

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

LevelSlice contains a slice of the files within a level of the LSM. A LevelSlice is immutable once created, but may be used to construct a mutable LevelIterator over the slice's files.

LevelSlices should be constructed through one of the existing constructors, not manually initialized.

func NewLevelSliceKeySorted

func NewLevelSliceKeySorted(cmp base.Compare, files []*TableMetadata) LevelSlice

NewLevelSliceKeySorted constructs a LevelSlice over the provided files, sorted by the files smallest keys. TODO(jackson): Can we improve this interface or avoid needing to export a slice constructor like this?

func NewLevelSliceSeqSorted

func NewLevelSliceSeqSorted(files []*TableMetadata) LevelSlice

NewLevelSliceSeqSorted constructs a LevelSlice over the provided files, sorted by the L0 sequence number sort order. TODO(jackson): Can we improve this interface or avoid needing to export a slice constructor like this?

func NewLevelSliceSpecificOrder

func NewLevelSliceSpecificOrder(files []*TableMetadata) LevelSlice

NewLevelSliceSpecificOrder constructs a LevelSlice over the provided files, ordering the files by their order in the provided slice. It's used in tests. TODO(jackson): Update tests to avoid requiring this and remove it.

func (*LevelSlice) AggregateSizeSum added in v2.1.0

func (ls *LevelSlice) AggregateSizeSum() uint64

AggregateSizeSum sums the size of all sstables in the slice, inclusive of the estimated physical size of tables' blob references. Its runtime is linear in the length of the slice.

func (LevelSlice) All added in v2.1.0

func (ls LevelSlice) All() iter.Seq[*TableMetadata]

All returns an iterator over all files in the slice.

func (*LevelSlice) Empty

func (ls *LevelSlice) Empty() bool

Empty indicates whether the slice contains any files.

func (*LevelSlice) Iter

func (ls *LevelSlice) Iter() LevelIterator

Iter constructs a LevelIterator that iterates over the slice.

func (*LevelSlice) Len

func (ls *LevelSlice) Len() int

Len returns the number of files in the slice. Its runtime is constant.

func (*LevelSlice) NumVirtual

func (ls *LevelSlice) NumVirtual() uint64

NumVirtual returns the number of virtual sstables in the level. Its runtime is linear in the length of the slice.

func (LevelSlice) Overlaps

func (ls LevelSlice) Overlaps(cmp Compare, bounds base.UserKeyBounds) LevelSlice

Overlaps returns a new LevelSlice that reflects the portion of files with boundaries that overlap with the provided bounds.

func (LevelSlice) Reslice

func (ls LevelSlice) Reslice(resliceFunc func(start, end *LevelIterator)) LevelSlice

Reslice constructs a new slice backed by the same underlying level, with new start and end positions. Reslice invokes the provided function, passing two LevelIterators: one positioned to i's inclusive start and one positioned to i's inclusive end. The resliceFunc may move either iterator forward or backwards, including beyond the callee's original bounds to capture additional files from the underlying level. Reslice constructs and returns a new LevelSlice with the final bounds of the iterators after calling resliceFunc.

func (LevelSlice) String

func (ls LevelSlice) String() string

String implements fmt.Stringer.

func (*LevelSlice) TableSizeSum added in v2.1.0

func (ls *LevelSlice) TableSizeSum() uint64

TableSizeSum sums the size of all sstables in the slice. Its runtime is linear in the length of the slice.

func (*LevelSlice) VirtualTableSizeSum added in v2.1.0

func (ls *LevelSlice) VirtualTableSizeSum() uint64

VirtualTableSizeSum returns the sum of the sizes of the virtual sstables in the level.

type NewTableEntry added in v2.1.0

type NewTableEntry struct {
	Level int
	Meta  *TableMetadata
	// BackingFileNum is only set during manifest replay, and only for virtual
	// sstables.
	BackingFileNum base.DiskFileNum
}

NewTableEntry holds the state for a new sstable or one moved from a different level.

type ObsoleteFiles added in v2.1.0

type ObsoleteFiles struct {
	TableBackings []*TableBacking
	BlobFiles     []*PhysicalBlobFile
}

ObsoleteFiles holds a set of files that are no longer referenced by any referenced Version.

func (*ObsoleteFiles) AddBacking added in v2.1.0

func (of *ObsoleteFiles) AddBacking(fb *TableBacking)

AddBacking appends the provided TableBacking to the list of obsolete files.

func (*ObsoleteFiles) AddBlob added in v2.1.0

func (of *ObsoleteFiles) AddBlob(bm *PhysicalBlobFile)

AddBlob appends the provided BlobFileMetadata to the list of obsolete files.

func (*ObsoleteFiles) Count added in v2.1.0

func (of *ObsoleteFiles) Count() int

Count returns the number of files in the ObsoleteFiles.

type ObsoleteFilesSet added in v2.1.0

type ObsoleteFilesSet interface {
	// AddBacking appends the provided TableBacking to the list of obsolete
	// files.
	AddBacking(*TableBacking)
	// AddBlob appends the provided BlobFileMetadata to the list of obsolete
	// files.
	AddBlob(*PhysicalBlobFile)
}

ObsoleteFilesSet accumulates files that now have zero references.

type PartialOverlapAnnotationAggregator

type PartialOverlapAnnotationAggregator[T any] interface {
	AnnotationAggregator[T]
	AccumulatePartialOverlap(f *TableMetadata, dst *T, bounds base.UserKeyBounds) *T
}

A PartialOverlapAnnotationAggregator is an extension of AnnotationAggregator that allows for custom accumulation of range annotations for files that only partially overlap with the range.

type PhysicalBlobFile added in v2.1.0

type PhysicalBlobFile struct {
	// FileNum is an ID that uniquely identifies the blob file.
	FileNum base.DiskFileNum
	// Size is the size of the file, in bytes.
	Size uint64
	// ValueSize is the sum of the length of the uncompressed values stored in
	// this blob file.
	ValueSize uint64
	// File creation time in seconds since the epoch (1970-01-01 00:00:00
	// UTC).
	CreationTime uint64
	// contains filtered or unexported fields
}

PhysicalBlobFile is metadata describing a physical blob value file.

func ParsePhysicalBlobFileDebug added in v2.1.0

func ParsePhysicalBlobFileDebug(s string) (_ *PhysicalBlobFile, err error)

ParsePhysicalBlobFileDebug parses a PhysicalBlobFile from its string representation. This function is intended for use in tests. It's the inverse of PhysicalBlobFile.String().

In production code paths, the PhysicalBlobFile is serialized in a binary format within a version edit under the tag tagNewBlobFile.

func (*PhysicalBlobFile) SafeFormat added in v2.1.0

func (m *PhysicalBlobFile) SafeFormat(w redact.SafePrinter, _ rune)

SafeFormat implements redact.SafeFormatter.

func (*PhysicalBlobFile) String added in v2.1.0

func (m *PhysicalBlobFile) String() string

String implements fmt.Stringer.

type PickFileAggregator

type PickFileAggregator struct {
	// Filter takes a TableMetadata and returns whether it is eligible to be
	// picked by this PickFileAggregator. The second return value indicates
	// whether this eligibility is stable and thus cacheable.
	Filter func(f *TableMetadata) (eligible bool, cacheOK bool)
	// Compare compares two instances of TableMetadata and returns true if the
	// first one should be picked over the second one. It may assume that both
	// arguments are non-nil.
	Compare func(f1 *TableMetadata, f2 *TableMetadata) bool
}

PickFileAggregator implements the AnnotationAggregator interface. It defines an aggregator that picks a single file from a set of eligible files.

func (PickFileAggregator) Accumulate

func (fa PickFileAggregator) Accumulate(
	f *TableMetadata, dst *TableMetadata,
) (v *TableMetadata, cacheOK bool)

Accumulate implements AnnotationAggregator.Accumulate, accumulating a single file as long as it is eligible to be picked.

func (PickFileAggregator) Merge

Merge implements AnnotationAggregator.Merge by picking a single file based on the output of PickFileAggregator.Compare.

func (PickFileAggregator) Zero

Zero implements AnnotationAggregator.Zero, returning nil as the zero value.

type SumAggregator

type SumAggregator struct {
	AccumulateFunc               func(f *TableMetadata) (v uint64, cacheOK bool)
	AccumulatePartialOverlapFunc func(f *TableMetadata, bounds base.UserKeyBounds) uint64
}

SumAggregator defines an Aggregator which sums together a uint64 value across files.

func (SumAggregator) Accumulate

func (sa SumAggregator) Accumulate(f *TableMetadata, dst *uint64) (v *uint64, cacheOK bool)

Accumulate implements AnnotationAggregator.Accumulate, accumulating a single file's uint64 value.

func (SumAggregator) AccumulatePartialOverlap

func (sa SumAggregator) AccumulatePartialOverlap(
	f *TableMetadata, dst *uint64, bounds base.UserKeyBounds,
) *uint64

AccumulatePartialOverlap implements PartialOverlapAnnotationAggregator.AccumulatePartialOverlap, accumulating a single file's uint64 value for a file which only partially overlaps with the range defined by bounds.

func (SumAggregator) Merge

func (sa SumAggregator) Merge(src *uint64, dst *uint64) *uint64

Merge implements AnnotationAggregator.Merge by summing two uint64 values.

func (SumAggregator) Zero

func (sa SumAggregator) Zero(dst *uint64) *uint64

Zero implements AnnotationAggregator.Zero, returning a new uint64 set to 0.

type TableBacking added in v2.1.0

type TableBacking struct {
	DiskFileNum base.DiskFileNum
	Size        uint64
	// contains filtered or unexported fields
}

TableBacking either backs a single physical sstable, or one or more virtual sstables.

See the comment above the TableMetadata type for sstable terminology.

func (*TableBacking) IsUnused added in v2.1.0

func (b *TableBacking) IsUnused() bool

IsUnused returns if the backing is not being used by any tables in a version or btree.

func (*TableBacking) MustHaveRefs added in v2.1.0

func (b *TableBacking) MustHaveRefs()

MustHaveRefs asserts that the backing has a positive refcount.

func (*TableBacking) Ref added in v2.1.0

func (b *TableBacking) Ref()

Ref increments the backing's ref count.

func (*TableBacking) Unref added in v2.1.0

func (b *TableBacking) Unref() int32

Unref decrements the backing's ref count (and returns the new count).

type TableInfo

type TableInfo struct {
	// FileNum is the internal DB identifier for the table.
	FileNum base.FileNum
	// Size is the size of the file in bytes.
	Size uint64
	// Smallest is the smallest internal key in the table.
	Smallest InternalKey
	// Largest is the largest internal key in the table.
	Largest InternalKey
	// SmallestSeqNum is the smallest sequence number in the table.
	SmallestSeqNum base.SeqNum
	// LargestSeqNum is the largest sequence number in the table.
	LargestSeqNum base.SeqNum
	// contains filtered or unexported fields
}

TableInfo contains the common information for table related events.

func (*TableInfo) GetBlobReferenceFiles added in v2.1.0

func (t *TableInfo) GetBlobReferenceFiles() []base.BlobFileID

GetBlobReferenceFiles returns the list of blob file numbers referenced by the table.

type TableMetadata added in v2.1.0

type TableMetadata struct {
	// AllowedSeeks is used to determine if a file should be picked for
	// a read triggered compaction. It is decremented when read sampling
	// in pebble.Iterator after every after every positioning operation
	// that returns a user key (eg. Next, Prev, SeekGE, SeekLT, etc).
	AllowedSeeks atomic.Int64

	// TableBacking is the physical file that backs either physical or virtual
	// sstables.
	TableBacking *TableBacking

	// InitAllowedSeeks is the inital value of allowed seeks. This is used
	// to re-set allowed seeks on a file once it hits 0.
	InitAllowedSeeks int64
	// TableNum is the table number, unique across the lifetime of a DB.
	//
	// INVARIANT: when !TableMetadata.Virtual, TableNum == TableBacking.DiskFileNum.
	TableNum base.TableNum
	// Size is the size of the file, in bytes. Size is an approximate value for
	// virtual sstables.
	//
	// INVARIANTS:
	// - When !TableMetadata.Virtual, Size == TableBacking.Size.
	// - Size should be non-zero. Size 0 virtual sstables must not be created.
	Size uint64
	// File creation time in seconds since the epoch (1970-01-01 00:00:00
	// UTC). For ingested sstables, this corresponds to the time the file was
	// ingested. For virtual sstables, this corresponds to the wall clock time
	// when the TableMetadata for the virtual sstable was first created.
	CreationTime int64
	// LargestSeqNumAbsolute is an upper bound for the largest sequence number
	// in the table. This upper bound is guaranteed to be higher than any
	// sequence number any of the table's keys have held at any point in time
	// while the database has been open. Specifically, if the table contains
	// keys that have had their sequence numbers zeroed during a compaction,
	// LargestSeqNumAbsolute will be at least as high as the pre-zeroing
	// sequence number. LargestSeqNumAbsolute is NOT durably persisted, so after
	// a database restart it takes on the value of LargestSeqNum.
	LargestSeqNumAbsolute base.SeqNum
	// Lower and upper bounds for the smallest and largest sequence numbers in
	// the table, across both point and range keys. For physical sstables, these
	// values are tight bounds. For virtual sstables, there is no guarantee that
	// there will be keys with SmallestSeqNum or LargestSeqNum within virtual
	// sstable bounds.
	SmallestSeqNum base.SeqNum
	LargestSeqNum  base.SeqNum
	// PointKeyBounds.Smallest() and PointKeyBounds.Largest() are the inclusive bounds for the
	// internal point keys stored in the table. This includes RANGEDELs, which
	// alter point keys.
	// NB: these field should be set using ExtendPointKeyBounds. They are left
	// exported for reads as an optimization.
	PointKeyBounds InternalKeyBounds
	// RangeKeyBounds.Smallest() and RangeKeyBounds.Largest() are the inclusive bounds for the
	// internal range keys stored in the table.
	// NB: these field should be set using ExtendRangeKeyBounds. They are left
	// exported for reads as an optimization.
	RangeKeyBounds *InternalKeyBounds
	// BlobReferences is a list of blob files containing values that are
	// referenced by this sstable.
	BlobReferences BlobReferences
	// BlobReferenceDepth is the stack depth of blob files referenced by this
	// sstable. See the comment on the BlobReferenceDepth type for more details.
	//
	// INVARIANT: BlobReferenceDepth == 0 iff len(BlobReferences) == 0
	// INVARIANT: BlobReferenceDepth <= len(BlobReferences)
	BlobReferenceDepth BlobReferenceDepth

	// Stats describe table statistics. Protected by DB.mu.
	//
	// For virtual sstables, set stats upon virtual sstable creation as
	// asynchronous computation of stats is not currently supported.
	//
	// TODO(bananabrick): To support manifest replay for virtual sstables, we
	// probably need to compute virtual sstable stats asynchronously. Otherwise,
	// we'd have to write virtual sstable stats to the version edit.
	Stats TableStats

	// For L0 files only. Protected by DB.mu. Used to generate L0 sublevels and
	// pick L0 compactions. Only accurate for the most recent Version.
	// TODO(radu): this is very hacky and fragile. This information should live
	// inside l0Sublevels.
	SubLevel int
	L0Index  int

	// IsIntraL0Compacting is set to True if this file is part of an intra-L0
	// compaction. When it's true, IsCompacting must also return true. If
	// Compacting is true and IsIntraL0Compacting is false for an L0 file, the
	// file must be part of a compaction to Lbase.
	IsIntraL0Compacting bool
	CompactionState     CompactionState
	// True if compaction of this file has been explicitly requested.
	// Previously, RocksDB and earlier versions of Pebble allowed this
	// flag to be set by a user table property collector. Some earlier
	// versions of Pebble respected this flag, while other more recent
	// versions ignored this flag.
	//
	// More recently this flag has been repurposed to facilitate the
	// compaction of 'atomic compaction units'. Files marked for
	// compaction are compacted in a rewrite compaction at the lowest
	// possible compaction priority.
	//
	// NB: A count of files marked for compaction is maintained on
	// Version, and compaction picking reads cached annotations
	// determined by this field.
	//
	// Protected by DB.mu.
	MarkedForCompaction bool
	// HasPointKeys tracks whether the table contains point keys (including
	// RANGEDELs). If a table contains only range deletions, HasPointsKeys is
	// still true.
	HasPointKeys bool
	// HasRangeKeys tracks whether the table contains any range keys.
	HasRangeKeys bool
	// Virtual is true if the TableMetadata belongs to a virtual sstable.
	Virtual bool

	// VirtualParams are set only when Virtual is true.
	VirtualParams *virtual.VirtualReaderParams

	// SyntheticPrefix is used to prepend a prefix to all keys and/or override all
	// suffixes in a table; used for some virtual tables.
	SyntheticPrefixAndSuffix sstable.SyntheticPrefixAndSuffix
	// contains filtered or unexported fields
}

TableMetadata is maintained for leveled-ssts, i.e., they belong to a level of some version. TableMetadata does not contain the actual level of the sst, since such leveled-ssts can move across levels in different versions, while sharing the same TableMetadata. There are two kinds of leveled-ssts, physical and virtual. Underlying both leveled-ssts is a backing-sst, for which the only state is TableBacking. A backing-sst is level-less. It is possible for a backing-sst to be referred to by a physical sst in one version and by one or more virtual ssts in one or more versions. A backing-sst becomes obsolete and can be deleted once it is no longer required by any physical or virtual sst in any version.

We maintain some invariants:

  1. Each physical and virtual sst will have a unique TableMetadata.TableNum, and there will be exactly one TableMetadata associated with the TableNum.

  2. Within a version, a backing-sst is either only referred to by one physical sst or one or more virtual ssts.

  3. Once a backing-sst is referred to by a virtual sst in the latest version, it cannot go back to being referred to by a physical sst in any future version.

Once a physical sst is no longer needed by any version, we will no longer maintain the table metadata associated with it. We will still maintain the TableBacking associated with the physical sst if the backing sst is required by any virtual ssts in any version.

When using these fields in the context of a Virtual Table, These fields have additional invariants imposed on them, and/or slightly varying meanings:

  • boundTypeSmallest and boundTypeLargest (and their counterparts {Point,Range}KeyBounds.{Smallest(), Largest()}) remain tight bounds that represent a key at that exact bound. We make the effort to determine the next smallest or largest key in an sstable after virtualizing it, to maintain this tightness. If the largest is a sentinel key (IsExclusiveSentinel()), it could mean that a rangedel or range key ends at that user key, or has been truncated to that user key.
  • One invariant is that if a rangedel or range key is truncated on its upper bound, the virtual sstable *must* have a rangedel or range key sentinel key as its upper bound. This is because truncation yields an exclusive upper bound for the rangedel/rangekey, and if there are any points at that exclusive upper bound within the same virtual sstable, those could get uncovered by this truncation. We enforce this invariant in calls to keyspan.Truncate.
  • Size is an estimate of the size of the virtualized portion of this sstable. The underlying file's size is stored in TableBacking.Size, though it could also be estimated or could correspond to just the referenced portion of a file (eg. if the file originated on another node).
  • Size must be > 0.
  • SmallestSeqNum and LargestSeqNum are loose bounds for virtual sstables. This means that all keys in the virtual sstable must have seqnums within [SmallestSeqNum, LargestSeqNum], however there's no guarantee that there's a key with a seqnum at either of the bounds. Calculating tight seqnum bounds would be too expensive and deliver little value.
  • Note: These properties do not apply to external sstables, whose bounds are loose rather than tight, as we do not open them on ingest.

func ParseTableMetadataDebug added in v2.1.0

func ParseTableMetadataDebug(s string) (_ *TableMetadata, err error)

ParseTableMetadataDebug parses a TableMetadata from its DebugString representation.

func (*TableMetadata) AttachVirtualBacking added in v2.1.0

func (m *TableMetadata) AttachVirtualBacking(backing *TableBacking)

AttachVirtualBacking attaches an existing TableBacking for a virtual table.

The Smallest/Largest bounds must already be set to their final values.

func (*TableMetadata) ContainedWithinSpan added in v2.1.0

func (m *TableMetadata) ContainedWithinSpan(cmp Compare, start, end []byte) bool

ContainedWithinSpan returns true if the file key range completely overlaps with the given range ("end" is assumed to exclusive).

func (*TableMetadata) ContainsKeyType added in v2.1.0

func (m *TableMetadata) ContainsKeyType(kt KeyType) bool

ContainsKeyType returns whether or not the file contains keys of the provided type.

func (*TableMetadata) DebugString added in v2.1.0

func (m *TableMetadata) DebugString(format base.FormatKey, verbose bool) string

DebugString returns a verbose representation of TableMetadata, typically for use in tests and debugging, returning the file number and the point, range and overall bounds for the table.

func (*TableMetadata) EstimatedReferenceSize added in v2.1.0

func (m *TableMetadata) EstimatedReferenceSize() uint64

EstimatedReferenceSize returns the estimated physical size of all the file's blob references in the table. This sum, added to the sstable's size, yields an approximation of the overall size of the data represented by the table.

EstimatedReferenceSize is an estimate, but it's guaranteed to be stable over the lifetime of the table. This is necessary to correctly maintain incrementally-updated metrics.

func (*TableMetadata) ExtendPointKeyBounds added in v2.1.0

func (m *TableMetadata) ExtendPointKeyBounds(
	cmp Compare, smallest, largest InternalKey,
) *TableMetadata

ExtendPointKeyBounds attempts to extend the lower and upper point key bounds and overall table bounds with the given smallest and largest keys. The smallest and largest bounds may not be extended if the table already has a bound that is smaller or larger, respectively. The receiver is returned. NB: calling this method should be preferred to manually setting the bounds by manipulating the fields directly, to maintain certain invariants.

func (*TableMetadata) ExtendRangeKeyBounds added in v2.1.0

func (m *TableMetadata) ExtendRangeKeyBounds(
	cmp Compare, smallest, largest InternalKey,
) *TableMetadata

ExtendRangeKeyBounds attempts to extend the lower and upper range key bounds and overall table bounds with the given smallest and largest keys. The smallest and largest bounds may not be extended if the table already has a bound that is smaller or larger, respectively. The receiver is returned. NB: calling this method should be preferred to manually setting the bounds by manipulating the fields directly, to maintain certain invariants.

func (*TableMetadata) FragmentIterTransforms added in v2.1.0

func (m *TableMetadata) FragmentIterTransforms() sstable.FragmentIterTransforms

FragmentIterTransforms returns an sstable.FragmentIterTransforms populated according to the file.

func (*TableMetadata) InitPhysicalBacking added in v2.1.0

func (m *TableMetadata) InitPhysicalBacking()

InitPhysicalBacking allocates and sets the TableBacking which is required by a physical sstable TableMetadata.

Ensure that the state required by TableBacking, such as the TableNum, is already set on the TableMetadata before InitPhysicalBacking is called. Calling InitPhysicalBacking only after the relevant state has been set in the TableMetadata is not necessary in tests which don't rely on TableBacking.

func (*TableMetadata) InitVirtualBacking added in v2.1.0

func (m *TableMetadata) InitVirtualBacking(fileNum base.DiskFileNum, size uint64)

InitVirtualBacking creates a new TableBacking for a virtual table.

The Smallest/Largest bounds must already be set to their final values.

func (*TableMetadata) InternalKeyBounds added in v2.1.0

func (m *TableMetadata) InternalKeyBounds() (InternalKey, InternalKey)

InternalKeyBounds returns the set of overall table bounds.

func (*TableMetadata) IsCompacting added in v2.1.0

func (m *TableMetadata) IsCompacting() bool

IsCompacting returns true if this file's compaction state is CompactionStateCompacting. Protected by DB.mu.

func (*TableMetadata) IterTransforms added in v2.1.0

func (m *TableMetadata) IterTransforms() sstable.IterTransforms

IterTransforms returns an sstable.IterTransforms populated according to the file.

func (*TableMetadata) Largest added in v2.1.0

func (m *TableMetadata) Largest() InternalKey

Largest returns the largest key based on the bound type of boundTypeLargest.

func (*TableMetadata) LargestBound added in v2.1.0

func (m *TableMetadata) LargestBound(kt KeyType) (InternalKey, bool)

LargestBound returns the file's largest bound of the key type. It returns a false second return value if the file does not contain any keys of the key type.

func (*TableMetadata) Overlaps added in v2.1.0

func (m *TableMetadata) Overlaps(cmp Compare, bounds *base.UserKeyBounds) bool

Overlaps returns true if the file key range overlaps with the given user key bounds.

func (*TableMetadata) PhysicalMeta added in v2.1.0

func (m *TableMetadata) PhysicalMeta() *TableMetadata

func (*TableMetadata) Ref added in v2.1.0

func (m *TableMetadata) Ref()

Ref increments the table's ref count. If this is the table's first reference, Ref will increment the reference of the table's TableBacking.

func (*TableMetadata) SetCompactionState added in v2.1.0

func (m *TableMetadata) SetCompactionState(to CompactionState)

SetCompactionState transitions this file's compaction state to the given state. Protected by DB.mu.

func (*TableMetadata) Smallest added in v2.1.0

func (m *TableMetadata) Smallest() InternalKey

Smallest returns the smallest key based on the bound type of boundTypeSmallest.

func (*TableMetadata) SmallestBound added in v2.1.0

func (m *TableMetadata) SmallestBound(kt KeyType) (InternalKey, bool)

SmallestBound returns the file's smallest bound of the key type. It returns a false second return value if the file does not contain any keys of the key type.

func (*TableMetadata) StatsMarkValid added in v2.1.0

func (m *TableMetadata) StatsMarkValid()

StatsMarkValid marks the TableStats as valid. The caller must hold DB.mu while populating TableStats and calling StatsMarkValud. Once stats are populated, they must not be mutated.

func (*TableMetadata) StatsValid added in v2.1.0

func (m *TableMetadata) StatsValid() bool

StatsValid returns true if the table stats have been populated. If StatValid returns true, the Stats field may be read (with or without holding the database mutex).

func (*TableMetadata) String added in v2.1.0

func (m *TableMetadata) String() string

String implements fmt.Stringer, printing the file number and the overall table bounds.

func (*TableMetadata) SyntheticSeqNum added in v2.1.0

func (m *TableMetadata) SyntheticSeqNum() sstable.SyntheticSeqNum

SyntheticSeqNum returns a SyntheticSeqNum which is set when SmallestSeqNum equals LargestSeqNum.

func (*TableMetadata) TableInfo added in v2.1.0

func (m *TableMetadata) TableInfo() TableInfo

TableInfo returns a subset of the TableMetadata state formatted as a TableInfo.

func (*TableMetadata) Unref added in v2.1.0

func (m *TableMetadata) Unref(obsoleteFiles ObsoleteFilesSet)

Unref decrements the table's reference count. If the count reaches zero, the table releases its references on associated files. If the table's backing file becomes obsolete, it's inserted into the provided ObsoleteFiles.

func (*TableMetadata) UserKeyBounds added in v2.1.0

func (m *TableMetadata) UserKeyBounds() base.UserKeyBounds

UserKeyBounds returns the user key bounds that correspond to m.Smallest and Largest. Because we do not allow split user keys, the user key bounds of files within a level do not overlap.

func (*TableMetadata) UserKeyBoundsByType added in v2.1.0

func (m *TableMetadata) UserKeyBoundsByType(keyType KeyType) base.UserKeyBounds

UserKeyBoundsByType returns the user key bounds for the given key types. Note that the returned bounds are invalid when requesting KeyTypePoint but HasPointKeys is false, or when requesting KeyTypeRange and HasRangeKeys is false.

func (*TableMetadata) Validate added in v2.1.0

func (m *TableMetadata) Validate(cmp Compare, formatKey base.FormatKey) error

Validate validates the metadata for consistency with itself, returning an error if inconsistent.

func (*TableMetadata) ValidateVirtual added in v2.1.0

func (m *TableMetadata) ValidateVirtual(createdFrom *TableMetadata)

ValidateVirtual should be called once the TableMetadata for a virtual sstable is created to verify that the fields of the virtual sstable are sound.

func (*TableMetadata) VirtualMeta added in v2.1.0

func (m *TableMetadata) VirtualMeta() *TableMetadata

type TableStats

type TableStats struct {
	// The total number of entries in the table.
	NumEntries uint64
	// The number of point and range deletion entries in the table.
	NumDeletions uint64
	// NumRangeKeySets is the total number of range key sets in the table.
	//
	// NB: If there's a chance that the sstable contains any range key sets,
	// then NumRangeKeySets must be > 0.
	NumRangeKeySets uint64
	// Estimate of the total disk space that may be dropped by this table's
	// point deletions by compacting them.
	PointDeletionsBytesEstimate uint64
	// Estimate of the total disk space that may be dropped by this table's
	// range deletions by compacting them. This estimate is at data-block
	// granularity and is not updated if compactions beneath the table reduce
	// the amount of reclaimable disk space. It also does not account for
	// overlapping data in L0 and ignores L0 sublevels, but the error that
	// introduces is expected to be small. Similarly, multiple overlapping
	// RANGEDELs can in different levels can count the same data to be deleted
	// multiple times.
	//
	// Tables in the bottommost level of the LSM may have a nonzero estimate if
	// snapshots or move compactions prevented the elision of their range
	// tombstones. A table in the bottommost level that was ingested into L6
	// will have a zero estimate, because the file's sequence numbers indicate
	// that the tombstone cannot drop any data contained within the file itself.
	RangeDeletionsBytesEstimate uint64
	// Total size of value blocks and value index block.
	ValueBlocksSize uint64
	// CompressionType is the compression profile used for the table (or nil if
	// the profile name is not recognized).
	CompressionType *block.CompressionProfile
	// TombstoneDenseBlocksRatio is the ratio of data blocks in this table that
	// fulfills at least one of the following:
	// 1. The block contains at least options.Experimental.NumDeletionsThreshold
	//    point tombstones.
	// 2. The ratio of the uncompressed size of point tombstones to the
	//    uncompressed size of the block is at least
	//    options.Experimental.DeletionSizeRatioThreshold.
	// This statistic is used to determine eligibility for a tombstone density
	// compaction.
	TombstoneDenseBlocksRatio float64
	RawKeySize                uint64
	RawValueSize              uint64
}

TableStats contains statistics on a table used for compaction heuristics, and export via Metrics.

type Version

type Version struct {

	// L0SublevelFiles contains the L0 sublevels.
	L0SublevelFiles []LevelSlice

	Levels [NumLevels]LevelMetadata

	// RangeKeyLevels holds a subset of the same files as Levels that contain range
	// keys (i.e. fileMeta.HasRangeKeys == true). The memory amplification of this
	// duplication should be minimal, as range keys are expected to be rare.
	RangeKeyLevels [NumLevels]LevelMetadata

	// BlobFiles holds the set of physical blob files that are referenced by the
	// version. The BlobFileSet is responsible for maintaining reference counts
	// on physical blob files so that they remain on storage until they're no
	// longer referenced by any version.
	BlobFiles BlobFileSet

	// The callback to invoke when the last reference to a version is
	// removed. Will be called with list.mu held.
	Deleted func(obsolete ObsoleteFiles)

	// Stats holds aggregated stats about the version maintained from
	// version to version.
	Stats struct {
		// MarkedForCompaction records the count of files marked for
		// compaction within the version.
		MarkedForCompaction int
	}
	// contains filtered or unexported fields
}

Version is a collection of table metadata for on-disk tables at various levels. In-memory DBs are written to level-0 tables, and compactions migrate data from level N to level N+1. The tables map internal keys (which are a user key, a delete or set bit, and a sequence number) to user values.

The tables at level 0 are sorted by largest sequence number. Due to file ingestion, there may be overlap in the ranges of sequence numbers contain in level 0 sstables. In particular, it is valid for one level 0 sstable to have the seqnum range [1,100] while an adjacent sstable has the seqnum range [50,50]. This occurs when the [50,50] table was ingested and given a global seqnum. The ingestion code will have ensured that the [50,50] sstable will not have any keys that overlap with the [1,100] in the seqnum range [1,49]. The range of internal keys [fileMetadata.smallest, fileMetadata.largest] in each level 0 table may overlap.

The tables at any non-0 level are sorted by their internal key range and any two tables at the same non-0 level do not overlap.

The internal key ranges of two tables at different levels X and Y may overlap, for any X != Y.

Finally, for every internal key in a table at level X, there is no internal key in a higher level table that has both the same user key and a higher sequence number.

func NewInitialVersion added in v2.1.0

func NewInitialVersion(comparer *base.Comparer) *Version

NewInitialVersion creates a version with no files. The L0Organizer should be freshly created.

func NewVersionForTesting added in v2.1.0

func NewVersionForTesting(
	comparer *base.Comparer, l0Organizer *L0Organizer, files [7][]*TableMetadata,
) *Version

NewVersionForTesting constructs a new Version with the provided files. It requires the provided files are already well-ordered. The L0Organizer should be freshly created.

func ParseVersionDebug

func ParseVersionDebug(
	comparer *base.Comparer, l0Organizer *L0Organizer, s string,
) (*Version, error)

ParseVersionDebug parses a Version from its DebugString output.

func (*Version) AllLevelsAndSublevels added in v2.1.0

func (v *Version) AllLevelsAndSublevels() iter.Seq2[Layer, LevelSlice]

AllLevelsAndSublevels returns an iterator that produces a Layer, LevelSlice pair for each L0 sublevel (from top to bottom) and each level below L0.

func (*Version) CalculateInuseKeyRanges

func (v *Version) CalculateInuseKeyRanges(
	l0Organizer *L0Organizer, level, maxLevel int, smallest, largest []byte,
) []base.UserKeyBounds

CalculateInuseKeyRanges examines table metadata in levels [level, maxLevel] within bounds [smallest,largest], returning an ordered slice of key ranges that include all keys that exist within levels [level, maxLevel] and within [smallest,largest].

func (*Version) CheckOrdering

func (v *Version) CheckOrdering() error

CheckOrdering checks that the files are consistent with respect to increasing file numbers (for level 0 files) and increasing and non- overlapping internal key ranges (for level non-0 files).

func (*Version) Contains

func (v *Version) Contains(level int, m *TableMetadata) bool

Contains returns a boolean indicating whether the provided file exists in the version at the given level. If level is non-zero then Contains binary searches among the files. If level is zero, Contains scans the entire level.

func (*Version) DebugString

func (v *Version) DebugString() string

DebugString returns an alternative format to String() which includes sequence number and kind information for the sstable boundaries.

func (*Version) DebugStringFormatKey added in v2.1.0

func (v *Version) DebugStringFormatKey(fmtKey base.FormatKey) string

DebugStringFormatKey is like DebugString but allows overriding key formatting with the provided FormatKey.

func (*Version) Next

func (v *Version) Next() *Version

Next returns the next version in the list of versions.

func (*Version) Overlaps

func (v *Version) Overlaps(level int, bounds base.UserKeyBounds) LevelSlice

Overlaps returns all elements of v.files[level] whose user key range intersects the given bounds. If level is non-zero then the user key bounds of v.files[level] are assumed to not overlap (although they may touch). If level is zero then that assumption cannot be made, and the given bounds are expanded to the union of those matching bounds so far and the computation is repeated until the bounds stabilize. The returned files are a subsequence of the input files, i.e., the ordering is not changed.

func (*Version) Ref

func (v *Version) Ref()

Ref increments the version refcount.

func (*Version) Refs

func (v *Version) Refs() int32

Refs returns the number of references to the version.

func (*Version) String

func (v *Version) String() string

String implements fmt.Stringer, printing the TableMetadata for each level in the Version.

func (*Version) Unref

func (v *Version) Unref()

Unref decrements the version refcount. If the last reference to the version was removed, the version is removed from the list of versions and the Deleted callback is invoked. Requires that the VersionList mutex is NOT locked.

func (*Version) UnrefLocked

func (v *Version) UnrefLocked()

UnrefLocked decrements the version refcount. If the last reference to the version was removed, the version is removed from the list of versions and the Deleted callback is invoked. Requires that the VersionList mutex is already locked.

type VersionEdit

type VersionEdit struct {
	// ComparerName is the value of Options.Comparer.Name. This is only set in
	// the first VersionEdit in a manifest (either when the DB is created, or
	// when a new manifest is created) and is used to verify that the comparer
	// specified at Open matches the comparer that was previously used.
	ComparerName string

	// MinUnflushedLogNum is the smallest WAL log file number corresponding to
	// mutations that have not been flushed to an sstable.
	//
	// This is an optional field, and 0 represents it is not set.
	MinUnflushedLogNum base.DiskFileNum

	// ObsoletePrevLogNum is a historic artifact from LevelDB that is not used by
	// Pebble, RocksDB, or even LevelDB. Its use in LevelDB was deprecated in
	// 6/2011. We keep it around purely for informational purposes when
	// displaying MANIFEST contents.
	ObsoletePrevLogNum uint64

	// The next file number. A single counter is used to assign file numbers
	// for the WAL, MANIFEST, sstable, and OPTIONS files.
	NextFileNum uint64

	// LastSeqNum is an upper bound on the sequence numbers that have been
	// assigned in flushed WALs. Unflushed WALs (that will be replayed during
	// recovery) may contain sequence numbers greater than this value.
	LastSeqNum base.SeqNum

	// A file num may be present in both deleted files and new files when it
	// is moved from a lower level to a higher level (when the compaction
	// found that there was no overlapping file at the higher level).
	DeletedTables map[DeletedTableEntry]*TableMetadata
	NewTables     []NewTableEntry
	// CreatedBackingTables can be used to preserve the TableBacking associated
	// with a physical sstable. This is useful when virtual sstables in the
	// latest version are reconstructed during manifest replay, and we also need
	// to reconstruct the TableBacking which is required by these virtual
	// sstables.
	//
	// INVARIANT: The TableBacking associated with a physical sstable must only
	// be added as a backing file in the same version edit where the physical
	// sstable is first virtualized. This means that the physical sstable must
	// be present in DeletedFiles and that there must be at least one virtual
	// sstable with the same TableBacking as the physical sstable in NewFiles. A
	// file must be present in CreatedBackingTables in exactly one version edit.
	// The physical sstable associated with the TableBacking must also not be
	// present in NewFiles.
	CreatedBackingTables []*TableBacking
	// RemovedBackingTables is used to remove the TableBacking associated with a
	// virtual sstable. Note that a backing sstable can be removed as soon as
	// there are no virtual sstables in the latest version which are using the
	// backing sstable, but the backing sstable doesn't necessarily have to be
	// removed atomically with the version edit which removes the last virtual
	// sstable associated with the backing sstable. The removal can happen in a
	// future version edit.
	//
	// INVARIANT: A file must only be added to RemovedBackingTables if it was
	// added to CreateBackingTables in a prior version edit. The same version
	// edit also cannot have the same file present in both CreateBackingTables
	// and RemovedBackingTables. A file must be present in RemovedBackingTables
	// in exactly one version edit.
	RemovedBackingTables []base.DiskFileNum
	// NewBlobFiles holds the metadata for all new blob files introduced within
	// the version edit.
	NewBlobFiles []BlobFileMetadata
	// DeletedBlobFiles holds all physical blob files that became unused during
	// the version edit.
	//
	// A physical blob file may become unused if the corresponding BlobFileID
	// becomes unreferenced during the version edit. In this case the BlobFileID
	// is not referenced by any sstable in the resulting Version.
	//
	// A physical blob file may also become unused if it is being replaced by a
	// new physical blob file. In this case NewBlobFiles must contain a
	// BlobFileMetadata with the same BlobFileID.
	//
	// While replaying a MANIFEST, the values are nil. Otherwise the values must
	// not be nil.
	DeletedBlobFiles map[DeletedBlobFileEntry]*PhysicalBlobFile
}

VersionEdit holds the state for an edit to a Version along with other on-disk state (log numbers, next file number, and the last sequence number).

func ParseVersionEditDebug

func ParseVersionEditDebug(s string) (_ *VersionEdit, err error)

ParseVersionEditDebug parses a VersionEdit from its DebugString implementation.

It doesn't recognize all fields; this implementation can be filled in as needed.

func (*VersionEdit) DebugString

func (v *VersionEdit) DebugString(fmtKey base.FormatKey) string

DebugString is a more verbose version of String(). Use this in tests.

func (*VersionEdit) Decode

func (v *VersionEdit) Decode(r io.Reader) error

Decode decodes an edit from the specified reader.

Note that the Decode step will not set the TableBacking for virtual sstables and the responsibility is left to the caller. However, the Decode step will populate the NewFileEntry.BackingFileNum in VersionEdit.NewFiles.

func (*VersionEdit) Encode

func (v *VersionEdit) Encode(w io.Writer) error

Encode encodes an edit to the specified writer.

func (*VersionEdit) String

func (v *VersionEdit) String() string

String implements fmt.Stringer for a VersionEdit.

type VersionList

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

VersionList holds a list of versions. The versions are ordered from oldest to newest.

func (*VersionList) Back

func (l *VersionList) Back() *Version

Back returns the newest version in the list. Note that this version is only valid if Empty() returns true.

func (*VersionList) Empty

func (l *VersionList) Empty() bool

Empty returns true if the list is empty, and false otherwise.

func (*VersionList) Front

func (l *VersionList) Front() *Version

Front returns the oldest version in the list. Note that this version is only valid if Empty() returns true.

func (*VersionList) Init

func (l *VersionList) Init(mu *sync.Mutex)

Init initializes the version list.

func (*VersionList) PushBack

func (l *VersionList) PushBack(v *Version)

PushBack adds a new version to the back of the list. This new version becomes the "newest" version in the list.

func (*VersionList) Remove

func (l *VersionList) Remove(v *Version)

Remove removes the specified version from the list.

type VirtualBackings

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

VirtualBackings maintains information about the set of backings that support virtual tables in the latest version.

The VirtualBackings set internally maintains for each backing the number of virtual tables that use that backing and the sum of their virtual sizes. When a backing is added to the set, it initially is not associated with any tables. AddTable/RemoveTable are used to maintain the set of tables that are associated with a backing. Finally, a backing can only be removed from the set when it is no longer in use.

-- Protection API --

VirtualBackings exposes a Protect/Unprotect API. This is used to allow external file ingestions to reuse existing virtual backings. Because ingestions can run in parallel with other operations like compactions, it is possible for a backing to "go away" in-between the time the ingestion decides to use it and the time the ingestion installs a new version. The protection API solves this problem by keeping backings alive, even if they become otherwise unused by any tables.

Backing protection achieves two goals:

  • it must prevent the removal of the backing from the latest version, where removal means becoming part of a VersionEdit.RemovedBackingTables. This is achieved by treating the backing as "in use", preventing Unused() from reporting it.
  • it must prevent the backing from becoming obsolete (i.e. reaching a ref count of 0). To achieve this, VirtualBackings takes a ref on each backing when it is added; this ref must be released after the backing is removed (when it is ok for the backing to be reported as obsolete).

For example, say we have virtual table T1 with backing B1 and an ingestion tries to reuse the file. This is what will usually happen (the happy case):

  • latest version is V1 and it contains T1(B1).
  • ingestion request comes for another virtual portion of B1. Ingestion process finds B1 and calls Protect(B1).
  • ingestion completes, installs version V2 which has T1(B1) and a new T2(B1), and calls Unprotect(B1).

In this path, the Protect/Unprotect calls do nothing. But here is what could happen (the corner case):

  • latest version is V1 and it contains T1(B1).
  • ingestion request comes for another virtual portion of B1. Ingestion process finds B1 and calls Protect(B1).
  • compaction completes and installs version V2 which no longer has T1. But because B1 is protected, V2 still has B1.
  • ingestion completes, installs version V3 which has a new T2(B1) and calls Unprotect(B1).

If instead the ingestion fails to complete, the last step becomes:

  • ingestion fails, calls Unprotect(B1). B1 is now Unused() and the next version (applied by whatever next operation is) will remove B1.

func MakeVirtualBackings

func MakeVirtualBackings() VirtualBackings

MakeVirtualBackings returns empty initialized VirtualBackings.

func (*VirtualBackings) AddAndRef

func (bv *VirtualBackings) AddAndRef(backing *TableBacking)

AddAndRef adds a new backing to the set and takes a reference on it. Another backing for the same DiskFileNum must not exist.

The added backing is unused until it is associated with a table via AddTable or protected via Protect.

func (*VirtualBackings) AddTable

func (bv *VirtualBackings) AddTable(m *TableMetadata)

AddTable is used when a new table is using an exiting backing. The backing must be in the set already.

func (*VirtualBackings) Backings

func (bv *VirtualBackings) Backings() []*TableBacking

Backings returns all backings in the set, in unspecified order.

func (*VirtualBackings) DiskFileNums

func (bv *VirtualBackings) DiskFileNums() []base.DiskFileNum

DiskFileNums returns disk file nums of all the backing in the set, in sorted order.

func (*VirtualBackings) ForEach

func (bv *VirtualBackings) ForEach(fn func(backing *TableBacking))

ForEach calls fn on each backing, in unspecified order.

func (*VirtualBackings) Get

func (bv *VirtualBackings) Get(n base.DiskFileNum) (_ *TableBacking, ok bool)

Get returns the backing with the given DiskFileNum, if it is in the set.

func (*VirtualBackings) Protect

func (bv *VirtualBackings) Protect(n base.DiskFileNum)

Protect prevents a backing from being reported as unused until a corresponding Unprotect call is made. The backing must be in the set.

Multiple Protect calls can be made for the same backing; each must have a corresponding Unprotect call before the backing can become unused.

func (*VirtualBackings) Remove

func (bv *VirtualBackings) Remove(n base.DiskFileNum)

Remove removes a backing. The backing must not be in use; normally backings are removed once they are reported by Unused().

It is up to the caller to release the reference took by AddAndRef.

func (*VirtualBackings) RemoveTable

func (bv *VirtualBackings) RemoveTable(m *TableMetadata)

RemoveTable is used when a table using a backing is removed. The backing is not removed from the set, even if it becomes unused.

func (*VirtualBackings) Stats

func (bv *VirtualBackings) Stats() (count int, totalSize uint64)

Stats returns the number and total size of all the virtual backings.

func (*VirtualBackings) String

func (bv *VirtualBackings) String() string

func (*VirtualBackings) Unprotect

func (bv *VirtualBackings) Unprotect(n base.DiskFileNum)

Unprotect reverses a Protect call.

func (*VirtualBackings) Unused

func (bv *VirtualBackings) Unused() []*TableBacking

Unused returns all backings that are and no longer used by the latest version and are not protected, in DiskFileNum order.

func (*VirtualBackings) Usage

func (bv *VirtualBackings) Usage(n base.DiskFileNum) (useCount int, virtualizedSize uint64)

Usage returns information about the usage of a backing, specifically:

  • useCount: the number of virtual tables that use this backing;
  • virtualizedSize: the sum of sizes of virtual tables that use the backing.

During compaction picking, we compensate a virtual sstable file size by (TableBacking.Size - virtualizedSize) / useCount. The intuition is that if TableBacking.Size - virtualizedSize is high, then the space amplification due to virtual sstables is high, and we should pick the virtual sstable with a higher priority.

Jump to

Keyboard shortcuts

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