From cd7e3e5855e82a656f04109c17bd2d92c210c386 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Mon, 1 May 2023 18:37:27 +0200 Subject: [PATCH 01/17] kvpb: introduce NodeDecommissionedError This will be used in a follow-up commit to more precisely communicate that a connection attempt failed due to a decommissioned peer or source. It will be more important to get this right as we switch to a stateful connection pool. We don't want to attempt to reconnect to decommissioned nodes forever, and we also don't want to misrepresent them as "unavailable" for the purpose of network observability. So far we're relying on the `PermissionDenied` error code but this is already overloaded. It's better to provide an explicit gRPC status payload that leaves no room for interpretation. Epic: none Release note: None --- pkg/kv/kvpb/BUILD.bazel | 5 ++ pkg/kv/kvpb/errors.proto | 5 ++ pkg/kv/kvpb/node_decommissioned_error.go | 58 +++++++++++++++++++ pkg/kv/kvpb/node_decommissioned_error_test.go | 28 +++++++++ .../lint/passes/fmtsafe/functions.go | 3 +- 5 files changed, 98 insertions(+), 1 deletion(-) create mode 100644 pkg/kv/kvpb/node_decommissioned_error.go create mode 100644 pkg/kv/kvpb/node_decommissioned_error_test.go diff --git a/pkg/kv/kvpb/BUILD.bazel b/pkg/kv/kvpb/BUILD.bazel index 0efcca13a09f..36431b14574d 100644 --- a/pkg/kv/kvpb/BUILD.bazel +++ b/pkg/kv/kvpb/BUILD.bazel @@ -16,6 +16,7 @@ go_library( "data.go", "errors.go", "method.go", + "node_decommissioned_error.go", "replica_unavailable_error.go", ":gen-batch-generated", # keep ":gen-errordetailtype-stringer", # keep @@ -42,7 +43,10 @@ go_library( "@com_github_cockroachdb_redact//:redact", "@com_github_dustin_go_humanize//:go-humanize", "@com_github_gogo_protobuf//proto", + "@com_github_gogo_protobuf//types", + "@com_github_gogo_status//:status", "@com_github_golang_mock//gomock", # keep + "@org_golang_google_grpc//codes", "@org_golang_google_grpc//metadata", # keep ], ) @@ -53,6 +57,7 @@ go_test( "api_test.go", "batch_test.go", "errors_test.go", + "node_decommissioned_error_test.go", "replica_unavailable_error_test.go", "string_test.go", ], diff --git a/pkg/kv/kvpb/errors.proto b/pkg/kv/kvpb/errors.proto index a2ceb73f47e4..40d67d6ec2a4 100644 --- a/pkg/kv/kvpb/errors.proto +++ b/pkg/kv/kvpb/errors.proto @@ -80,6 +80,11 @@ message NotLeaseHolderError { message NodeUnavailableError { } +// A NodeDecommissionedError is returned when trying to connect from or to a +// peer that is known to have been decommissioned. +message NodeDecommissionedError { +} + // An UnsupportedRequestError indicates that the recipient node // does not know how to handle the type of request received. message UnsupportedRequestError { diff --git a/pkg/kv/kvpb/node_decommissioned_error.go b/pkg/kv/kvpb/node_decommissioned_error.go new file mode 100644 index 000000000000..ff2735281cc1 --- /dev/null +++ b/pkg/kv/kvpb/node_decommissioned_error.go @@ -0,0 +1,58 @@ +// Copyright 2023 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 kvpb + +import ( + "github.com/cockroachdb/errors" + _ "github.com/cockroachdb/errors/extgrpc" // for error encoder/decoder + "github.com/cockroachdb/redact" + ptypes "github.com/gogo/protobuf/types" + gogostatus "github.com/gogo/status" + "google.golang.org/grpc/codes" +) + +// NewDecommissionedStatusErrorf returns a GRPC status with the given error code +// and formatted message whose detail is a *NodeDecommissionedError. +func NewDecommissionedStatusErrorf(errorCode codes.Code, format string, args ...interface{}) error { + // Important: gogoproto ptypes and status, not google protobuf, see extgrpc pkg. + st := gogostatus.Newf(errorCode, format, args...).Proto() + det, err := ptypes.MarshalAny(&NodeDecommissionedError{}) + if err != nil { + return err + } + st.Details = append(st.Details, det) + return gogostatus.FromProto(st).Err() +} + +func (err *NodeDecommissionedError) SafeFormatError(p errors.Printer) (next error) { + p.Printf("node is decommissioned") + return nil +} + +func (err *NodeDecommissionedError) Error() string { + return redact.Sprint(err).StripMarkers() +} + +// IsDecommissionedStatusErr returns true if the error wraps a gRPC status error +// with a NodeDecommissionedError detail, i.e. it was created using +// NewDecommissionedStatusErrorf. +func IsDecommissionedStatusErr(err error) bool { + s, ok := gogostatus.FromError(errors.UnwrapAll(err)) + if !ok { + return false + } + for _, det := range s.Details() { + if _, ok := det.(*NodeDecommissionedError); ok { + return true + } + } + return false +} diff --git a/pkg/kv/kvpb/node_decommissioned_error_test.go b/pkg/kv/kvpb/node_decommissioned_error_test.go new file mode 100644 index 000000000000..8b4c7bae726b --- /dev/null +++ b/pkg/kv/kvpb/node_decommissioned_error_test.go @@ -0,0 +1,28 @@ +// Copyright 2023 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 kvpb + +import ( + "context" + "testing" + + "github.com/cockroachdb/errors" + "github.com/stretchr/testify/require" + "google.golang.org/grpc/codes" +) + +func TestNewDecommissionedStatusErrorf(t *testing.T) { + ctx := context.Background() + err := errors.Wrap(NewDecommissionedStatusErrorf(codes.Unauthenticated, "hello %s", "world"), "!") + require.True(t, IsDecommissionedStatusErr(err)) + ee := errors.EncodeError(ctx, err) + require.True(t, IsDecommissionedStatusErr(errors.DecodeError(ctx, ee))) +} diff --git a/pkg/testutils/lint/passes/fmtsafe/functions.go b/pkg/testutils/lint/passes/fmtsafe/functions.go index 2d3fbac90394..0e4fa759e23a 100644 --- a/pkg/testutils/lint/passes/fmtsafe/functions.go +++ b/pkg/testutils/lint/passes/fmtsafe/functions.go @@ -73,7 +73,8 @@ var requireConstFmt = map[string]bool{ "github.com/cockroachdb/cockroach/pkg/util/log/logcrash.ReportOrPanic": true, - "github.com/cockroachdb/cockroach/pkg/kv/kvpb.NewAmbiguousResultErrorf": true, + "github.com/cockroachdb/cockroach/pkg/kv/kvpb.NewAmbiguousResultErrorf": true, + "github.com/cockroachdb/cockroach/pkg/kv/kvpb.NewDecommissionedStatusErrorf": true, "(*github.com/cockroachdb/cockroach/pkg/util/tracing.Span).Recordf": true, "(*github.com/cockroachdb/cockroach/pkg/util/tracing.spanInner).Recordf": true, From 00f9c8f010f818bb5303278c70c4d26f2dbc4a1b Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Thu, 18 May 2023 12:59:01 +0200 Subject: [PATCH 02/17] rpc: update the round-trip-latency histogram improve documentation. Epic: None Release note: None --- pkg/rpc/clock_offset.go | 33 ++++++++++++++++++++++++--------- 1 file changed, 24 insertions(+), 9 deletions(-) diff --git a/pkg/rpc/clock_offset.go b/pkg/rpc/clock_offset.go index 7fcdc5ad6714..490703e4e544 100644 --- a/pkg/rpc/clock_offset.go +++ b/pkg/rpc/clock_offset.go @@ -29,7 +29,7 @@ import ( type RemoteClockMetrics struct { ClockOffsetMeanNanos *metric.Gauge ClockOffsetStdDevNanos *metric.Gauge - LatencyHistogramNanos metric.IHistogram + RoundTripLatency metric.IHistogram } // avgLatencyMeasurementAge determines how to exponentially weight the @@ -52,10 +52,22 @@ var ( Measurement: "Clock Offset", Unit: metric.Unit_NANOSECONDS, } - metaLatencyHistogramNanos = metric.Metadata{ - Name: "round-trip-latency", - Help: "Distribution of round-trip latencies with other nodes", - Measurement: "Roundtrip Latency", + + metaConnectionRoundTripLatency = metric.Metadata{ + // NB: the name is legacy and should not be changed since customers + // rely on it. + Name: "round-trip-latency", + Help: `Distribution of round-trip latencies with other nodes. + +This only reflects successful heartbeats and measures gRPC overhead as well as +possible head-of-line blocking. Elevated values in this metric may hint at +network issues and/or saturation, but they are no proof of them. CPU overload +can similarly elevate this metric. The operator should look towards OS-level +metrics such as packet loss, retransmits, etc, to conclusively diagnose network +issues. Heartbeats are not very frequent (~seconds), so they may not capture +rare or short-lived degradations. +`, + Measurement: "Round-trip time", Unit: metric.Unit_NANOSECONDS, } ) @@ -138,11 +150,14 @@ func newRemoteClockMonitor( r.metrics = RemoteClockMetrics{ ClockOffsetMeanNanos: metric.NewGauge(metaClockOffsetMeanNanos), ClockOffsetStdDevNanos: metric.NewGauge(metaClockOffsetStdDevNanos), - LatencyHistogramNanos: metric.NewHistogram(metric.HistogramOptions{ + RoundTripLatency: metric.NewHistogram(metric.HistogramOptions{ Mode: metric.HistogramModePreferHdrLatency, - Metadata: metaLatencyHistogramNanos, + Metadata: metaConnectionRoundTripLatency, Duration: histogramWindowInterval, - Buckets: metric.IOLatencyBuckets, + // NB: the choice of IO over Network buckets is somewhat debatable, but + // it's fine. Heartbeats can take >1s which the IO buckets can represent, + // but the Network buckets top out at 1s. + Buckets: metric.IOLatencyBuckets, }), } return &r @@ -262,7 +277,7 @@ func (r *RemoteClockMonitor) UpdateOffset( newLatencyf := float64(roundTripLatency.Nanoseconds()) prevAvg := info.avgNanos.Value() info.avgNanos.Add(newLatencyf) - r.metrics.LatencyHistogramNanos.RecordValue(roundTripLatency.Nanoseconds()) + r.metrics.RoundTripLatency.RecordValue(roundTripLatency.Nanoseconds()) // See: https://github.com/cockroachdb/cockroach/issues/96262 // See: https://github.com/cockroachdb/cockroach/issues/98066 From b83cb84f0bdc7e3715f596f49a2e5645c45d7aee Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Thu, 1 Jun 2023 08:57:06 +0200 Subject: [PATCH 03/17] circuit: allow triggering probe This will be used in the `rpc` package to remove an ugly workaround. Epic: None Release note: None --- pkg/util/circuit/circuitbreaker.go | 14 ++++++++++---- pkg/util/circuit/circuitbreaker_test.go | 22 ++++++++++++++++++++++ 2 files changed, 32 insertions(+), 4 deletions(-) diff --git a/pkg/util/circuit/circuitbreaker.go b/pkg/util/circuit/circuitbreaker.go index 32c24c6f8514..bdd10dd50de4 100644 --- a/pkg/util/circuit/circuitbreaker.go +++ b/pkg/util/circuit/circuitbreaker.go @@ -90,6 +90,12 @@ func (b *Breaker) Signal() Signal { return b.mu.errAndCh } +// Probe forces the breaker probe to run (if it is not already running), +// even if the breaker is not currently tripped. +func (b *Breaker) Probe() { + b.maybeTriggerProbe(true /* force */) +} + // HasMark returns whether the error has an error mark that is unique to this // breaker. In other words, the error originated at this Breaker. // @@ -146,7 +152,7 @@ func (b *Breaker) Report(err error) { // and we don't want a self-perpetuating loop of probe invocations. Instead, // we only probe when clients are actively asking the Breaker for its // status, via Breaker.Signal. - b.maybeTriggerProbe() + b.maybeTriggerProbe(false /* force */) } } @@ -224,9 +230,9 @@ func TestingSetTripped(b *Breaker, err error) (undo func()) { } } -func (b *Breaker) maybeTriggerProbe() { +func (b *Breaker) maybeTriggerProbe(force bool) { b.mu.Lock() - if b.mu.probing || b.mu.errAndCh.err == nil { + if b.mu.probing || (!force && b.mu.errAndCh.err == nil) { b.mu.Unlock() // A probe is already running or the breaker is not currently tripped. The // latter case can occur since maybeTriggerProbe is invoked from @@ -270,7 +276,7 @@ func (b *Breaker) maybeTriggerProbe() { func (b *Breaker) newErrAndCh() *errAndCh { return &errAndCh{ - maybeTriggerProbe: b.maybeTriggerProbe, + maybeTriggerProbe: func() { b.maybeTriggerProbe(false /* force */) }, ch: make(chan struct{}), } } diff --git a/pkg/util/circuit/circuitbreaker_test.go b/pkg/util/circuit/circuitbreaker_test.go index 20e9bad9c021..1fa9b1887314 100644 --- a/pkg/util/circuit/circuitbreaker_test.go +++ b/pkg/util/circuit/circuitbreaker_test.go @@ -384,6 +384,28 @@ func TestBreakerRealistic(t *testing.T) { }) } +func TestBreaker_Probe(t *testing.T) { + defer leaktest.AfterTest(t)() + _, eh := testLogBridge(t) + defer eh.RequireNumTrippedEqualsNumResets(t) + var ran bool + br := NewBreaker(Options{ + Name: "mybreaker", + AsyncProbe: func(report func(error), done func()) { + ran = true + done() + }, + EventHandler: eh, + }) + br.Probe() + testutils.SucceedsSoon(t, func() error { + if !ran { + return errors.New("probe did not run") + } + return nil + }) +} + func TestTestingSetTripped(t *testing.T) { defer leaktest.AfterTest(t)() _, tl := testLogBridge(t) From 2ffa52615afb128697175762761800f8974c9eda Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Thu, 4 May 2023 10:49:16 -0500 Subject: [PATCH 04/17] rpc: retain information about failed connections Prior to this change, `rpc.Context` did not remember a connection after it failed. A connection attempt would be created whenever we didn't have a functioning connection to the given `(nodeID, targetAddress, class)` and callers would be multiplexed onto it as long as that attempt was not known to have failed. The main problem with this setup was that it makes it difficult to provide good observability because a remote node that just hasn't been dialed yet and one that is down but between attempts looks the same. We could write code that periodically tries to fully connect all nodes in the cluster to each other, but even then exporting good metrics is challenging and while we're currently comfortable with a densely connected cluster, that may change over time. An additional challenge was presented by circuit breaking and logging of attempts. Without state retained, we were limited to a simple retry scheme with lots of redundant logging. It wasn't easy to log how long a connection had been unhealthy (as an unhealthy connection was represented as an absent connection), so in effect folks had to trawl through logs to grab the corresponding timestamps of first and last failure. Another piece of complexity were the RPC circuit breakers. These were implemented at the NodeDialer-level (i.e. one circuit breaker per `(NodeID,Class`)) but kept in the `rpc.Context` (which generally operates on `(NodeID, Addr, Class)` because gossip also used them. The library they were using uses recruitment of client traffic, which also isn't ideal as it could periodically subject a SQL query to a timeout failure. We'd really prefer it if the breakers probed without user traffic recruitment. This PR addresses the above shortcomings: - state is now retained across restarts on a `(NodeID, Addr, Class)` basis in the `rpc.Context. This has a few complications, for example we need to handle decommissioned nodes, as well as nodes that restart under a new listening address. - the NodeDialer-level circuit breakers are removed. - we're no longer recruiting user traffic for probes. Instead, we adopt the `util/circuit` package already used for Replica-level circuit breaking. This library uses an async background probe to determine when to heal the breaker. Epic: CRDB-21710 Release note: TODO(during review): explain the new metrics and any other changes. --- pkg/rpc/BUILD.bazel | 16 +- pkg/rpc/breaker.go | 10 +- pkg/rpc/clock_offset.go | 33 + pkg/rpc/connection.go | 204 ++++ pkg/rpc/context.go | 654 ++----------- pkg/rpc/context_test.go | 896 ++++++++---------- pkg/rpc/datadriven_test.go | 335 +++++++ pkg/rpc/down_node_test.go | 5 +- pkg/rpc/errors.go | 38 + pkg/rpc/metrics.go | 195 +++- pkg/rpc/nodedialer/BUILD.bazel | 2 + pkg/rpc/nodedialer/nodedialer.go | 27 +- pkg/rpc/nodedialer/nodedialer_test.go | 22 +- pkg/rpc/peer.go | 892 +++++++++++++++++ pkg/rpc/peer_map.go | 76 ++ pkg/rpc/peer_test.go | 207 ++++ pkg/rpc/settings.go | 123 +++ .../TestReconnection/decommission.txt | 79 ++ .../TestReconnection/restart_new_ip.txt | 76 ++ pkg/rpc/testdata/TestReconnection/single.txt | 71 ++ pkg/server/admin_test.go | 2 +- pkg/server/decommission.go | 11 +- .../src/store/nodes/nodes.fixtures.ts | 3 +- 23 files changed, 2870 insertions(+), 1107 deletions(-) create mode 100644 pkg/rpc/connection.go create mode 100644 pkg/rpc/datadriven_test.go create mode 100644 pkg/rpc/errors.go create mode 100644 pkg/rpc/peer.go create mode 100644 pkg/rpc/peer_map.go create mode 100644 pkg/rpc/peer_test.go create mode 100644 pkg/rpc/settings.go create mode 100644 pkg/rpc/testdata/TestReconnection/decommission.txt create mode 100644 pkg/rpc/testdata/TestReconnection/restart_new_ip.txt create mode 100644 pkg/rpc/testdata/TestReconnection/single.txt diff --git a/pkg/rpc/BUILD.bazel b/pkg/rpc/BUILD.bazel index 4f0bb0b77ea4..af221e8f25fc 100644 --- a/pkg/rpc/BUILD.bazel +++ b/pkg/rpc/BUILD.bazel @@ -12,13 +12,18 @@ go_library( "breaker.go", "clock_offset.go", "codec.go", + "connection.go", "connection_class.go", "context.go", "context_testutils.go", + "errors.go", "heartbeat.go", "keepalive.go", "metrics.go", + "peer.go", + "peer_map.go", "restricted_internal_client.go", + "settings.go", "snappy.go", "tls.go", ], @@ -39,7 +44,7 @@ go_library( "//pkg/settings/cluster", "//pkg/ts/tspb", "//pkg/util", - "//pkg/util/buildutil", + "//pkg/util/circuit", "//pkg/util/envutil", "//pkg/util/growstack", "//pkg/util/grpcutil", @@ -48,6 +53,7 @@ go_library( "//pkg/util/log/logcrash", "//pkg/util/log/severity", "//pkg/util/metric", + "//pkg/util/metric/aggmetric", "//pkg/util/netutil", "//pkg/util/stop", "//pkg/util/syncutil", @@ -89,14 +95,17 @@ go_test( "clock_offset_test.go", "codec_test.go", "context_test.go", + "datadriven_test.go", "down_node_test.go", "heartbeat_test.go", "helpers_test.go", "main_test.go", + "peer_test.go", "snappy_test.go", "tls_test.go", ], args = ["-test.timeout=55s"], + data = glob(["testdata/**"]), embed = [":rpc"], deps = [ "//pkg/base", @@ -115,9 +124,11 @@ go_test( "//pkg/settings/cluster", "//pkg/spanconfig", "//pkg/testutils", + "//pkg/testutils/datapathutils", "//pkg/testutils/skip", "//pkg/ts/tspb", "//pkg/util", + "//pkg/util/circuit", "//pkg/util/grpcutil", "//pkg/util/hlc", "//pkg/util/leaktest", @@ -129,8 +140,11 @@ go_test( "//pkg/util/timeutil", "//pkg/util/tracing", "//pkg/util/uuid", + "@com_github_cockroachdb_datadriven//:datadriven", "@com_github_cockroachdb_errors//:errors", "@com_github_cockroachdb_logtags//:logtags", + "@com_github_cockroachdb_redact//:redact", + "@com_github_gogo_status//:status", "@com_github_stretchr_testify//require", "@io_etcd_go_raft_v3//raftpb", "@org_golang_google_grpc//:go_default_library", diff --git a/pkg/rpc/breaker.go b/pkg/rpc/breaker.go index fde6394b0064..f0ae564d1d04 100644 --- a/pkg/rpc/breaker.go +++ b/pkg/rpc/breaker.go @@ -91,12 +91,18 @@ func newBackOff(clock backoff.Clock) backoff.BackOff { return b } -func newBreaker(ctx context.Context, name string, clock clock.Clock) *circuit.Breaker { +func newBreaker(ctx context.Context, name string, clock clock.Clock, noop bool) *circuit.Breaker { + shouldTrip := circuit.ThresholdTripFunc(1) + if noop { + shouldTrip = func(*circuit.Breaker) bool { + return false + } + } return circuit.NewBreakerWithOptions(&circuit.Options{ Name: name, BackOff: newBackOff(clock), Clock: clock, - ShouldTrip: circuit.ThresholdTripFunc(1), + ShouldTrip: shouldTrip, Logger: breakerLogger{ctx}, }) } diff --git a/pkg/rpc/clock_offset.go b/pkg/rpc/clock_offset.go index 490703e4e544..66f0cb4ec53b 100644 --- a/pkg/rpc/clock_offset.go +++ b/pkg/rpc/clock_offset.go @@ -391,3 +391,36 @@ func (r RemoteOffset) isHealthy(ctx context.Context, toleratedOffset time.Durati func (r RemoteOffset) isStale(ttl time.Duration, now time.Time) bool { return r.measuredAt().Add(ttl).Before(now) } + +func updateClockOffsetTracking( + ctx context.Context, + remoteClocks *RemoteClockMonitor, + nodeID roachpb.NodeID, + sendTime, serverTime, receiveTime time.Time, + toleratedOffset time.Duration, +) (time.Duration, RemoteOffset, error) { + pingDuration := receiveTime.Sub(sendTime) + if remoteClocks == nil { + // Only a server connecting to another server needs to check clock + // offsets. A CLI command does not need to update its local HLC, nor does + // it care that strictly about client-server latency, nor does it need to + // track the offsets. + + return pingDuration, RemoteOffset{}, nil + } + + var offset RemoteOffset + if pingDuration <= maximumPingDurationMult*toleratedOffset { + // Offset and error are measured using the remote clock reading + // technique described in + // http://se.inf.tu-dresden.de/pubs/papers/SRDS1994.pdf, page 6. + // However, we assume that drift and min message delay are 0, for + // now. + offset.MeasuredAt = receiveTime.UnixNano() + offset.Uncertainty = (pingDuration / 2).Nanoseconds() + remoteTimeNow := serverTime.Add(pingDuration / 2) + offset.Offset = remoteTimeNow.Sub(receiveTime).Nanoseconds() + } + remoteClocks.UpdateOffset(ctx, nodeID, offset, pingDuration) + return pingDuration, offset, remoteClocks.VerifyClockOffset(ctx) +} diff --git a/pkg/rpc/connection.go b/pkg/rpc/connection.go new file mode 100644 index 000000000000..2f58c3e3dafd --- /dev/null +++ b/pkg/rpc/connection.go @@ -0,0 +1,204 @@ +// Copyright 2023 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 rpc + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/util/circuit" + "github.com/cockroachdb/errors" + "google.golang.org/grpc" +) + +// Connection is a wrapper around grpc.ClientConn. It prevents the underlying +// connection from being used until it has been validated via heartbeat. +type Connection struct { + // Fields in this struct are only ever mutated from the circuit breaker probe, + // but they may be read widely (many callers hold a *Connection). + + // The following fields are populated on instantiation. + k peerKey + // breakerSignalFn is (*Breaker).Signal of the surrounding `*peer`. We consult + // this in Connect() to abort dial attempts when the breaker is tripped. + breakerSignalFn func() circuit.Signal + // connFuture is signaled with success (revealing the clientConn) once the + // initial heartbeat succeeds. If we fail to create a ClientConn or the + // ClientConn fails its first heartbeat, it's signaled with an error. + // + // connFuture can be signaled (like any mutation, from the probe only) without + // holding the surrounding mutex. + // + // It always has to be signaled eventually, regardless of the stopper + // draining, etc, since callers might be blocking on it. + connFuture connFuture +} + +// newConnectionToNodeID makes a Connection for the given node, class, and nontrivial Signal +// that should be queried in Connect(). +func newConnectionToNodeID(k peerKey, breakerSignal func() circuit.Signal) *Connection { + c := &Connection{ + breakerSignalFn: breakerSignal, + k: k, + connFuture: connFuture{ + ready: make(chan struct{}), + }, + } + return c +} + +// waitOrDefault blocks on initialHeartbeatDone and returns either an error or +// the unwrapped grpc connection. If the provided context or signal fire, they +// will short-circuit the waiting process. The signal may be nil in which case +// it is ignored. If a non-nil defErr is provided, waitOrDefault will never +// block but fall back to defErr in this case. +func (c *Connection) waitOrDefault( + ctx context.Context, defErr error, sig circuit.Signal, +) (*grpc.ClientConn, error) { + // Check the circuit breaker first. If it is already tripped now, we + // want it to take precedence over connFuture below (which is closed in + // the common case of a connection going bad after having been healthy + // for a while). + select { + case <-sig.C(): + return nil, sig.Err() + default: + } + + // Wait for either the breaker to trip, the caller to give up, or the waitCh + // to fire. Because we support both a default and no default in this method, + // there are two largely identical branches that should be kept in sync. + if defErr == nil { + select { + case <-c.connFuture.C(): + case <-sig.C(): + return nil, sig.Err() + case <-ctx.Done(): + return nil, errors.Wrapf(ctx.Err(), "while connecting to n%d at %s", c.k.NodeID, c.k.TargetAddr) + } + } else { + select { + case <-c.connFuture.C(): + case <-sig.C(): + return nil, sig.Err() + case <-ctx.Done(): + return nil, errors.Wrapf(ctx.Err(), "while connecting to n%d at %s", c.k.NodeID, c.k.TargetAddr) + default: + return nil, defErr + } + } + + // Done waiting, c.connFuture has resolved, return the result. Note that this + // conn could be unhealthy (or there may not even be a conn, i.e. Err() != + // nil), if that's what the caller wanted (ConnectNoBreaker). + return c.connFuture.Conn(), c.connFuture.Err() +} + +// Connect returns the underlying grpc.ClientConn after it has been validated, +// or an error if dialing or validation fails. Connect implements circuit +// breaking, i.e. there is a circuit breaker for each peer and if the breaker is +// tripped (which happens when a heartbeat fails), Connect will fail-fast with +// an error. In rare cases, this behavior is undesired and ConnectNoBreaker may +// be used instead. +func (c *Connection) Connect(ctx context.Context) (*grpc.ClientConn, error) { + return c.waitOrDefault(ctx, nil /* defErr */, c.breakerSignalFn()) +} + +type neverTripSignal struct{} + +func (s *neverTripSignal) Err() error { + return nil +} + +func (s *neverTripSignal) C() <-chan struct{} { + return nil +} + +// ConnectNoBreaker is like Connect but bypasses the circuit breaker, meaning +// that it will latch onto (or start) an existing connection attempt even if +// previous attempts have not succeeded. This may be preferable to Connect +// if the caller is already certain that a peer is available. +func (c *Connection) ConnectNoBreaker(ctx context.Context) (*grpc.ClientConn, error) { + // For ConnectNoBreaker we don't use the default Signal but pass a dummy one + // that never trips. (The probe tears down the Conn on quiesce so we don't rely + // on the Signal for that). + // + // Because peer probe attempts can become on-demand (when the peer suspects + // that it is stale and waiting to be deleted) we touch the real Signal to + // make sure a probe is attempted (async). This doesn't matter for this call - + // after all we're already tied to a *Connection - but it matters for future + // calls. We could hypothetically end up in a situation in which the + // *Connection is unhealthy and the breaker is tripped, but the *peer has + // deleteAfter set and is thus not currently running a probe. If there are no + // calls to Connect (which queries the Signal, starting a probe), the defunct + // *Connection will remain around forever. By simply reading the Signal here + // we make sure that calls to ConnectNoBreaker tip the probe off as well, + // avoiding this problem. + _ = c.Signal().Err() + return c.waitOrDefault(ctx, nil /* defErr */, &neverTripSignal{}) +} + +// Health returns an error indicating the success or failure of the connection's +// latest heartbeat. Returns ErrNotHeartbeated if the peer was just contacted for +// the first time and the first heartbeat has not occurred yet. +func (c *Connection) Health() error { + _, err := c.waitOrDefault(context.Background(), ErrNotHeartbeated, c.breakerSignalFn()) + return err +} + +func (c *Connection) Signal() circuit.Signal { + return c.breakerSignalFn() +} + +type connFuture struct { + ready chan struct{} + cc *grpc.ClientConn + err error +} + +var _ circuit.Signal = (*connFuture)(nil) + +func (s *connFuture) C() <-chan struct{} { + return s.ready +} + +// Err must only be called after C() has been closed. +func (s *connFuture) Err() error { + return s.err +} + +// Conn must only be called after C() has been closed. +func (s *connFuture) Conn() *grpc.ClientConn { + if s.err != nil { + return nil + } + return s.cc +} + +func (s *connFuture) Resolved() bool { + select { + case <-s.ready: + return true + default: + return false + } +} + +// Resolve is idempotent. Only the first call has any effect. +// Not thread safe. +func (s *connFuture) Resolve(cc *grpc.ClientConn, err error) { + select { + case <-s.ready: + // Already resolved, noop. + default: + s.cc, s.err = cc, err + close(s.ready) + } +} diff --git a/pkg/rpc/context.go b/pkg/rpc/context.go index 441d59f6f012..638d11c13c24 100644 --- a/pkg/rpc/context.go +++ b/pkg/rpc/context.go @@ -15,12 +15,10 @@ import ( "context" "crypto/tls" "encoding/binary" - "fmt" "hash/fnv" "io" "math" "net" - "sync/atomic" "time" circuit "github.com/cockroachdb/circuitbreaker" @@ -29,16 +27,13 @@ import ( "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" - "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/util" - "github.com/cockroachdb/cockroach/pkg/util/buildutil" - "github.com/cockroachdb/cockroach/pkg/util/envutil" + circuitbreaker "github.com/cockroachdb/cockroach/pkg/util/circuit" "github.com/cockroachdb/cockroach/pkg/util/growstack" "github.com/cockroachdb/cockroach/pkg/util/grpcutil" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/cockroachdb/cockroach/pkg/util/netutil" "github.com/cockroachdb/cockroach/pkg/util/stop" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -51,108 +46,12 @@ import ( "go.opentelemetry.io/otel/attribute" "google.golang.org/grpc" "google.golang.org/grpc/backoff" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/connectivity" "google.golang.org/grpc/credentials" "google.golang.org/grpc/credentials/insecure" "google.golang.org/grpc/encoding" "google.golang.org/grpc/metadata" - grpcstatus "google.golang.org/grpc/status" ) -func init() { - // Disable GRPC tracing. This retains a subset of messages for - // display on /debug/requests, which is very expensive for - // snapshots. Until we can be more selective about what is retained - // in traces, we must disable tracing entirely. - // https://github.com/grpc/grpc-go/issues/695 - grpc.EnableTracing = false -} - -const ( - // The coefficient by which the tolerated offset is multiplied to determine - // the maximum acceptable measurement latency. - maximumPingDurationMult = 2 -) - -const ( - defaultWindowSize = 65535 -) - -func getWindowSize(name string, c ConnectionClass, defaultSize int) int32 { - const maxWindowSize = defaultWindowSize * 32 - s := envutil.EnvOrDefaultInt(name, defaultSize) - if s > maxWindowSize { - log.Warningf(context.Background(), "%s value too large; trimmed to %d", name, maxWindowSize) - s = maxWindowSize - } - if s <= defaultWindowSize { - log.Warningf(context.Background(), - "%s RPC will use dynamic window sizes due to %s value lower than %d", c, name, defaultSize) - } - return int32(s) -} - -var ( - // for an RPC - initialWindowSize = getWindowSize( - "COCKROACH_RPC_INITIAL_WINDOW_SIZE", DefaultClass, defaultWindowSize*32) - initialConnWindowSize = initialWindowSize * 16 // for a connection - - // for RangeFeed RPC - rangefeedInitialWindowSize = getWindowSize( - "COCKROACH_RANGEFEED_RPC_INITIAL_WINDOW_SIZE", RangefeedClass, 2*defaultWindowSize /* 128K */) -) - -// errDialRejected is returned from client interceptors when the server's -// stopper is quiescing. The error is constructed to return true in -// `grpcutil.IsConnectionRejected` which prevents infinite retry loops during -// cluster shutdown, especially in unit testing. -var errDialRejected = grpcstatus.Error(codes.PermissionDenied, "refusing to dial; node is quiescing") - -// sourceAddr is the environment-provided local address for outgoing -// connections. -var sourceAddr = func() net.Addr { - const envKey = "COCKROACH_SOURCE_IP_ADDRESS" - if sourceAddr, ok := envutil.EnvString(envKey, 0); ok { - sourceIP := net.ParseIP(sourceAddr) - if sourceIP == nil { - panic(fmt.Sprintf("unable to parse %s '%s' as IP address", envKey, sourceAddr)) - } - return &net.TCPAddr{ - IP: sourceIP, - } - } - return nil -}() - -var enableRPCCompression = envutil.EnvOrDefaultBool("COCKROACH_ENABLE_RPC_COMPRESSION", true) - -type serverOpts struct { - interceptor func(fullMethod string) error -} - -// ServerOption is a configuration option passed to NewServer. -type ServerOption func(*serverOpts) - -// WithInterceptor adds an additional interceptor. The interceptor is called before -// streaming and unary RPCs and may inject an error. -func WithInterceptor(f func(fullMethod string) error) ServerOption { - return func(opts *serverOpts) { - if opts.interceptor == nil { - opts.interceptor = f - } else { - f := opts.interceptor - opts.interceptor = func(fullMethod string) error { - if err := f(fullMethod); err != nil { - return err - } - return f(fullMethod) - } - } - } -} - // NewServer sets up an RPC server. Depending on the ServerOptions, the Server // either expects incoming connections from KV nodes, or from tenant SQL // servers. @@ -179,14 +78,6 @@ type ClientInterceptorInfo struct { StreamInterceptors []grpc.StreamClientInterceptor } -type versionCompatError struct{} - -func (versionCompatError) Error() string { - return "version compatibility check failed on ping response" -} - -var VersionCompatError = versionCompatError{} - // NewServerEx is like NewServer, but also returns the interceptors that have // been registered with gRPC for the server. These interceptors can be used // manually when bypassing gRPC to call into the server (like the @@ -301,74 +192,12 @@ func NewServerEx( }, nil } -// Connection is a wrapper around grpc.ClientConn. It prevents the underlying -// connection from being used until it has been validated via heartbeat. -type Connection struct { - // The following fields are populated on instantiation. - - // remoteNodeID implies checking the remote node ID. 0 when unknown, - // non-zero to check with remote node. Never mutated. - remoteNodeID roachpb.NodeID - class ConnectionClass // never mutated - // err is nil initially; eventually set to the dial or heartbeat error that - // tore down the connection. - err atomic.Value - // initialHeartbeatDone is closed in `runHeartbeat` once grpcConn is populated - // and a heartbeat is successfully returned. This means that access to that - // field must read this channel first. - initialHeartbeatDone chan struct{} // closed after first heartbeat - grpcConn *grpc.ClientConn // present when initialHeartbeatDone is closed; must read that channel first -} - -func newConnectionToNodeID(remoteNodeID roachpb.NodeID, class ConnectionClass) *Connection { - c := &Connection{ - class: class, - initialHeartbeatDone: make(chan struct{}), - remoteNodeID: remoteNodeID, - } - return c -} - -// Connect returns the underlying grpc.ClientConn after it has been validated, -// or an error if dialing or validation fails. -func (c *Connection) Connect(ctx context.Context) (*grpc.ClientConn, error) { - - // Wait for initial heartbeat. - select { - case <-c.initialHeartbeatDone: - case <-ctx.Done(): - return nil, errors.Wrap(ctx.Err(), "connect") - } - - if err, _ := c.err.Load().(error); err != nil { - return nil, err // connection got destroyed - } - - return c.grpcConn, nil -} - -// Health returns an error indicating the success or failure of the -// connection's latest heartbeat. Returns ErrNotHeartbeated if the -// first heartbeat has not completed, which is the common case. -func (c *Connection) Health() error { - select { - case <-c.initialHeartbeatDone: - // NB: this path is rarely hit if the caller just pulled a fresh - // *Connection out of the connection pool (since this error is - // only populated upon removing from the pool). However, caller - // might have been holding on to this *Connection for some time. - err, _ := c.err.Load().(error) - return err - default: - // There might be a connection attempt going on, but not one that has proven - // conclusively that the peer is reachable and able to connect back to us. - // Ideally we could return ErrNoConnection, but it is hard to separate out - // these cases. - return ErrNotHeartbeated - } -} - -// Context contains the fields required by the rpc framework. +// Context is a pool of *grpc.ClientConn that are periodically health-checked, +// and for which circuit breaking and metrics are provided. Callers can obtain a +// *Connection via the non-blocking GRPCDialNode method. A *Connection is akin +// to a promise, with the `Connection.Connect` method blocking on the promise. A +// single underlying *grpc.ClientConn is maintained for each triplet of (NodeID, +// TargetAddress, ConnectionClass). // // TODO(tbg): rename at the very least the `ctx` receiver, but possibly the whole // thing. @@ -380,7 +209,7 @@ type Context struct { breakerClock breakerClock RemoteClocks *RemoteClockMonitor - MasterCtx context.Context + MasterCtx context.Context // cancel on stopper quiesce heartbeatInterval time.Duration heartbeatTimeout time.Duration @@ -389,7 +218,7 @@ type Context struct { localInternalClient RestrictedInternalClient - m connMap + peers peerMap // dialbackMap is a map of currently executing dialback connections. This map // is typically empty or close to empty. It only holds entries that are being @@ -411,8 +240,6 @@ type Context struct { clientUnaryInterceptors []grpc.UnaryClientInterceptor clientStreamInterceptors []grpc.StreamClientInterceptor - logClosingConnEvery log.EveryN - // loopbackDialFn, when non-nil, is used when the target of the dial // is ourselves (== AdvertiseAddr). // @@ -458,31 +285,6 @@ func (c *Context) SetLoopbackDialer(loopbackDialFn func(context.Context) (net.Co c.loopbackDialFn = loopbackDialFn } -// connKey is used as key in the Context.conns map. -// Connections which carry a different class but share a target and nodeID -// will always specify distinct connections. Different remote node IDs get -// distinct *Connection objects to ensure that we don't mis-route RPC -// requests in the face of address reuse. Gossip connections and other -// non-Internal users of the Context are free to dial nodes without -// specifying a node ID (see GRPCUnvalidatedDial()) however later calls to -// Dial with the same target and class with a node ID will create a new -// underlying connection which will not be reused by calls specifying the -// NodeID. -type connKey struct { - targetAddr string - // Note: this ought to be renamed, see: - // https://github.com/cockroachdb/cockroach/pull/73309 - nodeID roachpb.NodeID - class ConnectionClass -} - -var _ redact.SafeFormatter = connKey{} - -// SafeFormat implements the redact.SafeFormatter interface. -func (c connKey) SafeFormat(p redact.SafePrinter, _ rune) { - p.Printf("{n%d: %s (%v)}", c.nodeID, c.targetAddr, c.class) -} - // ContextOptions are passed to NewContext to set up a new *Context. // All pointer fields and TenantID are required. type ContextOptions struct { @@ -665,12 +467,11 @@ func NewContext(ctx context.Context, opts ContextOptions) *Context { breakerClock: breakerClock{ clock: opts.Clock, }, - rpcCompression: enableRPCCompression, - MasterCtx: masterCtx, - metrics: makeMetrics(), - heartbeatInterval: opts.Config.RPCHeartbeatInterval, - heartbeatTimeout: opts.Config.RPCHeartbeatTimeout, - logClosingConnEvery: log.Every(time.Second), + rpcCompression: enableRPCCompression, + MasterCtx: masterCtx, + metrics: makeMetrics(), + heartbeatInterval: opts.Config.RPCHeartbeatInterval, + heartbeatTimeout: opts.Config.RPCHeartbeatTimeout, } rpcCtx.dialbackMu.Lock() @@ -1624,10 +1425,10 @@ func (rpcCtx *Context) ConnHealth( if rpcCtx.GetLocalInternalClientForAddr(nodeID) != nil { return nil } - if conn, ok := rpcCtx.m.Get(connKey{target, nodeID, class}); ok { - return conn.Health() + if p, ok := rpcCtx.peers.get(peerKey{target, nodeID, class}); ok { + return p.c.Health() } - return ErrNoConnection + return ErrNotHeartbeated } type transportType bool @@ -1706,6 +1507,30 @@ func (rpcCtx *Context) dialOptsLocal() ([]grpc.DialOption, error) { return dialOpts, err } +// GetBreakerForAddr looks up a breaker for the matching (NodeID,Class,Addr). +// If it exists, it is unique. +// +// For testing purposes only. +// +// Do not call .Report() on the returned breaker. The probe manages the lifecycle +// of the breaker. +func (rpcCtx *Context) GetBreakerForAddr( + nodeID roachpb.NodeID, class ConnectionClass, addr net.Addr, +) (*circuitbreaker.Breaker, bool) { + sAddr := addr.String() + rpcCtx.peers.mu.RLock() + defer rpcCtx.peers.mu.RUnlock() + p, ok := rpcCtx.peers.mu.m[peerKey{ + TargetAddr: sAddr, + NodeID: nodeID, + Class: class, + }] + if !ok { + return nil, false + } + return p.b, true +} + // GetClientTLSConfig decides which TLS client configuration (& // certificates) to use to reach the remote node. func (rpcCtx *Context) GetClientTLSConfig() (*tls.Config, error) { @@ -2236,388 +2061,51 @@ func (rpcCtx *Context) GRPCDialPod( return rpcCtx.GRPCDialNode(target, roachpb.NodeID(remoteInstanceID), class) } -type connMap struct { - mu struct { - syncutil.RWMutex - m map[connKey]*Connection - } -} - -func (m *connMap) Get(k connKey) (*Connection, bool) { - m.mu.RLock() - defer m.mu.RUnlock() - c, ok := m.mu.m[k] - return c, ok -} - -func (m *connMap) Remove(k connKey, conn *Connection) error { - m.mu.Lock() - defer m.mu.Unlock() - mConn, found := m.mu.m[k] - if !found { - return errors.AssertionFailedf("no conn found for %+v", k) - } - if mConn != conn { - return errors.AssertionFailedf("conn for %+v not identical to those for which removal was requested", k) - } - - delete(m.mu.m, k) - return nil -} - -func (m *connMap) TryInsert(k connKey) (_ *Connection, inserted bool) { - m.mu.Lock() - defer m.mu.Unlock() - - if m.mu.m == nil { - m.mu.m = map[connKey]*Connection{} - } - - if c, lostRace := m.mu.m[k]; lostRace { - return c, false - } - - newConn := newConnectionToNodeID(k.nodeID, k.class) - - // NB: we used to also insert into `connKey{target, 0, class}` so that - // callers that may pass a zero NodeID could coalesce onto the connection - // with the "real" NodeID. This caused issues over the years[^1][^2] and - // was never necessary anyway, so we don't do it anymore. - // - // [^1]: https://github.com/cockroachdb/cockroach/issues/37200 - // [^2]: https://github.com/cockroachdb/cockroach/pull/89539 - m.mu.m[k] = newConn - return newConn, true -} - -func maybeFatal(ctx context.Context, err error) { - if err == nil || !buildutil.CrdbTestBuild { - return - } - log.FatalfDepth(ctx, 1, "%s", err) -} - // grpcDialNodeInternal connects to the remote node and sets up the async heartbeater. // This intentionally takes no `context.Context`; it uses one derived from rpcCtx.masterCtx. func (rpcCtx *Context) grpcDialNodeInternal( target string, remoteNodeID roachpb.NodeID, class ConnectionClass, ) *Connection { - k := connKey{target, remoteNodeID, class} - if conn, ok := rpcCtx.m.Get(k); ok { - // There's a cached connection. - return conn - } - - ctx := rpcCtx.makeDialCtx(target, remoteNodeID, class) - - conn, inserted := rpcCtx.m.TryInsert(k) - if !inserted { - // Someone else won the race. - return conn - } - - // We made a connection and registered it. Others might already be accessing - // it now, but it's our job to kick off the async goroutine that will do the - // dialing and health checking of this connection (including eventually - // removing this connection should it become unhealthy). Until that reports - // back, any other callers will block on `c.initialHeartbeatDone` in - // Connect(). - if err := rpcCtx.Stopper.RunAsyncTask( - ctx, - "rpc.Context: heartbeat", func(ctx context.Context) { - rpcCtx.metrics.HeartbeatLoopsStarted.Inc(1) - - // Run the heartbeat; this will block until the connection breaks for - // whatever reason. We don't actually have to do anything with the error, - // so we ignore it. - _ = rpcCtx.runHeartbeat(ctx, conn, target) - maybeFatal(ctx, rpcCtx.m.Remove(k, conn)) - - // Context gets canceled on server shutdown, and if that's likely why - // the connection ended don't increment the metric as a result. We don't - // want activity on the metric every time a node gracefully shuts down. - // - // NB: the ordering here is such that the metric is decremented *after* - // the connection is removed from the pool. No strong reason but feels - // nicer that way and makes for fewer test flakes. - if ctx.Err() == nil { - rpcCtx.metrics.HeartbeatLoopsExited.Inc(1) - } - }); err != nil { - // If node is draining (`err` will always equal stop.ErrUnavailable - // here), return special error (see its comments). - _ = err // ignore this error - conn.err.Store(errDialRejected) - close(conn.initialHeartbeatDone) - maybeFatal(ctx, rpcCtx.m.Remove(k, conn)) + k := peerKey{TargetAddr: target, NodeID: remoteNodeID, Class: class} + if p, ok := rpcCtx.peers.get(k); ok { + // There's a cached peer, so we have a cached connection, use it. + return p.c } - return conn -} - -// NewBreaker creates a new circuit breaker properly configured for RPC -// connections. name is used internally for logging state changes of the -// returned breaker. -func (rpcCtx *Context) NewBreaker(name string) *circuit.Breaker { - if rpcCtx.BreakerFactory != nil { - return rpcCtx.BreakerFactory() - } - return newBreaker(rpcCtx.MasterCtx, name, &rpcCtx.breakerClock) -} + // Slow path. Race to create a peer. + conns := &rpcCtx.peers -// ErrNotHeartbeated is returned by ConnHealth when we have not yet performed -// the first heartbeat. -var ErrNotHeartbeated = errors.New("not yet heartbeated") + conns.mu.Lock() + defer conns.mu.Unlock() -// ErrNoConnection is returned by ConnHealth when no connection exists to -// the node. -var ErrNoConnection = errors.New("no connection found") - -// TODO(baptist): Remove in 23.2 (or 24.1) once validating dialback works for all scenarios. -var useDialback = settings.RegisterBoolSetting( - settings.TenantReadOnly, - "rpc.dialback.enabled", - "if true, require bidirectional RPC connections between nodes to prevent one-way network unavailability", - true, -) - -// runHeartbeat synchronously runs the heartbeat loop for the given RPC -// connection. The ctx passed as argument must be derived from rpcCtx.masterCtx, -// so that it respects the same cancellation policy. -func (rpcCtx *Context) runHeartbeat( - ctx context.Context, conn *Connection, target string, -) (retErr error) { - defer func() { - var initialHeartbeatDone bool - select { - case <-conn.initialHeartbeatDone: - initialHeartbeatDone = true - default: - if retErr != nil { - retErr = &netutil.InitialHeartbeatFailedError{WrappedErr: retErr} - } - } - - if retErr != nil { - conn.err.Store(retErr) - if ctx.Err() == nil { - // If the remote peer is down, we'll get fail-fast errors and since we - // don't have circuit breakers at the rpcCtx level, we need to avoid a - // busy loop of corresponding logging. We ask the EveryN only if we're - // looking at an InitialHeartbeatFailedError; if we did manage to - // heartbeat at least once we're not in the busy loop case and want to - // log unconditionally. - if neverHealthy := errors.HasType( - retErr, (*netutil.InitialHeartbeatFailedError)(nil), - ); !neverHealthy || rpcCtx.logClosingConnEvery.ShouldLog() { - var buf redact.StringBuilder - if neverHealthy { - buf.Printf("unable to connect (is the peer up and reachable?): %v", retErr) - } else { - buf.Printf("closing connection after: %s", retErr) - } - log.Health.Errorf(ctx, "%s", buf) - } - } - } - if grpcConn := conn.grpcConn; grpcConn != nil { - _ = grpcConn.Close() // nolint:grpcconnclose - } - if initialHeartbeatDone { - rpcCtx.metrics.HeartbeatsNominal.Dec(1) - } else { - close(conn.initialHeartbeatDone) // unblock any waiters - } - if rpcCtx.RemoteClocks != nil { - rpcCtx.RemoteClocks.OnDisconnect(ctx, conn.remoteNodeID) - } - }() - - { - var err error - conn.grpcConn, err = rpcCtx.grpcDialRaw(ctx, target, conn.class, rpcCtx.testingDialOpts...) - if err != nil { - // Note that grpcConn will actually connect in the background, so it's - // unusual to hit this case. - return err - } - if rpcCtx.RemoteClocks != nil { - rpcCtx.RemoteClocks.OnConnect(ctx, conn.remoteNodeID) - } + if p, lostRace := conns.getRLocked(k); lostRace { + return p.c } - // Start heartbeat loop. + // Won race. Actually create a peer. - // The request object. Note that we keep the same object from - // heartbeat to heartbeat: we compute a new .Offset at the end of - // the current heartbeat as input to the next one. - request := &PingRequest{ - OriginAddr: rpcCtx.Config.AdvertiseAddr, - TargetNodeID: conn.remoteNodeID, - ServerVersion: rpcCtx.Settings.Version.BinaryVersion(), - LocalityAddress: rpcCtx.Config.LocalityAddresses, + if conns.mu.m == nil { + conns.mu.m = map[peerKey]*peer{} } - heartbeatClient := NewHeartbeatClient(conn.grpcConn) - - var heartbeatTimer timeutil.Timer - defer heartbeatTimer.Stop() - - // Give the first iteration a wait-free heartbeat attempt. - heartbeatTimer.Reset(0) - - // All errors are considered permanent. We already have a connection that - // should be healthy; and we don't allow gRPC to reconnect under the hood. - // So whenever anything goes wrong during heartbeating, we throw away the - // connection; a new one will be created by the connection pool as needed. - // This simple model should work well in practice and it avoids serious - // problems that could arise from keeping unhealthy connections in the pool. - connFailedCh := make(chan connectivity.State, 1) - first := true - for { - select { - case <-ctx.Done(): - return nil // server shutting down - case <-heartbeatTimer.C: - heartbeatTimer.Read = true - case <-connFailedCh: - // gRPC has signaled that the connection is now failed, which implies that - // we will need to start a new connection (since we set things up that way - // using onlyOnceDialer). But we go through the motions and run the - // heartbeat so that there is a unified path that reports the error, - // in order to provide a good UX. - } - - if err := rpcCtx.Stopper.RunTaskWithErr(ctx, "rpc heartbeat", func(ctx context.Context) error { - // Pick up any asynchronous update to clusterID and NodeID. - clusterID := rpcCtx.StorageClusterID.Get() - request.ClusterID = &clusterID - request.OriginNodeID = rpcCtx.NodeID.Get() - - interceptor := func(context.Context, *PingRequest) error { return nil } - if fn := rpcCtx.OnOutgoingPing; fn != nil { - interceptor = fn - } - - var response *PingResponse - sendTime := rpcCtx.Clock.Now() - ping := func(ctx context.Context) error { - if err := interceptor(ctx, request); err != nil { - return err - } - var err error - // Check the setting lazily to allow toggling on/off without a restart. - if rpcCtx.NeedsDialback && useDialback.Get(&rpcCtx.Settings.SV) { - if first { - request.NeedsDialback = PingRequest_BLOCKING - } else { - request.NeedsDialback = PingRequest_NON_BLOCKING - } - } else { - request.NeedsDialback = PingRequest_NONE - } - response, err = heartbeatClient.Ping(ctx, request) - return err - } - var err error - if rpcCtx.heartbeatTimeout > 0 { - err = timeutil.RunWithTimeout(ctx, "conn heartbeat", rpcCtx.heartbeatTimeout, ping) - } else { - err = ping(ctx) - } - - if err != nil { - log.VEventf(ctx, 2, "received error on ping response from n%d, %v", conn.remoteNodeID, err) - return err - } - - // We verify the cluster name on the initiator side (instead - // of the heartbeat service side, as done for the cluster ID - // and node ID checks) so that the operator who is starting a - // new node in a cluster and mistakenly joins the wrong - // cluster gets a chance to see the error message on their - // management console. - if !rpcCtx.Config.DisableClusterNameVerification && !response.DisableClusterNameVerification { - err = errors.Wrap( - checkClusterName(rpcCtx.Config.ClusterName, response.ClusterName), - "cluster name check failed on ping response") - if err != nil { - return err - } - } - - err = checkVersion(ctx, rpcCtx.Settings.Version, response.ServerVersion) - if err != nil { - err := errors.Mark(err, VersionCompatError) - return err - } - - // Only a server connecting to another server needs to check clock - // offsets. A CLI command does not need to update its local HLC, nor does - // it care that strictly about client-server latency, nor does it need to - // track the offsets. For BLOCKING requests we can not use this - // response for updating our clocks since the observed RTT latency can be - // inflated. - if rpcCtx.RemoteClocks != nil && request.NeedsDialback != PingRequest_BLOCKING { - receiveTime := rpcCtx.Clock.Now() - - // Only update the clock offset measurement if we actually got a - // successful response from the server. - pingDuration := receiveTime.Sub(sendTime) - if pingDuration > maximumPingDurationMult*rpcCtx.ToleratedOffset { - request.Offset.Reset() - } else { - // Offset and error are measured using the remote clock reading - // technique described in - // http://se.inf.tu-dresden.de/pubs/papers/SRDS1994.pdf, page 6. - // However, we assume that drift and min message delay are 0, for - // now. - request.Offset.MeasuredAt = receiveTime.UnixNano() - request.Offset.Uncertainty = (pingDuration / 2).Nanoseconds() - remoteTimeNow := timeutil.Unix(0, response.ServerTime).Add(pingDuration / 2) - request.Offset.Offset = remoteTimeNow.Sub(receiveTime).Nanoseconds() - } - rpcCtx.RemoteClocks.UpdateOffset(ctx, conn.remoteNodeID, request.Offset, pingDuration) - if err := rpcCtx.RemoteClocks.VerifyClockOffset(ctx); err != nil && rpcCtx.FatalOnOffsetViolation { - log.Ops.Fatalf(ctx, "%v", err) - } - } - - return nil - }); err != nil { - return err - } + p := rpcCtx.newPeer(k) + // (Asynchronously) Start the probe (= heartbeat loop). The breaker is healthy + // right now (it was just created) but the call to `.Probe` will launch the + // probe[1] regardless. + // + // [1]: see (*peer).launch. + p.b.Probe() - if first { - // First heartbeat succeeded. - rpcCtx.metrics.HeartbeatsNominal.Inc(1) - close(conn.initialHeartbeatDone) - log.Health.Infof(ctx, "connection is now ready") - // The connection should be `Ready` now since we just used it for a - // heartbeat RPC. Any additional state transition indicates that we need - // to remove it, and we want to do so reactively. Unfortunately, gRPC - // forces us to spin up a separate goroutine for this purpose even - // though it internally uses a channel. - // Note also that the implementation of this in gRPC is clearly racy, - // so consider this somewhat best-effort. - _ = rpcCtx.Stopper.RunAsyncTask(ctx, "conn state watcher", func(ctx context.Context) { - st := connectivity.Ready - for { - if !conn.grpcConn.WaitForStateChange(ctx, st) { - return - } - st = conn.grpcConn.GetState() - if st == connectivity.TransientFailure { - connFailedCh <- st - return - } - } - }) - } + // NB: we used to also insert into `peerKey{target, 0, class}` so that + // callers that may pass a zero NodeID could coalesce onto the connection + // with the "real" NodeID. This caused issues over the years[^1][^2] and + // was never necessary anyway, so we don't do it anymore. + // + // [^1]: https://github.com/cockroachdb/cockroach/issues/37200 + // [^2]: https://github.com/cockroachdb/cockroach/pull/89539 - heartbeatTimer.Reset(rpcCtx.heartbeatInterval) - first = false - } + conns.mu.m[k] = p + return p.snap().c } // NewHeartbeatService returns a HeartbeatService initialized from the Context. @@ -2649,7 +2137,7 @@ func (rpcCtx *Context) NewHeartbeatService() *HeartbeatService { // clear out any pending attempts as soon as a successful connection is // established. func (rpcCtx *Context) VerifyDialback( - ctx context.Context, request *PingRequest, response *PingResponse, locality roachpb.Locality, + ctx context.Context, request *PingRequest, _ *PingResponse, locality roachpb.Locality, ) error { if request.NeedsDialback == PingRequest_NONE { return nil diff --git a/pkg/rpc/context_test.go b/pkg/rpc/context_test.go index 212ebcce27b4..962ab342656d 100644 --- a/pkg/rpc/context_test.go +++ b/pkg/rpc/context_test.go @@ -42,6 +42,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" "github.com/cockroachdb/logtags" + gogostatus "github.com/gogo/status" "github.com/stretchr/testify/require" "google.golang.org/grpc" "google.golang.org/grpc/codes" @@ -112,6 +113,226 @@ func newTestContext( }) } +// TestReconnectAfterAddressChange verifies that when a peer restarts under a +// different IP, a connection can be established and removes the previous +// connection from the Context. +// +// TODO(tbg): try to extend TestDataDriven to cover these scenarios; +// they are pretty cumbersome to test as is. +func TestReconnectAfterAddressChange(t *testing.T) { + defer leaktest.AfterTest(t)() + + ctx := context.Background() + stopper := stop.NewStopper() + defer stopper.Stop(ctx) + + // Shared cluster ID by all RPC peers (this ensures that the peers + // don't talk to servers from unrelated tests by accident). + clusterID := uuid.MakeV4() + + clock := timeutil.DefaultTimeSource{} + maxOffset := time.Duration(250) + serverCtx := newTestContext(clusterID, clock, maxOffset, stopper) + + const serverNodeID = 1 + serverCtx.NodeID.Set(context.Background(), serverNodeID) + s1 := newTestServer(t, serverCtx) + s2 := newTestServer(t, serverCtx) + + heartbeat := &ManualHeartbeatService{ + ready: make(chan error), + stopper: stopper, + clock: clock, + maxOffset: maxOffset, + remoteClockMonitor: serverCtx.RemoteClocks, + version: serverCtx.Settings.Version, + nodeID: serverCtx.NodeID, + } + close(heartbeat.ready) + RegisterHeartbeatServer(s1, heartbeat) + RegisterHeartbeatServer(s2, heartbeat) + + clientCtx := newTestContext(clusterID, clock, maxOffset, stopper) + clientCtx.heartbeatInterval = 10 * time.Millisecond + + tmpStopper := stop.NewStopper() + defer tmpStopper.Stop(ctx) + ln1, err := netutil.ListenAndServeGRPC(tmpStopper, s1, util.TestAddr) + require.NoError(t, err) + + // NB: we need to create this listener while ln1 is also open, or ln2 might + // get the same listen address previously used by ln1, which means that the + // circuit breaker is shared between both targets and the test gets confused. + ln2, err := netutil.ListenAndServeGRPC(serverCtx.Stopper, s2, util.TestAddr) + require.NoError(t, err) + + k1 := peerKey{ + TargetAddr: ln1.Addr().String(), + NodeID: serverNodeID, + Class: DefaultClass, + } + k1U := peerKey{ + TargetAddr: k1.TargetAddr, + NodeID: 0, + Class: SystemClass, + } + + { + conn := clientCtx.GRPCDialNode(k1.TargetAddr, serverNodeID, DefaultClass) + _, err := conn.Connect(context.Background()) + require.NoError(t, err) + testutils.SucceedsSoon(t, func() error { + if err := conn.Health(); err != nil { + return err + } + _, m, _, ok := clientCtx.peers.getWithBreaker(k1) + require.True(t, ok) + am := clientCtx.Metrics() + if err := checkPeerMetrics(m, true, false, true); err != nil { + return err + } + return checkMetrics(am, 1, 0, 0, true) + }) + + // Also make an unvalidated conn because they're tricky because + // we don't have the privilege of knowing when a target address + // is "decommissioned" (which is usually the event that triggers + // soft-deletion from the rpc.Context). + connUnvalidated := clientCtx.GRPCUnvalidatedDial(k1.TargetAddr) + _, err = conn.Connect(context.Background()) + require.NoError(t, err) + testutils.SucceedsSoon(t, func() error { + if err := connUnvalidated.Health(); err != nil { + return err + } + + _, m, _, ok := clientCtx.peers.getWithBreaker(k1U) + require.True(t, ok) + am := clientCtx.Metrics() + if err := checkPeerMetrics(m, true, false, true); err != nil { + return err + } + return checkMetrics(am, 2, 0, 0, true) + }) + + } + + { + tmpStopper.Stop(ctx) + // Peer stays in map despite becoming unhealthy (due to listener closing). + testutils.SucceedsSoon(t, func() error { + _, err := clientCtx.GRPCDialNode(k1.TargetAddr, serverNodeID, DefaultClass).Connect(ctx) + if err == nil { + return errors.New("waiting for error") + } + + _, err = clientCtx.GRPCUnvalidatedDial(k1.TargetAddr).Connect(ctx) + if err == nil { + return errors.New("waiting for error (unvalidated dial)") + } + + { + _, m, _, ok := clientCtx.peers.getWithBreaker(k1) + require.True(t, ok) + if err := checkPeerMetrics(m, false, true, true); err != nil { + return errors.Wrap(err, "k1") + } + } + + // The unvalidated peer becomes on-demand only (deleteAfter set) because it sees an unhealthy + // sibling (k1). + { + p, m, _, ok := clientCtx.peers.getWithBreaker(k1U) + require.True(t, ok) + if p.deleteAfter == 0 { + return errors.New("zero deleteAfter") + } + if err := checkPeerMetrics(m, false, false, true); err != nil { + return errors.Wrap(err, "k1U") + } + } + + return checkMetrics(clientCtx.Metrics(), 0, 1, 1, true) + }) + } + + // Now connect to n1 but now listening on ln2. This should work. + t.Logf("reconnecting at %s", ln2.Addr()) + k2 := peerKey{ + TargetAddr: ln2.Addr().String(), + NodeID: serverNodeID, + Class: DefaultClass, + } + conn := clientCtx.GRPCDialNode(k2.TargetAddr, serverNodeID, DefaultClass) + _, err = conn.Connect(context.Background()) + require.NoError(t, err) + testutils.SucceedsSoon(t, conn.Health) + // Peer should be in map (with ln2's address), and the previous peer (with + // ln1's address) realized that it has been replaced. + _, _, _, ok := clientCtx.peers.getWithBreaker(k2) + require.True(t, ok) + var failedConnCount int64 + testutils.SucceedsSoon(t, func() error { + { + p, m, _, ok := clientCtx.peers.getWithBreaker(k1) + require.True(t, ok) + if p.deleteAfter == 0 { + return errors.New("deleteAfter still nil") + } + // Peer should count neither as healthy nor as unhealthy: it has empty metrics. + // We're mostly pretending it's not there. + if err := checkPeerMetrics(m, false, false, true); err != nil { + return err + } + + // Verify that we're still trying to reconnect on demand, i.e. connection + // failure count goes up as we redial. Note that GRPCDialNode itself doesn't + // lead to an increment, instead, it makes sure the probe makes an attempt + // but this is async. + _, err := clientCtx.GRPCDialNode(k1.TargetAddr, k1.NodeID, k1.Class).Connect(ctx) + require.Error(t, err) + + if failedConnCount == 0 { + failedConnCount = m.ConnectionFailures.Value() + } else if n := m.ConnectionFailures.Value(); n <= failedConnCount { + return errors.Errorf("waiting for ConnectionFailures to increment from %d", n) + } + } + + // k1U remains in the state from before, with deleteAfter set. + p, m, _, ok := clientCtx.peers.getWithBreaker(k1U) + require.True(t, ok) + if p.deleteAfter == 0 { + return errors.New("deleteAfter still nil") + } + if err := checkPeerMetrics(m, false, false, true); err != nil { + return err + } + + // The two connections to ln1 are "inactive". Only the connection to ln2 is + // healthy. + return checkMetrics(clientCtx.Metrics(), 1, 0, 2, true) + }) + + // Disable the circuit breaker setting and watch the map clear out. Because p1 and p1U + // have deleteAfter set, we need to access the breaker to make sure the probe runs at + // least once (to perform the deletion). + // (In real code, this would happen if a client calls .Connect). + enableRPCCircuitBreakers.Override(context.Background(), &clientCtx.Settings.SV, false) + testutils.SucceedsSoon(t, func() error { + if p1, _, b, ok := clientCtx.peers.getWithBreaker(k1); ok { + _ = b.Signal().Err() // make sure breaker is running + return errors.Errorf("p1 still there: %+v", p1) + } + if p1U, _, b, ok := clientCtx.peers.getWithBreaker(k1U); ok { + _ = b.Signal().Err() // make sure breaker is running + return errors.Errorf("p1U still there: %+v", p1U) + } + return nil + }) + require.NoError(t, checkMetrics(clientCtx.Metrics(), 1, 0, 0, true)) +} + // TestPingInterceptors checks that OnOutgoingPing and OnIncomingPing can inject errors. func TestPingInterceptors(t *testing.T) { defer leaktest.AfterTest(t)() @@ -165,10 +386,10 @@ func TestPingInterceptors(t *testing.T) { { _, err := rpcCtx.GRPCDialNode(remoteAddr, blockedOriginNodeID, SystemClass).Connect(ctx) require.True(t, errors.HasType(err, errBoomRecv)) - status, ok := status.FromError(errors.UnwrapAll(err)) + st, ok := status.FromError(errors.UnwrapAll(err)) require.True(t, ok) - require.Equal(t, codes.FailedPrecondition, status.Code()) - require.Equal(t, recvMsg, status.Message()) + require.Equal(t, codes.FailedPrecondition, st.Code()) + require.Equal(t, recvMsg, st.Message()) } } @@ -243,14 +464,25 @@ func testClockOffsetInPingRequestInternal(t *testing.T, clientOnly bool) { _, err = rpcCtxClient.GRPCDialNode(remoteAddr, 1, SystemClass).Connect(ctx) require.NoError(t, err) - t.Logf("first ping check") - firstPing := <-pings - require.Zero(t, firstPing.Offset.Offset) - require.Zero(t, firstPing.Offset.Uncertainty) - require.Zero(t, firstPing.Offset.MeasuredAt) + // The first ping establishes the TCP+TLS connection and uses a blocking dialback, + // so it's usually pretty noisy in terms of detecting clock offsets. The second + // ping thus has no previous useful offset either, but it returns measurements + // then used by the third ping. + t.Logf("first two pings check") + for i := 0; i < 2; i++ { + firstPing := <-pings + require.Zero(t, firstPing.Offset.Offset) + require.Zero(t, firstPing.Offset.Uncertainty) + require.Zero(t, firstPing.Offset.MeasuredAt) + } for i := 1; i < 3; i++ { t.Logf("ping %d check", i) - nextPing := <-pings + var nextPing PingRequest + select { + case nextPing = <-pings: + case <-time.After(testutils.DefaultSucceedsSoonDuration): + t.Fatalf("timed out waiting for ping #%d", i) + } if !clientOnly { require.NotZero(t, nextPing.Offset.Offset, i) require.NotZero(t, nextPing.Offset.Uncertainty, i) @@ -788,11 +1020,10 @@ func BenchmarkInternalClientAdapter(b *testing.B) { } } -// TestHeartbeatHealth verifies that the health status changes after -// heartbeats succeed or fail. -func TestHeartbeatHealth(t *testing.T) { +// TestConnectLoopback verifies that we correctly go through the +// internal server when dialing the local address. +func TestConnectLoopback(t *testing.T) { defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) stopper := stop.NewStopper() defer stopper.Stop(context.Background()) @@ -800,441 +1031,40 @@ func TestHeartbeatHealth(t *testing.T) { clock := timeutil.NewManualTime(timeutil.Unix(0, 1)) maxOffset := time.Duration(0) - // Shared cluster ID by all RPC peers (this ensures that the peers - // don't talk to servers from unrelated tests by accident). clusterID := uuid.MakeV4() - - const serverNodeID = 1 - const clientNodeID = 2 + const nodeID = 1 ctx := context.Background() - serverCtx := newTestContext(clusterID, clock, maxOffset, stop.NewStopper()) - defer serverCtx.Stopper.Stop(ctx) - serverCtx.NodeID.Set(context.Background(), serverNodeID) - s := newTestServer(t, serverCtx) - - heartbeat := &ManualHeartbeatService{ - ready: make(chan error), - stopper: stopper, - clock: clock, - maxOffset: maxOffset, - remoteClockMonitor: serverCtx.RemoteClocks, - version: serverCtx.Settings.Version, - nodeID: serverCtx.NodeID, - } - RegisterHeartbeatServer(s, heartbeat) - - errFailedHeartbeat := errors.New("failed heartbeat") - - var hbSuccess atomic.Value - hbSuccess.Store(true) - - go func() { - for { - var err error - if !hbSuccess.Load().(bool) { - err = errFailedHeartbeat - } - - select { - case <-stopper.ShouldQuiesce(): - return - case heartbeat.ready <- err: - } - } - }() - clientCtx := newTestContext(clusterID, clock, maxOffset, stopper) - clientCtx.NodeID.Set(context.Background(), clientNodeID) - - lisNotLocalServer, err := net.Listen("tcp", "127.0.0.1:0") - t.Logf("lisNotLocal: %s", lisNotLocalServer.Addr()) - require.NoError(t, err) - defer func() { - netutil.FatalIfUnexpected(lisNotLocalServer.Close()) - }() - lisLocalServer, err := net.Listen("tcp", "127.0.0.1:0") - t.Logf("lisLocalServer: %s", lisLocalServer.Addr()) - require.NoError(t, err) - require.NoError(t, err) - defer func() { - netutil.FatalIfUnexpected(lisLocalServer.Close()) - }() - - clientCtx.Config.Addr = lisNotLocalServer.Addr().String() - clientCtx.Config.AdvertiseAddr = lisLocalServer.Addr().String() - - // Make the interval shorter to speed up the test. - clientCtx.Config.RPCHeartbeatInterval = 1 * time.Millisecond - clientCtx.Config.RPCHeartbeatTimeout = 1 * time.Millisecond - - m := clientCtx.Metrics() - - ln, err := netutil.ListenAndServeGRPC(serverCtx.Stopper, s, util.TestAddr) - require.NoError(t, err) - remoteAddr := ln.Addr().String() - t.Logf("remoteAddr: %s [serves grpc on serverCtx]", ln.Addr()) - if _, err := clientCtx.GRPCDialNode(remoteAddr, serverNodeID, DefaultClass).Connect(ctx); err != nil { - t.Fatal(err) + clientCtx.NodeID.Set(context.Background(), nodeID) + loopLn := netutil.NewLoopbackListener(ctx, stopper) + clientCtx.loopbackDialFn = func(ctx context.Context) (net.Conn, error) { + return loopLn.Connect(ctx) } - - requireNominal := func(t *testing.T, exp int64) { - t.Helper() - require.Equal(t, exp, m.HeartbeatsNominal.Value()) - } - - // Wait for the connection. - testutils.SucceedsSoon(t, func() error { - err := clientCtx.TestingConnHealth(remoteAddr, serverNodeID) - if err != nil && !errors.Is(err, ErrNotHeartbeated) { - t.Fatal(err) - } - return err - }) - requireNominal(t, 1) - - // Should not get a healthy connection while heartbeats fail; nominal - // heartbeats should soon drop to zero as the heartbeat loop fails (which - // destroys the connection). - hbSuccess.Store(false) - testutils.SucceedsSoon(t, func() error { - n := clientCtx.Metrics().HeartbeatsNominal.Value() - if n != 0 { - return errors.Errorf("%d nominal heartbeats", n) - } - // It might take a brief moment for ConnHealth to come back - // as ErrNotHeartbeated, but it should be non-nil immediately. - err := clientCtx.TestingConnHealth(remoteAddr, serverNodeID) - require.Error(t, err) - if !errors.Is(err, ErrNotHeartbeated) { - return err - } - return nil - }) - - // Should become healthy in the presence of successful heartbeats. - hbSuccess.Store(true) - testutils.SucceedsSoon(t, func() error { - return clientCtx.TestingConnHealth(remoteAddr, serverNodeID) - }) - requireNominal(t, 1) - - // Should become unhealthy again in the presence of failing heartbeats. - hbSuccess.Store(false) - testutils.SucceedsSoon(t, func() error { - if n := m.HeartbeatsNominal.Value(); n > 0 { - return errors.Errorf("%d nominal heartbeats", n) - } - // See above for rationale. - err := clientCtx.TestingConnHealth(remoteAddr, serverNodeID) - require.Error(t, err) - if !errors.Is(err, ErrNotHeartbeated) { - return err - } - return nil - }) - - // Should become healthy in the presence of successful heartbeats. - hbSuccess.Store(true) - testutils.SucceedsSoon(t, func() error { - return clientCtx.TestingConnHealth(remoteAddr, serverNodeID) - }) - requireNominal(t, 1) - - // Ensure that non-existing connections return ErrNotHeartbeated. - lisNonExistentConnection, err := net.Listen("tcp", "127.0.0.1:0") - require.NoError(t, err) - defer func() { - netutil.FatalIfUnexpected(lisNonExistentConnection.Close()) - }() - t.Logf("lisNonExistent: %s", lisNonExistentConnection.Addr()) - if err := clientCtx.TestingConnHealth(lisNonExistentConnection.Addr().String(), 3); !errors.Is(err, ErrNotHeartbeated) { - t.Errorf("wanted ErrNotHeartbeated, not %v", err) - } - // The connection to Node 3 on the lisNonExistentConnection should be - // initializing and the server connection should be nominal. - testutils.SucceedsSoon(t, func() error { - return checkNominal(m, 1) - }) - // Ensure that there's no error connecting to the local node when an internal // server has been registered. clientCtx.SetLocalInternalServer( &internalServer{}, ServerInterceptorInfo{}, ClientInterceptorInfo{}) - require.NoError(t, clientCtx.TestingConnHealth(clientCtx.Config.AdvertiseAddr, clientNodeID)) - - // Connections should shut down again and now that we're nearing the test it's - // a good opportunity to check that what came up must go down. - hbSuccess.Store(false) - // Need to close these or we eat a 20s timeout. - netutil.FatalIfUnexpected(lisNonExistentConnection.Close()) - netutil.FatalIfUnexpected(lisNotLocalServer.Close()) - testutils.SucceedsSoon(t, func() error { - started, exited := m.HeartbeatLoopsStarted.Count(), m.HeartbeatLoopsExited.Count() - if started != exited { - return errors.Errorf("started(%d) != exited(%d)", started, exited) - } - return nil - }) -} - -func checkNominal(m *Metrics, exp int64) error { - if n := m.HeartbeatsNominal.Value(); n != exp { - return errors.Errorf("%d nominal, want %d", n, exp) - } - return nil -} - -// TestConnectionRemoveNodeIDZero verifies that when a connection initiated via -// GRPCDialNode fails, we also clean up the connection returned by -// GRPCUnvalidatedDial. -// -// See #37200. -func TestConnectionRemoveNodeIDZero(t *testing.T) { - defer leaktest.AfterTest(t)() - - ctx := context.Background() - stopper := stop.NewStopper() - defer stopper.Stop(ctx) - - clock := &timeutil.DefaultTimeSource{} - maxOffset := time.Nanosecond - clientCtx := newTestContext(uuid.MakeV4(), clock, maxOffset, stopper) - // Provoke an error. - _, err := clientCtx.GRPCDialNode("127.0.0.1:notaport", 1, DefaultClass).Connect(context.Background()) - if err == nil { - t.Fatal("expected some kind of error, got nil") - } - - // NB: this takes a moment because GRPCDialRaw only gives up on the initial - // connection after 1s (more precisely, the redialChan gets closed only after - // 1s), which seems difficult to configure ad-hoc. - testutils.SucceedsSoon(t, func() error { - var keys []connKey - clientCtx.m.mu.RLock() - defer clientCtx.m.mu.RUnlock() - for k := range clientCtx.m.mu.m { - keys = append(keys, k) - } - if len(keys) > 0 { - return errors.Errorf("still have connections %v", keys) - } - return nil - }) -} -type interceptingListener struct { - net.Listener - connCB func(net.Conn) -} - -func (ln *interceptingListener) Accept() (net.Conn, error) { - conn, err := ln.Listener.Accept() - if err == nil { - ln.connCB(conn) - } - return conn, err -} - -// TestHeartbeatHealth verifies that the health status changes after -// heartbeats succeed or fail due to transport failures. -func TestHeartbeatHealthTransport(t *testing.T) { - defer leaktest.AfterTest(t)() - - stopper := stop.NewStopper() - defer stopper.Stop(context.Background()) - - ctx := context.Background() - - // Shared cluster ID by all RPC peers (this ensures that the peers - // don't talk to servers from unrelated tests by accident). - clusterID := uuid.MakeV4() - - clock := timeutil.NewManualTime(timeutil.Unix(0, 1)) - maxOffset := time.Duration(0) - serverCtx := newTestContext(clusterID, clock, maxOffset, stopper) - const serverNodeID = 1 - serverCtx.NodeID.Set(context.Background(), serverNodeID) - // newTestServer with a custom listener. - tlsConfig, err := serverCtx.GetServerTLSConfig() - if err != nil { - t.Fatal(err) - } - s := grpc.NewServer(grpc.Creds(credentials.NewTLS(tlsConfig))) - RegisterHeartbeatServer(s, &HeartbeatService{ - clock: clock, - remoteClockMonitor: serverCtx.RemoteClocks, - clusterID: serverCtx.StorageClusterID, - nodeID: serverCtx.NodeID, - version: serverCtx.Settings.Version, - }) - - mu := struct { - syncutil.Mutex - conns []net.Conn - autoClose bool - }{} - ln := func() *interceptingListener { - ln, err := net.Listen("tcp", util.TestAddr.String()) - if err != nil { - t.Fatal(err) - } - return &interceptingListener{ - Listener: ln, - connCB: func(conn net.Conn) { - mu.Lock() - if mu.autoClose { - _ = conn.Close() - } else { - mu.conns = append(mu.conns, conn) - } - mu.Unlock() - }} - }() - - _ = stopper.RunAsyncTask(ctx, "wait-quiesce", func(context.Context) { - <-stopper.ShouldQuiesce() - netutil.FatalIfUnexpected(ln.Close()) - <-stopper.ShouldQuiesce() - s.Stop() - }) - - _ = stopper.RunAsyncTask(ctx, "serve", func(context.Context) { - netutil.FatalIfUnexpected(s.Serve(ln)) - }) - - remoteAddr := ln.Addr().String() - - clientCtx := newTestContext(clusterID, clock, maxOffset, stopper) - // Make the interval shorter to speed up the test. - clientCtx.Config.RPCHeartbeatInterval = 1 * time.Millisecond - clientCtx.Config.RPCHeartbeatTimeout = 1 * time.Millisecond - if _, err := clientCtx.GRPCDialNode(remoteAddr, serverNodeID, DefaultClass).Connect(context.Background()); err != nil { - t.Fatal(err) - } - // Everything is normal; should become healthy. - testutils.SucceedsSoon(t, func() error { - return clientCtx.TestingConnHealth(remoteAddr, serverNodeID) - }) - - closeConns := func() (numClosed int, _ error) { - mu.Lock() - defer mu.Unlock() - n := len(mu.conns) - for i := n - 1; i >= 0; i-- { - // This can spuriously return ErrClosed since the listener is closed - // before us. - if err := mu.conns[i].Close(); err != nil && !errors.Is(err, net.ErrClosed) { - return 0, err - } - mu.conns = mu.conns[:i] - } - return n, nil - } - - isUnhealthy := func(err error) bool { - // Most of the time, an unhealthy connection will get - // ErrNotHeartbeated, but there are brief periods during which we - // could get one of the grpc errors below (while the old - // connection is in the middle of closing). - if errors.Is(err, ErrNotHeartbeated) { - return true - } - // The expected code here is Unavailable, but at least on OSX you can also get - // - // rpc error: code = Internal desc = connection error: desc = "transport: authentication - // handshake failed: write tcp 127.0.0.1:53936->127.0.0.1:53934: write: broken pipe". - code := status.Code(errors.UnwrapAll(err)) - return code == codes.Unavailable || code == codes.Internal - } - - // Close all the connections until we see a failure on the main goroutine. - done := make(chan struct{}) - if err := stopper.RunAsyncTask(ctx, "busyloop-closer", func(ctx context.Context) { - for { - if _, err := closeConns(); err != nil { - log.Warningf(ctx, "%v", err) - } - select { - case <-done: - return - default: - } - } - }); err != nil { - t.Fatal(err) - } - - // We don't use SucceedsSoon because that internally uses doubling backoffs, and - // it doesn't need too much bad luck to run into the time limit. - for then := timeutil.Now(); ; { - err := func() error { - if err := clientCtx.TestingConnHealth(remoteAddr, serverNodeID); !isUnhealthy(err) { - return errors.Errorf("unexpected error: %v", err) // nolint:errwrap - } - return nil - }() - if err == nil { - break - } - if timeutil.Since(then) > 45*time.Second { - t.Fatal(err) - } - time.Sleep(10 * time.Millisecond) - } - - close(done) - - // We can reconnect and the connection becomes healthy again. - testutils.SucceedsSoon(t, func() error { - if _, err := clientCtx.GRPCDialNode(remoteAddr, serverNodeID, DefaultClass).Connect(context.Background()); err != nil { - return err - } - return clientCtx.TestingConnHealth(remoteAddr, serverNodeID) - }) - - // Close the listener and all the connections. Note that if we - // only closed the listener, recently-accepted-but-not-yet-handled - // connections could sneak in and randomly make the target healthy - // again. To avoid this, we flip the boolean below which is used in - // our handler callback to eagerly close any stragglers. - mu.Lock() - mu.autoClose = true - mu.Unlock() - if err := ln.Close(); err != nil { - t.Fatal(err) - } - - // Also terminate any existing connections. - if _, err := closeConns(); err != nil { - t.Fatal(err) - } - - // Should become unhealthy again now that the connection was closed. - testutils.SucceedsSoon(t, func() error { - err := clientCtx.TestingConnHealth(remoteAddr, serverNodeID) + sTCP := newTestServer(t, clientCtx) + sLoopback := newTestServer(t, clientCtx) + errLoopback := gogostatus.Newf(codes.DataLoss, "loopback!").Err() + RegisterHeartbeatServer(sLoopback, &ManualHeartbeatService{readyFn: func() error { + return errLoopback + }}) + require.NoError(t, stopper.RunAsyncTask(ctx, "listen-server", func(ctx context.Context) { + netutil.FatalIfUnexpected(sLoopback.Serve(loopLn)) + })) + ln, err := netutil.ListenAndServeGRPC(stopper, sTCP, util.TestAddr) + require.NoError(t, err) - if !isUnhealthy(err) { - return errors.Errorf("unexpected error: %v", err) // nolint:errwrap - } - return nil - }) + addr := ln.Addr().String() + clientCtx.Config.AdvertiseAddr = addr - // TODO(baptist): Better understand when this happens. It appears we can get - // spurious connections to other tests on a stress run. This has been - // happening for a while, but only comes out rarely when this package is - // stressed. This test is very aggressive since it is calling GRPCDialNode in - // a busy loop for 50ms. - expectedCluster := "doesn't match server cluster ID" - expectedNode := "doesn't match server node ID" - // Should stay unhealthy despite reconnection attempts. - for then := timeutil.Now(); timeutil.Since(then) < 50*clientCtx.Config.RPCHeartbeatTimeout; { - err := clientCtx.TestingConnHealth(remoteAddr, serverNodeID) - if !isUnhealthy(err) && !testutils.IsError(err, expectedCluster) && !testutils.IsError(err, expectedNode) { - t.Fatal(err) - } - } + // Connect and get the error that comes from loopbackLn, proving that we were routed there. + _, err = clientCtx.GRPCDialNode(addr, nodeID, DefaultClass).Connect(ctx) + require.Equal(t, codes.DataLoss, gogostatus.Code(errors.UnwrapAll(err)), "%+v", err) } func TestOffsetMeasurement(t *testing.T) { @@ -1349,7 +1179,13 @@ func TestFailedOffsetMeasurement(t *testing.T) { // Remove the timeout so that failure arises from exceeding the maximum // clock reading delay, not the timeout. clientCtx.heartbeatTimeout = 0 - go func() { heartbeat.ready <- nil }() // Allow one heartbeat for initialization. + // Allow two heartbeat for initialization. The first ping doesn't report an offset, + // the second one thus doesn't have an offset to work with, so it's only on the + // third one that's fully configured. + go func() { + heartbeat.ready <- nil + heartbeat.ready <- nil + }() if _, err := clientCtx.GRPCDialNode(remoteAddr, serverNodeID, DefaultClass).Connect(ctx); err != nil { t.Fatal(err) } @@ -1416,6 +1252,17 @@ func TestLatencyInfoCleanupOnClosedConnection(t *testing.T) { clientCtx.Config.RPCHeartbeatInterval = 1 * time.Millisecond clientCtx.Config.RPCHeartbeatTimeout = 1 * time.Millisecond + var hbDecommission atomic.Value + hbDecommission.Store(false) + clientCtx.OnOutgoingPing = func(ctx context.Context, req *PingRequest) error { + if hbDecommission.Load().(bool) { + return kvpb.NewDecommissionedStatusErrorf( + codes.PermissionDenied, "injected decommissioned error for n%d", req.TargetNodeID, + ) + } + return nil + } + conn, err := clientCtx.GRPCDialNode(remoteAddr, serverNodeID, DefaultClass).Connect(ctx) if err != nil { t.Fatal(err) @@ -1455,7 +1302,7 @@ func TestLatencyInfoCleanupOnClosedConnection(t *testing.T) { // Close last anotherConn to simulate network disruption. err = anotherConn.Close() // nolint:grpcconnclose require.NoError(t, err) - + hbDecommission.Store(true) testutils.SucceedsSoon(t, func() error { clientCtx.RemoteClocks.mu.Lock() defer clientCtx.RemoteClocks.mu.Unlock() @@ -1905,7 +1752,7 @@ func grpcRunKeepaliveTestCase(testCtx context.Context, c grpcKeepaliveTestCase) // Context.ConnHealth() for the remote node moves to UNAVAILABLE because of // the (application-level) heartbeats performed by rpc.Context, but the // behavior of our heartbeats in the face of transport failures is - // sufficiently tested in TestHeartbeatHealthTransport. + // sufficiently tested elsewhere. log.Infof(ctx, "test done") return nil } @@ -2472,7 +2319,7 @@ func TestRejectDialOnQuiesce(t *testing.T) { // First, we shouldn't be able to dial again, even though we already have a // connection. _, err = rpcCtx.GRPCDialNode(addr, serverNodeID, SystemClass).Connect(ctx) - require.ErrorIs(t, err, errDialRejected) + require.ErrorIs(t, err, errQuiescing) require.True(t, grpcutil.IsConnectionRejected(err)) require.True(t, grpcutil.IsAuthError(err)) @@ -2549,11 +2396,15 @@ func (d *trackingListener) Close() error { } func newRegisteredServer( - t testing.TB, stopper *stop.Stopper, clusterID uuid.UUID, nodeID roachpb.NodeID, + t testing.TB, + ctx context.Context, + stopper *stop.Stopper, + clusterID uuid.UUID, + nodeID roachpb.NodeID, ) (*Context, string, chan *PingRequest, *trackingListener) { clock := timeutil.NewManualTime(timeutil.Unix(0, 1)) // We don't want to stall sending to this channel. - pingChan := make(chan *PingRequest, 5) + pingChan := make(chan *PingRequest, 1) opts := ContextOptions{ TenantID: roachpb.SystemTenantID, @@ -2572,14 +2423,21 @@ func newRegisteredServer( rpcCtx := NewContext(context.Background(), opts) // This is normally set up inside the server, we want to hold onto all PingRequests that come through. rpcCtx.OnIncomingPing = func(ctx context.Context, req *PingRequest, resp *PingResponse) error { - pingChan <- req err := rpcCtx.VerifyDialback(ctx, req, resp, roachpb.Locality{}) + if err != nil { + t.Logf("dialback error: %s", err) + return err + } // On success store the ping to the channel for test analysis. - return err + select { + case pingChan <- req: + default: + } + return nil } - rpcCtx.NodeID.Set(context.Background(), nodeID) - rpcCtx.StorageClusterID.Set(context.Background(), clusterID) + rpcCtx.NodeID.Set(ctx, nodeID) + rpcCtx.StorageClusterID.Set(ctx, clusterID) s := newTestServer(t, rpcCtx) RegisterHeartbeatServer(s, rpcCtx.NewHeartbeatService()) @@ -2587,13 +2445,17 @@ func newRegisteredServer( ln, err := net.Listen("tcp", util.TestAddr.String()) require.Nil(t, err) tracker := trackingListener{Listener: ln} - _ = stopper.RunAsyncTask(context.Background(), "serve", func(context.Context) { + stopper.OnQuiesce(func() { netutil.FatalIfUnexpected(ln.Close()) }) + + rpcCtx.OnOutgoingPing = func(ctx context.Context, req *PingRequest) error { return nil } + + _ = stopper.RunAsyncTask(ctx, "serve", func(context.Context) { closeReason := s.Serve(&tracker) - log.Infof(context.Background(), "Closed listener with reason %v", closeReason) + log.Infof(ctx, "Closed listener with reason %v", closeReason) }) addr := ln.Addr().String() - log.Infof(context.Background(), "Listening on %s", addr) + log.Infof(ctx, "Listening on %s", addr) // This needs to be set once we know our address so that ping requests have // the correct reverse addr in them. rpcCtx.Config.AdvertiseAddr = addr @@ -2604,16 +2466,13 @@ func newRegisteredServer( // into bidirectional partitions. The test sets up two nodes that are pinging each func TestHeartbeatDialback(t *testing.T) { defer leaktest.AfterTest(t)() - ctx := context.Background() stopper := stop.NewStopper() defer stopper.Stop(ctx) clusterID := uuid.MakeV4() - ctx1, remoteAddr1, pingChan1, ln1 := newRegisteredServer(t, stopper, clusterID, 1) - ctx2, remoteAddr2, pingChan2, ln2 := newRegisteredServer(t, stopper, clusterID, 2) - defer func() { netutil.FatalIfUnexpected(ln1.Close()) }() - defer func() { netutil.FatalIfUnexpected(ln2.Close()) }() + ctx1, remoteAddr1, nonRejectedPings1, _ := newRegisteredServer(t, context.Background(), stopper, clusterID, 1) + ctx2, remoteAddr2, nonRejectedPings2, ln2 := newRegisteredServer(t, context.Background(), stopper, clusterID, 2) // Test an incorrect remoteNodeID, this should fail with a heartbeat error. // This invariant is important to make sure we don't try and connect to the @@ -2623,8 +2482,8 @@ func TestHeartbeatDialback(t *testing.T) { var respErr *netutil.InitialHeartbeatFailedError require.ErrorAs(t, err, &respErr) // Verify no heartbeat received in either direction. - require.Equal(t, 0, len(pingChan1)) - require.Equal(t, 0, len(pingChan2)) + require.Equal(t, 0, len(nonRejectedPings1)) + require.Equal(t, 0, len(nonRejectedPings2)) } // Initiate connection from node 1 to node 2 which will create a dialback @@ -2632,19 +2491,19 @@ func TestHeartbeatDialback(t *testing.T) { // reverse connection. { conn, err := ctx1.GRPCDialNode(remoteAddr2, 2, DefaultClass).Connect(ctx) + require.NoError(t, err) + require.NotNil(t, conn) defer func() { _ = conn.Close() // nolint:grpcconnclose }() - require.NoError(t, err) - require.NotNil(t, conn) - require.Equal(t, 1, len(pingChan2)) - pingReq := <-pingChan2 + require.Equal(t, 1, len(nonRejectedPings2)) + pingReq := <-nonRejectedPings2 require.Equal(t, PingRequest_BLOCKING, pingReq.NeedsDialback) - require.Equal(t, 0, len(pingChan1)) + require.Equal(t, 0, len(nonRejectedPings1)) } - //Now connect back in the opposite direction. This should not initiate any - //dialback since we are already connected. + // Now connect back in the opposite direction. This should not initiate any + // dialback since we are already connected. { conn, err := ctx1.GRPCDialNode(remoteAddr2, 2, DefaultClass).Connect(ctx) defer func() { @@ -2653,47 +2512,118 @@ func TestHeartbeatDialback(t *testing.T) { require.NoError(t, err) require.NotNil(t, conn) // The reverse connection was already set up, but we are still blocking. - pingReq := <-pingChan1 + pingReq := <-nonRejectedPings1 require.Equal(t, PingRequest_BLOCKING, pingReq.NeedsDialback) // At this point, node 1 has a fully established connection to node 2, however node 2 has not yet finished connecting back. require.Equal(t, nil, ctx1.ConnHealth(remoteAddr2, 2, DefaultClass)) } - // Verify we get non-blocking requests in both directions now. - require.Equal(t, PingRequest_NON_BLOCKING, (<-pingChan2).NeedsDialback) - require.Equal(t, PingRequest_NON_BLOCKING, (<-pingChan1).NeedsDialback) + // Verify we get non-blocking requests in both directions now. A blocking one might + // still be on the channel, so we need to retry. + testutils.SucceedsSoon(t, func() error { + nd1 := (<-nonRejectedPings1).NeedsDialback + nd2 := (<-nonRejectedPings2).NeedsDialback + if nd1 != PingRequest_NON_BLOCKING || nd2 != PingRequest_NON_BLOCKING { + return errors.Errorf("n1: %v n2: %v", nd1, nd2) + } + return nil + }) // Verify we are fully healthy in both directions (note the dialback is on the // system class). - require.Equal(t, nil, ctx1.ConnHealth(remoteAddr2, 2, DefaultClass)) - require.Equal(t, nil, ctx2.ConnHealth(remoteAddr1, 1, SystemClass)) + require.NoError(t, ctx1.ConnHealth(remoteAddr2, 2, DefaultClass)) + require.NoError(t, ctx2.ConnHealth(remoteAddr1, 1, SystemClass)) // Forcibly shut down listener 2 and the connection node1 -> node2. // Test the reverse connection also closes within ~RPCHeartbeatTimeout. - log.Info(ctx, "Closing node 2 listener") + t.Logf("Closing node 2 listener") _ = ln2.Close() - // Wait for a few more pings to go through to make sure it has a chance to - // shut down the reverse connection. Normally the connect attempt times out - // immediately and returns an error, but occasionally it needs to wait for the - // RPCHeartbeatTimeout (100 ms). Wait until pings have stopped in both - // directions for at least 1 second before checking health. + // n1 -> n2 can not be healthy for much longer since the listener for n2 is + // closed, and n2 should terminate its connection to n1 within ~100ms + // (heartbeat interval in this test) since n1 can't dial-back. We'll give a + // generous 10s for that to happen, but bail early once we see connections in + // both directions marked as unhealthy. + tr := timeutil.NewTimer() + defer tr.Stop() + tr.Reset(10 * time.Second) for { + strict := false + // After a short moment, should no longer see pings n2->n1 here because + // n1's dialback will fail (so they're rejected), even though n2's circuit + // breaker will regularly try to reach out. select { - case ping := <-pingChan1: - log.Infof(ctx, "Received %+v", ping) - case ping := <-pingChan2: - log.Infof(ctx, "Received %+v", ping) - case <-time.After(1 * time.Second): - err1 := ctx1.ConnHealth(remoteAddr2, 2, DefaultClass) - require.True(t, errors.Is(err1, ErrNoConnection), "%+v", err1) - err2 := ctx2.ConnHealth(remoteAddr1, 1, SystemClass) - require.True(t, errors.Is(err2, ErrNoConnection), "%+v", err2) + case ping := <-nonRejectedPings1: + t.Logf("Received ping n%d->n%d", ping.OriginNodeID, ping.TargetNodeID) + case ping := <-nonRejectedPings2: + t.Logf("Received ping n%d->n%d", ping.OriginNodeID, ping.TargetNodeID) + case <-tr.C: + tr.Read = true + strict = true + case <-time.After(time.Millisecond): + } + + errDef, errSys := ctx1.ConnHealth(remoteAddr2, 2, DefaultClass), ctx2.ConnHealth(remoteAddr1, 1, SystemClass) + if errors.Is(errDef, ErrNotHeartbeated) { + errDef = nil + } + if errors.Is(errSys, ErrNotHeartbeated) { + errSys = nil + } + if errors.HasType(errDef, (*netutil.InitialHeartbeatFailedError)(nil)) { + errDef = nil + } + if errors.HasType(errSys, (*netutil.InitialHeartbeatFailedError)(nil)) { + errSys = nil + } + if errDef == nil && errSys == nil { return } + + if strict { + require.NoError(t, errDef) + require.NoError(t, errSys) + } } } // TODO(baptist): Add a test using TestCluster to verify this works in a full // integration test. + +func checkMetrics(m *Metrics, healthy, unhealthy, inactive int64, checkDurations bool) error { + // NB: peers can be neither healthy nor unhealthy. This happens when + // a peer is first created and when its deleteAfter field is set. + if exp, n := healthy, m.ConnectionHealthy.Value(); exp != n { + return errors.Errorf("ConnectionHealthy = %d", n) + } + if exp, n := unhealthy, m.ConnectionUnhealthy.Value(); exp != n { + return errors.Errorf("ConnectionUnhealthy = %d", n) + } + if exp, n := inactive, m.ConnectionInactive.Value(); exp != n { + return errors.Errorf("ConnectionInactive = %d", n) + } + + if !checkDurations { + return nil + } + + // HealthyFor is nonzero if and only if there is at least one healthy connection, since it's + // the sum of the connections' HealthyFor, and on a connection this is nonzero if and only + // if the connection is healthy. + if hf := m.ConnectionHealthyFor.Value(); (hf > 0) != (healthy > 0) { + return errors.Errorf("#healthy is %d but ConnectionHealthyFor is %v", healthy, hf) + } + // UnhealthyFor is nonzero if and only if there is at least one unhealthy connection, for the + // same reasons as above. + if uf := m.ConnectionUnhealthyFor.Value(); (uf > 0) != (unhealthy > 0) { + return errors.Errorf("#unhealthy is %d but ConnectionUnHealthyFor is %v", unhealthy, uf) + } + + // Similar to the two above, only healthy connections should maintain the avg round-trip latency. + if v := m.ConnectionAvgRoundTripLatency.Value(); (v > 0) != (healthy > 0) { + return errors.Errorf("ConnectionAvgRoundTripLatency = %v", v) + } + + return nil +} diff --git a/pkg/rpc/datadriven_test.go b/pkg/rpc/datadriven_test.go new file mode 100644 index 000000000000..4ebfbdc201da --- /dev/null +++ b/pkg/rpc/datadriven_test.go @@ -0,0 +1,335 @@ +// Copyright 2023 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 rpc + +import ( + "context" + "fmt" + "regexp" + "strconv" + "strings" + "sync/atomic" + "testing" + "time" + + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/datapathutils" + "github.com/cockroachdb/cockroach/pkg/util" + "github.com/cockroachdb/cockroach/pkg/util/circuit" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/netutil" + "github.com/cockroachdb/cockroach/pkg/util/stop" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/uuid" + "github.com/cockroachdb/datadriven" + "github.com/cockroachdb/errors" + "github.com/cockroachdb/redact" + "github.com/stretchr/testify/require" + "google.golang.org/grpc" + "google.golang.org/grpc/codes" + grpcstatus "google.golang.org/grpc/status" +) + +/* +TestReconnection supports the following input data: + - tick + - dial class=(def|sys|rf) + - connect class=(def|sys|rf) + - set-hb-err + - reset-hb-err + - soon [healthy=] [unhealthy=] [inactive=] + Verify that metrics in report matching number of healthy and unhealthy connections. + +Time is automatically ticked by 1s per command. The environment currently sets +up n1 with two listeners, which are addressed via as `n1` and `n1'`. +Additional copies of n1 or additional nodes can be added in setupEnv() or via +(to be added) DSL. +*/ +func TestReconnection(t *testing.T) { + defer leaktest.AfterTest(t)() + datadriven.Walk(t, datapathutils.TestDataPath(t, t.Name()), func(t *testing.T, path string) { + env := setupEnv(t) + defer env.stopper.Stop(context.Background()) + datadriven.RunTest(t, path, func(t *testing.T, d *datadriven.TestData) string { + ctx, collect := tracing.ContextWithRecordingSpan(context.Background(), env.tracer, d.Cmd) + + defer collect() + env.clock.Advance(time.Second) + switch d.Cmd { + case "tick": + require.Len(t, d.CmdArgs, 1) + d, err := time.ParseDuration(d.CmdArgs[0].Key) + require.NoError(t, err) + env.clock.Advance(d) + log.Eventf(ctx, "%s", env.clock.Now()) + case "dial": + env.handleDial(env.lookupTarget(t, d.CmdArgs...), scanClass(t, d)) + case "connect": + env.handleConnect(ctx, env.lookupTarget(t, d.CmdArgs...), scanClass(t, d)) + case "show": + env.handleShow(ctx, env.lookupTarget(t, d.CmdArgs...), scanClass(t, d)) + case "set-hb-err": + var dc bool + d.MaybeScanArgs(t, "decommissioned", &dc) + var err error + if dc { + err = kvpb.NewDecommissionedStatusErrorf(codes.PermissionDenied, "injected decommissioned error") + } else { + err = errors.New("boom") + } + + env.handleSetHeartbeatError(err, env.lookupTargets(t, d.CmdArgs...)...) + case "reset-hb-err": + env.handleSetHeartbeatError(nil /* err */, env.lookupTargets(t, d.CmdArgs...)...) + case "soon": + var healthy int64 + var unhealthy int64 + var inactive int64 + d.MaybeScanArgs(t, "healthy", &healthy) + d.MaybeScanArgs(t, "unhealthy", &unhealthy) + d.MaybeScanArgs(t, "inactive", &inactive) + if err := env.handleSoon(healthy, unhealthy, inactive); err != nil { + t.Fatalf("%s: %v", d.Pos, err) + } + default: + log.Eventf(ctx, "unknown command: %s\n", d.Cmd) + } + _ = env + rec := collect() + var buf strings.Builder + for _, sp := range rec { + for _, ent := range sp.Logs { + msg := ent.Message + // This is a crude hack, but it gets the job done: the trace has the + // file:line printed at the beginning of the message in an unstructured + // format, we need to get rid of that to have stable output. + msg = redact.RedactableString(regexp.MustCompile(`^([^ ]+) (.*)`).ReplaceAllString(string(msg), `$2`)) + _, _ = fmt.Fprintln(&buf, msg) + } + } + if buf.Len() == 0 { + _, _ = buf.WriteString("ok") + } + return buf.String() + }) + }) +} + +type ddEnv struct { + clusterID uuid.UUID + clock *timeutil.ManualTime + maxOffset time.Duration + stopper *stop.Stopper + tracer *tracing.Tracer + servers []*ddServer + client *Context +} + +func setupEnv(t *testing.T) *ddEnv { + stopper := stop.NewStopper() + tracer := tracing.NewTracer() + tracer.SetRedactable(true) + // Shared cluster ID by all RPC peers (this ensures that the peers + // don't talk to servers from unrelated tests by accident). + clusterID := uuid.MakeV4() + clock := timeutil.NewManualTime(timeutil.Unix(0, 0)) + maxOffset := time.Duration(250) + + clientCtx := newTestContext(clusterID, clock, maxOffset, stopper) + clientCtx.heartbeatInterval = 10 * time.Millisecond + + env := &ddEnv{ + clusterID: clusterID, + clock: clock, + maxOffset: maxOffset, + stopper: stopper, + tracer: tracer, + client: clientCtx, + } + + // Add two servers with NodeID 1 so that tests can simulate the case of a node + // restarting under a new address. + env.addServer(t, roachpb.NodeID(1)) + env.addServer(t, roachpb.NodeID(1)) + + return env +} + +type ddServer struct { + nodeID roachpb.NodeID + addr string + context *Context + grpcServer *grpc.Server + hbService *ManualHeartbeatService + hbErr *atomic.Pointer[error] +} + +func (env *ddEnv) addServer(t *testing.T, nodeID roachpb.NodeID) { + serverCtx := newTestContext(env.clusterID, env.clock, env.maxOffset, env.stopper) + serverCtx.NodeID.Set(context.Background(), nodeID) + grpcServer := newTestServer(t, serverCtx) + hbErr := new(atomic.Pointer[error]) + hbService := &ManualHeartbeatService{ + readyFn: func() error { + if errp := hbErr.Load(); errp != nil && *errp != nil { + return *errp + } + return nil + }, + stopper: env.stopper, + clock: env.clock, + maxOffset: env.maxOffset, + remoteClockMonitor: serverCtx.RemoteClocks, + version: serverCtx.Settings.Version, + nodeID: serverCtx.NodeID, + } + RegisterHeartbeatServer(grpcServer, hbService) + + ln, err := netutil.ListenAndServeGRPC(serverCtx.Stopper, grpcServer, util.TestAddr) + if err != nil { + t.Fatal(err) + } + + env.servers = append(env.servers, &ddServer{ + nodeID: nodeID, + addr: ln.Addr().String(), + context: serverCtx, + grpcServer: grpcServer, + hbService: hbService, + hbErr: hbErr, + }) +} + +func (env *ddEnv) lookupServerWithSkip(nodeID roachpb.NodeID, skip int) *ddServer { + // NB: this code is intentionally dumb so that it can in principle handle + // out-of-order servers, in case we want to go dynamic at some point. + for i := range env.servers { + if env.servers[i].nodeID == nodeID { + if skip > 0 { + skip-- + continue + } + return env.servers[i] + } + } + return nil +} + +func (env *ddEnv) dial(srv *ddServer, class ConnectionClass) *Connection { + return env.client.GRPCDialNode(srv.addr, srv.nodeID, class) +} + +func (env *ddEnv) handleDial(to *ddServer, class ConnectionClass) { + env.dial(to, class) +} + +func (env *ddEnv) handleConnect(ctx context.Context, srv *ddServer, class ConnectionClass) { + if _, err := env.dial(srv, class).Connect(ctx); err != nil { + // Don't log errors because it introduces too much flakiness. For example, + // network errors look different on different systems, and in many tests + // the heartbeat that catches an error may either be the first one or not + // (and so sometimes there's an InitialHeartbeatFailedError, or not). That's + // on top of error messages containing nondetermistic text. + tripped := errors.Is(err, circuit.ErrBreakerOpen) + log.Eventf(ctx, "error code: %v [tripped=%t]", grpcstatus.Code(errors.UnwrapAll(err)), tripped) + } +} + +func (env *ddEnv) handleShow(ctx context.Context, srv *ddServer, class ConnectionClass) { + sn, _, b, ok := env.client.peers.getWithBreaker(peerKey{NodeID: srv.nodeID, TargetAddr: srv.addr, Class: class}) + if !ok { + log.Eventf(ctx, "%s", redact.SafeString("")) + return + } + // Read tripped status without signaling probe. + var tripped bool + select { + case <-b.Signal().C(): + tripped = true + default: + } + // Avoid printing timestamps since they're usually not + // deterministic; they're set by the probe but time advances + // by 1s on each datadriven command; they are not tightly + // synchronized. + now := env.clock.Now() + log.Eventf(ctx, `tripped: %t +inactive: %t +deletable: %t`, + redact.Safe(tripped), + redact.Safe(sn.deleteAfter != 0), + redact.Safe(sn.deletable(now))) +} + +func (env *ddEnv) handleSetHeartbeatError(err error, srvs ...*ddServer) { + for _, srv := range srvs { + srv.hbErr.Store(&err) + } +} + +func (env *ddEnv) handleSoon(healthy, unhealthy, inactive int64) error { + m := env.client.Metrics() + // NB: returning to caller leads to printing in output, which means failure will + // be associated with a position in the test file. Much better than failing the + // test using `t.Fatal`. + return testutils.SucceedsSoonError(func() error { + return checkMetrics(m, healthy, unhealthy, inactive, true) + }) +} + +// lookupTargets looks up the servers from the slice, where they are specified +// as keys notation `n'`. Each `'` skips one match, i.e. it n5' would +// be the second server with NodeID 5. +// Keys that don't match are ignored. +func (env *ddEnv) lookupTargets(t *testing.T, in ...datadriven.CmdArg) []*ddServer { + var out []*ddServer + re := regexp.MustCompile(`^n([0-9]+)('*)$`) + for _, to := range in { + matches := re.FindStringSubmatch(to.Key) + if len(matches) != 3 { + continue + } + nodeID, err := strconv.ParseInt(matches[1], 10, 64) + require.NoError(t, err) + srv := env.lookupServerWithSkip(roachpb.NodeID(nodeID), len(matches[2])) + require.NotNil(t, srv) + out = append(out, srv) + } + return out +} + +// lookupTarget is like lookupTargets, but asserts that there is exactly one +// target, which is then returned. +func (env *ddEnv) lookupTarget(t *testing.T, in ...datadriven.CmdArg) *ddServer { + srvs := env.lookupTargets(t, in...) + require.Len(t, srvs, 1) + return srvs[0] +} + +func scanClass(t *testing.T, d *datadriven.TestData) ConnectionClass { + var s string + d.ScanArgs(t, "class", &s) + switch s { + case "def": + return DefaultClass + case "sys": + return SystemClass + case "rf": + return RangefeedClass + default: + t.Fatalf("no such class: %s", s) + } + return 0 // unreachable +} diff --git a/pkg/rpc/down_node_test.go b/pkg/rpc/down_node_test.go index 7cac162e83f0..2f2a51a16ec2 100644 --- a/pkg/rpc/down_node_test.go +++ b/pkg/rpc/down_node_test.go @@ -60,8 +60,9 @@ func TestConnectingToDownNode(t *testing.T) { const n = 100 for i := 0; i < n; i++ { tBegin := timeutil.Now() - _, err = rpcCtx.GRPCDialNode(ln.Addr().String(), 1, DefaultClass).Connect(ctx) - require.True(t, errors.HasType(err, (*netutil.InitialHeartbeatFailedError)(nil))) + _, err = rpcCtx.GRPCDialNode(ln.Addr().String(), 1, DefaultClass). + Connect(ctx) + require.True(t, errors.HasType(err, (*netutil.InitialHeartbeatFailedError)(nil)), "%+v", err) dur += timeutil.Since(tBegin) } avg := dur / n diff --git a/pkg/rpc/errors.go b/pkg/rpc/errors.go new file mode 100644 index 000000000000..07697af78eae --- /dev/null +++ b/pkg/rpc/errors.go @@ -0,0 +1,38 @@ +// Copyright 2023 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 rpc + +import ( + "github.com/cockroachdb/errors" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" +) + +// errQuiescing is returned from client interceptors when the server's +// stopper is quiescing. The error is constructed to return true in +// `grpcutil.IsConnectionRejected` which prevents infinite retry loops during +// cluster shutdown, especially in unit testing. +var errQuiescing = status.Error(codes.PermissionDenied, "refusing to dial; node is quiescing") + +// ErrNotHeartbeated is returned by ConnHealth or Connection.Health when we have +// not yet performed the first heartbeat. This error will typically only be +// observed when checking the health during the first connection attempt to a +// node, as during subsequent periods of an unhealthy connection the circuit +// breaker error will be returned instead. +var ErrNotHeartbeated = errors.New("not yet heartbeated") + +type versionCompatError struct{} + +func (versionCompatError) Error() string { + return "version compatibility check failed on ping response" +} + +var VersionCompatError = versionCompatError{} diff --git a/pkg/rpc/metrics.go b/pkg/rpc/metrics.go index 0b21e41abf1f..161bdd39b095 100644 --- a/pkg/rpc/metrics.go +++ b/pkg/rpc/metrics.go @@ -10,7 +10,11 @@ package rpc -import "github.com/cockroachdb/cockroach/pkg/util/metric" +import ( + "github.com/VividCortex/ewma" + "github.com/cockroachdb/cockroach/pkg/util/metric" + "github.com/cockroachdb/cockroach/pkg/util/metric/aggmetric" +) var ( // The below gauges store the current state of running heartbeat loops. @@ -21,45 +25,200 @@ var ( // Together these metrics should provide a picture of the state of current // connections. - metaHeartbeatsNominal = metric.Metadata{ - Name: "rpc.heartbeats.nominal", - Help: "Gauge of current connections in the nominal state (i.e. at least one successful heartbeat)", + metaConnectionHealthy = metric.Metadata{ + Name: "rpc.connection.healthy", + Help: "Gauge of current connections in a healthy state (i.e. bidirectionally connected and heartbeating)", + Measurement: "Connections", + Unit: metric.Unit_COUNT, + } + + metaConnectionUnhealthy = metric.Metadata{ + Name: "rpc.connection.unhealthy", + Help: "Gauge of current connections in an unhealthy state (not bidirectionally connected or heartbeating)", Measurement: "Connections", Unit: metric.Unit_COUNT, } - metaHeartbeatLoopsStarted = metric.Metadata{ - Name: "rpc.heartbeats.loops.started", - Help: "Counter of connection attempts", + metaConnectionInactive = metric.Metadata{ + Name: "rpc.connection.inactive", + Help: "Gauge of current connections in an inactive state and pending deletion; " + + "these are not healthy but are not tracked as unhealthy either because " + + "there is reason to believe that the connection is no longer relevant," + + "for example if the node has since been seen under a new address", Measurement: "Connections", Unit: metric.Unit_COUNT, } - metaHeartbeatLoopsExited = metric.Metadata{ - Name: "rpc.heartbeats.loops.exited", + + metaConnectionHealthyNanos = metric.Metadata{ + Name: "rpc.connection.healthy_nanos", + Help: `Gauge of nanoseconds of healthy connection time + +On the prometheus endpoint scraped with the cluster setting 'server.child_metrics.enabled' set, +the constituent parts of this metric are available on a per-peer basis and one can read off +for how long a given peer has been connected`, + Measurement: "Nanoseconds", + Unit: metric.Unit_NANOSECONDS, + } + + metaConnectionUnhealthyNanos = metric.Metadata{ + Name: "rpc.connection.unhealthy_nanos", + Help: `Gauge of nanoseconds of unhealthy connection time. + +On the prometheus endpoint scraped with the cluster setting 'server.child_metrics.enabled' set, +the constituent parts of this metric are available on a per-peer basis and one can read off +for how long a given peer has been unreachable`, + Measurement: "Nanoseconds", + Unit: metric.Unit_NANOSECONDS, + } + + metaConnectionHeartbeats = metric.Metadata{ + Name: "rpc.connection.heartbeats", + Help: `Counter of successful heartbeats.`, + Measurement: "Heartbeats", + Unit: metric.Unit_COUNT, + } + + metaConnectionFailures = metric.Metadata{ + Name: "rpc.connection.failures", Help: `Counter of failed connections. -This includes both healthy connections that then terminated as well -as connection attempts that failed outright during initial dialing. +This includes both the event in which a healthy connection terminates as well as +unsuccessful reconnection attempts. -Connections that are terminated as part of shutdown are excluded. +Connections that are terminated as part of local node shutdown are excluded. +Decommissioned peers are excluded. `, Measurement: "Connections", Unit: metric.Unit_COUNT, } + + metaConnectionAvgRoundTripLatency = metric.Metadata{ + Name: "rpc.connection.avg_round_trip_latency", + Unit: metric.Unit_NANOSECONDS, + Help: `Sum of exponentially weighted moving average of round-trip latencies, as measured through a gRPC RPC. + +Dividing this Gauge by rpc.connection.healthy gives an approximation of average +latency, but the top-level round-trip-latency histogram is more useful. Instead, +users should consult the label families of this metric if they are available +(which requires prometheus and the cluster setting 'server.child_metrics.enabled'); +these provide per-peer moving averages. + +This metric does not track failed connection. A failed connection's contribution +is reset to zero. +`, + Measurement: "Latency", + } ) func makeMetrics() Metrics { + childLabels := []string{"remote_node_id", "remote_addr", "class"} return Metrics{ - HeartbeatLoopsStarted: metric.NewCounter(metaHeartbeatLoopsStarted), - HeartbeatLoopsExited: metric.NewCounter(metaHeartbeatLoopsExited), - HeartbeatsNominal: metric.NewGauge(metaHeartbeatsNominal), + ConnectionHealthy: aggmetric.NewGauge(metaConnectionHealthy, childLabels...), + ConnectionUnhealthy: aggmetric.NewGauge(metaConnectionUnhealthy, childLabels...), + ConnectionInactive: aggmetric.NewGauge(metaConnectionInactive, childLabels...), + ConnectionHealthyFor: aggmetric.NewGauge(metaConnectionHealthyNanos, childLabels...), + ConnectionUnhealthyFor: aggmetric.NewGauge(metaConnectionUnhealthyNanos, childLabels...), + ConnectionHeartbeats: aggmetric.NewCounter(metaConnectionHeartbeats, childLabels...), + ConnectionFailures: aggmetric.NewCounter(metaConnectionFailures, childLabels...), + ConnectionAvgRoundTripLatency: aggmetric.NewGauge(metaConnectionAvgRoundTripLatency, childLabels...), } } // Metrics is a metrics struct for Context metrics. // Field X is documented in metaX. type Metrics struct { - HeartbeatLoopsStarted *metric.Counter - HeartbeatLoopsExited *metric.Counter - HeartbeatsNominal *metric.Gauge + ConnectionHealthy *aggmetric.AggGauge + ConnectionUnhealthy *aggmetric.AggGauge + ConnectionInactive *aggmetric.AggGauge + ConnectionHealthyFor *aggmetric.AggGauge + ConnectionUnhealthyFor *aggmetric.AggGauge + ConnectionHeartbeats *aggmetric.AggCounter + ConnectionFailures *aggmetric.AggCounter + ConnectionAvgRoundTripLatency *aggmetric.AggGauge +} + +// peerMetrics are metrics that are kept on a per-peer basis. +// Their lifecycle follows that of the associated peer, i.e. +// they are acquired on peer creation, and are released when +// the peer is destroyed. +type peerMetrics struct { + // IMPORTANT: update Metrics.release when adding any gauges here. It's + // notoriously easy to leak child gauge values when removing peers. All gauges + // must be reset before removing, and there must not be any chance that + // they're set again because even if they're unlinked from the parent, they + // will continue to add to the parent! + + // 1 on first heartbeat success (via reportHealthy), reset after + // runHeartbeatUntilFailure returns. + ConnectionHealthy *aggmetric.Gauge + // Reset on first successful heartbeat (via reportHealthy), 1 after + // runHeartbeatUntilFailure returns. + ConnectionUnhealthy *aggmetric.Gauge + // Set when the peer is inactive, i.e. `deleteAfter` is set but it is still in + // the peer map (i.e. likely a superseded connection, but we're not sure yet). + // For such peers the probe only runs on demand and the connection is not + // healthy but also not tracked as unhealthy. + ConnectionInactive *aggmetric.Gauge + // Updated on each successful heartbeat from a local var, reset after + // runHeartbeatUntilFailure returns. + ConnectionHealthyFor *aggmetric.Gauge + // Updated from p.mu.disconnected before each loop around in breakerProbe.run, + // reset on first heartbeat success (via reportHealthy). + ConnectionUnhealthyFor *aggmetric.Gauge + // Updated on each successful heartbeat, reset (along with roundTripLatency) + // after runHeartbeatUntilFailure returns. + AvgRoundTripLatency *aggmetric.Gauge + // roundTripLatency is the source for the AvgRoundTripLatency gauge. We don't + // want to maintain a full histogram per peer, so instead on each heartbeat we + // update roundTripLatency and flush the result into AvgRoundTripLatency. + roundTripLatency ewma.MovingAverage // *not* thread safe + + // Counters. + + // Incremented after each successful heartbeat. + ConnectionHeartbeats *aggmetric.Counter + // Updated before each loop around in breakerProbe.run. + ConnectionFailures *aggmetric.Counter +} + +func (m *Metrics) acquire(k peerKey) peerMetrics { + labelVals := []string{k.NodeID.String(), k.TargetAddr, k.Class.String()} + return peerMetrics{ + ConnectionHealthy: m.ConnectionHealthy.AddChild(labelVals...), + ConnectionUnhealthy: m.ConnectionUnhealthy.AddChild(labelVals...), + ConnectionInactive: m.ConnectionInactive.AddChild(labelVals...), + ConnectionHealthyFor: m.ConnectionHealthyFor.AddChild(labelVals...), + ConnectionUnhealthyFor: m.ConnectionUnhealthyFor.AddChild(labelVals...), + ConnectionHeartbeats: m.ConnectionHeartbeats.AddChild(labelVals...), + ConnectionFailures: m.ConnectionFailures.AddChild(labelVals...), + AvgRoundTripLatency: m.ConnectionAvgRoundTripLatency.AddChild(labelVals...), + + // We use a SimpleEWMA which uses the zero value to mean "uninitialized" + // and operates on a ~60s decay rate. + // + // Note that this is *not* thread safe. + roundTripLatency: &ewma.SimpleEWMA{}, + } +} + +func (pm *peerMetrics) release() { + // All the gauges should be zero now, or the aggregate will be off forever. + // Note that this isn't true for counters, as the aggregate *should* track + // the count of all children that ever existed, even if they have been + // released. (Releasing a peer doesn't "undo" past heartbeats). + pm.ConnectionHealthy.Update(0) + pm.ConnectionUnhealthy.Update(0) + pm.ConnectionInactive.Update(0) + pm.ConnectionHealthyFor.Update(0) + pm.ConnectionUnhealthyFor.Update(0) + pm.AvgRoundTripLatency.Update(0) + pm.roundTripLatency.Set(0) + + pm.ConnectionHealthy.Unlink() + pm.ConnectionUnhealthy.Unlink() + pm.ConnectionHealthyFor.Unlink() + pm.ConnectionUnhealthyFor.Unlink() + pm.ConnectionHeartbeats.Unlink() + pm.ConnectionFailures.Unlink() + pm.AvgRoundTripLatency.Unlink() } diff --git a/pkg/rpc/nodedialer/BUILD.bazel b/pkg/rpc/nodedialer/BUILD.bazel index 8d2462648aab..39f914a42c91 100644 --- a/pkg/rpc/nodedialer/BUILD.bazel +++ b/pkg/rpc/nodedialer/BUILD.bazel @@ -11,6 +11,7 @@ go_library( "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/rpc", + "//pkg/util/circuit", "//pkg/util/log", "//pkg/util/stop", "//pkg/util/syncutil", @@ -46,6 +47,7 @@ go_test( "@com_github_stretchr_testify//assert", "@com_github_stretchr_testify//require", "@org_golang_google_grpc//:go_default_library", + "@org_golang_google_grpc//codes", ], ) diff --git a/pkg/rpc/nodedialer/nodedialer.go b/pkg/rpc/nodedialer/nodedialer.go index ba753a904c8e..32e2fdf5877c 100644 --- a/pkg/rpc/nodedialer/nodedialer.go +++ b/pkg/rpc/nodedialer/nodedialer.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" + circuit2 "github.com/cockroachdb/cockroach/pkg/util/circuit" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/stop" "github.com/cockroachdb/cockroach/pkg/util/syncutil" @@ -119,9 +120,8 @@ func (n *Dialer) Dial( } // DialNoBreaker is like Dial, but will not check the circuit breaker before -// trying to connect. The breaker is notified of the outcome. This function -// should only be used when there is good reason to believe that the node is -// reachable. +// trying to connect. This function should only be used when there is good +// reason to believe that the node is reachable. func (n *Dialer) DialNoBreaker( ctx context.Context, nodeID roachpb.NodeID, class rpc.ConnectionClass, ) (_ *grpc.ClientConn, err error) { @@ -196,7 +196,12 @@ func (n *Dialer) dial( log.Health.Warningf(ctx, "unable to connect to n%d: %s", nodeID, err) } }() - conn, err := n.rpcContext.GRPCDialNode(addr.String(), nodeID, class).Connect(ctx) + rpcConn := n.rpcContext.GRPCDialNode(addr.String(), nodeID, class) + connect := rpcConn.Connect + if !checkBreaker { + connect = rpcConn.ConnectNoBreaker + } + conn, err := connect(ctx) if err != nil { // If we were canceled during the dial, don't trip the breaker. if ctxErr := ctx.Err(); ctxErr != nil { @@ -255,7 +260,9 @@ func (n *Dialer) ConnHealth(nodeID roachpb.NodeID, class rpc.ConnectionClass) er // "hint" to use a connection if it already exists, but simultaneously kick off // a connection attempt in the background if it doesn't and always return // immediately. It is only used today by DistSQL and it should probably be -// removed and moved into that code. +// removed and moved into that code. Also, as of #99191, we have stateful +// circuit breakers that probe in the background and so whatever exactly it +// is the caller really wants can likely be achieved by more direct means. func (n *Dialer) ConnHealthTryDial(nodeID roachpb.NodeID, class rpc.ConnectionClass) error { err := n.ConnHealth(nodeID, class) if err == nil || !n.getBreaker(nodeID, class).Ready() { @@ -279,6 +286,16 @@ func (n *Dialer) GetCircuitBreaker( return n.getBreaker(nodeID, class).Breaker } +func (n *Dialer) GetCircuitBreakerNew( + nodeID roachpb.NodeID, class rpc.ConnectionClass, +) (*circuit2.Breaker, bool) { + addr, err := n.resolver(nodeID) + if err != nil { + return nil, false + } + return n.rpcContext.GetBreakerForAddr(nodeID, class, addr) +} + func (n *Dialer) getBreaker(nodeID roachpb.NodeID, class rpc.ConnectionClass) *wrappedBreaker { breakers := &n.breakers[class] value, ok := breakers.Load(int64(nodeID)) diff --git a/pkg/rpc/nodedialer/nodedialer_test.go b/pkg/rpc/nodedialer/nodedialer_test.go index fd1501c9506c..c5eb043783b2 100644 --- a/pkg/rpc/nodedialer/nodedialer_test.go +++ b/pkg/rpc/nodedialer/nodedialer_test.go @@ -16,6 +16,7 @@ import ( "math/rand" "net" "sync" + "sync/atomic" "testing" "time" @@ -37,6 +38,7 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "google.golang.org/grpc" + "google.golang.org/grpc/codes" ) const staticNodeID = 1 @@ -135,8 +137,19 @@ func TestConnHealth(t *testing.T) { defer stopper.Stop(ctx) nd := New(rpcCtx, newSingleNodeResolver(staticNodeID, ln.Addr())) - // When no connection exists, we expect ConnHealth to return ErrNoConnection. - require.Equal(t, rpc.ErrNoConnection, nd.ConnHealth(staticNodeID, rpc.DefaultClass)) + var hbDecommission atomic.Value + hbDecommission.Store(false) + rpcCtx.OnOutgoingPing = func(ctx context.Context, req *rpc.PingRequest) error { + if hbDecommission.Load().(bool) { + return kvpb.NewDecommissionedStatusErrorf( + codes.PermissionDenied, "target node n%s is decommissioned", req.TargetNodeID, + ) + } + return nil + } + + // When no connection exists, we expect ConnHealth to return ErrNotHeartbeated. + require.Equal(t, rpc.ErrNotHeartbeated, nd.ConnHealth(staticNodeID, rpc.DefaultClass)) // After dialing the node, ConnHealth should return nil. _, err := nd.Dial(ctx, staticNodeID, rpc.DefaultClass) @@ -147,7 +160,7 @@ func TestConnHealth(t *testing.T) { // ConnHealth should still error for other node ID and class. require.Error(t, nd.ConnHealth(9, rpc.DefaultClass)) - require.Equal(t, rpc.ErrNoConnection, nd.ConnHealth(staticNodeID, rpc.SystemClass)) + require.Equal(t, rpc.ErrNotHeartbeated, nd.ConnHealth(staticNodeID, rpc.SystemClass)) // When the heartbeat errors, ConnHealth should eventually error too. hb.setErr(errors.New("boom")) @@ -185,6 +198,7 @@ func TestConnHealth(t *testing.T) { // Closing the remote connection should fail ConnHealth. require.NoError(t, ln.popConn().Close()) + hbDecommission.Store(true) require.Eventually(t, func() bool { return nd.ConnHealth(staticNodeID, rpc.DefaultClass) != nil }, time.Second, 10*time.Millisecond) @@ -204,7 +218,7 @@ func TestConnHealthTryDial(t *testing.T) { nd := New(rpcCtx, newSingleNodeResolver(staticNodeID, ln.Addr())) // Make sure no connection exists yet, via ConnHealth(). - require.Equal(t, rpc.ErrNoConnection, nd.ConnHealth(staticNodeID, rpc.DefaultClass)) + require.Equal(t, rpc.ErrNotHeartbeated, nd.ConnHealth(staticNodeID, rpc.DefaultClass)) // When no connection exists, we expect ConnHealthTryDial to dial the node, // which will return ErrNoHeartbeat at first but eventually succeed. diff --git a/pkg/rpc/peer.go b/pkg/rpc/peer.go new file mode 100644 index 000000000000..3f85fcf1b0b6 --- /dev/null +++ b/pkg/rpc/peer.go @@ -0,0 +1,892 @@ +// Copyright 2023 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 rpc + +import ( + "context" + "fmt" + "runtime/pprof" + "time" + + "github.com/VividCortex/ewma" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" + "github.com/cockroachdb/cockroach/pkg/util/circuit" + "github.com/cockroachdb/cockroach/pkg/util/grpcutil" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/netutil" + "github.com/cockroachdb/cockroach/pkg/util/stop" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/cockroachdb/errors" + "github.com/cockroachdb/logtags" + "github.com/cockroachdb/redact" + "google.golang.org/grpc" + "google.golang.org/grpc/connectivity" + "google.golang.org/grpc/status" +) + +// A peer is a remote node that we are trying to maintain a healthy RPC +// connection (for a given connection class not known to the peer itself) to. It +// maintains metrics on our connection state to the peer (see the embedded +// peerMetrics) and maintains a circuit breaker whose probe is the heartbeat +// loop; the breaker trips whenever a heartbeat fails and resets whenever a +// heartbeat succeeds. +// Usually, the probe is always active (either attempting to heal the connection +// or maintaining its health), but when a peer looks likely to be obsolete (for +// example, remote node is noticed as having been decommissioned) as indicated +// by the `deleteAfter` field being nonzero, the probe only runs on-demand, that +// is, whenever usage of the `peer` is detected (this is done by the circuit +// breaker); see (*peerMap).shouldDeleteAfter. +// When the current time has surpassed `deleteAfter`, the peer will be taken out +// of its surrounding rpc.Context and will no longer probe; see +// (*peer).maybeDelete. +// See (*peer).launch for details on the probe (heartbeat loop) itself. +type peer struct { + peerMetrics + k peerKey + opts *ContextOptions + heartbeatInterval time.Duration + heartbeatTimeout time.Duration + dial func(ctx context.Context, target string, class ConnectionClass) (*grpc.ClientConn, error) + // b maintains connection health. This breaker's async probe is always + // active - it is the heartbeat loop and manages `mu.c.` (including + // recreating it after the connection fails and has to be redialed). + // + // NB: at the time of writing, we don't use the breaking capabilities, + // i.e. we don't check the circuit breaker in `Connect`. We will do that + // once the circuit breaker is mature, and then retire the breakers + // returned by Context.getBreaker. + // + // Currently what will happen when a peer is down is that `c` will be + // recreated (blocking new callers to `Connect()`), a connection attempt + // will be made, and callers will see the failure to this attempt. + // + // With the breaker, callers would be turned away eagerly until there + // is a known-healthy connection. + // + // mu must *NOT* be held while operating on `b`. This is because the async + // probe will sometimes have to synchronously acquire mu before spawning off. + b *circuit.Breaker + logDisconnectEvery log.EveryN + mu struct { + syncutil.Mutex + // Copies of PeerSnap may be leaked outside of lock, since the memory within + // is never mutated in place. + PeerSnap + } + remoteClocks *RemoteClockMonitor + // NB: lock order: peers.mu then peers.mu.m[k].mu (but better to avoid + // overlapping critical sections) + peers *peerMap +} + +// PeerSnap is the state of a peer. +type PeerSnap struct { + c *Connection // never nil, only mutated in the breaker probe + + // Timestamp of latest successful initial heartbeat on `c`. This + // is never cleared: it only ever moves forward. If the peer is + // currently unhealthy (i.e. breaker tripped), `disconnected` will + // be larger than `connected`. Otherwise, the peer is healthy. + // + // Example: + // t=100: peer created (connected=0, disconnected=0) + // t=120: peer heartbeats successfully (connected=120, disconnected=0) + // t=130: peer heartbeats successfully (connected=120, disconnected=0) + // t=150: peer fails heartbeat (connected=120, disconnected=150) + // t=160: peer fails heartbeat (connected=120, disconnected=150) + // t=170: peer heartbeats successfully (connected=170, disconnected=150) + // t=200: peer fails heartbeat (connected=170, disconnected=200). + // + // INVARIANT: connected > disconnected <=> c.initialHeartbeatDone closed. + // (Assuming monotonic local walltime). + connected time.Time + // disconnected is zero initially, reset on successful heartbeat, set on + // heartbeat teardown if zero. In other words, does not move forward across + // subsequent connection failures - it tracks the first disconnect since + // having been healthy. See comment on `connected` for example. + // + // INVARIANT: disconnected != 0 <=> connected == 0. + // INVARIANT: disconnected != 0 <=> breaker tripped + // (manual manipulation of the breaker is a programming error) + disconnected time.Time + // deleteAfter is nonzero if the peer is failed and "likely not useful any more". + // This must be set from the probe for that peer, and makes probing on-demand: + // each attempt to access the breaker triggers a probe, if one is not + // inflight, but the probe will terminate on failure. + // + // If the peer becomes healthy again, deleteAfter is cleared. + // + // Comments in shouldDeleteAfter document when a deletedAfter is set. This is somewhat + // complex, owing to the possibility of listening addresses swapping around between + // nodes, as well as the existence of unvalidated (NodeID-less) connections. + // + // INVARIANT: if deleteAfter != nil, all gauge contributions to metrics are zero. + // INVARIANT: deleteAfter != 0 => disconnected != 0 + deleteAfter time.Duration + // deleted indicates that the peer was removed from map and unregistered from + // metrics. The probe must not do any more work (in particular the gauges must + // no longer be touched). + // + // INVARIANT: deleted once => deleted forever + // INVARIANT: deleted => deleteAfter > 0 + deleted bool +} + +func (p *peer) snap() PeerSnap { + p.mu.Lock() + defer p.mu.Unlock() + return p.mu.PeerSnap +} + +// newPeer returns circuit breaker that trips when connection (associated +// with provided peerKey) is failed. The breaker's probe *is* the heartbeat loop +// and is thus running at all times. The exception is a decommissioned node, for +// which the probe simply exits (any future connection attempts to the same peer +// will trigger the probe but the probe will exit again), and a superseded peer, +// i.e. one for which a node restarted with a different IP address and we're the +// "old", unhealth, peer. +// +// Multiple peers to a given node can temporarily exist at any given point +// in time (if the node restarts under a different IP). We assume that +// ultimately one of those will become unhealthy and repeatedly fail its +// probe. On probe failure, we check the node map for duplicates and if a +// healthy duplicate exists, remove ourselves from the map. In the worst +// case, we are actually the new connection to the remote and got tripped +// up by a transient error while the old connection hasn't realized yet that +// it's dead - in that case, because we immediately remove ourselves from the +// map, the next attempt to dial the node will start from a blank slate. In +// other words, even with this theoretical race, the situation will sort itself +// out quickly. +func (rpcCtx *Context) newPeer(k peerKey) *peer { + // Initialization here is a bit circular. The peer holds the breaker. The + // breaker probe references the peer because it needs to replace the one-shot + // Connection when it makes a new connection in the probe. And (all but the + // first incarnation of) the Connection also holds on to the breaker since the + // Connect method needs to do the short-circuiting (if a Connection is created + // while the breaker is tripped, we want to block in Connect only once we've + // seen the first heartbeat succeed). + p := &peer{ + peerMetrics: rpcCtx.metrics.acquire(k), + logDisconnectEvery: log.Every(time.Minute), + k: k, + remoteClocks: rpcCtx.RemoteClocks, + opts: &rpcCtx.ContextOptions, + peers: &rpcCtx.peers, + dial: func(ctx context.Context, target string, class ConnectionClass) (*grpc.ClientConn, error) { + return rpcCtx.grpcDialRaw(ctx, target, class, rpcCtx.testingDialOpts...) + }, + heartbeatInterval: rpcCtx.heartbeatInterval, + heartbeatTimeout: rpcCtx.heartbeatTimeout, + } + var b *circuit.Breaker + + ctx := rpcCtx.makeDialCtx(k.TargetAddr, k.NodeID, k.Class) + b = circuit.NewBreaker(circuit.Options{ + Name: "breaker", // log tags already represent `k` + AsyncProbe: func(report func(error), done func()) { + pprof.Do(ctx, pprof.Labels("tags", logtags.FromContext(ctx).String()), func(ctx context.Context) { + p.launch(ctx, report, done) + }) + }, + // Use a noop EventHandler; we do our own logging in the probe since we'll + // have better information. + EventHandler: &circuit.EventLogger{Log: func(buf redact.StringBuilder) {}}, + }) + p.b = b + c := newConnectionToNodeID(k, b.Signal) + p.mu.PeerSnap = PeerSnap{c: c} + + return p +} + +func (p *peer) breakerDisabled() bool { + return !enableRPCCircuitBreakers.Get(&p.opts.Settings.SV) +} + +// launch starts the probe in the background. The probe typically runs forever[1], +// and has the following high-level structure (dashes reflect call depth). +// +// - run: loops "forever", except when connection is pending deletion, see [1]. +// -- runOnce: starts a new *ClientConn and maintains it until it errors out. +// --- runSingleHeartbeat: performs the first heartbeat. +// --- onInitialHeartbeatSucceeded: signals the conn future (*ClientConn now accessible). +// --- runHeartbeatUntilFailure: performs subsequent heartbeats until error occurs. +// ---- onSubsequentHeartbeatSucceeded: metric updates. +// - onHeartbeatFailed: state transition into failed state (breaker, logging, etc). +// +// [1]: see comment on `peer` for exceptions, and (*peer).shouldDeleteAfter for +// an entry point into the code. In brief, if an unhealthy peer is suspected of +// being obsolete, the probe only runs when the breaker is checked by a caller. +// After a generous timeout, the peer is removed if still unhealthy. +func (p *peer) launch(ctx context.Context, report func(error), done func()) { + // Acquire mu just to show that we can, as the caller is supposed + // to not hold the lock. + p.mu.Lock() + _ = 0 // bypass empty crit section lint + p.mu.Unlock() + + taskName := fmt.Sprintf("conn to n%d@%s/%s", p.k.NodeID, p.k.TargetAddr, p.k.Class) + + log.VEventf(ctx, 1, "probe starting") + if err := p.opts.Stopper.RunAsyncTask(ctx, taskName, func(ctx context.Context) { + p.run(ctx, report, done) + }); err != nil { + // Stopper draining. Since we're trying to launch a probe, we know the + // breaker is tripped. We overwrite the error since we want errQuiescing + // (which has a gRPC status), not kvpb.NodeUnavailableError. + err = errQuiescing + report(err) + // We also need to resolve connFuture because a caller may be waiting on + // (*Connection).ConnectNoBreaker, and they need to be signaled as well + // but aren't listening to the stopper. + p.mu.c.connFuture.Resolve(nil, errQuiescing) + done() + } +} + +func (p *peer) run(ctx context.Context, report func(error), done func()) { + var t timeutil.Timer + defer t.Stop() + defer done() + defer log.VEventf(ctx, 1, "probe stopping") + // Immediately run probe after breaker circuit is tripped, optimizing for the + // case in which we can immediately reconnect. + t.Reset(0) + for { + if p.snap().deleted { + return + } + + // NB: we don't need to close initialHeartbeatDone in these error cases. + // Connect() is cancellation-sensitive as well. + select { + case <-ctx.Done(): + // Stopper quiescing, node shutting down. Mirroring what breakerProbe.launch + // does when it can't launch an async task: leave the broken connection around, + // no need to close initialHeartbeatDone, just report errQuiescing and quit. + report(errQuiescing) + return + case <-t.C: + t.Read = true + // Retry every second. Note that if runHeartbeatUntilFailure takes >1, we'll + // retry immediately once it returns. This means that a connection breaking + // for the first time is usually followed by an immediate redial attempt. + t.Reset(p.heartbeatInterval) + } + + // Peer is currently initializing (first use) or unhealthy (looped around + // from earlier attempt). `runOnce` will try to establish a connection and + // keep it healthy for as long as possible. On first error, it will return + // back to us. + err := p.runOnce(ctx, report) + // If ctx is done, Stopper is draining. Unconditionally override the error + // to clean up the logging in this case. + if ctx.Err() != nil { + err = errQuiescing + } + + // Transition peer into unhealthy state. + now := p.opts.Clock.Now() + p.onHeartbeatFailed(ctx, err, now, report) + + // Release peer and delete from map, if appropriate. We'll detect + // whether this happened after looping around. + p.maybeDelete(ctx, now) + + if errors.Is(err, errQuiescing) { + // Heartbeat loop ended due to shutdown. Exit the probe, it won't be + // started again since that means running an async task through the + // Stopper. + return + } + + p.mu.Lock() + p.mu.c = newConnectionToNodeID(p.k, p.mu.c.breakerSignalFn) + p.mu.Unlock() + + if p.snap().deleteAfter != 0 { + // Peer is in inactive mode, and we just finished up a probe, so + // end the probe. Another one will be started if anyone accesses + // the breaker. + return + } + } +} + +func (p *peer) runOnce(ctx context.Context, report func(error)) error { + cc, err := p.dial(ctx, p.k.TargetAddr, p.k.Class) + if err != nil { + return err + } + defer func() { + _ = cc.Close() // nolint:grpcconnclose + }() + + // Set up notifications on a channel when gRPC tears down, so that we + // can trigger another instant heartbeat for expedited circuit breaker + // tripping. + connFailedCh := make(chan connectivity.State, 1) + launchConnStateWatcher(ctx, p.opts.Stopper, cc, connFailedCh) + + if p.remoteClocks != nil { + p.remoteClocks.OnConnect(ctx, p.k.NodeID) + defer p.remoteClocks.OnDisconnect(ctx, p.k.NodeID) + } + + if err := runSingleHeartbeat( + ctx, NewHeartbeatClient(cc), p.k, p.peerMetrics.roundTripLatency, nil /* no remote clocks */, p.opts, p.heartbeatTimeout, PingRequest_BLOCKING, + ); err != nil { + return err + } + + p.onInitialHeartbeatSucceeded(ctx, p.opts.Clock.Now(), cc, report) + + return p.runHeartbeatUntilFailure(ctx, connFailedCh) +} + +func runSingleHeartbeat( + ctx context.Context, + heartbeatClient HeartbeatClient, + k peerKey, + roundTripLatency ewma.MovingAverage, + remoteClocks *RemoteClockMonitor, // nil if no RemoteClocks update should be made + opts *ContextOptions, + heartbeatTimeout time.Duration, + preferredDialback PingRequest_DialbackType, +) error { + if !opts.NeedsDialback || !useDialback.Get(&opts.Settings.SV) { + preferredDialback = PingRequest_NONE + } + + // Pick up any asynchronous update to clusterID and NodeID. + clusterID := opts.StorageClusterID.Get() + + var lastOffset RemoteOffset + if remoteClocks != nil { + lastOffset = remoteClocks.GetOffset(k.NodeID) + } + + // The request object. Note that we keep the same object from + // heartbeat to heartbeat: we compute a new .Offset at the end of + // the current heartbeat as input to the next one. + request := &PingRequest{ + OriginAddr: opts.Config.AdvertiseAddr, + TargetNodeID: k.NodeID, + ServerVersion: opts.Settings.Version.BinaryVersion(), + LocalityAddress: opts.Config.LocalityAddresses, + ClusterID: &clusterID, + OriginNodeID: opts.NodeID.Get(), + NeedsDialback: preferredDialback, + Offset: lastOffset, + } + + interceptor := func(context.Context, *PingRequest) error { return nil } + if fn := opts.OnOutgoingPing; fn != nil { + interceptor = fn + } + + var response *PingResponse + sendTime := opts.Clock.Now() + ping := func(ctx context.Context) error { + if err := interceptor(ctx, request); err != nil { + return err + } + var err error + + response, err = heartbeatClient.Ping(ctx, request) + return err + } + var err error + if heartbeatTimeout > 0 { + err = timeutil.RunWithTimeout(ctx, "conn heartbeat", heartbeatTimeout, ping) + } else { + err = ping(ctx) + } + + if err != nil { + log.VEventf(ctx, 2, "received error on ping response from n%d, %v", k.NodeID, err) + return err + } + + // We verify the cluster name on the initiator side (instead + // of the heartbeat service side, as done for the cluster ID + // and node ID checks) so that the operator who is starting a + // new node in a cluster and mistakenly joins the wrong + // cluster gets a chance to see the error message on their + // management console. + if !opts.Config.DisableClusterNameVerification && !response.DisableClusterNameVerification { + err = errors.Wrap( + checkClusterName(opts.Config.ClusterName, response.ClusterName), + "cluster name check failed on ping response") + if err != nil { + return err + } + } + + err = checkVersion(ctx, opts.Settings.Version, response.ServerVersion) + if err != nil { + err := errors.Mark(err, VersionCompatError) + return err + } + + receiveTime := opts.Clock.Now() + + pingDuration, _, err := updateClockOffsetTracking( + ctx, remoteClocks, k.NodeID, + sendTime, timeutil.Unix(0, response.ServerTime), receiveTime, + opts.ToleratedOffset, + ) + if err != nil { + if opts.FatalOnOffsetViolation { + log.Ops.Fatalf(ctx, "%v", err) + } + } else { + roundTripLatency.Add(float64(pingDuration.Nanoseconds())) // source for metrics + } + + return nil +} + +// runHeartbeatUntilFailure synchronously runs the heartbeat loop for the given +// RPC connection, returning once a heartbeat fails. The ctx passed as argument +// must be derived from rpcCtx.masterCtx, so that it respects the same +// cancellation policy. +func (p *peer) runHeartbeatUntilFailure( + ctx context.Context, connFailedCh <-chan connectivity.State, +) error { + var heartbeatTimer timeutil.Timer + defer heartbeatTimer.Stop() + // NB: the caller just sent the initial heartbeat, so we don't + // queue for an immedidate heartbeat but wait out the interval. + heartbeatTimer.Reset(p.heartbeatInterval) + + // If we get here, we know `connFuture` has been resolved (due to the + // initial heartbeat having succeeded), so we have a Conn() we can + // use. + heartbeatClient := NewHeartbeatClient(p.snap().c.connFuture.Conn()) + + for { + select { + case <-ctx.Done(): + return ctx.Err() // likely server shutdown + case <-heartbeatTimer.C: + heartbeatTimer.Read = true + case <-connFailedCh: + // gRPC has signaled that the connection is now failed, which implies that + // we will need to start a new connection (since we set things up that way + // using onlyOnceDialer). But we go through the motions and run the + // heartbeat so that there is a unified path that reports the error, + // in order to provide a good UX. + } + + if err := runSingleHeartbeat( + ctx, heartbeatClient, p.k, p.peerMetrics.roundTripLatency, p.remoteClocks, + p.opts, p.heartbeatTimeout, PingRequest_NON_BLOCKING, + ); err != nil { + return err + } + + p.onSubsequentHeartbeatSucceeded(ctx, p.opts.Clock.Now()) + heartbeatTimer.Reset(p.heartbeatInterval) + } +} + +func logOnHealthy(ctx context.Context, disconnected, now time.Time) { + var buf redact.StringBuilder + _, _ = redact.Fprintf(&buf, "connection is now healthy") + // When the breaker was first created, we tripped it but disconnected will + // have been zero, so don't log a bogus duration in that case. + if !disconnected.IsZero() { + _, _ = redact.Fprintf(&buf, " (after %s)", now.Sub(disconnected).Round(time.Second)) + } + log.Health.InfofDepth(ctx, 1, "%s", buf) +} + +func (p *peer) onInitialHeartbeatSucceeded( + ctx context.Context, now time.Time, cc *grpc.ClientConn, report func(err error), +) { + // First heartbeat succeeded. By convention we update the breaker + // before updating the peer. The other way is fine too, just the + // tests need to be written a certain way to avoid being flaky. + report(nil) + + p.mu.Lock() + defer p.mu.Unlock() + p.mu.connected = now + // If the probe was inactive, the fact that we managed to heartbeat implies + // that it ought not have been. + p.mu.deleteAfter = 0 + + // Gauge updates. + p.ConnectionHealthy.Update(1) + p.ConnectionUnhealthy.Update(0) + p.ConnectionInactive.Update(0) + // ConnectionHealthyFor is already zero. + p.ConnectionUnhealthyFor.Update(0) + // AvgRoundTripLatency is already zero. We don't use the initial + // ping since it has overhead of TLS handshake, blocking dialback, etc. + + // Counter updates. + p.ConnectionHeartbeats.Inc(1) + // ConnectionFailures is not updated here. + + // Close the channel last which is helpful for unit tests that + // first waitOrDefault for a healthy conn to then check metrics. + p.mu.c.connFuture.Resolve(cc, nil /* err */) + + logOnHealthy(ctx, p.mu.disconnected, now) +} + +func (p *peer) onSubsequentHeartbeatSucceeded(_ context.Context, now time.Time) { + // Gauge updates. + // ConnectionHealthy is already one. + // ConnectionUnhealthy is already zero. + p.ConnectionHealthyFor.Update(now.Sub(p.snap().connected).Nanoseconds() + 1) // add 1ns for unit tests w/ manual clock + // ConnectionInactive is already zero. + // ConnectionUnhealthyFor is already zero. + p.AvgRoundTripLatency.Update(int64(p.roundTripLatency.Value()) + 1) // add 1ns for unit tests w/ manual clock + + // Counter updates. + p.ConnectionHeartbeats.Inc(1) + // ConnectionFailures is not updated here. +} + +func maybeLogOnFailedHeartbeat( + ctx context.Context, + now time.Time, + err, prevErr error, + snap PeerSnap, // already accounting for `err` + every *log.EveryN, +) { + if errors.Is(err, errQuiescing) { + return + } + // If the error is wrapped in InitialHeartbeatFailedError, unwrap it because that + // error is noise in the logging we're doing here. + if ihb := (*netutil.InitialHeartbeatFailedError)(nil); errors.As(err, &ihb) { + err = ihb.WrappedErr + } + if prevErr == nil && !snap.connected.IsZero() { + // If we just disconnected now after having been healthy, log without rate + // limiting. + + logErr := err + if errors.Is(logErr, grpcutil.ErrConnectionInterrupted) { + // + // We use onlyOnceDialer to prevent gRPC from redialing internally, + // but this means that whenever the connection drops, we get a gRPC + // error that comes from it internally trying to redial and hitting + // the onlyOnceDialer (which refuses with ErrConnectionInterrupted). + // The only actionable information in that is + // ErrConnectionInterrupted; the true reason for the disconnect is + // sadly not available to us (even gRPC logging doesn't contain it + // as far as I can tell!), so the best we can do is be succinct. + // + // We'll basically always hit this path outside of tests when the + // connection breaks (but not when the remote node version + // mismatches, gets decommissioned, etc). + logErr = grpcutil.ErrConnectionInterrupted + } + log.Health.Errorf(ctx, "disconnected (was healthy for %s): %v", + now.Sub(snap.connected).Round(time.Millisecond), logErr, + ) + } else { + // Logging on each failed reconnection is quite noisy and often doesn't + // add anything. So log only on first error, on code change, when V(1) + // is set, or every ~minute. Errors not originating from gRPC are always + // logged. + prevStatus, havePrev := status.FromError(errors.UnwrapAll(prevErr)) + curStatus, _ := status.FromError(errors.UnwrapAll(err)) + if shouldLog := !havePrev || + curStatus.Code() != prevStatus.Code() || + every.ShouldLog() || + log.V(1); shouldLog { + var buf redact.StringBuilder + buf.SafeString("failed connection attempt") + if !snap.disconnected.IsZero() { + _, _ = redact.Fprintf(&buf, " (last connected %s ago)", now.Sub(snap.disconnected).Round(time.Millisecond)) + } else { + _, _ = redact.Fprintf(&buf, " (never connected)") + } + log.Health.Errorf(ctx, "%v: %v", buf, err) + } + } +} + +func (p *peer) onHeartbeatFailed( + ctx context.Context, err error, now time.Time, report func(err error), +) { + prevErr := p.b.Signal().Err() + // For simplicity, we have the convention that this method always returns + // with an error. This is easier to reason about since we're the probe, + // and - morally speaking - the connection is healthy as long as the + // probe is running and happy. We don't want to consider a connection + // healthy when the probe is not running but didn't report an error. + if err == nil { + err = errors.AssertionFailedf("unexpected connection shutdown") + } + + // There might be other peers in the map that are pending deletion, but which + // are no longer seeing activity. To eventually clear them out, we check all + // conns when any conn fails. This avoids the need to have an extra goroutine + // sitting in all of the goroutine stacks we look at during support. + touchOldPeers(p.peers, now) + + // We're a bit careful with the locking here to avoid acquiring p.peers.mu + // while holding p.peer.mu. + var deleteAfter time.Duration + if ts := p.peers.shouldDeleteAfter(p.k, err); ts != 0 { + deleteAfter = ts + } + + p.mu.Lock() + defer p.mu.Unlock() + ls := &p.mu.PeerSnap // "locked snap" + + if !ls.c.connFuture.Resolved() { + // If the initial heartbeat failed (or we got an error creating the + // *grpc.ClientConn), wrap the error. More importantly, resolve connFuture; + // someone might be waiting on it in ConnectNoBreaker who is not paying + // attention to the circuit breaker. + err = &netutil.InitialHeartbeatFailedError{WrappedErr: err} + ls.c.connFuture.Resolve(nil /* cc */, err) + } + // By convention, we stick to updating breaker before updating peer + // to make it easier to write non-flaky tests. + report(err) + + if ls.disconnected.IsZero() || ls.disconnected.Before(ls.connected) { + ls.disconnected = now + } + // If we're not already soft-deleted and soft deletion is indicated now, + // mark as such. + if ls.deleteAfter == 0 && deleteAfter != 0 { + ls.deleteAfter = deleteAfter + } + + maybeLogOnFailedHeartbeat(ctx, now, err, prevErr, *ls, &p.logDisconnectEvery) + + nConnUnhealthy := int64(1) + nConnInactive := int64(0) + connUnhealthyFor := now.Sub(ls.disconnected).Nanoseconds() + 1 // 1ns for unit tests w/ manual clock + if ls.deleteAfter != 0 { + // The peer got marked as pending deletion, so the probe becomes lazy + // (i.e. we terminate the for-loop here and only probe again when someone + // consults the breaker). Reset the gauges, causing this peer to not be + // reflected in aggregate stats any longer. + nConnUnhealthy = 0 + nConnInactive = 1 + connUnhealthyFor = 0 + } + // Gauge updates. + p.ConnectionHealthy.Update(0) + p.ConnectionUnhealthy.Update(nConnUnhealthy) + p.ConnectionInactive.Update(nConnInactive) + p.ConnectionHealthyFor.Update(0) + p.ConnectionUnhealthyFor.Update(connUnhealthyFor) + // NB: keep this last for TestGrpcDialInternal_ReconnectPeer. + p.AvgRoundTripLatency.Update(0) + p.roundTripLatency.Set(0) + // Counter updates. + p.ConnectionFailures.Inc(1) +} + +func (p PeerSnap) deletable(now time.Time) bool { + if p.deleteAfter == 0 { + return false + } + ts := p.disconnected.Add(p.deleteAfter) + return now.After(ts) +} + +// hasSiblingConn takes a peer as identified by `self` and tries to find a sibling: +// - if self.NodeID != 0, finds another conn with matching NodeID but different TargetAddr. +// - if self.NodeID == 0, finds another conn with matching TargetAddr. +// +// In both cases, if such a conn exists that became healthy *after* ours became +// unhealthy, `healthy` will be true. If no such conn exists, (false, false) is +// returned. +func hasSiblingConn(peers map[peerKey]*peer, self peerKey) (healthy, ok bool) { + for other, otherPeer := range peers { + if self == other { + continue // exclude self + } + // NB: we're careful not to call snap() on self because it might be locked + // already. + otherSigCh := otherPeer.b.Signal().C() + + if self.NodeID == 0 { + if other.TargetAddr != self.TargetAddr { + continue + } + // We're a GRPCUnvalidatedDial, which has a tricky life cycle because no + // NodeID is associated. We can't detect an IP address change in the same + // way we can for the NodeID!=0 branch below, nor do we get an event + // telling is we're decommissioned. + // + // We do the simple thing: if there's any another connection to the same + // address (no matter who) and it's healthy, report that. The caller can + // figure out what to do with that information. + } else { + if self.NodeID != other.NodeID || self.TargetAddr == other.TargetAddr { + continue + } + // We're a validated (i.e. with NodeID) connection and found another + // healthy peer matching our NodeID but not our address. Node has + // restarted under a new IP! We are likely obsolete. + // + // Note that if "we" are actually the newer connection (hypothetically + // this could happen if the other connection is "still" healthy, and we + // are "temporarily" unhealthy) then we'll remove the wrong connection, + // but there will be a reconnection attempt (recreating the myKey peer), + // so while causing a hiccup it wouldn't wedge anything. + } + ok = true + select { + // We don't just check `c.Health` because that will trigger + // the probe. If otherSnap belongs to an inactive peer, we + // don't want to randomly do that all of the time; only + // a direct access to the peer by a client should start + // the probe. Checking that the breaker channel is open + // accomplishes that. + case <-otherSigCh: + default: + healthy = true + } + } + return healthy, ok +} + +func (peers *peerMap) shouldDeleteAfter(myKey peerKey, err error) time.Duration { + peers.mu.RLock() + defer peers.mu.RUnlock() + + sibHealthy, ok := hasSiblingConn(peers.mu.m, myKey) + + var deleteAfter time.Duration + if kvpb.IsDecommissionedStatusErr(err) { + deleteAfter = 24 * time.Hour + } else if myKey.NodeID != 0 && ok && sibHealthy { + // We're a NodeID-keyed conn and found another conn to the NodeID that is + // healthy, meaning our TargetAddr is stale. This would be common in k8s + // deployments without stable IPs, where a node would receive a new address + // after each restart. In such a setting, it's usually expected that nobody + // tries to connect under the old address any more after a few minutes, so + // delete more aggressively. + deleteAfter = 5 * time.Minute + } else if myKey.NodeID == 0 { + if !ok || !sibHealthy { + // If we're an unvalidated connection, if we have a healthy sibling we + // assume our connection is still relevant. However, if we *don't* have a + // sibling, what are we to do? We may or may no longer be relevant. Stick + // around for up to 24h, then get deleted, so we avoid log spam (due to + // frequent recreation of the peer) but also we don't leak. If we have a + // sibling but it's not healthy, giving it a 24h grace period is also + // fine; this simplifies the logic because our unhealthy sibling might + // have detected that the IP has moved, but we don't want to have to peek + // into its state too much; we'll spend the next 24h testing the + // connection only when it's accessed, and then remove ourselves. + deleteAfter = 24 * time.Hour + } + } + + return deleteAfter +} + +func touchOldPeers(peers *peerMap, now time.Time) { + var sigs []circuit.Signal + peers.mu.RLock() + for _, p := range peers.mu.m { + if p.snap().deletable(now) { + sigs = append(sigs, p.b.Signal()) + } + } + peers.mu.RUnlock() + + // Now, outside of the lock, query all of the collected Signals which will tip + // off the respective probes, which will perform self-removal from the map. To + // simplify logic, we only allow a peer's probe itself to remove the peer. + // We could do this under peers.mu today but it seems better not to, since + // there is always a chance that the breaker's `AsyncProbe` will one day be + // update to acquire `peers.mu` as well, which would cause locking issues. + for _, sig := range sigs { + // NB: we don't assert that Err() != nil because (at least in theory) + // the connection may have become healthy again in some scenarios. + _ = sig.Err() + } +} + +func (p *peer) maybeDelete(ctx context.Context, now time.Time) { + // If the peer can be deleted, delete it now. + // + // Also delete unconditionally if circuit breakers are (now) disabled. We want + // to allow that setting to bypass an as wide as possible class of issues, so + // we completely yank the peer out of the map. + snap := p.snap() + + if snap.deleted { + log.VEventf(ctx, 2, "peer already deleted") + return + } + + if !p.breakerDisabled() && !snap.deletable(now) { + return + } + + log.VEventf(ctx, 1, "deleting peer") + + // Lock order: map, then peer. But here we can do better and + // not hold both mutexes at the same time. + // + // Release metrics in the same critical section as p.deleted=true + // to make sure the metrics are not updated after release, since that + // causes the aggregate metrics to drift. + // + // We delete from the map first, then mark the peer as deleted. The converse + // works too, but it makes for flakier tests because it's possible to see the + // metrics change but the peer still being in the map. + + p.peers.mu.Lock() + delete(p.peers.mu.m, p.k) + p.peers.mu.Unlock() + + p.mu.Lock() + p.mu.deleted = true + p.peerMetrics.release() + p.mu.Unlock() + +} + +func launchConnStateWatcher( + ctx context.Context, stopper *stop.Stopper, grpcConn *grpc.ClientConn, ch chan connectivity.State, +) { + // The connection should be `Ready` now since we just used it for a + // heartbeat RPC. Any additional state transition indicates that we need + // to remove it, and we want to do so reactively. Unfortunately, gRPC + // forces us to spin up a separate goroutine for this purpose even + // though it internally uses a channel. + // Note also that the implementation of this in gRPC is clearly racy, + // so consider this somewhat best-effort. + _ = stopper.RunAsyncTask(ctx, "conn state watcher", func(ctx context.Context) { + st := connectivity.Ready + for { + if !grpcConn.WaitForStateChange(ctx, st) { + return + } + st = grpcConn.GetState() + if st == connectivity.TransientFailure || st == connectivity.Shutdown { + ch <- st + return + } + } + }) +} diff --git a/pkg/rpc/peer_map.go b/pkg/rpc/peer_map.go new file mode 100644 index 000000000000..33cffad9d29d --- /dev/null +++ b/pkg/rpc/peer_map.go @@ -0,0 +1,76 @@ +// Copyright 2023 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 rpc + +import ( + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/util/circuit" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" + "github.com/cockroachdb/redact" +) + +// peerKey is used as key in the Context.peers map. +// Connections which carry a different class but share a target and nodeID +// will always specify distinct connections. Different remote node IDs get +// distinct *Connection objects to ensure that we don't mis-route RPC +// requests in the face of address reuse. Gossip connections and other +// non-Internal users of the Context are free to dial nodes without +// specifying a node ID (see GRPCUnvalidatedDial()) however later calls to +// Dial with the same target and class with a node ID will create a new +// underlying connection which will not be reused by calls specifying the +// NodeID. +type peerKey struct { + TargetAddr string + // NodeID of remote node, 0 when unknown, non-zero to check with remote node. + // Never mutated. + NodeID roachpb.NodeID + Class ConnectionClass +} + +var _ redact.SafeFormatter = peerKey{} + +// SafeFormat implements the redact.SafeFormatter interface. +func (c peerKey) SafeFormat(p redact.SafePrinter, _ rune) { + p.Printf("{n%d: %s (%v)}", c.NodeID, c.TargetAddr, c.Class) +} + +type peerMap struct { + mu struct { + syncutil.RWMutex + m map[peerKey]*peer + } +} + +func (peers *peerMap) getWithBreaker(k peerKey) (PeerSnap, peerMetrics, *circuit.Breaker, bool) { + peers.mu.RLock() + defer peers.mu.RUnlock() + p := peers.mu.m[k] + if p == nil { + return PeerSnap{}, peerMetrics{}, nil, false + } + return p.snap(), p.peerMetrics, p.b, true +} + +// Conn returns a read-only version of the peer and a boolean indicating +// whether the peer exists. +func (peers *peerMap) get(k peerKey) (PeerSnap, bool) { + peers.mu.RLock() + defer peers.mu.RUnlock() + return peers.getRLocked(k) +} + +func (peers *peerMap) getRLocked(k peerKey) (PeerSnap, bool) { + p, ok := peers.mu.m[k] + if !ok { + return PeerSnap{}, false + } + return p.snap(), true +} diff --git a/pkg/rpc/peer_test.go b/pkg/rpc/peer_test.go new file mode 100644 index 000000000000..27c1bbd43f18 --- /dev/null +++ b/pkg/rpc/peer_test.go @@ -0,0 +1,207 @@ +// Copyright 2023 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 rpc + +import ( + "testing" + + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/util/circuit" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/errors" +) + +type pmb map[peerKey]*peer // peer map builder + +func (m pmb) k(k peerKey) pmb { + return m.kh(k, true /* healthy */) +} + +func (m pmb) kh(k peerKey, healthy bool) pmb { + if m == nil { + m = pmb{} + } + p := &peer{} + p.b = circuit.NewBreaker(circuit.Options{ + Name: "test", + }) + if !healthy { + circuit.TestingSetTripped(p.b, errors.New("boom")) + } + m[k] = p + return m +} + +func (m pmb) p(nodeID roachpb.NodeID, targetAddr string, class ConnectionClass, healthy bool) pmb { + k := peerKey{TargetAddr: targetAddr, NodeID: nodeID, Class: class} + return m.kh(k, healthy) +} + +func Test_hasSiblingConn(t *testing.T) { + defer leaktest.AfterTest(t)() + + const ( + t1 = "1.1.1.1:1111" + t2 = "2.2.2.2:2222" + t3 = "3.3.3.3:3333" + n1 = roachpb.NodeID(1) + n2 = roachpb.NodeID(2) + n3 = roachpb.NodeID(3) + def = DefaultClass + sys = SystemClass + ) + + defSelf := func() peerKey { + return peerKey{ + TargetAddr: t1, + NodeID: n1, + Class: def, + } + } + + defZeroSelf := func() peerKey { + return peerKey{ + TargetAddr: t1, + Class: def, + } + } + + tests := []struct { + name string + m pmb + self peerKey + wantHealthy bool + wantOk bool + }{ + // In the empty map, defSelf has no sibling. + { + name: "empty", + }, + + // First, we test various combinations where our peerKey has a NodeID. + // We'll test the NodeID==0 case below. + + // In a map that contains only itself, defSelf has no sibling. + { + name: "single", + m: (&pmb{}).k(defSelf()), + }, + // In a map that contains defSelf and another healthy but unrelated peer, + // defSelf finds no sibling. + { + name: "unrelated", + m: (&pmb{}).k(defSelf()).p(n2, t2, sys, true /* healthy */), + }, + // Same as previous, but with a curveball: n2 also hails from defSelf's + // address. (In other words, n2 must have restarted with a listener + // formerly used by n1, or vice versa). Not a sibling - we're looking + // for the same NodeID on a different TargetAddr. + { + name: "unrelated-switcheroo", + m: (&pmb{}).k(defSelf()).p(n2, t1, def, true /* healthy */), + }, + // Unrelated peer that has a zero NodeID. + { + name: "unrelated-switcheroo", + m: (&pmb{}).k(defSelf()).p(0, t3, def, true /* healthy */), + }, + // If a healthy sibling is present, it is found. + { + name: "related-healthy", + m: (&pmb{}).k(defSelf()).p(n1, t2, def, true /* healthy */), + wantOk: true, wantHealthy: true, + }, + // Like last, but sibling has a different connection class - outcome is the + // same. + { + name: "related-healthy-sys", + m: (&pmb{}).k(defSelf()).p(n1, t2, sys, true /* healthy */), + wantOk: true, wantHealthy: true, + }, + // If an unhealthy sibling is present, it is found. + { + name: "related-unhealthy", + m: (&pmb{}).k(defSelf()).p(n1, t2, def, false /* healthy */), + wantOk: true, wantHealthy: false, + }, + // Ditto with other class. + { + name: "related-unhealthy-sys", + m: (&pmb{}).k(defSelf()).p(n1, t2, sys, false /* healthy */), + wantOk: true, wantHealthy: false, + }, + + // Now we test the NodeID == 0 case. + + // Peer is not sibling of self. + { + name: "zero-single", + m: (&pmb{}).k(defZeroSelf()), + self: defZeroSelf(), + }, + // Peer is not sibling of other peers that have non-matching target addrs. + { + name: "zero-unrelated", + m: (&pmb{}). + k(defZeroSelf()). + p(n2, t2, def, true). + p(n3, t3, def, true). + p(0, t2, sys, true), + self: defZeroSelf(), + }, + // Peer is sibling of NodeID-peer with matching addr. + { + name: "zero-matching", + m: (&pmb{}). + k(defZeroSelf()). + p(1, t1, sys, true), + self: defZeroSelf(), + wantOk: true, + wantHealthy: true, + }, + // Ditto but peer is not healthy. + { + name: "zero-matching-unhealthy", + m: (&pmb{}). + k(defZeroSelf()). + p(1, t1, sys, false), + self: defZeroSelf(), + wantOk: true, + wantHealthy: false, + }, + // Peer is sibling of zero-peer with matching addr. (It necessarily has + // a different class, or it would be the same peer). + { + name: "zero-matching-zero", + m: (&pmb{}). + k(defZeroSelf()). + p(0, t1, sys, true), + self: defZeroSelf(), + wantOk: true, + wantHealthy: true, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + self := tt.self + if self.TargetAddr == "" { + self = defSelf() + } + gotHealthy, gotOk := hasSiblingConn(tt.m, self) + if gotHealthy != tt.wantHealthy { + t.Errorf("hasSiblingConn() gotHealthy = %v, want %v", gotHealthy, tt.wantHealthy) + } + if gotOk != tt.wantOk { + t.Errorf("hasSiblingConn() gotOk = %v, want %v", gotOk, tt.wantOk) + } + }) + } +} diff --git a/pkg/rpc/settings.go b/pkg/rpc/settings.go new file mode 100644 index 000000000000..d3c59c938797 --- /dev/null +++ b/pkg/rpc/settings.go @@ -0,0 +1,123 @@ +// Copyright 2023 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 rpc + +import ( + "context" + "fmt" + "net" + + "github.com/cockroachdb/cockroach/pkg/settings" + "github.com/cockroachdb/cockroach/pkg/util/envutil" + "github.com/cockroachdb/cockroach/pkg/util/log" + "google.golang.org/grpc" +) + +func init() { + // Disable GRPC tracing. This retains a subset of messages for + // display on /debug/requests, which is very expensive for + // snapshots. Until we can be more selective about what is retained + // in traces, we must disable tracing entirely. + // https://github.com/grpc/grpc-go/issues/695 + grpc.EnableTracing = false +} + +var enableRPCCircuitBreakers = settings.RegisterBoolSetting( + settings.TenantReadOnly, + "rpc.circuit_breaker.enabled", + "enables stateful management of failed connections, including circuit breaking "+ + "when in unhealthy state; only use in case of issues - logging may be suboptimal "+ + "and metrics related to connection errors will not be populated correctly", + envutil.EnvOrDefaultBool("COCKROACH_RPC_CIRCUIT_BREAKERS_ENABLED", true), +) + +// TODO(baptist): Remove in 23.2 (or 24.1) once validating dialback works for all scenarios. +var useDialback = settings.RegisterBoolSetting( + settings.TenantReadOnly, + "rpc.dialback.enabled", + "if true, require bidirectional RPC connections between nodes to prevent one-way network unavailability", + true, +) + +var enableRPCCompression = envutil.EnvOrDefaultBool("COCKROACH_ENABLE_RPC_COMPRESSION", true) + +func getWindowSize(name string, c ConnectionClass, defaultSize int) int32 { + const maxWindowSize = defaultWindowSize * 32 + s := envutil.EnvOrDefaultInt(name, defaultSize) + if s > maxWindowSize { + log.Warningf(context.Background(), "%s value too large; trimmed to %d", name, maxWindowSize) + s = maxWindowSize + } + if s <= defaultWindowSize { + log.Warningf(context.Background(), + "%s RPC will use dynamic window sizes due to %s value lower than %d", c, name, defaultSize) + } + return int32(s) +} + +const ( + defaultWindowSize = 65535 + // The coefficient by which the tolerated offset is multiplied to determine + // the maximum acceptable measurement latency. + maximumPingDurationMult = 2 +) + +var ( + // for an RPC + initialWindowSize = getWindowSize( + "COCKROACH_RPC_INITIAL_WINDOW_SIZE", DefaultClass, defaultWindowSize*32) + initialConnWindowSize = initialWindowSize * 16 // for a connection + + // for RangeFeed RPC + rangefeedInitialWindowSize = getWindowSize( + "COCKROACH_RANGEFEED_RPC_INITIAL_WINDOW_SIZE", RangefeedClass, 2*defaultWindowSize /* 128K */) +) + +// sourceAddr is the environment-provided local address for outgoing +// connections. +var sourceAddr = func() net.Addr { + const envKey = "COCKROACH_SOURCE_IP_ADDRESS" + if sourceAddr, ok := envutil.EnvString(envKey, 0); ok { + sourceIP := net.ParseIP(sourceAddr) + if sourceIP == nil { + panic(fmt.Sprintf("unable to parse %s '%s' as IP address", envKey, sourceAddr)) + } + return &net.TCPAddr{ + IP: sourceIP, + } + } + return nil +}() + +type serverOpts struct { + interceptor func(fullMethod string) error +} + +// ServerOption is a configuration option passed to NewServer. +type ServerOption func(*serverOpts) + +// WithInterceptor adds an additional interceptor. The interceptor is called before +// streaming and unary RPCs and may inject an error. +func WithInterceptor(f func(fullMethod string) error) ServerOption { + return func(opts *serverOpts) { + if opts.interceptor == nil { + opts.interceptor = f + } else { + f := opts.interceptor + opts.interceptor = func(fullMethod string) error { + if err := f(fullMethod); err != nil { + return err + } + return f(fullMethod) + } + } + } +} diff --git a/pkg/rpc/testdata/TestReconnection/decommission.txt b/pkg/rpc/testdata/TestReconnection/decommission.txt new file mode 100644 index 000000000000..07f6990e8c0d --- /dev/null +++ b/pkg/rpc/testdata/TestReconnection/decommission.txt @@ -0,0 +1,79 @@ +# Verify that when a node is decommissioned, it transitions +# into inactive mode and is deleted once enough time has passed. + +connect n1 class=def +---- +ok + +soon healthy=1 +---- +ok + +set-hb-err n1 +---- +ok + +soon unhealthy=1 +---- +ok + +reset-hb-err n1 +---- +ok + +set-hb-err n1 decommissioned=true +---- +ok + +soon inactive=1 +---- +ok + +show n1 class=def +---- +tripped: true +inactive: true +deletable: false + +tick 25h +---- +1970-01-02 01:00:09 +0000 UTC + +show n1 class=def +---- +tripped: true +inactive: true +deletable: true + +# We want to check that n1 gets removed since lots of +# time has passed, and we do so indirectly: we make +# another peer that fails a heartbeat. Whenever that +# happens we also groom the peers map, which should +# end up removing n1. The same would happen if we directly +# dialed n1 but in practice that isn't the frequent path +# since n1 as a node has long disappeared from the cluster. + +connect n1' class=sys +---- +ok + +set-hb-err n1' +---- +ok + +# Note that the inactive peer is gone. +soon unhealthy=1 +---- +ok + +show n1 class=def +---- + + +reset-hb-err n1' +---- +ok + +soon healthy=1 +---- +ok diff --git a/pkg/rpc/testdata/TestReconnection/restart_new_ip.txt b/pkg/rpc/testdata/TestReconnection/restart_new_ip.txt new file mode 100644 index 000000000000..b14ceeea3719 --- /dev/null +++ b/pkg/rpc/testdata/TestReconnection/restart_new_ip.txt @@ -0,0 +1,76 @@ +# This test demonstrates that the rpc connection +# pool handles the case in which a node is listening +# on two separate addresses. +# Even though this is not something that should occur +# in production, misconfiguration could conceivably +# create that situation. Also, nodes restarting and +# moving between IP addresses in the process could, +# for a short moment, lead to a situation that looks +# almost indistinguishable. +# +# The test then proceeds to demonstrate that when one +# of the two connections goes unhealthy, it will enter +# inactive mode, but can also recover from that when +# needed (and possible). +# +# All in all, this test demonstrates satisfactory +# handling of the case in which a node restarts +# under a new IP address. +connect n1 class=def +---- +ok + +soon healthy=1 +---- +ok + +connect n1' class=def +---- +ok + +soon healthy=2 +---- +ok + +set-hb-err n1 +---- +ok + +# When n1 fails, it sees the healthy +# connection to n1' and goes into inactive +# mode. +soon healthy=1 inactive=1 +---- +ok + +set-hb-err n1' +---- +ok + +# When n1' also trips, it will be marked +# as unhealthy. +soon healthy=0 unhealthy=1 inactive=1 +---- +ok + +reset-hb-err n1 n1' +---- +ok + +# Despite heartbeats not failing, the connection +# to n1 remains inactive. It needs some activity +# to trigger the probe. +soon healthy=1 inactive=1 +---- +ok + +# This attempt kicks off a probe, but doesn't wait for the probe. +connect n1 class=def +---- +error code: ‹Unknown› [tripped=true] + +# However, the probe *was* kicked off and heals the peer +# soon. +soon healthy=2 +---- +ok diff --git a/pkg/rpc/testdata/TestReconnection/single.txt b/pkg/rpc/testdata/TestReconnection/single.txt new file mode 100644 index 000000000000..b5af01c75f9f --- /dev/null +++ b/pkg/rpc/testdata/TestReconnection/single.txt @@ -0,0 +1,71 @@ +# This basic test operates against a single server. It establishes a healthy +# DefaultClass connection, then interrupts the heartbeat, lets the peer reconnect, +# then adds a SystemClass and RangeFeedClass connection, and interrupts heartbeats +# again. At each step, metrics are verified via `soon`. +dial n1 class=def +---- +ok + +soon healthy=1 +---- +ok + +connect n1 class=def +---- +ok + +set-hb-err n1 +---- +ok + +soon unhealthy=1 +---- +ok + +show n1 class=def +---- +tripped: true +inactive: false +deletable: false + +# Trying to reconnect, we are stopped by the +# circuit breaker. +connect n1 class=def +---- +error code: ‹Unknown› [tripped=true] + +reset-hb-err n1 +---- +ok + +soon n1 healthy=1 +---- +ok + +connect n1 class=def +---- +ok + +connect n1 class=sys +---- +ok + +soon n1 healthy=2 +---- +ok + +connect n1 class=rf +---- +ok + +soon healthy=3 +---- +ok + +set-hb-err n1 +---- +ok + +soon n1 unhealthy=3 +---- +ok diff --git a/pkg/server/admin_test.go b/pkg/server/admin_test.go index b23e7f653784..53b70fd59070 100644 --- a/pkg/server/admin_test.go +++ b/pkg/server/admin_test.go @@ -3190,7 +3190,7 @@ func TestAdminDecommissionedOperations(t *testing.T) { // This will cause SuccessWithin to retry. return err } - require.Equal(t, tc.expectCode, s.Code()) + require.Equal(t, tc.expectCode, s.Code(), "%+v", err) return nil }, 10*time.Second) }) diff --git a/pkg/server/decommission.go b/pkg/server/decommission.go index e76722e0e19b..b8c34d167864 100644 --- a/pkg/server/decommission.go +++ b/pkg/server/decommission.go @@ -16,6 +16,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/allocatorimpl" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" @@ -138,16 +139,12 @@ func getPingCheckDecommissionFn( return nodeTombStorage, func(ctx context.Context, nodeID roachpb.NodeID, errorCode codes.Code) error { ts, err := nodeTombStorage.IsDecommissioned(ctx, nodeID) if err != nil { - // An error here means something very basic is not working. Better to terminate - // than to limp along. - log.Fatalf(ctx, "unable to read decommissioned status for n%d: %v", nodeID, err) + return errors.Wrapf(err, "unable to read decommissioned status for n%d", nodeID) } if !ts.IsZero() { - // The node was decommissioned. - return grpcstatus.Errorf(errorCode, + return kvpb.NewDecommissionedStatusErrorf(errorCode, "n%d was permanently removed from the cluster at %s; it is not allowed to rejoin the cluster", - nodeID, ts, - ) + nodeID, ts) } // The common case - target node is not decommissioned. return nil diff --git a/pkg/ui/workspaces/cluster-ui/src/store/nodes/nodes.fixtures.ts b/pkg/ui/workspaces/cluster-ui/src/store/nodes/nodes.fixtures.ts index 63b39d1c497a..5e75b01f2a6e 100644 --- a/pkg/ui/workspaces/cluster-ui/src/store/nodes/nodes.fixtures.ts +++ b/pkg/ui/workspaces/cluster-ui/src/store/nodes/nodes.fixtures.ts @@ -306,7 +306,8 @@ export const getNodeStatus = (): INodeStatus => { "round-trip-latency-p99.999": 1114111, "rpc.heartbeats.failed": 0, "rpc.heartbeats.initializing": 0, - "rpc.heartbeats.loops.exited": 0, + "rpc.heartbeats.connection_failures": 0, + "rpc.heartbeats.round-trip.latency": 0, "rpc.heartbeats.loops.started": 1, "rpc.heartbeats.nominal": 1, "schedules.error": 0, From 51d2876626b494f20f3a08f9a888222d5b0ddc04 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Thu, 11 May 2023 14:42:48 +0200 Subject: [PATCH 05/17] nodedialer: remove test for deprecated breaker from TestConnHealth{,TryDial} Epic: none Release note: None --- pkg/rpc/nodedialer/nodedialer_test.go | 19 ------------------- 1 file changed, 19 deletions(-) diff --git a/pkg/rpc/nodedialer/nodedialer_test.go b/pkg/rpc/nodedialer/nodedialer_test.go index c5eb043783b2..0f294f674437 100644 --- a/pkg/rpc/nodedialer/nodedialer_test.go +++ b/pkg/rpc/nodedialer/nodedialer_test.go @@ -187,15 +187,6 @@ func TestConnHealth(t *testing.T) { }, time.Second, 10*time.Millisecond) } - // Tripping the breaker should return ErrBreakerOpen. - br := nd.getBreaker(staticNodeID, rpc.DefaultClass) - br.Trip() - require.Equal(t, circuit.ErrBreakerOpen, nd.ConnHealth(staticNodeID, rpc.DefaultClass)) - - // Resetting the breaker should recover ConnHealth. - br.Reset() - require.NoError(t, nd.ConnHealth(staticNodeID, rpc.DefaultClass)) - // Closing the remote connection should fail ConnHealth. require.NoError(t, ln.popConn().Close()) hbDecommission.Store(true) @@ -241,16 +232,6 @@ func TestConnHealthTryDial(t *testing.T) { return nd.ConnHealthTryDial(staticNodeID, rpc.DefaultClass) == nil }, time.Second, 10*time.Millisecond) - // Tripping the breaker should return ErrBreakerOpen. - br := nd.getBreaker(staticNodeID, rpc.DefaultClass) - br.Trip() - require.Equal(t, circuit.ErrBreakerOpen, nd.ConnHealthTryDial(staticNodeID, rpc.DefaultClass)) - - // But it should eventually recover, when the breaker allows it. - require.Eventually(t, func() bool { - return nd.ConnHealthTryDial(staticNodeID, rpc.DefaultClass) == nil - }, 5*time.Second, 10*time.Millisecond) - // Closing the remote connection should eventually recover. require.NoError(t, ln.popConn().Close()) require.Eventually(t, func() bool { From fd576f40b4d4a21e98179b7fd18cd0524886c031 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Thu, 11 May 2023 15:59:05 +0200 Subject: [PATCH 06/17] nodedialer: modernize TestDialNoBreaker Test the functionality via the new circuit breakers, i.e. this test will continue working if we remove the old breakers (which it no longer tests). Epic: none Release note: None --- pkg/rpc/nodedialer/BUILD.bazel | 1 + pkg/rpc/nodedialer/nodedialer_test.go | 63 ++++++++++----------------- 2 files changed, 25 insertions(+), 39 deletions(-) diff --git a/pkg/rpc/nodedialer/BUILD.bazel b/pkg/rpc/nodedialer/BUILD.bazel index 39f914a42c91..9bf284a8241c 100644 --- a/pkg/rpc/nodedialer/BUILD.bazel +++ b/pkg/rpc/nodedialer/BUILD.bazel @@ -35,6 +35,7 @@ go_test( "//pkg/rpc", "//pkg/settings/cluster", "//pkg/testutils", + "//pkg/util/circuit", "//pkg/util/hlc", "//pkg/util/leaktest", "//pkg/util/log", diff --git a/pkg/rpc/nodedialer/nodedialer_test.go b/pkg/rpc/nodedialer/nodedialer_test.go index 0f294f674437..15fe8dad866b 100644 --- a/pkg/rpc/nodedialer/nodedialer_test.go +++ b/pkg/rpc/nodedialer/nodedialer_test.go @@ -27,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/testutils" + circuitbreaker "github.com/cockroachdb/cockroach/pkg/util/circuit" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -77,51 +78,35 @@ func TestDialNoBreaker(t *testing.T) { testutils.SucceedsSoon(t, func() error { return nd.ConnHealth(staticNodeID, rpc.DefaultClass) }) - breaker := nd.GetCircuitBreaker(staticNodeID, rpc.DefaultClass) - assert.True(t, breaker.Ready()) - // Test that DialNoBreaker is successful normally. - _, err = nd.DialNoBreaker(ctx, staticNodeID, rpc.DefaultClass) - assert.Nil(t, err, "failed to dial") - assert.True(t, breaker.Ready()) - assert.Equal(t, breaker.Failures(), int64(0)) + require.NoError(t, rpcCtx.ConnHealth(ln.Addr().String(), staticNodeID, rpc.DefaultClass)) - // Now trip the breaker and check that DialNoBreaker will go ahead - // and dial anyway, and on top of that open the breaker again (since - // the dial will succeed). - breaker.Trip() - require.True(t, breaker.Tripped()) + // Test that DialNoBreaker is successful normally. + conn := rpcCtx.GRPCDialNode(ln.Addr().String(), staticNodeID, rpc.DefaultClass) + require.NoError(t, conn.Signal().Err()) _, err = nd.DialNoBreaker(ctx, staticNodeID, rpc.DefaultClass) require.NoError(t, err) - require.False(t, breaker.Tripped()) + // Ditto regular dial. + _, err = nd.Dial(ctx, staticNodeID, rpc.DefaultClass) + require.NoError(t, err) + + injErr := errors.New("injected error") + + // Mock-trip the breaker. (This leaves the connection intact). + { + b, ok := rpcCtx.GetBreakerForAddr(staticNodeID, rpc.DefaultClass, ln.Addr()) + require.True(t, ok) + undo := circuitbreaker.TestingSetTripped(b, injErr) + defer undo() + } + // Regular Dial should be refused, but DialNoBreaker will + // still work. + + _, err = nd.Dial(ctx, staticNodeID, rpc.DefaultClass) + require.True(t, errors.Is(err, injErr), "%+v", err) - // Test that resolver errors also trip the breaker, just like - // they would for regular Dial. - boom := fmt.Errorf("boom") - nd = New(rpcCtx, func(roachpb.NodeID) (net.Addr, error) { - return nil, boom - }) - breaker = nd.GetCircuitBreaker(staticNodeID, rpc.DefaultClass) - _, err = nd.DialNoBreaker(ctx, staticNodeID, rpc.DefaultClass) - assert.Equal(t, errors.Cause(err), boom) - assert.Equal(t, breaker.Failures(), int64(1)) - - // Test that connection errors are reported to the breaker even - // with DialNoBreaker. - // To do this, we have to trick grpc into never successfully dialing - // the server, because if it succeeds once then it doesn't try again - // to perform a connection. To trick grpc in this way, we have to - // set up a server without the heartbeat service running. Without - // getting a heartbeat, the nodedialer will throw an error thinking - // that it wasn't able to successfully make a connection. - _, ln, _ = newTestServer(t, clock, stopper, false /* useHeartbeat */) - nd = New(rpcCtx, newSingleNodeResolver(staticNodeID, ln.Addr())) - breaker = nd.GetCircuitBreaker(staticNodeID, rpc.DefaultClass) - assert.True(t, breaker.Ready()) - assert.Equal(t, breaker.Failures(), int64(0)) _, err = nd.DialNoBreaker(ctx, staticNodeID, rpc.DefaultClass) - assert.NotNil(t, err, "expected dial error") - assert.Equal(t, breaker.Failures(), int64(1)) + require.NoError(t, err) } func TestConnHealth(t *testing.T) { From 62a02da6cd6efaf8eb5e0fa364b06f4edb50fbe9 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Thu, 11 May 2023 22:18:18 +0200 Subject: [PATCH 07/17] nodedialer: remove TestConcurrentCancellationAndTimeout It was specific to the nodedialer-level circuit breakers, which will be removed. Epic: none Release note: None --- pkg/rpc/nodedialer/nodedialer_test.go | 34 --------------------------- 1 file changed, 34 deletions(-) diff --git a/pkg/rpc/nodedialer/nodedialer_test.go b/pkg/rpc/nodedialer/nodedialer_test.go index 15fe8dad866b..3c9cb0891ab9 100644 --- a/pkg/rpc/nodedialer/nodedialer_test.go +++ b/pkg/rpc/nodedialer/nodedialer_test.go @@ -262,40 +262,6 @@ func TestConnHealthInternal(t *testing.T) { require.Error(t, nd.ConnHealth(7, rpc.DefaultClass)) } -func TestConcurrentCancellationAndTimeout(t *testing.T) { - defer leaktest.AfterTest(t)() - stopper, _, _, _, nd := setUpNodedialerTest(t, staticNodeID) - defer stopper.Stop(context.Background()) - ctx := context.Background() - breaker := nd.GetCircuitBreaker(staticNodeID, rpc.DefaultClass) - // Test that when a context is canceled during dialing we always return that - // error but we never trip the breaker. - const N = 1000 - var wg sync.WaitGroup - for i := 0; i < N; i++ { - wg.Add(2) - // Jiggle when we cancel relative to when we dial to try to hit cases where - // cancellation happens during the call to GRPCDial. - iCtx, cancel := context.WithTimeout(ctx, randDuration(time.Millisecond)) - go func() { - time.Sleep(randDuration(time.Millisecond)) - cancel() - wg.Done() - }() - go func() { - time.Sleep(randDuration(time.Millisecond)) - _, err := nd.Dial(iCtx, 1, rpc.DefaultClass) - if err != nil && - !errors.IsAny(err, context.Canceled, context.DeadlineExceeded) { - t.Errorf("got an unexpected error from Dial: %v", err) - } - wg.Done() - }() - } - wg.Wait() - assert.Equal(t, breaker.Failures(), int64(0)) -} - func TestResolverErrorsTrip(t *testing.T) { defer leaktest.AfterTest(t)() stopper, rpcCtx, _, _, _ := setUpNodedialerTest(t, staticNodeID) From 0b53325954e47662212f4a31e144d378d9e0693a Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Thu, 11 May 2023 22:25:40 +0200 Subject: [PATCH 08/17] nodedialer: remove breaker check from TestNodedialerPositive This was testing the deprecated breaker. Epic: none Release note: None --- pkg/rpc/nodedialer/nodedialer_test.go | 5 ----- 1 file changed, 5 deletions(-) diff --git a/pkg/rpc/nodedialer/nodedialer_test.go b/pkg/rpc/nodedialer/nodedialer_test.go index 3c9cb0891ab9..e8dccae4776c 100644 --- a/pkg/rpc/nodedialer/nodedialer_test.go +++ b/pkg/rpc/nodedialer/nodedialer_test.go @@ -15,7 +15,6 @@ import ( "fmt" "math/rand" "net" - "sync" "sync/atomic" "testing" "time" @@ -49,13 +48,9 @@ func TestNodedialerPositive(t *testing.T) { stopper, _, _, _, nd := setUpNodedialerTest(t, staticNodeID) defer stopper.Stop(context.Background()) // Ensure that dialing works. - breaker := nd.GetCircuitBreaker(1, rpc.DefaultClass) - assert.True(t, breaker.Ready()) ctx := context.Background() _, err := nd.Dial(ctx, staticNodeID, rpc.DefaultClass) assert.Nil(t, err, "failed to dial") - assert.True(t, breaker.Ready()) - assert.Equal(t, breaker.Failures(), int64(0)) } func TestDialNoBreaker(t *testing.T) { From 9dbb69459e07ff7bec98c8d77a26904c267f95f1 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Thu, 11 May 2023 22:26:23 +0200 Subject: [PATCH 09/17] nodedialer: remove Test{Disconnects,ResolverErrors}Trip Epic: none Release note: None --- pkg/rpc/nodedialer/nodedialer_test.go | 94 --------------------------- 1 file changed, 94 deletions(-) diff --git a/pkg/rpc/nodedialer/nodedialer_test.go b/pkg/rpc/nodedialer/nodedialer_test.go index e8dccae4776c..f9cd4c6a75ed 100644 --- a/pkg/rpc/nodedialer/nodedialer_test.go +++ b/pkg/rpc/nodedialer/nodedialer_test.go @@ -13,7 +13,6 @@ package nodedialer import ( "context" "fmt" - "math/rand" "net" "sync/atomic" "testing" @@ -257,94 +256,6 @@ func TestConnHealthInternal(t *testing.T) { require.Error(t, nd.ConnHealth(7, rpc.DefaultClass)) } -func TestResolverErrorsTrip(t *testing.T) { - defer leaktest.AfterTest(t)() - stopper, rpcCtx, _, _, _ := setUpNodedialerTest(t, staticNodeID) - defer stopper.Stop(context.Background()) - boom := fmt.Errorf("boom") - nd := New(rpcCtx, func(id roachpb.NodeID) (net.Addr, error) { - return nil, boom - }) - _, err := nd.Dial(context.Background(), staticNodeID, rpc.DefaultClass) - assert.Equal(t, errors.Cause(err), boom) - breaker := nd.GetCircuitBreaker(staticNodeID, rpc.DefaultClass) - assert.False(t, breaker.Ready()) -} - -func TestDisconnectsTrip(t *testing.T) { - defer leaktest.AfterTest(t)() - stopper, _, ln, hb, nd := setUpNodedialerTest(t, staticNodeID) - defer stopper.Stop(context.Background()) - ctx := context.Background() - breaker := nd.GetCircuitBreaker(staticNodeID, rpc.DefaultClass) - - // Now close the underlying connection from the server side and set the - // heartbeat service to return errors. This will eventually lead to the client - // connection being removed and Dial attempts to return an error. - // While this is going on there will be many clients attempting to - // connect. These connecting clients will send interesting errors they observe - // on the errChan. Once an error from Dial is observed the test re-enables the - // heartbeat service. The test will confirm that the only errors they record - // in to the breaker are interesting ones as determined by shouldTrip. - hb.setErr(fmt.Errorf("boom")) - underlyingNetConn := ln.popConn() - require.NoError(t, underlyingNetConn.Close()) - const N = 1000 - breakerEventChan := make(chan circuit.ListenerEvent, N) - breaker.AddListener(breakerEventChan) - errChan := make(chan error, N) - shouldTrip := func(err error) bool { - return err != nil && - !errors.IsAny(err, context.DeadlineExceeded, context.Canceled, circuit.ErrBreakerOpen) - } - var wg sync.WaitGroup - for i := 0; i < N; i++ { - wg.Add(2) - iCtx, cancel := context.WithTimeout(ctx, randDuration(time.Millisecond)) - go func() { - time.Sleep(randDuration(time.Millisecond)) - cancel() - wg.Done() - }() - go func() { - time.Sleep(randDuration(time.Millisecond)) - _, err := nd.Dial(iCtx, 1, rpc.DefaultClass) - if shouldTrip(err) { - errChan <- err - } - wg.Done() - }() - } - go func() { wg.Wait(); close(errChan) }() - var errorsSeen int - for range errChan { - if errorsSeen == 0 { - hb.setErr(nil) - } - errorsSeen++ - } - breaker.RemoveListener(breakerEventChan) - close(breakerEventChan) - var failsSeen int - for ev := range breakerEventChan { - if ev.Event == circuit.BreakerFail { - failsSeen++ - } - } - // Ensure that all of the interesting errors were seen by the breaker. - require.Equal(t, errorsSeen, failsSeen) - - // Ensure that the connection eventually becomes healthy if we fix the - // heartbeat and keep dialing. - hb.setErr(nil) - testutils.SucceedsSoon(t, func() error { - if _, err := nd.Dial(ctx, staticNodeID, rpc.DefaultClass); err != nil { - return err - } - return nd.ConnHealth(staticNodeID, rpc.DefaultClass) - }) -} - func setUpNodedialerTest( t *testing.T, nodeID roachpb.NodeID, ) ( @@ -370,11 +281,6 @@ func setUpNodedialerTest( return stopper, rpcCtx, ln, hb, nd } -// randDuration returns a uniform random duration between 0 and max. -func randDuration(max time.Duration) time.Duration { - return time.Duration(rand.Intn(int(max))) -} - func newTestServer( t testing.TB, clock hlc.WallClock, stopper *stop.Stopper, useHeartbeat bool, ) (*grpc.Server, *interceptingListener, *heartbeatService) { From 74c0dacb0255bd96fe9679d8847899aeea4fbba7 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Thu, 11 May 2023 22:27:14 +0200 Subject: [PATCH 10/17] importer: use new rpc breaker in TestImportRetriesBreakerOpenFailure Epic: none Release note: None --- pkg/sql/importer/BUILD.bazel | 2 ++ pkg/sql/importer/import_stmt_test.go | 48 +++++++++++++++++++++++----- 2 files changed, 42 insertions(+), 8 deletions(-) diff --git a/pkg/sql/importer/BUILD.bazel b/pkg/sql/importer/BUILD.bazel index f814ed31fc86..ad06983ce7ab 100644 --- a/pkg/sql/importer/BUILD.bazel +++ b/pkg/sql/importer/BUILD.bazel @@ -187,6 +187,7 @@ go_test( "//pkg/kv/kvserver/kvserverbase", "//pkg/roachpb", "//pkg/rpc", + "//pkg/rpc/nodedialer", "//pkg/security/securityassets", "//pkg/security/securitytest", "//pkg/security/username", @@ -231,6 +232,7 @@ go_test( "//pkg/testutils/sqlutils", "//pkg/testutils/testcluster", "//pkg/util", + "//pkg/util/circuit", "//pkg/util/ctxgroup", "//pkg/util/encoding/csv", "//pkg/util/envutil", diff --git a/pkg/sql/importer/import_stmt_test.go b/pkg/sql/importer/import_stmt_test.go index a7fa18f3a531..06b63dc86e4a 100644 --- a/pkg/sql/importer/import_stmt_test.go +++ b/pkg/sql/importer/import_stmt_test.go @@ -44,6 +44,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" + "github.com/cockroachdb/cockroach/pkg/rpc/nodedialer" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql" @@ -71,6 +72,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/testutils/skip" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/util" + "github.com/cockroachdb/cockroach/pkg/util/circuit" "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/encoding/csv" "github.com/cockroachdb/cockroach/pkg/util/ioctx" @@ -2972,8 +2974,16 @@ func TestImportRetriesBreakerOpenFailure(t *testing.T) { func(raw jobs.Resumer) jobs.Resumer { r := raw.(*importResumer) r.testingKnobs.beforeRunDSP = func() error { - aboutToRunDSP <- struct{}{} - <-allowRunDSP + select { + case aboutToRunDSP <- struct{}{}: + case <-time.After(testutils.DefaultSucceedsSoonDuration): + return errors.New("timed out on aboutToRunDSP") + } + select { + case <-allowRunDSP: + case <-time.After(testutils.DefaultSucceedsSoonDuration): + return errors.New("timed out on allowRunDSP") + } return nil } return r @@ -3002,16 +3012,38 @@ func TestImportRetriesBreakerOpenFailure(t *testing.T) { // On the first attempt, we trip the node 3 breaker between distsql planning // and actually running the plan. - <-aboutToRunDSP - breaker := tc.Server(0).DistSQLServer().(*distsql.ServerImpl).PodNodeDialer.GetCircuitBreaker(roachpb.NodeID(3), rpc.DefaultClass) - breaker.Break() - allowRunDSP <- struct{}{} + select { + case <-aboutToRunDSP: + case <-time.After(testutils.DefaultSucceedsSoonDuration): + t.Fatal("timed out on aboutToRunDSP") + } + { + b, ok := tc.Server(0).NodeDialer().(*nodedialer.Dialer).GetCircuitBreakerNew(roachpb.NodeID(3), rpc.DefaultClass) + require.True(t, ok) + undo := circuit.TestingSetTripped(b, errors.New("boom")) + defer undo() + } + + timeout := testutils.DefaultSucceedsSoonDuration + select { + case allowRunDSP <- struct{}{}: + case <-time.After(timeout): + t.Fatalf("timed out on allowRunDSP attempt 1") + } // The failure above should be retried. We expect this to succeed even if we // don't reset the breaker because node 3 should no longer be included in // the plan. - <-aboutToRunDSP - allowRunDSP <- struct{}{} + select { + case <-aboutToRunDSP: + case <-time.After(timeout): + t.Fatalf("timed out on aboutToRunDSP") + } + select { + case allowRunDSP <- struct{}{}: + case <-time.After(timeout): + t.Fatalf("timed out on allowRunDSP") + } require.NoError(t, g.Wait()) } From a35f3947e00f00b767957296aae8d88a87aaef42 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Fri, 12 May 2023 08:12:01 +0200 Subject: [PATCH 11/17] nodedialer: remove use of old breaker in TestConnHealthInternal Epic: none Release note: None --- pkg/rpc/nodedialer/BUILD.bazel | 1 - pkg/rpc/nodedialer/nodedialer_test.go | 12 ++++-------- 2 files changed, 4 insertions(+), 9 deletions(-) diff --git a/pkg/rpc/nodedialer/BUILD.bazel b/pkg/rpc/nodedialer/BUILD.bazel index 9bf284a8241c..cc78101bc80c 100644 --- a/pkg/rpc/nodedialer/BUILD.bazel +++ b/pkg/rpc/nodedialer/BUILD.bazel @@ -43,7 +43,6 @@ go_test( "//pkg/util/syncutil", "//pkg/util/timeutil", "//pkg/util/uuid", - "@com_github_cockroachdb_circuitbreaker//:circuitbreaker", "@com_github_cockroachdb_errors//:errors", "@com_github_stretchr_testify//assert", "@com_github_stretchr_testify//require", diff --git a/pkg/rpc/nodedialer/nodedialer_test.go b/pkg/rpc/nodedialer/nodedialer_test.go index f9cd4c6a75ed..e97a0f80cb0e 100644 --- a/pkg/rpc/nodedialer/nodedialer_test.go +++ b/pkg/rpc/nodedialer/nodedialer_test.go @@ -18,7 +18,6 @@ import ( "testing" "time" - circuit "github.com/cockroachdb/circuitbreaker" "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -244,13 +243,10 @@ func TestConnHealthInternal(t *testing.T) { require.NoError(t, nd.ConnHealth(staticNodeID, rpc.DefaultClass)) require.NoError(t, nd.ConnHealth(staticNodeID, rpc.SystemClass)) - // However, it does respect the breaker. - br := nd.getBreaker(staticNodeID, rpc.DefaultClass) - br.Trip() - require.Equal(t, circuit.ErrBreakerOpen, nd.ConnHealth(staticNodeID, rpc.DefaultClass)) - - br.Reset() - require.NoError(t, nd.ConnHealth(staticNodeID, rpc.DefaultClass)) + // We don't have a breaker for it. This is a proxy for "we have no + // internal dialing for it". + _, ok := nd.GetCircuitBreakerNew(staticNodeID, rpc.DefaultClass) + require.False(t, ok) // Other nodes still fail though. require.Error(t, nd.ConnHealth(7, rpc.DefaultClass)) From 6c76b59cac515b7fb5cdc7e35a6dff55342741f0 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Fri, 12 May 2023 14:39:37 +0200 Subject: [PATCH 12/17] kvserver: use new breakers in RaftTransport Epic: none Release note: None --- pkg/kv/kvserver/BUILD.bazel | 1 - pkg/kv/kvserver/client_raft_test.go | 20 +++++++++++--------- pkg/kv/kvserver/helpers_test.go | 7 +++---- pkg/kv/kvserver/raft_transport.go | 2 +- pkg/kv/kvserver/raft_transport_test.go | 5 +++-- pkg/rpc/nodedialer/nodedialer.go | 6 ------ pkg/rpc/nodedialer/nodedialer_test.go | 2 +- pkg/sql/importer/import_stmt_test.go | 2 +- 8 files changed, 20 insertions(+), 25 deletions(-) diff --git a/pkg/kv/kvserver/BUILD.bazel b/pkg/kv/kvserver/BUILD.bazel index fe8136ee1aca..f65dd9a9c596 100644 --- a/pkg/kv/kvserver/BUILD.bazel +++ b/pkg/kv/kvserver/BUILD.bazel @@ -472,7 +472,6 @@ go_test( "//pkg/util/tracing", "//pkg/util/tracing/tracingpb", "//pkg/util/uuid", - "@com_github_cockroachdb_circuitbreaker//:circuitbreaker", "@com_github_cockroachdb_cockroach_go_v2//crdb", "@com_github_cockroachdb_datadriven//:datadriven", "@com_github_cockroachdb_errors//:errors", diff --git a/pkg/kv/kvserver/client_raft_test.go b/pkg/kv/kvserver/client_raft_test.go index 5f900541d328..3129ad9f436e 100644 --- a/pkg/kv/kvserver/client_raft_test.go +++ b/pkg/kv/kvserver/client_raft_test.go @@ -52,6 +52,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/testutils/storageutils" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" "github.com/cockroachdb/cockroach/pkg/util" + "github.com/cockroachdb/cockroach/pkg/util/circuit" "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" @@ -2613,9 +2614,11 @@ func TestReportUnreachableHeartbeats(t *testing.T) { // Shut down a raft transport via the circuit breaker, and wait for two // election timeouts to trigger an election if reportUnreachable broke // heartbeat transmission to the other store. - cb := tc.Servers[followerIdx].RaftTransport().GetCircuitBreaker( + b, ok := tc.Servers[followerIdx].RaftTransport().GetCircuitBreaker( tc.Target(followerIdx).NodeID, rpc.DefaultClass) - cb.Break() + require.True(t, ok) + undo := circuit.TestingSetTripped(b, errors.New("boom")) + defer undo() // Send a command to ensure Raft is aware of lost follower so that it won't // quiesce (which would prevent heartbeats). @@ -2698,10 +2701,12 @@ func TestReportUnreachableRemoveRace(t *testing.T) { // Pseudo-partition partitionedMaybeLeaseholderIdx away from everyone else. We do this by tripping // the circuit breaker on all other nodes. t.Logf("partitioning") + var undos []func() for i := range tc.Servers { if i != partitionedMaybeLeaseholderIdx { - cb := tc.Servers[i].RaftTransport().GetCircuitBreaker(tc.Target(partitionedMaybeLeaseholderIdx).NodeID, rpc.DefaultClass) - cb.Break() + b, ok := tc.Servers[i].RaftTransport().GetCircuitBreaker(tc.Target(partitionedMaybeLeaseholderIdx).NodeID, rpc.DefaultClass) + require.True(t, ok) + undos = append(undos, circuit.TestingSetTripped(b, errors.New("boom"))) } } @@ -2709,11 +2714,8 @@ func TestReportUnreachableRemoveRace(t *testing.T) { heartbeatInterval := tc.GetFirstStoreFromServer(t, partitionedMaybeLeaseholderIdx).GetStoreConfig().CoalescedHeartbeatsInterval time.Sleep(heartbeatInterval) t.Logf("resolving partition") - for i := range tc.Servers { - if i != partitionedMaybeLeaseholderIdx { - cb := tc.Servers[i].RaftTransport().GetCircuitBreaker(tc.Target(partitionedMaybeLeaseholderIdx).NodeID, rpc.DefaultClass) - cb.Reset() - } + for _, undo := range undos { + undo() } t.Logf("waiting for replicaGC of removed leader replica") diff --git a/pkg/kv/kvserver/helpers_test.go b/pkg/kv/kvserver/helpers_test.go index 074c1080aec2..2226f55c8f2b 100644 --- a/pkg/kv/kvserver/helpers_test.go +++ b/pkg/kv/kvserver/helpers_test.go @@ -21,7 +21,6 @@ import ( "testing" "time" - circuit "github.com/cockroachdb/circuitbreaker" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator" @@ -40,7 +39,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util" - circuit2 "github.com/cockroachdb/cockroach/pkg/util/circuit" + "github.com/cockroachdb/cockroach/pkg/util/circuit" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/quotapool" @@ -237,7 +236,7 @@ func (r *Replica) Store() *Store { return r.store } -func (r *Replica) Breaker() *circuit2.Breaker { +func (r *Replica) Breaker() *circuit.Breaker { return r.breaker.wrapped } @@ -498,7 +497,7 @@ func (r *Replica) TripBreaker() { // connection attempts to the specified node. func (t *RaftTransport) GetCircuitBreaker( nodeID roachpb.NodeID, class rpc.ConnectionClass, -) *circuit.Breaker { +) (*circuit.Breaker, bool) { return t.dialer.GetCircuitBreaker(nodeID, class) } diff --git a/pkg/kv/kvserver/raft_transport.go b/pkg/kv/kvserver/raft_transport.go index 2d2ef8a41e31..41d6f4cca719 100644 --- a/pkg/kv/kvserver/raft_transport.go +++ b/pkg/kv/kvserver/raft_transport.go @@ -517,7 +517,7 @@ func (t *RaftTransport) SendAsync( panic("snapshots must be sent using SendSnapshot") } - if !t.dialer.GetCircuitBreaker(toNodeID, class).Ready() { + if b, ok := t.dialer.GetCircuitBreaker(toNodeID, class); ok && b.Signal().Err() != nil { return false } diff --git a/pkg/kv/kvserver/raft_transport_test.go b/pkg/kv/kvserver/raft_transport_test.go index 46d0dc09b0a8..c478056a11ea 100644 --- a/pkg/kv/kvserver/raft_transport_test.go +++ b/pkg/kv/kvserver/raft_transport_test.go @@ -445,7 +445,8 @@ func TestRaftTransportCircuitBreaker(t *testing.T) { StoreID: 1, ReplicaID: 1, } - clientTransport := rttc.AddNode(clientReplica.NodeID) + _ = rttc.AddNode(clientReplica.NodeID) + rttc.GossipNode(serverReplica.NodeID, &util.UnresolvedAddr{NetworkField: "invalid", AddressField: "127.0.0.1:999999999"}) // Sending repeated messages should begin dropping once the circuit breaker // does trip. @@ -464,7 +465,7 @@ func TestRaftTransportCircuitBreaker(t *testing.T) { // snuck in. testutils.SucceedsSoon(t, func() error { if !rttc.Send(clientReplica, serverReplica, 1, raftpb.Message{Commit: 2}) { - clientTransport.GetCircuitBreaker(serverReplica.NodeID, rpc.DefaultClass).Reset() + return errors.New("messages still dropped") } select { case req := <-serverChannel.ch: diff --git a/pkg/rpc/nodedialer/nodedialer.go b/pkg/rpc/nodedialer/nodedialer.go index 32e2fdf5877c..1a588d851671 100644 --- a/pkg/rpc/nodedialer/nodedialer.go +++ b/pkg/rpc/nodedialer/nodedialer.go @@ -282,12 +282,6 @@ func (n *Dialer) ConnHealthTryDial(nodeID roachpb.NodeID, class rpc.ConnectionCl // dialing to that node through this NodeDialer. func (n *Dialer) GetCircuitBreaker( nodeID roachpb.NodeID, class rpc.ConnectionClass, -) *circuit.Breaker { - return n.getBreaker(nodeID, class).Breaker -} - -func (n *Dialer) GetCircuitBreakerNew( - nodeID roachpb.NodeID, class rpc.ConnectionClass, ) (*circuit2.Breaker, bool) { addr, err := n.resolver(nodeID) if err != nil { diff --git a/pkg/rpc/nodedialer/nodedialer_test.go b/pkg/rpc/nodedialer/nodedialer_test.go index e97a0f80cb0e..a7a7fa25e2ae 100644 --- a/pkg/rpc/nodedialer/nodedialer_test.go +++ b/pkg/rpc/nodedialer/nodedialer_test.go @@ -245,7 +245,7 @@ func TestConnHealthInternal(t *testing.T) { // We don't have a breaker for it. This is a proxy for "we have no // internal dialing for it". - _, ok := nd.GetCircuitBreakerNew(staticNodeID, rpc.DefaultClass) + _, ok := nd.GetCircuitBreaker(staticNodeID, rpc.DefaultClass) require.False(t, ok) // Other nodes still fail though. diff --git a/pkg/sql/importer/import_stmt_test.go b/pkg/sql/importer/import_stmt_test.go index 06b63dc86e4a..07d5a380c9e9 100644 --- a/pkg/sql/importer/import_stmt_test.go +++ b/pkg/sql/importer/import_stmt_test.go @@ -3018,7 +3018,7 @@ func TestImportRetriesBreakerOpenFailure(t *testing.T) { t.Fatal("timed out on aboutToRunDSP") } { - b, ok := tc.Server(0).NodeDialer().(*nodedialer.Dialer).GetCircuitBreakerNew(roachpb.NodeID(3), rpc.DefaultClass) + b, ok := tc.Server(0).NodeDialer().(*nodedialer.Dialer).GetCircuitBreaker(roachpb.NodeID(3), rpc.DefaultClass) require.True(t, ok) undo := circuit.TestingSetTripped(b, errors.New("boom")) defer undo() From 8a0198345e8deb9766574b93fc0d5b6a176a6ec7 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Mon, 15 May 2023 09:46:58 +0200 Subject: [PATCH 13/17] kvserver: remove outdated comment The previous breaker was traffic-recruiting, and so this deadlocks ```go if breaker.Tripped(nodeID) { return errors.New("tripped") } dial(nodeID) ``` because the dial will never be attempted. The new breakers don't have this issue because they're running a probe in the background (which is triggered by any inquiry to the breaker), so the breaker will untrip once the connection is ready to be used (at which point the dial will succeed). Epic: none Release note: None --- pkg/kv/kvserver/raft_transport.go | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/pkg/kv/kvserver/raft_transport.go b/pkg/kv/kvserver/raft_transport.go index 41d6f4cca719..1e7f5d150bbf 100644 --- a/pkg/kv/kvserver/raft_transport.go +++ b/pkg/kv/kvserver/raft_transport.go @@ -582,10 +582,7 @@ func (t *RaftTransport) startProcessNewQueue( } defer cleanup(q) defer t.queues[class].Delete(int64(toNodeID)) - // NB: we dial without a breaker here because the caller has already - // checked the breaker. Checking it again can cause livelock, see: - // https://github.com/cockroachdb/cockroach/issues/68419 - conn, err := t.dialer.DialNoBreaker(ctx, toNodeID, class) + conn, err := t.dialer.Dial(ctx, toNodeID, class) if err != nil { // DialNode already logs sufficiently, so just return. return From e4c0dc240009ad0986c8250040c55605c247c418 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Mon, 15 May 2023 09:52:38 +0200 Subject: [PATCH 14/17] nodedialer: remove uses of deprecated breaker Epic: none Release note: None --- pkg/rpc/nodedialer/nodedialer.go | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/pkg/rpc/nodedialer/nodedialer.go b/pkg/rpc/nodedialer/nodedialer.go index 1a588d851671..980af8442e39 100644 --- a/pkg/rpc/nodedialer/nodedialer.go +++ b/pkg/rpc/nodedialer/nodedialer.go @@ -234,11 +234,6 @@ func (n *Dialer) ConnHealth(nodeID roachpb.NodeID, class rpc.ConnectionClass) er if n == nil || n.resolver == nil { return errors.New("no node dialer configured") } - // NB: Don't call Ready(). The breaker protocol would require us to follow - // that up with a dial, which we won't do as this is called in hot paths. - if n.getBreaker(nodeID, class).Tripped() { - return circuit.ErrBreakerOpen - } addr, err := n.resolver(nodeID) if err != nil { return err @@ -265,7 +260,7 @@ func (n *Dialer) ConnHealth(nodeID roachpb.NodeID, class rpc.ConnectionClass) er // is the caller really wants can likely be achieved by more direct means. func (n *Dialer) ConnHealthTryDial(nodeID roachpb.NodeID, class rpc.ConnectionClass) error { err := n.ConnHealth(nodeID, class) - if err == nil || !n.getBreaker(nodeID, class).Ready() { + if err == nil { return err } addr, err := n.resolver(nodeID) From ca910d950691a99745c3ea17e7763255d3e9f4d0 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Mon, 15 May 2023 10:15:48 +0200 Subject: [PATCH 15/17] gossip: don't use deprecated breaker Epic: none Release note: None --- pkg/gossip/BUILD.bazel | 1 - pkg/gossip/client.go | 30 +++++++++++++++--------------- pkg/gossip/client_test.go | 4 ++-- pkg/gossip/gossip.go | 12 +----------- pkg/gossip/gossip_test.go | 2 +- 5 files changed, 19 insertions(+), 30 deletions(-) diff --git a/pkg/gossip/BUILD.bazel b/pkg/gossip/BUILD.bazel index 4d06c27969b6..cb6d13e4d9d6 100644 --- a/pkg/gossip/BUILD.bazel +++ b/pkg/gossip/BUILD.bazel @@ -39,7 +39,6 @@ go_library( "//pkg/util/syncutil", "//pkg/util/timeutil", "//pkg/util/uuid", - "@com_github_cockroachdb_circuitbreaker//:circuitbreaker", "@com_github_cockroachdb_errors//:errors", "@com_github_cockroachdb_logtags//:logtags", "@com_github_cockroachdb_redact//:redact", diff --git a/pkg/gossip/client.go b/pkg/gossip/client.go index 23f376acfdc6..173ac2cdc473 100644 --- a/pkg/gossip/client.go +++ b/pkg/gossip/client.go @@ -17,7 +17,6 @@ import ( "sync" "time" - circuit "github.com/cockroachdb/circuitbreaker" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/util" @@ -65,15 +64,13 @@ func newClient(ambient log.AmbientContext, addr net.Addr, nodeMetrics Metrics) * } } +var logFailedStartEvery = log.Every(5 * time.Second) + // start dials the remote addr and commences gossip once connected. Upon exit, // the client is sent on the disconnected channel. This method starts client // processing in a goroutine and returns immediately. func (c *client) startLocked( - g *Gossip, - disconnected chan *client, - rpcCtx *rpc.Context, - stopper *stop.Stopper, - breaker *circuit.Breaker, + g *Gossip, disconnected chan *client, rpcCtx *rpc.Context, stopper *stop.Stopper, ) { // Add a placeholder for the new outgoing connection because we may not know // the ID of the node we're connecting to yet. This will be resolved in @@ -98,23 +95,26 @@ func (c *client) startLocked( disconnected <- c }() - consecFailures := breaker.ConsecFailures() - var stream Gossip_GossipClient - if err := breaker.Call(func() error { + stream, err := func() (Gossip_GossipClient, error) { // Note: avoid using `grpc.WithBlock` here. This code is already // asynchronous from the caller's perspective, so the only effect of // `WithBlock` here is blocking shutdown - at the time of this writing, // that ends ups up making `kv` tests take twice as long. conn, err := rpcCtx.GRPCUnvalidatedDial(c.addr.String()).Connect(ctx) if err != nil { - return err + return nil, err } - if stream, err = NewGossipClient(conn).Gossip(ctx); err != nil { - return err + stream, err := NewGossipClient(conn).Gossip(ctx) + if err != nil { + return nil, err + } + if err := c.requestGossip(g, stream); err != nil { + return nil, err } - return c.requestGossip(g, stream) - }, 0); err != nil { - if consecFailures == 0 { + return stream, nil + }() + if err != nil { + if logFailedStartEvery.ShouldLog() { log.Warningf(ctx, "failed to start gossip client to %s: %s", c.addr, err) } return diff --git a/pkg/gossip/client_test.go b/pkg/gossip/client_test.go index 6f696a4a259f..0c01ef7208db 100644 --- a/pkg/gossip/client_test.go +++ b/pkg/gossip/client_test.go @@ -162,7 +162,7 @@ func gossipSucceedsSoon( // If the client wasn't able to connect, restart it. g := gossip[client] g.mu.Lock() - client.startLocked(g, disconnected, rpcContext, stopper, rpcContext.NewBreaker("")) + client.startLocked(g, disconnected, rpcContext, stopper) g.mu.Unlock() default: } @@ -319,7 +319,7 @@ func TestClientNodeID(t *testing.T) { case <-disconnected: // The client hasn't been started or failed to start, loop and try again. local.mu.Lock() - c.startLocked(local, disconnected, rpcContext, stopper, rpcContext.NewBreaker("")) + c.startLocked(local, disconnected, rpcContext, stopper) local.mu.Unlock() } } diff --git a/pkg/gossip/gossip.go b/pkg/gossip/gossip.go index dbbf1ba09489..a59db6f1e643 100644 --- a/pkg/gossip/gossip.go +++ b/pkg/gossip/gossip.go @@ -57,7 +57,6 @@ import ( "time" "unsafe" - circuit "github.com/cockroachdb/circuitbreaker" "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/config/zonepb" @@ -233,8 +232,6 @@ type Gossip struct { clientsMu struct { syncutil.Mutex clients []*client - // One breaker per client for the life of the process. - breakers map[string]*circuit.Breaker } disconnected chan *client // Channel of disconnected clients @@ -307,7 +304,6 @@ func New( stopper.AddCloser(stop.CloserFn(g.server.AmbientContext.FinishEventLog)) registry.AddMetric(g.outgoing.gauge) - g.clientsMu.breakers = map[string]*circuit.Breaker{} g.mu.Lock() // Add ourselves as a SystemConfig watcher. @@ -1456,17 +1452,11 @@ func (g *Gossip) signalConnectedLocked() { func (g *Gossip) startClientLocked(addr util.UnresolvedAddr, rpcContext *rpc.Context) { g.clientsMu.Lock() defer g.clientsMu.Unlock() - breaker, ok := g.clientsMu.breakers[addr.String()] - if !ok { - name := fmt.Sprintf("gossip %v->%v", rpcContext.Config.Addr, addr) - breaker = rpcContext.NewBreaker(name) - g.clientsMu.breakers[addr.String()] = breaker - } ctx := g.AnnotateCtx(context.TODO()) log.VEventf(ctx, 1, "starting new client to %s", addr) c := newClient(g.server.AmbientContext, &addr, g.serverMetrics) g.clientsMu.clients = append(g.clientsMu.clients, c) - c.startLocked(g, g.disconnected, rpcContext, g.server.stopper, breaker) + c.startLocked(g, g.disconnected, rpcContext, g.server.stopper) } // removeClientLocked removes the specified client. Called when a client diff --git a/pkg/gossip/gossip_test.go b/pkg/gossip/gossip_test.go index 846ac786ea1c..4ee8fd47282d 100644 --- a/pkg/gossip/gossip_test.go +++ b/pkg/gossip/gossip_test.go @@ -521,7 +521,7 @@ func TestGossipNoForwardSelf(t *testing.T) { localAddr := local.GetNodeAddr() c := newClient(log.MakeTestingAmbientCtxWithNewTracer(), localAddr, makeMetrics()) peer.mu.Lock() - c.startLocked(peer, disconnectedCh, peerCtx, stopper, peerCtx.NewBreaker("")) + c.startLocked(peer, disconnectedCh, peerCtx, stopper) peer.mu.Unlock() disconnectedClient := <-disconnectedCh From edf0e7072cfd655df6cffa4c3db9f17d13dee897 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Mon, 15 May 2023 10:20:56 +0200 Subject: [PATCH 16/17] nodedialer,rpc: remove deprecated breaker Farewell, old friend. Epic: none Release note: None --- go.mod | 2 +- pkg/rpc/BUILD.bazel | 3 - pkg/rpc/breaker.go | 125 ------------------------------- pkg/rpc/context.go | 8 +- pkg/rpc/nodedialer/BUILD.bazel | 2 - pkg/rpc/nodedialer/nodedialer.go | 64 ++-------------- 6 files changed, 9 insertions(+), 195 deletions(-) delete mode 100644 pkg/rpc/breaker.go diff --git a/go.mod b/go.mod index 2d0b73df4766..42cc863ca273 100644 --- a/go.mod +++ b/go.mod @@ -134,7 +134,6 @@ require ( github.com/edsrzf/mmap-go v1.0.0 github.com/elastic/gosigar v0.14.1 github.com/emicklei/dot v0.15.0 - github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a github.com/fatih/color v1.9.0 github.com/fraugster/parquet-go v0.10.0 github.com/fsnotify/fsnotify v1.5.1 @@ -278,6 +277,7 @@ require ( github.com/eapache/go-resiliency v1.3.0 // indirect github.com/eapache/go-xerial-snappy v0.0.0-20230111030713-bf00bc1b83b6 // indirect github.com/eapache/queue v1.1.0 // indirect + github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a // indirect github.com/fatih/structs v1.1.0 // indirect github.com/felixge/httpsnoop v1.0.1 // indirect github.com/ghodss/yaml v1.0.0 // indirect diff --git a/pkg/rpc/BUILD.bazel b/pkg/rpc/BUILD.bazel index af221e8f25fc..cd8f6d98e92f 100644 --- a/pkg/rpc/BUILD.bazel +++ b/pkg/rpc/BUILD.bazel @@ -9,7 +9,6 @@ go_library( "addjoin.go", "auth.go", "auth_tenant.go", - "breaker.go", "clock_offset.go", "codec.go", "connection.go", @@ -61,12 +60,10 @@ go_library( "//pkg/util/tracing", "//pkg/util/tracing/grpcinterceptor", "//pkg/util/uuid", - "@com_github_cenkalti_backoff//:backoff", "@com_github_cockroachdb_circuitbreaker//:circuitbreaker", "@com_github_cockroachdb_errors//:errors", "@com_github_cockroachdb_logtags//:logtags", "@com_github_cockroachdb_redact//:redact", - "@com_github_facebookgo_clock//:clock", "@com_github_gogo_protobuf//proto", "@com_github_golang_protobuf//proto:go_default_library", "@com_github_golang_snappy//:snappy", diff --git a/pkg/rpc/breaker.go b/pkg/rpc/breaker.go deleted file mode 100644 index f0ae564d1d04..000000000000 --- a/pkg/rpc/breaker.go +++ /dev/null @@ -1,125 +0,0 @@ -// Copyright 2016 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 rpc - -import ( - "context" - "time" - - "github.com/cenkalti/backoff" - circuit "github.com/cockroachdb/circuitbreaker" - "github.com/cockroachdb/cockroach/pkg/util/hlc" - "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/facebookgo/clock" -) - -const rpcBreakerMaxBackoff = time.Second - -// breakerClock is an implementation of clock.Clock that internally uses an -// hlc.WallClock. It is used to adapt the WallClock to the circuit breaker -// clocks. Note that it only implements the After() and Now() methods needed by -// circuit breakers and backoffs. -type breakerClock struct { - clock hlc.WallClock -} - -var _ clock.Clock = &breakerClock{} - -func (c *breakerClock) After(d time.Duration) <-chan time.Time { - // TODO(andrei): This is broken, in that the returned timer has nothing to do - // with c.clock. Fix this once hlc.HybridManualClock implements - // timeutil.TimeSource, which will allow the breakerClock to wrap a - // timeutil.TimeSource, and not a - return time.After(d) -} - -func (c *breakerClock) AfterFunc(d time.Duration, f func()) *clock.Timer { - panic("unimplemented") -} - -func (c *breakerClock) Now() time.Time { - return c.clock.Now() -} - -func (c *breakerClock) Sleep(d time.Duration) { - panic("unimplemented") -} - -func (c *breakerClock) Tick(d time.Duration) <-chan time.Time { - panic("unimplemented") -} - -func (c *breakerClock) Ticker(d time.Duration) *clock.Ticker { - panic("unimplemented") -} - -func (c *breakerClock) Timer(d time.Duration) *clock.Timer { - panic("unimplemented") -} - -// newBackOff creates a new exponential backoff properly configured for RPC -// connection backoff. -func newBackOff(clock backoff.Clock) backoff.BackOff { - // This exponential backoff limits the circuit breaker to 1 second - // intervals between successive attempts to resolve a node address - // and connect via GRPC. - // - // NB (nota Ben): MaxInterval should be less than the Raft election timeout - // (1.5s) to avoid disruptions. A newly restarted node will be in follower - // mode with no knowledge of the Raft leader. If it doesn't hear from a - // leader before the election timeout expires, it will start to campaign, - // which can be disruptive. Therefore the leader needs to get in touch (via - // Raft heartbeats) with such nodes within one election timeout of their - // restart, which won't happen if their backoff is too high. - b := &backoff.ExponentialBackOff{ - InitialInterval: 500 * time.Millisecond, - RandomizationFactor: 0.5, - Multiplier: 1.5, - MaxInterval: rpcBreakerMaxBackoff, - MaxElapsedTime: 0, - Clock: clock, - } - b.Reset() - return b -} - -func newBreaker(ctx context.Context, name string, clock clock.Clock, noop bool) *circuit.Breaker { - shouldTrip := circuit.ThresholdTripFunc(1) - if noop { - shouldTrip = func(*circuit.Breaker) bool { - return false - } - } - return circuit.NewBreakerWithOptions(&circuit.Options{ - Name: name, - BackOff: newBackOff(clock), - Clock: clock, - ShouldTrip: shouldTrip, - Logger: breakerLogger{ctx}, - }) -} - -// breakerLogger implements circuit.Logger to expose logging from the -// circuitbreaker package. Debugf is logged with a vmodule level of 2 so to see -// the circuitbreaker debug messages set --vmodule=breaker=2 -type breakerLogger struct { - ctx context.Context -} - -func (r breakerLogger) Debugf(format string, v ...interface{}) { - if log.V(2) { - log.Dev.InfofDepth(r.ctx, 1, format, v...) - } -} - -func (r breakerLogger) Infof(format string, v ...interface{}) { - log.Ops.InfofDepth(r.ctx, 1, format, v...) -} diff --git a/pkg/rpc/context.go b/pkg/rpc/context.go index 638d11c13c24..fb3d30d9309a 100644 --- a/pkg/rpc/context.go +++ b/pkg/rpc/context.go @@ -207,7 +207,6 @@ type Context struct { ContextOptions *SecurityContext - breakerClock breakerClock RemoteClocks *RemoteClockMonitor MasterCtx context.Context // cancel on stopper quiesce @@ -462,11 +461,8 @@ func NewContext(ctx context.Context, opts ContextOptions) *Context { secCtx.useNodeAuth = opts.UseNodeAuth rpcCtx := &Context{ - ContextOptions: opts, - SecurityContext: secCtx, - breakerClock: breakerClock{ - clock: opts.Clock, - }, + ContextOptions: opts, + SecurityContext: secCtx, rpcCompression: enableRPCCompression, MasterCtx: masterCtx, metrics: makeMetrics(), diff --git a/pkg/rpc/nodedialer/BUILD.bazel b/pkg/rpc/nodedialer/BUILD.bazel index cc78101bc80c..733d4891f69a 100644 --- a/pkg/rpc/nodedialer/BUILD.bazel +++ b/pkg/rpc/nodedialer/BUILD.bazel @@ -14,9 +14,7 @@ go_library( "//pkg/util/circuit", "//pkg/util/log", "//pkg/util/stop", - "//pkg/util/syncutil", "//pkg/util/tracing", - "@com_github_cockroachdb_circuitbreaker//:circuitbreaker", "@com_github_cockroachdb_errors//:errors", "@org_golang_google_grpc//:go_default_library", ], diff --git a/pkg/rpc/nodedialer/nodedialer.go b/pkg/rpc/nodedialer/nodedialer.go index 980af8442e39..bc7899352bfa 100644 --- a/pkg/rpc/nodedialer/nodedialer.go +++ b/pkg/rpc/nodedialer/nodedialer.go @@ -12,12 +12,9 @@ package nodedialer import ( "context" - "fmt" "net" "time" - "unsafe" - circuit "github.com/cockroachdb/circuitbreaker" "github.com/cockroachdb/cockroach/pkg/kv/kvbase" "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -25,20 +22,11 @@ import ( circuit2 "github.com/cockroachdb/cockroach/pkg/util/circuit" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/stop" - "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/errors" "google.golang.org/grpc" ) -// No more than one failure to connect to a given node will be logged in the given interval. -const logPerNodeFailInterval = time.Minute - -type wrappedBreaker struct { - *circuit.Breaker - log.EveryN -} - // An AddressResolver translates NodeIDs into addresses. type AddressResolver func(roachpb.NodeID) (net.Addr, error) @@ -49,8 +37,6 @@ type Dialer struct { rpcContext *rpc.Context resolver AddressResolver testingKnobs DialerTestingKnobs - - breakers [rpc.NumConnectionClasses]syncutil.IntMap // map[roachpb.NodeID]*wrappedBreaker } // DialerOpt contains configuration options for a Dialer. @@ -109,14 +95,12 @@ func (n *Dialer) Dial( if ctxErr := ctx.Err(); ctxErr != nil { return nil, errors.Wrap(ctxErr, "dial") } - breaker := n.getBreaker(nodeID, class) addr, err := n.resolver(nodeID) if err != nil { err = errors.Wrapf(err, "failed to resolve n%d", nodeID) - breaker.Fail(err) return nil, err } - return n.dial(ctx, nodeID, addr, breaker, true /* checkBreaker */, class) + return n.dial(ctx, nodeID, addr, true, class) } // DialNoBreaker is like Dial, but will not check the circuit breaker before @@ -130,12 +114,9 @@ func (n *Dialer) DialNoBreaker( } addr, err := n.resolver(nodeID) if err != nil { - if ctx.Err() == nil { - n.getBreaker(nodeID, class).Fail(err) - } return nil, err } - return n.dial(ctx, nodeID, addr, n.getBreaker(nodeID, class), false /* checkBreaker */, class) + return n.dial(ctx, nodeID, addr, false, class) } // DialInternalClient is a specialization of DialClass for callers that @@ -164,20 +145,20 @@ func (n *Dialer) DialInternalClient( return nil, errors.Wrap(err, "resolver error") } log.VEventf(ctx, 2, "sending request to %s", addr) - conn, err := n.dial(ctx, nodeID, addr, n.getBreaker(nodeID, class), true /* checkBreaker */, class) + conn, err := n.dial(ctx, nodeID, addr, true, class) if err != nil { return nil, err } return TracingInternalClient{InternalClient: kvpb.NewInternalClient(conn)}, nil } -// dial performs the dialing of the remote connection. If breaker is nil, -// then perform this logic without using any breaker functionality. +// dial performs the dialing of the remote connection. If checkBreaker +// is set (which it usually is), circuit breakers for the peer will be +// checked. func (n *Dialer) dial( ctx context.Context, nodeID roachpb.NodeID, addr net.Addr, - breaker *wrappedBreaker, checkBreaker bool, class rpc.ConnectionClass, ) (_ *grpc.ClientConn, err error) { @@ -186,16 +167,6 @@ func (n *Dialer) dial( if ctxErr := ctx.Err(); ctxErr != nil { return nil, errors.Wrap(ctxErr, ctxWrapMsg) } - if checkBreaker && !breaker.Ready() { - err = errors.Wrapf(circuit.ErrBreakerOpen, "unable to dial n%d", nodeID) - return nil, err - } - defer func() { - // Enforce a minimum interval between warnings for failed connections. - if err != nil && ctx.Err() == nil && breaker != nil && breaker.ShouldLog() { - log.Health.Warningf(ctx, "unable to connect to n%d: %s", nodeID, err) - } - }() rpcConn := n.rpcContext.GRPCDialNode(addr.String(), nodeID, class) connect := rpcConn.Connect if !checkBreaker { @@ -208,21 +179,9 @@ func (n *Dialer) dial( return nil, errors.Wrap(ctxErr, ctxWrapMsg) } err = errors.Wrapf(err, "failed to connect to n%d at %v", nodeID, addr) - if breaker != nil { - breaker.Fail(err) - } return nil, err } - // TODO(bdarnell): Reconcile the different health checks and circuit breaker - // behavior in this file. Note that this different behavior causes problems - // for higher-levels in the system. For example, DistSQL checks for - // ConnHealth when scheduling processors, but can then see attempts to send - // RPCs fail when dial fails due to an open breaker. Reset the breaker here - // as a stop-gap before the reconciliation occurs. - if breaker != nil { - breaker.Success() - } return conn, nil } @@ -285,17 +244,6 @@ func (n *Dialer) GetCircuitBreaker( return n.rpcContext.GetBreakerForAddr(nodeID, class, addr) } -func (n *Dialer) getBreaker(nodeID roachpb.NodeID, class rpc.ConnectionClass) *wrappedBreaker { - breakers := &n.breakers[class] - value, ok := breakers.Load(int64(nodeID)) - if !ok { - name := fmt.Sprintf("rpc %v [n%d]", n.rpcContext.Config.Addr, nodeID) - breaker := &wrappedBreaker{Breaker: n.rpcContext.NewBreaker(name), EveryN: log.Every(logPerNodeFailInterval)} - value, _ = breakers.LoadOrStore(int64(nodeID), unsafe.Pointer(breaker)) - } - return (*wrappedBreaker)(value) -} - // Latency returns the exponentially weighted moving average latency to the // given node ID. Returns a latency of 0 with no error if we don't have enough // samples to compute a reliable average. From f210860a40a9530dde3da0e62619fb4dc610e16a Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Tue, 6 Jun 2023 12:58:18 +0200 Subject: [PATCH 17/17] rpc: remove TestReconnectAfterAddressChange There is now stronger coverage in the datadriven tests, see `TestReconnection`. Epic: none Release note: None --- pkg/rpc/context_test.go | 220 ---------------------------------------- 1 file changed, 220 deletions(-) diff --git a/pkg/rpc/context_test.go b/pkg/rpc/context_test.go index 962ab342656d..f35ddc5742e3 100644 --- a/pkg/rpc/context_test.go +++ b/pkg/rpc/context_test.go @@ -113,226 +113,6 @@ func newTestContext( }) } -// TestReconnectAfterAddressChange verifies that when a peer restarts under a -// different IP, a connection can be established and removes the previous -// connection from the Context. -// -// TODO(tbg): try to extend TestDataDriven to cover these scenarios; -// they are pretty cumbersome to test as is. -func TestReconnectAfterAddressChange(t *testing.T) { - defer leaktest.AfterTest(t)() - - ctx := context.Background() - stopper := stop.NewStopper() - defer stopper.Stop(ctx) - - // Shared cluster ID by all RPC peers (this ensures that the peers - // don't talk to servers from unrelated tests by accident). - clusterID := uuid.MakeV4() - - clock := timeutil.DefaultTimeSource{} - maxOffset := time.Duration(250) - serverCtx := newTestContext(clusterID, clock, maxOffset, stopper) - - const serverNodeID = 1 - serverCtx.NodeID.Set(context.Background(), serverNodeID) - s1 := newTestServer(t, serverCtx) - s2 := newTestServer(t, serverCtx) - - heartbeat := &ManualHeartbeatService{ - ready: make(chan error), - stopper: stopper, - clock: clock, - maxOffset: maxOffset, - remoteClockMonitor: serverCtx.RemoteClocks, - version: serverCtx.Settings.Version, - nodeID: serverCtx.NodeID, - } - close(heartbeat.ready) - RegisterHeartbeatServer(s1, heartbeat) - RegisterHeartbeatServer(s2, heartbeat) - - clientCtx := newTestContext(clusterID, clock, maxOffset, stopper) - clientCtx.heartbeatInterval = 10 * time.Millisecond - - tmpStopper := stop.NewStopper() - defer tmpStopper.Stop(ctx) - ln1, err := netutil.ListenAndServeGRPC(tmpStopper, s1, util.TestAddr) - require.NoError(t, err) - - // NB: we need to create this listener while ln1 is also open, or ln2 might - // get the same listen address previously used by ln1, which means that the - // circuit breaker is shared between both targets and the test gets confused. - ln2, err := netutil.ListenAndServeGRPC(serverCtx.Stopper, s2, util.TestAddr) - require.NoError(t, err) - - k1 := peerKey{ - TargetAddr: ln1.Addr().String(), - NodeID: serverNodeID, - Class: DefaultClass, - } - k1U := peerKey{ - TargetAddr: k1.TargetAddr, - NodeID: 0, - Class: SystemClass, - } - - { - conn := clientCtx.GRPCDialNode(k1.TargetAddr, serverNodeID, DefaultClass) - _, err := conn.Connect(context.Background()) - require.NoError(t, err) - testutils.SucceedsSoon(t, func() error { - if err := conn.Health(); err != nil { - return err - } - _, m, _, ok := clientCtx.peers.getWithBreaker(k1) - require.True(t, ok) - am := clientCtx.Metrics() - if err := checkPeerMetrics(m, true, false, true); err != nil { - return err - } - return checkMetrics(am, 1, 0, 0, true) - }) - - // Also make an unvalidated conn because they're tricky because - // we don't have the privilege of knowing when a target address - // is "decommissioned" (which is usually the event that triggers - // soft-deletion from the rpc.Context). - connUnvalidated := clientCtx.GRPCUnvalidatedDial(k1.TargetAddr) - _, err = conn.Connect(context.Background()) - require.NoError(t, err) - testutils.SucceedsSoon(t, func() error { - if err := connUnvalidated.Health(); err != nil { - return err - } - - _, m, _, ok := clientCtx.peers.getWithBreaker(k1U) - require.True(t, ok) - am := clientCtx.Metrics() - if err := checkPeerMetrics(m, true, false, true); err != nil { - return err - } - return checkMetrics(am, 2, 0, 0, true) - }) - - } - - { - tmpStopper.Stop(ctx) - // Peer stays in map despite becoming unhealthy (due to listener closing). - testutils.SucceedsSoon(t, func() error { - _, err := clientCtx.GRPCDialNode(k1.TargetAddr, serverNodeID, DefaultClass).Connect(ctx) - if err == nil { - return errors.New("waiting for error") - } - - _, err = clientCtx.GRPCUnvalidatedDial(k1.TargetAddr).Connect(ctx) - if err == nil { - return errors.New("waiting for error (unvalidated dial)") - } - - { - _, m, _, ok := clientCtx.peers.getWithBreaker(k1) - require.True(t, ok) - if err := checkPeerMetrics(m, false, true, true); err != nil { - return errors.Wrap(err, "k1") - } - } - - // The unvalidated peer becomes on-demand only (deleteAfter set) because it sees an unhealthy - // sibling (k1). - { - p, m, _, ok := clientCtx.peers.getWithBreaker(k1U) - require.True(t, ok) - if p.deleteAfter == 0 { - return errors.New("zero deleteAfter") - } - if err := checkPeerMetrics(m, false, false, true); err != nil { - return errors.Wrap(err, "k1U") - } - } - - return checkMetrics(clientCtx.Metrics(), 0, 1, 1, true) - }) - } - - // Now connect to n1 but now listening on ln2. This should work. - t.Logf("reconnecting at %s", ln2.Addr()) - k2 := peerKey{ - TargetAddr: ln2.Addr().String(), - NodeID: serverNodeID, - Class: DefaultClass, - } - conn := clientCtx.GRPCDialNode(k2.TargetAddr, serverNodeID, DefaultClass) - _, err = conn.Connect(context.Background()) - require.NoError(t, err) - testutils.SucceedsSoon(t, conn.Health) - // Peer should be in map (with ln2's address), and the previous peer (with - // ln1's address) realized that it has been replaced. - _, _, _, ok := clientCtx.peers.getWithBreaker(k2) - require.True(t, ok) - var failedConnCount int64 - testutils.SucceedsSoon(t, func() error { - { - p, m, _, ok := clientCtx.peers.getWithBreaker(k1) - require.True(t, ok) - if p.deleteAfter == 0 { - return errors.New("deleteAfter still nil") - } - // Peer should count neither as healthy nor as unhealthy: it has empty metrics. - // We're mostly pretending it's not there. - if err := checkPeerMetrics(m, false, false, true); err != nil { - return err - } - - // Verify that we're still trying to reconnect on demand, i.e. connection - // failure count goes up as we redial. Note that GRPCDialNode itself doesn't - // lead to an increment, instead, it makes sure the probe makes an attempt - // but this is async. - _, err := clientCtx.GRPCDialNode(k1.TargetAddr, k1.NodeID, k1.Class).Connect(ctx) - require.Error(t, err) - - if failedConnCount == 0 { - failedConnCount = m.ConnectionFailures.Value() - } else if n := m.ConnectionFailures.Value(); n <= failedConnCount { - return errors.Errorf("waiting for ConnectionFailures to increment from %d", n) - } - } - - // k1U remains in the state from before, with deleteAfter set. - p, m, _, ok := clientCtx.peers.getWithBreaker(k1U) - require.True(t, ok) - if p.deleteAfter == 0 { - return errors.New("deleteAfter still nil") - } - if err := checkPeerMetrics(m, false, false, true); err != nil { - return err - } - - // The two connections to ln1 are "inactive". Only the connection to ln2 is - // healthy. - return checkMetrics(clientCtx.Metrics(), 1, 0, 2, true) - }) - - // Disable the circuit breaker setting and watch the map clear out. Because p1 and p1U - // have deleteAfter set, we need to access the breaker to make sure the probe runs at - // least once (to perform the deletion). - // (In real code, this would happen if a client calls .Connect). - enableRPCCircuitBreakers.Override(context.Background(), &clientCtx.Settings.SV, false) - testutils.SucceedsSoon(t, func() error { - if p1, _, b, ok := clientCtx.peers.getWithBreaker(k1); ok { - _ = b.Signal().Err() // make sure breaker is running - return errors.Errorf("p1 still there: %+v", p1) - } - if p1U, _, b, ok := clientCtx.peers.getWithBreaker(k1U); ok { - _ = b.Signal().Err() // make sure breaker is running - return errors.Errorf("p1U still there: %+v", p1U) - } - return nil - }) - require.NoError(t, checkMetrics(clientCtx.Metrics(), 1, 0, 0, true)) -} - // TestPingInterceptors checks that OnOutgoingPing and OnIncomingPing can inject errors. func TestPingInterceptors(t *testing.T) { defer leaktest.AfterTest(t)()