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

Serializable txn improvements #21140

Conversation

spencerkimball
Copy link
Member

The concept is straightforward: store read spans encountered during
the course of executing a transaction, up to a maximum limit of bytes.
Upon encountering a transaction retry error with reason=RETRY_SERIALIZABLE,
attempt to "update" all of the read spans in order to avoid a full txn
restart. For the case of write-only SERIALIZABLE transactions, we're now
able to set the original timestamp equal to the commit timestamp before
committing, entirely circumventing the possibility of a serializable
restart. Otherwise, if all read spans can be updated, the original timestamp
is moved to the commit timestamp (without a transaction restart), and the
EndTransaction batch is re-submitted.

UpdateRead is a new KV RPC which re-reads a span, updating the timestamp
cache and checking whether any more recent writes occurred to the span.
It uses a time-bounded iterator for efficiency. Any writes or intents
which have appeared between the transaction's original timestamp and its
commit timestamp result in an error.

This change should allow long-lived SERIALIZABLE transactions in the same
style as SNAPSHOT, and should decrease the occurence of transaction retry
errors which currently must be handled by client code.

See #21078

@spencerkimball spencerkimball requested review from bdarnell, tbg and a team December 31, 2017 16:05
@cockroach-teamcity
Copy link
Member

This change is Reviewable

@spencerkimball
Copy link
Member Author

spencerkimball commented Dec 31, 2017

[DNM] this is a prototype of improvements to serializable txns so that we can allow long-lived transactions and reduce restarts, even with the max safe timestamp changes proposed in #21056.

@petermattis
Copy link
Collaborator

I haven't looked at the code, but are you aware that SQL very rarely has pure write-only transactions? INSERT uses conditional puts to detect unique constrain violations. UPDATE, DELETE and UPSERT internally perform reads in most cases. I believe there is a fast-path in both DELETE and UPSERT to use write-only KV operations if there is only a single index and some other conditions apply, but that seems difficult to explain to users

@spencerkimball
Copy link
Member Author

That’s interesting to know. The point of this isn’t improving write-only txns, though that seemed an easy addition. But your comment makes me realize there’s further work to do with conditional puts here. Need to treat those as reads if the conditional put fails.

@spencerkimball
Copy link
Member Author

@petermattis actually I think this is correct for conditional puts as is, and won't count those as read spans. So the zero-read-span optimization ought to be able to work with "write-only" sql workloads where we're doing simple primary key constraint checking. Like big numbers of upserts.

@spencerkimball spencerkimball force-pushed the serializable-txn-improvements branch 2 times, most recently from 4ae6396 to 1ebd7d8 Compare January 1, 2018 15:31
@bdarnell
Copy link
Contributor

bdarnell commented Jan 1, 2018

This is great; I wish we had thought to do this a long time ago.

Not every request that updates the timestamp cache does it in the same way (code), but UpdateRead will update the tscache in the same way no matter what. We need to at least remember when a read span was a part of a DeleteRange request so that it can be updated properly. I'm not sure whether the ResumeSpan work in batch.go is enough for Scan and ReverseScan or if we need to look at MaxSpanRequestKeys too.


Reviewed 4 of 29 files at r1, 12 of 12 files at r2.
Review status: 16 of 36 files reviewed at latest revision, 7 unresolved discussions, some commit checks failed.


