Skip to content

Commit

Permalink
Merge #86512 #86521
Browse files Browse the repository at this point in the history
86512: kvserver/rangefeed: use `RangeKeyChanged()` in catchup scans r=tbg a=erikgrinaker

This avoids a key comparison in the hot path. However, we do not have
any benchmarks that emit range keys (they are always filtered out by the
`MVCCIncrementalIterator`), so it doesn't show up here:

```
name                                                                  old time/op  new time/op  delta
CatchUpScan/mixed-case/withDiff=true/perc=0.00/numRangeKeys=0-24       525ms ± 1%   520ms ± 1%    ~     (p=0.056 n=5+5)
CatchUpScan/mixed-case/withDiff=true/perc=0.00/numRangeKeys=1-24       570ms ± 0%   559ms ± 0%  -1.98%  (p=0.008 n=5+5)
CatchUpScan/mixed-case/withDiff=true/perc=0.00/numRangeKeys=100-24     624ms ± 1%   618ms ± 0%  -0.96%  (p=0.008 n=5+5)
CatchUpScan/mixed-case/withDiff=true/perc=50.00/numRangeKeys=0-24      1.03s ± 0%   1.02s ± 0%  -0.56%  (p=0.016 n=5+5)
CatchUpScan/mixed-case/withDiff=true/perc=50.00/numRangeKeys=1-24      1.21s ± 0%   1.21s ± 0%    ~     (p=0.056 n=5+5)
CatchUpScan/mixed-case/withDiff=true/perc=50.00/numRangeKeys=100-24    1.71s ± 0%   1.70s ± 1%    ~     (p=0.056 n=5+5)
CatchUpScan/mixed-case/withDiff=true/perc=75.00/numRangeKeys=0-24      898ms ± 0%   906ms ± 0%  +0.86%  (p=0.008 n=5+5)
CatchUpScan/mixed-case/withDiff=true/perc=75.00/numRangeKeys=1-24      1.08s ± 1%   1.07s ± 1%    ~     (p=0.421 n=5+5)
CatchUpScan/mixed-case/withDiff=true/perc=75.00/numRangeKeys=100-24    1.44s ± 0%   1.44s ± 0%    ~     (p=0.841 n=5+5)
CatchUpScan/mixed-case/withDiff=true/perc=95.00/numRangeKeys=0-24      190ms ± 1%   189ms ± 1%    ~     (p=0.056 n=5+5)
CatchUpScan/mixed-case/withDiff=true/perc=95.00/numRangeKeys=1-24      215ms ± 0%   215ms ± 1%    ~     (p=0.421 n=5+5)
CatchUpScan/mixed-case/withDiff=true/perc=95.00/numRangeKeys=100-24    325ms ± 1%   324ms ± 1%    ~     (p=0.548 n=5+5)
CatchUpScan/mixed-case/withDiff=true/perc=99.00/numRangeKeys=0-24      158ms ± 1%   158ms ± 1%    ~     (p=0.690 n=5+5)
CatchUpScan/mixed-case/withDiff=true/perc=99.00/numRangeKeys=1-24      181ms ± 0%   182ms ± 0%  +0.73%  (p=0.008 n=5+5)
CatchUpScan/mixed-case/withDiff=true/perc=99.00/numRangeKeys=100-24    294ms ± 0%   293ms ± 1%    ~     (p=0.421 n=5+5)
CatchUpScan/mixed-case/withDiff=false/perc=0.00/numRangeKeys=0-24      782ms ± 0%   788ms ± 1%  +0.81%  (p=0.008 n=5+5)
CatchUpScan/mixed-case/withDiff=false/perc=0.00/numRangeKeys=1-24      901ms ± 0%   906ms ± 0%  +0.55%  (p=0.008 n=5+5)
CatchUpScan/mixed-case/withDiff=false/perc=0.00/numRangeKeys=100-24    918ms ± 1%   920ms ± 0%    ~     (p=0.310 n=5+5)
CatchUpScan/mixed-case/withDiff=false/perc=50.00/numRangeKeys=0-24     663ms ± 1%   671ms ± 0%  +1.21%  (p=0.008 n=5+5)
CatchUpScan/mixed-case/withDiff=false/perc=50.00/numRangeKeys=1-24     793ms ± 0%   799ms ± 0%  +0.68%  (p=0.008 n=5+5)
CatchUpScan/mixed-case/withDiff=false/perc=50.00/numRangeKeys=100-24   803ms ± 0%   813ms ± 1%  +1.25%  (p=0.008 n=5+5)
CatchUpScan/mixed-case/withDiff=false/perc=75.00/numRangeKeys=0-24     594ms ± 0%   601ms ± 0%  +1.28%  (p=0.008 n=5+5)
CatchUpScan/mixed-case/withDiff=false/perc=75.00/numRangeKeys=1-24     722ms ± 0%   730ms ± 1%  +1.17%  (p=0.008 n=5+5)
CatchUpScan/mixed-case/withDiff=false/perc=75.00/numRangeKeys=100-24   733ms ± 0%   742ms ± 1%  +1.20%  (p=0.008 n=5+5)
CatchUpScan/mixed-case/withDiff=false/perc=95.00/numRangeKeys=0-24     172ms ± 0%   170ms ± 1%  -0.70%  (p=0.032 n=5+5)
CatchUpScan/mixed-case/withDiff=false/perc=95.00/numRangeKeys=1-24     194ms ± 0%   194ms ± 1%    ~     (p=0.421 n=5+5)
CatchUpScan/mixed-case/withDiff=false/perc=95.00/numRangeKeys=100-24   303ms ± 0%   303ms ± 1%    ~     (p=0.841 n=5+5)
CatchUpScan/mixed-case/withDiff=false/perc=99.00/numRangeKeys=0-24     153ms ± 0%   154ms ± 0%  +0.59%  (p=0.016 n=5+5)
CatchUpScan/mixed-case/withDiff=false/perc=99.00/numRangeKeys=1-24     177ms ± 1%   177ms ± 1%    ~     (p=0.690 n=5+5)
CatchUpScan/mixed-case/withDiff=false/perc=99.00/numRangeKeys=100-24   288ms ± 1%   288ms ± 0%    ~     (p=0.841 n=5+5)
```

