Skip to content

Commit

Permalink
storage: optimize MVCCExportToSST range key processing
Browse files Browse the repository at this point in the history
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
  • Loading branch information
erikgrinaker committed Aug 22, 2022
1 parent a36edf7 commit d27f8d6
Showing 1 changed file with 57 additions and 66 deletions.
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 d27f8d6

Please sign in to comment.