manifest

package
v1.1.0 Latest Latest
Warning

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

Go to latest
Published: Jan 9, 2024 License: BSD-3-Clause Imports: 17 Imported by: 0

Documentation

Index

Constants

View Source
const NumLevels = 7

NumLevels is the number of levels a Version contains.

Variables

This section is empty.

Functions

func CheckOrdering

func CheckOrdering(
	cmp Compare, format base.FormatKey, level Level, files LevelIterator, ordering OrderingInvariants,
) 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).

The ordering field may be passed AllowSplitUserKeys to allow adjacent files that are both inclusive of the same user key. Pebble no longer creates version edits installing such files, and Pebble databases with sufficiently high format major version should no longer have any such files within their LSM. TODO(jackson): Remove AllowSplitUserKeys when we remove support for the earlier format major versions.

func LevelToInt

func LevelToInt(l Level) int

LevelToInt returns the int representation of a Level

func SortBySeqNum

func SortBySeqNum(files []*FileMetadata)

SortBySeqNum sorts the specified files by increasing sequence number.

func SortBySmallest

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

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

Types

type Annotator

type Annotator 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 interface{}) (v interface{})

	// 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(m *FileMetadata, dst interface{}) (v interface{}, 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 interface{}, dst interface{}) interface{}
}

An Annotator defines a computation over a level's FileMetadata. If the computation is stable and uses inputs that are fixed for the lifetime of a FileMetadata, 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.

type BulkVersionEdit

type BulkVersionEdit struct {
	Added   [NumLevels]map[base.FileNum]*FileMetadata
	Deleted [NumLevels]map[base.FileNum]*FileMetadata

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

	// AddedByFileNum maps file number to file metadata for all added files
	// from accumulated version edits. AddedByFileNum 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 AddedByFileNum to correctly populate the BulkVersionEdit's Deleted
	// field with non-nil *FileMetadata.
	AddedByFileNum map[base.FileNum]*FileMetadata

	// 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 file must not be added and removed from a given level 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: If a file 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.

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,
	cmp Compare,
	formatKey base.FormatKey,
	flushSplitBytes int64,
	readCompactionRate int64,
	zombies map[base.DiskFileNum]uint64,
	orderingInvariants OrderingInvariants,
) (*Version, error)

Apply applies the delta b to the current version to produce a new version. The new version is consistent with respect to the comparer cmp.

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

On success, if a non-nil zombies map is provided to Apply, the map is updated with file numbers and files sizes of deleted files. These files are considered zombies because they are no longer referenced by the returned Version, but cannot be deleted from disk as they are still in use by the incoming Version.

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 DeletedFileEntry

type DeletedFileEntry struct {
	Level   int
	FileNum base.FileNum
}

DeletedFileEntry holds the state for a file deletion from a level. The file itself might still be referenced by another level.

type FileBacking added in v1.1.0

type FileBacking struct {

	// VirtualizedSize is set iff the backing sst is only referred to by
	// virtual ssts in the latest version. VirtualizedSize is the sum of the
	// virtual sstable sizes of all of the virtual sstables in the latest
	// version which are backed by the physical sstable. When a virtual
	// sstable is removed from the latest version, we will decrement the
	// VirtualizedSize. During compaction picking, we'll compensate a
	// virtual sstable file size by
	// (FileBacking.Size - FileBacking.VirtualizedSize) / latestVersionRefs.
	// The intuition is that if FileBacking.Size - FileBacking.VirtualizedSize
	// is high, then the space amplification due to virtual sstables is
	// high, and we should pick the virtual sstable with a higher priority.
	//
	// TODO(bananabrick): Compensate the virtual sstable file size using
	// the VirtualizedSize during compaction picking and test.
	VirtualizedSize atomic.Uint64
	DiskFileNum     base.DiskFileNum
	Size            uint64
	// contains filtered or unexported fields
}

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

See the comment above the FileMetadata type for sstable terminology.

type FileMetadata

