diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html
index c478d897c640..bd053e5909cb 100644
--- a/docs/generated/settings/settings.html
+++ b/docs/generated/settings/settings.html
@@ -59,6 +59,6 @@
trace.debug.enable | boolean | false | if set, traces for recent requests can be seen in the /debug page |
trace.lightstep.token | string |
| if set, traces go to Lightstep using this token |
trace.zipkin.collector | string |
| if set, traces go to the given Zipkin instance (example: '127.0.0.1:9411'); ignored if trace.lightstep.token is set. |
-version | custom validation | 2.0-5 | set the active cluster version in the format '.'. |
+version | custom validation | 2.0-6 | set the active cluster version in the format '.'. |
diff --git a/pkg/cmd/roachtest/tpcc.go b/pkg/cmd/roachtest/tpcc.go
index 0d2d8c8175da..cf990e1d099f 100644
--- a/pkg/cmd/roachtest/tpcc.go
+++ b/pkg/cmd/roachtest/tpcc.go
@@ -369,7 +369,7 @@ func registerTPCCBench(r *registry) {
LoadWarehouses: 1000,
EstimatedMax: 325,
- StoreDirVersion: "2.0-5",
+ StoreDirVersion: "2.0-6",
},
{
Nodes: 3,
@@ -394,7 +394,7 @@ func registerTPCCBench(r *registry) {
LoadWarehouses: 5000,
EstimatedMax: 500,
- StoreDirVersion: "2.0-5",
+ StoreDirVersion: "2.0-6",
},
// objective 3, key result 2.
{
diff --git a/pkg/internal/client/batch.go b/pkg/internal/client/batch.go
index 19c5a3a38b3e..ba714452693f 100644
--- a/pkg/internal/client/batch.go
+++ b/pkg/internal/client/batch.go
@@ -167,8 +167,8 @@ func (b *Batch) fillResults(ctx context.Context) {
// instead; this effectively just leaks here.
// TODO(tschottdorf): returning an error here seems
// to get swallowed.
- panic(errors.Errorf("not enough responses for calls: %+v, %+v",
- b.reqs, b.response))
+ panic(errors.Errorf("not enough responses for calls: (%T) %+v\nresponses: %+v",
+ args, args, b.response))
}
}
}
diff --git a/pkg/internal/client/client_test.go b/pkg/internal/client/client_test.go
index 6033753970b9..28e4bdc04555 100644
--- a/pkg/internal/client/client_test.go
+++ b/pkg/internal/client/client_test.go
@@ -798,14 +798,18 @@ func TestReadConsistencyTypes(t *testing.T) {
t.Run(rc.String(), func(t *testing.T) {
// Mock out DistSender's sender function to check the read consistency for
// outgoing BatchRequests and return an empty reply.
- factory := client.TxnSenderFactoryFunc(func(_ client.TxnType) client.TxnSender {
- return client.TxnSenderFunc(func(_ context.Context, ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) {
- if ba.ReadConsistency != rc {
- return nil, roachpb.NewErrorf("BatchRequest has unexpected ReadConsistency %s", ba.ReadConsistency)
- }
- return ba.CreateReply(), nil
- })
+ factory := client.TxnSenderFactoryFunc(func(client.TxnType) client.TxnSender {
+ return client.TxnSenderAdapter{
+ StartTrackingWrapped: func(context.Context) error { panic("unimplemented") },
+ Wrapped: func(_ context.Context, ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) {
+ if ba.ReadConsistency != rc {
+ return nil, roachpb.NewErrorf("BatchRequest has unexpected ReadConsistency %s", ba.ReadConsistency)
+ }
+ return ba.CreateReply(), nil
+ },
+ }
})
+
clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond)
db := client.NewDB(factory, clock)
ctx := context.TODO()
@@ -971,10 +975,13 @@ func TestNodeIDAndObservedTimestamps(t *testing.T) {
// Mock out sender function to check that created transactions
// have the observed timestamp set for the configured node ID.
- factory := client.TxnSenderFactoryFunc(func(_ client.TxnType) client.TxnSender {
- return client.TxnSenderFunc(func(_ context.Context, ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) {
- return ba.CreateReply(), nil
- })
+ factory := client.TxnSenderFactoryFunc(func(client.TxnType) client.TxnSender {
+ return client.TxnSenderAdapter{
+ StartTrackingWrapped: func(context.Context) error { panic("unimplemented") },
+ Wrapped: func(_ context.Context, ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) {
+ return ba.CreateReply(), nil
+ },
+ }
})
clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond)
diff --git a/pkg/internal/client/db.go b/pkg/internal/client/db.go
index 719345f9d601..df37cca60bd2 100644
--- a/pkg/internal/client/db.go
+++ b/pkg/internal/client/db.go
@@ -199,11 +199,22 @@ type DB struct {
ctx DBContext
}
-// GetSender returns a transaction-capable Sender instance. The same
-// sender must be used for the entirety of a transaction. Get a new
-// instance to start a new transaction.
+// GetSender returns a Sender that can be used to send requests through.
+// Note that a new Sender created; it is not shared.
+//
+// The Sender returned should not be used for sending transactional requests.
+// Use db.Txn() or db.NewTxn() for that.
func (db *DB) GetSender() Sender {
- return db.factory.New(RootTxn)
+ // We pass nil for the txn here because we don't have a txn on hand.
+ // That's why this method says to not use the Sender for transactional
+ // requests, plus the fact that if a Sender is used directly, the caller needs
+ // to be mindful of the need to start a heartbeat loop when writing.
+ //
+ // Note that even non-transactional requests need to go through a
+ // TxnCoordSender because batches that get split need to be wrapped in
+ // transactions (and the TxnCoordSender handles that). So we can't simply
+ // return the wrapped handler here.
+ return db.factory.New(RootTxn, nil /* txn */)
}
// GetFactory returns the DB's TxnSenderFactory.
diff --git a/pkg/internal/client/sender.go b/pkg/internal/client/sender.go
index 3ab13975e815..aeddc164f016 100644
--- a/pkg/internal/client/sender.go
+++ b/pkg/internal/client/sender.go
@@ -69,7 +69,7 @@ type TxnSender interface {
GetMeta() roachpb.TxnCoordMeta
// AugmentMeta combines the TxnCoordMeta from another distributed
// TxnSender which is part of the same transaction.
- AugmentMeta(meta roachpb.TxnCoordMeta)
+ AugmentMeta(ctx context.Context, meta roachpb.TxnCoordMeta)
// OnFinish invokes the supplied closure when the sender has finished
// with the txn (i.e. it's been abandoned, aborted, or committed).
// The error passed is meant to indicate to an extant distributed
@@ -78,16 +78,21 @@ type TxnSender interface {
// if this method is invoked multiple times, the most recent callback
// is the only one which will be invoked.
OnFinish(func(error))
+
+ // StartTracking starts a heartbeat loop and tracking of intents.
+ StartTracking(ctx context.Context) error
}
// TxnSenderFactory is the interface used to create new instances
// of TxnSender.
type TxnSenderFactory interface {
- // New returns a new instance of TxnSender. The typ parameter
- // specifies whether the sender is the root or one of potentially
- // many child "leaf" nodes in a tree of transaction objects, as is
- // created during a DistSQL flow.
- New(typ TxnType) TxnSender
+ // New returns a new instance of TxnSender.
+ // typ specifies whether the sender is the root or one of potentially many
+ // child "leaf" nodes in a tree of transaction objects, as is created during a
+ // DistSQL flow.
+ // txn is the transaction whose requests this sender will carry. It can be nil
+ // if the sender will not be used for transactional requests.
+ New(typ TxnType, txn *roachpb.Transaction) TxnSender
// WrappedSender returns the TxnSenderFactory's wrapped Sender.
WrappedSender() Sender
}
@@ -103,33 +108,44 @@ func (f SenderFunc) Send(
return f(ctx, ba)
}
-// TxnSenderFunc is an adapter to allow the use of ordinary functions
-// as TxnSenders with GetMeta or AugmentMeta panicing with unimplemented.
-// This is a helper mechanism to facilitate testing.
-type TxnSenderFunc func(context.Context, roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error)
+// TxnSenderAdapter is an adapter to allow the use of ordinary functions as
+// TxnSenders with GetMeta or AugmentMeta panicing with unimplemented. This is
+// a helper mechanism to facilitate testing.
+type TxnSenderAdapter struct {
+ Wrapped func(context.Context, roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error)
+ StartTrackingWrapped func(context.Context) error
+}
// Send calls f(ctx, c).
-func (f TxnSenderFunc) Send(
+func (f TxnSenderAdapter) Send(
ctx context.Context, ba roachpb.BatchRequest,
) (*roachpb.BatchResponse, *roachpb.Error) {
- return f(ctx, ba)
+ return f.Wrapped(ctx, ba)
}
// GetMeta is part of the TxnSender interface.
-func (f TxnSenderFunc) GetMeta() roachpb.TxnCoordMeta { panic("unimplemented") }
+func (f TxnSenderAdapter) GetMeta() roachpb.TxnCoordMeta { panic("unimplemented") }
// AugmentMeta is part of the TxnSender interface.
-func (f TxnSenderFunc) AugmentMeta(_ roachpb.TxnCoordMeta) { panic("unimplemented") }
+func (f TxnSenderAdapter) AugmentMeta(context.Context, roachpb.TxnCoordMeta) { panic("unimplemented") }
// OnFinish is part of the TxnSender interface.
-func (f TxnSenderFunc) OnFinish(_ func(error)) { panic("unimplemented") }
+func (f TxnSenderAdapter) OnFinish(_ func(error)) { panic("unimplemented") }
+
+// StartTracking is part the TxnSender interface.
+func (f TxnSenderAdapter) StartTracking(ctx context.Context) error {
+ if f.StartTrackingWrapped != nil {
+ return f.StartTrackingWrapped(ctx)
+ }
+ panic("unimplemented")
+}
// TxnSenderFactoryFunc is an adapter to allow the use of ordinary functions
// as TxnSenderFactories. This is a helper mechanism to facilitate testing.
type TxnSenderFactoryFunc func(TxnType) TxnSender
// New calls f().
-func (f TxnSenderFactoryFunc) New(typ TxnType) TxnSender {
+func (f TxnSenderFactoryFunc) New(typ TxnType, _ *roachpb.Transaction) TxnSender {
return f(typ)
}
diff --git a/pkg/internal/client/txn.go b/pkg/internal/client/txn.go
index c098404baf24..1b7ae6765219 100644
--- a/pkg/internal/client/txn.go
+++ b/pkg/internal/client/txn.go
@@ -73,10 +73,7 @@ type Txn struct {
// txnAnchorKey is the key at which to anchor the transaction record. If
// unset, the first key written in the transaction will be used.
txnAnchorKey roachpb.Key
- // writingTxnRecord is set when the Txn is in the middle of writing
- // its transaction record. It is used to assure that even in the presence
- // of concurrent requests, only one sends a BeginTxnRequest.
- writingTxnRecord bool
+ state txnState
// see IsFinalized()
finalized bool
// previousIDs holds the set of all previous IDs that the Txn's Proto has
@@ -99,6 +96,40 @@ type Txn struct {
}
}
+// txnState represents states relating to whether Begin/EndTxn requests need to
+// be sent.
+type txnState int
+
+const (
+ // txnReadOnly means that the transaction never sent any writes. There's no
+ // transaction record, so an EndTxn does not need to be sent.
+ txnReadOnly txnState = iota
+ // txnWriting means that the transaction has sent some writes (and so it also
+ // sent a BeginTxn). An EndTransaction must be sent to resolve intents and/or
+ // to cleanup the txn record.
+ // txnWriting does not guarantee that the transaction record has been written.
+ // In case the BeginTxn batch encoutered an error, it might not have been. In
+ // this case, a rollback will get an error (ignored by SQL).
+ txnWriting
+ // txnWriteInOldEpoch means that the txn has been writing in an old epoch,
+ // but then restarted with a new epoch, and there have been no writes sent
+ // since then. This means that an EndTransaction(commit=false) needs to be
+ // sent to clean up intents. It also means that a BeginTransaction needs to be
+ // sent on the first write: the TransactionRestartError might have been
+ // received by the batch with the BeginTransaction in it, in which case there
+ // is no transaction record (and so it needs to be created).
+ // We could be smarter about not transitioning to this state if there's ever
+ // been a successful write (in which case we know that there is a txn record
+ // and a BeginTransaction is not necessary) but as of May 2018 we don't do
+ // that. Note that the server accepts a BeginTxn with a higher epoch if a
+ // transaction record already exists.
+ txnWriteInOldEpoch
+ // txnError means that the txn had performed some writes and then a batch got
+ // a non-retriable error. Further batches except EndTransaction(commit=false)
+ // will be rejected.
+ txnError
+)
+
// NewTxn returns a new txn. The typ parameter specifies whether this
// transaction is the top level (root), or one of potentially many
// distributed transactions (leaf).
@@ -141,7 +172,7 @@ func NewTxnWithProto(
proto.AssertInitialized(context.TODO())
txn := &Txn{db: db, typ: typ, gatewayNodeID: gatewayNodeID}
txn.mu.Proto = proto
- txn.mu.sender = db.factory.New(typ)
+ txn.mu.sender = db.factory.New(typ, &proto)
return txn
}
@@ -298,7 +329,7 @@ func (txn *Txn) CommitTimestamp() hlc.Timestamp {
func (txn *Txn) SetTxnAnchorKey(key roachpb.Key) error {
txn.mu.Lock()
defer txn.mu.Unlock()
- if txn.mu.Proto.Writing || txn.mu.writingTxnRecord {
+ if txn.mu.state != txnReadOnly {
return errors.Errorf("transaction anchor key already set")
}
txn.mu.txnAnchorKey = key
@@ -596,6 +627,14 @@ func (txn *Txn) Rollback(ctx context.Context) error {
}
func (txn *Txn) rollback(ctx context.Context) *roachpb.Error {
+ // TODO(andrei): It's common for rollbacks to fail with
+ // TransactionStatusError: txn record not found (REASON_TXN_NOT_FOUND),
+ // in case the txn record was never written (e.g. if a 1PC failed). One would
+ // think that, depending on the error received by a 1PC batch, we'd know if a
+ // txn record was written and, if it wasn't, we could short-circuit the
+ // rollback. There's two tricky things about that, though: a) we'd need to
+ // know whether the DistSender has split what looked like a 1PC batch to the
+ // client and b) ambiguous errors.
log.VEventf(ctx, 2, "rolling back transaction")
return txn.sendEndTxnReq(ctx, false /* commit */, nil)
}
@@ -614,19 +653,16 @@ func (txn *Txn) OnFinish(onFinishFn func(error)) {
txn.mu.sender.OnFinish(onFinishFn)
}
-// maybeFinishReadonly provides a fast-path for finishing a read-only
+// finishReadonlyLocked provides a fast-path for finishing a read-only
// transaction without going through the overhead of creating an
// EndTransactionRequest only to not send it.
//
// NB: The logic here must be kept in sync with the logic in txn.Send.
//
// TODO(andrei): Can we share this code with txn.Send?
-func (txn *Txn) maybeFinishReadonly(commit bool, deadline *hlc.Timestamp) (bool, *roachpb.Error) {
- txn.mu.Lock()
- defer txn.mu.Unlock()
- if txn.mu.Proto.Writing || txn.mu.writingTxnRecord {
- return false, nil
- }
+func (txn *Txn) finishReadonlyLocked(
+ ctx context.Context, commit bool, deadline *hlc.Timestamp,
+) *roachpb.Error {
txn.mu.finalized = true
// Check that read only transactions do not violate their deadline. This can NOT
// happen since the txn deadline is normally updated when it is about to expire
@@ -644,7 +680,7 @@ func (txn *Txn) maybeFinishReadonly(commit bool, deadline *hlc.Timestamp) (bool,
// 4. new timestamp violates deadline
// 5. txn retries the read
// 6. commit fails - only thanks to this code path?
- return false, roachpb.NewErrorWithTxn(roachpb.NewTransactionStatusError(
+ return roachpb.NewErrorWithTxn(roachpb.NewTransactionStatusError(
"deadline exceeded before transaction finalization"), &txn.mu.Proto)
}
if commit {
@@ -652,18 +688,35 @@ func (txn *Txn) maybeFinishReadonly(commit bool, deadline *hlc.Timestamp) (bool,
} else {
txn.mu.Proto.Status = roachpb.ABORTED
}
- return true, nil
+ return nil
}
func (txn *Txn) sendEndTxnReq(
ctx context.Context, commit bool, deadline *hlc.Timestamp,
) *roachpb.Error {
- if ok, err := txn.maybeFinishReadonly(commit, deadline); ok || err != nil {
- return err
+ txn.mu.Lock()
+ if txn.mu.state == txnReadOnly {
+ defer txn.mu.Unlock()
+ return txn.finishReadonlyLocked(ctx, commit, deadline)
+ }
+
+ var swallowErr bool
+ if txn.mu.state == txnWriteInOldEpoch && commit {
+ // If there was a write in an old epoch (and no writes in the current
+ // epoch), we need to send a rollback. We'll ignore the error from it; the
+ // commit is successful at this point.
+ log.VEventf(ctx, 2, "old epoch write turning commit into rollback")
+ commit = false
+ swallowErr = true
}
+ txn.mu.Unlock()
+
var ba roachpb.BatchRequest
ba.Add(endTxnReq(commit, deadline, txn.systemConfigTrigger))
_, pErr := txn.Send(ctx, ba)
+ if swallowErr {
+ return nil
+ }
return pErr
}
@@ -878,6 +931,14 @@ func (txn *Txn) Send(
txn.mu.Lock()
defer txn.mu.Unlock()
+ if txn.mu.state == txnError {
+ singleAbort := ba.IsSingleEndTransactionRequest() &&
+ !ba.Requests[0].GetInner().(*roachpb.EndTransactionRequest).Commit
+ if !singleAbort {
+ return roachpb.NewError(&roachpb.TxnAlreadyEncounteredErrorError{})
+ }
+ }
+
sender = txn.mu.sender
if txn.mu.Proto.Status != roachpb.PENDING || txn.mu.finalized {
return roachpb.NewErrorf(
@@ -899,14 +960,27 @@ func (txn *Txn) Send(
txn.mu.active = true
}
- needBeginTxn = !(txn.mu.Proto.Writing || txn.mu.writingTxnRecord) && haveTxnWrite
- needEndTxn := txn.mu.Proto.Writing || txn.mu.writingTxnRecord || haveTxnWrite
+ needBeginTxn = haveTxnWrite && (txn.mu.state != txnWriting)
+ // We need the EndTxn if we've ever written before or if we're writing now.
+ needEndTxn := haveTxnWrite || txn.mu.state != txnReadOnly
elideEndTxn = haveEndTxn && !needEndTxn
// If we're not yet writing in this txn, but intend to, insert a
// begin transaction request before the first write command and update
// transaction state accordingly.
if needBeginTxn {
+ // Unless it's a 1PC, ask the TxnCoordSender to track the transaction.
+ if txn.mu.state == txnReadOnly && !haveEndTxn {
+ if err := txn.mu.sender.StartTracking(ctx); err != nil {
+ return roachpb.NewError(err)
+ }
+ }
+ // We're about to send a BeginTxn, so move to the Writing state.
+ txn.mu.state = txnWriting
+ // From now on, all requests need to be checked against the AbortCache on
+ // the server side.
+ txn.mu.Proto.Writing = true
+
// Set txn key based on the key of the first transactional write if
// not already set. If the transaction already has a key (we're in a
// restart), make sure we keep the anchor key the same.
@@ -930,9 +1004,6 @@ func (txn *Txn) Send(
copy(ba.Requests, oldRequests[:firstWriteIdx])
ba.Requests[firstWriteIdx].MustSetInner(bt)
copy(ba.Requests[firstWriteIdx+1:], oldRequests[firstWriteIdx:])
- // We're going to be writing the transaction record by sending the
- // begin transaction request.
- txn.mu.writingTxnRecord = true
}
if elideEndTxn {
@@ -976,9 +1047,7 @@ func (txn *Txn) Send(
txn.mu.Lock()
defer txn.mu.Unlock()
- // If we inserted a begin transaction request, remove it here. We also
- // unset the flag writingTxnRecord flag in case another ever needs to
- // be sent again (for instance, if we're aborted and need to restart).
+ // If we inserted a begin transaction request, remove it here.
if needBeginTxn {
if br != nil && br.Responses != nil {
br.Responses = append(br.Responses[:firstWriteIdx], br.Responses[firstWriteIdx+1:]...)
@@ -994,8 +1063,6 @@ func (txn *Txn) Send(
pErr.SetErrorIndex(idx - 1)
}
}
-
- txn.mu.writingTxnRecord = false
}
if haveEndTxn {
if pErr == nil || !endTxnRequest.Commit {
@@ -1006,12 +1073,19 @@ func (txn *Txn) Send(
}
}
+ // If the request was part of a previous attempt, don't return any results.
+ if requestTxnID != txn.mu.Proto.ID || ba.Txn.Epoch != txn.mu.Proto.Epoch {
+ return nil, roachpb.NewError(&roachpb.TxnPrevAttemptError{})
+ }
+
if pErr != nil {
if log.V(1) {
log.Infof(ctx, "failed batch: %s", pErr)
}
+ var retriable bool
switch t := pErr.GetDetail().(type) {
case *roachpb.HandledRetryableTxnError:
+ retriable = true
retryErr := t
if requestTxnID != retryErr.TxnID {
// KV should not return errors for transactions other than the one that sent
@@ -1019,21 +1093,25 @@ func (txn *Txn) Send(
log.Fatalf(ctx, "retryable error for the wrong txn. "+
"requestTxnID: %s, retryErr.TxnID: %s. retryErr: %s",
requestTxnID, retryErr.TxnID, retryErr)
- } else if requestTxnID == txn.mu.Proto.ID {
- // Our requestTxnID still matches the proto, so update the state.
- // If it doesn't match here, it means a concurrent request through
- // this Txn object has already aborted and restarted the txn.
- txn.updateStateOnRetryableErrLocked(ctx, retryErr)
}
+ txn.updateStateOnRetryableErrLocked(ctx, retryErr)
}
// Note that unhandled retryable txn errors are allowed from leaf
// transactions. We pass them up through distributed SQL flows to
// the root transactions, at the receiver.
- if txn.typ == RootTxn && pErr.TransactionRestart != roachpb.TransactionRestart_NONE {
- log.Fatalf(ctx,
- "unexpected retryable error at the client.Txn level: (%T) %s",
- pErr.GetDetail(), pErr)
+ if pErr.TransactionRestart != roachpb.TransactionRestart_NONE {
+ retriable = true
+ if txn.typ == RootTxn {
+ log.Fatalf(ctx,
+ "unexpected retryable error at the client.Txn level: (%T) %s",
+ pErr.GetDetail(), pErr)
+ }
+ }
+
+ if !retriable && txn.mu.state == txnWriting {
+ txn.mu.state = txnError
}
+
return nil, pErr
}
@@ -1041,9 +1119,6 @@ func (txn *Txn) Send(
if br.Error != nil {
panic(roachpb.ErrorUnexpectedlySet(txn.mu.sender, br))
}
- if br.Txn != nil && br.Txn.ID != txn.mu.Proto.ID {
- return nil, roachpb.NewError(&roachpb.TxnPrevAttemptError{})
- }
// Only successful requests can carry an updated Txn in their response
// header. Some errors (e.g. a restart) have a Txn attached to them as
@@ -1108,11 +1183,11 @@ func (txn *Txn) GetTxnCoordMeta() roachpb.TxnCoordMeta {
// AugmentTxnCoordMeta augments this transaction's TxnCoordMeta
// information with the supplied meta. For use with GetTxnCoordMeta().
-func (txn *Txn) AugmentTxnCoordMeta(meta roachpb.TxnCoordMeta) {
+func (txn *Txn) AugmentTxnCoordMeta(ctx context.Context, meta roachpb.TxnCoordMeta) {
txn.mu.Lock()
defer txn.mu.Unlock()
txn.mu.Proto.Update(&meta.Txn)
- txn.mu.sender.AugmentMeta(meta)
+ txn.mu.sender.AugmentMeta(ctx, meta)
}
// UpdateStateOnRemoteRetryableErr updates the Txn, and the
@@ -1167,9 +1242,11 @@ func (txn *Txn) updateStateOnRetryableErrLocked(
// attempt. The txn inside pErr was correctly prepared for this by
// TxnCoordSender.
txn.mu.Proto = *newTxn
+ // We're starting a fresh transaction, with no state.
+ txn.mu.state = txnReadOnly
// Create a new txn sender.
- txn.mu.sender = txn.db.factory.New(txn.typ)
+ txn.mu.sender = txn.db.factory.New(txn.typ, newTxn)
} else {
// Update the transaction proto with the one to be used for the next
// attempt. The txn inside pErr was correctly prepared for this by
@@ -1178,6 +1255,12 @@ func (txn *Txn) updateStateOnRetryableErrLocked(
// we rely on the associativity of Transaction.Update to sort out this
// lack of ordering guarantee.
txn.mu.Proto.Update(newTxn)
+
+ // If we've been writing, we'll need to send a BeginTxn with the next
+ // request.
+ if txn.mu.state == txnWriting {
+ txn.mu.state = txnWriteInOldEpoch
+ }
}
}
diff --git a/pkg/internal/client/txn_test.go b/pkg/internal/client/txn_test.go
index 4e8d4bf8dcbc..1bc64b2e14b1 100644
--- a/pkg/internal/client/txn_test.go
+++ b/pkg/internal/client/txn_test.go
@@ -70,7 +70,11 @@ func TestTxnSnowballTrace(t *testing.T) {
// 0.275ms 0.171ms event:client.Txn did AutoCommit. err:
//txn: "internal/client/txn_test.go:67 TestTxnSnowballTrace" id= key=/Min rw=false pri=0.00000000 iso=SERIALIZABLE stat=COMMITTED epo=0 ts=0.000000000,0 orig=0.000000000,0 max=0.000000000,0 wto=false rop=false
// 0.278ms 0.173ms event:txn complete
- found, err := regexp.MatchString(".*event:inside txn\n.*event:client.Txn did AutoCommit. err: \n.*\n.*event:txn complete.*", dump)
+ found, err := regexp.MatchString(
+ // The (?s) makes "." match \n. This makes the test resilient to other log
+ // lines being interspersed.
+ "(?s).*event:inside txn\n.*event:client.Txn did AutoCommit. err: \n.*\n.*event:txn complete.*",
+ dump)
if err != nil {
t.Fatal(err)
}
@@ -82,55 +86,58 @@ func TestTxnSnowballTrace(t *testing.T) {
func newTestTxnFactory(
createReply func(roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error),
) TxnSenderFactoryFunc {
- return TxnSenderFactoryFunc(func(_ TxnType) TxnSender {
- return TxnSenderFunc(func(_ context.Context, ba roachpb.BatchRequest,
- ) (*roachpb.BatchResponse, *roachpb.Error) {
- if ba.UserPriority == 0 {
- ba.UserPriority = 1
- }
+ return TxnSenderFactoryFunc(func(TxnType) TxnSender {
+ return TxnSenderAdapter{
+ StartTrackingWrapped: func(context.Context) error { return nil },
+ Wrapped: func(_ context.Context, ba roachpb.BatchRequest,
+ ) (*roachpb.BatchResponse, *roachpb.Error) {
+ if ba.UserPriority == 0 {
+ ba.UserPriority = 1
+ }
- var br *roachpb.BatchResponse
- var pErr *roachpb.Error
- if createReply != nil {
- br, pErr = createReply(ba)
- } else {
- br = ba.CreateReply()
- }
- if pErr != nil {
- return nil, pErr
- }
- var writing bool
- status := roachpb.PENDING
- for i, req := range ba.Requests {
- args := req.GetInner()
- if _, ok := args.(*roachpb.PutRequest); ok {
- testPutRespCopy := testPutResp
- union := &br.Responses[i] // avoid operating on copy
- union.MustSetInner(&testPutRespCopy)
+ var br *roachpb.BatchResponse
+ var pErr *roachpb.Error
+ if createReply != nil {
+ br, pErr = createReply(ba)
+ } else {
+ br = ba.CreateReply()
}
- if roachpb.IsTransactionWrite(args) {
- writing = true
+ if pErr != nil {
+ return nil, pErr
}
- }
- if args, ok := ba.GetArg(roachpb.EndTransaction); ok {
- et := args.(*roachpb.EndTransactionRequest)
- writing = true
- if et.Commit {
- status = roachpb.COMMITTED
- } else {
- status = roachpb.ABORTED
+ var writing bool
+ status := roachpb.PENDING
+ for i, req := range ba.Requests {
+ args := req.GetInner()
+ if _, ok := args.(*roachpb.PutRequest); ok {
+ testPutRespCopy := testPutResp
+ union := &br.Responses[i] // avoid operating on copy
+ union.MustSetInner(&testPutRespCopy)
+ }
+ if roachpb.IsTransactionWrite(args) {
+ writing = true
+ }
}
- }
- if ba.Txn != nil && br.Txn == nil {
- txnClone := ba.Txn.Clone()
- br.Txn = &txnClone
- if pErr == nil {
- br.Txn.Writing = writing
- br.Txn.Status = status
+ if args, ok := ba.GetArg(roachpb.EndTransaction); ok {
+ et := args.(*roachpb.EndTransactionRequest)
+ writing = true
+ if et.Commit {
+ status = roachpb.COMMITTED
+ } else {
+ status = roachpb.ABORTED
+ }
}
- }
- return br, pErr
- })
+ if ba.Txn != nil && br.Txn == nil {
+ txnClone := ba.Txn.Clone()
+ br.Txn = &txnClone
+ if pErr == nil {
+ br.Txn.Writing = writing
+ br.Txn.Status = status
+ }
+ }
+ return br, pErr
+ },
+ }
})
}
diff --git a/pkg/kv/dist_sender_server_test.go b/pkg/kv/dist_sender_server_test.go
index f7c2799d0191..21de65a9dd12 100644
--- a/pkg/kv/dist_sender_server_test.go
+++ b/pkg/kv/dist_sender_server_test.go
@@ -1774,8 +1774,7 @@ func TestTxnStarvation(t *testing.T) {
// TestTxnCoordSenderHeartbeatFailurePostSplit verifies that on
// heartbeat timeout, the transaction is aborted asynchronously,
// leaving abort span entries which cause concurrent reads to fail
-// with txn aborted errors on both the range the transaction started
-// on and a separate range involved in the same transaction.
+// with txn aborted errors on a range different that the txn's anchor.
//
// Note that this is a post-split version of TestTxnCoordSenderGCTimeout.
func TestTxnCoordSenderHeartbeatFailurePostSplit(t *testing.T) {
@@ -1836,7 +1835,6 @@ func TestTxnCoordSenderHeartbeatFailurePostSplit(t *testing.T) {
}()
return errCh
}
- errChA := startReader(keyA)
errChB := startReader(keyB)
stores := s.GetStores().(*storage.Stores)
@@ -1863,9 +1861,6 @@ func TestTxnCoordSenderHeartbeatFailurePostSplit(t *testing.T) {
// Now signal the inflight readers to continue; they should witness
// abort span entries.
close(signal)
- if err := <-errChA; !testutils.IsError(err, "txn aborted") {
- t.Errorf("expected transaction aborted error reading %s; got %s", keyA, err)
- }
if err := <-errChB; !testutils.IsError(err, "txn aborted") {
t.Errorf("expected transaction aborted error reading %s; got %s", keyB, err)
}
@@ -1955,31 +1950,8 @@ func TestTxnCoordSenderRetries(t *testing.T) {
return err
},
retryable: func(ctx context.Context, txn *client.Txn) error {
- return txn.InitPut(ctx, "a", "put", false) // put to advance txn ts
- },
- },
- {
- name: "forwarded timestamp with get and initput value exists",
- beforeTxnStart: func(ctx context.Context, db *client.DB) error {
- return db.Put(ctx, "a", "put")
- },
- afterTxnStart: func(ctx context.Context, db *client.DB) error {
- _, err := db.Get(ctx, "a") // read key to set ts cache
- return err
- },
- retryable: func(ctx context.Context, txn *client.Txn) error {
- err := txn.InitPut(ctx, "a", "init-put", false) // init-put to advance txn ts
- // Swallow expected condition failed error.
- if _, ok := err.(*roachpb.ConditionFailedError); !ok {
- if err != nil {
- return errors.New("expected condition failed error")
- }
- return err
- }
- log.Infof(ctx, "Swallowed error")
- return nil
+ return txn.InitPut(ctx, "a", "put", false /* failOnTombstones */) // put to advance txn ts
},
- // No retries, this is a straight failure.
},
{
name: "forwarded timestamp with get and cput",
diff --git a/pkg/kv/txn_coord_sender.go b/pkg/kv/txn_coord_sender.go
index 7f64698a8016..35ce8e15224f 100644
--- a/pkg/kv/txn_coord_sender.go
+++ b/pkg/kv/txn_coord_sender.go
@@ -100,6 +100,14 @@ const (
type TxnCoordSender struct {
mu struct {
syncutil.Mutex
+
+ // tracking is set if the TxnCoordSender has a heartbeat loop running for
+ // the transaction record. It also means that the TxnCoordSender is
+ // accumulating intents for the transaction.
+ // tracking is set by the client just before a BeginTransaction request is
+ // sent. If set, an EndTransaction will also be sent eventually to clean up.
+ tracking bool
+
// meta contains all coordinator state which may be passed between
// distributed TxnCoordSenders via MetaRelease() and MetaAugment().
meta roachpb.TxnCoordMeta
@@ -259,17 +267,30 @@ func NewTxnCoordSenderFactory(
}
}
-// New is part of the TxnCoordSenderFactory interface.
-func (tcf *TxnCoordSenderFactory) New(typ client.TxnType) client.TxnSender {
+// New is part of the TxnSenderFactory interface.
+func (tcf *TxnCoordSenderFactory) New(
+ typ client.TxnType, txn *roachpb.Transaction,
+) client.TxnSender {
tcs := &TxnCoordSender{
typ: typ,
TxnCoordSenderFactory: tcf,
}
tcs.mu.meta.RefreshValid = true
+
+ // If a transaction was passed in bind the TxnCoordSender to it.
+ // TODO(andrei): Ideally, if a transaction is not passed it, we should take
+ // that to mean that a TxnCoordSender is not needed and we should return the
+ // wrapped sender directly. However, there are tests that pass nil and still
+ // send transactional requests. That's why the TxnCoordSender is still
+ // littered with code handling the case where it is not yet bound to a
+ // transaction.
+ if txn != nil {
+ tcs.mu.meta.Txn = txn.Clone()
+ }
return tcs
}
-// WrappedSender is part of the TxnCoordSenderFactory interface.
+// WrappedSender is part of the TxnSenderFactory interface.
func (tcf *TxnCoordSenderFactory) WrappedSender() client.Sender {
return tcf.wrapped
}
@@ -295,11 +316,16 @@ func (tc *TxnCoordSender) GetMeta() roachpb.TxnCoordMeta {
}
// AugmentMeta is part of the client.TxnSender interface.
-func (tc *TxnCoordSender) AugmentMeta(meta roachpb.TxnCoordMeta) {
+func (tc *TxnCoordSender) AugmentMeta(ctx context.Context, meta roachpb.TxnCoordMeta) {
tc.mu.Lock()
defer tc.mu.Unlock()
+
+ if tc.mu.meta.Txn.ID == (uuid.UUID{}) {
+ log.Fatalf(ctx, "cannot AugmentMeta on unbound TxnCoordSender. meta id: %s", meta.Txn.ID)
+ }
+
// Sanity check: don't combine if the meta is for a different txn ID.
- if tc.mu.meta.Txn.ID != (uuid.UUID{}) && tc.mu.meta.Txn.ID != meta.Txn.ID {
+ if tc.mu.meta.Txn.ID != meta.Txn.ID {
return
}
tc.mu.meta.Txn.Update(&meta.Txn)
@@ -343,15 +369,12 @@ func (tc *TxnCoordSender) OnFinish(onFinishFn func(error)) {
tc.mu.onFinishFn = onFinishFn
}
-// Send implements the batch.Sender interface. If the request is part of a
-// transaction, the TxnCoordSender adds the transaction to a map of active
-// transactions and begins heartbeating it. Every subsequent request for the
-// same transaction updates the lastUpdate timestamp to prevent live
-// transactions from being considered abandoned and garbage collected.
+// Send implements the batch.Sender interface.
+//
// Read/write mutating requests have their key or key range added to the
// transaction's interval tree of key ranges for eventual cleanup via resolved
-// write intents; they're tagged to an outgoing EndTransaction request, with
-// the receiving replica in charge of resolving them.
+// write intents; they're tagged to an outgoing EndTransaction request, with the
+// receiving replica in charge of resolving them.
func (tc *TxnCoordSender) Send(
ctx context.Context, ba roachpb.BatchRequest,
) (*roachpb.BatchResponse, *roachpb.Error) {
@@ -369,6 +392,10 @@ func (tc *TxnCoordSender) Send(
startNS := tc.clock.PhysicalNow()
if ba.Txn != nil {
+ if tc.mu.meta.Txn.ID == (uuid.UUID{}) {
+ log.Fatalf(ctx, "cannot send transactional request through unbound TxnCoordSender")
+ }
+
ctx = log.WithLogTag(ctx, "txn", uuid.ShortStringer(ba.Txn.ID))
if log.V(2) {
ctx = log.WithLogTag(ctx, "ts", ba.Txn.Timestamp)
@@ -409,14 +436,24 @@ func (tc *TxnCoordSender) Send(
tc.mu.Lock()
defer tc.mu.Unlock()
- if tc.mu.meta.Txn.ID == (uuid.UUID{}) {
- // Ensure that the txn is bound.
- tc.mu.meta.Txn = ba.Txn.Clone()
- }
- if ba.Txn.Writing {
- if pErr := tc.maybeRejectClientLocked(ctx, ba.Txn.ID); pErr != nil {
- return pErr
- }
+ // Copy a few fields from the request's txn. This is technically only
+ // required during the first send, as these fields are set before
+ // the first send and can't change afterwards. Keeping these fields in
+ // sync between the TxnCoordSender and the client.Txn is needed because,
+ // when the TxnCoordSender synthesizes TransactionAbortedErrors, it
+ // creates a new proto that it passes to the client.Txn and then these
+ // fields are used when creating that proto that will then be used for the
+ // client.Txn. On subsequent retries of the transaction, it's important
+ // for the values of these fields to have been preserved because that
+ // makes future calls to txn.SetIsolation() and such no-ops.
+ // If this makes no sense it's because the TxnCoordSender having a copy of
+ // the Transaction proto generally makes no sense.
+ tc.mu.meta.Txn.Name = ba.Txn.Name
+ tc.mu.meta.Txn.Isolation = ba.Txn.Isolation
+ tc.mu.meta.Txn.Priority = ba.Txn.Priority
+
+ if pErr := tc.maybeRejectClientLocked(ctx, ba.Txn.ID); pErr != nil {
+ return pErr
}
tc.mu.meta.CommandCount += int32(len(ba.Requests))
@@ -851,24 +888,19 @@ func (tc *TxnCoordSender) maybeRejectClientLocked(
// validateTxn validates properties of a txn specified on a request.
// The transaction is expected to be initialized by the time it reaches
-// the TxnCoordSender. Furthermore, no transactional writes are allowed
-// unless preceded by a begin transaction request within the same batch.
-// The exception is if the transaction is already in state txn.Writing=true.
+// the TxnCoordSender.
func (tc *TxnCoordSender) validateTxnForBatch(ctx context.Context, ba *roachpb.BatchRequest) error {
if len(ba.Requests) == 0 {
return errors.Errorf("empty batch with txn")
}
ba.Txn.AssertInitialized(ctx)
- // Check for a begin transaction to set txn key based on the key of
- // the first transactional write. Also enforce that no transactional
- // writes occur before a begin transaction.
var haveBeginTxn bool
for _, req := range ba.Requests {
args := req.GetInner()
if _, ok := args.(*roachpb.BeginTransactionRequest); ok {
- if haveBeginTxn || ba.Txn.Writing {
- return errors.Errorf("begin transaction requested twice in the same txn: %s", ba.Txn)
+ if haveBeginTxn {
+ return errors.Errorf("begin transaction requested twice in the same batch: %s", ba.Txn)
}
if ba.Txn.Key == nil {
return errors.Errorf("transaction with BeginTxnRequest missing anchor key: %v", ba)
@@ -977,8 +1009,17 @@ func (tc *TxnCoordSender) heartbeatLoop(ctx context.Context) {
// then heartbeat loop ignores the timeout check and this case is
// responsible for client timeouts.
log.VEventf(ctx, 2, "transaction heartbeat stopped: %s", ctx.Err())
- tc.tryAsyncAbort(ctx)
- return
+
+ // Check if the closer channel had also been closed; in that case, that
+ // takes priority.
+ select {
+ case <-closer:
+ // Transaction finished normally.
+ return
+ default:
+ tc.tryAsyncAbort(ctx)
+ return
+ }
case <-tc.stopper.ShouldQuiesce():
return
}
@@ -1000,10 +1041,15 @@ func (tc *TxnCoordSender) tryAsyncAbort(ctx context.Context) {
// raced here. That's fine (and probably better than the alternative, which
// is missing new intents sometimes). Note that the txn may be uninitialized
// here if a failure occurred before the first write succeeded.
- if txn.Status != roachpb.PENDING || txn.ID == (uuid.UUID{}) {
+ if txn.Status != roachpb.PENDING {
return
}
+ // Update out status to Aborted, since we're about to send a rollback. Besides
+ // being sane, this prevents the heartbeat loop from incrementing an
+ // "Abandons" metric.
+ tc.mu.meta.Txn.Status = roachpb.ABORTED
+
// NB: use context.Background() here because we may be called when the
// caller's context has been canceled.
if err := tc.stopper.RunAsyncTask(
@@ -1136,6 +1182,39 @@ func (tc *TxnCoordSender) heartbeat(ctx context.Context) bool {
return true
}
+// StartTracking is part of the client.TxnSender interface.
+func (tc *TxnCoordSender) StartTracking(ctx context.Context) error {
+ tc.mu.Lock()
+ defer tc.mu.Unlock()
+
+ tc.mu.tracking = true
+ tc.mu.firstUpdateNanos = tc.clock.PhysicalNow()
+
+ // Only heartbeat the txn record if we're the root transaction.
+ if tc.typ != client.RootTxn {
+ return nil
+ }
+
+ log.VEventf(ctx, 2, "coordinator spawns heartbeat loop")
+ // Create a channel to stop the heartbeat with the lock held
+ // to avoid a race between the async task and a subsequent commit.
+ tc.mu.txnEnd = make(chan struct{})
+ if err := tc.stopper.RunAsyncTask(
+ ctx, "kv.TxnCoordSender: heartbeat loop", func(ctx context.Context) {
+ tc.heartbeatLoop(ctx)
+ }); err != nil {
+ // The system is already draining and we can't start the
+ // heartbeat. We refuse new transactions for now because
+ // they're likely not going to have all intents committed.
+ // In principle, we can relax this as needed though.
+ tc.cleanupTxnLocked(ctx, aborted)
+ duration, restarts, status := tc.finalTxnStatsLocked()
+ tc.updateStats(duration, restarts, status, false /* onePC */)
+ return err
+ }
+ return nil
+}
+
// updateState updates the transaction state in both the success and
// error cases, applying those updates to the corresponding txnMeta
// object when adequate. It also updates retryable errors with the
@@ -1284,71 +1363,20 @@ func (tc *TxnCoordSender) updateState(
// a serializable retry error. We can use the set of read spans to
// avoid retrying the transaction if all the spans can be updated to
// the current transaction timestamp.
- //
- // A tricky edge case is that of a transaction which "fails" on the
- // first writing request, but actually manages to write some intents
- // (for example, due to being multi-range). In this case, there will
- // be an error, but the transaction will be marked as Writing and the
- // coordinator must track the state, for the client's retry will be
- // performed with a Writing transaction which the coordinator rejects
- // unless it is tracking it (on top of it making sense to track it;
- // after all, it **has** laid down intents and only the coordinator
- // can augment a potential EndTransaction call). See #3303.
- //
- // An extension of this case is that of a transaction which receives an
- // ambiguous result error on its first writing request. Here, the
- // transaction will not be marked as Writing, but still could have laid
- // down intents (we don't know, it's ambiguous!). As with the other case,
- // we still track the possible writes so they can be cleaned up cleanup
- // to avoid dangling intents. However, since the Writing flag is not
- // set in these cases, it may be possible that the request was read-only.
- // This is ok, since the following block will be a no-op if the batch
- // contained no transactional write requests.
- _, ambiguousErr := pErr.GetDetail().(*roachpb.AmbiguousResultError)
- if pErr == nil || ambiguousErr || newTxn.Writing {
+ if tc.mu.tracking {
// Adding the intents even on error reduces the likelihood of dangling
// intents blocking concurrent writers for extended periods of time.
// See #3346.
tc.appendAndCondenseIntentsLocked(ctx, ba, br)
-
- // Initialize the first update time and maybe start the heartbeat.
- if tc.mu.firstUpdateNanos == 0 && len(tc.mu.meta.Intents) > 0 {
- // If the transaction is already over, there's no point in
- // launching a one-off heartbeat which will shut down right
- // away. If we ended up here with an error, we'll always start
- // the coordinator - the transaction has laid down intents, so
- // we expect it to be committed/aborted at some point in the
- // future.
- if _, isEnding := ba.GetArg(roachpb.EndTransaction); pErr != nil || !isEnding {
- log.Event(ctx, "coordinator spawns")
- tc.mu.firstUpdateNanos = startNS
-
- // Only heartbeat the txn record if we're the root transaction.
- if tc.typ == client.RootTxn {
- // Create a channel to stop the heartbeat with the lock held
- // to avoid a race between the async task and a subsequent commit.
- tc.mu.txnEnd = make(chan struct{})
- if err := tc.stopper.RunAsyncTask(
- ctx, "kv.TxnCoordSender: heartbeat loop", func(ctx context.Context) {
- tc.heartbeatLoop(ctx)
- }); err != nil {
- // The system is already draining and we can't start the
- // heartbeat. We refuse new transactions for now because
- // they're likely not going to have all intents committed.
- // In principle, we can relax this as needed though.
- tc.cleanupTxnLocked(ctx, aborted)
- duration, restarts, status := tc.finalTxnStatsLocked()
- tc.updateStats(duration, restarts, status, false)
- return roachpb.NewError(err)
- }
- }
- } else {
- // If this was a successful one phase commit, update stats
- // directly as they won't otherwise be updated on heartbeat
- // loop shutdown.
- etArgs, ok := br.Responses[len(br.Responses)-1].GetInner().(*roachpb.EndTransactionResponse)
- tc.updateStats(tc.clock.PhysicalNow()-startNS, 0, newTxn.Status, ok && etArgs.OnePhaseCommit)
- }
+ } else {
+ // If this was a successful one phase commit, update stats
+ // directly as they won't otherwise be updated on heartbeat
+ // loop shutdown.
+ _, isBeginning := ba.GetArg(roachpb.BeginTransaction)
+ _, isEnding := ba.GetArg(roachpb.EndTransaction)
+ if pErr == nil && isBeginning && isEnding {
+ etArgs, ok := br.Responses[len(br.Responses)-1].GetInner().(*roachpb.EndTransactionResponse)
+ tc.updateStats(tc.clock.PhysicalNow()-startNS, 0, newTxn.Status, ok && etArgs.OnePhaseCommit)
}
}
diff --git a/pkg/kv/txn_coord_sender_test.go b/pkg/kv/txn_coord_sender_test.go
index 3a15de7e0d4d..c3a5d18f3a3f 100644
--- a/pkg/kv/txn_coord_sender_test.go
+++ b/pkg/kv/txn_coord_sender_test.go
@@ -688,9 +688,9 @@ func TestTxnCoordSenderCancel(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
- tc := s.DB.GetSender().(*TxnCoordSender)
- origSender := tc.TxnCoordSenderFactory.wrapped
- tc.TxnCoordSenderFactory.wrapped = client.SenderFunc(
+ factory := s.DB.GetFactory().(*TxnCoordSenderFactory)
+ origSender := factory.WrappedSender()
+ factory.wrapped = client.SenderFunc(
func(ctx context.Context, args roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) {
if _, hasET := args.GetArg(roachpb.EndTransaction); hasET {
// Cancel the transaction while also sending it along. This tickled a
@@ -1109,7 +1109,7 @@ func TestTxnCoordIdempotentCleanup(t *testing.T) {
}
// TestTxnMultipleCoord checks that multiple txn coordinators can be
-// used by a single transaction, and their state can be combined.
+// used for reads by a single transaction, and their state can be combined.
func TestTxnMultipleCoord(t *testing.T) {
defer leaktest.AfterTest(t)()
s := createTestDB(t)
@@ -1118,66 +1118,36 @@ func TestTxnMultipleCoord(t *testing.T) {
ctx := context.Background()
txn := client.NewTxn(s.DB, 0 /* gatewayNodeID */, client.RootTxn)
tc := txn.Sender().(*TxnCoordSender)
- defer teardownHeartbeat(tc)
// Start the transaction.
key := roachpb.Key("a")
- if err := txn.Put(ctx, key, []byte("value")); err != nil {
- t.Fatalf("expected error %s", err)
+ if _, err := txn.Get(ctx, key); err != nil {
+ t.Fatal(err)
}
// New create a second, leaf coordinator.
txn2 := client.NewTxnWithProto(s.DB, 0 /* gatewayNodeID */, client.LeafTxn, *txn.Proto())
- tc2 := txn2.Sender().(*TxnCoordSender)
- defer teardownHeartbeat(tc2)
// Start the second transaction.
key2 := roachpb.Key("b")
- if err := txn2.Put(ctx, key2, []byte("value2")); err != nil {
- t.Fatalf("expected error %s", err)
- }
-
- // Verify heartbeat started on root txn.
- testutils.SucceedsSoon(t, func() error {
- tc.mu.Lock()
- defer tc.mu.Unlock()
- if tc.mu.txnEnd == nil {
- return errors.New("expected heartbeat on root coordinator")
- }
- return nil
- })
- // Verify no heartbeat started on leaf txn.
- tc2.mu.Lock()
- if tc2.mu.txnEnd != nil {
- t.Fatalf("unexpected heartbeat on leaf coordinator")
- }
- tc2.mu.Unlock()
-
- // Verify it's an error to commit on the leaf txn node.
- ba := txn2.NewBatch()
- ba.AddRawRequest(&roachpb.EndTransactionRequest{Commit: true})
- if err := txn2.Run(context.TODO(), ba); !testutils.IsError(err, "cannot commit on a leaf transaction coordinator") {
- t.Fatalf("expected cannot commit on leaf coordinator error; got %v", err)
+ if _, err := txn2.Get(ctx, key2); err != nil {
+ t.Fatal(err)
}
// Augment txn with txn2's meta & commit.
- txn.AugmentTxnCoordMeta(txn2.GetTxnCoordMeta())
+ txn.AugmentTxnCoordMeta(ctx, txn2.GetTxnCoordMeta())
// Verify presence of both intents.
tc.mu.Lock()
- if a, e := tc.mu.meta.Intents, []roachpb.Span{{Key: key}, {Key: key2}}; !reflect.DeepEqual(a, e) {
- t.Fatalf("expected intents %+v; got %+v", e, a)
- }
- if a, e := tc.mu.intentsSizeBytes, int64(len(key)+len(key2)); a != e {
- t.Fatalf("expected intentsSizeBytes %d; got %d", e, a)
+ if a, e := tc.mu.meta.RefreshReads, []roachpb.Span{{Key: key}, {Key: key2}}; !reflect.DeepEqual(a, e) {
+ tc.mu.Unlock()
+ t.Fatalf("expected read spans %+v; got %+v", e, a)
}
tc.mu.Unlock()
- ba = txn.NewBatch()
+ ba := txn.NewBatch()
ba.AddRawRequest(&roachpb.EndTransactionRequest{Commit: true})
- if err := txn.Run(context.TODO(), ba); err != nil {
+ if err := txn.Run(ctx, ba); err != nil {
t.Fatal(err)
}
-
- verifyCleanup(key, s.Eng, t, tc, tc2)
}
// TestTxnCoordSenderSingleRoundtripTxn checks that a batch which completely
@@ -1201,7 +1171,6 @@ func TestTxnCoordSenderSingleRoundtripTxn(t *testing.T) {
ambient, cluster.MakeTestingClusterSettings(),
senderFn, clock, false, stopper, MakeTxnMetrics(metric.TestSampleInterval),
)
- tc := factory.New(client.RootTxn)
// Stop the stopper manually, prior to trying the transaction. This has the
// effect of returning a NodeUnavailableError for any attempts at launching
@@ -1214,6 +1183,7 @@ func TestTxnCoordSenderSingleRoundtripTxn(t *testing.T) {
ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: key}})
ba.Add(&roachpb.EndTransactionRequest{})
txn := roachpb.MakeTransaction("test", key, 0, 0, clock.Now(), 0)
+ tc := factory.New(client.RootTxn, &txn)
ba.Txn = &txn
_, pErr := tc.Send(context.Background(), ba)
if pErr != nil {
@@ -1263,8 +1233,6 @@ func TestTxnCoordSenderErrorWithIntent(t *testing.T) {
stopper,
MakeTxnMetrics(metric.TestSampleInterval),
)
- tc := factory.New(client.RootTxn)
- defer teardownHeartbeat(tc.(*TxnCoordSender))
var ba roachpb.BatchRequest
key := roachpb.Key("test")
@@ -1272,6 +1240,8 @@ func TestTxnCoordSenderErrorWithIntent(t *testing.T) {
ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: key}})
ba.Add(&roachpb.EndTransactionRequest{})
txn := roachpb.MakeTransaction("test", key, 0, 0, clock.Now(), 0)
+ tc := factory.New(client.RootTxn, &txn)
+ defer teardownHeartbeat(tc.(*TxnCoordSender))
ba.Txn = &txn
_, pErr := tc.Send(context.Background(), ba)
if !testutils.IsPError(pErr, test.errMsg) {
@@ -1418,7 +1388,7 @@ func checkTxnMetricsOnce(
func setupMetricsTest(t *testing.T) (*localtestcluster.LocalTestCluster, TxnMetrics, func()) {
s := createTestDB(t)
metrics := MakeTxnMetrics(metric.TestSampleInterval)
- s.DB.GetSender().(*TxnCoordSender).TxnCoordSenderFactory.metrics = metrics
+ s.DB.GetFactory().(*TxnCoordSenderFactory).metrics = metrics
return s, metrics, func() {
s.Stop()
}
@@ -1459,7 +1429,8 @@ func TestTxnOnePhaseCommit(t *testing.T) {
defer cleanupFn()
value := []byte("value")
- if err := s.DB.Txn(context.TODO(), func(ctx context.Context, txn *client.Txn) error {
+ ctx := context.TODO()
+ if err := s.DB.Txn(ctx, func(ctx context.Context, txn *client.Txn) error {
key := []byte("key-commit")
b := txn.NewBatch()
b.Put(key, value)
@@ -1467,6 +1438,20 @@ func TestTxnOnePhaseCommit(t *testing.T) {
}); err != nil {
t.Fatal(err)
}
+ kv, err := s.DB.Get(ctx, []byte("key-commit"))
+ if err != nil {
+ t.Fatal(err)
+ }
+ if kv.Value == nil {
+ t.Fatal("expected value not found")
+ }
+ val, err := kv.Value.GetBytes()
+ if err != nil {
+ t.Fatal(err)
+ }
+ if !bytes.Equal(val, value) {
+ t.Fatalf("expected: %s, got: %s", value, val)
+ }
checkTxnMetrics(t, metrics, "commit 1PC txn", 1, 1 /* 1PC */, 0, 0, 0)
}
diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go
index 1c2107d0a7fc..40703cff4a05 100644
--- a/pkg/roachpb/api.pb.go
+++ b/pkg/roachpb/api.pb.go
@@ -147,6 +147,7 @@
UnhandledRetryableError
HandledRetryableTxnError
TxnPrevAttemptError
+ TxnAlreadyEncounteredErrorError
IntegerOverflowError
MixedSuccessError
BatchTimestampBeforeGCError
diff --git a/pkg/roachpb/batch.go b/pkg/roachpb/batch.go
index 1473893fff7a..228422166c6a 100644
--- a/pkg/roachpb/batch.go
+++ b/pkg/roachpb/batch.go
@@ -404,6 +404,9 @@ func (ba BatchRequest) String() string {
req := arg.GetInner()
if _, ok := req.(*NoopRequest); ok {
str = append(str, req.Method().String())
+ } else if et, ok := req.(*EndTransactionRequest); ok {
+ h := req.Header()
+ str = append(str, fmt.Sprintf("%s(commit:%t) [%s]", req.Method(), et.Commit, h.Key))
} else {
h := req.Header()
str = append(str, fmt.Sprintf("%s [%s,%s)", req.Method(), h.Key, h.EndKey))
diff --git a/pkg/roachpb/data.go b/pkg/roachpb/data.go
index 14fb79c1eb28..42021c304e6f 100644
--- a/pkg/roachpb/data.go
+++ b/pkg/roachpb/data.go
@@ -877,6 +877,9 @@ func (t *Transaction) Restart(
t.WriteTooOld = false
t.RetryOnPush = false
t.Sequence = 0
+ // Reset Writing. Since we're using a new epoch, we don't care about the abort
+ // cache.
+ t.Writing = false
}
// BumpEpoch increments the transaction's epoch, allowing for an in-place
diff --git a/pkg/roachpb/data.pb.go b/pkg/roachpb/data.pb.go
index f101f99ae1fe..ce3a4d203eb6 100644
--- a/pkg/roachpb/data.pb.go
+++ b/pkg/roachpb/data.pb.go
@@ -438,8 +438,12 @@ type Transaction struct {
// The list of observed timestamps is kept sorted by NodeID. Use
// Transaction.UpdateObservedTimestamp to maintain the sorted order.
ObservedTimestamps []ObservedTimestamp `protobuf:"bytes,8,rep,name=observed_timestamps,json=observedTimestamps" json:"observed_timestamps"`
- // Writing is true if the transaction has previously executed a successful
- // write request, i.e. a request that may have left intents (across retries).
+ // Writing is true if the transaction has previously sent a Begin transaction
+ // (i.e. if it ever attempted to perform a write, so if it ever attempted to
+ // leave intents (across retries)). The flag will be set even if the BeginTxn
+ // batch failed.
+ // When set, the AbortCache must be checked by reads so that they don't miss
+ // to see the txn's previous writes.
Writing bool `protobuf:"varint,9,opt,name=writing,proto3" json:"writing,omitempty"`
// If this is true, the transaction must retry. Relevant only for
// SNAPSHOT transactions: a SERIALIZABLE transaction would have to
diff --git a/pkg/roachpb/data.proto b/pkg/roachpb/data.proto
index e2d60c94f297..5cb31c256947 100644
--- a/pkg/roachpb/data.proto
+++ b/pkg/roachpb/data.proto
@@ -335,8 +335,12 @@ message Transaction {
// The list of observed timestamps is kept sorted by NodeID. Use
// Transaction.UpdateObservedTimestamp to maintain the sorted order.
repeated ObservedTimestamp observed_timestamps = 8 [(gogoproto.nullable) = false];
- // Writing is true if the transaction has previously executed a successful
- // write request, i.e. a request that may have left intents (across retries).
+ // Writing is true if the transaction has previously sent a Begin transaction
+ // (i.e. if it ever attempted to perform a write, so if it ever attempted to
+ // leave intents (across retries)). The flag will be set even if the BeginTxn
+ // batch failed.
+ // When set, the AbortCache must be checked by reads so that they don't miss
+ // to see the txn's previous writes.
bool writing = 9;
// If this is true, the transaction must retry. Relevant only for
// SNAPSHOT transactions: a SERIALIZABLE transaction would have to
diff --git a/pkg/roachpb/errors.go b/pkg/roachpb/errors.go
index de86c087b96c..d435660c5270 100644
--- a/pkg/roachpb/errors.go
+++ b/pkg/roachpb/errors.go
@@ -628,6 +628,16 @@ func (*TxnPrevAttemptError) message(_ *Error) string {
var _ ErrorDetailInterface = &TxnPrevAttemptError{}
+func (e *TxnAlreadyEncounteredErrorError) Error() string {
+ return e.message(nil)
+}
+
+func (*TxnAlreadyEncounteredErrorError) message(_ *Error) string {
+ return "txn already encountered an error; cannot be used anymore"
+}
+
+var _ ErrorDetailInterface = &TxnAlreadyEncounteredErrorError{}
+
func (e *IntegerOverflowError) Error() string {
return e.message(nil)
}
diff --git a/pkg/roachpb/errors.pb.go b/pkg/roachpb/errors.pb.go
index 1d14641d70f1..0dd7e57fa1a9 100644
--- a/pkg/roachpb/errors.pb.go
+++ b/pkg/roachpb/errors.pb.go
@@ -526,6 +526,19 @@ func (m *TxnPrevAttemptError) String() string { return proto.CompactT
func (*TxnPrevAttemptError) ProtoMessage() {}
func (*TxnPrevAttemptError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{24} }
+// TxnAlreadyEncounteredErrorError indicates that an operation tried to use a
+// transaction that already received an error from a previous request. Once that
+// happens, client.Txn rejects future requests.
+type TxnAlreadyEncounteredErrorError struct {
+}
+
+func (m *TxnAlreadyEncounteredErrorError) Reset() { *m = TxnAlreadyEncounteredErrorError{} }
+func (m *TxnAlreadyEncounteredErrorError) String() string { return proto.CompactTextString(m) }
+func (*TxnAlreadyEncounteredErrorError) ProtoMessage() {}
+func (*TxnAlreadyEncounteredErrorError) Descriptor() ([]byte, []int) {
+ return fileDescriptorErrors, []int{25}
+}
+
// An IntegerOverflowError indicates that an operation was aborted because
// it would have caused an integeter overflow.
type IntegerOverflowError struct {
@@ -537,7 +550,7 @@ type IntegerOverflowError struct {
func (m *IntegerOverflowError) Reset() { *m = IntegerOverflowError{} }
func (m *IntegerOverflowError) String() string { return proto.CompactTextString(m) }
func (*IntegerOverflowError) ProtoMessage() {}
-func (*IntegerOverflowError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{25} }
+func (*IntegerOverflowError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{26} }
// A MixedSuccessError indicates that some portion of the batch
// request may have succeeded, but the batch as a whole failed with
@@ -549,7 +562,7 @@ type MixedSuccessError struct {
func (m *MixedSuccessError) Reset() { *m = MixedSuccessError{} }
func (m *MixedSuccessError) String() string { return proto.CompactTextString(m) }
func (*MixedSuccessError) ProtoMessage() {}
-func (*MixedSuccessError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{26} }
+func (*MixedSuccessError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{27} }
// A BatchTimestampBeforeGCError indicates that a request's timestamp was
// before the GC threshold.
@@ -562,7 +575,7 @@ func (m *BatchTimestampBeforeGCError) Reset() { *m = BatchTimestampBefor
func (m *BatchTimestampBeforeGCError) String() string { return proto.CompactTextString(m) }
func (*BatchTimestampBeforeGCError) ProtoMessage() {}
func (*BatchTimestampBeforeGCError) Descriptor() ([]byte, []int) {
- return fileDescriptorErrors, []int{27}
+ return fileDescriptorErrors, []int{28}
}
// ErrorDetail is a union type containing all available errors.
@@ -593,18 +606,19 @@ type ErrorDetail struct {
// The following three are ErrorDetails (and proto messages) because they
// needs to be communicated from the TxnCoordSender and Txn to the upper
// layers through the Sender interface.
- HandledRetryableTxnError *HandledRetryableTxnError `protobuf:"bytes,28,opt,name=handled_retryable_txn_error,json=handledRetryableTxnError" json:"handled_retryable_txn_error,omitempty"`
- TxnAbortedAsyncErr *TxnPrevAttemptError `protobuf:"bytes,30,opt,name=txn_aborted_async_err,json=txnAbortedAsyncErr" json:"txn_aborted_async_err,omitempty"`
- IntegerOverflow *IntegerOverflowError `protobuf:"bytes,31,opt,name=integer_overflow,json=integerOverflow" json:"integer_overflow,omitempty"`
- UnsupportedRequest *UnsupportedRequestError `protobuf:"bytes,32,opt,name=unsupported_request,json=unsupportedRequest" json:"unsupported_request,omitempty"`
- MixedSuccess *MixedSuccessError `protobuf:"bytes,33,opt,name=mixed_success,json=mixedSuccess" json:"mixed_success,omitempty"`
- TimestampBefore *BatchTimestampBeforeGCError `protobuf:"bytes,34,opt,name=timestamp_before,json=timestampBefore" json:"timestamp_before,omitempty"`
+ HandledRetryableTxnError *HandledRetryableTxnError `protobuf:"bytes,28,opt,name=handled_retryable_txn_error,json=handledRetryableTxnError" json:"handled_retryable_txn_error,omitempty"`
+ TxnAbortedAsyncErr *TxnPrevAttemptError `protobuf:"bytes,30,opt,name=txn_aborted_async_err,json=txnAbortedAsyncErr" json:"txn_aborted_async_err,omitempty"`
+ IntegerOverflow *IntegerOverflowError `protobuf:"bytes,31,opt,name=integer_overflow,json=integerOverflow" json:"integer_overflow,omitempty"`
+ UnsupportedRequest *UnsupportedRequestError `protobuf:"bytes,32,opt,name=unsupported_request,json=unsupportedRequest" json:"unsupported_request,omitempty"`
+ MixedSuccess *MixedSuccessError `protobuf:"bytes,33,opt,name=mixed_success,json=mixedSuccess" json:"mixed_success,omitempty"`
+ TimestampBefore *BatchTimestampBeforeGCError `protobuf:"bytes,34,opt,name=timestamp_before,json=timestampBefore" json:"timestamp_before,omitempty"`
+ TxnAlreadyEncounteredError *TxnAlreadyEncounteredErrorError `protobuf:"bytes,35,opt,name=txn_already_encountered_error,json=txnAlreadyEncounteredError" json:"txn_already_encountered_error,omitempty"`
}
func (m *ErrorDetail) Reset() { *m = ErrorDetail{} }
func (m *ErrorDetail) String() string { return proto.CompactTextString(m) }
func (*ErrorDetail) ProtoMessage() {}
-func (*ErrorDetail) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{28} }
+func (*ErrorDetail) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{29} }
// ErrPosition describes the position of an error in a Batch. A simple nullable
// primitive field would break compatibility with proto3, where primitive fields
@@ -616,7 +630,7 @@ type ErrPosition struct {
func (m *ErrPosition) Reset() { *m = ErrPosition{} }
func (m *ErrPosition) String() string { return proto.CompactTextString(m) }
func (*ErrPosition) ProtoMessage() {}
-func (*ErrPosition) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{29} }
+func (*ErrPosition) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{30} }
// Error is a generic representation including a string message
// and information about retryability.
@@ -646,7 +660,7 @@ type Error struct {
func (m *Error) Reset() { *m = Error{} }
func (*Error) ProtoMessage() {}
-func (*Error) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{30} }
+func (*Error) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{31} }
func init() {
proto.RegisterType((*NotLeaseHolderError)(nil), "cockroach.roachpb.NotLeaseHolderError")
@@ -674,6 +688,7 @@ func init() {
proto.RegisterType((*UnhandledRetryableError)(nil), "cockroach.roachpb.UnhandledRetryableError")
proto.RegisterType((*HandledRetryableTxnError)(nil), "cockroach.roachpb.HandledRetryableTxnError")
proto.RegisterType((*TxnPrevAttemptError)(nil), "cockroach.roachpb.TxnPrevAttemptError")
+ proto.RegisterType((*TxnAlreadyEncounteredErrorError)(nil), "cockroach.roachpb.TxnAlreadyEncounteredErrorError")
proto.RegisterType((*IntegerOverflowError)(nil), "cockroach.roachpb.IntegerOverflowError")
proto.RegisterType((*MixedSuccessError)(nil), "cockroach.roachpb.MixedSuccessError")
proto.RegisterType((*BatchTimestampBeforeGCError)(nil), "cockroach.roachpb.BatchTimestampBeforeGCError")
@@ -1303,6 +1318,27 @@ func (this *TxnPrevAttemptError) Equal(that interface{}) bool {
}
return true
}
+func (this *TxnAlreadyEncounteredErrorError) Equal(that interface{}) bool {
+ if that == nil {
+ return this == nil
+ }
+
+ that1, ok := that.(*TxnAlreadyEncounteredErrorError)
+ if !ok {
+ that2, ok := that.(TxnAlreadyEncounteredErrorError)
+ if ok {
+ that1 = &that2
+ } else {
+ return false
+ }
+ }
+ if that1 == nil {
+ return this == nil
+ } else if this == nil {
+ return false
+ }
+ return true
+}
func (this *IntegerOverflowError) Equal(that interface{}) bool {
if that == nil {
return this == nil
@@ -1484,6 +1520,9 @@ func (this *ErrorDetail) Equal(that interface{}) bool {
if !this.TimestampBefore.Equal(that1.TimestampBefore) {
return false
}
+ if !this.TxnAlreadyEncounteredError.Equal(that1.TxnAlreadyEncounteredError) {
+ return false
+ }
return true
}
func (this *ErrPosition) Equal(that interface{}) bool {
@@ -2250,6 +2289,24 @@ func (m *TxnPrevAttemptError) MarshalTo(dAtA []byte) (int, error) {
return i, nil
}
+func (m *TxnAlreadyEncounteredErrorError) Marshal() (dAtA []byte, err error) {
+ size := m.Size()
+ dAtA = make([]byte, size)
+ n, err := m.MarshalTo(dAtA)
+ if err != nil {
+ return nil, err
+ }
+ return dAtA[:n], nil
+}
+
+func (m *TxnAlreadyEncounteredErrorError) MarshalTo(dAtA []byte) (int, error) {
+ var i int
+ _ = i
+ var l int
+ _ = l
+ return i, nil
+}
+
func (m *IntegerOverflowError) Marshal() (dAtA []byte, err error) {
size := m.Size()
dAtA = make([]byte, size)
@@ -2651,6 +2708,18 @@ func (m *ErrorDetail) MarshalTo(dAtA []byte) (int, error) {
}
i += n48
}
+ if m.TxnAlreadyEncounteredError != nil {
+ dAtA[i] = 0x9a
+ i++
+ dAtA[i] = 0x2
+ i++
+ i = encodeVarintErrors(dAtA, i, uint64(m.TxnAlreadyEncounteredError.Size()))
+ n49, err := m.TxnAlreadyEncounteredError.MarshalTo(dAtA[i:])
+ if err != nil {
+ return 0, err
+ }
+ i += n49
+ }
return i, nil
}
@@ -2701,11 +2770,11 @@ func (m *Error) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x22
i++
i = encodeVarintErrors(dAtA, i, uint64(m.UnexposedTxn.Size()))
- n49, err := m.UnexposedTxn.MarshalTo(dAtA[i:])
+ n50, err := m.UnexposedTxn.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n49
+ i += n50
}
dAtA[i] = 0x28
i++
@@ -2714,30 +2783,30 @@ func (m *Error) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x32
i++
i = encodeVarintErrors(dAtA, i, uint64(m.Detail.Size()))
- n50, err := m.Detail.MarshalTo(dAtA[i:])
+ n51, err := m.Detail.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n50
+ i += n51
}
if m.Index != nil {
dAtA[i] = 0x3a
i++
i = encodeVarintErrors(dAtA, i, uint64(m.Index.Size()))
- n51, err := m.Index.MarshalTo(dAtA[i:])
+ n52, err := m.Index.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n51
+ i += n52
}
dAtA[i] = 0x42
i++
i = encodeVarintErrors(dAtA, i, uint64(m.Now.Size()))
- n52, err := m.Now.MarshalTo(dAtA[i:])
+ n53, err := m.Now.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n52
+ i += n53
return i, nil
}
@@ -2991,6 +3060,12 @@ func (m *TxnPrevAttemptError) Size() (n int) {
return n
}
+func (m *TxnAlreadyEncounteredErrorError) Size() (n int) {
+ var l int
+ _ = l
+ return n
+}
+
func (m *IntegerOverflowError) Size() (n int) {
var l int
_ = l
@@ -3134,6 +3209,10 @@ func (m *ErrorDetail) Size() (n int) {
l = m.TimestampBefore.Size()
n += 2 + l + sovErrors(uint64(l))
}
+ if m.TxnAlreadyEncounteredError != nil {
+ l = m.TxnAlreadyEncounteredError.Size()
+ n += 2 + l + sovErrors(uint64(l))
+ }
return n
}
@@ -3263,6 +3342,9 @@ func (this *ErrorDetail) GetValue() interface{} {
if this.TimestampBefore != nil {
return this.TimestampBefore
}
+ if this.TxnAlreadyEncounteredError != nil {
+ return this.TxnAlreadyEncounteredError
+ }
return nil
}
@@ -3322,6 +3404,8 @@ func (this *ErrorDetail) SetValue(value interface{}) bool {
this.MixedSuccess = vt
case *BatchTimestampBeforeGCError:
this.TimestampBefore = vt
+ case *TxnAlreadyEncounteredErrorError:
+ this.TxnAlreadyEncounteredError = vt
default:
return false
}
@@ -5599,6 +5683,56 @@ func (m *TxnPrevAttemptError) Unmarshal(dAtA []byte) error {
}
return nil
}
+func (m *TxnAlreadyEncounteredErrorError) Unmarshal(dAtA []byte) error {
+ l := len(dAtA)
+ iNdEx := 0
+ for iNdEx < l {
+ preIndex := iNdEx
+ var wire uint64
+ for shift := uint(0); ; shift += 7 {
+ if shift >= 64 {
+ return ErrIntOverflowErrors
+ }
+ if iNdEx >= l {
+ return io.ErrUnexpectedEOF
+ }
+ b := dAtA[iNdEx]
+ iNdEx++
+ wire |= (uint64(b) & 0x7F) << shift
+ if b < 0x80 {
+ break
+ }
+ }
+ fieldNum := int32(wire >> 3)
+ wireType := int(wire & 0x7)
+ if wireType == 4 {
+ return fmt.Errorf("proto: TxnAlreadyEncounteredErrorError: wiretype end group for non-group")
+ }
+ if fieldNum <= 0 {
+ return fmt.Errorf("proto: TxnAlreadyEncounteredErrorError: illegal tag %d (wire type %d)", fieldNum, wire)
+ }
+ switch fieldNum {
+ default:
+ iNdEx = preIndex
+ skippy, err := skipErrors(dAtA[iNdEx:])
+ if err != nil {
+ return err
+ }
+ if skippy < 0 {
+ return ErrInvalidLengthErrors
+ }
+ if (iNdEx + skippy) > l {
+ return io.ErrUnexpectedEOF
+ }
+ iNdEx += skippy
+ }
+ }
+
+ if iNdEx > l {
+ return io.ErrUnexpectedEOF
+ }
+ return nil
+}
func (m *IntegerOverflowError) Unmarshal(dAtA []byte) error {
l := len(dAtA)
iNdEx := 0
@@ -6831,6 +6965,39 @@ func (m *ErrorDetail) Unmarshal(dAtA []byte) error {
return err
}
iNdEx = postIndex
+ case 35:
+ if wireType != 2 {
+ return fmt.Errorf("proto: wrong wireType = %d for field TxnAlreadyEncounteredError", wireType)
+ }
+ var msglen int
+ for shift := uint(0); ; shift += 7 {
+ if shift >= 64 {
+ return ErrIntOverflowErrors
+ }
+ if iNdEx >= l {
+ return io.ErrUnexpectedEOF
+ }
+ b := dAtA[iNdEx]
+ iNdEx++
+ msglen |= (int(b) & 0x7F) << shift
+ if b < 0x80 {
+ break
+ }
+ }
+ if msglen < 0 {
+ return ErrInvalidLengthErrors
+ }
+ postIndex := iNdEx + msglen
+ if postIndex > l {
+ return io.ErrUnexpectedEOF
+ }
+ if m.TxnAlreadyEncounteredError == nil {
+ m.TxnAlreadyEncounteredError = &TxnAlreadyEncounteredErrorError{}
+ }
+ if err := m.TxnAlreadyEncounteredError.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+ return err
+ }
+ iNdEx = postIndex
default:
iNdEx = preIndex
skippy, err := skipErrors(dAtA[iNdEx:])
@@ -7275,150 +7442,153 @@ var (
func init() { proto.RegisterFile("roachpb/errors.proto", fileDescriptorErrors) }
var fileDescriptorErrors = []byte{
- // 2315 bytes of a gzipped FileDescriptorProto
- 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x98, 0xcf, 0x53, 0x1b, 0xc9,
- 0x15, 0xc7, 0x19, 0x24, 0x10, 0x3c, 0x21, 0x24, 0xda, 0x98, 0x1d, 0xe3, 0xb5, 0x60, 0xd9, 0xfc,
- 0xb0, 0x9d, 0x0a, 0x6c, 0x79, 0xe3, 0x4d, 0xad, 0x93, 0x1c, 0x04, 0x12, 0xac, 0x0c, 0x48, 0xa4,
- 0x11, 0xeb, 0x65, 0x37, 0x55, 0x53, 0xc3, 0x4c, 0x23, 0x66, 0x2d, 0xcd, 0x28, 0x3d, 0x3d, 0x20,
- 0x2e, 0x39, 0xe7, 0x98, 0xdc, 0x72, 0x48, 0x55, 0x5c, 0xc9, 0x29, 0x95, 0x4b, 0x0e, 0xa9, 0x1c,
- 0x73, 0xf6, 0x31, 0xc7, 0x54, 0xaa, 0x42, 0x25, 0xe4, 0x92, 0xaa, 0xfc, 0x07, 0x3e, 0xa5, 0xfa,
- 0xc7, 0x68, 0x46, 0x68, 0x06, 0x2b, 0xbe, 0x49, 0xaf, 0x5f, 0x7f, 0xfb, 0xf5, 0x8f, 0xf7, 0xfa,
- 0xd3, 0x03, 0x8b, 0xd4, 0x33, 0xad, 0xb3, 0xde, 0xc9, 0x06, 0xa1, 0xd4, 0xa3, 0xfe, 0x7a, 0x8f,
- 0x7a, 0xcc, 0x43, 0x0b, 0x96, 0x67, 0xbd, 0x14, 0x2d, 0xeb, 0xaa, 0x7d, 0x79, 0x29, 0x74, 0xec,
- 0x12, 0x66, 0xda, 0x26, 0x33, 0xa5, 0xeb, 0x32, 0x0a, 0xed, 0x31, 0x9b, 0x1e, 0x30, 0xa7, 0xb3,
- 0x71, 0xd6, 0xb1, 0x36, 0x98, 0xd3, 0x25, 0x3e, 0x33, 0xbb, 0x3d, 0xd5, 0xb2, 0xd8, 0xf6, 0xda,
- 0x9e, 0xf8, 0xb9, 0xc1, 0x7f, 0x49, 0xeb, 0xda, 0x9f, 0x27, 0xe1, 0x4e, 0xc3, 0x63, 0x7b, 0xc4,
- 0xf4, 0xc9, 0x67, 0x5e, 0xc7, 0x26, 0xb4, 0xc6, 0xa3, 0x41, 0x55, 0xc8, 0x51, 0xd2, 0xeb, 0x38,
- 0x96, 0xa9, 0x6b, 0xab, 0xda, 0xc3, 0xfc, 0x93, 0x6f, 0xac, 0x8f, 0x04, 0xb6, 0x8e, 0xa5, 0x47,
- 0x95, 0xf8, 0x16, 0x75, 0x7a, 0xcc, 0xa3, 0x9b, 0xd9, 0xd7, 0x57, 0x2b, 0x13, 0x38, 0xec, 0x8a,
- 0x76, 0x60, 0xae, 0xc3, 0x95, 0x8d, 0x33, 0x21, 0xad, 0x4f, 0x8e, 0x2f, 0x85, 0xf3, 0x9d, 0x28,
- 0x26, 0xf4, 0x14, 0x66, 0xa8, 0xe9, 0xb6, 0x89, 0xe1, 0xd8, 0x7a, 0x66, 0x55, 0x7b, 0x98, 0xd9,
- 0x5c, 0xe6, 0x23, 0x5d, 0x5f, 0xad, 0xe4, 0x30, 0xb7, 0xd7, 0xab, 0x6f, 0xa2, 0x9f, 0x38, 0x27,
- 0x7c, 0xeb, 0x36, 0x5a, 0x87, 0x29, 0xa1, 0xa2, 0x67, 0xc5, 0xc0, 0x7a, 0xc2, 0xc0, 0x62, 0xe6,
- 0x58, 0xba, 0xa1, 0x0f, 0x01, 0xac, 0xc0, 0x67, 0x5e, 0xd7, 0xe8, 0xfa, 0x6d, 0x7d, 0x6a, 0x55,
- 0x7b, 0x38, 0xab, 0xa6, 0x34, 0x2b, 0xed, 0xfb, 0x7e, 0xfb, 0x59, 0xf6, 0x3f, 0xaf, 0x56, 0xb4,
- 0xb5, 0xf7, 0x61, 0xb1, 0xe1, 0xd9, 0xe4, 0xc8, 0x35, 0xcf, 0x4d, 0xa7, 0x63, 0x9e, 0x74, 0x88,
- 0x58, 0x38, 0xd5, 0xba, 0x02, 0xef, 0x1d, 0xb9, 0x7e, 0xd0, 0xeb, 0x79, 0x94, 0x11, 0x1b, 0x93,
- 0x9f, 0x06, 0xc4, 0x67, 0x71, 0x87, 0x1f, 0x03, 0x12, 0xd1, 0x36, 0x3c, 0xb6, 0xed, 0x05, 0xae,
- 0x2d, 0x57, 0x3d, 0x3e, 0x4d, 0x6d, 0xec, 0x69, 0x2a, 0xc9, 0xdf, 0x4c, 0xc2, 0x5d, 0xd1, 0xb4,
- 0x4b, 0x2e, 0xf7, 0x1d, 0xbf, 0x6b, 0x32, 0xeb, 0x4c, 0xca, 0x7e, 0x0c, 0x0b, 0x54, 0x86, 0x60,
- 0xf8, 0xcc, 0xa4, 0xcc, 0x78, 0x49, 0x2e, 0x85, 0xfe, 0xdc, 0x66, 0xee, 0xcd, 0xd5, 0x4a, 0x66,
- 0x97, 0x5c, 0xe2, 0xa2, 0xf2, 0x38, 0xe4, 0x0e, 0xbb, 0xe4, 0x12, 0x6d, 0x40, 0x68, 0x32, 0x88,
- 0x6b, 0x8b, 0x2e, 0x93, 0xc3, 0x5d, 0x0a, 0xaa, 0xbd, 0xe6, 0xda, 0xbc, 0xc3, 0x3e, 0x94, 0xba,
- 0x6a, 0x58, 0x62, 0x1b, 0x22, 0x36, 0xb1, 0x57, 0xf9, 0x27, 0x6b, 0x49, 0x1b, 0xce, 0xdb, 0x63,
- 0xdb, 0x5d, 0x8c, 0xfa, 0x8a, 0x26, 0xb4, 0x0b, 0x45, 0x3f, 0x68, 0xb7, 0x89, 0xcf, 0x06, 0x6a,
- 0xd9, 0xb1, 0xd5, 0xe6, 0x07, 0x5d, 0x45, 0x8b, 0x5a, 0xa1, 0xff, 0x4e, 0xc2, 0x1a, 0x26, 0xa6,
- 0xfd, 0xc2, 0x61, 0x67, 0x8e, 0x7b, 0xe4, 0x5a, 0x84, 0x32, 0xd3, 0x71, 0xd9, 0x65, 0xdd, 0x65,
- 0x84, 0x9e, 0x9b, 0x1d, 0xb9, 0x5c, 0xcf, 0x61, 0x9e, 0x12, 0xd3, 0x36, 0x06, 0x19, 0xa4, 0x52,
- 0xe0, 0x41, 0x6c, 0x60, 0x9e, 0x66, 0xeb, 0x67, 0x1d, 0x6b, 0xbd, 0x15, 0x3a, 0xa9, 0x83, 0x52,
- 0xe0, 0x5d, 0x07, 0x46, 0x84, 0x01, 0x91, 0xbe, 0xe3, 0x33, 0xc7, 0x6d, 0xc7, 0xf4, 0x26, 0xc7,
- 0xd7, 0x5b, 0x08, 0xbb, 0x47, 0x9a, 0x9b, 0x50, 0xe8, 0x9a, 0xfd, 0x98, 0x5c, 0x66, 0x0c, 0x39,
- 0x3c, 0xd7, 0x35, 0xfb, 0x91, 0xc6, 0x57, 0x70, 0xc7, 0x3b, 0xf1, 0x09, 0x3d, 0x27, 0xb1, 0x79,
- 0xfa, 0x7a, 0x76, 0x35, 0x93, 0x92, 0xa0, 0x4d, 0xe5, 0x7d, 0x33, 0x3e, 0xe4, 0xdd, 0x6c, 0xf0,
- 0xa3, 0x1c, 0x68, 0x51, 0xd3, 0xf5, 0x4d, 0x8b, 0x39, 0x9e, 0x5b, 0x39, 0x11, 0xa9, 0x10, 0xcf,
- 0x01, 0x13, 0x16, 0x63, 0x0e, 0x07, 0x81, 0xaf, 0x8e, 0xeb, 0x16, 0x40, 0x2f, 0xf0, 0xcf, 0x08,
- 0x31, 0x58, 0xdf, 0x55, 0x6b, 0x5f, 0x4e, 0x08, 0x29, 0xd6, 0x39, 0xcc, 0x52, 0xd9, 0xaf, 0xd5,
- 0x77, 0xd5, 0x10, 0xa7, 0x70, 0x37, 0xe6, 0x85, 0x09, 0xa3, 0x97, 0x72, 0x8c, 0x1d, 0x98, 0xa6,
- 0xc4, 0xf4, 0x3d, 0xa9, 0x3f, 0xff, 0xe4, 0xd1, 0xed, 0xfa, 0xa2, 0x27, 0x16, 0x1d, 0xd4, 0x50,
- 0xaa, 0xbb, 0x1a, 0xa7, 0x0c, 0x4b, 0x43, 0xde, 0xbd, 0x8e, 0x79, 0x19, 0x9f, 0xea, 0x5f, 0xb4,
- 0x21, 0x87, 0x43, 0x66, 0xb2, 0xc0, 0x97, 0x91, 0x2c, 0x41, 0x86, 0x17, 0x1b, 0x2d, 0x56, 0x6c,
- 0xb8, 0x01, 0x35, 0x06, 0x11, 0x4e, 0x8a, 0x08, 0x3f, 0xba, 0x3d, 0xc2, 0x98, 0xe4, 0x7a, 0x52,
- 0xa0, 0x6b, 0x9f, 0xc0, 0xb4, 0xb4, 0x23, 0x04, 0xf3, 0xb8, 0x56, 0x39, 0x6c, 0x36, 0x8c, 0xa3,
- 0xc6, 0x6e, 0xa3, 0xf9, 0xa2, 0x51, 0x9a, 0x40, 0x3a, 0x2c, 0x2a, 0x5b, 0xeb, 0x8b, 0x86, 0xd1,
- 0x68, 0xb6, 0x8c, 0xed, 0xe6, 0x51, 0xa3, 0x5a, 0xd2, 0xd4, 0x04, 0x8e, 0xa1, 0xf4, 0x82, 0x3a,
- 0x8c, 0xf0, 0x4c, 0x71, 0x65, 0x25, 0x43, 0x9f, 0x42, 0xce, 0x11, 0x7f, 0x7d, 0x5d, 0x13, 0xe7,
- 0xe6, 0x5e, 0x42, 0x88, 0xb2, 0x43, 0x78, 0x31, 0x28, 0x7f, 0x29, 0xfa, 0x3c, 0x3b, 0x33, 0x59,
- 0xca, 0xac, 0xfd, 0x41, 0x53, 0xda, 0x2d, 0xcf, 0x6b, 0x76, 0x54, 0x25, 0xac, 0xc0, 0xec, 0x3b,
- 0xa5, 0x5f, 0xd4, 0x0b, 0x35, 0xa0, 0x64, 0x5a, 0x2c, 0x30, 0x3b, 0xef, 0x96, 0x78, 0x45, 0xd9,
- 0x79, 0x60, 0x56, 0x0b, 0xb1, 0x0c, 0xa8, 0xd9, 0xe3, 0x05, 0xdd, 0xa1, 0xc4, 0x6f, 0xf5, 0xdd,
- 0xf8, 0x2e, 0x1f, 0xc3, 0xe2, 0x96, 0xe7, 0xda, 0x0e, 0xdf, 0x8f, 0x6d, 0xd3, 0xe9, 0xa8, 0xe3,
- 0x8e, 0x7e, 0x00, 0x73, 0x2a, 0x92, 0x73, 0xb3, 0x13, 0x10, 0x35, 0x9f, 0xa4, 0xdb, 0xe8, 0x73,
- 0xde, 0x8e, 0xf3, 0xd2, 0x5b, 0xfc, 0x51, 0xd2, 0x7f, 0xd2, 0x00, 0xc9, 0xab, 0x8a, 0x7c, 0x4d,
- 0xac, 0x30, 0x91, 0x50, 0x19, 0x72, 0x5d, 0xe2, 0xfb, 0x66, 0x9b, 0x0c, 0x1d, 0xa0, 0xd0, 0x88,
- 0x7e, 0x08, 0xb3, 0xaa, 0x48, 0x13, 0x5b, 0x4d, 0x3e, 0xf5, 0x12, 0x0c, 0x57, 0x70, 0xd0, 0x01,
- 0x3d, 0x83, 0x99, 0xb0, 0xfa, 0xa8, 0x1a, 0xf3, 0xb6, 0xce, 0x03, 0x7f, 0x15, 0xf6, 0xf7, 0x61,
- 0xf6, 0x90, 0xb8, 0xe3, 0x05, 0x3b, 0x74, 0x28, 0x2e, 0x60, 0xb1, 0xd2, 0x3d, 0x71, 0xda, 0x81,
- 0x17, 0xf8, 0x98, 0xf8, 0x41, 0x87, 0x8d, 0x37, 0xe1, 0x4f, 0x21, 0x7f, 0x41, 0xcd, 0x5e, 0x8f,
- 0xd8, 0x06, 0xa1, 0xf4, 0x96, 0x29, 0x0b, 0x39, 0x0c, 0xca, 0xb9, 0x46, 0xc3, 0x3d, 0x7c, 0xc0,
- 0x2f, 0xd1, 0x53, 0xb6, 0x43, 0xbd, 0xa0, 0x57, 0x25, 0x1d, 0x72, 0xa3, 0x66, 0x11, 0x58, 0x52,
- 0xa8, 0xb2, 0xe5, 0x51, 0x1a, 0xf4, 0xf8, 0x56, 0xcb, 0xc8, 0x3e, 0x80, 0x59, 0x01, 0x72, 0xc6,
- 0xcd, 0x6c, 0x9e, 0x11, 0xe6, 0x7d, 0xbf, 0x8d, 0xd6, 0x60, 0xb6, 0x47, 0x3d, 0x8b, 0xf8, 0xbe,
- 0xda, 0x8d, 0x99, 0x41, 0xdd, 0x0a, 0xcd, 0x83, 0x93, 0x84, 0xd4, 0x30, 0xf1, 0xa4, 0xf8, 0x11,
- 0x80, 0x22, 0xab, 0x10, 0x10, 0xa6, 0x36, 0xcb, 0x0a, 0x10, 0x66, 0x95, 0xbf, 0x40, 0x84, 0xe8,
- 0x0f, 0xdf, 0x4e, 0xf9, 0xd3, 0x8e, 0xc8, 0xe3, 0x90, 0x79, 0x74, 0x94, 0x3c, 0x7c, 0x6e, 0x4d,
- 0x20, 0x0f, 0xe1, 0x2d, 0xc9, 0x43, 0xfd, 0xc4, 0x39, 0xe1, 0x3b, 0x90, 0xdc, 0xe7, 0xb4, 0x73,
- 0x66, 0xba, 0x76, 0x87, 0xb3, 0x0e, 0xa3, 0x97, 0x03, 0x1c, 0x42, 0x4f, 0x20, 0xdb, 0xab, 0x51,
- 0x7a, 0xcb, 0x91, 0x17, 0x7e, 0x6a, 0x1d, 0x84, 0xef, 0xda, 0x3f, 0x34, 0xd0, 0x3f, 0xbb, 0xa1,
- 0x16, 0x66, 0x5a, 0x6a, 0xb9, 0xfc, 0x0a, 0xa6, 0x59, 0xdf, 0xe5, 0xe1, 0x4b, 0x4a, 0xa9, 0xf2,
- 0xa6, 0xbf, 0x5f, 0xad, 0x7c, 0xdc, 0x76, 0xd8, 0x59, 0x70, 0xb2, 0x6e, 0x79, 0xdd, 0x8d, 0xc1,
- 0xe0, 0xf6, 0x49, 0xf4, 0x7b, 0xa3, 0xf7, 0xb2, 0xbd, 0x21, 0xa8, 0x39, 0x08, 0x1c, 0x7b, 0xfd,
- 0xe8, 0xa8, 0x5e, 0xbd, 0xbe, 0x5a, 0x99, 0x6a, 0xf5, 0xdd, 0x7a, 0x15, 0x4f, 0xb1, 0xbe, 0x5b,
- 0xb7, 0xd1, 0x36, 0xe4, 0x59, 0x54, 0x6a, 0x55, 0x2e, 0x8c, 0x77, 0x25, 0xc5, 0x3b, 0xaa, 0xe5,
- 0xba, 0x0f, 0x77, 0x5a, 0x7d, 0xf7, 0x80, 0x92, 0xf3, 0x0a, 0x63, 0xa4, 0xdb, 0x1b, 0x02, 0xc3,
- 0x5f, 0x6a, 0xb0, 0xc8, 0x6b, 0x66, 0x9b, 0xd0, 0xe6, 0x39, 0xa1, 0xa7, 0x1d, 0xef, 0x42, 0x4e,
- 0xfc, 0x1e, 0x64, 0x12, 0xb0, 0x8d, 0xdb, 0xd0, 0x23, 0x28, 0x58, 0x01, 0xa5, 0xc4, 0x65, 0xaa,
- 0xc0, 0x4c, 0x8a, 0x1d, 0x94, 0x01, 0xcc, 0xa9, 0x26, 0x51, 0x4d, 0xd0, 0x77, 0xa1, 0xe8, 0xb8,
- 0x16, 0x25, 0xdd, 0xc8, 0x39, 0x13, 0x73, 0x9e, 0x1f, 0x34, 0xc6, 0x8b, 0xcf, 0x3e, 0x2c, 0xec,
- 0x3b, 0x7d, 0x62, 0x1f, 0x06, 0x16, 0x3f, 0xa5, 0xe1, 0xce, 0xe6, 0x54, 0xf2, 0xbc, 0x6d, 0x73,
- 0x71, 0xe8, 0xa8, 0xe4, 0x7e, 0xaf, 0xc1, 0xfd, 0x4d, 0x8e, 0x7a, 0x51, 0xc9, 0x25, 0xa7, 0x1e,
- 0x25, 0x3b, 0x5b, 0x83, 0xda, 0xdf, 0x7a, 0xa7, 0xda, 0x1f, 0xe1, 0x0d, 0x97, 0x38, 0xa3, 0xc4,
- 0xe7, 0xef, 0x8e, 0xff, 0xa7, 0xe8, 0x47, 0xbd, 0x54, 0xac, 0xbf, 0x45, 0x90, 0x17, 0x51, 0x55,
- 0x09, 0x33, 0x9d, 0x0e, 0x3a, 0x80, 0x92, 0xeb, 0x31, 0x63, 0xe8, 0x55, 0x23, 0x43, 0xfc, 0x56,
- 0xc2, 0xf4, 0x13, 0x5e, 0x56, 0x78, 0xde, 0x1d, 0x32, 0xa2, 0x7d, 0x28, 0x4a, 0xe6, 0xe7, 0xba,
- 0xa7, 0x3c, 0x23, 0x55, 0xc0, 0xdf, 0x4c, 0xe3, 0xdc, 0xa1, 0xcc, 0xc5, 0x05, 0x1a, 0xb7, 0xa1,
- 0xcf, 0x01, 0x49, 0xb9, 0x97, 0xe4, 0xd2, 0x08, 0x99, 0x5a, 0x9d, 0xd8, 0x87, 0x69, 0x8a, 0x37,
- 0x5f, 0x0c, 0xb8, 0x44, 0x6f, 0x98, 0xd1, 0xcf, 0x60, 0x55, 0x40, 0xf1, 0x85, 0x60, 0x67, 0x23,
- 0x88, 0xe0, 0xd9, 0x70, 0x14, 0x3d, 0x2b, 0x3e, 0x7f, 0x9a, 0xf8, 0xbc, 0x7b, 0x1b, 0x75, 0xe3,
- 0x07, 0xf4, 0x36, 0x1f, 0x0e, 0xac, 0xb1, 0x4c, 0x32, 0x4c, 0x89, 0x93, 0xe2, 0x8d, 0x96, 0x7f,
- 0xf2, 0xf8, 0xf6, 0x54, 0x8c, 0xb3, 0x27, 0x46, 0x6c, 0xa4, 0x01, 0x61, 0x28, 0xc5, 0xc5, 0x39,
- 0x45, 0xea, 0xd3, 0x42, 0xf9, 0xdb, 0xb7, 0x2b, 0x0f, 0xa0, 0x15, 0x17, 0xd9, 0xb0, 0x15, 0x1d,
- 0xc1, 0x42, 0x5c, 0x93, 0xf2, 0x4a, 0xa6, 0xe7, 0x52, 0xf7, 0x21, 0x11, 0x53, 0x71, 0x3c, 0x2c,
- 0x61, 0x46, 0x5f, 0x40, 0x7c, 0x02, 0xfc, 0x3d, 0xc7, 0x02, 0x5f, 0x9f, 0x11, 0xba, 0x8f, 0xc6,
- 0x46, 0x44, 0x1c, 0x8f, 0x4d, 0xda, 0xd1, 0x36, 0xcc, 0x5d, 0x70, 0x0c, 0x33, 0x24, 0xa4, 0xe9,
- 0xb3, 0x42, 0xf3, 0xc3, 0x04, 0xcd, 0x9b, 0x24, 0x88, 0xf3, 0x17, 0x91, 0x05, 0xed, 0x40, 0x41,
- 0xea, 0x30, 0xcf, 0x33, 0x78, 0xfe, 0xc1, 0xed, 0x42, 0xb1, 0x1b, 0x4e, 0x09, 0x49, 0x0b, 0xcf,
- 0x0c, 0xaf, 0x67, 0x50, 0xc5, 0x5a, 0xe2, 0x31, 0x90, 0x4f, 0xcd, 0x8c, 0x51, 0x28, 0xc3, 0x05,
- 0x2f, 0x6e, 0xe3, 0x9b, 0x6c, 0x85, 0x74, 0x66, 0x9c, 0x0a, 0x3c, 0xd3, 0xe7, 0x52, 0x37, 0x39,
- 0x09, 0xe4, 0x70, 0xd1, 0x1a, 0xb6, 0xa2, 0x3d, 0x98, 0x97, 0xa5, 0x80, 0x2a, 0x2c, 0xd3, 0x0b,
- 0xa9, 0x11, 0x8e, 0xe2, 0x1b, 0x2e, 0x74, 0xe2, 0x36, 0x1e, 0xa1, 0xeb, 0xd9, 0xc4, 0x08, 0xa2,
- 0x8f, 0x0a, 0xfa, 0x7c, 0x6a, 0x84, 0x49, 0x9f, 0x1f, 0x70, 0xd1, 0x1d, 0xb6, 0xa2, 0x8f, 0x20,
- 0xeb, 0x13, 0xd7, 0xd6, 0x8b, 0x42, 0xe7, 0xfd, 0x04, 0x9d, 0x01, 0xa0, 0x61, 0xe1, 0x29, 0x2b,
- 0xc8, 0x29, 0x33, 0xda, 0x1c, 0x81, 0x0c, 0x5b, 0x32, 0x90, 0x5e, 0xba, 0xa5, 0x82, 0x24, 0xe0,
- 0x12, 0xaf, 0x20, 0xc3, 0x66, 0x7e, 0x72, 0x43, 0x7a, 0xb1, 0x06, 0xec, 0xa4, 0x2f, 0xa4, 0x9e,
- 0xdc, 0x64, 0xce, 0xc2, 0x0b, 0xf4, 0xa6, 0x5d, 0x94, 0x50, 0xa5, 0x1c, 0x9e, 0x39, 0x94, 0x5e,
- 0x42, 0x47, 0xb8, 0x8a, 0xbf, 0xd9, 0x63, 0xb6, 0x9b, 0x29, 0x46, 0xc5, 0x6b, 0x4e, 0x5f, 0x1a,
- 0x27, 0xc5, 0x62, 0x2f, 0xbf, 0xa1, 0x14, 0x93, 0x76, 0xbe, 0xc1, 0x66, 0x48, 0xb5, 0x06, 0x15,
- 0x58, 0xab, 0x2f, 0xa7, 0x6e, 0x70, 0x12, 0x00, 0xe3, 0xa2, 0x39, 0x6c, 0xe5, 0x93, 0x97, 0xe4,
- 0x16, 0xdd, 0x1f, 0xf7, 0x53, 0x27, 0x3f, 0x4a, 0x7e, 0xb8, 0xe0, 0xc7, 0x6d, 0xe8, 0x6b, 0xb8,
- 0xaf, 0x48, 0x4e, 0x96, 0x2c, 0x7e, 0x88, 0x78, 0xea, 0x19, 0x82, 0x62, 0xf5, 0xf7, 0x85, 0xf4,
- 0x77, 0x12, 0xa4, 0xd3, 0x88, 0x0d, 0xeb, 0x67, 0x69, 0x2c, 0x77, 0x0c, 0x77, 0xb9, 0xb2, 0xaa,
- 0xe5, 0x86, 0xe9, 0x5f, 0xba, 0x96, 0xc0, 0xf6, 0x72, 0xea, 0x8d, 0x9a, 0x00, 0x4e, 0x18, 0xb1,
- 0x7e, 0x58, 0xc9, 0x2b, 0x5c, 0xa2, 0x46, 0x29, 0x5f, 0x69, 0x47, 0x52, 0x94, 0xe1, 0x29, 0x8c,
- 0xd2, 0x57, 0x52, 0x57, 0x3a, 0x09, 0xb8, 0x70, 0xd1, 0x19, 0xb6, 0xf2, 0x2b, 0x28, 0x88, 0x3e,
- 0xea, 0x19, 0xea, 0x9d, 0xa4, 0xaf, 0xa6, 0x5e, 0x41, 0x29, 0x9f, 0x00, 0x31, 0x0a, 0x46, 0x1a,
- 0x50, 0x1d, 0x0a, 0x5d, 0xce, 0x58, 0x86, 0x2f, 0x21, 0x4b, 0xff, 0x20, 0xf5, 0x5b, 0xe9, 0x08,
- 0x8b, 0xe1, 0xb9, 0x6e, 0xcc, 0x84, 0x8e, 0xa1, 0x34, 0x78, 0xf1, 0x1a, 0x27, 0x02, 0xad, 0xf4,
- 0x35, 0xa1, 0xb6, 0x9e, 0xa0, 0x76, 0x0b, 0x89, 0xe1, 0x22, 0x1b, 0xb6, 0x3f, 0x9b, 0x79, 0xfd,
- 0x6a, 0x45, 0x53, 0xcf, 0xb4, 0x7b, 0xa5, 0xe5, 0xe7, 0xd9, 0x99, 0x07, 0xa5, 0xf2, 0xda, 0x86,
- 0x60, 0xa4, 0x03, 0xcf, 0x17, 0xa5, 0x11, 0x2d, 0xc3, 0x94, 0xe3, 0xda, 0xa4, 0xaf, 0x5e, 0x28,
- 0x92, 0xac, 0xa4, 0x49, 0x51, 0xd5, 0x1f, 0x33, 0x30, 0x35, 0xde, 0x7b, 0xee, 0x27, 0xc3, 0xd7,
- 0x3e, 0x25, 0xe2, 0x03, 0xa6, 0xe0, 0x99, 0xf9, 0xc4, 0x13, 0x3e, 0x94, 0x8c, 0xc2, 0x39, 0xfc,
- 0x50, 0xc5, 0x46, 0x5a, 0xd0, 0x16, 0x14, 0x02, 0x97, 0xf4, 0x7b, 0x9e, 0x4f, 0x6c, 0x71, 0xbf,
- 0x64, 0xc7, 0x21, 0x7b, 0x3c, 0x37, 0xe8, 0xc4, 0xef, 0x95, 0x0d, 0xc8, 0x7b, 0xd4, 0x69, 0x3b,
- 0xae, 0xc1, 0x6b, 0xaf, 0x20, 0x92, 0xa9, 0xcd, 0x79, 0x3e, 0xe6, 0x9b, 0xab, 0x95, 0x69, 0x5e,
- 0xa5, 0xeb, 0x55, 0x0c, 0xd2, 0x85, 0xff, 0x43, 0x9f, 0xc0, 0xb4, 0x2d, 0x68, 0x52, 0x31, 0x46,
- 0x39, 0x0d, 0x9c, 0x25, 0x73, 0x62, 0xe5, 0x8d, 0xbe, 0x17, 0xae, 0x6b, 0xee, 0xb6, 0x6e, 0xe1,
- 0x36, 0xa8, 0x15, 0x47, 0x4f, 0x21, 0xe3, 0x7a, 0x17, 0x8a, 0x10, 0xc6, 0x82, 0x60, 0xee, 0xff,
- 0x6c, 0xe6, 0x57, 0xaf, 0x56, 0x26, 0xa2, 0x07, 0xf9, 0xe3, 0x5f, 0x6b, 0x37, 0x3e, 0x71, 0x0d,
- 0x3e, 0x88, 0xc9, 0x6f, 0x47, 0x2d, 0x7c, 0x6c, 0x8c, 0x7c, 0x55, 0x7a, 0x0f, 0xee, 0xc8, 0x96,
- 0x17, 0xb8, 0xde, 0xaa, 0x19, 0xad, 0x66, 0xd3, 0x68, 0xee, 0x55, 0x4b, 0x1a, 0x5a, 0x02, 0x24,
- 0x1b, 0xaa, 0xb5, 0xbd, 0x5a, 0xab, 0x66, 0xe0, 0x4a, 0x63, 0xa7, 0x56, 0x9a, 0x8c, 0xec, 0x87,
- 0x35, 0x5c, 0xaf, 0xec, 0xd5, 0xbf, 0xac, 0x6c, 0xee, 0xd5, 0x4a, 0x19, 0x74, 0x0f, 0xee, 0x4a,
- 0xfb, 0x41, 0xf3, 0xf0, 0xb0, 0xbe, 0xb9, 0x57, 0x33, 0x70, 0xed, 0x60, 0xaf, 0x72, 0x5c, 0xca,
- 0x2e, 0x67, 0x7f, 0xfe, 0xbb, 0xf2, 0xc4, 0xe3, 0x67, 0x80, 0x46, 0x77, 0x1e, 0xcd, 0x40, 0xb6,
- 0xd1, 0x6c, 0xd4, 0x4a, 0x13, 0x28, 0x0f, 0xb9, 0xcd, 0xca, 0xd6, 0x6e, 0x73, 0x7b, 0xbb, 0xa4,
- 0xa1, 0x02, 0xcc, 0xd6, 0xf7, 0xf7, 0x6b, 0xd5, 0x7a, 0xa5, 0x55, 0x2b, 0x4d, 0x6e, 0x7e, 0xf0,
- 0xfa, 0x5f, 0xe5, 0x89, 0xd7, 0xd7, 0x65, 0xed, 0xaf, 0xd7, 0x65, 0xed, 0x6f, 0xd7, 0x65, 0xed,
- 0x9f, 0xd7, 0x65, 0xed, 0x17, 0xff, 0x2e, 0x4f, 0x7c, 0x99, 0x53, 0xeb, 0xfa, 0xbf, 0x00, 0x00,
- 0x00, 0xff, 0xff, 0x75, 0x40, 0xad, 0xe7, 0xac, 0x19, 0x00, 0x00,
+ // 2366 bytes of a gzipped FileDescriptorProto
+ 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x98, 0xcd, 0x6f, 0x23, 0x49,
+ 0xf9, 0xc7, 0xe3, 0xd8, 0x89, 0x93, 0xc7, 0x71, 0xec, 0xd4, 0x64, 0xb2, 0x3d, 0x99, 0x1d, 0x3b,
+ 0x9b, 0xfd, 0xfd, 0xd8, 0xd9, 0x45, 0x24, 0xab, 0x2c, 0xb3, 0x68, 0x07, 0x38, 0x38, 0xb1, 0x93,
+ 0xf5, 0x24, 0xb1, 0x43, 0xc5, 0xd9, 0xd9, 0xec, 0x22, 0xb5, 0x3a, 0xdd, 0x15, 0xa7, 0x77, 0xec,
+ 0x6e, 0x53, 0x5d, 0x9d, 0x38, 0x17, 0xce, 0x1c, 0xe1, 0x06, 0x12, 0x12, 0x23, 0x71, 0x42, 0x5c,
+ 0x38, 0x20, 0x8e, 0x9c, 0xe7, 0xc8, 0x11, 0x21, 0x11, 0x41, 0xb8, 0x20, 0xf1, 0x1f, 0xcc, 0x09,
+ 0xd5, 0x4b, 0xbb, 0xdb, 0x71, 0xb7, 0xc7, 0xcc, 0xcd, 0x7e, 0xea, 0xa9, 0x6f, 0x3d, 0xf5, 0xf2,
+ 0x3c, 0xf5, 0xa9, 0x86, 0x65, 0xea, 0x1a, 0xe6, 0x45, 0xef, 0x6c, 0x93, 0x50, 0xea, 0x52, 0x6f,
+ 0xa3, 0x47, 0x5d, 0xe6, 0xa2, 0x25, 0xd3, 0x35, 0x5f, 0x88, 0x96, 0x0d, 0xd5, 0xbe, 0xba, 0x12,
+ 0x38, 0x76, 0x09, 0x33, 0x2c, 0x83, 0x19, 0xd2, 0x75, 0x15, 0x05, 0xf6, 0x88, 0x4d, 0xf3, 0x99,
+ 0xdd, 0xd9, 0xbc, 0xe8, 0x98, 0x9b, 0xcc, 0xee, 0x12, 0x8f, 0x19, 0xdd, 0x9e, 0x6a, 0x59, 0x6e,
+ 0xbb, 0x6d, 0x57, 0xfc, 0xdc, 0xe4, 0xbf, 0xa4, 0x75, 0xfd, 0x4f, 0xd3, 0x70, 0xaf, 0xe1, 0xb2,
+ 0x03, 0x62, 0x78, 0xe4, 0x73, 0xb7, 0x63, 0x11, 0x5a, 0xe3, 0xd1, 0xa0, 0x2a, 0x64, 0x29, 0xe9,
+ 0x75, 0x6c, 0xd3, 0xd0, 0x52, 0x6b, 0xa9, 0xc7, 0xb9, 0xad, 0xff, 0xdb, 0x18, 0x09, 0x6c, 0x03,
+ 0x4b, 0x8f, 0x2a, 0xf1, 0x4c, 0x6a, 0xf7, 0x98, 0x4b, 0xb7, 0x33, 0xaf, 0x6e, 0xca, 0x53, 0x38,
+ 0xe8, 0x8a, 0xf6, 0x60, 0xa1, 0xc3, 0x95, 0xf5, 0x0b, 0x21, 0xad, 0x4d, 0x4f, 0x2e, 0x85, 0x73,
+ 0x9d, 0x30, 0x26, 0xf4, 0x04, 0xe6, 0xa8, 0xe1, 0xb4, 0x89, 0x6e, 0x5b, 0x5a, 0x7a, 0x2d, 0xf5,
+ 0x38, 0xbd, 0xbd, 0xca, 0x47, 0xba, 0xbd, 0x29, 0x67, 0x31, 0xb7, 0xd7, 0xab, 0xaf, 0xc3, 0x9f,
+ 0x38, 0x2b, 0x7c, 0xeb, 0x16, 0xda, 0x80, 0x19, 0xa1, 0xa2, 0x65, 0xc4, 0xc0, 0x5a, 0xcc, 0xc0,
+ 0x62, 0xe6, 0x58, 0xba, 0xa1, 0xf7, 0x01, 0x4c, 0xdf, 0x63, 0x6e, 0x57, 0xef, 0x7a, 0x6d, 0x6d,
+ 0x66, 0x2d, 0xf5, 0x78, 0x5e, 0x4d, 0x69, 0x5e, 0xda, 0x0f, 0xbd, 0xf6, 0xd3, 0xcc, 0xbf, 0x5f,
+ 0x96, 0x53, 0xeb, 0xef, 0xc2, 0x72, 0xc3, 0xb5, 0xc8, 0x89, 0x63, 0x5c, 0x1a, 0x76, 0xc7, 0x38,
+ 0xeb, 0x10, 0xb1, 0x70, 0xaa, 0xb5, 0x0c, 0xef, 0x9c, 0x38, 0x9e, 0xdf, 0xeb, 0xb9, 0x94, 0x11,
+ 0x0b, 0x93, 0x9f, 0xf8, 0xc4, 0x63, 0x51, 0x87, 0x1f, 0x01, 0x12, 0xd1, 0x36, 0x5c, 0xb6, 0xeb,
+ 0xfa, 0x8e, 0x25, 0x57, 0x3d, 0x3a, 0xcd, 0xd4, 0xc4, 0xd3, 0x54, 0x92, 0xbf, 0x99, 0x86, 0xfb,
+ 0xa2, 0x69, 0x9f, 0x5c, 0x1f, 0xda, 0x5e, 0xd7, 0x60, 0xe6, 0x85, 0x94, 0xfd, 0x04, 0x96, 0xa8,
+ 0x0c, 0x41, 0xf7, 0x98, 0x41, 0x99, 0xfe, 0x82, 0x5c, 0x0b, 0xfd, 0x85, 0xed, 0xec, 0xeb, 0x9b,
+ 0x72, 0x7a, 0x9f, 0x5c, 0xe3, 0x82, 0xf2, 0x38, 0xe6, 0x0e, 0xfb, 0xe4, 0x1a, 0x6d, 0x42, 0x60,
+ 0xd2, 0x89, 0x63, 0x89, 0x2e, 0xd3, 0xc3, 0x5d, 0xf2, 0xaa, 0xbd, 0xe6, 0x58, 0xbc, 0xc3, 0x21,
+ 0x14, 0xbb, 0x6a, 0x58, 0x62, 0xe9, 0x22, 0x36, 0xb1, 0x57, 0xb9, 0xad, 0xf5, 0xb8, 0x0d, 0xe7,
+ 0xed, 0x91, 0xed, 0x2e, 0x84, 0x7d, 0x45, 0x13, 0xda, 0x87, 0x82, 0xe7, 0xb7, 0xdb, 0xc4, 0x63,
+ 0x03, 0xb5, 0xcc, 0xc4, 0x6a, 0x8b, 0x83, 0xae, 0xa2, 0x45, 0xad, 0xd0, 0x7f, 0xa6, 0x61, 0x1d,
+ 0x13, 0xc3, 0x7a, 0x6e, 0xb3, 0x0b, 0xdb, 0x39, 0x71, 0x4c, 0x42, 0x99, 0x61, 0x3b, 0xec, 0xba,
+ 0xee, 0x30, 0x42, 0x2f, 0x8d, 0x8e, 0x5c, 0xae, 0x67, 0xb0, 0x48, 0x89, 0x61, 0xe9, 0x83, 0x0c,
+ 0x52, 0x29, 0xf0, 0x28, 0x32, 0x30, 0x4f, 0xb3, 0x8d, 0x8b, 0x8e, 0xb9, 0xd1, 0x0a, 0x9c, 0xd4,
+ 0x41, 0xc9, 0xf3, 0xae, 0x03, 0x23, 0xc2, 0x80, 0x48, 0xdf, 0xf6, 0x98, 0xed, 0xb4, 0x23, 0x7a,
+ 0xd3, 0x93, 0xeb, 0x2d, 0x05, 0xdd, 0x43, 0xcd, 0x6d, 0xc8, 0x77, 0x8d, 0x7e, 0x44, 0x2e, 0x3d,
+ 0x81, 0x1c, 0x5e, 0xe8, 0x1a, 0xfd, 0x50, 0xe3, 0x6b, 0xb8, 0xe7, 0x9e, 0x79, 0x84, 0x5e, 0x92,
+ 0xc8, 0x3c, 0x3d, 0x2d, 0xb3, 0x96, 0x4e, 0x48, 0xd0, 0xa6, 0xf2, 0xbe, 0x1b, 0x1f, 0x72, 0xef,
+ 0x36, 0x78, 0x61, 0x0e, 0xb4, 0xa8, 0xe1, 0x78, 0x86, 0xc9, 0x6c, 0xd7, 0xa9, 0x9c, 0x89, 0x54,
+ 0x88, 0xe6, 0x80, 0x01, 0xcb, 0x11, 0x87, 0x23, 0xdf, 0x53, 0xc7, 0x75, 0x07, 0xa0, 0xe7, 0x7b,
+ 0x17, 0x84, 0xe8, 0xac, 0xef, 0xa8, 0xb5, 0x2f, 0xc5, 0x84, 0x14, 0xe9, 0x1c, 0x64, 0xa9, 0xec,
+ 0xd7, 0xea, 0x3b, 0x6a, 0x88, 0x73, 0xb8, 0x1f, 0xf1, 0xc2, 0x84, 0xd1, 0x6b, 0x39, 0xc6, 0x1e,
+ 0xcc, 0x52, 0x62, 0x78, 0xae, 0xd4, 0x5f, 0xdc, 0xfa, 0x70, 0xbc, 0xbe, 0xe8, 0x89, 0x45, 0x07,
+ 0x35, 0x94, 0xea, 0xae, 0xc6, 0x29, 0xc1, 0xca, 0x90, 0x77, 0xaf, 0x63, 0x5c, 0x47, 0xa7, 0xfa,
+ 0xe7, 0xd4, 0x90, 0xc3, 0x31, 0x33, 0x98, 0xef, 0xc9, 0x48, 0x56, 0x20, 0xcd, 0x8b, 0x4d, 0x2a,
+ 0x52, 0x6c, 0xb8, 0x01, 0x35, 0x06, 0x11, 0x4e, 0x8b, 0x08, 0x3f, 0x1e, 0x1f, 0x61, 0x44, 0x72,
+ 0x23, 0x2e, 0xd0, 0xf5, 0x4f, 0x61, 0x56, 0xda, 0x11, 0x82, 0x45, 0x5c, 0xab, 0x1c, 0x37, 0x1b,
+ 0xfa, 0x49, 0x63, 0xbf, 0xd1, 0x7c, 0xde, 0x28, 0x4e, 0x21, 0x0d, 0x96, 0x95, 0xad, 0xf5, 0x65,
+ 0x43, 0x6f, 0x34, 0x5b, 0xfa, 0x6e, 0xf3, 0xa4, 0x51, 0x2d, 0xa6, 0xd4, 0x04, 0x4e, 0xa1, 0xf8,
+ 0x9c, 0xda, 0x8c, 0xf0, 0x4c, 0x71, 0x64, 0x25, 0x43, 0x9f, 0x41, 0xd6, 0x16, 0x7f, 0x3d, 0x2d,
+ 0x25, 0xce, 0xcd, 0x83, 0x98, 0x10, 0x65, 0x87, 0xe0, 0x62, 0x50, 0xfe, 0x52, 0xf4, 0x59, 0x66,
+ 0x6e, 0xba, 0x98, 0x5e, 0xff, 0x7d, 0x4a, 0x69, 0xb7, 0x5c, 0xb7, 0xd9, 0x51, 0x95, 0xb0, 0x02,
+ 0xf3, 0x6f, 0x95, 0x7e, 0x61, 0x2f, 0xd4, 0x80, 0xa2, 0x61, 0x32, 0xdf, 0xe8, 0xbc, 0x5d, 0xe2,
+ 0x15, 0x64, 0xe7, 0x81, 0x59, 0x2d, 0xc4, 0x2a, 0xa0, 0x66, 0x8f, 0x17, 0x74, 0x9b, 0x12, 0xaf,
+ 0xd5, 0x77, 0xa2, 0xbb, 0x7c, 0x0a, 0xcb, 0x3b, 0xae, 0x63, 0xd9, 0x7c, 0x3f, 0x76, 0x0d, 0xbb,
+ 0xa3, 0x8e, 0x3b, 0xfa, 0x3e, 0x2c, 0xa8, 0x48, 0x2e, 0x8d, 0x8e, 0x4f, 0xd4, 0x7c, 0xe2, 0x6e,
+ 0xa3, 0x2f, 0x78, 0x3b, 0xce, 0x49, 0x6f, 0xf1, 0x47, 0x49, 0xff, 0x31, 0x05, 0x48, 0x5e, 0x55,
+ 0xe4, 0x1b, 0x62, 0x06, 0x89, 0x84, 0x4a, 0x90, 0xed, 0x12, 0xcf, 0x33, 0xda, 0x64, 0xe8, 0x00,
+ 0x05, 0x46, 0xf4, 0x03, 0x98, 0x57, 0x45, 0x9a, 0x58, 0x6a, 0xf2, 0x89, 0x97, 0x60, 0xb0, 0x82,
+ 0x83, 0x0e, 0xe8, 0x29, 0xcc, 0x05, 0xd5, 0x47, 0xd5, 0x98, 0x37, 0x75, 0x1e, 0xf8, 0xab, 0xb0,
+ 0xbf, 0x07, 0xf3, 0xc7, 0xc4, 0x99, 0x2c, 0xd8, 0xa1, 0x43, 0x71, 0x05, 0xcb, 0x95, 0xee, 0x99,
+ 0xdd, 0xf6, 0x5d, 0xdf, 0xc3, 0xc4, 0xf3, 0x3b, 0x6c, 0xb2, 0x09, 0x7f, 0x06, 0xb9, 0x2b, 0x6a,
+ 0xf4, 0x7a, 0xc4, 0xd2, 0x09, 0xa5, 0x63, 0xa6, 0x2c, 0xe4, 0x30, 0x28, 0xe7, 0x1a, 0x0d, 0xf6,
+ 0xf0, 0x11, 0xbf, 0x44, 0xcf, 0xd9, 0x1e, 0x75, 0xfd, 0x5e, 0x95, 0x74, 0xc8, 0x9d, 0x9a, 0x45,
+ 0x60, 0x45, 0xa1, 0xca, 0x8e, 0x4b, 0xa9, 0xdf, 0xe3, 0x5b, 0x2d, 0x23, 0x7b, 0x0f, 0xe6, 0x05,
+ 0xc8, 0xe9, 0x77, 0xb3, 0x79, 0x4e, 0x98, 0x0f, 0xbd, 0x36, 0x5a, 0x87, 0xf9, 0x1e, 0x75, 0x4d,
+ 0xe2, 0x79, 0x6a, 0x37, 0xe6, 0x06, 0x75, 0x2b, 0x30, 0x0f, 0x4e, 0x12, 0x52, 0xc3, 0x44, 0x93,
+ 0xe2, 0x87, 0x00, 0x8a, 0xac, 0x02, 0x40, 0x98, 0xd9, 0x2e, 0x29, 0x40, 0x98, 0x57, 0xfe, 0x02,
+ 0x11, 0xc2, 0x3f, 0x7c, 0x3b, 0xe5, 0x4f, 0x2b, 0x24, 0x8f, 0x63, 0xe6, 0xd2, 0x51, 0xf2, 0xf0,
+ 0xb8, 0x35, 0x86, 0x3c, 0x84, 0xb7, 0x24, 0x0f, 0xf5, 0x13, 0x67, 0x85, 0xef, 0x40, 0xf2, 0x90,
+ 0xd3, 0xce, 0x85, 0xe1, 0x58, 0x1d, 0xce, 0x3a, 0x8c, 0x5e, 0x0f, 0x70, 0x08, 0x6d, 0x41, 0xa6,
+ 0x57, 0xa3, 0x74, 0xcc, 0x91, 0x17, 0x7e, 0x6a, 0x1d, 0x84, 0xef, 0xfa, 0xdf, 0x53, 0xa0, 0x7d,
+ 0x7e, 0x47, 0x2d, 0xc8, 0xb4, 0xc4, 0x72, 0xf9, 0x35, 0xcc, 0xb2, 0xbe, 0xc3, 0xc3, 0x97, 0x94,
+ 0x52, 0xe5, 0x4d, 0x7f, 0xbb, 0x29, 0x7f, 0xd2, 0xb6, 0xd9, 0x85, 0x7f, 0xb6, 0x61, 0xba, 0xdd,
+ 0xcd, 0xc1, 0xe0, 0xd6, 0x59, 0xf8, 0x7b, 0xb3, 0xf7, 0xa2, 0xbd, 0x29, 0xa8, 0xd9, 0xf7, 0x6d,
+ 0x6b, 0xe3, 0xe4, 0xa4, 0x5e, 0xbd, 0xbd, 0x29, 0xcf, 0xb4, 0xfa, 0x4e, 0xbd, 0x8a, 0x67, 0x58,
+ 0xdf, 0xa9, 0x5b, 0x68, 0x17, 0x72, 0x2c, 0x2c, 0xb5, 0x2a, 0x17, 0x26, 0xbb, 0x92, 0xa2, 0x1d,
+ 0xd5, 0x72, 0x3d, 0x84, 0x7b, 0xad, 0xbe, 0x73, 0x44, 0xc9, 0x65, 0x85, 0x31, 0xd2, 0xed, 0x0d,
+ 0x81, 0xe1, 0x07, 0x50, 0x6e, 0xf5, 0x9d, 0x4a, 0x87, 0x63, 0xc4, 0x75, 0xcd, 0x31, 0x5d, 0x9f,
+ 0xb3, 0x89, 0x3a, 0x84, 0x51, 0xc7, 0x5f, 0xa4, 0x60, 0x99, 0x17, 0xd7, 0x36, 0xa1, 0xcd, 0x4b,
+ 0x42, 0xcf, 0x3b, 0xee, 0x95, 0x5c, 0xa1, 0x07, 0x90, 0x8e, 0xe1, 0x3b, 0x6e, 0x43, 0x1f, 0x42,
+ 0xde, 0xf4, 0x29, 0x25, 0x0e, 0x53, 0x95, 0x68, 0x5a, 0x6c, 0xb5, 0x8c, 0x74, 0x41, 0x35, 0x89,
+ 0xb2, 0x83, 0xbe, 0x03, 0x05, 0xdb, 0x31, 0x29, 0xe9, 0x86, 0xce, 0xe9, 0x88, 0xf3, 0xe2, 0xa0,
+ 0x31, 0x5a, 0xa5, 0x0e, 0x61, 0xe9, 0xd0, 0xee, 0x13, 0xeb, 0xd8, 0x37, 0xf9, 0x71, 0x0e, 0x8e,
+ 0x40, 0x56, 0x65, 0xd9, 0x9b, 0x4e, 0x01, 0x0e, 0x1c, 0x95, 0xdc, 0xef, 0x52, 0xf0, 0x70, 0x9b,
+ 0x33, 0x61, 0x58, 0x9b, 0xc9, 0xb9, 0x4b, 0xc9, 0xde, 0xce, 0xe0, 0x92, 0x68, 0xbd, 0xd5, 0x25,
+ 0x11, 0x72, 0x10, 0x97, 0xb8, 0xa0, 0xc4, 0xe3, 0x0f, 0x94, 0xff, 0xe5, 0x76, 0x08, 0x7b, 0xa9,
+ 0x58, 0x7f, 0x75, 0x0f, 0x72, 0x22, 0xaa, 0x2a, 0x61, 0x86, 0xdd, 0x41, 0x47, 0x50, 0x74, 0x5c,
+ 0xa6, 0x0f, 0x3d, 0x7f, 0x64, 0x88, 0xdf, 0x8a, 0x99, 0x7e, 0xcc, 0x13, 0x0c, 0x2f, 0x3a, 0x43,
+ 0x46, 0x74, 0x08, 0x05, 0xf9, 0x38, 0xe0, 0xba, 0xe7, 0x3c, 0x75, 0x55, 0xc0, 0xff, 0x9f, 0x04,
+ 0xc4, 0x43, 0x29, 0x8e, 0xf3, 0x34, 0x6a, 0x43, 0x5f, 0x00, 0x92, 0x72, 0x2f, 0xc8, 0xb5, 0x1e,
+ 0xc0, 0xb7, 0x3a, 0xda, 0x8f, 0x93, 0x14, 0xef, 0x3e, 0x2d, 0x70, 0x91, 0xde, 0x31, 0xa3, 0x9f,
+ 0xc2, 0x9a, 0xa0, 0xe7, 0x2b, 0x01, 0xd9, 0xba, 0x1f, 0x52, 0xb6, 0x6e, 0x2b, 0xcc, 0x56, 0x20,
+ 0xff, 0x24, 0xf6, 0x1d, 0xf8, 0x26, 0x3c, 0xc7, 0x8f, 0xe8, 0x38, 0x1f, 0x4e, 0xb6, 0x91, 0x94,
+ 0xd3, 0x0d, 0xc9, 0x9d, 0xe2, 0x31, 0x97, 0xdb, 0xfa, 0x68, 0x7c, 0xce, 0x46, 0x21, 0x15, 0x23,
+ 0x36, 0xd2, 0x80, 0x30, 0x14, 0xa3, 0xe2, 0x1c, 0x37, 0xb5, 0x59, 0xa1, 0xfc, 0xc1, 0x78, 0xe5,
+ 0x01, 0xdd, 0xe2, 0x02, 0x1b, 0xb6, 0xa2, 0x13, 0x58, 0x8a, 0x6a, 0x52, 0x5e, 0xf2, 0xb4, 0x6c,
+ 0xe2, 0x3e, 0xc4, 0xf2, 0x2c, 0x8e, 0x86, 0x25, 0xcc, 0xe8, 0x4b, 0x88, 0x4e, 0x80, 0x3f, 0xfc,
+ 0x98, 0xef, 0x69, 0x73, 0x42, 0xf7, 0xc3, 0x89, 0x59, 0x12, 0x47, 0x63, 0x93, 0x76, 0xb4, 0x0b,
+ 0x0b, 0x57, 0x9c, 0xd7, 0x74, 0x49, 0x73, 0xda, 0xbc, 0xd0, 0x7c, 0x3f, 0x46, 0xf3, 0x2e, 0x32,
+ 0xe2, 0xdc, 0x55, 0x68, 0x41, 0x7b, 0x90, 0x97, 0x3a, 0xcc, 0x75, 0x75, 0x9e, 0x7f, 0x30, 0x5e,
+ 0x28, 0x72, 0x15, 0x2a, 0x21, 0x69, 0xe1, 0x99, 0xe1, 0xf6, 0x74, 0xaa, 0xa0, 0x4c, 0xbc, 0x1a,
+ 0x72, 0x89, 0x99, 0x31, 0x4a, 0x6f, 0x38, 0xef, 0x46, 0x6d, 0x7c, 0x93, 0xcd, 0x00, 0xe3, 0xf4,
+ 0x73, 0xc1, 0x71, 0xda, 0x42, 0xe2, 0x26, 0xc7, 0x11, 0x1f, 0x2e, 0x98, 0xc3, 0x56, 0x74, 0x00,
+ 0x8b, 0xb2, 0x14, 0x50, 0xc5, 0x6f, 0x5a, 0x3e, 0x31, 0xc2, 0x51, 0xce, 0xc3, 0xf9, 0x4e, 0xd4,
+ 0xc6, 0x23, 0x74, 0x5c, 0x8b, 0xe8, 0x7e, 0xf8, 0xf5, 0x41, 0x5b, 0x4c, 0x8c, 0x30, 0xee, 0x3b,
+ 0x05, 0x2e, 0x38, 0xc3, 0x56, 0xf4, 0x31, 0x64, 0x3c, 0xe2, 0x58, 0x5a, 0x41, 0xe8, 0xbc, 0x1b,
+ 0xa3, 0x33, 0x20, 0x39, 0x2c, 0x3c, 0x65, 0x05, 0x39, 0x67, 0x7a, 0x9b, 0xb3, 0x92, 0x6e, 0x49,
+ 0x58, 0xd2, 0x8a, 0x63, 0x2a, 0x48, 0x0c, 0x57, 0xf1, 0x0a, 0x32, 0x6c, 0xe6, 0x27, 0x37, 0xc0,
+ 0x1c, 0x73, 0x00, 0x59, 0xda, 0x52, 0xe2, 0xc9, 0x8d, 0x07, 0x32, 0xbc, 0x44, 0xef, 0xda, 0x45,
+ 0x09, 0x55, 0xca, 0xc1, 0x99, 0x43, 0xc9, 0x25, 0x74, 0x04, 0xc0, 0xf8, 0xe3, 0x3e, 0x62, 0xbb,
+ 0x9b, 0x62, 0x54, 0x3c, 0xfb, 0xb4, 0x95, 0x49, 0x52, 0x2c, 0xf2, 0x44, 0x1c, 0x4a, 0x31, 0x69,
+ 0xe7, 0x1b, 0x6c, 0x04, 0xf8, 0xab, 0x53, 0xc1, 0xbf, 0xda, 0x6a, 0xe2, 0x06, 0xc7, 0x91, 0x32,
+ 0x2e, 0x18, 0xc3, 0x56, 0x3e, 0x79, 0x89, 0x78, 0xe1, 0xfd, 0xf1, 0x30, 0x71, 0xf2, 0xa3, 0x88,
+ 0x88, 0xf3, 0x5e, 0xd4, 0x86, 0xbe, 0x81, 0x87, 0x0a, 0xf9, 0x64, 0xc9, 0xe2, 0x87, 0x88, 0xa7,
+ 0x9e, 0x2e, 0x70, 0x57, 0x7b, 0x57, 0x48, 0x7f, 0x3b, 0x46, 0x3a, 0x09, 0xed, 0xb0, 0x76, 0x91,
+ 0x04, 0x7d, 0xa7, 0x70, 0x9f, 0x2b, 0xab, 0x5a, 0xae, 0x1b, 0xde, 0xb5, 0x63, 0x0a, 0xbe, 0x2f,
+ 0x25, 0xde, 0xa8, 0x31, 0x84, 0x85, 0x11, 0xeb, 0x07, 0x95, 0xbc, 0xc2, 0x25, 0x6a, 0x94, 0xf2,
+ 0x95, 0xb6, 0x25, 0x45, 0xe9, 0xae, 0xc2, 0x28, 0xad, 0x9c, 0xb8, 0xd2, 0x71, 0xc0, 0x85, 0x0b,
+ 0xf6, 0xb0, 0x95, 0x5f, 0x41, 0x7e, 0xf8, 0xf5, 0x4f, 0x57, 0x0f, 0x2a, 0x6d, 0x2d, 0xf1, 0x0a,
+ 0x4a, 0xf8, 0x56, 0x88, 0x91, 0x3f, 0xd2, 0x80, 0xea, 0x90, 0xef, 0x72, 0xc6, 0xd2, 0x3d, 0x09,
+ 0x59, 0xda, 0x7b, 0x89, 0x1f, 0x55, 0x47, 0x58, 0x0c, 0x2f, 0x74, 0x23, 0x26, 0x74, 0x0a, 0xc5,
+ 0xc1, 0xd3, 0x58, 0x3f, 0x13, 0x68, 0xa5, 0xad, 0x0b, 0xb5, 0x8d, 0x18, 0xb5, 0x31, 0x24, 0x86,
+ 0x0b, 0x6c, 0xd8, 0x8e, 0x7c, 0x78, 0x24, 0x76, 0x4c, 0x72, 0xac, 0x4e, 0x42, 0x90, 0x55, 0xe7,
+ 0xe3, 0x7d, 0x31, 0xce, 0x56, 0xfc, 0xce, 0x8d, 0xc3, 0x5f, 0xbc, 0xca, 0x12, 0x1d, 0x9e, 0xce,
+ 0xbd, 0x7a, 0x59, 0x4e, 0xa9, 0x67, 0xe4, 0x83, 0xe2, 0xea, 0xb3, 0xcc, 0xdc, 0xa3, 0x62, 0x69,
+ 0x7d, 0x53, 0xa0, 0xd9, 0x91, 0xeb, 0x89, 0x8a, 0x8c, 0x56, 0x61, 0xc6, 0x76, 0x2c, 0xd2, 0x57,
+ 0x2f, 0x28, 0x09, 0x74, 0xd2, 0xa4, 0x60, 0xee, 0x0f, 0x69, 0x98, 0x99, 0xec, 0xbd, 0xf9, 0xe3,
+ 0x61, 0xda, 0xa0, 0x44, 0x7c, 0x60, 0x15, 0x18, 0xb5, 0x18, 0x9b, 0x58, 0x43, 0x35, 0x40, 0x38,
+ 0x07, 0x1f, 0xd2, 0xd8, 0x48, 0x0b, 0xda, 0x81, 0xbc, 0xef, 0x90, 0x7e, 0xcf, 0xf5, 0x88, 0x25,
+ 0xae, 0xb5, 0xcc, 0x24, 0x2f, 0x0f, 0xbc, 0x30, 0xe8, 0xc4, 0xaf, 0xb3, 0x4d, 0xc8, 0xb9, 0xd4,
+ 0x6e, 0xdb, 0x8e, 0xce, 0x4b, 0xbe, 0x00, 0xa1, 0x99, 0xed, 0x45, 0x3e, 0xe6, 0xeb, 0x9b, 0xf2,
+ 0x2c, 0xbf, 0x1c, 0xea, 0x55, 0x0c, 0xd2, 0x85, 0xff, 0x43, 0x9f, 0xc2, 0xac, 0x25, 0x20, 0x56,
+ 0xa1, 0x4d, 0x29, 0x89, 0xd7, 0x25, 0xea, 0x62, 0xe5, 0x8d, 0xbe, 0x1b, 0xac, 0x6b, 0x76, 0x5c,
+ 0xb7, 0x60, 0x1b, 0xd4, 0x8a, 0xa3, 0x27, 0x90, 0x76, 0xdc, 0x2b, 0x05, 0x26, 0x13, 0xb1, 0x37,
+ 0xf7, 0x7f, 0x3a, 0xf7, 0xcb, 0x97, 0xe5, 0xa9, 0xf0, 0x83, 0xc1, 0x47, 0xbf, 0x4e, 0xdd, 0xf9,
+ 0x04, 0x37, 0xf8, 0x60, 0x27, 0xbf, 0x6d, 0xb5, 0xf0, 0xa9, 0x3e, 0xf2, 0xd5, 0xeb, 0x1d, 0xb8,
+ 0x27, 0x5b, 0x9e, 0xe3, 0x7a, 0xab, 0xa6, 0xb7, 0x9a, 0x4d, 0xbd, 0x79, 0x50, 0x2d, 0xa6, 0xd0,
+ 0x0a, 0x20, 0xd9, 0x50, 0xad, 0x1d, 0xd4, 0x5a, 0x35, 0x1d, 0x57, 0x1a, 0x7b, 0xb5, 0xe2, 0x74,
+ 0x68, 0x3f, 0xae, 0xe1, 0x7a, 0xe5, 0xa0, 0xfe, 0x55, 0x65, 0xfb, 0xa0, 0x56, 0x4c, 0xa3, 0x07,
+ 0x70, 0x5f, 0xda, 0x8f, 0x9a, 0xc7, 0xc7, 0xf5, 0xed, 0x83, 0x9a, 0x8e, 0x6b, 0x47, 0x07, 0x95,
+ 0xd3, 0x62, 0x66, 0x35, 0xf3, 0xb3, 0xdf, 0x96, 0xa6, 0x3e, 0x7a, 0x0a, 0x68, 0x74, 0xe7, 0xd1,
+ 0x1c, 0x64, 0x1a, 0xcd, 0x46, 0xad, 0x38, 0x85, 0x72, 0x90, 0xdd, 0xae, 0xec, 0xec, 0x37, 0x77,
+ 0x77, 0x8b, 0x29, 0x94, 0x87, 0xf9, 0xfa, 0xe1, 0x61, 0xad, 0x5a, 0xaf, 0xb4, 0x6a, 0xc5, 0xe9,
+ 0xed, 0xf7, 0x5e, 0xfd, 0xb3, 0x34, 0xf5, 0xea, 0xb6, 0x94, 0xfa, 0xcb, 0x6d, 0x29, 0xf5, 0xd7,
+ 0xdb, 0x52, 0xea, 0x1f, 0xb7, 0xa5, 0xd4, 0xcf, 0xff, 0x55, 0x9a, 0xfa, 0x2a, 0xab, 0xd6, 0xf5,
+ 0xbf, 0x01, 0x00, 0x00, 0xff, 0xff, 0x85, 0xdd, 0x5b, 0xa8, 0x4c, 0x1a, 0x00, 0x00,
}
diff --git a/pkg/roachpb/errors.proto b/pkg/roachpb/errors.proto
index 8dff18b9bca1..41267c98ced1 100644
--- a/pkg/roachpb/errors.proto
+++ b/pkg/roachpb/errors.proto
@@ -346,6 +346,13 @@ message TxnPrevAttemptError {
option (gogoproto.equal) = true;
}
+// TxnAlreadyEncounteredErrorError indicates that an operation tried to use a
+// transaction that already received an error from a previous request. Once that
+// happens, client.Txn rejects future requests.
+message TxnAlreadyEncounteredErrorError{
+ option (gogoproto.equal) = true;
+}
+
// An IntegerOverflowError indicates that an operation was aborted because
// it would have caused an integeter overflow.
message IntegerOverflowError {
@@ -413,6 +420,7 @@ message ErrorDetail {
optional UnsupportedRequestError unsupported_request = 32;
optional MixedSuccessError mixed_success = 33;
optional BatchTimestampBeforeGCError timestamp_before = 34;
+ optional TxnAlreadyEncounteredErrorError txn_already_encountered_error = 35;
}
// TransactionRestart indicates how an error should be handled in a
diff --git a/pkg/roachpb/string_test.go b/pkg/roachpb/string_test.go
index 58ed25b3be8a..e2d35243d8ef 100644
--- a/pkg/roachpb/string_test.go
+++ b/pkg/roachpb/string_test.go
@@ -72,7 +72,7 @@ func TestBatchRequestString(t *testing.T) {
}
br.Requests = append(br.Requests, roachpb.RequestUnion{EndTransaction: &roachpb.EndTransactionRequest{}})
- e := fmt.Sprintf(`[txn: %s], Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), ... 76 skipped ..., Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), EndTransaction [/Min,/Min)`,
+ e := fmt.Sprintf(`[txn: %s], Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), ... 76 skipped ..., Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), EndTransaction(commit:false) [/Min]`,
br.Txn.Short())
if e != br.String() {
t.Fatalf("e = %s\nv = %s", e, br.String())
diff --git a/pkg/server/updates_test.go b/pkg/server/updates_test.go
index 2b85d0199fe4..a32eb13ce3fb 100644
--- a/pkg/server/updates_test.go
+++ b/pkg/server/updates_test.go
@@ -419,7 +419,7 @@ func TestReportUsage(t *testing.T) {
"diagnostics.reporting.send_crash_reports": "false",
"server.time_until_store_dead": "1m30s",
"trace.debug.enable": "false",
- "version": "2.0-5",
+ "version": "2.0-6",
"cluster.secret": "",
} {
if got, ok := r.last.AlteredSettings[key]; !ok {
diff --git a/pkg/settings/cluster/cockroach_versions.go b/pkg/settings/cluster/cockroach_versions.go
index 7e393977ca72..81b7fd226e3b 100644
--- a/pkg/settings/cluster/cockroach_versions.go
+++ b/pkg/settings/cluster/cockroach_versions.go
@@ -53,6 +53,7 @@ const (
VersionRangeAppliedStateKey
VersionImportFormats
VersionSecondaryLookupJoins
+ VersionClientSideWritingFlag
// Add new versions here (step one of two).
@@ -218,6 +219,13 @@ var versionsSingleton = keyedVersions([]keyedVersion{
Key: VersionSecondaryLookupJoins,
Version: roachpb.Version{Major: 2, Minor: 0, Unstable: 5},
},
+ {
+ // VersionClientsideWritingFlag is https://github.com/cockroachdb/cockroach/pull/25541.
+ // Before this version, the Transaction.Writing flag used to be set by the
+ // server. After, it is set only by the client.
+ Key: VersionClientSideWritingFlag,
+ Version: roachpb.Version{Major: 2, Minor: 0, Unstable: 6},
+ },
// Add new versions here (step two of two).
diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go
index b7e17d9da8e3..4b78f86fe53f 100644
--- a/pkg/sql/conn_executor.go
+++ b/pkg/sql/conn_executor.go
@@ -1561,6 +1561,9 @@ func (ex *connExecutor) synchronizeParallelStmts(ctx context.Context) error {
return 1
}
return 2
+ case *roachpb.TxnAlreadyEncounteredErrorError:
+ // Another parallel stmt got an error that caused this one.
+ return 5
case *roachpb.TxnPrevAttemptError:
// Symptom of concurrent retry.
return 3
diff --git a/pkg/sql/distsql_running.go b/pkg/sql/distsql_running.go
index 7b4f10dbc139..340b91982ee9 100644
--- a/pkg/sql/distsql_running.go
+++ b/pkg/sql/distsql_running.go
@@ -376,7 +376,7 @@ func (r *distSQLReceiver) Push(
if meta.TxnMeta != nil {
if r.txn != nil {
if r.txn.ID() == meta.TxnMeta.Txn.ID {
- r.txn.AugmentTxnCoordMeta(*meta.TxnMeta)
+ r.txn.AugmentTxnCoordMeta(r.ctx, *meta.TxnMeta)
}
} else {
r.resultWriter.SetError(
diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal b/pkg/sql/logictest/testdata/logic_test/crdb_internal
index e1a65b8ff040..5892d284af98 100644
--- a/pkg/sql/logictest/testdata/logic_test/crdb_internal
+++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal
@@ -244,7 +244,7 @@ select crdb_internal.set_vmodule('')
query T
select crdb_internal.node_executable_version()
----
-2.0-5
+2.0-6
query ITTT colnames
select node_id, component, field, regexp_replace(regexp_replace(value, '^\d+$', ''), e':\\d+', ':') as value from crdb_internal.node_runtime_info
@@ -332,4 +332,4 @@ select * from crdb_internal.gossip_alerts
query T
select crdb_internal.node_executable_version()
----
-2.0-5
+2.0-6
diff --git a/pkg/sql/logictest/testdata/logic_test/parallel_stmts b/pkg/sql/logictest/testdata/logic_test/parallel_stmts
index 9c842a182967..00ae6796bd38 100644
--- a/pkg/sql/logictest/testdata/logic_test/parallel_stmts
+++ b/pkg/sql/logictest/testdata/logic_test/parallel_stmts
@@ -359,17 +359,6 @@ SHOW TIME ZONE
----
UTC
-query TT
-SHOW CREATE TABLE kv
-----
-kv CREATE TABLE kv (
- k INT NOT NULL,
- v INT NULL,
- CONSTRAINT "primary" PRIMARY KEY (k ASC),
- FAMILY "primary" (k, v),
- CONSTRAINT check_v CHECK (v < 100)
- )
-
query T
SHOW TRANSACTION STATUS
----
diff --git a/pkg/sql/logictest/testdata/logic_test/show_trace b/pkg/sql/logictest/testdata/logic_test/show_trace
index b548d89e9a2d..c57f5a814572 100644
--- a/pkg/sql/logictest/testdata/logic_test/show_trace
+++ b/pkg/sql/logictest/testdata/logic_test/show_trace
@@ -226,6 +226,8 @@ SELECT operation, message FROM [SHOW KV TRACE FOR SESSION]
sql txn CPut /Table/54/1/1/0 -> /TUPLE/2:2:Int/2
dist sender querying next range at /Table/54/1/1/0
dist sender r1: sending batch 1 CPut, 1 BeginTxn, 1 EndTxn, 1 InitPut to (n1,s1):1
+dist sender querying next range at /Table/54/1/1/0
+dist sender r1: sending batch 1 EndTxn to (n1,s1):1
statement error duplicate key value
set tracing=kv;
@@ -238,6 +240,8 @@ SELECT operation, message FROM [SHOW KV TRACE FOR SESSION]
sql txn CPut /Table/54/1/2/0 -> /TUPLE/2:2:Int/2
dist sender querying next range at /Table/54/1/2/0
dist sender r1: sending batch 1 CPut, 1 BeginTxn, 1 EndTxn, 1 InitPut to (n1,s1):1
+dist sender querying next range at /Table/54/1/2/0
+dist sender r1: sending batch 1 EndTxn to (n1,s1):1
query TT
SET tracing=kv;
@@ -283,6 +287,8 @@ sql txn Del /Table/54/2/3/0
sql txn CPut /Table/54/2/2/0 -> /BYTES/Š
dist sender querying next range at /Table/54/1/2/0
dist sender r1: sending batch 1 Put, 1 CPut, 1 Del, 1 BeginTxn, 1 EndTxn to (n1,s1):1
+dist sender querying next range at /Table/54/1/2/0
+dist sender r1: sending batch 1 EndTxn to (n1,s1):1
query TT
SET tracing=kv;
diff --git a/pkg/sql/sem/tree/stmt.go b/pkg/sql/sem/tree/stmt.go
index 98a9ea598903..f991ad06952e 100644
--- a/pkg/sql/sem/tree/stmt.go
+++ b/pkg/sql/sem/tree/stmt.go
@@ -134,6 +134,9 @@ type HiddenFromShowQueries interface {
// IndependentFromParallelizedPriors is a pseudo-interface to be implemented
// by statements which do not force parallel statement execution synchronization
// when they run.
+// NB: Only statements that don't send any requests using the current
+// transaction can implement this. Otherwise, the statement will fail if any of
+// the parallel statements has encoutered a KV error (which toasts the txn).
type IndependentFromParallelizedPriors interface {
independentFromParallelizedPriors()
}
@@ -639,8 +642,7 @@ func (*ShowColumns) StatementType() StatementType { return Rows }
// StatementTag returns a short string identifying the type of statement.
func (*ShowColumns) StatementTag() string { return "SHOW COLUMNS" }
-func (*ShowColumns) hiddenFromStats() {}
-func (*ShowColumns) independentFromParallelizedPriors() {}
+func (*ShowColumns) hiddenFromStats() {}
// StatementType implements the Statement interface.
func (*ShowCreateTable) StatementType() StatementType { return Rows }
@@ -648,8 +650,7 @@ func (*ShowCreateTable) StatementType() StatementType { return Rows }
// StatementTag returns a short string identifying the type of statement.
func (*ShowCreateTable) StatementTag() string { return "SHOW CREATE TABLE" }
-func (*ShowCreateTable) hiddenFromStats() {}
-func (*ShowCreateTable) independentFromParallelizedPriors() {}
+func (*ShowCreateTable) hiddenFromStats() {}
// StatementType implements the Statement interface.
func (*ShowCreateView) StatementType() StatementType { return Rows }
@@ -657,8 +658,7 @@ func (*ShowCreateView) StatementType() StatementType { return Rows }
// StatementTag returns a short string identifying the type of statement.
func (*ShowCreateView) StatementTag() string { return "SHOW CREATE VIEW" }
-func (*ShowCreateView) hiddenFromStats() {}
-func (*ShowCreateView) independentFromParallelizedPriors() {}
+func (*ShowCreateView) hiddenFromStats() {}
// StatementType implements the Statement interface.
func (*ShowCreateSequence) StatementType() StatementType { return Rows }
@@ -666,8 +666,7 @@ func (*ShowCreateSequence) StatementType() StatementType { return Rows }
// StatementTag implements the Statement interface.
func (*ShowCreateSequence) StatementTag() string { return "SHOW CREATE SEQUENCE" }
-func (*ShowCreateSequence) hiddenFromStats() {}
-func (*ShowCreateSequence) independentFromParallelizedPriors() {}
+func (*ShowCreateSequence) hiddenFromStats() {}
// StatementType implements the Statement interface.
func (*ShowBackup) StatementType() StatementType { return Rows }
@@ -675,8 +674,7 @@ func (*ShowBackup) StatementType() StatementType { return Rows }
// StatementTag returns a short string identifying the type of statement.
func (*ShowBackup) StatementTag() string { return "SHOW BACKUP" }
-func (*ShowBackup) hiddenFromStats() {}
-func (*ShowBackup) independentFromParallelizedPriors() {}
+func (*ShowBackup) hiddenFromStats() {}
// StatementType implements the Statement interface.
func (*ShowDatabases) StatementType() StatementType { return Rows }
@@ -684,8 +682,7 @@ func (*ShowDatabases) StatementType() StatementType { return Rows }
// StatementTag returns a short string identifying the type of statement.
func (*ShowDatabases) StatementTag() string { return "SHOW DATABASES" }
-func (*ShowDatabases) hiddenFromStats() {}
-func (*ShowDatabases) independentFromParallelizedPriors() {}
+func (*ShowDatabases) hiddenFromStats() {}
// StatementType implements the Statement interface.
func (*ShowTrace) StatementType() StatementType { return Rows }
@@ -701,8 +698,7 @@ func (*ShowGrants) StatementType() StatementType { return Rows }
// StatementTag returns a short string identifying the type of statement.
func (*ShowGrants) StatementTag() string { return "SHOW GRANTS" }
-func (*ShowGrants) hiddenFromStats() {}
-func (*ShowGrants) independentFromParallelizedPriors() {}
+func (*ShowGrants) hiddenFromStats() {}
// StatementType implements the Statement interface.
func (*ShowIndex) StatementType() StatementType { return Rows }
@@ -710,8 +706,7 @@ func (*ShowIndex) StatementType() StatementType { return Rows }
// StatementTag returns a short string identifying the type of statement.
func (*ShowIndex) StatementTag() string { return "SHOW INDEX" }
-func (*ShowIndex) hiddenFromStats() {}
-func (*ShowIndex) independentFromParallelizedPriors() {}
+func (*ShowIndex) hiddenFromStats() {}
// StatementType implements the Statement interface.
func (*ShowQueries) StatementType() StatementType { return Rows }
@@ -737,8 +732,7 @@ func (*ShowRoleGrants) StatementType() StatementType { return Rows }
// StatementTag returns a short string identifying the type of statement.
func (*ShowRoleGrants) StatementTag() string { return "SHOW GRANTS ON ROLE" }
-func (*ShowRoleGrants) hiddenFromStats() {}
-func (*ShowRoleGrants) independentFromParallelizedPriors() {}
+func (*ShowRoleGrants) hiddenFromStats() {}
// StatementType implements the Statement interface.
func (*ShowSessions) StatementType() StatementType { return Rows }
@@ -755,8 +749,7 @@ func (*ShowTableStats) StatementType() StatementType { return Rows }
// StatementTag returns a short string identifying the type of statement.
func (*ShowTableStats) StatementTag() string { return "SHOW STATISTICS" }
-func (*ShowTableStats) hiddenFromStats() {}
-func (*ShowTableStats) independentFromParallelizedPriors() {}
+func (*ShowTableStats) hiddenFromStats() {}
// StatementType implements the Statement interface.
func (*ShowHistogram) StatementType() StatementType { return Rows }
@@ -764,8 +757,7 @@ func (*ShowHistogram) StatementType() StatementType { return Rows }
// StatementTag returns a short string identifying the type of statement.
func (*ShowHistogram) StatementTag() string { return "SHOW HISTOGRAM" }
-func (*ShowHistogram) hiddenFromStats() {}
-func (*ShowHistogram) independentFromParallelizedPriors() {}
+func (*ShowHistogram) hiddenFromStats() {}
// StatementType implements the Statement interface.
func (*ShowSyntax) StatementType() StatementType { return Rows }
@@ -793,8 +785,7 @@ func (*ShowUsers) StatementType() StatementType { return Rows }
// StatementTag returns a short string identifying the type of statement.
func (*ShowUsers) StatementTag() string { return "SHOW USERS" }
-func (*ShowUsers) hiddenFromStats() {}
-func (*ShowUsers) independentFromParallelizedPriors() {}
+func (*ShowUsers) hiddenFromStats() {}
// StatementType implements the Statement interface.
func (*ShowRoles) StatementType() StatementType { return Rows }
@@ -802,8 +793,7 @@ func (*ShowRoles) StatementType() StatementType { return Rows }
// StatementTag returns a short string identifying the type of statement.
func (*ShowRoles) StatementTag() string { return "SHOW ROLES" }
-func (*ShowRoles) hiddenFromStats() {}
-func (*ShowRoles) independentFromParallelizedPriors() {}
+func (*ShowRoles) hiddenFromStats() {}
// StatementType implements the Statement interface.
func (*ShowZoneConfig) StatementType() StatementType { return Rows }
@@ -825,16 +815,13 @@ func (*ShowFingerprints) StatementType() StatementType { return Rows }
// StatementTag returns a short string identifying the type of statement.
func (*ShowFingerprints) StatementTag() string { return "SHOW EXPERIMENTAL_FINGERPRINTS" }
-func (*ShowFingerprints) independentFromParallelizedPriors() {}
-
// StatementType implements the Statement interface.
func (*ShowConstraints) StatementType() StatementType { return Rows }
// StatementTag returns a short string identifying the type of statement.
func (*ShowConstraints) StatementTag() string { return "SHOW CONSTRAINTS" }
-func (*ShowConstraints) hiddenFromStats() {}
-func (*ShowConstraints) independentFromParallelizedPriors() {}
+func (*ShowConstraints) hiddenFromStats() {}
// StatementType implements the Statement interface.
func (*ShowTables) StatementType() StatementType { return Rows }
@@ -842,8 +829,7 @@ func (*ShowTables) StatementType() StatementType { return Rows }
// StatementTag returns a short string identifying the type of statement.
func (*ShowTables) StatementTag() string { return "SHOW TABLES" }
-func (*ShowTables) hiddenFromStats() {}
-func (*ShowTables) independentFromParallelizedPriors() {}
+func (*ShowTables) hiddenFromStats() {}
// StatementType implements the Statement interface.
func (*ShowSchemas) StatementType() StatementType { return Rows }
@@ -851,8 +837,7 @@ func (*ShowSchemas) StatementType() StatementType { return Rows }
// StatementTag returns a short string identifying the type of statement.
func (*ShowSchemas) StatementTag() string { return "SHOW SCHEMAS" }
-func (*ShowSchemas) hiddenFromStats() {}
-func (*ShowSchemas) independentFromParallelizedPriors() {}
+func (*ShowSchemas) hiddenFromStats() {}
// StatementType implements the Statement interface.
func (*Split) StatementType() StatementType { return Rows }
diff --git a/pkg/sql/txn_restart_test.go b/pkg/sql/txn_restart_test.go
index 18787910417e..3eab77ed9241 100644
--- a/pkg/sql/txn_restart_test.go
+++ b/pkg/sql/txn_restart_test.go
@@ -738,7 +738,7 @@ END;
t.Fatal(err)
}
if count != 1 {
- t.Fatalf("Expected 1 rows, got %d", count)
+ t.Fatalf("Expected 1 rows, got %d from %s", count, table)
}
}
})
diff --git a/pkg/sql/txn_state_test.go b/pkg/sql/txn_state_test.go
index f38943d24d0d..c48f3fec95d6 100644
--- a/pkg/sql/txn_state_test.go
+++ b/pkg/sql/txn_state_test.go
@@ -57,12 +57,13 @@ func makeTestContext() testContext {
manual := hlc.NewManualClock(123)
clock := hlc.NewClock(manual.UnixNano, time.Nanosecond)
factory := client.TxnSenderFactoryFunc(func(client.TxnType) client.TxnSender {
- return client.TxnSenderFunc(
- func(context.Context, roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) {
+ return client.TxnSenderAdapter{
+ Wrapped: func(context.Context, roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) {
return nil, nil
},
- )
+ }
})
+
settings := cluster.MakeTestingClusterSettings()
return testContext{
manualClock: manual,
diff --git a/pkg/storage/batcheval/cmd_begin_transaction.go b/pkg/storage/batcheval/cmd_begin_transaction.go
index d43ee84d5e10..ccdf0a432509 100644
--- a/pkg/storage/batcheval/cmd_begin_transaction.go
+++ b/pkg/storage/batcheval/cmd_begin_transaction.go
@@ -20,6 +20,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/roachpb"
+ "github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/storage/batcheval/result"
"github.com/cockroachdb/cockroach/pkg/storage/engine"
"github.com/cockroachdb/cockroach/pkg/storage/spanset"
@@ -48,6 +49,9 @@ func declareKeysBeginTransaction(
) {
DeclareKeysWriteTransaction(desc, header, req, spans)
spans.Add(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeTxnSpanGCThresholdKey(header.RangeID)})
+ spans.Add(spanset.SpanReadOnly, roachpb.Span{
+ Key: keys.AbortSpanKey(header.RangeID, header.Txn.ID),
+ })
}
// BeginTransaction writes the initial transaction record. Fails in
@@ -130,7 +134,10 @@ func BeginTransaction(
// we bump the record's heartbeat timestamp right before laying it down.
reply.Txn.LastHeartbeat.Forward(cArgs.EvalCtx.Clock().Now())
+ if !cArgs.EvalCtx.ClusterSettings().Version.IsActive(cluster.VersionClientSideWritingFlag) {
+ reply.Txn.Writing = true
+ }
+
// Write the txn record.
- reply.Txn.Writing = true
return result.Result{}, engine.MVCCPutProto(ctx, batch, cArgs.Stats, key, hlc.Timestamp{}, nil, reply.Txn)
}
diff --git a/pkg/storage/replica.go b/pkg/storage/replica.go
index c615a8a5f827..f3648e8529f5 100644
--- a/pkg/storage/replica.go
+++ b/pkg/storage/replica.go
@@ -2942,12 +2942,13 @@ func (r *Replica) evaluateProposal(
if pErr != nil {
pErr = r.maybeSetCorrupt(ctx, pErr)
- // Restore the original txn's Writing bool if the error specifies
- // a transaction.
- if txn := pErr.GetTxn(); txn != nil {
- if ba.Txn == nil {
- log.Fatalf(ctx, "error had a txn but batch is non-transactional. Err txn: %s", txn)
- }
+ txn := pErr.GetTxn()
+ if txn != nil && ba.Txn == nil {
+ log.Fatalf(ctx, "error had a txn but batch is non-transactional. Err txn: %s", txn)
+ }
+ if txn != nil && !r.ClusterSettings().Version.IsActive(cluster.VersionClientSideWritingFlag) {
+ // Restore the original txn's Writing bool if the error specifies a
+ // transaction.
if txn.ID == ba.Txn.ID {
txn.Writing = ba.Txn.Writing
}
@@ -5229,12 +5230,14 @@ func (r *Replica) evaluateWriteBatch(
if pErr == nil && (ba.Timestamp == br.Timestamp ||
(retryLocally && !isEndTransactionExceedingDeadline(br.Timestamp, *etArg))) {
clonedTxn := ba.Txn.Clone()
- clonedTxn.Writing = true
clonedTxn.Status = roachpb.COMMITTED
// Make sure the returned txn has the actual commit
// timestamp. This can be different if the stripped batch was
// executed at the server's hlc now timestamp.
clonedTxn.Timestamp = br.Timestamp
+ if !r.ClusterSettings().Version.IsActive(cluster.VersionClientSideWritingFlag) {
+ clonedTxn.Writing = true
+ }
// If the end transaction is not committed, clear the batch and mark the status aborted.
if !etArg.Commit {
diff --git a/pkg/storage/replica_test.go b/pkg/storage/replica_test.go
index f0e496842e60..02b4678fb9c0 100644
--- a/pkg/storage/replica_test.go
+++ b/pkg/storage/replica_test.go
@@ -5368,7 +5368,6 @@ func TestPushTxnUpgradeExistingTxn(t *testing.T) {
expTxn.Timestamp = test.expTS
expTxn.Status = roachpb.ABORTED
expTxn.LastHeartbeat = test.startTS
- expTxn.Writing = true
if !reflect.DeepEqual(expTxn, reply.PusheeTxn) {
t.Fatalf("unexpected push txn in trial %d: %s", i, pretty.Diff(expTxn, reply.PusheeTxn))
diff --git a/pkg/storage/store_test.go b/pkg/storage/store_test.go
index be859a273504..cf5f2f039fc9 100644
--- a/pkg/storage/store_test.go
+++ b/pkg/storage/store_test.go
@@ -77,7 +77,7 @@ type testSenderFactory struct {
store *Store
}
-func (f *testSenderFactory) New(typ client.TxnType) client.TxnSender {
+func (f *testSenderFactory) New(typ client.TxnType, _ *roachpb.Transaction) client.TxnSender {
return &testSender{store: f.store}
}
@@ -93,10 +93,12 @@ type testSender struct {
func (db *testSender) GetMeta() roachpb.TxnCoordMeta { panic("unimplemented") }
-func (db *testSender) AugmentMeta(roachpb.TxnCoordMeta) { panic("unimplemented") }
+func (db *testSender) AugmentMeta(context.Context, roachpb.TxnCoordMeta) { panic("unimplemented") }
func (db *testSender) OnFinish(func(error)) { panic("unimplemented") }
+func (db *testSender) StartTracking(context.Context) error { panic("unimplemented") }
+
// Send forwards the call to the single store. This is a poor man's
// version of kv.TxnCoordSender, but it serves the purposes of
// supporting tests in this package. Transactions are not supported.