Skip to content

Commit

Permalink
db: add compaction callbacks for unexpected SingleDelete behavior
Browse files Browse the repository at this point in the history
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
#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 cockroachdb/cockroach#115881
  • Loading branch information
sumeerbhola committed Dec 13, 2023
1 parent ba91feb commit 93aaf52
Show file tree
Hide file tree
Showing 9 changed files with 386 additions and 54 deletions.
4 changes: 3 additions & 1 deletion compaction.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
255 changes: 217 additions & 38 deletions compaction_iter.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -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
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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:
Expand All @@ -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.
Expand Down Expand Up @@ -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
Expand Down
22 changes: 21 additions & 1 deletion compaction_iter_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -119,7 +125,7 @@ func TestCompactionIter(t *testing.T) {
return m, nil
}
}

resetSingleDelStats()
return newCompactionIter(
DefaultComparer.Compare,
DefaultComparer.Equal,
Expand All @@ -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,
)
}
Expand Down Expand Up @@ -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:
Expand Down
7 changes: 7 additions & 0 deletions metamorphic/options.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}

Expand Down
Loading

0 comments on commit 93aaf52

Please sign in to comment.