Skip to content

Commit

Permalink
Merge #56603
Browse files Browse the repository at this point in the history
56603: roachpb: use EncodedError in Error r=knz a=tbg

roachpb: use EncodedError in Error

`roachpb.Error` has long been poor to work with. It could only transport
structured errors for a hard-coded enum message (`ErrorDetail`) and had
a complex and circuituous interaction with the standard library's
`error` type: `pErr.GoError()` would return an `error` that was
sometimes just `pErr` itself (hidden behind an identical type); other
times it was an `UnhandledRetryableError` wrapping `pErr`, and then
of course sometimes it would be the structured error from which `pErr`
was created in the first place.

Thanks to `cockroachdb/errors`, there's a much better option around,
namely `errorspb.EncodedError` which has powerful capabilities to
essentially carry any error across RPC boundaries (and notably does so
"out of the box" for protobuf-backed errors) while retaining its
original structure, as long as the error type is known on both sides of
the RPC.

The simplifications that result from this primitive would be enough to
motivate this change in itself, but additionally after this change we
will be able to reliably detect context cancellations, etc, without
relying on error-prone and insecure string matching.

Going into more detail, this commit makes the following changes:

- adds an `EncodedError` to `Error`
- deprecates the following fields on `Error`, which are henceforth
  deduced from the `EncodedError` where present:
  - `transaction_restart`
  - `message`
  - `detail`
  These fields are populated for the time being so that 20.2 nodes
  understand the errors emitted by 21.1 nodes. In 21.2, we can drop
  them all.
- deprecates `(*Error).SetDetail`, which has only one real caller
  anyway (and that caller will be able to replace EncodedError instead
  once 21.2 comes around).
- makes `(*Error).SafeFormat` redact properly based on `EncodedError`
  (yay!), though there's a larger TODO that's left for a follow-up
  change (around the fact that error details can depend on the
  surrounding Error for rendering, particularly relevant when SetTxn
  is called)

We can capitalize on EncodedError before the 21.2 cycle, but this
is left for future PRs.

Fixes #54939.

Release note: None


Co-authored-by: Tobias Grieger <tobias.b.grieger@gmail.com>
  • Loading branch information
