Skip to content

Commit

Permalink
compact: pull out code for compacting spans
Browse files Browse the repository at this point in the history
This commit pulls out the pieces of code which compact range del and
range key spans in their own structures (which we can now test
independently).
  • Loading branch information
RaduBerinde committed May 6, 2024
1 parent 94cfeb2 commit 1210811
Show file tree
Hide file tree
Showing 7 changed files with 336 additions and 151 deletions.
105 changes: 21 additions & 84 deletions internal/compact/iterator.go
Original file line number Diff line number Diff line change
Expand Up @@ -165,10 +165,10 @@ type Iter struct {
// keys.
iter base.InternalIterator

delElider pointTombstoneElider
rangeDelElider rangeTombstoneElider
rangeKeyElider rangeTombstoneElider
err error
delElider pointTombstoneElider
rangeDelCompactor RangeDelSpanCompactor
rangeKeyCompactor RangeKeySpanCompactor
err error
// `key.UserKey` is set to `keyBuf` caused by saving `i.iterKV.UserKey`
// and `key.Trailer` is set to `i.iterKV.Trailer`. This is the
// case on return from all public methods -- these methods return `key`.
Expand Down Expand Up @@ -324,8 +324,8 @@ func NewIter(

i.frontiers.Init(i.cmp)
i.delElider.Init(i.cmp, cfg.TombstoneElision)
i.rangeDelElider.Init(i.cmp, cfg.TombstoneElision)
i.rangeKeyElider.Init(i.cmp, cfg.RangeKeyElision)
i.rangeDelCompactor = MakeRangeDelSpanCompactor(i.cmp, i.cfg.Comparer.Equal, cfg.Snapshots, cfg.TombstoneElision)
i.rangeKeyCompactor = MakeRangeKeySpanCompactor(i.cmp, i.cfg.Comparer.Equal, cfg.Snapshots, cfg.RangeKeyElision)
return i
}

Expand Down Expand Up @@ -1367,34 +1367,14 @@ func (i *Iter) TombstonesUpTo(key []byte) []keyspan.Span {
toReturn, i.tombstones = i.splitSpans(i.tombstones, key)

result := toReturn[:0]
var tmp keyspan.Span
for _, span := range toReturn {
// Apply the snapshot stripe rules, keeping only the latest tombstone for
// each snapshot stripe.
currentIdx := -1
keys := make([]keyspan.Key, 0, min(len(span.Keys), len(i.cfg.Snapshots)+1))
for _, k := range span.Keys {
idx := i.cfg.Snapshots.Index(k.SeqNum())
if currentIdx == idx {
continue
}
if idx == 0 && i.rangeDelElider.ShouldElide(span.Start, span.End) {
// This is the last snapshot stripe and the range tombstone
// can be elided.
break
}

keys = append(keys, k)
if idx == 0 {
// This is the last snapshot stripe.
break
}
currentIdx = idx
}
if len(keys) > 0 {
i.rangeDelCompactor.Compact(&span, &tmp)
if !tmp.Empty() {
result = append(result, keyspan.Span{
Start: i.cloneKey(span.Start),
End: i.cloneKey(span.End),
Keys: keys,
Start: i.cloneKey(tmp.Start),
End: i.cloneKey(tmp.End),
Keys: slices.Clone(tmp.Keys),
})
}
}
Expand Down Expand Up @@ -1430,58 +1410,15 @@ func (i *Iter) RangeKeysUpTo(key []byte) []keyspan.Span {
toReturn, i.rangeKeys = i.splitSpans(i.rangeKeys, key)

result := toReturn[:0]
for _, s := range toReturn {
elideInLastStripe := func(keys []keyspan.Key) []keyspan.Key {
// Unsets and deletes in the last snapshot stripe can be elided.
k := 0
for j := range keys {
if (keys[j].Kind() == base.InternalKeyKindRangeKeyUnset || keys[j].Kind() == base.InternalKeyKindRangeKeyDelete) &&
i.rangeKeyElider.ShouldElide(s.Start, s.End) {
continue
}
keys[k] = keys[j]
k++
}
keys = keys[:k]
return keys
}

var dst keyspan.Span
// snapshots are in ascending order, while s.keys are in descending seqnum
// order. Partition s.keys by snapshot stripes, and call rangekey.Coalesce
// on each partition.
dst.Start = s.Start
dst.End = s.End
dst.Keys = dst.Keys[:0]
x, y := len(i.cfg.Snapshots)-1, 0
usedLen := 0
for x >= 0 {
start := y
for y < len(s.Keys) && !base.Visible(s.Keys[y].SeqNum(), i.cfg.Snapshots[x], base.InternalKeySeqNumMax) {
// Include y in current partition.
y++
}
if y > start {
keysDst := dst.Keys[usedLen:cap(dst.Keys)]
rangekey.Coalesce(i.cmp, i.cfg.Comparer.Equal, s.Keys[start:y], &keysDst)
if y == len(s.Keys) {
// This is the last snapshot stripe. Unsets and deletes can be elided.
keysDst = elideInLastStripe(keysDst)
}
usedLen += len(keysDst)
dst.Keys = append(dst.Keys, keysDst...)
}
x--
}
if y < len(s.Keys) {
keysDst := dst.Keys[usedLen:cap(dst.Keys)]
rangekey.Coalesce(i.cmp, i.cfg.Comparer.Equal, s.Keys[y:], &keysDst)
keysDst = elideInLastStripe(keysDst)
usedLen += len(keysDst)
dst.Keys = append(dst.Keys, keysDst...)
}
if len(dst.Keys) > 0 {
result = append(result, dst)
var tmp keyspan.Span
for _, span := range toReturn {
i.rangeKeyCompactor.Compact(&span, &tmp)
if !tmp.Empty() {
result = append(result, keyspan.Span{
Start: i.cloneKey(tmp.Start),
End: i.cloneKey(tmp.End),
Keys: slices.Clone(tmp.Keys),
})
}
}
return result
Expand Down
48 changes: 0 additions & 48 deletions internal/compact/iterator_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -290,54 +290,6 @@ func TestCompactionIter(t *testing.T) {
runTest(t, "testdata/iter_delete_sized")
}

// TestIterRangeKeys tests the range key coalescing and striping logic.
func TestIterRangeKeys(t *testing.T) {
datadriven.RunTest(t, "testdata/iter_range_keys", func(t *testing.T, td *datadriven.TestData) string {
switch td.Cmd {
case "transform":
var snapshots []uint64
var keyRanges []base.UserKeyBounds
td.MaybeScanArgs(t, "snapshots", &snapshots)
if arg, ok := td.Arg("in-use-key-ranges"); ok {
for _, keyRange := range arg.Vals {
parts := strings.SplitN(keyRange, "-", 2)
start := []byte(strings.TrimSpace(parts[0]))
end := []byte(strings.TrimSpace(parts[1]))
keyRanges = append(keyRanges, base.UserKeyBoundsInclusive(start, end))
}
}
span := keyspan.ParseSpan(td.Input)
for i := range span.Keys {
if i > 0 {
if span.Keys[i-1].Trailer < span.Keys[i].Trailer {
return "span keys not sorted"
}
}
}

cfg := IterConfig{
Comparer: base.DefaultComparer,
Snapshots: snapshots,
AllowZeroSeqNum: false,
TombstoneElision: NoTombstoneElision(),
RangeKeyElision: ElideTombstonesOutsideOf(keyRanges),
}
pointIter, rangeDelIter, rangeKeyIter := makeInputIters(nil, nil, nil)
iter := NewIter(cfg, pointIter, rangeDelIter, rangeKeyIter)
iter.AddRangeKeySpan(&span)

outSpans := iter.RangeKeysUpTo(nil)
var b strings.Builder
for i := range outSpans {
fmt.Fprintf(&b, "%s\n", outSpans[i].String())
}
return b.String()
default:
return fmt.Sprintf("unknown command: %s", td.Cmd)
}
})
}

// makeInputIters creates the iterators necessthat can be used to create a compaction
// Iter.
func makeInputIters(
Expand Down
171 changes: 171 additions & 0 deletions internal/compact/spans.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,171 @@
// Copyright 2024 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 compact

import (
"github.com/cockroachdb/pebble/internal/base"
"github.com/cockroachdb/pebble/internal/invariants"
"github.com/cockroachdb/pebble/internal/keyspan"
"github.com/cockroachdb/pebble/internal/rangekey"
)

// RangeDelSpanCompactor coalesces RANGEDELs within snapshot stripes and elides
// RANGEDELs in the last stripe if possible.
type RangeDelSpanCompactor struct {
cmp base.Compare
equal base.Equal
snapshots Snapshots
elider rangeTombstoneElider
}

// MakeRangeDelSpanCompactor creates a new compactor for RANGEDEL spans.
func MakeRangeDelSpanCompactor(
cmp base.Compare, equal base.Equal, snapshots Snapshots, elision TombstoneElision,
) RangeDelSpanCompactor {
c := RangeDelSpanCompactor{
cmp: cmp,
equal: equal,
snapshots: snapshots,
}
c.elider.Init(cmp, elision)
return c
}

// Compact compacts the given range del span and stores the results in the
// given output span, reusing its slices.
//
// Compaction of a span entails coalescing RANGEDELs keys within snapshot
// stripes, and eliding RANGEDELs in the last stripe if possible.
//
// It is possible for the output span to be empty after the call (if all
// RANGEDELs in the span are elided).
//
// The spans that are passed to Compact calls must be ordered and
// non-overlapping.
func (c *RangeDelSpanCompactor) Compact(span, output *keyspan.Span) {
if invariants.Enabled && span.KeysOrder != keyspan.ByTrailerDesc {
panic("pebble: span's keys unexpectedly not in trailer order")
}
output.Reset()
// Apply the snapshot stripe rules, keeping only the latest tombstone for
// each snapshot stripe.
currentIdx := -1
for _, k := range span.Keys {
idx := c.snapshots.Index(k.SeqNum())
if currentIdx == idx {
continue
}
if idx == 0 && c.elider.ShouldElide(span.Start, span.End) {
// This is the last snapshot stripe and the range tombstone
// can be elided.
break
}

output.Keys = append(output.Keys, k)
if idx == 0 {
// This is the last snapshot stripe.
break
}
currentIdx = idx
}
if len(output.Keys) > 0 {
output.Start = append(output.Start, span.Start...)
output.End = append(output.End, span.End...)
output.KeysOrder = span.KeysOrder
}
}

// RangeKeySpanCompactor coalesces range keys within snapshot stripes and elides
// RangeKeyDelete and RangeKeyUnsets when possible. It is used as a container
// for at most one "compacted" span.
type RangeKeySpanCompactor struct {
cmp base.Compare
equal base.Equal
snapshots Snapshots
elider rangeTombstoneElider
}

// MakeRangeKeySpanCompactor creates a new compactor for range key spans.
func MakeRangeKeySpanCompactor(
cmp base.Compare, equal base.Equal, snapshots Snapshots, elision TombstoneElision,
) RangeKeySpanCompactor {
c := RangeKeySpanCompactor{
cmp: cmp,
equal: equal,
snapshots: snapshots,
}
c.elider.Init(cmp, elision)
return c
}

// Compact compacts the given range key span and stores the results in the
// given output span, reusing its slices.
//
// Compaction of a span entails coalescing range keys within snapshot
// stripes, and eliding RangeKeyUnset/RangeKeyDelete in the last stripe if
// possible.
//
// It is possible for the output span to be empty after the call (if all range
// keys in the span are elided).
//
// The spans that are passed to Compact calls must be ordered and
// non-overlapping.
func (c *RangeKeySpanCompactor) Compact(span, output *keyspan.Span) {
if invariants.Enabled && span.KeysOrder != keyspan.ByTrailerDesc {
panic("pebble: span's keys unexpectedly not in trailer order")
}
// snapshots are in ascending order, while s.keys are in descending seqnum
// order. Partition s.keys by snapshot stripes, and call rangekey.Coalesce
// on each partition.
output.Reset()
x, y := len(c.snapshots)-1, 0
usedLen := 0
for x >= 0 {
start := y
for y < len(span.Keys) && !base.Visible(span.Keys[y].SeqNum(), c.snapshots[x], base.InternalKeySeqNumMax) {
// Include y in current partition.
y++
}
if y > start {
keysDst := output.Keys[usedLen:cap(output.Keys)]
rangekey.Coalesce(c.cmp, c.equal, span.Keys[start:y], &keysDst)
if y == len(span.Keys) {
// This is the last snapshot stripe. Unsets and deletes can be elided.
keysDst = c.elideInLastStripe(span.Start, span.End, keysDst)
}
usedLen += len(keysDst)
output.Keys = append(output.Keys, keysDst...)
}
x--
}
if y < len(span.Keys) {
keysDst := output.Keys[usedLen:cap(output.Keys)]
rangekey.Coalesce(c.cmp, c.equal, span.Keys[y:], &keysDst)
keysDst = c.elideInLastStripe(span.Start, span.End, keysDst)
usedLen += len(keysDst)
output.Keys = append(output.Keys, keysDst...)
}
if len(output.Keys) > 0 {
output.Start = append(output.Start, span.Start...)
output.End = append(output.End, span.End...)
output.KeysOrder = span.KeysOrder
}
}

func (c *RangeKeySpanCompactor) elideInLastStripe(
start, end []byte, keys []keyspan.Key,
) []keyspan.Key {
// Unsets and deletes in the last snapshot stripe can be elided.
k := 0
for j := range keys {
if (keys[j].Kind() == base.InternalKeyKindRangeKeyUnset || keys[j].Kind() == base.InternalKeyKindRangeKeyDelete) &&
c.elider.ShouldElide(start, end) {
continue
}
keys[k] = keys[j]
k++
}
return keys[:k]
}
Loading

0 comments on commit 1210811

Please sign in to comment.