type FileMetadata 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

	// FileBacking is the state which backs either a physical or virtual
	// sstables.
	FileBacking *FileBacking

	// 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
	// FileNum is the file number.
	//
	// INVARIANT: when !FileMetadata.Virtual, FileNum == FileBacking.DiskFileNum.
	FileNum base.FileNum
	// Size is the size of the file, in bytes. Size is an approximate value for
	// virtual sstables.
	//
	// INVARIANTS:
	// - When !FileMetadata.Virtual, Size == FileBacking.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 FileMetadata for the virtual sstable was first created.
	CreationTime int64
	// 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 uint64
	LargestSeqNum  uint64
	// SmallestPointKey and LargestPointKey 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.
	SmallestPointKey InternalKey
	LargestPointKey  InternalKey
	// SmallestRangeKey and LargestRangeKey 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.
	SmallestRangeKey InternalKey
	LargestRangeKey  InternalKey
	// Smallest and Largest are the inclusive bounds for the internal keys stored
	// in the table, across both point and range keys.
	// NB: these fields are derived from their point and range key equivalents,
	// and are updated via the MaybeExtend{Point,Range}KeyBounds methods.
	Smallest InternalKey
	Largest  InternalKey
	// 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.
	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 FileMetadata belongs to a virtual sstable.
	Virtual bool
	// contains filtered or unexported fields
}

FileMetadata is maintained for leveled-ssts, i.e., they belong to a level of some version. FileMetadata does not contain the actual level of the sst, since such leveled-ssts can move across levels in different versions, while sharing the same FileMetadata. There are two kinds of leveled-ssts, physical and virtual. Underlying both leveled-ssts is a backing-sst, for which the only state is FileBacking. 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 FileMetadata.FileNum, and there will be exactly one FileMetadata associated with the FileNum.

  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 file metadata associated with it. We will still maintain the FileBacking associated with the physical sst if the backing sst is required by any virtual ssts in any version.

func ParseFileMetadataDebug

func ParseFileMetadataDebug(s string) (*FileMetadata, error)

ParseFileMetadataDebug parses a FileMetadata from its DebugString representation.

func (*FileMetadata) ContainedWithinSpan added in v1.1.0

func (m *FileMetadata) 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 (*FileMetadata) ContainsKeyType

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

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

func (*FileMetadata) DebugString

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

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

func (*FileMetadata) ExtendPointKeyBounds

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

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 (*FileMetadata) ExtendRangeKeyBounds

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

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 (*FileMetadata) InitPhysicalBacking added in v1.1.0

func (m *FileMetadata) InitPhysicalBacking()

InitPhysicalBacking allocates and sets the FileBacking which is required by a physical sstable FileMetadata.

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

func (*FileMetadata) InitProviderBacking added in v1.1.0

func (m *FileMetadata) InitProviderBacking(fileNum base.DiskFileNum)

InitProviderBacking creates a new FileBacking for a file backed by an objstorage.Provider.

func (*FileMetadata) IsCompacting

func (m *FileMetadata) IsCompacting() bool

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

func (*FileMetadata) LargestBound

func (m *FileMetadata) 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 (*FileMetadata) LatestRef added in v1.1.0

func (m *FileMetadata) LatestRef()

LatestRef increments the latest ref count associated with the backing sstable.

func (*FileMetadata) LatestRefs added in v1.1.0

func (m *FileMetadata) LatestRefs() int32

LatestRefs returns the latest ref count associated with the backing sstable.

func (*FileMetadata) LatestUnref added in v1.1.0

func (m *FileMetadata) LatestUnref() int32

LatestUnref decrements the latest ref count associated with the backing sstable.

func (*FileMetadata) Overlaps

func (m *FileMetadata) Overlaps(cmp Compare, start []byte, end []byte, exclusiveEnd bool) bool

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

func (*FileMetadata) PhysicalMeta added in v1.1.0

func (m *FileMetadata) PhysicalMeta() PhysicalFileMeta

PhysicalMeta should be the only source of creating the PhysicalFileMeta wrapper type.

func (*FileMetadata) Ref added in v1.1.0