craig[bot] and tbg committed Nov 13, 2020
2 parents 8a94ee8 + 93ce5d0 commit e7a5761
Show file tree
Hide file tree
Showing 21 changed files with 496 additions and 316 deletions.
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/backup_processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -224,7 +224,7 @@ func runBackupProcessor(
span.span, span.attempts+1, header.UserPriority.String())
rawRes, pErr := kv.SendWrappedWith(ctx, flowCtx.Cfg.DB.NonTransactionalSender(), header, req)
if pErr != nil {
if err := pErr.Detail.GetWriteIntent(); err != nil {
if _, ok := pErr.GetDetail().(*roachpb.WriteIntentError); ok {
span.lastTried = timeutil.Now()
span.attempts++
todo <- span
Expand Down
6 changes: 3 additions & 3 deletions pkg/kv/kvclient/kvcoord/txn_coord_sender.go
Original file line number Diff line number Diff line change
Expand Up @@ -608,7 +608,7 @@ func (tc *TxnCoordSender) maybeRejectClientLocked(
newTxn := roachpb.PrepareTransactionForRetry(
ctx, abortedErr, roachpb.NormalUserPriority, tc.clock)
return roachpb.NewError(roachpb.NewTransactionRetryWithProtoRefreshError(
abortedErr.Message, tc.mu.txn.ID, newTxn))
abortedErr.String(), tc.mu.txn.ID, newTxn))
case protoStatus != roachpb.PENDING || hbObservedStatus != roachpb.PENDING:
// The transaction proto is in an unexpected state.
return roachpb.NewErrorf(
Expand Down Expand Up @@ -687,7 +687,7 @@ func (tc *TxnCoordSender) handleRetryableErrLocked(

// We'll pass a TransactionRetryWithProtoRefreshError up to the next layer.
retErr := roachpb.NewTransactionRetryWithProtoRefreshError(
pErr.Message,
pErr.String(),
errTxnID, // the id of the transaction that encountered the error
newTxn)

Expand Down Expand Up @@ -747,7 +747,7 @@ func (tc *TxnCoordSender) updateStateLocked(
return nil
}

if pErr.TransactionRestart != roachpb.TransactionRestart_NONE {
if pErr.TransactionRestart() != roachpb.TransactionRestart_NONE {
if tc.typ == kv.LeafTxn {
// Leaves handle retriable errors differently than roots. The leaf
// transaction is not supposed to be used any more after a retriable
Expand Down
6 changes: 2 additions & 4 deletions pkg/kv/kvserver/client_merge_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1590,9 +1590,7 @@ func TestStoreRangeMergeCheckConsistencyAfterSubsumption(t *testing.T) {
if et := r.GetEndTxn(); et != nil && et.InternalCommitTrigger.GetMergeTrigger() != nil {
mergeEndTxnReceived <- ba.Txn
<-abortMergeTxn
return &roachpb.Error{
Message: "abort the merge for test",
}
return roachpb.NewError(errors.New("abort the merge for test"))
}
}
return nil
Expand Down Expand Up @@ -1653,7 +1651,7 @@ func TestStoreRangeMergeCheckConsistencyAfterSubsumption(t *testing.T) {

pErr := <-mergeErr
require.IsType(t, &roachpb.Error{}, pErr)
require.Regexp(t, "abort the merge for test", pErr.Message)
require.Regexp(t, "abort the merge for test", pErr.String())

testutils.SucceedsSoon(t, func() error {
pErr := <-checkConsistencyResp
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/closed_timestamp_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1239,12 +1239,12 @@ func retryOnError(f func(*roachpb.Error) bool) respFunc {
}

var retryOnRangeKeyMismatch = retryOnError(func(pErr *roachpb.Error) bool {
_, isRangeKeyMismatch := pErr.Detail.Value.(*roachpb.ErrorDetail_RangeKeyMismatch)
_, isRangeKeyMismatch := pErr.GetDetail().(*roachpb.RangeKeyMismatchError)
return isRangeKeyMismatch
})

var retryOnRangeNotFound = retryOnError(func(pErr *roachpb.Error) bool {
_, isRangeNotFound := pErr.Detail.Value.(*roachpb.ErrorDetail_RangeNotFound)
_, isRangeNotFound := pErr.GetDetail().(*roachpb.RangeNotFoundError)
return isRangeNotFound
})

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -161,7 +161,7 @@ on-txn-updated txn=txn1 status=aborted
----
[-] update txn: aborting txn1
[4] sequence req1r: detected pusher aborted
[4] sequence req1r: sequencing complete, returned error: TransactionAbortedError(ABORT_REASON_PUSHER_ABORTED)
[4] sequence req1r: sequencing complete, returned error: TransactionAbortedError(ABORT_REASON_PUSHER_ABORTED): <nil>
[6] sequence req3r: resolving intent "a" for txn 00000001 with ABORTED status
[6] sequence req3r: acquiring latches
[6] sequence req3r: scanning lock table for conflicting locks
Expand Down Expand Up @@ -380,7 +380,7 @@ on-txn-updated txn=txn1 status=aborted
[4] sequence req4w: scanning lock table for conflicting locks
[4] sequence req4w: sequencing complete, returned guard
[5] sequence req1w2: detected pusher aborted
[5] sequence req1w2: sequencing complete, returned error: TransactionAbortedError(ABORT_REASON_PUSHER_ABORTED)
[5] sequence req1w2: sequencing complete, returned error: TransactionAbortedError(ABORT_REASON_PUSHER_ABORTED): <nil>
[7] sequence req3w2: resolving intent "a" for txn 00000001 with ABORTED status
[7] sequence req3w2: pushing txn 00000004 to detect request deadlock
[7] sequence req3w2: blocked on select in concurrency_test.(*cluster).PushTransaction
Expand Down Expand Up @@ -612,7 +612,7 @@ on-txn-updated txn=txn4 status=aborted
----
[-] update txn: aborting txn4
[4] sequence req4w: detected pusher aborted
[4] sequence req4w: sequencing complete, returned error: TransactionAbortedError(ABORT_REASON_PUSHER_ABORTED)
[4] sequence req4w: sequencing complete, returned error: TransactionAbortedError(ABORT_REASON_PUSHER_ABORTED): <nil>
[5] sequence req1w2: acquiring latches
[5] sequence req1w2: scanning lock table for conflicting locks
[5] sequence req1w2: sequencing complete, returned guard
Expand Down Expand Up @@ -836,7 +836,7 @@ on-txn-updated txn=txn1 status=aborted
----
[-] update txn: aborting txn1
[5] sequence req1w2: detected pusher aborted
[5] sequence req1w2: sequencing complete, returned error: TransactionAbortedError(ABORT_REASON_PUSHER_ABORTED)
[5] sequence req1w2: sequencing complete, returned error: TransactionAbortedError(ABORT_REASON_PUSHER_ABORTED): <nil>
[6] sequence req3w2: resolving intent "a" for txn 00000001 with ABORTED status
[6] sequence req3w2: acquiring latches
[6] sequence req3w2: scanning lock table for conflicting locks
Expand Down Expand Up @@ -1081,7 +1081,7 @@ on-txn-updated txn=txn4 status=aborted
----
[-] update txn: aborting txn4
[5] sequence req4w: detected pusher aborted
[5] sequence req4w: sequencing complete, returned error: TransactionAbortedError(ABORT_REASON_PUSHER_ABORTED)
[5] sequence req4w: sequencing complete, returned error: TransactionAbortedError(ABORT_REASON_PUSHER_ABORTED): <nil>
[6] sequence req3w2: acquiring latches
[6] sequence req3w2: scanning lock table for conflicting locks
[6] sequence req3w2: sequencing complete, returned guard
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -210,7 +210,7 @@ on-txn-updated txn=txn2 status=aborted
----
[-] update txn: aborting txn2
[2] handle write intent error req1: detected pusher aborted
[2] handle write intent error req1: handled conflicting intents on "k", returned error: TransactionAbortedError(ABORT_REASON_PUSHER_ABORTED)
[2] handle write intent error req1: handled conflicting intents on "k", returned error: TransactionAbortedError(ABORT_REASON_PUSHER_ABORTED): <nil>

debug-lock-table
----
Expand Down
10 changes: 7 additions & 3 deletions pkg/kv/kvserver/replica_evaluate.go
Original file line number Diff line number Diff line change
Expand Up @@ -246,10 +246,14 @@ func evaluateBatch(

// If an EndTxn wants to restart because of a write too old, we
// might have a better error to return to the client.
retErr, ok := pErr.GetDetail().(*roachpb.TransactionRetryError)
if ok && retErr.Reason == roachpb.RETRY_WRITE_TOO_OLD &&
if retErr, ok := pErr.GetDetail().(*roachpb.TransactionRetryError); ok &&
retErr.Reason == roachpb.RETRY_WRITE_TOO_OLD &&
args.Method() == roachpb.EndTxn && writeTooOldState.err != nil {
pErr.SetDetail(writeTooOldState.err)
// TODO(tbg): this is the only real caller of this method. In 21.2,
// we should be able to simply replace EncodedError. Or even better,
// we stop manually crafting and mutating errors in the way this code
// does.
pErr.DeprecatedSetDetail(writeTooOldState.err)
// Don't defer this error. We could perhaps rely on the client observing
// the WriteTooOld flag and retry the batch, but we choose not too.
writeTooOldState.cantDeferWTOE = true
Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvserver/replica_range_lease.go
Original file line number Diff line number Diff line change
Expand Up @@ -423,6 +423,7 @@ func (p *pendingLeaseRequest) requestLeaseAsync(
// Don't send the same transaction object twice; this can lead to races.
if pErr != nil {
pErrClone := *pErr
// TODO(tbg): why?
pErrClone.SetTxn(pErr.GetTxn())
llHandle.resolve(&pErrClone)
} else {
Expand Down
6 changes: 3 additions & 3 deletions pkg/kv/kvserver/replica_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2290,16 +2290,16 @@ func TestLeaseConcurrent(t *testing.T) {
pErrs[i] = <-pErrCh
}

newMsg := "moob"
newErr := errors.New("moob")
for i, pErr := range pErrs {
if withError != (pErr != nil) {
t.Errorf("%d: wanted error: %t, got error %v", i, withError, pErr)
}
if testutils.IsPError(pErr, newMsg) {
if testutils.IsPError(pErr, newErr.Error()) {
t.Errorf("%d: errors shared memory: %v", i, pErr)
} else if testutils.IsPError(pErr, origMsg) {
// Mess with anyone holding the same reference.
pErr.Message = newMsg
pErr.EncodedError = errors.EncodeError(context.Background(), newErr)
} else if pErr != nil {
t.Errorf("%d: unexpected error: %s", i, pErr)
}
Expand Down
5 changes: 5 additions & 0 deletions pkg/kv/kvserver/store_send.go
Original file line number Diff line number Diff line change
Expand Up @@ -244,6 +244,11 @@ func (s *Store) Send(
return true // continue visiting
}
t.AppendRangeInfo(ctx, desc, l)
// We have to write `t` back to `pErr` so that it picks up the changes.
//
// TODO(tbg): avoid DeprecatedSetDetail. Instead, collect the range infos
// separately and make a new pErr at the end.
pErr.DeprecatedSetDetail(t)
return true // continue visiting
})
case *roachpb.RaftGroupDeletedError:
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/txn.go
Original file line number Diff line number Diff line change
Expand Up @@ -1040,7 +1040,7 @@ func (txn *Txn) UpdateStateOnRemoteRetryableErr(ctx context.Context, pErr *roach
txn.mu.Lock()
defer txn.mu.Unlock()

if pErr.TransactionRestart == roachpb.TransactionRestart_NONE {
if pErr.TransactionRestart() == roachpb.TransactionRestart_NONE {
log.Fatalf(ctx, "unexpected non-retryable error: %s", pErr)
}

Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/txn_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -287,11 +287,11 @@ func TestRunTransactionRetryOnErrors(t *testing.T) {
pErr = roachpb.NewErrorWithTxn(test.err, ba.Txn)
}

if pErr.TransactionRestart != roachpb.TransactionRestart_NONE {
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().
return nil, roachpb.NewError(roachpb.NewTransactionRetryWithProtoRefreshError(
pErr.Message, ba.Txn.ID, *ba.Txn))
"foo", ba.Txn.ID, *ba.Txn))
}
return nil, pErr
}
Expand Down
1 change: 1 addition & 0 deletions pkg/roachpb/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,7 @@ go_library(
"//vendor/github.com/aws/aws-sdk-go/aws/credentials",
"//vendor/github.com/cockroachdb/apd/v2:apd",
"//vendor/github.com/cockroachdb/errors",
"//vendor/github.com/cockroachdb/errors/errorspb",
"//vendor/github.com/cockroachdb/redact",
"//vendor/github.com/gogo/protobuf/proto",
"//vendor/github.com/gogo/protobuf/sortkeys",
Expand Down
2 changes: 1 addition & 1 deletion pkg/roachpb/data.go
Original file line number Diff line number Diff line change
Expand Up @@ -1361,7 +1361,7 @@ func (tr *TransactionRecord) AsTransaction() Transaction {
func PrepareTransactionForRetry(
ctx context.Context, pErr *Error, pri UserPriority, clock *hlc.Clock,
) Transaction {
if pErr.TransactionRestart == TransactionRestart_NONE {
if pErr.TransactionRestart() == TransactionRestart_NONE {
log.Fatalf(ctx, "invalid retryable err (%T): %s", pErr.GetDetail(), pErr)
}

Expand Down
Loading

0 comments on commit e7a5761

Please sign in to comment.