pkg/kv/txn_coord_sender.go, line 1106 at r2 (raw file):

		var readKeysSize int64
		for _, s := range readKeys {
			readKeysSize += int64(len(s.Key) + len(s.EndKey))

It's probably worthwhile to track a running size in a field of txnMeta.


pkg/kv/txn_coord_sender.go, line 1205 at r2 (raw file):

	if _, batchErr := tc.wrapped.Send(ctx, updateReadBa); batchErr != nil {
		if log.V(4) {
			log.Infof(ctx, "failed to update txn reads (%s); propagating original retry error", batchErr)

This should be a log.VEventf so it goes in the request trace. Level 4 also seems too high to me (i'd have used 2), but I don't have strong opinions about this.


pkg/roachpb/api.go, line 896 at r2 (raw file):

// they do not list isTxnRead as a flag.
func (*ConditionalPutRequest) flags() int {
	return isRead | isWrite | isTxn | isTxnWrite | updatesTSCache | consultsTSCache

ConditionalPut is the only method for which updatesTSCache and isTxnRead differ. Does it need to update the ts cache? I don't think so, by the same argument as for isTxnRead. In that case we don't need a new flag and can just reuse updatesTSCache.


pkg/roachpb/api.go, line 996 at r2 (raw file):

func (*AdminScatterRequest) flags() int             { return isAdmin | isAlone | isRange }
func (*AddSSTableRequest) flags() int               { return isWrite | isAlone | isRange }
func (*UpdateReadRequest) flags() int               { return isRead | isTxn | isRange | updatesTSCache }

If you comment out updatesTSCache on UpdateReadRequest, do a bunch of our existing tests fail? If not, we need to work on higher-level test coverage of this feature.


pkg/roachpb/api.proto, line 1115 at r2 (raw file):

// verifies that no writes have occurred since the original
// transaction timestamp and updates the timestamp cache with a span
// at the current transactin timestamp.

s/transactin/transaction/

Document what this returns (just success or failure? what kind of error?)


pkg/storage/batcheval/cmd_update_read.go, line 1 at r2 (raw file):

// Copyright 2014 The Cockroach Authors.

s/2014/2017/


pkg/storage/batcheval/cmd_update_read.go, line 81 at r2 (raw file):

		}
		// Return an error if a new intent was written to the span.
		if ts := i.Txn.Timestamp; h.Txn.OrigTimestamp.Less(ts) {

If you get an intent older than OrigTimestamp, something weird has happened. The original read should not have succeeded. We may want to return an error in this case.


Comments from Reviewable

@spencerkimball spencerkimball force-pushed the serializable-txn-improvements branch from 1ebd7d8 to a4d46b5 Compare January 2, 2018 15:55
@spencerkimball
Copy link
Member Author

I've changed UpdateRead to properly update either the read or write timestamp cache.

Could you elaborate on why you don't believe the ResumeSpan stuff is sufficient? I saw the code also in Replica.updateTimestampCache which consults the response rows in order to update the timestamp cache. I assumed that code was old; it wasn't clear to me why resume span wasn't being used instead of the convoluted checks to MaxSpanRequestKeys. ResumeSpan is never set unless MaxSpanRequestKeys is exceeded. And using ResumeSpan there would avoid the call to Key.Next() in the ScanRequest case. Maybe I'm missing something though.


Review status: 4 of 36 files reviewed at latest revision, 7 unresolved discussions.


pkg/kv/txn_coord_sender.go, line 1106 at r2 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

It's probably worthwhile to track a running size in a field of txnMeta.

Yes good idea. I already had to make the same suggested change to the write intents because it was showing up in profiling building huge transactions.


pkg/kv/txn_coord_sender.go, line 1205 at r2 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

This should be a log.VEventf so it goes in the request trace. Level 4 also seems too high to me (i'd have used 2), but I don't have strong opinions about this.

Done.


pkg/roachpb/api.go, line 896 at r2 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

ConditionalPut is the only method for which updatesTSCache and isTxnRead differ. Does it need to update the ts cache? I don't think so, by the same argument as for isTxnRead. In that case we don't need a new flag and can just reuse updatesTSCache.

I had this same thought and explored it. The problem is that these two ideas (needing to update the timestamp cache, vs. needing to rescan the span for newer updates) are different. Primarily, it occurs to me that ConditionalPut should update the timestamp cache on ConditionFailedErrors , because we're sending back a signal that can very well be interpreted as a read (the actual value is stored in the error – that's surely a read). We currently don't do this, btw. Do you think this is a problem? Seems like one to me.

In any case, because these really are subtly different concepts, I think it's wise to have a new bool.


pkg/roachpb/api.go, line 996 at r2 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

If you comment out updatesTSCache on UpdateReadRequest, do a bunch of our existing tests fail? If not, we need to work on higher-level test coverage of this feature.

I hadn't even gotten through tests because I wanted to get this idea prototyped first to see if this direction seemed like an adequate solution to the various problems people have brought up with max safe timestamp, large transactions, and serializability.


pkg/roachpb/api.proto, line 1115 at r2 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

s/transactin/transaction/

Document what this returns (just success or failure? what kind of error?)

It's just a generic error. Do you think we need to make a specific error for this for some reason?


pkg/storage/batcheval/cmd_update_read.go, line 1 at r2 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

s/2014/2017/

Done.


pkg/storage/batcheval/cmd_update_read.go, line 81 at r2 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

If you get an intent older than OrigTimestamp, something weird has happened. The original read should not have succeeded. We may want to return an error in this case.

Done.


Comments from Reviewable

@bdarnell
Copy link
Contributor

bdarnell commented Jan 3, 2018

Could you elaborate on why you don't believe the ResumeSpan stuff is sufficient?

I'm not sure about this, it just caught my attention that we're doing it differently in the two places, and I think they should be the same (i.e. change the existing code to use ResumeSpan).


Reviewed 2 of 14 files at r3, 12 of 12 files at r5.
Review status: 18 of 36 files reviewed at latest revision, 8 unresolved discussions, some commit checks failed.


pkg/roachpb/api.go, line 896 at r2 (raw file):

Previously, spencerkimball (Spencer Kimball) wrote…

I had this same thought and explored it. The problem is that these two ideas (needing to update the timestamp cache, vs. needing to rescan the span for newer updates) are different. Primarily, it occurs to me that ConditionalPut should update the timestamp cache on ConditionFailedErrors , because we're sending back a signal that can very well be interpreted as a read (the actual value is stored in the error – that's surely a read). We currently don't do this, btw. Do you think this is a problem? Seems like one to me.

In any case, because these really are subtly different concepts, I think it's wise to have a new bool.

I see. Yes, operations that update the timestamp cache should do so even if they return an error (as long as that error is based on data read. ConditionFailedError may be the only one of these for now, but it's probably better to be conservative and assume that any error from a read command should still update the timestamp cache).

Since it's impossible to continue a transaction after one of these errors, it doesn't really matter whether we add them to the UpdateRead list or not. However, I think it's more consistent to add them to the list (which will subsequently get ignored since we'll never get a retry error from a pushed timestamp) than to maintain that these are different concepts (or at least I'm not seeing the subtle difference that justifies a separate flag)


pkg/roachpb/api.proto, line 1115 at r2 (raw file):

Previously, spencerkimball (Spencer Kimball) wrote…

It's just a generic error. Do you think we need to make a specific error for this for some reason?

Mainly for logging/monitoring purposes. A failure to update the read timestamps because there was a conflicting write is different from an UpdateRead that fails due to an RPC error.


pkg/roachpb/batch.go, line 233 at r5 (raw file):

// keys affected by the request. The supplied function is called with
// each span and a bool indicating whether the span updates the write
// timestamp cache.

If it updates the write tscache, is "ReadSpanIterate" the appropriate name? Maybe TSCacheSpanIterate?


pkg/roachpb/batch.go, line 247 at r5 (raw file):

			var writeCache bool
			switch req.(type) {
			case *DeleteRangeRequest:

We should avoid duplicating this knowledge here. Maybe a new api.go flag for updatesWriteTSCache?


pkg/roachpb/batch_test.go, line 247 at r5 (raw file):

	var spans []Span
	var delRngSpans []Span
	fn := func(span Span, delRng bool) {

s/delRng/write/g. We may have more like DelRange in the future.


pkg/storage/batcheval/cmd_update_read.go, line 20 at r5 (raw file):

	"context"

	"google.golang.org/appengine/log"

Wrong log package.


Comments from Reviewable

@spencerkimball
Copy link
Member Author

See #21304 for fixes to the resume spans.


Review status: 18 of 36 files reviewed at latest revision, 8 unresolved discussions, some commit checks failed.


pkg/roachpb/api.go, line 896 at r2 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

I see. Yes, operations that update the timestamp cache should do so even if they return an error (as long as that error is based on data read. ConditionFailedError may be the only one of these for now, but it's probably better to be conservative and assume that any error from a read command should still update the timestamp cache).

Since it's impossible to continue a transaction after one of these errors, it doesn't really matter whether we add them to the UpdateRead list or not. However, I think it's more consistent to add them to the list (which will subsequently get ignored since we'll never get a retry error from a pushed timestamp) than to maintain that these are different concepts (or at least I'm not seeing the subtle difference that justifies a separate flag)

I think it would be very unconservative indeed to drastically change the behavior by setting the timestamp cache on all read errors... Better to just correct the ConditionFailedError; for the life of me, I can't think of another case – although as you'll see in #21304 I found yet another bug with how we have been updating the timestamp cache. eek.

I think we still need this flag. The difference is subtle, but it's important for our ability to avoid restarts on common insert / upsert workloads which are only doing uniqueness constraint checking on the primary key via conditional puts. If we eliminate this distinction and require UpdateRead on ConditionalPuts, then we'll have to restart if any other reader set the timestamp cache where we're inserting, moving the txn timestamp forward.


pkg/roachpb/api.proto, line 1115 at r2 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

Mainly for logging/monitoring purposes. A failure to update the read timestamps because there was a conflicting write is different from an UpdateRead that fails due to an RPC error.

I thought RPC errors ended up as SendError. We have lots of generic errors returned via fmt.Errorf. My feeling is not to add to errors.proto unless callers need to use the error.


pkg/roachpb/batch.go, line 233 at r5 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

If it updates the write tscache, is "ReadSpanIterate" the appropriate name? Maybe TSCacheSpanIterate?

I renamed everything given the presence of read and write spans. The RPC is now UpdateSpan instead of UpdateRead and there are two flags: isUpdateRead and isUpdateWrite.


pkg/roachpb/batch.go, line 247 at r5 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

We should avoid duplicating this knowledge here. Maybe a new api.go flag for updatesWriteTSCache?

See above.


pkg/roachpb/batch_test.go, line 247 at r5 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

s/delRng/write/g. We may have more like DelRange in the future.

Done.


pkg/storage/batcheval/cmd_update_read.go, line 20 at r5 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

Wrong log package.

Done.


Comments from Reviewable

@spencerkimball spencerkimball force-pushed the serializable-txn-improvements branch 2 times, most recently from 526d941 to d1b1d11 Compare January 8, 2018 01:53
@bdarnell
Copy link
Contributor

bdarnell commented Jan 9, 2018

Reviewed 7 of 29 files at r1, 2 of 14 files at r6, 17 of 17 files at r9.
Review status: 32 of 43 files reviewed at latest revision, 5 unresolved discussions, some commit checks failed.


pkg/kv/txn_coord_sender.go, line 71 at r9 (raw file):

	keysSize int64

	// updateKeys stores key ranges which were read during a transaction.

updateKeys sounds weird to me - we haven't updated these keys. Maybe depKeys, since we depend on them being unchanged? (I'd also suggest EnsureUnchanged instead of UpdateSpan for the new command)


pkg/roachpb/api.go, line 896 at r2 (raw file):

Previously, spencerkimball (Spencer Kimball) wrote…

I think it would be very unconservative indeed to drastically change the behavior by setting the timestamp cache on all read errors... Better to just correct the ConditionFailedError; for the life of me, I can't think of another case – although as you'll see in #21304 I found yet another bug with how we have been updating the timestamp cache. eek.

I think we still need this flag. The difference is subtle, but it's important for our ability to avoid restarts on common insert / upsert workloads which are only doing uniqueness constraint checking on the primary key via conditional puts. If we eliminate this distinction and require UpdateRead on ConditionalPuts, then we'll have to restart if any other reader set the timestamp cache where we're inserting, moving the txn timestamp forward.

Now that we set the timestamp cache on ConditionFailedError (#21297), is the distinction between isUpdateRead (formerly isTxnRead) and updatesTSCache still relevant? If so, can you explain the "common insert/upsert workload" you're concerned about?


pkg/roachpb/batch.go, line 233 at r5 (raw file):

Previously, spencerkimball (Spencer Kimball) wrote…

I renamed everything given the presence of read and write spans. The RPC is now UpdateSpan instead of UpdateRead and there are two flags: isUpdateRead and isUpdateWrite.

While UpdateSpan (kind of) works for the new rpc method, I think it's confusing to have all these other variables with UpdateSpan in their names. It sounds like it's referring to spans that have been updated/written to, instead of spans whose status needs to be updated when our timestamp is pushed.


pkg/storage/engine/mvcc.go, line 1912 at r9 (raw file):

// range of write intents. In this case, checks which make sense only
// in the context of resolving a specific point intent are skipped.
func mvccResolveWriteIntent(

Document the new return value.


Comments from Reviewable

@spencerkimball spencerkimball force-pushed the serializable-txn-improvements branch from d1b1d11 to 11e7909 Compare January 21, 2018 21:56
@spencerkimball spencerkimball force-pushed the serializable-txn-improvements branch from 11e7909 to 2bd2411 Compare January 30, 2018 00:36
@spencerkimball spencerkimball requested review from a team January 30, 2018 00:36
@spencerkimball
Copy link
Member Author

@bdarnell, this has undergone some big changes due to @tschottdorf's suggestion that we handle things other than just SERIALIZABLE_RETRY. I think we should schedule some time tomorrow to go through it together.


Review status: 0 of 42 files reviewed at latest revision, 5 unresolved discussions, some commit checks pending.


pkg/kv/txn_coord_sender.go, line 71 at r9 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

updateKeys sounds weird to me - we haven't updated these keys. Maybe depKeys, since we depend on them being unchanged? (I'd also suggest EnsureUnchanged instead of UpdateSpan for the new command)

I did some pretty big changes here. No longer called updateKeys.


pkg/roachpb/api.go, line 896 at r2 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

Now that we set the timestamp cache on ConditionFailedError (#21297), is the distinction between isUpdateRead (formerly isTxnRead) and updatesTSCache still relevant? If so, can you explain the "common insert/upsert workload" you're concerned about?

Now that we're handling WriteTooOldErrors as well (most recent change), we must update ConditionalPuts. There still seems to be no good way to avoid having these new flags though. They are subtly different from updatesTSCache. If we get rid of them, we'll end up needing to special case other code, such as when we collect the update spans.


pkg/roachpb/batch.go, line 233 at r5 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

While UpdateSpan (kind of) works for the new rpc method, I think it's confusing to have all these other variables with UpdateSpan in their names. It sounds like it's referring to spans that have been updated/written to, instead of spans whose status needs to be updated when our timestamp is pushed.

Lots of refactoring...let me know if this is still a problem.


pkg/storage/engine/mvcc.go, line 1912 at r9 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

Document the new return value.

Done.


Comments from Reviewable

@spencerkimball spencerkimball requested review from andreimatei and removed request for tbg January 30, 2018 00:58
@bdarnell
Copy link
Contributor

Reviewed 10 of 10 files at r13.
Review status: all files reviewed at latest revision, 1 unresolved discussion.


pkg/kv/txn_coord_sender.go, line 584 at r11 (raw file):

Previously, spencerkimball (Spencer Kimball) wrote…

I think not using the word Ambiguous is a good idea. How about MixedSuccessError to more clearly connote failure is involved and the results are hard to interpret?

MixedSuccessError sounds good to me.


Comments from Reviewable

@spencerkimball spencerkimball force-pushed the serializable-txn-improvements branch 2 times, most recently from 76eb1fe to 0cd1bd5 Compare February 1, 2018 03:44
@spencerkimball
Copy link
Member Author

Had to add a new version as it turns out that if you send an old server a Request type it doesn't understand in a batch it panics. We should return an error instead. I filed #22290 to track that.

@spencerkimball spencerkimball force-pushed the serializable-txn-improvements branch from 0cd1bd5 to 976dae7 Compare February 1, 2018 12:06
@spencerkimball
Copy link
Member Author

Did some more thinking this morning about InitPut and realized we shouldn't need to update the timestamp cache or track refresh spans for it. Updated the comments and added unittests.

Also improved comments in mvccResolveIntent.

@tbg
Copy link
Member

tbg commented Feb 1, 2018 via email

@spencerkimball spencerkimball force-pushed the serializable-txn-improvements branch from 976dae7 to 240b329 Compare February 1, 2018 15:42
@spencerkimball
Copy link
Member Author

@tschottdorf thanks for pointing that out. I closed my issue as a duplicate.

The concept is straightforward: store spans encountered during
the course of executing a transaction, up to a maximum limit of bytes.
Upon encountering a transaction retry error with `reason=RETRY_SERIALIZABLE`,
attempt to "refresh" all of the spans in order to avoid a full txn
restart. For the case of write-only SERIALIZABLE transactions, we're now
able to set the original timestamp equal to the commit timestamp before
committing, entirely circumventing the possibility of a serializable
restart. Otherwise, if all spans can be updated, the original timestamp
is moved to the commit timestamp (without a transaction restart), and the
`EndTransaction` batch is re-submitted.

`Refresh(Range)` is a new KV RPC which re-reads a span, updating the timestamp
cache and checking whether any more recent writes occurred to the span.
It uses a time-bounded iterator for efficiency. Any writes or intents
which have appeared between the transaction's original timestamp and its
commit timestamp result in an error. Note that `Refresh(Range)` has a bool
indicating whether it should update the read or write timestamp cache.
Only `DeleteRange` creates update spans which need to update the write
timestamp cache.

This change should allow long-lived SERIALIZABLE transactions in the same
style as SNAPSHOT, and should decrease the occurence of transaction retry
errors which currently must be handled by client code.

See cockroachdb#21078

Release note (sql): significantly reduce the likelihood of serializable
restarts seen by clients due to concurrent workloads.
@spencerkimball spencerkimball force-pushed the serializable-txn-improvements branch from 240b329 to f7f6472 Compare February 1, 2018 16:15
@bdarnell
Copy link
Contributor

bdarnell commented Feb 1, 2018

Reviewed 8 of 8 files at r14.
Review status: all files reviewed at latest revision, 1 unresolved discussion, all commit checks successful.


Comments from Reviewable

@spencerkimball spencerkimball merged commit e1e0e22 into cockroachdb:master Feb 1, 2018
@spencerkimball spencerkimball deleted the serializable-txn-improvements branch February 1, 2018 18:27
andreimatei added a commit to andreimatei/cockroach that referenced this pull request Mar 19, 2018
... if it's still possible for the SQL connExecutor to retry them (i.e.
we haven't sent results to the client for the respective txn yet).

This patch re-introduces code we used to have, but was deleted in cockroachdb#21140.
The motivation for deleting the code was that retriable errors were
supposed to be rarer because of the "read refreshing" mechanism, but
that mechanism doesn't apply to transactions that "observe their commit
timestamp". A txn can observe its timestamp by calling
cluster_logical_timestamp(), or in other ways. In particular, schema
change txns always observe their ts.

Fixes cockroachdb#22933

Release note: Retriable errors on schema change operations are less
likely to be returned to clients; more operations are retried
internally.
andreimatei added a commit to andreimatei/cockroach that referenced this pull request Mar 19, 2018
... if it's still possible for the SQL connExecutor to retry them (i.e.
we haven't sent results to the client for the respective txn yet).

This patch re-introduces code we used to have, but was deleted in cockroachdb#21140.
The motivation for deleting the code was that retriable errors were
supposed to be rarer because of the "read refreshing" mechanism, but
that mechanism doesn't apply to transactions that "observe their commit
timestamp". A txn can observe its timestamp by calling
cluster_logical_timestamp(), or in other ways. In particular, schema
change txns always observe their ts.

Fixes cockroachdb#22933

Release note: Retriable errors on schema change operations are less
likely to be returned to clients; more operations are retried
internally.
andreimatei added a commit to andreimatei/cockroach that referenced this pull request Mar 20, 2018
... if it's still possible for the SQL connExecutor to retry them (i.e.
we haven't sent results to the client for the respective txn yet).

This patch re-introduces code we used to have, but was deleted in cockroachdb#21140.
The motivation for deleting the code was that retriable errors were
supposed to be rarer because of the "read refreshing" mechanism, but
that mechanism doesn't apply to transactions that "observe their commit
timestamp". A txn can observe its timestamp by calling
cluster_logical_timestamp(), or in other ways. In particular, schema
change txns always observe their ts.

Fixes cockroachdb#22933

Release note: Retriable errors on schema change operations are less
likely to be returned to clients; more operations are retried
internally.
nvanbenschoten added a commit to nvanbenschoten/cockroach that referenced this pull request Mar 27, 2018
Proposals that return errors but still want to go through Raft to
lay down intents were refactored away in cockroachdb#21140. This allows us to
simplify the error handling in `evaluateProposalInner`.

Release note: None
andreimatei added a commit to andreimatei/cockroach that referenced this pull request Aug 1, 2018
There used to be a discrepancy between deadline exceeded errors detected
on the server side and those detected on the client side for elided
EndTransaction requests. No longer since cockroachdb#21140.

Release note: None
craig bot pushed a commit that referenced this pull request Aug 1, 2018
28117: kv: get rid of test FUD r=andreimatei a=andreimatei

Some TCS tests were employing a teardownHeartbeat() method. That's not
needed - all the test indirectly stop a stopper that also tears down
dangling heartbeat loops.

Release note: None

28124: client: delete stale comment r=andreimatei a=andreimatei

There used to be a discrepancy between deadline exceeded errors detected
on the server side and those detected on the client side for elided
EndTransaction requests. No longer since #21140.

Release note: None

Co-authored-by: Andrei Matei <[email protected]>
nvanbenschoten added a commit to nvanbenschoten/cockroach that referenced this pull request Mar 15, 2019
Proposals that return errors but still want to go through Raft to
lay down intents were refactored away in cockroachdb#21140. This allows us to
simplify the error handling in `evaluateProposalInner`.

Release note: None
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.

5 participants