func (m *FileMetadata) Ref()

Ref increments the ref count associated with the backing sstable.

func (*FileMetadata) Refs

func (m *FileMetadata) Refs() int32

Refs returns the refcount of backing sstable.

func (*FileMetadata) SetCompactionState

func (m *FileMetadata) SetCompactionState(to CompactionState)

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

func (*FileMetadata) SmallestBound

func (m *FileMetadata) 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 (*FileMetadata) StatsMarkValid

func (m *FileMetadata) 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 (*FileMetadata) StatsValid

func (m *FileMetadata) 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 (*FileMetadata) String

func (m *FileMetadata) String() string

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

func (*FileMetadata) TableInfo

func (m *FileMetadata) TableInfo() TableInfo

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

func (*FileMetadata) Unref added in v1.1.0

func (m *FileMetadata) Unref() int32

Unref decrements the ref count associated with the backing sstable.

func (*FileMetadata) Validate

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

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

func (*FileMetadata) ValidateVirtual added in v1.1.0

func (m *FileMetadata) ValidateVirtual(createdFrom *FileMetadata)

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

func (*FileMetadata) VirtualMeta added in v1.1.0

func (m *FileMetadata) VirtualMeta() VirtualFileMeta

VirtualMeta should be the only source of creating the VirtualFileMeta wrapper type.

type InternalKey

type InternalKey = base.InternalKey

InternalKey exports the base.InternalKey type.

func KeyRange

func KeyRange(ucmp Compare, iters ...LevelIterator) (smallest, largest InternalKey)

KeyRange returns the minimum smallest and maximum largest internalKey for all the FileMetadata in iters.

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 {
	Smallest  InternalKey
	Largest   InternalKey
	IsIntraL0 bool
}

L0Compaction describes an active compaction with inputs from L0.

type L0CompactionFiles

