Skip to content

Commit

Permalink
Merge #76478
Browse files Browse the repository at this point in the history
76478: batcheval: add MVCC-compliant `RevertRange` variant r=aliher1911,dt a=erikgrinaker

**roachpb: add `isAlone` for `RevertRangeRequest`**

Since `RevertRange` mutates MVCC history, we want them to be alone in a
batch. The DistSender will split any batches that have multiple such
requests.

Release note: None

**storage: add `NextKeyIgnoringTime()` for `MVCCIncrementalIterator`**

This patch adds a method `NextKeyIgnoringTime()` for
`MVCCIncrementalIterator`. This can be used to find the next key (as
opposed to version) of the iterator, ignoring the time bounds. It's
similar to `NextIgnoringTime()`, but calls `NextKey()` instead of
`Next()` on the underlying iterator.

Release note: None

**batcheval: add MVCC-compliant `RevertRange` variant**

This adds a new parameter `ExperimentalPreserveHistory` which, rather
than clearing keys above the target time, will write new values or
tombstones that reflect the state at the target time. For long runs of
new keys, this will instead drop an MVCC range tombstone. This makes the
command respect e.g. MVCC immutability, the closed timestamp, and
timestamp cache.

Note that MVCC range tombstones are currently experimental, and as such
this parameter is also experimental. Callers must call
`storage.CanUseExperimentalMVCCRangeTombstones()` before using it.

Resolves #70416.

Release note: None

Co-authored-by: Erik Grinaker <[email protected]>
  • Loading branch information
