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

kv: commit-wait before running commit triggers and resolving intents #63971

Merged

Conversation

nvanbenschoten
Copy link
Member

@nvanbenschoten nvanbenschoten commented Apr 21, 2021

Fixes a serious bug revealed by #63747.

This commit fixes a bug revealed by kvnemesis where a range-merge watcher on the
right-hand side of a range merge could incorrectly determine that a range merge
txn had succeeded, when in reality, it had failed. The watcher would then put
the RHS leaseholder replica into a stalled state by setting r.mu.destroyStatus
to destroyReasonMergePending, effectively stalling any operation on the range
indefinitely.

The setup for this bug was that a range was operating with a global_reads zone
configuration attribute, so it was pushing all writers into the future. The
range was split and then rapidly merged back together. During the merge txn, a
range-merge watcher (see maybeWatchForMergeLocked) began monitoring the state
of the range merge txn. The problem was that at the time that the range merge
txn committed, neither the meta descriptor version written by the merge or even
the meta descriptor version written by the split were visible to the watcher's
follow-up query. Because the watcher read below the split txn's descriptor, it
came to the wrong conclusion about the merge.

It is interesting to think about what is going wrong here, because it's not
immediately obvious who is at fault. If a transaction has a commit timestamp in
the future of present time, it will need to commit-wait before acknowledging the
client. Typically, this is performed in the TxnCoordSender after the transaction
has committed and resolved its intents (see TxnCoordSender.maybeCommitWait). It
is safe to wait after a future-time transaction has committed and resolved
intents without compromising linearizability because the uncertainty interval of
concurrent and later readers ensures atomic visibility of the effects of the
transaction. In other words, all of the transaction's intents will become
visible and will remain visible at once, which is sometimes called "monotonic
reads". This is true even if the resolved intents are at a high enough timestamp
such that they are not visible to concurrent readers immediately after they are
resolved, but only become visible sometime during the writer's commit-wait
sleep. This property is central to the correctness of non-blocking transactions.
See: https://github.com/cockroachdb/cockroach/blob/master/docs/RFCS/20200811_non_blocking_txns.md

However, if a transaction has a commit trigger, the side-effects of the trigger
will go into effect immediately upon applying the corresponding Raft log entry.
This poses a problem, because we do not want part of a transaction's effects
(e.g. its commit trigger) to become visible to onlookers before the rest of its
effects do (e.g. its intent writes).

To avoid this problem, this commit adds special server-side logic to perform the
commit-wait stage of a transaction with a commit trigger early, before its
EndTxn evaluates and its commit trigger fires. This results in the transaction
waiting longer to commit, run its commit trigger, and resolve its intents, but
it is otherwise safe and effective.

Interestingly, this is quite similar to how Spanner handles its commit-wait rule:

Before allowing any coordinator replica to apply the commit record, the
coordinator leader waits until TT.after(s), so as to obey the commit-wait rule
described in Section 4.1.2. Because the coordinator leader chose s based on
TT.now().latest, and now waits until that timestamp is guaranteed to be in the
past, the expected wait is at least 2 ∗ . This wait is typically overlapped with
Paxos communication. After commit wait, the coordinator sends the commit
timestamp to the client and all other participant leaders. Each participant
leader logs the transaction’s outcome through Paxos. All participants apply at
the same timestamp and then release locks.

Of course, the whole point of non-blocking transactions is that we release locks
early and use clocks (through uncertainty intervals + a reader-side commit-wait
rule) to enforce consistency, so we don't want to make this change for standard
transactions.

Before this change, I could hit the bug in about 5 minutes of stressing
kvnemesis on a roachprod cluster. After this change, I've been able to run
kvnemesis for a few hours without issue.

Release note (bug fix): Fixed a rare bug present in betas where rapid range
splits and merges on a GLOBAL table could lead to a stuck leaseholder replica.
The situation is no longer possible.

cc. @cockroachdb/kv

@cockroach-teamcity
Copy link
Member

