rangekey

package
v1.1.0 Latest Latest
Warning

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

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

Documentation

Overview

Package rangekey provides facilities for encoding, decoding and merging range keys.

Range keys map a span of keyspan `[start, end)`, at an optional suffix, to a value.

Encoding

Unlike other Pebble keys, range keys encode several fields of information: start key, end key, suffix and value. Internally within Pebble and its sstables, all keys including range keys are represented as a key-value tuple. Range keys map to internal key-value tuples by mapping the start key to the key and encoding the remainder of the fields in the value.

## `RANGEKEYSET`

A `RANGEKEYSET` represents one more range keys set over a single region of user key space. Each represented range key must have a unique suffix. A `RANGEKEYSET` encapsulates a start key, an end key and a set of SuffixValue pairs.

A `RANGEKEYSET` key's user key holds the start key. Its value is a varstring end key, followed by a set of SuffixValue pairs. A `RANGEKEYSET` may have multiple SuffixValue pairs if the keyspan was set at multiple unique suffix values.

## `RANGEKEYUNSET`

A `RANGEKEYUNSET` represents the removal of range keys at specific suffixes over a single region of user key space. A `RANGEKEYUNSET` encapsulates a start key, an end key and a set of suffixes.

A `RANGEKEYUNSET` key's user key holds the start key. Its value is a varstring end key, followed by a set of suffixes. A `RANGEKEYUNSET` may have multiple suffixes if the keyspan was unset at multiple unique suffixes.

## `RANGEKEYDEL`

A `RANGEKEYDEL` represents the removal of all range keys over a single region of user key space, regardless of suffix. A `RANGEKEYDEL` encapsulates a start key and an end key. The end key is stored in the value, without any varstring length prefixing.

Index

Constants

This section is empty.

Variables

This section is empty.

Functions

func Coalesce

func Coalesce(cmp base.Compare, eq base.Equal, keys []keyspan.Key, dst *[]keyspan.Key) error

Coalesce imposes range key semantics and coalesces range keys with the same bounds. Coalesce drops any keys shadowed by more recent sets, unsets or deletes. Coalesce modifies the provided span's Keys slice, reslicing the slice to remove dropped keys.

Coalescence has subtle behavior with respect to sequence numbers. Coalesce depends on a keyspan.Span's Keys being sorted in sequence number descending order. The first key has the largest sequence number. The returned coalesced span includes only the largest sequence number. All other sequence numbers are forgotten. When a compaction constructs output range keys from a coalesced span, it produces at most one RANGEKEYSET, one RANGEKEYUNSET and one RANGEKEYDEL. Each one of these keys adopt the largest sequence number.

This has the potentially surprising effect of 'promoting' a key to a higher sequence number. This is okay, because:

  • There are no other overlapping keys within the coalesced span of sequence numbers (otherwise they would be in the compaction, due to the LSM invariant).
  • Range key sequence numbers are never compared to point key sequence numbers. Range keys and point keys have parallel existences.
  • Compactions only coalesce within snapshot stripes.

Additionally, internal range keys at the same sequence number have subtle mechanics:

  • RANGEKEYSETs shadow RANGEKEYUNSETs of the same suffix.
  • RANGEKEYDELs only apply to keys at lower sequence numbers.

This is required for ingestion. Ingested sstables are assigned a single sequence number for the file, at which all of the file's keys are visible. The RANGEKEYSET, RANGEKEYUNSET and RANGEKEYDEL key kinds are ordered such that among keys with equal sequence numbers (thus ordered by their kinds) the keys do not affect one another. Ingested sstables are expected to be consistent with respect to the set/unset suffixes: A given suffix should be set or unset but not both.

The resulting dst Keys slice is sorted by Trailer.

func Decode

func Decode(ik base.InternalKey, v []byte, keysDst []keyspan.Key) (keyspan.Span, error)

Decode takes an internal key pair encoding range key(s) and returns a decoded keyspan containing the keys. If keysDst is provided, keys will be appended to keysDst.

func DecodeEndKey

func DecodeEndKey(kind base.InternalKeyKind, data []byte) (endKey, value []byte, ok bool)

DecodeEndKey reads the end key from the beginning of a range key (RANGEKEYSET, RANGEKEYUNSET or RANGEKEYDEL)'s physical encoded value. Both sets and unsets encode the range key, plus additional data in the value.

func Encode

func Encode(s *keyspan.Span, emit func(k base.InternalKey, v []byte) error) error

Encode takes a Span containing only range keys. It invokes the provided closure with the encoded internal keys that represent the Span's state. The keys and values passed to emit are only valid until the closure returns. If emit returns an error, Encode stops and returns the error.

func EncodeSetValue

func EncodeSetValue(dst []byte, endKey []byte, suffixValues []SuffixValue) int

EncodeSetValue encodes a RangeKeySet's value into dst. The length of dst must be greater than or equal to EncodedSetValueLen. EncodeSetValue returns the number of bytes written, which should always equal the EncodedSetValueLen with the same arguments.

func EncodeUnsetValue

func EncodeUnsetValue(dst []byte, endKey []byte, suffixes [][]byte) int

EncodeUnsetValue encodes a RangeKeyUnset's value into dst. The length of dst must be greater than or equal to EncodedUnsetValueLen. EncodeUnsetValue returns the number of bytes written, which should always equal the EncodedUnsetValueLen with the same arguments.

func EncodedSetValueLen

func EncodedSetValueLen(endKey []byte, suffixValues []SuffixValue) int

EncodedSetValueLen precomputes the length of a RangeKeySet's value when encoded. It may be used to construct a buffer of the appropriate size before encoding.

func EncodedUnsetValueLen

func EncodedUnsetValueLen(endKey []byte, suffixes [][]byte) int

