Skip to content

Commit

Permalink
Merge #86056 #86084
Browse files Browse the repository at this point in the history
86056: storage: add `MVCCIterator.RangeKeyChanged()` r=jbowens a=erikgrinaker

This patch adds `RangeKeyChanged()` to the MVCC iterator interface,
returning `true` if the latest positioning operation caused the
overlapping range keys to change. Previously returned `RangeKeys()` and
`RangeBounds()` values are valid as long as this returns `false`.

Support has been implemented in most iterators, but not yet for
`MVCCIncrementalIterator` which is less trivial (in particular due to
the mixing of operations that respect and ignore time bounds via e.g.
`NextIgnoringTime`).

No callers make use of this yet.

`intentInterleavingIter` likely takes a slight performance hit from
this, but benchmarks are inconclusive. This will possibly be made up for
by replacing `HasPointAndRange()` calls in hot paths with
`RangeKeyChanged()`, even in the no-range-key case.

```
MVCCScan_Pebble/rows=1/versions=1/valueSize=64/numRangeKeys=0-24                 5.55µs ± 1%    5.56µs ± 1%     ~     (p=0.921 n=9+10)
MVCCScan_Pebble/rows=1/versions=1/valueSize=64/numRangeKeys=1-24                 12.1µs ± 1%    12.2µs ± 3%     ~     (p=0.196 n=10+10)
MVCCScan_Pebble/rows=1/versions=1/valueSize=64/numRangeKeys=100-24                165µs ± 1%     164µs ± 1%   -0.83%  (p=0.002 n=10+8)
MVCCScan_Pebble/rows=100/versions=1/valueSize=64/numRangeKeys=0-24               38.5µs ± 1%    39.0µs ± 2%   +1.38%  (p=0.002 n=10+10)
MVCCScan_Pebble/rows=100/versions=1/valueSize=64/numRangeKeys=1-24               71.8µs ± 1%    72.2µs ± 0%   +0.51%  (p=0.035 n=10+9)
MVCCScan_Pebble/rows=100/versions=1/valueSize=64/numRangeKeys=100-24              179µs ± 1%     177µs ± 1%   -0.88%  (p=0.002 n=10+8)
MVCCScan_Pebble/rows=10000/versions=1/valueSize=64/numRangeKeys=0-24             2.76ms ± 1%    2.79ms ± 0%   +1.16%  (p=0.000 n=10+10)
MVCCScan_Pebble/rows=10000/versions=1/valueSize=64/numRangeKeys=1-24             5.41ms ± 1%    5.48ms ± 1%   +1.21%  (p=0.000 n=10+10)
MVCCScan_Pebble/rows=10000/versions=1/valueSize=64/numRangeKeys=100-24           6.52ms ± 1%    6.53ms ± 2%     ~     (p=0.739 n=10+10)
MVCCReverseScan_Pebble/rows=1/versions=1/valueSize=64/numRangeKeys=0-24          5.96µs ± 1%    6.08µs ± 1%   +1.97%  (p=0.000 n=9+9)
MVCCReverseScan_Pebble/rows=1/versions=1/valueSize=64/numRangeKeys=1-24          12.1µs ± 1%    12.2µs ± 1%   +0.56%  (p=0.014 n=9+10)
MVCCReverseScan_Pebble/rows=1/versions=1/valueSize=64/numRangeKeys=100-24        92.3µs ± 1%    91.2µs ± 1%   -1.18%  (p=0.000 n=10+9)
MVCCReverseScan_Pebble/rows=100/versions=1/valueSize=64/numRangeKeys=0-24        51.7µs ± 2%    51.7µs ± 1%     ~     (p=0.529 n=10+10)
MVCCReverseScan_Pebble/rows=100/versions=1/valueSize=64/numRangeKeys=1-24        85.6µs ± 1%    86.0µs ± 1%     ~     (p=0.093 n=10+10)
MVCCReverseScan_Pebble/rows=100/versions=1/valueSize=64/numRangeKeys=100-24       183µs ± 1%     183µs ± 1%     ~     (p=0.481 n=10+10)
MVCCReverseScan_Pebble/rows=10000/versions=1/valueSize=64/numRangeKeys=0-24      3.97ms ± 1%    3.98ms ± 1%     ~     (p=0.063 n=10+10)
MVCCReverseScan_Pebble/rows=10000/versions=1/valueSize=64/numRangeKeys=1-24      6.70ms ± 1%    6.73ms ± 0%   +0.39%  (p=0.028 n=10+9)
MVCCReverseScan_Pebble/rows=10000/versions=1/valueSize=64/numRangeKeys=100-24    7.78ms ± 1%    7.84ms ± 1%   +0.77%  (p=0.005 n=10+10)
MVCCGet_Pebble/batch=false/versions=1/valueSize=8/numRangeKeys=0-24              5.35µs ± 2%    5.34µs ± 2%     ~     (p=0.493 n=10+10)
MVCCGet_Pebble/batch=false/versions=1/valueSize=8/numRangeKeys=1-24              11.4µs ± 1%    11.6µs ± 1%   +1.67%  (p=0.000 n=10+8)
MVCCGet_Pebble/batch=false/versions=1/valueSize=8/numRangeKeys=100-24             179µs ± 1%     176µs ± 1%   -1.66%  (p=0.000 n=10+9)
MVCCGet_Pebble/batch=false/versions=10/valueSize=8/numRangeKeys=0-24             6.47µs ± 2%    6.43µs ± 1%     ~     (p=0.315 n=10+10)
MVCCGet_Pebble/batch=false/versions=10/valueSize=8/numRangeKeys=1-24             17.3µs ± 1%    17.6µs ± 2%   +1.90%  (p=0.000 n=10+10)
MVCCGet_Pebble/batch=false/versions=10/valueSize=8/numRangeKeys=100-24            184µs ± 2%     182µs ± 1%   -0.91%  (p=0.010 n=10+9)
MVCCGet_Pebble/batch=false/versions=100/valueSize=8/numRangeKeys=0-24            15.2µs ± 2%    15.2µs ± 4%     ~     (p=0.631 n=10+10)
MVCCGet_Pebble/batch=false/versions=100/valueSize=8/numRangeKeys=1-24            58.0µs ± 2%    57.6µs ± 3%     ~     (p=0.218 n=10+10)
MVCCGet_Pebble/batch=false/versions=100/valueSize=8/numRangeKeys=100-24           230µs ± 1%     225µs ± 2%   -2.15%  (p=0.000 n=10+10)
MVCCGet_Pebble/batch=true/versions=1/valueSize=8/numRangeKeys=0-24               3.69µs ± 2%    3.63µs ± 1%   -1.60%  (p=0.001 n=9+10)
MVCCGet_Pebble/batch=true/versions=1/valueSize=8/numRangeKeys=1-24               8.59µs ± 1%    8.68µs ± 1%   +1.07%  (p=0.003 n=10+10)
MVCCGet_Pebble/batch=true/versions=1/valueSize=8/numRangeKeys=100-24              169µs ± 1%     165µs ± 1%   -2.17%  (p=0.000 n=10+10)
MVCCGet_Pebble/batch=true/versions=10/valueSize=8/numRangeKeys=0-24              4.63µs ± 1%    4.61µs ± 2%     ~     (p=0.356 n=9+10)
MVCCGet_Pebble/batch=true/versions=10/valueSize=8/numRangeKeys=1-24              14.2µs ± 1%    14.3µs ± 0%   +0.97%  (p=0.000 n=10+8)
MVCCGet_Pebble/batch=true/versions=10/valueSize=8/numRangeKeys=100-24             173µs ± 1%     169µs ± 1%   -2.26%  (p=0.000 n=10+10)
MVCCGet_Pebble/batch=true/versions=100/valueSize=8/numRangeKeys=0-24             11.9µs ± 4%    11.8µs ± 3%     ~     (p=0.720 n=10+9)
MVCCGet_Pebble/batch=true/versions=100/valueSize=8/numRangeKeys=1-24             51.9µs ± 1%    52.2µs ± 1%     ~     (p=0.156 n=10+9)
MVCCGet_Pebble/batch=true/versions=100/valueSize=8/numRangeKeys=100-24            213µs ± 1%     209µs ± 2%   -1.74%  (p=0.000 n=10+10)
```

Touches #84379.

Release note: None

86084: storage: omit `MVCCIterator.Stats()` call during `MVCCScan`/`MVCCGet` r=nicktrav a=erikgrinaker

`MVCCScan` and `MVCCGet` attach iterator statistics to an active trace.
However, they would fetch the iterator statistics even if no trace was
active, which has a significant cost. This patch avoids fetching the
stats when there is no active trace.

```
name                                                                 old time/op    new time/op    delta
MVCCGet_Pebble/batch=true/versions=1/valueSize=8/numRangeKeys=0-24     3.59µs ± 0%    3.34µs ± 0%  -7.02%  (p=0.000 n=10+8)
MVCCGet_Pebble/batch=true/versions=1/valueSize=8/numRangeKeys=1-24     7.43µs ± 1%    7.01µs ± 0%  -5.70%  (p=0.000 n=10+9)
MVCCGet_Pebble/batch=true/versions=10/valueSize=8/numRangeKeys=0-24    4.62µs ± 1%    4.36µs ± 1%  -5.53%  (p=0.000 n=10+10)
MVCCGet_Pebble/batch=true/versions=10/valueSize=8/numRangeKeys=1-24    10.4µs ± 1%    10.0µs ± 1%  -3.79%  (p=0.000 n=10+10)
```

