diff --git a/pkg/kv/kvserver/batcheval/cmd_revert_range.go b/pkg/kv/kvserver/batcheval/cmd_revert_range.go index 1afcfbd39f87..7e7fe6e05a58 100644 --- a/pkg/kv/kvserver/batcheval/cmd_revert_range.go +++ b/pkg/kv/kvserver/batcheval/cmd_revert_range.go @@ -51,12 +51,8 @@ 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{ - // 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, + LowerBound: from, UpperBound: to, + MinTimestampHint: since.Next() /* make exclusive */, MaxTimestampHint: until, }) defer iter.Close() iter.SeekGE(storage.MVCCKey{Key: from}) @@ -82,39 +78,29 @@ 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 revert in specified time range") + log.VEventf(ctx, 2, "no keys to clear in specified time range") return result.Result{}, nil } - log.VEventf(ctx, 2, "reverting keys with timestamp (%v, %v]", - args.TargetTime, cArgs.Header.Timestamp) + log.VEventf(ctx, 2, "clearing keys with timestamp (%v, %v]", args.TargetTime, cArgs.Header.Timestamp) - 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}}, - } - } + resume, err := storage.MVCCClearTimeRange(ctx, readWriter, cArgs.Stats, args.Key, args.EndKey, + args.TargetTime, cArgs.Header.Timestamp, cArgs.Header.MaxSpanRequestKeys, + maxRevertRangeBatchBytes, + args.EnableTimeBoundIteratorOptimization) if err != nil { return result.Result{}, err } diff --git a/pkg/roachpb/api.go b/pkg/roachpb/api.go index 2514b5ef09e3..7d484b1d65b3 100644 --- a/pkg/roachpb/api.go +++ b/pkg/roachpb/api.go @@ -778,8 +778,8 @@ func (crr *ClearRangeRequest) ShallowCopy() Request { } // ShallowCopy implements the Request interface. -func (rrr *RevertRangeRequest) ShallowCopy() Request { - shallowCopy := *rrr +func (crr *RevertRangeRequest) ShallowCopy() Request { + shallowCopy := *crr return &shallowCopy } @@ -1270,14 +1270,9 @@ func (*ClearRangeRequest) flags() flag { return isWrite | isRange | isAlone | 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 - } +// 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 | isAlone | bypassesReplicaCircuitBreaker } diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index a8ca0c9de814..98d31f644d6b 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -402,34 +402,18 @@ message ClearRangeResponse { } -// 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. +// 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. message RevertRangeRequest { RequestHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; - // TargetTime specifies a the time to which to "revert" the range to. Any - // versions later than TargetTime will be undone. TargetTime must be higher + // 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 // 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 diff --git a/pkg/roachpb/api_test.go b/pkg/roachpb/api_test.go index 9abfa2cace00..dea42892987a 100644 --- a/pkg/roachpb/api_test.go +++ b/pkg/roachpb/api_test.go @@ -321,7 +321,6 @@ func TestFlagCombinations(t *testing.T) { &DeleteRangeRequest{UseExperimentalRangeTombstone: true}, &GetRequest{KeyLocking: lock.Exclusive}, &ReverseScanRequest{KeyLocking: lock.Exclusive}, - &RevertRangeRequest{ExperimentalPreserveHistory: true}, &ScanRequest{KeyLocking: lock.Exclusive}, } diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index 3bd3548e1e3e..4fddb07b7515 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -1967,8 +1967,6 @@ func MVCCMerge( // // If the underlying iterator encounters an intent with a timestamp in the span // (startTime, endTime], or any inline meta, this method will return an error. -// -// TODO(erikgrinaker): This should clear any range keys as well. func MVCCClearTimeRange( _ context.Context, rw ReadWriter, @@ -2233,6 +2231,9 @@ func MVCCDeleteRange( // This method is EXPERIMENTAL: range keys are under active development, and // have severe limitations including being ignored by all KV and MVCC APIs and // only being stored in memory. +// +// TODO(erikgrinaker): Needs handling of conflicts (e.g. WriteTooOldError), +// MVCCStats, and tests. func ExperimentalMVCCDeleteRangeUsingTombstone( ctx context.Context, rw ReadWriter, @@ -2246,16 +2247,8 @@ func ExperimentalMVCCDeleteRangeUsingTombstone( } else if len(intents) > 0 { return &roachpb.WriteIntentError{Intents: intents} } - return experimentalMVCCDeleteRangeUsingTombstoneInternal(ctx, rw, ms, MVCCRangeKey{ - StartKey: startKey, EndKey: endKey, Timestamp: timestamp}) -} - -// TODO(erikgrinaker): Needs handling of conflicts (e.g. WriteTooOldError), -// MVCCStats, and tests. -func experimentalMVCCDeleteRangeUsingTombstoneInternal( - ctx context.Context, rw ReadWriter, ms *enginepb.MVCCStats, rangeKey MVCCRangeKey, -) error { - return rw.ExperimentalPutMVCCRangeKey(rangeKey, nil) + return rw.ExperimentalPutMVCCRangeKey(MVCCRangeKey{ + StartKey: startKey, EndKey: endKey, Timestamp: timestamp}, nil) } func recordIteratorStats(traceSpan *tracing.Span, iteratorStats IteratorStats) { @@ -2274,222 +2267,6 @@ func recordIteratorStats(traceSpan *tracing.Span, iteratorStats IteratorStats) { } } -// ExperimentalMVCCRevertRange will revert a range back to its state as of some -// past timestamp, writing tombstones or key updates as appropriate at the given -// write timestamp. Long runs of key deletions will be written using MVCC range -// tombstones. -// -// This function cannot be used in a transaction. However, it will scan for -// existing intents and return a WriteIntentError, and scan for newer writes -// and return WriteTooOldError. -// -// This function is EXPERIMENTAL. Range tombstones are not supported throughout -// the MVCC API, and the on-disk format is unstable. -// -// TODO(erikgrinaker): Handle range keys. -func ExperimentalMVCCRevertRange( - ctx context.Context, - rw ReadWriter, - ms *enginepb.MVCCStats, - startKey, endKey roachpb.Key, - writeTimestamp hlc.Timestamp, - revertTimestamp hlc.Timestamp, - deleteRangeThreshold int, - maxBatchSize int64, - maxBatchBytes int64, - maxIntents int64, -) (*roachpb.Span, error) { - // We must resolve any intents within the span, so we may as well scan for - // separated intents before doing any work. - if intents, err := ScanIntents(ctx, rw, startKey, endKey, maxIntents, 0); err != nil { - return nil, err - } else if len(intents) > 0 { - return nil, &roachpb.WriteIntentError{Intents: intents} - } - - // We accumulate point deletes in deleteBuf until we either reach - // deleteRangeThreshold and switch to using a range deletion tombstone - // anchored at deleteRangeStart, or until we hit a visible key at which - // point we flush the deleteBuf as point deletes. - var deleteRangeStart roachpb.Key - var deleteBuf []roachpb.Key - var deleteBufIdx int - var deleteBufBytes int64 - if deleteRangeThreshold > 1 { - deleteBuf = make([]roachpb.Key, deleteRangeThreshold-1) - } - - putBuf := newPutBuffer() - defer putBuf.release() - - var batchSize, batchBytes int64 - - flushDeletes := func(nonMatch roachpb.Key) error { - if len(deleteRangeStart) > 0 { - err := experimentalMVCCDeleteRangeUsingTombstoneInternal(ctx, rw, ms, MVCCRangeKey{ - StartKey: deleteRangeStart, EndKey: nonMatch, Timestamp: writeTimestamp}) - deleteRangeStart = nil - batchBytes += int64(encodedMVCCKeyLength(MVCCKey{Key: nonMatch})) // account for end key - return err - } - - if deleteBufIdx > 0 { - iter := newMVCCIterator(rw, false, IterOptions{Prefix: true}) - defer iter.Close() - for i := 0; i < deleteBufIdx; i++ { - err := mvccPutInternal( - ctx, rw, iter, ms, deleteBuf[i], writeTimestamp, nil, nil, putBuf, nil) - if err != nil { - return err - } - } - deleteBufIdx = 0 - deleteBufBytes = 0 - } - return nil - } - - revert := func(k roachpb.Key, v []byte) (*roachpb.Key, error) { - // For non-deletions, we have to flush any pending deletes first. This may also - // flush a range tombstone, which will add to batchBytes. - if len(v) > 0 { - if err := flushDeletes(k); err != nil { - return nil, err - } - } - - // If the batch is full, return a resume key after flushing any deletes. - if batchSize >= maxBatchSize || batchBytes >= maxBatchBytes { - err := flushDeletes(k) - return &k, err - } - bytes := int64(encodedMVCCKeyLength(MVCCKey{Key: k, Timestamp: writeTimestamp}) + len(v)) - - if len(v) > 0 || len(deleteBuf) == 0 { - batchSize++ - batchBytes += bytes - iter := newMVCCIterator(rw, false, IterOptions{Prefix: true}) - defer iter.Close() - return nil, mvccPutInternal(ctx, rw, iter, ms, k, writeTimestamp, v, nil, putBuf, nil) - - } else if len(deleteRangeStart) == 0 { - // We're currently buffering point deletions. - if deleteBufIdx < len(deleteBuf) { - deleteBuf[deleteBufIdx] = append(deleteBuf[deleteBufIdx][:0], k...) - deleteBufIdx++ - deleteBufBytes += bytes - batchSize++ - batchBytes += bytes - } else { - // Buffer is full -- switch to tracking the start of the range delete. We - // remove the buffered keys from the batch size, and instead only track - // the range key. - batchSize -= int64(deleteBufIdx) - 1 // -1 accounts for the range key - batchBytes -= deleteBufBytes - - int64(encodedMVCCKeyLength(MVCCKey{Key: deleteBuf[0], Timestamp: writeTimestamp})) - deleteRangeStart = deleteBuf[0] - deleteBufIdx = 0 - deleteBufBytes = 0 - } - } - return nil, nil - } - - // We set up an incremental iterator from the revert time to look for any - // changes that need to be reverted. However, we also need to inspect older - // values to e.g. find the value to revert to or make sure we don't drop range - // tombstones across them -- we do this by using the IgnoringTime() methods on - // the MVCCIncrementalIterator. - iter := NewMVCCIncrementalIterator(rw, MVCCIncrementalIterOptions{ - EnableTimeBoundIteratorOptimization: true, - EndKey: endKey, - StartTime: revertTimestamp, - EndTime: writeTimestamp, // puts will error on any newer versions - }) - defer iter.Close() - - // TODO(erikgrinaker): Consider rewriting the below to iterate over keys and - // versions separately, which might make the logic clearer. Also consider - // using a struct to manage the deletion state. - var revertKey roachpb.Key - var revertValue, revertValueFrom []byte - iter.SeekGE(MVCCKey{Key: startKey}) - for { - if ok, err := iter.Valid(); err != nil { - return nil, err - } else if !ok { - break - } - - key := iter.UnsafeKey() - - if key.Timestamp.IsEmpty() { - return nil, errors.Errorf("encountered inline key %s", key) - } - - // If a key was scheduled for reversion, revert it when the key changes, - // but only if the original value differs from the latest value. - if len(revertKey) > 0 && !revertKey.Equal(key.Key) { - if !bytes.Equal(revertValue, revertValueFrom) { - if resumeKey, err := revert(revertKey, revertValue); err != nil || resumeKey != nil { - return &roachpb.Span{Key: *resumeKey, EndKey: endKey}, err - } - } - revertKey, revertValue, revertValueFrom = nil, nil, nil // TODO(erikgrinaker): reuse slices - } - - if revertTimestamp.Less(key.Timestamp) { - // Schedule this key for reversion. - if len(revertKey) == 0 { - // TODO(erikgrinaker): reuse byte slices - revertKey = key.Key.Clone() - revertValueFrom = append([]byte(nil), iter.Value()...) - } - - // Move the iterator to the next key, even if <= resumeTimestamp. If it - // finds an old version of this key, it will set the value to revert to. - iter.NextIgnoringTime() - - } else if bytes.Equal(revertKey, key.Key) { - // This is the version of revertKey that we should revert back to. If it - // is visible we can move to the next TBI key, because it's going to flush - // any pending deletes. However, if it's a tombstone we have to move to - // the next key even if it's outside of the TBI bounds to see if it's - // visible and avoid dropping a range tombstone across it. - revertValue = append([]byte(nil), iter.Value()...) - if len(revertValue) > 0 { - iter.Next() - } else { - iter.NextKeyIgnoringTime() - } - - } else if len(iter.Value()) > 0 { - // This is a different visible key at or below the revert timestamp. We - // have to flush any deletes up to here to avoid dropping a range - // tombstone across it. - if err := flushDeletes(key.Key); err != nil { - return nil, err - } - iter.Next() - - } else { - // This is a tombstone for a different key. We have to move to the next - // key (ignoring TBI) to check whether it could be a visible key outside - // of the time bounds -- if it is, we don't want to drop a range tombstone - // across it. - iter.NextKeyIgnoringTime() - } - } - - // Handle a revert at the very end of the iteration. - if len(revertKey) > 0 && !bytes.Equal(revertValue, revertValueFrom) { - if resumeKey, err := revert(revertKey, revertValue); err != nil || resumeKey != nil { - return &roachpb.Span{Key: *resumeKey, EndKey: endKey}, err - } - } - return nil, flushDeletes(endKey) -} - func mvccScanToBytes( ctx context.Context, iter MVCCIterator, diff --git a/pkg/storage/mvcc_history_test.go b/pkg/storage/mvcc_history_test.go index 9451f0beb91d..9f6b865591c1 100644 --- a/pkg/storage/mvcc_history_test.go +++ b/pkg/storage/mvcc_history_test.go @@ -69,7 +69,6 @@ import ( // // clear_range k= end= // clear_range_key k= end= [ts=[,]] -// revert_range [ts=[,]] k= end= revertTS=[,int] [deleteRangeThreshold=] [maxBatchSize=] [maxBatchBytes=] // // Where `` can be a simple string, or a string // prefixed by the following characters: @@ -428,7 +427,6 @@ var commands = map[string]cmd{ "iter_range_keys": {typReadOnly, cmdIterRangeKeys}, "merge": {typDataUpdate, cmdMerge}, "put": {typDataUpdate, cmdPut}, - "revert_range": {typDataUpdate, cmdRevertRange}, "scan": {typReadOnly, cmdScan}, } @@ -606,27 +604,7 @@ func cmdCheckIntent(e *evalCtx) error { func cmdClearRange(e *evalCtx) error { key, endKey := e.getKeyRange() - if err := e.engine.ClearMVCCRangeAndIntents(key, endKey); err != nil { - return err - } - // TODO(erikgrinaker): Consider removing range keys in ClearMVCCRangeAndIntents. - iter := NewMVCCRangeKeyIterator(e.engine, MVCCRangeKeyIterOptions{ - LowerBound: key, - UpperBound: endKey, - }) - defer iter.Close() - for { - if ok, err := iter.Valid(); err != nil { - return err - } else if !ok { - break - } - if err := e.engine.ExperimentalClearMVCCRangeKey(iter.Key()); err != nil { - return err - } - iter.Next() - } - return nil + return e.engine.ClearMVCCRangeAndIntents(key, endKey) } func cmdClearRangeKey(e *evalCtx) error { @@ -836,39 +814,6 @@ func cmdPut(e *evalCtx) error { }) } -func cmdRevertRange(e *evalCtx) error { - ts := e.getTs(nil) - revertTS := e.getTsWithName(nil, "revertTS") - key, endKey := e.getKeyRange() - - deleteRangeThreshold := 100 - if e.hasArg("deleteRangeThreshold") { - e.scanArg("deleteRangeThreshold", &deleteRangeThreshold) - } - - maxBatchSize := 1000 - if e.hasArg("maxBatchSize") { - e.scanArg("maxBatchSize", &maxBatchSize) - } - - maxBatchBytes := int(1e6) - if e.hasArg("maxBatchBytes") { - e.scanArg("maxBatchBytes", &maxBatchBytes) - } - - return e.withWriter("revertRange", func(rw ReadWriter) error { - resumeSpan, err := ExperimentalMVCCRevertRange(e.ctx, rw, nil, key, endKey, ts, revertTS, - deleteRangeThreshold, int64(maxBatchSize), int64(maxBatchBytes), 1000) - if err != nil { - return err - } - if resumeSpan != nil { - e.results.buf.Printf("revert_range: resume span [%s,%s)\n", resumeSpan.Key, resumeSpan.EndKey) - } - return err - }) -} - func cmdScan(e *evalCtx) error { txn := e.getTxn(optional) key, endKey := e.getKeyRange() diff --git a/pkg/storage/testdata/mvcc_histories/revert_range b/pkg/storage/testdata/mvcc_histories/revert_range deleted file mode 100644 index 9b39bbadf828..000000000000 --- a/pkg/storage/testdata/mvcc_histories/revert_range +++ /dev/null @@ -1,275 +0,0 @@ -# TODO(erikgrinaker): Test this with existing range keys too. -# TODO(erikgrinaker): Conflict and intent tests. -# TODO(erikgrinaker): Instead of duplicating the data set, we can use -# clear_range to clear the revert_range write timestamp. However, this requires -# support for clearing range keys in MVCCClearTimeRange. - -run ok -clear_range k=a end=z -put k=a v=1 ts=1 -put k=a v=2 ts=2 -put k=b v=2 ts=2 -del k=b ts=3 -put k=c v=1 ts=1 -del k=c ts=2 -put k=c v=3 ts=3 -put k=d v=1 ts=1 -put k=e v=2 ts=2 -del k=f ts=1 -put k=g v=3 ts=3 -revert_range k=a end=z ts=5 revertTS=1 deleteRangeThreshold=2 -scan ts=5 k=a end=z # NB: does not respect range tombstones yet ----- -scan: "a" -> /BYTES/1 @5.000000000,0 -scan: "b" -> / @3.000000000,0 -scan: "c" -> /BYTES/1 @5.000000000,0 -scan: "d" -> /BYTES/1 @1.000000000,0 -scan: "e" -> /BYTES/2 @2.000000000,0 -scan: "f" -> / @1.000000000,0 -scan: "g" -> /BYTES/3 @3.000000000,0 ->> at end: -range key: {e-z}/5.000000000,0 -> [] -data: "a"/5.000000000,0 -> /BYTES/1 -data: "a"/2.000000000,0 -> /BYTES/2 -data: "a"/1.000000000,0 -> /BYTES/1 -data: "b"/3.000000000,0 -> / -data: "b"/2.000000000,0 -> /BYTES/2 -data: "c"/5.000000000,0 -> /BYTES/1 -data: "c"/3.000000000,0 -> /BYTES/3 -data: "c"/2.000000000,0 -> / -data: "c"/1.000000000,0 -> /BYTES/1 -data: "d"/1.000000000,0 -> /BYTES/1 -data: "e"/2.000000000,0 -> /BYTES/2 -data: "f"/1.000000000,0 -> / -data: "g"/3.000000000,0 -> /BYTES/3 - -run ok -clear_range k=a end=z -put k=a v=1 ts=1 -put k=a v=2 ts=2 -put k=b v=2 ts=2 -del k=b ts=3 -put k=c v=1 ts=1 -del k=c ts=2 -put k=c v=3 ts=3 -put k=d v=1 ts=1 -put k=e v=2 ts=2 -del k=f ts=1 -put k=g v=3 ts=3 -revert_range k=a end=z ts=5 revertTS=2 deleteRangeThreshold=2 -scan ts=5 k=a end=z # NB: does not respect range tombstones yet ----- -scan: "a" -> /BYTES/2 @2.000000000,0 -scan: "b" -> /BYTES/2 @5.000000000,0 -scan: "c" -> / @5.000000000,0 -scan: "d" -> /BYTES/1 @1.000000000,0 -scan: "e" -> /BYTES/2 @2.000000000,0 -scan: "f" -> / @1.000000000,0 -scan: "g" -> / @5.000000000,0 ->> at end: -data: "a"/2.000000000,0 -> /BYTES/2 -data: "a"/1.000000000,0 -> /BYTES/1 -data: "b"/5.000000000,0 -> /BYTES/2 -data: "b"/3.000000000,0 -> / -data: "b"/2.000000000,0 -> /BYTES/2 -data: "c"/5.000000000,0 -> / -data: "c"/3.000000000,0 -> /BYTES/3 -data: "c"/2.000000000,0 -> / -data: "c"/1.000000000,0 -> /BYTES/1 -data: "d"/1.000000000,0 -> /BYTES/1 -data: "e"/2.000000000,0 -> /BYTES/2 -data: "f"/1.000000000,0 -> / -data: "g"/5.000000000,0 -> / -data: "g"/3.000000000,0 -> /BYTES/3 - -run ok -clear_range k=a end=z -put k=a v=1 ts=1 -put k=a v=2 ts=2 -put k=b v=2 ts=2 -del k=b ts=3 -put k=c v=1 ts=1 -del k=c ts=2 -put k=c v=3 ts=3 -put k=d v=1 ts=1 -put k=e v=2 ts=2 -del k=f ts=1 -put k=g v=3 ts=3 -revert_range k=a end=z ts=5 revertTS=3 deleteRangeThreshold=2 -scan ts=5 k=a end=z # NB: does not respect range tombstones yet ----- -scan: "a" -> /BYTES/2 @2.000000000,0 -scan: "b" -> / @3.000000000,0 -scan: "c" -> /BYTES/3 @3.000000000,0 -scan: "d" -> /BYTES/1 @1.000000000,0 -scan: "e" -> /BYTES/2 @2.000000000,0 -scan: "f" -> / @1.000000000,0 -scan: "g" -> /BYTES/3 @3.000000000,0 ->> at end: -data: "a"/2.000000000,0 -> /BYTES/2 -data: "a"/1.000000000,0 -> /BYTES/1 -data: "b"/3.000000000,0 -> / -data: "b"/2.000000000,0 -> /BYTES/2 -data: "c"/3.000000000,0 -> /BYTES/3 -data: "c"/2.000000000,0 -> / -data: "c"/1.000000000,0 -> /BYTES/1 -data: "d"/1.000000000,0 -> /BYTES/1 -data: "e"/2.000000000,0 -> /BYTES/2 -data: "f"/1.000000000,0 -> / -data: "g"/3.000000000,0 -> /BYTES/3 - -run ok -clear_range k=a end=z -put k=a v=1 ts=1 -put k=a v=2 ts=2 -put k=b v=2 ts=2 -del k=b ts=3 -put k=c v=1 ts=1 -del k=c ts=2 -put k=c v=3 ts=3 -put k=d v=1 ts=1 -put k=e v=2 ts=2 -del k=f ts=1 -put k=g v=3 ts=3 -revert_range k=c end=z ts=5 revertTS=1 deleteRangeThreshold=2 maxBatchSize=1 -scan ts=5 k=a end=z # NB: does not respect range tombstones yet ----- -revert_range: resume span ["e","z") -scan: "a" -> /BYTES/2 @2.000000000,0 -scan: "b" -> / @3.000000000,0 -scan: "c" -> /BYTES/1 @5.000000000,0 -scan: "d" -> /BYTES/1 @1.000000000,0 -scan: "e" -> /BYTES/2 @2.000000000,0 -scan: "f" -> / @1.000000000,0 -scan: "g" -> /BYTES/3 @3.000000000,0 ->> at end: -data: "a"/2.000000000,0 -> /BYTES/2 -data: "a"/1.000000000,0 -> /BYTES/1 -data: "b"/3.000000000,0 -> / -data: "b"/2.000000000,0 -> /BYTES/2 -data: "c"/5.000000000,0 -> /BYTES/1 -data: "c"/3.000000000,0 -> /BYTES/3 -data: "c"/2.000000000,0 -> / -data: "c"/1.000000000,0 -> /BYTES/1 -data: "d"/1.000000000,0 -> /BYTES/1 -data: "e"/2.000000000,0 -> /BYTES/2 -data: "f"/1.000000000,0 -> / -data: "g"/3.000000000,0 -> /BYTES/3 - -run ok -clear_range k=a end=z -put k=a v=1 ts=1 -put k=a v=2 ts=2 -put k=b v=2 ts=2 -del k=b ts=3 -put k=c v=1 ts=1 -del k=c ts=2 -put k=c v=3 ts=3 -put k=d v=1 ts=1 -put k=e v=2 ts=2 -del k=f ts=1 -put k=g v=3 ts=3 -revert_range k=a end=z ts=5 revertTS=1 deleteRangeThreshold=10 maxBatchSize=3 -scan ts=5 k=a end=z # NB: does not respect range tombstones yet ----- -revert_range: resume span ["g","z") -scan: "a" -> /BYTES/1 @5.000000000,0 -scan: "b" -> / @3.000000000,0 -scan: "c" -> /BYTES/1 @5.000000000,0 -scan: "d" -> /BYTES/1 @1.000000000,0 -scan: "e" -> / @5.000000000,0 -scan: "f" -> / @1.000000000,0 -scan: "g" -> /BYTES/3 @3.000000000,0 ->> at end: -data: "a"/5.000000000,0 -> /BYTES/1 -data: "a"/2.000000000,0 -> /BYTES/2 -data: "a"/1.000000000,0 -> /BYTES/1 -data: "b"/3.000000000,0 -> / -data: "b"/2.000000000,0 -> /BYTES/2 -data: "c"/5.000000000,0 -> /BYTES/1 -data: "c"/3.000000000,0 -> /BYTES/3 -data: "c"/2.000000000,0 -> / -data: "c"/1.000000000,0 -> /BYTES/1 -data: "d"/1.000000000,0 -> /BYTES/1 -data: "e"/5.000000000,0 -> / -data: "e"/2.000000000,0 -> /BYTES/2 -data: "f"/1.000000000,0 -> / -data: "g"/3.000000000,0 -> /BYTES/3 - -run ok -clear_range k=a end=z -put k=a v=1 ts=1 -put k=a v=2 ts=2 -put k=b v=2 ts=2 -del k=b ts=3 -put k=c v=1 ts=1 -del k=c ts=2 -put k=c v=3 ts=3 -put k=d v=1 ts=1 -put k=e v=2 ts=2 -del k=f ts=1 -put k=g v=3 ts=3 -revert_range k=a end=z ts=5 revertTS=1 deleteRangeThreshold=2 maxBatchSize=3 -scan ts=5 k=a end=z # NB: does not respect range tombstones yet ----- -revert_range: resume span ["g","z") -scan: "a" -> /BYTES/1 @5.000000000,0 -scan: "b" -> / @3.000000000,0 -scan: "c" -> /BYTES/1 @5.000000000,0 -scan: "d" -> /BYTES/1 @1.000000000,0 -scan: "e" -> / @5.000000000,0 -scan: "f" -> / @1.000000000,0 -scan: "g" -> /BYTES/3 @3.000000000,0 ->> at end: -data: "a"/5.000000000,0 -> /BYTES/1 -data: "a"/2.000000000,0 -> /BYTES/2 -data: "a"/1.000000000,0 -> /BYTES/1 -data: "b"/3.000000000,0 -> / -data: "b"/2.000000000,0 -> /BYTES/2 -data: "c"/5.000000000,0 -> /BYTES/1 -data: "c"/3.000000000,0 -> /BYTES/3 -data: "c"/2.000000000,0 -> / -data: "c"/1.000000000,0 -> /BYTES/1 -data: "d"/1.000000000,0 -> /BYTES/1 -data: "e"/5.000000000,0 -> / -data: "e"/2.000000000,0 -> /BYTES/2 -data: "f"/1.000000000,0 -> / -data: "g"/3.000000000,0 -> /BYTES/3 - -run ok -clear_range k=a end=z -put k=a v=1 ts=1 -put k=a v=2 ts=2 -put k=b v=2 ts=2 -del k=b ts=3 -put k=c v=1 ts=1 -del k=c ts=2 -put k=c v=3 ts=3 -put k=d v=1 ts=1 -put k=e v=2 ts=2 -del k=f ts=1 -put k=g v=3 ts=3 -revert_range k=c end=z ts=5 revertTS=1 deleteRangeThreshold=3 maxBatchBytes=1 -scan ts=5 k=a end=z # NB: does not respect range tombstones yet ----- -revert_range: resume span ["e","z") -scan: "a" -> /BYTES/2 @2.000000000,0 -scan: "b" -> / @3.000000000,0 -scan: "c" -> /BYTES/1 @5.000000000,0 -scan: "d" -> /BYTES/1 @1.000000000,0 -scan: "e" -> /BYTES/2 @2.000000000,0 -scan: "f" -> / @1.000000000,0 -scan: "g" -> /BYTES/3 @3.000000000,0 ->> at end: -data: "a"/2.000000000,0 -> /BYTES/2 -data: "a"/1.000000000,0 -> /BYTES/1 -data: "b"/3.000000000,0 -> / -data: "b"/2.000000000,0 -> /BYTES/2 -data: "c"/5.000000000,0 -> /BYTES/1 -data: "c"/3.000000000,0 -> /BYTES/3 -data: "c"/2.000000000,0 -> / -data: "c"/1.000000000,0 -> /BYTES/1 -data: "d"/1.000000000,0 -> /BYTES/1 -data: "e"/2.000000000,0 -> /BYTES/2 -data: "f"/1.000000000,0 -> / -data: "g"/3.000000000,0 -> /BYTES/3