diff --git a/internal/rangekey/bound_iter.go b/internal/keyspan/bound_iter.go similarity index 95% rename from internal/rangekey/bound_iter.go rename to internal/keyspan/bound_iter.go index dec80c0262..3642ec209e 100644 --- a/internal/rangekey/bound_iter.go +++ b/internal/keyspan/bound_iter.go @@ -2,7 +2,7 @@ // of this source code is governed by a BSD-style license that can be found in // the LICENSE file. -package rangekey +package keyspan import ( "bytes" @@ -10,7 +10,6 @@ import ( "github.com/cockroachdb/pebble/internal/base" "github.com/cockroachdb/pebble/internal/invariants" - "github.com/cockroachdb/pebble/internal/keyspan" ) type boundKind int8 @@ -38,7 +37,7 @@ type boundKey struct { // Start keys have user keys equal to the boundKey's key field. If the // boundKind is boundKindFragmentEnd, all fragments' End keys equal the // boundKey's key field. - fragments []keyspan.Span + fragments []Span } func (k boundKey) valid() bool { @@ -83,31 +82,31 @@ func (k boundKey) String() string { // // Along with the key and bound kind, fragmentBoundIterator surfaces the set of // all the fragments that share that bound. fragmentBoundIterator is used -// internally by the range-key merging iterator. +// internally by the span merging iterator. // // Note that fragmentBoundIterator's interface differs from FragmentIterator. // The fragmentBoundIterator iterates over individual bounds, including end -// boundaries, whereas FragmentIterator returns whole keyspan.Spans ordered by +// boundaries, whereas FragmentIterator returns whole Spans ordered by // Start key. The fragmentBoundIterator's interface is designed this way to // simplify the mergingIter implementation, which must include both start and // end boundaries in its heap. type fragmentBoundIterator struct { // iter holds the underlying fragment iterator. iterSpan is always set to // the span at iter's current position. - iter keyspan.FragmentIterator - iterSpan keyspan.Span + iter FragmentIterator + iterSpan Span // start and end hold the user keys for the iterator's current fragment. One // of either the start or the end is the current iterator position, // indicated by boundKind. All of the underlying iterators fragments with // the bounds [start, end) are contained within the fragments field. start []byte end []byte - fragments []keyspan.Span + fragments []Span boundKind boundKind dir int8 } -func (i *fragmentBoundIterator) init(fragmentIter keyspan.FragmentIterator) { +func (i *fragmentBoundIterator) init(fragmentIter FragmentIterator) { *i = fragmentBoundIterator{iter: fragmentIter} } @@ -291,8 +290,8 @@ func (i *fragmentBoundIterator) initBackwardSpan(cmp base.Compare) boundKey { func (i *fragmentBoundIterator) clearFragments() { for j := range i.fragments { - // Clear any pointers into range key blocks to avoid retaining them. - i.fragments[j] = keyspan.Span{} + // Clear any pointers into blocks to avoid retaining them. + i.fragments[j] = Span{} } i.fragments = i.fragments[:0] } diff --git a/internal/rangekey/bound_iter_test.go b/internal/keyspan/bound_iter_test.go similarity index 81% rename from internal/rangekey/bound_iter_test.go rename to internal/keyspan/bound_iter_test.go index 64dc591a19..b4fb28a8f5 100644 --- a/internal/rangekey/bound_iter_test.go +++ b/internal/keyspan/bound_iter_test.go @@ -2,7 +2,7 @@ // of this source code is governed by a BSD-style license that can be found in // the LICENSE file. -package rangekey +package keyspan import ( "bytes" @@ -12,8 +12,6 @@ import ( "github.com/cockroachdb/pebble/internal/base" "github.com/cockroachdb/pebble/internal/datadriven" - "github.com/cockroachdb/pebble/internal/keyspan" - "github.com/stretchr/testify/require" ) func TestFragmentBoundIterator(t *testing.T) { @@ -32,19 +30,12 @@ func TestFragmentBoundIterator(t *testing.T) { datadriven.RunTest(t, "testdata/fragment_bound_iterator", func(td *datadriven.TestData) string { switch td.Cmd { case "define": - var spans []keyspan.Span + var spans []Span lines := strings.Split(strings.TrimSpace(td.Input), "\n") for _, line := range lines { - startKey, value := Parse(line) - endKey, v, ok := DecodeEndKey(startKey.Kind(), value) - require.True(t, ok) - spans = append(spans, keyspan.Span{ - Start: startKey, - End: endKey, - Value: v, - }) + spans = append(spans, parseSpanWithKind(t, line)) } - iter.init(keyspan.NewIter(cmp, spans)) + iter.init(NewIter(cmp, spans)) return "OK" case "iter": buf.Reset() diff --git a/internal/rangekey/merging_iter.go b/internal/keyspan/merging_iter.go similarity index 92% rename from internal/rangekey/merging_iter.go rename to internal/keyspan/merging_iter.go index 66995efe71..904440cd3d 100644 --- a/internal/rangekey/merging_iter.go +++ b/internal/keyspan/merging_iter.go @@ -2,7 +2,7 @@ // of this source code is governed by a BSD-style license that can be found in // the LICENSE file. -package rangekey +package keyspan import ( "bytes" @@ -11,7 +11,6 @@ import ( "github.com/cockroachdb/pebble/internal/base" "github.com/cockroachdb/pebble/internal/invariants" - "github.com/cockroachdb/pebble/internal/keyspan" ) // TODO(jackson): Document memory safety requirements. Once we're reading from @@ -28,11 +27,6 @@ import ( // seeks would require introducing key comparisons to switchTo{Min,Max}Heap // where there currently are none. -// TODO(jackson): Consider moving this to the internal/keyspan package and -// removing any explicit mention of range keys. The algorithm works for all key -// span typees, and could be used to fragment RANGEDELs across levels during a -// compaction. - // Fragments holds a set of fragments all with the bounds [Start, End). // Individual, sorted fragments may be retrieved using the At method. type Fragments struct { @@ -51,17 +45,17 @@ func (f *Fragments) Count() int { } // At retrieves the i-th span. -func (f *Fragments) At(i int) keyspan.Span { +func (f *Fragments) At(i int) Span { // Construct the fragment with f.spans[i].Start's trailer, so it adopts its // sequence number and kind, and with f.spans[i]'s Value. - return keyspan.Span{ + return Span{ Start: base.InternalKey{UserKey: f.Start, Trailer: f.spans[i].Start.Trailer}, End: f.End, Value: f.spans[i].Value, } } -type bySeqKind []*keyspan.Span +type bySeqKind []*Span func (s bySeqKind) Len() int { return len(s) } func (s bySeqKind) Swap(i, j int) { s[i], s[j] = s[j], s[i] } @@ -76,9 +70,8 @@ func (s bySeqKind) Less(i, j int) bool { } } -// MergingIter merges range keys across levels of the LSM, exposing a fragment -// iterator that yields range key spans fragmented at unique range key -// boundaries. +// MergingIter merges fragments across levels of the LSM, exposing a fragment +// iterator that yields sets of spans fragmented at unique user key boundaries. // // A MergingIter is initialized with an arbitrary number of child iterators over // fragmented spans. Each child iterator exposes fragmented key spans, such that @@ -94,7 +87,7 @@ func (s bySeqKind) Less(i, j int) bool { // The MergingIter configured with c1 and c2 further fragments the key spans and // surfaces the key spans: // -// a-b a-b b-c b-c b-c c--e c--e c--e c--e e-g h-j +// [a-b a-b] [b-c b-c b-c] [c--e c--e c--e c--e] [e-g] [h-j] // // This is the result of fragmenting all of c1 and c2's fragments at every // unique key bound (a, b, c, e, g, h, j). @@ -218,9 +211,9 @@ func (s bySeqKind) Less(i, j int) bool { // part of a boundKey. In the above example, i1 and i2 are positioned at end // boundaries, so findNextFragmentSet collects [a,c) and [a,p). These spans // contain the merging iterator's [m.start, m.end) span, but they may also -// extend beyond the m.start and m.end. Before returning the spans to the -// caller, these spans are truncated to m.start and m.end user keys, preserving -// the existing spans' sequence numbers, key kinds and values. +// extend beyond the m.start and m.end. The merging iterator returns the +// fragments with the merging iter's m.start and m.end bounds, preserving the +// underlying keys spans' sequence numbers, key kinds and values. // // It may be the case that findNextFragmentSet finds no levels positioned at end // boundaries, in which case the span [m.start, m.end) overlaps with nothing. In @@ -236,11 +229,11 @@ type MergingIter struct { // Invariant: None of the levels' iterators contain spans with a bound // between start and end. For all bounds b, b ≤ start || b ≥ end. start, end []byte - // fragments holds all of the range key fragments across all levels that - // overlap the key span [start, end), sorted by sequence number and kind - // descending. The fragments are not truncated to start and end. This slice - // is reconstituted in synthesizeFragments from each mergiingIterLevel's - // fragments every time the [start, end) bounds change. + // fragments holds all of the fragments across all levels that overlap the + // key span [start, end), sorted by sequence number and kind descending. The + // fragments are not truncated to start and end. This slice is reconstituted + // in synthesizeFragments from each mergiingIterLevel's fragments every time + // the [start, end) bounds change. // // Each element points into a child iterator's memory, so the spans may not // be directly modified. @@ -258,7 +251,7 @@ type mergingIterLevel struct { } // Init initializes the merging iterator with the provided fragment iterators. -func (m *MergingIter) Init(cmp base.Compare, iters ...keyspan.FragmentIterator) { +func (m *MergingIter) Init(cmp base.Compare, iters ...FragmentIterator) { levels, items := m.levels, m.heap.items *m = MergingIter{ @@ -278,9 +271,8 @@ func (m *MergingIter) Init(cmp base.Compare, iters ...keyspan.FragmentIterator) } } -// SeekGE implements base.InternalIterator.SeekGE. Like other implementations of -// keyspan.FragmentIterator, MergingIter seeks based on the span's Start key, -// returning the fragment with the smallest Start ≥ key. +// SeekGE moves the iterator to the first set of fragments with a start key +// greater than or equal to key. func (m *MergingIter) SeekGE(key []byte, trySeekUsingNext bool) Fragments { m.invalidate() // clear state about current position for i := range m.levels { @@ -291,9 +283,8 @@ func (m *MergingIter) SeekGE(key []byte, trySeekUsingNext bool) Fragments { return m.findNextFragmentSet() } -// SeekLT implements base.InternalIterator.SeekLT. Like other implementations of -// keyspan.FragmentIterator, MergingIter seeks based on fragements' Start keys, -// returning the fragment with the largest Start that's < key. +// SeekLT moves the iterator to the last set of fragments with a start key +// less than key. func (m *MergingIter) SeekLT(key []byte) Fragments { // TODO(jackson): Evaluate whether there's an implementation of SeekLT // independent of SeekGE that is more efficient. It's tricky, because the @@ -321,7 +312,7 @@ func (m *MergingIter) SeekLT(key []byte) Fragments { return m.Prev() } -// First implements base.InternalIterator.First. +// First seeks the iterator to the first set of fragments. func (m *MergingIter) First() Fragments { m.invalidate() // clear state about current position for i := range m.levels { @@ -332,7 +323,7 @@ func (m *MergingIter) First() Fragments { return m.findNextFragmentSet() } -// Last implements base.InternalIterator.Last. +// Last seeks the iterator to the last set of fragments. func (m *MergingIter) Last() Fragments { m.invalidate() // clear state about current position for i := range m.levels { @@ -343,7 +334,7 @@ func (m *MergingIter) Last() Fragments { return m.findPrevFragmentSet() } -// Next implements base.InternalIterator.Next. +// Next advances the iterator to the next set of fragments. func (m *MergingIter) Next() Fragments { if m.err != nil { return Fragments{} @@ -360,7 +351,7 @@ func (m *MergingIter) Next() Fragments { return m.findNextFragmentSet() } -// Prev implements base.InternalIterator.Prev. +// Prev advances the iterator to the prev set of fragments. func (m *MergingIter) Prev() Fragments { if m.err != nil { return Fragments{} @@ -377,7 +368,7 @@ func (m *MergingIter) Prev() Fragments { return m.findPrevFragmentSet() } -// Error implements (base.InternalIterator).Error. +// Error returns any accumulated error. func (m *MergingIter) Error() error { if m.heap.len() == 0 || m.err != nil { return m.err @@ -385,14 +376,16 @@ func (m *MergingIter) Error() error { return m.levels[m.heap.items[0].index].iter.iter.Error() } -// SetBounds implements (base.InternalIterator).SetBounds. +// SetBounds sets the lower and upper bounds for the iterator. Note that the +// result of Next and Prev will be undefined until the iterator has been +// repositioned with SeekGE, SeekLT, First, or Last. func (m *MergingIter) SetBounds(lower, upper []byte) { for i := range m.levels { m.levels[i].iter.iter.SetBounds(lower, upper) } } -// Close implements (base.InternalIterator).Close. +// Close closes the iterator, releasing all acquired resources. func (m *MergingIter) Close() error { for i := range m.levels { if err := m.levels[i].iter.iter.Close(); err != nil && m.err == nil { @@ -404,6 +397,7 @@ func (m *MergingIter) Close() error { return m.err } +// String implements fmt.Stringer. func (m *MergingIter) String() string { return "merging-keyspan" } @@ -778,11 +772,11 @@ func (m *MergingIter) prevEntry() { } } -// clonedIters makes a clone of the merging iterator's underlying iterators and +// ClonedIters makes a clone of the merging iterator's underlying iterators and // returns them. -func (m *MergingIter) clonedIters() []keyspan.FragmentIterator { +func (m *MergingIter) ClonedIters() []FragmentIterator { // TODO(jackson): Remove when range-key state is included in readState. - var iters []keyspan.FragmentIterator + var iters []FragmentIterator for l := range m.levels { iters = append(iters, m.levels[l].iter.iter.Clone()) } @@ -892,3 +886,12 @@ func (h *mergingIterHeap) down(i0, n int) bool { } return i > i0 } + +// firstError returns the first non-nil error of err0 and err1, or nil if both +// are nil. +func firstError(err0, err1 error) error { + if err0 != nil { + return err0 + } + return err1 +} diff --git a/internal/rangekey/merging_iter_test.go b/internal/keyspan/merging_iter_test.go similarity index 86% rename from internal/rangekey/merging_iter_test.go rename to internal/keyspan/merging_iter_test.go index 09fd91b67b..2be672d4c2 100644 --- a/internal/rangekey/merging_iter_test.go +++ b/internal/keyspan/merging_iter_test.go @@ -2,7 +2,7 @@ // of this source code is governed by a BSD-style license that can be found in // the LICENSE file. -package rangekey +package keyspan import ( "bytes" @@ -11,14 +11,35 @@ import ( "strings" "testing" "time" + "unicode" "github.com/cockroachdb/pebble/internal/base" "github.com/cockroachdb/pebble/internal/datadriven" - "github.com/cockroachdb/pebble/internal/keyspan" "github.com/cockroachdb/pebble/internal/testkeys" "github.com/stretchr/testify/require" ) +func parseSpanWithKind(t testing.TB, s string) Span { + // parseSpanWithKind expects a string like + // + // a.RANGEKEYSET.10 : c + // + // The value is optional. + fields := strings.FieldsFunc(s, func(r rune) bool { return r == ':' || unicode.IsSpace(r) }) + if len(fields) < 2 { + t.Fatalf("key span string representation should have 2+ fields, found %d in %q", len(fields), s) + } + var value []byte + if len(fields[2:]) > 0 { + value = []byte(strings.Join(fields[2:], " ")) + } + return Span{ + Start: base.ParseInternalKey(fields[0]), + End: []byte(fields[1]), + Value: value, + } +} + func TestMergingIter(t *testing.T) { cmp := base.DefaultComparer.Compare var buf bytes.Buffer @@ -39,27 +60,20 @@ func TestMergingIter(t *testing.T) { datadriven.RunTest(t, "testdata/merging_iter", func(td *datadriven.TestData) string { switch td.Cmd { case "define": - var iters []keyspan.FragmentIterator - var spans []keyspan.Span + var iters []FragmentIterator + var spans []Span lines := strings.Split(strings.TrimSpace(td.Input), "\n") for _, line := range lines { if line == "--" { - iters = append(iters, keyspan.NewIter(cmp, spans)) + iters = append(iters, NewIter(cmp, spans)) spans = nil continue } - startKey, value := Parse(line) - endKey, v, ok := DecodeEndKey(startKey.Kind(), value) - require.True(t, ok) - spans = append(spans, keyspan.Span{ - Start: startKey, - End: endKey, - Value: v, - }) + spans = append(spans, parseSpanWithKind(t, line)) } if len(spans) > 0 { - iters = append(iters, keyspan.NewIter(cmp, spans)) + iters = append(iters, NewIter(cmp, spans)) } iter.Init(cmp, iters...) return fmt.Sprintf("%d levels", len(iters)) @@ -114,7 +128,7 @@ func TestMergingIter(t *testing.T) { // TestMergingIter_FragmenterEquivalence tests for equivalence between the // fragmentation performed on-the-fly by the MergingIter and the fragmentation -// performed by the keyspan.Fragmenter. +// performed by the Fragmenter. // // It does this by producing 1-10 levels of well-formed fragments. Generated // fragments may overlap other levels arbitrarily, but within their level @@ -122,8 +136,8 @@ func TestMergingIter(t *testing.T) { // key bounds. // // The test then feeds all the fragments, across all levels, into a Fragmenter -// and produces a keyspan.Iter over those fragments. The test also constructs a -// MergingIter with a separate keyspan.Iter for each level. It runs a random +// and produces a Iter over those fragments. The test also constructs a +// MergingIter with a separate Iter for each level. It runs a random // series of operations, applying each operation to both. It asserts that each // operation has identical results on both iterators. func TestMergingIter_FragmenterEquivalence(t *testing.T) { @@ -150,9 +164,9 @@ func testFragmenterEquivalenceOnce(t *testing.T, seed int64) { ks := testkeys.Alpha(rng.Intn(3) + 1) // Generate between 1 and 10 levels of fragment iterators. - levels := make([][]keyspan.Span, rng.Intn(10)+1) - iters := make([]keyspan.FragmentIterator, len(levels)) - var allSpans []keyspan.Span + levels := make([][]Span, rng.Intn(10)+1) + iters := make([]FragmentIterator, len(levels)) + var allSpans []Span var buf bytes.Buffer for l := 0; l < len(levels); l++ { fmt.Fprintf(&buf, "level %d: ", l) @@ -167,7 +181,7 @@ func testFragmenterEquivalenceOnce(t *testing.T, seed int64) { fragmentCount := uint64(rng.Intn(3) + 1) for f := fragmentCount; f > 0; f-- { - s := keyspan.Span{ + s := Span{ Start: base.MakeInternalKey( startUserKey, uint64((len(levels)-l)*3)+f, @@ -187,20 +201,20 @@ func testFragmenterEquivalenceOnce(t *testing.T, seed int64) { } keyspaceStartIdx = spanEndIdx } - iters[l] = keyspan.NewIter(cmp, levels[l]) + iters[l] = NewIter(cmp, levels[l]) fmt.Fprintln(&buf) } // Fragment the spans across the levels. - var allFragmented []keyspan.Span - f := keyspan.Fragmenter{ + var allFragmented []Span + f := Fragmenter{ Cmp: cmp, Format: testkeys.Comparer.FormatKey, - Emit: func(spans []keyspan.Span) { + Emit: func(spans []Span) { allFragmented = append(allFragmented, spans...) }, } - keyspan.Sort(f.Cmp, allSpans) + Sort(f.Cmp, allSpans) for _, s := range allSpans { f.Add(s) } @@ -217,7 +231,7 @@ func testFragmenterEquivalenceOnce(t *testing.T, seed int64) { } t.Logf("%d levels:\n%s\n", len(levels), buf.String()) - fragmenterIter := keyspan.NewIter(f.Cmp, allFragmented) + fragmenterIter := NewIter(f.Cmp, allFragmented) mergingIter := &mergingIterAdapter{MergingIter: &MergingIter{}} mergingIter.MergingIter.Init(f.Cmp, iters...) @@ -300,8 +314,8 @@ func testFragmenterEquivalenceOnce(t *testing.T, seed int64) { } // mergingIterAdapter adapts MergingIter, which returns Fragments, to fulfill -// the keyspan.FragmentIterator interface. It's used by -// TestMergingIter_FragmenterEquivalence to compare a keyspan.Iter with a +// the FragmentIterator interface. It's used by +// TestMergingIter_FragmenterEquivalence to compare a Iter with a // MergingIter, despite their different interfaces. type mergingIterAdapter struct { *MergingIter @@ -310,9 +324,9 @@ type mergingIterAdapter struct { frags Fragments } -var _ keyspan.FragmentIterator = &mergingIterAdapter{} +var _ FragmentIterator = &mergingIterAdapter{} -func (i *mergingIterAdapter) Clone() keyspan.FragmentIterator { +func (i *mergingIterAdapter) Clone() FragmentIterator { panic("unimplemented") } @@ -320,9 +334,9 @@ func (i *mergingIterAdapter) End() []byte { return i.Current().End } -func (i *mergingIterAdapter) Current() keyspan.Span { +func (i *mergingIterAdapter) Current() Span { if i.index < 0 || i.index >= i.frags.Count() { - return keyspan.Span{} + return Span{} } return i.frags.At(i.index) } diff --git a/internal/rangekey/testdata/fragment_bound_iterator b/internal/keyspan/testdata/fragment_bound_iterator similarity index 100% rename from internal/rangekey/testdata/fragment_bound_iterator rename to internal/keyspan/testdata/fragment_bound_iterator diff --git a/internal/rangekey/testdata/merging_iter b/internal/keyspan/testdata/merging_iter similarity index 100% rename from internal/rangekey/testdata/merging_iter rename to internal/keyspan/testdata/merging_iter diff --git a/internal/rangekey/iter.go b/internal/rangekey/iter.go index 0b4de60194..5bbc3279a8 100644 --- a/internal/rangekey/iter.go +++ b/internal/rangekey/iter.go @@ -57,8 +57,8 @@ type Iterator interface { // Iter handles 'coalescing' spans on-the-fly, including dropping key spans that // are no longer relevant. type Iter struct { - miter MergingIter - iterFrags Fragments + miter keyspan.MergingIter + iterFrags keyspan.Fragments coalescer Coalescer curr CoalescedSpan err error @@ -93,7 +93,7 @@ func (i *Iter) Clone() Iterator { // Init the new Iter to ensure err is cleared. newIter := &Iter{} newIter.Init(i.coalescer.items.cmp, i.coalescer.formatKey, i.coalescer.visibleSeqNum, - i.miter.clonedIters()...) + i.miter.ClonedIters()...) return newIter }