Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
92825: storage: never elide resume key in MVCCExportToSST r=sumeerbhola,irfansharif a=stevendanna

MVCCExportToSST checks if no data has been written to the SST and avoids calling Finish() on the writer in that case. However, it was previously also returning an empty resume key in that case, regardless of the resume key's previous value.

Returning an empty resume key was OK since it was assumed that if the SST had no data, then there couldn't possibly be anything to resume.

However, when this method was extended to support cooperative resource management, it now has two cases in which the main iteration loop of the export may return a resume key despite not having written any data.

First, the Elastic CPU work handler is called unconditionally. If the CPU work handler was already over the limit on the first call, then we return immediately and with no data and a resume key set.

Second, the time-based resource limiter is called after the first iteration of the loop. While typically allowing one iteration will result in data being written to the SST before the exhaustion condition can be hit; if there is a long string of deletion tombstones that are not being added to our response, then it is possible that we've completed iterations of the loop without any data being written.

As a result of dropping this resume key, users of ExportRequest such as backup would then assume that their work was done. As a result, a backup may not contain all of the expected data.

Elastic CPU limiting and the time-based resource limiter are disabled by default in previous releases.

Elastic CPU limiting is enabled by default on master.

Epic: None

Release note (bug fix): Fix a bug that would result in incomplete backups when non-default, non-public resource limiting settings (kv.bulk_sst.max_request_time or
admission.elastic_cpu.enabled) were enabled.

Co-authored-by: Steven Danna <[email protected]>
  • Loading branch information
craig[bot] and stevendanna committed Dec 12, 2022
2 parents 9257286 + 3e927d2 commit 30fce41
Show file tree
Hide file tree
Showing 6 changed files with 342 additions and 37 deletions.
18 changes: 14 additions & 4 deletions pkg/kv/kvserver/batcheval/cmd_export.go
Original file line number Diff line number Diff line change
Expand Up @@ -218,11 +218,21 @@ func evalExport(
}
data := destFile.Data()

// NB: This should only happen on the first page of results. If there were
// more data to be read that lead to pagination then we'd see it in this
// page. Break out of the loop because there must be no data to export.
// NB: This should only happen in two cases:
//
// 1. There was nothing to export for this span.
//
// 2. We hit a resource constraint that led to an
// early exit and thus have a resume key despite
// not having data.
if summary.DataSize == 0 {
break
if resume.Key != nil {
start = resume.Key
resumeKeyTS = resume.Timestamp
continue
} else {
break
}
}

span := roachpb.Span{Key: start}
Expand Down
88 changes: 60 additions & 28 deletions pkg/storage/mvcc.go
Original file line number Diff line number Diff line change
Expand Up @@ -5904,9 +5904,14 @@ func MVCCExportToSST(
}

if summary.DataSize == 0 {
// If no records were added to the sstable, skip completing it and return a
// nil slice – the export code will discard it anyway (based on 0 DataSize).
return roachpb.BulkOpSummary{}, MVCCKey{}, nil
// If no records were added to the sstable, skip
// completing it and return an empty summary.
//
// We still propogate the resumeKey because our
// iteration may have been halted because of resource
// limitiations before any keys were added to the
// returned SST.
return roachpb.BulkOpSummary{}, resumeKey, nil
}

return summary, resumeKey, sstWriter.Finish()
Expand Down Expand Up @@ -5977,6 +5982,7 @@ func mvccExportToWriter(
rangeKeyMasking = opts.EndTS
}

