Skip to content

Commit

Permalink
WIP on updating MVCCDelete
Browse files Browse the repository at this point in the history
This commit makes it so that `MVCCDelete` as well as `DeleteRequest`
always return whether the key was actually deleted. If approved, it'll
be squashed into the previous commit.
  • Loading branch information
yuzefovich committed Aug 8, 2022
1 parent b281022 commit acf1184
Show file tree
Hide file tree
Showing 51 changed files with 276 additions and 146 deletions.
2 changes: 1 addition & 1 deletion pkg/cli/debug.go
Original file line number Diff line number Diff line change
Expand Up @@ -1400,7 +1400,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{
Expand Down
4 changes: 2 additions & 2 deletions pkg/clusterversion/cockroach_versions.go
Original file line number Diff line number Diff line change
Expand Up @@ -337,8 +337,8 @@ const (
UsersHaveIDs
// SetUserIDNotNull sets the user_id column in system.users to not null.
SetUserIDNotNull
// DeleteRequestReturnKey is the version where the ReturnKey argument of the
// DeleteRequest is introduced.
// DeleteRequestReturnKey is the version where the DeleteRequest began
// populating the FoundKey value in the response.
DeleteRequestReturnKey

// *************************************************
Expand Down
22 changes: 9 additions & 13 deletions pkg/kv/batch.go
Original file line number Diff line number Diff line change
Expand Up @@ -242,12 +242,9 @@ func (b *Batch) fillResults(ctx context.Context) {
}
case *roachpb.DeleteRequest:
if result.Err == nil {
deleteRequest := args.(*roachpb.DeleteRequest)
if deleteRequest.ReturnKey {
resp := reply.(*roachpb.DeleteResponse)
if resp.FoundKey {
result.Keys = []roachpb.Key{deleteRequest.Key}
}
resp := reply.(*roachpb.DeleteResponse)
if resp.FoundKey {
result.Keys = []roachpb.Key{args.(*roachpb.DeleteRequest).Key}
}
}
case *roachpb.DeleteRangeRequest:
Expand Down Expand Up @@ -628,7 +625,7 @@ func (b *Batch) Del(keys ...interface{}) {
b.initResult(0, len(keys), notRaw, err)
return
}
reqs = append(reqs, roachpb.NewDelete(k, false /* returnKey */))
reqs = append(reqs, roachpb.NewDelete(k))
b.approxMutationReqBytes += len(k)
}
b.appendReqs(reqs...)
Expand All @@ -638,19 +635,18 @@ func (b *Batch) Del(keys ...interface{}) {
// DelKey deletes one key.
//
// A new result will be appended to the batch which will contain 0 rows and
// Result.Err will indicate success or failure. The returnKey argument
// determines whether the key is included in Result.Keys to indicate that the
// delete actually deleted something.
// Result.Err will indicate success or failure. The key will be included in
// Result.Keys if it was actually deleted.
//
// key can be either a byte slice or a string.
func (b *Batch) DelKey(key interface{}, returnKey bool) {
func (b *Batch) DelKey(key interface{}) {
k, err := marshalKey(key)
if err != nil {
b.initResult(0, 1, notRaw, err)
b.initResult(0, 0, notRaw, err)
return
}
b.approxMutationReqBytes += len(k)
b.appendReqs(roachpb.NewDelete(k, returnKey))
b.appendReqs(roachpb.NewDelete(k))
b.initResult(1, 0, notRaw, nil)
}

Expand Down
15 changes: 7 additions & 8 deletions pkg/kv/db.go
Original file line number Diff line number Diff line change
Expand Up @@ -535,19 +535,18 @@ func (db *DB) Del(ctx context.Context, keys ...interface{}) error {

// DelKey deletes one key.
//
// The returned roachpb.Key will contain the key deleted if the returnKey
// parameter is true and the key was actually deleted, or will be nil if the
// parameter is false of the key was already non-existent.
// The returned roachpb.Key will contain the key if it was actually deleted.
//
// key can be either a byte slice or a string.
func (db *DB) DelKey(ctx context.Context, key interface{}, returnKey bool) (roachpb.Key, error) {
func (db *DB) DelKey(ctx context.Context, key interface{}) (roachpb.Key, error) {
b := &Batch{}
b.DelKey(key, returnKey)
b.DelKey(key)
r, err := getOneResult(db.Run(ctx, b), b)
if returnKey {
return r.Keys[0], err
var deletedKey roachpb.Key
if len(r.Keys) > 0 {
deletedKey = r.Keys[0]
}
return roachpb.Key{}, err
return deletedKey, err
}

// DelRange deletes the rows between begin (inclusive) and end (exclusive).
Expand Down
60 changes: 25 additions & 35 deletions pkg/kv/db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -480,47 +480,37 @@ func TestDB_Del(t *testing.T) {
}

for _, useDelKey := range []bool{false, true} {
for _, returnKey := range []bool{false, true} {
if !useDelKey && returnKey {
// Del doesn't have a returnKey argument.
continue
}
if err := db.Put(context.Background(), "ab", "2"); err != nil {
if err := db.Put(context.Background(), "ab", "2"); err != nil {
t.Fatal(err)
}
if useDelKey {
key, err := db.DelKey(context.Background(), "ab")
if err != nil {
t.Fatal(err)
}
if useDelKey {
if key, err := db.DelKey(context.Background(), "ab", returnKey); err != nil {
t.Fatal(err)
} else if returnKey {
checkKeys(t, []string{"ab"}, []roachpb.Key{key})
// Also try deleting a non-existent key and verify that no
// key is returned.
if key, err = db.DelKey(context.Background(), "ad", returnKey); err != nil {
t.Fatal(err)
} else if len(key) > 0 {
t.Errorf("expected deleted key to be empty when deleting a non-existent key, got %v", key)
}
} else {
if len(key) > 0 {
t.Errorf("expected deleted key to be empty when returnKeys set to false, got %v", key)
}
}
} else {
if err := db.Del(context.Background(), "ab"); err != nil {
t.Fatal(err)
}
}
rows, err := db.Scan(context.Background(), "a", "b", 100)
if err != nil {
checkKeys(t, []string{"ab"}, []roachpb.Key{key})
// Also try deleting a non-existent key and verify that no key is
// returned.
if key, err = db.DelKey(context.Background(), "ad"); err != nil {
t.Fatal(err)
} else if len(key) > 0 {
t.Errorf("expected deleted key to be empty when deleting a non-existent key, got %v", key)
}
expected := map[string][]byte{
"aa": []byte("1"),
"ac": []byte("3"),
} else {
if err := db.Del(context.Background(), "ab"); err != nil {
t.Fatal(err)
}
checkRows(t, expected, rows)
checkLen(t, len(expected), len(rows))
}
rows, err := db.Scan(context.Background(), "a", "b", 100)
if err != nil {
t.Fatal(err)
}
expected := map[string][]byte{
"aa": []byte("1"),
"ac": []byte("3"),
}
checkRows(t, expected, rows)
checkLen(t, len(expected), len(rows))
}
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvnemesis/doc.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@
//
// TODO
// - CPut/InitPut/Increment
// - Delete (with and without ReturnKey)
// - DelKey
// - ClearRange/RevertRange
// - AdminRelocateRange
// - AdminUnsplit
Expand Down
3 changes: 2 additions & 1 deletion pkg/kv/kvserver/abortspan/abortspan.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down
12 changes: 4 additions & 8 deletions pkg/kv/kvserver/batcheval/cmd_delete.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,14 +30,10 @@ func Delete(
h := cArgs.Header

var err error
if args.ReturnKey {
reply := resp.(*roachpb.DeleteResponse)
reply.FoundKey, err = storage.MVCCDeleteReturningExistence(
ctx, readWriter, cArgs.Stats, args.Key, h.Timestamp, cArgs.Now, h.Txn,
)
} else {
err = storage.MVCCDelete(ctx, readWriter, cArgs.Stats, args.Key, h.Timestamp, cArgs.Now, h.Txn)
}
reply := resp.(*roachpb.DeleteResponse)
reply.FoundKey, err = storage.MVCCDelete(
ctx, readWriter, cArgs.Stats, args.Key, h.Timestamp, cArgs.Now, h.Txn,
)
// NB: even if MVCC returns an error, it may still have written an intent
// into the batch. This allows callers to consume errors like WriteTooOld
// without re-evaluating the batch. This behavior isn't particularly
Expand Down
3 changes: 2 additions & 1 deletion pkg/kv/kvserver/batcheval/cmd_delete_range_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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))
require.NoError(t, storage.MVCCDeleteRangeUsingTombstone(ctx, rw, nil, roachpb.Key("Z"), roachpb.Key("a"), hlc.Timestamp{WallTime: 100e9}, localTS, nil, nil, 0))
Expand Down
3 changes: 2 additions & 1 deletion pkg/kv/kvserver/batcheval/cmd_end_transaction.go
Original file line number Diff line number Diff line change
Expand Up @@ -628,7 +628,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
Expand Down
3 changes: 2 additions & 1 deletion pkg/kv/kvserver/batcheval/cmd_query_intent_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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.)
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/batcheval/intent_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/loqrecovery/apply.go
Original file line number Diff line number Diff line change
Expand Up @@ -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{
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/replica_raftstorage.go
Original file line number Diff line number Diff line change
Expand Up @@ -700,7 +700,7 @@ func (r *Replica) append(
for i := lastIndex + 1; i <= prevLastIndex; i++ {
// Note that the caller is in charge of deleting any sideloaded payloads
// (which they must only do *after* the batch has committed).
err := storage.MVCCDelete(ctx, eng, &diff, keys.RaftLogKeyFromPrefix(prefix, i),
_, err := storage.MVCCDelete(ctx, eng, &diff, keys.RaftLogKeyFromPrefix(prefix, i),
hlc.Timestamp{}, hlc.ClockTimestamp{}, nil)
if err != nil {
return 0, 0, 0, err
Expand Down
3 changes: 1 addition & 2 deletions pkg/roachpb/api.go
Original file line number Diff line number Diff line change
Expand Up @@ -1194,12 +1194,11 @@ func NewInitPut(key Key, value Value, failOnTombstones bool) Request {
}

// NewDelete returns a Request initialized to delete the value at key.
func NewDelete(key Key, returnKey bool) Request {
func NewDelete(key Key) Request {
return &DeleteRequest{
RequestHeader: RequestHeader{
Key: key,
},
ReturnKey: returnKey,
}
}

Expand Down
4 changes: 1 addition & 3 deletions pkg/roachpb/api.proto
Original file line number Diff line number Diff line change
Expand Up @@ -319,14 +319,12 @@ message IncrementResponse {
// A DeleteRequest is the argument to the Delete() method.
message DeleteRequest {
RequestHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true];
// return the key that is deleted in the response.
bool return_key = 2;
}

// A DeleteResponse is the return value from the Delete() method.
message DeleteResponse {
ResponseHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true];
// True if return_key was set and there was a key that got deleted.
// True if there was a key that got deleted.
bool found_key = 2;
}

Expand Down
6 changes: 3 additions & 3 deletions pkg/sql/delete_range.go
Original file line number Diff line number Diff line change
Expand Up @@ -90,8 +90,8 @@ func (d *deleteRangeNode) startExec(params runParams) error {
}

// Configure the fetcher, which is only used to decode the returned keys
// from the DeleteKey and the DeleteRange, and is never used to actually
// fetch kvs.
// from the DelKey and the DelRange operations, and is never used to
// actually fetch kvs.
var spec descpb.IndexFetchSpec
if err := rowenc.InitIndexFetchSpec(
&spec, params.ExecCfg().Codec, d.desc, d.desc.GetPrimaryIndex(), nil, /* columnIDs */
Expand Down Expand Up @@ -172,7 +172,7 @@ func (d *deleteRangeNode) deleteSpans(params runParams, b *kv.Batch, spans roach
if traceKV {
log.VEventf(ctx, 2, "Del %s", span.Key)
}
b.DelKey(span.Key, true /* returnKey */)
b.DelKey(span.Key)
} else {
if traceKV {
log.VEventf(ctx, 2, "DelRange %s - %s", span.Key, span.EndKey)
Expand Down
2 changes: 1 addition & 1 deletion pkg/storage/bench_pebble_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -300,7 +300,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)
}
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/storage/engine_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/storage/metamorphic/operations.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
Loading

0 comments on commit acf1184

Please sign in to comment.