Documentation
¶
Overview ¶
Package keyspanimpl contains Pebble-specific implementations of keyspan fragment iterators.
Index ¶
- type LevelIter
- func (l *LevelIter) Close()
- func (l *LevelIter) DebugTree(tp treeprinter.Node)
- func (l *LevelIter) First() (*keyspan.Span, error)
- func (l *LevelIter) Init(ctx context.Context, opts keyspan.SpanIterOptions, cmp base.Compare, ...)
- func (l *LevelIter) Last() (*keyspan.Span, error)
- func (l *LevelIter) Next() (*keyspan.Span, error)
- func (l *LevelIter) Prev() (*keyspan.Span, error)
- func (l *LevelIter) SeekGE(key []byte) (*keyspan.Span, error)
- func (l *LevelIter) SeekLT(key []byte) (*keyspan.Span, error)
- func (l *LevelIter) SetContext(ctx context.Context)
- func (l *LevelIter) String() string
- func (l *LevelIter) WrapChildren(wrap keyspan.WrapFn)
- type MergingBuffers
- type MergingIter
- func (m *MergingIter) AddLevel(iter keyspan.FragmentIterator)
- func (m *MergingIter) Close()
- func (m *MergingIter) DebugString() string
- func (m *MergingIter) DebugTree(tp treeprinter.Node)
- func (m *MergingIter) First() (*keyspan.Span, error)
- func (m *MergingIter) Init(comparer *base.Comparer, transformer keyspan.Transformer, bufs *MergingBuffers, ...)
- func (m *MergingIter) Last() (*keyspan.Span, error)
- func (m *MergingIter) Next() (*keyspan.Span, error)
- func (m *MergingIter) Prev() (*keyspan.Span, error)
- func (m *MergingIter) SeekGE(key []byte) (*keyspan.Span, error)
- func (m *MergingIter) SeekLT(key []byte) (*keyspan.Span, error)
- func (m *MergingIter) SetContext(ctx context.Context)
- func (m *MergingIter) String() string
- func (m *MergingIter) WrapChildren(wrap keyspan.WrapFn)
- type TableNewSpanIter
Constants ¶
This section is empty.
Variables ¶
This section is empty.
Functions ¶
This section is empty.
Types ¶
type LevelIter ¶
type LevelIter struct {
// contains filtered or unexported fields
}
LevelIter provides a merged view of spans from sstables in an L1+ level or an L0 sublevel.
LevelIter takes advantage of level invariants to only have one sstable span block open at one time, opened using the newIter function passed in.
A LevelIter is configured with a key type that is either KeyTypePoint (corresponding to range dels) or KeyTypeRange (corresponding to range keys). The key type decides which bounds we use for the files (and which files we filter out).
LevelIter supports emitting "straddling spans": these are empty spans that cover the gaps between the keyspaces of adjacent files. This is an optimization to avoid unnecessarily loading files in cases where spans are very sparse (in the context of merging spans from multiple levels). We currently produce straddling spans only in range key mode.
TODO(radu): investigate enabling straddling spans for rangedel mode.
func NewLevelIter ¶
func NewLevelIter( ctx context.Context, opts keyspan.SpanIterOptions, cmp base.Compare, newIter TableNewSpanIter, files manifest.LevelIterator, level manifest.Layer, keyType manifest.KeyType, ) *LevelIter
NewLevelIter returns a LevelIter.
newIter must create a range del iterator for the given file if keyType is KeyTypePoint or a range key iterator if keyType is KeyTypeRange.
func (*LevelIter) DebugTree ¶
func (l *LevelIter) DebugTree(tp treeprinter.Node)
DebugTree is part of the FragmentIterator interface.
func (*LevelIter) Init ¶
func (l *LevelIter) Init( ctx context.Context, opts keyspan.SpanIterOptions, cmp base.Compare, newIter TableNewSpanIter, files manifest.LevelIterator, level manifest.Layer, keyType manifest.KeyType, )
Init initializes a LevelIter.
newIter must create a range del iterator for the given file if keyType is KeyTypePoint or a range key iterator if keyType is KeyTypeRange.
func (*LevelIter) SetContext ¶
SetContext is part of the FragmentIterator interface.
func (*LevelIter) WrapChildren ¶
WrapChildren implements FragmentIterator.
type MergingBuffers ¶
type MergingBuffers struct {
// contains filtered or unexported fields
}
MergingBuffers holds buffers used while merging keyspans.
func (*MergingBuffers) PrepareForReuse ¶
func (bufs *MergingBuffers) PrepareForReuse()
PrepareForReuse discards any excessively large buffers.
type MergingIter ¶
type MergingIter struct { *MergingBuffers // contains filtered or unexported fields }
MergingIter merges spans across levels of the LSM, exposing an iterator over spans that yields sets of spans fragmented at unique user key boundaries.
A MergingIter is initialized with an arbitrary number of child iterators over fragmented spans. Each child iterator exposes fragmented key spans, such that overlapping keys are surfaced in a single Span. Key spans from one child iterator may overlap key spans from another child iterator arbitrarily.
The spans combined by MergingIter will return spans with keys sorted by trailer descending. If the MergingIter is configured with a Transformer, it's permitted to modify the ordering of the spans' keys returned by MergingIter.
Algorithm ¶
The merging iterator wraps child iterators, merging and fragmenting spans across levels. The high-level algorithm is:
- Initialize the heap with bound keys from child iterators' spans.
- Find the next [or previous] two unique user keys' from bounds.
- Consider the span formed between the two unique user keys a candidate span.
- Determine if any of the child iterators' spans overlap the candidate span. 4a. If any of the child iterator's current bounds are end keys (during forward iteration) or start keys (during reverse iteration), then all the spans with that bound overlap the candidate span. 4b. Apply the configured transform, which may remove keys. 4c. If no spans overlap, forget the smallest (forward iteration) or largest (reverse iteration) unique user key and advance the iterators to the next unique user key. Start again from 3.
Detailed algorithm ¶
Each level (i0, i1, ...) has a user-provided input FragmentIterator. The merging iterator steps through individual boundaries of the underlying spans separately. If the underlying FragmentIterator has fragments [a,b){#2,#1} [b,c){#1} the mergingIterLevel.{next,prev} step through:
(a, start), (b, end), (b, start), (c, end)
Note that (a, start) and (b, end) are observed ONCE each, despite two keys sharing those bounds. Also note that (b, end) and (b, start) are two distinct iterator positions of a mergingIterLevel.
The merging iterator maintains a heap (min during forward iteration, max during reverse iteration) containing the boundKeys. Each boundKey is a 3-tuple holding the bound user key, whether the bound is a start or end key and the set of keys from that level that have that bound. The heap orders based on the boundKey's user key only.
The merging iterator is responsible for merging spans across levels to determine which span is next, but it's also responsible for fragmenting overlapping spans. Consider the example:
i0: b---d e-----h i1: a---c h-----k i2: a------------------------------p fragments: a-b-c-d-e-----h-----k----------p
None of the individual child iterators contain a span with the exact bounds [c,d), but the merging iterator must produce a span [c,d). To accomplish this, the merging iterator visits every span between unique boundary user keys. In the above example, this is:
[a,b), [b,c), [c,d), [d,e), [e, h), [h, k), [k, p)
The merging iterator first initializes the heap to prepare for iteration. The description below discusses the mechanics of forward iteration after a call to First, but the mechanics are similar for reverse iteration and other positioning methods.
During a call to First, the heap is initialized by seeking every mergingIterLevel to the first bound of the first fragment. In the above example, this seeks the child iterators to:
i0: (b, boundKindFragmentStart, [ [b,d) ]) i1: (a, boundKindFragmentStart, [ [a,c) ]) i2: (a, boundKindFragmentStart, [ [a,p) ])
After fixing up the heap, the root of the heap is a boundKey with the smallest user key ('a' in the example). Once the heap is setup for iteration in the appropriate direction and location, the merging iterator uses find{Next,Prev}FragmentSet to find the next/previous span bounds.
During forward iteration, the root of the heap's user key is the start key key of next merged span. findNextFragmentSet sets m.start to this user key. The heap may contain other boundKeys with the same user key if another level has a fragment starting or ending at the same key, so the findNextFragmentSet method pulls from the heap until it finds the first key greater than m.start. This key is used as the end key.
In the above example, this results in m.start = 'a', m.end = 'b' and child iterators in the following positions:
i0: (b, boundKindFragmentStart, [ [b,d) ]) i1: (c, boundKindFragmentEnd, [ [a,c) ]) i2: (p, boundKindFragmentEnd, [ [a,p) ])
With the user key bounds of the next merged span established, findNextFragmentSet must determine which, if any, fragments overlap the span. During forward iteration any child iterator that is now positioned at an end boundary has an overlapping span. (Justification: The child iterator's end boundary is ≥ m.end. The corresponding start boundary must be ≤ m.start since there were no other user keys between m.start and m.end. So the fragments associated with the iterator's current end boundary have start and end bounds such that start ≤ m.start < m.end ≤ end).
findNextFragmentSet iterates over the levels, collecting keys from any child iterators positioned at end boundaries. In the above example, i1 and i2 are positioned at end boundaries, so findNextFragmentSet collects the keys of [a,c) and [a,p). These spans contain the merging iterator's [m.start, m.end) span, but they may also extend beyond the m.start and m.end. The merging iterator returns the keys with the merging iter's m.start and m.end bounds, preserving the underlying keys' sequence numbers, key kinds and values.
A MergingIter is configured with a Transform that's applied to the span before surfacing it to the iterator user. A Transform may remove keys arbitrarily, but it may not modify the values themselves.
It may be the case that findNextFragmentSet finds no levels positioned at end boundaries, or that there are no spans remaining after applying a transform, in which case the span [m.start, m.end) overlaps with nothing. In this case findNextFragmentSet loops, repeating the above process again until it finds a span that does contain keys.
Memory safety ¶
The FragmentIterator interface only guarantees stability of a Span and its associated slices until the next positioning method is called. Adjacent Spans may be contained in different sstables, requring the FragmentIterator implementation to close one sstable, releasing its memory, before opening the next. Most of the state used by the MergingIter is derived from spans at current child iterator positions only, ensuring state is stable. The one exception is the start bound during forward iteration and the end bound during reverse iteration.
If the heap root originates from an end boundary when findNextFragmentSet begins, a Next on the heap root level may invalidate the end boundary. To accommodate this, find{Next,Prev}FragmentSet copy the initial boundary if the subsequent Next/Prev would move to the next span.
func (*MergingIter) AddLevel ¶
func (m *MergingIter) AddLevel(iter keyspan.FragmentIterator)
AddLevel adds a new level to the bottom of the merging iterator. AddLevel must be called after Init and before any other method.
func (*MergingIter) Close ¶
func (m *MergingIter) Close()
Close closes the iterator, releasing all acquired resources.
func (*MergingIter) DebugString ¶
func (m *MergingIter) DebugString() string
DebugString returns a string representing the current internal state of the merging iterator and its heap for debugging purposes.
func (*MergingIter) DebugTree ¶
func (m *MergingIter) DebugTree(tp treeprinter.Node)
DebugTree is part of the FragmentIterator interface.
func (*MergingIter) First ¶
func (m *MergingIter) First() (*keyspan.Span, error)
First seeks the iterator to the first span.
func (*MergingIter) Init ¶
func (m *MergingIter) Init( comparer *base.Comparer, transformer keyspan.Transformer, bufs *MergingBuffers, iters ...keyspan.FragmentIterator, )
Init initializes the merging iterator with the provided fragment iterators.
func (*MergingIter) Last ¶
func (m *MergingIter) Last() (*keyspan.Span, error)
Last seeks the iterator to the last span.
func (*MergingIter) Next ¶
func (m *MergingIter) Next() (*keyspan.Span, error)
Next advances the iterator to the next span.
func (*MergingIter) Prev ¶
func (m *MergingIter) Prev() (*keyspan.Span, error)
Prev advances the iterator to the previous span.
func (*MergingIter) SeekGE ¶
func (m *MergingIter) SeekGE(key []byte) (*keyspan.Span, error)
SeekGE moves the iterator to the first span covering a key greater than or equal to the given key. This is equivalent to seeking to the first span with an end key greater than the given key.
func (*MergingIter) SeekLT ¶
func (m *MergingIter) SeekLT(key []byte) (*keyspan.Span, error)
SeekLT moves the iterator to the last span covering a key less than the given key. This is equivalent to seeking to the last span with a start key less than the given key.
func (*MergingIter) SetContext ¶
func (m *MergingIter) SetContext(ctx context.Context)
SetContext is part of the FragmentIterator interface.
func (*MergingIter) WrapChildren ¶
func (m *MergingIter) WrapChildren(wrap keyspan.WrapFn)
WrapChildren implements FragmentIterator.
type TableNewSpanIter ¶
type TableNewSpanIter func( ctx context.Context, file *manifest.FileMetadata, iterOptions keyspan.SpanIterOptions, ) (keyspan.FragmentIterator, error)
TableNewSpanIter creates a new iterator for range key spans for the given file.