Skip to content

Commit

Permalink
db: changes to make iterators respect the bounds specified by the caller
Browse files Browse the repository at this point in the history
- Iterator ensures that keys used for seek respect both lower and upper
  bounds.
- levelIter returns synthetic keys corresponding to the iterator bounds,
  when they are present, instead of the file bounds. Additionally,
  the mergingIter is told that it has done so using the
  isSyntheticIterBoundsKey field -- this prevents the mergingIter to
  get stuck at such a key when it becomes the top of the heap.

The latter is needed to retrict the cases where the mergingIter
breaks an invariant that the iterators below it will not be positioned
more than one key outside the iterator bounds. Such an invariant is
needed for a later change that will optimize seeks by using next.
Say there is an upper bound k1, and the mergingIter breaks the
invariant by positioning a levelIter at k3 > k1. This is acceptable
only if any keys k2 such that k1 < k2 < k3 are deleted by a range
tombstone. The old code returned the file bounds as the synthetic
keys which could be arbitrarily far away from the iterator bounds.
Using the above example, it could have returned k3. If k3 becomes
the top element of the heap, and happens to be deleted by a tombstone
[k3, k4) (the tombstone would be at a higher level), the code
would seek to k4, while k2 was not deleted, which means iterators
can be more than one (internal) key outide the iterator bounds.
  • Loading branch information
sumeerbhola committed Oct 1, 2020
1 parent 91e8175 commit 879f3bf
Show file tree
Hide file tree
Showing 9 changed files with 112 additions and 41 deletions.
1 change: 1 addition & 0 deletions db.go
Original file line number Diff line number Diff line change
Expand Up @@ -771,6 +771,7 @@ func (d *DB) newIterInternal(
li.initRangeDel(&mlevels[0].rangeDelIter)
li.initSmallestLargestUserKey(&mlevels[0].smallestUserKey, &mlevels[0].largestUserKey,
&mlevels[0].isLargestUserKeyRangeDelSentinel)
li.initIsSyntheticIterBoundsKey(&mlevels[0].isSyntheticIterBoundsKey)
mlevels[0].iter = li
mlevels = mlevels[1:]
}
Expand Down
14 changes: 9 additions & 5 deletions internal/base/iterator.go
Original file line number Diff line number Diff line change
Expand Up @@ -53,11 +53,15 @@ import "fmt"
// 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. 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. [TODO(peter): can we eliminate this exception?]
// 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.
//
// An iterator must be closed after use, but it is not necessary to read an
// iterator until exhaustion.
Expand Down
12 changes: 11 additions & 1 deletion iterator.go
Original file line number Diff line number Diff line change
Expand Up @@ -315,6 +315,8 @@ func (i *Iterator) SeekGE(key []byte) bool {
i.prefix = nil
if lowerBound := i.opts.GetLowerBound(); lowerBound != nil && i.cmp(key, lowerBound) < 0 {
key = lowerBound
} else if upperBound := i.opts.GetUpperBound(); upperBound != nil && i.cmp(key, upperBound) > 0 {
key = upperBound
}

i.iterKey, i.iterValue = i.iter.SeekGE(key)
Expand Down Expand Up @@ -383,6 +385,12 @@ func (i *Iterator) SeekPrefixGE(key []byte) bool {
return false
}
key = lowerBound
} else if upperBound := i.opts.GetUpperBound(); upperBound != nil && i.cmp(key, upperBound) > 0 {
if n := i.split(upperBound); !bytes.Equal(i.prefix, upperBound[:n]) {
i.err = errors.New("pebble: SeekPrefixGE supplied with key outside of upper bound")
return false
}
key = upperBound
}

