diff --git a/pkg/internal/client/db_test.go b/pkg/internal/client/db_test.go index 87db9a1c3ddd..92c07ec9276f 100644 --- a/pkg/internal/client/db_test.go +++ b/pkg/internal/client/db_test.go @@ -395,6 +395,7 @@ func TestCommonMethods(t *testing.T) { {txnType, "SetSystemConfigTrigger"}: {}, {txnType, "SetTxnAnchorKey"}: {}, {txnType, "UpdateDeadlineMaybe"}: {}, + {txnType, "UpdateStateOnDetachedErr"}: {}, {txnType, "AddCommitTrigger"}: {}, {txnType, "CommandCount"}: {}, {txnType, "IsRetryableErrMeantForTxn"}: {}, diff --git a/pkg/internal/client/sender.go b/pkg/internal/client/sender.go index 5fb7dde29226..9416bc0e248c 100644 --- a/pkg/internal/client/sender.go +++ b/pkg/internal/client/sender.go @@ -28,6 +28,18 @@ type Sender interface { Send(context.Context, roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) } +// !!! comment +type SenderWithDistSQLBackdoor interface { + Sender + + UpdateStateOnDetachedErr( + ctx context.Context, + txn roachpb.Transaction, + pri roachpb.UserPriority, + err error, + ) roachpb.Transaction +} + // SenderFunc is an adapter to allow the use of ordinary functions // as Senders. type SenderFunc func(context.Context, roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) diff --git a/pkg/internal/client/txn.go b/pkg/internal/client/txn.go index b849a488208d..afc0ecd0bbf4 100644 --- a/pkg/internal/client/txn.go +++ b/pkg/internal/client/txn.go @@ -640,13 +640,17 @@ func (txn *Txn) Exec( } } + if _, ok := err.(*roachpb.InternalRetryableTxnError); ok { + log.Fatalf(ctx, "unexpected InternalRetryableTxnError at the txn.Exec level: %s", err) + } + retErr, retryable := err.(*roachpb.RetryableTxnError) - if retryable && !txn.IsRetryableErrMeantForTxn(retErr) { + if retryable && !txn.IsRetryableErrMeantForTxn(*retErr) { // Make sure the txn record that err carries is for this txn. // If it's not, we terminate the "retryable" character of the error. We // might get a RetryableTxnError if the closure ran another transaction // internally and let the error propagate upwards. - return errors.Wrap(retErr, "retryable error from another txn") + return errors.Wrapf(retErr, "retryable error from another txn. Current txn ID: %v", txn.Proto().ID) } if !opt.AutoRetry || !retryable { break @@ -663,23 +667,25 @@ func (txn *Txn) Exec( // IsRetryableErrMeantForTxn returns true if err is a retryable // error meant to restart this client transaction. -func (txn *Txn) IsRetryableErrMeantForTxn(err *roachpb.RetryableTxnError) bool { +func (txn *Txn) IsRetryableErrMeantForTxn(retryErr roachpb.RetryableTxnError) bool { txn.mu.Lock() defer txn.mu.Unlock() - return txn.isRetryableErrMeantForTxnLocked(err) + return txn.isRetryableErrMeantForTxnLocked(retryErr) } -func (txn *Txn) isRetryableErrMeantForTxnLocked(err *roachpb.RetryableTxnError) bool { +func (txn *Txn) isRetryableErrMeantForTxnLocked(retryErr roachpb.RetryableTxnError) bool { + errTxnID := retryErr.TxnID + // Make sure the txn record that err carries is for this txn. // First check if the error was meant for a previous incarnation // of the transaction. - if err.TxnID != nil { - if _, ok := txn.mu.previousIDs[*err.TxnID]; ok { + if errTxnID != nil { + if _, ok := txn.mu.previousIDs[*errTxnID]; ok { return true } } // If not, make sure it was meant for this transaction. - return roachpb.TxnIDEqual(err.TxnID, txn.mu.Proto.ID) + return roachpb.TxnIDEqual(errTxnID, txn.mu.Proto.ID) } // send runs the specified calls synchronously in a single batch and @@ -806,6 +812,7 @@ func (txn *Txn) send( } // Send call through the DB. + requestTxnID, requestEpoch := ba.Txn.ID, ba.Txn.Epoch br, pErr := txn.db.send(ctx, ba) // Lock for the entire response postlude. @@ -846,7 +853,13 @@ func (txn *Txn) send( if log.V(1) { log.Infof(ctx, "failed batch: %s", pErr) } - txn.updateStateOnErrLocked(pErr.GoError()) + if pErr.TransactionRestart != roachpb.TransactionRestart_NONE { + retryErr, ok := pErr.GetDetail().(*roachpb.HandledRetryableError) + if !ok { + log.Fatalf(ctx, "unexpected retryable error at the client.Txn level: (%T) %s", pErr.GetDetail(), pErr) + } + txn.updateStateOnRetryableErrLocked(ctx, *retryErr, ba.UserPriority, requestTxnID, requestEpoch) + } return nil, pErr } @@ -861,10 +874,8 @@ func (txn *Txn) send( } // Only successful requests can carry an updated Txn in their response - // header. Any error (e.g. a restart) can have a Txn attached to them as - // well; those update our local state in the same way for the next attempt. - // The exception is if our transaction was aborted and needs to restart - // from scratch, in which case we do just that. + // header. Some errors (e.g. a restart) have a Txn attached to them as + // well; these errors have been handled above. txn.mu.Proto.Update(br.Txn) } @@ -913,73 +924,84 @@ func firstWriteIndex(ba roachpb.BatchRequest) (int, *roachpb.Error) { } // UpdateStateOnErr updates the Txn, and the Transaction proto inside it, in -// response to an error encountered when running a request through the txn. If +// response to an error encountered when running a request through the txn. If // the error is not a RetryableTxnError, then this is a no-op. For a retryable // error, the Transaction proto is either initialized with the updated proto // from the error, or a new Transaction proto is initialized. // +// pri is the priority that should be used if the transaction needs to be +// restarted and we're giving the restarted transaction the chance to get a +// higher priority. +// // TODO(andrei,nvanbenschoten): document whether calling this repeatedly within // the same epoch of the txn is safe or not, and generally what protection we // have for racing calls. We protect against calls for old IDs, but what about // old epochs and/or the current epoch? -func (txn *Txn) UpdateStateOnErr(err error) { - txn.mu.Lock() - txn.updateStateOnErrLocked(err) - txn.mu.Unlock() +func (txn *Txn) UpdateStateOnDetachedErr(ctx context.Context, err error, pri roachpb.UserPriority) { + // !!! + // // Here we assume that the Txn hasn't changed asynchronously since we + // // started executing the query; we're relying on DistSQL queries not being + // // executed concurrently with anything else using this txn. + // + // txn.mu.Lock() + // defer txn.mu.Unlock() + // + // prevID := txn.mu.Proto.ID + // txn.mu.Proto = txn.db.GetSender().(SenderWithDistSQLBackdoor).UpdateStateOnDetachedErr(ctx, txn.mu.Proto, pri, err) + // // TODO(andrei): Remove the prevID == nil check after #15024 merges and all + // // transactions get IDs earlier. + // if prevID != txn.mu.Proto.ID && prevID != nil { + // txn.recordPreviousTxnIDLocked(*prevID) + // } } -// updateStateOnErrLocked is like UpdateStateOnErr, but assumes that txn.mu is -// locked. -func (txn *Txn) updateStateOnErrLocked(err error) { - retryErr, ok := err.(*roachpb.RetryableTxnError) - if !ok { +// updateStateOnRetryableErrLocked updates the Transaction proto inside txn. +// +// requestTxnID and requestEpoch identify the state of the transaction at the +// time when the request that generated retryErr was sent. These are used to see +// if the information in the error is obsolete by now. +func (txn *Txn) updateStateOnRetryableErrLocked( + ctx context.Context, + retryErr roachpb.HandledRetryableError, + pri roachpb.UserPriority, + requestTxnID *uuid.UUID, + requestEpoch uint32, +) { + if !roachpb.TxnIDEqual(requestTxnID, txn.mu.Proto.ID) { return } - if !txn.isRetryableErrMeantForTxnLocked(retryErr) { - // If this happens, something is wrong; we've received an error that - // wasn't meant for this transaction. This is a sign that we either - // somehow ran another txn inside our txn and didn't properly terminate - // its error, or our transaction got a TransactionAbortedError (and the - // proto was reset), was retried, and then we still somehow managed to get - // an error meant for the previous incarnation of the transaction. - // Letting this wrong error slip here can cause us to retry the wrong - // transaction. - panic(fmt.Sprintf("Got a retryable error meant for a different transaction. "+ - "txn.mu.Proto.ID: %v, pErr.ID: %v", txn.mu.Proto.ID, retryErr.TxnID)) + newTxn := retryErr.Transaction + if newTxn == nil { + log.Fatalf(ctx, "Retryable error without a txn. "+ + "The txn should have always been filled by TxnCoordSender. err: %s", retryErr) + } + if newTxn.ID == nil { + // newTxn.ID == nil means the cause was a TransactionAbortedError; + // we're going to initialized a new Transaction, and so have to save the + // old transaction ID so that concurrent requests or delayed responses + // that that throw errors know that these errors were sent to the correct + // transaction, even once the proto is reset. + txn.recordPreviousTxnIDLocked(*txn.mu.Proto.ID) } - - // Reset the statement count as this is a retryable txn error. - txn.mu.commandCount = 0 // Only update the proto if the retryable error is meant for the current - // incarnation of the transaction. In other words, only update it if - // retryErr.TxnID is not in txn.mu.previousIDs. - if roachpb.TxnIDEqual(retryErr.TxnID, txn.mu.Proto.ID) { - if retryErr.Transaction != nil { - txn.mu.Proto.Update(retryErr.Transaction) - } else { - // Transaction == nil means the cause was a TransactionAbortedError. We'll - // init a new Transaction, and also save the old transaction ID so that - // concurrent requests or delayed responses that that throw errors know - // that these errors were sent to the correct transaction, even once the - // proto is reset. - if txn.mu.previousIDs == nil { - txn.mu.previousIDs = make(map[uuid.UUID]struct{}) - } - txn.mu.previousIDs[*txn.mu.Proto.ID] = struct{}{} + // incarnation of the transaction (i.e. the error was generated by a request + // that was sent during the current epoch). + if requestEpoch == txn.mu.Proto.Epoch { + // Reset the statement count as this is a retryable txn error. + txn.mu.commandCount = 0 - // Next, reset the transaction proto so we start anew on restart. - txn.mu.Proto = roachpb.Transaction{ - TxnMeta: enginepb.TxnMeta{ - Isolation: txn.mu.Proto.Isolation, - }, - Name: txn.mu.Proto.Name, - } - // Acts as a minimum priority on restart. - if retryErr.RetryPriority != nil { - txn.mu.Proto.Priority = *retryErr.RetryPriority - } - } + // Overwrite the transaction proto with the one to be used for the next + // attempt. The txn inside pErr was correctly prepared for this by + // TxnCoordSender. + txn.mu.Proto = *newTxn + } +} + +func (txn *Txn) recordPreviousTxnIDLocked(prevTxnID uuid.UUID) { + if txn.mu.previousIDs == nil { + txn.mu.previousIDs = make(map[uuid.UUID]struct{}) } + txn.mu.previousIDs[*txn.mu.Proto.ID] = struct{}{} } diff --git a/pkg/internal/client/txn_test.go b/pkg/internal/client/txn_test.go index 8e6b5daacf41..0526d9ee92a9 100644 --- a/pkg/internal/client/txn_test.go +++ b/pkg/internal/client/txn_test.go @@ -18,6 +18,7 @@ package client import ( "bytes" + "fmt" "reflect" "regexp" "testing" @@ -204,25 +205,6 @@ func TestTxnRequestTxnTimestamp(t *testing.T) { } } -// TestTxnResetTxnOnAbort verifies transaction is reset on abort. -func TestTxnResetTxnOnAbort(t *testing.T) { - defer leaktest.AfterTest(t)() - clock := hlc.NewClock(hlc.UnixNano, 0) - db := NewDB(newTestSender(func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { - return nil, roachpb.NewErrorWithTxn(&roachpb.TransactionAbortedError{}, ba.Txn) - }), clock) - - txn := NewTxn(db) - _, pErr := txn.send(context.Background(), testPut()) - if _, ok := pErr.GetDetail().(*roachpb.TransactionAbortedError); !ok { - t.Fatalf("expected TransactionAbortedError, got %v", pErr) - } - - if txn.Proto().ID != nil { - t.Error("expected txn to be cleared") - } -} - // TestTransactionConfig verifies the proper unwrapping and // re-wrapping of the client's sender when starting a transaction. // Also verifies that the UserPriority is propagated to the @@ -456,10 +438,14 @@ func TestEndWriteRestartReadOnlyTransaction(t *testing.T) { t.Fatal(err) } ok = true - // Return an immediate txn retry error. We need to go through the pErr - // and back to get a RetryableTxnError. - return roachpb.NewErrorWithTxn( - roachpb.NewTransactionRetryError(), txn.Proto()).GoError() + // Return an immediate txn retry error. + // HACK ALERT: to do without a TxnCoordSender, we jump through hoops to + // get the retryable error expected by db.Txn(). + intRetErr := roachpb.NewErrorWithTxn( + roachpb.NewTransactionRetryError(), txn.Proto()).GoError().(*roachpb.InternalRetryableTxnError) + return roachpb.NewError( + roachpb.NewHandledRetryableError( + intRetErr.Error(), intRetErr.TxnID, *intRetErr.Transaction)).GoError() } if !success { return errors.New("aborting on purpose") @@ -508,7 +494,12 @@ func TestTransactionKeyNotChangedInRestart(t *testing.T) { if attempt == 0 { // Abort the first attempt so that we need to retry with // a new transaction proto. - return nil, roachpb.NewErrorWithTxn(roachpb.NewTransactionRetryError(), ba.Txn) + // + // HACK ALERT: to do without a TxnCoordSender, we jump through hoops to + // get the retryable error expected by db.Txn(). + intRetErr := roachpb.NewErrorWithTxn( + roachpb.NewTransactionRetryError(), ba.Txn).GoError().(*roachpb.InternalRetryableTxnError) + return nil, roachpb.NewError(roachpb.NewHandledRetryableError(intRetErr.Error(), intRetErr.TxnID, *intRetErr.Transaction)) } return ba.CreateReply(), nil }), clock) @@ -574,135 +565,58 @@ func TestRunTransactionRetryOnErrors(t *testing.T) { {&roachpb.TransactionStatusError{}, false}, } - for i, test := range testCases { - count := 0 - db := NewDB(newTestSender( - func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { - - if _, ok := ba.GetArg(roachpb.Put); ok { - count++ - if count == 1 { - return nil, roachpb.NewErrorWithTxn(test.err, ba.Txn) + for _, test := range testCases { + t.Run(fmt.Sprintf("%T", test.err), func(t *testing.T) { + count := 0 + db := NewDB(newTestSender( + func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + + if _, ok := ba.GetArg(roachpb.Put); ok { + count++ + if count == 1 { + var pErr *roachpb.Error + if _, ok := test.err.(*roachpb.ReadWithinUncertaintyIntervalError); ok { + // This error requires an observed timestamp to have been + // recorded on the origin node. + ba.Txn.UpdateObservedTimestamp(1, hlc.Timestamp{WallTime: 1, Logical: 1}) + pErr = roachpb.NewErrorWithTxn(test.err, ba.Txn) + pErr.OriginNode = 1 + } else { + pErr = roachpb.NewErrorWithTxn(test.err, ba.Txn) + } + + if pErr.TransactionRestart != roachpb.TransactionRestart_NONE { + // HACK ALERT: to do without a TxnCoordSender, we jump through hoops to + // get the retryable error expected by db.Txn(). + intRetErr := pErr.GoError().(*roachpb.InternalRetryableTxnError) + intRetErr.Transaction = ba.Txn + return nil, roachpb.NewError(roachpb.NewHandledRetryableError( + intRetErr.Error(), intRetErr.TxnID, *intRetErr.Transaction)) + } + return nil, pErr + } } + return ba.CreateReply(), nil + }), clock) + err := db.Txn(context.TODO(), func(ctx context.Context, txn *Txn) error { + return txn.Put(ctx, "a", "b") + }) + if test.retry { + if err != nil { + t.Fatalf("expected success on retry; got %s", err) } - return ba.CreateReply(), nil - }), clock) - err := db.Txn(context.TODO(), func(ctx context.Context, txn *Txn) error { - return txn.Put(ctx, "a", "b") - }) - if test.retry { - if count != 2 { - t.Errorf("%d: expected one retry; got %d", i, count-1) - } - if err != nil { - t.Errorf("%d: expected success on retry; got %s", i, err) - } - } else { - if count != 1 { - t.Errorf("%d: expected no retries; got %d", i, count) - } - if reflect.TypeOf(err) != reflect.TypeOf(test.err) { - t.Errorf("%d: expected error of type %T; got %T", i, test.err, err) - } - } - } -} - -// Test that the a txn gets a fresh OrigTimestamp with every retry. -func TestAbortedRetryRenewsTimestamp(t *testing.T) { - defer leaktest.AfterTest(t)() - - // Create a TestSender that aborts a transaction 2 times before succeeding. - mc := hlc.NewManualClock(123) - clock := hlc.NewClock(mc.UnixNano, time.Nanosecond) - count := 0 - db := NewDB(newTestSender(func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { - if _, ok := ba.GetArg(roachpb.Put); ok { - mc.Increment(1) - count++ - if count < 3 { - return nil, roachpb.NewErrorWithTxn(&roachpb.TransactionAbortedError{}, ba.Txn) - } - } - return ba.CreateReply(), nil - }), clock) - - txnClosure := func(ctx context.Context, txn *Txn, opt *TxnExecOptions) error { - // Ensure the KV transaction is created. - return txn.Put(ctx, "a", "b") - } - - txn := NewTxn(db) - - // Request a client-defined timestamp. - refTimestamp := clock.Now() - execOpt := TxnExecOptions{ - AutoRetry: true, - AutoCommit: true, - AssignTimestampImmediately: true, - } - - // Perform the transaction. - if err := txn.Exec(context.Background(), execOpt, txnClosure); err != nil { - t.Fatal(err) - } - - // Check the timestamp was preserved. - if txn.Proto().OrigTimestamp.WallTime == refTimestamp.WallTime { - t.Errorf("expected txn orig ts to be different than %s", refTimestamp) - } -} - -// TestAbortTransactionOnCommitErrors verifies that transactions are -// aborted on the correct errors. -func TestAbortTransactionOnCommitErrors(t *testing.T) { - defer leaktest.AfterTest(t)() - clock := hlc.NewClock(hlc.UnixNano, 0) - - testCases := []struct { - err error - abort bool - }{ - {roachpb.NewReadWithinUncertaintyIntervalError(hlc.Timestamp{}, hlc.Timestamp{}), true}, - {&roachpb.TransactionAbortedError{}, false}, - {&roachpb.TransactionPushError{}, true}, - {&roachpb.TransactionRetryError{}, true}, - {&roachpb.RangeNotFoundError{}, true}, - {&roachpb.RangeKeyMismatchError{}, true}, - {&roachpb.TransactionStatusError{}, true}, - } - - for _, test := range testCases { - var commit, abort bool - db := NewDB(newTestSender(func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { - - switch t := ba.Requests[0].GetInner().(type) { - case *roachpb.EndTransactionRequest: - if t.Commit { - commit = true - return nil, roachpb.NewErrorWithTxn(test.err, ba.Txn) + if count != 2 { + t.Fatalf("expected one retry; got %d", count-1) + } + } else { + if count != 1 { + t.Errorf("expected no retries; got %d", count) + } + if reflect.TypeOf(err) != reflect.TypeOf(test.err) { + t.Errorf("expected error of type %T; got %T", test.err, err) } - abort = true } - return ba.CreateReply(), nil - }), clock) - - txn := NewTxn(db) - if pErr := txn.Put(context.Background(), "a", "b"); pErr != nil { - t.Fatalf("put failed: %s", pErr) - } - if pErr := txn.CommitOrCleanup(context.Background()); pErr == nil { - t.Fatalf("unexpected commit success") - } - - if !commit { - t.Errorf("%T: failed to find commit", test.err) - } - if test.abort && !abort { - t.Errorf("%T: failed to find abort", test.err) - } else if !test.abort && abort { - t.Errorf("%T: found unexpected abort", test.err) - } + }) } } @@ -853,8 +767,13 @@ func TestWrongTxnRetry(t *testing.T) { if err := innerTxn.Put(ctx, "x", "y"); err != nil { t.Fatal(err) } - return roachpb.NewErrorWithTxn(&roachpb.TransactionPushError{ - PusheeTxn: *outerTxn.Proto()}, innerTxn.Proto()).GoError() + // HACK ALERT: to do without a TxnCoordSender, we jump through hoops to + // get the retryable error expected by db.Txn(). + + intRetErr := roachpb.NewErrorWithTxn(&roachpb.TransactionPushError{ + PusheeTxn: *outerTxn.Proto()}, innerTxn.Proto()).GoError().(*roachpb.InternalRetryableTxnError) + return roachpb.NewError(roachpb.NewHandledRetryableError( + intRetErr.Error(), intRetErr.TxnID, *intRetErr.Transaction)).GoError() } innerTxn := NewTxn(db) err := innerTxn.Exec(ctx, execOpt, innerClosure) diff --git a/pkg/kv/dist_sender_server_test.go b/pkg/kv/dist_sender_server_test.go index 4d58bda8aaf1..066c14515cf8 100644 --- a/pkg/kv/dist_sender_server_test.go +++ b/pkg/kv/dist_sender_server_test.go @@ -1147,8 +1147,8 @@ func TestPropagateTxnOnError(t *testing.T) { err := txn.CommitInBatch(ctx, b) if epoch == 1 { if retErr, ok := err.(*roachpb.RetryableTxnError); ok { - if _, ok := retErr.Cause.(*roachpb.ReadWithinUncertaintyIntervalError); !ok { - t.Errorf("expected ReadWithinUncertaintyIntervalError, but got: %s", retErr.Cause) + if !testutils.IsError(retErr, "ReadWithinUncertaintyIntervalError") { + t.Errorf("expected ReadWithinUncertaintyIntervalError, but got: %s", retErr) } } else { t.Errorf("expected a retryable error, but got: %s", err) diff --git a/pkg/kv/txn_coord_sender.go b/pkg/kv/txn_coord_sender.go index 802d1c667ab0..d24d6b833d84 100644 --- a/pkg/kv/txn_coord_sender.go +++ b/pkg/kv/txn_coord_sender.go @@ -511,8 +511,13 @@ func (tc *TxnCoordSender) maybeRejectClientLocked( case txnMeta.txn.Status == roachpb.ABORTED: txn := txnMeta.txn.Clone() tc.cleanupTxnLocked(ctx, txn) - return roachpb.NewErrorWithTxn(roachpb.NewTransactionAbortedError(), - &txn) + abortedErr := roachpb.NewErrorWithTxn(roachpb.NewTransactionAbortedError(), &txn) + retryErr := abortedErr.GoError().(*roachpb.InternalRetryableTxnError) + // TODO(andrei): figure out a UserPriority to use here. + newTxn := roachpb.PrepareTransactionForRetry( + retryErr, roachpb.NormalUserPriority, txn.Isolation, txn.Name) + return roachpb.NewError(roachpb.NewHandledRetryableError( + retryErr.Error(), retryErr.TxnID, newTxn)) case txnMeta.txn.Status == roachpb.COMMITTED: txn := txnMeta.txn.Clone() tc.cleanupTxnLocked(ctx, txn) @@ -555,9 +560,20 @@ func (tc *TxnCoordSender) validateTxnForBatch(ba *roachpb.BatchRequest) error { return nil } -// cleanupTxnLocked is called when a transaction ends. The transaction record -// is updated and the heartbeat goroutine signaled to clean up the transaction +// CleanupTxn is called when a transaction ends. The transaction record is +// updated and the heartbeat goroutine signaled to clean up the transaction // gracefully. +// +// TODO(andrei): the TxnCoordSender should get out of the business of deciding +// when the transaction should be cleaned up; this decision should be delegated +// to the client. Also see #10511. +func (tc *TxnCoordSender) CleanupTxn(ctx context.Context, txn roachpb.Transaction) { + tc.txnMu.Lock() + tc.cleanupTxnLocked(ctx, txn) + tc.txnMu.Unlock() +} + +// cleanupTxnLocked is like CleanupTxn, but assumes that tc.txnMu is locked. func (tc *TxnCoordSender) cleanupTxnLocked(ctx context.Context, txn roachpb.Transaction) { log.Event(ctx, "coordinator stops") txnMeta, ok := tc.txnMu.txns[*txn.ID] @@ -791,8 +807,13 @@ func (tc *TxnCoordSender) heartbeat(ctx context.Context, txnID uuid.UUID) bool { // 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 certain errors with the +// object when adequate. It also updates retryable errors with the // updated transaction for use by client restarts. +// +// startNS is the time when the request that's updating the state has been sent. +// This is not used if the request is known to not be the one in charge of +// starting tracking the transaction - i.e. this is the case for DistSQL, which +// just does reads and passes 0. func (tc *TxnCoordSender) updateState( ctx context.Context, startNS int64, @@ -809,59 +830,67 @@ func (tc *TxnCoordSender) updateState( return pErr } + txnID := *ba.Txn.ID var newTxn roachpb.Transaction - newTxn.Update(ba.Txn) if pErr == nil { + newTxn.Update(ba.Txn) newTxn.Update(br.Txn) - } else if errTxn := pErr.GetTxn(); errTxn != nil { - newTxn.Update(errTxn) - } - - switch t := pErr.GetDetail().(type) { - case *roachpb.OpRequiresTxnError: - panic("OpRequiresTxnError must not happen at this level") - case *roachpb.ReadWithinUncertaintyIntervalError: - // If the reader encountered a newer write within the uncertainty - // interval, we advance the txn's timestamp just past the last observed - // timestamp from the node. - restartTS, ok := newTxn.GetObservedTimestamp(pErr.OriginNode) - if !ok { - pErr = roachpb.NewError(errors.Errorf("no observed timestamp for node %d found on uncertainty restart", pErr.OriginNode)) + } else { + var retryErr *roachpb.InternalRetryableTxnError + if pErr.TransactionRestart != roachpb.TransactionRestart_NONE { + goErr := pErr.GoError() + var ok bool + retryErr, ok = goErr.(*roachpb.InternalRetryableTxnError) + if !ok { + // Something went wrong with the conversion; some assertion must have + // failed in the GoError() call. + log.Errorf(ctx, "Expected conversion to InternalRetryableTxnError "+ + "to succeed, but it didn't. The conversion returned: %s", goErr) + pErr = roachpb.NewError(goErr) + } + } + if retryErr != nil { + if !roachpb.TxnIDEqual(pErr.GetTxn().ID, &txnID) { + // KV should not return errors for transactions other that the one in + // the BatchRequest. + log.Fatalf(ctx, "retryable error for the wrong txn. ba.Txn: %s. pErr: %s", + ba.Txn, pErr) + } + newTxn = roachpb.PrepareTransactionForRetry( + retryErr, ba.UserPriority, ba.Txn.Isolation, ba.Txn.Name, + ) + if newTxn.ID == nil { + // Clean up the freshly aborted transaction in defer(), avoiding a + // race with the state update below. + // + // TODO(andrei): If the epoch that our map is aware of has already been + // incremented compared to ba.Txn, perhaps we shouldn't abort the txn + // here. This would match client.Txn, who will ignore this error. + defer tc.cleanupTxnLocked(ctx, *ba.Txn) + } + // Pass a HandledRetryableError up to the next layer. + pErr = roachpb.NewError( + roachpb.NewHandledRetryableError(retryErr.Error(), retryErr.TxnID, newTxn)) } else { - newTxn.Timestamp.Forward(restartTS) - newTxn.Restart(ba.UserPriority, newTxn.Priority, newTxn.Timestamp) + // We got a non-retryable error. + + newTxn.Update(ba.Txn) + if errTxn := pErr.GetTxn(); errTxn != nil { + // Writes can carry back intents in errors, so we have to keep track of + // them. + newTxn.Update(errTxn) + } + + // Update the txn in the error to reflect the TxnCoordSender's state. + // + // Avoid changing existing errors because sometimes they escape into + // goroutines and data races can occur. + pErrShallow := *pErr + pErrShallow.SetTxn(&newTxn) // SetTxn clones newTxn + pErr = &pErrShallow } - case *roachpb.TransactionAbortedError: - // Increase timestamp if applicable. - newTxn.Timestamp.Forward(pErr.GetTxn().Timestamp) - newTxn.Priority = pErr.GetTxn().Priority - // Clean up the freshly aborted transaction in defer(), avoiding a - // race with the state update below. - defer tc.cleanupTxnLocked(ctx, newTxn) - case *roachpb.TransactionPushError: - // Increase timestamp if applicable, ensuring that we're - // just ahead of the pushee. - newTxn.Timestamp.Forward(t.PusheeTxn.Timestamp) - newTxn.Restart(ba.UserPriority, t.PusheeTxn.Priority-1, newTxn.Timestamp) - case *roachpb.TransactionRetryError: - // Increase timestamp so on restart, we're ahead of any timestamp - // cache entries or newer versions which caused the restart. - newTxn.Restart(ba.UserPriority, pErr.GetTxn().Priority, newTxn.Timestamp) - case *roachpb.WriteTooOldError: - newTxn.Restart(ba.UserPriority, newTxn.Priority, t.ActualTimestamp) - case nil: - // Nothing to do here, avoid the default case. - default: - // Do not clean up the transaction since we're leaving cancellation of - // the transaction up to the client. For example, on seeing an error, - // like TransactionStatusError or ConditionFailedError, the client - // will call Txn.CleanupOnError() which will cleanup the transaction - // and its intents. Therefore leave the transaction in the PENDING - // state and do not call cleanTxnLocked(). } - txnID := *newTxn.ID - txnMeta := tc.txnMu.txns[txnID] // For successful transactional requests, keep the written intents and // the updated transaction record to be sent along with the reply. @@ -937,15 +966,53 @@ func (tc *TxnCoordSender) updateState( txnMeta.setLastUpdate(tc.clock.PhysicalNow()) } - if pErr != nil && pErr.GetTxn() != nil { - // Avoid changing existing errors because sometimes they escape into - // goroutines and data races can occur. - pErrShallow := *pErr - pErrShallow.SetTxn(&newTxn) // SetTxn clones newTxn - pErr = &pErrShallow + return pErr +} + +// UpdateStateOnDetachedErr is part of the SenderWithDistSQLBackdoor interface. +func (tc *TxnCoordSender) UpdateStateOnDetachedErr( + ctx context.Context, + txn roachpb.Transaction, + pri roachpb.UserPriority, + err error, +) roachpb.Transaction { + tc.txnMu.Lock() + defer tc.txnMu.Unlock() + + retryErr, ok := err.(*roachpb.InternalRetryableTxnError) + if !ok { + // Nothing to update for non-retryable errors. In updateState() we read + // transaction state from all errors to keep track of intents, but that + // doesn't apply here (DistSQL only does reads). + return txn + } + + // TODO(andrei): We should check: + // roachpb.TxnIDEqual(retryErr.TxnID, txn.mu.Proto.ID) + // once #15024 merges. Until then, the check can fail spuriously as DistSQL + // transactions might not have an ID at the gateway. + + newTxn := roachpb.PrepareTransactionForRetry( + retryErr, pri, txn.Isolation, txn.Name, + ) + + // TODO(andrei): Remove the IsInitialized() check once #15024 is merged. + if txn.IsInitialized() { + if newTxn.ID == nil { + // Clean up the freshly aborted transaction. + // !!! maybe TODO is obsolete + // TODO(andrei): We're cleaning up the previous txn here, although the + // client.Txn tries to do the same thing (if it was configured with an + // onKVTxnSwitched). Remove this duplication of work and concerns. + tc.cleanupTxnLocked(ctx, txn) + } else { + txnMeta := tc.txnMu.txns[*txn.ID] + txnMeta.txn.Update(&newTxn) + txnMeta.setLastUpdate(tc.clock.PhysicalNow()) + } } - return pErr + return newTxn } // TODO(tschottdorf): this method is somewhat awkward but unless we want to diff --git a/pkg/kv/txn_coord_sender_test.go b/pkg/kv/txn_coord_sender_test.go index 52c1477f2467..f553475dbdcf 100644 --- a/pkg/kv/txn_coord_sender_test.go +++ b/pkg/kv/txn_coord_sender_test.go @@ -585,9 +585,9 @@ func TestTxnCoordSenderAddIntentOnError(t *testing.T) { func assertTransactionRetryError(t *testing.T, e error) { if retErr, ok := e.(*roachpb.RetryableTxnError); ok { - if _, ok := retErr.Cause.(*roachpb.TransactionRetryError); !ok { - t.Fatalf("expected a TransactionRetryError, but got %s (%T)", - retErr.Cause, retErr.Cause) + if !testutils.IsError(retErr, "TransactionRetryError") { + t.Fatalf("expected the cause to be TransactionRetryError, but got %s", + retErr) } } else { t.Fatalf("expected a retryable error, but got %s (%T)", e, e) @@ -596,9 +596,9 @@ func assertTransactionRetryError(t *testing.T, e error) { func assertTransactionAbortedError(t *testing.T, e error) { if retErr, ok := e.(*roachpb.RetryableTxnError); ok { - if _, ok := retErr.Cause.(*roachpb.TransactionAbortedError); !ok { - t.Fatalf("expected a TransactionAbortedError, but got %s (%T)", - retErr.Cause, retErr.Cause) + if !testutils.IsError(retErr, "TransactionAbortedError") { + t.Fatalf("expected the cause to be TransactionAbortedError, but got %s", + retErr) } } else { t.Fatalf("expected a retryable error, but got %s (%T)", e, e) @@ -790,15 +790,21 @@ func TestTxnCoordSenderTxnUpdatedOnError(t *testing.T) { plus10 := origTS.Add(10, 10) plus20 := plus10.Add(10, 0) testCases := []struct { - pErr *roachpb.Error + // The test's name. + name string + pErrGen func(txn *roachpb.Transaction) *roachpb.Error expEpoch uint32 expPri int32 expTS, expOrigTS hlc.Timestamp - nodeSeen bool + // Is set, we're expecting that the Transaction proto is re-initialized (as + // opposed to just having the epoch incremented). + expNewTransaction bool + nodeSeen bool }{ { // No error, so nothing interesting either. - pErr: nil, + name: "nil", + pErrGen: func(_ *roachpb.Transaction) *roachpb.Error { return nil }, expEpoch: 0, expPri: 1, expTS: origTS, @@ -807,15 +813,17 @@ func TestTxnCoordSenderTxnUpdatedOnError(t *testing.T) { { // On uncertainty error, new epoch begins and node is seen. // Timestamp moves ahead of the existing write. - pErr: func() *roachpb.Error { - pErr := roachpb.NewErrorWithTxn( - roachpb.NewReadWithinUncertaintyIntervalError(hlc.Timestamp{}, hlc.Timestamp{}), - &roachpb.Transaction{}) + name: "ReadWithinUncertaintyIntervalError", + pErrGen: func(txn *roachpb.Transaction) *roachpb.Error { const nodeID = 1 - pErr.GetTxn().UpdateObservedTimestamp(nodeID, plus10) + txn.UpdateObservedTimestamp(nodeID, plus10) + pErr := roachpb.NewErrorWithTxn( + roachpb.NewReadWithinUncertaintyIntervalError( + hlc.Timestamp{}, hlc.Timestamp{}), + txn) pErr.OriginNode = nodeID return pErr - }(), + }, expEpoch: 1, expPri: 1, expTS: plus10, @@ -825,21 +833,26 @@ func TestTxnCoordSenderTxnUpdatedOnError(t *testing.T) { { // On abort, nothing changes but we get a new priority to use for // the next attempt. - pErr: roachpb.NewErrorWithTxn(&roachpb.TransactionAbortedError{}, - &roachpb.Transaction{ - TxnMeta: enginepb.TxnMeta{Timestamp: plus20, Priority: 10}, - }), - expPri: 10, + name: "TransactionAbortedError", + pErrGen: func(txn *roachpb.Transaction) *roachpb.Error { + txn.Timestamp = plus20 + txn.Priority = 10 + return roachpb.NewErrorWithTxn(&roachpb.TransactionAbortedError{}, txn) + }, + expNewTransaction: true, + expPri: 10, }, { // On failed push, new epoch begins just past the pushed timestamp. // Additionally, priority ratchets up to just below the pusher's. - pErr: roachpb.NewErrorWithTxn(&roachpb.TransactionPushError{ - PusheeTxn: roachpb.Transaction{ - TxnMeta: enginepb.TxnMeta{Timestamp: plus10, Priority: int32(10)}, - }, + name: "TransactionPushError", + pErrGen: func(txn *roachpb.Transaction) *roachpb.Error { + return roachpb.NewErrorWithTxn(&roachpb.TransactionPushError{ + PusheeTxn: roachpb.Transaction{ + TxnMeta: enginepb.TxnMeta{Timestamp: plus10, Priority: int32(10)}, + }, + }, txn) }, - &roachpb.Transaction{}), expEpoch: 1, expPri: 9, expTS: plus10, @@ -847,11 +860,12 @@ func TestTxnCoordSenderTxnUpdatedOnError(t *testing.T) { }, { // On retry, restart with new epoch, timestamp and priority. - pErr: roachpb.NewErrorWithTxn(&roachpb.TransactionRetryError{}, - &roachpb.Transaction{ - TxnMeta: enginepb.TxnMeta{Timestamp: plus10, Priority: int32(10)}, - }, - ), + name: "TransactionRetryError", + pErrGen: func(txn *roachpb.Transaction) *roachpb.Error { + txn.Timestamp = plus10 + txn.Priority = 10 + return roachpb.NewErrorWithTxn(&roachpb.TransactionRetryError{}, txn) + }, expEpoch: 1, expPri: 10, expTS: plus10, @@ -859,60 +873,94 @@ func TestTxnCoordSenderTxnUpdatedOnError(t *testing.T) { }, } - for i, test := range testCases { - stopper := stop.NewStopper() + for _, test := range testCases { + t.Run(test.name, func(t *testing.T) { + stopper := stop.NewStopper() - manual := hlc.NewManualClock(origTS.WallTime) - clock := hlc.NewClock(manual.UnixNano, 20*time.Nanosecond) + manual := hlc.NewManualClock(origTS.WallTime) + clock := hlc.NewClock(manual.UnixNano, 20*time.Nanosecond) - var senderFn client.SenderFunc = func(_ context.Context, ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { - var reply *roachpb.BatchResponse - if test.pErr == nil { - reply = ba.CreateReply() + var senderFn client.SenderFunc = func( + _ context.Context, ba roachpb.BatchRequest, + ) (*roachpb.BatchResponse, *roachpb.Error) { + var reply *roachpb.BatchResponse + pErr := test.pErrGen(ba.Txn) + if pErr == nil { + reply = ba.CreateReply() + } + return reply, pErr } - return reply, test.pErr - } - ambient := log.AmbientContext{Tracer: tracing.NewTracer()} - ts := NewTxnCoordSender( - ambient, - senderFn, - clock, - false, - stopper, - MakeTxnMetrics(metric.TestSampleInterval), - ) - db := client.NewDB(ts, clock) - txn := client.NewTxn(db) - txn.InternalSetPriority(1) - txn.SetDebugName("test txn") - key := roachpb.Key("test-key") - _, err := txn.Get(context.TODO(), key) - teardownHeartbeats(ts) - stopper.Stop(context.TODO()) - - if test.pErr != nil && err == nil { - t.Fatalf("expected an error") - } - if txn.Proto().Epoch != test.expEpoch { - t.Errorf("%d: expected epoch = %d; got %d", - i, test.expEpoch, txn.Proto().Epoch) - } - if txn.Proto().Priority != test.expPri { - t.Errorf("%d: expected priority = %d; got %d", - i, test.expPri, txn.Proto().Priority) - } - if txn.Proto().Timestamp != test.expTS { - t.Errorf("%d: expected timestamp to be %s; got %s", - i, test.expTS, txn.Proto().Timestamp) - } - if txn.Proto().OrigTimestamp != test.expOrigTS { - t.Errorf("%d: expected orig timestamp to be %s; got %s", - i, test.expOrigTS, txn.Proto().OrigTimestamp) - } - if ns := txn.Proto().ObservedTimestamps; (len(ns) != 0) != test.nodeSeen { - t.Errorf("%d: expected nodeSeen=%t, but list of hosts is %v", - i, test.nodeSeen, ns) - } + ambient := log.AmbientContext{Tracer: tracing.NewTracer()} + ts := NewTxnCoordSender( + ambient, + senderFn, + clock, + false, + stopper, + MakeTxnMetrics(metric.TestSampleInterval), + ) + db := client.NewDB(ts, clock) + key := roachpb.Key("test-key") + origTxnProto := *roachpb.NewTransaction( + "test txn", + key, + roachpb.UserPriority(0), + enginepb.SERIALIZABLE, + clock.Now(), + clock.MaxOffset().Nanoseconds(), + ) + // TODO(andrei): I've monkeyed with the priorities on this initial + // Transaction to keep the test happy from a previous version in which the + // Transaction was not initialized before use (which became insufficient + // when we started testing that TransactionAbortedError's properly + // re-initializes the proto), but this deserves cleanup. I think this test + // is strict in what updated priorities it expects and also our mechanism + // for assigning exact priorities doesn't work properly when faced with + // updates. + origTxnProto.Priority = 1 + txn := client.NewTxnWithProto(db, origTxnProto) + txn.InternalSetPriority(1) + + _, err := txn.Get(context.TODO(), key) + teardownHeartbeats(ts) + stopper.Stop(context.TODO()) + + if test.name != "nil" && err == nil { + t.Fatalf("expected an error") + } + txnReset := !roachpb.TxnIDEqual(origTxnProto.ID, txn.Proto().ID) + if txnReset != test.expNewTransaction { + t.Fatalf("expected txn reset: %t and got: %t", test.expNewTransaction, txnReset) + } + if test.expNewTransaction { + // Check that the timestamp has been reset. If the transaction reset code + // changes, this should simply assert that the timestamp was advanced. + zeroTS := hlc.Timestamp{} + if txn.Proto().Timestamp != zeroTS || txn.Proto().OrigTimestamp != zeroTS { + t.Fatalf("expected txn with no timestamp, but got: %s", txn.Proto()) + } + } + if txn.Proto().Epoch != test.expEpoch { + t.Errorf("expected epoch = %d; got %d", + test.expEpoch, txn.Proto().Epoch) + } + if txn.Proto().Priority != test.expPri { + t.Errorf("expected priority = %d; got %d", + test.expPri, txn.Proto().Priority) + } + if txn.Proto().Timestamp != test.expTS { + t.Errorf("expected timestamp to be %s; got %s", + test.expTS, txn.Proto().Timestamp) + } + if txn.Proto().OrigTimestamp != test.expOrigTS { + t.Errorf("expected orig timestamp to be %s; got %s", + test.expOrigTS, txn.Proto().OrigTimestamp) + } + if ns := txn.Proto().ObservedTimestamps; (len(ns) != 0) != test.nodeSeen { + t.Errorf("expected nodeSeen=%t, but list of hosts is %v", + test.nodeSeen, ns) + } + }) } } @@ -1555,3 +1603,104 @@ func TestContextDoneNil(t *testing.T) { t.Error("context.Background().Done()'s behavior has changed") } } + +// TestAbortTransactionOnCommitErrors verifies that transactions are +// aborted on the correct errors. +func TestAbortTransactionOnCommitErrors(t *testing.T) { + defer leaktest.AfterTest(t)() + clock := hlc.NewClock(hlc.UnixNano, 0) + + testCases := []struct { + err error + errFn func(roachpb.Transaction) *roachpb.Error + abort bool + }{ + { + errFn: func(txn roachpb.Transaction) *roachpb.Error { + const nodeID = 1 + // ReadWithUncertaintyErrors need a clock to have been recorded on the + // origin. + txn.UpdateObservedTimestamp(nodeID, makeTS(123, 0)) + return roachpb.NewErrorWithTxn( + roachpb.NewReadWithinUncertaintyIntervalError(hlc.Timestamp{}, hlc.Timestamp{}), + &txn) + }, + abort: true}, + {err: &roachpb.TransactionAbortedError{}, abort: false}, + {err: &roachpb.TransactionPushError{}, abort: true}, + {err: &roachpb.TransactionRetryError{}, abort: true}, + {err: &roachpb.RangeNotFoundError{}, abort: true}, + {err: &roachpb.RangeKeyMismatchError{}, abort: true}, + {err: &roachpb.TransactionStatusError{}, abort: true}, + } + + for _, test := range testCases { + t.Run(fmt.Sprintf("%T", test.err), func(t *testing.T) { + var commit, abort bool + + stopper := stop.NewStopper() + defer stopper.Stop(context.TODO()) + var senderFn client.SenderFunc = func( + _ context.Context, ba roachpb.BatchRequest, + ) (*roachpb.BatchResponse, *roachpb.Error) { + br := ba.CreateReply() + + switch req := ba.Requests[0].GetInner().(type) { + case *roachpb.BeginTransactionRequest: + if _, ok := ba.Requests[1].GetInner().(*roachpb.PutRequest); !ok { + t.Fatalf("expected Put") + } + union := &br.Responses[0] // avoid operating on copy + union.MustSetInner(&roachpb.BeginTransactionResponse{}) + union = &br.Responses[1] // avoid operating on copy + union.MustSetInner(&roachpb.PutResponse{}) + if ba.Txn != nil && br.Txn == nil { + txnClone := ba.Txn.Clone() + br.Txn = &txnClone + br.Txn.Writing = true + br.Txn.Status = roachpb.PENDING + } + case *roachpb.EndTransactionRequest: + if req.Commit { + commit = true + if test.errFn != nil { + return nil, test.errFn(*ba.Txn) + } + return nil, roachpb.NewErrorWithTxn(test.err, ba.Txn) + } + abort = true + default: + t.Fatalf("unexpected batch: %s", ba) + } + return br, nil + } + ambient := log.AmbientContext{Tracer: tracing.NewTracer()} + ts := NewTxnCoordSender( + ambient, + senderFn, + clock, + false, + stopper, + MakeTxnMetrics(metric.TestSampleInterval), + ) + db := client.NewDB(ts, clock) + + txn := client.NewTxn(db) + if pErr := txn.Put(context.Background(), "a", "b"); pErr != nil { + t.Fatalf("put failed: %s", pErr) + } + if pErr := txn.CommitOrCleanup(context.Background()); pErr == nil { + t.Fatalf("unexpected commit success") + } + + if !commit { + t.Errorf("%T: failed to find commit", test.err) + } + if test.abort && !abort { + t.Errorf("%T: failed to find abort", test.err) + } else if !test.abort && abort { + t.Errorf("%T: found unexpected abort", test.err) + } + }) + } +} diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go index 127c4b0a7da3..577ba3aa698b 100644 --- a/pkg/roachpb/api.pb.go +++ b/pkg/roachpb/api.pb.go @@ -130,7 +130,8 @@ ErrorDetail ErrPosition Error - RetryableTxnError + InternalRetryableTxnError + HandledRetryableError InternalTimeSeriesData InternalTimeSeriesSample RaftTruncatedState diff --git a/pkg/roachpb/data.go b/pkg/roachpb/data.go index 97a400493055..a63a73cb012a 100644 --- a/pkg/roachpb/data.go +++ b/pkg/roachpb/data.go @@ -956,6 +956,48 @@ func (t Transaction) GetObservedTimestamp(nodeID NodeID) (hlc.Timestamp, bool) { return s.get(nodeID) } +// PrepareTransactionForRetry returns a new Transaction to be used for retrying +// the original Transaction. Depending on the error, this might return an +// already-existing Transaction with an incremented epoch, or a completely new +// Transaction. +// +// The caller should generally check that the error was +// meant for this Transaction before calling this. +// +// pri is the priority that should be used when giving the restarted transaction +// the chance to get a higher priority. +// +// In case retryErr tells us that a new Transaction needs to be created, +// isolation and name help initialize this new transaction. +// +// TODO(andrei): This param is confusing: it's passed as the priority of the +// Batch request that caused the retry. And it's not used if a new Transaction +// needs to be created; in this case the priority will be initialized later, +// before running the first batch in the new retry. Feels like something more +// consistent should be done about this. +func PrepareTransactionForRetry( + retryErr *InternalRetryableTxnError, pri UserPriority, isolation enginepb.IsolationType, name string, +) Transaction { + var newTxn Transaction + if retryErr.Transaction != nil { + newTxn = retryErr.Transaction.Clone() + newTxn.Restart(pri, newTxn.Priority, newTxn.Timestamp) + } else { + // Transaction == nil means the cause was a TransactionAbortedError. We'll + // init a new Transaction. + newTxn = Transaction{ + TxnMeta: enginepb.TxnMeta{ + Isolation: isolation, + }, + Name: name, + } + if retryErr.RetryPriority != nil { + newTxn.Priority = *retryErr.RetryPriority + } + } + return newTxn +} + var _ fmt.Stringer = &Lease{} func (l Lease) String() string { diff --git a/pkg/roachpb/errors.go b/pkg/roachpb/errors.go index 3e3ffaecf881..ec5b64ba308f 100644 --- a/pkg/roachpb/errors.go +++ b/pkg/roachpb/errors.go @@ -22,14 +22,28 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/caller" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/uuid" + "github.com/pkg/errors" ) +// RetryableTxnError represents a retryable transaction error - the transaction +// that caused it should be re-run. +type RetryableTxnError struct { + Msg string + TxnID *uuid.UUID +} + func (e *RetryableTxnError) Error() string { return e.Msg } var _ error = &RetryableTxnError{} +func (e *InternalRetryableTxnError) Error() string { + return e.Msg +} + +var _ error = &InternalRetryableTxnError{} + // NewRetryableTxnError creates a shim RetryableTxnError that // reports the given cause when converted to String(). This can be // used to fake/force a retry at the SQL layer. @@ -93,6 +107,7 @@ func NewError(err error) *Error { } else { e.setGoError(err) } + return e } @@ -150,6 +165,10 @@ func (e *Error) GoError() error { return nil } if e.TransactionRestart != TransactionRestart_NONE { + if tErr, ok := e.GetDetail().(*HandledRetryableError); ok { + return &RetryableTxnError{Msg: tErr.Msg, TxnID: tErr.TxnID} + } + var txnID *uuid.UUID if e.GetTxn() != nil { txnID = e.GetTxn().ID @@ -158,14 +177,61 @@ func (e *Error) GoError() error { // TransactionAbortedError will not carry a Transaction, signaling to the // recipient to start a brand new txn. txn := e.GetTxn() - if _, ok := e.GetDetail().(*TransactionAbortedError); ok { - txn = nil + retryPriority := int32(-1) + retryTimestamp := hlc.MinTimestamp + if txn != nil { + txnClone := txn.Clone() + txn = &txnClone + switch tErr := e.GetDetail().(type) { + case *TransactionAbortedError: + retryTimestamp = txn.Timestamp + retryPriority = txn.Priority + txn = nil + case *ReadWithinUncertaintyIntervalError: + // If the reader encountered a newer write within the uncertainty + // interval, we advance the txn's timestamp just past the last observed + // timestamp from the node. + ts, ok := txn.GetObservedTimestamp(e.OriginNode) + if !ok { + return errors.Wrapf( + tErr, + "no observed timestamp for node %d found on uncertainty restart", + e.OriginNode) + } + txn.Timestamp.Forward(ts) + case *TransactionPushError: + // Increase timestamp if applicable, ensuring that we're just ahead of + // the pushee. + txn.Timestamp.Forward(tErr.PusheeTxn.Timestamp) + txn.UpgradePriority(tErr.PusheeTxn.Priority - 1) + case *TransactionRetryError: + // Nothing to do. Transaction.Timestamp has already been forwarded to be + // ahead of any timestamp cache entries or newer versions which caused + // the restart. + case *WriteTooOldError: + // Increase the timestamp to the ts at which we've actually written. + txn.Timestamp.Forward(tErr.ActualTimestamp) + default: + // Assert that we've covered all the retryable errors. + if _, ok := tErr.(transactionRestartError); ok { + return errors.Wrapf(tErr, "retryable error of type %T not handled in "+ + "pErr.GoErr(): %s", tErr, tErr) + } + } } - return &RetryableTxnError{ + + err := &InternalRetryableTxnError{ Msg: e.Message, TxnID: txnID, Transaction: txn, } + if retryPriority != -1 { + err.RetryPriority = &retryPriority + } + if retryTimestamp != hlc.MinTimestamp { + err.RetryTimestamp = &retryTimestamp + } + return err } return e.GetDetail() } @@ -351,11 +417,11 @@ func (e *AmbiguousResultError) message(_ *Error) string { var _ ErrorDetailInterface = &AmbiguousResultError{} func (e *TransactionAbortedError) Error() string { - return "txn aborted" + return "TransactionAbortedError: txn aborted" } func (e *TransactionAbortedError) message(pErr *Error) string { - return fmt.Sprintf("txn aborted %s", pErr.GetTxn()) + return fmt.Sprintf("TransactionAbortedError: txn aborted %s", pErr.GetTxn()) } func (*TransactionAbortedError) canRestartTransaction() TransactionRestart { @@ -365,11 +431,34 @@ func (*TransactionAbortedError) canRestartTransaction() TransactionRestart { var _ ErrorDetailInterface = &TransactionAbortedError{} var _ transactionRestartError = &TransactionAbortedError{} +func (e *HandledRetryableError) Error() string { + return e.message(nil) +} + +func (e *HandledRetryableError) message(_ *Error) string { + return fmt.Sprintf("HandledRetryableError: %s", e.Msg) +} + +func (*HandledRetryableError) canRestartTransaction() TransactionRestart { + // TODO(andrei): make this dependent on the type of the original error + return TransactionRestart_IMMEDIATE +} + +var _ ErrorDetailInterface = &HandledRetryableError{} +var _ transactionRestartError = &HandledRetryableError{} + // NewTransactionAbortedError initializes a new TransactionAbortedError. func NewTransactionAbortedError() *TransactionAbortedError { return &TransactionAbortedError{} } +// NewHandledRetryableError initializes a new HandledRetryableError. +func NewHandledRetryableError( + msg string, txnID *uuid.UUID, txn Transaction, +) *HandledRetryableError { + return &HandledRetryableError{Msg: msg, TxnID: txnID, Transaction: &txn} +} + // NewTransactionPushError initializes a new TransactionPushError. // The argument is copied. func NewTransactionPushError(pusheeTxn Transaction) *TransactionPushError { @@ -401,14 +490,12 @@ func NewTransactionRetryError() *TransactionRetryError { return &TransactionRetryError{} } -// TODO(kaneda): Delete this method once we fully unimplement error for every -// error detail. func (e *TransactionRetryError) Error() string { - return fmt.Sprintf("retry txn") + return fmt.Sprintf("TransactionRetryError: retry txn") } func (e *TransactionRetryError) message(pErr *Error) string { - return fmt.Sprintf("retry txn %s", pErr.GetTxn()) + return fmt.Sprintf("TransactionRetryError: retry txn %s", pErr.GetTxn()) } var _ ErrorDetailInterface = &TransactionRetryError{} @@ -468,7 +555,8 @@ func (e *WriteTooOldError) Error() string { } func (e *WriteTooOldError) message(_ *Error) string { - return fmt.Sprintf("write at timestamp %s too old; wrote at %s", e.Timestamp, e.ActualTimestamp) + return fmt.Sprintf("WriteTooOldError: write at timestamp %s too old; wrote at %s", + e.Timestamp, e.ActualTimestamp) } var _ ErrorDetailInterface = &WriteTooOldError{} @@ -495,7 +583,9 @@ func (e *ReadWithinUncertaintyIntervalError) Error() string { } func (e *ReadWithinUncertaintyIntervalError) message(_ *Error) string { - return fmt.Sprintf("read at time %s encountered previous write with future timestamp %s within uncertainty interval", e.ReadTimestamp, e.ExistingTimestamp) + return fmt.Sprintf("ReadWithinUncertaintyIntervalError: read at time %s encountered "+ + "previous write with future timestamp %s within uncertainty interval", + e.ReadTimestamp, e.ExistingTimestamp) } var _ ErrorDetailInterface = &ReadWithinUncertaintyIntervalError{} diff --git a/pkg/roachpb/errors.pb.go b/pkg/roachpb/errors.pb.go index 03fdc798e043..38f58a404e8c 100644 --- a/pkg/roachpb/errors.pb.go +++ b/pkg/roachpb/errors.pb.go @@ -375,6 +375,7 @@ type ErrorDetail struct { Send *SendError `protobuf:"bytes,15,opt,name=send" json:"send,omitempty"` AmbiguousResult *AmbiguousResultError `protobuf:"bytes,26,opt,name=ambiguous_result,json=ambiguousResult" json:"ambiguous_result,omitempty"` StoreNotFound *StoreNotFoundError `protobuf:"bytes,27,opt,name=store_not_found,json=storeNotFound" json:"store_not_found,omitempty"` + HandledRetryableError *HandledRetryableError `protobuf:"bytes,28,opt,name=handled_retryable_error,json=handledRetryableError" json:"handled_retryable_error,omitempty"` // TODO(kaneda): Following are added to preserve the type when // converting Go errors from/to proto Errors. Revisit this design. RaftGroupDeleted *RaftGroupDeletedError `protobuf:"bytes,16,opt,name=raft_group_deleted,json=raftGroupDeleted" json:"raft_group_deleted,omitempty"` @@ -411,9 +412,6 @@ type Error struct { // of retryable errors. // // Not to be accessed directly - use Error.GetTxn(). - // - // NOTE: TxnCoordSender currently has a role in updating the epoch on the - // return path for most retryable errors. UnexposedTxn *Transaction `protobuf:"bytes,4,opt,name=unexposed_txn,json=unexposedTxn" json:"unexposed_txn,omitempty"` // Node at which the error was generated (zero if does not apply). OriginNode NodeID `protobuf:"varint,5,opt,name=origin_node,json=originNode,casttype=NodeID" json:"origin_node"` @@ -432,16 +430,21 @@ func (m *Error) Reset() { *m = Error{} } func (*Error) ProtoMessage() {} func (*Error) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{23} } -// RetryableTxnError tells the recipient that the transaction it attempted to -// use to perform an operation must be retried. This is returned generally as a -// result of a transaction conflict. +// InternalRetryableTxnError tells the recipient that the transaction it +// attempted to use to perform an operation must be retried. This is returned +// generally as a result of a transaction conflict. // // This is not an ErrorDetail; it is a higher-level error than a pErr. It is -// created based on a TransactionRetryError or a TransactionAbortedError. It -// implements the go error interface and is used by the layers above KV. It is a -// proto because DistSQL needs to send it over the wire. It needs to be in this -// package because it both depends on and is depended on by roachpb. -type RetryableTxnError struct { +// created based details such as TransactionRetryError or +// TransactionAbortedError. It implements the go error interface and is used by +// the TxnCoordSender and DistSQL. It is a proto because DistSQL needs to send +// it over the wire. It needs to be in this package because it both depends on +// and is depended on by roachpb. +// +// The TxnCoordSender will transform this into a HandledRetryableError (an +// ErrorDetail), so that restart information can be passed to the next layer up +// (the client.Txn) through the client.Sender() interface. +type InternalRetryableTxnError struct { // A user-readable message. Msg string `protobuf:"bytes,1,opt,name=msg" json:"msg"` // The ID of the transaction being restarted. The client is supposed to check @@ -455,16 +458,47 @@ type RetryableTxnError struct { // If not set, the client is supposed to use a brand new Transaction. This is // set by all retryable errors except TransactionAbortedError to communicate // the new timestamp and epoch for the next attempt. + // + // When retrying, a client is supposed to call Transaction.Restart() on + // this transaction. Transaction *Transaction `protobuf:"bytes,3,opt,name=transaction" json:"transaction,omitempty"` // If set, this is a lower-bound on the priority of the restarted transaction. // This can only be set if transaction is not set. RetryPriority *int32 `protobuf:"varint,4,opt,name=retry_priority,json=retryPriority" json:"retry_priority,omitempty"` + // If set, a retry should use this timestamp. + RetryTimestamp *cockroach_util_hlc.Timestamp `protobuf:"bytes,5,opt,name=retry_timestamp,json=retryTimestamp" json:"retry_timestamp,omitempty"` } -func (m *RetryableTxnError) Reset() { *m = RetryableTxnError{} } -func (m *RetryableTxnError) String() string { return proto.CompactTextString(m) } -func (*RetryableTxnError) ProtoMessage() {} -func (*RetryableTxnError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{24} } +func (m *InternalRetryableTxnError) Reset() { *m = InternalRetryableTxnError{} } +func (m *InternalRetryableTxnError) String() string { return proto.CompactTextString(m) } +func (*InternalRetryableTxnError) ProtoMessage() {} +func (*InternalRetryableTxnError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{24} } + +// HandledRetryableError is an error detail representing a retryable error that +// has been "handled" by the TxnCoordSender. This contains the final form of +// the Transaction proto that should be used on next attempts. This error is +// handled by the client.Txn, which transforms it into a RetryableTxnError. +type HandledRetryableError struct { + // A user-readable message. + Msg string `protobuf:"bytes,1,opt,name=msg" json:"msg"` + // The ID of the transaction being restarted. The client is supposed to check + // this against the ID of its transaction and make sure the retryable error + // is meant for its level and didn't escape from some inner transaction. + // + // TODO(andrei): make this non-nullable (and thus required) if we manage to + // make Transaction.ID non-nullable. + TxnID *github_com_cockroachdb_cockroach_pkg_util_uuid.UUID `protobuf:"bytes,2,opt,name=txn_id,json=txnId,customtype=github.com/cockroachdb/cockroach/pkg/util/uuid.UUID" json:"txn_id,omitempty"` + // The Transaction that should be used by next attempts. Depending on the + // original cause of this method, this can either be the same Transaction as + // before, but with an incremented epoch and timestamp, or a completely new + // Transaction. + Transaction *Transaction `protobuf:"bytes,3,opt,name=transaction" json:"transaction,omitempty"` +} + +func (m *HandledRetryableError) Reset() { *m = HandledRetryableError{} } +func (m *HandledRetryableError) String() string { return proto.CompactTextString(m) } +func (*HandledRetryableError) ProtoMessage() {} +func (*HandledRetryableError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{25} } func init() { proto.RegisterType((*NotLeaseHolderError)(nil), "cockroach.roachpb.NotLeaseHolderError") @@ -491,7 +525,8 @@ func init() { proto.RegisterType((*ErrorDetail)(nil), "cockroach.roachpb.ErrorDetail") proto.RegisterType((*ErrPosition)(nil), "cockroach.roachpb.ErrPosition") proto.RegisterType((*Error)(nil), "cockroach.roachpb.Error") - proto.RegisterType((*RetryableTxnError)(nil), "cockroach.roachpb.RetryableTxnError") + proto.RegisterType((*InternalRetryableTxnError)(nil), "cockroach.roachpb.InternalRetryableTxnError") + proto.RegisterType((*HandledRetryableError)(nil), "cockroach.roachpb.HandledRetryableError") proto.RegisterEnum("cockroach.roachpb.TransactionRestart", TransactionRestart_name, TransactionRestart_value) } func (m *NotLeaseHolderError) Marshal() (dAtA []byte, err error) { @@ -1301,6 +1336,18 @@ func (m *ErrorDetail) MarshalTo(dAtA []byte) (int, error) { } i += n35 } + if m.HandledRetryableError != nil { + dAtA[i] = 0xe2 + i++ + dAtA[i] = 0x1 + i++ + i = encodeVarintErrors(dAtA, i, uint64(m.HandledRetryableError.Size())) + n36, err := m.HandledRetryableError.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n36 + } return i, nil } @@ -1351,11 +1398,11 @@ func (m *Error) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintErrors(dAtA, i, uint64(m.UnexposedTxn.Size())) - n36, err := m.UnexposedTxn.MarshalTo(dAtA[i:]) + n37, err := m.UnexposedTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n36 + i += n37 } dAtA[i] = 0x28 i++ @@ -1364,34 +1411,34 @@ func (m *Error) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x32 i++ i = encodeVarintErrors(dAtA, i, uint64(m.Detail.Size())) - n37, err := m.Detail.MarshalTo(dAtA[i:]) + n38, err := m.Detail.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n37 + i += n38 } if m.Index != nil { dAtA[i] = 0x3a i++ i = encodeVarintErrors(dAtA, i, uint64(m.Index.Size())) - n38, err := m.Index.MarshalTo(dAtA[i:]) + n39, err := m.Index.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n38 + i += n39 } dAtA[i] = 0x42 i++ i = encodeVarintErrors(dAtA, i, uint64(m.Now.Size())) - n39, err := m.Now.MarshalTo(dAtA[i:]) + n40, err := m.Now.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n39 + i += n40 return i, nil } -func (m *RetryableTxnError) Marshal() (dAtA []byte, err error) { +func (m *InternalRetryableTxnError) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalTo(dAtA) @@ -1401,7 +1448,7 @@ func (m *RetryableTxnError) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *RetryableTxnError) MarshalTo(dAtA []byte) (int, error) { +func (m *InternalRetryableTxnError) MarshalTo(dAtA []byte) (int, error) { var i int _ = i var l int @@ -1414,27 +1461,79 @@ func (m *RetryableTxnError) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintErrors(dAtA, i, uint64(m.TxnID.Size())) - n40, err := m.TxnID.MarshalTo(dAtA[i:]) + n41, err := m.TxnID.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n40 + i += n41 } if m.Transaction != nil { dAtA[i] = 0x1a i++ i = encodeVarintErrors(dAtA, i, uint64(m.Transaction.Size())) - n41, err := m.Transaction.MarshalTo(dAtA[i:]) + n42, err := m.Transaction.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n41 + i += n42 } if m.RetryPriority != nil { dAtA[i] = 0x20 i++ i = encodeVarintErrors(dAtA, i, uint64(*m.RetryPriority)) } + if m.RetryTimestamp != nil { + dAtA[i] = 0x2a + i++ + i = encodeVarintErrors(dAtA, i, uint64(m.RetryTimestamp.Size())) + n43, err := m.RetryTimestamp.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n43 + } + return i, nil +} + +func (m *HandledRetryableError) 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 *HandledRetryableError) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0xa + i++ + i = encodeVarintErrors(dAtA, i, uint64(len(m.Msg))) + i += copy(dAtA[i:], m.Msg) + if m.TxnID != nil { + dAtA[i] = 0x12 + i++ + i = encodeVarintErrors(dAtA, i, uint64(m.TxnID.Size())) + n44, err := m.TxnID.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n44 + } + if m.Transaction != nil { + dAtA[i] = 0x1a + i++ + i = encodeVarintErrors(dAtA, i, uint64(m.Transaction.Size())) + n45, err := m.Transaction.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n45 + } return i, nil } @@ -1749,6 +1848,10 @@ func (m *ErrorDetail) Size() (n int) { l = m.StoreNotFound.Size() n += 2 + l + sovErrors(uint64(l)) } + if m.HandledRetryableError != nil { + l = m.HandledRetryableError.Size() + n += 2 + l + sovErrors(uint64(l)) + } return n } @@ -1783,7 +1886,7 @@ func (m *Error) Size() (n int) { return n } -func (m *RetryableTxnError) Size() (n int) { +func (m *InternalRetryableTxnError) Size() (n int) { var l int _ = l l = len(m.Msg) @@ -1799,6 +1902,26 @@ func (m *RetryableTxnError) Size() (n int) { if m.RetryPriority != nil { n += 1 + sovErrors(uint64(*m.RetryPriority)) } + if m.RetryTimestamp != nil { + l = m.RetryTimestamp.Size() + n += 1 + l + sovErrors(uint64(l)) + } + return n +} + +func (m *HandledRetryableError) Size() (n int) { + var l int + _ = l + l = len(m.Msg) + n += 1 + l + sovErrors(uint64(l)) + if m.TxnID != nil { + l = m.TxnID.Size() + n += 1 + l + sovErrors(uint64(l)) + } + if m.Transaction != nil { + l = m.Transaction.Size() + n += 1 + l + sovErrors(uint64(l)) + } return n } @@ -1879,6 +2002,9 @@ func (this *ErrorDetail) GetValue() interface{} { if this.StoreNotFound != nil { return this.StoreNotFound } + if this.HandledRetryableError != nil { + return this.HandledRetryableError + } return nil } @@ -1926,6 +2052,8 @@ func (this *ErrorDetail) SetValue(value interface{}) bool { this.AmbiguousResult = vt case *StoreNotFoundError: this.StoreNotFound = vt + case *HandledRetryableError: + this.HandledRetryableError = vt default: return false } @@ -4504,6 +4632,39 @@ func (m *ErrorDetail) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 28: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field HandledRetryableError", 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.HandledRetryableError == nil { + m.HandledRetryableError = &HandledRetryableError{} + } + if err := m.HandledRetryableError.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipErrors(dAtA[iNdEx:]) @@ -4840,7 +5001,7 @@ func (m *Error) Unmarshal(dAtA []byte) error { } return nil } -func (m *RetryableTxnError) Unmarshal(dAtA []byte) error { +func (m *InternalRetryableTxnError) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -4863,10 +5024,10 @@ func (m *RetryableTxnError) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: RetryableTxnError: wiretype end group for non-group") + return fmt.Errorf("proto: InternalRetryableTxnError: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: RetryableTxnError: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: InternalRetryableTxnError: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -4983,6 +5144,183 @@ func (m *RetryableTxnError) Unmarshal(dAtA []byte) error { } } m.RetryPriority = &v + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RetryTimestamp", 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.RetryTimestamp == nil { + m.RetryTimestamp = &cockroach_util_hlc.Timestamp{} + } + if err := m.RetryTimestamp.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + 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 *HandledRetryableError) 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: HandledRetryableError: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: HandledRetryableError: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Msg", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowErrors + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthErrors + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Msg = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TxnID", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowErrors + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthErrors + } + postIndex := iNdEx + byteLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + var v github_com_cockroachdb_cockroach_pkg_util_uuid.UUID + m.TxnID = &v + if err := m.TxnID.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Transaction", 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.Transaction == nil { + m.Transaction = &Transaction{} + } + if err := m.Transaction.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipErrors(dAtA[iNdEx:]) @@ -5112,116 +5450,120 @@ var ( func init() { proto.RegisterFile("cockroach/pkg/roachpb/errors.proto", fileDescriptorErrors) } var fileDescriptorErrors = []byte{ - // 1761 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x58, 0x4f, 0x73, 0xdb, 0xc6, - 0x15, 0x17, 0x24, 0xd2, 0x14, 0x1f, 0x45, 0x91, 0xda, 0x28, 0x0a, 0xac, 0x36, 0x94, 0x82, 0xfc, - 0xa9, 0x92, 0x03, 0x99, 0x3a, 0x75, 0x67, 0xe2, 0xb6, 0x33, 0x91, 0x44, 0xc9, 0xa5, 0x15, 0xc9, - 0x9e, 0x95, 0x14, 0x77, 0x9a, 0xce, 0x60, 0x20, 0x60, 0x4d, 0xa1, 0x06, 0xb1, 0xf0, 0x62, 0x61, - 0x91, 0x97, 0x7e, 0x86, 0x1e, 0xdb, 0x5b, 0x2e, 0xbd, 0x75, 0x3a, 0xfd, 0x10, 0x3d, 0xf8, 0xd8, - 0x63, 0xa7, 0x07, 0x4d, 0xab, 0x9e, 0xfb, 0x05, 0x74, 0xca, 0xec, 0x1f, 0x90, 0xa0, 0x08, 0xd2, - 0x9c, 0xdc, 0xc0, 0xdf, 0xbe, 0xf7, 0xc3, 0xdb, 0x7d, 0xfb, 0xde, 0xef, 0x81, 0x60, 0xb9, 0xd4, - 0x7d, 0xc9, 0xa8, 0xe3, 0x5e, 0xb6, 0xa2, 0x97, 0xdd, 0x96, 0x7c, 0x8a, 0x2e, 0x5a, 0x84, 0x31, - 0xca, 0xe2, 0x66, 0xc4, 0x28, 0xa7, 0x68, 0x6d, 0x68, 0xd3, 0xd4, 0xeb, 0x9b, 0x1f, 0xe5, 0xbb, - 0xf5, 0x08, 0x77, 0x3c, 0x87, 0x3b, 0xca, 0x71, 0x73, 0x3b, 0xdf, 0x2a, 0x63, 0xf1, 0xc9, 0xb8, - 0x45, 0xc2, 0xfd, 0xa0, 0x75, 0x19, 0xb8, 0x2d, 0xee, 0xf7, 0x48, 0xcc, 0x9d, 0x5e, 0xa4, 0xed, - 0xd6, 0xbb, 0xb4, 0x4b, 0xe5, 0x63, 0x4b, 0x3c, 0x29, 0xd4, 0xfa, 0xfb, 0x22, 0xbc, 0x73, 0x42, - 0xf9, 0xd7, 0xc4, 0x89, 0xc9, 0xaf, 0x69, 0xe0, 0x11, 0x76, 0x20, 0xe2, 0x46, 0x6d, 0x28, 0x31, - 0x12, 0x05, 0xbe, 0xeb, 0x98, 0xc6, 0xb6, 0xb1, 0x53, 0x79, 0xf0, 0x51, 0x73, 0x62, 0x0b, 0x4d, - 0xac, 0x2c, 0xda, 0x24, 0x76, 0x99, 0x1f, 0x71, 0xca, 0xf6, 0x0a, 0x6f, 0xae, 0xb7, 0x16, 0x70, - 0xea, 0x8a, 0x1e, 0xc3, 0x4a, 0x20, 0x98, 0xed, 0x4b, 0x49, 0x6d, 0x2e, 0xce, 0x4f, 0x85, 0x2b, - 0xc1, 0x28, 0x26, 0xf4, 0x10, 0x96, 0x99, 0x13, 0x76, 0x89, 0xed, 0x7b, 0xe6, 0xd2, 0xb6, 0xb1, - 0xb3, 0xb4, 0xb7, 0x29, 0xde, 0x74, 0x73, 0xbd, 0x55, 0xc2, 0x02, 0xef, 0xb4, 0x6f, 0x47, 0x8f, - 0xb8, 0x24, 0x6d, 0x3b, 0x1e, 0x6a, 0x42, 0x51, 0xb2, 0x98, 0x05, 0xf9, 0x62, 0x33, 0xe7, 0xc5, - 0x72, 0xe7, 0x58, 0x99, 0xa1, 0x0f, 0x01, 0xdc, 0x24, 0xe6, 0xb4, 0x67, 0xf7, 0xe2, 0xae, 0x59, - 0xdc, 0x36, 0x76, 0xca, 0x7a, 0x4b, 0x65, 0x85, 0x1f, 0xc7, 0x5d, 0x6b, 0x03, 0xd6, 0x4f, 0xa8, - 0x47, 0xce, 0x43, 0xe7, 0xb5, 0xe3, 0x07, 0xce, 0x45, 0x40, 0xe4, 0x91, 0x59, 0x47, 0x80, 0x64, - 0x00, 0x27, 0x94, 0x1f, 0xd2, 0x24, 0xf4, 0xd4, 0x41, 0x66, 0x23, 0x37, 0xe6, 0x8e, 0xdc, 0xfa, - 0xf3, 0x22, 0xbc, 0x2b, 0xc1, 0x23, 0x32, 0x38, 0xf6, 0xe3, 0x9e, 0xc3, 0xdd, 0x4b, 0x45, 0xf8, - 0x05, 0xac, 0x31, 0xf2, 0x2a, 0x21, 0x31, 0xb7, 0x63, 0xee, 0x30, 0x6e, 0xbf, 0x24, 0x03, 0xc9, - 0xbc, 0xb2, 0x57, 0xba, 0xbd, 0xde, 0x5a, 0x3a, 0x22, 0x03, 0x5c, 0xd3, 0x16, 0xa7, 0xc2, 0xe0, - 0x88, 0x0c, 0x50, 0x0b, 0x52, 0xc8, 0x26, 0xa1, 0x27, 0x5d, 0x16, 0xc7, 0x5d, 0xaa, 0x7a, 0xfd, - 0x20, 0xf4, 0x84, 0xc3, 0x31, 0xd4, 0x7b, 0xfa, 0xb5, 0xc4, 0xb3, 0x65, 0x54, 0xf2, 0xe0, 0x2b, - 0x0f, 0xac, 0xbc, 0xec, 0x89, 0xf5, 0x4c, 0xee, 0x6a, 0x23, 0x5f, 0xb9, 0x84, 0x8e, 0xa0, 0x16, - 0x27, 0xdd, 0x2e, 0x89, 0xf9, 0x90, 0xad, 0x30, 0x37, 0xdb, 0xea, 0xd0, 0x55, 0xae, 0x58, 0xff, - 0x30, 0xc0, 0xc2, 0xc4, 0xf1, 0x9e, 0xfb, 0xfc, 0xd2, 0x0f, 0xcf, 0x43, 0x97, 0x30, 0xee, 0xf8, - 0x21, 0x1f, 0x74, 0x42, 0x4e, 0xd8, 0x6b, 0x27, 0x50, 0x07, 0xf5, 0x04, 0x56, 0x19, 0x71, 0x3c, - 0x7b, 0x58, 0x08, 0xfa, 0x26, 0xbf, 0x9f, 0x79, 0xa5, 0xa8, 0x96, 0xe6, 0x65, 0xe0, 0x36, 0xcf, - 0x52, 0x23, 0x9d, 0xef, 0xaa, 0x70, 0x1d, 0x82, 0x08, 0x03, 0x22, 0x7d, 0x3f, 0xe6, 0x7e, 0xd8, - 0xcd, 0xf0, 0x2d, 0xce, 0xcf, 0xb7, 0x96, 0xba, 0x0f, 0x17, 0xac, 0xfb, 0xf0, 0xde, 0x19, 0x73, - 0xc2, 0xd8, 0x71, 0xb9, 0x4f, 0xc3, 0xdd, 0x0b, 0xca, 0x38, 0x51, 0x97, 0xc6, 0xfa, 0x16, 0xd6, - 0x33, 0x4b, 0xcf, 0x92, 0x58, 0xe7, 0x7e, 0x1f, 0x20, 0x4a, 0xe2, 0x4b, 0x42, 0x6c, 0xde, 0x0f, - 0xf5, 0x76, 0x1a, 0x39, 0x27, 0x98, 0x71, 0x4e, 0xef, 0xaf, 0xf2, 0x3b, 0xeb, 0x87, 0xd6, 0x7b, - 0xf0, 0x6e, 0x66, 0x1d, 0x13, 0xce, 0x06, 0xea, 0xad, 0x26, 0x6c, 0x8c, 0x2d, 0x44, 0x81, 0xa3, - 0x57, 0x3e, 0x1f, 0x5b, 0x39, 0xe5, 0x0e, 0x4f, 0x62, 0x15, 0xd1, 0x06, 0x2c, 0x89, 0x52, 0x31, - 0x32, 0xa5, 0x22, 0x00, 0xeb, 0x14, 0xea, 0xcf, 0x99, 0xcf, 0x89, 0x48, 0x49, 0xc8, 0x95, 0xed, - 0x97, 0x50, 0xf2, 0xe5, 0xcf, 0xd8, 0x34, 0xb6, 0x97, 0x76, 0x2a, 0x0f, 0xee, 0xe7, 0x84, 0xae, - 0x1c, 0xd2, 0x46, 0xa2, 0xed, 0x9f, 0x14, 0x96, 0x17, 0xeb, 0x4b, 0xd6, 0x5f, 0x0c, 0xcd, 0x7a, - 0x46, 0xe9, 0xd3, 0x40, 0x17, 0xd8, 0x2e, 0x94, 0x7f, 0x50, 0x86, 0x47, 0x5e, 0xe8, 0x04, 0xea, - 0x8e, 0xcb, 0x13, 0x27, 0xf8, 0x61, 0xb9, 0xad, 0x29, 0xe7, 0x51, 0x66, 0xd7, 0x01, 0x3d, 0x8d, - 0x30, 0x79, 0x95, 0xf8, 0x8c, 0xc4, 0x67, 0xfd, 0x50, 0x1d, 0xe2, 0x29, 0xac, 0xef, 0xd3, 0xd0, - 0xf3, 0xc5, 0x11, 0x1e, 0x3a, 0x7e, 0xa0, 0x93, 0x8d, 0x7e, 0x01, 0x2b, 0xfa, 0xed, 0xaf, 0x9d, - 0x20, 0x21, 0x7a, 0x0f, 0x79, 0xbd, 0xea, 0x1b, 0xb1, 0x8e, 0x2b, 0xca, 0x5a, 0xfe, 0xb0, 0xfe, - 0x66, 0x00, 0x52, 0x2d, 0x8c, 0xfc, 0x9e, 0xb8, 0xe9, 0x05, 0x42, 0x0d, 0x28, 0xf5, 0x48, 0x1c, - 0x3b, 0x5d, 0x32, 0x96, 0x9a, 0x14, 0x44, 0xbf, 0x84, 0xb2, 0xae, 0x77, 0xe2, 0xe9, 0xad, 0x4e, - 0x6d, 0x8e, 0xe9, 0x79, 0x0d, 0x1d, 0xd0, 0x23, 0x58, 0x4e, 0xaf, 0xb3, 0x6e, 0x0a, 0x6f, 0x73, - 0x1e, 0xda, 0x5b, 0x3f, 0x85, 0xf2, 0x29, 0x09, 0xe7, 0x0b, 0x53, 0xa7, 0xfd, 0x15, 0xac, 0xef, - 0xf6, 0x2e, 0xfc, 0x6e, 0x42, 0x93, 0x18, 0x93, 0x38, 0x09, 0xf8, 0x7c, 0x9b, 0xfc, 0x12, 0x2a, - 0x57, 0xcc, 0x89, 0x22, 0xe2, 0xd9, 0x84, 0xb1, 0x19, 0xdb, 0x94, 0x74, 0x18, 0xb4, 0xf1, 0x01, - 0x63, 0xa2, 0x46, 0xb0, 0xf3, 0x82, 0x3f, 0x66, 0x34, 0x89, 0xda, 0x24, 0x20, 0xc3, 0xca, 0xb4, - 0x61, 0x43, 0x4b, 0xd5, 0x3e, 0x65, 0x2c, 0x89, 0x44, 0x32, 0x55, 0x34, 0x1f, 0x40, 0x59, 0x4a, - 0xbe, 0x7d, 0xb7, 0x1e, 0x96, 0x25, 0x7c, 0x1c, 0x77, 0x91, 0x05, 0xe5, 0x88, 0x51, 0x97, 0xc4, - 0xb1, 0x3e, 0xf5, 0xe5, 0x61, 0x75, 0xa6, 0xb0, 0x75, 0x0a, 0x48, 0xbf, 0x20, 0x7b, 0xc9, 0x7f, - 0x05, 0xa0, 0x35, 0x35, 0xd5, 0x91, 0xe2, 0x5e, 0x43, 0xeb, 0x48, 0x59, 0xdb, 0x4b, 0x25, 0x19, - 0xfd, 0x10, 0x09, 0x53, 0x8f, 0x9e, 0x90, 0xa6, 0x53, 0x4e, 0xd9, 0xa4, 0x34, 0xc5, 0x02, 0xcd, - 0x91, 0x26, 0x69, 0xad, 0xa4, 0x49, 0x3f, 0xe2, 0x92, 0xb4, 0xed, 0x78, 0xd6, 0x6d, 0x15, 0x2a, - 0x92, 0xa0, 0x4d, 0xb8, 0xe3, 0x07, 0xe8, 0x19, 0xd4, 0x43, 0xca, 0xed, 0x31, 0xa1, 0x57, 0x77, - 0xf8, 0x93, 0x9c, 0xb3, 0xce, 0x19, 0x36, 0xf0, 0x6a, 0x38, 0x06, 0xa2, 0x63, 0xa8, 0x29, 0xcd, - 0x14, 0xbc, 0x2f, 0x44, 0xc0, 0x3a, 0x79, 0x1f, 0x4f, 0x53, 0x8b, 0xb1, 0x8d, 0xe1, 0x2a, 0xcb, - 0x62, 0xe8, 0x1b, 0x40, 0x8a, 0xee, 0x25, 0x19, 0xd8, 0xa9, 0x32, 0xe9, 0x8b, 0xbb, 0x33, 0x8d, - 0xf1, 0xae, 0xee, 0xe2, 0x3a, 0xbb, 0x03, 0xa3, 0x3f, 0xc0, 0xb6, 0x14, 0x98, 0x2b, 0xa9, 0x43, - 0x76, 0x32, 0x12, 0x22, 0xdb, 0xd7, 0x4a, 0xa4, 0x55, 0xee, 0x61, 0xee, 0xc4, 0xf3, 0x36, 0x05, - 0xc3, 0xef, 0xb3, 0x59, 0x36, 0xe8, 0x5b, 0x78, 0x87, 0x8f, 0xba, 0xb2, 0xed, 0x28, 0x05, 0x91, - 0x63, 0x4b, 0xe5, 0xc1, 0x67, 0xb3, 0x65, 0x21, 0x2b, 0x37, 0x18, 0xf1, 0x89, 0x05, 0x84, 0xa1, - 0x9e, 0x25, 0x17, 0xf2, 0x61, 0xde, 0x93, 0xcc, 0x3f, 0x99, 0xcd, 0x3c, 0x54, 0x2b, 0x5c, 0xe3, - 0xe3, 0x28, 0x3a, 0x87, 0xb5, 0x2c, 0x27, 0x13, 0xd2, 0x63, 0x96, 0xa6, 0xe6, 0x21, 0x57, 0xa5, - 0x70, 0x36, 0x2c, 0x09, 0xa3, 0xdf, 0x40, 0x76, 0x03, 0x62, 0x2a, 0xe2, 0x49, 0x6c, 0x2e, 0x4b, - 0xde, 0x4f, 0x67, 0xf3, 0x66, 0xa4, 0x0c, 0x67, 0x63, 0x53, 0x38, 0x3a, 0x84, 0x95, 0x2b, 0xa1, - 0x37, 0xb6, 0xd2, 0x21, 0xb3, 0x2c, 0x39, 0x3f, 0xcc, 0xe1, 0xbc, 0x2b, 0x76, 0xb8, 0x72, 0x35, - 0x42, 0xd0, 0x63, 0xa8, 0x2a, 0x1e, 0x4e, 0xa9, 0x4d, 0x03, 0xcf, 0x84, 0xd9, 0x44, 0x99, 0xd2, - 0xd7, 0x44, 0x0a, 0x11, 0x95, 0x41, 0x23, 0x9b, 0x69, 0x69, 0x91, 0x53, 0x40, 0x65, 0x6a, 0x65, - 0x4c, 0x6a, 0x10, 0xae, 0xd2, 0x2c, 0x26, 0x92, 0xec, 0xa6, 0x92, 0x64, 0xbf, 0x90, 0x9a, 0x64, - 0xae, 0x4c, 0x4d, 0x72, 0x9e, 0x7a, 0xe1, 0x9a, 0x3b, 0x8e, 0xa2, 0xaf, 0x61, 0x55, 0xb5, 0x02, - 0xa6, 0x15, 0xc9, 0xac, 0x4e, 0x8d, 0x70, 0x52, 0xb9, 0x70, 0x35, 0xc8, 0x62, 0x22, 0xc2, 0x90, - 0x7a, 0xc4, 0x4e, 0x46, 0xd3, 0xb6, 0xb9, 0x3a, 0x35, 0xc2, 0xbc, 0xb9, 0x1c, 0xd7, 0xc2, 0x71, - 0x14, 0x7d, 0x0e, 0x85, 0x98, 0x84, 0x9e, 0x59, 0x93, 0x3c, 0x3f, 0xce, 0xe1, 0x19, 0x2a, 0x14, - 0x96, 0x96, 0xaa, 0x83, 0xbc, 0xe0, 0x76, 0x57, 0xe8, 0x81, 0xed, 0x29, 0x41, 0x30, 0xeb, 0x33, - 0x3a, 0x48, 0x8e, 0x76, 0x88, 0x0e, 0x32, 0x0e, 0x8b, 0x9b, 0x9b, 0xb6, 0x75, 0x77, 0x28, 0x27, - 0xe6, 0xda, 0xd4, 0x9b, 0x9b, 0x2f, 0x3d, 0x78, 0x8d, 0xdd, 0xc5, 0x65, 0x0b, 0xd5, 0xcc, 0xe9, - 0x9d, 0x43, 0xd3, 0x5b, 0xe8, 0x84, 0xe0, 0x88, 0xf9, 0x37, 0x83, 0xdd, 0x2d, 0x31, 0x26, 0x67, - 0x43, 0x73, 0x63, 0x9e, 0x12, 0xcb, 0xcc, 0x91, 0x63, 0x25, 0xa6, 0x70, 0x91, 0x60, 0x27, 0x15, - 0x77, 0x9b, 0x49, 0x75, 0x37, 0x37, 0xa7, 0x26, 0x38, 0x6f, 0x0e, 0xc0, 0x35, 0x67, 0x1c, 0x15, - 0x9b, 0x57, 0xc2, 0x36, 0xd2, 0x8f, 0x1f, 0x4d, 0xdd, 0xfc, 0xa4, 0x30, 0xe2, 0x6a, 0x9c, 0xc5, - 0x1e, 0x15, 0xde, 0x7c, 0xb7, 0x65, 0x3c, 0x29, 0x2c, 0xdf, 0xaf, 0x6f, 0x5a, 0x9f, 0x4a, 0xed, - 0x7b, 0x46, 0x63, 0x79, 0xe5, 0xd1, 0x26, 0x14, 0xfd, 0xd0, 0x23, 0x7d, 0x2d, 0xc9, 0x4a, 0xcd, - 0x15, 0x64, 0xfd, 0x75, 0x09, 0x8a, 0xf3, 0x0d, 0x2a, 0xbf, 0x1b, 0x6f, 0xe4, 0x8c, 0xc8, 0x0f, - 0x3b, 0xa9, 0x50, 0xab, 0xb9, 0x31, 0x8f, 0x1d, 0xaf, 0x34, 0xd6, 0x94, 0x88, 0x4f, 0xac, 0xa0, - 0x7d, 0xa8, 0x26, 0x21, 0xe9, 0x47, 0x34, 0x26, 0x9e, 0xec, 0x18, 0x85, 0x79, 0xbe, 0x1b, 0xf0, - 0xca, 0xd0, 0x49, 0x74, 0x8a, 0x16, 0x54, 0x28, 0xf3, 0xbb, 0x7e, 0x68, 0x8b, 0x6a, 0x92, 0x1a, - 0x53, 0xdc, 0x5b, 0x15, 0xef, 0xbc, 0xbd, 0xde, 0xba, 0x27, 0xea, 0xae, 0xd3, 0xc6, 0xa0, 0x4c, - 0xc4, 0x2f, 0xf4, 0x73, 0xb8, 0xe7, 0xc9, 0xf9, 0x40, 0xab, 0x46, 0x63, 0xda, 0xdc, 0xa5, 0xa6, - 0x08, 0xac, 0xad, 0xd1, 0xcf, 0xd2, 0x13, 0x2d, 0xcd, 0x72, 0x4b, 0x13, 0xa0, 0xcf, 0x1a, 0x3d, - 0x84, 0xa5, 0x90, 0x5e, 0xe9, 0x9e, 0x3f, 0xd7, 0xd0, 0x2e, 0xec, 0x1f, 0x15, 0xfe, 0xf4, 0xdd, - 0xd6, 0x82, 0x9e, 0x32, 0xff, 0x6f, 0xc0, 0x9a, 0xd4, 0x13, 0xd1, 0x23, 0xd2, 0x86, 0x39, 0xed, - 0xfb, 0x06, 0x3d, 0x87, 0x7b, 0xbc, 0x1f, 0x8a, 0xc9, 0x49, 0x7d, 0x47, 0x7f, 0xf5, 0xef, 0xeb, - 0xad, 0x2f, 0xba, 0x3e, 0xbf, 0x4c, 0x2e, 0x9a, 0x2e, 0xed, 0xb5, 0x86, 0x11, 0x78, 0x17, 0xad, - 0x9c, 0x3f, 0x68, 0x92, 0xc4, 0xf7, 0x9a, 0xe7, 0xe7, 0x9d, 0xf6, 0xcd, 0xf5, 0x56, 0xf1, 0xac, - 0x1f, 0x76, 0xda, 0xb8, 0xc8, 0xfb, 0x61, 0xc7, 0x43, 0x5f, 0x41, 0x25, 0x93, 0x43, 0x3d, 0xa5, - 0xbc, 0x2d, 0x57, 0x59, 0x17, 0xf4, 0xb1, 0xf8, 0xee, 0xe5, 0x6c, 0x60, 0x47, 0xcc, 0xa7, 0xcc, - 0xe7, 0x03, 0x99, 0xf0, 0xa2, 0x28, 0x69, 0xce, 0x06, 0xcf, 0x34, 0xf8, 0xd9, 0x23, 0x40, 0x93, - 0xd7, 0x08, 0x2d, 0x43, 0xe1, 0xe4, 0xe9, 0xc9, 0x41, 0x7d, 0x01, 0x55, 0xa0, 0xb4, 0xb7, 0xbb, - 0x7f, 0xf4, 0xf4, 0xf0, 0xb0, 0x6e, 0xa0, 0x2a, 0x94, 0x3b, 0xc7, 0xc7, 0x07, 0xed, 0xce, 0xee, - 0xd9, 0x41, 0x7d, 0x71, 0xef, 0x83, 0x37, 0xff, 0x6d, 0x2c, 0xbc, 0xb9, 0x69, 0x18, 0xff, 0xbc, - 0x69, 0x18, 0xff, 0xba, 0x69, 0x18, 0xff, 0xb9, 0x69, 0x18, 0x7f, 0xfc, 0x5f, 0x63, 0xe1, 0xb7, - 0x25, 0x1d, 0xde, 0xf7, 0x01, 0x00, 0x00, 0xff, 0xff, 0xd8, 0x81, 0x12, 0xf2, 0x16, 0x13, 0x00, - 0x00, + // 1827 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x58, 0x4f, 0x6f, 0x1b, 0xc7, + 0x15, 0x17, 0x25, 0xd2, 0x14, 0x1f, 0x45, 0x91, 0x9a, 0xc8, 0xf2, 0x5a, 0x4d, 0x28, 0x65, 0xf3, + 0xa7, 0x4a, 0x0e, 0x62, 0xea, 0xd4, 0x05, 0xe2, 0xb6, 0x40, 0x24, 0x51, 0x72, 0x68, 0x45, 0xb2, + 0x31, 0x92, 0xe2, 0xa2, 0x29, 0xb0, 0x1d, 0xed, 0x8e, 0xc9, 0xad, 0x97, 0x3b, 0xf4, 0xec, 0xac, + 0x45, 0x5e, 0xfa, 0x19, 0x7a, 0x6c, 0x6f, 0xb9, 0xf4, 0x56, 0x14, 0xfd, 0x02, 0xbd, 0xf5, 0x60, + 0xf4, 0xd4, 0x63, 0x51, 0x14, 0x42, 0xab, 0x7e, 0x0b, 0x9f, 0x8a, 0xf9, 0xb3, 0xe4, 0x52, 0x5c, + 0xca, 0x84, 0x6f, 0xb9, 0x2d, 0x7f, 0xf3, 0xde, 0x6f, 0xfe, 0xbc, 0x79, 0xef, 0xf7, 0x86, 0x60, + 0xbb, 0xcc, 0x7d, 0xce, 0x19, 0x71, 0x3b, 0x8d, 0xde, 0xf3, 0x76, 0x43, 0x7d, 0xf5, 0xce, 0x1b, + 0x94, 0x73, 0xc6, 0xa3, 0xed, 0x1e, 0x67, 0x82, 0xa1, 0x95, 0xa1, 0xcd, 0xb6, 0x19, 0x5f, 0xff, + 0x30, 0xdb, 0xad, 0x4b, 0x05, 0xf1, 0x88, 0x20, 0xda, 0x71, 0x7d, 0x33, 0xdb, 0x2a, 0x65, 0xf1, + 0xf1, 0xb8, 0x45, 0x2c, 0xfc, 0xa0, 0xd1, 0x09, 0xdc, 0x86, 0xf0, 0xbb, 0x34, 0x12, 0xa4, 0xdb, + 0x33, 0x76, 0xab, 0x6d, 0xd6, 0x66, 0xea, 0xb3, 0x21, 0xbf, 0x34, 0x6a, 0xff, 0x65, 0x1e, 0xde, + 0x39, 0x66, 0xe2, 0x6b, 0x4a, 0x22, 0xfa, 0x15, 0x0b, 0x3c, 0xca, 0xf7, 0xe5, 0xba, 0x51, 0x13, + 0x8a, 0x9c, 0xf6, 0x02, 0xdf, 0x25, 0x56, 0x6e, 0x33, 0xb7, 0x55, 0xbe, 0xf7, 0xe1, 0xf6, 0xc4, + 0x16, 0xb6, 0xb1, 0xb6, 0x68, 0xd2, 0xc8, 0xe5, 0x7e, 0x4f, 0x30, 0xbe, 0x9b, 0x7f, 0x75, 0xb9, + 0x31, 0x87, 0x13, 0x57, 0xf4, 0x10, 0x96, 0x02, 0xc9, 0xec, 0x74, 0x14, 0xb5, 0x35, 0x3f, 0x3b, + 0x15, 0x2e, 0x07, 0xa3, 0x35, 0xa1, 0xfb, 0xb0, 0xc8, 0x49, 0xd8, 0xa6, 0x8e, 0xef, 0x59, 0x0b, + 0x9b, 0xb9, 0xad, 0x85, 0xdd, 0x75, 0x39, 0xd3, 0xd5, 0xe5, 0x46, 0x11, 0x4b, 0xbc, 0xd5, 0x7c, + 0x3d, 0xfa, 0xc4, 0x45, 0x65, 0xdb, 0xf2, 0xd0, 0x36, 0x14, 0x14, 0x8b, 0x95, 0x57, 0x13, 0x5b, + 0x19, 0x13, 0xab, 0x9d, 0x63, 0x6d, 0x86, 0x3e, 0x00, 0x70, 0xe3, 0x48, 0xb0, 0xae, 0xd3, 0x8d, + 0xda, 0x56, 0x61, 0x33, 0xb7, 0x55, 0x32, 0x5b, 0x2a, 0x69, 0xfc, 0x28, 0x6a, 0xdb, 0x6b, 0xb0, + 0x7a, 0xcc, 0x3c, 0x7a, 0x16, 0x92, 0x97, 0xc4, 0x0f, 0xc8, 0x79, 0x40, 0xd5, 0x91, 0xd9, 0x87, + 0x80, 0xd4, 0x02, 0x8e, 0x99, 0x38, 0x60, 0x71, 0xe8, 0xe9, 0x83, 0x4c, 0xaf, 0x3c, 0x37, 0xf3, + 0xca, 0xed, 0x3f, 0xcc, 0xc3, 0x6d, 0x05, 0x1e, 0xd2, 0xc1, 0x91, 0x1f, 0x75, 0x89, 0x70, 0x3b, + 0x9a, 0xf0, 0x73, 0x58, 0xe1, 0xf4, 0x45, 0x4c, 0x23, 0xe1, 0x44, 0x82, 0x70, 0xe1, 0x3c, 0xa7, + 0x03, 0xc5, 0xbc, 0xb4, 0x5b, 0x7c, 0x7d, 0xb9, 0xb1, 0x70, 0x48, 0x07, 0xb8, 0x6a, 0x2c, 0x4e, + 0xa4, 0xc1, 0x21, 0x1d, 0xa0, 0x06, 0x24, 0x90, 0x43, 0x43, 0x4f, 0xb9, 0xcc, 0x8f, 0xbb, 0x54, + 0xcc, 0xf8, 0x7e, 0xe8, 0x49, 0x87, 0x23, 0xa8, 0x75, 0xcd, 0xb4, 0xd4, 0x73, 0xd4, 0xaa, 0xd4, + 0xc1, 0x97, 0xef, 0xd9, 0x59, 0xd1, 0x93, 0xe3, 0xa9, 0xd8, 0x55, 0x47, 0xbe, 0x6a, 0x08, 0x1d, + 0x42, 0x35, 0x8a, 0xdb, 0x6d, 0x1a, 0x89, 0x21, 0x5b, 0x7e, 0x66, 0xb6, 0xe5, 0xa1, 0xab, 0x1a, + 0xb1, 0xff, 0x96, 0x03, 0x1b, 0x53, 0xe2, 0x3d, 0xf5, 0x45, 0xc7, 0x0f, 0xcf, 0x42, 0x97, 0x72, + 0x41, 0xfc, 0x50, 0x0c, 0x5a, 0xa1, 0xa0, 0xfc, 0x25, 0x09, 0xf4, 0x41, 0x3d, 0x82, 0x65, 0x4e, + 0x89, 0xe7, 0x0c, 0x13, 0xc1, 0xdc, 0xe4, 0xf7, 0x52, 0x53, 0xca, 0x6c, 0xd9, 0xee, 0x04, 0xee, + 0xf6, 0x69, 0x62, 0x64, 0xe2, 0x5d, 0x91, 0xae, 0x43, 0x10, 0x61, 0x40, 0xb4, 0xef, 0x47, 0xc2, + 0x0f, 0xdb, 0x29, 0xbe, 0xf9, 0xd9, 0xf9, 0x56, 0x12, 0xf7, 0xe1, 0x80, 0x7d, 0x17, 0xee, 0x9c, + 0x72, 0x12, 0x46, 0xc4, 0x15, 0x3e, 0x0b, 0x77, 0xce, 0x19, 0x17, 0x54, 0x5f, 0x1a, 0xfb, 0x5b, + 0x58, 0x4d, 0x0d, 0x3d, 0x89, 0x23, 0x13, 0xfb, 0x3d, 0x80, 0x5e, 0x1c, 0x75, 0x28, 0x75, 0x44, + 0x3f, 0x34, 0xdb, 0xa9, 0x67, 0x9c, 0x60, 0xca, 0x39, 0xb9, 0xbf, 0xda, 0xef, 0xb4, 0x1f, 0xda, + 0x77, 0xe0, 0x76, 0x6a, 0x1c, 0x53, 0xc1, 0x07, 0x7a, 0x56, 0x0b, 0xd6, 0xc6, 0x06, 0x7a, 0x01, + 0x31, 0x23, 0x9f, 0x8d, 0x8d, 0x9c, 0x08, 0x22, 0xe2, 0x48, 0xaf, 0x68, 0x0d, 0x16, 0x64, 0xaa, + 0xe4, 0x52, 0xa9, 0x22, 0x01, 0xfb, 0x04, 0x6a, 0x4f, 0xb9, 0x2f, 0xa8, 0x0c, 0x49, 0x28, 0xb4, + 0xed, 0x17, 0x50, 0xf4, 0xd5, 0xcf, 0xc8, 0xca, 0x6d, 0x2e, 0x6c, 0x95, 0xef, 0xdd, 0xcd, 0x58, + 0xba, 0x76, 0x48, 0x0a, 0x89, 0xb1, 0x7f, 0x94, 0x5f, 0x9c, 0xaf, 0x2d, 0xd8, 0x7f, 0xcc, 0x19, + 0xd6, 0x53, 0xc6, 0x1e, 0x07, 0x26, 0xc1, 0x76, 0xa0, 0xf4, 0x56, 0x11, 0x1e, 0x79, 0xa1, 0x63, + 0xa8, 0x11, 0x57, 0xc4, 0x24, 0x78, 0xbb, 0xd8, 0x56, 0xb5, 0xf3, 0x28, 0xb2, 0xab, 0x80, 0x1e, + 0xf7, 0x30, 0x7d, 0x11, 0xfb, 0x9c, 0x46, 0xa7, 0xfd, 0x50, 0x1f, 0xe2, 0x09, 0xac, 0xee, 0xb1, + 0xd0, 0xf3, 0xe5, 0x11, 0x1e, 0x10, 0x3f, 0x30, 0xc1, 0x46, 0x3f, 0x85, 0x25, 0x33, 0xfb, 0x4b, + 0x12, 0xc4, 0xd4, 0xec, 0x21, 0xab, 0x56, 0x7d, 0x23, 0xc7, 0x71, 0x59, 0x5b, 0xab, 0x1f, 0xf6, + 0x9f, 0x73, 0x80, 0x74, 0x09, 0xa3, 0xbf, 0xa1, 0x6e, 0x72, 0x81, 0x50, 0x1d, 0x8a, 0x5d, 0x1a, + 0x45, 0xa4, 0x4d, 0xc7, 0x42, 0x93, 0x80, 0xe8, 0x67, 0x50, 0x32, 0xf9, 0x4e, 0x3d, 0xb3, 0xd5, + 0xa9, 0xc5, 0x31, 0x39, 0xaf, 0xa1, 0x03, 0x7a, 0x00, 0x8b, 0xc9, 0x75, 0x36, 0x45, 0xe1, 0x4d, + 0xce, 0x43, 0x7b, 0xfb, 0x47, 0x50, 0x3a, 0xa1, 0xe1, 0x6c, 0xcb, 0x34, 0x61, 0x7f, 0x01, 0xab, + 0x3b, 0xdd, 0x73, 0xbf, 0x1d, 0xb3, 0x38, 0xc2, 0x34, 0x8a, 0x03, 0x31, 0xdb, 0x26, 0xbf, 0x80, + 0xf2, 0x05, 0x27, 0xbd, 0x1e, 0xf5, 0x1c, 0xca, 0xf9, 0x0d, 0xdb, 0x54, 0x74, 0x18, 0x8c, 0xf1, + 0x3e, 0xe7, 0x32, 0x47, 0x30, 0x79, 0x26, 0x1e, 0x72, 0x16, 0xf7, 0x9a, 0x34, 0xa0, 0xc3, 0xcc, + 0x74, 0x60, 0xcd, 0x48, 0xd5, 0x1e, 0xe3, 0x3c, 0xee, 0xc9, 0x60, 0xea, 0xd5, 0xbc, 0x0f, 0x25, + 0x25, 0xf9, 0xce, 0xf5, 0x7c, 0x58, 0x54, 0xf0, 0x51, 0xd4, 0x46, 0x36, 0x94, 0x7a, 0x9c, 0xb9, + 0x34, 0x8a, 0xcc, 0xa9, 0x2f, 0x0e, 0xb3, 0x33, 0x81, 0xed, 0x13, 0x40, 0x66, 0x82, 0xf4, 0x25, + 0xff, 0x39, 0x80, 0xd1, 0xd4, 0x44, 0x47, 0x0a, 0xbb, 0x75, 0xa3, 0x23, 0x25, 0x63, 0xaf, 0x94, + 0x64, 0xf4, 0x43, 0x06, 0x4c, 0x7f, 0x7a, 0x52, 0x9a, 0x4e, 0x04, 0xe3, 0x93, 0xd2, 0x14, 0x49, + 0x34, 0x43, 0x9a, 0x94, 0xb5, 0x96, 0x26, 0xf3, 0x89, 0x8b, 0xca, 0xb6, 0xe5, 0xd9, 0xff, 0x5e, + 0x86, 0xb2, 0x22, 0x68, 0x52, 0x41, 0xfc, 0x00, 0x3d, 0x81, 0x5a, 0xc8, 0x84, 0x33, 0x26, 0xf4, + 0xfa, 0x0e, 0x7f, 0x9c, 0x71, 0xd6, 0x19, 0xcd, 0x06, 0x5e, 0x0e, 0xc7, 0x40, 0x74, 0x04, 0x55, + 0xad, 0x99, 0x92, 0xf7, 0x99, 0x5c, 0xb0, 0x09, 0xde, 0x47, 0xd3, 0xd4, 0x62, 0x6c, 0x63, 0xb8, + 0xc2, 0xd3, 0x18, 0xfa, 0x06, 0x90, 0xa6, 0x7b, 0x4e, 0x07, 0x4e, 0xa2, 0x4c, 0xe6, 0xe2, 0x6e, + 0x4d, 0x63, 0xbc, 0xae, 0xbb, 0xb8, 0xc6, 0xaf, 0xc1, 0xe8, 0xb7, 0xb0, 0xa9, 0x04, 0xe6, 0x42, + 0xe9, 0x90, 0x13, 0x8f, 0x84, 0xc8, 0xf1, 0x8d, 0x12, 0x19, 0x95, 0xbb, 0x9f, 0xd9, 0xf1, 0xbc, + 0x49, 0xc1, 0xf0, 0x7b, 0xfc, 0x26, 0x1b, 0xf4, 0x2d, 0xbc, 0x23, 0x46, 0x55, 0xd9, 0x21, 0x5a, + 0x41, 0x54, 0xdb, 0x52, 0xbe, 0xf7, 0xe9, 0xcd, 0xb2, 0x90, 0x96, 0x1b, 0x8c, 0xc4, 0xc4, 0x00, + 0xc2, 0x50, 0x4b, 0x93, 0x4b, 0xf9, 0xb0, 0x6e, 0x29, 0xe6, 0x1f, 0xde, 0xcc, 0x3c, 0x54, 0x2b, + 0x5c, 0x15, 0xe3, 0x28, 0x3a, 0x83, 0x95, 0x34, 0x27, 0x97, 0xd2, 0x63, 0x15, 0xa7, 0xc6, 0x21, + 0x53, 0xa5, 0x70, 0x7a, 0x59, 0x0a, 0x46, 0xbf, 0x80, 0xf4, 0x06, 0x64, 0x57, 0x24, 0xe2, 0xc8, + 0x5a, 0x54, 0xbc, 0x9f, 0xdc, 0xcc, 0x9b, 0x92, 0x32, 0x9c, 0x5e, 0x9b, 0xc6, 0xd1, 0x01, 0x2c, + 0x5d, 0x48, 0xbd, 0x71, 0xb4, 0x0e, 0x59, 0x25, 0xc5, 0xf9, 0x41, 0x06, 0xe7, 0x75, 0xb1, 0xc3, + 0xe5, 0x8b, 0x11, 0x82, 0x1e, 0x42, 0x45, 0xf3, 0x08, 0xc6, 0x1c, 0x16, 0x78, 0x16, 0xdc, 0x4c, + 0x94, 0x4a, 0x7d, 0x43, 0xa4, 0x11, 0x99, 0x19, 0xac, 0xe7, 0x70, 0x23, 0x2d, 0xaa, 0x0b, 0x28, + 0x4f, 0xcd, 0x8c, 0x49, 0x0d, 0xc2, 0x15, 0x96, 0xc6, 0x64, 0x90, 0xdd, 0x44, 0x92, 0x9c, 0x67, + 0x4a, 0x93, 0xac, 0xa5, 0xa9, 0x41, 0xce, 0x52, 0x2f, 0x5c, 0x75, 0xc7, 0x51, 0xf4, 0x35, 0x2c, + 0xeb, 0x52, 0xc0, 0x8d, 0x22, 0x59, 0x95, 0xa9, 0x2b, 0x9c, 0x54, 0x2e, 0x5c, 0x09, 0xd2, 0x98, + 0x5c, 0x61, 0xc8, 0x3c, 0xea, 0xc4, 0xa3, 0x6e, 0xdb, 0x5a, 0x9e, 0xba, 0xc2, 0xac, 0xbe, 0x1c, + 0x57, 0xc3, 0x71, 0x14, 0x7d, 0x06, 0xf9, 0x88, 0x86, 0x9e, 0x55, 0x55, 0x3c, 0xef, 0x66, 0xf0, + 0x0c, 0x15, 0x0a, 0x2b, 0x4b, 0x5d, 0x41, 0x9e, 0x09, 0xa7, 0x2d, 0xf5, 0xc0, 0xf1, 0xb4, 0x20, + 0x58, 0xb5, 0x1b, 0x2a, 0x48, 0x86, 0x76, 0xc8, 0x0a, 0x32, 0x0e, 0xcb, 0x9b, 0x9b, 0x94, 0x75, + 0x77, 0x28, 0x27, 0xd6, 0xca, 0xd4, 0x9b, 0x9b, 0x2d, 0x3d, 0x78, 0x85, 0x5f, 0xc7, 0x55, 0x09, + 0x35, 0xcc, 0xc9, 0x9d, 0x43, 0xd3, 0x4b, 0xe8, 0x84, 0xe0, 0xc8, 0xfe, 0x37, 0x85, 0x5d, 0x4f, + 0x31, 0xae, 0x7a, 0x43, 0x6b, 0x6d, 0x96, 0x14, 0x4b, 0xf5, 0x91, 0x63, 0x29, 0xa6, 0x71, 0x19, + 0x60, 0x92, 0x88, 0xbb, 0xc3, 0x95, 0xba, 0x5b, 0xeb, 0x53, 0x03, 0x9c, 0xd5, 0x07, 0xe0, 0x2a, + 0x19, 0x47, 0xe5, 0xe6, 0xb5, 0xb0, 0x8d, 0xf4, 0xe3, 0x07, 0x53, 0x37, 0x3f, 0x29, 0x8c, 0xb8, + 0x12, 0xa5, 0x31, 0xf4, 0x6b, 0xb8, 0xd3, 0x21, 0xa1, 0x17, 0xc8, 0xa7, 0x8b, 0x2c, 0x38, 0xf2, + 0x12, 0x39, 0x4a, 0xd4, 0xad, 0x77, 0xa7, 0x5e, 0x81, 0xaf, 0xb4, 0x07, 0x4e, 0x1c, 0x34, 0xf3, + 0xed, 0x4e, 0x16, 0xfc, 0x20, 0xff, 0xea, 0xbb, 0x8d, 0xdc, 0xa3, 0xfc, 0xe2, 0xdd, 0xda, 0xba, + 0xfd, 0x89, 0x52, 0xd7, 0x27, 0x2c, 0x52, 0x49, 0x85, 0xd6, 0xa1, 0xe0, 0x87, 0x1e, 0xed, 0x1b, + 0xd1, 0xd7, 0xfd, 0x82, 0x86, 0xec, 0x3f, 0x2d, 0x40, 0x61, 0xb6, 0x56, 0xe8, 0x57, 0xe3, 0x52, + 0xc1, 0xa9, 0x7a, 0x3a, 0x2a, 0x0d, 0x5c, 0xce, 0x3c, 0x95, 0xb1, 0x00, 0x2a, 0x63, 0x43, 0x89, + 0xc4, 0xc4, 0x08, 0xda, 0x83, 0x4a, 0x1c, 0xd2, 0x7e, 0x8f, 0x45, 0xd4, 0x53, 0x35, 0x29, 0x3f, + 0xcb, 0xcb, 0x04, 0x2f, 0x0d, 0x9d, 0x64, 0x2d, 0x6a, 0x40, 0x99, 0x71, 0xbf, 0xed, 0x87, 0x8e, + 0xcc, 0x57, 0xa5, 0x62, 0x85, 0xdd, 0x65, 0x39, 0xe7, 0xeb, 0xcb, 0x8d, 0x5b, 0x32, 0xb3, 0x5b, + 0x4d, 0x0c, 0xda, 0x44, 0xfe, 0x42, 0x3f, 0x81, 0x5b, 0x9e, 0xea, 0x40, 0x8c, 0x2e, 0xd5, 0xa7, + 0x75, 0x76, 0xba, 0x4f, 0xc1, 0xc6, 0x1a, 0xfd, 0x38, 0x39, 0xd1, 0xe2, 0x4d, 0x6e, 0x49, 0x00, + 0xcc, 0x59, 0xa3, 0xfb, 0xb0, 0x10, 0xb2, 0x0b, 0xa3, 0x2a, 0x33, 0x3d, 0x0b, 0xa4, 0xfd, 0x83, + 0xfc, 0xef, 0xbf, 0xdb, 0x98, 0x33, 0x7d, 0xec, 0x5f, 0xe7, 0xe1, 0xae, 0x12, 0xef, 0x90, 0x04, + 0xc3, 0x0b, 0x90, 0x94, 0xe6, 0x69, 0x2f, 0x29, 0xf4, 0x14, 0x6e, 0x89, 0x7e, 0x28, 0x7b, 0x34, + 0xfd, 0x62, 0xff, 0xf2, 0x5f, 0x97, 0x1b, 0x9f, 0xb7, 0x7d, 0xd1, 0x89, 0xcf, 0xb7, 0x5d, 0xd6, + 0x6d, 0x0c, 0x57, 0xe2, 0x9d, 0x37, 0x32, 0xfe, 0x0a, 0x8a, 0x63, 0xdf, 0xdb, 0x3e, 0x3b, 0x6b, + 0x35, 0xaf, 0x2e, 0x37, 0x0a, 0xa7, 0xfd, 0xb0, 0xd5, 0xc4, 0x05, 0xd1, 0x0f, 0x5b, 0x1e, 0xfa, + 0x12, 0xca, 0xa9, 0x58, 0x9a, 0x7e, 0xe8, 0x4d, 0x31, 0x4b, 0xbb, 0xa0, 0x8f, 0xe4, 0x0b, 0x5b, + 0xf0, 0x81, 0xd3, 0xe3, 0x3e, 0xe3, 0xbe, 0x18, 0xa8, 0xc0, 0x17, 0x64, 0xf1, 0x10, 0x7c, 0xf0, + 0xc4, 0x80, 0xe8, 0x40, 0xd6, 0x22, 0x69, 0x36, 0x7a, 0x5d, 0x15, 0x66, 0x38, 0x46, 0xac, 0xc9, + 0x47, 0xcf, 0xaa, 0xbf, 0xe7, 0xe0, 0x76, 0x66, 0x5a, 0x7d, 0x0f, 0xcf, 0xee, 0xd3, 0x07, 0x80, + 0x26, 0x73, 0x0c, 0x2d, 0x42, 0xfe, 0xf8, 0xf1, 0xf1, 0x7e, 0x6d, 0x0e, 0x95, 0xa1, 0xb8, 0xbb, + 0xb3, 0x77, 0xf8, 0xf8, 0xe0, 0xa0, 0x96, 0x43, 0x15, 0x28, 0xb5, 0x8e, 0x8e, 0xf6, 0x9b, 0xad, + 0x9d, 0xd3, 0xfd, 0xda, 0xfc, 0xee, 0xfb, 0xaf, 0xfe, 0x5b, 0x9f, 0x7b, 0x75, 0x55, 0xcf, 0xfd, + 0xe3, 0xaa, 0x9e, 0xfb, 0xe7, 0x55, 0x3d, 0xf7, 0x9f, 0xab, 0x7a, 0xee, 0x77, 0xff, 0xab, 0xcf, + 0xfd, 0xb2, 0x68, 0xe6, 0xfd, 0x7f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x60, 0x1a, 0x52, 0x23, 0x95, + 0x14, 0x00, 0x00, } diff --git a/pkg/roachpb/errors.proto b/pkg/roachpb/errors.proto index c2f63945f2c4..e958e53b332e 100644 --- a/pkg/roachpb/errors.proto +++ b/pkg/roachpb/errors.proto @@ -239,6 +239,7 @@ message ErrorDetail { reserved 25; optional AmbiguousResultError ambiguous_result = 26; optional StoreNotFoundError store_not_found = 27; + optional HandledRetryableError handled_retryable_error = 28; // TODO(kaneda): Following are added to preserve the type when // converting Go errors from/to proto Errors. Revisit this design. @@ -290,9 +291,6 @@ message Error { // of retryable errors. // // Not to be accessed directly - use Error.GetTxn(). - // - // NOTE: TxnCoordSender currently has a role in updating the epoch on the - // return path for most retryable errors. optional Transaction unexposed_txn = 4; // Node at which the error was generated (zero if does not apply). @@ -313,19 +311,24 @@ message Error { reserved 2; } -// RetryableTxnError tells the recipient that the transaction it attempted to -// use to perform an operation must be retried. This is returned generally as a -// result of a transaction conflict. +// InternalRetryableTxnError tells the recipient that the transaction it +// attempted to use to perform an operation must be retried. This is returned +// generally as a result of a transaction conflict. // // This is not an ErrorDetail; it is a higher-level error than a pErr. It is -// created based on a TransactionRetryError or a TransactionAbortedError. It -// implements the go error interface and is used by the layers above KV. It is a -// proto because DistSQL needs to send it over the wire. It needs to be in this -// package because it both depends on and is depended on by roachpb. -message RetryableTxnError { +// created based details such as TransactionRetryError or +// TransactionAbortedError. It implements the go error interface and is used by +// the TxnCoordSender and DistSQL. It is a proto because DistSQL needs to send +// it over the wire. It needs to be in this package because it both depends on +// and is depended on by roachpb. +// +// The TxnCoordSender will transform this into a HandledRetryableError (an +// ErrorDetail), so that restart information can be passed to the next layer up +// (the client.Txn) through the client.Sender() interface. +message InternalRetryableTxnError { // A user-readable message. optional string msg = 1 [(gogoproto.nullable) = false]; - + // The ID of the transaction being restarted. The client is supposed to check // this against the ID of its transaction and make sure the retryable error // is meant for its level and didn't escape from some inner transaction. @@ -333,16 +336,60 @@ message RetryableTxnError { // TODO(andrei): make this non-nullable (and thus required) if we manage to // make Transaction.ID non-nullable. optional bytes txn_id = 2 [ - (gogoproto.customtype) = "github.com/cockroachdb/cockroach/pkg/util/uuid.UUID", - (gogoproto.customname) = "TxnID"]; + (gogoproto.customtype) = "github.com/cockroachdb/cockroach/pkg/util/uuid.UUID", + (gogoproto.customname) = "TxnID"]; // The Transaction that the client is supposed to use for the next attempt. // If not set, the client is supposed to use a brand new Transaction. This is // set by all retryable errors except TransactionAbortedError to communicate // the new timestamp and epoch for the next attempt. + // + // When retrying, a client is supposed to call Transaction.Restart() on + // this transaction. optional Transaction transaction = 3; + // The fields below are filled in when "transaction" is not present (i.e. as a + // result of a TransactionAbortedError). They're supposed to be + // used with the Transaction.Restart() call. If transaction is not set, these + // will also not be set. + // + // NOTE(andrei): I think a better design would be for the server to always be + // in charge of updating the Transaction to the final form in which it should + // be used by the client. I couldn't easily do that, though, due to the + // current mechanics through which pErrs and their details are partially + // filled in by multiple layers. + // I've also tried to apply these fields to the transaction in pErr.GoError(). + // The problem I hit in doing that was that Transaction.Restart() needs a + // request's UserPriority, and pErr.GoError() doesn't have a request. + // If set, this is a lower-bound on the priority of the restarted transaction. // This can only be set if transaction is not set. optional int32 retry_priority = 4; + // If set, a retry should use this timestamp. + optional util.hlc.Timestamp retry_timestamp = 5; +} + +// HandledRetryableError is an error detail representing a retryable error that +// has been "handled" by the TxnCoordSender. This contains the final form of +// the Transaction proto that should be used on next attempts. This error is +// handled by the client.Txn, which transforms it into a RetryableTxnError. +message HandledRetryableError { + // A user-readable message. + optional string msg = 1 [(gogoproto.nullable) = false]; + + // The ID of the transaction being restarted. The client is supposed to check + // this against the ID of its transaction and make sure the retryable error + // is meant for its level and didn't escape from some inner transaction. + // + // TODO(andrei): make this non-nullable (and thus required) if we manage to + // make Transaction.ID non-nullable. + optional bytes txn_id = 2 [ + (gogoproto.customtype) = "github.com/cockroachdb/cockroach/pkg/util/uuid.UUID", + (gogoproto.customname) = "TxnID"]; + + // The Transaction that should be used by next attempts. Depending on the + // original cause of this method, this can either be the same Transaction as + // before, but with an incremented epoch and timestamp, or a completely new + // Transaction. + optional Transaction transaction = 3; } diff --git a/pkg/roachpb/errors_test.go b/pkg/roachpb/errors_test.go index 0ba64ab9f37d..c43d3c6096fd 100644 --- a/pkg/roachpb/errors_test.go +++ b/pkg/roachpb/errors_test.go @@ -52,7 +52,7 @@ func TestSetTxn(t *testing.T) { e := NewError(NewTransactionAbortedError()) txn := NewTransaction("test", Key("a"), 1, enginepb.SERIALIZABLE, hlc.Timestamp{}, 0) e.SetTxn(txn) - if !strings.HasPrefix(e.Message, "txn aborted \"test\"") { + if !strings.HasPrefix(e.Message, "TransactionAbortedError: txn aborted \"test\"") { t.Errorf("unexpected message: %s", e.Message) } } diff --git a/pkg/server/server.go b/pkg/server/server.go index ce51cb36d809..74e6db820cd8 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -291,18 +291,19 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { // Set up Executor execCfg := sql.ExecutorConfig{ - AmbientCtx: s.cfg.AmbientCtx, - NodeID: &s.nodeIDContainer, - DB: s.db, - Gossip: s.gossip, - DistSender: s.distSender, - RPCContext: s.rpcContext, - LeaseManager: s.leaseMgr, - Clock: s.clock, - DistSQLSrv: s.distSQLServer, - HistogramWindowInterval: s.cfg.HistogramWindowInterval(), - RangeDescriptorCache: s.distSender.RangeDescriptorCache(), - LeaseHolderCache: s.distSender.LeaseHolderCache(), + AmbientCtx: s.cfg.AmbientCtx, + NodeID: &s.nodeIDContainer, + DB: s.db, + Gossip: s.gossip, + DistSender: s.distSender, + RPCContext: s.rpcContext, + LeaseManager: s.leaseMgr, + Clock: s.clock, + DistSQLSrv: s.distSQLServer, + HistogramWindowInterval: s.cfg.HistogramWindowInterval(), + RangeDescriptorCache: s.distSender.RangeDescriptorCache(), + LeaseHolderCache: s.distSender.LeaseHolderCache(), + StopHeartbeatingTransactionRecord: s.txnCoordSender.CleanupTxn, } if s.cfg.TestingKnobs.SQLExecutor != nil { execCfg.TestingKnobs = s.cfg.TestingKnobs.SQLExecutor.(*sql.ExecutorTestingKnobs) diff --git a/pkg/sql/backfill.go b/pkg/sql/backfill.go index b6faa4e75627..49ec4cec9386 100644 --- a/pkg/sql/backfill.go +++ b/pkg/sql/backfill.go @@ -414,13 +414,16 @@ func (sc *SchemaChanger) distBackfill( } // TODO(andrei): pass the right caches. I think this will crash without // them. - recv := makeDistSQLReceiver( + recv, err := makeDistSQLReceiver( ctx, nil, /* sink */ nil, /* rangeCache */ nil, /* leaseCache */ nil, /* txn - the flow does not run wholly in a txn */ ) + if err != nil { + return err + } planCtx := sc.distSQLPlanner.NewPlanningCtx(ctx, txn) plan, err := sc.distSQLPlanner.CreateBackfiller( &planCtx, backfillType, *tableDesc, duration, chunkSize, spans, otherTableDescs, diff --git a/pkg/sql/distsql_physical_planner_test.go b/pkg/sql/distsql_physical_planner_test.go index 51ad17a55044..40f355106641 100644 --- a/pkg/sql/distsql_physical_planner_test.go +++ b/pkg/sql/distsql_physical_planner_test.go @@ -283,7 +283,11 @@ func TestDistSQLReceiverUpdatesCaches(t *testing.T) { rangeCache := kv.NewRangeDescriptorCache(nil /* db */, 2<<10 /* size */) leaseCache := kv.NewLeaseHolderCache(2 << 10 /* size */) - r := makeDistSQLReceiver(context.TODO(), nil /* sink */, rangeCache, leaseCache) + r, err := makeDistSQLReceiver( + context.TODO(), nil /* sink */, rangeCache, leaseCache, nil /* txn */) + if err != nil { + t.Fatal(err) + } descs := []roachpb.RangeDescriptor{ {RangeID: 1, StartKey: roachpb.RKey("a"), EndKey: roachpb.RKey("c")}, diff --git a/pkg/sql/distsql_running.go b/pkg/sql/distsql_running.go index df96b26f13e3..d59028892cb0 100644 --- a/pkg/sql/distsql_running.go +++ b/pkg/sql/distsql_running.go @@ -247,14 +247,14 @@ func makeDistSQLReceiver( rangeCache *kv.RangeDescriptorCache, leaseCache *kv.LeaseHolderCache, txn *client.Txn, -) distSQLReceiver { +) (distSQLReceiver, error) { return distSQLReceiver{ ctx: ctx, rows: sink, rangeCache: rangeCache, leaseCache: leaseCache, txn: txn, - } + }, nil } // Push is part of the RowReceiver interface. @@ -269,7 +269,10 @@ func (r *distSQLReceiver) Push( // itself in non-error cases. Those updates are not necessary if we're // just doing reads. Once DistSQL starts performing writes, we'll need // to perform such updates too. - r.txn.UpdateStateOnErr(meta.Err) + // TODO(andrei): Figure out a story for the priority we're passing + // below. In the TxnCoordSender world, this is coming from the current + // Batch's priority. + r.txn.UpdateStateOnDetachedErr(r.ctx, meta.Err, roachpb.NormalUserPriority) } r.err = meta.Err } diff --git a/pkg/sql/distsqlrun/base.go b/pkg/sql/distsqlrun/base.go index 9f2ecca53645..3860b5e24edd 100644 --- a/pkg/sql/distsqlrun/base.go +++ b/pkg/sql/distsqlrun/base.go @@ -623,7 +623,12 @@ func (e *Error) String() string { func NewError(err error) *Error { if pgErr, ok := pgerror.GetPGCause(err); ok { return &Error{Detail: &Error_PGError{PGError: pgErr}} - } else if retryErr, ok := err.(*roachpb.RetryableTxnError); ok { + } else if retryErr, ok := err.(*roachpb.InternalRetryableTxnError); ok { + // !!! I think the cast above is impossible since DistSQL goes through a + // local TxnCoordSender which always converts these to + // HandledRetryableError. But that's not what we want; we want the global + // TxnCoordSender to do the txn restarting. So I think DistSQL should bypass + // the local txnCoordSender. return &Error{ Detail: &Error_RetryableTxnError{ RetryableTxnError: retryErr, diff --git a/pkg/sql/distsqlrun/data.pb.go b/pkg/sql/distsqlrun/data.pb.go index af510a177937..155a9d0eb77c 100644 --- a/pkg/sql/distsqlrun/data.pb.go +++ b/pkg/sql/distsqlrun/data.pb.go @@ -209,7 +209,7 @@ type Error_PGError struct { PGError *cockroach_pgerror.Error `protobuf:"bytes,1,opt,name=pg_error,json=pgError,oneof"` } type Error_RetryableTxnError struct { - RetryableTxnError *cockroach_roachpb2.RetryableTxnError `protobuf:"bytes,2,opt,name=retryableTxnError,oneof"` + RetryableTxnError *cockroach_roachpb2.InternalRetryableTxnError `protobuf:"bytes,2,opt,name=retryableTxnError,oneof"` } func (*Error_PGError) isError_Detail() {} @@ -229,7 +229,7 @@ func (m *Error) GetPGError() *cockroach_pgerror.Error { return nil } -func (m *Error) GetRetryableTxnError() *cockroach_roachpb2.RetryableTxnError { +func (m *Error) GetRetryableTxnError() *cockroach_roachpb2.InternalRetryableTxnError { if x, ok := m.GetDetail().(*Error_RetryableTxnError); ok { return x.RetryableTxnError } @@ -280,7 +280,7 @@ func _Error_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Buffer) if wire != proto.WireBytes { return true, proto.ErrInternalBadWireType } - msg := new(cockroach_roachpb2.RetryableTxnError) + msg := new(cockroach_roachpb2.InternalRetryableTxnError) err := b.DecodeMessage(msg) m.Detail = &Error_RetryableTxnError{msg} return true, err @@ -1410,7 +1410,7 @@ func (m *Error) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - v := &cockroach_roachpb2.RetryableTxnError{} + v := &cockroach_roachpb2.InternalRetryableTxnError{} if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -2911,75 +2911,75 @@ var ( func init() { proto.RegisterFile("cockroach/pkg/sql/distsqlrun/data.proto", fileDescriptorData) } var fileDescriptorData = []byte{ - // 1110 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x56, 0x4f, 0x6f, 0x1b, 0xc5, - 0x1b, 0xf6, 0xda, 0x8e, 0xff, 0xbc, 0x76, 0xfa, 0x73, 0x47, 0x3f, 0x21, 0x2b, 0x14, 0x3b, 0x5d, - 0x02, 0x04, 0x54, 0xd6, 0x6d, 0x38, 0x20, 0xe8, 0x81, 0xda, 0xf1, 0x26, 0x36, 0x24, 0x71, 0x34, - 0x4e, 0x85, 0x82, 0x90, 0x56, 0x9b, 0xdd, 0x89, 0xb3, 0xea, 0x66, 0x77, 0x33, 0x3b, 0xdb, 0xc4, - 0x17, 0x38, 0x73, 0xa2, 0x47, 0x8e, 0xfd, 0x0a, 0x7c, 0x8b, 0x1c, 0x39, 0x56, 0x20, 0x45, 0x60, - 0x3e, 0x03, 0x12, 0xe2, 0x84, 0x66, 0x76, 0xd6, 0x7f, 0xe2, 0x98, 0xb6, 0xea, 0x6d, 0x76, 0xe6, - 0x79, 0x9e, 0x79, 0xde, 0x67, 0xde, 0xf1, 0x18, 0x3e, 0xb0, 0x7c, 0xeb, 0x09, 0xf5, 0x4d, 0xeb, - 0xa4, 0x11, 0x3c, 0x19, 0x34, 0xc2, 0x33, 0xb7, 0x61, 0x3b, 0x21, 0x0b, 0xcf, 0x5c, 0x1a, 0x79, - 0x0d, 0xdb, 0x64, 0xa6, 0x16, 0x50, 0x9f, 0xf9, 0xa8, 0x3a, 0x06, 0x6a, 0xe1, 0x99, 0xab, 0x4d, - 0x40, 0x2b, 0xf5, 0x59, 0x09, 0x31, 0x0a, 0x8e, 0x1a, 0x66, 0xe0, 0xc4, 0xd4, 0x95, 0xd5, 0x9b, - 0x01, 0x13, 0xf1, 0x15, 0xf5, 0x66, 0x04, 0xa1, 0xd4, 0xa7, 0xa1, 0xc4, 0x7c, 0x3c, 0xef, 0x34, - 0x18, 0x9c, 0x3b, 0x94, 0x34, 0x82, 0x81, 0x00, 0xce, 0xc2, 0x3f, 0x9a, 0x87, 0x87, 0x67, 0xee, - 0x91, 0x19, 0x92, 0x46, 0xc8, 0x68, 0x64, 0xb1, 0x88, 0x12, 0x7b, 0xb1, 0x74, 0x82, 0x25, 0x9e, - 0xe5, 0xdb, 0xc4, 0x36, 0x6c, 0x93, 0x45, 0xa7, 0x12, 0xfe, 0xff, 0x81, 0x3f, 0xf0, 0xc5, 0xb0, - 0xc1, 0x47, 0xf1, 0xac, 0xfa, 0xb3, 0x02, 0x4b, 0x3a, 0x77, 0x80, 0x5a, 0x50, 0x08, 0x06, 0x86, - 0x70, 0x53, 0x55, 0x56, 0x95, 0xf5, 0xd2, 0x46, 0x55, 0x9b, 0xa4, 0x27, 0xdd, 0x6a, 0x02, 0xdb, - 0x2a, 0x8d, 0xae, 0xea, 0xf9, 0xfd, 0x6d, 0xf1, 0xd1, 0x49, 0xe1, 0x7c, 0x30, 0x88, 0x35, 0x0e, - 0xe0, 0x36, 0x25, 0x8c, 0x0e, 0xcd, 0x23, 0x97, 0x1c, 0x5c, 0x78, 0x62, 0xb2, 0x9a, 0x16, 0x62, - 0x6b, 0x53, 0x62, 0x32, 0x29, 0x0d, 0x5f, 0xc7, 0x76, 0x52, 0x78, 0x5e, 0xe0, 0xf3, 0xec, 0x4f, - 0xcf, 0xeb, 0xa9, 0x56, 0x01, 0x72, 0x36, 0x61, 0xa6, 0xe3, 0xaa, 0x5b, 0x00, 0xfa, 0x45, 0x40, - 0x49, 0x18, 0x3a, 0xbe, 0x87, 0x6a, 0x90, 0x7f, 0x4a, 0x28, 0x1f, 0x0a, 0xdb, 0xc5, 0x56, 0xf6, - 0xf2, 0xaa, 0x9e, 0xc2, 0xc9, 0x24, 0xaa, 0x42, 0x96, 0x5c, 0x04, 0xb1, 0x8d, 0x64, 0x51, 0xcc, - 0xa8, 0x7f, 0x2b, 0x50, 0xe8, 0x51, 0x9b, 0x50, 0xc7, 0x1b, 0xa0, 0x2e, 0xe4, 0x2d, 0xdf, 0x8d, - 0x4e, 0xbd, 0xb0, 0xaa, 0xac, 0x66, 0xd6, 0x4b, 0x1b, 0x1f, 0x6a, 0x8b, 0x7a, 0x47, 0x4b, 0x48, - 0xda, 0xa6, 0x60, 0x24, 0x3b, 0x4a, 0xfe, 0xca, 0x73, 0x05, 0x72, 0xf1, 0x0a, 0x7a, 0x47, 0xa8, - 0x1a, 0x8e, 0x7d, 0x21, 0xcc, 0x2d, 0x4b, 0x68, 0xce, 0xf2, 0xdd, 0xae, 0x7d, 0x81, 0xbe, 0x86, - 0xa2, 0xed, 0x50, 0x62, 0x31, 0xee, 0x9e, 0x1b, 0xbc, 0xb5, 0xf1, 0xc9, 0x2b, 0x6f, 0xab, 0xb5, - 0x13, 0xaa, 0x54, 0x9d, 0x68, 0xa9, 0x35, 0x28, 0x8e, 0x57, 0x51, 0x1e, 0x32, 0xcd, 0xfe, 0x66, - 0x25, 0x85, 0x0a, 0x90, 0x6d, 0xeb, 0xfd, 0xcd, 0x8a, 0xa2, 0xfe, 0xa5, 0x00, 0xea, 0x33, 0x4a, - 0xcc, 0x53, 0xdd, 0xb3, 0x03, 0xdf, 0xf1, 0x58, 0x3f, 0x20, 0x16, 0xfa, 0x0a, 0xb2, 0x6c, 0x18, - 0x10, 0xe1, 0xf5, 0xd6, 0xc6, 0x83, 0xc5, 0x56, 0xe6, 0xb9, 0xda, 0xc1, 0x30, 0x20, 0x49, 0xbc, - 0x5c, 0x04, 0x7d, 0x06, 0xc5, 0x50, 0xc0, 0x0c, 0xc7, 0x16, 0xc5, 0x2d, 0xb5, 0xee, 0xf0, 0xe5, - 0xd1, 0x55, 0xbd, 0x10, 0xf3, 0xbb, 0xed, 0x7f, 0xa6, 0xc6, 0xb8, 0x10, 0xc3, 0xbb, 0x36, 0x7a, - 0x0f, 0x4a, 0xcc, 0xa4, 0x03, 0xc2, 0x0c, 0xd3, 0xb6, 0x69, 0x35, 0x33, 0x75, 0x74, 0x10, 0x2f, - 0x34, 0x6d, 0x9b, 0xaa, 0xf7, 0x21, 0xcb, 0x77, 0x45, 0x45, 0x58, 0xda, 0xe9, 0x6d, 0x36, 0x77, - 0x2a, 0x29, 0x04, 0x90, 0xc3, 0xfa, 0x6e, 0xef, 0x40, 0xaf, 0x28, 0xe8, 0x36, 0x2c, 0xf7, 0x0f, - 0xf7, 0x36, 0x0d, 0xac, 0xf7, 0xf7, 0x7b, 0x7b, 0x7d, 0xbd, 0x92, 0x56, 0x7f, 0x4b, 0xc3, 0x72, - 0xd7, 0x0b, 0x22, 0xd6, 0x1f, 0x7a, 0x96, 0x28, 0x79, 0x6b, 0xa6, 0xe4, 0x7b, 0x8b, 0x4b, 0x9e, - 0xa1, 0xcd, 0x57, 0xdb, 0x86, 0x82, 0x2f, 0xcf, 0x47, 0x76, 0xbc, 0xfa, 0xf2, 0x93, 0x94, 0x0a, - 0x63, 0x26, 0xda, 0x81, 0x7c, 0x1c, 0x42, 0x58, 0xcd, 0x88, 0x2e, 0xbc, 0xf7, 0x3a, 0x67, 0x90, - 0x34, 0xa2, 0x94, 0x40, 0x5f, 0x42, 0x39, 0xee, 0x49, 0x83, 0x5b, 0x0c, 0xab, 0x59, 0x21, 0x79, - 0xf7, 0x9a, 0xa4, 0xfc, 0xd1, 0x90, 0x5d, 0x35, 0x55, 0x58, 0xc9, 0x1a, 0xcf, 0x84, 0xaa, 0x2a, - 0xb3, 0x5e, 0x86, 0xe2, 0xe3, 0xbd, 0x1e, 0x6e, 0xeb, 0x58, 0x6f, 0x57, 0x52, 0xa8, 0x04, 0xf9, - 0xe4, 0x43, 0x51, 0x7f, 0x4c, 0x43, 0xa5, 0x17, 0xb1, 0x20, 0x62, 0xd8, 0x8f, 0x18, 0xa1, 0x22, - 0xe0, 0xee, 0x4c, 0xc0, 0x8d, 0xff, 0x08, 0xe5, 0x1a, 0x73, 0x3e, 0xe3, 0xa9, 0x74, 0xd2, 0x6f, - 0x9e, 0xce, 0x5d, 0x28, 0x9f, 0x98, 0xe1, 0x89, 0x91, 0x5c, 0x7b, 0x1e, 0xf8, 0x32, 0x2e, 0xf1, - 0xb9, 0x38, 0x8a, 0x50, 0xfd, 0x42, 0x16, 0x5d, 0x81, 0xf2, 0x7e, 0xb3, 0xdf, 0x37, 0x0e, 0x3a, - 0xb8, 0xf7, 0x78, 0xbb, 0x13, 0xf7, 0xd9, 0x6e, 0x17, 0xe3, 0x1e, 0xae, 0x28, 0x3c, 0x83, 0xd6, - 0xa1, 0xd1, 0x69, 0xf6, 0x3b, 0x95, 0x34, 0x2a, 0x43, 0xa1, 0x75, 0x68, 0xe0, 0xe6, 0xde, 0xb6, - 0x5e, 0xc9, 0xa8, 0xcf, 0x14, 0x28, 0xb6, 0xf9, 0x8f, 0x70, 0xd7, 0x3b, 0xf6, 0xd1, 0x16, 0x14, - 0xc4, 0x2f, 0x33, 0xef, 0x91, 0x38, 0x8e, 0xb5, 0x05, 0x67, 0x21, 0x38, 0xba, 0xc4, 0x26, 0x5d, - 0x92, 0x70, 0xd1, 0x43, 0x19, 0x69, 0xdc, 0x67, 0xaf, 0x7c, 0x9e, 0x82, 0xa4, 0x7e, 0x07, 0xb7, - 0xf6, 0xa9, 0x6f, 0x47, 0x16, 0xa1, 0x1d, 0x62, 0xda, 0x84, 0xa2, 0x07, 0x90, 0x3f, 0x76, 0xfd, - 0x73, 0x7e, 0x4d, 0xb9, 0xab, 0x72, 0xab, 0xca, 0xe1, 0xbf, 0x5e, 0xd5, 0x73, 0x5b, 0xae, 0x7f, - 0xde, 0x6d, 0x8f, 0xc6, 0x23, 0x9c, 0xe3, 0xc0, 0xae, 0xfd, 0x06, 0x77, 0x5b, 0xfd, 0x1e, 0xca, - 0xc9, 0xfe, 0x6d, 0x93, 0x99, 0xe8, 0x6d, 0x28, 0x52, 0xf3, 0xdc, 0x38, 0x1a, 0x32, 0x12, 0xc6, - 0xfb, 0xe3, 0x02, 0x35, 0xcf, 0x5b, 0xfc, 0x1b, 0x61, 0x28, 0x9c, 0x12, 0x66, 0xf2, 0x47, 0x57, - 0x1e, 0xf9, 0xfd, 0xc5, 0x47, 0x8e, 0xc9, 0xa9, 0xcf, 0x48, 0x22, 0xbe, 0x2b, 0x79, 0x49, 0x7a, - 0x89, 0x8e, 0xfa, 0x42, 0x81, 0xff, 0x4d, 0x40, 0x61, 0x68, 0x0e, 0x08, 0x7a, 0x04, 0xb9, 0x13, - 0x11, 0x86, 0x7c, 0xfa, 0xd6, 0x17, 0xef, 0x32, 0x1b, 0x1e, 0x96, 0x3c, 0xd4, 0x84, 0x1c, 0x1b, - 0x06, 0xf1, 0xed, 0xe7, 0x3e, 0xdf, 0x5d, 0xac, 0x30, 0x6e, 0x88, 0xe4, 0x35, 0x88, 0x89, 0xe8, - 0x11, 0x64, 0x45, 0xa1, 0x19, 0x61, 0xe1, 0xfd, 0x97, 0x5b, 0x68, 0x4f, 0xca, 0x13, 0x4c, 0xf5, - 0x87, 0x34, 0xbc, 0x75, 0x73, 0x0a, 0xe8, 0x5b, 0x00, 0x6a, 0x7a, 0x03, 0x62, 0x38, 0xde, 0xb1, - 0x2f, 0xab, 0x7c, 0xf8, 0xba, 0x59, 0x6a, 0x98, 0x4b, 0x70, 0xeb, 0x61, 0x27, 0x85, 0x8b, 0x34, - 0xf9, 0x42, 0x9f, 0xc2, 0x12, 0x99, 0x7a, 0xec, 0xeb, 0x8b, 0x85, 0x93, 0x77, 0x3e, 0xc6, 0xaf, - 0xf4, 0x00, 0x26, 0x9a, 0xa8, 0x79, 0xcd, 0x24, 0x0f, 0xf2, 0xce, 0x4d, 0x7f, 0x1c, 0x12, 0x4a, - 0xf2, 0xf2, 0x8d, 0x9d, 0xb4, 0xf2, 0xb0, 0xf4, 0xd4, 0x74, 0x23, 0xd2, 0x5a, 0xbb, 0xfc, 0xa3, - 0x96, 0xba, 0x1c, 0xd5, 0x94, 0x5f, 0x46, 0x35, 0xe5, 0xc5, 0xa8, 0xa6, 0xfc, 0x3e, 0xaa, 0x29, - 0xcf, 0xfe, 0xac, 0xa5, 0xbe, 0x81, 0x89, 0x9d, 0x7f, 0x03, 0x00, 0x00, 0xff, 0xff, 0xe0, 0xda, - 0xe0, 0xb0, 0x4a, 0x0a, 0x00, 0x00, + // 1114 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x56, 0x4f, 0x6f, 0x1b, 0x45, + 0x1c, 0xf5, 0xda, 0x8e, 0xff, 0xfc, 0xec, 0x14, 0x77, 0x84, 0x90, 0x15, 0x8a, 0xdd, 0x2e, 0x05, + 0x02, 0x0a, 0xeb, 0x36, 0x1c, 0x10, 0xf4, 0x40, 0xed, 0x78, 0x13, 0x1b, 0x92, 0x38, 0x1a, 0xa7, + 0x42, 0x41, 0x95, 0x56, 0x93, 0xdd, 0x89, 0x63, 0x75, 0xb3, 0xbb, 0x99, 0x1d, 0x37, 0xf1, 0x05, + 0xce, 0x9c, 0xe8, 0x91, 0x63, 0xbf, 0x06, 0xdf, 0x20, 0x47, 0x8e, 0x15, 0x48, 0x11, 0x98, 0xcf, + 0x80, 0x84, 0x38, 0xa1, 0x99, 0x9d, 0xf5, 0x9f, 0x38, 0xa6, 0xad, 0x7a, 0x9b, 0x9d, 0x79, 0xef, + 0xcd, 0xfb, 0xbd, 0xf9, 0x8d, 0xc7, 0xf0, 0x91, 0xed, 0xdb, 0x4f, 0x98, 0x4f, 0xec, 0xe3, 0x5a, + 0xf0, 0xa4, 0x57, 0x0b, 0x4f, 0xdd, 0x9a, 0xd3, 0x0f, 0x79, 0x78, 0xea, 0xb2, 0x81, 0x57, 0x73, + 0x08, 0x27, 0x46, 0xc0, 0x7c, 0xee, 0xa3, 0xf2, 0x18, 0x68, 0x84, 0xa7, 0xae, 0x31, 0x01, 0xad, + 0x54, 0x67, 0x25, 0xe4, 0x28, 0x38, 0xac, 0x91, 0xa0, 0x1f, 0x51, 0x57, 0x6e, 0x5f, 0x0f, 0x98, + 0x88, 0xaf, 0xe8, 0xd7, 0x23, 0x28, 0x63, 0x3e, 0x0b, 0x15, 0xe6, 0xd3, 0x79, 0xa7, 0x41, 0xef, + 0xac, 0xcf, 0x68, 0x2d, 0xe8, 0x49, 0xe0, 0x2c, 0xfc, 0x93, 0x79, 0x78, 0x78, 0xea, 0x1e, 0x92, + 0x90, 0xd6, 0x42, 0xce, 0x06, 0x36, 0x1f, 0x30, 0xea, 0x2c, 0x96, 0x8e, 0xb1, 0xd4, 0xb3, 0x7d, + 0x87, 0x3a, 0x96, 0x43, 0xf8, 0xe0, 0x44, 0xc1, 0xdf, 0xee, 0xf9, 0x3d, 0x5f, 0x0e, 0x6b, 0x62, + 0x14, 0xcd, 0xea, 0xbf, 0x68, 0xb0, 0x64, 0x0a, 0x07, 0xa8, 0x01, 0xb9, 0xa0, 0x67, 0x49, 0x37, + 0x65, 0xed, 0xb6, 0xb6, 0x5a, 0x58, 0x2f, 0x1b, 0x93, 0xf4, 0x94, 0x5b, 0x43, 0x62, 0x1b, 0x85, + 0xd1, 0x65, 0x35, 0xbb, 0xb7, 0x25, 0x3f, 0x5a, 0x09, 0x9c, 0x0d, 0x7a, 0x91, 0xc6, 0x63, 0xb8, + 0xc9, 0x28, 0x67, 0x43, 0x72, 0xe8, 0xd2, 0xfd, 0x73, 0x4f, 0x4e, 0x96, 0x93, 0x52, 0x6c, 0x6d, + 0x4a, 0x4c, 0x25, 0x65, 0xb4, 0x3d, 0x4e, 0x99, 0x47, 0x5c, 0x7c, 0x95, 0xd3, 0x4a, 0xe0, 0x79, + 0xa1, 0x2f, 0xd3, 0x3f, 0x3f, 0xaf, 0x26, 0x1a, 0x39, 0xc8, 0x38, 0x94, 0x93, 0xbe, 0xab, 0x6f, + 0x02, 0x98, 0xe7, 0x01, 0xa3, 0x61, 0xd8, 0xf7, 0x3d, 0x54, 0x81, 0xec, 0x53, 0xca, 0xc4, 0x50, + 0xda, 0xcf, 0x37, 0xd2, 0x17, 0x97, 0xd5, 0x04, 0x8e, 0x27, 0x51, 0x19, 0xd2, 0xf4, 0x3c, 0x88, + 0xec, 0xc4, 0x8b, 0x72, 0x46, 0xff, 0x47, 0x83, 0x5c, 0x87, 0x39, 0x94, 0xf5, 0xbd, 0x1e, 0x6a, + 0x43, 0xd6, 0xf6, 0xdd, 0xc1, 0x89, 0x17, 0x96, 0xb5, 0xdb, 0xa9, 0xd5, 0xc2, 0xfa, 0xc7, 0xc6, + 0xa2, 0x1e, 0x32, 0x62, 0x92, 0xb1, 0x21, 0x19, 0xf1, 0x8e, 0x8a, 0xbf, 0xf2, 0x5c, 0x83, 0x4c, + 0xb4, 0x82, 0xde, 0x93, 0xaa, 0x56, 0xdf, 0x39, 0x97, 0xe6, 0x96, 0x15, 0x34, 0x63, 0xfb, 0x6e, + 0xdb, 0x39, 0x47, 0xdf, 0x42, 0xde, 0xe9, 0x33, 0x6a, 0x73, 0xe1, 0x5e, 0x18, 0xbc, 0xb1, 0xfe, + 0xd9, 0x2b, 0x6f, 0x6b, 0x34, 0x63, 0xaa, 0x52, 0x9d, 0x68, 0xe9, 0x15, 0xc8, 0x8f, 0x57, 0x51, + 0x16, 0x52, 0xf5, 0xee, 0x46, 0x29, 0x81, 0x72, 0x90, 0x6e, 0x9a, 0xdd, 0x8d, 0x92, 0xa6, 0xff, + 0xad, 0x01, 0xea, 0x72, 0x46, 0xc9, 0x89, 0xe9, 0x39, 0x81, 0xdf, 0xf7, 0x78, 0x37, 0xa0, 0x36, + 0xfa, 0x06, 0xd2, 0x7c, 0x18, 0x50, 0xe9, 0xf5, 0xc6, 0xfa, 0xfd, 0xc5, 0x56, 0xe6, 0xb9, 0xc6, + 0xfe, 0x30, 0xa0, 0x71, 0xbc, 0x42, 0x04, 0x7d, 0x01, 0xf9, 0x50, 0xc2, 0xac, 0xbe, 0x23, 0x8b, + 0x5b, 0x6a, 0xdc, 0x12, 0xcb, 0xa3, 0xcb, 0x6a, 0x2e, 0xe2, 0xb7, 0x9b, 0xff, 0x4e, 0x8d, 0x71, + 0x2e, 0x82, 0xb7, 0x1d, 0xf4, 0x01, 0x14, 0x38, 0x61, 0x3d, 0xca, 0x2d, 0xe2, 0x38, 0xac, 0x9c, + 0x9a, 0x3a, 0x3a, 0x88, 0x16, 0xea, 0x8e, 0xc3, 0xf4, 0x7b, 0x90, 0x16, 0xbb, 0xa2, 0x3c, 0x2c, + 0x6d, 0x77, 0x36, 0xea, 0xdb, 0xa5, 0x04, 0x02, 0xc8, 0x60, 0x73, 0xa7, 0xb3, 0x6f, 0x96, 0x34, + 0x74, 0x13, 0x96, 0xbb, 0x07, 0xbb, 0x1b, 0x16, 0x36, 0xbb, 0x7b, 0x9d, 0xdd, 0xae, 0x59, 0x4a, + 0xea, 0xbf, 0x27, 0x61, 0xb9, 0xed, 0x05, 0x03, 0xde, 0x1d, 0x7a, 0xb6, 0x2c, 0x79, 0x73, 0xa6, + 0xe4, 0xb5, 0xc5, 0x25, 0xcf, 0xd0, 0xe6, 0xab, 0x6d, 0x42, 0xce, 0x57, 0xe7, 0xa3, 0x3a, 0x5f, + 0x7f, 0xf9, 0x49, 0x2a, 0x85, 0x31, 0x13, 0x6d, 0x43, 0x36, 0x0a, 0x21, 0x2c, 0xa7, 0x64, 0x17, + 0xae, 0xbd, 0xce, 0x19, 0xc4, 0x8d, 0xa8, 0x24, 0xd0, 0xd7, 0x50, 0x8c, 0x7a, 0xd2, 0x12, 0x16, + 0xc3, 0x72, 0x5a, 0x4a, 0xde, 0xb9, 0x22, 0xa9, 0x7e, 0x3c, 0x54, 0x57, 0x4d, 0x15, 0x56, 0xb0, + 0xc7, 0x33, 0xa1, 0xae, 0xab, 0xac, 0x97, 0x21, 0xff, 0x68, 0xb7, 0x83, 0x9b, 0x26, 0x36, 0x9b, + 0xa5, 0x04, 0x2a, 0x40, 0x36, 0xfe, 0xd0, 0xf4, 0x9f, 0x92, 0x50, 0xea, 0x0c, 0x78, 0x30, 0xe0, + 0xd8, 0x1f, 0x70, 0xca, 0x64, 0xc0, 0xed, 0x99, 0x80, 0x6b, 0xff, 0x13, 0xca, 0x15, 0xe6, 0x7c, + 0xc6, 0x53, 0xe9, 0x24, 0xdf, 0x3c, 0x9d, 0x3b, 0x50, 0x3c, 0x26, 0xe1, 0xb1, 0x15, 0x5f, 0x7b, + 0x11, 0xf8, 0x32, 0x2e, 0x88, 0xb9, 0x28, 0x8a, 0x50, 0xff, 0x4a, 0x15, 0x5d, 0x82, 0xe2, 0x5e, + 0xbd, 0xdb, 0xb5, 0xf6, 0x5b, 0xb8, 0xf3, 0x68, 0xab, 0x15, 0xf5, 0xd9, 0x4e, 0x1b, 0xe3, 0x0e, + 0x2e, 0x69, 0x22, 0x83, 0xc6, 0x81, 0xd5, 0xaa, 0x77, 0x5b, 0xa5, 0x24, 0x2a, 0x42, 0xae, 0x71, + 0x60, 0xe1, 0xfa, 0xee, 0x96, 0x59, 0x4a, 0xe9, 0xcf, 0x34, 0xc8, 0x37, 0xc5, 0x8f, 0x71, 0xdb, + 0x3b, 0xf2, 0xd1, 0x26, 0xe4, 0xe4, 0x2f, 0xb4, 0xe8, 0x91, 0x28, 0x8e, 0xbb, 0x0b, 0xce, 0x42, + 0x72, 0x4c, 0x85, 0x8d, 0xbb, 0x24, 0xe6, 0xa2, 0x07, 0x2a, 0xd2, 0xa8, 0xcf, 0x5e, 0xf9, 0x3c, + 0x25, 0x49, 0xff, 0x1e, 0x6e, 0xec, 0x31, 0xdf, 0x19, 0xd8, 0x94, 0xb5, 0x28, 0x71, 0x28, 0x43, + 0xf7, 0x21, 0x7b, 0xe4, 0xfa, 0x67, 0xe2, 0x9a, 0x0a, 0x57, 0xc5, 0x46, 0x59, 0xc0, 0x7f, 0xbb, + 0xac, 0x66, 0x36, 0x5d, 0xff, 0xac, 0xdd, 0x1c, 0x8d, 0x47, 0x38, 0x23, 0x80, 0x6d, 0xe7, 0x0d, + 0xee, 0xb6, 0xfe, 0x03, 0x14, 0xe3, 0xfd, 0x9b, 0x84, 0x13, 0xf4, 0x2e, 0xe4, 0x19, 0x39, 0xb3, + 0x0e, 0x87, 0x9c, 0x86, 0xd1, 0xfe, 0x38, 0xc7, 0xc8, 0x59, 0x43, 0x7c, 0x23, 0x0c, 0xb9, 0x13, + 0xca, 0x89, 0x78, 0x7c, 0xd5, 0x91, 0xdf, 0x5b, 0x7c, 0xe4, 0x98, 0x9e, 0xf8, 0x9c, 0xc6, 0xe2, + 0x3b, 0x8a, 0x17, 0xa7, 0x17, 0xeb, 0xe8, 0x2f, 0x34, 0x78, 0x6b, 0x02, 0x0a, 0x43, 0xd2, 0xa3, + 0xe8, 0x21, 0x64, 0x8e, 0x65, 0x18, 0xea, 0x09, 0x5c, 0x5d, 0xbc, 0xcb, 0x6c, 0x78, 0x58, 0xf1, + 0x50, 0x1d, 0x32, 0x7c, 0x18, 0x44, 0xb7, 0x5f, 0xf8, 0x7c, 0x7f, 0xb1, 0xc2, 0xb8, 0x21, 0xe2, + 0xd7, 0x20, 0x22, 0xa2, 0x87, 0x90, 0x96, 0x85, 0xa6, 0xa4, 0x85, 0x0f, 0x5f, 0x6e, 0xa1, 0x39, + 0x29, 0x4f, 0x32, 0xf5, 0x1f, 0x93, 0xf0, 0xce, 0xf5, 0x29, 0xa0, 0xc7, 0x00, 0x8c, 0x78, 0x3d, + 0x6a, 0xf5, 0xbd, 0x23, 0x5f, 0x55, 0xf9, 0xe0, 0x75, 0xb3, 0x34, 0xb0, 0x90, 0x10, 0xd6, 0xc3, + 0x56, 0x02, 0xe7, 0x59, 0xfc, 0x85, 0x3e, 0x87, 0x25, 0x3a, 0xf5, 0xe8, 0x57, 0x17, 0x0b, 0xc7, + 0xef, 0x7c, 0x84, 0x5f, 0xe9, 0x00, 0x4c, 0x34, 0x51, 0xfd, 0x8a, 0x49, 0x11, 0xe4, 0xad, 0x6b, + 0xfe, 0x40, 0x8c, 0x29, 0xf1, 0xcb, 0x37, 0x76, 0xd2, 0xc8, 0xc2, 0xd2, 0x53, 0xe2, 0x0e, 0x68, + 0xe3, 0xee, 0xc5, 0x9f, 0x95, 0xc4, 0xc5, 0xa8, 0xa2, 0xfd, 0x3a, 0xaa, 0x68, 0x2f, 0x46, 0x15, + 0xed, 0x8f, 0x51, 0x45, 0x7b, 0xf6, 0x57, 0x25, 0xf1, 0x1d, 0x4c, 0xec, 0xfc, 0x17, 0x00, 0x00, + 0xff, 0xff, 0xb1, 0x4a, 0x0a, 0x32, 0x52, 0x0a, 0x00, 0x00, } diff --git a/pkg/sql/distsqlrun/data.proto b/pkg/sql/distsqlrun/data.proto index c20596261274..56cfc8e8264f 100644 --- a/pkg/sql/distsqlrun/data.proto +++ b/pkg/sql/distsqlrun/data.proto @@ -36,7 +36,7 @@ message Error { oneof detail { pgerror.Error pg_error = 1 [(gogoproto.customname) = "PGError"]; - roachpb.RetryableTxnError retryableTxnError = 2; + roachpb.InternalRetryableTxnError retryableTxnError = 2; // TODO(andrei): Add AmbiguousResultError here once DistSQL starts executing // writes. } diff --git a/pkg/sql/executor.go b/pkg/sql/executor.go index 6b372b0e30e2..eaaacf6c7c52 100644 --- a/pkg/sql/executor.go +++ b/pkg/sql/executor.go @@ -252,6 +252,17 @@ type ExecutorConfig struct { // Caches updated by DistSQL. RangeDescriptorCache *kv.RangeDescriptorCache LeaseHolderCache *kv.LeaseHolderCache + + // StopheartbeatingTransactionRecord needs to be set for the Executor to be + // able to run DistSQL queries. This will be called whenever a query running + // through DistSQL encounters a TransactionAbortedError, and so the + // Transaction proto is re-initialized and the old transaction record should + // be cleaned up. This needs to be hooked up to the TxnCoordSender's cleanup + // routine. + // + // On non-DistSQL query execution, the TxnCoordSender is directly involved and + // this indirection is not used. + StopHeartbeatingTransactionRecord func(ctx context.Context, prevTxn roachpb.Transaction) } var _ base.ModuleTestingKnobs = &ExecutorTestingKnobs{} @@ -1348,12 +1359,15 @@ func commitSQLTransaction(txnState *txnState, commitType commitType) (Result, er func (e *Executor) execDistSQL(planner *planner, tree planNode, result *Result) error { // Note: if we just want the row count, result.Rows is nil here. ctx := planner.session.Ctx() - recv := makeDistSQLReceiver( + recv, err := makeDistSQLReceiver( ctx, result.Rows, e.cfg.RangeDescriptorCache, e.cfg.LeaseHolderCache, planner.txn, ) - err := e.distSQLPlanner.PlanAndRun(ctx, planner.txn, tree, &recv) + if err != nil { + return err + } + err = e.distSQLPlanner.PlanAndRun(ctx, planner.txn, tree, &recv) if err != nil { return err } diff --git a/pkg/sql/session.go b/pkg/sql/session.go index 958a23a39503..3a4048242b0e 100644 --- a/pkg/sql/session.go +++ b/pkg/sql/session.go @@ -684,7 +684,7 @@ func (ts *txnState) updateStateAndCleanupOnErr(err error, e *Executor) { } if retErr, ok := err.(*roachpb.RetryableTxnError); !ok || !ts.willBeRetried() || - !ts.txn.IsRetryableErrMeantForTxn(retErr) { + !ts.txn.IsRetryableErrMeantForTxn(*retErr) { // We can't or don't want to retry this txn, so the txn is over. e.TxnAbortCount.Inc(1) diff --git a/pkg/sql/txn_restart_test.go b/pkg/sql/txn_restart_test.go index 87cb2e46d10d..d2bf95ca43e7 100644 --- a/pkg/sql/txn_restart_test.go +++ b/pkg/sql/txn_restart_test.go @@ -1236,6 +1236,7 @@ INSERT INTO t.test (k, v) VALUES ('test_key', 'test_val'); if err != nil { t.Fatal(err) } + // We're going to use FORCE_RETRY() to generate an error for a different // transaction than the one we initiate. We need call that function in a // transaction that already has an id (so, it can't be the first query in the @@ -1250,7 +1251,7 @@ INSERT INTO t.test (k, v) VALUES ('test_key', 'test_val'); if isRetryableErr(err) { t.Fatalf("expected non-retryable error, got: %s", err) } - if !testutils.IsError(err, "pq: retryable error from another txn: forced by crdb_internal.force_retry()") { + if !testutils.IsError(err, "pq: retryable error from another txn.* forced by crdb_internal.force_retry()") { t.Fatalf("unexpected error: %v", err) } if err := tx.Rollback(); err != nil { diff --git a/pkg/storage/client_replica_test.go b/pkg/storage/client_replica_test.go index f74eebde96e1..a21a79b423ed 100644 --- a/pkg/storage/client_replica_test.go +++ b/pkg/storage/client_replica_test.go @@ -40,6 +40,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util/caller" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/stop" @@ -258,7 +259,7 @@ func TestTxnPutOutOfOrder(t *testing.T) { return err } if !bytes.Equal(actual.ValueBytes(), updatedVal) { - t.Fatalf("unexpected get result: %s", actual) + return errors.Errorf("unexpected get result: %s", actual) } if epoch == 0 { @@ -272,7 +273,9 @@ func TestTxnPutOutOfOrder(t *testing.T) { }) if epoch != 2 { - errChan <- errors.Errorf("unexpected number of txn retries: %d", epoch) + file, line, _ := caller.Lookup(0) + errChan <- errors.Errorf("%s:%d unexpected number of txn retries. "+ + "Expected epoch 2, got: %d.", file, line, epoch) } else { errChan <- nil } diff --git a/pkg/storage/replica_command.go b/pkg/storage/replica_command.go index 7fe5ce8e1fbe..6b1a2565dcbf 100644 --- a/pkg/storage/replica_command.go +++ b/pkg/storage/replica_command.go @@ -664,6 +664,11 @@ func evalEndTransaction( } else if !ok { return EvalResult{}, roachpb.NewTransactionStatusError("does not exist") } + // We're using existingTxn on the reply, even though it can be stale compared + // to the Transaction in the request (e.g. the Sequence can be stale). This is + // OK since we're processing an EndTransaction and so there's not going to be + // more requests using the transaction from this reply (or, in case of a + // restart, we'll reset the Transaction anyway). reply.Txn = &existingTxn // Verify that we can either commit it or abort it (according