2 Star 2 Fork 1

cockroachdb/cockroach

Create your Gitee Account
Explore and code with more than 12 million developers,Free private repositories !:)
Sign up
Clone or Download
errors.go 16.85 KB
Copy Edit Raw Blame History
123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620
// Copyright 2014 The Cockroach Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
// implied. See the License for the specific language governing
// permissions and limitations under the License.
package roachpb
import (
"bytes"
"fmt"
"github.com/cockroachdb/cockroach/pkg/util/caller"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
)
func (e *UnhandledRetryableError) Error() string {
return e.PErr.Message
}
var _ error = &UnhandledRetryableError{}
// ErrorUnexpectedlySet creates a string to panic with when a response (typically
// a roachpb.BatchResponse) unexpectedly has Error set in its response header.
func ErrorUnexpectedlySet(culprit, response interface{}) string {
return fmt.Sprintf("error is unexpectedly set, culprit is %T:\n%+v", culprit, response)
}
// transactionRestartError is an interface implemented by errors that cause
// a transaction to be restarted.
type transactionRestartError interface {
canRestartTransaction() TransactionRestart
}
// GetDetail returns an error detail associated with the error.
func (e *Error) GetDetail() ErrorDetailInterface {
if e == nil {
return nil
}
if e.Detail == nil {
// Unknown error detail; return the generic error.
return (*internalError)(e)
}
if err, ok := e.Detail.GetValue().(ErrorDetailInterface); ok {
return err
}
// Unknown error detail; return the generic error.
return (*internalError)(e)
}
// NewError creates an Error from the given error.
func NewError(err error) *Error {
if err == nil {
return nil
}
e := &Error{}
if intErr, ok := err.(*internalError); ok {
*e = *(*Error)(intErr)
} else {
e.setGoError(err)
}
return e
}
// NewErrorWithTxn creates an Error from the given error and a transaction.
//
// txn is cloned before being stored in Error.
func NewErrorWithTxn(err error, txn *Transaction) *Error {
e := NewError(err)
e.SetTxn(txn)
return e
}
// NewErrorf creates an Error from the given error message. It is a
// passthrough to fmt.Errorf, with an additional prefix containing the
// filename and line number.
func NewErrorf(format string, a ...interface{}) *Error {
// Cannot use errors.Errorf here due to cyclic dependency.
file, line, _ := caller.Lookup(1)
s := fmt.Sprintf("%s:%d: ", file, line)
return NewError(fmt.Errorf(s+format, a...))
}
// String implements fmt.Stringer.
func (e *Error) String() string {
if e == nil {
return "<nil>"
}
return e.Message
}
type internalError Error
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
// message returns an error message.
message(*Error) string
}
// GoError returns a Go error converted from Error.
func (e *Error) GoError() error {
if e == nil {
return nil
}
if e.TransactionRestart != TransactionRestart_NONE {
return &UnhandledRetryableError{
PErr: *e,
}
}
return e.GetDetail()
}
// setGoError sets Error using err.
func (e *Error) setGoError(err error) {
if e.Message != "" {
panic("cannot re-use roachpb.Error")
}
if sErr, ok := err.(ErrorDetailInterface); ok {
e.Message = sErr.message(e)
} else {
e.Message = err.Error()
}
var isTxnError bool
if r, ok := err.(transactionRestartError); ok {
isTxnError = true
e.TransactionRestart = r.canRestartTransaction()
}
// If the specific error type exists in the detail union, set it.
detail := &ErrorDetail{}
if detail.SetValue(err) {
e.Detail = detail
} else if _, isInternalError := err.(*internalError); !isInternalError && isTxnError {
panic(fmt.Sprintf("transactionRestartError %T must be an ErrorDetail", err))
}
}
// SetTxn sets the txn and resets the error message. txn is cloned before being
// stored in the Error.
// TODO(kaneda): Unexpose this method and make callers use NewErrorWithTxn.
func (e *Error) SetTxn(txn *Transaction) {
e.UnexposedTxn = txn
if txn != nil {
txnClone := txn.Clone()
e.UnexposedTxn = &txnClone
}
if e.Detail != nil {
if sErr, ok := e.Detail.GetValue().(ErrorDetailInterface); ok {
// Refresh the message as the txn is updated.
e.Message = sErr.message(e)
}
}
}
// GetTxn returns the txn.
func (e *Error) GetTxn() *Transaction {
if e == nil {
return nil
}
return e.UnexposedTxn
}
// UpdateTxn updates the txn.
func (e *Error) UpdateTxn(o *Transaction) {
if e == nil {
return
}
if e.UnexposedTxn == nil {
e.UnexposedTxn = o
} else {
e.UnexposedTxn.Update(o)
}
}
// SetErrorIndex sets the index of the error.
func (e *Error) SetErrorIndex(index int32) {
e.Index = &ErrPosition{Index: index}
}
func (e *NodeUnavailableError) Error() string {
return e.message(nil)
}
func (*NodeUnavailableError) message(_ *Error) string {
return "node unavailable; try another peer"
}
var _ ErrorDetailInterface = &NodeUnavailableError{}
func (e *NotLeaseHolderError) Error() string {
return e.message(nil)
}
func (e *NotLeaseHolderError) message(_ *Error) string {
const prefix = "[NotLeaseHolderError] "
if e.CustomMsg != "" {
return prefix + e.CustomMsg
}
if e.LeaseHolder == nil {
return fmt.Sprintf("%sr%d: replica %s not lease holder; lease holder unknown", prefix, e.RangeID, e.Replica)
} else if e.Lease != nil {
return fmt.Sprintf("%sr%d: replica %s not lease holder; current lease is %s", prefix, e.RangeID, e.Replica, e.Lease)
}
return fmt.Sprintf("%sr%d: replica %s not lease holder; replica %s is", prefix, e.RangeID, e.Replica, *e.LeaseHolder)
}
var _ ErrorDetailInterface = &NotLeaseHolderError{}
func (e *LeaseRejectedError) Error() string {
return e.message(nil)
}
func (e *LeaseRejectedError) message(_ *Error) string {
return fmt.Sprintf("cannot replace lease %s with %s: %s", e.Existing, e.Requested, e.Message)
}
var _ ErrorDetailInterface = &LeaseRejectedError{}
// NewSendError creates a SendError.
func NewSendError(msg string) *SendError {
return &SendError{Message: msg}
}
func (s SendError) Error() string {
return s.message(nil)
}
func (s *SendError) message(_ *Error) string {
return "failed to send RPC: " + s.Message
}
var _ ErrorDetailInterface = &SendError{}
// NewRangeNotFoundError initializes a new RangeNotFoundError.
func NewRangeNotFoundError(rangeID RangeID) *RangeNotFoundError {
return &RangeNotFoundError{
RangeID: rangeID,
}
}
func (e *RangeNotFoundError) Error() string {
return e.message(nil)
}
func (e *RangeNotFoundError) message(_ *Error) string {
return fmt.Sprintf("r%d was not found", e.RangeID)
}
var _ ErrorDetailInterface = &RangeNotFoundError{}
// NewRangeKeyMismatchError initializes a new RangeKeyMismatchError.
func NewRangeKeyMismatchError(start, end Key, desc *RangeDescriptor) *RangeKeyMismatchError {
if desc != nil && !desc.IsInitialized() {
// We must never send uninitialized ranges back to the client (nil
// is fine) guard against regressions of #6027.
panic(fmt.Sprintf("descriptor is not initialized: %+v", desc))
}
return &RangeKeyMismatchError{
RequestStartKey: start,
RequestEndKey: end,
MismatchedRange: desc,
}
}
func (e *RangeKeyMismatchError) Error() string {
return e.message(nil)
}
func (e *RangeKeyMismatchError) message(_ *Error) string {
if e.MismatchedRange != nil {
return fmt.Sprintf("key range %s-%s outside of bounds of range %s-%s",
e.RequestStartKey, e.RequestEndKey, e.MismatchedRange.StartKey, e.MismatchedRange.EndKey)
}
return fmt.Sprintf("key range %s-%s could not be located within a range on store", e.RequestStartKey, e.RequestEndKey)
}
var _ ErrorDetailInterface = &RangeKeyMismatchError{}
// NewAmbiguousResultError initializes a new AmbiguousResultError with
// an explanatory message.
func NewAmbiguousResultError(msg string) *AmbiguousResultError {
return &AmbiguousResultError{Message: msg}
}
func (e *AmbiguousResultError) Error() string {
return e.message(nil)
}
func (e *AmbiguousResultError) message(_ *Error) string {
return fmt.Sprintf("result is ambiguous (%s)", e.Message)
}
var _ ErrorDetailInterface = &AmbiguousResultError{}
func (e *TransactionAbortedError) Error() string {
return "TransactionAbortedError: txn aborted"
}
func (e *TransactionAbortedError) message(pErr *Error) string {
return fmt.Sprintf("TransactionAbortedError: txn aborted %s", pErr.GetTxn())
}
func (*TransactionAbortedError) canRestartTransaction() TransactionRestart {
return TransactionRestart_IMMEDIATE
}
var _ ErrorDetailInterface = &TransactionAbortedError{}
var _ transactionRestartError = &TransactionAbortedError{}
func (e *HandledRetryableTxnError) Error() string {
return e.message(nil)
}
func (e *HandledRetryableTxnError) message(_ *Error) string {
return fmt.Sprintf("HandledRetryableTxnError: %s", e.Msg)
}
var _ ErrorDetailInterface = &HandledRetryableTxnError{}
// NewTransactionAbortedError initializes a new TransactionAbortedError.
func NewTransactionAbortedError() *TransactionAbortedError {
return &TransactionAbortedError{}
}
// NewHandledRetryableTxnError initializes a new HandledRetryableTxnError.
//
// txnID is the ID of the transaction being restarted.
// txn is the transaction that the client should use for the next attempts.
func NewHandledRetryableTxnError(
msg string, txnID uuid.UUID, txn Transaction,
) *HandledRetryableTxnError {
return &HandledRetryableTxnError{Msg: msg, TxnID: txnID, Transaction: txn}
}
// NewTransactionPushError initializes a new TransactionPushError.
// The argument is copied.
func NewTransactionPushError(pusheeTxn Transaction) *TransactionPushError {
// Note: this error will cause a txn restart. The error that the client
// receives contains a txn that might have a modified priority.
return &TransactionPushError{PusheeTxn: pusheeTxn.Clone()}
}
func (e *TransactionPushError) Error() string {
return e.message(nil)
}
func (e *TransactionPushError) message(pErr *Error) string {
if pErr.GetTxn() == nil {
return fmt.Sprintf("failed to push %s", e.PusheeTxn)
}
return fmt.Sprintf("txn %s failed to push %s", pErr.GetTxn(), e.PusheeTxn)
}
var _ ErrorDetailInterface = &TransactionPushError{}
var _ transactionRestartError = &TransactionPushError{}
func (*TransactionPushError) canRestartTransaction() TransactionRestart {
return TransactionRestart_IMMEDIATE
}
// NewTransactionRetryError initializes a new TransactionRetryError.
func NewTransactionRetryError(reason TransactionRetryReason) *TransactionRetryError {
return &TransactionRetryError{
Reason: reason,
}
}
func (e *TransactionRetryError) Error() string {
return fmt.Sprintf("TransactionRetryError: retry txn (%s)", e.Reason)
}
func (e *TransactionRetryError) message(pErr *Error) string {
return fmt.Sprintf("TransactionRetryError: retry txn (%s): %s", e.Reason, pErr.GetTxn())
}
var _ ErrorDetailInterface = &TransactionRetryError{}
var _ transactionRestartError = &TransactionRetryError{}
func (*TransactionRetryError) canRestartTransaction() TransactionRestart {
return TransactionRestart_IMMEDIATE
}
// NewTransactionReplayError initializes a new TransactionReplayError.
func NewTransactionReplayError() *TransactionReplayError {
return &TransactionReplayError{}
}
func (e *TransactionReplayError) Error() string {
return fmt.Sprintf("replay txn")
}
func (e *TransactionReplayError) message(pErr *Error) string {
return fmt.Sprintf("replay txn %s", pErr.GetTxn())
}
var _ ErrorDetailInterface = &TransactionReplayError{}
// NewTransactionStatusError initializes a new TransactionStatusError from
// the given message.
func NewTransactionStatusError(msg string) *TransactionStatusError {
return &TransactionStatusError{Msg: msg}
}
func (e *TransactionStatusError) Error() string {
return "TransactionStatusError: " + e.Msg
}
func (e *TransactionStatusError) message(pErr *Error) string {
return fmt.Sprintf("txn %s: %s", pErr.GetTxn(), e.Msg)
}
var _ ErrorDetailInterface = &TransactionStatusError{}
func (e *WriteIntentError) Error() string {
return e.message(nil)
}
func (e *WriteIntentError) message(_ *Error) string {
var buf bytes.Buffer
buf.WriteString("conflicting intents on ")
// If we have a lot of intents, we only want to show the first and the last.
const maxBegin = 5
const maxEnd = 5
var begin, end []Intent
if len(e.Intents) <= maxBegin+maxEnd {
begin = e.Intents
} else {
begin = e.Intents[0:maxBegin]
end = e.Intents[len(e.Intents)-maxEnd : len(e.Intents)]
}
for i := range begin {
if i > 0 {
buf.WriteString(", ")
}
buf.WriteString(begin[i].Key.String())
}
if end != nil {
buf.WriteString(" ... ")
for i := range end {
if i > 0 {
buf.WriteString(", ")
}
buf.WriteString(end[i].Key.String())
}
}
return buf.String()
}
var _ ErrorDetailInterface = &WriteIntentError{}
func (e *WriteTooOldError) Error() string {
return e.message(nil)
}
func (e *WriteTooOldError) message(_ *Error) string {
return fmt.Sprintf("WriteTooOldError: write at timestamp %s too old; wrote at %s",
e.Timestamp, e.ActualTimestamp)
}
var _ ErrorDetailInterface = &WriteTooOldError{}
var _ transactionRestartError = &WriteTooOldError{}
func (*WriteTooOldError) canRestartTransaction() TransactionRestart {
return TransactionRestart_IMMEDIATE
}
// NewReadWithinUncertaintyIntervalError creates a new uncertainty retry error.
// The read and existing timestamps are purely informational and used for
// formatting the error message.
func NewReadWithinUncertaintyIntervalError(
readTS, existingTS hlc.Timestamp,
) *ReadWithinUncertaintyIntervalError {
return &ReadWithinUncertaintyIntervalError{
ReadTimestamp: readTS,
ExistingTimestamp: existingTS,
}
}
func (e *ReadWithinUncertaintyIntervalError) Error() string {
return e.message(nil)
}
func (e *ReadWithinUncertaintyIntervalError) message(_ *Error) string {
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{}
var _ transactionRestartError = &ReadWithinUncertaintyIntervalError{}
func (*ReadWithinUncertaintyIntervalError) canRestartTransaction() TransactionRestart {
return TransactionRestart_IMMEDIATE
}
func (e *OpRequiresTxnError) Error() string {
return e.message(nil)
}
func (e *OpRequiresTxnError) message(_ *Error) string {
return "the operation requires transactional context"
}
var _ ErrorDetailInterface = &OpRequiresTxnError{}
func (e *ConditionFailedError) Error() string {
return e.message(nil)
}
func (e *ConditionFailedError) message(_ *Error) string {
return fmt.Sprintf("unexpected value: %s", e.ActualValue)
}
var _ ErrorDetailInterface = &ConditionFailedError{}
func (e *RaftGroupDeletedError) Error() string {
return e.message(nil)
}
func (*RaftGroupDeletedError) message(_ *Error) string {
return "raft group deleted"
}
var _ ErrorDetailInterface = &RaftGroupDeletedError{}
func (e *ReplicaCorruptionError) Error() string {
return e.message(nil)
}
func (e *ReplicaCorruptionError) message(_ *Error) string {
msg := fmt.Sprintf("replica corruption (processed=%t)", e.Processed)
if e.ErrorMsg != "" {
msg += ": " + e.ErrorMsg
}
return msg
}
var _ ErrorDetailInterface = &ReplicaCorruptionError{}
// NewReplicaTooOldError initializes a new ReplicaTooOldError.
func NewReplicaTooOldError(replicaID ReplicaID) *ReplicaTooOldError {
return &ReplicaTooOldError{
ReplicaID: replicaID,
}
}
func (e *ReplicaTooOldError) Error() string {
return e.message(nil)
}
func (*ReplicaTooOldError) message(_ *Error) string {
return "sender replica too old, discarding message"
}
var _ ErrorDetailInterface = &ReplicaTooOldError{}
// NewStoreNotFoundError initializes a new StoreNotFoundError.
func NewStoreNotFoundError(storeID StoreID) *StoreNotFoundError {
return &StoreNotFoundError{
StoreID: storeID,
}
}
func (e *StoreNotFoundError) Error() string {
return e.message(nil)
}
func (e *StoreNotFoundError) message(_ *Error) string {
return fmt.Sprintf("store %d was not found", e.StoreID)
}
var _ ErrorDetailInterface = &StoreNotFoundError{}
func (e *UntrackedTxnError) Error() string {
return e.message(nil)
}
func (*UntrackedTxnError) message(_ *Error) string {
return "writing transaction timed out or ran on multiple coordinators"
}
var _ ErrorDetailInterface = &UntrackedTxnError{}
func (e *TxnPrevAttemptError) Error() string {
return e.message(nil)
}
func (*TxnPrevAttemptError) message(_ *Error) string {
return "response meant for previous incarnation of transaction"
}
var _ ErrorDetailInterface = &TxnPrevAttemptError{}
马建仓 AI 助手
尝试更多
代码解读
代码找茬
代码优化
1
https://gitee.com/mirrors_cockroachdb/cockroach.git
git@gitee.com:mirrors_cockroachdb/cockroach.git
mirrors_cockroachdb
cockroach
cockroach
v1.1.8

Search