-
Notifications
You must be signed in to change notification settings - Fork 3.7k
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: tpcc returns retriable errors to the user #28898
Comments
I've added more logging to the client - soon we'll know what pgerror code these errors are returning. Plenty of errors are being returned properly as 40001 - it's just a matter of tracking down which ones aren't. |
As far as I can tell, the only spurious error that gets returned is I could have sworn I saw a non-replay-txn error pop up, but now I can't reproduce it. I'll keep trying though. |
I've figured out the issue. Errors emitted from remote flows don't get properly unwrapped and turned into pgwire errors. I've made a test that shows this:
This should automatically retry, but it doesn't, since I've forced the data on What I don't understand is why this hasn't always been a problem. Nothing about this was changed during the DistSQL merge. Perhaps another error reporting mechanism was changed recently? |
Looks like this has always been a problem. I confirmed that it reproduces on release-2.0. The next question is why it didn't come up in TPCC back then. Perhaps these sorts of errors were inadvertently made more likely. |
The root cause of this issue seems to be the fact that the TransactionCoordSender is currently misconfigured to return a We found this by adding the diff:
which produced a stack trace on an in-the-wild cluster:
This should never happen. |
Btw, I've verified that the cause of the "replay txn" error is the scenario described in #28270 (comment)
|
So I'm looking at the badly wrapped |
I verified that we are indeed trying to start a flow in an aborted txn. We should add protection against that, as it obviously it will result in a bad time. |
Well in the cluster I'm running (roachprod 4 node, 3 tpcc workloads running concurrently) I'm seeing heartbeats taking up to 5s. And those heartbeats find their transactions to be aborted. |
Before this patch, we had a race between a heartbeat find out (async) that a txn has been aborted and a client using the txn to create DistSQL flows. We could end up creating flows in aborted txns, which are going to have a bad time: at the moment, leaf TxnCoordSenders don't react well to what appears to be an async abort: tcs.maybeRejectClientLocked() returns a HandledRetryableError, but the DistSQL infrastructure only expects raw retriable errors - so the HandledRetryableError was ironically losing its "retryable" character and was making it to a sql client with the wrong pgwire code. This patch resolves the situation by atomically checking that the txn is still good (i.e. status==PENDING) before extracting its metadata that is sent to leaves. Fixes cockroachdb#28898 Release note: Fix an issue where, under severe load, clients were sometimes receiving retryable errors with a non-retryable error code (a client would get an error with the message "HandledRetryableError: ..." but an internal error code instead of the expected retryable error code).
Before this patch, we had a race between a heartbeat find out (async) that a txn has been aborted and a client using the txn to create DistSQL flows. We could end up creating flows in aborted txns, which are going to have a bad time: at the moment, leaf TxnCoordSenders don't react well to what appears to be an async abort: tcs.maybeRejectClientLocked() returns a HandledRetryableError, but the DistSQL infrastructure only expects raw retriable errors - so the HandledRetryableError was ironically losing its "retryable" character and was making it to a sql client with the wrong pgwire code. This patch resolves the situation by atomically checking that the txn is still good (i.e. status==PENDING) before extracting its metadata that is sent to leaves. Fixes cockroachdb#28898 Fixes cockroachdb#29271 Release note (bug fix): Fix an issue where, under severe load, clients were sometimes receiving retryable errors with a non-retryable error code (a client would get an error with the message "HandledRetryableError: ..." but an internal error code instead of the expected retryable error code).
Before this patch, we had a race between a heartbeat find out (async) that a txn has been aborted and a client using the txn to create DistSQL flows. We could end up creating flows in aborted txns, which are going to have a bad time: at the moment, leaf TxnCoordSenders don't react well to what appears to be an async abort: tcs.maybeRejectClientLocked() returns a HandledRetryableError, but the DistSQL infrastructure only expects raw retriable errors - so the HandledRetryableError was ironically losing its "retryable" character and was making it to a sql client with the wrong pgwire code. This patch resolves the situation by atomically checking that the txn is still good (i.e. status==PENDING) before extracting its metadata that is sent to leaves. Fixes cockroachdb#28898 Fixes cockroachdb#29271 Release note (bug fix): Fix an issue where, under severe load, clients were sometimes receiving retryable errors with a non-retryable error code (a client would get an error with the message "HandledRetryableError: ..." but an internal error code instead of the expected retryable error code).
Before this patch, we had a race between a heartbeat find out (async) that a txn has been aborted and a client using the txn to create DistSQL flows. We could end up creating flows in aborted txns, which are going to have a bad time: at the moment, leaf TxnCoordSenders don't react well to what appears to be an async abort: tcs.maybeRejectClientLocked() returns a HandledRetryableError, but the DistSQL infrastructure only expects raw retriable errors - so the HandledRetryableError was ironically losing its "retryable" character and was making it to a sql client with the wrong pgwire code. This patch resolves the situation by atomically checking that the txn is still good (i.e. status==PENDING) before extracting its metadata that is sent to leaves. Fixes cockroachdb#28898 Fixes cockroachdb#29271 Release note (bug fix): Fix an issue where, under severe load, clients were sometimes receiving retryable errors with a non-retryable error code (a client would get an error with the message "HandledRetryableError: ..." but an internal error code instead of the expected retryable error code).
29455: distsqlrun,kv,client: don't create flows in aborted txns r=andreimatei a=andreimatei Before this patch, we had a race between a heartbeat find out (async) that a txn has been aborted and a client using the txn to create DistSQL flows. We could end up creating flows in aborted txns, which are going to have a bad time: at the moment, leaf TxnCoordSenders don't react well to what appears to be an async abort: tcs.maybeRejectClientLocked() returns a HandledRetryableError, but the DistSQL infrastructure only expects raw retriable errors - so the HandledRetryableError was ironically losing its "retryable" character and was making it to a sql client with the wrong pgwire code. This patch resolves the situation by atomically checking that the txn is still good (i.e. status==PENDING) before extracting its metadata that is sent to leaves. Fixes #28898 Fixes #29271 Release note: Fix an issue where, under severe load, clients were sometimes receiving retryable errors with a non-retryable error code (a client would get an error with the message "HandledRetryableError: ..." but an internal error code instead of the expected retryable error code). Co-authored-by: Andrei Matei <andrei@cockroachlabs.com>
29036: kv: prevent a leaf TCS from returning HandledRetryableError r=andreimatei a=andreimatei A root TxnCoordSender takes retryable errors, prepares the transaction for a retry and the returns a HandledRetryableError to the client. Leaf TCS does not prepare transactions for retries, and so has no business creating HandledRetryableErrors. Instead, it returns raw error which have to make their way (through DistSQL streams) to the root, which will "handle" them. Before this patch, there was a codepath where a leaf TCS would erroneously return a HandledRetryableError. This was confusing DistSQL which was not plumbing it properly and it was making its way to the client with a non-retriable code. Fixes #28898 Release note: None Co-authored-by: Andrei Matei <andrei@cockroachlabs.com>
29998: release-2.1: kv: prevent a leaf TCS from returning HandledRetryableError r=andreimatei a=andreimatei Backport 1/1 commits from #29036. /cc @cockroachdb/release --- A root TxnCoordSender takes retryable errors, prepares the transaction for a retry and the returns a HandledRetryableError to the client. Leaf TCS does not prepare transactions for retries, and so has no business creating HandledRetryableErrors. Instead, it returns raw error which have to make their way (through DistSQL streams) to the root, which will "handle" them. Before this patch, there was a codepath where a leaf TCS would erroneously return a HandledRetryableError. This was confusing DistSQL which was not plumbing it properly and it was making its way to the client with a non-retriable code. Fixes #28898 Release note: None Co-authored-by: Andrei Matei <andrei@cockroachlabs.com>
Running TPCC with latest master sometimes produces retriable errors that get propagated all the way up to the user, like the "replay txn" error.
@asubiotto theorizes that this could be because of #24798.
@andreimatei, what do you think?
The text was updated successfully, but these errors were encountered: