Skip to content
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: employ transactional idempotency to refresh mixed-success batches #35140

Merged
merged 3 commits into from
Nov 12, 2019

Conversation

nvanbenschoten
Copy link
Member

@nvanbenschoten nvanbenschoten commented Feb 22, 2019

This change builds on the introduction of idempotency properties introduced into the MVCC layer in #33001. It exploits this property to remove a previous restriction that could prevent transactions from refreshing and result in transaction retries. The restriction was that batches which had experienced some success while writing could not refresh even if another part of the batch required a refresh. This was because it was unclear which parts of the batch had succeeded in performing writes and which parts of the batch had not. Without this knowledge, it was unsafe to re-issue the batch because that could result in duplicating writes (e.g. performing an increment twice).

A lot has changed since then. We now have proper sequence numbers on requests, we no longer send BeginTransaction requests (which threw errors on replays), and we now have an MVCC layer that is idempotent for writes within a transaction. With this foundation in place, we can now safely re-issue any write within a transaction that we're unsure about. As long as writes remain well-sequenced (writes to the same key aren't reordered), everything should behave as expected.

The best way to see that the change works is through the test cases in TestTxnCoordSenderRetries that now succeed. Without #33001, those all fail.

@nvanbenschoten nvanbenschoten requested review from tbg and a team February 22, 2019 03:36
@cockroach-teamcity
Copy link
Member

This change is Reviewable

Copy link
Member

@tbg tbg left a 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, 1 of 1 files at r2, 4 of 4 files at r3.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @andreimatei and @nvanbenschoten)


pkg/kv/txn_interceptor_span_refresher.go, line 190 at r3 (raw file):

		}
		// Unwrap the MixedSuccessError.
		pErr = aPSErr.Wrapped

