Skip to content

Commit

Permalink
kvserver: add Time Bound Iteration to DeleteRange
Browse files Browse the repository at this point in the history
Previously, a kv client could only pass an AOST timestamp to a DeleteRange
request. Now, the user can pass a lower bound timestamp, causing
the kvserver to leverage time bound iteration while issuing delete requests.

Specifically, the server uses an MVCCIncrementalIterator to iterate over the
target span at the client provided time bounds, track a continuous run of keys
in that time bound, and flush the run via point and MVCC range tombstones
depending on the size of the run.

In a future pr, this operation will replace the use of RevertRange during IMPORT
INTO rollbacks to make them MVCC compatible.

Informs cockroachdb#70428

Release note: none
  • Loading branch information
msbutler committed Jul 25, 2022
1 parent 1c90519 commit aead1ca
Show file tree
Hide file tree
Showing 6 changed files with 999 additions and 110 deletions.
58 changes: 54 additions & 4 deletions pkg/kv/kvserver/batcheval/cmd_delete_range.go
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,8 @@ func declareKeysDeleteRange(
}
}

const maxDeleteRangeBatchBytes = 32 << 20

// DeleteRange deletes the range of key/value pairs specified by
// start and end keys.
func DeleteRange(
Expand All @@ -67,7 +69,14 @@ func DeleteRange(
h := cArgs.Header
reply := resp.(*roachpb.DeleteRangeResponse)

// Use experimental MVCC range tombstone if requested.
if args.Predicates != (roachpb.DeleteRangePredicates{}) && !args.UseRangeTombstone {
// This ensures predicate based DeleteRange piggybacks on the version gate,
// roachpb api flags, and latch declarations used by the UseRangeTombstone.
return result.Result{}, errors.AssertionFailedf(
"UseRangeTombstones must be passed with predicate based Delete Range")
}

// Use MVCC range tombstone if requested.
if args.UseRangeTombstone {
if cArgs.Header.Txn != nil {
return result.Result{}, ErrTransactionUnsupported
Expand All @@ -79,14 +88,55 @@ func DeleteRange(
return result.Result{}, errors.AssertionFailedf(
"ReturnKeys can't be used with range tombstones")
}

desc := cArgs.EvalCtx.Desc()
leftPeekBound, rightPeekBound := rangeTombstonePeekBounds(
args.Key, args.EndKey, desc.StartKey.AsRawKey(), desc.EndKey.AsRawKey())
maxIntents := storage.MaxIntentsPerWriteIntentError.Get(&cArgs.EvalCtx.ClusterSettings().SV)

err := storage.MVCCDeleteRangeUsingTombstone(ctx, readWriter, cArgs.Stats,
args.Key, args.EndKey, h.Timestamp, cArgs.Now, leftPeekBound, rightPeekBound, maxIntents)
if args.Predicates == (roachpb.DeleteRangePredicates{}) {
// If no predicate parameters are passed, use the fast path.
err := storage.MVCCDeleteRangeUsingTombstone(ctx, readWriter, cArgs.Stats,
args.Key, args.EndKey, h.Timestamp, cArgs.Now, leftPeekBound, rightPeekBound, maxIntents)
return result.Result{}, err
}

if h.MaxSpanRequestKeys == 0 {
// In production, MaxSpanRequestKeys must be greater than zero to ensure
// the DistSender serializes predicate based DeleteRange requests across
// ranges. This ensures that only one resumeSpan gets returned to the
// client.
//
// Also, note that DeleteRangeUsingTombstone requests pass the isAlone
// flag in roachpb.api.proto, ensuring the requests do not intermingle with
// other types of requests, preventing further resume span muddling.
return result.Result{}, errors.AssertionFailedf(
"MaxSpanRequestKeys must be greater than zero when using predicated based DeleteRange")
}
// TODO (msbutler): Tune the threshold once DeleteRange and DeleteRangeUsingTombstone have
// been further optimized.
defaultRangeTombstoneThreshold := int64(64)
resumeSpan, err := storage.MVCCPredicateDeleteRange(ctx, readWriter, cArgs.Stats,
args.Key, args.EndKey, h.Timestamp, cArgs.Now, leftPeekBound, rightPeekBound,
args.Predicates, h.MaxSpanRequestKeys, maxDeleteRangeBatchBytes,
defaultRangeTombstoneThreshold, maxIntents)

if resumeSpan != nil {
reply.ResumeSpan = resumeSpan
reply.ResumeReason = roachpb.RESUME_KEY_LIMIT

// Note: While MVCCPredicateDeleteRange _could_ return reply.NumKeys, as
// the number of keys iterated through, doing so could lead to a
// significant performance drawback. The DistSender would have used
// NumKeys to subtract the number of keys processed by one range from the
// MaxSpanRequestKeys limit given to the next range. In this case, that's
// specifically not what we want, because this request does not use the
// normal meaning of MaxSpanRequestKeys (i.e. number of keys to return).
// Here, MaxSpanRequest keys is used to limit the number of tombstones
// written. Thus, setting NumKeys would just reduce the limit available to
// the next range for no good reason.
}
// Return result is always empty, since the reply is populated into the
// passed in resp pointer.
return result.Result{}, err
}

Expand Down
Loading

0 comments on commit aead1ca

Please sign in to comment.