Skip to content

Commit

Permalink
storage: add MVCC range tombstone handling for scans and gets
Browse files Browse the repository at this point in the history
This patch adds MVCC range tombstone handling for scans and gets. In the
basic case, this simply means that point keys below an MVCC range
tombstone are not visible.

When tombstones are requested by the caller, the MVCC range tombstones
themselves are never exposed, to avoid having to explicitly handle these
throughout the codebase. Instead, synthetic MVCC point tombstones are
emitted at the start of MVCC range tombstones and wherever they overlap
a point key (above and below). Additionally, point gets return synthetic
point tombstones even if no existing point key exists.

The point tombstone synthesis is implemented by a new
`pointSynthesizingIterator`, an `MVCCIterator` implementation that wraps
an existing `MVCCIterator`. This allows e.g. `pebbleMVCCScanner` to
remain unchanged, and automatically take MVCC range tombstones into
account for e.g. conflict/uncertainty checks.

Point tombstone synthesis must be enabled even when the caller has not
requested tombstones, because they must always be taken into account for
conflict/uncertainty checks. However, in these cases we enable range key
masking below the read timestamp, omitting any covered points since
these are no longer needed.

Release note: None
  • Loading branch information
erikgrinaker committed May 1, 2022
1 parent 905b1df commit b79bdae
Show file tree
Hide file tree
Showing 8 changed files with 2,313 additions and 29 deletions.
1 change: 1 addition & 0 deletions pkg/storage/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ go_library(
"pebble_iterator.go",
"pebble_merge.go",
"pebble_mvcc_scanner.go",
"point_synthesizing_iter.go",
"replicas_storage.go",
"resource_limiter.go",
"row_counter.go",
Expand Down
82 changes: 54 additions & 28 deletions pkg/storage/mvcc.go
Original file line number Diff line number Diff line change
Expand Up @@ -689,12 +689,34 @@ func (opts *MVCCGetOptions) validate() error {
return nil
}

func newMVCCIterator(reader Reader, inlineMeta bool, opts IterOptions) MVCCIterator {
// newMVCCIterator is a convenience function that sets up a suitable iterator
// for internal MVCC operations. In particular, this never exposes range keys
// (i.e. MVCC range tombstones), but instead synthesizes MVCC point tombstones
// around existing point keys that overlap an MVCC range tombstone, as well as
// at the start of MVCC range tombstones. It does this even if tombstones are
// not returned, since they must be visible to conflict/uncertainty checks.
func newMVCCIterator(
reader Reader, timestamp hlc.Timestamp, returnTombstones bool, opts IterOptions,
) MVCCIterator {
if opts.KeyTypes != IterKeyTypePointsOnly {
panic(errors.AssertionFailedf("can't request range keys"))
}
opts.KeyTypes = IterKeyTypePointsAndRanges
// Disable separated intents if reading inline.
iterKind := MVCCKeyAndIntentsIterKind
if inlineMeta {
if timestamp.IsEmpty() {
iterKind = MVCCKeyIterKind
}
return reader.NewMVCCIterator(iterKind, opts)
// If we're not returning tombstones, enable range key masking.
if !returnTombstones && opts.RangeKeyMaskingBelow.IsEmpty() {
opts.RangeKeyMaskingBelow = timestamp
}
// Prefix iterators are generally used for point operations (e.g. gets or puts
// as opposed to scans). For these, we also synthesize point tombstones for
// MVCC range tombstones at a SeekGE key, to emit them even if no existing
// point key exists -- this is necessary for conflict/uncertainty checks.
emitOnSeekGE := opts.Prefix
return newPointSynthesizingIter(reader.NewMVCCIterator(iterKind, opts), emitOnSeekGE)
}

// MVCCGet returns the most recent value for the specified key whose timestamp
Expand All @@ -703,7 +725,10 @@ func newMVCCIterator(reader Reader, inlineMeta bool, opts IterOptions) MVCCItera
//
// In tombstones mode, if the most recent value is a deletion tombstone, the
// result will be a non-nil roachpb.Value whose RawBytes field is nil.
// Otherwise, a deletion tombstone results in a nil roachpb.Value.
// Otherwise, a deletion tombstone results in a nil roachpb.Value. MVCC range
// tombstones will be emitted as synthetic point tombstones (these synthetic
// point tombstones may not be visible to a scan if there is no existing point
// key at this key).
//
// In inconsistent mode, if an intent is encountered, it will be placed in the
// dedicated return parameter. By contrast, in consistent mode, an intent will
Expand All @@ -724,7 +749,7 @@ func newMVCCIterator(reader Reader, inlineMeta bool, opts IterOptions) MVCCItera
func MVCCGet(
ctx context.Context, reader Reader, key roachpb.Key, timestamp hlc.Timestamp, opts MVCCGetOptions,
) (*roachpb.Value, *roachpb.Intent, error) {
iter := newMVCCIterator(reader, timestamp.IsEmpty(), IterOptions{Prefix: true})
iter := newMVCCIterator(reader, timestamp, opts.Tombstones, IterOptions{Prefix: true})
defer iter.Close()
value, intent, err := mvccGet(ctx, iter, key, timestamp, opts)
return value.ToPointer(), intent, err
Expand Down Expand Up @@ -1063,10 +1088,7 @@ func MVCCDelete(
timestamp hlc.Timestamp,
txn *roachpb.Transaction,
) error {
iter := newMVCCIterator(rw, timestamp.IsEmpty(), IterOptions{
KeyTypes: IterKeyTypePointsAndRanges,
Prefix: true,
})
iter := newMVCCIterator(rw, timestamp, false /* returnTombstones */, IterOptions{Prefix: true})
defer iter.Close()

return mvccPutUsingIter(ctx, rw, iter, ms, key, timestamp, noValue, txn, nil /* valueFn */)
Expand Down Expand Up @@ -1687,10 +1709,7 @@ func MVCCIncrement(
txn *roachpb.Transaction,
inc int64,
) (int64, error) {
iter := newMVCCIterator(rw, timestamp.IsEmpty(), IterOptions{
KeyTypes: IterKeyTypePointsAndRanges,
Prefix: true,
})
iter := newMVCCIterator(rw, timestamp, false /* returnTombstones */, IterOptions{Prefix: true})
defer iter.Close()

var int64Val int64
Expand Down Expand Up @@ -1762,10 +1781,7 @@ func MVCCConditionalPut(
allowIfDoesNotExist CPutMissingBehavior,
txn *roachpb.Transaction,
) error {
iter := newMVCCIterator(rw, timestamp.IsEmpty(), IterOptions{
KeyTypes: IterKeyTypePointsAndRanges,
Prefix: true,
})
iter := newMVCCIterator(rw, timestamp, false /* returnTombstones */, IterOptions{Prefix: true})
defer iter.Close()

return mvccConditionalPutUsingIter(ctx, rw, iter, ms, key, timestamp, value, expVal, allowIfDoesNotExist, txn)
Expand Down Expand Up @@ -1843,10 +1859,7 @@ func MVCCInitPut(
failOnTombstones bool,
txn *roachpb.Transaction,
) error {
iter := newMVCCIterator(rw, timestamp.IsEmpty(), IterOptions{
KeyTypes: IterKeyTypePointsAndRanges,
Prefix: true,
})
iter := newMVCCIterator(rw, timestamp, false /* returnTombstones */, IterOptions{Prefix: true})
defer iter.Close()
return mvccInitPutUsingIter(ctx, rw, iter, ms, key, timestamp, value, failOnTombstones, txn)
}
Expand Down Expand Up @@ -2221,7 +2234,7 @@ func MVCCDeleteRange(

buf := newPutBuffer()
defer buf.release()
iter := newMVCCIterator(rw, timestamp.IsEmpty(), IterOptions{Prefix: true})
iter := newMVCCIterator(rw, timestamp, false /* returnTombstones */, IterOptions{Prefix: true})
defer iter.Close()

var keys []roachpb.Key
Expand Down Expand Up @@ -2741,7 +2754,10 @@ type MVCCScanResult struct {
// In tombstones mode, if the most recent value for a key is a deletion
// tombstone, the scan result will contain a roachpb.KeyValue for that key whose
// RawBytes field is nil. Otherwise, the key-value pair will be omitted from the
// result entirely.
// result entirely. For MVCC range tombstones, synthetic point tombstones are
// returned at the start of the range tombstone and when then overlap a point
// key (note that this may emit spurious point tombstones at the MVCCScan start
// key if it truncates an MVCC range tombstone).
//
// When scanning inconsistently, any encountered intents will be placed in the
// dedicated result parameter. By contrast, when scanning consistently, any
Expand All @@ -2764,7 +2780,10 @@ func MVCCScan(
timestamp hlc.Timestamp,
opts MVCCScanOptions,
) (MVCCScanResult, error) {
iter := newMVCCIterator(reader, timestamp.IsEmpty(), IterOptions{LowerBound: key, UpperBound: endKey})
iter := newMVCCIterator(reader, timestamp, opts.Tombstones, IterOptions{
LowerBound: key,
UpperBound: endKey,
})
defer iter.Close()
return mvccScanToKvs(ctx, iter, key, endKey, timestamp, opts)
}
Expand All @@ -2777,7 +2796,10 @@ func MVCCScanToBytes(
timestamp hlc.Timestamp,
opts MVCCScanOptions,
) (MVCCScanResult, error) {
iter := newMVCCIterator(reader, timestamp.IsEmpty(), IterOptions{LowerBound: key, UpperBound: endKey})
iter := newMVCCIterator(reader, timestamp, opts.Tombstones, IterOptions{
LowerBound: key,
UpperBound: endKey,
})
defer iter.Close()
return mvccScanToBytes(ctx, iter, key, endKey, timestamp, opts)
}
Expand Down Expand Up @@ -2811,7 +2833,9 @@ func MVCCScanAsTxn(
// the reverse flag is set, the iterator will be moved in reverse order. If the
// scan options specify an inconsistent scan, all "ignored" intents will be
// returned. In consistent mode, intents are only ever returned as part of a
// WriteIntentError.
// WriteIntentError. In Tombstones mode, MVCC range tombstones are emitted as
// synthetic point tombstones around overlapping point keys and the start of
// MVCC range tombstones.
func MVCCIterate(
ctx context.Context,
reader Reader,
Expand All @@ -2820,8 +2844,10 @@ func MVCCIterate(
opts MVCCScanOptions,
f func(roachpb.KeyValue) error,
) ([]roachpb.Intent, error) {
iter := newMVCCIterator(
reader, timestamp.IsEmpty(), IterOptions{LowerBound: key, UpperBound: endKey})
iter := newMVCCIterator(reader, timestamp, opts.Tombstones, IterOptions{
LowerBound: key,
UpperBound: endKey,
})
defer iter.Close()

var intents []roachpb.Intent
Expand Down
5 changes: 4 additions & 1 deletion pkg/storage/mvcc_history_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -70,7 +70,7 @@ import (
// put_rangekey k=<key> end=<key> ts=<int>[,<int>]
// scan [t=<name>] [ts=<int>[,<int>]] [resolve [status=<txnstatus>]] k=<key> [end=<key>] [inconsistent] [tombstones] [reverse] [failOnMoreRecent] [localUncertaintyLimit=<int>[,<int>]] [globalUncertaintyLimit=<int>[,<int>]] [max=<max>] [targetbytes=<target>] [avoidExcess] [allowEmpty]
//
// iter [k=<key>] [end=<key>] [kind=key|keyAndIntents] [types=pointsOnly|pointsWithRanges|pointsAndRanges|rangesOnly] [maskBelow=<int>[,<int>]]
// iter [k=<key>] [end=<key>] [kind=key|keyAndIntents] [types=pointsOnly|pointsWithRanges|pointsAndRanges|rangesOnly] [pointSynthesis [emitOnSeekGE]] [maskBelow=<int>[,<int>]]
// iter_seek_ge k=<key> [ts=<int>[,<int>]]
// iter_seek_lt k=<key> [ts=<int>[,<int>]]
// iter_seek_intent_ge k=<key> txn=<name>
Expand Down Expand Up @@ -1002,6 +1002,9 @@ func cmdIterNew(e *evalCtx) error {
MVCCIterator: r.NewMVCCIterator(kind, opts),
closeReader: closeReader,
}
if e.hasArg("pointSynthesis") {
e.iter = newPointSynthesizingIter(e.iter, e.hasArg("emitOnSeekGE"))
}
return nil
}

Expand Down
6 changes: 6 additions & 0 deletions pkg/storage/mvcc_key.go
Original file line number Diff line number Diff line change
Expand Up @@ -62,6 +62,12 @@ func (k MVCCKey) Next() MVCCKey {
}
}

// Clone returns a copy of the key.
func (k MVCCKey) Clone() MVCCKey {
k.Key = k.Key.Clone()
return k
}

// Compare returns -1 if this key is less than the given key, 0 if they're
// equal, or 1 if this is greater. Comparison is by key,timestamp, where larger
// timestamps sort before smaller ones except empty ones which sort first (like
Expand Down
Loading

0 comments on commit b79bdae

Please sign in to comment.