Documentation
¶
Index ¶
- Constants
- Variables
- func AssertionFailedf(format string, args ...interface{}) error
- func CatchErrorPanic(f func() error) (err error)
- func CheckComparer(c *Comparer, prefixes [][]byte, suffixes [][]byte) error
- func CloseHelper(closer io.Closer) io.Closer
- func CorruptionErrorf(format string, args ...interface{}) error
- func DebugTree(iter IteratorDebug) string
- func InternalCompare(userCmp Compare, a, b InternalKey) int
- func MakeFilename(fileType FileType, dfn DiskFileNum) string
- func MakeFilepath(fs vfs.FS, dirname string, fileType FileType, dfn DiskFileNum) string
- func MarkCorruptionError(err error) error
- func MinUserKey(cmp Compare, a, b []byte) []byte
- func MustExist(fs vfs.FS, filename string, fataler Fataler, err error)
- func ParseFilename(fs vfs.FS, filename string) (fileType FileType, dfn DiskFileNum, ok bool)
- func SharedPrefixLen(a, b []byte) int
- func Visible(seqNum SeqNum, snapshot, batchSnapshot SeqNum) bool
- type AbbreviatedKey
- type AppendValueMerger
- type ArchiveCleaner
- type AtomicSeqNum
- type AttributeAndLen
- type BlockPropertyFilter
- type BoundaryKind
- type Cleaner
- type Compare
- type ComparePointSuffixes
- type CompareRangeSuffixes
- type Comparer
- type DeletableValueMerger
- type DeleteCleaner
- type DiskFileNum
- type Equal
- type FakeIter
- func (f *FakeIter) Close() error
- func (f *FakeIter) DebugTree(tp treeprinter.Node)
- func (f *FakeIter) Error() error
- func (f *FakeIter) First() *InternalKV
- func (f *FakeIter) KV() *InternalKV
- func (f *FakeIter) Last() *InternalKV
- func (f *FakeIter) Next() *InternalKV
- func (f *FakeIter) NextPrefix(succKey []byte) *InternalKV
- func (f *FakeIter) Prev() *InternalKV
- func (f *FakeIter) SeekGE(key []byte, flags SeekGEFlags) *InternalKV
- func (f *FakeIter) SeekLT(key []byte, flags SeekLTFlags) *InternalKV
- func (f *FakeIter) SeekPrefixGE(prefix, key []byte, flags SeekGEFlags) *InternalKV
- func (f *FakeIter) SetBounds(lower, upper []byte)
- func (f *FakeIter) SetCloseErr(closeErr error)
- func (f *FakeIter) SetContext(_ context.Context)
- func (f *FakeIter) String() string
- func (f *FakeIter) Valid() bool
- type Fataler
- type FileInfo
- type FileNum
- type FileType
- type FilterPolicy
- type FilterType
- type FilterWriter
- type FormatBytes
- type FormatKey
- type FormatValue
- type GaugeSampleMetric
- type ImmediateSuccessor
- type InMemLogger
- type InternalIterator
- type InternalIteratorStats
- type InternalKV
- func (kv *InternalKV) InPlaceValue() []byte
- func (kv *InternalKV) IsExclusiveSentinel() bool
- func (kv *InternalKV) Kind() InternalKeyKind
- func (kv *InternalKV) SeqNum() SeqNum
- func (kv *InternalKV) Value(buf []byte) (val []byte, callerOwned bool, err error)
- func (kv *InternalKV) Visible(snapshot, batchSnapshot SeqNum) bool
- type InternalKey
- func DecodeInternalKey(encodedKey []byte) InternalKey
- func MakeExclusiveSentinelKey(kind InternalKeyKind, userKey []byte) InternalKey
- func MakeInternalKey(userKey []byte, seqNum SeqNum, kind InternalKeyKind) InternalKey
- func MakeRangeDeleteSentinelKey(userKey []byte) InternalKey
- func MakeSearchKey(userKey []byte) InternalKey
- func ParseInternalKey(s string) InternalKey
- func ParseInternalKeyRange(s string) (start, end InternalKey)
- func (k InternalKey) Clone() InternalKey
- func (k *InternalKey) CopyFrom(k2 InternalKey)
- func (k InternalKey) Encode(buf []byte)
- func (k InternalKey) EncodeTrailer() [8]byte
- func (k InternalKey) IsExclusiveSentinel() bool
- func (k InternalKey) IsUpperBoundFor(cmp Compare, userKey []byte) bool
- func (k InternalKey) Kind() InternalKeyKind
- func (k InternalKey) Pretty(f FormatKey) fmt.Formatter
- func (k InternalKey) Separator(cmp Compare, sep Separator, buf []byte, other InternalKey) InternalKey
- func (k InternalKey) SeqNum() SeqNum
- func (k *InternalKey) SetKind(kind InternalKeyKind)
- func (k *InternalKey) SetSeqNum(seqNum SeqNum)
- func (k InternalKey) Size() int
- func (k InternalKey) String() string
- func (k InternalKey) Successor(cmp Compare, succ Successor, buf []byte) InternalKey
- func (k InternalKey) Valid() bool
- func (k InternalKey) Visible(snapshot, batchSnapshot SeqNum) bool
- type InternalKeyKind
- type InternalKeyTrailer
- type IteratorDebug
- type LazyFetcher
- type LazyValue
- func (lv *LazyValue) Clone(buf []byte, fetcher *LazyFetcher) (LazyValue, []byte)
- func (lv *LazyValue) InPlaceValue() []byte
- func (lv *LazyValue) Len() int
- func (lv *LazyValue) TryGetShortAttribute() (ShortAttribute, bool)
- func (lv *LazyValue) Value(buf []byte) (val []byte, callerOwned bool, err error)
- type Logger
- type LoggerAndTracer
- type LoggerWithNoopTracer
- type Merge
- type Merger
- type NeedsFileContents
- type NoopLoggerAndTracer
- func (l NoopLoggerAndTracer) Errorf(format string, args ...interface{})
- func (l NoopLoggerAndTracer) Eventf(ctx context.Context, format string, args ...interface{})
- func (l NoopLoggerAndTracer) Fatalf(format string, args ...interface{})
- func (l NoopLoggerAndTracer) Infof(format string, args ...interface{})
- func (l NoopLoggerAndTracer) IsTracingEnabled(ctx context.Context) bool
- type SeekGEFlags
- func (s SeekGEFlags) BatchJustRefreshed() bool
- func (s SeekGEFlags) DisableBatchJustRefreshed() SeekGEFlags
- func (s SeekGEFlags) DisableRelativeSeek() SeekGEFlags
- func (s SeekGEFlags) DisableTrySeekUsingNext() SeekGEFlags
- func (s SeekGEFlags) EnableBatchJustRefreshed() SeekGEFlags
- func (s SeekGEFlags) EnableRelativeSeek() SeekGEFlags
- func (s SeekGEFlags) EnableTrySeekUsingNext() SeekGEFlags
- func (s SeekGEFlags) RelativeSeek() bool
- func (s SeekGEFlags) TrySeekUsingNext() bool
- type SeekLTFlags
- type Separator
- type SeqNum
- type ShortAttribute
- type ShortAttributeExtractor
- type Split
- type Successor
- type ThroughputMetric
- type TopLevelIterator
- type UserKeyBoundary
- type UserKeyBounds
- func ParseUserKeyBounds(s string) UserKeyBounds
- func UserKeyBoundsEndExclusive(start []byte, end []byte) UserKeyBounds
- func UserKeyBoundsEndExclusiveIf(start []byte, end []byte, exclusive bool) UserKeyBounds
- func UserKeyBoundsFromInternal(smallest, largest InternalKey) UserKeyBounds
- func UserKeyBoundsInclusive(start []byte, end []byte) UserKeyBounds
- func (b *UserKeyBounds) ContainsBounds(cmp Compare, other *UserKeyBounds) bool
- func (b *UserKeyBounds) ContainsInternalKey(cmp Compare, key InternalKey) bool
- func (b *UserKeyBounds) ContainsUserKey(cmp Compare, userKey []byte) bool
- func (b UserKeyBounds) Format(fmtKey FormatKey) string
- func (b *UserKeyBounds) Overlaps(cmp Compare, other *UserKeyBounds) bool
- func (b UserKeyBounds) String() string
- func (b *UserKeyBounds) Valid(cmp Compare) bool
- type ValueFetcher
- type ValueMerger
Constants ¶
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.
const ( DefaultBlockRestartInterval = 16 DefaultBlockSize = 4096 DefaultBlockSizeThreshold = 90 SizeClassAwareBlockSizeThreshold = 60 )
SSTable block defaults.
const InternalTrailerLen = 8
InternalTrailerLen is the number of bytes used to encode InternalKey.Trailer.
const MaxShortAttribute = 7
MaxShortAttribute is the maximum value of the short attribute (3 bits).
const SeekGEFlagsNone = SeekGEFlags(0)
SeekGEFlagsNone is the default value of SeekGEFlags, with all flags disabled.
const SeekLTFlagsNone = SeekLTFlags(0)
SeekLTFlagsNone is the default value of SeekLTFlags, with all flags disabled.
Variables ¶
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.
var DefaultLogger defaultLogger
DefaultLogger logs to the Go stdlib logs.
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.
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.
var ErrNotFound = errors.New("pebble: not found")
ErrNotFound means that a get or delete call did not find the requested key.
var InvalidInternalKey = MakeInternalKey(nil, SeqNumZero, InternalKeyKindInvalid)
InvalidInternalKey is an invalid internal key for which Valid() will return false.
Functions ¶
func AssertionFailedf ¶
AssertionFailedf creates an assertion error and panics in invariants.Enabled builds. It should only be used when it indicates a bug.
func CatchErrorPanic ¶
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 ¶
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 ¶
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 ¶
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 ¶
MakeFilepath builds a filepath from components.
func MarkCorruptionError ¶
MarkCorruptionError marks given error as a corruption error.
func MinUserKey ¶
MinUserKey returns the smaller of two user keys. If one of the keys is nil, the other one is returned.
func MustExist ¶
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 ¶
ParseFilename parses the components from a filename.
func SharedPrefixLen ¶
SharedPrefixLen returns the largest i such that a[:i] equals b[:i]. This function can be useful in implementing the Comparer interface.
Types ¶
type AbbreviatedKey ¶
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 ¶
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 ¶
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.
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 Compare ¶
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 ¶
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 ¶
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 ¶
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 ¶
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) 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 ¶
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) DebugTree ¶
func (f *FakeIter) DebugTree(tp treeprinter.Node)
DebugTree 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) SetCloseErr ¶
SetCloseErr causes future calls to Error() and Close() to return this error.
func (*FakeIter) SetContext ¶
SetContext 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.
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.
type FormatKey ¶
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 ¶
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 ¶
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) 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 ¶
func (s *InternalIteratorStats) Merge(from InternalIteratorStats)
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) 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 ¶
func (t InternalKeyTrailer) Kind() InternalKeyKind
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.
[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.
[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 ¶
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 ¶
InPlaceValue returns the value under the assumption that it is in-place. This is for Pebble-internal code.
func (*LazyValue) TryGetShortAttribute ¶
func (lv *LazyValue) TryGetShortAttribute() (ShortAttribute, bool)
TryGetShortAttribute returns the ShortAttribute and a bool indicating whether the ShortAttribute was populated.
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 ¶
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 ¶
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.
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 ¶
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:
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.
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
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))
DefaultSplit is a trivial implementation of Split which always returns the full key.
type Successor ¶
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.