Skip to content

Commit

Permalink
sstable: add index separator comparison methods
Browse files Browse the repository at this point in the history
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.
  • Loading branch information
jbowens committed Oct 18, 2024
1 parent 1d2e9e8 commit ff784a8
Show file tree
Hide file tree
Showing 6 changed files with 137 additions and 115 deletions.
11 changes: 11 additions & 0 deletions sstable/block/block.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
13 changes: 13 additions & 0 deletions sstable/colblk/index_block.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)]
Expand Down
2 changes: 1 addition & 1 deletion sstable/copier.go
Original file line number Diff line number Diff line change
Expand Up @@ -255,7 +255,7 @@ func intersectingIndexEntries(
}
}
}
if r.Compare(end.UserKey, top.Separator()) <= 0 {
if top.SeparatorGT(end.UserKey, true /* inclusively */) {
break
}
}
Expand Down
110 changes: 50 additions & 60 deletions sstable/reader_iter_single_lvl.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -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.
Expand Down Expand Up @@ -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 {
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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
}
Expand Down Expand Up @@ -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.
}
Expand Down Expand Up @@ -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
}
Expand Down Expand Up @@ -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.
Expand All @@ -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 {
Expand Down Expand Up @@ -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
}
Expand Down Expand Up @@ -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
}
Expand All @@ -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
}
Expand Down
Loading

0 comments on commit ff784a8

Please sign in to comment.