Release justification: bug fixes and low-risk updates to new functionality

Release note: None

86521: storage: optimize `MVCCExportToSST` range key processing  r=tbg a=erikgrinaker

This patch optimizes `MVCCExportToSST` by making use of
`RangeKeyChanged()`, `CloneInto()`, and `PutRawMVCCRangeKey()`.
Unfortunately, this did not appear to have a significant impact on
performance in the 0/1 range key cases, as most of the gains were
already realized by previously migrating `MVCCIncrementalIterator` to
`RangeKeyChanged()`.

```
name                                                                                      old time/op  new time/op   delta
MVCCExportToSST/numKeys=64/numRevisions=1/exportAllRevisions=false/numRangeKeys=0-24      69.8µs ± 0%   69.9µs ± 1%    ~     (p=0.841 n=5+5)
MVCCExportToSST/numKeys=64/numRevisions=1/exportAllRevisions=false/numRangeKeys=1-24       116µs ± 7%    113µs ± 6%    ~     (p=0.548 n=5+5)
MVCCExportToSST/numKeys=64/numRevisions=1/exportAllRevisions=true/numRangeKeys=0-24       66.9µs ± 1%   66.6µs ± 1%    ~     (p=0.151 n=5+5)
MVCCExportToSST/numKeys=64/numRevisions=1/exportAllRevisions=true/numRangeKeys=1-24       98.4µs ± 7%  105.6µs ± 4%    ~     (p=0.095 n=5+5)
MVCCExportToSST/numKeys=64/numRevisions=10/exportAllRevisions=false/numRangeKeys=0-24      171µs ± 4%    166µs ± 4%    ~     (p=0.222 n=5+5)
MVCCExportToSST/numKeys=64/numRevisions=10/exportAllRevisions=false/numRangeKeys=1-24      229µs ± 4%    230µs ± 3%    ~     (p=0.841 n=5+5)
MVCCExportToSST/numKeys=64/numRevisions=10/exportAllRevisions=true/numRangeKeys=0-24       277µs ± 3%    270µs ± 2%    ~     (p=0.151 n=5+5)
MVCCExportToSST/numKeys=64/numRevisions=10/exportAllRevisions=true/numRangeKeys=1-24       345µs ± 2%    346µs ± 1%    ~     (p=1.000 n=5+5)
MVCCExportToSST/numKeys=1024/numRevisions=1/exportAllRevisions=false/numRangeKeys=0-24     617µs ± 1%    614µs ± 1%    ~     (p=0.310 n=5+5)
MVCCExportToSST/numKeys=1024/numRevisions=1/exportAllRevisions=false/numRangeKeys=1-24     762µs ± 1%    756µs ± 1%    ~     (p=0.151 n=5+5)
MVCCExportToSST/numKeys=1024/numRevisions=1/exportAllRevisions=true/numRangeKeys=0-24      571µs ± 2%    556µs ± 1%  -2.60%  (p=0.032 n=5+5)
MVCCExportToSST/numKeys=1024/numRevisions=1/exportAllRevisions=true/numRangeKeys=1-24      705µs ± 1%    700µs ± 1%    ~     (p=0.222 n=5+5)
MVCCExportToSST/numKeys=1024/numRevisions=10/exportAllRevisions=false/numRangeKeys=0-24   2.17ms ± 0%   2.19ms ± 1%  +0.80%  (p=0.008 n=5+5)
MVCCExportToSST/numKeys=1024/numRevisions=10/exportAllRevisions=false/numRangeKeys=1-24   2.87ms ± 1%   2.91ms ± 1%  +1.53%  (p=0.032 n=5+5)
MVCCExportToSST/numKeys=1024/numRevisions=10/exportAllRevisions=true/numRangeKeys=0-24    3.80ms ± 0%   3.79ms ± 1%    ~     (p=0.548 n=5+5)
MVCCExportToSST/numKeys=1024/numRevisions=10/exportAllRevisions=true/numRangeKeys=1-24    4.77ms ± 0%   4.75ms ± 0%    ~     (p=0.095 n=5+5)
MVCCExportToSST/numKeys=8192/numRevisions=1/exportAllRevisions=false/numRangeKeys=0-24    4.63ms ± 1%   4.57ms ± 1%  -1.24%  (p=0.008 n=5+5)
MVCCExportToSST/numKeys=8192/numRevisions=1/exportAllRevisions=false/numRangeKeys=1-24    5.61ms ± 0%   5.65ms ± 1%  +0.66%  (p=0.032 n=5+5)
MVCCExportToSST/numKeys=8192/numRevisions=1/exportAllRevisions=true/numRangeKeys=0-24     4.21ms ± 1%   4.14ms ± 0%  -1.69%  (p=0.008 n=5+5)
MVCCExportToSST/numKeys=8192/numRevisions=1/exportAllRevisions=true/numRangeKeys=1-24     5.23ms ± 0%   5.17ms ± 1%  -1.16%  (p=0.008 n=5+5)
MVCCExportToSST/numKeys=8192/numRevisions=10/exportAllRevisions=false/numRangeKeys=0-24   17.6ms ± 1%   17.8ms ± 2%    ~     (p=0.222 n=5+5)
MVCCExportToSST/numKeys=8192/numRevisions=10/exportAllRevisions=false/numRangeKeys=1-24   23.3ms ± 0%   23.6ms ± 0%  +1.10%  (p=0.008 n=5+5)
MVCCExportToSST/numKeys=8192/numRevisions=10/exportAllRevisions=true/numRangeKeys=0-24    30.9ms ± 0%   30.7ms ± 0%  -0.51%  (p=0.008 n=5+5)
MVCCExportToSST/numKeys=8192/numRevisions=10/exportAllRevisions=true/numRangeKeys=1-24    38.5ms ± 0%   38.4ms ± 0%    ~     (p=0.310 n=5+5)
MVCCExportToSST/numKeys=65536/numRevisions=1/exportAllRevisions=false/numRangeKeys=0-24   36.6ms ± 0%   36.1ms ± 1%  -1.46%  (p=0.008 n=5+5)
MVCCExportToSST/numKeys=65536/numRevisions=1/exportAllRevisions=false/numRangeKeys=1-24   44.5ms ± 0%   44.6ms ± 0%    ~     (p=0.310 n=5+5)
MVCCExportToSST/numKeys=65536/numRevisions=1/exportAllRevisions=true/numRangeKeys=0-24    33.4ms ± 1%   32.8ms ± 0%  -1.70%  (p=0.008 n=5+5)
MVCCExportToSST/numKeys=65536/numRevisions=1/exportAllRevisions=true/numRangeKeys=1-24    41.1ms ± 1%   40.7ms ± 0%  -1.04%  (p=0.008 n=5+5)
MVCCExportToSST/numKeys=65536/numRevisions=10/exportAllRevisions=false/numRangeKeys=0-24   152ms ± 1%    152ms ± 4%    ~     (p=1.000 n=5+5)
MVCCExportToSST/numKeys=65536/numRevisions=10/exportAllRevisions=false/numRangeKeys=1-24   197ms ± 0%    200ms ± 0%  +1.61%  (p=0.008 n=5+5)
MVCCExportToSST/numKeys=65536/numRevisions=10/exportAllRevisions=true/numRangeKeys=0-24    255ms ± 1%    253ms ± 0%  -0.56%  (p=0.032 n=5+5)
MVCCExportToSST/numKeys=65536/numRevisions=10/exportAllRevisions=true/numRangeKeys=1-24    314ms ± 0%    314ms ± 0%    ~     (p=0.730 n=4+5)
```