craig[bot] and erikgrinaker committed Feb 22, 2022
2 parents 9c0d08d + e1acd75 commit 4c65a10
Show file tree
Hide file tree
Showing 9 changed files with 860 additions and 33 deletions.
44 changes: 29 additions & 15 deletions pkg/kv/kvserver/batcheval/cmd_revert_range.go
Original file line number Diff line number Diff line change
Expand Up @@ -51,8 +51,12 @@ func isEmptyKeyTimeRange(
// that there is *a* key in the SST that is in the time range. Thus we should
// proceed to iteration that actually checks timestamps on each key.
iter := readWriter.NewMVCCIterator(storage.MVCCKeyIterKind, storage.IterOptions{
LowerBound: from, UpperBound: to,
MinTimestampHint: since.Next() /* make exclusive */, MaxTimestampHint: until,
// TODO(erikgrinaker): Make sure TBIs respect range keys too.
KeyTypes: storage.IterKeyTypePointsAndRanges, // revert any range keys as well
LowerBound: from,
UpperBound: to,
MinTimestampHint: since.Next(), // exclusive
MaxTimestampHint: until,
})
defer iter.Close()
iter.SeekGE(storage.MVCCKey{Key: from})
Expand All @@ -78,29 +82,39 @@ func RevertRange(

args := cArgs.Args.(*roachpb.RevertRangeRequest)
reply := resp.(*roachpb.RevertRangeResponse)
pd := result.Result{
Replicated: kvserverpb.ReplicatedEvalResult{
MVCCHistoryMutation: &kvserverpb.ReplicatedEvalResult_MVCCHistoryMutation{
Spans: []roachpb.Span{{Key: args.Key, EndKey: args.EndKey}},
},
},
}

if empty, err := isEmptyKeyTimeRange(
readWriter, args.Key, args.EndKey, args.TargetTime, cArgs.Header.Timestamp,
); err != nil {
return result.Result{}, err
} else if empty {
log.VEventf(ctx, 2, "no keys to clear in specified time range")
log.VEventf(ctx, 2, "no keys to revert in specified time range")
return result.Result{}, nil
}

log.VEventf(ctx, 2, "clearing keys with timestamp (%v, %v]", args.TargetTime, cArgs.Header.Timestamp)
log.VEventf(ctx, 2, "reverting keys with timestamp (%v, %v]",
args.TargetTime, cArgs.Header.Timestamp)

resume, err := storage.MVCCClearTimeRange(ctx, readWriter, cArgs.Stats, args.Key, args.EndKey,
args.TargetTime, cArgs.Header.Timestamp, cArgs.Header.MaxSpanRequestKeys,
maxRevertRangeBatchBytes,
args.EnableTimeBoundIteratorOptimization)
var pd result.Result
var resume *roachpb.Span
var err error
if args.ExperimentalPreserveHistory {
const deleteRangeThreshold = 100
maxIntents := storage.MaxIntentsPerWriteIntentError.Get(&cArgs.EvalCtx.ClusterSettings().SV)
// TODO(erikgrinaker): Write a test for this once MVCC range tombstones are
// properly written to batches and replicated.
// TODO(erikgrinaker): Test that this records MVCC logical ops correctly.
resume, err = storage.ExperimentalMVCCRevertRange(ctx, readWriter, cArgs.Stats,
args.Key, args.EndKey, cArgs.Header.Timestamp, args.TargetTime, deleteRangeThreshold,
cArgs.Header.MaxSpanRequestKeys, maxRevertRangeBatchBytes, maxIntents)
} else {
resume, err = storage.MVCCClearTimeRange(ctx, readWriter, cArgs.Stats, args.Key, args.EndKey,
args.TargetTime, cArgs.Header.Timestamp, cArgs.Header.MaxSpanRequestKeys,
maxRevertRangeBatchBytes, args.EnableTimeBoundIteratorOptimization)
pd.Replicated.MVCCHistoryMutation = &kvserverpb.ReplicatedEvalResult_MVCCHistoryMutation{
Spans: []roachpb.Span{{Key: args.Key, EndKey: args.EndKey}},
}
}
if err != nil {
return result.Result{}, err
}
Expand Down
17 changes: 12 additions & 5 deletions pkg/roachpb/api.go
Original file line number Diff line number Diff line change
Expand Up @@ -778,8 +778,8 @@ func (crr *ClearRangeRequest) ShallowCopy() Request {
}

// ShallowCopy implements the Request interface.
func (crr *RevertRangeRequest) ShallowCopy() Request {
shallowCopy := *crr
func (rrr *RevertRangeRequest) ShallowCopy() Request {
shallowCopy := *rrr
return &shallowCopy
}

Expand Down Expand Up @@ -1270,9 +1270,16 @@ func (*ClearRangeRequest) flags() flag {
return isWrite | isRange | isAlone | bypassesReplicaCircuitBreaker
}

// Note that RevertRange commands cannot be part of a transaction as
// they clear all MVCC versions above their target time.
func (*RevertRangeRequest) flags() flag { return isWrite | isRange | bypassesReplicaCircuitBreaker }
// Note that RevertRange commands cannot be part of a transaction, as they
// either clear MVCC versions or write MVCC range tombstones, neither of which
// is supported within transactions.
func (rrr *RevertRangeRequest) flags() flag {
if rrr.ExperimentalPreserveHistory {
return isRead | isWrite | isRange | isAlone | updatesTSCache | appliesTSCache |
bypassesReplicaCircuitBreaker
}
return isWrite | isRange | isAlone | bypassesReplicaCircuitBreaker
}

func (sr *ScanRequest) flags() flag {
maybeLocking := flagForLockStrength(sr.KeyLocking)
Expand Down
26 changes: 21 additions & 5 deletions pkg/roachpb/api.proto
Original file line number Diff line number Diff line change
Expand Up @@ -402,18 +402,34 @@ message ClearRangeResponse {
}


// A RevertRangeRequest specifies a range of keys in which to clear all MVCC
// revisions more recent than some TargetTime from the underlying engine, thus
// reverting the range (from the perspective of an MVCC scan) to that time.
// A RevertRangeRequest specifies a range of keys to revert to some past time.
// By default, it will clear all revision more recent that TargetTime from the
// underlying engine. However, this violates several guarantees including MVCC
// immutability, the closed timestamp, timestamp cache, and others. See the
// ExperimentalPreserveHistory parameter which will uphold these guarantees.
message RevertRangeRequest {
RequestHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true];

// TargetTime specifies a the time to which to "revert" the range by clearing
// any MVCC key with a strictly higher timestamp. TargetTime must be higher
// TargetTime specifies a the time to which to "revert" the range to. Any
// versions later than TargetTime will be undone. TargetTime must be higher
// than the GC Threshold for the replica - so that it is assured that the keys
// for that time are still there — or the request will fail.
util.hlc.Timestamp target_time = 2 [(gogoproto.nullable) = false];

// ExperimentalPreserveHistory will preserve MVCC history by, rather than
// clearing newer versions, deleting them using tombstones or updating them
// back to their original value as of the target time. Long runs of key
// deletions will use an MVCC range tombstone instead. This respects the
// closed timestamp and timestamp cache.
//
// The caller must check storage.CanUseExperimentalMVCCRangeTombstones()
// before enabling this parameter.
//
// This parameter is EXPERIMENTAL: range tombstones are under active
// development, and have severe limitations including being ignored by all
// KV and MVCC APIs and only being stored in memory.
bool experimental_preserve_history = 5;

bool enable_time_bound_iterator_optimization = 3;

// IgnoreGcThreshold can be set by a caller to ignore the target-time when
Expand Down
1 change: 1 addition & 0 deletions pkg/roachpb/api_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -321,6 +321,7 @@ func TestFlagCombinations(t *testing.T) {
&DeleteRangeRequest{UseExperimentalRangeTombstone: true},
&GetRequest{KeyLocking: lock.Exclusive},
&ReverseScanRequest{KeyLocking: lock.Exclusive},
&RevertRangeRequest{ExperimentalPreserveHistory: true},
&ScanRequest{KeyLocking: lock.Exclusive},
}

Expand Down
Loading

0 comments on commit 4c65a10

Please sign in to comment.