diff --git a/compaction.go b/compaction.go index 90348e74c7..4304260c51 100644 --- a/compaction.go +++ b/compaction.go @@ -154,7 +154,12 @@ type grandparentLimitSplitter struct { func (g *grandparentLimitSplitter) shouldSplitBefore( key *InternalKey, tw *sstable.Writer, ) compactionSplitSuggestion { - if g.limit != nil && g.c.cmp(key.UserKey, g.limit) > 0 { + // NB: The limit must be applied using >= since g.limit may be used as the + // `splitterSuggestion` ultimately passed to `compactionIter.Tombstones` to + // serve as an *exclusive* end boundary truncation point. If we used > then, + // we may have already added a key with the user key `g.limit` to the + // previous sstable. + if g.limit != nil && g.c.cmp(key.UserKey, g.limit) >= 0 { return splitNow } return noSplit @@ -854,14 +859,6 @@ func seekGT(iter *manifest.LevelIterator, cmp base.Compare, key []byte) *manifes // current heuristic stops output of a table if the addition of another key // would cause the table to overlap more than 10x the target file size at // level N. See maxGrandparentOverlapBytes. -// -// TODO(peter): Stopping compaction output in the middle of a user-key creates -// 2 sstables that need to be compacted together as an "atomic compaction -// unit". This is unfortunate as it removes the benefit of stopping output to -// an sstable in order to prevent a large compaction with the next level. Seems -// better to adjust findGrandparentLimit to not stop output in the middle of a -// user-key. Perhaps this isn't a problem if the compaction picking heuristics -// always pick the right (older) sibling for compaction first. func (c *compaction) findGrandparentLimit(start []byte) []byte { iter := c.grandparents.Iter() var overlappedBytes uint64 @@ -870,8 +867,8 @@ func (c *compaction) findGrandparentLimit(start []byte) []byte { // To ensure forward progress we always return a larger user // key than where we started. See comments above clients of // this function for how this is used. - if overlappedBytes > c.maxOverlapBytes && c.cmp(start, f.Largest.UserKey) < 0 { - return f.Largest.UserKey + if overlappedBytes > c.maxOverlapBytes && c.cmp(start, f.Smallest.UserKey) < 0 { + return f.Smallest.UserKey } } return nil @@ -1056,13 +1053,15 @@ func (c *compaction) newInputIter(newIters tableNewIters) (_ internalIterator, r c.closers = append(c.closers, rangeDelIter) rangeDelIter = noCloseIter{rangeDelIter} - // Truncate the range tombstones returned by the iterator to the upper - // bound of the atomic compaction unit. Note that we need do this - // truncation at read time in order to handle RocksDB generated sstables - // which do not truncate range tombstones to atomic compaction unit - // boundaries at write time. Because we're doing the truncation at read - // time, we follow RocksDB's lead and do not truncate tombstones to - // atomic unit boundaries at compaction time. + // Truncate the range tombstones returned by the iterator to the + // upper bound of the atomic compaction unit. Note that we need do + // this truncation at read time in order to handle sstables + // generated by RocksDB and earlier versions of Pebble which do not + // truncate range tombstones to atomic compaction unit boundaries at + // write time. + // + // The current Pebble compaction logic DOES truncate tombstones to + // atomic unit boundaries at compaction time too. atomicUnit, _ := expandToAtomicUnit(c.cmp, f.Slice(), true /* disableIsCompacting */) lowerBound, upperBound := manifest.KeyRange(c.cmp, atomicUnit.Iter()) // Range deletion tombstones are often written to sstables @@ -2205,7 +2204,7 @@ func (d *DB) runCompaction( // compactionIter.Tombstones via keyspan.Fragmenter.FlushTo, and by the // WriterMetadata.LargestRangeDel.UserKey. splitKey = append([]byte(nil), splitKey...) - for _, v := range iter.Tombstones(splitKey, splitL0Outputs) { + for _, v := range iter.Tombstones(splitKey) { if tw == nil { if err := newOutput(); err != nil { return err @@ -2272,10 +2271,9 @@ func (d *DB) runCompaction( outputMetrics.Size += int64(meta.Size) outputMetrics.NumFiles++ - // The handling of range boundaries is a bit complicated. if n := len(ve.NewFiles); n > 1 { - // This is not the first output file. Bound the smallest range key by the - // previous tables largest key. + // This is not the first output file. Ensure the sstable boundaries + // are nonoverlapping. prevMeta := ve.NewFiles[n-2].Meta if writerMeta.SmallestRangeDel.UserKey != nil { c := d.cmp(writerMeta.SmallestRangeDel.UserKey, prevMeta.Largest.UserKey) @@ -2284,40 +2282,17 @@ func (d *DB) runCompaction( "pebble: smallest range tombstone start key is less than previous sstable largest key: %s < %s", writerMeta.SmallestRangeDel.Pretty(d.opts.Comparer.FormatKey), prevMeta.Largest.Pretty(d.opts.Comparer.FormatKey)) - } - if c == 0 && prevMeta.Largest.SeqNum() <= writerMeta.SmallestRangeDel.SeqNum() { - // The user key portion of the range boundary start key is equal to - // the previous table's largest key. We need the tables to be - // key-space partitioned, so force the boundary to a key that we know - // is larger than the previous table's largest key. - if prevMeta.Largest.SeqNum() == 0 { - // If the seqnum of the previous table's largest key is 0, we can't - // decrement it. This should never happen as we take care in the - // main compaction loop to avoid generating an sstable with a - // largest key containing a zero seqnum. - return errors.Errorf( - "pebble: previous sstable largest key unexpectedly has 0 seqnum: %s", - prevMeta.Largest.Pretty(d.opts.Comparer.FormatKey)) - } - // TODO(peter): Technically, this produces a small gap with the - // previous sstable. The largest key of the previous table may be - // b#5.SET. The smallest key of the new sstable may then become - // b#4.RANGEDEL even though the tombstone is [b,z)#6. If iteration - // ever precisely truncates sstable boundaries, the key b#5.DEL at - // a lower level could slip through. Note that this can't ever - // actually happen, though, because the only way for two records to - // have the same seqnum is via ingestion. And even if it did happen - // revealing a deletion tombstone is not problematic. Slightly more - // worrisome is the combination of b#5.MERGE, b#5.SET and - // b#4.RANGEDEL, but we can't ever see b#5.MERGE and b#5.SET at - // different levels in the tree due to the ingestion argument and - // atomic compaction units. - // - // TODO(sumeer): Incorporate the the comment in - // https://github.com/cockroachdb/pebble/pull/479#pullrequestreview-340600654 - // into docs/range_deletions.md and reference the correctness - // argument here. Note that that comment might be slightly incorrect. - writerMeta.SmallestRangeDel.SetSeqNum(prevMeta.Largest.SeqNum() - 1) + } else if c == 0 && !prevMeta.Largest.IsExclusiveSentinel() { + // The user key portion of the range boundary start key is + // equal to the previous table's largest key user key, and + // the previous table's largest key is not exclusive. This + // violates the invariant that tables are key-space + // partitioned. + return errors.Errorf( + "pebble: invariant violation: previous sstable largest key %s, current sstable smallest rangedel: %s", + prevMeta.Largest.Pretty(d.opts.Comparer.FormatKey), + writerMeta.SmallestRangeDel.Pretty(d.opts.Comparer.FormatKey), + ) } } } @@ -2365,12 +2340,10 @@ func (d *DB) runCompaction( c.largest.Pretty(d.opts.Comparer.FormatKey)) } } - // Verify that when splitting an output to L0, we never split different - // revisions of the same user key across two different sstables. - if splitL0Outputs { - if err := c.errorOnUserKeyOverlap(ve); err != nil { - return err - } + // Verify that we never split different revisions of the same user key + // across two different sstables. + if err := c.errorOnUserKeyOverlap(ve); err != nil { + return err } if err := meta.Validate(d.cmp, d.opts.Comparer.FormatKey); err != nil { return err @@ -2384,40 +2357,29 @@ func (d *DB) runCompaction( // the splitterGroup can be composed of multiple splitters. In this case, // we start off with splitters for file sizes, grandparent limits, and (for // L0 splits) L0 limits, before wrapping them in an splitterGroup. - // - // There is a complication here: We may not be able to switch SSTables right - // away when we are splitting an L0 output. We do not split the same user - // key across different sstables within one flush or intra-L0 compaction, so - // the userKeyChangeSplitter ensures we are at a user key change boundary - // when doing a split. outputSplitters := []compactionOutputSplitter{ &fileSizeSplitter{maxFileSize: c.maxOutputFileSize}, &grandparentLimitSplitter{c: c, ve: ve}, } - var splitter compactionOutputSplitter if splitL0Outputs { - // outputSplitters[0] is the file size splitter, which doesn't guarantee - // that all advised splits will be at user key change boundaries. Wrap - // it in a userKeyChangeSplitter. - outputSplitters[0] = &userKeyChangeSplitter{ - cmp: c.cmp, - splitter: outputSplitters[0], - unsafePrevUserKey: func() []byte { - // Return the largest point key written to tw or the start of - // the current range deletion in the fragmenter, whichever is - // greater. - prevPoint := prevPointKey.UnsafeKey() - if c.cmp(prevPoint.UserKey, c.rangeDelFrag.Start()) > 0 { - return prevPoint.UserKey - } - return c.rangeDelFrag.Start() - }, - } outputSplitters = append(outputSplitters, &l0LimitSplitter{c: c, ve: ve}) } - splitter = &splitterGroup{ - cmp: c.cmp, - splitters: outputSplitters, + // We do not split the same user key across different sstables within one + // flush or compaction, so the userKeyChangeSplitter ensures we are at a + // user key change boundary when doing a split. + var splitter compactionOutputSplitter = &userKeyChangeSplitter{ + cmp: c.cmp, + splitter: &splitterGroup{cmp: c.cmp, splitters: outputSplitters}, + unsafePrevUserKey: func() []byte { + // Return the largest point key written to tw or the start of + // the current range deletion in the fragmenter, whichever is + // greater. + prevPoint := prevPointKey.UnsafeKey() + if c.cmp(prevPoint.UserKey, c.rangeDelFrag.Start()) > 0 { + return prevPoint.UserKey + } + return c.rangeDelFrag.Start() + }, } // NB: we avoid calling maybeThrottle on a nilPacer because the cost of @@ -2440,7 +2402,6 @@ func (d *DB) runCompaction( splitterSuggestion := splitter.onNewOutput(key) // Each inner loop iteration processes one key from the input iterator. - prevPointSeqNum := InternalKeySeqNumMax for ; key != nil; key, val = iter.Next() { if split := splitter.shouldSplitBefore(key, tw); split == splitNow { break @@ -2468,21 +2429,11 @@ func (d *DB) runCompaction( if err := tw.Add(*key, val); err != nil { return nil, pendingOutputs, err } - prevPointSeqNum = key.SeqNum() } // A splitter requested a split, and we're ready to finish the output. // We need to choose the key at which to split any pending range // tombstones. - // - // There's a complication here. We need to ensure that for a user key - // k we never end up with one output's largest key as k#0 and the - // next output's smallest key as k#RANGEDEL,#x where x > 0. This is a - // problem because k#RANGEDEL,#x sorts before k#0. Normally, we just - // adjust the seqnum of the next output's smallest boundary to be - // less, but that's not possible with the zero seqnum. We can avoid - // this case with careful picking of where to split pending range - // tombstones. var splitKey []byte switch { case key != nil: @@ -2520,26 +2471,12 @@ func (d *DB) runCompaction( // flush up to the next grandparent limit greater than // `splitterSuggestion` instead. splitKey = nil - case key == nil && prevPointSeqNum != 0: + case key == nil: // The last key added did not have a zero sequence number, so // we'll always be able to adjust the next table's smallest key. - // NB: Because of the splitter's `onNewOutput` contract, - // `splitterSuggestion` must be >= any key previously added to the - // current output sstable. + // NB: Because of the userKeyChangeSplitter, `splitterSuggestion` + // must be > any key previously added to the current output sstable. splitKey = splitterSuggestion - case key == nil && prevPointSeqNum == 0: - // The last key added did have a zero sequence number. The - // splitters' suggested split point might have the same user key, - // which would cause the next output to have an unadjustable - // smallest key. To prevent that, we ignore the splitter's - // suggestion, leaving splitKey nil to flush all pending range - // tombstones. - // TODO(jackson): This case is only problematic if splitKey equals - // the user key of the last point key added. We don't need to - // flush *all* range tombstones to the current sstable. We could - // flush up to the next grandparent limit greater than - // `splitterSuggestion` instead. - splitKey = nil default: return nil, nil, errors.New("pebble: not reached") } diff --git a/compaction_iter.go b/compaction_iter.go index c93b42bb09..fe3cca627a 100644 --- a/compaction_iter.go +++ b/compaction_iter.go @@ -769,17 +769,14 @@ func (i *compactionIter) Close() error { // exclude specifies if the specified key is exclusive or inclusive. // When exclude = true, all returned range tombstones are truncated to the // specified key. -func (i *compactionIter) Tombstones(key []byte, exclude bool) []keyspan.Span { - switch { - case key == nil: +func (i *compactionIter) Tombstones(key []byte) []keyspan.Span { + if key == nil { i.rangeDelFrag.Finish() - case exclude: + } else { // The specified end key is exclusive; no versions of the specified // user key (including range tombstones covering that key) should // be flushed yet. i.rangeDelFrag.TruncateAndFlushTo(key) - default: - i.rangeDelFrag.FlushTo(key) } tombstones := i.tombstones i.tombstones = nil diff --git a/compaction_iter_test.go b/compaction_iter_test.go index fe4da0b1c5..c2fdde9da4 100644 --- a/compaction_iter_test.go +++ b/compaction_iter_test.go @@ -191,7 +191,7 @@ func TestCompactionIter(t *testing.T) { if len(parts) == 2 { key = []byte(parts[1]) } - for _, v := range iter.Tombstones(key, false) { + for _, v := range iter.Tombstones(key) { fmt.Fprintf(&b, "%s-%s#%d\n", v.Start.UserKey, v.End, v.Start.SeqNum()) } diff --git a/compaction_picker.go b/compaction_picker.go index 6a3cd47726..4155477db4 100644 --- a/compaction_picker.go +++ b/compaction_picker.go @@ -374,6 +374,11 @@ func (pc *pickedCompaction) grow(sm, la InternalKey, maxExpandedBytes uint64) bo // disableIsCompacting is true, isCompacting always returns false. This helps // avoid spurious races from being detected when this method is used outside // of compaction picking code. +// +// TODO(jackson): Compactions and flushes no longer split a user key between two +// sstables. We could perform a migration, re-compacting any sstables with split +// user keys, which would allow us to remove atomic compaction unit expansion +// code. func expandToAtomicUnit( cmp Compare, inputs manifest.LevelSlice, disableIsCompacting bool, ) (slice manifest.LevelSlice, isCompacting bool) { diff --git a/docs/range_deletions.md b/docs/range_deletions.md index e1c9ac6564..19b8f06a38 100644 --- a/docs/range_deletions.md +++ b/docs/range_deletions.md @@ -1,5 +1,9 @@ # Range Deletions +TODO: The following explanation of range deletions does not take into account +the recent change to prohibit splitting of a user key between sstables. This +change simplifies the logic, removing 'improperly truncated range tombstones.' + TODO: The following explanation of range deletions ignores the kind/trailer that appears at the end of keys after the sequence number. This should be harmless but need to add a justification on why diff --git a/internal/keyspan/fragmenter.go b/internal/keyspan/fragmenter.go index 76e5319443..276709a146 100644 --- a/internal/keyspan/fragmenter.go +++ b/internal/keyspan/fragmenter.go @@ -88,8 +88,8 @@ type Fragmenter struct { // flushed. All pending spans have the same Start.UserKey. pending []Span // doneBuf is used to buffer completed span fragments when flushing to a - // specific key (e.g. FlushTo). It is cached in the Fragmenter to allow - // reuse. + // specific key (e.g. TruncateAndFlushTo). It is cached in the Fragmenter to + // allow reuse. doneBuf []Span // sortBuf is used to sort fragments by end key when flushing. sortBuf spansByEndKey @@ -266,88 +266,15 @@ func (f *Fragmenter) Empty() bool { return f.finished || len(f.pending) == 0 } -// FlushTo flushes all of the fragments with a start key <= key. Used during -// compaction to force emitting of spans which straddle an sstable boundary. -// Note that the emitted spans are not truncated to the specified key. Consider +// TruncateAndFlushTo flushes all of the fragments with a start key <= key, +// truncating spans to the specified end key. Used during compaction to force +// emitting of spans which straddle an sstable boundary. Consider // the scenario: // // a---------k#10 // f#8 // f#7 -// -// If the compaction logic splits f#8 and f#7 into different sstables, we can't -// truncate the span [a,k) at f. Doing so could produce an sstable with the -// records: -// -// a----f#10 -// f#8 -// -// The span [a,f) does not cover the key f. -func (f *Fragmenter) FlushTo(key []byte) { - if f.finished { - panic("pebble: span fragmenter already finished") - } - - if f.flushedKey != nil { - switch c := f.Cmp(key, f.flushedKey); { - case c < 0: - panic(fmt.Sprintf("pebble: flush-to key (%s) < flushed key (%s)", - f.Format(key), f.Format(f.flushedKey))) - } - } - f.flushedKey = append(f.flushedKey[:0], key...) - - if len(f.pending) > 0 { - // Since all of the pending spans have the same start key, we only need - // to compare against the first one. - switch c := f.Cmp(f.pending[0].Start.UserKey, key); { - case c > 0: - panic(fmt.Sprintf("pebble: keys must be in order: %s > %s", - f.Format(f.pending[0].Start.UserKey), f.Format(key))) - } - // Note that we explicitly do not return early here if Start.UserKey == - // key. Similar to the scenario described above, consider: - // - // f----k#10 - // f#8 - // f#7 - // - // If the compaction logic splits f#8 and f#7 into different sstables, we - // have to emit the span [f,k) in both sstables. - } - - // At this point we know that the new start key is greater than the pending - // spans start keys. We flush all span fragments with a start key - // <= key. - f.flush(f.pending, key) - - // Truncate the pending spans to start with key, filtering any which - // would become empty. - pending := f.pending - f.pending = f.pending[:0] - for _, s := range pending { - if f.Cmp(key, s.End) < 0 { - // s: a--+--e - // new: c------ - f.pending = append(f.pending, Span{ - Start: base.MakeInternalKey(key, s.Start.SeqNum(), s.Start.Kind()), - End: s.End, - Value: s.Value, - }) - } - } -} - -// TruncateAndFlushTo is similar to FlushTo, except it also truncates spans to -// the specified end key by calling truncateAndFlush. Only called in compactions -// where we can guarantee that all versions of UserKeys < key have been written, -// or in other words, where we can ensure we don't split a user key across two -// sstables. Going back to the scenario from above: -// -// a---------k#10 -// f#8 -// f#7 -// +/// // Let's say the next user key after f is g. Calling TruncateAndFlushTo(g) will // flush this span: // diff --git a/internal/keyspan/fragmenter_test.go b/internal/keyspan/fragmenter_test.go index 16782b221f..86d9cfb3d5 100644 --- a/internal/keyspan/fragmenter_test.go +++ b/internal/keyspan/fragmenter_test.go @@ -45,14 +45,7 @@ func buildSpans( }, } for _, line := range strings.Split(s, "\n") { - if strings.HasPrefix(line, "flush-to ") { - parts := strings.Split(line, " ") - if len(parts) != 2 { - t.Fatalf("expected 2 components, but found %d: %s", len(parts), line) - } - f.FlushTo([]byte(parts[1])) - continue - } else if strings.HasPrefix(line, "truncate-and-flush-to ") { + if strings.HasPrefix(line, "truncate-and-flush-to ") { parts := strings.Split(line, " ") if len(parts) != 2 { t.Fatalf("expected 2 components, but found %d: %s", len(parts), line) @@ -204,30 +197,6 @@ func TestFragmenterDeleted(t *testing.T) { }) } -func TestFragmenterFlushTo(t *testing.T) { - cmp := base.DefaultComparer.Compare - fmtKey := base.DefaultComparer.FormatKey - - datadriven.RunTest(t, "testdata/fragmenter_flush_to", func(d *datadriven.TestData) string { - switch d.Cmd { - case "build": - return func() (result string) { - defer func() { - if r := recover(); r != nil { - result = fmt.Sprint(r) - } - }() - - spans := buildSpans(t, cmp, fmtKey, d.Input, base.InternalKeyKindRangeDelete) - return formatSpans(spans) - }() - - default: - return fmt.Sprintf("unknown command: %s", d.Cmd) - } - }) -} - func TestFragmenterTruncateAndFlushTo(t *testing.T) { cmp := base.DefaultComparer.Compare fmtKey := base.DefaultComparer.FormatKey diff --git a/internal/keyspan/testdata/fragmenter_flush_to b/internal/keyspan/testdata/fragmenter_flush_to deleted file mode 100644 index a4b935358a..0000000000 --- a/internal/keyspan/testdata/fragmenter_flush_to +++ /dev/null @@ -1,107 +0,0 @@ -build -2: a--c -flush-to c -1: b--d ----- -pebble: start key (b) < flushed key (c) - -build -flush-to c -1: b--d ----- -pebble: start key (b) < flushed key (c) - -build -flush-to c -flush-to b ----- -pebble: flush-to key (b) < flushed key (c) - -build -3: a--d -2: d--g -flush-to c ----- -pebble: flush-to key (c) < flushed key (d) - -# Note that the duplication of the fragments is expected. The flush-to -# is emitting tombstones to different sstables. - -build -3: a--d -flush-to a ----- -3: a--d -3: a--d - -build -3: a--d -2: d--g -flush-to d ----- -3: a--d -2: d--g -2: d--g - -build -2: a----f -flush-to c ----- -2: a----f -2: c--f - -build -2: a----f -flush-to f ----- -2: a----f - -build -3: a-c -1: a-----g -flush-to d ----- -3: a-c -1: a-c -1: c---g -1: d--g - -build -2: a---e -1: a------h -flush-to c ----- -2: a---e -1: a---e -2: c-e -1: c-e -1: e--h - -build -3: a-c -2: a---e -1: a-----g -flush-to d -3: d----i ----- -3: a-c -2: a-c -1: a-c -2: c-e -1: c-e -3: de -2: de -1: de -3: e-g -1: e-g -3: g-i - -build -3: a-c -2: a-----g -flush-to e ----- -3: a-c -2: a-c -2: c---g -2: e-g diff --git a/internal/keyspan/testdata/fragmenter_values b/internal/keyspan/testdata/fragmenter_values index 8e2e5fcbc8..7462ae8b4b 100644 --- a/internal/keyspan/testdata/fragmenter_values +++ b/internal/keyspan/testdata/fragmenter_values @@ -63,26 +63,3 @@ truncate-and-flush-to d 3: e-g orange 1: e-g coconut 3: g-i orange - -# NB: Unlike the above truncate-and-flush-to calls, a flush-to does not truncate -# the end boundary. In this case, the fragments beginning at `c` are not -# truncated to `d`, they're flushed with the bounadries formed by fragmentation -# (`e`) -build -3: a-c apple -2: a---e banana -1: a-----g coconut -flush-to d -3: d----i orange ----- -3: a-c apple -2: a-c banana -1: a-c coconut -2: c-e banana -1: c-e coconut -3: de orange -2: de banana -1: de coconut -3: e-g orange -1: e-g coconut -3: g-i orange diff --git a/range_del_test.go b/range_del_test.go index e2d7389c75..85dd7c3cda 100644 --- a/range_del_test.go +++ b/range_del_test.go @@ -267,20 +267,18 @@ func TestRangeDelCompactionTruncation(t *testing.T) { 1: 000008:[a#3,RANGEDEL-b#72057594037927935,RANGEDEL] 2: - 000012:[b#4,SET-b#4,SET] - 000013:[b#3,RANGEDEL-c#72057594037927935,RANGEDEL] + 000012:[b#4,SET-c#72057594037927935,RANGEDEL] 3: - 000014:[c#5,SET-d#72057594037927935,RANGEDEL] + 000013:[c#5,SET-d#72057594037927935,RANGEDEL] `) } else { expectLSM(` 1: 000008:[a#3,RANGEDEL-b#72057594037927935,RANGEDEL] 2: - 000012:[b#4,SETWITHDEL-b#4,SETWITHDEL] - 000013:[b#3,RANGEDEL-c#72057594037927935,RANGEDEL] + 000012:[b#4,SETWITHDEL-c#72057594037927935,RANGEDEL] 3: - 000014:[c#5,SET-d#72057594037927935,RANGEDEL] + 000013:[c#5,SET-d#72057594037927935,RANGEDEL] `) } @@ -359,17 +357,15 @@ func TestRangeDelCompactionTruncation2(t *testing.T) { require.NoError(t, d.Compact([]byte("b"), []byte("b\x00"))) expectLSM(` 6: - 000008:[a#3,RANGEDEL-b#2,SET] - 000009:[b#1,RANGEDEL-d#72057594037927935,RANGEDEL] + 000008:[a#3,RANGEDEL-d#72057594037927935,RANGEDEL] `) require.NoError(t, d.Set([]byte("c"), bytes.Repeat([]byte("d"), 100), nil)) require.NoError(t, d.Compact([]byte("c"), []byte("c\x00"))) expectLSM(` 6: - 000012:[a#3,RANGEDEL-b#2,SET] - 000013:[b#1,RANGEDEL-c#72057594037927935,RANGEDEL] - 000014:[c#4,SET-d#72057594037927935,RANGEDEL] + 000011:[a#3,RANGEDEL-c#72057594037927935,RANGEDEL] + 000012:[c#4,SET-d#72057594037927935,RANGEDEL] `) } @@ -435,8 +431,7 @@ func TestRangeDelCompactionTruncation3(t *testing.T) { } expectLSM(` 3: - 000012:[a#3,RANGEDEL-b#2,SET] - 000013:[b#1,RANGEDEL-d#72057594037927935,RANGEDEL] + 000008:[a#3,RANGEDEL-d#72057594037927935,RANGEDEL] `) require.NoError(t, d.Set([]byte("c"), bytes.Repeat([]byte("d"), 100), nil)) @@ -444,19 +439,18 @@ func TestRangeDelCompactionTruncation3(t *testing.T) { require.NoError(t, d.Compact([]byte("c"), []byte("c\x00"))) expectLSM(` 3: - 000017:[a#3,RANGEDEL-b#2,SET] - 000018:[b#1,RANGEDEL-c#72057594037927935,RANGEDEL] + 000012:[a#3,RANGEDEL-c#72057594037927935,RANGEDEL] 4: - 000019:[c#4,SET-d#72057594037927935,RANGEDEL] + 000013:[c#4,SET-d#72057594037927935,RANGEDEL] `) require.NoError(t, d.Compact([]byte("c"), []byte("c\x00"))) expectLSM(` 3: - 000017:[a#3,RANGEDEL-b#2,SET] - 000018:[b#1,RANGEDEL-c#72057594037927935,RANGEDEL] + 000012:[a#3,RANGEDEL-c#72057594037927935,RANGEDEL] 5: - 000019:[c#4,SET-d#72057594037927935,RANGEDEL]`) + 000013:[c#4,SET-d#72057594037927935,RANGEDEL] +`) if _, _, err := d.Get([]byte("b")); err != ErrNotFound { t.Fatalf("expected not found, but found %v", err) @@ -465,10 +459,9 @@ func TestRangeDelCompactionTruncation3(t *testing.T) { require.NoError(t, d.Compact([]byte("a"), []byte("a\x00"))) expectLSM(` 4: - 000020:[a#3,RANGEDEL-b#2,SET] - 000021:[b#1,RANGEDEL-c#72057594037927935,RANGEDEL] + 000012:[a#3,RANGEDEL-c#72057594037927935,RANGEDEL] 5: - 000019:[c#4,SET-d#72057594037927935,RANGEDEL] + 000013:[c#4,SET-d#72057594037927935,RANGEDEL] `) if v, _, err := d.Get([]byte("b")); err != ErrNotFound { diff --git a/testdata/compaction_find_grandparent_limit b/testdata/compaction_find_grandparent_limit index 6950848fe8..e1c88ba1c2 100644 --- a/testdata/compaction_find_grandparent_limit +++ b/testdata/compaction_find_grandparent_limit @@ -17,27 +17,29 @@ e-f 2 compact max-overlap=1 a b c d e f ---- -a-b -c-d -e-f +a-c +d-e +f-f compact max-overlap=2 a b c d e f ---- -a-d -e-f +a-c +d-e +f-f compact max-overlap=4 a b c d e f ---- -a-f +a-e +f-f compact max-overlap=1 b c d e f g h i j ---- -b-d -e-f -g-j +b-c +d-e +f-j compact max-overlap=1 a g h i j @@ -54,8 +56,8 @@ e-f compact max-overlap=1 c d e f ---- -c-d -e-f +c-e +f-f # Unequal size grandparents define @@ -67,13 +69,15 @@ e-f 3 compact max-overlap=1 a b c d e f ---- -a-d -e-f +a-c +d-e +f-f compact max-overlap=3 a b c d e f ---- -a-f +a-e +f-f # Unequal size grandparents define @@ -85,12 +89,12 @@ e-f 1 compact max-overlap=1 a b c d e f ---- -a-b -c-d -e-f +a-c +d-e +f-f compact max-overlap=3 a b c d e f ---- -a-d -e-f +a-c +d-f diff --git a/testdata/compaction_iter b/testdata/compaction_iter index 34bf1c9599..2f17e2bcff 100644 --- a/testdata/compaction_iter +++ b/testdata/compaction_iter @@ -553,7 +553,7 @@ tombstones ---- a#10,15:k f#9,1:f -a-k#10 +a-f#10 . f#8,1:f f-k#10 @@ -574,7 +574,6 @@ tombstones ---- f#10,15:k f#9,1:f -f-k#10 . f#8,1:f f-k#10 @@ -600,7 +599,6 @@ b#2,15:d c#3,15:e d#4,1:d b-c#2 -c-d#3 . c-d#3 d-e#3 @@ -619,8 +617,6 @@ b#2,15:d c#3,15:e d#4,1:d b-c#2 -c-d#3 -c-d#2 . c-d#3 c-d#2 @@ -643,7 +639,7 @@ tombstones a#1,1:a b#2,15:d c#4,1:d -b-d#2 +b-c#2 . c-d#2 . diff --git a/testdata/compaction_iter_set_with_del b/testdata/compaction_iter_set_with_del index da78252543..796984e0c7 100644 --- a/testdata/compaction_iter_set_with_del +++ b/testdata/compaction_iter_set_with_del @@ -553,7 +553,7 @@ tombstones ---- a#10,15:k f#9,1:f -a-k#10 +a-f#10 . f#8,1:f f-k#10 @@ -574,7 +574,6 @@ tombstones ---- f#10,15:k f#9,1:f -f-k#10 . f#8,1:f f-k#10 @@ -600,7 +599,6 @@ b#2,15:d c#3,15:e d#4,1:d b-c#2 -c-d#3 . c-d#3 d-e#3 @@ -619,8 +617,6 @@ b#2,15:d c#3,15:e d#4,1:d b-c#2 -c-d#3 -c-d#2 . c-d#3 c-d#2 @@ -643,7 +639,7 @@ tombstones a#1,1:a b#2,15:d c#4,1:d -b-d#2 +b-c#2 . c-d#2 . diff --git a/testdata/manual_compaction b/testdata/manual_compaction index d6111e3c17..3ff926f98d 100644 --- a/testdata/manual_compaction +++ b/testdata/manual_compaction @@ -92,9 +92,8 @@ range-deletions-bytes-estimate: 1552 compact a-e L1 ---- 2: - 000008:[a#3,SET-b#72057594037927935,RANGEDEL] - 000009:[b#2,RANGEDEL-d#72057594037927935,RANGEDEL] - 000010:[d#2,RANGEDEL-e#72057594037927935,RANGEDEL] + 000008:[a#3,SET-c#72057594037927935,RANGEDEL] + 000009:[c#2,RANGEDEL-e#72057594037927935,RANGEDEL] 3: 000006:[a#0,SET-b#0,SET] 000007:[c#0,SET-d#0,SET] @@ -105,7 +104,7 @@ wait-pending-table-stats num-entries: 2 num-deletions: 1 point-deletions-bytes-estimate: 0 -range-deletions-bytes-estimate: 31 +range-deletions-bytes-estimate: 776 # Same as above, except range tombstone covers multiple grandparent file boundaries. @@ -140,9 +139,9 @@ L3 compact a-e L1 ---- 2: - 000010:[a#3,SET-b#72057594037927935,RANGEDEL] - 000011:[b#2,RANGEDEL-d#72057594037927935,RANGEDEL] - 000012:[d#2,RANGEDEL-f#72057594037927935,RANGEDEL] + 000010:[a#3,SET-c#72057594037927935,RANGEDEL] + 000011:[c#2,RANGEDEL-e#72057594037927935,RANGEDEL] + 000012:[e#2,RANGEDEL-f#72057594037927935,RANGEDEL] 000013:[f#2,RANGEDEL-g#72057594037927935,RANGEDEL] 3: 000006:[a#0,SET-b#0,SET] @@ -215,9 +214,8 @@ L3 compact a-e L1 ---- 2: - 000008:[a#3,RANGEDEL-b#72057594037927935,RANGEDEL] - 000009:[b#3,RANGEDEL-d#72057594037927935,RANGEDEL] - 000010:[d#3,RANGEDEL-e#72057594037927935,RANGEDEL] + 000008:[a#3,RANGEDEL-c#72057594037927935,RANGEDEL] + 000009:[c#3,RANGEDEL-e#72057594037927935,RANGEDEL] 3: 000006:[a#0,SET-b#0,SET] 000007:[c#0,SET-d#0,SET] @@ -446,8 +444,8 @@ L3 compact a-e L1 ---- 2: - 000007:[a#4,SET-b#2,SET] - 000008:[b#1,RANGEDEL-e#72057594037927935,RANGEDEL] + 000007:[a#4,SET-a#4,SET] + 000008:[b#3,RANGEDEL-e#72057594037927935,RANGEDEL] 3: 000006:[b#1,SET-b#1,SET] @@ -483,8 +481,8 @@ L3 compact a-e L1 ---- 2: - 000007:[a#4,SET-b#3,SET] - 000008:[b#2,RANGEDEL-e#72057594037927935,RANGEDEL] + 000007:[a#4,SET-a#4,SET] + 000008:[b#3,RANGEDEL-e#72057594037927935,RANGEDEL] 3: 000006:[b#1,SET-b#1,SET] @@ -524,8 +522,8 @@ L3 compact a-e L1 ---- 2: - 000007:[a#4,SET-b#4,SET] - 000008:[b#2,RANGEDEL-e#72057594037927935,RANGEDEL] + 000007:[a#4,SET-a#4,SET] + 000008:[b#4,SET-e#72057594037927935,RANGEDEL] 3: 000006:[b#1,SET-b#1,SET] @@ -577,7 +575,8 @@ a:3 compact a-e L1 ---- 2: - 000007:[a#3,SET-e#72057594037927935,RANGEDEL] + 000007:[a#3,SET-c#72057594037927935,RANGEDEL] + 000008:[c#3,RANGEDEL-e#72057594037927935,RANGEDEL] 3: 000005:[a#2,RANGEDEL-b#72057594037927935,RANGEDEL] 000006:[c#0,SET-d#0,SET] @@ -612,8 +611,7 @@ L3 compact a-f L1 ---- 2: - 000006:[a#3,SET-c#72057594037927935,RANGEDEL] - 000007:[c#3,RANGEDEL-e#72057594037927935,RANGEDEL] + 000006:[a#3,SET-e#72057594037927935,RANGEDEL] 3: 000005:[a#2,RANGEDEL-b#72057594037927935,RANGEDEL] diff --git a/testdata/manual_compaction_set_with_del b/testdata/manual_compaction_set_with_del index e73e310266..1a907f01d2 100644 --- a/testdata/manual_compaction_set_with_del +++ b/testdata/manual_compaction_set_with_del @@ -92,9 +92,8 @@ range-deletions-bytes-estimate: 1552 compact a-e L1 ---- 2: - 000008:[a#3,SETWITHDEL-b#72057594037927935,RANGEDEL] - 000009:[b#2,RANGEDEL-d#72057594037927935,RANGEDEL] - 000010:[d#2,RANGEDEL-e#72057594037927935,RANGEDEL] + 000008:[a#3,SETWITHDEL-c#72057594037927935,RANGEDEL] + 000009:[c#2,RANGEDEL-e#72057594037927935,RANGEDEL] 3: 000006:[a#0,SET-b#0,SET] 000007:[c#0,SET-d#0,SET] @@ -105,7 +104,7 @@ wait-pending-table-stats num-entries: 2 num-deletions: 1 point-deletions-bytes-estimate: 0 -range-deletions-bytes-estimate: 31 +range-deletions-bytes-estimate: 776 # Same as above, except range tombstone covers multiple grandparent file boundaries. @@ -140,9 +139,9 @@ L3 compact a-e L1 ---- 2: - 000010:[a#3,SETWITHDEL-b#72057594037927935,RANGEDEL] - 000011:[b#2,RANGEDEL-d#72057594037927935,RANGEDEL] - 000012:[d#2,RANGEDEL-f#72057594037927935,RANGEDEL] + 000010:[a#3,SETWITHDEL-c#72057594037927935,RANGEDEL] + 000011:[c#2,RANGEDEL-e#72057594037927935,RANGEDEL] + 000012:[e#2,RANGEDEL-f#72057594037927935,RANGEDEL] 000013:[f#2,RANGEDEL-g#72057594037927935,RANGEDEL] 3: 000006:[a#0,SET-b#0,SET] @@ -215,9 +214,8 @@ L3 compact a-e L1 ---- 2: - 000008:[a#3,RANGEDEL-b#72057594037927935,RANGEDEL] - 000009:[b#3,RANGEDEL-d#72057594037927935,RANGEDEL] - 000010:[d#3,RANGEDEL-e#72057594037927935,RANGEDEL] + 000008:[a#3,RANGEDEL-c#72057594037927935,RANGEDEL] + 000009:[c#3,RANGEDEL-e#72057594037927935,RANGEDEL] 3: 000006:[a#0,SET-b#0,SET] 000007:[c#0,SET-d#0,SET] @@ -446,8 +444,8 @@ L3 compact a-e L1 ---- 2: - 000007:[a#4,SET-b#2,SET] - 000008:[b#1,RANGEDEL-e#72057594037927935,RANGEDEL] + 000007:[a#4,SET-a#4,SET] + 000008:[b#3,RANGEDEL-e#72057594037927935,RANGEDEL] 3: 000006:[b#1,SET-b#1,SET] @@ -483,8 +481,8 @@ L3 compact a-e L1 ---- 2: - 000007:[a#4,SET-b#3,SET] - 000008:[b#2,RANGEDEL-e#72057594037927935,RANGEDEL] + 000007:[a#4,SET-a#4,SET] + 000008:[b#3,RANGEDEL-e#72057594037927935,RANGEDEL] 3: 000006:[b#1,SET-b#1,SET] @@ -524,8 +522,8 @@ L3 compact a-e L1 ---- 2: - 000007:[a#4,SET-b#4,SET] - 000008:[b#2,RANGEDEL-e#72057594037927935,RANGEDEL] + 000007:[a#4,SET-a#4,SET] + 000008:[b#4,SET-e#72057594037927935,RANGEDEL] 3: 000006:[b#1,SET-b#1,SET] @@ -577,7 +575,8 @@ a:3 compact a-e L1 ---- 2: - 000007:[a#3,SET-e#72057594037927935,RANGEDEL] + 000007:[a#3,SET-c#72057594037927935,RANGEDEL] + 000008:[c#3,RANGEDEL-e#72057594037927935,RANGEDEL] 3: 000005:[a#2,RANGEDEL-b#72057594037927935,RANGEDEL] 000006:[c#0,SET-d#0,SET] @@ -612,8 +611,7 @@ L3 compact a-f L1 ---- 2: - 000006:[a#3,SET-c#72057594037927935,RANGEDEL] - 000007:[c#3,RANGEDEL-e#72057594037927935,RANGEDEL] + 000006:[a#3,SET-e#72057594037927935,RANGEDEL] 3: 000005:[a#2,RANGEDEL-b#72057594037927935,RANGEDEL]