i.iterKey, i.iterValue = i.iter.SeekPrefixGE(i.prefix, key)
Expand All @@ -395,8 +403,10 @@ func (i *Iterator) SeekPrefixGE(key []byte) bool {
func (i *Iterator) SeekLT(key []byte) bool {
i.err = nil // clear cached iteration error
i.prefix = nil
if upperBound := i.opts.GetUpperBound(); upperBound != nil && i.cmp(key, upperBound) >= 0 {
if upperBound := i.opts.GetUpperBound(); upperBound != nil && i.cmp(key, upperBound) > 0 {
key = upperBound
} else if lowerBound := i.opts.GetLowerBound(); lowerBound != nil && i.cmp(key, lowerBound) < 0 {
key = lowerBound
}

i.iterKey, i.iterValue = i.iter.SeekLT(key)
Expand Down
87 changes: 66 additions & 21 deletions level_iter.go
Original file line number Diff line number Diff line change
Expand Up @@ -72,8 +72,8 @@ type levelIter struct {
iter internalIterator
iterFile *fileMetadata
newIters tableNewIters
// When rangeDelIterPtr != nil, the caller requires that a range del iterator
// corresponding to the current file be placed in *rangeDelIterPtr. When this
// When rangeDelIterPtr != nil, the caller requires that *rangeDelIterPtr must
// point to a range del iterator corresponding to the current file. When this
// iterator returns nil, *rangeDelIterPtr should also be set to nil. Whenever
// a non-nil internalIterator is placed in rangeDelIterPtr, a copy is placed
// in rangeDelIterCopy. This is done for the following special case:
Expand Down Expand Up @@ -135,6 +135,12 @@ type levelIter struct {
smallestUserKey, largestUserKey *[]byte
isLargestUserKeyRangeDelSentinel *bool

// Set to true iff the key returned by this 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.
isSyntheticIterBoundsKey *bool

// bytesIterated keeps track of the number of bytes iterated during compaction.
bytesIterated *uint64

Expand Down Expand Up @@ -193,6 +199,10 @@ func (l *levelIter) initSmallestLargestUserKey(
l.isLargestUserKeyRangeDelSentinel = isLargestUserKeyRangeDelSentinel
}

func (l *levelIter) initIsSyntheticIterBoundsKey(isSyntheticIterBoundsKey *bool) {
l.isSyntheticIterBoundsKey = isSyntheticIterBoundsKey
}

func (l *levelIter) findFileGE(key []byte) *fileMetadata {
// Find the earliest file whose largest key is >= ikey.
//
Expand Down Expand Up @@ -257,6 +267,9 @@ func (l *levelIter) initTableBounds(f *fileMetadata) int {
func (l *levelIter) loadFile(file *fileMetadata, dir int) bool {
l.smallestBoundary = nil
l.largestBoundary = nil
if l.isSyntheticIterBoundsKey != nil {
*l.isSyntheticIterBoundsKey = false
}
if l.iterFile == file {
if l.err != nil {
return false
Expand Down Expand Up @@ -353,6 +366,9 @@ func (l *levelIter) verify(key *InternalKey, val []byte) (*InternalKey, []byte)

func (l *levelIter) SeekGE(key []byte) (*InternalKey, []byte) {
l.err = nil // clear cached iteration error
if l.isSyntheticIterBoundsKey != nil {
*l.isSyntheticIterBoundsKey = false
}

// NB: the top-level Iterator has already adjusted key based on
// IterOptions.LowerBound.
Expand All @@ -367,6 +383,9 @@ func (l *levelIter) SeekGE(key []byte) (*InternalKey, []byte) {

func (l *levelIter) SeekPrefixGE(prefix, key []byte) (*InternalKey, []byte) {
l.err = nil // clear cached iteration error
if l.isSyntheticIterBoundsKey != nil {
*l.isSyntheticIterBoundsKey = false
}

// NB: the top-level Iterator has already adjusted key based on
// IterOptions.LowerBound.
Expand All @@ -383,16 +402,31 @@ func (l *levelIter) SeekPrefixGE(prefix, key []byte) (*InternalKey, []byte) {
// this case the same as SeekGE where an upper-bound resides within the
// sstable and generate a synthetic boundary key.
if l.rangeDelIterPtr != nil && *l.rangeDelIterPtr != nil {
if l.tableOpts.UpperBound != nil {
l.syntheticBoundary.UserKey = l.tableOpts.UpperBound
l.syntheticBoundary.Trailer = InternalKeyRangeDeleteSentinel
l.largestBoundary = &l.syntheticBoundary
if l.isSyntheticIterBoundsKey != nil {
*l.isSyntheticIterBoundsKey = true
}
return l.verify(l.largestBoundary, nil)
}
l.syntheticBoundary = l.iterFile.Largest
l.syntheticBoundary.SetKind(InternalKeyKindRangeDelete)
l.largestBoundary = &l.syntheticBoundary
if l.isSyntheticIterBoundsKey != nil {
*l.isSyntheticIterBoundsKey = false
}
return l.verify(l.largestBoundary, nil)
}
return l.verify(l.skipEmptyFileForward())
}

func (l *levelIter) SeekLT(key []byte) (*InternalKey, []byte) {
l.err = nil // clear cached iteration error
if l.isSyntheticIterBoundsKey != nil {
*l.isSyntheticIterBoundsKey = false
}

// NB: the top-level Iterator has already adjusted key based on
// IterOptions.UpperBound.
Expand All @@ -407,6 +441,9 @@ func (l *levelIter) SeekLT(key []byte) (*InternalKey, []byte) {

func (l *levelIter) First() (*InternalKey, []byte) {
l.err = nil // clear cached iteration error
if l.isSyntheticIterBoundsKey != nil {
*l.isSyntheticIterBoundsKey = false
}

// NB: the top-level Iterator will call SeekGE if IterOptions.LowerBound is
// set.
Expand All @@ -421,6 +458,9 @@ func (l *levelIter) First() (*InternalKey, []byte) {

func (l *levelIter) Last() (*InternalKey, []byte) {
l.err = nil // clear cached iteration error
if l.isSyntheticIterBoundsKey != nil {
*l.isSyntheticIterBoundsKey = false
}

// NB: the top-level Iterator will call SeekLT if IterOptions.UpperBound is
// set.
Expand All @@ -437,6 +477,9 @@ func (l *levelIter) Next() (*InternalKey, []byte) {
if l.err != nil || l.iter == nil {
return nil, nil
}
if l.isSyntheticIterBoundsKey != nil {
*l.isSyntheticIterBoundsKey = false
}

switch {
case l.largestBoundary != nil:
Expand Down Expand Up @@ -474,6 +517,9 @@ func (l *levelIter) Prev() (*InternalKey, []byte) {
if l.err != nil || l.iter == nil {
return nil, nil
}
if l.isSyntheticIterBoundsKey != nil {
*l.isSyntheticIterBoundsKey = false
}

switch {
case l.smallestBoundary != nil:
Expand Down Expand Up @@ -534,20 +580,18 @@ func (l *levelIter) skipEmptyFileForward() (*InternalKey, []byte) {
// boundary key so that mergingIter can use the range tombstone iterator
// until the other levels have reached this boundary.
//
// It is safe to set the boundary key kind to RANGEDEL because we're
// never going to look at subsequent sstables (we've reached the upper
// bound).
// It is safe to set the boundary key to the UpperBound user key
// with the RANGEDEL sentinel since it is the smallest InternalKey
// that matches the exclusive upper bound, and does not represent
// a real key.
if l.tableOpts.UpperBound != nil {
if *l.rangeDelIterPtr != nil {
// TODO(peter): Rather than using f.Largest, can we use
// l.tableOpts.UpperBound and set the seqnum to 0? We know the upper
// bound resides within the table boundaries. Not clear if this is
// kosher with respect to the invariant that only one record for a
// given user key will have seqnum 0. See Iterator.nextUserKey for an
// optimization that requires this.
l.syntheticBoundary = l.iterFile.Largest
l.syntheticBoundary.SetKind(InternalKeyKindRangeDelete)
l.syntheticBoundary.UserKey = l.tableOpts.UpperBound
l.syntheticBoundary.Trailer = InternalKeyRangeDeleteSentinel
l.largestBoundary = &l.syntheticBoundary
if l.isSyntheticIterBoundsKey != nil {
*l.isSyntheticIterBoundsKey = true
}
return l.largestBoundary, nil
}
// Else there are no range deletions in this sstable. This
Expand Down Expand Up @@ -597,17 +641,18 @@ func (l *levelIter) skipEmptyFileBackward() (*InternalKey, []byte) {
// synthetic boundary key so that mergingIter can use the range tombstone
// iterator until the other levels have reached this boundary.
//
// It is safe to set the boundary key kind to RANGEDEL because we're
// never going to look at earlier sstables (we've reached the lower
// bound).
// It is safe to set the boundary key to the LowerBound user key
// with the RANGEDEL sentinel since it is the smallest InternalKey
// that is within the inclusive lower bound, and does not
// represent a real key.
if l.tableOpts.LowerBound != nil {
if *l.rangeDelIterPtr != nil {
// TODO(peter): Rather than using f.Smallest, can we use
// l.tableOpts.LowerBound and set the seqnum to InternalKeySeqNumMax?
// We know the lower bound resides within the table boundaries.
l.syntheticBoundary = l.iterFile.Smallest
l.syntheticBoundary.SetKind(InternalKeyKindRangeDelete)
l.syntheticBoundary.UserKey = l.tableOpts.LowerBound
l.syntheticBoundary.Trailer = InternalKeyRangeDeleteSentinel
l.smallestBoundary = &l.syntheticBoundary
if l.isSyntheticIterBoundsKey != nil {
*l.isSyntheticIterBoundsKey = true
}
return l.smallestBoundary, nil
}
// Else there are no range deletions in this sstable. This
Expand Down
15 changes: 12 additions & 3 deletions merging_iter.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ type mergingIterLevel struct {
// below.
smallestUserKey, largestUserKey []byte
isLargestUserKeyRangeDelSentinel bool
isSyntheticIterBoundsKey bool

// tombstone caches the tombstone rangeDelIter is currently pointed at. If
// tombstone.Empty() is true, there are no further tombstones within the
Expand Down Expand Up @@ -390,8 +391,9 @@ func (m *mergingIter) switchToMinHeap() {
// 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 || (l.iterKey.Kind() == base.InternalKeyKindRangeDelete &&
m.heap.cmp(l.iterKey.UserKey, m.lower) < 0) {
if l.iterKey == nil || (m.lower != nil && l.isSyntheticIterBoundsKey &&
l.iterKey.Trailer == InternalKeyRangeDeleteSentinel &&
m.heap.cmp(l.iterKey.UserKey, m.lower) <= 0) {
if m.lower != nil {
l.iterKey, l.iterValue = l.iter.SeekGE(m.lower)
} else {
Expand Down Expand Up @@ -464,7 +466,8 @@ func (m *mergingIter) switchToMaxHeap() {
// 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 || (l.iterKey.Kind() == base.InternalKeyKindRangeDelete &&
if l.iterKey == nil || (m.upper != nil && l.isSyntheticIterBoundsKey &&
l.iterKey.Trailer == InternalKeyRangeDeleteSentinel &&
m.heap.cmp(l.iterKey.UserKey, m.upper) >= 0) {
if m.upper != nil {
l.iterKey, l.iterValue = l.iter.SeekLT(m.upper)
Expand Down Expand Up @@ -610,6 +613,9 @@ func (m *mergingIter) isNextEntryDeleted(item *mergingIterItem) bool {
func (m *mergingIter) findNextEntry() (*InternalKey, []byte) {
for m.heap.len() > 0 && m.err == nil {
item := &m.heap.items[0]
if m.levels[item.index].isSyntheticIterBoundsKey {
break
}
if m.isNextEntryDeleted(item) {
continue
}
Expand Down Expand Up @@ -752,6 +758,9 @@ func (m *mergingIter) isPrevEntryDeleted(item *mergingIterItem) bool {
func (m *mergingIter) findPrevEntry() (*InternalKey, []byte) {
for m.heap.len() > 0 && m.err == nil {
item := &m.heap.items[0]
if m.levels[item.index].isSyntheticIterBoundsKey {
break
}
if m.isPrevEntryDeleted(item) {
continue
}
Expand Down
2 changes: 2 additions & 0 deletions merging_iter_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -248,6 +248,7 @@ func TestMergingIterCornerCases(t *testing.T) {
li.initRangeDel(&levelIters[i].rangeDelIter)
li.initSmallestLargestUserKey(
&levelIters[i].smallestUserKey, &levelIters[i].largestUserKey, &levelIters[i].isLargestUserKeyRangeDelSentinel)
li.initIsSyntheticIterBoundsKey(&levelIters[i].isSyntheticIterBoundsKey)
}
miter := &mergingIter{}
miter.init(nil /* opts */, cmp, levelIters...)
Expand Down Expand Up @@ -561,6 +562,7 @@ func BenchmarkMergingIterSeqSeekGEWithBounds(b *testing.B) {
l.initSmallestLargestUserKey(
&mils[level].smallestUserKey, &mils[level].largestUserKey,
&mils[level].isLargestUserKeyRangeDelSentinel)
l.initIsSyntheticIterBoundsKey(&mils[level].isSyntheticIterBoundsKey)
mils[level].iter = l
}
m := &mergingIter{}
Expand Down
4 changes: 2 additions & 2 deletions testdata/level_iter_boundaries
Original file line number Diff line number Diff line change
Expand Up @@ -344,7 +344,7 @@ next
next
----
e#5,1:z
h#72057594037927935,15:
f#72057594037927935,15:
.

file-pos
Expand All @@ -371,7 +371,7 @@ prev
prev
----
f#6,1:z
e#5,15:
f#72057594037927935,15:
.

file-pos
Expand Down
14 changes: 7 additions & 7 deletions testdata/level_iter_seek
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@ iter
set-bounds upper=d
seek-ge d
----
f/d-e#6#5,15:
d/d-e#6#72057594037927935,15:

iter
set-bounds upper=d
Expand All @@ -51,9 +51,9 @@ next
next
----
c/d-e#6#7,1:c
f#5,15:
d#72057594037927935,15:
c#7,1:c
f#5,15:
d#72057594037927935,15:
.

iter
Expand All @@ -65,7 +65,7 @@ iter
set-bounds lower=d
seek-lt d
----
c/d-e#6#7,15:
d/d-e#6#72057594037927935,15:

iter
set-bounds lower=d
Expand All @@ -76,9 +76,9 @@ prev
prev
----
f/d-e#6#5,1:f
c#7,15:
d#72057594037927935,15:
f#5,1:f
c#7,15:
d#72057594037927935,15:
.

# Verify that First() in the presence of an upper-bound pauses at the
Expand All @@ -97,7 +97,7 @@ iter
set-bounds upper=f
first
----
f#5,15:
f#72057594037927935,15:

# Verify that Last() in the presence of a lower-bound pauses at the
# table containing the lower-bound.
Expand Down
Loading

0 comments on commit 879f3bf

Please sign in to comment.