diff --git a/pkg/storage/engine.go b/pkg/storage/engine.go index 062efb361e16..a6418c39c4c6 100644 --- a/pkg/storage/engine.go +++ b/pkg/storage/engine.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/storage/fs" + "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/envutil" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/iterutil" @@ -630,27 +631,26 @@ type Writer interface { // returns. ApplyBatchRepr(repr []byte, sync bool) error - // ClearMVCC removes the item from the db with the given MVCCKey. It - // requires that the timestamp is non-empty (see - // {ClearUnversioned,ClearIntent} if the timestamp is empty). Note that - // clear actually removes entries from the storage engine, rather than - // inserting MVCC tombstones. + // ClearMVCC removes the point key with the given MVCCKey from the db. It does + // not affect range keys. It requires that the timestamp is non-empty (see + // ClearUnversioned or ClearIntent if the timestamp is empty). Note that clear + // actually removes entries from the storage engine, rather than inserting + // MVCC tombstones. // // It is safe to modify the contents of the arguments after it returns. ClearMVCC(key MVCCKey) error // ClearUnversioned removes an unversioned item from the db. It is for use // with inline metadata (not intents) and other unversioned keys (like - // Range-ID local keys). + // Range-ID local keys). It does not affect range keys. // // It is safe to modify the contents of the arguments after it returns. ClearUnversioned(key roachpb.Key) error - // ClearIntent removes an intent from the db. Unlike - // {ClearMVCC,ClearUnversioned} this is a higher-level method that may make - // changes in parts of the key space that are not only a function of the - // input, and may choose to use a single-clear under the covers. - // txnDidNotUpdateMeta allows for performance optimization when set to true, - // and has semantics defined in MVCCMetadata.TxnDidNotUpdateMeta (it can - // be conservatively set to false). + // ClearIntent removes an intent from the db. Unlike ClearMVCC and + // ClearUnversioned, this is a higher-level method that may make changes in + // parts of the key space that are not only a function of the input, and may + // choose to use a single-clear under the covers. txnDidNotUpdateMeta allows + // for performance optimization when set to true, and has semantics defined in + // MVCCMetadata.TxnDidNotUpdateMeta (it can be conservatively set to false). // // It is safe to modify the contents of the arguments after it returns. // @@ -660,18 +660,20 @@ type Writer interface { // decrease, we can stop tracking txnDidNotUpdateMeta and still optimize // ClearIntent by always doing single-clear. ClearIntent(key roachpb.Key, txnDidNotUpdateMeta bool, txnUUID uuid.UUID) error - // ClearEngineKey removes the item from the db with the given EngineKey. - // Note that clear actually removes entries from the storage engine. This is - // a general-purpose and low-level method that should be used sparingly, - // only when the other Clear* methods are not applicable. + // ClearEngineKey removes the given point key from the engine. It does not + // affect range keys. Note that clear actually removes entries from the + // storage engine. This is a general-purpose and low-level method that should + // be used sparingly, only when the other Clear* methods are not applicable. // // It is safe to modify the contents of the arguments after it returns. ClearEngineKey(key EngineKey) error - // ClearRawRange removes a set of entries from start (inclusive) to end - // (exclusive) using a Pebble range tombstone. It can be applied to a range - // consisting of MVCCKeys or the more general EngineKeys -- it simply uses the - // roachpb.Key parameters as the Key field of an EngineKey. + // ClearRawRange removes both point keys and range keys from start (inclusive) + // to end (exclusive) using a Pebble range tombstone. It can be applied to a + // range consisting of MVCCKeys or the more general EngineKeys -- it simply + // uses the roachpb.Key parameters as the Key field of an EngineKey. This + // implies that it does not clear intents unless the intent lock table is + // targeted explicitly. // // Similar to the other Clear* methods, this method actually removes entries // from the storage engine. It is safe to modify the contents of the arguments @@ -679,17 +681,17 @@ type Writer interface { ClearRawRange(start, end roachpb.Key) error // ClearMVCCRange removes MVCC keys from start (inclusive) to end (exclusive) // using a Pebble range tombstone. It will remove everything in the span, - // including intents. + // including intents and range keys. // // Similar to the other Clear* methods, this method actually removes entries // from the storage engine. It is safe to modify the contents of the arguments // after it returns. ClearMVCCRange(start, end roachpb.Key) error - // ClearMVCCVersions removes MVCC versions from start (inclusive) to end - // (exclusive) using a Pebble range tombstone. It is meant for efficiently + // ClearMVCCVersions removes MVCC point key versions from start (inclusive) to + // end (exclusive) using a Pebble range tombstone. It is meant for efficiently // clearing a subset of versions of a key, since the parameters are MVCCKeys // and not roachpb.Keys, but it can also be used across multiple keys. It will - // ignore intents, leaving them in place. + // ignore intents and range keys, leaving them in place. // // Similar to the other Clear* methods, this method actually removes entries // from the storage engine. It is safe to modify the contents of the arguments @@ -697,7 +699,7 @@ type Writer interface { ClearMVCCVersions(start, end MVCCKey) error // ClearMVCCIteratorRange removes all keys in the given span using an MVCC // iterator, by clearing individual keys (including intents) with Pebble point - // tombstones. + // tombstones. It will also clear all range keys in the span. // // Similar to the other Clear* methods, this method actually removes entries // from the storage engine. It is safe to modify the contents of the arguments @@ -724,7 +726,10 @@ type Writer interface { // boundaries will be cleared. Clears are idempotent. // // This method is primarily intended for MVCC garbage collection and similar - // internal use. + // internal use. It will do an internal scan across the span first to check + // whether it contains any range keys at all, and clear the smallest single + // span that covers all range keys (if any), to avoid dropping Pebble range + // tombstones across unnecessary spans. // // This method is EXPERIMENTAL: range keys are under active development, and // have severe limitations including being ignored by all KV and MVCC APIs and @@ -1210,9 +1215,10 @@ func WriteSyncNoop(eng Engine) error { } // ClearRangeWithHeuristic clears the keys from start (inclusive) to end -// (exclusive). Depending on the number of keys, it will either use ClearRawRange -// or clear individual keys. It works with EngineKeys, so don't expect it to -// find and clear separated intents if [start, end) refers to MVCC key space. +// (exclusive), including any range keys. Depending on the number of keys, it +// will either use ClearRawRange or clear individual keys. It works with +// EngineKeys, so don't expect it to find and clear separated intents if +// [start,end) refers to MVCC key space. func ClearRangeWithHeuristic(reader Reader, writer Writer, start, end roachpb.Key) error { iter := reader.NewEngineIterator(IterOptions{UpperBound: end}) defer iter.Close() @@ -1225,7 +1231,9 @@ func ClearRangeWithHeuristic(reader Reader, writer Writer, start, end roachpb.Ke // perhaps we should fix Pebble to handle large numbers of tombstones in an // sstable better. Note that we are referring to storage-level tombstones here, // and not MVCC tombstones. - const clearRangeMinKeys = 64 + clearRangeMinKeys := util.ConstantWithMetamorphicTestRange( + "clear-range-with-heuristic-minkeys", 64, 1, 64) + // Peek into the range to see whether it's large enough to justify // ClearRawRange. Note that the work done here is bounded by // clearRangeMinKeys, so it will be fairly cheap even for large @@ -1239,16 +1247,13 @@ func ClearRangeWithHeuristic(reader Reader, writer Writer, start, end roachpb.Ke for valid { count++ if count > clearRangeMinKeys { - break + return writer.ClearRawRange(start, end) } valid, err = iter.NextEngineKey() } if err != nil { return err } - if count > clearRangeMinKeys { - return writer.ClearRawRange(start, end) - } valid, err = iter.SeekEngineKeyGE(EngineKey{Key: start}) for valid { var k EngineKey @@ -1260,7 +1265,10 @@ func ClearRangeWithHeuristic(reader Reader, writer Writer, start, end roachpb.Ke } valid, err = iter.NextEngineKey() } - return err + if err != nil { + return err + } + return writer.ExperimentalClearAllMVCCRangeKeys(start, end) } var ingestDelayL0Threshold = settings.RegisterIntSetting( diff --git a/pkg/storage/engine_test.go b/pkg/storage/engine_test.go index 60248eb5de3b..5daf3bcbe3a4 100644 --- a/pkg/storage/engine_test.go +++ b/pkg/storage/engine_test.go @@ -949,71 +949,6 @@ func TestEngineScan2(t *testing.T) { } } -func testEngineDeleteRange(t *testing.T, clearRange func(engine Engine, start, end MVCCKey) error) { - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - - keys := []MVCCKey{ - mvccKey("a"), - mvccKey("aa"), - mvccKey("aaa"), - mvccKey("ab"), - mvccKey("abc"), - mvccKey(roachpb.RKeyMax), - } - - insertKeys(keys, engine, t) - - // Scan all keys (non-inclusive of final key). - verifyScan(localMax, roachpb.KeyMax, 10, keys[:5], engine, t) - - // Delete a range of keys - if err := clearRange(engine, mvccKey("aa"), mvccKey("abc")); err != nil { - t.Fatal(err) - } - // Verify what's left - verifyScan(localMax, roachpb.KeyMax, 10, - []MVCCKey{mvccKey("a"), mvccKey("abc")}, engine, t) - }) - } -} - -func TestEngineDeleteRange(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - testEngineDeleteRange(t, func(engine Engine, start, end MVCCKey) error { - return engine.ClearMVCCVersions(start, end) - }) -} - -func TestEngineDeleteRangeBatch(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - testEngineDeleteRange(t, func(engine Engine, start, end MVCCKey) error { - batch := engine.NewUnindexedBatch(true /* writeOnly */) - defer batch.Close() - if err := batch.ClearMVCCVersions(start, end); err != nil { - return err - } - batch2 := engine.NewUnindexedBatch(true /* writeOnly */) - defer batch2.Close() - if err := batch2.ApplyBatchRepr(batch.Repr(), false); err != nil { - return err - } - return batch2.Commit(false) - }) -} - -func TestEngineDeleteRangeIterator(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - testEngineDeleteRange(t, func(engine Engine, start, end MVCCKey) error { - return engine.ClearMVCCIteratorRange(start.Key, end.Key) - }) -} - func TestSnapshot(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -1717,6 +1652,156 @@ func TestScanIntents(t *testing.T) { } } +// TestEngineClearRange tests Clear*Range methods and related helpers. +func TestEngineClearRange(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + + // Set up initial dataset, where [b-g) will be cleared. + // [] is intent, o---o is MVCC range tombstone. + // + // 6 [a6][b6] [e6] [g6] + // 5 a5 b5 c5 + // 4 o-------------------o o---o + // 3 e3 + // 2 o-------------------o g2 + // 1 c1 o---------------o + // a b c d e f g h + // + // After a complete clear, the remaining state will be: + // + // 6 [a6] [g6] + // 5 a5 + // 4 o---o o---o + // 3 + // 2 o---o g2 + // 1 o---o + // a b c d e f g h + // + // However, certain clearers cannot clear intents or range keys. + writeInitialData := func(t *testing.T, rw ReadWriter) { + var localTS hlc.ClockTimestamp + txn := roachpb.MakeTransaction("test", nil, roachpb.NormalUserPriority, wallTS(6), 1, 1) + require.NoError(t, MVCCPut(ctx, rw, nil, roachpb.Key("c"), wallTS(1), localTS, stringValue("c1").Value, nil)) + require.NoError(t, rw.ExperimentalPutMVCCRangeKey(rangeKey("d", "h", 1), MVCCValue{})) + require.NoError(t, rw.ExperimentalPutMVCCRangeKey(rangeKey("a", "f", 2), MVCCValue{})) + require.NoError(t, MVCCPut(ctx, rw, nil, roachpb.Key("g"), wallTS(2), localTS, stringValue("g2").Value, nil)) + require.NoError(t, MVCCPut(ctx, rw, nil, roachpb.Key("e"), wallTS(3), localTS, stringValue("e3").Value, nil)) + require.NoError(t, rw.ExperimentalPutMVCCRangeKey(rangeKey("a", "f", 4), MVCCValue{})) + require.NoError(t, rw.ExperimentalPutMVCCRangeKey(rangeKey("g", "h", 4), MVCCValue{})) + require.NoError(t, MVCCPut(ctx, rw, nil, roachpb.Key("a"), wallTS(5), localTS, stringValue("a2").Value, nil)) + require.NoError(t, MVCCPut(ctx, rw, nil, roachpb.Key("b"), wallTS(5), localTS, stringValue("b2").Value, nil)) + require.NoError(t, MVCCPut(ctx, rw, nil, roachpb.Key("c"), wallTS(5), localTS, stringValue("c2").Value, nil)) + require.NoError(t, MVCCPut(ctx, rw, nil, roachpb.Key("a"), wallTS(6), localTS, stringValue("a6").Value, &txn)) + require.NoError(t, MVCCPut(ctx, rw, nil, roachpb.Key("b"), wallTS(6), localTS, stringValue("b6").Value, &txn)) + require.NoError(t, MVCCPut(ctx, rw, nil, roachpb.Key("e"), wallTS(6), localTS, stringValue("e6").Value, &txn)) + require.NoError(t, MVCCPut(ctx, rw, nil, roachpb.Key("g"), wallTS(6), localTS, stringValue("g6").Value, &txn)) + } + start, end := roachpb.Key("b"), roachpb.Key("g") + + testcases := map[string]struct { + clearRange func(ReadWriter, roachpb.Key, roachpb.Key) error + clearsIntents bool + clearsRangeKeys bool + }{ + "ClearRawRange": { + clearRange: func(rw ReadWriter, start, end roachpb.Key) error { + return rw.ClearRawRange(start, end) + }, + clearsIntents: false, + clearsRangeKeys: true, + }, + + "ClearMVCCRange": { + clearRange: func(rw ReadWriter, start, end roachpb.Key) error { + return rw.ClearMVCCRange(start, end) + }, + clearsIntents: true, + clearsRangeKeys: true, + }, + + "ClearMVCCIteratorRange": { + clearRange: func(rw ReadWriter, start, end roachpb.Key) error { + return rw.ClearMVCCIteratorRange(start, end) + }, + clearsIntents: true, + clearsRangeKeys: true, + }, + + "ClearMVCCVersions": { + clearRange: func(rw ReadWriter, start, end roachpb.Key) error { + return rw.ClearMVCCVersions(MVCCKey{Key: start}, MVCCKey{Key: end}) + }, + clearsIntents: false, + clearsRangeKeys: false, + }, + + "ClearRangeWithHeuristic": { + clearRange: func(rw ReadWriter, start, end roachpb.Key) error { + return ClearRangeWithHeuristic(rw, rw, start, end) + }, + clearsIntents: false, + clearsRangeKeys: true, + }, + } + testutils.RunTrueAndFalse(t, "batch", func(t *testing.T, useBatch bool) { + for name, tc := range testcases { + t.Run(name, func(t *testing.T) { + eng := NewDefaultInMemForTesting() + defer eng.Close() + writeInitialData(t, eng) + + rw := ReadWriter(eng) + if useBatch { + batch := eng.NewBatch() + defer batch.Close() + rw = batch + } + + require.NoError(t, tc.clearRange(rw, start, end)) + + // We always expect all point keys to be cleared. We'll find provisional + // values for the intents. + require.Equal(t, []MVCCKey{ + pointKey("a", 6), pointKey("a", 5), pointKey("g", 6), pointKey("g", 2), + }, scanPointKeys(t, rw)) + + // Which separated intents we find will depend on the clearer. + if tc.clearsIntents { + require.Equal(t, []roachpb.Key{roachpb.Key("a"), roachpb.Key("g")}, scanIntentKeys(t, rw)) + } else { + require.Equal(t, []roachpb.Key{ + roachpb.Key("a"), roachpb.Key("b"), roachpb.Key("e"), roachpb.Key("g"), + }, scanIntentKeys(t, rw)) + } + + // Which range keys we find will depend on the clearer. + if tc.clearsRangeKeys { + require.Equal(t, []MVCCRangeKeyValue{ + rangeKV("a", "b", 4, MVCCValue{}), + rangeKV("a", "b", 2, MVCCValue{}), + rangeKV("g", "h", 4, MVCCValue{}), + rangeKV("g", "h", 1, MVCCValue{}), + }, scanRangeKeys(t, rw)) + } else { + require.Equal(t, []MVCCRangeKeyValue{ + rangeKV("a", "d", 4, MVCCValue{}), + rangeKV("a", "d", 2, MVCCValue{}), + rangeKV("d", "f", 4, MVCCValue{}), + rangeKV("d", "f", 2, MVCCValue{}), + rangeKV("d", "f", 1, MVCCValue{}), + rangeKV("f", "g", 1, MVCCValue{}), + rangeKV("g", "h", 4, MVCCValue{}), + rangeKV("g", "h", 1, MVCCValue{}), + }, scanRangeKeys(t, rw)) + } + }) + } + }) +} + // TestEngineIteratorVisibility checks iterator visibility for various readers. // See comment on Engine.NewMVCCIterator for detailed visibility semantics. func TestEngineIteratorVisibility(t *testing.T) { @@ -1983,7 +2068,8 @@ func TestEngineRangeKeyMutations(t *testing.T) { rangeKV("g", "h", 1, MVCCValue{}), }, scanRangeKeys(t, rw)) - // Clear all range keys in the [c-f) span. + // Clear all range keys in the [c-f) span. Twice for idempotency. + require.NoError(t, rw.ExperimentalClearAllMVCCRangeKeys(roachpb.Key("c"), roachpb.Key("f"))) require.NoError(t, rw.ExperimentalClearAllMVCCRangeKeys(roachpb.Key("c"), roachpb.Key("f"))) require.Equal(t, []MVCCRangeKeyValue{ rangeKV("a", "c", 1, MVCCValue{}), @@ -2125,6 +2211,64 @@ func TestEngineRangeKeysUnsupported(t *testing.T) { } } +// TestUnindexedBatchClearAllRangeKeys tests that range keys are properly +// cleared via an unindexed batch. This tests an optimization in +// pebbleBatch.ExperimentalClearAllMVCCRangeKeys that tightens the span bounds +// to existing keys to avoid dropping unnecessary Pebble range tombstones, which +// must handle the range keys in the unindexed batch correctly. +func TestUnindexedBatchClearAllRangeKeys(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + eng := NewDefaultInMemForTesting() + defer eng.Close() + + // Write a range key [a-d)@1 into the engine. + require.NoError(t, eng.ExperimentalPutMVCCRangeKey(rangeKey("a", "d", 1), MVCCValue{})) + + // Set up an unindexed batch, and write [e-h)@2 into it. Then clear + // all range keys in the [c-f) span via the batch and commit it. + batch := eng.NewUnindexedBatch(true /* writeOnly */) + defer batch.Close() + + require.NoError(t, batch.ExperimentalPutMVCCRangeKey(rangeKey("e", "h", 2), MVCCValue{})) + require.NoError(t, batch.ExperimentalClearAllMVCCRangeKeys(roachpb.Key("c"), roachpb.Key("f"))) + require.NoError(t, batch.Commit(false /* sync */)) + + // Read range keys back from engine. We should find [a-c)@1 and [f-h)@2. + require.Equal(t, []MVCCRangeKeyValue{ + rangeKV("a", "c", 1, MVCCValue{}), + rangeKV("f", "h", 2, MVCCValue{}), + }, scanRangeKeys(t, eng)) + + // Now, set up another unindexed batch without any range keys, and clear + // all range keys in [b-g) via it. + batch = eng.NewUnindexedBatch(true /* writeOnly */) + defer batch.Close() + + require.NoError(t, batch.ExperimentalClearAllMVCCRangeKeys(roachpb.Key("b"), roachpb.Key("g"))) + require.NoError(t, batch.Commit(false /* sync */)) + + // Read range keys back from engine. We should find [a-b)@1 and [g-h)@2. + require.Equal(t, []MVCCRangeKeyValue{ + rangeKV("a", "b", 1, MVCCValue{}), + rangeKV("g", "h", 2, MVCCValue{}), + }, scanRangeKeys(t, eng)) + + // Now clear everything. Twice, for idempotency. + batch = eng.NewUnindexedBatch(true /* writeOnly */) + defer batch.Close() + require.NoError(t, batch.ExperimentalClearAllMVCCRangeKeys(roachpb.Key("a"), roachpb.Key("z"))) + require.NoError(t, batch.Commit(false /* sync */)) + + batch = eng.NewUnindexedBatch(true /* writeOnly */) + defer batch.Close() + require.NoError(t, batch.ExperimentalClearAllMVCCRangeKeys(roachpb.Key("a"), roachpb.Key("z"))) + require.NoError(t, batch.Commit(false /* sync */)) + + require.Empty(t, scanRangeKeys(t, eng)) +} + // scanRangeKeys scans all range keys from the reader. func scanRangeKeys(t *testing.T, r Reader) []MVCCRangeKeyValue { t.Helper() @@ -2135,10 +2279,9 @@ func scanRangeKeys(t *testing.T, r Reader) []MVCCRangeKeyValue { UpperBound: keys.MaxKey, }) defer iter.Close() - iter.SeekGE(MVCCKey{Key: keys.LocalMax}) var rangeKeys []MVCCRangeKeyValue - for { + for iter.SeekGE(MVCCKey{Key: keys.LocalMax}); ; iter.Next() { ok, err := iter.Valid() require.NoError(t, err) if !ok { @@ -2147,11 +2290,46 @@ func scanRangeKeys(t *testing.T, r Reader) []MVCCRangeKeyValue { for _, rangeKey := range iter.RangeKeys() { rangeKeys = append(rangeKeys, rangeKey.Clone()) } - iter.Next() } return rangeKeys } +// scanPointKeys scans all point keys from the reader, excluding intents. +func scanPointKeys(t *testing.T, r Reader) []MVCCKey { + t.Helper() + + iter := r.NewMVCCIterator(MVCCKeyIterKind, IterOptions{ + LowerBound: keys.LocalMax, + UpperBound: keys.MaxKey, + }) + defer iter.Close() + + var pointKeys []MVCCKey + for iter.SeekGE(MVCCKey{Key: keys.LocalMax}); ; iter.Next() { + ok, err := iter.Valid() + require.NoError(t, err) + if !ok { + break + } + pointKeys = append(pointKeys, iter.Key()) + } + return pointKeys +} + +// scanIntentKeys scans all separated intents from the reader, ignoring +// provisional values. +func scanIntentKeys(t *testing.T, r Reader) []roachpb.Key { + t.Helper() + + var intentKeys []roachpb.Key + intents, err := ScanIntents(context.Background(), r, keys.LocalMax, keys.MaxKey, 0, 0) + require.NoError(t, err) + for _, intent := range intents { + intentKeys = append(intentKeys, intent.Key.Clone()) + } + return intentKeys +} + // scanIter scans all point/range keys from the iterator, and returns a combined // slice of MVCCRangeKeyValue and MVCCKeyValue in order. func scanIter(t *testing.T, iter MVCCIterator) []interface{} { diff --git a/pkg/storage/mvcc_history_test.go b/pkg/storage/mvcc_history_test.go index fb850b205c38..b6853cb6fe4c 100644 --- a/pkg/storage/mvcc_history_test.go +++ b/pkg/storage/mvcc_history_test.go @@ -82,6 +82,7 @@ import ( // // clear k= [ts=[,]] // clear_range k= end= +// clear_rangekey k= end= ts=[,] // // Where `` can be a simple string, or a string // prefixed by the following characters: @@ -477,17 +478,18 @@ var commands = map[string]cmd{ // TODO(nvanbenschoten): test "resolve_intent_range". "check_intent": {typReadOnly, cmdCheckIntent}, - "clear": {typDataUpdate, cmdClear}, - "clear_range": {typDataUpdate, cmdClearRange}, - "cput": {typDataUpdate, cmdCPut}, - "del": {typDataUpdate, cmdDelete}, - "del_range": {typDataUpdate, cmdDeleteRange}, - "get": {typReadOnly, cmdGet}, - "increment": {typDataUpdate, cmdIncrement}, - "merge": {typDataUpdate, cmdMerge}, - "put": {typDataUpdate, cmdPut}, - "put_rangekey": {typDataUpdate, cmdPutRangeKey}, - "scan": {typReadOnly, cmdScan}, + "clear": {typDataUpdate, cmdClear}, + "clear_range": {typDataUpdate, cmdClearRange}, + "clear_rangekey": {typDataUpdate, cmdClearRangeKey}, + "cput": {typDataUpdate, cmdCPut}, + "del": {typDataUpdate, cmdDelete}, + "del_range": {typDataUpdate, cmdDeleteRange}, + "get": {typReadOnly, cmdGet}, + "increment": {typDataUpdate, cmdIncrement}, + "merge": {typDataUpdate, cmdMerge}, + "put": {typDataUpdate, cmdPut}, + "put_rangekey": {typDataUpdate, cmdPutRangeKey}, + "scan": {typReadOnly, cmdScan}, "iter_new": {typReadOnly, cmdIterNew}, "iter_seek_ge": {typReadOnly, cmdIterSeekGE}, @@ -685,9 +687,21 @@ func cmdClear(e *evalCtx) error { func cmdClearRange(e *evalCtx) error { key, endKey := e.getKeyRange() + // NB: We can't test ClearRawRange or ClearRangeUsingHeuristic here, because + // it does not handle separated intents. + if util.ConstantWithMetamorphicTestBool("clear-range-using-iterator", false) { + return e.engine.ClearMVCCIteratorRange(key, endKey) + } return e.engine.ClearMVCCRange(key, endKey) } +func cmdClearRangeKey(e *evalCtx) error { + key, endKey := e.getKeyRange() + ts := e.getTs(nil) + return e.engine.ExperimentalClearMVCCRangeKey( + MVCCRangeKey{StartKey: key, EndKey: endKey, Timestamp: ts}) +} + func cmdCPut(e *evalCtx) error { txn := e.getTxn(optional) ts := e.getTs(txn) diff --git a/pkg/storage/mvcc_key_test.go b/pkg/storage/mvcc_key_test.go index 447c803a9580..51509ed3ba92 100644 --- a/pkg/storage/mvcc_key_test.go +++ b/pkg/storage/mvcc_key_test.go @@ -474,7 +474,7 @@ func TestMVCCRangeKeyValidate(t *testing.T) { } func pointKey(key string, ts int) MVCCKey { - return MVCCKey{Key: roachpb.Key(key), Timestamp: hlc.Timestamp{WallTime: int64(ts)}} + return MVCCKey{Key: roachpb.Key(key), Timestamp: wallTS(ts)} } func pointKV(key string, ts int, value string) MVCCKeyValue { @@ -488,7 +488,7 @@ func rangeKey(start, end string, ts int) MVCCRangeKey { return MVCCRangeKey{ StartKey: roachpb.Key(start), EndKey: roachpb.Key(end), - Timestamp: hlc.Timestamp{WallTime: int64(ts)}, + Timestamp: wallTS(ts), } } @@ -505,3 +505,7 @@ func rangeKV(start, end string, ts int, v MVCCValue) MVCCRangeKeyValue { Value: valueBytes, } } + +func wallTS(ts int) hlc.Timestamp { + return hlc.Timestamp{WallTime: int64(ts)} +} diff --git a/pkg/storage/pebble.go b/pkg/storage/pebble.go index f382644d01f1..f05e6621148f 100644 --- a/pkg/storage/pebble.go +++ b/pkg/storage/pebble.go @@ -1244,7 +1244,11 @@ func (p *Pebble) SingleClearEngineKey(key EngineKey) error { // ClearRawRange implements the Engine interface. func (p *Pebble) ClearRawRange(start, end roachpb.Key) error { - return p.clearRange(MVCCKey{Key: start}, MVCCKey{Key: end}) + startKey, endKey := EncodeMVCCKey(MVCCKey{Key: start}), EncodeMVCCKey(MVCCKey{Key: end}) + if err := p.db.DeleteRange(startKey, endKey, pebble.Sync); err != nil { + return err + } + return p.ExperimentalClearAllMVCCRangeKeys(start, end) } // ClearMVCCRange implements the Engine interface. @@ -1255,13 +1259,7 @@ func (p *Pebble) ClearMVCCRange(start, end roachpb.Key) error { // ClearMVCCVersions implements the Engine interface. func (p *Pebble) ClearMVCCVersions(start, end MVCCKey) error { - return p.clearRange(start, end) -} - -func (p *Pebble) clearRange(start, end MVCCKey) error { - bufStart := EncodeMVCCKey(start) - bufEnd := EncodeMVCCKey(end) - return p.db.DeleteRange(bufStart, bufEnd, pebble.Sync) + return p.db.DeleteRange(EncodeMVCCKey(start), EncodeMVCCKey(end), pebble.Sync) } // ClearMVCCIteratorRange implements the Engine interface. @@ -1301,8 +1299,19 @@ func (p *Pebble) ExperimentalClearAllMVCCRangeKeys(start, end roachpb.Key) error if err := rangeKey.Validate(); err != nil { return err } - return p.db.Experimental().RangeKeyDelete( - EncodeMVCCKeyPrefix(start), EncodeMVCCKeyPrefix(end), pebble.Sync) + // Look for any range keys in the span before dropping a range tombstone, and + // use the smallest possible span that covers them, to avoid dropping range + // tombstones across unnecessary spans. We don't worry about races here, + // because this is a non-MVCC operation where the caller must guarantee + // appropriate isolation. + clearFrom, clearTo, err := pebbleFindRangeKeySpan(p.db, + EncodeMVCCKeyPrefix(start), EncodeMVCCKeyPrefix(end)) + if err != nil { + return err + } else if clearFrom == nil || clearTo == nil { + return nil + } + return p.db.Experimental().RangeKeyDelete(clearFrom, clearTo, pebble.Sync) } // ExperimentalPutMVCCRangeKey implements the Engine interface. @@ -2574,3 +2583,40 @@ func pebbleExportToSst( return rows.BulkOpSummary, MVCCKey{Key: resumeKey, Timestamp: resumeTS}, nil } + +// pebbleFindRangeKeySpan returns the minimum span within the given bounds that +// covers all contained range keys. If there are no range keys within the +// bounds, this returns nil keys. +func pebbleFindRangeKeySpan(r pebble.Reader, lower, upper []byte) ([]byte, []byte, error) { + iter := r.NewIter(&pebble.IterOptions{ + KeyTypes: pebble.IterKeyTypeRangesOnly, + LowerBound: lower, + UpperBound: upper, + }) + defer func() { + // We handle errors during iteration. + _ = iter.Close() + }() + + // Look for a range key. If none are found, return nil bounds. + if !iter.SeekGE(lower) { + return nil, nil, iter.Error() + } + rangeStart, _ := iter.RangeBounds() + start := append([]byte{}, rangeStart...) + + // Find the end of the span. + if !iter.SeekLT(upper) { + if err := iter.Error(); err != nil { + return nil, nil, err + } + return nil, nil, errors.AssertionFailedf("unexpected missing range key in %s-%s", lower, upper) + } + _, rangeEnd := iter.RangeBounds() + end := append([]byte{}, rangeEnd...) + + if bytes.Compare(start, end) >= 0 { + return nil, nil, errors.AssertionFailedf("range key end %s at or before start %s", end, start) + } + return start, end, nil +} diff --git a/pkg/storage/pebble_batch.go b/pkg/storage/pebble_batch.go index d99ff1ff875d..5fd5f2201024 100644 --- a/pkg/storage/pebble_batch.go +++ b/pkg/storage/pebble_batch.go @@ -49,10 +49,11 @@ type pebbleBatch struct { prefixEngineIter pebbleIterator normalEngineIter pebbleIterator - iter cloneableIter - writeOnly bool - iterUnused bool - closed bool + iter cloneableIter + writeOnly bool + iterUnused bool + containsRangeKeys bool + closed bool wrappedIntentWriter intentDemuxWriter // scratch space for wrappedIntentWriter. @@ -374,7 +375,11 @@ func (p *pebbleBatch) SingleClearEngineKey(key EngineKey) error { // ClearRawRange implements the Batch interface. func (p *pebbleBatch) ClearRawRange(start, end roachpb.Key) error { - return p.clearRange(MVCCKey{Key: start}, MVCCKey{Key: end}) + p.buf = EncodeMVCCKeyToBuf(p.buf[:0], MVCCKey{Key: start}) + if err := p.batch.DeleteRange(p.buf, EncodeMVCCKey(MVCCKey{Key: end}), nil); err != nil { + return err + } + return p.ExperimentalClearAllMVCCRangeKeys(start, end) } // ClearMVCCRange implements the Batch interface. @@ -386,13 +391,8 @@ func (p *pebbleBatch) ClearMVCCRange(start, end roachpb.Key) error { // ClearMVCCVersions implements the Batch interface. func (p *pebbleBatch) ClearMVCCVersions(start, end MVCCKey) error { - return p.clearRange(start, end) -} - -func (p *pebbleBatch) clearRange(start, end MVCCKey) error { p.buf = EncodeMVCCKeyToBuf(p.buf[:0], start) - buf2 := EncodeMVCCKey(end) - return p.batch.DeleteRange(p.buf, buf2, nil) + return p.batch.DeleteRange(p.buf, EncodeMVCCKey(end), nil) } // ClearIterRange implements the Batch interface. @@ -416,7 +416,7 @@ func (p *pebbleBatch) ClearMVCCIteratorRange(start, end roachpb.Key) error { return err } } - return nil + return p.ExperimentalClearAllMVCCRangeKeys(start, end) } // ExperimentalClearMVCCRangeKey implements the Engine interface. @@ -443,8 +443,30 @@ func (p *pebbleBatch) ExperimentalClearAllMVCCRangeKeys(start, end roachpb.Key) if err := rangeKey.Validate(); err != nil { return err } - return p.batch.Experimental().RangeKeyDelete( - EncodeMVCCKeyPrefix(start), EncodeMVCCKeyPrefix(end), nil) + // Look for any range keys in the span, and use the smallest possible span + // that covers them, to avoid dropping range tombstones across unnecessary + // spans. We don't worry about races here, because this is a non-MVCC + // operation where the caller must guarantee appropriate isolation. + // + // If we're using an unindexed batch, then we have to read from the database. + // However, if the unindexed batch itself contains range keys then we can't + // know where they are, so we have to delete the full span. This seems + // unlikely to ever happen. + clearFrom, clearTo := EncodeMVCCKeyPrefix(start), EncodeMVCCKeyPrefix(end) + if p.batch.Indexed() || !p.containsRangeKeys { + var err error + r := pebble.Reader(p.batch) + if !p.batch.Indexed() { + r = p.db + } + clearFrom, clearTo, err = pebbleFindRangeKeySpan(r, clearFrom, clearTo) + if err != nil { + return err + } else if clearFrom == nil || clearTo == nil { + return nil + } + } + return p.batch.Experimental().RangeKeyDelete(clearFrom, clearTo, nil) } // ExperimentalPutMVCCRangeKey implements the Batch interface. @@ -464,12 +486,18 @@ func (p *pebbleBatch) ExperimentalPutMVCCRangeKey(rangeKey MVCCRangeKey, value M if err != nil { return errors.Wrapf(err, "failed to encode MVCC value for range key %s", rangeKey) } - return p.batch.Experimental().RangeKeySet( + if err := p.batch.Experimental().RangeKeySet( EncodeMVCCKeyPrefix(rangeKey.StartKey), EncodeMVCCKeyPrefix(rangeKey.EndKey), EncodeMVCCTimestampSuffix(rangeKey.Timestamp), valueBytes, - nil) + nil); err != nil { + return err + } + // Mark the batch as containing range keys. See + // ExperimentalClearAllMVCCRangeKeys for why. + p.containsRangeKeys = true + return nil } // Merge implements the Batch interface. diff --git a/pkg/storage/sst_writer.go b/pkg/storage/sst_writer.go index 9f023e31c740..f9dc2b20826f 100644 --- a/pkg/storage/sst_writer.go +++ b/pkg/storage/sst_writer.go @@ -128,6 +128,8 @@ func (fw *SSTWriter) Finish() error { } // ClearRawRange implements the Writer interface. +// +// TODO(erikgrinaker): This must clear range keys when SSTs support them. func (fw *SSTWriter) ClearRawRange(start, end roachpb.Key) error { return fw.clearRange(MVCCKey{Key: start}, MVCCKey{Key: end}) } @@ -153,8 +155,10 @@ func (fw *SSTWriter) ExperimentalClearMVCCRangeKey(MVCCRangeKey) error { } // ExperimentalClearAllMVCCRangeKeys implements the Writer interface. +// +// TODO(erikgrinaker): This must clear range keys when SSTs support them. func (fw *SSTWriter) ExperimentalClearAllMVCCRangeKeys(roachpb.Key, roachpb.Key) error { - panic("not implemented") + return nil } func (fw *SSTWriter) clearRange(start, end MVCCKey) error { diff --git a/pkg/storage/testdata/mvcc_histories/range_key_clear b/pkg/storage/testdata/mvcc_histories/range_key_clear new file mode 100644 index 000000000000..f90f4fcb9b77 --- /dev/null +++ b/pkg/storage/testdata/mvcc_histories/range_key_clear @@ -0,0 +1,293 @@ +# Tests MVCC range key clearing. +# +# Sets up following dataset, where x is tombstone, o-o is range tombstone, [] is intent. +# +# T +# 7 [a7] [d7] [j7] [l7] +# 6 f6 +# 5 o---------------o k5 +# 4 x x d4 f4 g4 +# 3 o-------o e3 o-------oh3 +# 2 a2 f2 g2 +# 1 o---------------------------------------o +# a b c d e f g h i j k +# +run ok +put_rangekey k=a end=k ts=1 +put_rangekey k=l end=m ts=1 +put_rangekey k=b end=d ts=3 +put k=a ts=2 v=a2 +del k=a ts=4 +del k=b ts=4 +put k=d ts=4 v=d4 +put k=e ts=3 v=e3 +put k=f ts=2 v=f2 +put k=g ts=2 v=g2 +put_rangekey k=f end=h ts=3 +put k=f ts=4 v=f4 +put k=f ts=6 v=f6 +put k=g ts=4 v=g4 +put_rangekey k=c end=g ts=5 +put k=h ts=3 v=h3 +del k=h ts=4 +put k=k ts=5 v=k5 +with t=A + txn_begin ts=7 + put k=a v=a7 + put k=d v=d7 + put k=j v=j7 + put k=l v=l7 +---- +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=7.000000000,0 wto=false gul=0,0 +rangekey: {a-b}/[1.000000000,0=/] +rangekey: {b-c}/[3.000000000,0=/ 1.000000000,0=/] +rangekey: {c-d}/[5.000000000,0=/ 3.000000000,0=/ 1.000000000,0=/] +rangekey: {d-f}/[5.000000000,0=/ 1.000000000,0=/] +rangekey: {f-g}/[5.000000000,0=/ 3.000000000,0=/ 1.000000000,0=/] +rangekey: {g-h}/[3.000000000,0=/ 1.000000000,0=/] +rangekey: {h-k}/[1.000000000,0=/] +rangekey: {l-m}/[1.000000000,0=/] +meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "a"/7.000000000,0 -> /BYTES/a7 +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/4.000000000,0 -> / +meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> /BYTES/d7 +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "e"/3.000000000,0 -> /BYTES/e3 +data: "f"/6.000000000,0 -> /BYTES/f6 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "f"/2.000000000,0 -> /BYTES/f2 +data: "g"/4.000000000,0 -> /BYTES/g4 +data: "g"/2.000000000,0 -> /BYTES/g2 +data: "h"/4.000000000,0 -> / +data: "h"/3.000000000,0 -> /BYTES/h3 +meta: "j"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "j"/7.000000000,0 -> /BYTES/j7 +data: "k"/5.000000000,0 -> /BYTES/k5 +meta: "l"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "l"/7.000000000,0 -> /BYTES/l7 + +# Clear a few range key segments. +run ok +clear_rangekey k=f end=g ts=3 +---- +>> at end: +rangekey: {a-b}/[1.000000000,0=/] +rangekey: {b-c}/[3.000000000,0=/ 1.000000000,0=/] +rangekey: {c-d}/[5.000000000,0=/ 3.000000000,0=/ 1.000000000,0=/] +rangekey: {d-g}/[5.000000000,0=/ 1.000000000,0=/] +rangekey: {g-h}/[3.000000000,0=/ 1.000000000,0=/] +rangekey: {h-k}/[1.000000000,0=/] +rangekey: {l-m}/[1.000000000,0=/] +meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "a"/7.000000000,0 -> /BYTES/a7 +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/4.000000000,0 -> / +meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> /BYTES/d7 +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "e"/3.000000000,0 -> /BYTES/e3 +data: "f"/6.000000000,0 -> /BYTES/f6 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "f"/2.000000000,0 -> /BYTES/f2 +data: "g"/4.000000000,0 -> /BYTES/g4 +data: "g"/2.000000000,0 -> /BYTES/g2 +data: "h"/4.000000000,0 -> / +data: "h"/3.000000000,0 -> /BYTES/h3 +meta: "j"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "j"/7.000000000,0 -> /BYTES/j7 +data: "k"/5.000000000,0 -> /BYTES/k5 +meta: "l"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "l"/7.000000000,0 -> /BYTES/l7 + +run ok +clear_rangekey k=e end=f ts=1 +---- +>> at end: +rangekey: {a-b}/[1.000000000,0=/] +rangekey: {b-c}/[3.000000000,0=/ 1.000000000,0=/] +rangekey: {c-d}/[5.000000000,0=/ 3.000000000,0=/ 1.000000000,0=/] +rangekey: {d-e}/[5.000000000,0=/ 1.000000000,0=/] +rangekey: {e-f}/[5.000000000,0=/] +rangekey: {f-g}/[5.000000000,0=/ 1.000000000,0=/] +rangekey: {g-h}/[3.000000000,0=/ 1.000000000,0=/] +rangekey: {h-k}/[1.000000000,0=/] +rangekey: {l-m}/[1.000000000,0=/] +meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "a"/7.000000000,0 -> /BYTES/a7 +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/4.000000000,0 -> / +meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> /BYTES/d7 +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "e"/3.000000000,0 -> /BYTES/e3 +data: "f"/6.000000000,0 -> /BYTES/f6 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "f"/2.000000000,0 -> /BYTES/f2 +data: "g"/4.000000000,0 -> /BYTES/g4 +data: "g"/2.000000000,0 -> /BYTES/g2 +data: "h"/4.000000000,0 -> / +data: "h"/3.000000000,0 -> /BYTES/h3 +meta: "j"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "j"/7.000000000,0 -> /BYTES/j7 +data: "k"/5.000000000,0 -> /BYTES/k5 +meta: "l"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "l"/7.000000000,0 -> /BYTES/l7 + +# Clearing segments is idempotent and works on missing segments. +run ok +clear_rangekey k=f end=g ts=3 +---- +>> at end: +rangekey: {a-b}/[1.000000000,0=/] +rangekey: {b-c}/[3.000000000,0=/ 1.000000000,0=/] +rangekey: {c-d}/[5.000000000,0=/ 3.000000000,0=/ 1.000000000,0=/] +rangekey: {d-e}/[5.000000000,0=/ 1.000000000,0=/] +rangekey: {e-f}/[5.000000000,0=/] +rangekey: {f-g}/[5.000000000,0=/ 1.000000000,0=/] +rangekey: {g-h}/[3.000000000,0=/ 1.000000000,0=/] +rangekey: {h-k}/[1.000000000,0=/] +rangekey: {l-m}/[1.000000000,0=/] +meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "a"/7.000000000,0 -> /BYTES/a7 +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/4.000000000,0 -> / +meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> /BYTES/d7 +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "e"/3.000000000,0 -> /BYTES/e3 +data: "f"/6.000000000,0 -> /BYTES/f6 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "f"/2.000000000,0 -> /BYTES/f2 +data: "g"/4.000000000,0 -> /BYTES/g4 +data: "g"/2.000000000,0 -> /BYTES/g2 +data: "h"/4.000000000,0 -> / +data: "h"/3.000000000,0 -> /BYTES/h3 +meta: "j"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "j"/7.000000000,0 -> /BYTES/j7 +data: "k"/5.000000000,0 -> /BYTES/k5 +meta: "l"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "l"/7.000000000,0 -> /BYTES/l7 + +run ok +clear_rangekey k=a end=z ts=10 +---- +>> at end: +rangekey: {a-b}/[1.000000000,0=/] +rangekey: {b-c}/[3.000000000,0=/ 1.000000000,0=/] +rangekey: {c-d}/[5.000000000,0=/ 3.000000000,0=/ 1.000000000,0=/] +rangekey: {d-e}/[5.000000000,0=/ 1.000000000,0=/] +rangekey: {e-f}/[5.000000000,0=/] +rangekey: {f-g}/[5.000000000,0=/ 1.000000000,0=/] +rangekey: {g-h}/[3.000000000,0=/ 1.000000000,0=/] +rangekey: {h-k}/[1.000000000,0=/] +rangekey: {l-m}/[1.000000000,0=/] +meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "a"/7.000000000,0 -> /BYTES/a7 +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/4.000000000,0 -> / +meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> /BYTES/d7 +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "e"/3.000000000,0 -> /BYTES/e3 +data: "f"/6.000000000,0 -> /BYTES/f6 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "f"/2.000000000,0 -> /BYTES/f2 +data: "g"/4.000000000,0 -> /BYTES/g4 +data: "g"/2.000000000,0 -> /BYTES/g2 +data: "h"/4.000000000,0 -> / +data: "h"/3.000000000,0 -> /BYTES/h3 +meta: "j"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "j"/7.000000000,0 -> /BYTES/j7 +data: "k"/5.000000000,0 -> /BYTES/k5 +meta: "l"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "l"/7.000000000,0 -> /BYTES/l7 + +# Now clear a few spans. +run ok +clear_range k=c end=d +---- +>> at end: +rangekey: {a-b}/[1.000000000,0=/] +rangekey: {b-c}/[3.000000000,0=/ 1.000000000,0=/] +rangekey: {d-e}/[5.000000000,0=/ 1.000000000,0=/] +rangekey: {e-f}/[5.000000000,0=/] +rangekey: {f-g}/[5.000000000,0=/ 1.000000000,0=/] +rangekey: {g-h}/[3.000000000,0=/ 1.000000000,0=/] +rangekey: {h-k}/[1.000000000,0=/] +rangekey: {l-m}/[1.000000000,0=/] +meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "a"/7.000000000,0 -> /BYTES/a7 +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/4.000000000,0 -> / +meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> /BYTES/d7 +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "e"/3.000000000,0 -> /BYTES/e3 +data: "f"/6.000000000,0 -> /BYTES/f6 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "f"/2.000000000,0 -> /BYTES/f2 +data: "g"/4.000000000,0 -> /BYTES/g4 +data: "g"/2.000000000,0 -> /BYTES/g2 +data: "h"/4.000000000,0 -> / +data: "h"/3.000000000,0 -> /BYTES/h3 +meta: "j"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "j"/7.000000000,0 -> /BYTES/j7 +data: "k"/5.000000000,0 -> /BYTES/k5 +meta: "l"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "l"/7.000000000,0 -> /BYTES/l7 + +run ok +clear_range k=f end=g +---- +>> at end: +rangekey: {a-b}/[1.000000000,0=/] +rangekey: {b-c}/[3.000000000,0=/ 1.000000000,0=/] +rangekey: {d-e}/[5.000000000,0=/ 1.000000000,0=/] +rangekey: {e-f}/[5.000000000,0=/] +rangekey: {g-h}/[3.000000000,0=/ 1.000000000,0=/] +rangekey: {h-k}/[1.000000000,0=/] +rangekey: {l-m}/[1.000000000,0=/] +meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "a"/7.000000000,0 -> /BYTES/a7 +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/4.000000000,0 -> / +meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> /BYTES/d7 +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "e"/3.000000000,0 -> /BYTES/e3 +data: "g"/4.000000000,0 -> /BYTES/g4 +data: "g"/2.000000000,0 -> /BYTES/g2 +data: "h"/4.000000000,0 -> / +data: "h"/3.000000000,0 -> /BYTES/h3 +meta: "j"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "j"/7.000000000,0 -> /BYTES/j7 +data: "k"/5.000000000,0 -> /BYTES/k5 +meta: "l"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "l"/7.000000000,0 -> /BYTES/l7 + +# Clear the entire span. +run ok +clear_range k=a end=z +---- +>> at end: + + +# Error conditions. We don't check clear_range, because it uses a metamorphic +# bool to switch between ClearMVCCRange and ClearMVCCIteratorRange, and these +# fail in different ways. +run error +clear_rangekey k=c end=a ts=1 +---- +>> at end: + +error: (*withstack.withStack:) invalid range key {c-a}/1.000000000,0: start key "c" is at or after end key "a"