It doesn't matter for correctness what error we see here, right? I'm asking because there may have been more errors than this one (if you're sending to ten ranges in parallel, you might get 9 different errors and one success, but only one of the errors will be reflected here). So what we do is handle the one we see, but we could also just throw it away and try to refresh unconditionally? You had this PR the other day about the tricky interaction between WriteTooOld returned from a CPut, for example. You may miss a WriteTooOld this way, for example, and fail to even attempt to refresh it (because CPut has this special casing)? It looks like this all hinges on what this "prefix" of the batch is that you're observing below.


pkg/storage/replica_test.go, line 3101 at r2 (raw file):

// TestReplicaTxnIdempotency verifies that transactions run successfully and in
// an idempotent manner when replaying the same requests.

Hmm, it's not exactly idempotent, right? For example, a DeleteRange could succeed the first time, but when it's replayed another intent (by another txn) is laid down into the "empty" keyspace touched by the DeleteRange. Now this won't usually cause an error because that other DeleteRange is forced above the original request's timestamp (and that txn has observed a timestamp), so when we replay it won't observe the intent, but you mention that the timestamp in the replay can be higher (say some other read happened on the key range at higher ts in the meantime) in which case it would see the intent and would have to fail.


pkg/storage/replica_test.go, line 3102 at r2 (raw file):

// TestReplicaTxnIdempotency verifies that transactions run successfully and in
// an idempotent manner when replaying the same requests.
func TestReplicaTxnIdempotency(t *testing.T) {

10606 lines in this file before your change -- I know it's arbitrary to start now -- but maybe you can start a new file with this test (I'm sure we'll find siblings to move into the same file, too, though not necessarily in this PR)

replica_txn_test.go?


pkg/storage/replica_test.go, line 3414 at r2 (raw file):

			afterTxnStart: func(txn *roachpb.Transaction, key []byte) error {
				txnHighTS := txn.Clone()
				txnHighTS.Timestamp.Add(1, 0)

don't you need to assign the result back to txnHighTS.Timestamp? I think this line is a no-op as written.

In this test case and others like it, do you want to check that the response transactions reflects the higher timestamp of the current and previous write?


pkg/storage/replica_test.go, line 3447 at r2 (raw file):

			run: func(txn *roachpb.Transaction, key []byte) error {
				txnHighTS := txn.Clone()
				txnHighTS.Timestamp.Add(1, 0)

Ditto as above.

Copy link
Contributor

@andreimatei andreimatei left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM

lovely PR.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @nvanbenschoten and @tbg)


pkg/kv/txn_interceptor_span_refresher.go, line 190 at r3 (raw file):

Previously, tbg (Tobias Grieger) wrote…

It doesn't matter for correctness what error we see here, right? I'm asking because there may have been more errors than this one (if you're sending to ten ranges in parallel, you might get 9 different errors and one success, but only one of the errors will be reflected here). So what we do is handle the one we see, but we could also just throw it away and try to refresh unconditionally? You had this PR the other day about the tricky interaction between WriteTooOld returned from a CPut, for example. You may miss a WriteTooOld this way, for example, and fail to even attempt to refresh it (because CPut has this special casing)? It looks like this all hinges on what this "prefix" of the batch is that you're observing below.

I think what matters is that DistSender reports non-retriable errors over retriable errors if both are encountered, right?


pkg/storage/replica_test.go, line 3071 at r2 (raw file):

}

// TestReplicaAbortSpanOnlyWithIntent verifies that a transactional command

wait, what's this test about? Is it a good thing that some things don't check the abort span? I guess the heartbeat doesn't check it because it goes to the transaction's range, but why are we testing this?


pkg/storage/replica_test.go, line 3330 at r2 (raw file):

		},
		{
			// A request issued after a request with a larger sequence has

maybe you can clarify this comment a bit; it took me a second to understand why it fails and also the relationship with the next test. Say something about how the seq-2 request is not re-issued; it's issued for the first time. And so this would indicate a faulty client, right?


pkg/storage/replica_test.go, line 3353 at r2 (raw file):

		},
		{
			// Unlike the previous case, here a request is reissued after a

say something about overlapping batches


pkg/storage/replica_test.go, line 3462 at r2 (raw file):

		},
		{
			// If part of a batch has already succeeded and another part hasn't

say that this would happen after a merge?
And also say that at the DistSender level this is generally an important property, but here we're testing at a replica level.

Copy link
Member Author

@nvanbenschoten nvanbenschoten left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @andreimatei, @nvanbenschoten, and @tbg)


pkg/kv/txn_interceptor_span_refresher.go, line 190 at r3 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

I think what matters is that DistSender reports non-retriable errors over retriable errors if both are encountered, right?

It looks like we simply return the first error that we see. However, we update the error's transaction proto when we see multiple errors, which is enough to avoid multiple refreshes when returning a more optimal error would have resulted in only one.

This is all ok because any non-retriable error should be reproducible after the refresh.


pkg/storage/replica_test.go, line 3071 at r2 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

wait, what's this test about? Is it a good thing that some things don't check the abort span? I guess the heartbeat doesn't check it because it goes to the transaction's range, but why are we testing this?

You tell me 39f6950. Either way, I might start pulling on this, but not here.


pkg/storage/replica_test.go, line 3101 at r2 (raw file):

For example, a DeleteRange could succeed the first time, but when it's replayed another intent (by another txn) is laid down into the "empty" keyspace touched by the DeleteRange.

Great point, this seems to be unique to DeleteRange, because of its use of the write timestamp cache. All other requests "guard" their entire read-set with an intent, so they won't run into this kind of issue.

Now this won't usually cause an error because that other DeleteRange is forced above the original request's timestamp (and that txn has observed a timestamp), so when we replay it won't observe the intent, but you mention that the timestamp in the replay can be higher (say some other read happened on the key range at higher ts in the meantime) in which case it would see the intent and would have to fail.

I think it's actually worse than that. DeleteRange reads at the max timestamp regardless of the timestamp of the request itself. This is described here:

// In order to detect the potential write intent by another concurrent
// transaction with a newer timestamp, we need to use the max timestamp for
// scan.
scanTs := hlc.MaxTimestamp

So in the case you described, even a replay at the same timestamp could see new intents. This isn't really an issue because the request that wrote these intents could not have overlapped with any of the keys that the DeleteRange deleted, but it does stretch the definition of idempotency. I'm curious what your take on this is.

Similarly (and in response to your question below), the way reissued requests with different timestamp works is that we leave the intent as is, regardless of whether the new request had a higher or lower timestamp. We could move the intent if we wanted, but that would result in an unnecessary write. Reissuing a request with slightly different parameters doesn't fit into the definition of idempotency so I think it's fine to take whatever stance we want here, but it's worth being explicit about the expected behavior.


pkg/storage/replica_test.go, line 3102 at r2 (raw file):

Previously, tbg (Tobias Grieger) wrote…

10606 lines in this file before your change -- I know it's arbitrary to start now -- but maybe you can start a new file with this test (I'm sure we'll find siblings to move into the same file, too, though not necessarily in this PR)

replica_txn_test.go?

Will do once we resolve everything else.


pkg/storage/replica_test.go, line 3330 at r2 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

maybe you can clarify this comment a bit; it took me a second to understand why it fails and also the relationship with the next test. Say something about how the seq-2 request is not re-issued; it's issued for the first time. And so this would indicate a faulty client, right?

Done.


pkg/storage/replica_test.go, line 3353 at r2 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

say something about overlapping batches

Done.


pkg/storage/replica_test.go, line 3414 at r2 (raw file):

Previously, tbg (Tobias Grieger) wrote…

don't you need to assign the result back to txnHighTS.Timestamp? I think this line is a no-op as written.

In this test case and others like it, do you want to check that the response transactions reflects the higher timestamp of the current and previous write?

Nice catch!


pkg/storage/replica_test.go, line 3447 at r2 (raw file):

Previously, tbg (Tobias Grieger) wrote…

Ditto as above.

Done.


pkg/storage/replica_test.go, line 3462 at r2 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

say that this would happen after a merge?
And also say that at the DistSender level this is generally an important property, but here we're testing at a replica level.

Done.

@tbg tbg added the X-noremind Bots won't notify about PRs with X-noremind label Jun 19, 2019
Copy link
Contributor

@andreimatei andreimatei left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM

Let's get this beaut back on the saddle!
The MixedSuccessError is in my way for fiddling with WriteTooOldErrors because the error only detects mixed successes in cases where some sub-batch returns an error, and not also in cases where some part wants to force a refresh without returning an error (like for example through the Txn.WriteTooOld flag).

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @andreimatei, @nvanbenschoten, and @tbg)


pkg/kv/txn_interceptor_span_refresher.go, line 185 at r6 (raw file):

	// TODO(nvanbenschoten): remove this in 19.2 when we remove MixedSuccessError.
	if aPSErr, ok := pErr.GetDetail().(*roachpb.MixedSuccessError); ok {
		if !sr.st.Version.IsActive(cluster.VersionSequencedReads) {

this migration is done
Waiting does wonders.


pkg/roachpb/errors.proto, line 427 at r6 (raw file):

// request may have succeeded, but the batch as a whole failed with
// the wrapped error.
// TODO(nvanbenschoten): Remove this in 19.2.

20.2 now :P


pkg/storage/replica_test.go, line 3071 at r2 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

You tell me 39f6950. Either way, I might start pulling on this, but not here.

I'm confused about the txn heartbeat part of the test was passing before my commit which you linked. Before my commit, I think the heartbeat was supposed to check the AbortSpan and get an error, but the tests asserts that it doesn't...
In any case, I guess this test shows that the heartbeat behavior is funky... We don't want a transaction record to be created after the transaction is cleaned up by another pusher, do we?


pkg/storage/replica_test.go, line 3101 at r2 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

For example, a DeleteRange could succeed the first time, but when it's replayed another intent (by another txn) is laid down into the "empty" keyspace touched by the DeleteRange.

Great point, this seems to be unique to DeleteRange, because of its use of the write timestamp cache. All other requests "guard" their entire read-set with an intent, so they won't run into this kind of issue.

Now this won't usually cause an error because that other DeleteRange is forced above the original request's timestamp (and that txn has observed a timestamp), so when we replay it won't observe the intent, but you mention that the timestamp in the replay can be higher (say some other read happened on the key range at higher ts in the meantime) in which case it would see the intent and would have to fail.

I think it's actually worse than that. DeleteRange reads at the max timestamp regardless of the timestamp of the request itself. This is described here:

// In order to detect the potential write intent by another concurrent
// transaction with a newer timestamp, we need to use the max timestamp for
// scan.
scanTs := hlc.MaxTimestamp

So in the case you described, even a replay at the same timestamp could see new intents. This isn't really an issue because the request that wrote these intents could not have overlapped with any of the keys that the DeleteRange deleted, but it does stretch the definition of idempotency. I'm curious what your take on this is.

Similarly (and in response to your question below), the way reissued requests with different timestamp works is that we leave the intent as is, regardless of whether the new request had a higher or lower timestamp. We could move the intent if we wanted, but that would result in an unnecessary write. Reissuing a request with slightly different parameters doesn't fit into the definition of idempotency so I think it's fine to take whatever stance we want here, but it's worth being explicit about the expected behavior.

It seems to me that the properties we want are:

  1. A request doesn't "conflict" with itself: if request is evaluated and applied and then it get evaluated and applied again, the 2nd iteration doesn't run into any trouble because of the first. If the 2nd iteration runs into problems that the first one didn't (i.e. conflicts with other concurrent transactions), I think that's fine. The client is going to arbitrarily get the results of the first or the second evaluation and I think either is fine (right?).
  2. A request R should not prevent refreshes of the transaction that don't suspect that R (or parts of it) has succeeded. I.e. the success should not prevent the RefreshRequest itself from succeeding, and then it should also not prevent R' from evaluating, where R` is R with bumped read and write timestamps. (Cause that's what refreshes do - they retry a batch, and part of that batch might have succeeded (the MixedSuccessError case)).

And so, the way I see it, the fact that the DelRange request finds new intents a second time around if it's "retried" at a different timestamp (or even if its retried at the same timestamp (according to what Nathan is showing), like it would when the DistSender retries it, seems fine to me.

I'm not sure what this does to the terminology, though. Clearly the DelRange request is not "idempotent". But it's safe to re-evaluate, at either the same timestamp (DistSender) or at a different one (after a Refresh).

My 2c.

Copy link
Contributor

@andreimatei andreimatei left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @andreimatei, @nvanbenschoten, and @tbg)


pkg/storage/replica_test.go, line 3071 at r2 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

I'm confused about the txn heartbeat part of the test was passing before my commit which you linked. Before my commit, I think the heartbeat was supposed to check the AbortSpan and get an error, but the tests asserts that it doesn't...
In any case, I guess this test shows that the heartbeat behavior is funky... We don't want a transaction record to be created after the transaction is cleaned up by another pusher, do we?

Nvm, Nathan just reminded me that the timestamp cache protects us here

@nvanbenschoten
Copy link
Member Author

I'm planning to revive this today.

Write idempotency was already tested well in mvcc_test.go:
- TestMVCCWriteWithSequence
- TestMVCCDeleteRangeWithSequence
- TestMVCCIdempotentTransactions

This commit adds a higher-level test to give us more confidence.

Release note: None
@nvanbenschoten nvanbenschoten removed the X-noremind Bots won't notify about PRs with X-noremind label Nov 11, 2019
@nvanbenschoten nvanbenschoten force-pushed the nvanbenschoten/storeStuff branch 2 times, most recently from e44ca6b to d016499 Compare November 11, 2019 21:50
…tches

This change builds on the introduction of idempotency properties introduced into
the MVCC layer in cockroachdb#33001. It exploits this property to remove a previous
restriction that could prevent transactions from refreshing and result in
transaction retries. The restriction was that batches which had experienced some
success while writing could not refresh even if another part of the batch
required a refresh. This was because it was unclear which parts of the batch had
succeeded in performing writes and which parts of the batch had not. Without
this knowledge, it was unsafe to re-issue the batch because that could result
in duplicating writes (e.g. performing an increment twice).

A lot has changed since then. We now have proper sequence numbers on requests,
we no longer send `BeginTransaction` requests (which threw errors on replays), and
we now have an MVCC layer that is idempotent for writes within a transaction.
With this foundation in place, we can now safely re-issue any write within a
transaction that we're unsure about. As long as writes remain well sequenced
(writes to the same key aren't reordered), everything should behave as expected.

The best way to see that the change works is through the test cases in
`TestTxnCoordSenderRetries` that now succeed. Without cockroachdb#33001, those all fail.

Because we can now refresh mixed-success batches, this commit allows us
to remove the `MixedSuccessError` type entirely. No longer will it haunt
our sender interface with its ambiguity and awkward error wrapping. No
longer will it stand in the way of read refreshes with its ignorant
stubbornness. The days of its tyranny are over. The war is won.

Release note (performance improvement): Transactions are able to refresh
their read timestamp even after the partial success of a batch.
This option is no longer used as of 19.2, so we can remove it in 20.1.
Copy link
Member Author

@nvanbenschoten nvanbenschoten left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Updated to remove the MixedSuccessError entirely and to clean up some of the strangling migration left over from reading at specific sequence numbers.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @andreimatei and @tbg)


pkg/kv/txn_interceptor_span_refresher.go, line 185 at r6 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

this migration is done
Waiting does wonders.

Done.


pkg/roachpb/errors.proto, line 427 at r6 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

20.2 now :P

Done.


pkg/storage/replica_test.go, line 3101 at r2 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

It seems to me that the properties we want are:

  1. A request doesn't "conflict" with itself: if request is evaluated and applied and then it get evaluated and applied again, the 2nd iteration doesn't run into any trouble because of the first. If the 2nd iteration runs into problems that the first one didn't (i.e. conflicts with other concurrent transactions), I think that's fine. The client is going to arbitrarily get the results of the first or the second evaluation and I think either is fine (right?).
  2. A request R should not prevent refreshes of the transaction that don't suspect that R (or parts of it) has succeeded. I.e. the success should not prevent the RefreshRequest itself from succeeding, and then it should also not prevent R' from evaluating, where R` is R with bumped read and write timestamps. (Cause that's what refreshes do - they retry a batch, and part of that batch might have succeeded (the MixedSuccessError case)).

And so, the way I see it, the fact that the DelRange request finds new intents a second time around if it's "retried" at a different timestamp (or even if its retried at the same timestamp (according to what Nathan is showing), like it would when the DistSender retries it, seems fine to me.

I'm not sure what this does to the terminology, though. Clearly the DelRange request is not "idempotent". But it's safe to re-evaluate, at either the same timestamp (DistSender) or at a different one (after a Refresh).

My 2c.

Yes, that all matches my thinking and our offline chats. Thanks for laying it out in writing.

So to summarize: all requests other than DelRange are idempotent if issued at the same timestamp. All requests including DelRange are safe to retry if issued at the same or a higher timestamp.

@andreimatei
Copy link
Contributor

andreimatei commented Nov 12, 2019 via email

@nvanbenschoten
Copy link
Member Author

bors r+

craig bot pushed a commit that referenced this pull request Nov 12, 2019
35140: storage: employ transactional idempotency to refresh mixed-success batches r=nvanbenschoten a=nvanbenschoten

This change builds on the introduction of idempotency properties introduced into the MVCC layer in #33001. It exploits this property to remove a previous restriction that could prevent transactions from refreshing and result in transaction retries. The restriction was that batches which had experienced some success while writing could not refresh even if another part of the batch required a refresh. This was because it was unclear which parts of the batch had succeeded in performing writes and which parts of the batch had not. Without this knowledge, it was unsafe to re-issue the batch because that could result in duplicating writes (e.g. performing an increment twice).

A lot has changed since then. We now have proper sequence numbers on requests, we no longer send `BeginTransaction` requests (which threw errors on replays), and we now have an MVCC layer that is idempotent for writes within a transaction. With this foundation in place, we can now safely re-issue any write within a transaction that we're unsure about. As long as writes remain well-sequenced (writes to the same key aren't reordered), everything should behave as expected.

The best way to see that the change works is through the test cases in `TestTxnCoordSenderRetries` that now succeed. Without #33001, those all fail.

Co-authored-by: Nathan VanBenschoten <nvanbenschoten@gmail.com>
@craig
Copy link
Contributor

craig bot commented Nov 12, 2019

Build succeeded

andreimatei added a commit to andreimatei/cockroach that referenced this pull request Mar 25, 2020
The scenario that this patch addresses is the following (from cockroachdb#46431):
1. txn1 sends Put(a) + Put(b) + EndTxn
2. DistSender splits the Put(a) from the rest.
3. Put(a) succeeds, but the rest catches some retriable error.
4. TxnCoordSender gets the retriable error. The fact that a sub-batch
  succeeded is lost. We used to care about that fact, but we've
  successively gotten rid of that tracking across cockroachdb#35140 and cockroachdb#44661.
5. we refresh everything that came before this batch. The refresh
  succeeds.
6. we re-send the batch. It gets split again. The part with the EndTxn
  executes first. The transaction is now STAGING. More than that, the txn
  is in fact implicitly committed - the intent on a is already there since
  the previous attempt and, because it's at a lower timestamp than the txn
  record, it counts as golden for the purposes of verifying the implicit
  commit condition.
7. some other transaction wonders in, sees that txn1 is in its way, and
  transitions it to explicitly committed.
8. the Put(a) now tries to evaluate. It gets really confused. I guess
  that different things can happen; none of them good. One thing that I
  believe we've observed in cockroachdb#46299 is that, if there's another txn's
  intent there already, the Put will try to push it, enter the
  txnWaitQueue, eventually observe that its own txn is committed and
  return an error. The client thus gets an error (and a non-ambiguous one
  to boot) although the txn is committed. Even worse perhaps, I think it's
  possible for a request to return wrong results instead of an error.

This patch fixes it by inhibiting the parallel commit when the EndTxn
batch is retried. This way, there's never a STAGING record.

Release note (bug fix): A rare bug causing errors to be returned for
successfully committed transactions was fixed. The most common error
message was "TransactionStatusError: already committed".

Release justification: serious bug fix

Fixes cockroachdb#46341
andreimatei added a commit to andreimatei/cockroach that referenced this pull request Mar 31, 2020
The scenario that this patch addresses is the following (from cockroachdb#46431):
1. txn1 sends Put(a) + Put(b) + EndTxn
2. DistSender splits the Put(a) from the rest.
3. Put(a) succeeds, but the rest catches some retriable error.
4. TxnCoordSender gets the retriable error. The fact that a sub-batch
  succeeded is lost. We used to care about that fact, but we've
  successively gotten rid of that tracking across cockroachdb#35140 and cockroachdb#44661.
5. we refresh everything that came before this batch. The refresh
  succeeds.
6. we re-send the batch. It gets split again. The part with the EndTxn
  executes first. The transaction is now STAGING. More than that, the txn
  is in fact implicitly committed - the intent on a is already there since
  the previous attempt and, because it's at a lower timestamp than the txn
  record, it counts as golden for the purposes of verifying the implicit
  commit condition.
7. some other transaction wonders in, sees that txn1 is in its way, and
  transitions it to explicitly committed.
8. the Put(a) now tries to evaluate. It gets really confused. I guess
  that different things can happen; none of them good. One thing that I
  believe we've observed in cockroachdb#46299 is that, if there's another txn's
  intent there already, the Put will try to push it, enter the
  txnWaitQueue, eventually observe that its own txn is committed and
  return an error. The client thus gets an error (and a non-ambiguous one
  to boot) although the txn is committed. Even worse perhaps, I think it's
  possible for a request to return wrong results instead of an error.

This patch fixes it by inhibiting the parallel commit when the EndTxn
batch is retried. This way, there's never a STAGING record.

Release note (bug fix): A rare bug causing errors to be returned for
successfully committed transactions was fixed. The most common error
message was "TransactionStatusError: already committed".

Release justification: serious bug fix

Fixes cockroachdb#46341
andreimatei added a commit to andreimatei/cockroach that referenced this pull request Mar 31, 2020
The scenario that this patch addresses is the following (from cockroachdb#46431):
1. txn1 sends Put(a) + Put(b) + EndTxn
2. DistSender splits the Put(a) from the rest.
3. Put(a) succeeds, but the rest catches some retriable error.
4. TxnCoordSender gets the retriable error. The fact that a sub-batch
  succeeded is lost. We used to care about that fact, but we've
  successively gotten rid of that tracking across cockroachdb#35140 and cockroachdb#44661.
5. we refresh everything that came before this batch. The refresh
  succeeds.
6. we re-send the batch. It gets split again. The part with the EndTxn
  executes first. The transaction is now STAGING. More than that, the txn
  is in fact implicitly committed - the intent on a is already there since
  the previous attempt and, because it's at a lower timestamp than the txn
  record, it counts as golden for the purposes of verifying the implicit
  commit condition.
7. some other transaction wonders in, sees that txn1 is in its way, and
  transitions it to explicitly committed.
8. the Put(a) now tries to evaluate. It gets really confused. I guess
  that different things can happen; none of them good. One thing that I
  believe we've observed in cockroachdb#46299 is that, if there's another txn's
  intent there already, the Put will try to push it, enter the
  txnWaitQueue, eventually observe that its own txn is committed and
  return an error. The client thus gets an error (and a non-ambiguous one
  to boot) although the txn is committed. Even worse perhaps, I think it's
  possible for a request to return wrong results instead of an error.

This patch fixes it by inhibiting the parallel commit when the EndTxn
batch is retried. This way, there's never a STAGING record.

Release note (bug fix): A rare bug causing errors to be returned for
successfully committed transactions was fixed. The most common error
message was "TransactionStatusError: already committed".

Release justification: serious bug fix

Fixes cockroachdb#46341
andreimatei added a commit to andreimatei/cockroach that referenced this pull request Apr 1, 2020
The scenario that this patch addresses is the following (from cockroachdb#46431):
1. txn1 sends Put(a) + Put(b) + EndTxn
2. DistSender splits the Put(a) from the rest.
3. Put(a) succeeds, but the rest catches some retriable error.
4. TxnCoordSender gets the retriable error. The fact that a sub-batch
  succeeded is lost. We used to care about that fact, but we've
  successively gotten rid of that tracking across cockroachdb#35140 and cockroachdb#44661.
5. we refresh everything that came before this batch. The refresh
  succeeds.
6. we re-send the batch. It gets split again. The part with the EndTxn
  executes first. The transaction is now STAGING. More than that, the txn
  is in fact implicitly committed - the intent on a is already there since
  the previous attempt and, because it's at a lower timestamp than the txn
  record, it counts as golden for the purposes of verifying the implicit
  commit condition.
7. some other transaction wonders in, sees that txn1 is in its way, and
  transitions it to explicitly committed.
8. the Put(a) now tries to evaluate. It gets really confused. I guess
  that different things can happen; none of them good. One thing that I
  believe we've observed in cockroachdb#46299 is that, if there's another txn's
  intent there already, the Put will try to push it, enter the
  txnWaitQueue, eventually observe that its own txn is committed and
  return an error. The client thus gets an error (and a non-ambiguous one
  to boot) although the txn is committed. Even worse perhaps, I think it's
  possible for a request to return wrong results instead of an error.

This patch fixes it by inhibiting the parallel commit when the EndTxn
batch is retried. This way, there's never a STAGING record.

Release note (bug fix): A rare bug causing errors to be returned for
successfully committed transactions was fixed. The most common error
message was "TransactionStatusError: already committed".

Release justification: serious bug fix

Fixes cockroachdb#46341
craig bot pushed a commit that referenced this pull request Apr 1, 2020
46596: kvclient/kvcoord: inhibit parallel commit when retrying EndTxn request r=andreimatei a=andreimatei

The scenario that this patch addresses is the following (from #46431):
1. txn1 sends Put(a) + Put(b) + EndTxn
2. DistSender splits the Put(a) from the rest.
3. Put(a) succeeds, but the rest catches some retriable error.
4. TxnCoordSender gets the retriable error. The fact that a sub-batch
  succeeded is lost. We used to care about that fact, but we've
  successively gotten rid of that tracking across #35140 and #44661.
5. we refresh everything that came before this batch. The refresh
  succeeds.
6. we re-send the batch. It gets split again. The part with the EndTxn
  executes first. The transaction is now STAGING. More than that, the txn
  is in fact implicitly committed - the intent on a is already there since
  the previous attempt and, because it's at a lower timestamp than the txn
  record, it counts as golden for the purposes of verifying the implicit
  commit condition.
7. some other transaction wonders in, sees that txn1 is in its way, and
  transitions it to explicitly committed.
8. the Put(a) now tries to evaluate. It gets really confused. I guess
  that different things can happen; none of them good. One thing that I
  believe we've observed in #46299 is that, if there's another txn's
  intent there already, the Put will try to push it, enter the
  txnWaitQueue, eventually observe that its own txn is committed and
  return an error. The client thus gets an error (and a non-ambiguous one
  to boot) although the txn is committed. Even worse perhaps, I think it's
  possible for a request to return wrong results instead of an error.

This patch fixes it by inhibiting the parallel commit when the EndTxn
batch is retried. This way, there's never a STAGING record.

Release note (bug fix): A rare bug causing errors to be returned for
successfully committed transactions was fixed. The most common error
message was "TransactionStatusError: already committed".

Release justification: serious bug fix

Fixes #46341

Co-authored-by: Andrei Matei <andrei@cockroachlabs.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

4 participants