base

package
v2.0.2 Latest Latest
Warning

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

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

Documentation

Index

Constants

View Source
const (
	InternalKeyKindDelete  InternalKeyKind = 0
	InternalKeyKindSet     InternalKeyKind = 1
	InternalKeyKindMerge   InternalKeyKind = 2
	InternalKeyKindLogData InternalKeyKind = 3

	// InternalKeyKindSingleDelete (SINGLEDEL) is a performance optimization
	// solely for compactions (to reduce write amp and space amp). Readers other
	// than compactions should treat SINGLEDEL as equivalent to a DEL.
	// Historically, it was simpler for readers other than compactions to treat
	// SINGLEDEL as equivalent to DEL, but as of the introduction of
	// InternalKeyKindSSTableInternalObsoleteBit, this is also necessary for
	// correctness.
	InternalKeyKindSingleDelete InternalKeyKind = 7
	//InternalKeyKindColumnFamilySingleDelete InternalKeyKind = 8
	//InternalKeyKindBeginPrepareXID          InternalKeyKind = 9
	//InternalKeyKindEndPrepareXID            InternalKeyKind = 10
	//InternalKeyKindCommitXID                InternalKeyKind = 11
	//InternalKeyKindRollbackXID              InternalKeyKind = 12
	//InternalKeyKindNoop                     InternalKeyKind = 13
	//InternalKeyKindColumnFamilyRangeDelete  InternalKeyKind = 14
	InternalKeyKindRangeDelete InternalKeyKind = 15

	// InternalKeyKindSeparator is a key used for separator / successor keys
	// written to sstable block indexes.
	//
	// NOTE: the RocksDB value has been repurposed. This was done to ensure that
	// keys written to block indexes with value "17" (when 17 happened to be the
	// max value, and InternalKeyKindMax was therefore set to 17), remain stable
	// when new key kinds are supported in Pebble.
	InternalKeyKindSeparator InternalKeyKind = 17

	// InternalKeyKindSetWithDelete keys are SET keys that have met with a
	// DELETE or SINGLEDEL key in a prior compaction. This key kind is
	// specific to Pebble. See
	// https://github.com/cockroachdb/pebble/v2/issues/1255.
	InternalKeyKindSetWithDelete InternalKeyKind = 18

	// InternalKeyKindRangeKeyDelete removes all range keys within a key range.
	// See the internal/rangekey package for more details.
	InternalKeyKindRangeKeyDelete InternalKeyKind = 19
	// InternalKeyKindRangeKeySet and InternalKeyKindRangeUnset represent
	// keys that set and unset values associated with ranges of key
	// space. See the internal/rangekey package for more details.
	InternalKeyKindRangeKeyUnset InternalKeyKind = 20
	InternalKeyKindRangeKeySet   InternalKeyKind = 21

	InternalKeyKindRangeKeyMin InternalKeyKind = InternalKeyKindRangeKeyDelete
	InternalKeyKindRangeKeyMax InternalKeyKind = InternalKeyKindRangeKeySet

	// InternalKeyKindIngestSST is used to distinguish a batch that corresponds to
	// the WAL entry for ingested sstables that are added to the flushable
	// queue. This InternalKeyKind cannot appear amongst other key kinds in a
	// batch (with the exception of alongside InternalKeyKindExcise), or in an sstable.
	InternalKeyKindIngestSST InternalKeyKind = 22

	// InternalKeyKindDeleteSized keys behave identically to
	// InternalKeyKindDelete keys, except that they hold an associated uint64
	// value indicating the (len(key)+len(value)) of the shadowed entry the
	// tombstone is expected to delete. This value is used to inform compaction
	// heuristics, but is not required to be accurate for correctness.
	InternalKeyKindDeleteSized InternalKeyKind = 23

	// InternalKeyKindExcise is used to persist the Excise part of an IngestAndExcise
	// to a WAL. An Excise is similar to a RangeDel+RangeKeyDel combined, in that it
	// deletes all point and range keys in a given key range while also immediately
	// truncating sstables to exclude this key span. This InternalKeyKind cannot
	// appear amongst other key kinds in a batch (with the exception of alongside
	// InternalKeyKindIngestSST), or in an sstable.
	InternalKeyKindExcise InternalKeyKind = 24

	// This maximum value isn't part of the file format. Future extensions may
	// increase this value.
	//
	// When constructing an internal key to pass to DB.Seek{GE,LE},
	// internalKeyComparer sorts decreasing by kind (after sorting increasing by
	// user key and decreasing by sequence number). Thus, use InternalKeyKindMax,
	// which sorts 'less than or equal to' any other valid internalKeyKind, when
	// searching for any kind of internal key formed by a certain user key and
	// seqNum.
	InternalKeyKindMax InternalKeyKind = 24

	// InternalKeyKindMaxForSSTable is the largest valid key kind that can exist
	// in an SSTable. This should usually equal InternalKeyKindMax, except
	// if the current InternalKeyKindMax is a kind that is never added to an
	// SSTable or memtable (eg. InternalKeyKindExcise).
	InternalKeyKindMaxForSSTable InternalKeyKind = InternalKeyKindDeleteSized

	// Internal to the sstable format. Not exposed by any sstable iterator.
	// Declared here to prevent definition of valid key kinds that set this bit.
	InternalKeyKindSSTableInternalObsoleteBit  InternalKeyKind = 64
	InternalKeyKindSSTableInternalObsoleteMask InternalKeyKind = 191

	// InternalKeyZeroSeqnumMaxTrailer is the largest trailer with a
	// zero sequence number.
	InternalKeyZeroSeqnumMaxTrailer InternalKeyTrailer = 255

	// A marker for an invalid key.
	InternalKeyKindInvalid InternalKeyKind = InternalKeyKindSSTableInternalObsoleteMask

	// InternalKeyRangeDeleteSentinel is the marker for a range delete sentinel
	// key. This sequence number and kind are used for the upper stable boundary
	// when a range deletion tombstone is the largest key in an sstable. This is
	// necessary because sstable boundaries are inclusive, while the end key of a
	// range deletion tombstone is exclusive.
	InternalKeyRangeDeleteSentinel = (InternalKeyTrailer(SeqNumMax) << 8) | InternalKeyTrailer(InternalKeyKindRangeDelete)

	// InternalKeyBoundaryRangeKey is the marker for a range key boundary. This
	// sequence number and kind are used during interleaved range key and point
	// iteration to allow an iterator to stop at range key start keys where
	// there exists no point key.
	InternalKeyBoundaryRangeKey = (InternalKeyTrailer(SeqNumMax) << 8) | InternalKeyTrailer(InternalKeyKindRangeKeySet)
)

These constants are part of the file format, and should not be changed.

View Source
const (
	DefaultBlockRestartInterval      = 16
	DefaultBlockSize                 = 4096
	DefaultBlockSizeThreshold        = 90
	SizeClassAwareBlockSizeThreshold = 60
)

SSTable block defaults.

View Source
const InternalTrailerLen = 8

InternalTrailerLen is the number of bytes used to encode InternalKey.Trailer.

View Source
const MaxShortAttribute = 7

MaxShortAttribute is the maximum value of the short attribute (3 bits).

View Source
const SeekGEFlagsNone = SeekGEFlags(0)

SeekGEFlagsNone is the default value of SeekGEFlags, with all flags disabled.

View Source
const SeekLTFlagsNone = SeekLTFlags(0)

SeekLTFlagsNone is the default value of SeekLTFlags, with all flags disabled.

Variables

View Source
var DefaultComparer = &Comparer{
	ComparePointSuffixes: bytes.Compare,
	CompareRangeSuffixes: bytes.Compare,
	Compare:              bytes.Compare,
	Equal:                bytes.Equal,

	AbbreviatedKey: func(key []byte) uint64 {
		if len(key) >= 8 {
			return binary.BigEndian.Uint64(key)
		}
		var v uint64
		for _, b := range key {
			v <<= 8
			v |= uint64(b)
		}
		return v << uint(8*(8-len(key)))
	},

	Split: DefaultSplit,

	FormatKey: DefaultFormatter,

	Separator: func(dst, a, b []byte) []byte {
		i, n := SharedPrefixLen(a, b), len(dst)
		dst = append(dst, a...)

		min := len(a)
		if min > len(b) {
			min = len(b)
		}
		if i >= min {

			return dst
		}

		if a[i] >= b[i] {

			return dst
		}

		if i < len(b)-1 || a[i]+1 < b[i] {
			i += n
			dst[i]++
			return dst[:i+1]
		}

		i += n + 1
		for ; i < len(dst); i++ {
			if dst[i] != 0xff {
				dst[i]++
				return dst[:i+1]
			}
		}
		return dst
	},

	Successor: func(dst, a []byte) (ret []byte) {
		for i := 0; i < len(a); i++ {
			if a[i] != 0xff {
				dst = append(dst, a[:i+1]...)
				dst[len(dst)-1]++
				return dst
			}
		}

		return append(dst, a...)
	},

	ImmediateSuccessor: func(dst, a []byte) (ret []byte) {
		return append(append(dst, a...), 0x00)
	},

	Name: "leveldb.BytewiseComparator",
}

DefaultComparer is the default implementation of the Comparer interface. It uses the natural ordering, consistent with bytes.Compare.

View Source
var DefaultLogger defaultLogger

DefaultLogger logs to the Go stdlib logs.

View Source
var DefaultMerger = &Merger{
	Merge: func(key, value []byte) (ValueMerger, error) {
		res := &AppendValueMerger{}
		res.buf = append(res.buf, value...)
		return res, nil
	},

	Name: "pebble.concatenate",
}

DefaultMerger is the default implementation of the Merger interface. It concatenates the two values to merge.

View Source
var ErrCorruption = errors.New("pebble: corruption")

ErrCorruption is a marker to indicate that data in a file (WAL, MANIFEST, sstable) isn't in the expected format.

View Source
var ErrNotFound = errors.New("pebble: not found")

ErrNotFound means that a get or delete call did not find the requested key.

InvalidInternalKey is an invalid internal key for which Valid() will return false.

Functions

func AssertionFailedf

func AssertionFailedf(format string, args ...interface{}) error

