// Copyright 2018 The LevelDB-Go and Pebble Authors. All rights reserved. Use // of this source code is governed by a BSD-style license that can be found in // the LICENSE file. package pebble import ( "bytes" "context" "fmt" "runtime/debug" "unsafe" "github.com/cockroachdb/errors" "github.com/cockroachdb/pebble/internal/base" "github.com/cockroachdb/pebble/internal/invariants" "github.com/cockroachdb/pebble/internal/keyspan" ) type mergingIterLevel struct { index int iter internalIterator // rangeDelIter is set to the range-deletion iterator for the level. When // configured with a levelIter, this pointer changes as sstable boundaries // are crossed. See levelIter.initRangeDel and the Range Deletions comment // below. rangeDelIter keyspan.FragmentIterator // iterKey and iterValue cache the current key and value iter are pointed at. iterKey *InternalKey iterValue base.LazyValue // levelIter is non-nil if this level's iter is ultimately backed by a // *levelIter. The handle in iter may have wrapped the levelIter with // intermediary internalIterator implementations. levelIter *levelIter // levelIterBoundaryContext's fields are set when using levelIter, in order // to surface sstable boundary keys and file-level context. See levelIter // comment and the Range Deletions comment below. levelIterBoundaryContext // tombstone caches the tombstone rangeDelIter is currently pointed at. If // tombstone is nil, there are no further tombstones within the // current sstable in the current iterator direction. The cached tombstone is // only valid for the levels in the range [0,heap[0].index]. This avoids // positioning tombstones at lower levels which cannot possibly shadow the // current key. tombstone *keyspan.Span } type levelIterBoundaryContext struct { // smallestUserKey and largestUserKey are populated with the smallest and // largest boundaries of the current file. smallestUserKey, largestUserKey []byte // isLargestUserKeyExclusive is set to true when a file's largest boundary // is an exclusive key, (eg, a range deletion sentinel). If true, the file // does not contain any keys with the provided user key, and the // largestUserKey bound is exclusive. isLargestUserKeyExclusive bool // isSyntheticIterBoundsKey is set to true iff the key returned by the level // iterator is a synthetic key derived from the iterator bounds. This is used // to prevent the mergingIter from being stuck at such a synthetic key if it // becomes the top element of the heap. When used with a user-facing Iterator, // the only range deletions exposed by this mergingIter should be those with // `isSyntheticIterBoundsKey || isIgnorableBoundaryKey`. isSyntheticIterBoundsKey bool // isIgnorableBoundaryKey is set to true iff the key returned by the level // iterator is a file boundary key that should be ignored when returning to // the parent iterator. File boundary keys are used by the level iter to // keep a levelIter file's range deletion iterator open as long as other // levels within the merging iterator require it. When used with a user-facing // Iterator, the only range deletions exposed by this mergingIter should be // those with `isSyntheticIterBoundsKey || isIgnorableBoundaryKey`. isIgnorableBoundaryKey bool } // mergingIter provides a merged view of multiple iterators from different // levels of the LSM. // // The core of a mergingIter is a heap of internalIterators (see // mergingIterHeap). The heap can operate as either a min-heap, used during // forward iteration (First, SeekGE, Next) or a max-heap, used during reverse // iteration (Last, SeekLT, Prev). The heap is initialized in calls to First, // Last, SeekGE, and SeekLT. A call to Next or Prev takes the current top // element on the heap, advances its iterator, and then "fixes" the heap // property. When one of the child iterators is exhausted during Next/Prev // iteration, it is removed from the heap. // // # Range Deletions // // A mergingIter can optionally be configured with a slice of range deletion // iterators. The range deletion iterator slice must exactly parallel the point // iterators and the range deletion iterator must correspond to the same level // in the LSM as the point iterator. Note that each memtable and each table in // L0 is a different "level" from the mergingIter perspective. So level 0 below // does not correspond to L0 in the LSM. // // A range deletion iterator iterates over fragmented range tombstones. Range // tombstones are fragmented by splitting them at any overlapping points. This // fragmentation guarantees that within an sstable tombstones will either be // distinct or will have identical start and end user keys. While range // tombstones are fragmented within an sstable, the start and end keys are not truncated // to sstable boundaries. This is necessary because the tombstone end key is // exclusive and does not have a sequence number. Consider an sstable // containing the range tombstone [a,c)#9 and the key "b#8". The tombstone must // delete "b#8", yet older versions of "b" might spill over to the next // sstable. So the boundary key for this sstable must be "b#8". Adjusting the // end key of tombstones to be optionally inclusive or contain a sequence // number would be possible solutions (such solutions have potentially serious // issues: tombstones have exclusive end keys since an inclusive deletion end can // be converted to an exclusive one while the reverse transformation is not possible; // the semantics of a sequence number for the end key of a range tombstone are murky). // // The approach taken here performs an // implicit truncation of the tombstone to the sstable boundaries. // // During initialization of a mergingIter, the range deletion iterators for // batches, memtables, and L0 tables are populated up front. Note that Batches // and memtables index unfragmented tombstones. Batch.newRangeDelIter() and // memTable.newRangeDelIter() fragment and cache the tombstones on demand. The // L1-L6 range deletion iterators are populated by levelIter. When configured // to load range deletion iterators, whenever a levelIter loads a table it // loads both the point iterator and the range deletion // iterator. levelIter.rangeDelIter is configured to point to the right entry // in mergingIter.levels. The effect of this setup is that // mergingIter.levels[i].rangeDelIter always contains the fragmented range // tombstone for the current table in level i that the levelIter has open. // // Another crucial mechanism of levelIter is that it materializes fake point // entries for the table boundaries if the boundary is range deletion // key. Consider a table that contains only a range tombstone [a-e)#10. The // sstable boundaries for this table will be a#10,15 and // e#72057594037927935,15. During forward iteration levelIter will return // e#72057594037927935,15 as a key. During reverse iteration levelIter will // return a#10,15 as a key. These sentinel keys act as bookends to point // iteration and allow mergingIter to keep a table and its associated range // tombstones loaded as long as there are keys at lower levels that are within // the bounds of the table. // // The final piece to the range deletion puzzle is the LSM invariant that for a // given key K newer versions of K can only exist earlier in the level, or at // higher levels of the tree. For example, if K#4 exists in L3, k#5 can only // exist earlier in the L3 or in L0, L1, L2 or a memtable. Get very explicitly // uses this invariant to find the value for a key by walking the LSM level by // level. For range deletions, this invariant means that a range deletion at // level N will necessarily shadow any keys within its bounds in level Y where // Y > N. One wrinkle to this statement is that it only applies to keys that // lie within the sstable bounds as well, but we get that guarantee due to the // way the range deletion iterator and point iterator are bound together by a // levelIter. // // Tying the above all together, we get a picture where each level (index in // mergingIter.levels) is composed of both point operations (pX) and range // deletions (rX). The range deletions for level X shadow both the point // operations and range deletions for level Y where Y > X allowing mergingIter // to skip processing entries in that shadow. For example, consider the // scenario: // // r0: a---e // r1: d---h // r2: g---k // r3: j---n // r4: m---q // // This is showing 5 levels of range deletions. Consider what happens upon // SeekGE("b"). We first seek the point iterator for level 0 (the point values // are not shown above) and we then seek the range deletion iterator. That // returns the tombstone [a,e). This tombstone tells us that all keys in the // range [a,e) in lower levels are deleted so we can skip them. So we can // adjust the seek key to "e", the tombstone end key. For level 1 we seek to // "e" and find the range tombstone [d,h) and similar logic holds. By the time // we get to level 4 we're seeking to "n". // // One consequence of not truncating tombstone end keys to sstable boundaries // is the seeking process described above cannot always seek to the tombstone // end key in the older level. For example, imagine in the above example r3 is // a partitioned level (i.e., L1+ in our LSM), and the sstable containing [j, // n) has "k" as its upper boundary. In this situation, compactions involving // keys at or after "k" can output those keys to r4+, even if they're newer // than our tombstone [j, n). So instead of seeking to "n" in r4 we can only // seek to "k". To achieve this, the instance variable `largestUserKey.` // maintains the upper bounds of the current sstables in the partitioned // levels. In this example, `levels[3].largestUserKey` holds "k", telling us to // limit the seek triggered by a tombstone in r3 to "k". // // During actual iteration levels can contain both point operations and range // deletions. Within a level, when a range deletion contains a point operation // the sequence numbers must be checked to determine if the point operation is // newer or older than the range deletion tombstone. The mergingIter maintains // the invariant that the range deletion iterators for all levels newer that // the current iteration key (L < m.heap.items[0].index) are positioned at the // next (or previous during reverse iteration) range deletion tombstone. We // know those levels don't contain a range deletion tombstone that covers the // current key because if they did the current key would be deleted. The range // deletion iterator for the current key's level is positioned at a range // tombstone covering or past the current key. The position of all of other // range deletion iterators is unspecified. Whenever a key from those levels // becomes the current key, their range deletion iterators need to be // positioned. This lazy positioning avoids seeking the range deletion // iterators for keys that are never considered. (A similar bit of lazy // evaluation can be done for the point iterators, but is still TBD). // // For a full example, consider the following setup: // // p0: o // r0: m---q // // p1: n p // r1: g---k // // p2: b d i // r2: a---e q----v // // p3: e // r3: // // If we start iterating from the beginning, the first key we encounter is "b" // in p2. When the mergingIter is pointing at a valid entry, the range deletion // iterators for all of the levels < m.heap.items[0].index are positioned at // the next range tombstone past the current key. So r0 will point at [m,q) and // r1 at [g,k). When the key "b" is encountered, we check to see if the current // tombstone for r0 or r1 contains it, and whether the tombstone for r2, [a,e), // contains and is newer than "b". // // Advancing the iterator finds the next key at "d". This is in the same level // as the previous key "b" so we don't have to reposition any of the range // deletion iterators, but merely check whether "d" is now contained by any of // the range tombstones at higher levels or has stepped past the range // tombstone in its own level or higher levels. In this case, there is nothing to be done. // // Advancing the iterator again finds "e". Since "e" comes from p3, we have to // position the r3 range deletion iterator, which is empty. "e" is past the r2 // tombstone of [a,e) so we need to advance the r2 range deletion iterator to // [q,v). // // The next key is "i". Because this key is in p2, a level above "e", we don't // have to reposition any range deletion iterators and instead see that "i" is // covered by the range tombstone [g,k). The iterator is immediately advanced // to "n" which is covered by the range tombstone [m,q) causing the iterator to // advance to "o" which is visible. // // TODO(peter,rangedel): For testing, advance the iterator through various // scenarios and have each step display the current state (i.e. the current // heap and range-del iterator positioning). type mergingIter struct { logger Logger split Split dir int snapshot uint64 batchSnapshot uint64 levels []mergingIterLevel heap mergingIterHeap err error prefix []byte lower []byte upper []byte stats *InternalIteratorStats // levelsPositioned, if non-nil, is a slice of the same length as levels. // It's used by NextPrefix to record which levels have already been // repositioned. It's created lazily by the first call to NextPrefix. levelsPositioned []bool combinedIterState *combinedIterState // Used in some tests to disable the random disabling of seek optimizations. forceEnableSeekOpt bool } // mergingIter implements the base.InternalIterator interface. var _ base.InternalIterator = (*mergingIter)(nil) // newMergingIter returns an iterator that merges its input. Walking the // resultant iterator will return all key/value pairs of all input iterators // in strictly increasing key order, as defined by cmp. It is permissible to // pass a nil split parameter if the caller is never going to call // SeekPrefixGE. // // The input's key ranges may overlap, but there are assumed to be no duplicate // keys: if iters[i] contains a key k then iters[j] will not contain that key k. // // None of the iters may be nil. func newMergingIter( logger Logger, stats *base.InternalIteratorStats, cmp Compare, split Split, iters ...internalIterator, ) *mergingIter { m := &mergingIter{} levels := make([]mergingIterLevel, len(iters)) for i := range levels { levels[i].iter = iters[i] } m.init(&IterOptions{logger: logger}, stats, cmp, split, levels...) return m } func (m *mergingIter) init( opts *IterOptions, stats *base.InternalIteratorStats, cmp Compare, split Split, levels ...mergingIterLevel, ) { m.err = nil // clear cached iteration error m.logger = opts.getLogger() if opts != nil { m.lower = opts.LowerBound m.upper = opts.UpperBound } m.snapshot = InternalKeySeqNumMax m.batchSnapshot = InternalKeySeqNumMax m.levels = levels m.heap.cmp = cmp m.split = split m.stats = stats if cap(m.heap.items) < len(levels) { m.heap.items = make([]*mergingIterLevel, 0, len(levels)) } else { m.heap.items = m.heap.items[:0] } for l := range m.levels { m.levels[l].index = l } } func (m *mergingIter) initHeap() { m.heap.items = m.heap.items[:0] for i := range m.levels { if l := &m.levels[i]; l.iterKey != nil { m.heap.items = append(m.heap.items, l) } else { m.err = firstError(m.err, l.iter.Error()) if m.err != nil { return } } } m.heap.init() } func (m *mergingIter) initMinHeap() { m.dir = 1 m.heap.reverse = false m.initHeap() m.initMinRangeDelIters(-1) } // The level of the previous top element was oldTopLevel. Note that all range delete // iterators < oldTopLevel are positioned past the key of the previous top element and // the range delete iterator == oldTopLevel is positioned at or past the key of the // previous top element. We need to position the range delete iterators from oldTopLevel + 1 // to the level of the current top element. func (m *mergingIter) initMinRangeDelIters(oldTopLevel int) { if m.heap.len() == 0 { return } // Position the range-del iterators at levels <= m.heap.items[0].index. item := m.heap.items[0] for level := oldTopLevel + 1; level <= item.index; level++ { l := &m.levels[level] if l.rangeDelIter == nil { continue } l.tombstone = l.rangeDelIter.SeekGE(item.iterKey.UserKey) } } func (m *mergingIter) initMaxHeap() { m.dir = -1 m.heap.reverse = true m.initHeap() m.initMaxRangeDelIters(-1) } // The level of the previous top element was oldTopLevel. Note that all range delete // iterators < oldTopLevel are positioned before the key of the previous top element and // the range delete iterator == oldTopLevel is positioned at or before the key of the // previous top element. We need to position the range delete iterators from oldTopLevel + 1 // to the level of the current top element. func (m *mergingIter) initMaxRangeDelIters(oldTopLevel int) { if m.heap.len() == 0 { return } // Position the range-del iterators at levels <= m.heap.items[0].index. item := m.heap.items[0] for level := oldTopLevel + 1; level <= item.index; level++ { l := &m.levels[level] if l.rangeDelIter == nil { continue } l.tombstone = keyspan.SeekLE(m.heap.cmp, l.rangeDelIter, item.iterKey.UserKey) } } func (m *mergingIter) switchToMinHeap() { if m.heap.len() == 0 { if m.lower != nil { m.SeekGE(m.lower, base.SeekGEFlagsNone) } else { m.First() } return } // We're switching from using a max heap to a min heap. We need to advance // any iterator that is less than or equal to the current key. Consider the // scenario where we have 2 iterators being merged (user-key:seq-num): // // i1: *a:2 b:2 // i2: a:1 b:1 // // The current key is a:2 and i2 is pointed at a:1. When we switch to forward // iteration, we want to return a key that is greater than a:2. key := m.heap.items[0].iterKey cur := m.heap.items[0] for i := range m.levels { l := &m.levels[i] if l == cur { continue } // If the iterator is exhausted, it may be out of bounds if range // deletions modified our search key as we descended. we need to // reposition it within the search bounds. If the current key is a // range tombstone, the iterator might still be exhausted but at a // sstable boundary sentinel. It would be okay to reposition an // interator like this only through successive Next calls, except that // it would violate the levelIter's invariants by causing it to return // a key before the lower bound. // // bounds = [ f, _ ) // L0: [ b ] [ f* z ] // L1: [ a |----| k y ] // L2: [ c (d) ] [ e g m ] // L3: [ x ] // // * - current key [] - table bounds () - heap item // // In the above diagram, the L2 iterator is positioned at a sstable // boundary (d) outside the lower bound (f). It arrived here from a // seek whose seek-key was modified by a range tombstone. If we called // Next on the L2 iterator, it would return e, violating its lower // bound. Instead, we seek it to >= f and Next from there. if l.iterKey == nil || (m.lower != nil && l.isSyntheticIterBoundsKey && l.iterKey.IsExclusiveSentinel() && m.heap.cmp(l.iterKey.UserKey, m.lower) <= 0) { if m.lower != nil { l.iterKey, l.iterValue = l.iter.SeekGE(m.lower, base.SeekGEFlagsNone) } else { l.iterKey, l.iterValue = l.iter.First() } } for ; l.iterKey != nil; l.iterKey, l.iterValue = l.iter.Next() { if base.InternalCompare(m.heap.cmp, *key, *l.iterKey) < 0 { // key < iter-key break } // key >= iter-key } } // Special handling for the current iterator because we were using its key // above. The iterator cur.iter may still be exhausted at a sstable boundary // sentinel. Similar to the logic applied to the other levels, in these // cases we seek the iterator to the first key in order to avoid violating // levelIter's invariants. See the example in the for loop above. if m.lower != nil && cur.isSyntheticIterBoundsKey && cur.iterKey.IsExclusiveSentinel() && m.heap.cmp(cur.iterKey.UserKey, m.lower) <= 0 { cur.iterKey, cur.iterValue = cur.iter.SeekGE(m.lower, base.SeekGEFlagsNone) } else { cur.iterKey, cur.iterValue = cur.iter.Next() } m.initMinHeap() } func (m *mergingIter) switchToMaxHeap() { if m.heap.len() == 0 { if m.upper != nil { m.SeekLT(m.upper, base.SeekLTFlagsNone) } else { m.Last() } return } // We're switching from using a min heap to a max heap. We need to backup any // iterator that is greater than or equal to the current key. Consider the // scenario where we have 2 iterators being merged (user-key:seq-num): // // i1: a:2 *b:2 // i2: a:1 b:1 // // The current key is b:2 and i2 is pointing at b:1. When we switch to // reverse iteration, we want to return a key that is less than b:2. key := m.heap.items[0].iterKey cur := m.heap.items[0] for i := range m.levels { l := &m.levels[i] if l == cur { continue } // If the iterator is exhausted, it may be out of bounds if range // deletions modified our search key as we descended. we need to // reposition it within the search bounds. If the current key is a // range tombstone, the iterator might still be exhausted but at a // sstable boundary sentinel. It would be okay to reposition an // interator like this only through successive Prev calls, except that // it would violate the levelIter's invariants by causing it to return // a key beyond the upper bound. // // bounds = [ _, g ) // L0: [ b ] [ f* z ] // L1: [ a |-------| k y ] // L2: [ c d ] h [(i) m ] // L3: [ e x ] // // * - current key [] - table bounds () - heap item // // In the above diagram, the L2 iterator is positioned at a sstable // boundary (i) outside the upper bound (g). It arrived here from a // seek whose seek-key was modified by a range tombstone. If we called // Prev on the L2 iterator, it would return h, violating its upper // bound. Instead, we seek it to < g, and Prev from there. if l.iterKey == nil || (m.upper != nil && l.isSyntheticIterBoundsKey && l.iterKey.IsExclusiveSentinel() && m.heap.cmp(l.iterKey.UserKey, m.upper) >= 0) { if m.upper != nil { l.iterKey, l.iterValue = l.iter.SeekLT(m.upper, base.SeekLTFlagsNone) } else { l.iterKey, l.iterValue = l.iter.Last() } } for ; l.iterKey != nil; l.iterKey, l.iterValue = l.iter.Prev() { if base.InternalCompare(m.heap.cmp, *key, *l.iterKey) > 0 { // key > iter-key break } // key <= iter-key } } // Special handling for the current iterator because we were using its key // above. The iterator cur.iter may still be exhausted at a sstable boundary // sentinel. Similar to the logic applied to the other levels, in these // cases we seek the iterator to in order to avoid violating levelIter's // invariants by Prev-ing through files. See the example in the for loop // above. if m.upper != nil && cur.isSyntheticIterBoundsKey && cur.iterKey.IsExclusiveSentinel() && m.heap.cmp(cur.iterKey.UserKey, m.upper) >= 0 { cur.iterKey, cur.iterValue = cur.iter.SeekLT(m.upper, base.SeekLTFlagsNone) } else { cur.iterKey, cur.iterValue = cur.iter.Prev() } m.initMaxHeap() } // maybeNextEntryWithinPrefix steps to the next entry, as long as the iteration // prefix has not already been exceeded. If it has, it exhausts the iterator by // resetting the heap to empty. func (m *mergingIter) maybeNextEntryWithinPrefix(l *mergingIterLevel) { if s := m.split(l.iterKey.UserKey); !bytes.Equal(m.prefix, l.iterKey.UserKey[:s]) { // The item at the root of the heap already exceeds the iteration // prefix. We should not advance any more. Clear the heap to reflect // that the iterator is now exhausted (within this prefix, at // least). m.heap.items = m.heap.items[:0] return } m.nextEntry(l, nil /* succKey */) } // nextEntry unconditionally steps to the next entry. item is the current top // item in the heap. // // nextEntry should be called directly when not in prefix-iteration mode, or by // Next. During prefix iteration mode, all other callers should use // maybeNextEntryWithinPrefix which will avoid advancing the iterator if the // current iteration prefix has been exhausted. See the comment within // nextEntry's body for an explanation of why other callers should call // maybeNextEntryWithinPrefix, which will ensure the documented invariant is // preserved. func (m *mergingIter) nextEntry(l *mergingIterLevel, succKey []byte) { // INVARIANT: If in prefix iteration mode, item.iterKey must have a prefix equal // to m.prefix. This invariant is important for ensuring TrySeekUsingNext // optimizations behave correctly. // // During prefix iteration, the iterator does not have a full view of the // LSM. Some level iterators may omit keys that are known to fall outside // the seek prefix (eg, due to sstable bloom filter exclusion). It's // important that in such cases we don't position any iterators beyond // m.prefix, because doing so may interfere with future seeks. // // Let prefixes P1 < P2 < P3. Imagine a SeekPrefixGE to prefix P1, followed // by a SeekPrefixGE to prefix P2. Imagine there exist live keys at prefix // P2, but they're not visible to the SeekPrefixGE(P1) (because of // bloom-filter exclusion or a range tombstone that deletes prefix P1 but // not P2). If the SeekPrefixGE(P1) is allowed to move any level iterators // to P3, the SeekPrefixGE(P2, TrySeekUsingNext=true) may mistakenly think // the level contains no point keys or range tombstones within the prefix // P2. Care is taken to avoid ever advancing the iterator beyond the current // prefix. If nextEntry is ever invoked while we're already beyond the // current prefix, we're violating the invariant. if invariants.Enabled && m.prefix != nil { if s := m.split(l.iterKey.UserKey); !bytes.Equal(m.prefix, l.iterKey.UserKey[:s]) { m.logger.Fatalf("mergingIter: prefix violation: nexting beyond prefix %q; existing heap root %q\n%s", m.prefix, l.iterKey, debug.Stack()) } } oldTopLevel := l.index oldRangeDelIter := l.rangeDelIter if succKey == nil { l.iterKey, l.iterValue = l.iter.Next() } else { l.iterKey, l.iterValue = l.iter.NextPrefix(succKey) } if l.iterKey != nil { if m.heap.len() > 1 { m.heap.fix(0) } if l.rangeDelIter != oldRangeDelIter { // The rangeDelIter changed which indicates that the l.iter moved to the // next sstable. We have to update the tombstone for oldTopLevel as well. oldTopLevel-- } } else { m.err = l.iter.Error() if m.err == nil { m.heap.pop() } } // The cached tombstones are only valid for the levels // [0,oldTopLevel]. Updated the cached tombstones for any levels in the range // [oldTopLevel+1,heap[0].index]. m.initMinRangeDelIters(oldTopLevel) } // isNextEntryDeleted starts from the current entry (as the next entry) and if // it is deleted, moves the iterators forward as needed and returns true, else // it returns false. item is the top item in the heap. // // During prefix iteration mode, isNextEntryDeleted will exhaust the iterator by // clearing the heap if the deleted key(s) extend beyond the iteration prefix // during prefix-iteration mode. func (m *mergingIter) isNextEntryDeleted(item *mergingIterLevel) bool { // Look for a range deletion tombstone containing item.iterKey at higher // levels (level < item.index). If we find such a range tombstone we know // it deletes the key in the current level. Also look for a range // deletion at the current level (level == item.index). If we find such a // range deletion we need to check whether it is newer than the current // entry. for level := 0; level <= item.index; level++ { l := &m.levels[level] if l.rangeDelIter == nil || l.tombstone == nil { // If l.tombstone is nil, there are no further tombstones // in the current sstable in the current (forward) iteration // direction. continue } if m.heap.cmp(l.tombstone.End, item.iterKey.UserKey) <= 0 { // The current key is at or past the tombstone end key. // // NB: for the case that this l.rangeDelIter is provided by a levelIter we know that // the levelIter must be positioned at a key >= item.iterKey. So it is sufficient to seek the // current l.rangeDelIter (since any range del iterators that will be provided by the // levelIter in the future cannot contain item.iterKey). Also, it is possible that we // will encounter parts of the range delete that should be ignored -- we handle that // below. l.tombstone = l.rangeDelIter.SeekGE(item.iterKey.UserKey) } if l.tombstone == nil { continue } // Reasoning for correctness of untruncated tombstone handling when the untruncated // tombstone is at a higher level: // The iterator corresponding to this tombstone is still in the heap so it must be // positioned >= item.iterKey. Which means the Largest key bound of the sstable containing this // tombstone is >= item.iterKey. So the upper limit of this tombstone cannot be file-bounds-constrained // to < item.iterKey. But it is possible that item.key < smallestUserKey, in which // case this tombstone should be ignored. // // Example 1: // sstable bounds [c#8, g#12] containing a tombstone [b, i)#7, and key is c#6. The // smallestUserKey is c, so we know the key is within the file bounds and the tombstone // [b, i) covers it. // // Example 2: // Same sstable bounds but key is b#10. The smallestUserKey is c, so the tombstone [b, i) // does not cover this key. // // For a tombstone at the same level as the key, the file bounds are trivially satisfied. if (l.smallestUserKey == nil || m.heap.cmp(l.smallestUserKey, item.iterKey.UserKey) <= 0) && l.tombstone.VisibleAt(m.snapshot) && l.tombstone.Contains(m.heap.cmp, item.iterKey.UserKey) { if level < item.index { // We could also do m.seekGE(..., level + 1). The levels from // [level + 1, item.index) are already after item.iterKey so seeking them may be // wasteful. // We can seek up to the min of largestUserKey and tombstone.End. // // Using example 1 above, we can seek to the smaller of g and i, which is g. // // Another example, where the sstable bounds are [c#8, i#InternalRangeDelSentinel], // and the tombstone is [b, i)#8. Seeking to i is correct since it is seeking up to // the exclusive bound of the tombstone. We do not need to look at // isLargestKeyRangeDelSentinel. // // Progress argument: Since this file is at a higher level than item.iterKey we know // that the iterator in this file must be positioned within its bounds and at a key // X > item.iterKey (otherwise it would be the min of the heap). It is not // possible for X.UserKey == item.iterKey.UserKey, since it is incompatible with // X > item.iterKey (a lower version cannot be in a higher sstable), so it must be that // X.UserKey > item.iterKey.UserKey. Which means l.largestUserKey > item.key.UserKey. // We also know that l.tombstone.End > item.iterKey.UserKey. So the min of these, // seekKey, computed below, is > item.iterKey.UserKey, so the call to seekGE() will // make forward progress. seekKey := l.tombstone.End if l.largestUserKey != nil && m.heap.cmp(l.largestUserKey, seekKey) < 0 { seekKey = l.largestUserKey } // This seek is not directly due to a SeekGE call, so we don't know // enough about the underlying iterator positions, and so we keep the // try-seek-using-next optimization disabled. Additionally, if we're in // prefix-seek mode and a re-seek would have moved us past the original // prefix, we can remove all merging iter levels below the rangedel // tombstone's level and return immediately instead of re-seeking. This // is correct since those levels cannot provide a key that matches the // prefix, and is also visible. Additionally, this is important to make // subsequent `TrySeekUsingNext` work correctly, as a re-seek on a // different prefix could have resulted in this iterator skipping visible // keys at prefixes in between m.prefix and seekKey, that are currently // not in the heap due to a bloom filter mismatch. // // Additionally, we set the relative-seek flag. This is // important when iterating with lazy combined iteration. If // there's a range key between this level's current file and the // file the seek will land on, we need to detect it in order to // trigger construction of the combined iterator. if m.prefix != nil { if n := m.split(seekKey); !bytes.Equal(m.prefix, seekKey[:n]) { for i := item.index; i < len(m.levels); i++ { // Remove this level from the heap. Setting iterKey and iterValue // to their zero values should be sufficient for initMinHeap to not // re-initialize the heap with them in it. Other fields in // mergingIterLevel can remain as-is; the iter/rangeDelIter needs // to stay intact for future trySeekUsingNexts to work, the level // iter boundary context is owned by the levelIter which is not // being repositioned, and any tombstones in these levels will be // irrelevant for us anyway. m.levels[i].iterKey = nil m.levels[i].iterValue = base.LazyValue{} } // TODO(bilal): Consider a more efficient way of removing levels from // the heap without reinitializing all of it. This would likely // necessitate tracking the heap positions of each mergingIterHeap // item in the mergingIterLevel, and then swapping that item in the // heap with the last-positioned heap item, and shrinking the heap by // one. m.initMinHeap() return true } } m.seekGE(seekKey, item.index, base.SeekGEFlagsNone.EnableRelativeSeek()) return true } if l.tombstone.CoversAt(m.snapshot, item.iterKey.SeqNum()) { if m.prefix == nil { m.nextEntry(item, nil /* succKey */) } else { m.maybeNextEntryWithinPrefix(item) } return true } } } return false } // Starting from the current entry, finds the first (next) entry that can be returned. func (m *mergingIter) findNextEntry() (*InternalKey, base.LazyValue) { for m.heap.len() > 0 && m.err == nil { item := m.heap.items[0] if m.levels[item.index].isSyntheticIterBoundsKey { break } m.addItemStats(item) // Skip ignorable boundary keys. These are not real keys and exist to // keep sstables open until we've surpassed their end boundaries so that // their range deletions are visible. if m.levels[item.index].isIgnorableBoundaryKey { if m.prefix == nil { m.nextEntry(item, nil /* succKey */) } else { m.maybeNextEntryWithinPrefix(item) } continue } // Check if the heap root key is deleted by a range tombstone in a // higher level. If it is, isNextEntryDeleted will advance the iterator // to a later key (through seeking or nexting). if m.isNextEntryDeleted(item) { m.stats.PointsCoveredByRangeTombstones++ continue } // Check if the key is visible at the iterator sequence numbers. if !item.iterKey.Visible(m.snapshot, m.batchSnapshot) { if m.prefix == nil { m.nextEntry(item, nil /* succKey */) } else { m.maybeNextEntryWithinPrefix(item) } continue } // The heap root is visible and not deleted by any range tombstones. // Return it. return item.iterKey, item.iterValue } return nil, base.LazyValue{} } // Steps to the prev entry. item is the current top item in the heap. func (m *mergingIter) prevEntry(l *mergingIterLevel) { oldTopLevel := l.index oldRangeDelIter := l.rangeDelIter if l.iterKey, l.iterValue = l.iter.Prev(); l.iterKey != nil { if m.heap.len() > 1 { m.heap.fix(0) } if l.rangeDelIter != oldRangeDelIter && l.rangeDelIter != nil { // The rangeDelIter changed which indicates that the l.iter moved to the // previous sstable. We have to update the tombstone for oldTopLevel as // well. oldTopLevel-- } } else { m.err = l.iter.Error() if m.err == nil { m.heap.pop() } } // The cached tombstones are only valid for the levels // [0,oldTopLevel]. Updated the cached tombstones for any levels in the range // [oldTopLevel+1,heap[0].index]. m.initMaxRangeDelIters(oldTopLevel) } // isPrevEntryDeleted() starts from the current entry (as the prev entry) and if it is deleted, // moves the iterators backward as needed and returns true, else it returns false. item is the top // item in the heap. func (m *mergingIter) isPrevEntryDeleted(item *mergingIterLevel) bool { // Look for a range deletion tombstone containing item.iterKey at higher // levels (level < item.index). If we find such a range tombstone we know // it deletes the key in the current level. Also look for a range // deletion at the current level (level == item.index). If we find such a // range deletion we need to check whether it is newer than the current // entry. for level := 0; level <= item.index; level++ { l := &m.levels[level] if l.rangeDelIter == nil || l.tombstone == nil { // If l.tombstone is nil, there are no further tombstones // in the current sstable in the current (reverse) iteration // direction. continue } if m.heap.cmp(item.iterKey.UserKey, l.tombstone.Start) < 0 { // The current key is before the tombstone start key. // // NB: for the case that this l.rangeDelIter is provided by a levelIter we know that // the levelIter must be positioned at a key < item.iterKey. So it is sufficient to seek the // current l.rangeDelIter (since any range del iterators that will be provided by the // levelIter in the future cannot contain item.iterKey). Also, it is it is possible that we // will encounter parts of the range delete that should be ignored -- we handle that // below. l.tombstone = keyspan.SeekLE(m.heap.cmp, l.rangeDelIter, item.iterKey.UserKey) } if l.tombstone == nil { continue } // Reasoning for correctness of untruncated tombstone handling when the untruncated // tombstone is at a higher level: // // The iterator corresponding to this tombstone is still in the heap so it must be // positioned <= item.iterKey. Which means the Smallest key bound of the sstable containing this // tombstone is <= item.iterKey. So the lower limit of this tombstone cannot have been // file-bounds-constrained to > item.iterKey. But it is possible that item.key >= Largest // key bound of this sstable, in which case this tombstone should be ignored. // // Example 1: // sstable bounds [c#8, g#12] containing a tombstone [b, i)#7, and key is f#6. The // largestUserKey is g, so we know the key is within the file bounds and the tombstone // [b, i) covers it. // // Example 2: // Same sstable but the key is g#6. This cannot happen since the [b, i)#7 untruncated // tombstone was involved in a compaction which must have had a file to the right of this // sstable that is part of the same atomic compaction group for future compactions. That // file must have bounds that cover g#6 and this levelIter must be at that file. // // Example 3: // sstable bounds [c#8, g#RangeDelSentinel] containing [b, i)#7 and the key is g#10. // This key is not deleted by this tombstone. We need to look at // isLargestUserKeyExclusive. // // For a tombstone at the same level as the key, the file bounds are trivially satisfied. // Default to within bounds. withinLargestSSTableBound := true if l.largestUserKey != nil { cmpResult := m.heap.cmp(l.largestUserKey, item.iterKey.UserKey) withinLargestSSTableBound = cmpResult > 0 || (cmpResult == 0 && !l.isLargestUserKeyExclusive) } if withinLargestSSTableBound && l.tombstone.Contains(m.heap.cmp, item.iterKey.UserKey) && l.tombstone.VisibleAt(m.snapshot) { if level < item.index { // We could also do m.seekLT(..., level + 1). The levels from // [level + 1, item.index) are already before item.iterKey so seeking them may be // wasteful. // We can seek up to the max of smallestUserKey and tombstone.Start.UserKey. // // Using example 1 above, we can seek to the larger of c and b, which is c. // // Progress argument: We know that the iterator in this file is positioned within // its bounds and at a key X < item.iterKey (otherwise it would be the max of the heap). // So smallestUserKey <= item.iterKey.UserKey and we already know that // l.tombstone.Start.UserKey <= item.iterKey.UserKey. So the seekKey computed below // is <= item.iterKey.UserKey, and since we do a seekLT() we will make backwards // progress. seekKey := l.tombstone.Start if l.smallestUserKey != nil && m.heap.cmp(l.smallestUserKey, seekKey) > 0 { seekKey = l.smallestUserKey } // We set the relative-seek flag. This is important when // iterating with lazy combined iteration. If there's a range // key between this level's current file and the file the seek // will land on, we need to detect it in order to trigger // construction of the combined iterator. m.seekLT(seekKey, item.index, base.SeekLTFlagsNone.EnableRelativeSeek()) return true } if l.tombstone.CoversAt(m.snapshot, item.iterKey.SeqNum()) { m.prevEntry(item) return true } } } return false } // Starting from the current entry, finds the first (prev) entry that can be returned. func (m *mergingIter) findPrevEntry() (*InternalKey, base.LazyValue) { for m.heap.len() > 0 && m.err == nil { item := m.heap.items[0] if m.levels[item.index].isSyntheticIterBoundsKey { break } m.addItemStats(item) if m.isPrevEntryDeleted(item) { m.stats.PointsCoveredByRangeTombstones++ continue } if item.iterKey.Visible(m.snapshot, m.batchSnapshot) && (!m.levels[item.index].isIgnorableBoundaryKey) { return item.iterKey, item.iterValue } m.prevEntry(item) } return nil, base.LazyValue{} } // Seeks levels >= level to >= key. Additionally uses range tombstones to extend the seeks. func (m *mergingIter) seekGE(key []byte, level int, flags base.SeekGEFlags) { // When seeking, we can use tombstones to adjust the key we seek to on each // level. Consider the series of range tombstones: // // 1: a---e // 2: d---h // 3: g---k // 4: j---n // 5: m---q // // If we SeekGE("b") we also find the tombstone "b" resides within in the // first level which is [a,e). Regardless of whether this tombstone deletes // "b" in that level, we know it deletes "b" in all lower levels, so we // adjust the search key in the next level to the tombstone end key "e". We // then SeekGE("e") in the second level and find the corresponding tombstone // [d,h). This process continues and we end up seeking for "h" in the 3rd // level, "k" in the 4th level and "n" in the last level. // // TODO(peter,rangedel): In addition to the above we can delay seeking a // level (and any lower levels) when the current iterator position is // contained within a range tombstone at a higher level. // Deterministically disable the TrySeekUsingNext optimizations sometimes in // invariant builds to encourage the metamorphic tests to surface bugs. Note // that we cannot disable the optimization within individual levels. It must // be disabled for all levels or none. If one lower-level iterator performs // a fresh seek whereas another takes advantage of its current iterator // position, the heap can become inconsistent. Consider the following // example: // // L5: [ [b-c) ] [ d ]* // L6: [ b ] [e]* // // Imagine a SeekGE(a). The [b-c) range tombstone deletes the L6 point key // 'b', resulting in the iterator positioned at d with the heap: // // {L5: d, L6: e} // // A subsequent SeekGE(b) is seeking to a larger key, so the caller may set // TrySeekUsingNext()=true. If the L5 iterator used the TrySeekUsingNext // optimization but the L6 iterator did not, the iterator would have the // heap: // // {L6: b, L5: d} // // Because the L5 iterator has already advanced to the next sstable, the // merging iterator cannot observe the [b-c) range tombstone and will // mistakenly return L6's deleted point key 'b'. if invariants.Enabled && flags.TrySeekUsingNext() && !m.forceEnableSeekOpt && disableSeekOpt(key, uintptr(unsafe.Pointer(m))) { flags = flags.DisableTrySeekUsingNext() } for ; level < len(m.levels); level++ { if invariants.Enabled && m.lower != nil && m.heap.cmp(key, m.lower) < 0 { m.logger.Fatalf("mergingIter: lower bound violation: %s < %s\n%s", key, m.lower, debug.Stack()) } l := &m.levels[level] if m.prefix != nil { l.iterKey, l.iterValue = l.iter.SeekPrefixGE(m.prefix, key, flags) } else { l.iterKey, l.iterValue = l.iter.SeekGE(key, flags) } // If this level contains overlapping range tombstones, alter the seek // key accordingly. Caveat: If we're performing lazy-combined iteration, // we cannot alter the seek key: Range tombstones don't delete range // keys, and there might exist live range keys within the range // tombstone's span that need to be observed to trigger a switch to // combined iteration. if rangeDelIter := l.rangeDelIter; rangeDelIter != nil && (m.combinedIterState == nil || m.combinedIterState.initialized) { // The level has a range-del iterator. Find the tombstone containing // the search key. // // For untruncated tombstones that are possibly file-bounds-constrained, we are using a // levelIter which will set smallestUserKey and largestUserKey. Since the levelIter // is at this file we know that largestUserKey >= key, so we know that the // tombstone we find cannot be file-bounds-constrained in its upper bound to something < key. // We do need to compare with smallestUserKey to ensure that the tombstone is not // file-bounds-constrained in its lower bound. // // See the detailed comments in isNextEntryDeleted() on why similar containment and // seeking logic is correct. The subtle difference here is that key is a user key, // so we can have a sstable with bounds [c#8, i#InternalRangeDelSentinel], and the // tombstone is [b, k)#8 and the seek key is i: levelIter.SeekGE(i) will move past // this sstable since it realizes the largest key is a InternalRangeDelSentinel. l.tombstone = rangeDelIter.SeekGE(key) if l.tombstone != nil && l.tombstone.VisibleAt(m.snapshot) && l.tombstone.Contains(m.heap.cmp, key) && (l.smallestUserKey == nil || m.heap.cmp(l.smallestUserKey, key) <= 0) { // NB: Based on the comment above l.largestUserKey >= key, and based on the // containment condition tombstone.End > key, so the assignment to key results // in a monotonically non-decreasing key across iterations of this loop. // // The adjustment of key here can only move it to a larger key. Since // the caller of seekGE guaranteed that the original key was greater // than or equal to m.lower, the new key will continue to be greater // than or equal to m.lower. if l.largestUserKey != nil && m.heap.cmp(l.largestUserKey, l.tombstone.End) < 0 { // Truncate the tombstone for seeking purposes. Note that this can over-truncate // but that is harmless for this seek optimization. key = l.largestUserKey } else { key = l.tombstone.End } } } } m.initMinHeap() } func (m *mergingIter) String() string { return "merging" } // SeekGE implements base.InternalIterator.SeekGE. 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. func (m *mergingIter) SeekGE(key []byte, flags base.SeekGEFlags) (*InternalKey, base.LazyValue) { m.err = nil // clear cached iteration error m.prefix = nil m.seekGE(key, 0 /* start level */, flags) return m.findNextEntry() } // SeekPrefixGE implements base.InternalIterator.SeekPrefixGE. 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. func (m *mergingIter) SeekPrefixGE( prefix, key []byte, flags base.SeekGEFlags, ) (*base.InternalKey, base.LazyValue) { m.err = nil // clear cached iteration error m.prefix = prefix m.seekGE(key, 0 /* start level */, flags) return m.findNextEntry() } // Seeks levels >= level to < key. Additionally uses range tombstones to extend the seeks. func (m *mergingIter) seekLT(key []byte, level int, flags base.SeekLTFlags) { // See the comment in seekGE regarding using tombstones to adjust the seek // target per level. m.prefix = nil for ; level < len(m.levels); level++ { if invariants.Enabled && m.upper != nil && m.heap.cmp(key, m.upper) > 0 { m.logger.Fatalf("mergingIter: upper bound violation: %s > %s\n%s", key, m.upper, debug.Stack()) } l := &m.levels[level] l.iterKey, l.iterValue = l.iter.SeekLT(key, flags) // If this level contains overlapping range tombstones, alter the seek // key accordingly. Caveat: If we're performing lazy-combined iteration, // we cannot alter the seek key: Range tombstones don't delete range // keys, and there might exist live range keys within the range // tombstone's span that need to be observed to trigger a switch to // combined iteration. if rangeDelIter := l.rangeDelIter; rangeDelIter != nil && (m.combinedIterState == nil || m.combinedIterState.initialized) { // The level has a range-del iterator. Find the tombstone containing // the search key. // // For untruncated tombstones that are possibly file-bounds-constrained we are using a // levelIter which will set smallestUserKey and largestUserKey. Since the levelIter // is at this file we know that smallestUserKey <= key, so we know that the // tombstone we find cannot be file-bounds-constrained in its lower bound to something > key. // We do need to compare with largestUserKey to ensure that the tombstone is not // file-bounds-constrained in its upper bound. // // See the detailed comments in isPrevEntryDeleted() on why similar containment and // seeking logic is correct. // Default to within bounds. withinLargestSSTableBound := true if l.largestUserKey != nil { cmpResult := m.heap.cmp(l.largestUserKey, key) withinLargestSSTableBound = cmpResult > 0 || (cmpResult == 0 && !l.isLargestUserKeyExclusive) } l.tombstone = keyspan.SeekLE(m.heap.cmp, rangeDelIter, key) if l.tombstone != nil && l.tombstone.VisibleAt(m.snapshot) && l.tombstone.Contains(m.heap.cmp, key) && withinLargestSSTableBound { // NB: Based on the comment above l.smallestUserKey <= key, and based // on the containment condition tombstone.Start.UserKey <= key, so the // assignment to key results in a monotonically non-increasing key // across iterations of this loop. // // The adjustment of key here can only move it to a smaller key. Since // the caller of seekLT guaranteed that the original key was less than // or equal to m.upper, the new key will continue to be less than or // equal to m.upper. if l.smallestUserKey != nil && m.heap.cmp(l.smallestUserKey, l.tombstone.Start) >= 0 { // Truncate the tombstone for seeking purposes. Note that this can over-truncate // but that is harmless for this seek optimization. key = l.smallestUserKey } else { key = l.tombstone.Start } } } } m.initMaxHeap() } // SeekLT implements base.InternalIterator.SeekLT. Note that SeekLT only checks // the lower bound. It is up to the caller to ensure that key is less than the // upper bound. func (m *mergingIter) SeekLT(key []byte, flags base.SeekLTFlags) (*InternalKey, base.LazyValue) { m.err = nil // clear cached iteration error m.prefix = nil m.seekLT(key, 0 /* start level */, flags) return m.findPrevEntry() } // First implements base.InternalIterator.First. Note that First only checks // the upper bound. It is up to the caller to ensure that key is greater than // or equal to the lower bound (e.g. via a call to SeekGE(lower)). func (m *mergingIter) First() (*InternalKey, base.LazyValue) { m.err = nil // clear cached iteration error m.prefix = nil m.heap.items = m.heap.items[:0] for i := range m.levels { l := &m.levels[i] l.iterKey, l.iterValue = l.iter.First() } m.initMinHeap() return m.findNextEntry() } // Last implements base.InternalIterator.Last. Note that Last only checks the // lower bound. It is up to the caller to ensure that key is less than the // upper bound (e.g. via a call to SeekLT(upper)) func (m *mergingIter) Last() (*InternalKey, base.LazyValue) { m.err = nil // clear cached iteration error m.prefix = nil for i := range m.levels { l := &m.levels[i] l.iterKey, l.iterValue = l.iter.Last() } m.initMaxHeap() return m.findPrevEntry() } func (m *mergingIter) Next() (*InternalKey, base.LazyValue) { if m.err != nil { return nil, base.LazyValue{} } if m.dir != 1 { m.switchToMinHeap() return m.findNextEntry() } if m.heap.len() == 0 { return nil, base.LazyValue{} } // NB: It's okay to call nextEntry directly even during prefix iteration // mode (as opposed to indirectly through maybeNextEntryWithinPrefix). // During prefix iteration mode, we rely on the caller to not call Next if // the iterator has already advanced beyond the iteration prefix. See the // comment above the base.InternalIterator interface. m.nextEntry(m.heap.items[0], nil /* succKey */) return m.findNextEntry() } func (m *mergingIter) NextPrefix(succKey []byte) (*InternalKey, LazyValue) { if m.dir != 1 { panic("pebble: cannot switch directions with NextPrefix") } if m.err != nil || m.heap.len() == 0 { return nil, LazyValue{} } if m.levelsPositioned == nil { m.levelsPositioned = make([]bool, len(m.levels)) } else { for i := range m.levelsPositioned { m.levelsPositioned[i] = false } } // The heap root necessarily must be positioned at a key < succKey, because // NextPrefix was invoked. root := &m.heap.items[0] m.levelsPositioned[(*root).index] = true if invariants.Enabled && m.heap.cmp((*root).iterKey.UserKey, succKey) >= 0 { m.logger.Fatalf("pebble: invariant violation: NextPrefix(%q) called on merging iterator already positioned at %q", succKey, (*root).iterKey) } m.nextEntry(*root, succKey) // NB: root is a pointer to the heap root. nextEntry may have changed // the heap root, so we must not expect root to still point to the same // level (or to even be valid, if the heap is now exhaused). for m.heap.len() > 0 { if m.levelsPositioned[(*root).index] { // A level we've previously positioned is at the top of the heap, so // there are no other levels positioned at keys < succKey. We've // advanced as far as we need to. break } // Since this level was not the original heap root when NextPrefix was // called, we don't know whether this level's current key has the // previous prefix or a new one. if m.heap.cmp((*root).iterKey.UserKey, succKey) >= 0 { break } m.levelsPositioned[(*root).index] = true m.nextEntry(*root, succKey) } return m.findNextEntry() } func (m *mergingIter) Prev() (*InternalKey, base.LazyValue) { if m.err != nil { return nil, base.LazyValue{} } if m.dir != -1 { if m.prefix != nil { m.err = errors.New("pebble: unsupported reverse prefix iteration") return nil, base.LazyValue{} } m.switchToMaxHeap() return m.findPrevEntry() } if m.heap.len() == 0 { return nil, base.LazyValue{} } m.prevEntry(m.heap.items[0]) return m.findPrevEntry() } func (m *mergingIter) Error() error { if m.heap.len() == 0 || m.err != nil { return m.err } return m.levels[m.heap.items[0].index].iter.Error() } func (m *mergingIter) Close() error { for i := range m.levels { iter := m.levels[i].iter if err := iter.Close(); err != nil && m.err == nil { m.err = err } if rangeDelIter := m.levels[i].rangeDelIter; rangeDelIter != nil { if err := rangeDelIter.Close(); err != nil && m.err == nil { m.err = err } } } m.levels = nil m.heap.items = m.heap.items[:0] return m.err } func (m *mergingIter) SetBounds(lower, upper []byte) { m.prefix = nil m.lower = lower m.upper = upper for i := range m.levels { m.levels[i].iter.SetBounds(lower, upper) } m.heap.clear() } func (m *mergingIter) SetContext(ctx context.Context) { for i := range m.levels { m.levels[i].iter.SetContext(ctx) } } func (m *mergingIter) DebugString() string { var buf bytes.Buffer sep := "" for m.heap.len() > 0 { item := m.heap.pop() fmt.Fprintf(&buf, "%s%s", sep, item.iterKey) sep = " " } if m.dir == 1 { m.initMinHeap() } else { m.initMaxHeap() } return buf.String() } func (m *mergingIter) ForEachLevelIter(fn func(li *levelIter) bool) { for _, ml := range m.levels { if ml.levelIter != nil { if done := fn(ml.levelIter); done { break } } } } func (m *mergingIter) addItemStats(l *mergingIterLevel) { m.stats.PointCount++ m.stats.KeyBytes += uint64(len(l.iterKey.UserKey)) m.stats.ValueBytes += uint64(len(l.iterValue.ValueOrHandle)) } var _ internalIterator = &mergingIter{}