diff --git a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go index c999672787ce..b1b75dc34454 100644 --- a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go +++ b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go @@ -187,7 +187,7 @@ func EndTxn( return result.Result{}, roachpb.NewTransactionStatusError("could not commit in one phase as requested") } if args.Commit && args.Poison { - return result.Result{}, errors.Errorf("cannot poison during a committing EndTxn request") + return result.Result{}, errors.AssertionFailedf("cannot poison during a committing EndTxn request") } key := keys.TransactionKey(h.Txn.Key, h.Txn.ID) @@ -1089,11 +1089,11 @@ func mergeTrigger( ) (result.Result, error) { desc := rec.Desc() if !bytes.Equal(desc.StartKey, merge.LeftDesc.StartKey) { - return result.Result{}, errors.Errorf("LHS range start keys do not match: %s != %s", + return result.Result{}, errors.AssertionFailedf("LHS range start keys do not match: %s != %s", desc.StartKey, merge.LeftDesc.StartKey) } if !desc.EndKey.Less(merge.LeftDesc.EndKey) { - return result.Result{}, errors.Errorf("original LHS end key is not less than the post merge end key: %s >= %s", + return result.Result{}, errors.AssertionFailedf("original LHS end key is not less than the post merge end key: %s >= %s", desc.EndKey, merge.LeftDesc.EndKey) } diff --git a/pkg/kv/kvserver/batcheval/cmd_migrate.go b/pkg/kv/kvserver/batcheval/cmd_migrate.go index 8ccc98faf548..51d48d1d91f5 100644 --- a/pkg/kv/kvserver/batcheval/cmd_migrate.go +++ b/pkg/kv/kvserver/batcheval/cmd_migrate.go @@ -71,7 +71,7 @@ func Migrate( fn, ok := migrationRegistry[migrationVersion] if !ok { - return result.Result{}, errors.Newf("migration for %s not found", migrationVersion) + return result.Result{}, errors.AssertionFailedf("migration for %s not found", migrationVersion) } pd, err := fn(ctx, readWriter, cArgs) if err != nil { diff --git a/pkg/kv/kvserver/batcheval/cmd_push_txn.go b/pkg/kv/kvserver/batcheval/cmd_push_txn.go index aa32e73105fb..a1bb18d3edaa 100644 --- a/pkg/kv/kvserver/batcheval/cmd_push_txn.go +++ b/pkg/kv/kvserver/batcheval/cmd_push_txn.go @@ -117,18 +117,18 @@ func PushTxn( // push the transaction to. Transactions should not be pushed into the // future past the leaseholder's lease expiration or their effect may // not be fully reflected in a future leaseholder's timestamp cache. - return result.Result{}, errors.Errorf("request timestamp %s less than PushTo timestamp %s", + return result.Result{}, errors.AssertionFailedf("request timestamp %s less than PushTo timestamp %s", h.Timestamp, args.PushTo) } if h.WriteTimestamp().Less(args.PusheeTxn.MinTimestamp) { // This condition must hold for the timestamp cache access in // SynthesizeTxnFromMeta and the timestamp cache update in // Replica.updateTimestampCache to be safe. - return result.Result{}, errors.Errorf("request timestamp %s less than pushee txn MinTimestamp %s", + return result.Result{}, errors.AssertionFailedf("request timestamp %s less than pushee txn MinTimestamp %s", h.Timestamp, args.PusheeTxn.MinTimestamp) } if !bytes.Equal(args.Key, args.PusheeTxn.Key) { - return result.Result{}, errors.Errorf("request key %s should match pushee txn key %s", + return result.Result{}, errors.AssertionFailedf("request key %s should match pushee txn key %s", args.Key, args.PusheeTxn.Key) } key := keys.TransactionKey(args.PusheeTxn.Key, args.PusheeTxn.ID) @@ -286,7 +286,7 @@ func PushTxn( // timestamp beneath this timestamp. reply.PusheeTxn.WriteTimestamp.Forward(args.PushTo) default: - return result.Result{}, errors.Errorf("unexpected push type: %v", pushType) + return result.Result{}, errors.AssertionFailedf("unexpected push type: %v", pushType) } // If the transaction record was already present, persist the updates to it. diff --git a/pkg/kv/kvserver/batcheval/cmd_query_intent.go b/pkg/kv/kvserver/batcheval/cmd_query_intent.go index 62d5cc937672..8d1e50f8b131 100644 --- a/pkg/kv/kvserver/batcheval/cmd_query_intent.go +++ b/pkg/kv/kvserver/batcheval/cmd_query_intent.go @@ -65,7 +65,7 @@ func QueryIntent( if h.WriteTimestamp().Less(args.Txn.WriteTimestamp) { // This condition must hold for the timestamp cache update in // Replica.updateTimestampCache to be safe. - return result.Result{}, errors.Errorf("QueryIntent request timestamp %s less than txn WriteTimestamp %s", + return result.Result{}, errors.AssertionFailedf("QueryIntent request timestamp %s less than txn WriteTimestamp %s", h.Timestamp, args.Txn.WriteTimestamp) } diff --git a/pkg/kv/kvserver/batcheval/cmd_query_txn.go b/pkg/kv/kvserver/batcheval/cmd_query_txn.go index 9524adb3d2c4..0b60aa744392 100644 --- a/pkg/kv/kvserver/batcheval/cmd_query_txn.go +++ b/pkg/kv/kvserver/batcheval/cmd_query_txn.go @@ -53,11 +53,11 @@ func QueryTxn( if h.WriteTimestamp().Less(args.Txn.MinTimestamp) { // This condition must hold for the timestamp cache access in // SynthesizeTxnFromMeta to be safe. - return result.Result{}, errors.Errorf("QueryTxn request timestamp %s less than txn MinTimestamp %s", + return result.Result{}, errors.AssertionFailedf("QueryTxn request timestamp %s less than txn MinTimestamp %s", h.Timestamp, args.Txn.MinTimestamp) } if !args.Key.Equal(args.Txn.Key) { - return result.Result{}, errors.Errorf("QueryTxn request key %s does not match txn key %s", + return result.Result{}, errors.AssertionFailedf("QueryTxn request key %s does not match txn key %s", args.Key, args.Txn.Key) } key := keys.TransactionKey(args.Txn.Key, args.Txn.ID) diff --git a/pkg/kv/kvserver/batcheval/cmd_recover_txn.go b/pkg/kv/kvserver/batcheval/cmd_recover_txn.go index 950858d2941a..3904c956d0d2 100644 --- a/pkg/kv/kvserver/batcheval/cmd_recover_txn.go +++ b/pkg/kv/kvserver/batcheval/cmd_recover_txn.go @@ -59,11 +59,11 @@ func RecoverTxn( // This condition must hold for the timestamp cache access in // SynthesizeTxnFromMeta and the timestamp cache update in // Replica.updateTimestampCache to be safe. - return result.Result{}, errors.Errorf("RecoverTxn request timestamp %s less than txn MinTimestamp %s", + return result.Result{}, errors.AssertionFailedf("RecoverTxn request timestamp %s less than txn MinTimestamp %s", h.Timestamp, args.Txn.MinTimestamp) } if !args.Key.Equal(args.Txn.Key) { - return result.Result{}, errors.Errorf("RecoverTxn request key %s does not match txn key %s", + return result.Result{}, errors.AssertionFailedf("RecoverTxn request key %s does not match txn key %s", args.Key, args.Txn.Key) } key := keys.TransactionKey(args.Txn.Key, args.Txn.ID) diff --git a/pkg/kv/kvserver/batcheval/cmd_refresh_range.go b/pkg/kv/kvserver/batcheval/cmd_refresh_range.go index 93aa3ceddf88..65453a25d6b0 100644 --- a/pkg/kv/kvserver/batcheval/cmd_refresh_range.go +++ b/pkg/kv/kvserver/batcheval/cmd_refresh_range.go @@ -33,7 +33,7 @@ func RefreshRange( h := cArgs.Header if h.Txn == nil { - return result.Result{}, errors.Errorf("no transaction specified to %s", args.Method()) + return result.Result{}, errors.AssertionFailedf("no transaction specified to %s", args.Method()) } // We're going to refresh up to the transaction's read timestamp. diff --git a/pkg/kv/kvserver/batcheval/cmd_revert_range.go b/pkg/kv/kvserver/batcheval/cmd_revert_range.go index 90848d6fe8fe..afc1cae3fec2 100644 --- a/pkg/kv/kvserver/batcheval/cmd_revert_range.go +++ b/pkg/kv/kvserver/batcheval/cmd_revert_range.go @@ -20,7 +20,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/cockroachdb/errors" ) func init() { @@ -68,7 +67,7 @@ func RevertRange( ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp roachpb.Response, ) (result.Result, error) { if cArgs.Header.Txn != nil { - return result.Result{}, errors.New("cannot execute RevertRange within a transaction") + return result.Result{}, ErrTransactionUnsupported } log.VEventf(ctx, 2, "RevertRange %+v", cArgs.Args) diff --git a/pkg/kv/kvserver/batcheval/cmd_subsume.go b/pkg/kv/kvserver/batcheval/cmd_subsume.go index cc7b44015ff1..77797e62b7c4 100644 --- a/pkg/kv/kvserver/batcheval/cmd_subsume.go +++ b/pkg/kv/kvserver/batcheval/cmd_subsume.go @@ -77,7 +77,7 @@ func Subsume( desc := cArgs.EvalCtx.Desc() if !bytes.Equal(desc.StartKey, args.RightDesc.StartKey) || !bytes.Equal(desc.EndKey, args.RightDesc.EndKey) { - return result.Result{}, errors.Errorf("RHS range bounds do not match: %s != %s", + return result.Result{}, errors.AssertionFailedf("RHS range bounds do not match: %s != %s", args.RightDesc, desc) } @@ -85,7 +85,7 @@ func Subsume( // of operations in the AdminMerge transaction should make it impossible for // these ranges to be nonadjacent, but double check. if !bytes.Equal(args.LeftDesc.EndKey, desc.StartKey) { - return result.Result{}, errors.Errorf("ranges are not adjacent: %s != %s", + return result.Result{}, errors.AssertionFailedf("ranges are not adjacent: %s != %s", args.LeftDesc.EndKey, desc.StartKey) } @@ -97,13 +97,13 @@ func Subsume( if err != nil { return result.Result{}, errors.Errorf("fetching local range descriptor: %s", err) } else if intent == nil { - return result.Result{}, errors.New("range missing intent on its local descriptor") + return result.Result{}, errors.AssertionFailedf("range missing intent on its local descriptor") } val, _, err := storage.MVCCGetAsTxn(ctx, readWriter, descKey, cArgs.Header.Timestamp, intent.Txn) if err != nil { return result.Result{}, errors.Errorf("fetching local range descriptor as txn: %s", err) } else if val != nil { - return result.Result{}, errors.New("non-deletion intent on local range descriptor") + return result.Result{}, errors.AssertionFailedf("non-deletion intent on local range descriptor") } // We prevent followers of the RHS from being able to serve follower reads on diff --git a/pkg/kv/kvserver/batcheval/transaction.go b/pkg/kv/kvserver/batcheval/transaction.go index 08d394cdff84..1b522eefd243 100644 --- a/pkg/kv/kvserver/batcheval/transaction.go +++ b/pkg/kv/kvserver/batcheval/transaction.go @@ -26,7 +26,7 @@ import ( // ErrTransactionUnsupported is returned when a non-transactional command is // evaluated in the context of a transaction. -var ErrTransactionUnsupported = errors.New("not supported within a transaction") +var ErrTransactionUnsupported = errors.AssertionFailedf("not supported within a transaction") // VerifyTransaction runs sanity checks verifying that the transaction in the // header and the request are compatible. @@ -34,10 +34,10 @@ func VerifyTransaction( h roachpb.Header, args roachpb.Request, permittedStatuses ...roachpb.TransactionStatus, ) error { if h.Txn == nil { - return errors.Errorf("no transaction specified to %s", args.Method()) + return errors.AssertionFailedf("no transaction specified to %s", args.Method()) } if !bytes.Equal(args.Header().Key, h.Txn.Key) { - return errors.Errorf("request key %s should match txn key %s", args.Header().Key, h.Txn.Key) + return errors.AssertionFailedf("request key %s should match txn key %s", args.Header().Key, h.Txn.Key) } statusPermitted := false for _, s := range permittedStatuses {