Resolves #86083.
Touches #64503.

Release note: None

Co-authored-by: Erik Grinaker <[email protected]>
  • Loading branch information
craig[bot] and erikgrinaker committed Aug 13, 2022
3 parents 99d6225 + 179f935 + c739468 commit dff8a8b
Show file tree
Hide file tree
Showing 20 changed files with 514 additions and 352 deletions.
5 changes: 5 additions & 0 deletions pkg/kv/kvserver/rangefeed/task_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -209,6 +209,11 @@ func (s *testIterator) RangeKeys() storage.MVCCRangeKeyStack {
return storage.MVCCRangeKeyStack{}
}

// RangeKeyChanged implements SimpleMVCCIterator.
func (s *testIterator) RangeKeyChanged() bool {
return false
}

func TestInitResolvedTSScan(t *testing.T) {
defer leaktest.AfterTest(t)()
startKey := roachpb.RKey("d")
Expand Down
10 changes: 10 additions & 0 deletions pkg/kv/kvserver/spanset/batch.go
Original file line number Diff line number Diff line change
Expand Up @@ -189,6 +189,11 @@ func (i *MVCCIterator) RangeKeys() storage.MVCCRangeKeyStack {
return i.i.RangeKeys()
}

// RangeKeyChanged implements SimpleMVCCIterator.
func (i *MVCCIterator) RangeKeyChanged() bool {
return i.i.RangeKeyChanged()
}

// FindSplitKey is part of the storage.MVCCIterator interface.
func (i *MVCCIterator) FindSplitKey(
start, end, minSplitKey roachpb.Key, targetSize int64,
Expand Down Expand Up @@ -356,6 +361,11 @@ func (i *EngineIterator) EngineRangeKeys() []storage.EngineRangeKeyValue {
return i.i.EngineRangeKeys()
}

// RangeKeyChanged is part of the storage.EngineIterator interface.
func (i *EngineIterator) RangeKeyChanged() bool {
return i.i.RangeKeyChanged()
}

// UnsafeEngineKey is part of the storage.EngineIterator interface.
func (i *EngineIterator) UnsafeEngineKey() (storage.EngineKey, error) {
return i.i.UnsafeEngineKey()
Expand Down
17 changes: 12 additions & 5 deletions pkg/storage/engine.go
Original file line number Diff line number Diff line change
Expand Up @@ -165,16 +165,20 @@ type SimpleMVCCIterator interface {
// comment on SimpleMVCCIterator.
HasPointAndRange() (bool, bool)
// RangeBounds returns the range bounds for the current range key, or an
// empty span if there are none. The returned keys are only valid until the
// next iterator call.
// empty span if there are none. The returned keys are valid until the
// range key changes, see RangeKeyChanged().
RangeBounds() roachpb.Span
// RangeKeys returns a stack of all range keys (with different timestamps) at
// the current key position. When at a point key, it will return all range
// keys overlapping that point key. The stack is only valid until the next
// iterator operation. For details on range keys, see comment on
// SimpleMVCCIterator, or this tech note:
// keys overlapping that point key. The stack is valid until the range key
// changes, see RangeKeyChanged().
//
// For details on range keys, see SimpleMVCCIterator comment, or tech note:
// https://github.com/cockroachdb/cockroach/blob/master/docs/tech-notes/mvcc-range-tombstones.md
RangeKeys() MVCCRangeKeyStack
// RangeKeyChanged returns true if the previous seek or step moved to a
// different range key (or none at all). This includes an exhausted iterator.
RangeKeyChanged() bool
}

// IteratorStats is returned from {MVCCIterator,EngineIterator}.Stats.
Expand Down Expand Up @@ -299,6 +303,9 @@ type EngineIterator interface {
EngineRangeBounds() (roachpb.Span, error)
// EngineRangeKeys returns the engine range keys at the current position.
EngineRangeKeys() []EngineRangeKeyValue
// RangeKeyChanged returns true if the previous seek or step moved to a
// different range key (or none at all). This includes an exhausted iterator.
RangeKeyChanged() bool
// UnsafeEngineKey returns the same value as EngineKey, but the memory is
// invalidated on the next call to {Next,NextKey,Prev,SeekGE,SeekLT,Close}.
// REQUIRES: latest positioning function returned valid=true.
Expand Down
53 changes: 52 additions & 1 deletion pkg/storage/intent_interleaving_iter.go
Original file line number Diff line number Diff line change
Expand Up @@ -133,6 +133,17 @@ type intentInterleavingIter struct {
//
// NB: This value is not valid for intentCmp != 0.
iterBareRangeAtIntent bool
// rangeKeyChanged keeps track of RangeKeyChanged() for the current
// iterator position. This can't simply call through to the parent
// iterator for two reasons:
//
// - maybeSkipIntentRangeKey() may step the iterator forward from
// a bare range key onto a provisional value, which would cause
// RangeKeyChanged() to return false rather than true.
//
// - reverse iteration may prematurely move onto a range key when
// positioned on an intent not overlapping the range key.
rangeKeyChanged bool
// The current direction. +1 for forward, -1 for reverse.
dir int
valid bool
Expand Down Expand Up @@ -379,6 +390,20 @@ func (i *intentInterleavingIter) maybeSkipIntentRangeKey() error {
return nil
}

// maybeSuppressRangeKeyChanged will suppress i.rangeKeyChanged in the reverse
// direction if the underlying iterator has moved past an intent onto a
// different range key that should not be surfaced yet. Must be called after
// computePos().
func (i *intentInterleavingIter) maybeSuppressRangeKeyChanged() {
if util.RaceEnabled && i.dir > 0 {
panic(errors.AssertionFailedf("maybeSuppressRangeKeyChanged called in forward direction"))
}
if i.rangeKeyChanged && i.isCurAtIntentIterReverse() && i.intentCmp > 0 &&
i.iter.RangeBounds().EndKey.Compare(i.intentKey) <= 0 {
i.rangeKeyChanged = false
}
}

func (i *intentInterleavingIter) SeekGE(key MVCCKey) {
i.dir = +1
i.valid = true
Expand All @@ -391,6 +416,7 @@ func (i *intentInterleavingIter) SeekGE(key MVCCKey) {
if err := i.tryDecodeKey(); err != nil {
return
}
i.rangeKeyChanged = i.iter.RangeKeyChanged()
var intentSeekKey roachpb.Key
if key.Timestamp.IsEmpty() {
// Common case.
Expand Down Expand Up @@ -436,6 +462,7 @@ func (i *intentInterleavingIter) SeekIntentGE(key roachpb.Key, txnUUID uuid.UUID
if err := i.tryDecodeKey(); err != nil {
return
}
i.rangeKeyChanged = i.iter.RangeKeyChanged()
var engineKey EngineKey
engineKey, i.intentKeyBuf = LockTableKey{
Key: key,
Expand Down Expand Up @@ -619,6 +646,7 @@ func (i *intentInterleavingIter) Next() {
if err := i.tryDecodeKey(); err != nil {
return
}
i.rangeKeyChanged = i.iter.RangeKeyChanged()
var limitKey roachpb.Key
if i.iterValid && !i.prefix {
limitKey = i.makeUpperLimitKey()
Expand Down Expand Up @@ -649,6 +677,7 @@ func (i *intentInterleavingIter) Next() {
if err := i.tryDecodeKey(); err != nil {
return
}
i.rangeKeyChanged = i.iter.RangeKeyChanged()
i.intentCmp = 0
if !i.iterValid {
i.err = errors.Errorf("intent has no provisional value")
Expand Down Expand Up @@ -719,6 +748,7 @@ func (i *intentInterleavingIter) Next() {
if err = i.tryDecodeLockKey(iterState, err); err != nil {
return
}
i.rangeKeyChanged = false // already surfaced at the intent
// NB: doesn't need maybeSkipIntentRangeKey() as intentCmp > 0.
i.intentCmp = +1
if util.RaceEnabled && i.intentKey != nil {
Expand All @@ -737,6 +767,7 @@ func (i *intentInterleavingIter) Next() {
if err := i.tryDecodeKey(); err != nil {
return
}
i.rangeKeyChanged = i.iter.RangeKeyChanged()
if i.intentIterState == pebble.IterAtLimit && i.iterValid && !i.prefix {
// TODO(sumeer): could avoid doing this if i.iter has stepped to
// different version of same key.
Expand Down Expand Up @@ -777,6 +808,7 @@ func (i *intentInterleavingIter) NextKey() {
if err := i.tryDecodeKey(); err != nil {
return
}
i.rangeKeyChanged = i.iter.RangeKeyChanged()
var limitKey roachpb.Key
if i.iterValid && !i.prefix {
limitKey = i.makeUpperLimitKey()
Expand All @@ -798,6 +830,7 @@ func (i *intentInterleavingIter) NextKey() {
if err := i.tryDecodeKey(); err != nil {
return
}
i.rangeKeyChanged = i.iter.RangeKeyChanged()
if i.intentIterState == pebble.IterAtLimit && i.iterValid && !i.prefix {
limitKey := i.makeUpperLimitKey()
iterState, err := i.intentIter.NextEngineKeyWithLimit(limitKey)
Expand Down Expand Up @@ -940,6 +973,11 @@ func (i *intentInterleavingIter) RangeKeys() MVCCRangeKeyStack {
return i.iter.RangeKeys()
}

// RangeKeyChanged implements SimpleMVCCIterator.
func (i *intentInterleavingIter) RangeKeyChanged() bool {
return i.rangeKeyChanged
}

func (i *intentInterleavingIter) Close() {
i.iter.Close()
i.intentIter.Close()
Expand Down Expand Up @@ -1008,6 +1046,8 @@ func (i *intentInterleavingIter) SeekLT(key MVCCKey) {
return
}
i.computePos()
i.rangeKeyChanged = i.iter.RangeKeyChanged()
i.maybeSuppressRangeKeyChanged()
}

func (i *intentInterleavingIter) Prev() {
Expand All @@ -1034,6 +1074,8 @@ func (i *intentInterleavingIter) Prev() {
return
}
i.computePos()
i.rangeKeyChanged = i.iter.RangeKeyChanged()
i.maybeSuppressRangeKeyChanged()
return
}
// At least one of the iterators is not exhausted.
Expand All @@ -1059,6 +1101,8 @@ func (i *intentInterleavingIter) Prev() {
return
}
i.computePos()
i.rangeKeyChanged = i.iter.RangeKeyChanged()
i.maybeSuppressRangeKeyChanged()
} else {
// The intentIter is after the iter. We don't know whether the iter key
// has an intent. Note that the iter could itself be positioned at an
Expand All @@ -1069,6 +1113,7 @@ func (i *intentInterleavingIter) Prev() {
return
}
i.computePos()
i.maybeSuppressRangeKeyChanged()
}
}
if !i.valid {
Expand All @@ -1080,13 +1125,16 @@ func (i *intentInterleavingIter) Prev() {
// positioned on the start of a range key colocated with the intent.
// Stepping intentIter backward will ensure that intentKey is <= the key
// of iter (when neither is exhausted), but we may also need to step
// off the bare range key if there is one.
// off the bare range key if there is one, and account for the fact
// that the range key may have already changed on the intent.
if i.iterBareRangeAtIntent {
i.iter.Prev()
if err := i.tryDecodeKey(); err != nil {
return
}
}
i.rangeKeyChanged = i.iter.RangeKeyChanged() && (i.iterBareRangeAtIntent ||
i.iter.RangeBounds().EndKey.Compare(i.intentKey) <= 0)
var limitKey roachpb.Key
if i.iterValid {
limitKey = i.makeLowerLimitKey()
Expand All @@ -1108,6 +1156,7 @@ func (i *intentInterleavingIter) Prev() {
i.intentCmp = -1
if i.intentKey != nil {
i.computePos()
i.maybeSuppressRangeKeyChanged()
if i.intentCmp > 0 {
i.err = errors.Errorf("intentIter should not be after iter")
i.valid = false
Expand All @@ -1132,6 +1181,8 @@ func (i *intentInterleavingIter) Prev() {
}
}
i.computePos()
i.rangeKeyChanged = i.iter.RangeKeyChanged()
i.maybeSuppressRangeKeyChanged()
}
}

Expand Down
5 changes: 5 additions & 0 deletions pkg/storage/multi_iterator.go
Original file line number Diff line number Diff line change
Expand Up @@ -110,6 +110,11 @@ func (f *multiIterator) RangeKeys() MVCCRangeKeyStack {
panic("not implemented")
}

// RangeKeyChanged implements SimpleMVCCIterator.
func (f *multiIterator) RangeKeyChanged() bool {
panic("not implemented")
}

// Next advances the iterator to the next key/value in the iteration. After this
// call, Valid() will be true if the iterator was not positioned at the last
// key.
Expand Down
9 changes: 5 additions & 4 deletions pkg/storage/mvcc.go
Original file line number Diff line number Diff line change
Expand Up @@ -997,7 +997,7 @@ func mvccGet(
mvccScanner.get(ctx)

// If we have a trace, emit the scan stats that we produced.
recordIteratorStats(ctx, mvccScanner.stats())
recordIteratorStats(ctx, mvccScanner.parent)

if mvccScanner.err != nil {
return optionalValue{}, nil, mvccScanner.err
Expand Down Expand Up @@ -3363,12 +3363,13 @@ func MVCCDeleteRangeUsingTombstone(
return nil
}

func recordIteratorStats(ctx context.Context, iteratorStats IteratorStats) {
func recordIteratorStats(ctx context.Context, iter MVCCIterator) {
sp := tracing.SpanFromContext(ctx)
if sp.RecordingType() == tracingpb.RecordingOff {
// Short-circuit before allocating ScanStats object.
// Short-circuit before doing any work.
return
}
iteratorStats := iter.Stats()
stats := &iteratorStats.Stats
steps := stats.ReverseStepCount[pebble.InterfaceCall] + stats.ForwardStepCount[pebble.InterfaceCall]
seeks := stats.ReverseSeekCount[pebble.InterfaceCall] + stats.ForwardSeekCount[pebble.InterfaceCall]
Expand Down Expand Up @@ -3456,7 +3457,7 @@ func mvccScanToBytes(
res.NumBytes = mvccScanner.results.bytes

// If we have a trace, emit the scan stats that we produced.
recordIteratorStats(ctx, mvccScanner.stats())
recordIteratorStats(ctx, mvccScanner.parent)

res.Intents, err = buildScanIntents(mvccScanner.intentsRepr())
if err != nil {
Expand Down
Loading

0 comments on commit dff8a8b

Please sign in to comment.