AssertionFailedf creates an assertion error and panics in invariants.Enabled builds. It should only be used when it indicates a bug.

func CatchErrorPanic

func CatchErrorPanic(f func() error) (err error)

CatchErrorPanic runs a function and catches any panic that contains an error, returning that error. Used in tests, in particular to catch panics threw by AssertionFailedf.

func CheckComparer

func CheckComparer(c *Comparer, prefixes [][]byte, suffixes [][]byte) error

CheckComparer is a mini test suite that verifies a comparer implementation.

It takes lists of valid prefixes and suffixes. It is recommended that both lists have at least three elements.

func CloseHelper

func CloseHelper(closer io.Closer) io.Closer

CloseHelper wraps an io.Closer in a wrapper that ignores extra calls to Close. It is useful to ensure cleanup in error paths (using defer) without double-closing.

func CorruptionErrorf

func CorruptionErrorf(format string, args ...interface{}) error

CorruptionErrorf formats according to a format specifier and returns the string as an error value that is marked as a corruption error.

func DebugTree

func DebugTree(iter IteratorDebug) string

DebugTree returns the iterator tree as a multi-line string.

func InternalCompare

func InternalCompare(userCmp Compare, a, b InternalKey) int

InternalCompare compares two internal keys using the specified comparison function. For equal user keys, internal keys compare in descending sequence number order. For equal user keys and sequence numbers, internal keys compare in descending kind order (this may happen in practice among range keys).

func MakeFilename

func MakeFilename(fileType FileType, dfn DiskFileNum) string

MakeFilename builds a filename from components.

func MakeFilepath

func MakeFilepath(fs vfs.FS, dirname string, fileType FileType, dfn DiskFileNum) string

MakeFilepath builds a filepath from components.

func MarkCorruptionError

func MarkCorruptionError(err error) error

MarkCorruptionError marks given error as a corruption error.

func MinUserKey

func MinUserKey(cmp Compare, a, b []byte) []byte

MinUserKey returns the smaller of two user keys. If one of the keys is nil, the other one is returned.

func MustExist

func MustExist(fs vfs.FS, filename string, fataler Fataler, err error)

MustExist checks if err is an error indicating a file does not exist. If it is, it lists the containing directory's files to annotate the error with counts of the various types of files and invokes the provided fataler. See cockroachdb/cockroach#56490.

func ParseFilename

func ParseFilename(fs vfs.FS, filename string) (fileType FileType, dfn DiskFileNum, ok bool)

ParseFilename parses the components from a filename.

func SharedPrefixLen

func SharedPrefixLen(a, b []byte) int

SharedPrefixLen returns the largest i such that a[:i] equals b[:i]. This function can be useful in implementing the Comparer interface.

func Visible

func Visible(seqNum SeqNum, snapshot, batchSnapshot SeqNum) bool

Visible returns true if a key with the provided sequence number is visible at the specified snapshot sequence numbers.

Types

type AbbreviatedKey

type AbbreviatedKey func(key []byte) uint64

AbbreviatedKey returns a fixed length prefix of a user key such that

AbbreviatedKey(a) < AbbreviatedKey(b) implies a < b, and
AbbreviatedKey(a) > AbbreviatedKey(b) implies a > b.

If AbbreviatedKey(a) == AbbreviatedKey(b), an additional comparison is required to determine if the two keys are actually equal.

This helps optimize indexed batch comparisons for cache locality. If a Split function is specified, AbbreviatedKey usually returns the first eight bytes of the user key prefix in the order that gives the correct ordering.

type AppendValueMerger

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

AppendValueMerger concatenates merge operands in order from oldest to newest.

func (*AppendValueMerger) Finish

func (a *AppendValueMerger) Finish(includesBase bool) ([]byte, io.Closer, error)

Finish returns the buffer that was constructed on-demand in `Merge{OlderNewer}()` calls.

func (*AppendValueMerger) MergeNewer

func (a *AppendValueMerger) MergeNewer(value []byte) error

MergeNewer appends value to the result.

func (*AppendValueMerger) MergeOlder

func (a *AppendValueMerger) MergeOlder(value []byte) error

MergeOlder prepends value to the result, which involves allocating a new buffer.

type ArchiveCleaner

type ArchiveCleaner struct{}

ArchiveCleaner archives file instead delete.

func (ArchiveCleaner) Clean

func (ArchiveCleaner) Clean(fs vfs.FS, fileType FileType, path string) error

Clean archives file.

TODO(sumeer): for log files written to the secondary FS, the archiving will also write to the secondary. We should consider archiving to the primary.

func (ArchiveCleaner) String

func (ArchiveCleaner) String() string

type AtomicSeqNum

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

AtomicSeqNum is an atomic SeqNum.

func (*AtomicSeqNum) Add

func (asn *AtomicSeqNum) Add(delta SeqNum) SeqNum

Add atomically adds delta to asn and returns the new value.

func (*AtomicSeqNum) CompareAndSwap

func (asn *AtomicSeqNum) CompareAndSwap(old, new SeqNum) bool

CompareAndSwap executes the compare-and-swap operation.

func (*AtomicSeqNum) Load

func (asn *AtomicSeqNum) Load() SeqNum

Load atomically loads and returns the stored SeqNum.

func (*AtomicSeqNum) Store

func (asn *AtomicSeqNum) Store(s SeqNum)

Store atomically stores s.

type AttributeAndLen

type AttributeAndLen struct {
	ValueLen       int32
	ShortAttribute ShortAttribute
}

AttributeAndLen represents the pair of value length and the short attribute.

type BlockPropertyFilter

type BlockPropertyFilter interface {
	// Name returns the name of the block property collector.
	Name() string
	// Intersects returns true if the set represented by prop intersects with
	// the set in the filter.
	Intersects(prop []byte) (bool, error)
	// SyntheticSuffixIntersects runs Intersects, but only after using the passed in
	// suffix arg to modify a decoded copy of the passed in prop. This method only
	// needs to be implemented for filters which that will be used with suffix
	// replacement.
	SyntheticSuffixIntersects(prop []byte, suffix []byte) (bool, error)
}

BlockPropertyFilter is used in an Iterator to filter sstables and blocks within the sstable. It should not maintain any per-sstable state, and must be thread-safe.

type BoundaryKind

type BoundaryKind uint8

BoundaryKind indicates if a boundary is exclusive or inclusive.

const (
	Exclusive BoundaryKind = iota
	Inclusive
)

The two possible values of BoundaryKind.

Note that we prefer Exclusive to be the zero value, so that zero UserKeyBounds are not valid.

type Cleaner

type Cleaner interface {
	Clean(fs vfs.FS, fileType FileType, path string) error
}

Cleaner cleans obsolete files.

type Compare

type Compare func(a, b []byte) int

Compare returns -1, 0, or +1 depending on whether a is 'less than', 'equal to' or 'greater than' b.

Both a and b must be valid keys. Note that because of synthetic prefix functionality, the Compare function can be called on a key (either from the database or passed as an argument for an iterator operation) after the synthetic prefix has been removed. In general, this implies that removing any leading bytes from a prefix must yield another valid prefix.

A key a is less than b if a's prefix is byte-wise less than b's prefix, or if the prefixes are equal and a's suffix is less than b's suffix (according to ComparePointSuffixes).

In other words, if prefix(a) = a[:Split(a)] and suffix(a) = a[Split(a):]:

  Compare(a, b) = bytes.Compare(prefix(a), prefix(b)) if not 0,
	                otherwise ComparePointSuffixes(suffix(a), suffix(b))

Compare defaults to using the formula above but it can be customized if there is a (potentially faster) specialization or it has to compare suffixes differently.

type ComparePointSuffixes

type ComparePointSuffixes func(a, b []byte) int

ComparePointSuffixes compares two point key suffixes and returns -1, 0, or +1.

