diff --git a/compaction.go b/compaction.go index 1511b94259..ec0f68607f 100644 --- a/compaction.go +++ b/compaction.go @@ -247,13 +247,7 @@ func (f *fileSizeSplitter) shouldSplitBefore(key *InternalKey, tw *sstable.Write // grandparent boundary. f.atGrandparentBoundary = false - // If the key is a range tombstone, the EstimatedSize may not grow right - // away when a range tombstone is added to the fragmenter: It's dependent on - // whether or not the this new range deletion will start a new fragment. - // Range deletions are rare, so we choose to simply not split yet. - // TODO(jackson): Reconsider this, and consider range keys too as a part of - // #2321. - if key.Kind() == InternalKeyKindRangeDelete || tw == nil { + if tw == nil { return noSplit } @@ -625,12 +619,9 @@ type compaction struct { // The range key fragmenter. Similar to rangeDelFrag in that it gets range // keys from the compaction iter and fragments them for output to files. rangeKeyFrag keyspan.Fragmenter - // The range deletion tombstone iterator, that merges and fragments - // tombstones across levels. This iterator is included within the compaction - // input iterator as a single level. - // TODO(jackson): Remove this when the refactor of FragmentIterator, - // InterleavingIterator, etc is complete. - rangeDelIter keyspan.InternalIteratorShim + // rangeDelInterlaving is an interleaving iterator for range deletions, that + // interleaves range tombstones among the point keys. + rangeDelInterleaving keyspan.InterleavingIter // rangeKeyInterleaving is the interleaving iter for range keys. rangeKeyInterleaving keyspan.InterleavingIter @@ -1377,6 +1368,15 @@ func (c *compaction) newInputIter( } } + // If there's only one constituent point iterator, we can avoid the overhead + // of a *mergingIter. This is possible, for example, when performing a flush + // of a single memtable. Otherwise, combine all the iterators into a merging + // iter. + iter := iters[0] + if len(iters) > 1 { + iter = newMergingIter(c.logger, &c.stats, c.cmp, nil, iters...) + } + // In normal operation, levelIter iterates over the point operations in a // level, and initializes a rangeDelIter pointer for the range deletions in // each table. During compaction, we want to iterate over the merged view of @@ -1384,29 +1384,19 @@ func (c *compaction) newInputIter( // levelIter per level to iterate over the point operations, and collect up // all the range deletion files. // - // The range deletion levels are first combined with a keyspan.MergingIter - // (currently wrapped by a keyspan.InternalIteratorShim to satisfy the - // internal iterator interface). The resulting merged rangedel iterator is - // then included with the point levels in a single mergingIter. - // - // Combine all the rangedel iterators using a keyspan.MergingIterator and a - // InternalIteratorShim so that the range deletions may be interleaved in - // the compaction input. - // TODO(jackson): Replace the InternalIteratorShim with an interleaving - // iterator. + // The range deletion levels are combined with a keyspan.MergingIter. The + // resulting merged rangedel iterator is then included using an + // InterleavingIter. + // TODO(jackson): Consider using a defragmenting iterator to stitch together + // logical range deletions that were fragmented due to previous file + // boundaries. if len(rangeDelIters) > 0 { - c.rangeDelIter.Init(c.cmp, rangeDelIters...) - iters = append(iters, &c.rangeDelIter) + mi := &keyspan.MergingIter{} + mi.Init(c.cmp, keyspan.NoopTransform, new(keyspan.MergingBuffers), rangeDelIters...) + c.rangeDelInterleaving.Init(c.comparer, iter, mi, keyspan.InterleavingIterOpts{}) + iter = &c.rangeDelInterleaving } - // If there's only one constituent point iterator, we can avoid the overhead - // of a *mergingIter. This is possible, for example, when performing a flush - // of a single memtable. Otherwise, combine all the iterators into a merging - // iter. - iter := iters[0] - if len(iters) > 0 { - iter = newMergingIter(c.logger, &c.stats, c.cmp, nil, iters...) - } // If there are range key iterators, we need to combine them using // keyspan.MergingIter, and then interleave them among the points. if len(rangeKeyIters) > 0 { @@ -3402,7 +3392,7 @@ func (d *DB) runCompaction( // The interleaved range deletion might only be one of many with // these bounds. Some fragmenting is performed ahead of time by // keyspan.MergingIter. - if s := c.rangeDelIter.Span(); !s.Empty() { + if s := c.rangeDelInterleaving.Span(); !s.Empty() { // The memory management here is subtle. Range deletions // blocks do NOT use prefix compression, which ensures that // range deletion spans' memory is available as long we keep diff --git a/compaction_iter.go b/compaction_iter.go index c86173e711..4a10e65dbb 100644 --- a/compaction_iter.go +++ b/compaction_iter.go @@ -180,16 +180,9 @@ type compactionIter struct { iterKey *InternalKey iterValue []byte iterStripeChange stripeChangeType - // `skip` indicates whether the remaining skippable entries in the current - // snapshot stripe should be skipped or processed. An example of a non- - // skippable entry is a range tombstone as we need to return it from the - // `compactionIter`, even if a key covering its start key has already been - // seen in the same stripe. `skip` has no effect when `pos == iterPosNext`. - // - // TODO(jackson): If we use keyspan.InterleavingIter for range deletions, - // like we do for range keys, the only remaining 'non-skippable' key is - // the invalid key. We should be able to simplify this logic and remove this - // field. + // `skip` indicates whether the remaining entries in the current snapshot + // stripe should be skipped or processed. `skip` has no effect when `pos == + // iterPosNext`. skip bool // `pos` indicates the iterator position at the top of `Next()`. Its type's // (`iterPos`) values take on the following meanings in the context of @@ -206,13 +199,6 @@ type compactionIter struct { // compaction iterator was only returned because an open snapshot prevents // its elision. This field only applies to point keys, and not to range // deletions or range keys. - // - // For MERGE, it is possible that doing the merge is interrupted even when - // the next point key is in the same stripe. This can happen if the loop in - // mergeNext gets interrupted by sameStripeNonSkippable. - // sameStripeNonSkippable occurs due to RANGEDELs that sort before - // SET/MERGE/DEL with the same seqnum, so the RANGEDEL does not necessarily - // delete the subsequent SET/MERGE/DEL keys. snapshotPinned bool // forceObsoleteDueToRangeDel is set to true in a subset of the cases that // snapshotPinned is true. This value is true when the point is obsolete due @@ -249,6 +235,8 @@ type compactionIter struct { frontiers frontiers // Reference to the range deletion tombstone fragmenter (e.g., // `compaction.rangeDelFrag`). + // TODO(jackson): We can eliminate range{Del,Key}Frag now that fragmentation + // is performed upfront by keyspan.MergingIters. rangeDelFrag *keyspan.Fragmenter rangeKeyFrag *keyspan.Fragmenter // The fragmented tombstones. @@ -339,7 +327,7 @@ func (i *compactionIter) Next() (*InternalKey, []byte) { return nil, nil } - // Prior to this call to `Next()` we are in one of four situations with + // Prior to this call to `Next()` we are in one of three situations with // respect to `iterKey` and related state: // // - `!skip && pos == iterPosNext`: `iterKey` is already at the next key. @@ -348,24 +336,6 @@ func (i *compactionIter) Next() (*InternalKey, []byte) { // snapshot stripe. // - `skip && pos == iterPosCurForward`: We are at the key that has been returned. // To move forward we skip skippable entries in the stripe. - // - `skip && pos == iterPosNext && i.iterStripeChange == sameStripeNonSkippable`: - // This case may occur when skipping within a snapshot stripe and we - // encounter either: - // a) an invalid key kind; The previous call will have returned - // whatever key it was processing and deferred handling of the - // invalid key to this invocation of Next(). We're responsible for - // ignoring skip=true and falling into the invalid key kind case - // down below. - // b) an interleaved range delete; This is a wart of the current code - // structure. While skipping within a snapshot stripe, a range - // delete interleaved at its start key and sequence number - // interrupts the sequence of point keys. After we return the range - // delete to the caller, we need to pick up skipping at where we - // left off, so we preserve skip=true. - // TODO(jackson): This last case is confusing and can be removed if we - // interleave range deletions at the maximal sequence number using the - // keyspan interleaving iterator. This is the treatment given to range - // keys today. if i.pos == iterPosCurForward { if i.skip { i.skipInStripe() @@ -373,9 +343,7 @@ func (i *compactionIter) Next() (*InternalKey, []byte) { i.nextInStripe() } } else if i.skip { - if i.iterStripeChange != sameStripeNonSkippable { - panic(errors.AssertionFailedf("compaction iterator has skip=true, but iterator is at iterPosNext")) - } + panic(errors.AssertionFailedf("compaction iterator has skip=true, but iterator is at iterPosNext")) } i.pos = iterPosCurForward @@ -395,16 +363,20 @@ func (i *compactionIter) Next() (*InternalKey, []byte) { if i.iterKey.Kind() == InternalKeyKindRangeDelete || rangekey.IsRangeKey(i.iterKey.Kind()) { // Return the span so the compaction can use it for file truncation and add - // it to the relevant fragmenter. We do not set `skip` to true before - // returning as there may be a forthcoming point key with the same user key - // and sequence number. Such a point key must be visible (i.e., not skipped + // it to the relevant fragmenter. In the case of range deletions, we do not + // set `skip` to true before returning as there may be any number of point + // keys with the same user key and sequence numbers ≥ the range deletion's + // sequence number. Such point keys must be visible (i.e., not skipped // over) since we promise point keys are not deleted by range tombstones at - // the same sequence number. - // - // Although, note that `skip` may already be true before reaching here - // due to an earlier key in the stripe. Then it is fine to leave it set - // to true, as the earlier key must have had a higher sequence number. + // the same sequence number (or higher). // + // Note that `skip` must already be false here, because range keys and range + // deletions are interleaved at the maximal sequence numbers and neither will + // set `skip`=true. + if i.skip { + panic(errors.AssertionFailedf("pebble: compaction iterator: skip unexpectedly true")) + } + // NOTE: there is a subtle invariant violation here in that calling // saveKey and returning a reference to the temporary slice violates // the stability guarantee for range deletion keys. A potential @@ -526,9 +498,8 @@ func (i *compactionIter) Next() (*InternalKey, []byte) { origSnapshotIdx := i.curSnapshotIdx var valueMerger ValueMerger valueMerger, i.err = i.merge(i.iterKey.UserKey, i.iterValue) - var change stripeChangeType if i.err == nil { - change = i.mergeNext(valueMerger) + i.mergeNext(valueMerger) } var needDelete bool if i.err == nil { @@ -553,20 +524,8 @@ func (i *compactionIter) Next() (*InternalKey, []byte) { } continue } - // A non-skippable entry does not necessarily cover later merge - // operands, so we must not zero the current merge result's seqnum. - // - // For example, suppose the forthcoming two keys are a range - // tombstone, `[a, b)#3`, and a merge operand, `a#3`. Recall that - // range tombstones do not cover point keys at the same seqnum, so - // `a#3` is not deleted. The range tombstone will be seen first due - // to its larger value type. Since it is a non-skippable key, the - // current merge will not include `a#3`. If we zeroed the current - // merge result's seqnum, then it would conflict with the upcoming - // merge including `a#3`, whose seqnum will also be zeroed. - if change != sameStripeNonSkippable { - i.maybeZeroSeqnum(origSnapshotIdx) - } + + i.maybeZeroSeqnum(origSnapshotIdx) return &i.key, i.value } if i.err != nil { @@ -618,18 +577,13 @@ func (i *compactionIter) skipInStripe() { i.skip = true // TODO(sumeer): we can avoid the overhead of calling i.rangeDelFrag.Covers, // in this case of nextInStripe, since we are skipping all of them anyway. - for i.nextInStripe() == sameStripeSkippable { + for i.nextInStripe() == sameStripe { if i.err != nil { panic(i.err) } } - // Reset skip if we landed outside the original stripe. Otherwise, we landed - // in the same stripe on a non-skippable key. In that case we should preserve - // `i.skip == true` such that later keys in the stripe will continue to be - // skipped. - if i.iterStripeChange == newStripeNewKey || i.iterStripeChange == newStripeSameKey { - i.skip = false - } + // We landed outside the original stripe, so reset skip. + i.skip = false } func (i *compactionIter) iterNext() bool { @@ -643,24 +597,22 @@ func (i *compactionIter) iterNext() bool { } // stripeChangeType indicates how the snapshot stripe changed relative to the -// previous key. If no change, it also indicates whether the current entry is -// skippable. If the snapshot stripe changed, it also indicates whether the new -// stripe was entered because the iterator progressed onto an entirely new key -// or entered a new stripe within the same key. +// previous key. If the snapshot stripe changed, it also indicates whether the +// new stripe was entered because the iterator progressed onto an entirely new +// key or entered a new stripe within the same key. type stripeChangeType int const ( newStripeNewKey stripeChangeType = iota newStripeSameKey - sameStripeSkippable - sameStripeNonSkippable + sameStripe ) // nextInStripe advances the iterator and returns one of the above const ints // indicating how its state changed. // -// All sameStripeSkippable keys that are covered by a RANGEDEL will be skipped -// and not returned. +// All sameStripe keys that are covered by a RANGEDEL will be skipped and not +// returned. // // Calls to nextInStripe must be preceded by a call to saveKey to retain a // temporary reference to the original key, so that forward iteration can @@ -685,7 +637,16 @@ func (i *compactionIter) nextInStripeHelper() stripeChangeType { } key := i.iterKey - if !i.equal(i.key.UserKey, key.UserKey) { + // Is this a new key? There are two cases: + // + // 1. The new key has a different user key. + // 2. The previous key was an interleaved range deletion or range key + // boundary. These keys are interleaved in the same input iterator + // stream as point keys, but they do not obey the ordinary sequence + // number ordering within a user key. If the previous key was one + // of these keys, we consider the new key a `newStripeNewKey` to + // reflect that it's the beginning of a new stream of point keys. + if i.key.IsExclusiveSentinel() || !i.equal(i.key.UserKey, key.UserKey) { i.curSnapshotIdx, i.curSnapshotSeqNum = snapshotIndex(key.SeqNum(), i.snapshots) return newStripeNewKey } @@ -709,29 +670,20 @@ func (i *compactionIter) nextInStripeHelper() stripeChangeType { i.curSnapshotIdx, i.curSnapshotSeqNum = snapshotIndex(key.SeqNum(), i.snapshots) switch key.Kind() { - case InternalKeyKindRangeDelete: - // Range tombstones need to be exposed by the compactionIter to the upper level - // `compaction` object, so return them regardless of whether they are in the same - // snapshot stripe. - if i.curSnapshotIdx == origSnapshotIdx { - return sameStripeNonSkippable - } - return newStripeSameKey - case InternalKeyKindRangeKeySet, InternalKeyKindRangeKeyUnset, InternalKeyKindRangeKeyDelete: - // Range keys are interleaved at the max sequence number for a given user - // key, so we should not see any more range keys in this stripe. + case InternalKeyKindRangeKeySet, InternalKeyKindRangeKeyUnset, InternalKeyKindRangeKeyDelete, + InternalKeyKindRangeDelete: + // Range tombstones and range keys are interleaved at the max + // sequence number for a given user key, and the first key after one + // is always considered a newStripeNewKey, so we should never reach + // this. panic("unreachable") - case InternalKeyKindInvalid: - if i.curSnapshotIdx == origSnapshotIdx { - return sameStripeNonSkippable - } - return newStripeSameKey case InternalKeyKindDelete, InternalKeyKindSet, InternalKeyKindMerge, InternalKeyKindSingleDelete, InternalKeyKindSetWithDelete, InternalKeyKindDeleteSized: // Fall through default: + kind := i.iterKey.Kind() i.iterKey = nil - i.err = base.CorruptionErrorf("invalid internal key kind: %d", errors.Safe(i.iterKey.Kind())) + i.err = base.CorruptionErrorf("invalid internal key kind: %d", errors.Safe(kind)) i.valid = false return newStripeNewKey } @@ -740,7 +692,7 @@ func (i *compactionIter) nextInStripeHelper() stripeChangeType { if i.rangeDelFrag.Covers(*i.iterKey, i.curSnapshotSeqNum) == keyspan.CoversVisibly { continue } - return sameStripeSkippable + return sameStripe } return newStripeSameKey } @@ -767,56 +719,15 @@ func (i *compactionIter) setNext() { // Else, we continue to loop through entries in the stripe looking for a // DEL. Note that we may stop *before* encountering a DEL, if one exists. // - // NB: nextInStripe will skip sameStripeSkippable keys that are visibly - // covered by a RANGEDEL. This can include DELs -- this is fine since such - // DELs don't need to be combined with SET to make SETWITHDEL. + // NB: nextInStripe will skip sameStripe keys that are visibly covered by a + // RANGEDEL. This can include DELs -- this is fine since such DELs don't + // need to be combined with SET to make SETWITHDEL. for { switch i.nextInStripe() { case newStripeNewKey, newStripeSameKey: i.pos = iterPosNext return - case sameStripeNonSkippable: - i.pos = iterPosNext - // We iterated onto a key that we cannot skip. We can - // conservatively transform the original SET into a SETWITHDEL - // as an indication that there *may* still be a DEL/SINGLEDEL - // under this SET, even if we did not actually encounter one. - // - // This is safe to do, as: - // - // - in the case that there *is not* actually a DEL/SINGLEDEL - // under this entry, any SINGLEDEL above this now-transformed - // SETWITHDEL will become a DEL when the two encounter in a - // compaction. The DEL will eventually be elided in a - // subsequent compaction. The cost for ensuring correctness is - // that this entry is kept around for an additional compaction - // cycle(s). - // - // - in the case there *is* indeed a DEL/SINGLEDEL under us - // (but in a different stripe or sstable), then we will have - // already done the work to transform the SET into a - // SETWITHDEL, and we will skip any additional iteration when - // this entry is encountered again in a subsequent compaction. - // - // Ideally, this codepath would be smart enough to handle the - // case of SET <- RANGEDEL <- ... <- DEL/SINGLEDEL <- .... - // This requires preserving any RANGEDEL entries we encounter - // along the way, then emitting the original (possibly - // transformed) key, followed by the RANGEDELs. This requires - // a sizable refactoring of the existing code, as nextInStripe - // currently returns a sameStripeNonSkippable when it - // encounters a RANGEDEL. - // TODO(travers): optimize to handle the RANGEDEL case if it - // turns out to be a performance problem. - i.key.SetKind(InternalKeyKindSetWithDelete) - - // By setting i.skip=true, we are saying that after the - // non-skippable key is emitted (which is likely a RANGEDEL), - // the remaining point keys that share the same user key as this - // saved key should be skipped. - i.skip = true - return - case sameStripeSkippable: + case sameStripe: // We're still in the same stripe. If this is a // DEL/SINGLEDEL/DELSIZED, we stop looking and emit a SETWITHDEL. // Subsequent keys are eligible for skipping. @@ -837,7 +748,7 @@ func (i *compactionIter) setNext() { } } -func (i *compactionIter) mergeNext(valueMerger ValueMerger) stripeChangeType { +func (i *compactionIter) mergeNext(valueMerger ValueMerger) { // Save the current key. i.saveKey() i.valid = true @@ -845,20 +756,20 @@ func (i *compactionIter) mergeNext(valueMerger ValueMerger) stripeChangeType { // Loop looking for older values in the current snapshot stripe and merge // them. for { - if i.nextInStripe() != sameStripeSkippable { + if i.nextInStripe() != sameStripe { i.pos = iterPosNext - return i.iterStripeChange + return } if i.err != nil { panic(i.err) } // NB: MERGE#10+RANGEDEL#9 stays a MERGE, since nextInStripe skips - // sameStripeSkippable keys that are visibly covered by a RANGEDEL. There - // may be MERGE#7 that is invisibly covered and will be preserved, but - // there is no risk that MERGE#10 and MERGE#7 will get merged in the - // future as the RANGEDEL still exists and will be used in user-facing - // reads that see MERGE#10, and will also eventually cause MERGE#7 to be - // deleted in a compaction. + // sameStripe keys that are visibly covered by a RANGEDEL. There may be + // MERGE#7 that is invisibly covered and will be preserved, but there is + // no risk that MERGE#10 and MERGE#7 will get merged in the future as + // the RANGEDEL still exists and will be used in user-facing reads that + // see MERGE#10, and will also eventually cause MERGE#7 to be deleted in + // a compaction. key := i.iterKey switch key.Kind() { case InternalKeyKindDelete, InternalKeyKindSingleDelete, InternalKeyKindDeleteSized: @@ -881,7 +792,7 @@ func (i *compactionIter) mergeNext(valueMerger ValueMerger) stripeChangeType { // use of SingleDelete. i.key.SetKind(InternalKeyKindSetWithDelete) i.skip = true - return sameStripeSkippable + return case InternalKeyKindSet, InternalKeyKindSetWithDelete: // We've hit a Set or SetWithDel value. Merge with the existing @@ -891,11 +802,11 @@ func (i *compactionIter) mergeNext(valueMerger ValueMerger) stripeChangeType { i.err = valueMerger.MergeOlder(i.iterValue) if i.err != nil { i.valid = false - return sameStripeSkippable + return } i.key.SetKind(InternalKeyKindSet) i.skip = true - return sameStripeSkippable + return case InternalKeyKindMerge: // We've hit another Merge value. Merge with the existing value and @@ -903,13 +814,13 @@ func (i *compactionIter) mergeNext(valueMerger ValueMerger) stripeChangeType { i.err = valueMerger.MergeOlder(i.iterValue) if i.err != nil { i.valid = false - return sameStripeSkippable + return } default: i.err = base.CorruptionErrorf("invalid internal key kind: %d", errors.Safe(i.iterKey.Kind())) i.valid = false - return sameStripeSkippable + return } } } @@ -934,7 +845,7 @@ func (i *compactionIter) singleDeleteNext() bool { for { // If we find a key that can't be skipped, return true so that the // caller yields the SingleDelete to the caller. - if i.nextInStripe() != sameStripeSkippable { + if i.nextInStripe() != sameStripe { // This defers additional error checking regarding single delete // invariants to the compaction where the keys with the same user key as // the single delete are in the same stripe. @@ -944,7 +855,7 @@ func (i *compactionIter) singleDeleteNext() bool { if i.err != nil { panic(i.err) } - // INVARIANT: sameStripeSkippable. + // INVARIANT: sameStripe. key := i.iterKey kind := key.Kind() switch kind { @@ -970,33 +881,26 @@ func (i *compactionIter) singleDeleteNext() bool { // since we are at the key after the Set/Merge that was single deleted. change := i.nextInStripe() switch change { - case sameStripeSkippable, newStripeSameKey: + case sameStripe, newStripeSameKey: // On the same user key. nextKind := i.iterKey.Kind() switch nextKind { case InternalKeyKindSet, InternalKeyKindSetWithDelete, InternalKeyKindMerge: if i.singleDeleteInvariantViolationCallback != nil { - // sameStripeSkippable keys returned by nextInStripe() are already + // sameStripe keys returned by nextInStripe() are already // known to not be covered by a RANGEDEL, so it is an invariant // violation. The rare case is newStripeSameKey, where it is a // violation if not covered by a RANGEDEL. - if change == sameStripeSkippable || + if change == sameStripe || i.rangeDelFrag.Covers(*i.iterKey, i.curSnapshotSeqNum) == keyspan.NoCover { i.singleDeleteInvariantViolationCallback(i.key.UserKey) } } - case InternalKeyKindDelete, InternalKeyKindDeleteSized, InternalKeyKindSingleDelete, - InternalKeyKindRangeDelete: + case InternalKeyKindDelete, InternalKeyKindDeleteSized, InternalKeyKindSingleDelete: default: panic(errors.AssertionFailedf( "unexpected internal key kind: %d", errors.Safe(i.iterKey.Kind()))) } - case sameStripeNonSkippable: - // No ability to check whether there is another Set/Merge below with - // the same user key. - // - // TODO(sumeer): once range deletions are interleaved at the maximal - // sequence number, this case will go away. case newStripeNewKey: default: panic("unreachable") @@ -1054,25 +958,7 @@ func (i *compactionIter) skipDueToSingleDeleteElision() { // determined that the tombstone is in the final snapshot stripe, but we // stepped into a new stripe of the same key. panic(errors.AssertionFailedf("eliding single delete followed by same key in new stripe")) - case sameStripeNonSkippable: - // There's a key that we cannot skip. There are two possible cases: - // a. The key is invalid. This is an error. - // b. The key is a range deletion. - // The second case may also be an ineffectual single delete. However, it - // is possible that there is a SET that is at the same seqnum as the - // RANGEDEL, and so is not deleted by that RANGEDEL, and will be deleted - // by this single delete. So we cannot be certain that this is an - // ineffectual single delete. - // - // TODO(sumeer): the existing todo to interleave range deletions at the - // maximal sequence number will allow us to address this ambiguity. - // - // TODO(sumeer): by setting skip to true, the compactionIter is making a - // single delete stronger (like a del), which will hide bugs in the use of - // single delete. - i.skip = true - return - case sameStripeSkippable: + case sameStripe: kind := i.iterKey.Kind() switch kind { case InternalKeyKindDelete, InternalKeyKindDeleteSized, InternalKeyKindSingleDelete: @@ -1113,7 +999,7 @@ func (i *compactionIter) skipDueToSingleDeleteElision() { case newStripeSameKey: panic(errors.AssertionFailedf("eliding single delete followed by same key in new stripe")) case newStripeNewKey: - case sameStripeSkippable: + case sameStripe: // On the same key. nextKind := i.iterKey.Kind() switch nextKind { @@ -1121,18 +1007,11 @@ func (i *compactionIter) skipDueToSingleDeleteElision() { if i.singleDeleteInvariantViolationCallback != nil { i.singleDeleteInvariantViolationCallback(i.key.UserKey) } - case InternalKeyKindDelete, InternalKeyKindDeleteSized, InternalKeyKindSingleDelete, - InternalKeyKindRangeDelete: + case InternalKeyKindDelete, InternalKeyKindDeleteSized, InternalKeyKindSingleDelete: default: panic(errors.AssertionFailedf( "unexpected internal key kind: %d", errors.Safe(i.iterKey.Kind()))) } - case sameStripeNonSkippable: - // No ability to check whether there is another Set/Merge below with - // the same user key. - // - // TODO(sumeer): once range deletions are interleaved at the maximal - // sequence number, this case will go away. default: panic("unreachable") } @@ -1174,7 +1053,7 @@ func (i *compactionIter) deleteSizedNext() (*base.InternalKey, []byte) { // Loop through all the keys within this stripe that are skippable. i.pos = iterPosNext - for i.nextInStripe() == sameStripeSkippable { + for i.nextInStripe() == sameStripe { if i.err != nil { panic(i.err) } @@ -1305,13 +1184,12 @@ func (i *compactionIter) deleteSizedNext() (*base.InternalKey, []byte) { return nil, nil } } - // Reset skip if we landed outside the original stripe. Otherwise, we landed - // in the same stripe on a non-skippable key. In that case we should preserve - // `i.skip == true` such that later keys in the stripe will continue to be - // skipped. - if i.iterStripeChange == newStripeNewKey || i.iterStripeChange == newStripeSameKey { - i.skip = false + + if i.iterStripeChange == sameStripe { + panic(errors.AssertionFailedf("unexpectedly found iter stripe change = %d", i.iterStripeChange)) } + // We landed outside the original stripe. Reset skip. + i.skip = false if i.err != nil { return nil, nil } diff --git a/compaction_iter_test.go b/compaction_iter_test.go index 932abde16e..628720665f 100644 --- a/compaction_iter_test.go +++ b/compaction_iter_test.go @@ -81,11 +81,13 @@ func TestCompactionIter(t *testing.T) { var merge Merge var keys []InternalKey var rangeKeys []keyspan.Span + var rangeDels []keyspan.Span var vals [][]byte var snapshots []uint64 var elideTombstones bool var allowZeroSeqnum bool - var interleavingIter *keyspan.InterleavingIter + var rangeKeyInterleaving *keyspan.InterleavingIter + var rangeDelInterleaving *keyspan.InterleavingIter // The input to the data-driven test is dependent on the format major // version we are testing against. @@ -108,13 +110,19 @@ func TestCompactionIter(t *testing.T) { // susceptible to use-after-free bugs, we skip the zeroing of // RangeDelete keys. fi := &fakeIter{keys: keys, vals: vals} - interleavingIter = &keyspan.InterleavingIter{} - interleavingIter.Init( + rangeDelInterleaving = &keyspan.InterleavingIter{} + rangeDelInterleaving.Init( base.DefaultComparer, fi, + keyspan.NewIter(base.DefaultComparer.Compare, rangeDels), + keyspan.InterleavingIterOpts{}) + rangeKeyInterleaving = &keyspan.InterleavingIter{} + rangeKeyInterleaving.Init( + base.DefaultComparer, + rangeDelInterleaving, keyspan.NewIter(base.DefaultComparer.Compare, rangeKeys), keyspan.InterleavingIterOpts{}) - iter := invalidating.NewIter(interleavingIter, invalidating.IgnoreKinds(InternalKeyKindRangeDelete)) + iter := invalidating.NewIter(rangeKeyInterleaving, invalidating.IgnoreKinds(InternalKeyKindRangeDelete)) if merge == nil { merge = func(key, value []byte) (base.ValueMerger, error) { m := &debugMerger{} @@ -161,9 +169,37 @@ func TestCompactionIter(t *testing.T) { keys = keys[:0] vals = vals[:0] rangeKeys = rangeKeys[:0] + rangeDels = rangeDels[:0] + rangeDelFragmenter := keyspan.Fragmenter{ + Cmp: DefaultComparer.Compare, + Format: DefaultComparer.FormatKey, + Emit: func(s keyspan.Span) { + rangeDels = append(rangeDels, s) + }, + } for _, key := range strings.Split(d.Input, "\n") { + // If the line ends in a '}' assume it's a span. + if strings.HasSuffix(key, "}") { + s := keyspan.ParseSpan(strings.TrimSpace(key)) + rangeKeys = append(rangeKeys, s) + continue + } + j := strings.Index(key, ":") - keys = append(keys, base.ParseInternalKey(key[:j])) + ik := base.ParseInternalKey(key[:j]) + if rangekey.IsRangeKey(ik.Kind()) { + panic("range keys must be pre-fragmented and formatted as spans") + } + if ik.Kind() == base.InternalKeyKindRangeDelete { + rangeDelFragmenter.Add(keyspan.Span{ + Start: ik.UserKey, + End: []byte(key[j+1:]), + Keys: []keyspan.Key{{Trailer: ik.Trailer}}, + }) + continue + } + + keys = append(keys, ik) if strings.HasPrefix(key[j+1:], "varint(") { valueStr := strings.TrimSuffix(strings.TrimPrefix(key[j+1:], "varint("), ")") @@ -175,13 +211,7 @@ func TestCompactionIter(t *testing.T) { vals = append(vals, []byte(key[j+1:])) } } - return "" - - case "define-range-keys": - for _, key := range strings.Split(d.Input, "\n") { - s := keyspan.ParseSpan(strings.TrimSpace(key)) - rangeKeys = append(rangeKeys, s) - } + rangeDelFragmenter.Finish() return "" case "iter": @@ -286,19 +316,15 @@ func TestCompactionIter(t *testing.T) { v = fmt.Sprintf("varint(%d)", vn) } } - fmt.Fprintf(&b, "%s:%s%s%s\n", iter.Key(), v, snapshotPinned, forceObsolete) + fmt.Fprintf(&b, "%s:%s%s%s", iter.Key(), v, snapshotPinned, forceObsolete) if iter.Key().Kind() == InternalKeyKindRangeDelete { - iter.rangeDelFrag.Add(keyspan.Span{ - Start: append([]byte{}, iter.Key().UserKey...), - End: append([]byte{}, iter.Value()...), - Keys: []keyspan.Key{ - {Trailer: iter.Key().Trailer}, - }, - }) + iter.rangeDelFrag.Add(*rangeDelInterleaving.Span()) + fmt.Fprintf(&b, "; Span() = %s", *rangeDelInterleaving.Span()) } if rangekey.IsRangeKey(iter.Key().Kind()) { - iter.rangeKeyFrag.Add(*interleavingIter.Span()) + iter.rangeKeyFrag.Add(*rangeKeyInterleaving.Span()) } + fmt.Fprintln(&b) } else if err := iter.Error(); err != nil { fmt.Fprintf(&b, "err=%v\n", err) } else { diff --git a/error_test.go b/error_test.go index e525afd9ac..97cac7dcd3 100644 --- a/error_test.go +++ b/error_test.go @@ -198,7 +198,7 @@ func TestRequireReadError(t *testing.T) { require.NoError(t, d.Flush()) expectLSM(` 0.0: - 000007:[a1#13,SETWITHDEL-a2#inf,RANGEDEL] + 000007:[a1#13,SET-a2#inf,RANGEDEL] 6: 000005:[a1#10,SET-a2#11,SET] `, d, t) @@ -292,7 +292,7 @@ func TestCorruptReadError(t *testing.T) { require.NoError(t, d.Flush()) expectLSM(` 0.0: - 000007:[a1#13,SETWITHDEL-a2#inf,RANGEDEL] + 000007:[a1#13,SET-a2#inf,RANGEDEL] 6: 000005:[a1#10,SET-a2#11,SET] `, d, t) diff --git a/internal/keyspan/defragment_test.go b/internal/keyspan/defragment_test.go index b9856da2c0..c9a7b01afb 100644 --- a/internal/keyspan/defragment_test.go +++ b/internal/keyspan/defragment_test.go @@ -80,7 +80,7 @@ func TestDefragmentingIter(t *testing.T) { } } var miter MergingIter - miter.Init(cmp, noopTransform, new(MergingBuffers), NewIter(cmp, spans)) + miter.Init(cmp, NoopTransform, new(MergingBuffers), NewIter(cmp, spans)) innerIter := attachProbes(&miter, probeContext{log: &buf}, probes...) var iter DefragmentingIter iter.Init(comparer, innerIter, equal, reducer, new(DefragmentingBuffers)) @@ -164,9 +164,9 @@ func testDefragmentingIteRandomizedOnce(t *testing.T, seed int64) { fragmented = fragment(cmp, formatKey, fragmented) var originalInner MergingIter - originalInner.Init(cmp, noopTransform, new(MergingBuffers), NewIter(cmp, original)) + originalInner.Init(cmp, NoopTransform, new(MergingBuffers), NewIter(cmp, original)) var fragmentedInner MergingIter - fragmentedInner.Init(cmp, noopTransform, new(MergingBuffers), NewIter(cmp, fragmented)) + fragmentedInner.Init(cmp, NoopTransform, new(MergingBuffers), NewIter(cmp, fragmented)) var referenceIter, fragmentedIter DefragmentingIter referenceIter.Init(comparer, &originalInner, DefragmentInternal, StaticDefragmentReducer, new(DefragmentingBuffers)) diff --git a/internal/keyspan/interleaving_iter_test.go b/internal/keyspan/interleaving_iter_test.go index 116f037614..99202325bf 100644 --- a/internal/keyspan/interleaving_iter_test.go +++ b/internal/keyspan/interleaving_iter_test.go @@ -108,7 +108,7 @@ func runInterleavingIterTest(t *testing.T, filename string) { for _, line := range lines { spans = append(spans, ParseSpan(line)) } - keyspanIter.Init(cmp, noopTransform, new(MergingBuffers), NewIter(cmp, spans)) + keyspanIter.Init(cmp, NoopTransform, new(MergingBuffers), NewIter(cmp, spans)) hooks.maskSuffix = nil iter.Init(testkeys.Comparer, &pointIter, &keyspanIter, InterleavingIterOpts{Mask: &hooks}) diff --git a/internal/keyspan/internal_iter_shim.go b/internal/keyspan/internal_iter_shim.go deleted file mode 100644 index bb9e37bdf9..0000000000 --- a/internal/keyspan/internal_iter_shim.go +++ /dev/null @@ -1,125 +0,0 @@ -// Copyright 2022 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 keyspan - -import ( - "context" - - "github.com/cockroachdb/pebble/internal/base" -) - -// InternalIteratorShim is a temporary iterator type used as a shim between -// keyspan.MergingIter and base.InternalIterator. It's used temporarily for -// range deletions during compactions, allowing range deletions to be -// interleaved by a compaction input iterator. -// -// TODO(jackson): This type should be removed, and the usages converted to using -// an InterleavingIterator type that interleaves keyspan.Spans from a -// keyspan.FragmentIterator with point keys. -type InternalIteratorShim struct { - miter MergingIter - mbufs MergingBuffers - span *Span - iterKey base.InternalKey -} - -// Assert that InternalIteratorShim implements InternalIterator. -var _ base.InternalIterator = &InternalIteratorShim{} - -// Init initializes the internal iterator shim to merge the provided fragment -// iterators. -func (i *InternalIteratorShim) Init(cmp base.Compare, iters ...FragmentIterator) { - i.miter.Init(cmp, noopTransform, &i.mbufs, iters...) -} - -// Span returns the span containing the full set of keys over the key span at -// the current iterator position. -func (i *InternalIteratorShim) Span() *Span { - return i.span -} - -// SeekGE implements (base.InternalIterator).SeekGE. -func (i *InternalIteratorShim) SeekGE( - key []byte, flags base.SeekGEFlags, -) (*base.InternalKey, base.LazyValue) { - panic("unimplemented") -} - -// SeekPrefixGE implements (base.InternalIterator).SeekPrefixGE. -func (i *InternalIteratorShim) SeekPrefixGE( - prefix, key []byte, flags base.SeekGEFlags, -) (*base.InternalKey, base.LazyValue) { - panic("unimplemented") -} - -// SeekLT implements (base.InternalIterator).SeekLT. -func (i *InternalIteratorShim) SeekLT( - key []byte, flags base.SeekLTFlags, -) (*base.InternalKey, base.LazyValue) { - panic("unimplemented") -} - -// First implements (base.InternalIterator).First. -func (i *InternalIteratorShim) First() (*base.InternalKey, base.LazyValue) { - i.span = i.miter.First() - for i.span != nil && i.span.Empty() { - i.span = i.miter.Next() - } - if i.span == nil { - return nil, base.LazyValue{} - } - i.iterKey = base.InternalKey{UserKey: i.span.Start, Trailer: i.span.Keys[0].Trailer} - return &i.iterKey, base.MakeInPlaceValue(i.span.End) -} - -// Last implements (base.InternalIterator).Last. -func (i *InternalIteratorShim) Last() (*base.InternalKey, base.LazyValue) { - panic("unimplemented") -} - -// Next implements (base.InternalIterator).Next. -func (i *InternalIteratorShim) Next() (*base.InternalKey, base.LazyValue) { - i.span = i.miter.Next() - for i.span != nil && i.span.Empty() { - i.span = i.miter.Next() - } - if i.span == nil { - return nil, base.LazyValue{} - } - i.iterKey = base.InternalKey{UserKey: i.span.Start, Trailer: i.span.Keys[0].Trailer} - return &i.iterKey, base.MakeInPlaceValue(i.span.End) -} - -// NextPrefix implements (base.InternalIterator).NextPrefix. -func (i *InternalIteratorShim) NextPrefix([]byte) (*base.InternalKey, base.LazyValue) { - panic("unimplemented") -} - -// Prev implements (base.InternalIterator).Prev. -func (i *InternalIteratorShim) Prev() (*base.InternalKey, base.LazyValue) { - panic("unimplemented") -} - -// Error implements (base.InternalIterator).Error. -func (i *InternalIteratorShim) Error() error { - return i.miter.Error() -} - -// Close implements (base.InternalIterator).Close. -func (i *InternalIteratorShim) Close() error { - return i.miter.Close() -} - -// SetBounds implements (base.InternalIterator).SetBounds. -func (i *InternalIteratorShim) SetBounds(lower, upper []byte) { -} - -// SetContext implements (base.InternalIterator).SetContext. -func (i *InternalIteratorShim) SetContext(_ context.Context) {} - -// String implements fmt.Stringer. -func (i *InternalIteratorShim) String() string { - return i.miter.String() -} diff --git a/internal/keyspan/transformer.go b/internal/keyspan/transformer.go index e0152cf4d6..e886c90c2f 100644 --- a/internal/keyspan/transformer.go +++ b/internal/keyspan/transformer.go @@ -24,7 +24,8 @@ func (tf TransformerFunc) Transform(cmp base.Compare, in Span, out *Span) error return tf(cmp, in, out) } -var noopTransform Transformer = TransformerFunc(func(_ base.Compare, s Span, dst *Span) error { +// NoopTransform is a Transformer that performs no mutations. +var NoopTransform Transformer = TransformerFunc(func(_ base.Compare, s Span, dst *Span) error { dst.Start, dst.End = s.Start, s.End dst.Keys = append(dst.Keys[:0], s.Keys...) return nil diff --git a/range_del_test.go b/range_del_test.go index 07251ab33a..7d7f4c0ac0 100644 --- a/range_del_test.go +++ b/range_del_test.go @@ -356,7 +356,7 @@ func TestRangeDelCompactionTruncation(t *testing.T) { 1: 000008:[a#12,RANGEDEL-b#inf,RANGEDEL] 2: - 000012:[b#13,SETWITHDEL-c#inf,RANGEDEL] + 000012:[b#13,SET-c#inf,RANGEDEL] 3: 000013:[c#14,SET-d#inf,RANGEDEL] `) diff --git a/testdata/compaction_delete_only_hints b/testdata/compaction_delete_only_hints index e11d120295..5147ba6dea 100644 --- a/testdata/compaction_delete_only_hints +++ b/testdata/compaction_delete_only_hints @@ -221,7 +221,7 @@ L6 a.SET.20:b a.RANGEDEL.15:z ---- 6: - 000004:[a#20,SETWITHDEL-z#inf,RANGEDEL] + 000004:[a#20,SET-z#inf,RANGEDEL] # Note that this test depends on stats being present on the sstables, so we # collect hints here. We expect none, as the table is in L6. @@ -255,7 +255,7 @@ L0.000001 a-z seqnums(tombstone=5-27, file-smallest=0, type=point-key-only) close-snapshot 10 ---- -[JOB 100] compacted(elision-only) L6 [000004] (741B) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [000005] (662B), in 1.0s (2.0s total), output rate 662B/s +[JOB 100] compacted(elision-only) L6 [000004] (742B) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [000005] (663B), in 1.0s (2.0s total), output rate 663B/s # The deletion hint was removed by the elision-only compaction. get-hints diff --git a/testdata/compaction_iter_delete_sized b/testdata/compaction_iter_delete_sized index 950ec82cf6..dea08758bc 100644 --- a/testdata/compaction_iter_delete_sized +++ b/testdata/compaction_iter_delete_sized @@ -247,9 +247,7 @@ a.INVALID.1:c iter first -next ---- -a#2,18:b err=invalid internal key kind: INVALID define @@ -259,9 +257,7 @@ a.INVALID.1:c iter first -next ---- -a#2,2:b err=invalid internal key kind: INVALID define @@ -271,10 +267,10 @@ a.RANGEDEL.1:d iter first -tombstones +next ---- +a#72057594037927935,15:; Span() = a-d:{(#1,RANGEDEL)} err=invalid internal key kind: INVALID -. define a.MERGE.2:b @@ -306,11 +302,13 @@ first next next next +next tombstones ---- -a#2,18:b -a#1,15:c -b#4,15:d +a#72057594037927935,15:; Span() = a-b:{(#1,RANGEDEL)} +a#2,1:b +b#72057594037927935,15:; Span() = b-c:{(#4,RANGEDEL) (#1,RANGEDEL)} +c#72057594037927935,15:; Span() = c-d:{(#4,RANGEDEL)} . a-b#1 b-c#4 @@ -322,11 +320,13 @@ first next next next +next tombstones ---- +a#72057594037927935,15:; Span() = a-b:{(#1,RANGEDEL)} a#2,1:b -a#1,15:c -b#4,15:d +b#72057594037927935,15:; Span() = b-c:{(#4,RANGEDEL) (#1,RANGEDEL)} +c#72057594037927935,15:; Span() = c-d:{(#4,RANGEDEL)} . a-b#1 b-c#4 @@ -340,12 +340,14 @@ next next next next +next tombstones ---- -a#2,18:b (not pinned) (not force obsolete) -a#1,15:c (not pinned) (not force obsolete) -b#4,15:d (not pinned) (not force obsolete) +a#72057594037927935,15: (not pinned) (not force obsolete); Span() = a-b:{(#1,RANGEDEL)} +a#2,1:b (not pinned) (not force obsolete) +b#72057594037927935,15: (not pinned) (not force obsolete); Span() = b-c:{(#4,RANGEDEL) (#1,RANGEDEL)} b#2,1:e (pinned) (force obsolete) +c#72057594037927935,15: (not pinned) (force obsolete); Span() = c-d:{(#4,RANGEDEL)} . a-b#1 b-c#4 @@ -360,12 +362,14 @@ next next next next +next tombstones ---- -a#2,18:b (not pinned) (not force obsolete) -a#1,15:c (not pinned) (not force obsolete) -b#4,15:d (not pinned) (not force obsolete) +a#72057594037927935,15: (not pinned) (not force obsolete); Span() = a-b:{(#1,RANGEDEL)} +a#2,1:b (not pinned) (not force obsolete) +b#72057594037927935,15: (not pinned) (not force obsolete); Span() = b-c:{(#4,RANGEDEL) (#1,RANGEDEL)} b#2,1:e (pinned) (force obsolete) +c#72057594037927935,15: (not pinned) (force obsolete); Span() = c-d:{(#4,RANGEDEL)} c#3,1:f (pinned) (force obsolete) . a-b#1 @@ -390,7 +394,7 @@ next next tombstones ---- -a#3,15:e +a#72057594037927935,15:; Span() = a-e:{(#3,RANGEDEL)} b#4,1:b c#3,1:c e#1,1:e @@ -414,7 +418,7 @@ next next tombstones ---- -a#3,15:e +a#72057594037927935,15:; Span() = a-e:{(#3,RANGEDEL)} b#4,2:b c#3,2:c e#1,2:e @@ -443,10 +447,10 @@ next next tombstones ---- -a#3,15:c +a#72057594037927935,15:; Span() = a-c:{(#3,RANGEDEL)} b#5,2:de +d#72057594037927935,15:; Span() = d-f:{(#3,RANGEDEL)} d#5,2:bc -d#3,15:f . a-c#3 d-f#3 @@ -463,11 +467,15 @@ first next next next +next +next tombstones ---- -a#3,15:d -b#2,15:e -c#1,15:f +a#72057594037927935,15:; Span() = a-b:{(#3,RANGEDEL)} +b#72057594037927935,15:; Span() = b-c:{(#3,RANGEDEL) (#2,RANGEDEL)} +c#72057594037927935,15:; Span() = c-d:{(#3,RANGEDEL) (#2,RANGEDEL) (#1,RANGEDEL)} +d#72057594037927935,15:; Span() = d-e:{(#2,RANGEDEL) (#1,RANGEDEL)} +e#72057594037927935,15:; Span() = e-f:{(#1,RANGEDEL)} . a-b#3 b-c#3 @@ -483,17 +491,16 @@ next next tombstones ---- -a#3,15:d -b#2,15:e -c#1,15:f -. +a#72057594037927935,15:; Span() = a-b:{(#3,RANGEDEL)} +b#72057594037927935,15:; Span() = b-c:{(#3,RANGEDEL) (#2,RANGEDEL)} +c#72057594037927935,15:; Span() = c-d:{(#3,RANGEDEL) (#2,RANGEDEL) (#1,RANGEDEL)} +d#72057594037927935,15:; Span() = d-e:{(#2,RANGEDEL) (#1,RANGEDEL)} a-b#3 b-c#3 c-d#3 c-d#1 d-e#2 d-e#1 -e-f#1 . iter snapshots=3 @@ -501,11 +508,15 @@ first next next next +next +next tombstones ---- -a#3,15:d -b#2,15:e -c#1,15:f +a#72057594037927935,15:; Span() = a-b:{(#3,RANGEDEL)} +b#72057594037927935,15:; Span() = b-c:{(#3,RANGEDEL) (#2,RANGEDEL)} +c#72057594037927935,15:; Span() = c-d:{(#3,RANGEDEL) (#2,RANGEDEL) (#1,RANGEDEL)} +d#72057594037927935,15:; Span() = d-e:{(#2,RANGEDEL) (#1,RANGEDEL)} +e#72057594037927935,15:; Span() = e-f:{(#1,RANGEDEL)} . a-b#3 b-c#3 @@ -521,11 +532,15 @@ first next next next +next +next tombstones ---- -a#3,15:d -b#2,15:e -c#1,15:f +a#72057594037927935,15:; Span() = a-b:{(#3,RANGEDEL)} +b#72057594037927935,15:; Span() = b-c:{(#3,RANGEDEL) (#2,RANGEDEL)} +c#72057594037927935,15:; Span() = c-d:{(#3,RANGEDEL) (#2,RANGEDEL) (#1,RANGEDEL)} +d#72057594037927935,15:; Span() = d-e:{(#2,RANGEDEL) (#1,RANGEDEL)} +e#72057594037927935,15:; Span() = e-f:{(#1,RANGEDEL)} . a-b#3 b-c#3 @@ -551,7 +566,7 @@ tombstones f next tombstones ---- -a#10,15:k (not pinned) (not force obsolete) +a#72057594037927935,15: (not pinned) (not force obsolete); Span() = a-k:{(#10,RANGEDEL)} f#9,1:f (pinned) (force obsolete) a-f#10 . @@ -572,7 +587,7 @@ tombstones f next tombstones ---- -f#10,15:k +f#72057594037927935,15:; Span() = f-k:{(#10,RANGEDEL)} f#9,1:f . f#8,1:f @@ -589,17 +604,21 @@ d.SET.4:d iter first next +tombstones c +next +next next next -tombstones c tombstones ---- a#1,1:a -b#2,15:d -c#3,15:e -d#4,1:d +b#72057594037927935,15:; Span() = b-c:{(#2,RANGEDEL)} b-c#2 . +c#72057594037927935,15:; Span() = c-d:{(#3,RANGEDEL) (#2,RANGEDEL)} +d#72057594037927935,15:; Span() = d-e:{(#3,RANGEDEL)} +d#4,1:d +. c-d#3 d-e#3 . @@ -607,17 +626,21 @@ d-e#3 iter snapshots=3 first next +tombstones c +next +next next next -tombstones c tombstones ---- a#1,1:a -b#2,15:d -c#3,15:e -d#4,1:d +b#72057594037927935,15:; Span() = b-c:{(#2,RANGEDEL)} b-c#2 . +c#72057594037927935,15:; Span() = c-d:{(#3,RANGEDEL) (#2,RANGEDEL)} +d#72057594037927935,15:; Span() = d-e:{(#3,RANGEDEL)} +d#4,1:d +. c-d#3 c-d#2 d-e#3 @@ -633,12 +656,14 @@ iter first next next +next tombstones c tombstones ---- a#1,1:a -b#2,15:d +b#72057594037927935,15:; Span() = b-d:{(#2,RANGEDEL)} c#4,1:d +. b-c#2 . c-d#2 @@ -658,7 +683,7 @@ next next next ---- -a#2,15:d +a#72057594037927935,15:; Span() = a-d:{(#2,RANGEDEL)} a#2,1:a b#2,1:b c#2,1:c @@ -979,8 +1004,8 @@ next next tombstones ---- +a#72057594037927935,15:; Span() = a-c:{(#2,RANGEDEL)} a#3,7: -a#2,15:c . a-c#2 . @@ -998,7 +1023,7 @@ next next tombstones ---- -a#3,15:d +a#72057594037927935,15:; Span() = a-d:{(#3,RANGEDEL)} d#2,0: . a-d#3 @@ -1010,7 +1035,7 @@ next next next ---- -a#3,15:d +a#72057594037927935,15:; Span() = a-d:{(#3,RANGEDEL)} a#2,0: d#2,0: . @@ -1021,7 +1046,7 @@ next next next ---- -a#3,15:d +a#72057594037927935,15:; Span() = a-d:{(#3,RANGEDEL)} a#1,1:a d#2,0: . @@ -1031,7 +1056,7 @@ first next next ---- -a#3,15:d +a#72057594037927935,15:; Span() = a-d:{(#3,RANGEDEL)} d#2,0: . @@ -1049,7 +1074,7 @@ tombstones ---- a#2,2:a . -b#1,15:c +b#72057594037927935,15:; Span() = b-c:{(#1,RANGEDEL)} . b-c#1 . @@ -1064,12 +1089,10 @@ iter allow-zero-seqnum=true first next next -next tombstones ---- -a#2,2:v2 -a#1,15:b -a#0,2:v1 +a#72057594037927935,15:; Span() = a-b:{(#1,RANGEDEL)} +a#0,2:v1v2 . a-b#1 . @@ -1135,7 +1158,7 @@ first next next ---- -a#3,15:c +a#72057594037927935,15:; Span() = a-c:{(#3,RANGEDEL)} b#5,2:5 . @@ -1144,7 +1167,7 @@ first next next ---- -a#3,15:c +a#72057594037927935,15:; Span() = a-c:{(#3,RANGEDEL)} b#0,2:5 . @@ -1152,10 +1175,12 @@ iter snapshots=2 first next next +next ---- -a#3,15:c +a#72057594037927935,15:; Span() = a-c:{(#3,RANGEDEL)} b#5,2:5 b#1,2:1 +. define a.RANGEDEL.3:c @@ -1169,7 +1194,7 @@ first next next ---- -a#3,15:c +a#72057594037927935,15:; Span() = a-c:{(#3,RANGEDEL)} b#5,2:5 . @@ -1177,10 +1202,12 @@ iter snapshots=2 first next next +next ---- -a#3,15:c +a#72057594037927935,15:; Span() = a-c:{(#3,RANGEDEL)} b#5,2:5 b#1,2:1 +. # SET that meets a DEL is transformed into a SETWITHDEL. @@ -1303,8 +1330,8 @@ first next next ---- -a#0,18:c -a#2,15:z +a#72057594037927935,15:; Span() = a-z:{(#2,RANGEDEL)} +a#0,1:c . iter allow-zero-seqnum=true snapshots=3 @@ -1313,8 +1340,8 @@ next next next ---- +a#72057594037927935,15:; Span() = a-z:{(#2,RANGEDEL)} a#3,1:c -a#2,15:z a#0,1:b . @@ -1324,8 +1351,8 @@ next next next ---- -a#3,18:c -a#2,15:z +a#72057594037927935,15:; Span() = a-z:{(#2,RANGEDEL)} +a#3,1:c a#1,0: . @@ -1341,8 +1368,8 @@ first next next ---- -a#4,18:c -a#3,15:z +a#72057594037927935,15:; Span() = a-z:{(#3,RANGEDEL)} +a#4,1:c . # Invalid keys are emitted under SETWITHDEL. @@ -1354,9 +1381,7 @@ a.INVALID.1: iter first -next ---- -a#2,18:b err=invalid internal key kind: INVALID define @@ -1367,9 +1392,7 @@ a.SET.1:b iter first -next ---- -a#3,18:c err=invalid internal key kind: INVALID # SINGLEDEL that meets a SETWITHDEL is transformed into a DEL. @@ -1653,9 +1676,9 @@ tombstones # Test a very subtle sequence where a elision of tombstones is active, and a # unskippable RANGEDEL sits between a DELSIZED and the key it was intended to -# delete. The unskippable RANGEDEL breaks the skipping of keys within the -# snapshot stripe, but it's ultimately okay because we preserve skip=true across -# the RANGEDEL return. +# delete. In previous versions of the code, the RANGEDEL was interleaved at its +# sequence number and "unskippable" breaking the skipping of keys within the +# snapshot stripe. define a.DELSIZED.5:varint(4) @@ -1668,7 +1691,7 @@ first next tombstones ---- -a#4,15:d +a#72057594037927935,15:; Span() = a-d:{(#4,RANGEDEL)} . . @@ -1679,7 +1702,7 @@ first next tombstones ---- -a#4,15:d +a#72057594037927935,15:; Span() = a-d:{(#4,RANGEDEL)} . . @@ -1696,7 +1719,7 @@ first next tombstones ---- -a#4,15:d +a#72057594037927935,15:; Span() = a-d:{(#4,RANGEDEL)} . . @@ -1713,9 +1736,10 @@ first next tombstones ---- -a#4,15:d +a#72057594037927935,15:; Span() = a-d:{(#4,RANGEDEL)} . . +ineffectual-single-deletes: a # Perform a few variants of the above but with a range del with a seqnum equal to # keys. NB: When seqnums are equal, the order of keys with various kinds is: @@ -1726,15 +1750,12 @@ a#4,15:d # compaction iterator should always observe them first. define +a-z:{(#5,RANGEKEYDEL)} a.SINGLEDEL.6: a.SETWITHDEL.5:foo a.RANGEDEL.5:z ---- -define-range-keys -a-z:{(#5,RANGEKEYDEL)} ----- - # In the following case, the SINGLEDEL meets a SETWITHDEL, promoting the # SINGLEDEL into a DEL. @@ -1746,8 +1767,8 @@ next tombstones ---- a#72057594037927935,19: +a#72057594037927935,15:; Span() = a-z:{(#5,RANGEDEL)} a#6,0: -a#5,15:z . a-z#5 . @@ -1762,7 +1783,7 @@ next tombstones ---- a#72057594037927935,19: -a#5,15:z +a#72057594037927935,15:; Span() = a-z:{(#5,RANGEDEL)} . . @@ -1772,21 +1793,17 @@ a.RANGEDEL.5:d a.SET.5:foo ---- -# NB: In this case, the RANGEDEL acts as an unintentional snapshot stripe -# change. This is a code artifact, and we will be able to remove this behavior -# when range deletes are interleaved at the maximal sequence number by an -# interleaving iterator (like range keys are). +# NB: In this case, in previous versions of the code, the RANGEDEL acted as an +# unintentional snapshot stripe change. This was a code artifact that was fixed +# when we began interleaving range deletions at the maximal sequence number +# using an interleaving iterator (like range keys are). iter first next -next -next tombstones ---- -a#6,7: -a#5,15:d -a#5,1:foo +a#72057594037927935,15:; Span() = a-d:{(#5,RANGEDEL)} . a-d#5 . @@ -1796,7 +1813,7 @@ first next tombstones ---- -a#5,15:d +a#72057594037927935,15:; Span() = a-d:{(#5,RANGEDEL)} . . @@ -1807,17 +1824,20 @@ a.RANGEDEL.4:d a.SET.4:bar ---- -# The SINGLEDEL invariant checking can't see past the RANGEDEL and see that -# the a.SET.4 violates the invariant. This is a code artifact that will be -# improved when range deletes are interleaved at the maximal sequence number. +# In previous versions of the code, the SINGLEDEL invariant checking could not +# see past the RANGEDEL and see that the a.SET.4 violates the invariant. This +# was a code artifact that has been improved when we began interleaving range +# deletes at the maximal sequence number + iter first next next ---- -a#4,15:d +a#72057594037927935,15:; Span() = a-d:{(#4,RANGEDEL)} a#4,1:bar . +invariant-violation-single-deletes: a define a.SINGLEDEL.6: @@ -1832,8 +1852,8 @@ first next tombstones ---- +a#72057594037927935,15:; Span() = a-d:{(#5,RANGEDEL)} a#6,0: -a#5,15:d a-d#5 . @@ -1842,7 +1862,7 @@ first next tombstones ---- -a#5,15:d +a#72057594037927935,15:; Span() = a-d:{(#5,RANGEDEL)} . . @@ -1855,10 +1875,12 @@ a.SET.5:foo iter first next +next tombstones ---- -a#6,23:varint(3) -a#5,15:d +a#72057594037927935,15:; Span() = a-d:{(#5,RANGEDEL)} +a#6,0: +. a-d#5 . @@ -1867,7 +1889,7 @@ first next tombstones ---- -a#5,15:d +a#72057594037927935,15:; Span() = a-d:{(#5,RANGEDEL)} . . diff --git a/testdata/compaction_iter_set_with_del b/testdata/compaction_iter_set_with_del index 620c1cf3b8..1bbb74032b 100644 --- a/testdata/compaction_iter_set_with_del +++ b/testdata/compaction_iter_set_with_del @@ -247,9 +247,7 @@ a.INVALID.1:c iter first -next ---- -a#2,18:b err=invalid internal key kind: INVALID define @@ -259,9 +257,7 @@ a.INVALID.1:c iter first -next ---- -a#2,2:b err=invalid internal key kind: INVALID define @@ -271,9 +267,12 @@ a.RANGEDEL.1:d iter first +next tombstones ---- +a#72057594037927935,15:; Span() = a-d:{(#1,RANGEDEL)} err=invalid internal key kind: INVALID +a-d#1 . define @@ -306,11 +305,13 @@ first next next next +next tombstones ---- -a#2,18:b -a#1,15:c -b#4,15:d +a#72057594037927935,15:; Span() = a-b:{(#1,RANGEDEL)} +a#2,1:b +b#72057594037927935,15:; Span() = b-c:{(#4,RANGEDEL) (#1,RANGEDEL)} +c#72057594037927935,15:; Span() = c-d:{(#4,RANGEDEL)} . a-b#1 b-c#4 @@ -322,11 +323,13 @@ first next next next +next tombstones ---- +a#72057594037927935,15:; Span() = a-b:{(#1,RANGEDEL)} a#2,1:b -a#1,15:c -b#4,15:d +b#72057594037927935,15:; Span() = b-c:{(#4,RANGEDEL) (#1,RANGEDEL)} +c#72057594037927935,15:; Span() = c-d:{(#4,RANGEDEL)} . a-b#1 b-c#4 @@ -340,12 +343,14 @@ next next next next +next tombstones ---- -a#2,18:b -a#1,15:c -b#4,15:d +a#72057594037927935,15:; Span() = a-b:{(#1,RANGEDEL)} +a#2,1:b +b#72057594037927935,15:; Span() = b-c:{(#4,RANGEDEL) (#1,RANGEDEL)} b#2,1:e +c#72057594037927935,15:; Span() = c-d:{(#4,RANGEDEL)} . a-b#1 b-c#4 @@ -360,12 +365,14 @@ next next next next +next tombstones ---- -a#2,18:b -a#1,15:c -b#4,15:d +a#72057594037927935,15:; Span() = a-b:{(#1,RANGEDEL)} +a#2,1:b +b#72057594037927935,15:; Span() = b-c:{(#4,RANGEDEL) (#1,RANGEDEL)} b#2,1:e +c#72057594037927935,15:; Span() = c-d:{(#4,RANGEDEL)} c#3,1:f . a-b#1 @@ -390,7 +397,7 @@ next next tombstones ---- -a#3,15:e +a#72057594037927935,15:; Span() = a-e:{(#3,RANGEDEL)} b#4,1:b c#3,1:c e#1,1:e @@ -414,7 +421,7 @@ next next tombstones ---- -a#3,15:e +a#72057594037927935,15:; Span() = a-e:{(#3,RANGEDEL)} b#4,2:b c#3,2:c e#1,2:e @@ -443,10 +450,10 @@ next next tombstones ---- -a#3,15:c +a#72057594037927935,15:; Span() = a-c:{(#3,RANGEDEL)} b#5,2:de +d#72057594037927935,15:; Span() = d-f:{(#3,RANGEDEL)} d#5,2:bc -d#3,15:f . a-c#3 d-f#3 @@ -463,11 +470,15 @@ first next next next +next +next tombstones ---- -a#3,15:d -b#2,15:e -c#1,15:f +a#72057594037927935,15:; Span() = a-b:{(#3,RANGEDEL)} +b#72057594037927935,15:; Span() = b-c:{(#3,RANGEDEL) (#2,RANGEDEL)} +c#72057594037927935,15:; Span() = c-d:{(#3,RANGEDEL) (#2,RANGEDEL) (#1,RANGEDEL)} +d#72057594037927935,15:; Span() = d-e:{(#2,RANGEDEL) (#1,RANGEDEL)} +e#72057594037927935,15:; Span() = e-f:{(#1,RANGEDEL)} . a-b#3 b-c#3 @@ -481,11 +492,15 @@ first next next next +next +next tombstones ---- -a#3,15:d -b#2,15:e -c#1,15:f +a#72057594037927935,15:; Span() = a-b:{(#3,RANGEDEL)} +b#72057594037927935,15:; Span() = b-c:{(#3,RANGEDEL) (#2,RANGEDEL)} +c#72057594037927935,15:; Span() = c-d:{(#3,RANGEDEL) (#2,RANGEDEL) (#1,RANGEDEL)} +d#72057594037927935,15:; Span() = d-e:{(#2,RANGEDEL) (#1,RANGEDEL)} +e#72057594037927935,15:; Span() = e-f:{(#1,RANGEDEL)} . a-b#3 b-c#3 @@ -501,11 +516,15 @@ first next next next +next +next tombstones ---- -a#3,15:d -b#2,15:e -c#1,15:f +a#72057594037927935,15:; Span() = a-b:{(#3,RANGEDEL)} +b#72057594037927935,15:; Span() = b-c:{(#3,RANGEDEL) (#2,RANGEDEL)} +c#72057594037927935,15:; Span() = c-d:{(#3,RANGEDEL) (#2,RANGEDEL) (#1,RANGEDEL)} +d#72057594037927935,15:; Span() = d-e:{(#2,RANGEDEL) (#1,RANGEDEL)} +e#72057594037927935,15:; Span() = e-f:{(#1,RANGEDEL)} . a-b#3 b-c#3 @@ -521,11 +540,15 @@ first next next next +next +next tombstones ---- -a#3,15:d -b#2,15:e -c#1,15:f +a#72057594037927935,15:; Span() = a-b:{(#3,RANGEDEL)} +b#72057594037927935,15:; Span() = b-c:{(#3,RANGEDEL) (#2,RANGEDEL)} +c#72057594037927935,15:; Span() = c-d:{(#3,RANGEDEL) (#2,RANGEDEL) (#1,RANGEDEL)} +d#72057594037927935,15:; Span() = d-e:{(#2,RANGEDEL) (#1,RANGEDEL)} +e#72057594037927935,15:; Span() = e-f:{(#1,RANGEDEL)} . a-b#3 b-c#3 @@ -551,7 +574,7 @@ tombstones f next tombstones ---- -a#10,15:k +a#72057594037927935,15:; Span() = a-k:{(#10,RANGEDEL)} f#9,1:f a-f#10 . @@ -572,7 +595,7 @@ tombstones f next tombstones ---- -f#10,15:k +f#72057594037927935,15:; Span() = f-k:{(#10,RANGEDEL)} f#9,1:f . f#8,1:f @@ -589,17 +612,21 @@ d.SET.4:d iter first next +tombstones c +next +next next next -tombstones c tombstones ---- a#1,1:a -b#2,15:d -c#3,15:e -d#4,1:d +b#72057594037927935,15:; Span() = b-c:{(#2,RANGEDEL)} b-c#2 . +c#72057594037927935,15:; Span() = c-d:{(#3,RANGEDEL) (#2,RANGEDEL)} +d#72057594037927935,15:; Span() = d-e:{(#3,RANGEDEL)} +d#4,1:d +. c-d#3 d-e#3 . @@ -607,17 +634,21 @@ d-e#3 iter snapshots=3 first next +tombstones c +next +next next next -tombstones c tombstones ---- a#1,1:a -b#2,15:d -c#3,15:e -d#4,1:d +b#72057594037927935,15:; Span() = b-c:{(#2,RANGEDEL)} b-c#2 . +c#72057594037927935,15:; Span() = c-d:{(#3,RANGEDEL) (#2,RANGEDEL)} +d#72057594037927935,15:; Span() = d-e:{(#3,RANGEDEL)} +d#4,1:d +. c-d#3 c-d#2 d-e#3 @@ -637,7 +668,7 @@ tombstones c tombstones ---- a#1,1:a -b#2,15:d +b#72057594037927935,15:; Span() = b-d:{(#2,RANGEDEL)} c#4,1:d b-c#2 . @@ -658,7 +689,7 @@ next next next ---- -a#2,15:d +a#72057594037927935,15:; Span() = a-d:{(#2,RANGEDEL)} a#2,1:a b#2,1:b c#2,1:c @@ -979,8 +1010,8 @@ next next tombstones ---- +a#72057594037927935,15:; Span() = a-c:{(#2,RANGEDEL)} a#3,7: -a#2,15:c . a-c#2 . @@ -998,7 +1029,7 @@ next next tombstones ---- -a#3,15:d +a#72057594037927935,15:; Span() = a-d:{(#3,RANGEDEL)} d#2,0: . a-d#3 @@ -1010,7 +1041,7 @@ next next next ---- -a#3,15:d +a#72057594037927935,15:; Span() = a-d:{(#3,RANGEDEL)} a#2,0: d#2,0: . @@ -1021,7 +1052,7 @@ next next next ---- -a#3,15:d +a#72057594037927935,15:; Span() = a-d:{(#3,RANGEDEL)} a#1,1:a d#2,0: . @@ -1031,7 +1062,7 @@ first next next ---- -a#3,15:d +a#72057594037927935,15:; Span() = a-d:{(#3,RANGEDEL)} d#2,0: . @@ -1049,7 +1080,7 @@ tombstones ---- a#2,2:a . -b#1,15:c +b#72057594037927935,15:; Span() = b-c:{(#1,RANGEDEL)} . b-c#1 . @@ -1064,12 +1095,10 @@ iter allow-zero-seqnum=true first next next -next tombstones ---- -a#2,2:v2 -a#1,15:b -a#0,2:v1 +a#72057594037927935,15:; Span() = a-b:{(#1,RANGEDEL)} +a#0,2:v1v2 . a-b#1 . @@ -1135,7 +1164,7 @@ first next next ---- -a#3,15:c +a#72057594037927935,15:; Span() = a-c:{(#3,RANGEDEL)} b#5,2:5 . @@ -1144,7 +1173,7 @@ first next next ---- -a#3,15:c +a#72057594037927935,15:; Span() = a-c:{(#3,RANGEDEL)} b#0,2:5 . @@ -1153,7 +1182,7 @@ first next next ---- -a#3,15:c +a#72057594037927935,15:; Span() = a-c:{(#3,RANGEDEL)} b#5,2:5 b#1,2:1 @@ -1169,7 +1198,7 @@ first next next ---- -a#3,15:c +a#72057594037927935,15:; Span() = a-c:{(#3,RANGEDEL)} b#5,2:5 . @@ -1178,7 +1207,7 @@ first next next ---- -a#3,15:c +a#72057594037927935,15:; Span() = a-c:{(#3,RANGEDEL)} b#5,2:5 b#1,2:1 @@ -1291,6 +1320,7 @@ a#1,0: # SETWITHDEL-eligible entries at or under a RANGEDEL at the same user key should # be skipped. + define a.SET.3:c a.RANGEDEL.2:z @@ -1303,8 +1333,8 @@ first next next ---- -a#0,18:c -a#2,15:z +a#72057594037927935,15:; Span() = a-z:{(#2,RANGEDEL)} +a#0,1:c . iter allow-zero-seqnum=true snapshots=3 @@ -1313,8 +1343,8 @@ next next next ---- +a#72057594037927935,15:; Span() = a-z:{(#2,RANGEDEL)} a#3,1:c -a#2,15:z a#0,1:b . @@ -1324,8 +1354,8 @@ next next next ---- -a#3,18:c -a#2,15:z +a#72057594037927935,15:; Span() = a-z:{(#2,RANGEDEL)} +a#3,1:c a#1,0: . @@ -1341,8 +1371,8 @@ first next next ---- -a#4,18:c -a#3,15:z +a#72057594037927935,15:; Span() = a-z:{(#3,RANGEDEL)} +a#4,1:c . # Invalid keys are emitted under SETWITHDEL. @@ -1354,9 +1384,7 @@ a.INVALID.1: iter first -next ---- -a#2,18:b err=invalid internal key kind: INVALID define @@ -1367,9 +1395,7 @@ a.SET.1:b iter first -next ---- -a#3,18:c err=invalid internal key kind: INVALID # SINGLEDEL that meets a SETWITHDEL is transformed into a DEL. diff --git a/testdata/manual_compaction_set_with_del_sstable_Pebblev4 b/testdata/manual_compaction_set_with_del_sstable_Pebblev4 index 5807ec5287..19bed7b1ba 100644 --- a/testdata/manual_compaction_set_with_del_sstable_Pebblev4 +++ b/testdata/manual_compaction_set_with_del_sstable_Pebblev4 @@ -93,7 +93,7 @@ range-deletions-bytes-estimate: 1334 compact a-e L1 ---- 2: - 000008:[a#3,SETWITHDEL-c#inf,RANGEDEL] + 000008:[a#3,SET-c#inf,RANGEDEL] 000009:[c#2,RANGEDEL-e#inf,RANGEDEL] 3: 000006:[a#0,SET-b#0,SET] @@ -141,7 +141,7 @@ L3 compact a-e L1 ---- 2: - 000010:[a#3,SETWITHDEL-c#inf,RANGEDEL] + 000010:[a#3,SET-c#inf,RANGEDEL] 000011:[c#2,RANGEDEL-e#inf,RANGEDEL] 000012:[e#2,RANGEDEL-g#inf,RANGEDEL] 3: @@ -181,7 +181,7 @@ L3 compact a-e L1 ---- 2: - 000009:[a#3,SETWITHDEL-c#inf,RANGEDEL] + 000009:[a#3,SET-c#inf,RANGEDEL] 000010:[c#2,RANGEDEL-h#3,SET] 3: 000006:[a#0,SET-b#0,SET] @@ -354,7 +354,7 @@ compact a-e L1 0.0: 000004:[c#4,SET-c#4,SET] 2: - 000008:[a#3,SETWITHDEL-b#inf,RANGEDEL] + 000008:[a#3,SET-b#inf,RANGEDEL] 000009:[b#2,RANGEDEL-e#inf,RANGEDEL] 3: 000007:[b#1,SET-b#1,SET] diff --git a/testdata/range_del b/testdata/range_del index 53cca9d90e..19813d6b19 100644 --- a/testdata/range_del +++ b/testdata/range_del @@ -1011,13 +1011,13 @@ mem: 1 compact a-e ---- 1: - 000007:[a#11,SETWITHDEL-c#inf,RANGEDEL] + 000007:[a#11,SET-c#inf,RANGEDEL] 000008:[c#12,SET-e#inf,RANGEDEL] compact d-e ---- 1: - 000007:[a#11,SETWITHDEL-c#inf,RANGEDEL] + 000007:[a#11,SET-c#inf,RANGEDEL] 2: 000008:[c#12,SET-e#inf,RANGEDEL] @@ -1049,7 +1049,7 @@ mem: 1 compact a-e ---- 1: - 000007:[a#11,SETWITHDEL-c#inf,RANGEDEL] + 000007:[a#11,SET-c#inf,RANGEDEL] 000008:[c#12,SET-e#inf,RANGEDEL] compact a-b @@ -1057,7 +1057,7 @@ compact a-b 1: 000008:[c#12,SET-e#inf,RANGEDEL] 2: - 000007:[a#11,SETWITHDEL-c#inf,RANGEDEL] + 000007:[a#11,SET-c#inf,RANGEDEL] iter seq=13 seek-lt d @@ -1097,7 +1097,7 @@ mem: 1 compact a-b ---- 1: - 000008:[a#11,SETWITHDEL-c#inf,RANGEDEL] + 000008:[a#11,SET-c#inf,RANGEDEL] 000009:[c#12,SET-d#inf,RANGEDEL] 000010:[d#10,RANGEDEL-e#inf,RANGEDEL] 2: @@ -1106,7 +1106,7 @@ compact a-b compact d-e ---- 1: - 000008:[a#11,SETWITHDEL-c#inf,RANGEDEL] + 000008:[a#11,SET-c#inf,RANGEDEL] 000009:[c#12,SET-d#inf,RANGEDEL] 3: 000011:[d#10,RANGEDEL-e#inf,RANGEDEL] @@ -1121,7 +1121,7 @@ compact a-b L1 1: 000009:[c#12,SET-d#inf,RANGEDEL] 2: - 000008:[a#11,SETWITHDEL-c#inf,RANGEDEL] + 000008:[a#11,SET-c#inf,RANGEDEL] 3: 000011:[d#10,RANGEDEL-e#inf,RANGEDEL] @@ -1160,7 +1160,7 @@ compact a-b 0.0: 000007:[f#13,SET-f#13,SET] 1: - 000009:[a#11,SETWITHDEL-c#inf,RANGEDEL] + 000009:[a#11,SET-c#inf,RANGEDEL] 000010:[c#12,SET-d#inf,RANGEDEL] 000011:[d#10,RANGEDEL-e#inf,RANGEDEL] 2: @@ -1171,7 +1171,7 @@ compact d-e 0.0: 000007:[f#13,SET-f#13,SET] 1: - 000009:[a#11,SETWITHDEL-c#inf,RANGEDEL] + 000009:[a#11,SET-c#inf,RANGEDEL] 000010:[c#12,SET-d#inf,RANGEDEL] 3: 000012:[d#10,RANGEDEL-e#inf,RANGEDEL] @@ -1184,7 +1184,7 @@ c:v compact f-f L0 ---- 1: - 000009:[a#11,SETWITHDEL-c#inf,RANGEDEL] + 000009:[a#11,SET-c#inf,RANGEDEL] 000010:[c#12,SET-d#inf,RANGEDEL] 000007:[f#13,SET-f#13,SET] 3: @@ -1193,8 +1193,8 @@ compact f-f L0 compact a-f L1 ---- 2: - 000013:[a#11,SETWITHDEL-c#inf,RANGEDEL] - 000014:[c#12,SETWITHDEL-d#inf,RANGEDEL] + 000013:[a#11,SET-c#inf,RANGEDEL] + 000014:[c#12,SET-d#inf,RANGEDEL] 000015:[f#13,SET-f#13,SET] 3: 000012:[d#10,RANGEDEL-e#inf,RANGEDEL]