diff --git a/pkg/kv/kvclient/kvcoord/dist_sender.go b/pkg/kv/kvclient/kvcoord/dist_sender.go index 37df0a5a9f93..f8a1d216861e 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender.go @@ -1961,17 +1961,22 @@ func (ds *DistSender) sendToReplicas( ds.clock.Update(br.Now) } - // TODO(andrei): There are errors below that cause us to move to a - // different replica without updating our caches. This means that future - // requests will attempt the same useless replicas. - switch tErr := br.Error.GetDetail().(type) { - case nil: + if br.Error == nil { // If the server gave us updated range info, lets update our cache with it. + // + // TODO(andreimatei): shouldn't we do this unconditionally? Our cache knows how + // to disregard stale information. if len(br.RangeInfos) > 0 { log.VEventf(ctx, 2, "received updated range info: %s", br.RangeInfos) routing.EvictAndReplace(ctx, br.RangeInfos...) } return br, nil + } + + // TODO(andrei): There are errors below that cause us to move to a + // different replica without updating our caches. This means that future + // requests will attempt the same useless replicas. + switch tErr := br.Error.GetDetail().(type) { case *roachpb.StoreNotFoundError, *roachpb.NodeUnavailableError: // These errors are likely to be unique to the replica that reported // them, so no action is required before the next retry. @@ -2064,7 +2069,11 @@ func (ds *DistSender) maybeIncrementErrCounters(br *roachpb.BatchResponse, err e if err != nil { ds.metrics.ErrCounts[roachpb.CommunicationErrType].Inc(1) } else { - ds.metrics.ErrCounts[br.Error.GetDetail().Type()].Inc(1) + typ := roachpb.InternalErrType + if detail := br.Error.GetDetail(); detail != nil { + typ = detail.Type() + } + ds.metrics.ErrCounts[typ].Inc(1) } } diff --git a/pkg/kv/kvclient/kvcoord/txn_coord_sender.go b/pkg/kv/kvclient/kvcoord/txn_coord_sender.go index 9d3cd6b920c8..e39372f6751e 100644 --- a/pkg/kv/kvclient/kvcoord/txn_coord_sender.go +++ b/pkg/kv/kvclient/kvcoord/txn_coord_sender.go @@ -782,7 +782,7 @@ func (tc *TxnCoordSender) updateStateLocked( // rollback), but some errors are safe to allow continuing (in particular // ConditionFailedError). In particular, SQL can recover by rolling back to a // savepoint. - if roachpb.ErrPriority(pErr.GetDetail()) != roachpb.ErrorScoreUnambiguousError { + if roachpb.ErrPriority(pErr.GoError()) != roachpb.ErrorScoreUnambiguousError { tc.mu.txnState = txnError tc.mu.storedErr = roachpb.NewError(&roachpb.TxnAlreadyEncounteredErrorError{ PrevError: pErr.String(), diff --git a/pkg/kv/kvserver/closed_timestamp_test.go b/pkg/kv/kvserver/closed_timestamp_test.go index 0e0283f71592..c741efa6fb76 100644 --- a/pkg/kv/kvserver/closed_timestamp_test.go +++ b/pkg/kv/kvserver/closed_timestamp_test.go @@ -1035,7 +1035,7 @@ func countNotLeaseHolderErrors(ba roachpb.BatchRequest, repls []*kvserver.Replic atomic.AddInt64(¬LeaseholderErrs, 1) return nil } - return pErr.GetDetail() + return pErr.GoError() } return nil }) diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index 1a7cf1c776c6..95130821a54e 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -3897,7 +3897,7 @@ func TestSerializableDeadline(t *testing.T) { if _, ok := pErr.GetDetail().(*roachpb.TransactionRetryError); !ok || !testutils.IsError(err, expectedErrMsg) { t.Fatalf("expected %q, got: %s (%T)", expectedErrMsg, - err, pErr.GetDetail()) + err, pErr.GoError()) } } @@ -10009,7 +10009,7 @@ func TestReplicaServersideRefreshes(t *testing.T) { send := func(ba roachpb.BatchRequest) (hlc.Timestamp, error) { br, pErr := tc.Sender().Send(context.Background(), ba) if pErr != nil { - return hlc.Timestamp{}, pErr.GetDetail() + return hlc.Timestamp{}, pErr.GoError() } // Check that we didn't mess up the stats. diff --git a/pkg/roachpb/errors.go b/pkg/roachpb/errors.go index ffbf0c9ec14f..41306f8b666b 100644 --- a/pkg/roachpb/errors.go +++ b/pkg/roachpb/errors.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/caller" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" "github.com/cockroachdb/redact" @@ -106,15 +107,24 @@ const ( // ErrPriority computes the priority of the given error. func ErrPriority(err error) ErrorPriority { - if err == nil { + // TODO(tbg): this method could take an `*Error` if it weren't for SQL + // propagating these as an `error`. See `DistSQLReceiver.Push`. + var detail ErrorDetailInterface + switch tErr := err.(type) { + case nil: return 0 - } - switch v := err.(type) { + case ErrorDetailInterface: + detail = tErr + case *internalError: + detail = (*Error)(tErr).GetDetail() case *UnhandledRetryableError: - if _, ok := v.PErr.GetDetail().(*TransactionAbortedError); ok { + if _, ok := tErr.PErr.GetDetail().(*TransactionAbortedError); ok { return ErrorScoreTxnAbort } return ErrorScoreTxnRestart + } + + switch v := detail.(type) { case *TransactionRetryWithProtoRefreshError: if v.PrevTxnAborted() { return ErrorScoreTxnAbort @@ -136,7 +146,13 @@ func NewError(err error) *Error { return nil } e := &Error{} - e.SetDetail(err) + if intErr, ok := err.(*internalError); ok { + *e = *(*Error)(intErr) + } else if msg, ok := err.(ErrorDetailInterface); ok { + e.SetDetail(msg) + } else { + e.Message = err.Error() + } return e } @@ -172,12 +188,11 @@ func (e *Error) SafeFormat(s redact.SafePrinter, _ rune) { // sure to terminate it here as well. These are all hints that *Error is not // well constructed. switch t := e.GetDetail().(type) { - case *internalError: - // *internalError is just our starting point *Error, i.e. no detail was - // returned. All we have is a message that will get stripped during redaction. + case nil: + // No detail was returned. All we have is a message + // that will get stripped during redaction. // - // TODO(tbg): using cockroachdb/errors for this case would get us much more - // mileage and usability here. See also: + // TODO(tbg): improve this after this issue has been addressed: // https://github.com/cockroachdb/cockroach/issues/54939 s.Print(e.Message) default: @@ -199,28 +214,14 @@ func (e *Error) String() string { type internalError Error -// Type is part of the ErrorDetailInterface. -func (e *internalError) Type() ErrorDetailType { - return InternalErrType -} - func (e *internalError) Error() string { return (*Error)(e).String() } -func (e *internalError) message(_ *Error) string { - return (*Error)(e).String() -} - -func (e *internalError) canRestartTransaction() TransactionRestart { - return e.TransactionRestart -} - -var _ ErrorDetailInterface = &internalError{} - // ErrorDetailInterface is an interface for each error detail. type ErrorDetailInterface interface { error + protoutil.Message // message returns an error message. message(*Error) string // Type returns the error's type. @@ -275,7 +276,10 @@ const ( NumErrors int = 40 ) -// GoError returns a Go error converted from Error. +// GoError returns a Go error converted from Error. If the error is a transaction +// retry error, it returns the error itself wrapped in an UnhandledRetryableError. +// Otherwise, if an error detail is present, is is returned (i.e. the result will +// match GetDetail()). Otherwise, returns the error itself masqueraded as an `error`. func (e *Error) GoError() error { if e == nil { return nil @@ -286,48 +290,39 @@ func (e *Error) GoError() error { PErr: *e, } } - return e.GetDetail() + if detail := e.GetDetail(); detail != nil { + return detail + } + return (*internalError)(e) } // SetDetail sets the error detail for the error. The argument cannot be nil. -func (e *Error) SetDetail(err error) { - if err == nil { - panic("nil err argument") +func (e *Error) SetDetail(detail ErrorDetailInterface) { + if detail == nil { + panic("nil detail argument") } - if intErr, ok := err.(*internalError); ok { - *e = *(*Error)(intErr) + e.Message = detail.message(e) + if r, ok := detail.(transactionRestartError); ok { + e.TransactionRestart = r.canRestartTransaction() } else { - if sErr, ok := err.(ErrorDetailInterface); ok { - e.Message = sErr.message(e) - } else { - e.Message = err.Error() - } - if r, ok := err.(transactionRestartError); ok { - e.TransactionRestart = r.canRestartTransaction() - } else { - e.TransactionRestart = TransactionRestart_NONE - } - // If the specific error type exists in the detail union, set it. - if !e.Detail.SetInner(err) { - _, isInternalError := err.(*internalError) - if !isInternalError && e.TransactionRestart != TransactionRestart_NONE { - panic(errors.AssertionFailedf("transactionRestartError %T must be an ErrorDetail", err)) - } + e.TransactionRestart = TransactionRestart_NONE + } + // If the specific error type exists in the detail union, set it. + if !e.Detail.SetInner(detail) { + if e.TransactionRestart != TransactionRestart_NONE { + panic(errors.AssertionFailedf("transactionRestartError %T must be an ErrorDetail", detail)) } - e.checkTxnStatusValid() } + e.checkTxnStatusValid() } -// GetDetail returns an error detail associated with the error. +// GetDetail returns an error detail associated with the error, or nil otherwise. func (e *Error) GetDetail() ErrorDetailInterface { if e == nil { return nil } - if err, ok := e.Detail.GetInner().(ErrorDetailInterface); ok { - return err - } - // Unknown error detail; return the generic error. - return (*internalError)(e) + detail, _ := e.Detail.GetInner().(ErrorDetailInterface) + return detail } // SetTxn sets the error transaction and resets the error message. diff --git a/pkg/roachpb/errors_test.go b/pkg/roachpb/errors_test.go index 1e0bbcefa1a1..a4e478706467 100644 --- a/pkg/roachpb/errors_test.go +++ b/pkg/roachpb/errors_test.go @@ -14,8 +14,10 @@ import ( "strings" "testing" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/uuid" + "github.com/cockroachdb/errors" "github.com/cockroachdb/redact" "github.com/stretchr/testify/require" ) @@ -53,6 +55,33 @@ func TestSetTxn(t *testing.T) { } } +func TestErrPriority(t *testing.T) { + unhandledAbort := &UnhandledRetryableError{ + PErr: *NewError(&TransactionAbortedError{}), + } + unhandledRetry := &UnhandledRetryableError{ + PErr: *NewError(&ReadWithinUncertaintyIntervalError{}), + } + require.Equal(t, ErrorPriority(0), ErrPriority(nil)) + require.Equal(t, ErrorScoreTxnAbort, ErrPriority(unhandledAbort)) + require.Equal(t, ErrorScoreTxnRestart, ErrPriority(unhandledRetry)) + { + id1 := uuid.Must(uuid.NewV4()) + require.Equal(t, ErrorScoreTxnRestart, ErrPriority(&TransactionRetryWithProtoRefreshError{ + TxnID: id1, + Transaction: Transaction{TxnMeta: enginepb.TxnMeta{ID: id1}}, + })) + id2 := uuid.Nil + require.Equal(t, ErrorScoreTxnAbort, ErrPriority(&TransactionRetryWithProtoRefreshError{ + TxnID: id1, + Transaction: Transaction{TxnMeta: enginepb.TxnMeta{ID: id2}}, + })) + } + require.Equal(t, ErrorScoreUnambiguousError, ErrPriority(&ConditionFailedError{})) + require.Equal(t, ErrorScoreUnambiguousError, ErrPriority(NewError(&ConditionFailedError{}).GoError())) + require.Equal(t, ErrorScoreNonRetriable, ErrPriority(errors.New("foo"))) +} + func TestErrorTxn(t *testing.T) { var pErr *Error if txn := pErr.GetTxn(); txn != nil { diff --git a/pkg/server/node.go b/pkg/server/node.go index 62b0bd8ee361..e9d11ee18e6e 100644 --- a/pkg/server/node.go +++ b/pkg/server/node.go @@ -848,7 +848,7 @@ func (n *Node) batchInternal( br, pErr = n.stores.Send(ctx, *args) if pErr != nil { br = &roachpb.BatchResponse{} - log.VErrEventf(ctx, 3, "%T", pErr.GetDetail()) + log.VErrEventf(ctx, 3, "error from stores.Send: %s", pErr) } if br.Error != nil { panic(roachpb.ErrorUnexpectedlySet(n.stores, br))