For historical reasons (see https://github.com/cockroachdb/cockroach/issues/130533 for a summary), this function is distinct from CompareRangeSuffixes. Specifically, ComparePointSuffixes may treat two suffixes as equal whereas CompareRangeSuffixes might not. Unlike CompareRangeSuffixes, this function must agree with Compare.

The empty slice suffix must be 'less than' any non-empty suffix.

A full key k is composed of a prefix k[:Split(k)] and suffix k[Split(k):]. Suffixes are compared to break ties between equal prefixes.

type CompareRangeSuffixes

type CompareRangeSuffixes func(a, b []byte) int

CompareRangeSuffixes compares two suffixes where either or both suffix originates from a range key and returns -1, 0, or +1.

For historical reasons (see https://github.com/cockroachdb/cockroach/issues/130533 for a summary), we allow this function to be more strict than Compare. Specifically, Compare may treat two suffixes as equal whereas CompareRangeSuffixes might not.

CompareRangeSuffixes is allowed to be more strict than see ComparePointSuffixes, meaning it may return -1 or +1 when ComparePointSuffixes would return 0.

The empty slice suffix must be 'less than' any non-empty suffix.

type Comparer

type Comparer struct {
	// The following must always be specified.
	AbbreviatedKey AbbreviatedKey
	Separator      Separator
	Successor      Successor

	// ImmediateSuccessor must be specified if range keys are used.
	ImmediateSuccessor ImmediateSuccessor

	// Split defaults to a trivial implementation that returns the full key length
	// if it is not specified.
	Split Split

	// CompareRangeSuffixes defaults to bytes.Compare if it is not specified.
	CompareRangeSuffixes CompareRangeSuffixes
	// ComparePointSuffixes defaults to bytes.Compare if it is not specified.
	ComparePointSuffixes ComparePointSuffixes

	// Compare defaults to a generic implementation that uses Split,
	// bytes.Compare, and ComparePointSuffixes if it is not specified.
	Compare Compare
	// Equal defaults to using Compare() == 0 if it is not specified.
	Equal Equal
	// FormatKey defaults to the DefaultFormatter if it is not specified.
	FormatKey FormatKey

	// FormatValue is optional.
	FormatValue FormatValue

	// Name is the name of the comparer.
	//
	// The on-disk format stores the comparer name, and opening a database with a
	// different comparer from the one it was created with will result in an
	// error.
	Name string
}

Comparer defines a total ordering over the space of []byte keys: a 'less than' relationship.

func MakeAssertComparer

func MakeAssertComparer(c Comparer) Comparer

MakeAssertComparer creates a Comparer that is the same with the given Comparer except that it asserts that the Compare and Equal functions adhere to their specifications.

func (*Comparer) EnsureDefaults

func (c *Comparer) EnsureDefaults() *Comparer

EnsureDefaults ensures that all non-optional fields are set.

If c is nil, returns DefaultComparer.

If any fields need to be set, returns a modified copy of c.

type DeletableValueMerger

type DeletableValueMerger interface {
	ValueMerger

	// DeletableFinish enables a value merger to indicate that the result of a merge operation
	// is non-existent. See Finish for a description of includesBase.
	DeletableFinish(includesBase bool) (value []byte, delete bool, closer io.Closer, err error)
}

DeletableValueMerger is an extension to ValueMerger which allows indicating that the result of a merge operation is non-existent. Such non-existent entries will eventually be deleted during compaction. Note that during compaction, non-existence of the result of a merge means that the merge operands will not result in any record being output. This is not the same as transforming the merge operands into a deletion tombstone, as older merge operands will still be visible during iteration. Deletion of the merge operands in this way is akin to the way a SingleDelete+Set combine into non-existence while leaving older records for the same key unaffected.

type DeleteCleaner

type DeleteCleaner struct{}

DeleteCleaner deletes file.

func (DeleteCleaner) Clean

func (DeleteCleaner) Clean(fs vfs.FS, fileType FileType, path string) error

Clean removes file.

func (DeleteCleaner) String

func (DeleteCleaner) String() string

type DiskFileNum

type DiskFileNum uint64

A DiskFileNum identifies a file or object with exists on disk.

func ParseDiskFileNum

func ParseDiskFileNum(s string) (dfn DiskFileNum, ok bool)

ParseDiskFileNum parses the provided string as a disk file number.

func PhysicalTableDiskFileNum

func PhysicalTableDiskFileNum(n FileNum) DiskFileNum

PhysicalTableDiskFileNum converts the FileNum of a physical table to the backing DiskFileNum. The underlying numbers always match for physical tables.

func (DiskFileNum) SafeFormat

func (dfn DiskFileNum) SafeFormat(w redact.SafePrinter, verb rune)

SafeFormat implements redact.SafeFormatter.

func (DiskFileNum) String

func (dfn DiskFileNum) String() string

type Equal

type Equal func(a, b []byte) bool

Equal returns true if a and b are equivalent.

For a given Compare, Equal(a,b)=true iff Compare(a,b)=0; that is, Equal is a (potentially faster) specialization of Compare.

type FakeIter

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

FakeIter is an iterator over a fixed set of KVs.

func NewFakeIter

func NewFakeIter(kvs []InternalKV) *FakeIter

NewFakeIter returns an iterator over the given KVs.

func (*FakeIter) Close

func (f *FakeIter) Close() error

Close is part of the InternalIterator interface.

func (*FakeIter) DebugTree

func (f *FakeIter) DebugTree(tp treeprinter.Node)

DebugTree is part of the InternalIterator interface.

func (*FakeIter) Error

func (f *FakeIter) Error() error

Error is part of the InternalIterator interface.

func (*FakeIter) First

func (f *FakeIter) First() *InternalKV

First is part of the InternalIterator interface.

func (*FakeIter) KV

func (f *FakeIter) KV() *InternalKV

KV is part of the InternalIterator interface.

func (*FakeIter) Last

func (f *FakeIter) Last() *InternalKV

Last is part of the InternalIterator interface.

func (*FakeIter) Next

func (f *FakeIter) Next() *InternalKV

Next is part of the InternalIterator interface.

func (*FakeIter) NextPrefix

func (f *FakeIter) NextPrefix(succKey []byte) *InternalKV

NextPrefix is part of the InternalIterator interface.

func (*FakeIter) Prev

func (f *FakeIter) Prev() *InternalKV

Prev is part of the InternalIterator interface.

func (*FakeIter) SeekGE

func (f *FakeIter) SeekGE(key []byte, flags SeekGEFlags) *InternalKV

SeekGE is part of the InternalIterator interface.

func (*FakeIter) SeekLT

func (f *FakeIter) SeekLT(key []byte, flags SeekLTFlags) *InternalKV

SeekLT is part of the InternalIterator interface.

func (*FakeIter) SeekPrefixGE

func (f *FakeIter) SeekPrefixGE(prefix, key []byte, flags SeekGEFlags) *InternalKV

SeekPrefixGE is part of the InternalIterator interface.

func (*FakeIter) SetBounds

func (f *FakeIter) SetBounds(lower, upper []byte)

SetBounds is part of the InternalIterator interface.

func (*FakeIter) SetCloseErr

func (f *FakeIter) SetCloseErr(closeErr error)

SetCloseErr causes future calls to Error() and Close() to return this error.

func (*FakeIter) SetContext

func (f *FakeIter) SetContext(_ context.Context)

SetContext is part of the InternalIterator interface.

func (*FakeIter) String

func (f *FakeIter) String() string

func (*FakeIter) Valid

func (f *FakeIter) Valid() bool

Valid is part of the InternalIterator interface.

type Fataler

type Fataler interface {
	Fatalf(format string, args ...interface{})
}

A Fataler fatals a process with a message when called.

type FileInfo

type FileInfo struct {
	FileNum  DiskFileNum
	FileSize uint64
}

FileInfo provides some rudimentary information about a file.

type FileNum

type FileNum uint64

FileNum is an internal DB identifier for a table. Tables can be physical (in which case the FileNum also identifies the backing object) or virtual.

func PhysicalTableFileNum

func PhysicalTableFileNum(f DiskFileNum) FileNum

PhysicalTableFileNum converts the DiskFileNum backing a physical table into the table's FileNum. The underlying numbers always match for physical tables.

func (FileNum) SafeFormat

func (fn FileNum) SafeFormat(w redact.SafePrinter, _ rune)

SafeFormat implements redact.SafeFormatter.

func (FileNum) String

func (fn FileNum) String() string

String returns a string representation of the file number.

type FileType

type FileType int

FileType enumerates the types of files found in a DB.

const (
	FileTypeLog FileType = iota
	FileTypeLock
	FileTypeTable
	FileTypeManifest
	FileTypeOptions
	FileTypeOldTemp
	FileTypeTemp
)

The FileType enumeration.

type FilterPolicy

type FilterPolicy interface {
	// Name names the filter policy.
	Name() string

	// MayContain returns whether the encoded filter may contain given key.
	// False positives are possible, where it returns true for keys not in the
	// original set.
	MayContain(ftype FilterType, filter, key []byte) bool

	// NewWriter creates a new FilterWriter.
	NewWriter(ftype FilterType) FilterWriter
}

FilterPolicy is an algorithm for probabilistically encoding a set of keys. The canonical implementation is a Bloom filter.

Every FilterPolicy has a name. This names the algorithm itself, not any one particular instance. Aspects specific to a particular instance, such as the set of keys or any other parameters, will be encoded in the []byte filter returned by NewWriter.

The name may be written to files on disk, along with the filter data. To use these filters, the FilterPolicy name at the time of writing must equal the name at the time of reading. If they do not match, the filters will be ignored, which will not affect correctness but may affect performance.

type FilterType

type FilterType int

FilterType is the level at which to apply a filter: block or table.

const (
	TableFilter FilterType = iota
)

The available filter types.

func (FilterType) String

func (t FilterType) String() string

type FilterWriter

type FilterWriter interface {
	// AddKey adds a key to the current filter block.
	AddKey(key []byte)

	// Finish appends to dst an encoded filter tha holds the current set of
	// keys. The writer state is reset after the call to Finish allowing the
	// writer to be reused for the creation of additional filters.
	Finish(dst []byte) []byte
}

FilterWriter provides an interface for creating filter blocks. See FilterPolicy for more details about filters.

type FormatBytes

type FormatBytes []byte

FormatBytes formats a byte slice using hexadecimal escapes for non-ASCII data.

func (FormatBytes) Format

func (p FormatBytes) Format(s fmt.State, c rune)

Format implements the fmt.Formatter interface.

type FormatKey

type FormatKey func(key []byte) fmt.Formatter

FormatKey returns a formatter for the user key.

var DefaultFormatter FormatKey = func(key []byte) fmt.Formatter {
	return FormatBytes(key)
}

DefaultFormatter is the default implementation of user key formatting: non-ASCII data is formatted as escaped hexadecimal values.

type FormatValue

type FormatValue func(key, value []byte) fmt.Formatter

FormatValue returns a formatter for the user value. The key is also specified for the value formatter in order to support value formatting that is dependent on the key.

type GaugeSampleMetric

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

GaugeSampleMetric is used to measure a gauge value (e.g. queue length) by accumulating samples of that gauge.

func (*GaugeSampleMetric) AddSample

func (gsm *GaugeSampleMetric) AddSample(sample int64)

AddSample adds the given sample.

func (*GaugeSampleMetric) Mean

func (gsm *GaugeSampleMetric) Mean() float64

Mean returns the mean value.

func (*GaugeSampleMetric) Merge

func (gsm *GaugeSampleMetric) Merge(x GaugeSampleMetric)

Merge accumulates the information from another gauge metric.

func (*GaugeSampleMetric) Subtract

func (gsm *GaugeSampleMetric) Subtract(x GaugeSampleMetric)

Subtract subtracts the information from another gauge metric.

type ImmediateSuccessor

type ImmediateSuccessor func(dst, a []byte) []byte

ImmediateSuccessor is invoked with a prefix key ([Split(a) == len(a)]) and appends to dst the smallest prefix key that is larger than the given prefix a.

ImmediateSuccessor must generate a prefix key k such that:

Split(k) == len(k) and Compare(a, k) < 0

and there exists no representable prefix key k2 such that:

Split(k2) == len(k2) and Compare(a, k2) < 0 and Compare(k2, k) < 0

As an example, an implementation built on the natural byte ordering using bytes.Compare could append a `\0` to `a`.

The appended key must be valid to pass to Compare.

type InMemLogger

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

InMemLogger implements Logger using an in-memory buffer (used for testing). The buffer can be read via String() and cleared via Reset().

func (*InMemLogger) Errorf

func (b *InMemLogger) Errorf(format string, args ...interface{})

Errorf is part of the Logger interface.

func (*InMemLogger) Fatalf

func (b *InMemLogger) Fatalf(format string, args ...interface{})

Fatalf is part of the Logger interface.

func (*InMemLogger) Infof

func (b *InMemLogger) Infof(format string, args ...interface{})

Infof is part of the Logger interface.

func (*InMemLogger) Reset

func (b *InMemLogger) Reset()

Reset clears the internal buffer.

func (*InMemLogger) String

func (b *InMemLogger) String() string

String returns the current internal buffer.

type InternalIterator

type InternalIterator interface {
	// SeekGE moves the iterator to the first key/value pair whose key is greater
	// than or equal to the given key. Returns the key and value if the iterator
	// is pointing at a valid entry, and (nil, nilv) otherwise. Note that SeekGE
	// only checks the upper bound. It is up to the caller to ensure that key
	// is greater than or equal to the lower bound.
	SeekGE(key []byte, flags SeekGEFlags) *InternalKV

	// SeekPrefixGE moves the iterator to the first key/value pair whose key is
	// greater than or equal to the given key. Returns the key and value if the
	// iterator is pointing at a valid entry, and (nil, nilv) otherwise. Note that
	// SeekPrefixGE only checks the upper bound. It is up to the caller to ensure
	// that key is greater than or equal to the lower bound.
	//
	// The prefix argument is used by some InternalIterator implementations
	// (e.g.  sstable.Reader) to avoid expensive operations. This operation is
	// only useful when a user-defined Split function is supplied to the
	// Comparer for the DB. The supplied prefix will be the prefix of the given
	// key returned by that Split function. If the iterator is able to determine
	// that no key with the prefix exists, it can return (nil,nilv). Unlike
	// SeekGE, this is not an indication that iteration is exhausted. The prefix
	// byte slice is guaranteed to be stable until the next absolute positioning
	// operation.
	//
	// Note that the iterator may return keys not matching the prefix. It is up
	// to the caller to check if the prefix matches.
	//
	// Calling SeekPrefixGE places the receiver into prefix iteration mode. Once
	// in this mode, reverse iteration may not be supported and will return an
	// error. Note that pebble/Iterator.SeekPrefixGE has this same restriction on
	// not supporting reverse iteration in prefix iteration mode until a
	// different positioning routine (SeekGE, SeekLT, First or Last) switches the
	// iterator out of prefix iteration.
	SeekPrefixGE(prefix, key []byte, flags SeekGEFlags) *InternalKV

	// SeekLT moves the iterator to the last key/value pair whose key is less
	// than the given key. Returns the key and value if the iterator is pointing
	// at a valid entry, and (nil, nilv) otherwise. Note that SeekLT only checks
	// the lower bound. It is up to the caller to ensure that key is less than
	// the upper bound.
	SeekLT(key []byte, flags SeekLTFlags) *InternalKV

	// First moves the iterator the first key/value pair. Returns the key and
	// value if the iterator is pointing at a valid entry, and (nil, nilv)
	// otherwise. Note that First only checks the upper bound. It is up to the
	// caller to ensure that First() is not called when there is a lower bound,
	// and instead call SeekGE(lower).
	First() *InternalKV

	// Last moves the iterator the last key/value pair. Returns the key and
	// value if the iterator is pointing at a valid entry, and (nil, nilv)
	// otherwise. Note that Last only checks the lower bound. It is up to the
	// caller to ensure that Last() is not called when there is an upper bound,
	// and instead call SeekLT(upper).
	Last() *InternalKV

	// Next moves the iterator to the next key/value pair. Returns the key and
	// value if the iterator is pointing at a valid entry, and (nil, nilv)
	// otherwise. Note that Next only checks the upper bound. It is up to the
	// caller to ensure that key is greater than or equal to the lower bound.
	//
	// It is valid to call Next when the iterator is positioned before the first
	// key/value pair due to either a prior call to SeekLT or Prev which returned
	// (nil, nilv). It is not allowed to call Next when the previous call to SeekGE,
	// SeekPrefixGE or Next returned (nil, nilv).
	Next() *InternalKV

	// NextPrefix moves the iterator to the next key/value pair with a different
	// prefix than the key at the current iterator position. Returns the key and
	// value if the iterator is pointing at a valid entry, and (nil, nil)
	// otherwise. Note that NextPrefix only checks the upper bound. It is up to
	// the caller to ensure that key is greater than or equal to the lower
	// bound.
	//
	// NextPrefix is passed the immediate successor to the current prefix key. A
	// valid implementation of NextPrefix is to call SeekGE with succKey.
	//
	// It is not allowed to call NextPrefix when the previous call was a reverse
	// positioning operation or a call to a forward positioning method that
	// returned (nil, nilv). It is also not allowed to call NextPrefix when the
	// iterator is in prefix iteration mode.
	NextPrefix(succKey []byte) *InternalKV

	// Prev moves the iterator to the previous key/value pair. Returns the key
	// and value if the iterator is pointing at a valid entry, and (nil, nilv)
	// otherwise. Note that Prev only checks the lower bound. It is up to the
	// caller to ensure that key is less than the upper bound.
	//
	// It is valid to call Prev when the iterator is positioned after the last
	// key/value pair due to either a prior call to SeekGE or Next which returned
	// (nil, nilv). It is not allowed to call Prev when the previous call to SeekLT
	// or Prev returned (nil, nilv).
	Prev() *InternalKV

	// Error returns any accumulated error. It may not include errors returned
	// to the client when calling LazyValue.Value().
	Error() error

	// Close closes the iterator and returns any accumulated error. Exhausting
	// all the key/value pairs in a table is not considered to be an error.
	//
	// Once Close is called, the iterator should not be used again. Specific
	// implementations may support multiple calls to Close (but no other calls
	// after the first Close).
	Close() error

	// SetBounds sets the lower and upper bounds for the iterator. Note that the
	// result of Next and Prev will be undefined until the iterator has been
	// repositioned with SeekGE, SeekPrefixGE, SeekLT, First, or Last.
	//
	// The bounds provided must remain valid until a subsequent call to
	// SetBounds has returned. This requirement exists so that iterator
	// implementations may compare old and new bounds to apply low-level
	// optimizations.
	SetBounds(lower, upper []byte)

	// SetContext replaces the context provided at iterator creation, or the
	// last one provided by SetContext.
	SetContext(ctx context.Context)

	fmt.Stringer

	IteratorDebug
}

InternalIterator iterates over a DB's key/value pairs in key order. Unlike the Iterator interface, the returned keys are InternalKeys composed of the user-key, a sequence number and a key kind. In forward iteration, key/value pairs for identical user-keys are returned in descending sequence order. In reverse iteration, key/value pairs for identical user-keys are returned in ascending sequence order.

InternalIterators provide 5 absolute positioning methods and 2 relative positioning methods. The absolute positioning methods are:

- SeekGE - SeekPrefixGE - SeekLT - First - Last

The relative positioning methods are:

- Next - Prev

The relative positioning methods can be used in conjunction with any of the absolute positioning methods with one exception: SeekPrefixGE does not support reverse iteration via Prev. It is undefined to call relative positioning methods without ever calling an absolute positioning method.

InternalIterators can optionally implement a prefix iteration mode. This mode is entered by calling SeekPrefixGE and exited by any other absolute positioning method (SeekGE, SeekLT, First, Last). When in prefix iteration mode, a call to Next will advance to the next key which has the same "prefix" as the one supplied to SeekPrefixGE. Note that "prefix" in this context is not a strict byte prefix, but defined by byte equality for the result of the Comparer.Split method. An InternalIterator is not required to support prefix iteration mode, and can implement SeekPrefixGE by forwarding to SeekGE. When the iteration prefix is exhausted, it is not valid to call Next on an internal iterator that's already returned (nil,nilv) or a key beyond the prefix.

Bounds, [lower, upper), can be set on iterators, either using the SetBounds() function in the interface, or in implementation specific ways during iterator creation. The forward positioning routines (SeekGE, First, and Next) only check the upper bound. The reverse positioning routines (SeekLT, Last, and Prev) only check the lower bound. It is up to the caller to ensure that the forward positioning routines respect the lower bound and the reverse positioning routines respect the upper bound (i.e. calling SeekGE instead of First if there is a lower bound, and SeekLT instead of Last if there is an upper bound). This imposition is done in order to elevate that enforcement to the caller (generally pebble.Iterator or pebble.mergingIter) rather than having it duplicated in every InternalIterator implementation.

Additionally, the caller needs to ensure that SeekGE/SeekPrefixGE are not called with a key > the upper bound, and SeekLT is not called with a key < the lower bound. InternalIterator implementations are required to respect the iterator bounds, never returning records outside of the bounds with one exception: an iterator may generate synthetic RANGEDEL marker records. See levelIter.syntheticBoundary for the sole existing example of this behavior. Specifically, levelIter can return synthetic keys whose user key is equal to the lower/upper bound.

The bounds provided to an internal iterator must remain valid until a subsequent call to SetBounds has returned. This requirement exists so that iterator implementations may compare old and new bounds to apply low-level optimizations. The pebble.Iterator satisfies this requirement by maintaining two bound buffers and switching between them.

An iterator must be closed after use, but it is not necessary to read an iterator until exhaustion.

An iterator is not goroutine-safe, but it is safe to use multiple iterators concurrently, either in separate goroutines or switching between the iterators in a single goroutine.

It is also safe to use an iterator concurrently with modifying its underlying DB, if that DB permits modification. However, the resultant key/value pairs are not guaranteed to be a consistent snapshot of that DB at a particular point in time.

InternalIterators accumulate errors encountered during operation, exposing them through the Error method. All of the absolute positioning methods reset any accumulated error before positioning. Relative positioning methods return without advancing if the iterator has accumulated an error.

nilv == shorthand for LazyValue{}, which represents a nil value.

type InternalIteratorStats

type InternalIteratorStats struct {
	// Bytes in the loaded blocks. If the block was compressed, this is the
	// compressed bytes. Currently, only the index blocks, data blocks
	// containing points, and filter blocks are included.
	BlockBytes uint64
	// Subset of BlockBytes that were in the block cache.
	BlockBytesInCache uint64
	// BlockReadDuration accumulates the duration spent fetching blocks
	// due to block cache misses.
	// TODO(sumeer): this currently excludes the time spent in Reader creation,
	// and in reading the rangedel and rangekey blocks. Fix that.
	BlockReadDuration time.Duration

	// Bytes in keys that were iterated over. Currently, only point keys are
	// included.
	KeyBytes uint64
	// Bytes in values that were iterated over. Currently, only point values are
	// included. For separated values, this is the size of the handle.
	ValueBytes uint64
	// The count of points iterated over.
	PointCount uint64
	// Points that were iterated over that were covered by range tombstones. It
	// can be useful for discovering instances of
	// https://github.com/cockroachdb/pebble/v2/issues/1070.
	PointsCoveredByRangeTombstones uint64

	// Stats related to points in value blocks encountered during iteration.
	// These are useful to understand outliers, since typical user facing
	// iteration should tend to only look at the latest point, and hence have
	// the following stats close to 0.
	SeparatedPointValue struct {
		// Count is a count of points that were in value blocks. This is not a
		// subset of PointCount: PointCount is produced by mergingIter and if
		// positioned once, and successful in returning a point, will have a
		// PointCount of 1, regardless of how many sstables (and memtables etc.)
		// in the heap got positioned. The count here includes every sstable
		// iterator that got positioned in the heap.
		Count uint64
		// ValueBytes represent the total byte length of the values (in value
		// blocks) of the points corresponding to Count.
		ValueBytes uint64
		// ValueBytesFetched is the total byte length of the values (in value
		// blocks) that were retrieved.
		ValueBytesFetched uint64
	}
}

InternalIteratorStats contains miscellaneous stats produced by InternalIterators that are part of the InternalIterator tree. Not every field is relevant for an InternalIterator implementation. The field values are aggregated as one goes up the InternalIterator tree.

func (*InternalIteratorStats) Merge

Merge merges the stats in from into the given stats.

func (*InternalIteratorStats) SafeFormat

func (s *InternalIteratorStats) SafeFormat(p redact.SafePrinter, verb rune)

SafeFormat implements the redact.SafeFormatter interface.

func (*InternalIteratorStats) String

func (s *InternalIteratorStats) String() string

type InternalKV

type InternalKV struct {
	K InternalKey
	V LazyValue
}

InternalKV represents a single internal key-value pair.

func FakeKVs

func FakeKVs(keys ...string) []InternalKV

FakeKVs constructs InternalKVs from the given key strings, in the format "key:seq-num". The values are empty.

func MakeInternalKV

func MakeInternalKV(k InternalKey, v []byte) InternalKV

MakeInternalKV constructs an InternalKV with the provided internal key and value. The value is encoded in-place.

func (*InternalKV) InPlaceValue

func (kv *InternalKV) InPlaceValue() []byte

InPlaceValue returns the KV's in-place value.

func (*InternalKV) IsExclusiveSentinel

func (kv *InternalKV) IsExclusiveSentinel() bool

IsExclusiveSentinel returns whether this key excludes point keys with the same user key if used as an end boundary. See the comment on InternalKeyRangeDeletionSentinel.

func (*InternalKV) Kind

func (kv *InternalKV) Kind() InternalKeyKind

Kind returns the KV's internal key kind.

func (*InternalKV) SeqNum

func (kv *InternalKV) SeqNum() SeqNum

SeqNum returns the KV's internal key sequence number.

func (*InternalKV) Value

func (kv *InternalKV) Value(buf []byte) (val []byte, callerOwned bool, err error)

Value return's the KV's underlying value.

func (*InternalKV) Visible

func (kv *InternalKV) Visible(snapshot, batchSnapshot SeqNum) bool

Visible returns true if the key is visible at the specified snapshot sequence number.

type InternalKey

type InternalKey struct {
	UserKey []byte
	Trailer InternalKeyTrailer
}

InternalKey is a key used for the in-memory and on-disk partial DBs that make up a pebble DB.

It consists of the user key (as given by the code that uses package pebble) followed by 8-bytes of metadata:

  • 1 byte for the type of internal key: delete or set,
  • 7 bytes for a uint56 sequence number, in little-endian format.

func DecodeInternalKey

func DecodeInternalKey(encodedKey []byte) InternalKey

DecodeInternalKey decodes an encoded internal key. See InternalKey.Encode().

func MakeExclusiveSentinelKey

func MakeExclusiveSentinelKey(kind InternalKeyKind, userKey []byte) InternalKey

MakeExclusiveSentinelKey constructs an internal key that is an exclusive sentinel key, used as the upper boundary for an sstable when a ranged key is the largest key in an sstable.

func MakeInternalKey

func MakeInternalKey(userKey []byte, seqNum SeqNum, kind InternalKeyKind) InternalKey

MakeInternalKey constructs an internal key from a specified user key, sequence number and kind.

func MakeRangeDeleteSentinelKey

func MakeRangeDeleteSentinelKey(userKey []byte) InternalKey

MakeRangeDeleteSentinelKey constructs an internal key that is a range deletion sentinel key, used as the upper boundary for an sstable when a range deletion is the largest key in an sstable.

func MakeSearchKey

func MakeSearchKey(userKey []byte) InternalKey

MakeSearchKey constructs an internal key that is appropriate for searching for a the specified user key. The search key contain the maximal sequence number and kind ensuring that it sorts before any other internal keys for the same user key.

func ParseInternalKey

func ParseInternalKey(s string) InternalKey

ParseInternalKey parses the string representation of an internal key. The format is <user-key>#<seq-num>,<kind>. The older format <user-key>.<kind>.<seq-num> is also supported (for now).

If the seq-num starts with a "b" it is marked as a batch-seq-num (i.e. the SeqNumBatchBit bit is set).

func ParseInternalKeyRange

func ParseInternalKeyRange(s string) (start, end InternalKey)

ParseInternalKeyRange parses a string of the form:

[<user-key>#<seq-num>,<kind>-<user-key>#<seq-num>,<kind>]

func (InternalKey) Clone

func (k InternalKey) Clone() InternalKey

Clone clones the storage for the UserKey component of the key.

func (*InternalKey) CopyFrom

func (k *InternalKey) CopyFrom(k2 InternalKey)

CopyFrom converts this InternalKey into a clone of the passed-in InternalKey, reusing any space already used for the current UserKey.

func (InternalKey) Encode

func (k InternalKey) Encode(buf []byte)

Encode encodes the receiver into the buffer. The buffer must be large enough to hold the encoded data. See InternalKey.Size().

func (InternalKey) EncodeTrailer

func (k InternalKey) EncodeTrailer() [8]byte

EncodeTrailer returns the trailer encoded to an 8-byte array.

func (InternalKey) IsExclusiveSentinel

func (k InternalKey) IsExclusiveSentinel() bool

IsExclusiveSentinel returns whether this internal key excludes point keys with the same user key if used as an end boundary. See the comment on InternalKeyRangeDeletionSentinel.

func (InternalKey) IsUpperBoundFor

func (k InternalKey) IsUpperBoundFor(cmp Compare, userKey []byte) bool

IsUpperBoundFor returns true if a range ending in k contains the userKey: either userKey < k.UserKey or they are equal and k is not an exclusive sentinel.

func (InternalKey) Kind

func (k InternalKey) Kind() InternalKeyKind

Kind returns the kind component of the key.

func (InternalKey) Pretty

func (k InternalKey) Pretty(f FormatKey) fmt.Formatter

Pretty returns a formatter for the key.

func (InternalKey) Separator

func (k InternalKey) Separator(
	cmp Compare, sep Separator, buf []byte, other InternalKey,
) InternalKey

Separator returns a separator key such that k <= x && x < other, where less than is consistent with the Compare function. The buf parameter may be used to store the returned InternalKey.UserKey, though it is valid to pass a nil. See the Separator type for details on separator keys.

func (InternalKey) SeqNum

func (k InternalKey) SeqNum() SeqNum

SeqNum returns the sequence number component of the key.

func (*InternalKey) SetKind

func (k *InternalKey) SetKind(kind InternalKeyKind)

SetKind sets the kind component of the key.

func (*InternalKey) SetSeqNum

func (k *InternalKey) SetSeqNum(seqNum SeqNum)

SetSeqNum sets the sequence number component of the key.

func (InternalKey) Size

func (k InternalKey) Size() int

Size returns the encoded size of the key.

func (InternalKey) String

func (k InternalKey) String() string

String returns a string representation of the key.

func (InternalKey) Successor

func (k InternalKey) Successor(cmp Compare, succ Successor, buf []byte) InternalKey

Successor returns a successor key such that k <= x. A simple implementation may return k unchanged. The buf parameter may be used to store the returned InternalKey.UserKey, though it is valid to pass a nil.

func (InternalKey) Valid

func (k InternalKey) Valid() bool

Valid returns true if the key has a valid kind.

func (InternalKey) Visible

func (k InternalKey) Visible(snapshot, batchSnapshot SeqNum) bool

Visible returns true if the key is visible at the specified snapshot sequence number.

type InternalKeyKind

type InternalKeyKind uint8

InternalKeyKind enumerates the kind of key: a deletion tombstone, a set value, a merged value, etc.

func ParseKind

func ParseKind(s string) InternalKeyKind

ParseKind parses the string representation of an internal key kind.

func (InternalKeyKind) SafeFormat

func (k InternalKeyKind) SafeFormat(w redact.SafePrinter, _ rune)

SafeFormat implements redact.SafeFormatter.

func (InternalKeyKind) String

func (k InternalKeyKind) String() string

type InternalKeyTrailer

type InternalKeyTrailer uint64

InternalKeyTrailer encodes a SeqNum and an InternalKeyKind.

func MakeTrailer

func MakeTrailer(seqNum SeqNum, kind InternalKeyKind) InternalKeyTrailer

MakeTrailer constructs an internal key trailer from the specified sequence number and kind.

func (InternalKeyTrailer) Kind

Kind returns the key kind component of the trailer.

func (InternalKeyTrailer) SeqNum

func (t InternalKeyTrailer) SeqNum() SeqNum

SeqNum returns the sequence number component of the trailer.

func (InternalKeyTrailer) String

func (t InternalKeyTrailer) String() string

String imlements the fmt.Stringer interface.

type IteratorDebug

type IteratorDebug interface {
	// DebugTree prints the entire iterator stack, used for debugging.
	//
	// Each implementation should perform a single Child/Childf call on tp.
	DebugTree(tp treeprinter.Node)
}

IteratorDebug is an interface implemented by all internal iterators and fragment iterators.

type LazyFetcher

type LazyFetcher struct {
	// Fetcher, given a handle, returns the value.
	Fetcher ValueFetcher

	// Attribute includes the short attribute and value length.
	Attribute AttributeAndLen
	// contains filtered or unexported fields
}

LazyFetcher supports fetching a lazy value.

Fetcher and Attribute are to be initialized at creation time. The fields are arranged to reduce the sizeof this struct.

type LazyValue

type LazyValue struct {
	// ValueOrHandle represents a value, or a handle to be passed to ValueFetcher.
	// - Fetcher == nil: ValueOrHandle is a value.
	// - Fetcher != nil: ValueOrHandle is a handle and Fetcher.Attribute is
	//   initialized.
	// The ValueOrHandle exposed by InternalIterator or Iterator may not be stable
	// if the iterator is stepped. To make it stable, make a copy using Clone.
	ValueOrHandle []byte
	// Fetcher provides support for fetching an actually lazy value.
	Fetcher *LazyFetcher
}

LazyValue represents a value that may not already have been extracted. Currently, it can represent either an in-place value (stored with the key) or a value stored in the value section. However, the interface is general enough to support values that are stored in separate files.

LazyValue is used in the InternalIterator interface, such that all positioning calls return (*InternalKey, LazyValue). It is also exposed via the public Iterator for callers that need to remember a recent but not necessarily latest LazyValue, in case they need the actual value in the future. An example is a caller that is iterating in reverse and looking for the latest MVCC version for a key -- it cannot identify the latest MVCC version without stepping to the previous key-value pair e.g. storage.pebbleMVCCScanner in CockroachDB.

Performance note: It is important for this struct to not exceed a sizeof 32 bytes, for optimizing the common case of the in-place value. Prior to introducing LazyValue, we were passing around a []byte which is 24 bytes. Passing a 40 byte or larger struct causes performance to drop by 75% on some benchmarks that do tight iteration loops.

Memory management: This is subtle, but important for performance.

A LazyValue returned by an InternalIterator or Iterator is unstable in that repositioning the iterator will invalidate the memory inside it. A caller wishing to maintain that LazyValue needs to call LazyValue.Clone(). Note that this does not fetch the value if it is not in-place. Clone() should ideally not be called if LazyValue.Value() has been called, since the cloned LazyValue will forget the extracted/fetched value, and calling Value() on this clone will cause the value to be extracted again. That is, Clone() does not make any promise about the memory stability of the underlying value.

A user of an iterator that calls LazyValue.Value() wants as much as possible for the returned value []byte to point to iterator owned memory.

  1. [P1] The underlying iterator that owns that memory also needs a promise from that user that at any time there is at most one value []byte slice that the caller is expecting it to maintain. Otherwise, the underlying iterator has to maintain multiple such []byte slices which results in more complicated and inefficient code.

  2. [P2] The underlying iterator, in order to make the promise that it is maintaining the one value []byte slice, also needs a way to know when it is relieved of that promise. One way it is relieved of that promise is by being told that it is being repositioned. Typically, the owner of the value []byte slice is a sstable iterator, and it will know that it is relieved of the promise when it is repositioned. However, consider the case where the caller has used LazyValue.Clone() and repositioned the iterator (which is actually a tree of iterators). In this case the underlying sstable iterator may not even be open. LazyValue.Value() will still work (at a higher cost), but since the sstable iterator is not open, it does not have a mechanism to know when the retrieved value is no longer in use. We refer to this situation as "not satisfying P2". To handle this situation, the LazyValue.Value() method accepts a caller owned buffer, that the callee will use if needed. The callee explicitly tells the caller whether the []byte slice for the value is now owned by the caller. This will be true if the callee attempted to use buf and either successfully used it or allocated a new []byte slice.

To ground the above in reality, we consider three examples of callers of LazyValue.Value():

  • Iterator: it calls LazyValue.Value for its own use when merging values. When merging during reverse iteration, it may have cloned the LazyValue. In this case it calls LazyValue.Value() on the cloned value, merges it, and then calls LazyValue.Value() on the current iterator position and merges it. So it is honoring P1.

  • Iterator on behalf of Iterator clients: The Iterator.Value() method needs to call LazyValue.Value(). The client of Iterator is satisfying P1 because of the inherent Iterator interface constraint, i.e., it is calling Iterator.Value() on the current Iterator position. It is possible that the Iterator has cloned this LazyValue (for the reverse iteration case), which the client is unaware of, so the underlying sstable iterator may not be able to satisfy P2. This is ok because Iterator will call LazyValue.Value with its (reusable) owned buffer.

  • CockroachDB's pebbleMVCCScanner: This will use LazyValues from Iterator since during reverse iteration in order to find the highest version that satisfies a read it needs to clone the LazyValue, step back the iterator and then decide whether it needs the value from the previously cloned LazyValue. The pebbleMVCCScanner will satisfy P1. The P2 story is similar to the previous case in that it will call LazyValue.Value with its (reusable) owned buffer.

Corollary: callers that directly use InternalIterator can know that they have done nothing to interfere with promise P2 can pass in a nil buf and be sure that it will not trigger an allocation.

Repeated calling of LazyValue.Value: This is ok as long as the caller continues to satisfy P1. The previously fetched value will be remembered inside LazyValue to avoid fetching again. So if the caller's buffer is used the first time the value was fetched, it is still in use.

LazyValue fields are visible outside the package for use in InternalIterator implementations and in Iterator, but not meant for direct use by users of Pebble.

func MakeInPlaceValue

func MakeInPlaceValue(val []byte) LazyValue

MakeInPlaceValue constructs an in-place value.

func (*LazyValue) Clone

func (lv *LazyValue) Clone(buf []byte, fetcher *LazyFetcher) (LazyValue, []byte)

Clone creates a stable copy of the LazyValue, by appending bytes to buf. The fetcher parameter must be non-nil and may be over-written and used inside the returned LazyValue -- this is needed to avoid an allocation. Most callers have at most K cloned LazyValues, where K is hard-coded, so they can have a pool of exactly K LazyFetcher structs they can reuse in these calls. The alternative of allocating LazyFetchers from a sync.Pool is not viable since we have no code trigger for returning to the pool (LazyValues are simply GC'd).

NB: It is highly preferable that LazyValue.Value() has not been called, since the Clone will forget any previously extracted value, and a future call to Value will cause it to be fetched again. We do this since we don't want to reason about whether or not to clone an already extracted value inside the Fetcher (we don't). Property P1 applies here too: if lv1.Value() has been called, and then lv2 is created as a clone of lv1, then calling lv2.Value() can invalidate any backing memory maintained inside the fetcher for lv1 (even though these are the same values). We initially prohibited calling LazyValue.Clone() if LazyValue.Value() has been called, but there is at least one complex caller (pebbleMVCCScanner inside CockroachDB) where it is not easy to prove this invariant.

func (*LazyValue) InPlaceValue

func (lv *LazyValue) InPlaceValue() []byte

InPlaceValue returns the value under the assumption that it is in-place. This is for Pebble-internal code.

func (*LazyValue) Len

func (lv *LazyValue) Len() int

Len returns the length of the value.

func (*LazyValue) TryGetShortAttribute

func (lv *LazyValue) TryGetShortAttribute() (ShortAttribute, bool)

TryGetShortAttribute returns the ShortAttribute and a bool indicating whether the ShortAttribute was populated.

func (*LazyValue) Value

func (lv *LazyValue) Value(buf []byte) (val []byte, callerOwned bool, err error)

Value returns the underlying value.

type Logger

type Logger interface {
	Infof(format string, args ...interface{})
	Errorf(format string, args ...interface{})
	Fatalf(format string, args ...interface{})
}

Logger defines an interface for writing log messages.

type LoggerAndTracer

type LoggerAndTracer interface {
	Logger
	// Eventf formats and emits a tracing log, if tracing is enabled in the
	// current context. It can also emit to a regular log, if expensive
	// logging is enabled.
	Eventf(ctx context.Context, format string, args ...interface{})
	// IsTracingEnabled returns true if tracing is enabled for this context,
	// or expensive logging is enabled. It can be used as an optimization to
	// avoid calling Eventf (which will be a noop when tracing or expensive
	// logging is not enabled) to avoid the overhead of boxing the args.
	IsTracingEnabled(ctx context.Context) bool
}

LoggerAndTracer defines an interface for logging and tracing.

type LoggerWithNoopTracer

type LoggerWithNoopTracer struct {
	Logger
}

LoggerWithNoopTracer wraps a logger and does no tracing.

func (*LoggerWithNoopTracer) Eventf

func (*LoggerWithNoopTracer) Eventf(ctx context.Context, format string, args ...interface{})

Eventf implements LoggerAndTracer.

func (*LoggerWithNoopTracer) IsTracingEnabled

func (*LoggerWithNoopTracer) IsTracingEnabled(ctx context.Context) bool

IsTracingEnabled implements LoggerAndTracer.

type Merge

type Merge func(key, value []byte) (ValueMerger, error)

Merge creates a ValueMerger for the specified key initialized with the value of one merge operand.

type Merger

type Merger struct {
	Merge Merge

	// Name is the name of the merger.
	//
	// Pebble stores the merger name on disk, and opening a database with a
	// different merger from the one it was created with will result in an error.
	Name string
}

Merger defines an associative merge operation. The merge operation merges two or more values for a single key. A merge operation is requested by writing a value using {Batch,DB}.Merge(). The value at that key is merged with any existing value. It is valid to Set a value at a key and then Merge a new value. Similar to non-merged values, a merged value can be deleted by either Delete or DeleteRange.

The merge operation is invoked when a merge value is encountered during a read, either during a compaction or during iteration.

type NeedsFileContents

type NeedsFileContents interface {
	// contains filtered or unexported methods
}

NeedsFileContents is implemented by a cleaner that needs the contents of the files that it is being asked to clean.

type NoopLoggerAndTracer

type NoopLoggerAndTracer struct{}

NoopLoggerAndTracer does no logging and tracing. Remember that struct{} is special cased in Go and does not incur an allocation when it backs the interface LoggerAndTracer.

func (NoopLoggerAndTracer) Errorf

func (l NoopLoggerAndTracer) Errorf(format string, args ...interface{})

Errorf implements LoggerAndTracer.

func (NoopLoggerAndTracer) Eventf

func (l NoopLoggerAndTracer) Eventf(ctx context.Context, format string, args ...interface{})

Eventf implements LoggerAndTracer.

func (NoopLoggerAndTracer) Fatalf

func (l NoopLoggerAndTracer) Fatalf(format string, args ...interface{})

Fatalf implements LoggerAndTracer.

func (NoopLoggerAndTracer) Infof

func (l NoopLoggerAndTracer) Infof(format string, args ...interface{})

Infof implements LoggerAndTracer.

func (NoopLoggerAndTracer) IsTracingEnabled

func (l NoopLoggerAndTracer) IsTracingEnabled(ctx context.Context) bool

IsTracingEnabled implements LoggerAndTracer.

type SeekGEFlags

type SeekGEFlags uint8

SeekGEFlags holds flags that may configure the behavior of a forward seek. Not all flags are relevant to all iterators.

func (SeekGEFlags) BatchJustRefreshed

func (s SeekGEFlags) BatchJustRefreshed() bool

BatchJustRefreshed is set by Seek[Prefix]GE when an iterator's view of an indexed batch was just refreshed. It serves as a signal to the batch iterator to ignore the TrySeekUsingNext optimization, because the external knowledge imparted by the TrySeekUsingNext flag does not apply to the batch iterator's position. See (pebble.Iterator).batchJustRefreshed.

func (SeekGEFlags) DisableBatchJustRefreshed

func (s SeekGEFlags) DisableBatchJustRefreshed() SeekGEFlags

DisableBatchJustRefreshed returns the provided flags with the batch-just-refreshed bit unset.

func (SeekGEFlags) DisableRelativeSeek

func (s SeekGEFlags) DisableRelativeSeek() SeekGEFlags

DisableRelativeSeek returns the provided flags with the relative-seek flag disabled.

func (SeekGEFlags) DisableTrySeekUsingNext

func (s SeekGEFlags) DisableTrySeekUsingNext() SeekGEFlags

DisableTrySeekUsingNext returns the provided flags with the try-seek-using-next optimization disabled.

func (SeekGEFlags) EnableBatchJustRefreshed

func (s SeekGEFlags) EnableBatchJustRefreshed() SeekGEFlags

EnableBatchJustRefreshed returns the provided flags with the batch-just-refreshed bit set. See BatchJustRefreshed for an explanation of this flag.

func (SeekGEFlags) EnableRelativeSeek

func (s SeekGEFlags) EnableRelativeSeek() SeekGEFlags

EnableRelativeSeek returns the provided flags with the relative-seek flag enabled. See RelativeSeek for an explanation of this flag's use.

func (SeekGEFlags) EnableTrySeekUsingNext

func (s SeekGEFlags) EnableTrySeekUsingNext() SeekGEFlags

EnableTrySeekUsingNext returns the provided flags with the try-seek-using-next optimization enabled. See TrySeekUsingNext for an explanation of this optimization.

func (SeekGEFlags) RelativeSeek

func (s SeekGEFlags) RelativeSeek() bool

RelativeSeek is set when in the course of a forward positioning operation, a higher-level iterator seeks a lower-level iterator to a larger key than the one at the current iterator position.

Concretely, this occurs when the merging iterator observes a range deletion covering the key at a level's current position, and the merging iterator seeks the level to the range deletion's end key. During lazy-combined iteration, this flag signals to the level iterator that the seek is NOT an absolute-positioning operation from the perspective of the pebble.Iterator, and the level iterator must look for range keys in tables between the current iterator position and the new seeked position.

func (SeekGEFlags) TrySeekUsingNext

func (s SeekGEFlags) TrySeekUsingNext() bool

TrySeekUsingNext indicates whether a performance optimization was enabled by a caller, indicating the caller has not done any action to move this iterator beyond the first key that would be found if this iterator were to honestly do the intended seek. For example, say the caller did a SeekGE(k1...), followed by SeekGE(k2...) where k1 <= k2, without any intermediate positioning calls. The caller can safely specify true for this parameter in the second call. As another example, say the caller did do one call to Next between the two Seek calls, and k1 < k2. Again, the caller can safely specify a true value for this parameter. Note that a false value is always safe. The callee is free to ignore the true value if its implementation does not permit this optimization.

We make the caller do this determination since a string comparison of k1, k2 is not necessarily cheap, and there may be many iterators in the iterator stack. Doing it once at the root of the iterator stack is cheaper.

This optimization could also be applied to SeekLT (where it would be trySeekUsingPrev). We currently only do it for SeekPrefixGE and SeekGE because this is where this optimization helps the performance of CockroachDB. The SeekLT cases in CockroachDB are typically accompanied with bounds that change between seek calls, and is optimized inside certain iterator implementations, like singleLevelIterator, without any extra parameter passing (though the same amortization of string comparisons could be done to improve that optimization, by making the root of the iterator stack do it).

type SeekLTFlags

type SeekLTFlags uint8

SeekLTFlags holds flags that may configure the behavior of a reverse seek. Not all flags are relevant to all iterators.

func (SeekLTFlags) DisableRelativeSeek

func (s SeekLTFlags) DisableRelativeSeek() SeekLTFlags

DisableRelativeSeek returns the provided flags with the relative-seek flag disabled.

func (SeekLTFlags) EnableRelativeSeek

func (s SeekLTFlags) EnableRelativeSeek() SeekLTFlags

EnableRelativeSeek returns the provided flags with the relative-seek flag enabled. See RelativeSeek for an explanation of this flag's use.

func (SeekLTFlags) RelativeSeek

func (s SeekLTFlags) RelativeSeek() bool

RelativeSeek is set when in the course of a reverse positioning operation, a higher-level iterator seeks a lower-level iterator to a smaller key than the one at the current iterator position.

Concretely, this occurs when the merging iterator observes a range deletion covering the key at a level's current position, and the merging iterator seeks the level to the range deletion's start key. During lazy-combined iteration, this flag signals to the level iterator that the seek is NOT an absolute-positioning operation from the perspective of the pebble.Iterator, and the level iterator must look for range keys in tables between the current iterator position and the new seeked position.

type Separator

type Separator func(dst, a, b []byte) []byte

Separator is used to construct SSTable index blocks. A trivial implementation is `return append(dst, a...)`, but appending fewer bytes leads to smaller SSTables.

Given keys a, b for which Compare(a, b) < 0, Separator produces a key k such that:

1. Compare(a, k) <= 0, and 2. Compare(k, b) < 0.

For example, if a and b are the []byte equivalents of the strings "black" and "blue", then the function may append "blb" to dst.

type SeqNum

type SeqNum uint64

SeqNum is a sequence number defining precedence among identical keys. A key with a higher sequence number takes precedence over a key with an equal user key of a lower sequence number. Sequence numbers are stored durably within the internal key "trailer" as a 7-byte (uint56) uint, and the maximum sequence number is 2^56-1. As keys are committed to the database, they're assigned increasing sequence numbers. Readers use sequence numbers to read a consistent database state, ignoring keys with sequence numbers larger than the readers' "visible sequence number."

The database maintains an invariant that no two point keys with equal user keys may have equal sequence numbers. Keys with differing user keys may have equal sequence numbers. A point key and a range deletion or range key that include that point key can have equal sequence numbers - in that case, the range key does not apply to the point key. A key's sequence number may be changed to zero during compactions when it can be proven that no identical keys with lower sequence numbers exist.

const (
	// SeqNumZero is the zero sequence number, set by compactions if they can
	// guarantee there are no keys underneath an internal key.
	SeqNumZero SeqNum = 0
	// SeqNumStart is the first sequence number assigned to a key. Sequence
	// numbers 1-9 are reserved for potential future use.
	SeqNumStart SeqNum = 10
	// SeqNumMax is the largest valid sequence number.
	SeqNumMax SeqNum = 1<<56 - 1
	// SeqNumBatchBit is set on batch sequence numbers which prevents those
	// entries from being excluded from iteration.
	SeqNumBatchBit SeqNum = 1 << 55
)

func ParseSeqNum

func ParseSeqNum(s string) SeqNum

ParseSeqNum parses the string representation of a sequence number. "inf" is supported as the maximum sequence number (mainly used for exclusive end keys).

func (SeqNum) SafeFormat

func (s SeqNum) SafeFormat(w redact.SafePrinter, _ rune)

SafeFormat implements redact.SafeFormatter.

func (SeqNum) String

func (s SeqNum) String() string

type ShortAttribute

type ShortAttribute uint8

ShortAttribute encodes a user-specified attribute of the value.

type ShortAttributeExtractor

type ShortAttributeExtractor func(
	key []byte, keyPrefixLen int, value []byte) (ShortAttribute, error)

ShortAttributeExtractor is an extractor that given the value, will return the ShortAttribute.

type Split

type Split func(a []byte) int

Split returns the length of the prefix of the user key that corresponds to the key portion of an MVCC encoding scheme to enable the use of prefix bloom filters.

The method will only ever be called with valid MVCC keys, that is, keys that the user could potentially store in the database. Pebble does not know which keys are MVCC keys and which are not, and may call Split on both MVCC keys and non-MVCC keys.

A trivial MVCC scheme is one in which Split() returns len(a). This corresponds to assigning a constant version to each key in the database. For performance reasons, it is preferable to use a `nil` split in this case.

Let prefix(a) = a[:Split(a)] and suffix(a) = a[Split(a):]. The following properties must hold:

  1. A key consisting of just a prefix must sort before all other keys with that prefix:

    If len(suffix(a)) > 0, then Compare(prefix(a), a) < 0.

  2. Prefixes must be used to order keys before suffixes:

    If Compare(a, b) <= 0, then Compare(prefix(a), prefix(b)) <= 0. If Compare(prefix(a), prefix(b)) < 0, then Compare(a, b) < 0

  3. Suffixes themselves must be valid keys and comparable, respecting the same ordering as within a key:

    If Compare(prefix(a), prefix(b)) = 0, then Compare(a, b) = ComparePointSuffixes(suffix(a), suffix(b))

var DefaultSplit Split = func(key []byte) int { return len(key) }

DefaultSplit is a trivial implementation of Split which always returns the full key.

func (Split) Prefix

func (s Split) Prefix(k []byte) []byte

Prefix returns the prefix of the key k, using s to split the key.

type Successor

type Successor func(dst, a []byte) []byte

Successor appends to dst a shortened key k given a key a such that Compare(a, k) <= 0. A simple implementation may return a unchanged. The appended key k must be valid to pass to Compare.

type ThroughputMetric

type ThroughputMetric struct {
	// Bytes is the processes bytes by the component.
	Bytes int64
	// WorkDuration is the duration that the component spent doing work.
	WorkDuration time.Duration
	// IdleDuration is the duration that the component was idling, waiting for
	// work.
	IdleDuration time.Duration
}

ThroughputMetric is used to measure the byte throughput of some component that performs work in a single-threaded manner. The throughput can be approximated by Bytes/(WorkDuration+IdleTime). The idle time is represented separately, so that the user of this metric could approximate the peak throughput as Bytes/WorkTime. The metric is designed to be cumulative (see Merge).

func (*ThroughputMetric) Merge

func (tm *ThroughputMetric) Merge(x ThroughputMetric)

Merge accumulates the information from another throughput metric.

func (*ThroughputMetric) PeakRate

func (tm *ThroughputMetric) PeakRate() int64

PeakRate returns the approximate peak rate if there was no idling.

func (*ThroughputMetric) Rate

func (tm *ThroughputMetric) Rate() int64

Rate returns the observed rate.

func (*ThroughputMetric) Subtract

func (tm *ThroughputMetric) Subtract(x ThroughputMetric)

Subtract subtracts the information from another ThroughputMetric

func (*ThroughputMetric) Utilization

func (tm *ThroughputMetric) Utilization() float64

Utilization returns a percent [0, 1.0] indicating the percent of time work was performed.

type TopLevelIterator

type TopLevelIterator interface {
	InternalIterator

	// SeekPrefixGEStrict extends InternalIterator.SeekPrefixGE with a guarantee
	// that the iterator only returns keys matching the prefix.
	SeekPrefixGEStrict(prefix, key []byte, flags SeekGEFlags) *InternalKV
}

TopLevelIterator extends InternalIterator to include an additional absolute positioning method, SeekPrefixGEStrict.

type UserKeyBoundary

type UserKeyBoundary struct {
	Key  []byte
	Kind BoundaryKind
}

UserKeyBoundary represents the endpoint of a bound which can be exclusive or inclusive.

func UserKeyExclusive

func UserKeyExclusive(userKey []byte) UserKeyBoundary

UserKeyExclusive creates an exclusive user key boundary.

func UserKeyExclusiveIf

func UserKeyExclusiveIf(userKey []byte, exclusive bool) UserKeyBoundary

UserKeyExclusiveIf creates a user key boundary which can be either inclusive or exclusive.

func UserKeyInclusive

func UserKeyInclusive(userKey []byte) UserKeyBoundary

UserKeyInclusive creates an inclusive user key boundary.

func (UserKeyBoundary) CompareUpperBounds

func (eb UserKeyBoundary) CompareUpperBounds(cmp Compare, other UserKeyBoundary) int

CompareUpperBounds compares two UserKeyBoundaries as upper bounds (e.g. when they are used for UserKeyBounds.End).

func (UserKeyBoundary) IsUpperBoundFor

func (eb UserKeyBoundary) IsUpperBoundFor(cmp Compare, userKey []byte) bool

IsUpperBoundFor returns true if the boundary is an upper bound for the key; i.e. the key is less than the boundary key OR they are equal and the boundary is inclusive.

func (UserKeyBoundary) IsUpperBoundForInternalKey

func (eb UserKeyBoundary) IsUpperBoundForInternalKey(cmp Compare, key InternalKey) bool

IsUpperBoundForInternalKey returns true if boundary is an upper bound for the given internal key.

type UserKeyBounds

type UserKeyBounds struct {
	Start []byte
	End   UserKeyBoundary
}

UserKeyBounds is a user key interval with an inclusive start boundary and with an end boundary that can be either inclusive or exclusive.

func ParseUserKeyBounds

func ParseUserKeyBounds(s string) UserKeyBounds

ParseUserKeyBounds parses UserKeyBounds from a string representation of the form "[foo, bar]" or "[foo, bar)".

func UserKeyBoundsEndExclusive

func UserKeyBoundsEndExclusive(start []byte, end []byte) UserKeyBounds

UserKeyBoundsEndExclusive creates the bounds [start, end).

func UserKeyBoundsEndExclusiveIf

func UserKeyBoundsEndExclusiveIf(start []byte, end []byte, exclusive bool) UserKeyBounds

UserKeyBoundsEndExclusiveIf creates either [start, end] or [start, end) bounds.

func UserKeyBoundsFromInternal

func UserKeyBoundsFromInternal(smallest, largest InternalKey) UserKeyBounds

UserKeyBoundsFromInternal creates the bounds [smallest.UserKey, largest.UserKey] or [smallest.UserKey, largest.UserKey) if largest is an exclusive sentinel.

smallest must not be an exclusive sentinel.

func UserKeyBoundsInclusive

func UserKeyBoundsInclusive(start []byte, end []byte) UserKeyBounds

UserKeyBoundsInclusive creates the bounds [start, end].

func (*UserKeyBounds) ContainsBounds

func (b *UserKeyBounds) ContainsBounds(cmp Compare, other *UserKeyBounds) bool

ContainsBounds returns true if b completely overlaps other.

func (*UserKeyBounds) ContainsInternalKey

func (b *UserKeyBounds) ContainsInternalKey(cmp Compare, key InternalKey) bool

ContainsInternalKey returns true if the internal key is within the bounds.

func (*UserKeyBounds) ContainsUserKey

func (b *UserKeyBounds) ContainsUserKey(cmp Compare, userKey []byte) bool

ContainsUserKey returns true if the user key is within the bounds.

func (UserKeyBounds) Format

func (b UserKeyBounds) Format(fmtKey FormatKey) string

Format converts the bounds to a string of the form "[foo, bar]" or "[foo, bar)", using the given key formatter.

func (*UserKeyBounds) Overlaps

func (b *UserKeyBounds) Overlaps(cmp Compare, other *UserKeyBounds) bool

Overlaps returns true if the bounds overlap.

func (UserKeyBounds) String

func (b UserKeyBounds) String() string

func (*UserKeyBounds) Valid

func (b *UserKeyBounds) Valid(cmp Compare) bool

Valid returns true if the bounds contain at least a user key.

type ValueFetcher

type ValueFetcher interface {
	// Fetch returns the value, given the handle. It is acceptable to call the
	// ValueFetcher.Fetch as long as the DB is open. However, one should assume
	// there is a fast-path when the iterator tree has not moved off the sstable
	// iterator that initially provided this LazyValue. Hence, to utilize this
	// fast-path the caller should try to decide whether it needs the value or
	// not as soon as possible, with minimal possible stepping of the iterator.
	//
	// buf will be used if the fetcher cannot satisfy P2 (see earlier comment).
	// If the fetcher attempted to use buf *and* len(buf) was insufficient, it
	// will allocate a new slice for the value. In either case it will set
	// callerOwned to true.
	Fetch(
		ctx context.Context, handle []byte, valLen int32, buf []byte,
	) (val []byte, callerOwned bool, err error)
}

ValueFetcher is an interface for fetching a value.

type ValueMerger

type ValueMerger interface {
	// MergeNewer adds an operand that is newer than all existing operands.
	// The caller retains ownership of value.
	//
	// If an error is returned the merge is aborted and no other methods must
	// be called.
	MergeNewer(value []byte) error

	// MergeOlder adds an operand that is older than all existing operands.
	// The caller retains ownership of value.
	//
	// If an error is returned the merge is aborted and no other methods must
	// be called.
	MergeOlder(value []byte) error

	// Finish does any final processing of the added operands and returns a
	// result. The caller can assume the returned byte slice will not be mutated.
	//
	// Finish must be the last function called on the ValueMerger. The caller
	// must not call any other ValueMerger functions after calling Finish.
	//
	// If `includesBase` is true, the oldest merge operand was part of the
	// merge. This will always be the true during normal iteration, but may be
	// false during compaction when only a subset of operands may be
	// available. Note that `includesBase` is set to true conservatively: a false
	// value means that we could not definitely determine that the base merge
	// operand was included.
	//
	// If a Closer is returned, the returned slice will remain valid until it is
	// closed. The caller must arrange for the closer to be eventually closed.
	Finish(includesBase bool) ([]byte, io.Closer, error)
}

ValueMerger receives merge operands one by one. The operand received is either newer or older than all operands received so far as indicated by the function names, `MergeNewer()` and `MergeOlder()`. Once all operands have been received, the client will invoke `Finish()` to obtain the final result. The order of a merge is not changed after the first call to `MergeNewer()` or `MergeOlder()`, i.e. the same method is used to submit all operands.

The implementation may choose to merge values into the result immediately upon receiving each operand, or buffer operands until Finish() is called. For example, buffering may be useful to avoid (de)serializing partial merge results.

The merge operation must be associative. That is, for the values A, B, C:

Merge(A).MergeOlder(B).MergeOlder(C) == Merge(C).MergeNewer(B).MergeNewer(A)

Examples of merge operators are integer addition, list append, and string concatenation.

func NewDeletableSumValueMerger

func NewDeletableSumValueMerger(key, value []byte) (ValueMerger, error)

NewDeletableSumValueMerger return a ValueMerger which computes the sum of its arguments, but transforms a zero sum into a non-existent entry.

Jump to

Keyboard shortcuts

? : This menu
/ : Search site
f or F : Jump to
y or Y : Canonical URL
JackTT - Gopher 🇻🇳