type L0CompactionFiles struct {
	Files []*FileMetadata

	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 L0Sublevels

type L0Sublevels struct {
	// Levels are ordered from oldest sublevel to youngest sublevel in the
	// outer slice, and the inner slice contains non-overlapping files for
	// that sublevel in increasing key order. Levels is constructed from
	// levelFiles and is used by callers that require a LevelSlice. The below two
	// fields are treated as immutable once created in NewL0Sublevels.
	Levels []LevelSlice
	// contains filtered or unexported fields
}

L0Sublevels represents a sublevel view of SSTables in L0. Tables in one sublevel are non-overlapping in key ranges, and keys in higher-indexed sublevels shadow older versions in lower-indexed sublevels. These invariants are similar to the regular level invariants, except with higher indexed sublevels having newer keys as opposed to lower indexed levels.

There is no limit to the number of sublevels that can exist in L0 at any time, however read and compaction performance is best when there are as few sublevels as possible.

func NewL0Sublevels

func NewL0Sublevels(
	levelMetadata *LevelMetadata, cmp Compare, formatKey base.FormatKey, flushSplitMaxBytes int64,
) (*L0Sublevels, error)

NewL0Sublevels creates an L0Sublevels instance for a given set of L0 files. These files must all be in L0 and must be sorted by seqnum (see SortBySeqNum). During interval iteration, when flushSplitMaxBytes bytes are exceeded in the range of intervals since the last flush split key, a flush split key is added.

This method can be called without DB.mu being held, so any DB.mu protected fields in FileMetadata cannot be accessed here, such as Compacting and IsIntraL0Compacting. Those fields are accessed in InitCompactingFileInfo instead.

func (*L0Sublevels) AddL0Files

func (s *L0Sublevels) AddL0Files(
	files []*FileMetadata, flushSplitMaxBytes int64, levelMetadata *LevelMetadata,
) (*L0Sublevels, error)

AddL0Files incrementally builds a new L0Sublevels for when the only change since the receiver L0Sublevels was an addition of the specified files, with no L0 deletions. The common case of this is an ingestion or a flush. These files can "sit on top" of existing sublevels, creating at most one new sublevel for a flush (and possibly multiple for an ingestion), and at most 2*len(files) additions to s.orderedIntervals. No files must have been deleted from L0, and the added files must all be newer in sequence numbers than existing files in L0Sublevels. The files parameter must be sorted in seqnum order. The levelMetadata parameter corresponds to the new L0 post addition of files. This method is meant to be significantly more performant than NewL0Sublevels.

Note that this function can only be called once on a given receiver; it appends to some slices in s which is only safe when done once. This is okay, as the common case (generating a new L0Sublevels after a flush/ingestion) is only going to necessitate one call of this method on a given receiver. The returned value, if non-nil, can then have *L0Sublevels.AddL0Files called on it again, and so on. If [errInvalidL0SublevelsOpt] is returned as an error, it likely means the optimization could not be applied (i.e. files added were older than files already in the sublevels, which is possible around ingestions and in tests). Eg. it can happen when an ingested file was ingested without queueing a flush since it did not actually overlap with any keys in the memtable. Later on the memtable was flushed, and the memtable had keys spanning around the ingested file, producing a flushed file that overlapped with the ingested file in file bounds but not in keys. It's possible for that flushed file to have a lower LargestSeqNum than the ingested file if all the additions after the ingestion were to another flushed file that was split into a separate sstable during flush. Any other non-nil error means L0Sublevels generation failed in the same way as NewL0Sublevels would likely fail.

func (*L0Sublevels) ExtendL0ForBaseCompactionTo

func (s *L0Sublevels) 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 (*L0Sublevels) FlushSplitKeys

func (s *L0Sublevels) 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 (*L0Sublevels) InUseKeyRanges

func (s *L0Sublevels) InUseKeyRanges(smallest, largest []byte) []UserKeyRange

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

func (*L0Sublevels) InitCompactingFileInfo

func (s *L0Sublevels) 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 (*L0Sublevels) MaxDepthAfterOngoingCompactions

func (s *L0Sublevels) 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 (*L0Sublevels) PickBaseCompaction

func (s *L0Sublevels) PickBaseCompaction(
	minCompactionDepth int, baseFiles LevelSlice,
) (*L0CompactionFiles, error)

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 (*L0Sublevels) PickIntraL0Compaction

func (s *L0Sublevels) PickIntraL0Compaction(
	earliestUnflushedSeqNum uint64, minCompactionDepth int,
) (*L0CompactionFiles, error)

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 (*L0Sublevels) ReadAmplification

func (s *L0Sublevels) 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 (*L0Sublevels) String

func (s *L0Sublevels) String() string

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

func (*L0Sublevels) UpdateStateForStartedCompaction

func (s *L0Sublevels) 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 [FileMetadata]s passed in.

type Level

type Level uint32

Level encodes a level and optional sublevel for use in log and error messages. The encoding has the property that Level(0) == L0Sublevel(invalidSublevel).

func L0Sublevel

func L0Sublevel(sublevel int) Level

L0Sublevel returns a Level representing the specified L0 sublevel.

func (Level) String

func (l Level) String() string

type LevelFile

type LevelFile struct {
	*FileMetadata
	// 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) Current

func (i *LevelIterator) Current() *FileMetadata

Current returns the item at the current iterator position.

Current is deprecated. Callers should instead use the return value of a positioning operation.

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() *FileMetadata

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

func (*LevelIterator) Last

func (i *LevelIterator) Last() *FileMetadata

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

func (*LevelIterator) Next

func (i *LevelIterator) Next() *FileMetadata

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

func (*LevelIterator) Prev

func (i *LevelIterator) Prev() *FileMetadata

Prev moves the iterator the previous file and returns it.

func (*LevelIterator) SeekGE

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

SeekGE seeks to the first file in the iterator's file set with a largest user key greater than or equal to the provided user key. The iterator must have been constructed from L1+, 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) *FileMetadata

SeekLT seeks to the last file in the iterator's file set with a smallest user key less than the provided user key. The iterator must have been constructed from L1+, 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
	// VirtualSize is the size of the virtual sstables in the level.
	VirtualSize uint64
	// contains filtered or unexported fields
}

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

func (*LevelMetadata) Annotation

func (lm *LevelMetadata) Annotation(annotator Annotator) interface{}

Annotation lazily calculates and returns the annotation defined by Annotator. The Annotator is used as the key for pre-calculated values, so equal Annotators must be used to avoid duplicate computations and cached annotations. Annotation must not be called concurrently, and in practice this is achieved by requiring callers to hold DB.mu.

func (*LevelMetadata) Empty

func (lm *LevelMetadata) Empty() bool

Empty indicates whether there are any files in the level.

func (*LevelMetadata) Find

func (lm *LevelMetadata) Find(cmp base.Compare, m *FileMetadata) *LevelFile

Find finds the provided file in the level if it exists.

func (*LevelMetadata) InvalidateAnnotation

func (lm *LevelMetadata) InvalidateAnnotation(annotator Annotator)

InvalidateAnnotation clears any cached annotations defined by Annotator. The Annotator is used as the key for pre-calculated values, so equal Annotators must be used to clear the appropriate cached annotation. InvalidateAnnotation must not be called concurrently, and in practice this is achieved by requiring callers to hold DB.mu.

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) Size added in v1.1.0

