Skip to content

Commit

Permalink
db: remove compaction iter seqnum optimization, replace with assert
Browse files Browse the repository at this point in the history
Previously the compaction iterator would avoid user key comparisons in some
circumstances when invariants around sequence numbers implied user keys must be
different. This commit removes this optimization, replacing it with an
assertion that the sequence number invariants hold.

This is done out of caution and with the realization that these user key
comparisons form a relatively small portion of the CPU cost of a compaction.

Informs #3082.
Informs cockroachdb/cockroach#114421.
  • Loading branch information
jbowens committed Nov 20, 2023
1 parent b9be64b commit 2a2be74
Show file tree
Hide file tree
Showing 2 changed files with 24 additions and 20 deletions.
39 changes: 19 additions & 20 deletions compaction_iter.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@ import (
"github.com/cockroachdb/errors"
"github.com/cockroachdb/pebble/internal/base"
"github.com/cockroachdb/pebble/internal/bytealloc"
"github.com/cockroachdb/pebble/internal/invariants"
"github.com/cockroachdb/pebble/internal/keyspan"
"github.com/cockroachdb/pebble/internal/rangekey"
)
Expand Down Expand Up @@ -638,28 +637,28 @@ func (i *compactionIter) nextInStripeHelper() stripeChangeType {
}
key := i.iterKey

// NB: The below conditional is an optimization to avoid a user key
// comparison in many cases. Internal keys with the same user key are
// ordered in (strictly) descending order by trailer. If the new key has a
// greater or equal trailer, or the previous key had a zero sequence number,
// the new key must have a new user key.
//
// A couple things make these cases common:
// - Sequence-number zeroing ensures ~all of the keys in L6 have a zero
// sequence number.
// - Ingested sstables' keys all adopt the same sequence number.
if i.keyTrailer <= base.InternalKeyZeroSeqnumMaxTrailer || key.Trailer >= i.keyTrailer {
if invariants.Enabled && i.equal(i.key.UserKey, key.UserKey) {
prevKey := i.key
prevKey.Trailer = i.keyTrailer
panic(fmt.Sprintf("pebble: invariant violation: %s and %s out of order", key, prevKey))
}
i.curSnapshotIdx, i.curSnapshotSeqNum = snapshotIndex(key.SeqNum(), i.snapshots)
return newStripeNewKey
} else if !i.equal(i.key.UserKey, key.UserKey) {
if !i.equal(i.key.UserKey, key.UserKey) {
i.curSnapshotIdx, i.curSnapshotSeqNum = snapshotIndex(key.SeqNum(), i.snapshots)
return newStripeNewKey
}

// If i.key and key have the same user key, then
// 1. i.key must not have had a zero sequence number (or it would've be the last
// key with its user key).
// 2. i.key must have a strictly larger sequence number
// There's an exception in that either key may be a range delete. Range
// deletes may share a sequence number with a point key if the keys were
// ingested together. Range keys may also share the sequence number if they
// were ingested, but range keys are interleaved into the compaction
// iterator's input iterator at the maximal sequence number so their
// original sequence number will not be observed here.
if prevSeqNum := base.SeqNumFromTrailer(i.keyTrailer); (prevSeqNum == 0 || prevSeqNum <= key.SeqNum()) &&
i.key.Kind() != InternalKeyKindRangeDelete && key.Kind() != InternalKeyKindRangeDelete {
prevKey := i.key
prevKey.Trailer = i.keyTrailer
panic(errors.AssertionFailedf("pebble: invariant violation: %s and %s out of order", prevKey, key))
}

origSnapshotIdx := i.curSnapshotIdx
i.curSnapshotIdx, i.curSnapshotSeqNum = snapshotIndex(key.SeqNum(), i.snapshots)
switch key.Kind() {
Expand Down
5 changes: 5 additions & 0 deletions internal/base/internal.go
Original file line number Diff line number Diff line change
Expand Up @@ -370,6 +370,11 @@ func (k InternalKey) SeqNum() uint64 {
return k.Trailer >> 8
}

// SeqNumFromTrailer returns the sequence number component of a trailer.
func SeqNumFromTrailer(t uint64) uint64 {
return t >> 8
}

// Visible returns true if the key is visible at the specified snapshot
// sequence number.
func (k InternalKey) Visible(snapshot, batchSnapshot uint64) bool {
Expand Down

0 comments on commit 2a2be74

Please sign in to comment.