This change is Reviewable

@nvanbenschoten nvanbenschoten force-pushed the nvanbenschoten/commitWaitOnServer branch from 1093291 to 2967b1c Compare April 21, 2021 05:55
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 @nvanbenschoten and @tbg)


pkg/kv/kvserver/replica_send.go, line 370 at r1 (raw file):

		// to move a transaction to a COMMITTED status, but the transaction wanted
		// to perform a commit-wait before committing and resolving intents. We
		// don't want to do so while holding latches, so we threw the error to up

"to up here" refers to an "up here" and "down there" that might not be very clear to a reader - perhaps one holding the map upside down. Let's say something else.


pkg/kv/kvserver/batcheval/cmd_end_transaction.go, line 369 at r1 (raw file):

		)
		if err != nil {
			return result.Result{}, roachpb.NewReplicaCorruptionError(err)

What's up with this change, and the new ReplicaCorruptionErrors below? Perhaps it deserves a separate commit?


pkg/kv/kvserver/batcheval/cmd_end_transaction.go, line 643 at r1 (raw file):

	// will need to commit-wait before acknowledging the client. Typically, this
	// is performed in the TxnCoordSender after the transaction has committed and
	// resolved its intents (see TxnCoordSender.maybeCommitWait). It is safe to

You say this here and in the commit message, but is the "after... it resolved its intents" part true? I don't see the EndTxn processing waiting for intent resolution when the resolution takes the (normal) async path. In fact, we've wanted to do the waiting as a form of pushback on writers, haven't we?

if err := r.store.intentResolver.CleanupTxnIntentsAsync(


pkg/kv/kvserver/batcheval/cmd_end_transaction.go, line 674 at r1 (raw file):

		// Replica.executeBatchWithConcurrencyRetries catch this error, drop
		// latches, perform the commit-wait, re-acquire latches, and retry this
		// request.

spell out here or above that replicating the command with the effects of the {split,merge}Trigger below is the part can only be done after waiting.


pkg/kv/kvserver/batcheval/cmd_end_transaction.go, line 675 at r1 (raw file):

		// latches, perform the commit-wait, re-acquire latches, and retry this
		// request.
		return result.Result{}, roachpb.NewCommitWaitBeforeIntentResolutionError(txn.WriteTimestamp)

mmm this kind of "try evaluation, fail, catch the error higher up, try again" is a kind of action at a distance that's hard to trace. If we can, I think we should avoid.
Can an EndTxn with a trigger be batched with other requests that might change the commit timestamp? I'd say probably not, and even if it could I'm not sure we'd end up here if the write ts was pushed in the same battch. So I'd say let's recognize an EndTxn{commitTrigger} request in replica_send, do the waiting, and here just assert that there's no synthetic timestamp. WDYT?


pkg/roachpb/errors.proto, line 489 at r1 (raw file):

}

// A CommitWaitBeforeIntentResolutionError indicates that the request attempted

s/the request/an EndTxn(commit=true. Are there others? It'd be good to be specific.


pkg/roachpb/errors.proto, line 496 at r1 (raw file):

// effects (i.e. its intent writes) will be visible upon resolution.
//
// This error is handled by the Store and should not escape to higher levels.

s/should not/does not :)

@tbg tbg requested a review from andreimatei April 22, 2021 09:14
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.

This looks good to me, mod the comments. Random question I have, the way splits and merges work at all here is that they get bumped but lay down their intents, then can refresh (since they only access the descriptors they have intents on anyway) and then go for the commit trigger (which has the wait inserted here), right? And it doesn't seem that the commit-wait has any effect on user traffic.

Reviewed 12 of 12 files at r1.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @andreimatei and @nvanbenschoten)


pkg/kv/kvserver/batcheval/cmd_end_transaction.go, line 675 at r1 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