func (lm *LevelMetadata) Size() uint64

Size returns the cumulative size of all the files within the level.

func (*LevelMetadata) Slice

func (lm *LevelMetadata) Slice() LevelSlice

Slice constructs a slice containing the entire level.

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 []*FileMetadata) 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 []*FileMetadata) 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 []*FileMetadata) 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) Each

func (ls LevelSlice) Each(fn func(*FileMetadata))

Each invokes fn for each element 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 added in v1.1.0

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) 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) SizeSum

func (ls *LevelSlice) SizeSum() uint64

SizeSum sums the size of all files in the slice. Its runtime is linear in the length of the slice.

func (LevelSlice) String

func (ls LevelSlice) String() string

String implements fmt.Stringer.

func (*LevelSlice) VirtualSizeSum added in v1.1.0

func (ls *LevelSlice) VirtualSizeSum() uint64

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

type NewFileEntry

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

NewFileEntry holds the state for a new file or one moved from a different level.

type OrderingInvariants added in v1.1.0

type OrderingInvariants int8

OrderingInvariants dictates the file ordering invariants active.

const (
	// ProhibitSplitUserKeys indicates that adjacent files within a level cannot
	// contain the same user key.
	ProhibitSplitUserKeys OrderingInvariants = iota
	// AllowSplitUserKeys indicates that adjacent files within a level may
	// contain the same user key. This is only allowed by historical format
	// major versions.
	//
	// TODO(jackson): Remove.
	AllowSplitUserKeys
)

type PhysicalFileMeta added in v1.1.0

type PhysicalFileMeta struct {
	*FileMetadata
}

PhysicalFileMeta is used by functions which want a guarantee that their input belongs to a physical sst and not a virtual sst.

NB: This type should only be constructed by calling FileMetadata.PhysicalMeta.

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 uint64
	// LargestSeqNum is the largest sequence number in the table.
	LargestSeqNum uint64
}

TableInfo contains the common information for table related events.

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.
	//
	// 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
}

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

type UserKeyRange

type UserKeyRange struct {
	Start, End []byte
}

UserKeyRange encodes a key range in user key space. A UserKeyRange's Start and End boundaries are both inclusive.

type Version

