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/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 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/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..1e7f5d150bbf 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 } @@ -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 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/BUILD.bazel b/pkg/rpc/BUILD.bazel index 4f0bb0b77ea4..cd8f6d98e92f 100644 --- a/pkg/rpc/BUILD.bazel +++ b/pkg/rpc/BUILD.bazel @@ -9,16 +9,20 @@ go_library( "addjoin.go", "auth.go", "auth_tenant.go", - "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 +43,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 +52,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", @@ -55,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", @@ -89,14 +92,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 +121,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 +137,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 deleted file mode 100644 index fde6394b0064..000000000000 --- a/pkg/rpc/breaker.go +++ /dev/null @@ -1,119 +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) *circuit.Breaker { - return circuit.NewBreakerWithOptions(&circuit.Options{ - Name: name, - BackOff: newBackOff(clock), - Clock: clock, - ShouldTrip: circuit.ThresholdTripFunc(1), - 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/clock_offset.go b/pkg/rpc/clock_offset.go index 7fcdc5ad6714..66f0cb4ec53b 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 @@ -376,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..fb3d30d9309a 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. @@ -378,9 +207,8 @@ type Context struct { ContextOptions *SecurityContext - breakerClock breakerClock RemoteClocks *RemoteClockMonitor - MasterCtx context.Context + MasterCtx context.Context // cancel on stopper quiesce heartbeatInterval time.Duration heartbeatTimeout time.Duration @@ -389,7 +217,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 +239,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 +284,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 { @@ -660,17 +461,13 @@ func NewContext(ctx context.Context, opts ContextOptions) *Context { secCtx.useNodeAuth = opts.UseNodeAuth rpcCtx := &Context{ - ContextOptions: opts, - SecurityContext: secCtx, - breakerClock: breakerClock{ - clock: opts.Clock, - }, - rpcCompression: enableRPCCompression, - MasterCtx: masterCtx, - metrics: makeMetrics(), - heartbeatInterval: opts.Config.RPCHeartbeatInterval, - heartbeatTimeout: opts.Config.RPCHeartbeatTimeout, - logClosingConnEvery: log.Every(time.Second), + ContextOptions: opts, + SecurityContext: secCtx, + rpcCompression: enableRPCCompression, + MasterCtx: masterCtx, + metrics: makeMetrics(), + heartbeatInterval: opts.Config.RPCHeartbeatInterval, + heartbeatTimeout: opts.Config.RPCHeartbeatTimeout, } rpcCtx.dialbackMu.Lock() @@ -1624,10 +1421,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 +1503,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 +2057,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 -} + // Slow path. Race to create a peer. + conns := &rpcCtx.peers -// 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) -} + conns.mu.Lock() + defer conns.mu.Unlock() -// ErrNotHeartbeated is returned by ConnHealth when we have not yet performed -// the first heartbeat. -var ErrNotHeartbeated = errors.New("not yet heartbeated") - -// 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 +2133,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..f35ddc5742e3 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" @@ -165,10 +166,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 +244,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 +800,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 +811,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) - } - - 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) + clientCtx.NodeID.Set(context.Background(), nodeID) + loopLn := netutil.NewLoopbackListener(ctx, stopper) + clientCtx.loopbackDialFn = func(ctx context.Context) (net.Conn, error) { + return loopLn.Connect(ctx) } - // 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 +959,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 +1032,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 +1082,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 +1532,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 +2099,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 +2176,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 +2203,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 +2225,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 +2246,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 +2262,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 +2271,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 +2292,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..733d4891f69a 100644 --- a/pkg/rpc/nodedialer/BUILD.bazel +++ b/pkg/rpc/nodedialer/BUILD.bazel @@ -11,11 +11,10 @@ go_library( "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/rpc", + "//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", ], @@ -34,6 +33,7 @@ go_test( "//pkg/rpc", "//pkg/settings/cluster", "//pkg/testutils", + "//pkg/util/circuit", "//pkg/util/hlc", "//pkg/util/leaktest", "//pkg/util/log", @@ -41,11 +41,11 @@ 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", "@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..bc7899352bfa 100644 --- a/pkg/rpc/nodedialer/nodedialer.go +++ b/pkg/rpc/nodedialer/nodedialer.go @@ -12,32 +12,21 @@ 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" "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" "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) @@ -48,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. @@ -108,20 +95,17 @@ 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 -// 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) { @@ -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,38 +167,21 @@ 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 + rpcConn := n.rpcContext.GRPCDialNode(addr.String(), nodeID, class) + connect := rpcConn.Connect + if !checkBreaker { + connect = rpcConn.ConnectNoBreaker } - 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) - } - }() - conn, err := n.rpcContext.GRPCDialNode(addr.String(), nodeID, class).Connect(ctx) + 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 { 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 } @@ -229,11 +193,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 @@ -255,10 +214,12 @@ 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() { + if err == nil { return err } addr, err := n.resolver(nodeID) @@ -275,19 +236,12 @@ 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) 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)) +) (*circuit2.Breaker, bool) { + addr, err := n.resolver(nodeID) + if err != nil { + return nil, false } - return (*wrappedBreaker)(value) + return n.rpcContext.GetBreakerForAddr(nodeID, class, addr) } // Latency returns the exponentially weighted moving average latency to the diff --git a/pkg/rpc/nodedialer/nodedialer_test.go b/pkg/rpc/nodedialer/nodedialer_test.go index fd1501c9506c..a7a7fa25e2ae 100644 --- a/pkg/rpc/nodedialer/nodedialer_test.go +++ b/pkg/rpc/nodedialer/nodedialer_test.go @@ -13,19 +13,18 @@ package nodedialer import ( "context" "fmt" - "math/rand" "net" - "sync" + "sync/atomic" "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" "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" @@ -37,6 +36,7 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "google.golang.org/grpc" + "google.golang.org/grpc/codes" ) const staticNodeID = 1 @@ -46,13 +46,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) { @@ -75,51 +71,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()) + + require.NoError(t, rpcCtx.ConnHealth(ln.Addr().String(), staticNodeID, rpc.DefaultClass)) // 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) - assert.Nil(t, err, "failed to dial") - assert.True(t, breaker.Ready()) - assert.Equal(t, breaker.Failures(), int64(0)) - - // 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()) - _, 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) { @@ -135,8 +115,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 +138,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")) @@ -174,17 +165,9 @@ 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) require.Eventually(t, func() bool { return nd.ConnHealth(staticNodeID, rpc.DefaultClass) != nil }, time.Second, 10*time.Millisecond) @@ -204,7 +187,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. @@ -227,16 +210,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 { @@ -270,140 +243,15 @@ 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.GetCircuitBreaker(staticNodeID, rpc.DefaultClass) + require.False(t, ok) // Other nodes still fail though. 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) - 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, ) ( @@ -429,11 +277,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) { 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/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..07d5a380c9e9 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).GetCircuitBreaker(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()) } 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, 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, 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)