mmm this kind of "try evaluation, fail, catch the error higher up, try again" is a kind of action at a distance that's hard to trace. If we can, I think we should avoid.
Can an EndTxn with a trigger be batched with other requests that might change the commit timestamp? I'd say probably not, and even if it could I'm not sure we'd end up here if the write ts was pushed in the same battch. So I'd say let's recognize an EndTxn{commitTrigger} request in replica_send, do the waiting, and here just assert that there's no synthetic timestamp. WDYT?

I had the same thought. If we can pull the logic up and avoid the yo-yo (leaving only a hard error here), I think that would be preferable from a simplicity point of view.


pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go, line 1082 at r1 (raw file):

//
// - futureCommitTime: configures whether or not EndTxn request is sent on
//     behalf of a transaction with a synthetic commit timestamp that leads the

so syntheticCommitTime and futureCommitTime overlap? I see that you're skipping this below, but it looks like it would be less confusing if you had two dimensions:

testutils.RunTrueOrFalse(t, "commitTrigger", func(t *testing.T, commitTrigger bool) {
	for _, cfg := range []struct {
		name     string
		commitTS func(now hlc.Timestamp) hlc.Timestamp
	}{
		{"past", func(now hlc.Timestamp) hlc.Timestamp { return now }},
		{"past-syn", func(now hlc.Timestamp) hlc.Timestamp { return now.WithSynthetic(true) }},
		{"future-syn", func(now hlc.Timestamp) hlc.Timestamp { return now.Add(100, 0).WithSynthetic(true) }},
	} {
		t.Run(cfg.name, func(t *testing.T) {
			// ...
		})
	}
})

@nvanbenschoten nvanbenschoten force-pushed the nvanbenschoten/commitWaitOnServer branch from 2967b1c to a30f57e Compare April 27, 2021 04:57
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.

Thanks for the reviews! I restructured this to be more in line with the control flow you two preferred. I was initially concerned that waiting before evaluation has determined that the transaction will commit successfully or not would lead to wasted work, and that it would be nice to be able to handle cases where the batch is bumped to a higher timestamp after acquiring latches. The first is still a minor concern, but I don't think it will matter in practice. The second concern was misguided, as trying to handle this case opens us up to an infinite retry loop where the request is continuously pushed and rejected with a CommitWaitBeforeIntentResolutionError. Even though we don't expect this situation to be possible, it feels much safer to reject this kind of situation with a hard error instead of letting it silently run wild. PTAL at the new approach.

Random question I have, the way splits and merges work at all here is that they get bumped but lay down their intents, then can refresh (since they only access the descriptors they have intents on anyway) and then go for the commit trigger (which has the wait inserted here), right?

Yes, exactly.

And it doesn't seem that the commit-wait has any effect on user traffic.

I haven't noticed any. The intent writes laid down by a split or merge should not be read consistently by foreground traffic. All meta range reads use the READ_UNCOMMITTED consistency level, so they won't block on the intents. The only potential for blocking is due to latching, which is why we're careful not to commit-wait while holding latches here.

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


pkg/kv/kvserver/replica_send.go, line 370 at r1 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

"to up here" refers to an "up here" and "down there" that might not be very clear to a reader - perhaps one holding the map upside down. Let's say something else.

Removed.


pkg/kv/kvserver/batcheval/cmd_end_transaction.go, line 369 at r1 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

What's up with this change, and the new ReplicaCorruptionErrors below? Perhaps it deserves a separate commit?

It was a small refactor to break the assumption that all errors from RunCommitTrigger amount to replica corruption errors.


pkg/kv/kvserver/batcheval/cmd_end_transaction.go, line 643 at r1 (raw file):

Previously, andreimatei (Andrei Matei) wrote…
	// will need to commit-wait before acknowledging the client. Typically, this
	// is performed in the TxnCoordSender after the transaction has committed and
	// resolved its intents (see TxnCoordSender.maybeCommitWait). It is safe to

You say this here and in the commit message, but is the "after... it resolved its intents" part true? I don't see the EndTxn processing waiting for intent resolution when the resolution takes the (normal) async path. In fact, we've wanted to do the waiting as a form of pushback on writers, haven't we?

