Skip to content

Commit

Permalink
roachpb: introduce ReplicaUnavailableError
Browse files Browse the repository at this point in the history
Extracted from cockroachdb#71806.

For cockroachdb#33007, we are introducing per-Replica circuit breakers. When
tripped, they should return a structured error (cockroachdb#74500) that DistSender
(cockroachdb#74504) can handle and that the SQL layer can interpret and render
nicely in terms of tables and indexes (cockroachdb#74502).

This PR introduces this error and the necessary plumbing. It will be
easy to adjust the actual data within it with needed; we might find a
reason to do so during cockroachdb#74502.

I want to acknowledge that I am aware of the existence of
`sqlerrors.NewRangeUnavailableError`; there's possibly some interplay
between these two coming up in cockroachdb#74502 but we definitely need a
structured error that is emitted from KV (and in particular doesn't
cause wild import cycles) that is per *Replica*, as we envision (though
not initially) scenarios in which the range as a hole remains available
but individual Replicas are not.

Release note: None
  • Loading branch information
tbg committed Jan 17, 2022
1 parent e84001d commit 6c4bb94
Show file tree
Hide file tree
Showing 8 changed files with 114 additions and 2 deletions.
2 changes: 2 additions & 0 deletions .github/CODEOWNERS
Validating CODEOWNERS rules …
Original file line number Diff line number Diff line change
Expand Up @@ -235,9 +235,11 @@
/pkg/roachpb/metadata* @cockroachdb/kv-prs
/pkg/roachpb/method* @cockroachdb/kv-prs
/pkg/roachpb/mocks* @cockroachdb/kv-prs
/pkg/roachpb/replica_* @cockroachdb/kv-prs
/pkg/roachpb/span* @cockroachdb/kv-prs
/pkg/roachpb/string_test.go @cockroachdb/kv-prs
/pkg/roachpb/tenant* @cockroachdb/kv-prs
/pkg/roachpb/testdata/repl* @cockroachdb/kv-prs
/pkg/roachpb/version* @cockroachdb/server
/pkg/roachprod/ @cockroachdb/dev-inf
/pkg/rpc/ @cockroachdb/server-prs
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/replica_circuit_breaker.go
Original file line number Diff line number Diff line change
Expand Up @@ -205,7 +205,7 @@ func replicaUnavailableError(
var _ redact.SafeFormatter = desc
var _ redact.SafeFormatter = replDesc

err := errors.Errorf("replica %s of %s is unavailable", desc, replDesc)
err := roachpb.NewReplicaUnavailableError(desc, replDesc)
err = errors.Wrapf(
err,
"raft status: %+v", redact.Safe(rs), // raft status contains no PII
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/testdata/replica_unavailable_error.txt
Original file line number Diff line number Diff line change
@@ -1,3 +1,3 @@
echo
----
replicas on non-live nodes: (n2,s20):2 (lost quorum: true): raft status: {"id":"0","term":0,"vote":"0","commit":0,"lead":"0","raftState":"StateFollower","applied":0,"progress":{},"leadtransferee":"0"}: replica r10:‹{a-z}› [(n1,s10):1, (n2,s20):2, next=3, gen=0] of (n1,s10):1 is unavailable
replicas on non-live nodes: (n2,s20):2 (lost quorum: true): raft status: {"id":"0","term":0,"vote":"0","commit":0,"lead":"0","raftState":"StateFollower","applied":0,"progress":{},"leadtransferee":"0"}: replica (n1,s10):1 unable to serve request to r10:‹{a-z}› [(n1,s10):1, (n2,s20):2, next=3, gen=0]
4 changes: 4 additions & 0 deletions pkg/roachpb/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ go_library(
"metadata.go",
"metadata_replicas.go",
"method.go",
"replica_unavailable_error.go",
"span_config.go",
"span_group.go",
"tenant.go",
Expand Down Expand Up @@ -154,17 +155,20 @@ go_test(
"merge_spans_test.go",
"metadata_replicas_test.go",
"metadata_test.go",
"replica_unavailable_error_test.go",
"span_group_test.go",
"tenant_test.go",
"version_test.go",
],
data = glob(["testdata/**"]),
embed = [":with-mocks"], # keep
tags = ["no-remote"],
deps = [
"//pkg/cli/exit",
"//pkg/kv/kvserver/concurrency/lock",
"//pkg/storage/enginepb",
"//pkg/testutils/buildutil",
"//pkg/testutils/echotest",
"//pkg/testutils/zerofields",
"//pkg/util",
"//pkg/util/bitarray",
Expand Down
7 changes: 7 additions & 0 deletions pkg/roachpb/errors.proto
Original file line number Diff line number Diff line change
Expand Up @@ -373,6 +373,13 @@ message AmbiguousResultError {
optional Error wrapped_err = 2;
}

message ReplicaUnavailableError {
option (gogoproto.goproto_stringer) = false;

optional roachpb.RangeDescriptor desc = 2 [(gogoproto.nullable) = false];
optional roachpb.ReplicaDescriptor replica = 4 [(gogoproto.nullable) = false];
}

// A RaftGroupDeletedError indicates a raft group has been deleted for
// the replica.
message RaftGroupDeletedError {
Expand Down
58 changes: 58 additions & 0 deletions pkg/roachpb/replica_unavailable_error.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,58 @@
// Copyright 2022 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package roachpb

import (
"fmt"

"github.com/cockroachdb/errors"
"github.com/cockroachdb/redact"
)

// NewReplicaUnavailableError initializes a new *ReplicaUnavailableError. It is
// provided with the range descriptor known to the replica, and the relevant
// replica descriptor within.
func NewReplicaUnavailableError(desc *RangeDescriptor, replDesc ReplicaDescriptor) error {
return &ReplicaUnavailableError{
Desc: *desc,
Replica: replDesc,
}
}

var _ errors.SafeFormatter = (*ReplicaUnavailableError)(nil)
var _ fmt.Formatter = (*ReplicaUnavailableError)(nil)

// SafeFormatError implements errors.SafeFormatter.
func (e *ReplicaUnavailableError) SafeFormatError(p errors.Printer) error {
e.printTo(p.Printf)
return nil
}

// See https://github.com/cockroachdb/errors/issues/88.
func (e *ReplicaUnavailableError) printTo(printf func(string, ...interface{})) {
printf("replica %s unable to serve request to %s", e.Replica, e.Desc)
}

// Format implements fmt.Formatter.
func (e *ReplicaUnavailableError) Format(s fmt.State, verb rune) { errors.FormatError(e, s, verb) }

// Error implements error.
func (e *ReplicaUnavailableError) Error() string {
var s string
e.printTo(func(format string, args ...interface{}) {
s = fmt.Sprintf(format, args...)
})
return s
}

func (e *ReplicaUnavailableError) String() string {
return redact.Sprint(e).StripMarkers()
}
37 changes: 37 additions & 0 deletions pkg/roachpb/replica_unavailable_error_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,37 @@
// Copyright 2022 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package roachpb

import (
"context"
"fmt"
"path/filepath"
"testing"

"github.com/cockroachdb/cockroach/pkg/testutils/echotest"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/redact"
)

func TestReplicaUnavailableError(t *testing.T) {
ctx := context.Background()
var _ = (*ReplicaUnavailableError)(nil)
rDesc := ReplicaDescriptor{NodeID: 1, StoreID: 2, ReplicaID: 3}
var set ReplicaSet
set.AddReplica(rDesc)
desc := NewRangeDescriptor(123, RKeyMin, RKeyMax, set)

var err error = NewReplicaUnavailableError(desc, rDesc)
err = errors.DecodeError(ctx, errors.EncodeError(ctx, err))

s := fmt.Sprintf("%s\n%s", err, redact.Sprint(err))
echotest.Require(t, s, filepath.Join("testdata", "replica_unavailable_error.txt"))
}
4 changes: 4 additions & 0 deletions pkg/roachpb/testdata/replica_unavailable_error.txt
Original file line number Diff line number Diff line change
@@ -0,0 +1,4 @@
echo
----
replica (n1,s2):3 unable to serve request to r123:/M{in-ax} [(n1,s2):1, next=2, gen=0]
replica (n1,s2):3 unable to serve request to r123:‹/M{in-ax}› [(n1,s2):1, next=2, gen=0]

0 comments on commit 6c4bb94

Please sign in to comment.