Skip to content

Commit

Permalink
colblk: change CompareFirstUserKey to IsLowerBound
Browse files Browse the repository at this point in the history
This commit changes the `CompareFirstUserKey` API to `IsLowerBound`
which can be defined when there are no keys to present.
  • Loading branch information
RaduBerinde committed Oct 2, 2024
1 parent c6de2ae commit b3369d6
Show file tree
Hide file tree
Showing 7 changed files with 72 additions and 42 deletions.
15 changes: 12 additions & 3 deletions internal/itertest/datadriven.go
Original file line number Diff line number Diff line change
Expand Up @@ -142,6 +142,9 @@ func RunInternalIterCmdWriter(
parts := strings.Fields(line)
var key *base.InternalKey
var value []byte
if o.showCommands {
fmt.Fprintf(w, "%*s: ", min(maxCmdLen, 40), line)
}
switch parts[0] {
case "seek-ge":
if len(parts) < 2 || len(parts) > 3 {
Expand Down Expand Up @@ -228,13 +231,19 @@ func RunInternalIterCmdWriter(
*o.stats = base.InternalIteratorStats{}
}
continue
case "is-lower-bound":
// This command is specific to colblk.DataBlockIter.
if len(parts) != 2 {
fmt.Fprint(w, "is-lower-bound <key>\n")
return
}
i := iter.(interface{ IsLowerBound(key []byte) bool })
fmt.Fprintf(w, "%v\n", i.IsLowerBound([]byte(parts[1])))
continue
default:
fmt.Fprintf(w, "unknown op: %s", parts[0])
return
}
if o.showCommands {
fmt.Fprintf(w, "%*s: ", min(maxCmdLen, 40), line)
}
o.fmtKV(w, key, value, iter)
if !o.withoutNewlines {
fmt.Fprintln(w)
Expand Down
12 changes: 8 additions & 4 deletions sstable/block/block.go
Original file line number Diff line number Diff line change
Expand Up @@ -158,10 +158,14 @@ type DataBlockIterator interface {
// KV returns the key-value pair at the current iterator position. The
// iterator must be Valid().
KV() *base.InternalKV
// CompareFirstUserKey compares the provided key to the first user key
// contained within the data block. It's equivalent to performing
// Compare(firstUserKey, k)
CompareFirstUserKey(k []byte) int
// IsLowerBound returns true if all keys produced by this iterator are >= the
// given key. The function is best effort; false negatives are allowed.
//
// If IsLowerBound is true then Compare(First().UserKey, k) >= 0.
//
// If the iterator produces no keys (i.e. First() is nil), IsLowerBound can
// return true for any key.
IsLowerBound(k []byte) bool
// Invalidate invalidates the block iterator, removing references to the
// block it was initialized with. The iterator may continue to be used after
// a call to Invalidate, but all positioning methods should return false.
Expand Down
10 changes: 5 additions & 5 deletions sstable/colblk/cockroach_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -198,18 +198,18 @@ func (ks *cockroachKeySeeker) Init(r *DataBlockReader) error {
// contained within the data block. It's equivalent to performing
//
// Compare(firstUserKey, k)
func (ks *cockroachKeySeeker) CompareFirstUserKey(k []byte) int {
func (ks *cockroachKeySeeker) IsLowerBound(k []byte) bool {
prefix, untypedSuffix, wallTime, logicalTime := crdbtest.DecodeTimestamp(k)
if v := crdbtest.Compare(ks.prefixes.UnsafeFirstSlice(), prefix); v != 0 {
return v
return v > 0
}
if len(untypedSuffix) > 0 {
return crdbtest.Compare(ks.untypedSuffixes.At(0), untypedSuffix)
return crdbtest.Compare(ks.untypedSuffixes.At(0), untypedSuffix) >= 0
}
if v := cmp.Compare(ks.mvccWallTimes.At(0), wallTime); v != 0 {
return v
return v > 0
}
return cmp.Compare(uint32(ks.mvccLogical.At(0)), logicalTime)
return cmp.Compare(uint32(ks.mvccLogical.At(0)), logicalTime) >= 0
}

// SeekGE is part of the KeySeeker interface.
Expand Down
28 changes: 12 additions & 16 deletions sstable/colblk/data_block.go
Original file line number Diff line number Diff line change
Expand Up @@ -100,10 +100,9 @@ func (kcmp KeyComparison) PrefixEqual() bool { return kcmp.PrefixLen == kcmp.Com
type KeySeeker interface {
// Init initializes the iterator to read from the provided DataBlockReader.
Init(b *DataBlockReader) error
// CompareFirstUserKey compares the provided key to the first user key
// contained within the data block. It's equivalent to performing
// Compare(firstUserKey, k)
CompareFirstUserKey(k []byte) int
// IsLowerBound returns true if all keys in the data block are >= the given
// key. If the data block contains no keys, returns true.
IsLowerBound(k []byte) bool
// SeekGE returns the index of the first row with a key greater than or
// equal to [key].
//
Expand Down Expand Up @@ -297,18 +296,16 @@ func (ks *defaultKeySeeker) Init(r *DataBlockReader) error {
return nil
}

// CompareFirstUserKey compares the provided key to the first user key
// contained within the data block. It's equivalent to performing
//
// Compare(firstUserKey, k)
func (ks *defaultKeySeeker) CompareFirstUserKey(k []byte) int {
// IsLowerBound is part of the KeySeeker interface.
func (ks *defaultKeySeeker) IsLowerBound(k []byte) bool {
si := ks.comparer.Split(k)
if v := ks.comparer.Compare(ks.prefixes.UnsafeFirstSlice(), k[:si]); v != 0 {
return v
return v > 0
}
return ks.comparer.Compare(ks.suffixes.At(0), k[si:])
return ks.comparer.Compare(ks.suffixes.At(0), k[si:]) >= 0
}

// SeekGE is part of the KeySeeker interface.
func (ks *defaultKeySeeker) SeekGE(key []byte, currRow int, dir int8) (row int) {
si := ks.comparer.Split(key)
row, eq := ks.prefixes.Search(key[:si])
Expand Down Expand Up @@ -897,11 +894,10 @@ func (i *DataBlockIter) Init(
return i.keySeeker.Init(r)
}

// CompareFirstUserKey compares the provided key to the first user key
// contained within the data block. It's equivalent to performing
// Compare(firstUserKey, k).
func (i *DataBlockIter) CompareFirstUserKey(k []byte) int {
return i.keySeeker.CompareFirstUserKey(k)
// IsLowerBound implements the block.DataBlockIterator interface.
func (i *DataBlockIter) IsLowerBound(k []byte) bool {
// Note: we ignore HideObsoletePoints, but false negatives are allowed.
return i.keySeeker.IsLowerBound(k)
}

// SeekGE implements the base.InternalIterator interface.
Expand Down
26 changes: 26 additions & 0 deletions sstable/colblk/testdata/data_block/transforms
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,19 @@ seek-lt c: b@2#1234,SETWITHDEL:blueberry
prev: a@10#1234,SET:apple
next: b@5#1234,SET:banana

iter verbose synthetic-seq-num=1234
is-lower-bound a
is-lower-bound a@11
is-lower-bound a@10
is-lower-bound a@9
is-lower-bound b
----
is-lower-bound a: true
is-lower-bound a@11: true
is-lower-bound a@10: true
is-lower-bound a@9: false
is-lower-bound b: false

write-block
a@10#1,SET:apple obsolete
b@5#3,SET:banana
Expand All @@ -56,6 +69,19 @@ first: b@5:banana
next: c@1:clementine
next: .

iter hide-obsolete-points
is-lower-bound a@10
is-lower-bound b
is-lower-bound b@5
is-lower-bound b@4
is-lower-bound d
----
is-lower-bound a@10: true
is-lower-bound b: false
is-lower-bound b@5: false
is-lower-bound b@4: false
is-lower-bound d: false

iter hide-obsolete-points
first
next
Expand Down
13 changes: 5 additions & 8 deletions sstable/reader_iter_single_lvl.go
Original file line number Diff line number Diff line change
Expand Up @@ -429,13 +429,10 @@ func (i *singleLevelIterator[I, PI, D, PD]) initBoundsForAlreadyLoadedBlock() {
// TODO(radu): determine automatically if we need to call First or not and
// unify this function with initBounds().
i.blockLower = i.lower
if i.blockLower != nil {
// TODO(radu): this should be <= 0
if PD(&i.data).CompareFirstUserKey(i.blockLower) > 0 {
// The lower-bound is less than the first key in the block. No need
// to check the lower-bound again for this block.
i.blockLower = nil
}
if i.blockLower != nil && PD(&i.data).IsLowerBound(i.blockLower) {
// The lower-bound is less than the first key in the block. No need
// to check the lower-bound again for this block.
i.blockLower = nil
}
i.blockUpper = i.upper
// TODO(radu): this should be >= 0 if blockUpper is inclusive.
Expand Down Expand Up @@ -1103,7 +1100,7 @@ func (i *singleLevelIterator[I, PI, D, PD]) SeekLT(

var dontSeekWithinBlock bool
if !PD(&i.data).IsDataInvalidated() && PD(&i.data).Valid() && PI(&i.index).Valid() &&
boundsCmp < 0 && PD(&i.data).CompareFirstUserKey(key) < 0 {
boundsCmp < 0 && !PD(&i.data).IsLowerBound(key) {
// Fast-path: The bounds have moved backward, and this SeekLT is
// respecting the upper bound (guaranteed by Iterator). We know that
// the iterator must already be positioned within or just outside the
Expand Down
10 changes: 4 additions & 6 deletions sstable/rowblk/rowblk_iter.go
Original file line number Diff line number Diff line change
Expand Up @@ -521,12 +521,10 @@ func (i *Iter) cacheEntry() {
i.cachedBuf = append(i.cachedBuf, i.key...)
}

// CompareFirstUserKey compares the provided key to the first user key
// contained within the data block. It's equivalent to performing
//
// Compare(firstUserKey, k)
func (i *Iter) CompareFirstUserKey(k []byte) int {
return i.cmp(i.firstUserKey, k)
// IsLowerBound implements the block.DataBlockIterator interface.
func (i *Iter) IsLowerBound(k []byte) bool {
// Note: we ignore HideObsoletePoints, but false negatives are allowed.
return i.cmp(i.firstUserKey, k) >= 0
}

// SeekGE implements internalIterator.SeekGE, as documented in the pebble
Expand Down

0 comments on commit b3369d6

Please sign in to comment.