elasticCPUHandle := admission.ElasticCPUWorkHandleFromContext(ctx)
iter := NewMVCCIncrementalIterator(reader, MVCCIncrementalIterOptions{
KeyTypes: IterKeyTypePointsAndRanges,
StartKey: opts.StartKey.Key,
Expand All @@ -5989,12 +5995,33 @@ func mvccExportToWriter(
defer iter.Close()

paginated := opts.TargetSize > 0
trackKeyBoundary := paginated || opts.ResourceLimiter != nil
hasElasticCPULimiter := elasticCPUHandle != nil
hasTimeBasedResourceLimiter := opts.ResourceLimiter != nil

// trackKeyBoundary is true if we need to know whether the
// iteration has proceeded to a new key.
//
// If opts.ExportAllRevisions is false, then our iteration loop
// will use NextKey() and thus will always be on a new key.
//
// If opts.ExportAllRevisions is true, we only need to track
// key boundaries if we may return from our iteration before
// the EndKey. This can happen if the user has requested
// paginated results, or if we hit a resource limit.
trackKeyBoundary := opts.ExportAllRevisions && (paginated || hasTimeBasedResourceLimiter || hasElasticCPULimiter)

firstIteration := true
// skipTombstones controls whether we include tombstones.
//
// We want tombstones if we are exporting all reivions or if
// we have a StartTS. A non-empty StartTS is used by
// incremental backups and thus needs to see tombstones if
// that happens to be the latest value.
skipTombstones := !opts.ExportAllRevisions && opts.StartTS.IsEmpty()

var rows RowCounter
var curKey roachpb.Key // only used if exportAllRevisions
// Only used if trackKeyBoundary is true.
var curKey roachpb.Key
var resumeKey MVCCKey
var rangeKeys MVCCRangeKeyStack
var rangeKeysSize int64
Expand Down Expand Up @@ -6031,7 +6058,6 @@ func mvccExportToWriter(
return maxSize
}

elasticCPUHandle := admission.ElasticCPUWorkHandleFromContext(ctx)
iter.SeekGE(opts.StartKey)
for {
if ok, err := iter.Valid(); err != nil {
Expand All @@ -6044,20 +6070,24 @@ func mvccExportToWriter(

unsafeKey := iter.UnsafeKey()

isNewKey := !opts.ExportAllRevisions || !unsafeKey.Key.Equal(curKey)
if trackKeyBoundary && opts.ExportAllRevisions && isNewKey {
// isNewKey is true when we aren't tracking key
// boundaries because either we are not exporting all
// revisions or because we know we won't stop before a
// key boundary anyway.
isNewKey := !trackKeyBoundary || !unsafeKey.Key.Equal(curKey)
if trackKeyBoundary && isNewKey {
curKey = append(curKey[:0], unsafeKey.Key...)
}

// TODO(irfansharif): Remove this time-based resource limiter once
// enabling elastic CPU limiting by default. There needs to be a
// compelling reason to need two mechanisms.
if opts.ResourceLimiter != nil {
if firstIteration {
// Don't check resources on first iteration to ensure we can make some progress regardless
// of starvation. Otherwise operations could spin indefinitely.
if firstIteration {
firstIteration = false
} else {
firstIteration = false
} else {
// TODO(irfansharif): Remove this time-based resource limiter once
// enabling elastic CPU limiting by default. There needs to be a
// compelling reason to need two mechanisms.
if opts.ResourceLimiter != nil {
// In happy day case we want to only stop at key boundaries as it allows callers to use
// produced sst's directly. But if we can't find key boundary within reasonable number of
// iterations we would split mid key.
Expand All @@ -6077,16 +6107,19 @@ func mvccExportToWriter(
break
}
}
}

// Check if we're over our allotted CPU time + on a key boundary (we
// prefer callers being able to use SSTs directly). Going over limit is
// accounted for in admission control by penalizing the subsequent
// request, so doing it slightly is fine.
if overLimit, _ := elasticCPUHandle.OverLimit(); overLimit && isNewKey {
resumeKey = unsafeKey.Clone()
resumeKey.Timestamp = hlc.Timestamp{}
break
// Check if we're over our allotted CPU time + on a key boundary (we
// prefer callers being able to use SSTs directly). Going over limit is
// accounted for in admission control by penalizing the subsequent
// request, so doing it slightly is fine.
stopAllowed := isNewKey || opts.StopMidKey
if overLimit, _ := elasticCPUHandle.OverLimit(); overLimit && stopAllowed {
resumeKey = unsafeKey.Clone()
if isNewKey {
resumeKey.Timestamp = hlc.Timestamp{}
}
break
}
}

// When we encounter an MVCC range tombstone stack, we buffer it in
Expand Down Expand Up @@ -6144,7 +6177,7 @@ func mvccExportToWriter(
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) {
if curSize > 0 && paginated && (reachedTargetSize || reachedMaxSize) {
rangeKeys.Clear()
rangeKeysSize = 0
resumeKey = unsafeKey.Clone()
Expand Down Expand Up @@ -6306,9 +6339,8 @@ type MVCCExportOptions struct {
ExportAllRevisions bool
// If TargetSize is positive, it indicates that the export should produce SSTs
// which are roughly target size. Specifically, it will return an SST such that
// the last key is responsible for meeting or exceeding the targetSize. If the
// resumeKey is non-nil then the data size of the returned sst will be greater
// than or equal to the targetSize.
// the last key is responsible for meeting or exceeding the targetSize, unless the
// iteration has been stopped because of resource limitations.
TargetSize uint64
// If MaxSize is positive, it is an absolute maximum on byte size for the
// returned sst. If it is the case that the versions of the last key will lead
Expand Down
Loading

0 comments on commit 30fce41

Please sign in to comment.