Skip to content

Commit

Permalink
sql: emit point deletes during delete fastpath
Browse files Browse the repository at this point in the history
Previously, the "deleteRange" SQL operator, which is meant to be a
fast-path for cases in which an entire range of keys can be deleted,
always did what it said: emitted DeleteRange KV operations. This
precludes a crucial optimization: sending point deletes when the list of
deleted keys is exactly known.

For example, a query like `DELETE FROM kv WHERE k = 10000` uses the
"fast path" delete, since it has a contiguous set of keys to delete, and
it doesn't need to know the values that were deleted. But, in this case,
the performance is actually worse if we use a DeleteRange KV operation
for various reasons (see #53939), because:

- ranged KV writes (DeleteRangeRequest) cannot be pipelined because an
  enumeration of the intents that they will leave cannot be known ahead
  of time. They must therefore perform evaluation and replication synchronously.
- ranged KV writes (DeleteRangeRequest) result in ranged intent
  resolution, which is less efficient, especially until we re-enable
  time-bound iterators.

The reason we couldn't previously emit point deletes in this case is
that SQL needs to know whether it deleted something or not. This means
we can't do a "blind put" of a deletion: we need to actually understand
whether there was something that we were "overwriting" with our delete.

This commit adds an optional "return key" parameter to DelRequest, which
returns true if a value was actually deleted.

Additionally, the deleteRange SQL operator detects when it can emit
single-key deletes, and does so.

Release note (performance improvement): point deletes in SQL are now
more efficient during concurrent workloads.
  • Loading branch information
jordanlewis committed Apr 11, 2021
1 parent a39716b commit 2346f65
Show file tree
Hide file tree
Showing 16 changed files with 730 additions and 562 deletions.
23 changes: 20 additions & 3 deletions pkg/kv/batch.go
Original file line number Diff line number Diff line change
Expand Up @@ -226,8 +226,13 @@ func (b *Batch) fillResults(ctx context.Context) {
}
}
case *roachpb.DeleteRequest:
row := &result.Rows[k]
row.Key = []byte(args.(*roachpb.DeleteRequest).Key)
deleteRequest := args.(*roachpb.DeleteRequest)
if deleteRequest.ReturnKey && result.Err == nil {
resp := reply.(*roachpb.DeleteResponse)
if resp.FoundKey {
result.Keys = []roachpb.Key{deleteRequest.Key}
}
}
case *roachpb.DeleteRangeRequest:
if result.Err == nil {
result.Keys = reply.(*roachpb.DeleteRangeResponse).Keys
Expand Down Expand Up @@ -612,12 +617,24 @@ func (b *Batch) Del(keys ...interface{}) {
b.initResult(0, len(keys), notRaw, err)
return
}
reqs = append(reqs, roachpb.NewDelete(k))
reqs = append(reqs, roachpb.NewDelete(k, false /* returnKey */))
}
b.appendReqs(reqs...)
b.initResult(len(reqs), len(reqs), notRaw, nil)
}

// DelKey is like Del but it takes a single key, and allows choosing whether or
// not to return whether a key was deleted via the command.
func (b *Batch) DelKey(key interface{}, returnKey bool) {
k, err := marshalKey(key)
if err != nil {
b.initResult(0, 1, notRaw, err)
return
}
b.appendReqs(roachpb.NewDelete(k, returnKey))
b.initResult(0, 1, notRaw, err)
}

// DelRange deletes the rows between begin (inclusive) and end (exclusive).
//
// A new result will be appended to the batch which will contain 0 rows and
Expand Down
3 changes: 2 additions & 1 deletion pkg/kv/kvclient/kvcoord/dist_sender.go
Original file line number Diff line number Diff line change
Expand Up @@ -604,7 +604,8 @@ func (ds *DistSender) initAndVerifyBatch(
inner := req.GetInner()
switch inner.(type) {
case *roachpb.ScanRequest, *roachpb.ResolveIntentRangeRequest,
*roachpb.DeleteRangeRequest, *roachpb.RevertRangeRequest, *roachpb.ExportRequest:
*roachpb.DeleteRequest, *roachpb.DeleteRangeRequest,
*roachpb.RevertRangeRequest, *roachpb.ExportRequest:
// Accepted forward range requests.
if isReverse {
return roachpb.NewErrorf("batch with limit contains both forward and reverse scans")
Expand Down
13 changes: 12 additions & 1 deletion pkg/kv/kvserver/batcheval/cmd_delete.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,18 @@ func Delete(
args := cArgs.Args.(*roachpb.DeleteRequest)
h := cArgs.Header

err := storage.MVCCDelete(ctx, readWriter, cArgs.Stats, args.Key, h.Timestamp, h.Txn)
var err error
if args.ReturnKey {
reply := resp.(*roachpb.DeleteResponse)
reply.FoundKey, err = storage.MVCCDeleteReturningExistence(
ctx, readWriter, cArgs.Stats, args.Key, h.Timestamp, h.Txn)
if err != nil {
return result.Result{}, err
}
} else {
err = storage.MVCCDelete(ctx, readWriter, cArgs.Stats, args.Key, h.Timestamp, 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/roachpb/api.go
Original file line number Diff line number Diff line change
Expand Up @@ -1074,11 +1074,12 @@ func NewInitPut(key Key, value Value, failOnTombstones bool) Request {
}

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

Expand Down
Loading

0 comments on commit 2346f65

Please sign in to comment.