From d9bfcfc624ce2ec9f770e9932ed2b6c64b93f88c Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Mon, 8 Aug 2022 14:33:04 -0700 Subject: [PATCH] WIP on updating MVCCDelete This commit makes it so that `MVCCDelete` as well as `DeleteRequest` always return whether the key was actually deleted. If approved, it'll be squashed into the previous commit. --- pkg/cli/debug.go | 2 +- pkg/clusterversion/cockroach_versions.go | 4 +- pkg/kv/batch.go | 22 +++---- pkg/kv/db.go | 15 +++-- pkg/kv/db_test.go | 60 ++++++++----------- pkg/kv/kvnemesis/doc.go | 2 +- pkg/kv/kvserver/abortspan/abortspan.go | 3 +- pkg/kv/kvserver/batcheval/cmd_delete.go | 12 ++-- .../batcheval/cmd_delete_range_test.go | 3 +- .../kvserver/batcheval/cmd_end_transaction.go | 3 +- .../batcheval/cmd_query_intent_test.go | 3 +- .../cmd_query_resolved_timestamp_test.go | 9 ++- pkg/kv/kvserver/batcheval/intent_test.go | 2 +- pkg/kv/kvserver/loqrecovery/apply.go | 2 +- pkg/kv/kvserver/replica_raftstorage.go | 2 +- pkg/roachpb/api.go | 3 +- pkg/roachpb/api.proto | 4 +- pkg/sql/delete_range.go | 6 +- pkg/storage/bench_pebble_test.go | 2 +- pkg/storage/engine_test.go | 2 +- pkg/storage/metamorphic/operations.go | 2 +- pkg/storage/mvcc.go | 19 ++---- pkg/storage/mvcc_history_test.go | 15 ++--- pkg/storage/mvcc_incremental_iterator_test.go | 8 +-- pkg/storage/mvcc_stats_test.go | 20 +++---- pkg/storage/mvcc_test.go | 26 ++++---- .../testdata/mvcc_histories/clear_time_range | 20 +++++++ .../mvcc_histories/clear_time_range_limits | 30 ++++++++++ .../mvcc_histories/delete_range_predicate | 2 + pkg/storage/testdata/mvcc_histories/deletes | 9 +-- pkg/storage/testdata/mvcc_histories/export | 3 + .../intent_history_enable_separated | 1 + .../testdata/mvcc_histories/local_timestamp | 3 + pkg/storage/testdata/mvcc_histories/max_keys | 1 + .../mvcc_histories/put_after_rollback | 1 + .../testdata/mvcc_histories/range_key_clear | 3 + .../testdata/mvcc_histories/range_key_iter | 3 + .../mvcc_histories/range_key_iter_incremental | 3 + .../mvcc_histories/range_key_iter_read_as_of | 3 + .../mvcc_histories/range_key_point_synthesis | 2 + .../mvcc_histories/range_tombstone_conflicts | 1 + .../mvcc_histories/range_tombstone_gets | 2 + .../mvcc_histories/range_tombstone_scans | 2 + .../mvcc_histories/range_tombstone_stats | 9 +++ ...range_tombstone_stats_intent_resolve_abort | 15 +++++ ...ange_tombstone_stats_intent_resolve_commit | 15 +++++ ...ange_tombstone_stats_intent_resolve_pushed | 15 +++++ ...tombstone_stats_intent_resolve_range_abort | 15 +++++ ...ombstone_stats_intent_resolve_range_commit | 15 +++++ ...ombstone_stats_intent_resolve_range_pushed | 15 +++++ .../range_tombstone_stats_intent_rewrite | 30 ++++++++++ .../mvcc_histories/range_tombstone_writes | 2 + .../testdata/mvcc_histories/target_bytes | 1 + 53 files changed, 326 insertions(+), 146 deletions(-) diff --git a/pkg/cli/debug.go b/pkg/cli/debug.go index 3ce3db80f7f2..47cf06ef177b 100644 --- a/pkg/cli/debug.go +++ b/pkg/cli/debug.go @@ -1405,7 +1405,7 @@ func removeDeadReplicas( // A crude form of the intent resolution process: abort the // transaction by deleting its record. txnKey := keys.TransactionKey(intent.Txn.Key, intent.Txn.ID) - if err := storage.MVCCDelete(ctx, batch, &ms, txnKey, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil); err != nil { + if _, err := storage.MVCCDelete(ctx, batch, &ms, txnKey, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil); err != nil { return nil, err } update := roachpb.LockUpdate{ diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index 8cbd11d28cc2..3e0d14544f53 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -340,8 +340,8 @@ const ( // SQLSchemaTelemetryScheduledJobs adds an automatic schedule for SQL schema // telemetry logging jobs. SQLSchemaTelemetryScheduledJobs - // DeleteRequestReturnKey is the version where the ReturnKey argument of the - // DeleteRequest is introduced. + // DeleteRequestReturnKey is the version where the DeleteRequest began + // populating the FoundKey value in the response. DeleteRequestReturnKey // ************************************************* diff --git a/pkg/kv/batch.go b/pkg/kv/batch.go index eedc072e2798..96a5bca3c19a 100644 --- a/pkg/kv/batch.go +++ b/pkg/kv/batch.go @@ -242,12 +242,9 @@ func (b *Batch) fillResults(ctx context.Context) { } case *roachpb.DeleteRequest: if result.Err == nil { - deleteRequest := args.(*roachpb.DeleteRequest) - if deleteRequest.ReturnKey { - resp := reply.(*roachpb.DeleteResponse) - if resp.FoundKey { - result.Keys = []roachpb.Key{deleteRequest.Key} - } + resp := reply.(*roachpb.DeleteResponse) + if resp.FoundKey { + result.Keys = []roachpb.Key{args.(*roachpb.DeleteRequest).Key} } } case *roachpb.DeleteRangeRequest: @@ -628,7 +625,7 @@ func (b *Batch) Del(keys ...interface{}) { b.initResult(0, len(keys), notRaw, err) return } - reqs = append(reqs, roachpb.NewDelete(k, false /* returnKey */)) + reqs = append(reqs, roachpb.NewDelete(k)) b.approxMutationReqBytes += len(k) } b.appendReqs(reqs...) @@ -638,19 +635,18 @@ func (b *Batch) Del(keys ...interface{}) { // DelKey deletes one key. // // A new result will be appended to the batch which will contain 0 rows and -// Result.Err will indicate success or failure. The returnKey argument -// determines whether the key is included in Result.Keys to indicate that the -// delete actually deleted something. +// Result.Err will indicate success or failure. The key will be included in +// Result.Keys if it was actually deleted. // // key can be either a byte slice or a string. -func (b *Batch) DelKey(key interface{}, returnKey bool) { +func (b *Batch) DelKey(key interface{}) { k, err := marshalKey(key) if err != nil { - b.initResult(0, 1, notRaw, err) + b.initResult(0, 0, notRaw, err) return } b.approxMutationReqBytes += len(k) - b.appendReqs(roachpb.NewDelete(k, returnKey)) + b.appendReqs(roachpb.NewDelete(k)) b.initResult(1, 0, notRaw, nil) } diff --git a/pkg/kv/db.go b/pkg/kv/db.go index b9469c1c2bce..330879943256 100644 --- a/pkg/kv/db.go +++ b/pkg/kv/db.go @@ -535,19 +535,18 @@ func (db *DB) Del(ctx context.Context, keys ...interface{}) error { // DelKey deletes one key. // -// The returned roachpb.Key will contain the key deleted if the returnKey -// parameter is true and the key was actually deleted, or will be nil if the -// parameter is false of the key was already non-existent. +// The returned roachpb.Key will contain the key if it was actually deleted. // // key can be either a byte slice or a string. -func (db *DB) DelKey(ctx context.Context, key interface{}, returnKey bool) (roachpb.Key, error) { +func (db *DB) DelKey(ctx context.Context, key interface{}) (roachpb.Key, error) { b := &Batch{} - b.DelKey(key, returnKey) + b.DelKey(key) r, err := getOneResult(db.Run(ctx, b), b) - if returnKey { - return r.Keys[0], err + var deletedKey roachpb.Key + if len(r.Keys) > 0 { + deletedKey = r.Keys[0] } - return roachpb.Key{}, err + return deletedKey, err } // DelRange deletes the rows between begin (inclusive) and end (exclusive). diff --git a/pkg/kv/db_test.go b/pkg/kv/db_test.go index 904fbcd4d582..db12732626c0 100644 --- a/pkg/kv/db_test.go +++ b/pkg/kv/db_test.go @@ -480,47 +480,37 @@ func TestDB_Del(t *testing.T) { } for _, useDelKey := range []bool{false, true} { - for _, returnKey := range []bool{false, true} { - if !useDelKey && returnKey { - // Del doesn't have a returnKey argument. - continue - } - if err := db.Put(context.Background(), "ab", "2"); err != nil { + if err := db.Put(context.Background(), "ab", "2"); err != nil { + t.Fatal(err) + } + if useDelKey { + key, err := db.DelKey(context.Background(), "ab") + if err != nil { t.Fatal(err) } - if useDelKey { - if key, err := db.DelKey(context.Background(), "ab", returnKey); err != nil { - t.Fatal(err) - } else if returnKey { - checkKeys(t, []string{"ab"}, []roachpb.Key{key}) - // Also try deleting a non-existent key and verify that no - // key is returned. - if key, err = db.DelKey(context.Background(), "ad", returnKey); err != nil { - t.Fatal(err) - } else if len(key) > 0 { - t.Errorf("expected deleted key to be empty when deleting a non-existent key, got %v", key) - } - } else { - if len(key) > 0 { - t.Errorf("expected deleted key to be empty when returnKeys set to false, got %v", key) - } - } - } else { - if err := db.Del(context.Background(), "ab"); err != nil { - t.Fatal(err) - } - } - rows, err := db.Scan(context.Background(), "a", "b", 100) - if err != nil { + checkKeys(t, []string{"ab"}, []roachpb.Key{key}) + // Also try deleting a non-existent key and verify that no key is + // returned. + if key, err = db.DelKey(context.Background(), "ad"); err != nil { t.Fatal(err) + } else if len(key) > 0 { + t.Errorf("expected deleted key to be empty when deleting a non-existent key, got %v", key) } - expected := map[string][]byte{ - "aa": []byte("1"), - "ac": []byte("3"), + } else { + if err := db.Del(context.Background(), "ab"); err != nil { + t.Fatal(err) } - checkRows(t, expected, rows) - checkLen(t, len(expected), len(rows)) } + rows, err := db.Scan(context.Background(), "a", "b", 100) + if err != nil { + t.Fatal(err) + } + expected := map[string][]byte{ + "aa": []byte("1"), + "ac": []byte("3"), + } + checkRows(t, expected, rows) + checkLen(t, len(expected), len(rows)) } } diff --git a/pkg/kv/kvnemesis/doc.go b/pkg/kv/kvnemesis/doc.go index d8905dfdfe55..91ad9d3dcb6a 100644 --- a/pkg/kv/kvnemesis/doc.go +++ b/pkg/kv/kvnemesis/doc.go @@ -24,7 +24,7 @@ // // TODO // - CPut/InitPut/Increment -// - Delete (with and without ReturnKey) +// - DelKey // - ClearRange/RevertRange // - AdminRelocateRange // - AdminUnsplit diff --git a/pkg/kv/kvserver/abortspan/abortspan.go b/pkg/kv/kvserver/abortspan/abortspan.go index d6fd58c24c3e..e93e6043c1d1 100644 --- a/pkg/kv/kvserver/abortspan/abortspan.go +++ b/pkg/kv/kvserver/abortspan/abortspan.go @@ -120,7 +120,8 @@ func (sc *AbortSpan) Del( ctx context.Context, reader storage.ReadWriter, ms *enginepb.MVCCStats, txnID uuid.UUID, ) error { key := keys.AbortSpanKey(sc.rangeID, txnID) - return storage.MVCCDelete(ctx, reader, ms, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil /* txn */) + _, err := storage.MVCCDelete(ctx, reader, ms, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil /* txn */) + return err } // Put writes an entry for the specified transaction ID. diff --git a/pkg/kv/kvserver/batcheval/cmd_delete.go b/pkg/kv/kvserver/batcheval/cmd_delete.go index eed94d95863c..ffd0b29e6b47 100644 --- a/pkg/kv/kvserver/batcheval/cmd_delete.go +++ b/pkg/kv/kvserver/batcheval/cmd_delete.go @@ -30,14 +30,10 @@ func Delete( h := cArgs.Header var err error - if args.ReturnKey { - reply := resp.(*roachpb.DeleteResponse) - reply.FoundKey, err = storage.MVCCDeleteReturningExistence( - ctx, readWriter, cArgs.Stats, args.Key, h.Timestamp, cArgs.Now, h.Txn, - ) - } else { - err = storage.MVCCDelete(ctx, readWriter, cArgs.Stats, args.Key, h.Timestamp, cArgs.Now, h.Txn) - } + reply := resp.(*roachpb.DeleteResponse) + reply.FoundKey, err = storage.MVCCDelete( + ctx, readWriter, cArgs.Stats, args.Key, h.Timestamp, cArgs.Now, h.Txn, + ) // NB: even if MVCC returns an error, it may still have written an intent // into the batch. This allows callers to consume errors like WriteTooOld // without re-evaluating the batch. This behavior isn't particularly diff --git a/pkg/kv/kvserver/batcheval/cmd_delete_range_test.go b/pkg/kv/kvserver/batcheval/cmd_delete_range_test.go index b8cfb058cdde..a91d2c1773a0 100644 --- a/pkg/kv/kvserver/batcheval/cmd_delete_range_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_delete_range_test.go @@ -60,7 +60,8 @@ func TestDeleteRangeTombstone(t *testing.T) { require.NoError(t, storage.MVCCPut(ctx, rw, nil, roachpb.Key("b"), hlc.Timestamp{WallTime: 2e9}, localTS, roachpb.MakeValueFromString("b2"), nil)) require.NoError(t, storage.MVCCPut(ctx, rw, nil, roachpb.Key("c"), hlc.Timestamp{WallTime: 4e9}, localTS, roachpb.MakeValueFromString("c4"), nil)) require.NoError(t, storage.MVCCPut(ctx, rw, nil, roachpb.Key("d"), hlc.Timestamp{WallTime: 2e9}, localTS, roachpb.MakeValueFromString("d2"), nil)) - require.NoError(t, storage.MVCCDelete(ctx, rw, nil, roachpb.Key("d"), hlc.Timestamp{WallTime: 3e9}, localTS, nil)) + _, err := storage.MVCCDelete(ctx, rw, nil, roachpb.Key("d"), hlc.Timestamp{WallTime: 3e9}, localTS, nil) + require.NoError(t, err) require.NoError(t, storage.MVCCPut(ctx, rw, nil, roachpb.Key("i"), hlc.Timestamp{WallTime: 5e9}, localTS, roachpb.MakeValueFromString("i5"), &txn)) require.NoError(t, storage.MVCCDeleteRangeUsingTombstone(ctx, rw, nil, roachpb.Key("f"), roachpb.Key("h"), hlc.Timestamp{WallTime: 3e9}, localTS, nil, nil, 0, nil)) require.NoError(t, storage.MVCCDeleteRangeUsingTombstone(ctx, rw, nil, roachpb.Key("Z"), roachpb.Key("a"), hlc.Timestamp{WallTime: 100e9}, localTS, nil, nil, 0, nil)) diff --git a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go index 39b873d3f9b6..3f34882367a2 100644 --- a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go +++ b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go @@ -629,7 +629,8 @@ func updateFinalizedTxn( // BatchRequest writes. return nil } - return storage.MVCCDelete(ctx, readWriter, ms, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil) + _, err := storage.MVCCDelete(ctx, readWriter, ms, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil) + return err } txn.LockSpans = externalLocks txn.InFlightWrites = nil diff --git a/pkg/kv/kvserver/batcheval/cmd_query_intent_test.go b/pkg/kv/kvserver/batcheval/cmd_query_intent_test.go index 6fd2376d00fd..58b92d7d5dc1 100644 --- a/pkg/kv/kvserver/batcheval/cmd_query_intent_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_query_intent_test.go @@ -40,7 +40,8 @@ func TestQueryIntent(t *testing.T) { writeIntent := func(k roachpb.Key, ts int64) roachpb.Transaction { txn := roachpb.MakeTransaction("test", k, 0, makeTS(ts), 0, 1) - require.NoError(t, storage.MVCCDelete(ctx, db, nil, k, makeTS(ts), hlc.ClockTimestamp{}, &txn)) + _, err := storage.MVCCDelete(ctx, db, nil, k, makeTS(ts), hlc.ClockTimestamp{}, &txn) + require.NoError(t, err) return txn } diff --git a/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp_test.go b/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp_test.go index 9c644d0431a7..655c5b0ab004 100644 --- a/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp_test.go @@ -43,14 +43,17 @@ func TestQueryResolvedTimestamp(t *testing.T) { return hlc.Timestamp{WallTime: ts} } writeValue := func(k string, ts int64) { - require.NoError(t, storage.MVCCDelete(ctx, db, nil, roachpb.Key(k), makeTS(ts), hlc.ClockTimestamp{}, nil)) + _, err := storage.MVCCDelete(ctx, db, nil, roachpb.Key(k), makeTS(ts), hlc.ClockTimestamp{}, nil) + require.NoError(t, err) } writeIntent := func(k string, ts int64) { txn := roachpb.MakeTransaction("test", roachpb.Key(k), 0, makeTS(ts), 0, 1) - require.NoError(t, storage.MVCCDelete(ctx, db, nil, roachpb.Key(k), makeTS(ts), hlc.ClockTimestamp{}, &txn)) + _, err := storage.MVCCDelete(ctx, db, nil, roachpb.Key(k), makeTS(ts), hlc.ClockTimestamp{}, &txn) + require.NoError(t, err) } writeInline := func(k string) { - require.NoError(t, storage.MVCCDelete(ctx, db, nil, roachpb.Key(k), hlc.Timestamp{}, hlc.ClockTimestamp{}, nil)) + _, err := storage.MVCCDelete(ctx, db, nil, roachpb.Key(k), hlc.Timestamp{}, hlc.ClockTimestamp{}, nil) + require.NoError(t, err) } // Setup: (with separated intents the actual key layout in the store is not what is listed below.) diff --git a/pkg/kv/kvserver/batcheval/intent_test.go b/pkg/kv/kvserver/batcheval/intent_test.go index 16d997395b7a..561940e783ef 100644 --- a/pkg/kv/kvserver/batcheval/intent_test.go +++ b/pkg/kv/kvserver/batcheval/intent_test.go @@ -129,7 +129,7 @@ func TestCollectIntentsUsesSameIterator(t *testing.T) { txn := roachpb.MakeTransaction("test", key, roachpb.NormalUserPriority, ts, 0, 1) var err error if delete { - err = storage.MVCCDelete(ctx, db, nil, key, ts, hlc.ClockTimestamp{}, &txn) + _, err = storage.MVCCDelete(ctx, db, nil, key, ts, hlc.ClockTimestamp{}, &txn) } else { err = storage.MVCCPut(ctx, db, nil, key, ts, hlc.ClockTimestamp{}, val, &txn) } diff --git a/pkg/kv/kvserver/loqrecovery/apply.go b/pkg/kv/kvserver/loqrecovery/apply.go index e7ea0c259013..5e9866fc2ac2 100644 --- a/pkg/kv/kvserver/loqrecovery/apply.go +++ b/pkg/kv/kvserver/loqrecovery/apply.go @@ -257,7 +257,7 @@ func applyReplicaUpdate( // A crude form of the intent resolution process: abort the // transaction by deleting its record. txnKey := keys.TransactionKey(intent.Txn.Key, intent.Txn.ID) - if err := storage.MVCCDelete(ctx, readWriter, &ms, txnKey, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil); err != nil { + if _, err := storage.MVCCDelete(ctx, readWriter, &ms, txnKey, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil); err != nil { return PrepareReplicaReport{}, err } update := roachpb.LockUpdate{ diff --git a/pkg/kv/kvserver/replica_raftstorage.go b/pkg/kv/kvserver/replica_raftstorage.go index 5ace88204422..d12e5011af36 100644 --- a/pkg/kv/kvserver/replica_raftstorage.go +++ b/pkg/kv/kvserver/replica_raftstorage.go @@ -700,7 +700,7 @@ func (r *Replica) append( for i := lastIndex + 1; i <= prevLastIndex; i++ { // Note that the caller is in charge of deleting any sideloaded payloads // (which they must only do *after* the batch has committed). - err := storage.MVCCDelete(ctx, eng, &diff, keys.RaftLogKeyFromPrefix(prefix, i), + _, err := storage.MVCCDelete(ctx, eng, &diff, keys.RaftLogKeyFromPrefix(prefix, i), hlc.Timestamp{}, hlc.ClockTimestamp{}, nil) if err != nil { return 0, 0, 0, err diff --git a/pkg/roachpb/api.go b/pkg/roachpb/api.go index d782236c74d7..8bc9e4a47b69 100644 --- a/pkg/roachpb/api.go +++ b/pkg/roachpb/api.go @@ -1194,12 +1194,11 @@ func NewInitPut(key Key, value Value, failOnTombstones bool) Request { } // NewDelete returns a Request initialized to delete the value at key. -func NewDelete(key Key, returnKey bool) Request { +func NewDelete(key Key) Request { return &DeleteRequest{ RequestHeader: RequestHeader{ Key: key, }, - ReturnKey: returnKey, } } diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index 990de4fd3bf7..f65633c1cc8b 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -319,14 +319,12 @@ message IncrementResponse { // A DeleteRequest is the argument to the Delete() method. message DeleteRequest { RequestHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; - // return the key that is deleted in the response. - bool return_key = 2; } // A DeleteResponse is the return value from the Delete() method. message DeleteResponse { ResponseHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; - // True if return_key was set and there was a key that got deleted. + // True if there was a key that got deleted. bool found_key = 2; } diff --git a/pkg/sql/delete_range.go b/pkg/sql/delete_range.go index 00a88742defa..370c417bdcc6 100644 --- a/pkg/sql/delete_range.go +++ b/pkg/sql/delete_range.go @@ -90,8 +90,8 @@ func (d *deleteRangeNode) startExec(params runParams) error { } // Configure the fetcher, which is only used to decode the returned keys - // from the DeleteKey and the DeleteRange, and is never used to actually - // fetch kvs. + // from the DelKey and the DelRange operations, and is never used to + // actually fetch kvs. var spec descpb.IndexFetchSpec if err := rowenc.InitIndexFetchSpec( &spec, params.ExecCfg().Codec, d.desc, d.desc.GetPrimaryIndex(), nil, /* columnIDs */ @@ -172,7 +172,7 @@ func (d *deleteRangeNode) deleteSpans(params runParams, b *kv.Batch, spans roach if traceKV { log.VEventf(ctx, 2, "Del %s", span.Key) } - b.DelKey(span.Key, true /* returnKey */) + b.DelKey(span.Key) } else { if traceKV { log.VEventf(ctx, 2, "DelRange %s - %s", span.Key, span.EndKey) diff --git a/pkg/storage/bench_pebble_test.go b/pkg/storage/bench_pebble_test.go index a06a9962a167..f1b124d12105 100644 --- a/pkg/storage/bench_pebble_test.go +++ b/pkg/storage/bench_pebble_test.go @@ -326,7 +326,7 @@ func BenchmarkMVCCPutDelete_Pebble(b *testing.B) { if err := MVCCPut(ctx, db, nil, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, value, nil); err != nil { b.Fatal(err) } - if err := MVCCDelete(ctx, db, nil, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil); err != nil { + if _, err := MVCCDelete(ctx, db, nil, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil); err != nil { b.Fatal(err) } } diff --git a/pkg/storage/engine_test.go b/pkg/storage/engine_test.go index 462a3403a253..f6889010e8d3 100644 --- a/pkg/storage/engine_test.go +++ b/pkg/storage/engine_test.go @@ -187,7 +187,7 @@ func TestEngineBatchStaleCachedIterator(t *testing.T) { // Seek the iterator to `key` and clear the value (but without // telling the iterator about that). - if err := MVCCDelete(context.Background(), batch, nil, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil); err != nil { + if _, err := MVCCDelete(context.Background(), batch, nil, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil); err != nil { t.Fatal(err) } diff --git a/pkg/storage/metamorphic/operations.go b/pkg/storage/metamorphic/operations.go index c732de4c26b5..3125897f37d4 100644 --- a/pkg/storage/metamorphic/operations.go +++ b/pkg/storage/metamorphic/operations.go @@ -374,7 +374,7 @@ func (m mvccDeleteOp) run(ctx context.Context) string { writer := m.m.getReadWriter(m.writer) txn.Sequence++ - err := storage.MVCCDelete(ctx, writer, nil, m.key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn) + _, err := storage.MVCCDelete(ctx, writer, nil, m.key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn) if err != nil { if writeTooOldErr := (*roachpb.WriteTooOldError)(nil); errors.As(err, &writeTooOldErr) { txn.WriteTimestamp.Forward(writeTooOldErr.ActualTimestamp) diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index dca9db814a2d..f03433852819 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -1325,6 +1325,9 @@ func MVCCBlindPut( // Note that, when writing transactionally, the txn's timestamps // dictate the timestamp of the operation, and the timestamp parameter is // confusing and redundant. See the comment on mvccPutInternal for details. +// +// foundKey indicates whether the key that was passed in had a value already in +// the database. func MVCCDelete( ctx context.Context, rw ReadWriter, @@ -1333,21 +1336,6 @@ func MVCCDelete( timestamp hlc.Timestamp, localTimestamp hlc.ClockTimestamp, txn *roachpb.Transaction, -) error { - _, err := MVCCDeleteReturningExistence(ctx, rw, ms, key, timestamp, localTimestamp, txn) - return err -} - -// MVCCDeleteReturningExistence is like MVCCDelete, but it returns whether the -// key that was passed in had a value already in the database. -func MVCCDeleteReturningExistence( - ctx context.Context, - rw ReadWriter, - ms *enginepb.MVCCStats, - key roachpb.Key, - timestamp hlc.Timestamp, - localTimestamp hlc.ClockTimestamp, - txn *roachpb.Transaction, ) (foundKey bool, err error) { iter := newMVCCIterator(rw, timestamp, false /* rangeKeyMasking */, IterOptions{ KeyTypes: IterKeyTypePointsAndRanges, @@ -1355,6 +1343,7 @@ func MVCCDeleteReturningExistence( }) defer iter.Close() + // TODO(yuzefovich): can we avoid the actual put if foundKey is false? valueFn := func(value optionalValue) (roachpb.Value, error) { foundKey = len(value.RawBytes) > 0 return noValue, nil diff --git a/pkg/storage/mvcc_history_test.go b/pkg/storage/mvcc_history_test.go index 66e8b69708f2..65a30096c2e8 100644 --- a/pkg/storage/mvcc_history_test.go +++ b/pkg/storage/mvcc_history_test.go @@ -1000,20 +1000,13 @@ func cmdDelete(e *evalCtx) error { key := e.getKey() ts := e.getTs(txn) localTs := hlc.ClockTimestamp(e.getTsWithName("localTs")) - returnKey := e.hasArg("returnKey") resolve, resolveStatus := e.getResolve() return e.withWriter("del", func(rw ReadWriter) error { - if returnKey { - key, err := MVCCDeleteReturningExistence(e.ctx, rw, e.ms, key, ts, localTs, txn) - if err != nil { - return err - } - e.results.buf.Printf("del: found key %v\n", key) - } else { - if err := MVCCDelete(e.ctx, rw, e.ms, key, ts, localTs, txn); err != nil { - return err - } + deletedKey, err := MVCCDelete(e.ctx, rw, e.ms, key, ts, localTs, txn) + if err != nil { + return err } + e.results.buf.Printf("del: %v: found key %v\n", key, deletedKey) if resolve { return e.resolveIntent(rw, key, txn, resolveStatus, hlc.ClockTimestamp{}) } diff --git a/pkg/storage/mvcc_incremental_iterator_test.go b/pkg/storage/mvcc_incremental_iterator_test.go index 1c38e433bcc8..bb205fcc4063 100644 --- a/pkg/storage/mvcc_incremental_iterator_test.go +++ b/pkg/storage/mvcc_incremental_iterator_test.go @@ -491,7 +491,7 @@ func TestMVCCIncrementalIteratorNextIgnoringTime(t *testing.T) { }) // Exercise deletion. - if err := MVCCDelete(ctx, e, nil, testKey1, ts3, hlc.ClockTimestamp{}, nil); err != nil { + if _, err := MVCCDelete(ctx, e, nil, testKey1, ts3, hlc.ClockTimestamp{}, nil); err != nil { t.Fatal(err) } // Returns the kv_1_1_1 even though it is outside (startTime, endTime]. @@ -628,7 +628,7 @@ func TestMVCCIncrementalIteratorNextKeyIgnoringTime(t *testing.T) { }) // Exercise deletion. - if err := MVCCDelete(ctx, e, nil, testKey1, ts3, hlc.ClockTimestamp{}, nil); err != nil { + if _, err := MVCCDelete(ctx, e, nil, testKey1, ts3, hlc.ClockTimestamp{}, nil); err != nil { t.Fatal(err) } // Returns the kv_1_1_1 even though it is outside (startTime, endTime]. @@ -975,7 +975,7 @@ func TestMVCCIncrementalIterator(t *testing.T) { t.Run("kv [1-2)", assertEqualKVs(e, testKey1, testKey2, tsMin, tsMax, latest, kvs(kv1_2_2))) // Exercise deletion. - if err := MVCCDelete(ctx, e, nil, testKey1, ts3, hlc.ClockTimestamp{}, nil); err != nil { + if _, err := MVCCDelete(ctx, e, nil, testKey1, ts3, hlc.ClockTimestamp{}, nil); err != nil { t.Fatal(err) } t.Run("del", assertEqualKVs(e, localMax, keyMax, ts1, tsMax, latest, kvs(kv1Deleted3, kv2_2_2))) @@ -1043,7 +1043,7 @@ func TestMVCCIncrementalIterator(t *testing.T) { t.Run("kv [1-2)", assertEqualKVs(e, testKey1, testKey2, tsMin, tsMax, all, kvs(kv1_2_2, kv1_1_1))) // Exercise deletion. - if err := MVCCDelete(ctx, e, nil, testKey1, ts3, hlc.ClockTimestamp{}, nil); err != nil { + if _, err := MVCCDelete(ctx, e, nil, testKey1, ts3, hlc.ClockTimestamp{}, nil); err != nil { t.Fatal(err) } t.Run("del", assertEqualKVs(e, localMax, keyMax, ts1, tsMax, all, kvs(kv1Deleted3, kv1_2_2, kv2_2_2))) diff --git a/pkg/storage/mvcc_stats_test.go b/pkg/storage/mvcc_stats_test.go index 8a5d5fdbc851..7cf57b476246 100644 --- a/pkg/storage/mvcc_stats_test.go +++ b/pkg/storage/mvcc_stats_test.go @@ -135,7 +135,7 @@ func TestMVCCStatsDeleteCommitMovesTimestamp(t *testing.T) { TxnMeta: enginepb.TxnMeta{ID: uuid.MakeV4(), WriteTimestamp: ts3}, ReadTimestamp: ts3, } - if err := MVCCDelete(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { + if _, err := MVCCDelete(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { t.Fatal(err) } @@ -467,7 +467,7 @@ func TestMVCCStatsDeleteMovesTimestamp(t *testing.T) { } require.EqualValues(t, m2ValSize, expM2ValSize) - if err := MVCCDelete(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { + if _, err := MVCCDelete(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { t.Fatal(err) } @@ -525,7 +525,7 @@ func TestMVCCStatsPutMovesDeletionTimestamp(t *testing.T) { } // Write a deletion tombstone intent. - if err := MVCCDelete(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { + if _, err := MVCCDelete(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { t.Fatal(err) } @@ -650,7 +650,7 @@ func TestMVCCStatsDelDelCommitMovesTimestamp(t *testing.T) { ts3 := hlc.Timestamp{WallTime: 3e9} // Write a non-transactional tombstone at t=1s. - if err := MVCCDelete(ctx, engine, aggMS, key, ts1, hlc.ClockTimestamp{}, nil); err != nil { + if _, err := MVCCDelete(ctx, engine, aggMS, key, ts1, hlc.ClockTimestamp{}, nil); err != nil { t.Fatal(err) } @@ -679,7 +679,7 @@ func TestMVCCStatsDelDelCommitMovesTimestamp(t *testing.T) { TxnMeta: enginepb.TxnMeta{ID: uuid.MakeV4(), WriteTimestamp: ts2}, ReadTimestamp: ts2, } - if err := MVCCDelete(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { + if _, err := MVCCDelete(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { t.Fatal(err) } @@ -839,7 +839,7 @@ func TestMVCCStatsPutDelPutMovesTimestamp(t *testing.T) { TxnMeta: enginepb.TxnMeta{ID: uuid.MakeV4(), WriteTimestamp: ts2}, ReadTimestamp: ts2, } - if err := MVCCDelete(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { + if _, err := MVCCDelete(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { t.Fatal(err) } @@ -993,10 +993,10 @@ func TestMVCCStatsDelDelGC(t *testing.T) { ts2 := hlc.Timestamp{WallTime: 2e9} // Write tombstones at ts1 and ts2. - if err := MVCCDelete(ctx, engine, aggMS, key, ts1, hlc.ClockTimestamp{}, nil); err != nil { + if _, err := MVCCDelete(ctx, engine, aggMS, key, ts1, hlc.ClockTimestamp{}, nil); err != nil { t.Fatal(err) } - if err := MVCCDelete(ctx, engine, aggMS, key, ts2, hlc.ClockTimestamp{}, nil); err != nil { + if _, err := MVCCDelete(ctx, engine, aggMS, key, ts2, hlc.ClockTimestamp{}, nil); err != nil { t.Fatal(err) } @@ -1207,7 +1207,7 @@ func TestMVCCStatsPutWaitDeleteGC(t *testing.T) { // Delete the value at ts5. - if err := MVCCDelete(ctx, engine, aggMS, key, ts2, hlc.ClockTimestamp{}, nil); err != nil { + if _, err := MVCCDelete(ctx, engine, aggMS, key, ts2, hlc.ClockTimestamp{}, nil); err != nil { t.Fatal(err) } @@ -1659,7 +1659,7 @@ func TestMVCCStatsRandomized(t *testing.T) { return desc } actions["Del"] = func(s *state) string { - if err := MVCCDelete(ctx, s.eng, s.MS, s.key, s.TS, hlc.ClockTimestamp{}, s.Txn); err != nil { + if _, err := MVCCDelete(ctx, s.eng, s.MS, s.key, s.TS, hlc.ClockTimestamp{}, s.Txn); err != nil { return err.Error() } return "" diff --git a/pkg/storage/mvcc_test.go b/pkg/storage/mvcc_test.go index 1ec70356e633..b7b433eae457 100644 --- a/pkg/storage/mvcc_test.go +++ b/pkg/storage/mvcc_test.go @@ -323,7 +323,7 @@ func TestMVCCGetAndDelete(t *testing.T) { t.Fatal("the value should not be empty") } - err = MVCCDelete(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, nil) + _, err = MVCCDelete(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, nil) if err != nil { t.Fatal(err) } @@ -372,7 +372,7 @@ func TestMVCCWriteWithOlderTimestampAfterDeletionOfNonexistentKey(t *testing.T) engine := engineImpl.create() defer engine.Close() - if err := MVCCDelete(context.Background(), engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, nil); err != nil { + if _, err := MVCCDelete(context.Background(), engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, nil); err != nil { t.Fatal(err) } @@ -463,7 +463,7 @@ func TestMVCCDeleteMissingKey(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCDelete(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, nil); err != nil { + if _, err := MVCCDelete(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, nil); err != nil { t.Fatal(err) } // Verify nothing is written to the engine. @@ -501,7 +501,7 @@ func TestMVCCGetAndDeleteInTxn(t *testing.T) { txn.Sequence++ txn.WriteTimestamp = hlc.Timestamp{WallTime: 3} - if err := MVCCDelete(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { + if _, err := MVCCDelete(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { t.Fatal(err) } @@ -1742,7 +1742,7 @@ func TestMVCCUncommittedDeleteRangeVisible(t *testing.T) { t.Fatal(err) } - if err := MVCCDelete(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 2, Logical: 1}, hlc.ClockTimestamp{}, nil); err != nil { + if _, err := MVCCDelete(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 2, Logical: 1}, hlc.ClockTimestamp{}, nil); err != nil { t.Fatal(err) } @@ -1782,7 +1782,7 @@ func TestMVCCDeleteRangeOldTimestamp(t *testing.T) { if err != nil { t.Fatal(err) } - err = MVCCDelete(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 5}, hlc.ClockTimestamp{}, nil) + _, err = MVCCDelete(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 5}, hlc.ClockTimestamp{}, nil) if err != nil { t.Fatal(err) } @@ -2245,7 +2245,8 @@ func TestMVCCClearTimeRangeOnRandomData(t *testing.T) { key := roachpb.Key(fmt.Sprintf("%05d", k)) if rand.Float64() > 0.8 { - require.NoError(t, MVCCDelete(ctx, e, &ms, key, hlc.Timestamp{WallTime: ts}, hlc.ClockTimestamp{}, nil)) + _, err := MVCCDelete(ctx, e, &ms, key, hlc.Timestamp{WallTime: ts}, hlc.ClockTimestamp{}, nil) + require.NoError(t, err) } else { v := roachpb.MakeValueFromString(fmt.Sprintf("v-%d", i)) require.NoError(t, MVCCPut(ctx, e, &ms, key, hlc.Timestamp{WallTime: ts}, hlc.ClockTimestamp{}, v, nil)) @@ -2337,7 +2338,7 @@ func TestMVCCInitPut(t *testing.T) { } // Delete. - err = MVCCDelete(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 3}, hlc.ClockTimestamp{}, nil) + _, err = MVCCDelete(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 3}, hlc.ClockTimestamp{}, nil) if err != nil { t.Fatal(err) } @@ -4794,7 +4795,7 @@ func TestMVCCGarbageCollect(t *testing.T) { } for _, val := range test.vals[i : i+1] { if i == len(test.vals)-1 && test.isDeleted { - if err := MVCCDelete(ctx, engine, ms, test.key, val.Timestamp, hlc.ClockTimestamp{}, + if _, err := MVCCDelete(ctx, engine, ms, test.key, val.Timestamp, hlc.ClockTimestamp{}, nil); err != nil { t.Fatal(err) } @@ -4987,7 +4988,7 @@ func TestMVCCGarbageCollectIntent(t *testing.T) { TxnMeta: enginepb.TxnMeta{ID: uuid.MakeV4(), WriteTimestamp: ts2}, ReadTimestamp: ts2, } - if err := MVCCDelete(ctx, engine, nil, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { + if _, err := MVCCDelete(ctx, engine, nil, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { t.Fatal(err) } keys := []roachpb.GCRequest_GCKey{ @@ -5244,9 +5245,8 @@ func (d rangeTestData) populateEngine( hlc.ClockTimestamp{}, roachpb.MakeValueFromBytes(v.point.Value), v.txn), "failed to insert test value into engine (%s)", v.point.Key.String()) } else { - require.NoError(t, MVCCDelete(ctx, engine, ms, v.point.Key.Key, v.point.Key.Timestamp, - hlc.ClockTimestamp{}, v.txn), - "failed to insert tombstone value into engine (%s)", v.point.Key.String()) + _, err := MVCCDelete(ctx, engine, ms, v.point.Key.Key, v.point.Key.Timestamp, hlc.ClockTimestamp{}, v.txn) + require.NoError(t, err, "failed to insert tombstone value into engine (%s)", v.point.Key.String()) } ts = v.point.Key.Timestamp } else { diff --git a/pkg/storage/testdata/mvcc_histories/clear_time_range b/pkg/storage/testdata/mvcc_histories/clear_time_range index 4817b5135dd9..afdd3a4a586a 100644 --- a/pkg/storage/testdata/mvcc_histories/clear_time_range +++ b/pkg/storage/testdata/mvcc_histories/clear_time_range @@ -34,6 +34,11 @@ del k=i ts=4 put k=k ts=5 v=k5 put_rangekey k=m end=n ts=3 localTs=2 ---- +del: "a": found key true +del: "b": found key false +del: "f": found key true +del: "g": found key false +del: "i": found key true >> at end: rangekey: {a-b}/[1.000000000,0=/] rangekey: {b-c}/[3.000000000,0=/ 1.000000000,0=/] @@ -89,6 +94,11 @@ del k=i ts=4 put k=k ts=5 v=k5 put_rangekey k=m end=n ts=3 localTs=2 ---- +del: "a": found key true +del: "b": found key false +del: "f": found key true +del: "g": found key false +del: "i": found key true >> at end: rangekey: {a-b}/[1.000000000,0=/] rangekey: {b-c}/[3.000000000,0=/ 1.000000000,0=/] @@ -272,6 +282,11 @@ del k=i ts=4 put k=k ts=5 v=k5 put_rangekey k=m end=n ts=3 localTs=2 ---- +del: "a": found key true +del: "b": found key false +del: "f": found key true +del: "g": found key false +del: "i": found key true >> at end: rangekey: {a-b}/[1.000000000,0=/] rangekey: {b-c}/[3.000000000,0=/ 1.000000000,0=/] @@ -835,6 +850,11 @@ del k=i ts=4 put k=k ts=5 v=k5 put_rangekey k=m end=n ts=3 localTs=2 ---- +del: "a": found key true +del: "b": found key false +del: "f": found key true +del: "g": found key false +del: "i": found key true >> at end: rangekey: {a-b}/[1.000000000,0=/] rangekey: {b-c}/[3.000000000,0=/ 1.000000000,0=/] diff --git a/pkg/storage/testdata/mvcc_histories/clear_time_range_limits b/pkg/storage/testdata/mvcc_histories/clear_time_range_limits index d5457997b156..2e2671f5eaa2 100644 --- a/pkg/storage/testdata/mvcc_histories/clear_time_range_limits +++ b/pkg/storage/testdata/mvcc_histories/clear_time_range_limits @@ -43,6 +43,11 @@ del k=i ts=4 put k=k ts=5 v=k5 put_rangekey k=m end=n ts=3 localTs=2 ---- +del: "a": found key true +del: "b": found key false +del: "f": found key true +del: "g": found key false +del: "i": found key true >> at end: rangekey: {a-b}/[1.000000000,0=/] rangekey: {b-c}/[3.000000000,0=/ 1.000000000,0=/] @@ -97,6 +102,11 @@ del k=i ts=4 put k=k ts=5 v=k5 put_rangekey k=m end=n ts=3 localTs=2 ---- +del: "a": found key true +del: "b": found key false +del: "f": found key true +del: "g": found key false +del: "i": found key true >> at end: rangekey: {a-b}/[1.000000000,0=/] rangekey: {b-c}/[3.000000000,0=/ 1.000000000,0=/] @@ -151,6 +161,11 @@ del k=i ts=4 put k=k ts=5 v=k5 put_rangekey k=m end=n ts=3 localTs=2 ---- +del: "a": found key true +del: "b": found key false +del: "f": found key true +del: "g": found key false +del: "i": found key true >> at end: rangekey: {a-b}/[1.000000000,0=/] rangekey: {b-c}/[3.000000000,0=/ 1.000000000,0=/] @@ -206,6 +221,11 @@ del k=i ts=4 put k=k ts=5 v=k5 put_rangekey k=m end=n ts=3 localTs=2 ---- +del: "a": found key true +del: "b": found key false +del: "f": found key true +del: "g": found key false +del: "i": found key true >> at end: rangekey: {a-b}/[1.000000000,0=/] rangekey: {b-c}/[3.000000000,0=/ 1.000000000,0=/] @@ -339,6 +359,11 @@ del k=i ts=4 put k=k ts=5 v=k5 put_rangekey k=m end=n ts=3 localTs=2 ---- +del: "a": found key true +del: "b": found key false +del: "f": found key true +del: "g": found key false +del: "i": found key true >> at end: rangekey: {a-b}/[1.000000000,0=/] rangekey: {b-c}/[3.000000000,0=/ 1.000000000,0=/] @@ -542,6 +567,11 @@ del k=i ts=4 put k=k ts=5 v=k5 put_rangekey k=m end=n ts=3 localTs=2 ---- +del: "a": found key true +del: "b": found key false +del: "f": found key true +del: "g": found key false +del: "i": found key true >> at end: rangekey: {a-b}/[1.000000000,0=/] rangekey: {b-c}/[3.000000000,0=/ 1.000000000,0=/] diff --git a/pkg/storage/testdata/mvcc_histories/delete_range_predicate b/pkg/storage/testdata/mvcc_histories/delete_range_predicate index b3c8e31b4b0d..043f1368e22c 100644 --- a/pkg/storage/testdata/mvcc_histories/delete_range_predicate +++ b/pkg/storage/testdata/mvcc_histories/delete_range_predicate @@ -28,6 +28,8 @@ with t=A txn_begin ts=7 put k=i v=i7 ---- +del: "a": found key true +del: "g": found key true >> 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: {k-p}/[4.000000000,0=/] diff --git a/pkg/storage/testdata/mvcc_histories/deletes b/pkg/storage/testdata/mvcc_histories/deletes index 1195fc05c516..f8086fc774b6 100644 --- a/pkg/storage/testdata/mvcc_histories/deletes +++ b/pkg/storage/testdata/mvcc_histories/deletes @@ -8,6 +8,7 @@ with t=A txn_remove ---- >> del k=a resolve t=A +del: "a": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+784 >> at end: data: "a"/44.000000000,0 -> / @@ -50,9 +51,9 @@ data: "a"/44.000000000,0 -> / # Delete the key and check that the correct key was found. run ok -del k=a ts=47 returnKey +del k=a ts=47 ---- -del: found key true +del: "a": found key true >> at end: data: "a"/47.000000000,0 -> / data: "a"/46.000000000,0 -> /BYTES/abc @@ -61,9 +62,9 @@ data: "a"/44.000000000,0 -> / # Try deleting the same key again and check that no key was found this time. run ok -del k=a ts=48 returnKey +del k=a ts=48 ---- -del: found key false +del: "a": found key false >> at end: data: "a"/48.000000000,0 -> / data: "a"/47.000000000,0 -> / diff --git a/pkg/storage/testdata/mvcc_histories/export b/pkg/storage/testdata/mvcc_histories/export index 13a98cf77157..27df7b08cb16 100644 --- a/pkg/storage/testdata/mvcc_histories/export +++ b/pkg/storage/testdata/mvcc_histories/export @@ -40,6 +40,9 @@ with t=A put k=l v=l7 put k=o v=n7 ---- +del: "a": found key true +del: "b": found key false +del: "h": found key true >> 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=/] diff --git a/pkg/storage/testdata/mvcc_histories/intent_history_enable_separated b/pkg/storage/testdata/mvcc_histories/intent_history_enable_separated index 1e744cbad50d..723b5ce18616 100644 --- a/pkg/storage/testdata/mvcc_histories/intent_history_enable_separated +++ b/pkg/storage/testdata/mvcc_histories/intent_history_enable_separated @@ -47,6 +47,7 @@ stats: val_bytes=+17 live_bytes=+17 intent_bytes=+1 txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=3} lock=true stat=PENDING rts=2.000000000,0 wto=false gul=0,0 >> del k=a t=A called PutIntent("a", _, 00000000-0000-0000-0000-000000000002) +del: "a": found key true meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=3} ts=2.000000000,0 del=true klen=12 vlen=0 ih={{0 /BYTES/first}{1 /BYTES/second}} mergeTs= txnDidNotUpdateMeta=false data: "a"/2.000000000,0 -> / data: "a"/1.000000000,0 -> /BYTES/default diff --git a/pkg/storage/testdata/mvcc_histories/local_timestamp b/pkg/storage/testdata/mvcc_histories/local_timestamp index 928c844f6149..3bf61e0519e6 100644 --- a/pkg/storage/testdata/mvcc_histories/local_timestamp +++ b/pkg/storage/testdata/mvcc_histories/local_timestamp @@ -46,10 +46,13 @@ with ts=40 del localTs=50 k=k3 ---- >> del localTs=30 k=k1 ts=40 +del: "k1": found key true stats: key_bytes=+12 val_count=+1 val_bytes=+13 live_count=-1 live_bytes=-21 gc_bytes_age=+2760 >> del localTs=40 k=k2 ts=40 +del: "k2": found key true stats: key_bytes=+12 val_count=+1 live_count=-1 live_bytes=-21 gc_bytes_age=+1980 >> del localTs=50 k=k3 ts=40 +del: "k3": found key true stats: key_bytes=+12 val_count=+1 live_count=-1 live_bytes=-34 gc_bytes_age=+2760 >> at end: data: "k1"/40.000000000,0 -> {localTs=30.000000000,0}/ diff --git a/pkg/storage/testdata/mvcc_histories/max_keys b/pkg/storage/testdata/mvcc_histories/max_keys index d1ff72a1191c..74544f2e67d6 100644 --- a/pkg/storage/testdata/mvcc_histories/max_keys +++ b/pkg/storage/testdata/mvcc_histories/max_keys @@ -15,6 +15,7 @@ with ts=1,0 put k=/row3/1 v=r3b del k=aa ts=2,0 ---- +del: "aa": found key true >> at end: data: "a"/1.000000000,0 -> /BYTES/val-a data: "aa"/2.000000000,0 -> / diff --git a/pkg/storage/testdata/mvcc_histories/put_after_rollback b/pkg/storage/testdata/mvcc_histories/put_after_rollback index 89e8f2cf8d40..e4c8e6abf74a 100644 --- a/pkg/storage/testdata/mvcc_histories/put_after_rollback +++ b/pkg/storage/testdata/mvcc_histories/put_after_rollback @@ -33,6 +33,7 @@ with t=A k=k3 >> put v=a t=A k=k3 stats: key_count=+1 key_bytes=+15 val_count=+1 val_bytes=+60 live_count=+1 live_bytes=+75 intent_count=+1 intent_bytes=+18 separated_intent_count=+1 intent_age=+99 >> del t=A k=k3 +del: "k3": found key false stats: val_bytes=-8 live_count=-1 live_bytes=-75 gc_bytes_age=+6633 intent_bytes=-6 >> at end: txn: "A" meta={id=00000000 key="k2" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=40} lock=true stat=PENDING rts=1.000000000,0 wto=false gul=0,0 isn=1 diff --git a/pkg/storage/testdata/mvcc_histories/range_key_clear b/pkg/storage/testdata/mvcc_histories/range_key_clear index f90f4fcb9b77..a9bde12c2a11 100644 --- a/pkg/storage/testdata/mvcc_histories/range_key_clear +++ b/pkg/storage/testdata/mvcc_histories/range_key_clear @@ -38,6 +38,9 @@ with t=A put k=j v=j7 put k=l v=l7 ---- +del: "a": found key true +del: "b": found key false +del: "h": found key true >> 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=/] diff --git a/pkg/storage/testdata/mvcc_histories/range_key_iter b/pkg/storage/testdata/mvcc_histories/range_key_iter index 7179e17652de..e6667e353dd2 100644 --- a/pkg/storage/testdata/mvcc_histories/range_key_iter +++ b/pkg/storage/testdata/mvcc_histories/range_key_iter @@ -40,6 +40,9 @@ with t=A put k=m v=l7 put k=o v=n7 ---- +del: "a": found key true +del: "b": found key false +del: "h": found key true >> 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=/] diff --git a/pkg/storage/testdata/mvcc_histories/range_key_iter_incremental b/pkg/storage/testdata/mvcc_histories/range_key_iter_incremental index ae88c7509b7f..27d1030fe8fd 100644 --- a/pkg/storage/testdata/mvcc_histories/range_key_iter_incremental +++ b/pkg/storage/testdata/mvcc_histories/range_key_iter_incremental @@ -43,6 +43,9 @@ with t=B put k=d v=d8 put k=m v=m8 ---- +del: "a": found key true +del: "b": found key false +del: "h": found key true >> at end: txn: "B" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=8.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=8.000000000,0 wto=false gul=0,0 rangekey: {a-b}/[1.000000000,0=/] diff --git a/pkg/storage/testdata/mvcc_histories/range_key_iter_read_as_of b/pkg/storage/testdata/mvcc_histories/range_key_iter_read_as_of index 5b49bc68c726..b669ff8a8980 100644 --- a/pkg/storage/testdata/mvcc_histories/range_key_iter_read_as_of +++ b/pkg/storage/testdata/mvcc_histories/range_key_iter_read_as_of @@ -31,6 +31,9 @@ del k=h ts=4 put k=k ts=5 v=k5 put_rangekey k=m end=n ts=3 localTs=2 ---- +del: "a": found key true +del: "b": found key false +del: "h": found key true >> at end: rangekey: {a-b}/[1.000000000,0=/] rangekey: {b-c}/[3.000000000,0=/ 1.000000000,0=/] diff --git a/pkg/storage/testdata/mvcc_histories/range_key_point_synthesis b/pkg/storage/testdata/mvcc_histories/range_key_point_synthesis index 3f46ca4e91af..737c81014399 100644 --- a/pkg/storage/testdata/mvcc_histories/range_key_point_synthesis +++ b/pkg/storage/testdata/mvcc_histories/range_key_point_synthesis @@ -38,6 +38,8 @@ with t=A put k=d v=d7 put k=j v=j7 ---- +del: "a": found key true +del: "b": found key false >> 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=/] diff --git a/pkg/storage/testdata/mvcc_histories/range_tombstone_conflicts b/pkg/storage/testdata/mvcc_histories/range_tombstone_conflicts index ddc288deea55..45c81c9b909e 100644 --- a/pkg/storage/testdata/mvcc_histories/range_tombstone_conflicts +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_conflicts @@ -28,6 +28,7 @@ stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_b >> put k=e ts=1 v=e1 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=e ts=2 +del: "e": found key true stats: key_bytes=+12 val_count=+1 live_count=-1 live_bytes=-21 gc_bytes_age=+3234 >> put k=f ts=1 v=f1 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 diff --git a/pkg/storage/testdata/mvcc_histories/range_tombstone_gets b/pkg/storage/testdata/mvcc_histories/range_tombstone_gets index d68445ee75f6..367386d52afa 100644 --- a/pkg/storage/testdata/mvcc_histories/range_tombstone_gets +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_gets @@ -28,6 +28,8 @@ with t=A txn_begin k=e ts=6 put k=e v=e6 ---- +del: "b": found key false +del: "a": found key false >> at end: txn: "A" meta={id=00000000 key="e" pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=6.000000000,0 wto=false gul=0,0 rangekey: {a-c}/[2.000000000,0=/] diff --git a/pkg/storage/testdata/mvcc_histories/range_tombstone_scans b/pkg/storage/testdata/mvcc_histories/range_tombstone_scans index 1e7ed3e5ac24..43bdf798c581 100644 --- a/pkg/storage/testdata/mvcc_histories/range_tombstone_scans +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_scans @@ -28,6 +28,8 @@ with t=A txn_begin k=e ts=6 put k=e v=e6 ---- +del: "b": found key false +del: "a": found key false >> at end: txn: "A" meta={id=00000000 key="e" pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=6.000000000,0 wto=false gul=0,0 rangekey: {a-c}/[2.000000000,0=/] diff --git a/pkg/storage/testdata/mvcc_histories/range_tombstone_stats b/pkg/storage/testdata/mvcc_histories/range_tombstone_stats index a7f8b47c3644..a5ca6d5d3f13 100644 --- a/pkg/storage/testdata/mvcc_histories/range_tombstone_stats +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_stats @@ -39,24 +39,30 @@ stats: key_count=+1 key_bytes=+2 val_count=+1 val_bytes=+21 live_count=+1 live_b >> put k=b ts=1 v=b1 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=b ts=2 +del: "b": found key true stats: key_bytes=+12 val_count=+1 live_count=-1 live_bytes=-21 gc_bytes_age=+3234 >> del k=b ts=3 +del: "b": found key false stats: key_bytes=+12 val_count=+1 gc_bytes_age=+1162 >> put k=b ts=6 v=b6 stats: key_bytes=+12 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 gc_bytes_age=-194 >> del k=c ts=1 +del: "c": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 >> put k=c ts=3 v=c3 stats: key_bytes=+12 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 gc_bytes_age=-198 >> del k=c ts=5 +del: "c": found key true stats: key_bytes=+12 val_count=+1 live_count=-1 live_bytes=-21 gc_bytes_age=+3135 >> put k=e ts=1 v=e1 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=f ts=1 +del: "f": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 >> put k=h ts=1 v=h1 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=i ts=1 +del: "i": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 >> del_range_ts k=d end=j ts=3 stats: range_key_count=+1 range_key_bytes=+13 range_val_count=+1 live_count=-2 live_bytes=-42 gc_bytes_age=+5335 @@ -71,10 +77,13 @@ stats: key_bytes=+12 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 gc_b >> put k=f ts=6 v=f6 stats: key_bytes=+12 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 gc_bytes_age=-198 >> del k=g ts=5 +del: "g": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1330 >> del k=h ts=5 +del: "h": found key false stats: key_bytes=+12 val_count=+1 gc_bytes_age=+1136 >> del k=i ts=5 +del: "i": found key false stats: key_bytes=+12 val_count=+1 gc_bytes_age=+1132 >> at end: rangekey: {d-j}/[4.000000000,0={localTs=3.000000000,0}/ 3.000000000,0=/] diff --git a/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_abort b/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_abort index 518ee3a2ee8c..911c915356d0 100644 --- a/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_abort +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_abort @@ -62,18 +62,22 @@ with t=A >> put k=b v=b1 ts=1 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=c ts=1 +del: "c": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 >> put k=e v=e1 ts=1 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=f ts=1 +del: "f": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 >> put k=g v=g1 ts=1 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=h ts=1 +del: "h": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 >> put k=i v=i1 ts=1 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=j ts=1 +del: "j": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 >> del_range_ts k=g end=s ts=3 stats: range_key_count=+1 range_key_bytes=+13 range_val_count=+1 live_count=-2 live_bytes=-42 gc_bytes_age=+5335 @@ -82,10 +86,12 @@ stats: range_key_bytes=+9 range_val_count=+1 gc_bytes_age=+860 >> put k=n v=n5 ts=5 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=o ts=5 +del: "o": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1330 >> put k=q v=q5 ts=5 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=r ts=5 +del: "r": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1330 >> put k=a v=a6 t=A stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 @@ -94,10 +100,13 @@ stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_bytes=+48 gc_bytes_age=+178 >> put k=c v=c6 t=A stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-198 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 >> del k=d t=A +del: "d": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> del k=e t=A +del: "e": found key true stats: key_bytes=+12 val_count=+1 val_bytes=+48 live_count=-1 live_bytes=-21 gc_bytes_age=+7614 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> del k=f t=A +del: "f": found key false stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5630 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> put k=g v=g6 t=A stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-194 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 @@ -106,10 +115,13 @@ stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_ >> put k=i v=i6 t=A stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-194 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 >> del k=j t=A +del: "j": found key false stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5630 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> del k=k t=A +del: "k": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> del k=l t=A +del: "l": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> put k=m v=m6 t=A stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 @@ -118,10 +130,13 @@ stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_bytes=+48 gc_bytes_age=+178 >> put k=o v=o6 t=A stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-190 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 >> del k=p t=A +del: "p": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> del k=q t=A +del: "q": found key true stats: key_bytes=+12 val_count=+1 val_bytes=+48 live_count=-1 live_bytes=-21 gc_bytes_age=+7614 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> del k=r t=A +del: "r": found key false stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5638 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> at end: txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=6.000000000,0 wto=false gul=0,0 diff --git a/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_commit b/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_commit index d1ac8745f512..e6bacdbcded7 100644 --- a/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_commit +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_commit @@ -62,18 +62,22 @@ with t=A >> put k=b v=b1 ts=1 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=c ts=1 +del: "c": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 >> put k=e v=e1 ts=1 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=f ts=1 +del: "f": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 >> put k=g v=g1 ts=1 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=h ts=1 +del: "h": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 >> put k=i v=i1 ts=1 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=j ts=1 +del: "j": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 >> del_range_ts k=g end=s ts=3 stats: range_key_count=+1 range_key_bytes=+13 range_val_count=+1 live_count=-2 live_bytes=-42 gc_bytes_age=+5335 @@ -82,10 +86,12 @@ stats: range_key_bytes=+9 range_val_count=+1 gc_bytes_age=+860 >> put k=n v=n5 ts=5 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=o ts=5 +del: "o": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1330 >> put k=q v=q5 ts=5 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=r ts=5 +del: "r": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1330 >> put k=a v=a6 t=A stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 @@ -94,10 +100,13 @@ stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_bytes=+48 gc_bytes_age=+178 >> put k=c v=c6 t=A stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-198 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 >> del k=d t=A +del: "d": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> del k=e t=A +del: "e": found key true stats: key_bytes=+12 val_count=+1 val_bytes=+48 live_count=-1 live_bytes=-21 gc_bytes_age=+7614 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> del k=f t=A +del: "f": found key false stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5630 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> put k=g v=g6 t=A stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-194 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 @@ -106,10 +115,13 @@ stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_ >> put k=i v=i6 t=A stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-194 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 >> del k=j t=A +del: "j": found key false stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5630 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> del k=k t=A +del: "k": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> del k=l t=A +del: "l": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> put k=m v=m6 t=A stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 @@ -118,10 +130,13 @@ stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_bytes=+48 gc_bytes_age=+178 >> put k=o v=o6 t=A stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-190 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 >> del k=p t=A +del: "p": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> del k=q t=A +del: "q": found key true stats: key_bytes=+12 val_count=+1 val_bytes=+48 live_count=-1 live_bytes=-21 gc_bytes_age=+7614 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> del k=r t=A +del: "r": found key false stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5638 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> at end: txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=6.000000000,0 wto=false gul=0,0 diff --git a/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_pushed b/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_pushed index d927491cf8cb..36b0678e67da 100644 --- a/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_pushed +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_pushed @@ -62,18 +62,22 @@ with t=A >> put k=b v=b1 ts=1 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=c ts=1 +del: "c": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 >> put k=e v=e1 ts=1 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=f ts=1 +del: "f": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 >> put k=g v=g1 ts=1 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=h ts=1 +del: "h": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 >> put k=i v=i1 ts=1 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=j ts=1 +del: "j": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 >> del_range_ts k=g end=s ts=3 stats: range_key_count=+1 range_key_bytes=+13 range_val_count=+1 live_count=-2 live_bytes=-42 gc_bytes_age=+5335 @@ -82,10 +86,12 @@ stats: range_key_bytes=+9 range_val_count=+1 gc_bytes_age=+860 >> put k=n v=n5 ts=5 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=o ts=5 +del: "o": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1330 >> put k=q v=q5 ts=5 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=r ts=5 +del: "r": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1330 >> put k=a v=a6 t=A stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 @@ -94,10 +100,13 @@ stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_bytes=+48 gc_bytes_age=+178 >> put k=c v=c6 t=A stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-198 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 >> del k=d t=A +del: "d": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> del k=e t=A +del: "e": found key true stats: key_bytes=+12 val_count=+1 val_bytes=+48 live_count=-1 live_bytes=-21 gc_bytes_age=+7614 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> del k=f t=A +del: "f": found key false stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5630 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> put k=g v=g6 t=A stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-194 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 @@ -106,10 +115,13 @@ stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_ >> put k=i v=i6 t=A stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-194 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 >> del k=j t=A +del: "j": found key false stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5630 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> del k=k t=A +del: "k": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> del k=l t=A +del: "l": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> put k=m v=m6 t=A stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 @@ -118,10 +130,13 @@ stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_bytes=+48 gc_bytes_age=+178 >> put k=o v=o6 t=A stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-190 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 >> del k=p t=A +del: "p": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> del k=q t=A +del: "q": found key true stats: key_bytes=+12 val_count=+1 val_bytes=+48 live_count=-1 live_bytes=-21 gc_bytes_age=+7614 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> del k=r t=A +del: "r": found key false stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5638 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> at end: txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=6.000000000,0 wto=false gul=0,0 diff --git a/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_range_abort b/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_range_abort index 0bab1efcbfba..083ec9b99615 100644 --- a/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_range_abort +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_range_abort @@ -62,18 +62,22 @@ with t=A >> put k=b v=b1 ts=1 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=c ts=1 +del: "c": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 >> put k=e v=e1 ts=1 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=f ts=1 +del: "f": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 >> put k=g v=g1 ts=1 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=h ts=1 +del: "h": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 >> put k=i v=i1 ts=1 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=j ts=1 +del: "j": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 >> del_range_ts k=g end=s ts=3 stats: range_key_count=+1 range_key_bytes=+13 range_val_count=+1 live_count=-2 live_bytes=-42 gc_bytes_age=+5335 @@ -82,10 +86,12 @@ stats: range_key_bytes=+9 range_val_count=+1 gc_bytes_age=+860 >> put k=n v=n5 ts=5 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=o ts=5 +del: "o": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1330 >> put k=q v=q5 ts=5 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=r ts=5 +del: "r": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1330 >> put k=a v=a6 t=A stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 @@ -94,10 +100,13 @@ stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_bytes=+48 gc_bytes_age=+178 >> put k=c v=c6 t=A stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-198 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 >> del k=d t=A +del: "d": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> del k=e t=A +del: "e": found key true stats: key_bytes=+12 val_count=+1 val_bytes=+48 live_count=-1 live_bytes=-21 gc_bytes_age=+7614 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> del k=f t=A +del: "f": found key false stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5630 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> put k=g v=g6 t=A stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-194 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 @@ -106,10 +115,13 @@ stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_ >> put k=i v=i6 t=A stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-194 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 >> del k=j t=A +del: "j": found key false stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5630 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> del k=k t=A +del: "k": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> del k=l t=A +del: "l": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> put k=m v=m6 t=A stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 @@ -118,10 +130,13 @@ stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_bytes=+48 gc_bytes_age=+178 >> put k=o v=o6 t=A stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-190 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 >> del k=p t=A +del: "p": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> del k=q t=A +del: "q": found key true stats: key_bytes=+12 val_count=+1 val_bytes=+48 live_count=-1 live_bytes=-21 gc_bytes_age=+7614 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> del k=r t=A +del: "r": found key false stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5638 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> at end: txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=6.000000000,0 wto=false gul=0,0 diff --git a/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_range_commit b/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_range_commit index 5bc127d55963..f6aca715ee0d 100644 --- a/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_range_commit +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_range_commit @@ -62,18 +62,22 @@ with t=A >> put k=b v=b1 ts=1 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=c ts=1 +del: "c": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 >> put k=e v=e1 ts=1 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=f ts=1 +del: "f": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 >> put k=g v=g1 ts=1 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=h ts=1 +del: "h": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 >> put k=i v=i1 ts=1 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=j ts=1 +del: "j": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 >> del_range_ts k=g end=s ts=3 stats: range_key_count=+1 range_key_bytes=+13 range_val_count=+1 live_count=-2 live_bytes=-42 gc_bytes_age=+5335 @@ -82,10 +86,12 @@ stats: range_key_bytes=+9 range_val_count=+1 gc_bytes_age=+860 >> put k=n v=n5 ts=5 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=o ts=5 +del: "o": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1330 >> put k=q v=q5 ts=5 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=r ts=5 +del: "r": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1330 >> put k=a v=a6 t=A stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 @@ -94,10 +100,13 @@ stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_bytes=+48 gc_bytes_age=+178 >> put k=c v=c6 t=A stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-198 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 >> del k=d t=A +del: "d": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> del k=e t=A +del: "e": found key true stats: key_bytes=+12 val_count=+1 val_bytes=+48 live_count=-1 live_bytes=-21 gc_bytes_age=+7614 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> del k=f t=A +del: "f": found key false stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5630 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> put k=g v=g6 t=A stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-194 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 @@ -106,10 +115,13 @@ stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_ >> put k=i v=i6 t=A stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-194 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 >> del k=j t=A +del: "j": found key false stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5630 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> del k=k t=A +del: "k": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> del k=l t=A +del: "l": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> put k=m v=m6 t=A stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 @@ -118,10 +130,13 @@ stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_bytes=+48 gc_bytes_age=+178 >> put k=o v=o6 t=A stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-190 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 >> del k=p t=A +del: "p": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> del k=q t=A +del: "q": found key true stats: key_bytes=+12 val_count=+1 val_bytes=+48 live_count=-1 live_bytes=-21 gc_bytes_age=+7614 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> del k=r t=A +del: "r": found key false stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5638 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> at end: txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=6.000000000,0 wto=false gul=0,0 diff --git a/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_range_pushed b/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_range_pushed index 454076c6037c..43884df83412 100644 --- a/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_range_pushed +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_range_pushed @@ -62,18 +62,22 @@ with t=A >> put k=b v=b1 ts=1 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=c ts=1 +del: "c": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 >> put k=e v=e1 ts=1 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=f ts=1 +del: "f": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 >> put k=g v=g1 ts=1 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=h ts=1 +del: "h": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 >> put k=i v=i1 ts=1 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=j ts=1 +del: "j": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 >> del_range_ts k=g end=s ts=3 stats: range_key_count=+1 range_key_bytes=+13 range_val_count=+1 live_count=-2 live_bytes=-42 gc_bytes_age=+5335 @@ -82,10 +86,12 @@ stats: range_key_bytes=+9 range_val_count=+1 gc_bytes_age=+860 >> put k=n v=n5 ts=5 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=o ts=5 +del: "o": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1330 >> put k=q v=q5 ts=5 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=r ts=5 +del: "r": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1330 >> put k=a v=a6 t=A stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 @@ -94,10 +100,13 @@ stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_bytes=+48 gc_bytes_age=+178 >> put k=c v=c6 t=A stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-198 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 >> del k=d t=A +del: "d": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> del k=e t=A +del: "e": found key true stats: key_bytes=+12 val_count=+1 val_bytes=+48 live_count=-1 live_bytes=-21 gc_bytes_age=+7614 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> del k=f t=A +del: "f": found key false stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5630 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> put k=g v=g6 t=A stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-194 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 @@ -106,10 +115,13 @@ stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_ >> put k=i v=i6 t=A stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-194 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 >> del k=j t=A +del: "j": found key false stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5630 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> del k=k t=A +del: "k": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> del k=l t=A +del: "l": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> put k=m v=m6 t=A stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 @@ -118,10 +130,13 @@ stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_bytes=+48 gc_bytes_age=+178 >> put k=o v=o6 t=A stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-190 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 >> del k=p t=A +del: "p": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> del k=q t=A +del: "q": found key true stats: key_bytes=+12 val_count=+1 val_bytes=+48 live_count=-1 live_bytes=-21 gc_bytes_age=+7614 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> del k=r t=A +del: "r": found key false stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5638 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> at end: txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=6.000000000,0 wto=false gul=0,0 diff --git a/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_rewrite b/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_rewrite index 07f5a75714d9..379f6baaed84 100644 --- a/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_rewrite +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_rewrite @@ -62,18 +62,22 @@ with t=A >> put k=b v=b1 ts=1 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=c ts=1 +del: "c": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 >> put k=e v=e1 ts=1 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=f ts=1 +del: "f": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 >> put k=g v=g1 ts=1 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=h ts=1 +del: "h": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 >> put k=i v=i1 ts=1 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=j ts=1 +del: "j": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 >> del_range_ts k=g end=s ts=3 stats: range_key_count=+1 range_key_bytes=+13 range_val_count=+1 live_count=-2 live_bytes=-42 gc_bytes_age=+5335 @@ -82,10 +86,12 @@ stats: range_key_bytes=+9 range_val_count=+1 gc_bytes_age=+860 >> put k=n v=n5 ts=5 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=o ts=5 +del: "o": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1330 >> put k=q v=q5 ts=5 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> del k=r ts=5 +del: "r": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1330 >> put k=a v=a6 t=A stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 @@ -94,10 +100,13 @@ stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_bytes=+48 gc_bytes_age=+178 >> put k=c v=c6 t=A stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-198 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 >> del k=d t=A +del: "d": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> del k=e t=A +del: "e": found key true stats: key_bytes=+12 val_count=+1 val_bytes=+48 live_count=-1 live_bytes=-21 gc_bytes_age=+7614 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> del k=f t=A +del: "f": found key false stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5630 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> put k=g v=g6 t=A stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-194 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 @@ -106,10 +115,13 @@ stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_ >> put k=i v=i6 t=A stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-194 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 >> del k=j t=A +del: "j": found key false stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5630 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> del k=k t=A +del: "k": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> del k=l t=A +del: "l": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> put k=m v=m6 t=A stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 @@ -118,10 +130,13 @@ stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_bytes=+48 gc_bytes_age=+178 >> put k=o v=o6 t=A stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-190 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 >> del k=p t=A +del: "p": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> del k=q t=A +del: "q": found key true stats: key_bytes=+12 val_count=+1 val_bytes=+48 live_count=-1 live_bytes=-21 gc_bytes_age=+7614 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> del k=r t=A +del: "r": found key false stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5638 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 >> at end: txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=6.000000000,0 wto=false gul=0,0 @@ -207,10 +222,13 @@ stats: gc_bytes_age=-19 intent_age=-1 >> put k=c v=c7 t=A ts=7 stats: intent_age=-1 >> del k=d t=A ts=7 +del: "d": found key false stats: gc_bytes_age=-62 intent_age=-1 >> del k=e t=A ts=7 +del: "e": found key true stats: gc_bytes_age=-81 intent_age=-1 >> del k=f t=A ts=7 +del: "f": found key false stats: gc_bytes_age=-62 intent_age=-1 >> put k=g v=g7 t=A ts=7 stats: intent_age=-1 @@ -219,10 +237,13 @@ stats: intent_age=-1 >> put k=i v=i7 t=A ts=7 stats: intent_age=-1 >> del k=j t=A ts=7 +del: "j": found key false stats: gc_bytes_age=-62 intent_age=-1 >> del k=k t=A ts=7 +del: "k": found key false stats: gc_bytes_age=-62 intent_age=-1 >> del k=l t=A ts=7 +del: "l": found key false stats: gc_bytes_age=-62 intent_age=-1 >> put k=m v=m7 t=A ts=7 stats: intent_age=-1 @@ -231,10 +252,13 @@ stats: gc_bytes_age=-19 intent_age=-1 >> put k=o v=o7 t=A ts=7 stats: intent_age=-1 >> del k=p t=A ts=7 +del: "p": found key false stats: gc_bytes_age=-62 intent_age=-1 >> del k=q t=A ts=7 +del: "q": found key true stats: gc_bytes_age=-81 intent_age=-1 >> del k=r t=A ts=7 +del: "r": found key false stats: gc_bytes_age=-62 intent_age=-1 >> at end: txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=1 ts=7.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=7.000000000,0 wto=false gul=0,0 @@ -308,10 +332,13 @@ with t=A ts=8 put k=l v=l8 ---- >> del k=a t=A ts=8 +del: "a": found key false stats: val_bytes=-7 live_count=-1 live_bytes=-69 gc_bytes_age=+5704 intent_bytes=-7 intent_age=-1 >> del k=b t=A ts=8 +del: "b": found key true stats: val_bytes=-7 live_count=-1 live_bytes=-69 gc_bytes_age=+5685 intent_bytes=-7 intent_age=-1 >> del k=c t=A ts=8 +del: "c": found key false stats: val_bytes=-7 live_count=-1 live_bytes=-69 gc_bytes_age=+5704 intent_bytes=-7 intent_age=-1 >> put k=d v=d8 t=A ts=8 stats: val_bytes=+7 live_count=+1 live_bytes=+69 gc_bytes_age=-5766 intent_bytes=+7 intent_age=-1 @@ -320,10 +347,13 @@ stats: val_bytes=+7 live_count=+1 live_bytes=+69 gc_bytes_age=-5785 intent_bytes >> put k=f v=f8 t=A ts=8 stats: val_bytes=+7 live_count=+1 live_bytes=+69 gc_bytes_age=-5766 intent_bytes=+7 intent_age=-1 >> del k=g t=A ts=8 +del: "g": found key false stats: val_bytes=-7 live_count=-1 live_bytes=-69 gc_bytes_age=+5704 intent_bytes=-7 intent_age=-1 >> del k=h t=A ts=8 +del: "h": found key false stats: val_bytes=-7 live_count=-1 live_bytes=-69 gc_bytes_age=+5704 intent_bytes=-7 intent_age=-1 >> del k=i t=A ts=8 +del: "i": found key false stats: val_bytes=-7 live_count=-1 live_bytes=-69 gc_bytes_age=+5704 intent_bytes=-7 intent_age=-1 >> put k=j v=j8 t=A ts=8 stats: val_bytes=+7 live_count=+1 live_bytes=+69 gc_bytes_age=-5766 intent_bytes=+7 intent_age=-1 diff --git a/pkg/storage/testdata/mvcc_histories/range_tombstone_writes b/pkg/storage/testdata/mvcc_histories/range_tombstone_writes index 3c8f6e1ac780..c772c64e5c45 100644 --- a/pkg/storage/testdata/mvcc_histories/range_tombstone_writes +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_writes @@ -27,6 +27,8 @@ with t=A put k=d v=d7 put k=i v=i7 ---- +del: "a": found key true +del: "g": found key true >> 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: {k-p}/[4.000000000,0=/] diff --git a/pkg/storage/testdata/mvcc_histories/target_bytes b/pkg/storage/testdata/mvcc_histories/target_bytes index 0b42dfcb7cd8..b96fe938238a 100644 --- a/pkg/storage/testdata/mvcc_histories/target_bytes +++ b/pkg/storage/testdata/mvcc_histories/target_bytes @@ -8,6 +8,7 @@ with ts=1,0 put k=aa v=willbetombstoned del k=aa ts=250,1 ---- +del: "aa": found key true >> at end: data: "a"/1.000000000,0 -> /BYTES/nevergoingtobeseen data: "aa"/250.000000000,1 -> /