From 93aaf52c3becc0416f25d62dcc11de056c50c44c Mon Sep 17 00:00:00 2001 From: sumeerbhola Date: Fri, 8 Dec 2023 15:44:47 -0500 Subject: [PATCH] db: add compaction callbacks for unexpected SingleDelete behavior Two callbacks are added to Options: - IneffectualSingleDeleteCallback is called when a SingleDelete is being elided without encountering a Set/Merge/SetWithDelete. - SingleDeleteInvariantViolationCallback is called when a SingleDelete has at least a pair of Set/Merge/SetWithDelete below it, since it violates the usage requirement of a SingleDelete. The metamorphic test now fails on the SingleDeleteInvariantViolationCallback. The support for Merge to be SingleDeleted, that was introduced in https://github.com/cockroachdb/pebble/pull/3118, is slightly improved in that SingleDelete consumes a single Merge instead of turning into a Delete. A subsequent change in CockroachDB will fatal or increment a metric (based on cluster settings) for these callbacks. Informs https://github.com/cockroachdb/cockroach/issues/115881 --- compaction.go | 4 +- compaction_iter.go | 255 ++++++++++++++++++++++---- compaction_iter_test.go | 22 ++- metamorphic/options.go | 7 + metamorphic/options_test.go | 6 +- options.go | 24 +++ testdata/compaction_iter | 28 ++- testdata/compaction_iter_delete_sized | 66 ++++++- testdata/compaction_iter_set_with_del | 28 ++- 9 files changed, 386 insertions(+), 54 deletions(-) diff --git a/compaction.go b/compaction.go index f7b1fb3544..464dfa5c62 100644 --- a/compaction.go +++ b/compaction.go @@ -3019,7 +3019,9 @@ func (d *DB) runCompaction( iiter = invalidating.MaybeWrapIfInvariants(iiter) iter := newCompactionIter(c.cmp, c.equal, c.formatKey, d.merge, iiter, snapshots, &c.rangeDelFrag, &c.rangeKeyFrag, c.allowedZeroSeqNum, c.elideTombstone, - c.elideRangeTombstone, d.FormatMajorVersion()) + c.elideRangeTombstone, d.opts.Experimental.IneffectualSingleDeleteCallback, + d.opts.Experimental.SingleDeleteInvariantViolationCallback, + d.FormatMajorVersion()) var ( createdFiles []base.DiskFileNum diff --git a/compaction_iter.go b/compaction_iter.go index 55ee67dc82..299dbfc983 100644 --- a/compaction_iter.go +++ b/compaction_iter.go @@ -256,10 +256,12 @@ type compactionIter struct { // The fragmented range keys. rangeKeys []keyspan.Span // Byte allocator for the tombstone keys. - alloc bytealloc.A - allowZeroSeqNum bool - elideTombstone func(key []byte) bool - elideRangeTombstone func(start, end []byte) bool + alloc bytealloc.A + allowZeroSeqNum bool + elideTombstone func(key []byte) bool + elideRangeTombstone func(start, end []byte) bool + ineffectualSingleDeleteCallback func(userKey []byte) + singleDeleteInvariantViolationCallback func(userKey []byte) // The on-disk format major version. This informs the types of keys that // may be written to disk during a compaction. formatVersion FormatMajorVersion @@ -281,20 +283,24 @@ func newCompactionIter( allowZeroSeqNum bool, elideTombstone func(key []byte) bool, elideRangeTombstone func(start, end []byte) bool, + ineffectualSingleDeleteCallback func(userKey []byte), + singleDeleteInvariantViolationCallback func(userKey []byte), formatVersion FormatMajorVersion, ) *compactionIter { i := &compactionIter{ - equal: equal, - merge: merge, - iter: iter, - snapshots: snapshots, - frontiers: frontiers{cmp: cmp}, - rangeDelFrag: rangeDelFrag, - rangeKeyFrag: rangeKeyFrag, - allowZeroSeqNum: allowZeroSeqNum, - elideTombstone: elideTombstone, - elideRangeTombstone: elideRangeTombstone, - formatVersion: formatVersion, + equal: equal, + merge: merge, + iter: iter, + snapshots: snapshots, + frontiers: frontiers{cmp: cmp}, + rangeDelFrag: rangeDelFrag, + rangeKeyFrag: rangeKeyFrag, + allowZeroSeqNum: allowZeroSeqNum, + elideTombstone: elideTombstone, + elideRangeTombstone: elideRangeTombstone, + ineffectualSingleDeleteCallback: ineffectualSingleDeleteCallback, + singleDeleteInvariantViolationCallback: singleDeleteInvariantViolationCallback, + formatVersion: formatVersion, } i.rangeDelFrag.Cmp = cmp i.rangeDelFrag.Format = formatKey @@ -455,13 +461,17 @@ func (i *compactionIter) Next() (*InternalKey, []byte) { // If we're at the last snapshot stripe and the tombstone // can be elided skip skippable keys in the same stripe. i.saveKey() - i.skipInStripe() + if i.key.Kind() == InternalKeyKindSingleDelete { + i.skipDueToSingleDeleteElision() + } else { + i.skipInStripe() + if !i.skip && i.iterStripeChange != newStripeNewKey { + panic(errors.AssertionFailedf("pebble: skipInStripe in last stripe disabled skip without advancing to new key")) + } + } if i.iterStripeChange == newStripeSameKey { panic(errors.AssertionFailedf("pebble: skipInStripe in last stripe found a new stripe within the same key")) } - if !i.skip && i.iterStripeChange != newStripeNewKey { - panic(errors.AssertionFailedf("pebble: skipInStripe in last stripe disabled skip without advancing to new key")) - } continue } else { // We're not at the last snapshot stripe, so the tombstone @@ -928,42 +938,82 @@ func (i *compactionIter) singleDeleteNext() bool { // 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 { + // 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. i.pos = iterPosNext return i.err == nil } if i.err != nil { panic(i.err) } + // INVARIANT: sameStripeSkippable. key := i.iterKey - switch key.Kind() { - case InternalKeyKindDelete, InternalKeyKindMerge, InternalKeyKindSetWithDelete, InternalKeyKindDeleteSized: - // We've hit a Delete, DeleteSized, Merge, SetWithDelete, transform + kind := key.Kind() + switch kind { + case InternalKeyKindDelete, InternalKeyKindSetWithDelete, InternalKeyKindDeleteSized: + if (kind == InternalKeyKindDelete || kind == InternalKeyKindDeleteSized) && + i.ineffectualSingleDeleteCallback != nil { + i.ineffectualSingleDeleteCallback(i.key.UserKey) + } + // We've hit a Delete, DeleteSized, SetWithDelete, transform // the SingleDelete into a full Delete. i.key.SetKind(InternalKeyKindDelete) i.skip = true return true - case InternalKeyKindSet: - // This SingleDelete deletes the Set, and we can now elide the + case InternalKeyKindSet, InternalKeyKindMerge: + // This SingleDelete deletes the Set/Merge, and we can now elide the // SingleDel as well. We advance past the Set and return false to // indicate to the main compaction loop that we should NOT yield the // current SingleDel key to the compaction loop. - i.nextInStripe() - // TODO(jackson): We could assert that nextInStripe either a) - // stepped onto a new key, or b) stepped on to a Delete, DeleteSized - // or SingleDel key. This would detect improper uses of SingleDel, - // but only when all three internal keys meet in the same compaction - // which is not likely. + // + // NB: singleDeleteNext was called with i.pos == iterPosCurForward, and + // after the call to nextInStripe, we are still at iterPosCurForward, + // since we are at the key after the Set/Merge that was single deleted. + change := i.nextInStripe() + switch change { + case sameStripeSkippable, 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 + // 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 || + i.rangeDelFrag.Covers(*i.iterKey, i.curSnapshotSeqNum) == keyspan.NoCover { + i.singleDeleteInvariantViolationCallback(i.key.UserKey) + } + } + case InternalKeyKindDelete, InternalKeyKindDeleteSized, InternalKeyKindSingleDelete, + InternalKeyKindRangeDelete: + 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") + } i.valid = false return false case InternalKeyKindSingleDelete: - // Two single deletes met in a compaction. With proper deterministic - // use of SingleDelete, this should never happen. The expectation is - // that there's exactly 1 set beneath a single delete. Currently, we - // opt to skip it. - // TODO(jackson): Should we make this an error? This would also - // allow us to simplify the code a bit by removing the for loop. + // Two single deletes met in a compaction. The first single delete is + // ineffectual. + if i.ineffectualSingleDeleteCallback != nil { + i.ineffectualSingleDeleteCallback(i.key.UserKey) + } + // Continue to apply the second single delete. continue default: @@ -974,6 +1024,135 @@ func (i *compactionIter) singleDeleteNext() bool { } } +// skipDueToSingleDeleteElision is called when the SingleDelete is being +// elided because it is in the final snapshot stripe and there are no keys +// with the same user key in lower levels in the LSM (below the files in this +// compaction). +// +// TODO(sumeer): the only difference between singleDeleteNext and +// skipDueToSingleDeleteElision is the fact that the caller knows it will be +// eliding the single delete in the latter case. There are some similar things +// happening in both implementations. My first attempt at combining them into +// a single method was hard to comprehend. Try again. +func (i *compactionIter) skipDueToSingleDeleteElision() { + for { + stripeChange := i.nextInStripe() + if i.err != nil { + panic(i.err) + } + switch stripeChange { + case newStripeNewKey: + // The single delete is only now being elided, meaning it did not elide + // any keys earlier in its descent down the LSM. We stepped onto a new + // user key, meaning that even now at its moment of elision, it still + // hasn't elided any other keys. The single delete was ineffectual (a + // no-op). + if i.ineffectualSingleDeleteCallback != nil { + i.ineffectualSingleDeleteCallback(i.key.UserKey) + } + i.skip = false + return + case newStripeSameKey: + // This should be impossible. If we're eliding a single delete, we + // 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: + kind := i.iterKey.Kind() + switch kind { + case InternalKeyKindDelete, InternalKeyKindDeleteSized, InternalKeyKindSingleDelete: + if i.ineffectualSingleDeleteCallback != nil { + i.ineffectualSingleDeleteCallback(i.key.UserKey) + } + switch kind { + case InternalKeyKindDelete, InternalKeyKindDeleteSized: + i.skipInStripe() + return + case InternalKeyKindSingleDelete: + // Repeat the same with this SingleDelete. We don't want to simply + // call skipInStripe(), since it increases the strength of the + // SingleDel, which hides bugs in the use of single delete. + continue + default: + panic(errors.AssertionFailedf( + "unexpected internal key kind: %d", errors.Safe(i.iterKey.Kind()))) + } + case InternalKeyKindSetWithDelete: + // The SingleDelete should behave like a Delete. + i.skipInStripe() + return + case InternalKeyKindSet, InternalKeyKindMerge: + // This SingleDelete deletes the Set/Merge, and we are eliding the + // SingleDel as well. Step to the next key (this is not deleted by the + // SingleDelete). + // + // NB: skipDueToSingleDeleteElision was called with i.pos == + // iterPosCurForward, and after the call to nextInStripe, we are still + // at iterPosCurForward, since we are at the key after the Set/Merge + // that was single deleted. + change := i.nextInStripe() + if i.err != nil { + panic(i.err) + } + switch change { + case newStripeSameKey: + panic(errors.AssertionFailedf("eliding single delete followed by same key in new stripe")) + case newStripeNewKey: + case sameStripeSkippable: + // On the same key. + nextKind := i.iterKey.Kind() + switch nextKind { + case InternalKeyKindSet, InternalKeyKindSetWithDelete, InternalKeyKindMerge: + if i.singleDeleteInvariantViolationCallback != nil { + i.singleDeleteInvariantViolationCallback(i.key.UserKey) + } + case InternalKeyKindDelete, InternalKeyKindDeleteSized, InternalKeyKindSingleDelete, + InternalKeyKindRangeDelete: + 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") + } + // Whether in same stripe or new stripe, this key is not consumed by + // the SingleDelete. + i.skip = false + return + default: + panic(errors.AssertionFailedf( + "unexpected internal key kind: %d", errors.Safe(i.iterKey.Kind()))) + } + default: + panic("unreachable") + } + } +} + // deleteSizedNext processes a DELSIZED point tombstone. Unlike ordinary DELs, // these tombstones carry a value that's a varint indicating the size of the // entry (len(key)+len(value)) that the tombstone is expected to delete. @@ -1025,8 +1204,8 @@ func (i *compactionIter) deleteSizedNext() (*base.InternalKey, []byte) { // We can differentiate these two cases by examining the length of // the DELSIZED's value. A DELSIZED's value holds the size of both // the user key and value that it intends to delete. For any user - // key with a length > 1, a DELSIZED that has not deleted a key must - // have a value with a length > 1. + // key with a length > 0, a DELSIZED that has not deleted a key must + // have a value with a length > 0. // // We treat both cases the same functionally, adopting the identity // of the lower-sequence numbered tombstone. However in the second diff --git a/compaction_iter_test.go b/compaction_iter_test.go index 02b1d3541a..24f0259078 100644 --- a/compaction_iter_test.go +++ b/compaction_iter_test.go @@ -99,6 +99,12 @@ func TestCompactionIter(t *testing.T) { return "testdata/compaction_iter_delete_sized" } + var ineffectualSingleDeleteKeys []string + var invariantViolationSingleDeleteKeys []string + resetSingleDelStats := func() { + ineffectualSingleDeleteKeys = ineffectualSingleDeleteKeys[:0] + invariantViolationSingleDeleteKeys = invariantViolationSingleDeleteKeys[:0] + } newIter := func(formatVersion FormatMajorVersion) *compactionIter { // To adhere to the existing assumption that range deletion blocks in // SSTables are not released while iterating, and therefore not @@ -119,7 +125,7 @@ func TestCompactionIter(t *testing.T) { return m, nil } } - + resetSingleDelStats() return newCompactionIter( DefaultComparer.Compare, DefaultComparer.Equal, @@ -136,6 +142,12 @@ func TestCompactionIter(t *testing.T) { func(_, _ []byte) bool { return elideTombstones }, + func(userKey []byte) { + ineffectualSingleDeleteKeys = append(ineffectualSingleDeleteKeys, string(userKey)) + }, + func(userKey []byte) { + invariantViolationSingleDeleteKeys = append(invariantViolationSingleDeleteKeys, string(userKey)) + }, formatVersion, ) } @@ -301,6 +313,14 @@ func TestCompactionIter(t *testing.T) { if printMissizedDels { fmt.Fprintf(&b, "missized-dels=%d\n", iter.stats.countMissizedDels) } + if len(ineffectualSingleDeleteKeys) > 0 { + fmt.Fprintf(&b, "ineffectual-single-deletes: %s\n", + strings.Join(ineffectualSingleDeleteKeys, ",")) + } + if len(invariantViolationSingleDeleteKeys) > 0 { + fmt.Fprintf(&b, "invariant-violation-single-deletes: %s\n", + strings.Join(invariantViolationSingleDeleteKeys, ",")) + } return b.String() default: diff --git a/metamorphic/options.go b/metamorphic/options.go index 66c4a15313..5bf0ec688d 100644 --- a/metamorphic/options.go +++ b/metamorphic/options.go @@ -219,6 +219,13 @@ func defaultOptions() *pebble.Options { }}, BlockPropertyCollectors: blockPropertyCollectorConstructors, } + // TODO(sumeer): add IneffectualSingleDeleteCallback that panics by + // supporting a test option that does not generate ineffectual single + // deletes. + opts.Experimental.SingleDeleteInvariantViolationCallback = func( + userKey []byte) { + panic(errors.AssertionFailedf("single del invariant violations on key %q", userKey)) + } return opts } diff --git a/metamorphic/options_test.go b/metamorphic/options_test.go index 2b7f81be68..a09c0e21b0 100644 --- a/metamorphic/options_test.go +++ b/metamorphic/options_test.go @@ -69,10 +69,12 @@ func TestOptionsRoundtrip(t *testing.T) { "BlockPropertyCollectors:", "EventListener:", "MaxConcurrentCompactions:", - "Experimental.EnableValueBlocks:", "Experimental.DisableIngestAsFlushable:", - "Experimental.RemoteStorage:", + "Experimental.EnableValueBlocks:", + "Experimental.IneffectualSingleDeleteCallback:", "Experimental.IngestSplit:", + "Experimental.RemoteStorage:", + "Experimental.SingleDeleteInvariantViolationCallback:", // Floating points "Experimental.PointTombstoneWeight:", } diff --git a/options.go b/options.go index c4c6b12387..f777daa624 100644 --- a/options.go +++ b/options.go @@ -699,6 +699,30 @@ type Options struct { // CacheSizeBytesBytes is the size of the on-disk block cache for objects // on shared storage in bytes. If it is 0, no cache is used. SecondaryCacheSizeBytes int64 + + // IneffectualPointDeleteCallback is called in compactions/flushes if any + // single delete is being elided without deleting a point set/merge. + IneffectualSingleDeleteCallback func(userKey []byte) + + // SingleDeleteInvariantViolationCallback is called in compactions/flushes if any + // single delete has consumed a Set/Merge, and there is another immediately older + // Set/SetWithDelete/Merge. The user of Pebble has violated the invariant under + // which SingleDelete can be used correctly. + // + // Consider the sequence SingleDelete#3, Set#2, Set#1. There are three + // ways some of these keys can first meet in a compaction. + // + // - All 3 keys in the same compaction: this callback will detect the + // violation. + // + // - SingleDelete#3, Set#2 meet in a compaction first: Both keys will + // disappear. The violation will not be detected, and the DB will have + // Set#1 which is likely incorrect (from the user's perspective). + // + // - Set#2, Set#1 meet in a compaction first: The output will be Set#2, + // which will later be consumed by SingleDelete#3. The violation will + // not be detected and the DB will be correct. + SingleDeleteInvariantViolationCallback func(userKey []byte) } // Filters is a map from filter policy name to filter policy. It is used for diff --git a/testdata/compaction_iter b/testdata/compaction_iter index f4d652b816..b156f00101 100644 --- a/testdata/compaction_iter +++ b/testdata/compaction_iter @@ -682,6 +682,7 @@ iter elide-tombstones=true first ---- . +ineffectual-single-deletes: a define a.SINGLEDEL.2: @@ -694,6 +695,7 @@ next ---- a#2,7: . +ineffectual-single-deletes: a define a.SINGLEDEL.3: @@ -705,6 +707,7 @@ iter first ---- . +ineffectual-single-deletes: a define a.SET.3:a @@ -720,6 +723,7 @@ next a#3,1:a b#2,0: . +ineffectual-single-deletes: b define a.SINGLEDEL.2: @@ -732,11 +736,13 @@ next ---- a#2,0: . +ineffectual-single-deletes: a iter elide-tombstones=true first ---- . +ineffectual-single-deletes: a define a.SINGLEDEL.2: @@ -745,9 +751,7 @@ a.MERGE.1: iter first -next ---- -a#2,0: . iter elide-tombstones=true @@ -777,6 +781,15 @@ next a#2,1:b . +# We don't notice the ineffectual single delete since the SET causes all +# SingleDelete error checking to be skipped. +iter elide-tombstones=true +first +next +---- +a#2,1:b +. + define a.MERGE.6:b a.SINGLEDEL.5: @@ -791,7 +804,8 @@ a#6,18:b[base] . # Non-deterministic use of SINGLEDEL where there are two older SETs that have -# not been deleted or single deleted. It is permitted to shadow both. +# not been deleted or single deleted. It is permitted to shadow both, since +# MERGE turns into a SETWITHDELETE when it meets the SINGLEDEL. define a.MERGE.6:b a.SINGLEDEL.5: @@ -831,6 +845,7 @@ next ---- a#1,1:a . +invariant-violation-single-deletes: a define a.SINGLEDEL.3: @@ -838,12 +853,14 @@ a.MERGE.2:b a.MERGE.1:a ---- +# SINGLEDEL consumes the first MERGE. iter first next ---- -a#3,0: +a#1,2:a . +invariant-violation-single-deletes: a define a.SINGLEDEL.4: @@ -918,6 +935,7 @@ next ---- a#2,1:ab[base] . +invariant-violation-single-deletes: a iter snapshots=2 first @@ -927,6 +945,7 @@ next a#2,2:b a#1,1:a . +invariant-violation-single-deletes: a iter snapshots=3 first @@ -934,6 +953,7 @@ next ---- a#2,1:ab[base] . +invariant-violation-single-deletes: a iter snapshots=(2,3,4) first diff --git a/testdata/compaction_iter_delete_sized b/testdata/compaction_iter_delete_sized index d841bcbd44..950ec82cf6 100644 --- a/testdata/compaction_iter_delete_sized +++ b/testdata/compaction_iter_delete_sized @@ -679,6 +679,7 @@ iter elide-tombstones=true first ---- . +ineffectual-single-deletes: a define a.SINGLEDEL.2: @@ -691,6 +692,7 @@ next ---- a#2,7: . +ineffectual-single-deletes: a define a.SINGLEDEL.3: @@ -702,6 +704,7 @@ iter first ---- . +ineffectual-single-deletes: a define a.SET.3:a @@ -717,6 +720,7 @@ next a#3,1:a b#2,0: . +ineffectual-single-deletes: b define a.SINGLEDEL.2: @@ -729,11 +733,13 @@ next ---- a#2,0: . +ineffectual-single-deletes: a iter elide-tombstones=true first ---- . +ineffectual-single-deletes: a define a.SINGLEDEL.2: @@ -742,9 +748,7 @@ a.MERGE.1: iter first -next ---- -a#2,0: . iter elide-tombstones=true @@ -776,6 +780,15 @@ next a#2,18:b . +# We don't notice the ineffectual single delete since the SET causes all +# SingleDelete error checking to be skipped. +iter elide-tombstones=true +first +next +---- +a#2,18:b +. + define a.MERGE.6:b a.SINGLEDEL.5: @@ -790,7 +803,8 @@ a#6,18:b[base] . # Non-deterministic use of SINGLEDEL where there are two older SETs that have -# not been deleted or single deleted. It is permitted to shadow both. +# not been deleted or single deleted. It is permitted to shadow both, since +# MERGE turns into a SETWITHDELETE when it meets the SINGLEDEL. define a.MERGE.6:b a.SINGLEDEL.5: @@ -830,6 +844,7 @@ next ---- a#1,1:a . +invariant-violation-single-deletes: a define a.SINGLEDEL.3: @@ -837,12 +852,14 @@ a.MERGE.2:b a.MERGE.1:a ---- +# SINGLEDEL consumes the first MERGE. iter first next ---- -a#3,0: +a#1,2:a . +invariant-violation-single-deletes: a define a.SINGLEDEL.4: @@ -917,6 +934,7 @@ next ---- a#2,1:ab[base] . +invariant-violation-single-deletes: a iter snapshots=2 first @@ -926,6 +944,7 @@ next a#2,2:b a#1,1:a . +invariant-violation-single-deletes: a iter snapshots=3 first @@ -933,6 +952,7 @@ next ---- a#2,1:ab[base] . +invariant-violation-single-deletes: a iter snapshots=(2,3,4) first @@ -1780,6 +1800,25 @@ a#5,15:d . . +define +a.SINGLEDEL.6: +a.SET.5:foo +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. +iter +first +next +next +---- +a#4,15:d +a#4,1:bar +. + define a.SINGLEDEL.6: a.SETWITHDEL.5:foo @@ -1894,3 +1933,22 @@ next ---- a#5,18:foo[base] . + +define +a.SINGLEDEL.4: +a.MERGE.3:a3 +a.SET.2:a2 +b.SINGLEDEL.6: +b.SET.5:b5 +b.SETWITHDEL.4:b4 +---- + +iter +first +next +next +---- +a#2,1:a2 +b#4,18:b4 +. +invariant-violation-single-deletes: a,b diff --git a/testdata/compaction_iter_set_with_del b/testdata/compaction_iter_set_with_del index f5f08f79ce..620c1cf3b8 100644 --- a/testdata/compaction_iter_set_with_del +++ b/testdata/compaction_iter_set_with_del @@ -679,6 +679,7 @@ iter elide-tombstones=true first ---- . +ineffectual-single-deletes: a define a.SINGLEDEL.2: @@ -691,6 +692,7 @@ next ---- a#2,7: . +ineffectual-single-deletes: a define a.SINGLEDEL.3: @@ -702,6 +704,7 @@ iter first ---- . +ineffectual-single-deletes: a define a.SET.3:a @@ -717,6 +720,7 @@ next a#3,1:a b#2,0: . +ineffectual-single-deletes: b define a.SINGLEDEL.2: @@ -729,11 +733,13 @@ next ---- a#2,0: . +ineffectual-single-deletes: a iter elide-tombstones=true first ---- . +ineffectual-single-deletes: a define a.SINGLEDEL.2: @@ -742,9 +748,7 @@ a.MERGE.1: iter first -next ---- -a#2,0: . iter elide-tombstones=true @@ -776,6 +780,15 @@ next a#2,18:b . +# We don't notice the ineffectual single delete since the SET causes all +# SingleDelete error checking to be skipped. +iter elide-tombstones=true +first +next +---- +a#2,18:b +. + define a.MERGE.6:b a.SINGLEDEL.5: @@ -790,7 +803,8 @@ a#6,18:b[base] . # Non-deterministic use of SINGLEDEL where there are two older SETs that have -# not been deleted or single deleted. It is permitted to shadow both. +# not been deleted or single deleted. It is permitted to shadow both, since +# MERGE turns into a SETWITHDELETE when it meets the SINGLEDEL. define a.MERGE.6:b a.SINGLEDEL.5: @@ -830,6 +844,7 @@ next ---- a#1,1:a . +invariant-violation-single-deletes: a define a.SINGLEDEL.3: @@ -837,12 +852,14 @@ a.MERGE.2:b a.MERGE.1:a ---- +# SINGLEDEL consumes the first MERGE. iter first next ---- -a#3,0: +a#1,2:a . +invariant-violation-single-deletes: a define a.SINGLEDEL.4: @@ -917,6 +934,7 @@ next ---- a#2,1:ab[base] . +invariant-violation-single-deletes: a iter snapshots=2 first @@ -926,6 +944,7 @@ next a#2,2:b a#1,1:a . +invariant-violation-single-deletes: a iter snapshots=3 first @@ -933,6 +952,7 @@ next ---- a#2,1:ab[base] . +invariant-violation-single-deletes: a iter snapshots=(2,3,4) first