-
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
storage: leave intents behind after blind-writes experiencing write-too-old #44654
storage: leave intents behind after blind-writes experiencing write-too-old #44654
Conversation
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.
everything but the last 3 commits is elsewhere
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @nvanbenschoten)
Before this patch, when the DistSender would split a batch into multiple sub-batches and one of the sub-batches fails, it would collect responses for the successful ones and return them together with the error. This used to be pretty important before we had write idempotency, because it allowed the span refresher to only retry an EndTxn without also retring other writes in that batch (which would have failed). Since we've gotten idempotency in the meantime, we can retry those other writes. In fact, it's arguably better to do it: there's a tradeoff between refreshing and retrying. Currently the span refresher needs to refresh the read spans of the successful sub-batches, which refresh is at risk of failing under contention. This patch makes the span refresher retry the whole batch without considering partial successes. With this patch, refreshing the partial successes is no longer needed because we'll retry those requests. In other words, we'll refresh less and retry more. The existing policy of refreshing more and retrying less will start to be applied inconsistenly with cockroachdb#44654, where we start refreshing when the client sees a WriteTooOld flag - but we're forced to refresh the whole batch. Besides the rationalizations above, this patch allows us to simplify code by not having to deal with both responses and errors. We can thus get rid of the enthralling comment on the client.Sender.Send() stating: " // The contract about whether both a response and an error can be // returned varies between layers. " Release note: None
Before this patch, when the DistSender would split a batch into multiple sub-batches and one of the sub-batches fails, it would collect responses for the successful ones and return them together with the error. This used to be pretty important before we had write idempotency, because it allowed the span refresher to only retry an EndTxn without also retring other writes in that batch (which would have failed). Since we've gotten idempotency in the meantime, we can retry those other writes. In fact, it's arguably better to do it: there's a tradeoff between refreshing and retrying. Currently the span refresher needs to refresh the read spans of the successful sub-batches, which refresh is at risk of failing under contention. This patch makes the span refresher retry the whole batch without considering partial successes. With this patch, refreshing the partial successes is no longer needed because we'll retry those requests. In other words, we'll refresh less and retry more. The existing policy of refreshing more and retrying less will start to be applied inconsistenly with cockroachdb#44654, where we start refreshing when the client sees a WriteTooOld flag - but we're forced to refresh the whole batch. Besides the rationalizations above, this patch allows us to simplify code by not having to deal with both responses and errors. We can thus get rid of the enthralling comment on the client.Sender.Send() stating: " // The contract about whether both a response and an error can be // returned varies between layers. " Release note: None
a1f649b
to
02c3c16
Compare
Before this patch, when the DistSender would split a batch into multiple sub-batches and one of the sub-batches fails, it would collect responses for the successful ones and return them together with the error. This used to be pretty important before we had write idempotency, because it allowed the span refresher to only retry an EndTxn without also retring other writes in that batch (which would have failed). Since we've gotten idempotency in the meantime, we can retry those other writes. In fact, it's arguably better to do it: there's a tradeoff between refreshing and retrying. Currently the span refresher needs to refresh the read spans of the successful sub-batches, which refresh is at risk of failing under contention. This patch makes the span refresher retry the whole batch without considering partial successes. With this patch, refreshing the partial successes is no longer needed because we'll retry those requests. In other words, we'll refresh less and retry more. The existing policy of refreshing more and retrying less will start to be applied inconsistenly with cockroachdb#44654, where we start refreshing when the client sees a WriteTooOld flag - but we're forced to refresh the whole batch. Besides the rationalizations above, this patch allows us to simplify code by not having to deal with both responses and errors. We can thus get rid of the enthralling comment on the client.Sender.Send() stating: " // The contract about whether both a response and an error can be // returned varies between layers. " Release note: None
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 2 of 14 files at r4, 1 of 20 files at r6, 1 of 1 files at r10, 4 of 4 files at r11, 14 of 14 files at r12.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @andreimatei and @nvanbenschoten)
pkg/kv/dist_sender.go, line 651 at r12 (raw file):
// encountered in the transaction and retry just the EndTxn request to // avoid client-side serializable txn retries. // TODO(andrei,nvanbenschoten): Get rid of this returning both error and result,
Didn't you just address this?
pkg/kv/dist_sender_server_test.go, line 2407 at r12 (raw file):
lastAutoRetries = metrics.AutoRetries.Count() err := tc.retryable(ctx, txn)
You can undo this.
pkg/kv/txn_interceptor_span_refresher.go, line 231 at r12 (raw file):
) (_ *roachpb.BatchResponse, _ *roachpb.Error, largestRefreshTS hlc.Timestamp) { br, pErr := sr.sendHelper(ctx, ba) if pErr == nil && br.Txn.WriteTooOld {
Let's add a few tight tests around this logic in txn_interceptor_span_refresher_test.go
.
pkg/kv/txn_interceptor_span_refresher.go, line 231 at r12 (raw file):
) (_ *roachpb.BatchResponse, _ *roachpb.Error, largestRefreshTS hlc.Timestamp) { br, pErr := sr.sendHelper(ctx, ba) if pErr == nil && br.Txn.WriteTooOld {
Should we check whether ba.Txn.WriteTooOld
was already set? Will that be possible with this new eager refresh-or-error policy?
pkg/kv/txn_interceptor_span_refresher.go, line 250 at r12 (raw file):
// // For the refresh, we have two options: either refresh everything read // *before* this batch, and then retry to batch, or refresh the current
the batch
pkg/kv/txn_interceptor_span_refresher.go, line 258 at r12 (raw file):
bumpedTxn.ReadTimestamp = bumpedTxn.WriteTimestamp pErr = roachpb.NewErrorWithTxn( roachpb.NewTransactionRetryError(roachpb.RETRY_WRITE_TOO_OLD, ""),
You were the one who wanted the extraMsg
...
pkg/kv/txn_interceptor_span_refresher.go, line 266 at r12 (raw file):
} if pErr != nil { // Don't confuse layers above with both a result and an error. This layer is
I thought we got rid of this?
pkg/kv/txn_interceptor_span_refresher.go, line 272 at r12 (raw file):
// Terminate the txn.WriteTooOld flag here. We failed to refresh it away, so // turn it into a retriable error. if br.Txn.WriteTooOld {
We need this again? If so, let's add a test where it's necessary and also pull the logic into a helper.
pkg/kv/txn_interceptor_span_refresher.go, line 332 at r12 (raw file):
// Try updating the txn spans so we can retry. if ok := sr.tryUpdatingTxnSpans(ctx, retryTxn); !ok { return br, pErr, hlc.Timestamp{}
Why was this needed?
pkg/roachpb/api.proto, line 1788 at r12 (raw file):
// idempotent. We could just re-issue requests. See #26915. bool async_consensus = 13; reserved 7,12,14;
smallest nit possible: we put spaces between these everywhere else.
pkg/storage/replica_evaluate.go, line 288 at r12 (raw file):
// to be rejected if it were sent again with the same sequence number // after a refresh. // TODO(andrei): What we really want to do here is either speculatively
This is going to be a great change when it happens.
pkg/storage/replica_test.go, line 4513 at r10 (raw file):
} // Expect that keyB intent got leaked.
What does "leaked" mean? Anyone who runs into will notice that they're able to remove it, so it's not really leaked without recourse. I'd add a bit more here.
pkg/storage/replica_test.go, line 9599 at r12 (raw file):
ba.Add(&put) return send(ba)
nit: remove this diff
pkg/storage/replica_write.go, line 477 at r12 (raw file):
pErrOrWTO := pErr if pErr == nil && br.Txn != nil && br.Txn.WriteTooOld { bumpedTxn := br.Txn.Clone()
This seems like a lot of wasted allocations. Is there a clean way to pull this into canDoServersideRetry
and avoid them?
Before this patch, when the DistSender would split a batch into multiple sub-batches and one of the sub-batches fails, it would collect responses for the successful ones and return them together with the error. This used to be pretty important before we had write idempotency, because it allowed the span refresher to only retry an EndTxn without also retring other writes in that batch (which would have failed). Since we've gotten idempotency in the meantime, we can retry those other writes. In fact, it's arguably better to do it: there's a tradeoff between refreshing and retrying. Currently the span refresher needs to refresh the read spans of the successful sub-batches, which refresh is at risk of failing under contention. This patch makes the span refresher retry the whole batch without considering partial successes. With this patch, refreshing the partial successes is no longer needed because we'll retry those requests. In other words, we'll refresh less and retry more. The existing policy of refreshing more and retrying less will start to be applied inconsistenly with cockroachdb#44654, where we start refreshing when the client sees a WriteTooOld flag - but we're forced to refresh the whole batch. Besides the rationalizations above, this patch allows us to simplify code by not having to deal with both responses and errors. We can thus get rid of the enthralling comment on the client.Sender.Send() stating: " // The contract about whether both a response and an error can be // returned varies between layers. " Release note: None
44661: kv: refresh less and retry more r=andreimatei a=andreimatei Before this patch, when the DistSender would split a batch into multiple sub-batches and one of the sub-batches fails, it would collect responses for the successful ones and return them together with the error. This used to be pretty important before we had write idempotency, because it allowed the span refresher to only retry an EndTxn without also retring other writes in that batch (which would have failed). Since we've gotten idempotency in the meantime, we can retry those other writes. In fact, it's arguably better to do it: there's a tradeoff between refreshing and retrying. Currently the span refresher needs to refresh the read spans of the successful sub-batches, which refresh is at risk of failing under contention. This patch makes the span refresher retry the whole batch without considering partial successes. With this patch, refreshing the partial successes is no longer needed because we'll retry those requests. In other words, we'll refresh less and retry more. The existing policy of refreshing more and retrying less will start to be applied inconsistenly with #44654, where we start refreshing when the client sees a WriteTooOld flag - but we're forced to refresh the whole batch. Besides the rationalizations above, this patch allows us to simplify code by not having to deal with both responses and errors. We can thus get rid of the enthralling comment on the client.Sender.Send() stating: " // The contract about whether both a response and an error can be // returned varies between layers. " Release note: None Co-authored-by: Andrei Matei <andrei@cockroachlabs.com>
02c3c16
to
9c2269a
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)
pkg/kv/dist_sender.go, line 651 at r12 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Didn't you just address this?
done
pkg/kv/dist_sender_server_test.go, line 2407 at r12 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
You can undo this.
done
pkg/kv/txn_interceptor_span_refresher.go, line 231 at r12 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Should we check whether
ba.Txn.WriteTooOld
was already set? Will that be possible with this new eager refresh-or-error policy?
It's no longer possible for ba.Txn.WriteTooOld
to be set.
pkg/kv/txn_interceptor_span_refresher.go, line 231 at r12 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Let's add a few tight tests around this logic in
txn_interceptor_span_refresher_test.go
.
added one such tests. Were you thinking of more?
pkg/kv/txn_interceptor_span_refresher.go, line 250 at r12 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
the batch
done
pkg/kv/txn_interceptor_span_refresher.go, line 258 at r12 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
You were the one who wanted the
extraMsg
...
well but what would it say in this case?
pkg/kv/txn_interceptor_span_refresher.go, line 266 at r12 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
I thought we got rid of this?
yes, I've now rebased on the other change
pkg/kv/txn_interceptor_span_refresher.go, line 272 at r12 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
We need this again? If so, let's add a test where it's necessary and also pull the logic into a helper.
We didn't need this. It was a bad leftover. Done.
pkg/kv/txn_interceptor_span_refresher.go, line 332 at r12 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Why was this needed?
it wasn't. Good catch.
pkg/roachpb/api.proto, line 1788 at r12 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
smallest nit possible: we put spaces between these everywhere else.
done
pkg/storage/replica_evaluate.go, line 288 at r12 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
This is going to be a great change when it happens.
👍
pkg/storage/replica_test.go, line 4513 at r10 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
What does "leaked" mean? Anyone who runs into will notice that they're able to remove it, so it's not really leaked without recourse. I'd add a bit more here.
done
pkg/storage/replica_test.go, line 9599 at r12 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
nit: remove this diff
done
pkg/storage/replica_write.go, line 477 at r12 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
This seems like a lot of wasted allocations. Is there a clean way to pull this into
canDoServersideRetry
and avoid them?
ok, see now
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 1 of 19 files at r13, 4 of 4 files at r14, 14 of 14 files at r15.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @andreimatei)
pkg/kv/txn_interceptor_span_refresher.go, line 231 at r12 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
It's no longer possible for
ba.Txn.WriteTooOld
to be set.
Should we assert against that then? What about if this is a refresher on a leaf txn coordinator and the root is a 19.2 node?
pkg/kv/txn_interceptor_span_refresher.go, line 258 at r12 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
well but what would it say in this case?
"WriteTooOld flag converted to WriteTooOldError" or something
pkg/kv/txn_interceptor_span_refresher_test.go, line 105 at r15 (raw file):
// If name is not set, the test will use pErr.String(). name string // OnFirstSend, if set, is invoked to evaluate the batch. If not set, pErr()
Consider constraining this to something like br func(roachpb.BatchRequest) *roachpb.BatchResponse
and then require that exactly one of br
and pErr
are set.
Then you can also keep the assertions in onFirstSend
below.
pkg/kv/txn_interceptor_span_refresher_test.go, line 187 at r15 (raw file):
// Collect some refresh spans. var ba roachpb.BatchRequest clonedTxn := txn.Clone()
nit, just inline this: ba.Header = roachpb.Header{Txn: txn.Clone()}
pkg/storage/replica_test.go, line 4513 at r10 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
done
Did you mean to leave both comments?
pkg/storage/replica_write.go, line 477 at r12 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
ok, see now
👍
9c2269a
to
1a83149
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)
pkg/kv/txn_interceptor_span_refresher.go, line 231 at r12 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Should we assert against that then? What about if this is a refresher on a leaf txn coordinator and the root is a 19.2 node?
I had not considered 19.2 roots. I've added code to reset the flag in newLeafTxnCoordSender()
- I think that's better than handling the flag here.
I've also added an assertion.
pkg/kv/txn_interceptor_span_refresher.go, line 258 at r12 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
"WriteTooOld flag converted to WriteTooOldError" or something
done
pkg/kv/txn_interceptor_span_refresher_test.go, line 105 at r15 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Consider constraining this to something like
br func(roachpb.BatchRequest) *roachpb.BatchResponse
and then require that exactly one ofbr
andpErr
are set.Then you can also keep the assertions in
onFirstSend
below.
I've added an assertion that exactly one of the two is set. And I've re-instituted those assertions. But I've kept the signature.
pkg/kv/txn_interceptor_span_refresher_test.go, line 187 at r15 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
nit, just inline this:
ba.Header = roachpb.Header{Txn: txn.Clone()}
done
pkg/storage/replica_test.go, line 4513 at r10 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Did you mean to leave both comments?
fixed
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 4 of 4 files at r17, 15 of 15 files at r18.
Reviewable status: complete! 1 of 0 LGTMs obtained
1a83149
to
15a12ab
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.
bors r+
Reviewable status: complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @nvanbenschoten)
Build failed |
Release note: None
Remove InclusiveTimeBounds(), a vestige that used to be more complex for backwards compatibility with 19.1 reasons but became obsolete when we cleaned up the compatibility cruft. Release note: None
…oo-old Before this patch, any write running into a write-too-old condition resulted in a WriteTooOldError being returned by the server. Returning an error implies that no intents are left behind. This is unfortunate; we'd like to leave intents (or, in the future, other types of locks) behind so keep away other transactions. We've observed this resulting in the starvation of a class of transactions in a user's workload. This patch makes it so that blind writes (i.e. Puts - used by UPDATE, not CPuts) don't return WriteTooOldErrors any more. Instead, they return the a txn proto with the WriteTooOld flag set. This is the behavior they had before cockroachdb#38668. This patch retains the goal of cockroachdb#38668, however: the client now eagerly refreshes the transactions when it sees a WriteTooOld flag, and if the refresh succeeds, it returns a WriteTooOldError to the higher layers (SQL), allowing for automatic retries where applicable. Unfortunately, CPuts (used by INSERT) continue to return WriteTooOldErrors without leaving locks behind. Dealing with them requires more tenderness because they imply a read, and the timestamp of a read cannot be bumped as easily as that of a write. Touches cockroachdb#44653 Release note (sql change): UPDATEs returning a serialization failure error (code 40001) now leave behind a lock, helping the transaction succeed if it retries. This prevents starvation of transactions whose UPDATEs are prone to conflicts.
15a12ab
to
da5dc5b
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.
bors r+
Reviewable status: complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @nvanbenschoten)
44654: storage: leave intents behind after blind-writes experiencing write-too-old r=andreimatei a=andreimatei Before this patch, any write running into a write-too-old condition resulted in a WriteTooOldError being returned by the server. Returning an error implies that no intents are left behind. This is unfortunate; we'd like to leave intents (or, in the future, other types of locks) behind so keep away other transactions. We've observed this resulting in the starvation of a class of transactions in a user's workload. This patch makes it so that blind writes (i.e. Puts - used by UPDATE, not CPuts) don't return WriteTooOldErrors any more. Instead, they return the a txn proto with the WriteTooOld flag set. This is the behavior they had before #38668. This patch retains the goal of #38668, however: the client now eagerly refreshes the transactions when it sees a WriteTooOld flag, and if the refresh succeeds, it returns a WriteTooOldError to the higher layers (SQL), allowing for automatic retries where applicable. Unfortunately, CPuts (used by INSERT) continue to return WriteTooOldErrors without leaving locks behind. Dealing with them requires more tenderness because they imply a read, and the timestamp of a read cannot be bumped as easily as that of a write. Touches #44653 Release note (SQL change): UPDATEs returning a serialization failure error (code 40001) now leave behind a lock, helping the transaction succeed if it retries. This prevents starvation of transactions whose UPDATEs are prone to conflicts. Co-authored-by: Andrei Matei <andrei@cockroachlabs.com>
Build succeeded |
Before this patch, any write running into a write-too-old condition
resulted in a WriteTooOldError being returned by the server. Returning
an error implies that no intents are left behind. This is unfortunate;
we'd like to leave intents (or, in the future, other types of locks)
behind so keep away other transactions. We've observed this resulting in
the starvation of a class of transactions in a user's workload.
This patch makes it so that blind writes (i.e. Puts - used by UPDATE,
not CPuts) don't return WriteTooOldErrors any more. Instead, they return
the a txn proto with the WriteTooOld flag set. This is the behavior they
had before #38668. This patch retains the goal of #38668, however: the
client now eagerly refreshes the transactions when it sees a WriteTooOld
flag, and if the refresh succeeds, it returns a WriteTooOldError to the
higher layers (SQL), allowing for automatic retries where applicable.
Unfortunately, CPuts (used by INSERT) continue to return
WriteTooOldErrors without leaving locks behind. Dealing with them
requires more tenderness because they imply a read, and the timestamp of
a read cannot be bumped as easily as that of a write.
Touches #44653
Release note (SQL change): UPDATEs returning a serialization failure error (code
40001) now leave behind a lock, helping the transaction succeed if it
retries. This prevents starvation of transactions whose UPDATEs are
prone to conflicts.