Release justification: bug fixes and low-risk updates to new functionality

Release note: None

Co-authored-by: Erik Grinaker <[email protected]>
  • Loading branch information
craig[bot] and erikgrinaker committed Aug 22, 2022
3 parents 31bd044 + 7086670 + d27f8d6 commit 76f5b3a
Show file tree
Hide file tree
Showing 2 changed files with 86 additions and 96 deletions.
59 changes: 29 additions & 30 deletions pkg/kv/kvserver/rangefeed/catchup_scan.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@ import (
type simpleCatchupIter interface {
storage.SimpleMVCCIterator
NextIgnoringTime()
RangeKeyChangedIgnoringTime() bool
RangeKeysIgnoringTime() storage.MVCCRangeKeyStack
}

Expand All @@ -45,6 +46,10 @@ func (i simpleCatchupIterAdapter) NextIgnoringTime() {
i.SimpleMVCCIterator.Next()
}

func (i simpleCatchupIterAdapter) RangeKeyChangedIgnoringTime() bool {
return i.SimpleMVCCIterator.RangeKeyChanged()
}

func (i simpleCatchupIterAdapter) RangeKeysIgnoringTime() storage.MVCCRangeKeyStack {
return i.SimpleMVCCIterator.RangeKeys()
}
Expand Down Expand Up @@ -137,7 +142,6 @@ func (i *CatchUpIterator) CatchUpScan(outputFn outputEventFn, withDiff bool) err
// can't use NextKey.
var lastKey roachpb.Key
var meta enginepb.MVCCMetadata
var rangeKeysStart roachpb.Key
i.SeekGE(storage.MVCCKey{Key: i.span.Key})
for {
if ok, err := i.Valid(); err != nil {
Expand All @@ -146,46 +150,40 @@ func (i *CatchUpIterator) CatchUpScan(outputFn outputEventFn, withDiff bool) err
break
}

hasPoint, hasRange := i.HasPointAndRange()

// Emit any new MVCC range tombstones when their start key is encountered.
// Range keys can currently only be MVCC range tombstones.
//
// TODO(erikgrinaker): Find a faster/better way to detect range key changes
// that doesn't involve constant comparisons. Pebble probably already knows,
// we just need a way to ask it.
// Note that byte slice comparison in Go is smart enough to immediately bail
// if lengths are different. However, it isn't smart enough to compare from
// the end, which would really help since our keys share prefixes.
if hasRange {
if rangeBounds := i.RangeBounds(); !rangeBounds.Key.Equal(rangeKeysStart) {
rangeKeysStart = append(rangeKeysStart[:0], rangeBounds.Key...)

// NB: RangeKeyChangedIgnoringTime() may trigger because a previous
// NextIgnoringTime() call moved onto an MVCC range tombstone outside of the
// time bounds. In this case, HasPointAndRange() will return false,false and
// we step forward.
if i.RangeKeyChangedIgnoringTime() {
hasPoint, hasRange := i.HasPointAndRange()
if hasRange {
// Emit events for these MVCC range tombstones, in chronological order.
versions := i.RangeKeys().Versions
for j := len(versions) - 1; j >= 0; j-- {
rangeKeys := i.RangeKeys()
for j := rangeKeys.Len() - 1; j >= 0; j-- {
var span roachpb.Span
a, span.Key = a.Copy(rangeBounds.Key, 0)
a, span.EndKey = a.Copy(rangeBounds.EndKey, 0)
a, span.Key = a.Copy(rangeKeys.Bounds.Key, 0)
a, span.EndKey = a.Copy(rangeKeys.Bounds.EndKey, 0)
err := outputFn(&roachpb.RangeFeedEvent{
DeleteRange: &roachpb.RangeFeedDeleteRange{
Span: span,
Timestamp: versions[j].Timestamp,
Timestamp: rangeKeys.Versions[j].Timestamp,
},
})
if err != nil {
return err
}
}
}
}

// If there's no point key here (i.e. we found a bare range key above), then
// step onto the next key. This may be a point key version at the same key
// as the range key's start bound, or a later point/range key.
if !hasPoint {
i.Next()
continue
// If there's no point key here (e.g. we found a bare range key above), then
// step onto the next key. This may be a point key version at the same key
// as the range key's start bound, or a later point/range key.
if !hasPoint {
i.Next()
continue
}
}

unsafeKey := i.UnsafeKey()
Expand Down Expand Up @@ -281,10 +279,11 @@ func (i *CatchUpIterator) CatchUpScan(outputFn outputEventFn, withDiff bool) err
return errors.AssertionFailedf("unexpected previous value %s for key %s",
reorderBuf[l].Val.PrevValue, key)
}
// If an MVCC range tombstone exists between this value and the next
// one, we don't emit the value after all -- it should be a tombstone.
// The RangeKeysIgnoringTime() call is cheap, no need for caching.
if !i.RangeKeysIgnoringTime().HasBetween(ts, reorderBuf[l].Val.Value.Timestamp) {
// However, don't emit a value if an MVCC range tombstone existed
// between this value and the next one. The RangeKeysIgnoringTime()
// call is cheap, no need for caching.
rangeKeys := i.RangeKeysIgnoringTime()
if rangeKeys.IsEmpty() || !rangeKeys.HasBetween(ts, reorderBuf[l].Val.Value.Timestamp) {
// TODO(sumeer): find out if it is deliberate that we are not populating
// PrevValue.Timestamp.
reorderBuf[l].Val.PrevValue.RawBytes = val
Expand Down
123 changes: 57 additions & 66 deletions pkg/storage/mvcc.go
Original file line number Diff line number Diff line change
Expand Up @@ -5762,7 +5762,6 @@ func MVCCExportToSST(
var curKey roachpb.Key // only used if exportAllRevisions
var resumeKey MVCCKey
var rangeKeys MVCCRangeKeyStack
var rangeKeysEnd roachpb.Key
var rangeKeysSize int64

// maxRangeKeysSizeIfTruncated calculates the worst-case size of the currently
Expand Down Expand Up @@ -5790,7 +5789,8 @@ func MVCCExportToSST(
// We could be truncated in the middle of a point key version series, which
// would require adding on a \0 byte via Key.Next(), so let's assume that.
maxSize := rangeKeysSize
if s := maxSize + int64(rangeKeys.Len()*(len(resumeKey)-len(rangeKeysEnd)+1)); s > maxSize {
endKeySize := len(rangeKeys.Bounds.EndKey)
if s := maxSize + int64(rangeKeys.Len()*(len(resumeKey)-endKeySize+1)); s > maxSize {
maxSize = s
}
return maxSize
Expand Down Expand Up @@ -5845,14 +5845,8 @@ func MVCCExportToSST(
// limit). If we return a resume key then we need to truncate the final
// range key stack (and thus the SST) to the resume key, so we can't flush
// them until we've moved past.
hasPoint, hasRange := iter.HasPointAndRange()

// First, flush any pending range tombstones when we reach their end key.
//
// TODO(erikgrinaker): Optimize the range key case here, to avoid these
// comparisons. Pebble should expose an API to cheaply detect range key
// changes.
if len(rangeKeysEnd) > 0 && bytes.Compare(unsafeKey.Key, rangeKeysEnd) >= 0 {
if iter.RangeKeyChanged() {
// Flush any pending range tombstones.
for _, v := range rangeKeys.Versions {
mvccValue, ok, err := tryDecodeSimpleMVCCValue(v.Value)
if !ok && err == nil {
Expand All @@ -5863,66 +5857,64 @@ func MVCCExportToSST(
"decoding mvcc value %s", v.Value)
}
// Export only the inner roachpb.Value, not the MVCCValue header.
mvccValue = MVCCValue{Value: mvccValue.Value}
if err := sstWriter.PutMVCCRangeKey(rangeKeys.AsRangeKey(v), mvccValue); err != nil {
rawValue := mvccValue.Value.RawBytes
if err := sstWriter.PutRawMVCCRangeKey(rangeKeys.AsRangeKey(v), rawValue); err != nil {
return roachpb.BulkOpSummary{}, MVCCKey{}, err
}
}
rows.BulkOpSummary.DataSize += rangeKeysSize
rangeKeys, rangeKeysEnd, rangeKeysSize = MVCCRangeKeyStack{}, nil, 0
}
rangeKeys.Clear()
rangeKeysSize = 0

// If we find any new range keys and we haven't buffered any range keys yet,
// buffer them.
if hasRange && !skipTombstones && rangeKeys.IsEmpty() {
rangeKeys = iter.RangeKeys()
rangeKeysEnd = append(rangeKeysEnd[:0], rangeKeys.Bounds.EndKey...)
if !opts.ExportAllRevisions {
rangeKeys.Versions = rangeKeys.Versions[:1]
}
// Buffer any new range keys.
hasPoint, hasRange := iter.HasPointAndRange()
if hasRange && !skipTombstones {
if opts.ExportAllRevisions {
iter.RangeKeys().CloneInto(&rangeKeys)
} else {
rks := iter.RangeKeys()
rks.Versions = rks.Versions[:1]
rks.CloneInto(&rangeKeys)
}

// TODO(erikgrinaker): We should consider a CloneInto() method on the
// MVCCRangeKeyStack that allows reusing a byte buffer. See also TODO in
// Clone() about using a single allocation for the entire clone (all byte
// slices).
rangeKeys = rangeKeys.Clone()
for _, v := range rangeKeys.Versions {
rangeKeysSize += int64(
len(rangeKeys.Bounds.Key) + len(rangeKeys.Bounds.EndKey) + len(v.Value))
}

for _, v := range rangeKeys.Versions {
rangeKeysSize += int64(
len(rangeKeys.Bounds.Key) + len(rangeKeys.Bounds.EndKey) + len(v.Value))
// Check if the range keys exceed a limit, using similar logic as point
// keys. We have to check both the size of the range keys as they are (in
// case we emit them as-is), and the size of the range keys if they were
// to be truncated at the start key due to a resume span (which could
// happen if the next point key exceeds the max size).
//
// TODO(erikgrinaker): The limit logic here is a bit of a mess, but we're
// complying with the existing point key logic for now. We should get rid
// of some of the options and clean this up.
curSize := rows.BulkOpSummary.DataSize
reachedTargetSize := opts.TargetSize > 0 && uint64(curSize) >= opts.TargetSize
newSize := curSize + maxRangeKeysSizeIfTruncated(rangeKeys.Bounds.Key)
reachedMaxSize := opts.MaxSize > 0 && newSize > int64(opts.MaxSize)
if paginated && (reachedTargetSize || reachedMaxSize) {
rangeKeys.Clear()
rangeKeysSize = 0
resumeKey = unsafeKey.Clone()
break
}
if reachedMaxSize {
return roachpb.BulkOpSummary{}, MVCCKey{}, &ExceedMaxSizeError{
reached: newSize, maxSize: opts.MaxSize}
}
}

// Check if the range keys exceed a limit, using similar logic as point
// keys. We have to check both the size of the range keys as they are (in
// case we emit them as-is), and the size of the range keys if they were
// to be truncated at the start key due to a resume span (which could
// happen if the next point key exceeds the max size).
//
// TODO(erikgrinaker): The limit logic here is a bit of a mess, but we're
// complying with the existing point key logic for now. We should get rid
// of some of the options and clean this up.
curSize := rows.BulkOpSummary.DataSize
reachedTargetSize := opts.TargetSize > 0 && uint64(curSize) >= opts.TargetSize
newSize := curSize + maxRangeKeysSizeIfTruncated(rangeKeys.Bounds.Key)
reachedMaxSize := opts.MaxSize > 0 && newSize > int64(opts.MaxSize)
if paginated && (reachedTargetSize || reachedMaxSize) {
rangeKeys, rangeKeysEnd, rangeKeysSize = MVCCRangeKeyStack{}, nil, 0
resumeKey = unsafeKey.Clone()
break
}
if reachedMaxSize {
return roachpb.BulkOpSummary{}, MVCCKey{}, &ExceedMaxSizeError{
reached: newSize, maxSize: opts.MaxSize}
// If we're on a bare range key, step forward. We can't use NextKey()
// because there may be a point key at the range key's start bound.
if !hasPoint {
iter.Next()
continue
}
}

// If we're on a bare range key, step forward. We can't use NextKey()
// because there may be a point key version at the range key's start bound.
if !hasPoint {
iter.Next()
continue
}

// Process point keys.
unsafeValue := iter.UnsafeValue()
skip := false
Expand Down Expand Up @@ -6020,15 +6012,14 @@ func MVCCExportToSST(
// which overlaps at [c-c\0).
if !rangeKeys.IsEmpty() {
// Calculate the new rangeKeysSize due to the new resume bounds.
if len(resumeKey.Key) > 0 && rangeKeysEnd.Compare(resumeKey.Key) > 0 {
oldEndLen := len(rangeKeysEnd)
rangeKeysEnd = resumeKey.Key
if len(resumeKey.Key) > 0 && rangeKeys.Bounds.EndKey.Compare(resumeKey.Key) > 0 {
oldEndLen := len(rangeKeys.Bounds.EndKey)
rangeKeys.Bounds.EndKey = resumeKey.Key
if resumeKey.Timestamp.IsSet() {
rangeKeysEnd = rangeKeysEnd.Next()
rangeKeys.Bounds.EndKey = rangeKeys.Bounds.EndKey.Next()
}
rangeKeysSize += int64(rangeKeys.Len() * (len(rangeKeysEnd) - oldEndLen))
rangeKeysSize += int64(rangeKeys.Len() * (len(rangeKeys.Bounds.EndKey) - oldEndLen))
}
rangeKeys.Bounds.EndKey = rangeKeysEnd
for _, v := range rangeKeys.Versions {
mvccValue, ok, err := tryDecodeSimpleMVCCValue(v.Value)
if !ok && err == nil {
Expand All @@ -6039,8 +6030,8 @@ func MVCCExportToSST(
"decoding mvcc value %s", v.Value)
}
// Export only the inner roachpb.Value, not the MVCCValue header.
mvccValue = MVCCValue{Value: mvccValue.Value}
if err := sstWriter.PutMVCCRangeKey(rangeKeys.AsRangeKey(v), mvccValue); err != nil {
rawValue := mvccValue.Value.RawBytes
if err := sstWriter.PutRawMVCCRangeKey(rangeKeys.AsRangeKey(v), rawValue); err != nil {
return roachpb.BulkOpSummary{}, MVCCKey{}, err
}
}
Expand Down

0 comments on commit 76f5b3a

Please sign in to comment.