if err := r.store.intentResolver.CleanupTxnIntentsAsync(

Deleted.


pkg/kv/kvserver/batcheval/cmd_end_transaction.go, line 674 at r1 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

spell out here or above that replicating the command with the effects of the {split,merge}Trigger below is the part can only be done after waiting.

Deleted.


pkg/kv/kvserver/batcheval/cmd_end_transaction.go, line 675 at r1 (raw file):

Previously, tbg (Tobias Grieger) wrote…

I had the same thought. If we can pull the logic up and avoid the yo-yo (leaving only a hard error here), I think that would be preferable from a simplicity point of view.

Done.


pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go, line 1082 at r1 (raw file):

Previously, tbg (Tobias Grieger) wrote…

so syntheticCommitTime and futureCommitTime overlap? I see that you're skipping this below, but it looks like it would be less confusing if you had two dimensions:

testutils.RunTrueOrFalse(t, "commitTrigger", func(t *testing.T, commitTrigger bool) {
	for _, cfg := range []struct {
		name     string
		commitTS func(now hlc.Timestamp) hlc.Timestamp
	}{
		{"past", func(now hlc.Timestamp) hlc.Timestamp { return now }},
		{"past-syn", func(now hlc.Timestamp) hlc.Timestamp { return now.WithSynthetic(true) }},
		{"future-syn", func(now hlc.Timestamp) hlc.Timestamp { return now.Add(100, 0).WithSynthetic(true) }},
	} {
		t.Run(cfg.name, func(t *testing.T) {
			// ...
		})
	}
})

Done.


pkg/roachpb/errors.proto, line 489 at r1 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

s/the request/an EndTxn(commit=true. Are there others? It'd be good to be specific.

Removed.


pkg/roachpb/errors.proto, line 496 at r1 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

s/should not/does not :)

Removed.

This commit adds support to `(*hlc.Clock).SleepUntil` to listen to
context cancellation and terminate its sleep early if the context
that is passed to it is canceled.
@nvanbenschoten nvanbenschoten force-pushed the nvanbenschoten/commitWaitOnServer branch 2 times, most recently from 288fccb to 80e55cf Compare April 27, 2021 15:38
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_strong:

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


pkg/kv/kvserver/replica_send.go, line 146 at r3 (raw file):

	}
	et := args.(*roachpb.EndTxnRequest)
	if !et.Commit || et.InternalCommitTrigger == nil {

should we assert around here that, if there's a CommitTrigger, then there aren't any other requests in the batch other than the EndTxn? Cause if there were, then those might cause the WriteTimestamp to be bumped later, and could hit the assertion in RunCommitTrigger, right?

Fixes a serious bug revealed by cockroachdb#63747.

This commit fixes a bug revealed by kvnemesis where a range-merge watcher on the
right-hand side of a range merge could incorrectly determine that a range merge
txn had succeeded, when in reality, it had failed. The watcher would then put
the RHS leaseholder replica into a stalled state by setting `r.mu.destroyStatus`
to `destroyReasonMergePending`, effectively stalling any operation on the range
indefinitely.

The setup for this bug was that a range was operating with a `global_reads` zone
configuration attribute, so it was pushing all writers into the future. The
range was split and then rapidly merged back together. During the merge txn, a
range-merge watcher (see `maybeWatchForMergeLocked`) began monitoring the state
of the range merge txn. The problem was that at the time that the range merge
txn committed, neither the meta descriptor version written by the merge or even
the meta descriptor version written by the split were visible to the watcher's
follow-up query. Because the watcher read below the split txn's descriptor, it
came to the wrong conclusion about the merge.

It is interesting to think about what is going wrong here, because it's not
immediately obvious who is at fault. If a transaction has a commit timestamp in
the future of present time, it will need to commit-wait before acknowledging the
client. Typically, this is performed in the TxnCoordSender after the transaction
has committed and resolved its intents (see TxnCoordSender.maybeCommitWait). It
is safe to wait after a future-time transaction has committed and resolved
intents without compromising linearizability because the uncertainty interval of
concurrent and later readers ensures atomic visibility of the effects of the
transaction. In other words, all of the transaction's intents will become
visible and will remain visible at once, which is sometimes called "monotonic
reads". This is true even if the resolved intents are at a high enough timestamp
such that they are not visible to concurrent readers immediately after they are
resolved, but only become visible sometime during the writer's commit-wait
sleep. This property is central to the correctness of non-blocking transactions.
See: https://github.com/cockroachdb/cockroach/blob/master/docs/RFCS/20200811_non_blocking_txns.md

However, if a transaction has a commit trigger, the side-effects of the trigger
will go into effect immediately upon applying to corresponding Raft log entry.
This poses a problem, because we do not want part of a transaction's effects
(e.g. its commit trigger) to become visible to onlookers before the rest of its
effects do (e.g. its intent writes).

To avoid this problem, this commit adds special server-side logic to perform the
commit-wait stage of a transaction with a commit trigger early, before its
EndTxn evaluates and its commit trigger fires. This results in the transaction
waiting longer to commit, run its commit trigger, and resolve its intents, but
it is otherwise safe and effective.

Interestingly, this is quite similar to how Spanner handles its commit-wait rule:

> Before allowing any coordinator replica to apply the commit record, the
> coordinator leader waits until TT.after(s), so as to obey the commit-wait rule
> described in Section 4.1.2. Because the coordinator leader chose s based on
> TT.now().latest, and now waits until that timestamp is guaranteed to be in the
> past, the expected wait is at least 2 ∗ . This wait is typically overlapped with
> Paxos communication. After commit wait, the coordinator sends the commit
> timestamp to the client and all other participant leaders. Each participant
> leader logs the transaction’s outcome through Paxos. All participants apply at
> the same timestamp and then release locks.

Of course, the whole point of non-blocking transactions is that we release locks
early and use clocks (through uncertainty intervals + a reader-side commit-wait
rule) to enforce consistency, so we don't want to make this change for standard
transactions.

Before this change, I could hit the bug in about 5 minutes of stressing
kvnemesis on a roachprod cluster. After this change, I've been able to run
kvnemesis for a few hours without issue.

Release note (bug fix): Fixed a rare bug present in betas where rapid range
splits and merges on a GLOBAL table could lead to a stuck leaseholder replica.
The situation is no longer possible.
Allowing this is a major risk, as it would break node liveness updates, which
perform a 1PC transaction with a commit trigger and can not tolerate being
pushed into the future. We could try to block this at zone config update time,
but due to the complexities of zone config inheritance, we would either need to
be overly restrictive or risk missing cases which could wedge a cluster. This
commit opts to disable the issue more directly.
@nvanbenschoten nvanbenschoten force-pushed the nvanbenschoten/commitWaitOnServer branch from 80e55cf to 0abe89a Compare April 27, 2021 18:27
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.

TFTR!

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


pkg/kv/kvserver/replica_send.go, line 146 at r3 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

should we assert around here that, if there's a CommitTrigger, then there aren't any other requests in the batch other than the EndTxn? Cause if there were, then those might cause the WriteTimestamp to be bumped later, and could hit the assertion in RunCommitTrigger, right?

We do need to support 1PC transactions that contain commit-triggers (e.g. node liveness updates), so we can't make such an assertion here. But these types of transactions should not be run on a global_read range, and I'm ok with the assertion error we have being returned if they are. I added a follow on commit to make sure that we don't get into trouble with the global_reads attribute being set on the node liveness range.

@nvanbenschoten
Copy link
Member Author

bors r+

@craig
Copy link
Contributor

craig bot commented Apr 27, 2021

Build succeeded:

@craig craig bot merged commit 29305bf into cockroachdb:master Apr 27, 2021
@nvanbenschoten nvanbenschoten deleted the nvanbenschoten/commitWaitOnServer branch April 28, 2021 03:27
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants