-
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,kv: make transaction deadline exceeded errors retriable #35284
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.
Reviewed 1 of 1 files at r1, 20 of 20 files at r2.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @andreimatei)
pkg/roachpb/errors.proto, line 215 at r2 (raw file):
// An asynchronous write was observed to have failed. RETRY_ASYNC_WRITE_FAILURE = 5; // The transaction exceeded its deadline.
tab instead of space
pkg/storage/batcheval/cmd_end_transaction.go, line 278 at r2 (raw file):
} if IsEndTransactionExceedingDeadline(reply.Txn.Timestamp, *args) {
Shouldn't we delete this?
6b98071
to
3612e00
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/roachpb/errors.proto, line 215 at r2 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
tab instead of space
done
pkg/storage/batcheval/cmd_end_transaction.go, line 278 at r2 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Shouldn't we delete this?
yes, it was unreachable, thanks.
Heroes! |
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 @andreimatei and @nvanbenschoten)
pkg/kv/txn_coord_sender.go, line 617 at r3 (raw file):
deadline := ba.Requests[0].GetEndTransaction().Deadline txn := tc.mu.txn if deadline != nil && deadline.Less(txn.Timestamp) {
This should be !txn.Timestamp.Less(deadline). The deadline is not included as a valid timestamp.
pkg/kv/txn_interceptor_committer.go, line 96 at r3 (raw file):
// Check if the (read-only) txn was pushed above its deadline. deadline := et.Deadline if deadline != nil && deadline.Less(br.Txn.Timestamp) {
same
pkg/kv/txn_interceptor_committer.go, line 106 at r3 (raw file):
txnCpy := txn.Clone() return nil, roachpb.NewErrorWithTxn( roachpb.NewTransactionRetryError(roachpb.RETRY_COMMIT_DEADLINE_EXCEEDED, extraMsg), &txnCpy)
probably makes sense to abstract this out as a function. It's repeated here.
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 @andreimatei, @nvanbenschoten, and @vivekmenezes)
pkg/kv/txn_coord_sender.go, line 617 at r3 (raw file):
Previously, vivekmenezes wrote…
This should be !txn.Timestamp.Less(deadline). The deadline is not included as a valid timestamp.
well that's not how EndTransactionRequest
is documented and not what the server enforces:
return args.Deadline != nil && args.Deadline.Less(t) |
But indeed the leasing code documents to be returning exclusive deadlines.
Line 1362 in 38bb1e7
// commit-timestamp < expiration-time. Care must be taken to not modify |
So... which one should it be? I'm happy to change the server-side enforcement too.
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 @andreimatei and @nvanbenschoten)
pkg/kv/txn_coord_sender.go, line 617 at r3 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
well that's not how
EndTransactionRequest
is documented and not what the server enforces:
return args.Deadline != nil && args.Deadline.Less(t) But indeed the leasing code documents to be returning exclusive deadlines.
Line 1362 in 38bb1e7
// commit-timestamp < expiration-time. Care must be taken to not modify So... which one should it be? I'm happy to change the server-side enforcement too.
The deadline is not an acceptable timestamp. Let's fix it to accommodate that concept. Thanks!
@andreimatei wanna update this PR and get it into a mergeable state. Would love to see it get into 19.1 |
Don't know if I can do that today or tomorrow. |
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.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @nvanbenschoten and @vivekmenezes)
pkg/kv/txn_coord_sender.go, line 617 at r3 (raw file):
Previously, vivekmenezes wrote…
The deadline is not an acceptable timestamp. Let's fix it to accommodate that concept. Thanks!
done
pkg/kv/txn_interceptor_committer.go, line 96 at r3 (raw file):
Previously, vivekmenezes wrote…
same
Done.
pkg/kv/txn_interceptor_committer.go, line 106 at r3 (raw file):
Previously, vivekmenezes wrote…
probably makes sense to abstract this out as a function. It's repeated here.
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.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @andreimatei, @nvanbenschoten, and @vivekmenezes)
pkg/kv/txn_coord_sender.go, line 705 at r5 (raw file):
// txnCpy := txn.Clone() // pErr := roachpb.NewErrorWithTxn( // roachpb.NewTransactionRetryError(roachpb.RETRY_COMMIT_DEADLINE_EXCEEDED, extraMsg), &txnCpy)
Nit: remove comments here and below
pkg/storage/batcheval/cmd_end_transaction.go, line 399 at r5 (raw file):
retry, reason = true, roachpb.RETRY_COMMIT_DEADLINE_EXCEEDED } }
I suppose you could use the common function here too.
Before this patch, they were opaque TransactionStatusErrors. The belief is that we should only be seeing such errors when a transaction is pushed by minutes. Shockingly, this seems to hapen enough in our tests, for example as described here: cockroachdb#18684 (comment) This patch marks the error as retriable, since it technically is. This patch also changes the semantics of the EndTransactionRequest.Deadline field to make it exclusive so that it matches the nature of SQL leases. No migration needed. Touches cockroachdb#18684 Release note (sql change): "transaction deadline exceeded" errors are now returned to the client with a retriable code.
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 (waiting on @nvanbenschoten and @vivekmenezes)
pkg/kv/txn_coord_sender.go, line 705 at r5 (raw file):
Previously, vivekmenezes wrote…
Nit: remove comments here and below
done. not a nit, thanks
pkg/storage/batcheval/cmd_end_transaction.go, line 399 at r5 (raw file):
Previously, vivekmenezes wrote…
I suppose you could use the common function here too.
not easily. Unfortunately on the server side most code (like here) deals with errors
, and the transaction is attached to the error later.
35284: storage,kv: make transaction deadline exceeded errors retriable r=andreimatei a=andreimatei Before this patch, they were opaque TransactionStatusErrors. The belief is that we should only be seeing such errors when a transaction is pushed by minutes. Shockingly, this seems to hapen enough in our tests, for example as described here: #18684 (comment) This patch marks the error as retriable, since it technically is. This patch also changes the semantics of the EndTransactionRequest.Deadline field to make it exclusive so that it matches the nature of SQL leases. No migration needed. Touches #18684 Release note (sql change): "transaction deadline exceeded" errors are now returned to the client with a retriable code. 35793: storage: fix TestRangeInfo flake and re-enable follower reads by default r=ajwerner a=ajwerner This PR addresses a test flake introduced by enabling follower reads in conjunction with #35130 which makes follower reads more generally possible in the face of lease transfer. Fixes #35758. Release note: None 35865: roachtest: Skip flaky jepsen nemesis r=tbg a=bdarnell See #35599 Release note: None Co-authored-by: Andrei Matei <[email protected]> Co-authored-by: Andrew Werner <[email protected]> Co-authored-by: Ben Darnell <[email protected]>
Build succeeded |
As of cockroachdb#35284, this was already checked by IsEndTransactionTriggeringRetryError. Release note: None
As of cockroachdb#35284, this was already checked by IsEndTransactionTriggeringRetryError. Release note: None
Before this patch, they were opaque TransactionStatusErrors.
The belief is that we should only be seeing such errors when a
transaction is pushed by minutes. Shockingly, this seems to hapen enough
in our tests, for example as described here: #18684 (comment)
This patch marks the error as retriable, since it technically is.
This patch also changes the semantics of the
EndTransactionRequest.Deadline field to make it exclusive so that it
matches the nature of SQL leases. No migration needed.
Touches #18684
Release note (sql change): "transaction deadline exceeded" errors are
now returned to the client with a retriable code.