-
Notifications
You must be signed in to change notification settings - Fork 3.8k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
sql: emit point deletes during delete fastpath #63416
Conversation
1652475
to
2346f65
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Very nice!
I believe this should improve TPC-C's Delivery transaction, which issues the following query:
DELETE FROM new_order WHERE no_w_id = %d AND (no_d_id, no_o_id) IN (%s)
We've seen this statement cause the pipeline stalls you referenced in the commit message.
We have logic tests that exercise this statement, but they aren't showing a change in behavior. Is this because the logictests are showing the optimizer's representation before the CanSplitSpanIntoFamilySpans
transformation is applied? Or will this change not yet help with DELETE ... IN ...
statements?
Also, do you mind making the following change to pkg/kv/kvnemesis/doc.go
:
// guarantees.
//
// TODO
-// - CPut/InitPut/Increment/Delete
+// - CPut/InitPut/Increment
+// - Delete (with and without ReturnKey)
// - DeleteRange/ClearRange/RevertRange/ReverseScan
// - AdminRelocateRange
// - AdminUnsplit
Reviewed 16 of 16 files at r4.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @jordanlewis)
pkg/kv/batch.go, line 229 at r4 (raw file):
} case *roachpb.DeleteRequest: deleteRequest := args.(*roachpb.DeleteRequest)
I think we need to update the comments on kv.Result
's fields to reflect this change.
pkg/kv/batch.go, line 230 at r4 (raw file):
case *roachpb.DeleteRequest: deleteRequest := args.(*roachpb.DeleteRequest) if deleteRequest.ReturnKey && result.Err == nil {
nit: I'd pull the if result.Err == nil {
out front to mirror the other cases around here.
pkg/kv/batch.go, line 626 at r4 (raw file):
} // DelKey is like Del but it takes a single key, and allows choosing whether or
We'll want to add testing for this. Maybe expose it through the DB
object and then extend TestDB_Del
.
pkg/kv/kvserver/batcheval/cmd_delete.go, line 37 at r4 (raw file):
reply.FoundKey, err = storage.MVCCDeleteReturningExistence( ctx, readWriter, cArgs.Stats, args.Key, h.Timestamp, h.Txn) if err != nil {
Do we want this early return? We don't have it on the other branch.
pkg/roachpb/api.proto, line 268 at r4 (raw file):
RequestHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; // return the key that is deleted in the response.
We're going to need to introduce a new cluster version to guard against the use of this field before we have a guarantee that all kv nodes are sufficiently up to date.
This reminds me, we don't need to do the same for #61583, do we? We didn't extend the KV API, but we did begin properly accounting for kv and byte limits on GetRequests.
pkg/roachpb/api.proto, line 269 at r4 (raw file):
// return the key that is deleted in the response. bool return_key = 3;
= 2
?
pkg/storage/mvcc.go, line 1176 at r4 (raw file):
txn *roachpb.Transaction, ) error { iter := newMVCCIterator(rw, timestamp.IsEmpty(), IterOptions{Prefix: true})
Let's define this in terms of MVCCDeleteReturningExistence
to avoid redundancy:
_, err := MVCCDeleteReturningExistence(ctx, rw, ms, key, timestamp, txn)
return err
pkg/storage/mvcc.go, line 1184 at r4 (raw file):
// MVCCDeleteReturningExistence is like MVCCDelete, but it returns whether the key // that was passed in had a value already in the database. func MVCCDeleteReturningExistence(
We'll want to add some testing for this. I think we can do so by adding a new returnKey
option to the del
operator in TestMVCCHistories
and then extending pkg/storage/testdata/mvcc_histories/deletes
.
pkg/storage/mvcc.go, line 1197 at r4 (raw file):
valueFn := func(value optionalValue) ([]byte, error) { foundKey = len(value.RawBytes) > 0 return nil, nil
nit: we make this a little more explicit. return noValue.RawBytes, 0
2346f65
to
7b833ae
Compare
Thanks for the review; I really want to come back to this and figure out how to get it over the finish line. |
7b833ae
to
faefeaa
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I rebased and addressed all of the feedback. I hope we can get this in before the stability period, so @nvanbenschoten and @jordanlewis please give it a look :)
We have logic tests that exercise this statement, but they aren't showing a change in behavior. Is this because the logictests are showing the optimizer's representation before the
CanSplitSpanIntoFamilySpans
transformation is applied?
Yes, I believe so. The spans are only split during the execbuilding stage, and the tests you linked are transformation tests which don't get to the execbuilding. The actual logic tests do show the correct usage of Del operation when expected.
Also, do you mind making the following change to pkg/kv/kvnemesis/doc.go
Done.
Reviewed 2 of 4 files at r1, 15 of 38 files at r2, 13 of 15 files at r3, 31 of 31 files at r6.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @nvanbenschoten)
pkg/kv/batch.go
line 229 at r4 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
I think we need to update the comments on
kv.Result
's fields to reflect this change.
Done.
pkg/kv/batch.go
line 230 at r4 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
nit: I'd pull the
if result.Err == nil {
out front to mirror the other cases around here.
Done.
pkg/kv/batch.go
line 626 at r4 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
We'll want to add testing for this. Maybe expose it through the
DB
object and then extendTestDB_Del
.
Done.
pkg/kv/kvserver/batcheval/cmd_delete.go
line 37 at r4 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Do we want this early return? We don't have it on the other branch.
Removed.
pkg/roachpb/api.proto
line 268 at r4 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
We're going to need to introduce a new cluster version to guard against the use of this field before we have a guarantee that all kv nodes are sufficiently up to date.
This reminds me, we don't need to do the same for #61583, do we? We didn't extend the KV API, but we did begin properly accounting for kv and byte limits on GetRequests.
Done.
pkg/roachpb/api.proto
line 269 at r4 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
= 2
?
Done.
pkg/storage/mvcc.go
line 1176 at r4 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Let's define this in terms of
MVCCDeleteReturningExistence
to avoid redundancy:_, err := MVCCDeleteReturningExistence(ctx, rw, ms, key, timestamp, txn) return err
Done.
pkg/storage/mvcc.go
line 1184 at r4 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
We'll want to add some testing for this. I think we can do so by adding a new
returnKey
option to thedel
operator inTestMVCCHistories
and then extendingpkg/storage/testdata/mvcc_histories/deletes
.
Done.
pkg/storage/mvcc.go
line 1197 at r4 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
nit: we make this a little more explicit.
return noValue.RawBytes, 0
Done.
Thanks @yuzefovich, this is great. Is this related to the TPC-E experiment from @nvanbenschoten? |
No - I looked over the old open issues from the execution dashboard and found that this PR was still open. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @jordanlewis and @nvanbenschoten)
pkg/storage/mvcc.go
line 1348 at r6 (raw file):
valueFn := func(value optionalValue) (roachpb.Value, error) { foundKey = len(value.RawBytes) > 0
if foundKey
is false, can we avoid doing an actual put? Or is the expectation that this will be rare enough that we don't need to optimize?
Drive-by bike shed: could we have |
faefeaa
to
0f78699
Compare
0f78699
to
acf1184
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I like this suggestion, done. However, there were quite a few places to change (mostly tests though), so I kept those changes in a separate commit. It'll be squashed if we like it.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @nvanbenschoten, @sumeerbhola, and @yuzefovich)
pkg/storage/mvcc.go
line 1348 at r6 (raw file):
Previously, sumeerbhola wrote…
if
foundKey
is false, can we avoid doing an actual put? Or is the expectation that this will be rare enough that we don't need to optimize?
This is outside of the scope of this PR - it is an already present semantics of MVCCDelete
to use the put here. I left a TODO.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @nvanbenschoten, @sumeerbhola, and @yuzefovich)
pkg/storage/mvcc.go
line 1348 at r6 (raw file):
Previously, yuzefovich (Yahor Yuzefovich) wrote…
This is outside of the scope of this question - it is an already present semantics of
MVCCDelete
to use the put here. I left a TODO.
Ack
3397935
to
d9bfcfc
Compare
I'm hoping to land it before the stability period. @nvanbenschoten will you have time to review this this week? Maybe @erikgrinaker or @sumeerbhola could give this a proper review? @cockroachdb/sql-queries can someone volunteer to review this too? |
I'm completely swamped before stability, so would appreciate it if someone else had the bandwidth. |
I can give this a review in the next day or so. One thing I'll focus on is whether the behavior change of a DELETE on a non-existent row is acceptable. This PR will turn such a statement from a read to a read-write. Related to @sumeerbhola's question, it's not straightforward to avoid this change in behavior. A @andreimatei and I talked through #64723 + #53939 extensively at some point and I don't recall there being a single solution that was optimal in all cases. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks @yuzefovich, this is great. Is this related to the TPC-E experiment from @nvanbenschoten?
I think this actually comes up in TPC-C. See the query in #53939.
It should also help with TPC-E's TradeResult
and MarketFeed
transactions.
Reviewed 1 of 3 files at r7, 4 of 55 files at r10, 53 of 53 files at r11.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @jordanlewis, @sumeerbhola, and @yuzefovich)
pkg/kv/batch.go
line 635 at r11 (raw file):
} // DelKey deletes one key.
Could you remind me why we wanted to add this new method? What does it do that Del
does not? Is it to avoid a few intermediate heap allocations?
pkg/kv/db.go
line 132 at r11 (raw file):
// 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 DelKey and DelRange Rows is nil.
Isn't this also the case for Del
? This relates to the question of why we need DelKey at all.
pkg/kv/db_test.go
line 482 at r11 (raw file):
} for _, useDelKey := range []bool{false, true} {
nit: you could use testutils.RunTrueAndFalse
here.
pkg/kv/kvserver/batcheval/cmd_delete.go
line 33 at r11 (raw file):
var err error reply := resp.(*roachpb.DeleteResponse)
minor nit: it's common in these batcheval functions for the reply := resp.(...)
type assertion to come right after the h := cArgs.Header
line.
pkg/roachpb/api.proto
line 327 at r11 (raw file):
message DeleteResponse { ResponseHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; // True if there was a key that got deleted.
Let's add here that a tombstone is written unconditionally, regardless of whether a key is found.
pkg/storage/mvcc.go
line 1348 at r6 (raw file):
Previously, sumeerbhola wrote…
Ack
We can't skip the intent write if we want Delete
requests to be pipelined or to be part of parallel commits. These two optimizations require the write set of a batch to be known in advance and for successful completion of the batch to be recorded in an intent that can later be queried. #64723 (comment) could lift this requirement for pre-commit pipelining, but it would still not allow for conditional writes during a parallel commit (used during auto-commit stmts).
It's worth pointing out the three downsides of Delete
needing to be unconditional:
- we must perform an intent write and resolve that intent after the txn commits
- we must leave a useless MVCC tombstone for MVCC GC to clean up
- we might confuse CDC streams with useless MVCC tombstones
We may be able to do something about the second and third downside, even if we can't entirely address the first. The idea here would be to lean on our decomposition of intents into locks and provisional values. We could satisfy the needs of parallel commits by leaving just a replicated lock to mark the completion of a Delete
and omitting the corresponding provisional value write. This would confuse various places in the code, so it wouldn't be a trivial change, but it feels possible longer-term.
For this PR, I think we need to answer the following two questions:
- are there important workloads where this will cause a performance regression? I doubt it, but we need to ask the question.
- are we ok with the observable impact this has on changefeed semantics?
pkg/storage/mvcc.go
line 1348 at r11 (raw file):
// TODO(yuzefovich): can we avoid the actual put if foundKey is false? valueFn := func(value optionalValue) (roachpb.Value, error) { foundKey = len(value.RawBytes) > 0
foundKey = value.IsPresent()
pkg/storage/testdata/mvcc_histories/deletes
line 65 at r11 (raw file):
run ok del k=a ts=48
Could we also delete a non-existent key (no versions, tombstone or otherwise) and show that a value is still written?
d9bfcfc
to
b5ebc16
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for the review Nathan! Since you didn't say you were against the change to MVCCDelete
to return whether a key was found, I squashed two commits into one.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @nvanbenschoten, @sumeerbhola, and @yuzefovich)
pkg/kv/batch.go
line 635 at r11 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Could you remind me why we wanted to add this new method? What does it do that
Del
does not? Is it to avoid a few intermediate heap allocations?
I'm not sure why we chose this way - probably to resemble what we do in DelRange
. It does look like we can easily reuse the existing Del
method at the cost of few extra allocations, so if you think it's worth it, I'm happy to make the change - I don't have strong feelings either way.
pkg/kv/db.go
line 132 at r11 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Isn't this also the case for
Del
? This relates to the question of why we need DelKey at all.
For Del
we do initialize Rows
slice according to the number of keys to delete, but then we never modify that slice. This is because of initResults()
call where the second argument is numRows == len(reqs)
.
pkg/kv/db_test.go
line 482 at r11 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
nit: you could use
testutils.RunTrueAndFalse
here.
Done.
pkg/kv/kvserver/batcheval/cmd_delete.go
line 33 at r11 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
minor nit: it's common in these batcheval functions for the
reply := resp.(...)
type assertion to come right after theh := cArgs.Header
line.
Done.
pkg/roachpb/api.proto
line 327 at r11 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Let's add here that a tombstone is written unconditionally, regardless of whether a key is found.
Done.
pkg/storage/mvcc.go
line 1348 at r6 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
We can't skip the intent write if we want
Delete
requests to be pipelined or to be part of parallel commits. These two optimizations require the write set of a batch to be known in advance and for successful completion of the batch to be recorded in an intent that can later be queried. #64723 (comment) could lift this requirement for pre-commit pipelining, but it would still not allow for conditional writes during a parallel commit (used during auto-commit stmts).It's worth pointing out the three downsides of
Delete
needing to be unconditional:
- we must perform an intent write and resolve that intent after the txn commits
- we must leave a useless MVCC tombstone for MVCC GC to clean up
- we might confuse CDC streams with useless MVCC tombstones
We may be able to do something about the second and third downside, even if we can't entirely address the first. The idea here would be to lean on our decomposition of intents into locks and provisional values. We could satisfy the needs of parallel commits by leaving just a replicated lock to mark the completion of a
Delete
and omitting the corresponding provisional value write. This would confuse various places in the code, so it wouldn't be a trivial change, but it feels possible longer-term.For this PR, I think we need to answer the following two questions:
- are there important workloads where this will cause a performance regression? I doubt it, but we need to ask the question.
- are we ok with the observable impact this has on changefeed semantics?
Thanks for the context and explanation, Nathan. I'll defer to you on the first question, but for the second question it seems the answer is "yes" given this comment.
pkg/storage/mvcc.go
line 1348 at r11 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
foundKey = value.IsPresent()
Done.
pkg/storage/testdata/mvcc_histories/deletes
line 65 at r11 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Could we also delete a non-existent key (no versions, tombstone or otherwise) and show that a value is still written?
Done.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewed 13 of 13 files at r12.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @sumeerbhola and @yuzefovich)
pkg/kv/batch.go
line 635 at r11 (raw file):
Previously, yuzefovich (Yahor Yuzefovich) wrote…
I'm not sure why we chose this way - probably to resemble what we do in
DelRange
. It does look like we can easily reuse the existingDel
method at the cost of few extra allocations, so if you think it's worth it, I'm happy to make the change - I don't have strong feelings either way.
Yeah, I would suggest getting rid of this if it isn't meaningfully different from Del
. Otherwise, each person who reads this needs to independently determine which one to use and reverse engineer the fact that they're roughly the same thing.
pkg/kv/db.go
line 132 at r11 (raw file):
Previously, yuzefovich (Yahor Yuzefovich) wrote…
For
Del
we do initializeRows
slice according to the number of keys to delete, but then we never modify that slice. This is because ofinitResults()
call where the second argument isnumRows == len(reqs)
.
Now that we're populating Keys
in fillResults
for a DeleteRequest
, instead of Rows
, should we be doing that?
b5ebc16
to
6f223a8
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @nvanbenschoten, @sumeerbhola, and @yuzefovich)
pkg/kv/batch.go
line 635 at r11 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Yeah, I would suggest getting rid of this if it isn't meaningfully different from
Del
. Otherwise, each person who reads this needs to independently determine which one to use and reverse engineer the fact that they're roughly the same thing.
Done.
pkg/kv/db.go
line 132 at r11 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Now that we're populating
Keys
infillResults
for aDeleteRequest
, instead ofRows
, should we be doing that?
Done.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewed 25 of 25 files at r13.
Reviewable status: complete! 1 of 0 LGTMs obtained (waiting on @jordanlewis, @sumeerbhola, and @yuzefovich)
pkg/kv/txn.go
line 629 at r13 (raw file):
} // Del deletes one or more keys.
Could we add // The returned []roachpb.Key will contain the keys that were actually deleted.
here as well?
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 cockroachdb#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 (although this became less important since we re-enabled 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 modifies the DeleteResponse to always return a boolean indicating whether a key from the DeleteRequest 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.
6f223a8
to
00af7d7
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
TFTR!
bors r+
Reviewable status: complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @nvanbenschoten, @sumeerbhola, and @yuzefovich)
pkg/kv/txn.go
line 629 at r13 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Could we add
// The returned []roachpb.Key will contain the keys that were actually deleted.
here as well?
Done.
Build succeeded: |
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:
enumeration of the intents that they will leave cannot be known ahead
of time. They must therefore perform evaluation and replication
synchronously.
resolution, which is less efficient (although this became less
important since we re-enabled 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 modifies the DeleteResponse to always return a boolean
indicating whether a key from the DeleteRequest was actually deleted.
Additionally, the deleteRange SQL operator detects when it can emit
single-key deletes, and does so.
Closes #53939.
Release note (performance improvement): point deletes in SQL are now
more efficient during concurrent workloads.