From ff784a8912c944a9b1ac16fc986b864b93625452 Mon Sep 17 00:00:00 2001 From: Jackson Owens Date: Wed, 16 Oct 2024 17:50:54 -0400 Subject: [PATCH] sstable: add index separator comparison methods Extend the IndexBlockIterator interface with two new methods for performing comparisons against the separator at the current iterator position. This is a precursor to decomposing separator keys into prefix and suffix in columnar blocks, as it will allow the colblk index iterator to avoid materializing the separator key unnecessarily. --- sstable/block/block.go | 11 +++ sstable/colblk/index_block.go | 13 ++++ sstable/copier.go | 2 +- sstable/reader_iter_single_lvl.go | 110 +++++++++++++--------------- sstable/reader_iter_two_lvl.go | 103 +++++++++++++------------- sstable/rowblk/rowblk_index_iter.go | 13 ++++ 6 files changed, 137 insertions(+), 115 deletions(-) diff --git a/sstable/block/block.go b/sstable/block/block.go index 3050289912..67aa288790 100644 --- a/sstable/block/block.go +++ b/sstable/block/block.go @@ -209,6 +209,17 @@ type IndexBlockIterator interface { // guaranteed to be greater than or equal to every key contained within the // referenced block(s). Separator() []byte + // SeparatorLT returns true if the separator at the iterator's current + // position is strictly less than the provided key. For some + // implementations, it may be more performant to call SeparatorLT rather + // than explicitly performing Compare(Separator(), key) < 0. + SeparatorLT(key []byte) bool + // SeparatorGT returns true if the separator at the iterator's current + // position is strictly greater than (or equal, if orEqual=true) the + // provided key. For some implementations, it may be more performant to call + // SeparatorGT rather than explicitly performing a comparison using the key + // returned by Separator. + SeparatorGT(key []byte, orEqual bool) bool // BlockHandleWithProperties decodes the block handle with any encoded // properties at the iterator's current position. BlockHandleWithProperties() (HandleWithProperties, error) diff --git a/sstable/colblk/index_block.go b/sstable/colblk/index_block.go index e7e35aeed0..3ac8cd1893 100644 --- a/sstable/colblk/index_block.go +++ b/sstable/colblk/index_block.go @@ -295,6 +295,19 @@ func (i *IndexIter) Separator() []byte { return i.applyTransforms(key) } +// SeparatorLT returns true if the separator at the iterator's current +// position is strictly less than the provided key. +func (i *IndexIter) SeparatorLT(key []byte) bool { + return i.compare(i.Separator(), key) < 0 +} + +// SeparatorGT returns true if the separator at the iterator's current position +// is strictly greater than (or equal, if orEqual=true) the provided key. +func (i *IndexIter) SeparatorGT(key []byte, inclusively bool) bool { + cmp := i.compare(i.Separator(), key) + return cmp > 0 || (cmp == 0 && inclusively) +} + func (i *IndexIter) applyTransforms(key []byte) []byte { if i.syntheticSuffix.IsSet() { key = key[:i.split(key)] diff --git a/sstable/copier.go b/sstable/copier.go index 52213e1d98..467cca110a 100644 --- a/sstable/copier.go +++ b/sstable/copier.go @@ -255,7 +255,7 @@ func intersectingIndexEntries( } } } - if r.Compare(end.UserKey, top.Separator()) <= 0 { + if top.SeparatorGT(end.UserKey, true /* inclusively */) { break } } diff --git a/sstable/reader_iter_single_lvl.go b/sstable/reader_iter_single_lvl.go index 7bda676ef6..bc9aac4446 100644 --- a/sstable/reader_iter_single_lvl.go +++ b/sstable/reader_iter_single_lvl.go @@ -409,7 +409,7 @@ func (i *singleLevelIterator[I, PI, D, PD]) initBounds() { } i.blockUpper = i.upper // TODO(radu): this should be >= 0 if blockUpper is inclusive. - if i.blockUpper != nil && i.cmp(i.blockUpper, PI(&i.index).Separator()) > 0 { + if i.blockUpper != nil && PI(&i.index).SeparatorLT(i.blockUpper) { // The upper-bound is greater than the index key which itself is greater // than or equal to every key in the block. No need to check the // upper-bound again for this block. Even if blockUpper is inclusive @@ -430,7 +430,7 @@ func (i *singleLevelIterator[I, PI, D, PD]) initBoundsForAlreadyLoadedBlock() { } i.blockUpper = i.upper // TODO(radu): this should be >= 0 if blockUpper is inclusive. - if i.blockUpper != nil && i.cmp(i.blockUpper, PI(&i.index).Separator()) > 0 { + if i.blockUpper != nil && PI(&i.index).SeparatorLT(i.blockUpper) { // The upper-bound is greater than the index key which itself is greater // than or equal to every key in the block. No need to check the // upper-bound again for this block. @@ -758,17 +758,16 @@ func (i *singleLevelIterator[I, PI, D, PD]) seekGEHelper( var dontSeekWithinBlock bool if !PD(&i.data).IsDataInvalidated() && PD(&i.data).Valid() && PI(&i.index).Valid() && - boundsCmp > 0 && i.cmp(key, PI(&i.index).Separator()) <= 0 { + boundsCmp > 0 && PI(&i.index).SeparatorGT(key, true /* orEqual */) { // Fast-path: The bounds have moved forward and this SeekGE is - // respecting the lower bound (guaranteed by Iterator). We know that - // the iterator must already be positioned within or just outside the - // previous bounds. Therefore it cannot be positioned at a block (or - // the position within that block) that is ahead of the seek position. + // respecting the lower bound (guaranteed by Iterator). We know that the + // iterator must already be positioned within or just outside the + // previous bounds. Therefore it cannot be positioned at a block (or the + // position within that block) that is ahead of the seek position. // However it can be positioned at an earlier block. This fast-path to // use Next() on the block is only applied when we are already at the - // block that the slow-path (the else-clause) would load -- this is - // the motivation for the i.cmp(key, i.index.Key().UserKey) <= 0 - // predicate. + // block that the slow-path (the else-clause) would load -- this is the + // motivation for the IsSeparatorUpperBound(key, true) predicate. i.initBoundsForAlreadyLoadedBlock() kv, done := i.trySeekGEUsingNextWithinBlock(key) if done { @@ -824,18 +823,16 @@ func (i *singleLevelIterator[I, PI, D, PD]) seekGEHelper( return nil } if result == loadBlockIrrelevant { - // Enforce the upper bound here since don't want to bother moving - // to the next block if upper bound is already exceeded. Note that - // the next block starts with keys >= ikey.UserKey since even + // Enforce the upper bound here since don't want to bother moving to + // the next block if upper bound is already exceeded. Note that the + // next block may start with keys >= index.Separator() since even // though this is the block separator, the same user key can span - // multiple blocks. If upper is exclusive we use >= below, else - // we use >. - if i.upper != nil { - cmp := i.cmp(PI(&i.index).Separator(), i.upper) - if (!i.endKeyInclusive && cmp >= 0) || cmp > 0 { - i.exhaustedBounds = +1 - return nil - } + // multiple blocks. If upper is exclusive we pass orEqual=true + // below, else we require the separator to be strictly greater than + // upper. + if i.upper != nil && PI(&i.index).SeparatorGT(i.upper, !i.endKeyInclusive) { + i.exhaustedBounds = +1 + return nil } // Want to skip to the next block. dontSeekWithinBlock = true @@ -1135,7 +1132,7 @@ func (i *singleLevelIterator[I, PI, D, PD]) SeekLT( // that the previous block starts with keys <= ikey.UserKey since // even though this is the current block's separator, the same // user key can span multiple blocks. - if i.lower != nil && i.cmp(PI(&i.index).Separator(), i.lower) < 0 { + if i.lower != nil && PI(&i.index).SeparatorLT(i.lower) { i.exhaustedBounds = -1 return nil } @@ -1216,16 +1213,14 @@ func (i *singleLevelIterator[I, PI, D, PD]) firstInternal() *base.InternalKV { } else { // result == loadBlockIrrelevant. Enforce the upper bound here since // don't want to bother moving to the next block if upper bound is - // already exceeded. Note that the next block starts with keys >= - // ikey.UserKey since even though this is the block separator, the - // same user key can span multiple blocks. If upper is exclusive we - // use >= below, else we use >. - if i.upper != nil { - cmp := i.cmp(PI(&i.index).Separator(), i.upper) - if (!i.endKeyInclusive && cmp >= 0) || cmp > 0 { - i.exhaustedBounds = +1 - return nil - } + // already exceeded. Note that the next block may start with keys >= + // index.Separator() since even though this is the block separator, the + // same user key can span multiple blocks. If upper is exclusive we pass + // orEqual=true below, else we require the separator to be strictly + // greater than upper. + if i.upper != nil && PI(&i.index).SeparatorGT(i.upper, !i.endKeyInclusive) { + i.exhaustedBounds = +1 + return nil } // Else fall through to skipForward. } @@ -1282,7 +1277,7 @@ func (i *singleLevelIterator[I, PI, D, PD]) lastInternal() *base.InternalKV { // already exceeded. Note that the previous block starts with keys <= // key.UserKey since even though this is the current block's // separator, the same user key can span multiple blocks. - if i.lower != nil && i.cmp(PI(&i.index).Separator(), i.lower) < 0 { + if i.lower != nil && PI(&i.index).SeparatorLT(i.lower) { i.exhaustedBounds = -1 return nil } @@ -1354,7 +1349,7 @@ func (i *singleLevelIterator[I, PI, D, PD]) NextPrefix(succKey []byte) *base.Int PD(&i.data).Invalidate() return nil } - if i.cmp(succKey, PI(&i.index).Separator()) > 0 { + if PI(&i.index).SeparatorLT(succKey) { // Not in the next data block, so seek the index. if !PI(&i.index).SeekGE(succKey) { // The target key is greater than any key in the index block. @@ -1369,18 +1364,15 @@ func (i *singleLevelIterator[I, PI, D, PD]) NextPrefix(succKey []byte) *base.Int return nil } if result == loadBlockIrrelevant { - // Enforce the upper bound here since don't want to bother moving - // to the next block if upper bound is already exceeded. Note that - // the next block starts with keys >= ikey.UserKey since even - // though this is the block separator, the same user key can span - // multiple blocks. If upper is exclusive we use >= below, else we use - // >. - if i.upper != nil { - cmp := i.cmp(PI(&i.index).Separator(), i.upper) - if (!i.endKeyInclusive && cmp >= 0) || cmp > 0 { - i.exhaustedBounds = +1 - return nil - } + // Enforce the upper bound here since don't want to bother moving to the + // next block if upper bound is already exceeded. Note that the next + // block may start with keys >= index.Separator() since even though this + // is the block separator, the same user key can span multiple blocks. + // If upper is exclusive we pass orEqual=true below, else we require + // the separator to be strictly greater than upper. + if i.upper != nil && PI(&i.index).SeparatorGT(i.upper, !i.endKeyInclusive) { + i.exhaustedBounds = +1 + return nil } } else if kv := PD(&i.data).SeekGE(succKey, base.SeekGEFlagsNone); kv != nil { if i.blockUpper != nil { @@ -1436,18 +1428,16 @@ func (i *singleLevelIterator[I, PI, D, PD]) skipForward() *base.InternalKV { // being exhausted, and must be due to an error. panic("loadDataBlock should not have failed with no error") } - // result == loadBlockIrrelevant. Enforce the upper bound here - // since don't want to bother moving to the next block if upper - // bound is already exceeded. Note that the next block starts with - // keys >= key.UserKey since even though this is the block - // separator, the same user key can span multiple blocks. If upper - // is exclusive we use >= below, else we use >. - if i.upper != nil { - cmp := i.cmp(PI(&i.index).Separator(), i.upper) - if (!i.endKeyInclusive && cmp >= 0) || cmp > 0 { - i.exhaustedBounds = +1 - return nil - } + // result == loadBlockIrrelevant. Enforce the upper bound here since + // don't want to bother moving to the next block if upper bound is + // already exceeded. Note that the next block may start with keys >= + // index.Separator() since even though this is the block separator, + // the same user key can span multiple blocks. If upper is exclusive + // we pass orEqual=true below, else we require the separator to be + // strictly greater than upper. + if i.upper != nil && PI(&i.index).SeparatorGT(i.upper, !i.endKeyInclusive) { + i.exhaustedBounds = +1 + return nil } continue } @@ -1521,7 +1511,7 @@ func (i *singleLevelIterator[I, PI, D, PD]) skipBackward() *base.InternalKV { // bound is already exceeded. Note that the previous block starts with // keys <= key.UserKey since even though this is the current block's // separator, the same user key can span multiple blocks. - if i.lower != nil && i.cmp(PI(&i.index).Separator(), i.lower) < 0 { + if i.lower != nil && PI(&i.index).SeparatorLT(i.lower) { i.exhaustedBounds = -1 return nil } @@ -1533,7 +1523,7 @@ func (i *singleLevelIterator[I, PI, D, PD]) skipBackward() *base.InternalKV { // safe to assume that there are no keys if we keep skipping backwards. // Check the previous block, but check the lower bound before doing // that. - if i.lower != nil && i.cmp(PI(&i.index).Separator(), i.lower) < 0 { + if i.lower != nil && PI(&i.index).SeparatorLT(i.lower) { i.exhaustedBounds = -1 return nil } diff --git a/sstable/reader_iter_two_lvl.go b/sstable/reader_iter_two_lvl.go index bc7068fdd4..e9e9a97582 100644 --- a/sstable/reader_iter_two_lvl.go +++ b/sstable/reader_iter_two_lvl.go @@ -325,7 +325,7 @@ func (i *twoLevelIterator[I, PI, D, PD]) SeekGE( var dontSeekWithinSingleLevelIter bool if PI(&i.topLevelIndex).IsDataInvalidated() || !PI(&i.topLevelIndex).Valid() || PI(&i.secondLevel.index).IsDataInvalidated() || err != nil || - (i.secondLevel.boundsCmp <= 0 && !flags.TrySeekUsingNext()) || i.secondLevel.cmp(key, PI(&i.topLevelIndex).Separator()) > 0 { + (i.secondLevel.boundsCmp <= 0 && !flags.TrySeekUsingNext()) || PI(&i.topLevelIndex).SeparatorLT(key) { // Slow-path: need to position the topLevelIndex. // The previous exhausted state of singleLevelIterator is no longer @@ -344,17 +344,16 @@ func (i *twoLevelIterator[I, PI, D, PD]) SeekGE( return nil } if result == loadBlockIrrelevant { - // Enforce the upper bound here since don't want to bother moving - // to the next entry in the top level index if upper bound is - // already exceeded. Note that the next entry starts with keys >= - // ikey.InternalKey.UserKey since even though this is the block separator, the - // same user key can span multiple index blocks. If upper is - // exclusive we use >= below, else we use >. - if i.secondLevel.upper != nil { - cmp := i.secondLevel.cmp(PI(&i.topLevelIndex).Separator(), i.secondLevel.upper) - if (!i.secondLevel.endKeyInclusive && cmp >= 0) || cmp > 0 { - i.secondLevel.exhaustedBounds = +1 - } + // Enforce the upper bound here since don't want to bother moving to + // the next entry in the top level index if upper bound is already + // exceeded. Note that the next entry starts with keys >= + // topLevelIndex.Separator() since even though this is the block + // separator, the same user key can span multiple index blocks. If + // upper is exclusive we pass orEqual=true below, else we require + // the separator to be strictly greater than upper. + if i.secondLevel.upper != nil && PI(&i.topLevelIndex).SeparatorGT( + i.secondLevel.upper, !i.secondLevel.endKeyInclusive) { + i.secondLevel.exhaustedBounds = +1 } // Fall through to skipForward. dontSeekWithinSingleLevelIter = true @@ -493,7 +492,7 @@ func (i *twoLevelIterator[I, PI, D, PD]) SeekPrefixGE( var dontSeekWithinSingleLevelIter bool if PI(&i.topLevelIndex).IsDataInvalidated() || !PI(&i.topLevelIndex).Valid() || PI(&i.secondLevel.index).IsDataInvalidated() || err != nil || - (i.secondLevel.boundsCmp <= 0 && !flags.TrySeekUsingNext()) || i.secondLevel.cmp(key, PI(&i.topLevelIndex).Separator()) > 0 { + (i.secondLevel.boundsCmp <= 0 && !flags.TrySeekUsingNext()) || PI(&i.topLevelIndex).SeparatorLT(key) { // Slow-path: need to position the topLevelIndex. // The previous exhausted state of singleLevelIterator is no longer @@ -512,17 +511,16 @@ func (i *twoLevelIterator[I, PI, D, PD]) SeekPrefixGE( return nil } if result == loadBlockIrrelevant { - // Enforce the upper bound here since don't want to bother moving - // to the next entry in the top level index if upper bound is - // already exceeded. Note that the next entry starts with keys >= - // ikey.InternalKey.UserKey since even though this is the block separator, the - // same user key can span multiple index blocks. If upper is - // exclusive we use >= below, else we use >. - if i.secondLevel.upper != nil { - cmp := i.secondLevel.cmp(PI(&i.topLevelIndex).Separator(), i.secondLevel.upper) - if (!i.secondLevel.endKeyInclusive && cmp >= 0) || cmp > 0 { - i.secondLevel.exhaustedBounds = +1 - } + // Enforce the upper bound here since don't want to bother moving to + // the next entry in the top level index if upper bound is already + // exceeded. Note that the next entry starts with keys >= + // topLevelIndex.Separator() since even though this is the block + // separator, the same user key can span multiple index blocks. If + // upper is exclusive we pass orEqual=true below, else we require + // the separator to be strictly greater than upper. + if i.secondLevel.upper != nil && PI(&i.topLevelIndex).SeparatorGT( + i.secondLevel.upper, !i.secondLevel.endKeyInclusive) { + i.secondLevel.exhaustedBounds = +1 } // Fall through to skipForward. dontSeekWithinSingleLevelIter = true @@ -719,7 +717,7 @@ func (i *twoLevelIterator[I, PI, D, PD]) SeekLT( // exceeded. Note that the previous entry starts with keys <= // ikey.InternalKey.UserKey since even though this is the current block's // separator, the same user key can span multiple index blocks. - if i.secondLevel.lower != nil && i.secondLevel.cmp(PI(&i.topLevelIndex).Separator(), i.secondLevel.lower) < 0 { + if i.secondLevel.lower != nil && PI(&i.topLevelIndex).SeparatorLT(i.secondLevel.lower) { i.secondLevel.exhaustedBounds = -1 } } @@ -757,16 +755,15 @@ func (i *twoLevelIterator[I, PI, D, PD]) First() *base.InternalKV { // Else fall through to skipForward. } else { // result == loadBlockIrrelevant. Enforce the upper bound here since - // don't want to bother moving to the next entry in the top level - // index if upper bound is already exceeded. Note that the next entry - // starts with keys >= ikv.InternalKey.UserKey since even though this is the + // don't want to bother moving to the next entry in the top level index + // if upper bound is already exceeded. Note that the next entry starts + // with keys >= topLevelIndex.Separator() since even though this is the // block separator, the same user key can span multiple index blocks. - // If upper is exclusive we use >= below, else we use >. - if i.secondLevel.upper != nil { - cmp := i.secondLevel.cmp(PI(&i.topLevelIndex).Separator(), i.secondLevel.upper) - if (!i.secondLevel.endKeyInclusive && cmp >= 0) || cmp > 0 { - i.secondLevel.exhaustedBounds = +1 - } + // If upper is exclusive we pass orEqual=true below, else we require the + // separator to be strictly greater than upper. + if i.secondLevel.upper != nil && PI(&i.topLevelIndex).SeparatorGT( + i.secondLevel.upper, !i.secondLevel.endKeyInclusive) { + i.secondLevel.exhaustedBounds = +1 } } // NB: skipForward checks whether exhaustedBounds is already +1. @@ -812,7 +809,7 @@ func (i *twoLevelIterator[I, PI, D, PD]) Last() *base.InternalKV { // entry starts with keys <= ikv.InternalKey.UserKey since even though // this is the current block's separator, the same user key can span // multiple index blocks. - if i.secondLevel.lower != nil && i.secondLevel.cmp(PI(&i.topLevelIndex).Separator(), i.secondLevel.lower) < 0 { + if i.secondLevel.lower != nil && PI(&i.topLevelIndex).SeparatorLT(i.secondLevel.lower) { i.secondLevel.exhaustedBounds = -1 } } @@ -872,15 +869,14 @@ func (i *twoLevelIterator[I, PI, D, PD]) NextPrefix(succKey []byte) *base.Intern if result == loadBlockIrrelevant { // Enforce the upper bound here since don't want to bother moving to the // next entry in the top level index if upper bound is already exceeded. - // Note that the next entry starts with keys >= ikv.InternalKey.UserKey - // since even though this is the block separator, the same user key can - // span multiple index blocks. If upper is exclusive we use >= below, - // else we use >. - if i.secondLevel.upper != nil { - cmp := i.secondLevel.cmp(PI(&i.topLevelIndex).Separator(), i.secondLevel.upper) - if (!i.secondLevel.endKeyInclusive && cmp >= 0) || cmp > 0 { - i.secondLevel.exhaustedBounds = +1 - } + // Note that the next entry starts with keys >= + // topLevelIndex.Separator() since even though this is the block + // separator, the same user key can span multiple index blocks. If upper + // is exclusive we pass orEqual=true below, else we require the + // separator to be strictly greater than upper. + if i.secondLevel.upper != nil && PI(&i.topLevelIndex).SeparatorGT( + i.secondLevel.upper, !i.secondLevel.endKeyInclusive) { + i.secondLevel.exhaustedBounds = +1 } } else if kv := i.secondLevel.SeekGE(succKey, base.SeekGEFlagsNone); kv != nil { return i.secondLevel.maybeVerifyKey(kv) @@ -938,7 +934,7 @@ func (i *twoLevelIterator[I, PI, D, PD]) skipForward() *base.InternalKV { // guarantees wrt an iterator lower bound when we iterate forward. But we // must never return keys that are not inside the virtual table. useSeek := i.secondLevel.vState != nil && (!PI(&i.topLevelIndex).Valid() || - i.secondLevel.cmp(PI(&i.topLevelIndex).Separator(), i.secondLevel.vState.lower.UserKey) < 0) + PI(&i.topLevelIndex).SeparatorLT(i.secondLevel.vState.lower.UserKey)) i.secondLevel.exhaustedBounds = 0 if !PI(&i.topLevelIndex).Next() { @@ -968,14 +964,13 @@ func (i *twoLevelIterator[I, PI, D, PD]) skipForward() *base.InternalKV { // index if upper bound is already exceeded. Note that the next // entry starts with keys >= i.topLevelIndex.Separator() since even // though this is the block separator, the same user key can span - // multiple index blocks. If upper is exclusive we use >= below, - // else we use >. - if i.secondLevel.upper != nil { - cmp := i.secondLevel.cmp(PI(&i.topLevelIndex).Separator(), i.secondLevel.upper) - if (!i.secondLevel.endKeyInclusive && cmp >= 0) || cmp > 0 { - i.secondLevel.exhaustedBounds = +1 - // Next iteration will return. - } + // multiple index blocks. If upper is exclusive we pass orEqual=true + // below, else we require the separator to be strictly greater than + // upper. + if i.secondLevel.upper != nil && PI(&i.topLevelIndex).SeparatorGT( + i.secondLevel.upper, !i.secondLevel.endKeyInclusive) { + i.secondLevel.exhaustedBounds = +1 + // Next iteration will return. } } } @@ -1011,7 +1006,7 @@ func (i *twoLevelIterator[I, PI, D, PD]) skipBackward() *base.InternalKV { // previous entry starts with keys <= i.topLevelIndex.Separator() since // even though this is the current block's separator, the same user // key can span multiple index blocks. - if i.secondLevel.lower != nil && i.secondLevel.cmp(PI(&i.topLevelIndex).Separator(), i.secondLevel.lower) < 0 { + if i.secondLevel.lower != nil && PI(&i.topLevelIndex).SeparatorLT(i.secondLevel.lower) { i.secondLevel.exhaustedBounds = -1 // Next iteration will return. } diff --git a/sstable/rowblk/rowblk_index_iter.go b/sstable/rowblk/rowblk_index_iter.go index 170faa92eb..87e59736bb 100644 --- a/sstable/rowblk/rowblk_index_iter.go +++ b/sstable/rowblk/rowblk_index_iter.go @@ -71,6 +71,19 @@ func (i *IndexIter) Separator() []byte { return i.iter.ikv.K.UserKey } +// SeparatorLT returns true if the separator at the iterator's current +// position is strictly less than the provided key. +func (i *IndexIter) SeparatorLT(key []byte) bool { + return i.iter.cmp(i.iter.ikv.K.UserKey, key) < 0 +} + +// SeparatorGT returns true if the separator at the iterator's current position +// is strictly greater than (or equal, if orEqual=true) the provided key. +func (i *IndexIter) SeparatorGT(key []byte, inclusively bool) bool { + cmp := i.iter.cmp(i.iter.ikv.K.UserKey, key) + return cmp > 0 || (cmp == 0 && inclusively) +} + // BlockHandleWithProperties decodes the block handle with any encoded // properties at the iterator's current position. func (i *IndexIter) BlockHandleWithProperties() (block.HandleWithProperties, error) {