type Version struct {

	// 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
	// higher level 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.
	//
	// The L0Sublevels struct is mostly used for compaction picking. As most
	// internal data structures in it are only necessary for compaction picking
	// and not for iterator creation, the reference to L0Sublevels is nil'd
	// after this version becomes the non-newest version, to reduce memory
	// usage.
	//
	// L0Sublevels.Levels contains L0 files ordered by sublevels. All the files
	// in Levels[0] are in L0Sublevels.Levels. L0SublevelFiles is also set to
	// a reference to that slice, as that slice is necessary for iterator
	// creation and needs to outlast L0Sublevels.
	L0Sublevels     *L0Sublevels
	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

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

	// 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 file 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 AccumulateIncompleteAndApplySingleVE added in v1.1.0

func AccumulateIncompleteAndApplySingleVE(
	ve *VersionEdit,
	curr *Version,
	cmp Compare,
	formatKey base.FormatKey,
	flushSplitBytes int64,
	readCompactionRate int64,
	backingStateMap map[base.DiskFileNum]*FileBacking,
	addBackingFunc func(*FileBacking),
	removeBackingFunc func(base.DiskFileNum),
	orderingInvariants OrderingInvariants,
) (_ *Version, zombies map[base.DiskFileNum]uint64, _ error)

AccumulateIncompleteAndApplySingleVE should be called if a single version edit is to be applied to the provided curr Version and if the caller needs to update the versionSet.zombieTables map. This function exists separately from BulkVersionEdit.Apply because it is easier to reason about properties regarding BulkVersionedit.Accumulate/Apply and zombie table generation, if we know that exactly one version edit is being accumulated.

Note that the version edit passed into this function may be incomplete because compactions don't have the ref counting information necessary to populate VersionEdit.RemovedBackingTables. This function will complete such a version edit by populating RemovedBackingTables.

Invariant: Any file being deleted through ve must belong to the curr Version. We can't have a delete for some arbitrary file which does not exist in curr.

func NewVersion

func NewVersion(
	cmp Compare, formatKey base.FormatKey, flushSplitBytes int64, files [NumLevels][]*FileMetadata,
) *Version

NewVersion constructs a new Version with the provided files. It requires the provided files are already well-ordered. It's intended for testing.

func ParseVersionDebug

func ParseVersionDebug(
	cmp Compare, formatKey base.FormatKey, flushSplitBytes int64, s string,
) (*Version, error)

ParseVersionDebug parses a Version from its DebugString output.

func (*Version) CheckOrdering

func (v *Version) CheckOrdering(
	cmp Compare, format base.FormatKey, order OrderingInvariants,
) 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, cmp Compare, m *FileMetadata) 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(format base.FormatKey) string

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

func (*Version) InitL0Sublevels

func (v *Version) InitL0Sublevels(
	cmp Compare, formatKey base.FormatKey, flushSplitBytes int64,
) error

InitL0Sublevels initializes the L0Sublevels

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, cmp Compare, start, end []byte, exclusiveEnd bool,
) LevelSlice

Overlaps returns all elements of v.files[level] whose user key range intersects the given range. If level is non-zero then the user key ranges 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 [start, end] range is expanded to the union of those matching ranges so far and the computation is repeated until [start, end] stabilizes. 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 FileMetadata 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.FileNum

	// 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 base.FileNum

	// 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 uint64

	// 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).
	DeletedFiles map[DeletedFileEntry]*FileMetadata
	NewFiles     []NewFileEntry
	// CreatedBackingTables can be used to preserve the FileBacking 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 FileBacking which is required by these virtual
	// sstables.
	//
	// INVARIANT: The FileBacking 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 FileBacking as the physical sstable in NewFiles. A
	// file must be present in CreatedBackingTables in exactly one version edit.
	// The physical sstable associated with the FileBacking must also not be
	// present in NewFiles.
	CreatedBackingTables []*FileBacking
	// RemovedBackingTables is used to remove the FileBacking 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
}

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 (*VersionEdit) DebugString added in v1.1.0

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 FileBacking 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 VirtualFileMeta added in v1.1.0

type VirtualFileMeta struct {
	*FileMetadata
}

VirtualFileMeta is used by functions which want a guarantee that their input belongs to a virtual sst and not a physical sst.

A VirtualFileMeta inherits all the same fields as a FileMetadata. These fields have additional invariants imposed on them, and/or slightly varying meanings:

  • Smallest and Largest (and their counterparts {Smallest, Largest}{Point,Range}Key) 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 FileBacking.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.

NB: This type should only be constructed by calling FileMetadata.VirtualMeta.

Jump to

Keyboard shortcuts

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