-
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
roachtest: jepsen/3/register/split failed #30792
Comments
This looks like a real one! I'm still working on making sense of the output but jepsen ran to completion and found a non-linearizable history. |
Could be a red herring, but #30778 merged in the day before this failure. |
Ok. #30778 seems pretty innocuous. It is not removing a replica from the replicas map temporarily during committing a split. I'll poke around in the logs for this failure to see if anything jumps out at me. |
OK, I think these are the relevant portions of the logs. We see from the reads here (starting at line 15) that key 167 has value 1. Several writes (with values 2 and 4) are attempted, but they all fail (unambiguously, I think, although I don't see any explicit handling of ambiguity here), just after the nemesis causes a split at key 165.
Next time we touch key 167, it has value 4:
So either A) pending writes on the RHS of a split are reporting unambiguous failures when they are in fact being applied or B) jepsen is not correctly accounting for ambiguous failures in this test. |
What kind of errors are happening here? Fortunately we have packet captures.
The test considers any Let's approach this from the other direction and find queries involving key 167. This is a little tricky because of the use of prepared statements; we can't actually see the query and the parameters together. But the filter That narrows it down to a single blind write of value 4 to key 167. Here's the full dump: https://gist.github.com/bdarnell/f07d1cd944d7dc7c1c4a0ec15fd4d235 Prepared statement decoder ring:
So the query does a read followed by an update, then tries to commit and gets a RETRY_ASYNC_WRITE_FAILURE.
Repeating the bashreduce above with more detail on the TransactionRetryErrors shows that this variant is rare:
So it looks like async writes (possibly combined with splits) are failing in an ambiguous way that is getting reported to the client as unambiguous. cc @nvanbenschoten |
Some timing details that aren't in the gist:
Here's how the operations were interleaved (split is coming from a different clock, so it might be a bit off)
|
If we see an RETRY_ASYNC_WRITE_FAILURE error then the transaction should have been retried without committing, just like any other retry error that doesn't hit one of our optimizations. There are two interesting things to note here that aren't immediately clear:
Together, this means that if the It is however possible that the txn record exists from a previous epoch, but I don't see how that would cause issues because intents at the old epoch would not be committed. |
Is the following or some variation of it possible?
Independently of whether this is even at fault, shouldn't I'm also not sure how the split would've made the above possible. There's seemingly no reason the @bdarnell do we have gRPC packet captures too? It'll be quite nasy to decode but now might be the time to put together a tool that does it for us. |
The part of that which doesn't seem possible is the "magic". If {Begin,Put} returned successfully to the client then it would have already been proposed to Raft. If the leaseholder stays the same then the command will block the subsequent QueryIntent in the command queue until it has applied. If the leaseholder changed hands before the {Begin,Put} applied then the proposal would have gotten rejected beneath Raft. Either way, the {Begin,Put} shouldn't apply succesfully after the {QueryIntent,Commit} executes. |
Well, the "magic" replay could be that the RPC arrives twice some tryExecute stuff goes wrong. But yeah, it doesn't seem likely (but then again, this is probably a rare bug). |
My working theory is that the The question is, what is that error on the first attempt to commit? We can't see that here because we don't have grpc packet captures or verbose-enough logging. We could get that, but so far the error has proven difficult to reproduce. |
Looking at this again from @bdarnell's perspective, the |
I see one at least remotely plausible way for the write to be applied twice. If this line fires, we might return ambiguous results when refreshing commands: cockroach/pkg/storage/replica.go Line 4173 in bbf9825
but we could unconditionally overwrite that variable later cockroach/pkg/storage/replica.go Line 4345 in bbf9825
The justification is given in the comment above as saying "it's the most aggressive mode anyway", but it's not true: cockroach/pkg/storage/replica.go Lines 4880 to 4884 in bbf9825
We will get an unambiguous retry instead of an ambiguous one. The retry will then fail checking the intent and the result would be what we're seeing. Oh, it actually becomes easier because the snapshot reason assignment doesn't do it if there's also a new leader: cockroach/pkg/storage/replica.go Lines 4171 to 4174 in bbf9825
So it seems reasonable that this would happen when the replica has the batch pending, but commits it via a snapshot that gets applied simultaneously with a leadership change (not too unlikely: we were leader before, and put the command in the log but then lose leadership and fall behind, getting caught up by the new leader). I'm not confident that this is really the bug we're seeing here. Seems like a bug though. I hope we can get a repro. |
I'm trying to get a repro, but have been unsuccessful so far. Might have to also write a unit test trying to replicate the spirit of the jepsen test+nemesis. Incidentally to your findings, I've discussed recently with @bdarnell about getting rid of reproposals on "snapshot applied" conditions - because they're hard to test (i.e. we're not testing them) and supposed to be rare and of questionable value. Instead, we'd treat the respective proposals as ambiguous without trying to resolve the ambiguity in any direction. I was going to do that when I have a chance, and then presumably the funky code that you're pointing to would be forced to change. |
That sounds good to me. If what I wrote above is actually what caused the bug, you're going to have a terrible time reproducing this. It's a good idea to return a few more ambiguous errors here and to remove the incorrect code. |
Well, what kind of chaos do you reckon will make the bug you found more likely to surface? |
Snapshots and leaseholder changes are associated with ... poorly functioning clusters. So you need one of those, and then you're not getting good qps = low frequency of repro. I still "hope" (well, not exactly hope) that this is another bug that actually has to do with the split. |
But, we should at least check the logs to see whether there was a snapshot for this range. |
Ah, right. It did go to a replica, but if the transaction has already committed and resolved, evaluating the rollback results in an empty write batch and slips through a hole in our logging. So this doesn't really tell us anything new (it comes after the async write failure, and we already know that the damage is done at that point). OK, next line of inquiry: Is there anything other than EndTransaction that could have resolved this intent? There's a failed push in the middle of this run (we've seen that in some but not all of the occurrences of this bug, I believe, so this may not prove anything).
Here's the gateway-side logs for all push and resolve traffic targeting this key during the relevant second:
This appears to show a successful push and resolve. The failed push isn't visible here (are we not logging the right error? Also, I'm only seeing the gateway stuff coming from a single node, which is suspicious). Of course, we don't see here which txn is being pushed (the test is deliberately racing multiple transactions against each other), or what kind of pushes these are. Maybe we need more logging of successful commands and first-time proposals, not just errors and reproposals. |
OK, time to look at this from the other side. The EndTransaction(commit=True) is behaving normally as far as we can tell; is there any other way we could end up reading the value it was supposed to write? I can think of a few ways which seem far-fetched but would be consistent with the symptoms we're seeing.
I think the last of these is most plausible, but I haven't found any evidence that any of them are possible. |
Out of your last three points, the last one seems the more likely to me. In
one of the previous failures (and in the last one too) we saw a PushTxn get
unstuck concomitantly with the EndTxn - which suggests that something
happened to the txn record.
…On Sat, Nov 3, 2018, 1:21 PM Ben Darnell ***@***.*** wrote:
OK, time to look at this from the other side. The
EndTransaction(commit=True) is behaving normally as far as we can tell; is
there any other way we could end up reading the value it was supposed to
write? I can think of a few ways which seem far-fetched but would be
consistent with the symptoms we're seeing.
- If the Put were processed as a non-transactional write. However, we
see in the replica logs that the BeginTransaction and Put are processed in
the same batch and it seems very unlikely that this link would be severed.
- If the intent were still pending but the MVCCMetadata were somehow
skipped over at read time (due to a misused time-bounded iterator, an
off-by-one seek, or something like that)
- The intent resolved with a status of ABORTED but the value was not
correctly deleted. My specific concern here is that this is possible if
MVCCMetadata.Timestamp were out of sync with the timestamp encoded in the
MVCC key.
I think the last of these is most plausible, but I haven't found any
evidence that any of them are possible.
—
You are receiving this because you were assigned.
Reply to this email directly, view it on GitHub
<#30792 (comment)>,
or mute the thread
<https://github.com/notifications/unsubscribe-auth/AAXBcZlMygaNrmOnQihhMaAae67tvTKMks5urdCcgaJpZM4XAeZt>
.
|
Just to throw another more powerful debug to in the mix, can we just keep
the write ahead log? We added this functionality.
|
@andreimatei got another failure with full Raft logging enabled:
We're going to dig in later. |
I might be getting over-excited here, but I think I have something (it's not based on any of the new logging). @andreimatei commented yesterday that one thing he saw in a previous repro was that there was a transaction that was waiting on the intent and that got unblocked just around the time the committing proposals were reproposed (but apparently both bounced because of invalid lease index). This smelled to me like a side-effect leak: we're signaling the txnwait queue with the proto resulting from a commit operation. What if we accidentally did that even when the commit didn't happen? Looking into it, this seems true. Step by step, we're seeing the command pop out of Raft for the first time (so we populate cockroach/pkg/storage/replica.go Lines 5396 to 5404 in fb4a974
Now note how we try to apply a no-op proposal, but we're not resetting the local eval result (only the replicated one): cockroach/pkg/storage/replica.go Lines 5473 to 5478 in 25922fc
We then continue to populate cockroach/pkg/storage/replica.go Lines 5622 to 5631 in 25922fc
This eventually ends up in cockroach/pkg/storage/replica_proposal.go Lines 765 to 770 in c23d984
and so what is likely happening is that we're trying to commit a txn (but we don't), but give the committed proto to some waiting pushers, who will happily go and resolve the intent they had been blocked on: cockroach/pkg/storage/txnwait/txnqueue.go Lines 475 to 489 in c32a379
|
I think you've found it! I thought I had looked through This bug lay dormant for a long while - most of the stuff in LocalResult is harmless (e.g. On a "forced" error ( This should be a very simple patch (at least to test the hypothesis): just wrap the line |
Yeah, the theory seems very likely. The analysis is more interesting: the bug (as it relates to EndTxn specifically) was probably introduced with the txn wait queue. But the introduction of the |
This patch fixes a bug with the application of LocalResults: they were applied even in cases when the processing of the respective command failed. So, we were evaluating a batch, proposing a command, failing to apply the command (in particular because of the lease index check), but then still applying the LocalResult even though the replicated state was not updated. This bug had catastrophic consequences: part of the LocalResult is a list of UpdatedTxn - this is used to ping the txn wait queue and unblock waiters, communicating a final txn disposition to them. Now imagine an EndTxn batch that evaluates to a command that fails the lease check. The respective LocalResult is populate with out txn, updated to the COMMITTED disposition. The command fails to process, but we still ping the wait queue telling all the waiters that the transaction committed. The waiters, if any, now go and send ResolveIntent requests, which actually commit intents (thinking that the intent's txn committed). And so we end up with "partially committed transactions" - some of the writes from a txn are committed (the ones that happened to have waiters on their intents), others don't. In order for this bug to manifest itself, we need: - read/write contention on some keys - a command to fail processing. Generally this happens either because of the lease index check or because of the lease check; so either a Raft leadership change or a lease change can potentially cause the badness. In the case of a Raft leadership change, proposals can get dropped, which leads to reproposals, which seem to frequently(?) end up processing at the wrong lease index and thus be rejected (and it's the reproposal processing that triggers the bug). This kind of situation is triggered by the Jepsen register test, with various nemeses. The lease check failure can probably be triggered simply with lease transfers (e.g. lease rebalancing). Interestingly, the rate of occurence of this bug probably changed between 2.0 and 2.1 because of the introduction, in 2.1, of the QueryIntent request (and also, separately, because of increased usage of lease transfers; this claim was not verified though). Speaking of the Raft leadership change scenario, once the reproposal fail to be applied (but the wait queue is erroneously signalled), we also re-evaluate the EndTransaction batch. Unless something else goes wrong, in 2.0 this re-evaluation was likely succeeding. In 2.1, however, it tends to fail if there was a waiter on our txn because the EndTxn requests are usually bundled with QueryIntent requests - and these QueryIntents fail during re-evaluation because they don't find the intent - it was, by then, errneously committed by a waiter through a ResolveIntent. Except for transaction that wrote to multiple ranges: I think for those the QueryIntent requests are split off from the EndTransaction request by the DistSender, and so we're back to the situation in 2.0. Fixes cockroachdb#30792 Release note (bug fix): Fix a bug causing transactions to appear to be partially committed. CRDB was sometimes claiming to have failed to commit a transaction but some (or all) of its writes were actually persisted.
This patch fixes a bug with the application of LocalResults: they were applied even in cases when the processing of the respective command failed. So, we were evaluating a batch, proposing a command, failing to apply the command (in particular because of the lease index check), but then still applying the LocalResult even though the replicated state was not updated. This bug had catastrophic consequences: part of the LocalResult is a list of UpdatedTxn - this is used to ping the txn wait queue and unblock waiters, communicating a final txn disposition to them. Now imagine an EndTxn batch that evaluates to a command that fails the lease check. The respective LocalResult is populate with out txn, updated to the COMMITTED disposition. The command fails to process, but we still ping the wait queue telling all the waiters that the transaction committed. The waiters, if any, now go and send ResolveIntent requests, which actually commit intents (thinking that the intent's txn committed). And so we end up with "partially committed transactions" - some of the writes from a txn are committed (the ones that happened to have waiters on their intents), others don't. In order for this bug to manifest itself, we need: - read/write contention on some keys - a command to fail processing. Generally this happens either because of the lease index check or because of the lease check; so either a Raft leadership change or a lease change can potentially cause the badness. In the case of a Raft leadership change, proposals can get dropped, which leads to reproposals, which seem to frequently(?) end up processing at the wrong lease index and thus be rejected (and it's the reproposal processing that triggers the bug). This kind of situation is triggered by the Jepsen register test, with various nemeses. The lease check failure can probably be triggered simply with lease transfers (e.g. lease rebalancing). Interestingly, the rate of occurence of this bug probably changed between 2.0 and 2.1 because of the introduction, in 2.1, of the QueryIntent request (and also, separately, because of increased usage of lease transfers; this claim was not verified though). Speaking of the Raft leadership change scenario, once the reproposal fail to be applied (but the wait queue is erroneously signalled), we also re-evaluate the EndTransaction batch. Unless something else goes wrong, in 2.0 this re-evaluation was likely succeeding. In 2.1, however, it tends to fail if there was a waiter on our txn because the EndTxn requests are usually bundled with QueryIntent requests - and these QueryIntents fail during re-evaluation because they don't find the intent - it was, by then, errneously committed by a waiter through a ResolveIntent. Except for transaction that wrote to multiple ranges: I think for those the QueryIntent requests are split off from the EndTransaction request by the DistSender, and so we're back to the situation in 2.0. Fixes cockroachdb#30792 Release note (bug fix): Fix a bug causing transactions to appear to be partially committed. CRDB was sometimes claiming to have failed to commit a transaction but some (or all) of its writes were actually persisted.
32166: storage: don't apply local results if cmd processing failed r=andreimatei a=andreimatei This patch fixes a bug with the application of LocalResults: they were applied even in cases when the processing of the respective command failed. So, we were evaluating a batch, proposing a command, failing to apply the command (in particular because of the lease index check), but then still applying the LocalResult even though the replicated state was not updated. This bug had catastrophic consequences: part of the LocalResult is a list of UpdatedTxn - this is used to ping the txn wait queue and unblock waiters, communicating a final txn disposition to them. Now imagine an EndTxn batch that evaluates to a command that fails the lease check. The respective LocalResult is populate with out txn, updated to the COMMITTED disposition. The command fails to process, but we still ping the wait queue telling all the waiters that the transaction committed. The waiters, if any, now go and send ResolveIntent requests, which actually commit intents (thinking that the intent's txn committed). And so we end up with "partially committed transactions" - some of the writes from a txn are committed (the ones that happened to have waiters on their intents), others don't. In order for this bug to manifest itself, we need: - read/write contention on some keys - a command to fail processing. Generally this happens either because of the lease index check or because of the lease check; so either a Raft leadership change or a lease change can potentially cause the badness. In the case of a Raft leadership change, proposals can get dropped, which leads to reproposals, which seem to frequently(?) end up processing at the wrong lease index and thus be rejected (and it's the reproposal processing that triggers the bug). This kind of situation is triggered by the Jepsen register test, with various nemeses. The lease check failure can probably be triggered simply with lease transfers (e.g. lease rebalancing). Interestingly, the rate of occurence of this bug probably changed between 2.0 and 2.1 because of the introduction, in 2.1, of the QueryIntent request (and also, separately, because of increased usage of lease transfers; this claim was not verified though). Speaking of the Raft leadership change scenario, once the reproposal fail to be applied (but the wait queue is erroneously signalled), we also re-evaluate the EndTransaction batch. Unless something else goes wrong, in 2.0 this re-evaluation was likely succeeding. In 2.1, however, it tends to fail if there was a waiter on our txn because the EndTxn requests are usually bundled with QueryIntent requests - and these QueryIntents fail during re-evaluation because they don't find the intent - it was, by then, errneously committed by a waiter through a ResolveIntent. Except for transaction that wrote to multiple ranges: I think for those the QueryIntent requests are split off from the EndTransaction request by the DistSender, and so we're back to the situation in 2.0. Fixes #30792 Release note (bug fix): Fix a bug causing transactions to appear to be partially committed. CRDB was sometimes claiming to have failed to commit a transaction but some (or all) of its writes were actually persisted. Co-authored-by: Andrei Matei <andrei@cockroachlabs.com>
This patch fixes a bug with the application of LocalResults: they were applied even in cases when the processing of the respective command failed. So, we were evaluating a batch, proposing a command, failing to apply the command (in particular because of the lease index check), but then still applying the LocalResult even though the replicated state was not updated. This bug had catastrophic consequences: part of the LocalResult is a list of UpdatedTxn - this is used to ping the txn wait queue and unblock waiters, communicating a final txn disposition to them. Now imagine an EndTxn batch that evaluates to a command that fails the lease check. The respective LocalResult is populate with out txn, updated to the COMMITTED disposition. The command fails to process, but we still ping the wait queue telling all the waiters that the transaction committed. The waiters, if any, now go and send ResolveIntent requests, which actually commit intents (thinking that the intent's txn committed). And so we end up with "partially committed transactions" - some of the writes from a txn are committed (the ones that happened to have waiters on their intents), others don't. In order for this bug to manifest itself, we need: - read/write contention on some keys - a command to fail processing. Generally this happens either because of the lease index check or because of the lease check; so either a Raft leadership change or a lease change can potentially cause the badness. In the case of a Raft leadership change, proposals can get dropped, which leads to reproposals, which seem to frequently(?) end up processing at the wrong lease index and thus be rejected (and it's the reproposal processing that triggers the bug). This kind of situation is triggered by the Jepsen register test, with various nemeses. The lease check failure can probably be triggered simply with lease transfers (e.g. lease rebalancing). Interestingly, the rate of occurence of this bug probably changed between 2.0 and 2.1 because of the introduction, in 2.1, of the QueryIntent request (and also, separately, because of increased usage of lease transfers; this claim was not verified though). Speaking of the Raft leadership change scenario, once the reproposal fail to be applied (but the wait queue is erroneously signalled), we also re-evaluate the EndTransaction batch. Unless something else goes wrong, in 2.0 this re-evaluation was likely succeeding. In 2.1, however, it tends to fail if there was a waiter on our txn because the EndTxn requests are usually bundled with QueryIntent requests - and these QueryIntents fail during re-evaluation because they don't find the intent - it was, by then, errneously committed by a waiter through a ResolveIntent. Except for transaction that wrote to multiple ranges: I think for those the QueryIntent requests are split off from the EndTransaction request by the DistSender, and so we're back to the situation in 2.0. Fixes cockroachdb#30792 Release note (bug fix): Fix a bug causing transactions to appear to be partially committed. CRDB was sometimes claiming to have failed to commit a transaction but some (or all) of its writes were actually persisted.
This patch fixes a bug with the application of LocalResults: they were applied even in cases when the processing of the respective command failed. So, we were evaluating a batch, proposing a command, failing to apply the command (in particular because of the lease index check), but then still applying the LocalResult even though the replicated state was not updated. This bug had catastrophic consequences: part of the LocalResult is a list of UpdatedTxn - this is used to ping the txn wait queue and unblock waiters, communicating a final txn disposition to them. Now imagine an EndTxn batch that evaluates to a command that fails the lease check. The respective LocalResult is populate with out txn, updated to the COMMITTED disposition. The command fails to process, but we still ping the wait queue telling all the waiters that the transaction committed. The waiters, if any, now go and send ResolveIntent requests, which actually commit intents (thinking that the intent's txn committed). And so we end up with "partially committed transactions" - some of the writes from a txn are committed (the ones that happened to have waiters on their intents), others don't. In order for this bug to manifest itself, we need: - read/write contention on some keys - a command to fail processing. Generally this happens either because of the lease index check or because of the lease check; so either a Raft leadership change or a lease change can potentially cause the badness. In the case of a Raft leadership change, proposals can get dropped, which leads to reproposals, which seem to frequently(?) end up processing at the wrong lease index and thus be rejected (and it's the reproposal processing that triggers the bug). This kind of situation is triggered by the Jepsen register test, with various nemeses. The lease check failure can probably be triggered simply with lease transfers (e.g. lease rebalancing). Interestingly, the rate of occurence of this bug probably changed between 2.0 and 2.1 because of the introduction, in 2.1, of the QueryIntent request (and also, separately, because of increased usage of lease transfers; this claim was not verified though). Speaking of the Raft leadership change scenario, once the reproposal fail to be applied (but the wait queue is erroneously signalled), we also re-evaluate the EndTransaction batch. Unless something else goes wrong, in 2.0 this re-evaluation was likely succeeding. In 2.1, however, it tends to fail if there was a waiter on our txn because the EndTxn requests are usually bundled with QueryIntent requests - and these QueryIntents fail during re-evaluation because they don't find the intent - it was, by then, errneously committed by a waiter through a ResolveIntent. Except for transaction that wrote to multiple ranges: I think for those the QueryIntent requests are split off from the EndTransaction request by the DistSender, and so we're back to the situation in 2.0. Fixes cockroachdb#30792 Release note (bug fix): Fix a bug causing transactions to appear to be partially committed. CRDB was sometimes claiming to have failed to commit a transaction but some (or all) of its writes were actually persisted.
…aders Another test for cockroachdb#30792, this time testing the specific interaction between readers and rejected Raft commands. Release note: None
…aders Another test for cockroachdb#30792, this time testing the specific interaction between readers and rejected Raft commands. Release note: None
32236: storage: add a test for interactions between rejected commands and re… r=andreimatei a=andreimatei …aders Another test for #30792, this time testing the specific interaction between readers and rejected Raft commands. Release note: None Co-authored-by: Andrei Matei <andrei@cockroachlabs.com>
A recent commit (master only) reintroduced a bug that we ironically had spent a lot of time on [before]. In summary, it would allow the result of an EndTransaction which would in itself *not* apply to leak and would result in intents being committed even though their transaction ultimately would not: cockroachdb#34025 (comment) We've diagnosed this pretty quickly the second time around, but clearly we didn't do a good job at preventing the regression. I can see how this would happen as the method this code is in is notoriously difficult to test for it interfaces so much with everything else that it's difficult to unit test it; one needs to jump through lots of hoops to target it, and so we do it less than we ought to. I believe this wasn't released in any alpha (nor backported anywhere), so no release note is necessary. Fixes cockroachdb#34025. [before]: cockroachdb#30792 (comment) Release note: None
34651: server: rework TestClusterVersionBootstrapStrict r=andreimatei a=andreimatei This test... I'm not entirely sure what it was supposed to test to be honest, but it seemed to be more complicated than it needed to be. It forced and emphasized MinSupportedVersion being equal to BinaryServerVersion (which is generally not a thing). I've simplified it, making it not muck with the versions, while keep (I think) the things it was testing (to the extent that it was testing anything). This test was also in my way because it created servers that pretended to be versions that are not technically supported by the binary, and this kind of funkiness is making my life hard as I'm trying to rework the way in which versions are propagated and what knobs servers have, etc. Release note: None 34659: storage: don't leak committed protos to pushers on reproposal r=bdarnell,andreimatei a=tbg TODO: test ---- A recent commit (master only) reintroduced a bug that we ironically had spent a lot of time on [before]. In summary, it would allow the result of an EndTransaction which would in itself *not* apply to leak and would result in intents being committed even though their transaction ultimately would not: #34025 (comment) We've diagnosed this pretty quickly the second time around, but clearly we didn't do a good job at preventing the regression. I can see how this would happen as the method this code is in is notoriously difficult to test for it interfaces so much with everything else that it's difficult to unit test it; one needs to jump through lots of hoops to target it, and so we do it less than we ought to. I believe this wasn't released in any alpha (nor backported anywhere), so no release note is necessary. Fixes #34025. [before]: #30792 (comment) Release note: None Co-authored-by: Andrei Matei <andrei@cockroachlabs.com> Co-authored-by: Tobias Schottdorf <tobias.schottdorf@gmail.com>
SHA: https://github.com/cockroachdb/cockroach/commits/031748232a601aab191fd3fb00dc298ca20e4506
Parameters:
To repro, try:
Failed test: https://teamcity.cockroachdb.com/viewLog.html?buildId=932135&tab=buildLog
The text was updated successfully, but these errors were encountered: