diff --git a/internal/compact/iterator.go b/internal/compact/iterator.go index 074326e950..034c95e7af 100644 --- a/internal/compact/iterator.go +++ b/internal/compact/iterator.go @@ -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`. @@ -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 } @@ -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), }) } } @@ -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 diff --git a/internal/compact/iterator_test.go b/internal/compact/iterator_test.go index e05c012fbc..04450de341 100644 --- a/internal/compact/iterator_test.go +++ b/internal/compact/iterator_test.go @@ -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( diff --git a/internal/compact/spans.go b/internal/compact/spans.go new file mode 100644 index 0000000000..18b8ed6725 --- /dev/null +++ b/internal/compact/spans.go @@ -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] +} diff --git a/internal/compact/spans_test.go b/internal/compact/spans_test.go new file mode 100644 index 0000000000..fa02d813fb --- /dev/null +++ b/internal/compact/spans_test.go @@ -0,0 +1,92 @@ +// 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 ( + "fmt" + "strings" + "testing" + + "github.com/cockroachdb/datadriven" + "github.com/cockroachdb/pebble/internal/base" + "github.com/cockroachdb/pebble/internal/keyspan" +) + +// TestRangeDelSpanCompactor tests the range key coalescing and striping logic. +func TestRangeDelSpanCompactor(t *testing.T) { + var c RangeDelSpanCompactor + var output keyspan.Span + datadriven.RunTest(t, "testdata/range_del_compactor", func(t *testing.T, td *datadriven.TestData) string { + switch td.Cmd { + case "compact": + var snapshots []uint64 + td.MaybeScanArgs(t, "snapshots", &snapshots) + keyRanges := maybeParseInUseKeyRanges(td) + span := keyspan.ParseSpan(td.Input) + + c = MakeRangeDelSpanCompactor( + base.DefaultComparer.Compare, + base.DefaultComparer.Equal, + snapshots, + ElideTombstonesOutsideOf(keyRanges), + ) + + c.Compact(&span, &output) + if output.Empty() { + return "." + } + return output.String() + + default: + return fmt.Sprintf("unknown command: %s", td.Cmd) + } + }) +} + +// TestRangeKeySpanCompactor tests the range key coalescing and striping logic. +func TestRangeKeySpanCompactor(t *testing.T) { + var c RangeKeySpanCompactor + var output keyspan.Span + datadriven.RunTest(t, "testdata/range_key_compactor", func(t *testing.T, td *datadriven.TestData) string { + switch td.Cmd { + case "compact": + var snapshots []uint64 + td.MaybeScanArgs(t, "snapshots", &snapshots) + keyRanges := maybeParseInUseKeyRanges(td) + span := keyspan.ParseSpan(td.Input) + + c = MakeRangeKeySpanCompactor( + base.DefaultComparer.Compare, + base.DefaultComparer.Equal, + snapshots, + ElideTombstonesOutsideOf(keyRanges), + ) + + c.Compact(&span, &output) + if output.Empty() { + return "." + } + return output.String() + + default: + return fmt.Sprintf("unknown command: %s", td.Cmd) + } + }) +} + +func maybeParseInUseKeyRanges(td *datadriven.TestData) []base.UserKeyBounds { + arg, ok := td.Arg("in-use-key-ranges") + if !ok { + return nil + } + keyRanges := make([]base.UserKeyBounds, len(arg.Vals)) + for i, keyRange := range arg.Vals { + parts := strings.SplitN(keyRange, "-", 2) + start := []byte(strings.TrimSpace(parts[0])) + end := []byte(strings.TrimSpace(parts[1])) + keyRanges[i] = base.UserKeyBoundsInclusive(start, end) + } + return keyRanges +} diff --git a/internal/compact/testdata/range_del_compactor b/internal/compact/testdata/range_del_compactor new file mode 100644 index 0000000000..00680c3863 --- /dev/null +++ b/internal/compact/testdata/range_del_compactor @@ -0,0 +1,23 @@ +# We retain one key per snapshot plus the top one. +compact snapshots=(5,10,15) in-use-key-ranges=(a-z) +a-c:{(#18,RANGEDEL) (#17,RANGEDEL) (#15,RANGEDEL) (#13,RANGEDEL) (#11,RANGEDEL) (#9,RANGEDEL) (#7,RANGEDEL) (#4,RANGEDEL) (#1,RANGEDEL)} +---- +a-c:{(#18,RANGEDEL) (#13,RANGEDEL) (#9,RANGEDEL) (#4,RANGEDEL)} + +# Same as before, but elide in the last stripe. +compact snapshots=(5,10,15) +a-c:{(#18,RANGEDEL) (#17,RANGEDEL) (#15,RANGEDEL) (#13,RANGEDEL) (#11,RANGEDEL) (#9,RANGEDEL) (#7,RANGEDEL) (#4,RANGEDEL) (#1,RANGEDEL)} +---- +a-c:{(#18,RANGEDEL) (#13,RANGEDEL) (#9,RANGEDEL)} + +# No snapshots = retain just the top one. +compact in-use-key-ranges=(a-z) +a-c:{(#4,RANGEDEL) (#2,RANGEDEL)} +---- +a-c:{(#4,RANGEDEL)} + +# No snapshots and elision: retain nothing. +compact +a-c:{(#4,RANGEDEL) (#2,RANGEDEL)} +---- +. diff --git a/internal/compact/testdata/iter_range_keys b/internal/compact/testdata/range_key_compactor similarity index 85% rename from internal/compact/testdata/iter_range_keys rename to internal/compact/testdata/range_key_compactor index 6af63cabca..50f57a8577 100644 --- a/internal/compact/testdata/iter_range_keys +++ b/internal/compact/testdata/range_key_compactor @@ -1,44 +1,44 @@ # Test snapshot striping and coalescing. -transform snapshots=(5,10,15) in-use-key-ranges=(a-z) +compact snapshots=(5,10,15) in-use-key-ranges=(a-z) a-c:{(#9,RANGEKEYSET,@3,foo5) (#4,RANGEKEYSET,@3,foo3) (#3,RANGEKEYSET,@3,foo2)} ---- a-c:{(#9,RANGEKEYSET,@3,foo5) (#4,RANGEKEYSET,@3,foo3)} -transform snapshots=(5,10,15) in-use-key-ranges=(a-z) +compact snapshots=(5,10,15) in-use-key-ranges=(a-z) a-c:{(#9,RANGEKEYUNSET,@3) (#8,RANGEKEYSET,@3,foo5) (#4,RANGEKEYSET,@3,foo3) (#3,RANGEKEYSET,@3,foo2)} ---- a-c:{(#9,RANGEKEYUNSET,@3) (#4,RANGEKEYSET,@3,foo3)} -transform snapshots=(5,10,15) in-use-key-ranges=(a-z) +compact snapshots=(5,10,15) in-use-key-ranges=(a-z) a-c:{(#9,RANGEKEYDEL) (#8,RANGEKEYSET,@3,foo5) (#4,RANGEKEYSET,@3,foo3) (#3,RANGEKEYSET,@3,foo2)} ---- a-c:{(#9,RANGEKEYDEL) (#4,RANGEKEYSET,@3,foo3)} -transform snapshots=(5,10,15) in-use-key-ranges=(a-z) +compact snapshots=(5,10,15) in-use-key-ranges=(a-z) a-c:{(#11,RANGEKEYDEL) (#8,RANGEKEYSET,@3,foo5) (#4,RANGEKEYSET,@3,foo3) (#3,RANGEKEYSET,@3,foo2)} ---- a-c:{(#11,RANGEKEYDEL) (#8,RANGEKEYSET,@3,foo5) (#4,RANGEKEYSET,@3,foo3)} -transform in-use-key-ranges=(a-z) +compact in-use-key-ranges=(a-z) a-c:{(#11,RANGEKEYDEL) (#8,RANGEKEYSET,@3,foo5) (#4,RANGEKEYSET,@3,foo3) (#3,RANGEKEYSET,@3,foo2)} ---- a-c:{(#11,RANGEKEYDEL)} # Test that elision works on the last snapshot stripe. -transform snapshots=(5,10,15) +compact snapshots=(5,10,15) a-c:{(#11,RANGEKEYDEL) (#8,RANGEKEYSET,@3,foo5) (#4,RANGEKEYSET,@3,foo3) (#3,RANGEKEYSET,@3,foo2)} ---- a-c:{(#11,RANGEKEYDEL) (#8,RANGEKEYSET,@3,foo5) (#4,RANGEKEYSET,@3,foo3)} -transform snapshots=(3,10,15) +compact snapshots=(3,10,15) a-c:{(#11,RANGEKEYDEL) (#8,RANGEKEYSET,@3,foo5) (#4,RANGEKEYSET,@3,foo3) (#3,RANGEKEYUNSET,@4) (#2,RANGEKEYSET,@3,foo2)} ---- a-c:{(#11,RANGEKEYDEL) (#8,RANGEKEYSET,@3,foo5) (#3,RANGEKEYUNSET,@4) (#2,RANGEKEYSET,@3,foo2)} -transform snapshots=(2,10,15) +compact snapshots=(2,10,15) a-c:{(#11,RANGEKEYDEL) (#8,RANGEKEYSET,@3,foo5) (#4,RANGEKEYSET,@3,foo3) (#3,RANGEKEYUNSET,@4) (#2,RANGEKEYSET,@3,foo2)} ---- a-c:{(#11,RANGEKEYDEL) (#8,RANGEKEYSET,@3,foo5)} @@ -46,69 +46,71 @@ a-c:{(#11,RANGEKEYDEL) (#8,RANGEKEYSET,@3,foo5)} # The RANGEKEYDEL deletes all underlying keys and there are no snapshots or # in-use key ranges at play, so all keys should empty out. -transform +compact a-c:{(#11,RANGEKEYDEL) (#8,RANGEKEYSET,@3,foo5) (#4,RANGEKEYSET,@3,foo3) (#3,RANGEKEYSET,@3,foo2)} ---- +. # Test RANGEKEYDELs are preserved over in-use key ranges in the last snapshot stripe. # in-use key ranges cover keys that exist in lower levels of the LSM, so dropping # range keys in that space could cause correctness issues. -transform in-use-key-ranges=(b-d) +compact in-use-key-ranges=(b-d) a-c:{(#11,RANGEKEYDEL) (#8,RANGEKEYSET,@3,foo5) (#4,RANGEKEYSET,@3,foo3) (#3,RANGEKEYSET,@3,foo2)} ---- a-c:{(#11,RANGEKEYDEL)} # Test RANGEKEYSETs are preserved in the non-last snapshot stripe. -transform in-use-key-ranges=(b-d) snapshots=(8) +compact in-use-key-ranges=(b-d) snapshots=(8) a-c:{(#11,RANGEKEYDEL) (#8,RANGEKEYSET,@3,foo5) (#4,RANGEKEYSET,@3,foo3) (#3,RANGEKEYSET,@3,foo2)} ---- a-c:{(#11,RANGEKEYDEL) (#4,RANGEKEYSET,@3,foo3)} -transform +compact a-c:{(#13,RANGEKEYSET,@3,bar1) (#12,RANGEKEYSET,@2,bar2) (#11,RANGEKEYDEL) (#8,RANGEKEYSET,@3,foo5) (#4,RANGEKEYSET,@3,foo3) (#3,RANGEKEYSET,@3,foo2)} ---- a-c:{(#13,RANGEKEYSET,@3,bar1) (#12,RANGEKEYSET,@2,bar2)} # Test RANGEKEYUNSETs are preserved over in-use key ranges. -transform +compact a-c:{(#11,RANGEKEYUNSET,@3) (#8,RANGEKEYSET,@3,foo5) (#4,RANGEKEYSET,@3,foo3) (#3,RANGEKEYSET,@3,foo2)} ---- +. -transform in-use-key-ranges=(b-d) +compact in-use-key-ranges=(b-d) a-c:{(#11,RANGEKEYUNSET,@3) (#8,RANGEKEYSET,@3,foo5) (#4,RANGEKEYSET,@3,foo3) (#3,RANGEKEYSET,@3,foo2)} ---- a-c:{(#11,RANGEKEYUNSET,@3)} # Test cases where multiple keys have the same sequence number. -transform +compact a-c:{(#11,RANGEKEYSET,@3,foo5) (#11,RANGEKEYUNSET,@4) (#11,RANGEKEYDEL)} ---- a-c:{(#11,RANGEKEYSET,@3,foo5)} -transform +compact a-c:{(#11,RANGEKEYSET,@3,foo5) (#11,RANGEKEYUNSET,@3) (#11,RANGEKEYDEL)} ---- a-c:{(#11,RANGEKEYSET,@3,foo5)} # Test that UNSETs and DELs are retained over in-use key ranges. -transform in-use-key-ranges=(b-d) +compact in-use-key-ranges=(b-d) a-c:{(#11,RANGEKEYSET,@3,foo5) (#11,RANGEKEYUNSET,@4) (#11,RANGEKEYDEL)} ---- a-c:{(#11,RANGEKEYSET,@3,foo5) (#11,RANGEKEYUNSET,@4) (#11,RANGEKEYDEL)} # Test that sets shadow unset at the same prefix, even if elision is disabled. -transform in-use-key-ranges=(b-d) +compact in-use-key-ranges=(b-d) a-c:{(#11,RANGEKEYSET,@3,foo5) (#11,RANGEKEYUNSET,@3) (#11,RANGEKEYDEL)} ---- a-c:{(#11,RANGEKEYSET,@3,foo5) (#11,RANGEKEYDEL)} -transform in-use-key-ranges=(a-z) +compact in-use-key-ranges=(a-z) a-c:{(#11,RANGEKEYSET,@3,foo5) (#11,RANGEKEYUNSET,@3) (#11,RANGEKEYDEL) ---- a-c:{(#11,RANGEKEYSET,@3,foo5) (#11,RANGEKEYDEL)} diff --git a/internal/keyspan/span.go b/internal/keyspan/span.go index ae4444a405..dd823183e5 100644 --- a/internal/keyspan/span.go +++ b/internal/keyspan/span.go @@ -399,6 +399,14 @@ func (s *Span) CoversAt(snapshot, seqNum uint64) bool { return false } +// Reset clears the span's Start, End, and Keys fields, retaining the slices for +// reuse. +func (s *Span) Reset() { + s.Start = s.Start[:0] + s.End = s.End[:0] + s.Keys = s.Keys[:0] +} + // String returns a string representation of the span. func (s Span) String() string { return fmt.Sprint(prettySpan{Span: s, formatKey: base.DefaultFormatter})