diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index b0c5d20b4d9b..681377ec088f 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -286,4 +286,4 @@ trace.jaeger.agent string the address of a Jaeger agent to receive traces using trace.opentelemetry.collector string address of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as :. If no port is specified, 4317 will be used. trace.span_registry.enabled boolean true if set, ongoing traces can be seen at https:///#/debug/tracez trace.zipkin.collector string the address of a Zipkin instance to receive traces, as :. If no port is specified, 9411 will be used. -version version 22.1-44 set the active cluster version in the format '.' +version version 22.1-46 set the active cluster version in the format '.' diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 1cde863bee26..271f8d810528 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -217,6 +217,6 @@ trace.opentelemetry.collectorstringaddress of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as :. If no port is specified, 4317 will be used. trace.span_registry.enabledbooleantrueif set, ongoing traces can be seen at https:///#/debug/tracez trace.zipkin.collectorstringthe address of a Zipkin instance to receive traces, as :. If no port is specified, 9411 will be used. -versionversion22.1-44set the active cluster version in the format '.' +versionversion22.1-46set the active cluster version in the format '.' 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 672518a298b1..9dda67328cf7 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -343,6 +343,9 @@ const ( // SchemaChangeSupportsCreateFunction adds support of CREATE FUNCTION // statement. SchemaChangeSupportsCreateFunction + // DeleteRequestReturnKey is the version where the DeleteRequest began + // populating the FoundKey value in the response. + DeleteRequestReturnKey // ************************************************* // Step (1): Add new versions here. @@ -604,6 +607,10 @@ var versionsSingleton = keyedVersions{ Key: SchemaChangeSupportsCreateFunction, Version: roachpb.Version{Major: 22, Minor: 1, Internal: 44}, }, + { + Key: DeleteRequestReturnKey, + Version: roachpb.Version{Major: 22, Minor: 1, Internal: 46}, + }, // ************************************************* // Step (2): Add new versions here. // Do not add new versions to a patch release. diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index d5716a03f246..ad6e25e1cdf3 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -64,11 +64,12 @@ func _() { _ = x[SetUserIDNotNull-53] _ = x[SQLSchemaTelemetryScheduledJobs-54] _ = x[SchemaChangeSupportsCreateFunction-55] + _ = x[DeleteRequestReturnKey-56] } -const _Key_name = "V21_2Start22_1PebbleFormatBlockPropertyCollectorProbeRequestPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreSCRAMAuthenticationUnsafeLossOfQuorumRecoveryRangeLogAlterSystemProtectedTimestampAddColumnEnableProtectedTimestampsForTenantDeleteCommentsWithDroppedIndexesRemoveIncompatibleDatabasePrivilegesAddRaftAppliedIndexTermMigrationPostAddRaftAppliedIndexTermMigrationDontProposeWriteTimestampForLeaseTransfersEnablePebbleFormatVersionBlockPropertiesMVCCIndexBackfillerEnableLeaseHolderRemovalLooselyCoupledRaftLogTruncationChangefeedIdlenessEnableDeclarativeSchemaChangerRowLevelTTLPebbleFormatSplitUserKeysMarkedEnableNewStoreRebalancerClusterLocksVirtualTableAutoStatsTableSettingsSuperRegionsEnableNewChangefeedOptionsSpanCountTablePreSeedSpanCountTableSeedSpanCountTableV22_1Start22_2LocalTimestampsPebbleFormatSplitUserKeysMarkedCompactedEnsurePebbleFormatVersionRangeKeysEnablePebbleFormatVersionRangeKeysTrigramInvertedIndexesRemoveGrantPrivilegeMVCCRangeTombstonesUpgradeSequenceToBeReferencedByIDSampledStmtDiagReqsAddSSTableTombstonesSystemPrivilegesTableEnablePredicateProjectionChangefeedAlterSystemSQLInstancesAddLocalitySystemExternalConnectionsTableAlterSystemStatementStatisticsAddIndexRecommendationsRoleIDSequenceAddSystemUserIDColumnUsersHaveIDsSetUserIDNotNullSQLSchemaTelemetryScheduledJobsSchemaChangeSupportsCreateFunction" +const _Key_name = "V21_2Start22_1PebbleFormatBlockPropertyCollectorProbeRequestPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreSCRAMAuthenticationUnsafeLossOfQuorumRecoveryRangeLogAlterSystemProtectedTimestampAddColumnEnableProtectedTimestampsForTenantDeleteCommentsWithDroppedIndexesRemoveIncompatibleDatabasePrivilegesAddRaftAppliedIndexTermMigrationPostAddRaftAppliedIndexTermMigrationDontProposeWriteTimestampForLeaseTransfersEnablePebbleFormatVersionBlockPropertiesMVCCIndexBackfillerEnableLeaseHolderRemovalLooselyCoupledRaftLogTruncationChangefeedIdlenessEnableDeclarativeSchemaChangerRowLevelTTLPebbleFormatSplitUserKeysMarkedEnableNewStoreRebalancerClusterLocksVirtualTableAutoStatsTableSettingsSuperRegionsEnableNewChangefeedOptionsSpanCountTablePreSeedSpanCountTableSeedSpanCountTableV22_1Start22_2LocalTimestampsPebbleFormatSplitUserKeysMarkedCompactedEnsurePebbleFormatVersionRangeKeysEnablePebbleFormatVersionRangeKeysTrigramInvertedIndexesRemoveGrantPrivilegeMVCCRangeTombstonesUpgradeSequenceToBeReferencedByIDSampledStmtDiagReqsAddSSTableTombstonesSystemPrivilegesTableEnablePredicateProjectionChangefeedAlterSystemSQLInstancesAddLocalitySystemExternalConnectionsTableAlterSystemStatementStatisticsAddIndexRecommendationsRoleIDSequenceAddSystemUserIDColumnUsersHaveIDsSetUserIDNotNullSQLSchemaTelemetryScheduledJobsSchemaChangeSupportsCreateFunctionDeleteRequestReturnKey" -var _Key_index = [...]uint16{0, 5, 14, 48, 60, 88, 118, 146, 167, 186, 220, 258, 292, 324, 360, 392, 428, 470, 510, 529, 553, 584, 602, 632, 643, 674, 698, 722, 744, 756, 782, 796, 817, 835, 840, 849, 864, 904, 938, 972, 994, 1014, 1033, 1066, 1085, 1105, 1126, 1161, 1195, 1225, 1278, 1292, 1313, 1325, 1341, 1372, 1406} +var _Key_index = [...]uint16{0, 5, 14, 48, 60, 88, 118, 146, 167, 186, 220, 258, 292, 324, 360, 392, 428, 470, 510, 529, 553, 584, 602, 632, 643, 674, 698, 722, 744, 756, 782, 796, 817, 835, 840, 849, 864, 904, 938, 972, 994, 1014, 1033, 1066, 1085, 1105, 1126, 1161, 1195, 1225, 1278, 1292, 1313, 1325, 1341, 1372, 1406, 1428} func (i Key) String() string { if i < 0 || i >= Key(len(_Key_index)-1) { diff --git a/pkg/kv/batch.go b/pkg/kv/batch.go index 61329f958abf..a28f3d0b2cc0 100644 --- a/pkg/kv/batch.go +++ b/pkg/kv/batch.go @@ -241,8 +241,14 @@ func (b *Batch) fillResults(ctx context.Context) { } } case *roachpb.DeleteRequest: - row := &result.Rows[k] - row.Key = []byte(args.(*roachpb.DeleteRequest).Key) + if result.Err == nil { + resp := reply.(*roachpb.DeleteResponse) + if resp.FoundKey { + // Accumulate all keys that were deleted as part of a + // single Del() operation. + result.Keys = append(result.Keys, args.(*roachpb.DeleteRequest).Key) + } + } case *roachpb.DeleteRangeRequest: if result.Err == nil { result.Keys = reply.(*roachpb.DeleteRangeResponse).Keys @@ -609,8 +615,9 @@ func (b *Batch) ReverseScanForUpdate(s, e interface{}) { // Del deletes one or more keys. // -// A new result will be appended to the batch and each key will have a -// corresponding row in the returned Result. +// A new result will be appended to the batch which will contain 0 rows and +// Result.Err will indicate success or failure. Each 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) Del(keys ...interface{}) { @@ -618,14 +625,14 @@ func (b *Batch) Del(keys ...interface{}) { for _, key := range keys { k, err := marshalKey(key) if err != nil { - b.initResult(0, len(keys), notRaw, err) + b.initResult(len(keys), 0, notRaw, err) return } reqs = append(reqs, roachpb.NewDelete(k)) b.approxMutationReqBytes += len(k) } b.appendReqs(reqs...) - b.initResult(len(reqs), len(reqs), notRaw, nil) + b.initResult(len(reqs), 0, notRaw, nil) } // DelRange deletes the rows between begin (inclusive) and end (exclusive). diff --git a/pkg/kv/db.go b/pkg/kv/db.go index c5fbf590161c..e1ebbd3083e8 100644 --- a/pkg/kv/db.go +++ b/pkg/kv/db.go @@ -126,13 +126,13 @@ type Result struct { // Err contains any error encountered when performing the operation. Err error // Rows contains the key/value pairs for the operation. The number of rows - // returned varies by operation. For Get, Put, CPut, Inc and Del the number + // returned varies by operation. For Get, Put, CPut, and Inc the number // of rows returned is the number of keys operated on. For Scan the number of // rows returned is the number or rows matching the scan capped by the - // maxRows parameter and other options. For DelRange Rows is nil. + // maxRows parameter and other options. For Del and DelRange Rows is nil. Rows []KeyValue - // Keys is set by some operations instead of returning the rows themselves. + // Keys is set by Del and DelRange instead of returning the rows themselves. Keys []roachpb.Key // ResumeSpan is the span to be used on the next operation in a @@ -526,11 +526,14 @@ func (db *DB) ReverseScanForUpdate( // Del deletes one or more keys. // +// The returned []roachpb.Key will contain the keys that were actually deleted. +// // key can be either a byte slice or a string. -func (db *DB) Del(ctx context.Context, keys ...interface{}) error { +func (db *DB) Del(ctx context.Context, keys ...interface{}) ([]roachpb.Key, error) { b := &Batch{} b.Del(keys...) - return getOneErr(db.Run(ctx, b), b) + r, err := getOneResult(db.Run(ctx, b), b) + return r.Keys, 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 df8f986e06ec..867c50adaf9b 100644 --- a/pkg/kv/db_test.go +++ b/pkg/kv/db_test.go @@ -254,7 +254,7 @@ func TestDB_InitPut(t *testing.T) { if err := db.InitPut(ctx, "aa", "2", false); err == nil { t.Fatal("expected error from init put") } - if err := db.Del(ctx, "aa"); err != nil { + if _, err := db.Del(ctx, "aa"); err != nil { t.Fatal(err) } if err := db.InitPut(ctx, "aa", "2", true); err == nil { @@ -479,9 +479,16 @@ func TestDB_Del(t *testing.T) { if err := db.Run(context.Background(), b); err != nil { t.Fatal(err) } - if err := db.Del(context.Background(), "ab"); err != nil { + if _, err := db.Del(context.Background(), "ab"); err != nil { t.Fatal(err) } + // Also try deleting a non-existent key and verify that no key is + // returned. + if deletedKeys, err := db.Del(context.Background(), "ad"); err != nil { + t.Fatal(err) + } else if len(deletedKeys) > 0 { + t.Errorf("expected deleted key to be empty when deleting a non-existent key, got %v", deletedKeys) + } rows, err := db.Scan(context.Background(), "a", "b", 100) if err != nil { t.Fatal(err) @@ -652,7 +659,8 @@ func TestDBDecommissionedOperations(t *testing.T) { op func() error }{ {"Del", func() error { - return db.Del(ctx, key) + _, err := db.Del(ctx, key) + return err }}, {"DelRange", func() error { _, err := db.DelRange(ctx, key, keyEnd, false) diff --git a/pkg/kv/kvclient/kvcoord/dist_sender.go b/pkg/kv/kvclient/kvcoord/dist_sender.go index f284425a0709..acdda1f99dc0 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender.go @@ -685,7 +685,8 @@ func (ds *DistSender) initAndVerifyBatch( // Accepted reverse range requests. foundReverse = true - case *roachpb.QueryIntentRequest, *roachpb.EndTxnRequest, *roachpb.GetRequest: + case *roachpb.QueryIntentRequest, *roachpb.EndTxnRequest, + *roachpb.GetRequest, *roachpb.DeleteRequest: // Accepted point requests that can be in batches with limit. default: diff --git a/pkg/kv/kvclient/kvcoord/dist_sender_server_test.go b/pkg/kv/kvclient/kvcoord/dist_sender_server_test.go index 34e5f8cfb662..716639ec20fe 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender_server_test.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender_server_test.go @@ -2603,7 +2603,8 @@ func TestTxnCoordSenderRetries(t *testing.T) { { name: "write too old with initput failing on tombstone before", beforeTxnStart: func(ctx context.Context, db *kv.DB) error { - return db.Del(ctx, "iput") + _, err := db.Del(ctx, "iput") + return err }, afterTxnStart: func(ctx context.Context, db *kv.DB) error { return db.Put(ctx, "iput", "put2") @@ -2619,7 +2620,8 @@ func TestTxnCoordSenderRetries(t *testing.T) { return db.Put(ctx, "iput", "put") }, afterTxnStart: func(ctx context.Context, db *kv.DB) error { - return db.Del(ctx, "iput") + _, err := db.Del(ctx, "iput") + return err }, retryable: func(ctx context.Context, txn *kv.Txn) error { return txn.InitPut(ctx, "iput", "put", true) diff --git a/pkg/kv/kvclient/kvcoord/txn_coord_sender_test.go b/pkg/kv/kvclient/kvcoord/txn_coord_sender_test.go index 5086569a95ec..ff80d241ed4d 100644 --- a/pkg/kv/kvclient/kvcoord/txn_coord_sender_test.go +++ b/pkg/kv/kvclient/kvcoord/txn_coord_sender_test.go @@ -1911,7 +1911,10 @@ func TestCommitMutatingTransaction(t *testing.T) { pointWrite: true, }, { - f: func(ctx context.Context, txn *kv.Txn) error { return txn.Del(ctx, "a") }, + f: func(ctx context.Context, txn *kv.Txn) error { + _, err := txn.Del(ctx, "a") + return err + }, expMethod: roachpb.Delete, pointWrite: true, }, diff --git a/pkg/kv/kvclient/kvcoord/txn_correctness_test.go b/pkg/kv/kvclient/kvcoord/txn_correctness_test.go index 4def538cde43..2df43ee0bdb3 100644 --- a/pkg/kv/kvclient/kvcoord/txn_correctness_test.go +++ b/pkg/kv/kvclient/kvcoord/txn_correctness_test.go @@ -161,7 +161,8 @@ func readCmd(ctx context.Context, c *cmd, txn *kv.Txn) error { // deleteCmd deletes the value at the given key from the db. func deleteCmd(ctx context.Context, c *cmd, txn *kv.Txn) error { - return txn.Del(ctx, c.getKey()) + _, err := txn.Del(ctx, c.getKey()) + return err } // deleteRngCmd deletes the range of values from the db from [key, endKey). diff --git a/pkg/kv/kvclient/rangefeed/rangefeed_external_test.go b/pkg/kv/kvclient/rangefeed/rangefeed_external_test.go index b9002ec1c24a..a33a50528b55 100644 --- a/pkg/kv/kvclient/rangefeed/rangefeed_external_test.go +++ b/pkg/kv/kvclient/rangefeed/rangefeed_external_test.go @@ -439,7 +439,8 @@ func TestRangefeedValueTimestamps(t *testing.T) { { beforeDelTS := db.Clock().Now() - require.NoError(t, db.Del(ctx, mkKey("a"))) + _, err = db.Del(ctx, mkKey("a")) + require.NoError(t, err) afterDelTS := db.Clock().Now() v := <-rows @@ -454,7 +455,8 @@ func TestRangefeedValueTimestamps(t *testing.T) { { beforeDelTS := db.Clock().Now() - require.NoError(t, db.Del(ctx, mkKey("a"))) + _, err = db.Del(ctx, mkKey("a")) + require.NoError(t, err) afterDelTS := db.Clock().Now() v := <-rows diff --git a/pkg/kv/kvclient/rangefeed/rangefeedcache/cache_test.go b/pkg/kv/kvclient/rangefeed/rangefeedcache/cache_test.go index cd1e93c1ae81..ffdcb6186efa 100644 --- a/pkg/kv/kvclient/rangefeed/rangefeedcache/cache_test.go +++ b/pkg/kv/kvclient/rangefeed/rangefeedcache/cache_test.go @@ -108,7 +108,8 @@ func TestCache(t *testing.T) { require.NoError(t, txn.Put(ctx, mkKey("d"), 1)) }) writeAndCheck(t, func(t *testing.T, txn *kv.Txn) { - require.NoError(t, txn.Del(ctx, mkKey("a"))) + _, err := txn.Del(ctx, mkKey("a")) + require.NoError(t, err) }) writeAndCheck(t, func(t *testing.T, txn *kv.Txn) { _, err := txn.DelRange(ctx, mkKey("a"), mkKey("c"), false) diff --git a/pkg/kv/kvnemesis/applier.go b/pkg/kv/kvnemesis/applier.go index 2ea04f7eb7ec..952f2998c057 100644 --- a/pkg/kv/kvnemesis/applier.go +++ b/pkg/kv/kvnemesis/applier.go @@ -159,7 +159,7 @@ type clientI interface { ScanForUpdate(context.Context, interface{}, interface{}, int64) ([]kv.KeyValue, error) ReverseScan(context.Context, interface{}, interface{}, int64) ([]kv.KeyValue, error) ReverseScanForUpdate(context.Context, interface{}, interface{}, int64) ([]kv.KeyValue, error) - Del(context.Context, ...interface{}) error + Del(context.Context, ...interface{}) ([]roachpb.Key, error) DelRange(context.Context, interface{}, interface{}, bool) ([]roachpb.Key, error) Run(context.Context, *kv.Batch) error } @@ -208,8 +208,16 @@ func applyClientOp(ctx context.Context, db clientI, op *Operation, inTxn bool) { } } case *DeleteOperation: - err := db.Del(ctx, o.Key) - o.Result = resultError(ctx, err) + deletedKeys, err := db.Del(ctx, o.Key) + if err != nil { + o.Result = resultError(ctx, err) + } else { + o.Result.Type = ResultType_Keys + o.Result.Keys = make([][]byte, len(deletedKeys)) + for i, deletedKey := range deletedKeys { + o.Result.Keys[i] = deletedKey + } + } case *DeleteRangeOperation: if !inTxn { panic(errors.AssertionFailedf(`non-transactional DelRange operations currently unsupported`)) diff --git a/pkg/kv/kvnemesis/applier_test.go b/pkg/kv/kvnemesis/applier_test.go index 5df5f14889ac..6b68eb12ff10 100644 --- a/pkg/kv/kvnemesis/applier_test.go +++ b/pkg/kv/kvnemesis/applier_test.go @@ -85,13 +85,13 @@ func TestApplier(t *testing.T) { check(t, step(reverseScan(`a`, `c`)), `db0.ReverseScan(ctx, "a", "c", 0) // (["b":"2", "a":"1"], nil)`) check(t, step(reverseScanForUpdate(`a`, `b`)), `db1.ReverseScanForUpdate(ctx, "a", "b", 0) // (["a":"1"], nil)`) - check(t, step(del(`b`)), `db0.Del(ctx, "b") // nil`) + check(t, step(del(`b`)), `db0.Del(ctx, "b")`) check(t, step(get(`b`)), `db1.Get(ctx, "b") // (nil, nil)`) check(t, step(put(`c`, `3`)), `db0.Put(ctx, "c", 3) // nil`) check(t, step(put(`d`, `4`)), `db1.Put(ctx, "d", 4) // nil`) - check(t, step(del(`c`)), `db0.Del(ctx, "c") // nil`) + check(t, step(del(`c`)), `db0.Del(ctx, "c")`) check(t, step(scan(`a`, `e`)), `db1.Scan(ctx, "a", "e", 0) // (["a":"1", "d":"4"], nil)`) check(t, step(put(`c`, `5`)), `db0.Put(ctx, "c", 5) // nil`) 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 ee293240fb25..147218d637c0 100644 --- a/pkg/kv/kvserver/batcheval/cmd_delete.go +++ b/pkg/kv/kvserver/batcheval/cmd_delete.go @@ -28,8 +28,12 @@ func Delete( ) (result.Result, error) { args := cArgs.Args.(*roachpb.DeleteRequest) h := cArgs.Header + reply := resp.(*roachpb.DeleteResponse) - err := storage.MVCCDelete(ctx, readWriter, cArgs.Stats, args.Key, h.Timestamp, cArgs.Now, h.Txn) + var err error + 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_is_span_empty_test.go b/pkg/kv/kvserver/batcheval/cmd_is_span_empty_test.go index bb571a80267a..322b652de411 100644 --- a/pkg/kv/kvserver/batcheval/cmd_is_span_empty_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_is_span_empty_test.go @@ -79,7 +79,8 @@ func TestIsSpanEmpty(t *testing.T) { requireEmpty(t, mkKey(""), mkKey("x")) requireNotEmpty(t, mkKey(""), mkKey("").PrefixEnd()) - require.NoError(t, kvDB.Del(ctx, mkKey("x"))) + _, err := kvDB.Del(ctx, mkKey("x")) + require.NoError(t, err) requireEmpty(t, mkKey(""), mkKey("x")) requireNotEmpty(t, mkKey(""), mkKey("").PrefixEnd()) 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/client_merge_test.go b/pkg/kv/kvserver/client_merge_test.go index 5d599dcb77e2..576249402fe0 100644 --- a/pkg/kv/kvserver/client_merge_test.go +++ b/pkg/kv/kvserver/client_merge_test.go @@ -4119,10 +4119,10 @@ func TestStoreRangeMergeDuringShutdown(t *testing.T) { // Simulate a merge transaction by launching a transaction that lays down // intents on the two copies of the RHS range descriptor. txn := kv.NewTxn(ctx, store.DB(), 0 /* gatewayNodeID */) - if err := txn.Del(ctx, keys.RangeDescriptorKey(rhsDesc.StartKey)); err != nil { + if _, err := txn.Del(ctx, keys.RangeDescriptorKey(rhsDesc.StartKey)); err != nil { t.Fatal(err) } - if err := txn.Del(ctx, keys.RangeMetaKey(rhsDesc.StartKey)); err != nil { + if _, err := txn.Del(ctx, keys.RangeMetaKey(rhsDesc.StartKey)); err != nil { t.Fatal(err) } 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 142f65d397de..788ae283ba35 100644 --- a/pkg/kv/kvserver/replica_raftstorage.go +++ b/pkg/kv/kvserver/replica_raftstorage.go @@ -701,7 +701,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/kv/txn.go b/pkg/kv/txn.go index 65a7fdde0cc8..dfecaf039a9d 100644 --- a/pkg/kv/txn.go +++ b/pkg/kv/txn.go @@ -628,11 +628,14 @@ func (txn *Txn) Iterate( // Del deletes one or more keys. // +// The returned []roachpb.Key will contain the keys that were actually deleted. +// // key can be either a byte slice or a string. -func (txn *Txn) Del(ctx context.Context, keys ...interface{}) error { +func (txn *Txn) Del(ctx context.Context, keys ...interface{}) ([]roachpb.Key, error) { b := txn.NewBatch() b.Del(keys...) - return getOneErr(txn.Run(ctx, b), b) + r, err := getOneResult(txn.Run(ctx, b), b) + return r.Keys, err } // DelRange deletes the rows between begin (inclusive) and end (exclusive). diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index 5a4cf28bf1de..27c3244c725b 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -343,6 +343,9 @@ message DeleteRequest { // A DeleteResponse is the return value from the Delete() method. message DeleteResponse { ResponseHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; + // True if there was a key that got deleted. A tombstone is written + // unconditionally, regardless of whether the key is found. + bool found_key = 2; } // A DeleteRangeRequest is the argument to the DeleteRange() method. It diff --git a/pkg/sql/delete_range.go b/pkg/sql/delete_range.go index 70e54e2d8404..97a242774d17 100644 --- a/pkg/sql/delete_range.go +++ b/pkg/sql/delete_range.go @@ -14,6 +14,7 @@ import ( "bytes" "context" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog" @@ -88,8 +89,9 @@ func (d *deleteRangeNode) startExec(params runParams) error { return err } - // Configure the fetcher, which is only used to decode the returned keys from - // the DeleteRange, and is never used to actually fetch kvs. + // Configure the fetcher, which is only used to decode the returned keys + // from the Del 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 */ @@ -159,15 +161,24 @@ func (d *deleteRangeNode) startExec(params runParams) error { return nil } -// deleteSpans adds each input span to a DelRange command in the given batch. +// deleteSpans adds each input span to a Del or a DelRange command in the given +// batch. func (d *deleteRangeNode) deleteSpans(params runParams, b *kv.Batch, spans roachpb.Spans) { ctx := params.ctx traceKV := params.p.ExtendedEvalContext().Tracing.KVTracingEnabled() + canUsePointDelete := params.ExecCfg().Settings.Version.IsActive(ctx, clusterversion.DeleteRequestReturnKey) for _, span := range spans { - if traceKV { - log.VEventf(ctx, 2, "DelRange %s - %s", span.Key, span.EndKey) + if span.EndKey == nil && canUsePointDelete { + if traceKV { + log.VEventf(ctx, 2, "Del %s", span.Key) + } + b.Del(span.Key) + } else { + if traceKV { + log.VEventf(ctx, 2, "DelRange %s - %s", span.Key, span.EndKey) + } + b.DelRange(span.Key, span.EndKey, true /* returnKeys */) } - b.DelRange(span.Key, span.EndKey, true /* returnKeys */) } } diff --git a/pkg/sql/drop_database.go b/pkg/sql/drop_database.go index 912d5de6141a..539fcceea8b3 100644 --- a/pkg/sql/drop_database.go +++ b/pkg/sql/drop_database.go @@ -141,7 +141,7 @@ func (n *dropDatabaseNode) startExec(params runParams) error { // The public schema and temporary schemas are cleaned up by just removing // the existing namespace entries. key := catalogkeys.MakeSchemaNameKey(p.ExecCfg().Codec, n.dbDesc.GetID(), schemaToDelete.GetName()) - if err := p.txn.Del(ctx, key); err != nil { + if _, err := p.txn.Del(ctx, key); err != nil { return err } case catalog.SchemaUserDefined: diff --git a/pkg/sql/gcjob_test/gc_job_test.go b/pkg/sql/gcjob_test/gc_job_test.go index 716216890ceb..81c08812cb5b 100644 --- a/pkg/sql/gcjob_test/gc_job_test.go +++ b/pkg/sql/gcjob_test/gc_job_test.go @@ -303,11 +303,12 @@ SELECT job_id, status, running_status // Manually delete the table. require.NoError(t, kvDB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { nameKey := catalogkeys.MakePublicObjectNameKey(keys.SystemSQLCodec, dbID, "foo") - if err := txn.Del(ctx, nameKey); err != nil { + if _, err := txn.Del(ctx, nameKey); err != nil { return err } descKey := catalogkeys.MakeDescMetadataKey(keys.SystemSQLCodec, tableID) - return txn.Del(ctx, descKey) + _, err := txn.Del(ctx, descKey) + return err })) // Update the GC TTL to tickle the job to refresh the status and discover that // it has been removed. Use a SucceedsSoon to deal with races between setting diff --git a/pkg/sql/opt/exec/execbuilder/testdata/autocommit b/pkg/sql/opt/exec/execbuilder/testdata/autocommit index 4108e0c21f29..d04cbb3c44d6 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/autocommit +++ b/pkg/sql/opt/exec/execbuilder/testdata/autocommit @@ -506,7 +506,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r52: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1 +dist sender send r52: sending batch 1 Del, 1 EndTxn to (n1,s1):1 # Multi-row delete should auto-commit. query B @@ -767,7 +767,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r52: sending batch 1 DelRng to (n1,s1):1 +dist sender send r52: sending batch 1 Del to (n1,s1):1 dist sender send r52: sending batch 1 Scan to (n1,s1):1 dist sender send r52: sending batch 1 Del, 1 EndTxn to (n1,s1):1 diff --git a/pkg/sql/opt/exec/execbuilder/testdata/cascade b/pkg/sql/opt/exec/execbuilder/testdata/cascade index 9b14e74f0f00..2f47e363b03e 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/cascade +++ b/pkg/sql/opt/exec/execbuilder/testdata/cascade @@ -71,7 +71,7 @@ query T rowsort SELECT message FROM [SHOW TRACE FOR SESSION] WHERE message LIKE '%executing cascade %' OR message LIKE 'Del%' ---- -DelRange /Table/106/1/"a-pk1" - /Table/106/1/"a-pk1"/PrefixEnd +Del /Table/106/1/"a-pk1"/0 executing cascade for constraint b1_delete_cascade_fkey Del /Table/107/1/"b1-pk1"/0 Del /Table/107/1/"b1-pk2"/0 @@ -315,7 +315,7 @@ query T SELECT message FROM [SHOW TRACE FOR SESSION] WHERE message LIKE '%executing cascade %' OR message LIKE 'Del%' ---- -DelRange /Table/123/1/"a-pk1" - /Table/123/1/"a-pk1"/PrefixEnd +Del /Table/123/1/"a-pk1"/0 executing cascade for constraint b1_delete_cascade_fkey Del /Table/124/1/"b1-pk1"/0 Del /Table/124/1/"b1-pk2"/0 @@ -603,7 +603,7 @@ query T SELECT message FROM [SHOW TRACE FOR SESSION] WHERE message LIKE '%executing cascade %' OR message LIKE 'Del%' ---- -DelRange /Table/145/1/"a-pk1" - /Table/145/1/"a-pk1"/PrefixEnd +Del /Table/145/1/"a-pk1"/0 executing cascade for constraint b1_delete_cascade_fkey Del /Table/146/1/"b1-pk1"/0 Del /Table/146/1/"b1-pk2"/0 @@ -798,7 +798,8 @@ subtest DelRange statement ok CREATE TABLE delrng1 ( p INT PRIMARY KEY, - data INT + data INT, + FAMILY (p, data) ); CREATE TABLE delrng2 ( p INT, @@ -841,10 +842,10 @@ vectorized: true └── • fk-cascade fk: delrng2a_p_fkey -query T kvtrace(DelRange) +query T kvtrace(Del,DelRange) DELETE FROM delrng1 WHERE p = 1 ---- -DelRange /Table/165/1/1 - /Table/165/1/2 +Del /Table/165/1/1/0 DelRange /Table/166/1/1 - /Table/166/1/2 DelRange /Table/167/1/1 - /Table/167/1/2 DelRange /Table/168/1/1 - /Table/168/1/2 diff --git a/pkg/sql/opt/exec/execbuilder/testdata/delete b/pkg/sql/opt/exec/execbuilder/testdata/delete index 3969e3eea61f..951b49f347f9 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/delete +++ b/pkg/sql/opt/exec/execbuilder/testdata/delete @@ -249,10 +249,29 @@ SET tracing = off query TT SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] -WHERE message LIKE '%DelRange%' OR message LIKE '%sending batch%' +WHERE message LIKE '%Del%' OR message LIKE '%sending batch%' ---- -batch flow coordinator DelRange /Table/110/1/5 - /Table/110/1/6 -dist sender send r52: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1 +batch flow coordinator Del /Table/110/1/5/0 +dist sender send r52: sending batch 1 Del, 1 EndTxn to (n1,s1):1 + +# Ensure that we send DelRanges when doing a point delete operation on a table +# that has multiple column families. + +statement ok +CREATE TABLE multicf (a INT PRIMARY KEY, b INT, FAMILY (a), FAMILY (b)) + +statement ok +SELECT * FROM multicf + +statement ok +SET tracing = on,kv; DELETE FROM multicf WHERE a = 5; SET tracing = off + +query TT +SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] +WHERE message LIKE '%Del%' OR message LIKE '%sending batch%' +---- +batch flow coordinator DelRange /Table/111/1/5 - /Table/111/1/6 +dist sender send r52: sending batch 1 DelRng to (n1,s1):1 statement ok CREATE TABLE xyz ( diff --git a/pkg/sql/opt/exec/execbuilder/testdata/partial_index b/pkg/sql/opt/exec/execbuilder/testdata/partial_index index 1cb01631c140..a26c80159ccb 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/partial_index +++ b/pkg/sql/opt/exec/execbuilder/testdata/partial_index @@ -1070,7 +1070,7 @@ INSERT INTO t57085_c VALUES (10, 1, true), (20, 1, false), (30, 2, true); query T kvtrace DELETE FROM t57085_p WHERE p = 1; ---- -DelRange /Table/114/1/1 - /Table/114/1/2 +Del /Table/114/1/1/0 Scan /Table/115/{1-2} Del /Table/115/2/1/10/0 Del /Table/115/1/10/0 diff --git a/pkg/sql/opt_exec_factory.go b/pkg/sql/opt_exec_factory.go index 5f8b4679fed1..ee2ddbc867d1 100644 --- a/pkg/sql/opt_exec_factory.go +++ b/pkg/sql/opt_exec_factory.go @@ -18,6 +18,7 @@ import ( "net/url" "strings" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/featureflag" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -1721,7 +1722,16 @@ func (ef *execFactory) ConstructDeleteRange( var sb span.Builder sb.Init(ef.planner.EvalContext(), ef.planner.ExecCfg().Codec, tabDesc, tabDesc.GetPrimaryIndex()) - spans, err := sb.SpansFromConstraint(indexConstraint, span.NoopSplitter()) + splitter := span.NoopSplitter() + canUsePointDelete := ef.planner.ExecCfg().Settings.Version.IsActive( + ef.planner.EvalContext().Context, clusterversion.DeleteRequestReturnKey, + ) + if canUsePointDelete { + splitter = span.MakeSplitterForDelete( + tabDesc, tabDesc.GetPrimaryIndex(), needed, true, /* forDelete */ + ) + } + spans, err := sb.SpansFromConstraint(indexConstraint, splitter) if err != nil { return nil, err } diff --git a/pkg/sql/repair.go b/pkg/sql/repair.go index 2a20ee3fa142..cf49330f4673 100644 --- a/pkg/sql/repair.go +++ b/pkg/sql/repair.go @@ -626,7 +626,7 @@ func (p *planner) UnsafeDeleteNamespaceEntry( return pgerror.Newf(pgcode.ObjectNotInPrerequisiteState, "refusing to delete namespace entry for non-dropped descriptor") } - if err := p.txn.Del(ctx, key); err != nil { + if _, err := p.txn.Del(ctx, key); err != nil { return errors.Wrap(err, "failed to delete entry") } @@ -675,7 +675,7 @@ func (p *planner) UnsafeDeleteDescriptor(ctx context.Context, descID int64, forc } } descKey := catalogkeys.MakeDescMetadataKey(p.execCfg.Codec, id) - if err := p.txn.Del(ctx, descKey); err != nil { + if _, err := p.txn.Del(ctx, descKey); err != nil { return err } diff --git a/pkg/sql/schema_changer.go b/pkg/sql/schema_changer.go index ffda5c0b1cfd..8bf624bd74e5 100644 --- a/pkg/sql/schema_changer.go +++ b/pkg/sql/schema_changer.go @@ -699,7 +699,7 @@ func (sc *SchemaChanger) exec(ctx context.Context) error { switch desc.(type) { case catalog.SchemaDescriptor, catalog.DatabaseDescriptor: if desc.Dropped() { - if err := sc.execCfg.DB.Del(ctx, catalogkeys.MakeDescMetadataKey(sc.execCfg.Codec, desc.GetID())); err != nil { + if _, err := sc.execCfg.DB.Del(ctx, catalogkeys.MakeDescMetadataKey(sc.execCfg.Codec, desc.GetID())); err != nil { return err } } diff --git a/pkg/sql/schema_changer_test.go b/pkg/sql/schema_changer_test.go index 99301fe6e4b6..8492b291b47b 100644 --- a/pkg/sql/schema_changer_test.go +++ b/pkg/sql/schema_changer_test.go @@ -5275,7 +5275,7 @@ func TestIndexBackfillValidation(t *testing.T) { if err != nil { t.Error(err) } - if err := db.Del(context.Background(), kv[0].Key); err != nil { + if _, err := db.Del(context.Background(), kv[0].Key); err != nil { t.Error(err) } } @@ -5346,7 +5346,7 @@ func TestInvertedIndexBackfillValidation(t *testing.T) { if err != nil { t.Error(err) } - if err := db.Del(context.Background(), kv[0].Key); err != nil { + if _, err := db.Del(context.Background(), kv[0].Key); err != nil { t.Error(err) } } diff --git a/pkg/sql/scrub_test.go b/pkg/sql/scrub_test.go index b5a5f2fd0cbf..eea537d9c44f 100644 --- a/pkg/sql/scrub_test.go +++ b/pkg/sql/scrub_test.go @@ -247,7 +247,8 @@ func removeIndexEntryForDatums( if err != nil { return err } - return kvDB.Del(context.Background(), entry.Key) + _, err = kvDB.Del(context.Background(), entry.Key) + return err } // addIndexEntryForDatums adds an index entry for the given datums. It assumes the datums are in the diff --git a/pkg/sql/span/span_splitter.go b/pkg/sql/span/span_splitter.go index 103615d6e694..effb5cdc4932 100644 --- a/pkg/sql/span/span_splitter.go +++ b/pkg/sql/span/span_splitter.go @@ -44,8 +44,20 @@ func NoopSplitter() Splitter { // MakeSplitter returns a Splitter that splits spans into more specific spans // for the needed families. If span splitting is not possible/useful, returns // the NoopSplitter (which never splits). +// Note: this splitter should **not** be used for deletes. func MakeSplitter( table catalog.TableDescriptor, index catalog.Index, neededColOrdinals util.FastIntSet, +) Splitter { + return MakeSplitterForDelete(table, index, neededColOrdinals, false /* forDelete */) +} + +// MakeSplitterForDelete is the same as MakeSplitter but additionally specifies +// whether the splitter will be used for deletes. +func MakeSplitterForDelete( + table catalog.TableDescriptor, + index catalog.Index, + neededColOrdinals util.FastIntSet, + forDelete bool, ) Splitter { // We can only split a span into separate family specific point lookups if: // @@ -92,6 +104,13 @@ func MakeSplitter( return NoopSplitter() } + // * If we're performing a delete, the table must have just one column + // family, since we need to delete all column families during delete + // operations. + if forDelete && numFamilies > 1 { + return NoopSplitter() + } + return Splitter{ numKeyColumns: index.NumKeyColumns(), neededFamilies: neededFamilies, diff --git a/pkg/sql/sqlinstance/instancestorage/instancestorage.go b/pkg/sql/sqlinstance/instancestorage/instancestorage.go index c8fb19d0af82..2f0b92376586 100644 --- a/pkg/sql/sqlinstance/instancestorage/instancestorage.go +++ b/pkg/sql/sqlinstance/instancestorage/instancestorage.go @@ -232,7 +232,7 @@ func (s *Storage) getAllInstanceRows( func (s *Storage) ReleaseInstanceID(ctx context.Context, id base.SQLInstanceID) error { key := makeInstanceKey(s.codec, s.tableID, id) ctx = multitenant.WithTenantCostControlExemption(ctx) - if err := s.db.Del(ctx, key); err != nil { + if _, err := s.db.Del(ctx, key); err != nil { return errors.Wrapf(err, "could not delete instance %d", id) } return nil diff --git a/pkg/sql/sqlliveness/slstorage/slstorage.go b/pkg/sql/sqlliveness/slstorage/slstorage.go index 16b447241899..83e85529e01c 100644 --- a/pkg/sql/sqlliveness/slstorage/slstorage.go +++ b/pkg/sql/sqlliveness/slstorage/slstorage.go @@ -306,7 +306,8 @@ func (s *Storage) deleteOrFetchSession( // The session is expired and needs to be deleted. deleted = true expiration = hlc.Timestamp{} - return txn.Del(ctx, k) + _, err = txn.Del(ctx, k) + return err }); err != nil { return false, hlc.Timestamp{}, errors.Wrapf(err, "could not query session id: %s", sid) @@ -370,7 +371,7 @@ func (s *Storage) deleteExpiredSessions(ctx context.Context) { deleted++ } } - if err := txn.Del(ctx, toDel...); err != nil { + if _, err := txn.Del(ctx, toDel...); err != nil { return err } start = rows[len(rows)-1].Key.Next() diff --git a/pkg/sql/temporary_schema.go b/pkg/sql/temporary_schema.go index 8d33f94fc80b..43ec8e920e58 100644 --- a/pkg/sql/temporary_schema.go +++ b/pkg/sql/temporary_schema.go @@ -206,7 +206,7 @@ func cleanupSessionTempObjects( // dropped). So we remove the namespace table entry of the temporary // schema. key := catalogkeys.MakeSchemaNameKey(codec, dbDesc.GetID(), tempSchemaName) - if err := txn.Del(ctx, key); err != nil { + if _, err := txn.Del(ctx, key); err != nil { return err } } diff --git a/pkg/sql/type_change_test.go b/pkg/sql/type_change_test.go index e726f5606a19..035435b1f50b 100644 --- a/pkg/sql/type_change_test.go +++ b/pkg/sql/type_change_test.go @@ -62,7 +62,7 @@ CREATE TYPE d.t AS ENUM(); desc := desctestutils.TestingGetPublicTypeDescriptor(kvDB, keys.SystemSQLCodec, "d", "t") delTypeDesc = func() { // Delete the descriptor. - if err := kvDB.Del(ctx, catalogkeys.MakeDescMetadataKey(keys.SystemSQLCodec, desc.GetID())); err != nil { + if _, err := kvDB.Del(ctx, catalogkeys.MakeDescMetadataKey(keys.SystemSQLCodec, desc.GetID())); err != nil { t.Error(err) } } 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 68dc1e98a509..c771f97d520e 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,14 +1336,20 @@ func MVCCDelete( timestamp hlc.Timestamp, localTimestamp hlc.ClockTimestamp, txn *roachpb.Transaction, -) error { +) (foundKey bool, err error) { iter := newMVCCIterator(rw, timestamp, false /* rangeKeyMasking */, IterOptions{ KeyTypes: IterKeyTypePointsAndRanges, Prefix: true, }) defer iter.Close() - return mvccPutUsingIter(ctx, rw, iter, ms, key, timestamp, localTimestamp, noValue, txn, nil) + // TODO(yuzefovich): can we avoid the actual put if foundKey is false? + valueFn := func(value optionalValue) (roachpb.Value, error) { + foundKey = value.IsPresent() + return noValue, nil + } + err = mvccPutUsingIter(ctx, rw, iter, ms, key, timestamp, localTimestamp, noValue, txn, valueFn) + return foundKey, err } var noValue = roachpb.Value{} diff --git a/pkg/storage/mvcc_history_test.go b/pkg/storage/mvcc_history_test.go index 3efde0013836..e49fd9911509 100644 --- a/pkg/storage/mvcc_history_test.go +++ b/pkg/storage/mvcc_history_test.go @@ -1003,9 +1003,11 @@ func cmdDelete(e *evalCtx) error { localTs := hlc.ClockTimestamp(e.getTsWithName("localTs")) resolve, resolveStatus := e.getResolve() return e.withWriter("del", func(rw ReadWriter) error { - if err := MVCCDelete(e.ctx, rw, e.ms, key, ts, localTs, txn); err != nil { + 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 c7498f3e8304..47772257bddf 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 -> / @@ -34,3 +35,52 @@ with t=A ---- get: "a" -> / @44.000000000,0 >> at end: + +## Check delete with returnKey option. + +# Put a value. + +run ok +with ts=46 v=abc + put k=a +---- +>> at end: +data: "a"/46.000000000,0 -> /BYTES/abc +data: "a"/44.000000000,0 -> / + +# Delete the key and check that the correct key was found. + +run ok +del k=a ts=47 +---- +del: "a": found key true +>> at end: +data: "a"/47.000000000,0 -> / +data: "a"/46.000000000,0 -> /BYTES/abc +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 +---- +del: "a": found key false +>> at end: +data: "a"/48.000000000,0 -> / +data: "a"/47.000000000,0 -> / +data: "a"/46.000000000,0 -> /BYTES/abc +data: "a"/44.000000000,0 -> / + +# Try deleting a key that never existed and ensure that the value is still +# written. + +run ok +del k=b ts=49 +---- +del: "b": found key false +>> at end: +data: "a"/48.000000000,0 -> / +data: "a"/47.000000000,0 -> / +data: "a"/46.000000000,0 -> /BYTES/abc +data: "a"/44.000000000,0 -> / +data: "b"/49.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 -> / diff --git a/pkg/upgrade/upgrades/role_id_migration_test.go b/pkg/upgrade/upgrades/role_id_migration_test.go index 75993e94d99d..030f730d56bc 100644 --- a/pkg/upgrade/upgrades/role_id_migration_test.go +++ b/pkg/upgrade/upgrades/role_id_migration_test.go @@ -70,9 +70,9 @@ func runTestRoleIDMigration(t *testing.T, numUsers int) { tdb.Exec(t, `DROP SEQUENCE system.role_id_seq`) tdb.Exec(t, `REVOKE node FROM root`) - err := tc.Servers[0].DB().Del(ctx, catalogkeys.MakeDescMetadataKey(keys.SystemSQLCodec, keys.RoleIDSequenceID)) + _, err := tc.Servers[0].DB().Del(ctx, catalogkeys.MakeDescMetadataKey(keys.SystemSQLCodec, keys.RoleIDSequenceID)) require.NoError(t, err) - err = tc.Servers[0].DB().Del(ctx, keys.SystemSQLCodec.SequenceKey(uint32(keys.RoleIDSequenceID))) + _, err = tc.Servers[0].DB().Del(ctx, keys.SystemSQLCodec.SequenceKey(uint32(keys.RoleIDSequenceID))) require.NoError(t, err) // Remove entries from system.users.