EncodedUnsetValueLen precomputes the length of a RangeKeyUnset's value when encoded. It may be used to construct a buffer of the appropriate size before encoding.

func IsRangeKey

func IsRangeKey(kind base.InternalKeyKind) bool

IsRangeKey returns true if the given key kind is one of the range key kinds.

Types

type Buffers

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

Buffers holds various buffers used for range key iteration. They're exposed so that they may be pooled and reused between iterators.

func (*Buffers) PrepareForReuse

func (bufs *Buffers) PrepareForReuse()

PrepareForReuse discards any excessively large buffers.

type Encoder

type Encoder struct {
	Emit func(base.InternalKey, []byte) error
	// contains filtered or unexported fields
}

An Encoder encodes range keys into their on-disk InternalKey format. An Encoder holds internal buffers, reused between Emit calls.

func (*Encoder) Encode

func (e *Encoder) Encode(s *keyspan.Span) error

Encode takes a Span containing only range keys. It invokes the Encoder's Emit closure with the encoded internal keys that represent the Span's state. The keys and values passed to emit are only valid until the closure returns. If Emit returns an error, Encode stops and returns the error.

The encoded key-value pair passed to Emit is only valid until the closure completes.

type SuffixValue

type SuffixValue struct {
	Suffix []byte
	Value  []byte
}

SuffixValue represents a tuple of a suffix and a corresponding value. A physical RANGEKEYSET key may contain many logical RangeKeySets, each represented with a separate SuffixValue tuple.

type UserIteratorConfig

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

UserIteratorConfig holds state for constructing the range key iterator stack for user iteration. The range key iterator must merge range key spans across the levels of the LSM. This merging is performed by a keyspan.MergingIter on-the-fly. The UserIteratorConfig implements keyspan.Transformer, evaluating range-key semantics and shadowing, so the spans returned by a MergingIter are fully resolved.

The MergingIter is wrapped by a BoundedIter, which elides spans that are outside the iterator bounds (or the current prefix's bounds, during prefix iteration mode).

To provide determinisim during iteration, the BoundedIter is wrapped by a DefragmentingIter that defragments abutting spans with identical user-observable state.

At the top-level an InterleavingIter interleaves range keys with point keys and performs truncation to iterator bounds.

Below is an abbreviated diagram illustrating the mechanics of a SeekGE.

               InterleavingIter.SeekGE
                       │
            DefragmentingIter.SeekGE
                       │
               BoundedIter.SeekGE
                       │
      ╭────────────────┴───────────────╮
      │                                ├── defragmentBwd*
MergingIter.SeekGE                     │
      │                                ╰── defragmentFwd
      ╰─╶╶ per level╶╶ ─╮
                        │
                        │
                        ├── <?>.SeekLT
                        │
                        ╰── <?>.Next

func (*UserIteratorConfig) AddLevel

func (ui *UserIteratorConfig) AddLevel(iter keyspan.FragmentIterator)

AddLevel adds a new level to the bottom of the iterator stack. AddLevel must be called after Init and before any other method on the iterator.

func (*UserIteratorConfig) Init

func (ui *UserIteratorConfig) Init(
	comparer *base.Comparer,
	snapshot uint64,
	lower, upper []byte,
	hasPrefix *bool,
	prefix *[]byte,
	internalKeys bool,
	bufs *Buffers,
	iters ...keyspan.FragmentIterator,
) keyspan.FragmentIterator

Init initializes the range key iterator stack for user iteration. The resulting fragment iterator applies range key semantics, defragments spans according to their user-observable state and, if !internalKeys, removes all Keys other than RangeKeySets describing the current state of range keys. The resulting spans contain Keys sorted by suffix (unless internalKeys is true, in which case they remain sorted by trailer descending).

The snapshot sequence number parameter determines which keys are visible. Any keys not visible at the provided snapshot are ignored.

func (*UserIteratorConfig) NewLevelIter

func (ui *UserIteratorConfig) NewLevelIter() *keyspan.LevelIter

NewLevelIter returns a pointer to a newly allocated or reused keyspan.LevelIter. The caller is responsible for calling Init() on this instance.

func (*UserIteratorConfig) SetBounds

func (ui *UserIteratorConfig) SetBounds(lower, upper []byte)

SetBounds propagates bounds to the iterator stack. The fragment iterator interface ordinarily doesn't enforce bounds, so this is exposed as an explicit method on the user iterator config.

func (*UserIteratorConfig) ShouldDefragment

func (ui *UserIteratorConfig) ShouldDefragment(equal base.Equal, a, b *keyspan.Span) bool

ShouldDefragment implements the DefragmentMethod interface and configures a DefragmentingIter to defragment spans of range keys if their user-visible state is identical. This defragmenting method assumes the provided spans have already been transformed through (UserIterationConfig).Transform, so all RangeKeySets are user-visible sets and are already in Suffix order. This defragmenter checks for equality between set suffixes and values (ignoring sequence numbers). It's intended for use during user iteration, when the wrapped keyspan iterator is merging spans across all levels of the LSM.

func (*UserIteratorConfig) Transform

func (ui *UserIteratorConfig) Transform(cmp base.Compare, s keyspan.Span, dst *keyspan.Span) error

Transform implements the keyspan.Transformer interface for use with a keyspan.MergingIter. It transforms spans by resolving range keys at the provided snapshot sequence number. Shadowing of keys is resolved (eg, removal of unset keys, removal of keys overwritten by a set at the same suffix, etc) and then non-RangeKeySet keys are removed. The resulting transformed spans only contain RangeKeySets describing the state visible at the provided sequence number, and hold their Keys sorted by Suffix (except if internalKeys is true, then keys remain sorted by trailer.

Jump to

Keyboard shortcuts

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