From 326c0ecfbe9d5e1cbed6fadeeb30c664875d501e Mon Sep 17 00:00:00 2001 From: Matt Keeler Date: Fri, 12 Jan 2024 11:54:07 -0500 Subject: [PATCH] In-Memory gRPC (#19942) * Implement In-Process gRPC for use by controller caching/indexing This replaces the pipe base listener implementation we were previously using. The new style CAN avoid cloning resources which our controller caching/indexing is taking advantage of to not duplicate resource objects in memory. To maintain safety for controllers and for them to be able to modify data they get back from the cache and the resource service, the client they are presented in their runtime will be wrapped with an autogenerated client which clones request and response messages as they pass through the client. Another sizable change in this PR is to consolidate how server specific gRPC services get registered and managed. Before this was in a bunch of different methods and it was difficult to track down how gRPC services were registered. Now its all in one place. * Fix race in tests * Ensure the resource service is registered to the multiplexed handler for forwarding from client agents * Expose peer streaming on the internal handler --- .grpcmocks.yaml | 3 +- agent/consul/leader.go | 4 +- agent/consul/leader_ce_test.go | 6 +- agent/consul/leader_peering_test.go | 4 +- agent/consul/leader_test.go | 2 +- agent/consul/rpc.go | 4 +- agent/consul/server.go | 343 ++--------- agent/consul/server_ce.go | 5 +- agent/consul/server_grpc.go | 538 ++++++++++++++++++ agent/grpc-external/services/acl/server.go | 4 +- .../services/connectca/server.go | 4 +- .../services/dataplane/server.go | 4 +- agent/grpc-external/services/dns/server.go | 4 +- .../services/peerstream/server.go | 4 +- .../peerstream/subscription_manager.go | 4 + .../grpc-external/services/resource/server.go | 4 +- .../services/resource/testing/builder.go | 66 ++- .../services/serverdiscovery/server.go | 4 +- agent/grpc-external/testutils/acl.go | 21 +- agent/grpc-external/testutils/server.go | 2 +- agent/grpc-internal/handler.go | 11 +- agent/grpc-internal/server_test.go | 9 +- .../services/subscribe/subscribe_test.go | 74 ++- agent/grpc-internal/stats_test.go | 6 +- agent/grpc-middleware/stats.go | 7 + agent/rpc/operator/service.go | 4 +- agent/rpc/peering/service.go | 4 +- build-support/scripts/devtools.sh | 2 + go.mod | 2 + go.sum | 8 +- internal/catalog/catalogtest/run_test.go | 29 +- internal/controller/cache/clone.go | 93 +++ internal/controller/cache/clone_test.go | 214 +++++++ internal/controller/controller.go | 19 + internal/controller/controller_test.go | 28 + internal/controller/controllertest/builder.go | 96 ++++ internal/controller/mem_consistency_test.go | 343 +++++++++++ internal/controller/runner.go | 67 ++- internal/controller/testing.go | 6 + internal/resource/resourcetest/builder.go | 14 +- .../protoc-gen-grpc-clone/e2e/.mockery.yaml | 15 + .../protoc-gen-grpc-clone/e2e/e2e_test.go | 72 +++ .../e2e/mock_SimpleClient_test.go | 180 ++++++ .../e2e/mock_Simple_FlowClient_test.go | 356 ++++++++++++ .../e2e/proto/buf.gen.yaml | 22 + .../e2e/proto/cloning_stream.pb.go | 31 + .../e2e/proto/service.pb.go | 258 +++++++++ .../e2e/proto/service.proto | 22 + .../e2e/proto/service_cloning_grpc.pb.go | 69 +++ .../e2e/proto/service_grpc.pb.go | 167 ++++++ .../internal/generate/generate.go | 146 +++++ .../generate/templates/cloning-stream.tmpl | 31 + .../internal/generate/templates/file.tmpl | 14 + .../templates/server-stream-method.tmpl | 10 + .../internal/generate/templates/service.tmpl | 44 ++ .../generate/templates/unary-method.tmpl | 10 + internal/tools/protoc-gen-grpc-clone/main.go | 27 + logging/names.go | 6 + proto-public/buf.gen.yaml | 6 + proto-public/pbacl/acl_cloning_grpc.pb.go | 69 +++ .../pbconnectca/ca_cloning_grpc.pb.go | 69 +++ proto-public/pbconnectca/cloning_stream.pb.go | 31 + .../pbdataplane/dataplane_cloning_grpc.pb.go | 69 +++ proto-public/pbdns/dns_cloning_grpc.pb.go | 58 ++ proto-public/pbresource/cloning_stream.pb.go | 31 + .../pbresource/resource_cloning_grpc.pb.go | 124 ++++ .../pbserverdiscovery/cloning_stream.pb.go | 31 + .../serverdiscovery_cloning_grpc.pb.go | 58 ++ test/integration/consul-container/go.sum | 3 +- testing/deployer/topology/default_versions.go | 8 +- 70 files changed, 3665 insertions(+), 438 deletions(-) create mode 100644 agent/consul/server_grpc.go create mode 100644 internal/controller/cache/clone.go create mode 100644 internal/controller/cache/clone_test.go create mode 100644 internal/controller/controllertest/builder.go create mode 100644 internal/controller/mem_consistency_test.go create mode 100644 internal/tools/protoc-gen-grpc-clone/e2e/.mockery.yaml create mode 100644 internal/tools/protoc-gen-grpc-clone/e2e/e2e_test.go create mode 100644 internal/tools/protoc-gen-grpc-clone/e2e/mock_SimpleClient_test.go create mode 100644 internal/tools/protoc-gen-grpc-clone/e2e/mock_Simple_FlowClient_test.go create mode 100644 internal/tools/protoc-gen-grpc-clone/e2e/proto/buf.gen.yaml create mode 100644 internal/tools/protoc-gen-grpc-clone/e2e/proto/cloning_stream.pb.go create mode 100644 internal/tools/protoc-gen-grpc-clone/e2e/proto/service.pb.go create mode 100644 internal/tools/protoc-gen-grpc-clone/e2e/proto/service.proto create mode 100644 internal/tools/protoc-gen-grpc-clone/e2e/proto/service_cloning_grpc.pb.go create mode 100644 internal/tools/protoc-gen-grpc-clone/e2e/proto/service_grpc.pb.go create mode 100644 internal/tools/protoc-gen-grpc-clone/internal/generate/generate.go create mode 100644 internal/tools/protoc-gen-grpc-clone/internal/generate/templates/cloning-stream.tmpl create mode 100644 internal/tools/protoc-gen-grpc-clone/internal/generate/templates/file.tmpl create mode 100644 internal/tools/protoc-gen-grpc-clone/internal/generate/templates/server-stream-method.tmpl create mode 100644 internal/tools/protoc-gen-grpc-clone/internal/generate/templates/service.tmpl create mode 100644 internal/tools/protoc-gen-grpc-clone/internal/generate/templates/unary-method.tmpl create mode 100644 internal/tools/protoc-gen-grpc-clone/main.go create mode 100644 proto-public/pbacl/acl_cloning_grpc.pb.go create mode 100644 proto-public/pbconnectca/ca_cloning_grpc.pb.go create mode 100644 proto-public/pbconnectca/cloning_stream.pb.go create mode 100644 proto-public/pbdataplane/dataplane_cloning_grpc.pb.go create mode 100644 proto-public/pbdns/dns_cloning_grpc.pb.go create mode 100644 proto-public/pbresource/cloning_stream.pb.go create mode 100644 proto-public/pbresource/resource_cloning_grpc.pb.go create mode 100644 proto-public/pbserverdiscovery/cloning_stream.pb.go create mode 100644 proto-public/pbserverdiscovery/serverdiscovery_cloning_grpc.pb.go diff --git a/.grpcmocks.yaml b/.grpcmocks.yaml index 32067cf10f..97949b9ff2 100644 --- a/.grpcmocks.yaml +++ b/.grpcmocks.yaml @@ -2,8 +2,9 @@ # SPDX-License-Identifier: BUSL-1.1 with-expecter: true -all: true recursive: true +include-regex: ".*" +exclude-regex: "(serverStream|Is(Inmem|Cloning).*Client)" # We don't want the mocks within proto-public to prevent forcing a dependency # of the testify library on the modules usage. The mocks are only for # internal testing purposes. Other consumers can generated the mocks into diff --git a/agent/consul/leader.go b/agent/consul/leader.go index 3edfaf902d..1baa8cbcdc 100644 --- a/agent/consul/leader.go +++ b/agent/consul/leader.go @@ -350,13 +350,13 @@ func (s *Server) establishLeadership(ctx context.Context) error { } if s.useV2Tenancy { - if err := s.initTenancy(ctx, s.resourceServiceServer.Backend); err != nil { + if err := s.initTenancy(ctx, s.storageBackend); err != nil { return err } } if s.useV2Resources { - if err := s.initConsulService(ctx, s.insecureResourceServiceClient); err != nil { + if err := s.initConsulService(ctx, pbresource.NewResourceServiceClient(s.insecureSafeGRPCChan)); err != nil { return err } } diff --git a/agent/consul/leader_ce_test.go b/agent/consul/leader_ce_test.go index 23009cfbc3..86da505c3a 100644 --- a/agent/consul/leader_ce_test.go +++ b/agent/consul/leader_ce_test.go @@ -48,16 +48,16 @@ func TestServer_InitTenancy(t *testing.T) { Name: resource.DefaultNamespaceName, } - ns, err := s.resourceServiceServer.Backend.Read(context.Background(), storage.StrongConsistency, nsID) + ns, err := s.storageBackend.Read(context.Background(), storage.StrongConsistency, nsID) require.NoError(t, err) require.Equal(t, resource.DefaultNamespaceName, ns.Id.Name) // explicitly call initiTenancy to verify we do not re-create namespace - err = s.initTenancy(context.Background(), s.resourceServiceServer.Backend) + err = s.initTenancy(context.Background(), s.storageBackend) require.NoError(t, err) // read again - actual, err := s.resourceServiceServer.Backend.Read(context.Background(), storage.StrongConsistency, nsID) + actual, err := s.storageBackend.Read(context.Background(), storage.StrongConsistency, nsID) require.NoError(t, err) require.Equal(t, ns.Id.Uid, actual.Id.Uid) diff --git a/agent/consul/leader_peering_test.go b/agent/consul/leader_peering_test.go index 8db496273d..ae5d2ae83e 100644 --- a/agent/consul/leader_peering_test.go +++ b/agent/consul/leader_peering_test.go @@ -1426,7 +1426,7 @@ func TestLeader_PeeringMetrics_emitPeeringMetrics(t *testing.T) { require.NoError(t, s2.fsm.State().PeeringWrite(lastIdx, &pbpeering.PeeringWriteRequest{Peering: p2})) // connect the stream - mst1, err := s2.peeringServer.Tracker.Connected(s2PeerID1) + mst1, err := s2.peerStreamServer.Tracker.Connected(s2PeerID1) require.NoError(t, err) // mimic tracking exported services @@ -1437,7 +1437,7 @@ func TestLeader_PeeringMetrics_emitPeeringMetrics(t *testing.T) { }) // connect the stream - mst2, err := s2.peeringServer.Tracker.Connected(s2PeerID2) + mst2, err := s2.peerStreamServer.Tracker.Connected(s2PeerID2) require.NoError(t, err) // mimic tracking exported services diff --git a/agent/consul/leader_test.go b/agent/consul/leader_test.go index 0ddecb7633..619d6ae6da 100644 --- a/agent/consul/leader_test.go +++ b/agent/consul/leader_test.go @@ -64,7 +64,7 @@ func Test_InitConsulService(t *testing.T) { testrpc.WaitForRaftLeader(t, s.RPC, "dc1", testrpc.WithToken("root")) - client := s.insecureResourceServiceClient + client := pbresource.NewResourceServiceClient(s.insecureSafeGRPCChan) consulServiceID := &pbresource.ID{ Name: structs.ConsulServiceName, diff --git a/agent/consul/rpc.go b/agent/consul/rpc.go index dbb781951e..8c5ad14eff 100644 --- a/agent/consul/rpc.go +++ b/agent/consul/rpc.go @@ -244,7 +244,7 @@ func (s *Server) handleConn(conn net.Conn, isTLS bool) { s.handleInsecureConn(conn) case pool.RPCGRPC: - s.grpcHandler.Handle(conn) + s.internalGRPCHandler.Handle(conn) case pool.RPCRaftForwarding: s.handleRaftForwarding(conn) @@ -315,7 +315,7 @@ func (s *Server) handleNativeTLS(conn net.Conn) { s.handleSnapshotConn(tlsConn) case pool.ALPN_RPCGRPC: - s.grpcHandler.Handle(tlsConn) + s.internalGRPCHandler.Handle(tlsConn) case pool.ALPN_RPCRaftForwarding: s.handleRaftForwarding(tlsConn) diff --git a/agent/consul/server.go b/agent/consul/server.go index 8c6efac77f..8ef2f5e053 100644 --- a/agent/consul/server.go +++ b/agent/consul/server.go @@ -20,11 +20,10 @@ import ( "time" "github.com/armon/go-metrics" + "github.com/fullstorydev/grpchan/inprocgrpc" "go.etcd.io/bbolt" "golang.org/x/time/rate" "google.golang.org/grpc" - "google.golang.org/grpc/credentials/insecure" - "google.golang.org/grpc/reflection" "github.com/hashicorp/consul-net-rpc/net/rpc" "github.com/hashicorp/go-connlimit" @@ -39,7 +38,6 @@ import ( "github.com/hashicorp/serf/serf" "github.com/hashicorp/consul/acl" - "github.com/hashicorp/consul/acl/resolver" "github.com/hashicorp/consul/agent/blockingquery" "github.com/hashicorp/consul/agent/connect" "github.com/hashicorp/consul/agent/consul/authmethod" @@ -53,14 +51,7 @@ import ( "github.com/hashicorp/consul/agent/consul/usagemetrics" "github.com/hashicorp/consul/agent/consul/wanfed" "github.com/hashicorp/consul/agent/consul/xdscapacity" - aclgrpc "github.com/hashicorp/consul/agent/grpc-external/services/acl" - "github.com/hashicorp/consul/agent/grpc-external/services/connectca" - "github.com/hashicorp/consul/agent/grpc-external/services/dataplane" "github.com/hashicorp/consul/agent/grpc-external/services/peerstream" - resourcegrpc "github.com/hashicorp/consul/agent/grpc-external/services/resource" - "github.com/hashicorp/consul/agent/grpc-external/services/serverdiscovery" - agentgrpc "github.com/hashicorp/consul/agent/grpc-internal" - "github.com/hashicorp/consul/agent/grpc-internal/services/subscribe" "github.com/hashicorp/consul/agent/hcp" hcpclient "github.com/hashicorp/consul/agent/hcp/client" logdrop "github.com/hashicorp/consul/agent/log-drop" @@ -68,7 +59,6 @@ import ( "github.com/hashicorp/consul/agent/pool" "github.com/hashicorp/consul/agent/router" "github.com/hashicorp/consul/agent/rpc/middleware" - "github.com/hashicorp/consul/agent/rpc/operator" "github.com/hashicorp/consul/agent/rpc/peering" "github.com/hashicorp/consul/agent/structs" "github.com/hashicorp/consul/agent/token" @@ -82,6 +72,7 @@ import ( "github.com/hashicorp/consul/internal/resource" "github.com/hashicorp/consul/internal/resource/demo" "github.com/hashicorp/consul/internal/resource/reaper" + "github.com/hashicorp/consul/internal/storage" raftstorage "github.com/hashicorp/consul/internal/storage/raft" "github.com/hashicorp/consul/internal/tenancy" "github.com/hashicorp/consul/lib" @@ -90,7 +81,6 @@ import ( "github.com/hashicorp/consul/logging" "github.com/hashicorp/consul/proto-public/pbmesh/v2beta1/pbproxystate" "github.com/hashicorp/consul/proto-public/pbresource" - "github.com/hashicorp/consul/proto/private/pbsubscribe" "github.com/hashicorp/consul/tlsutil" "github.com/hashicorp/consul/types" cslversion "github.com/hashicorp/consul/version" @@ -139,8 +129,7 @@ const ( // and wait for a periodic reconcile. reconcileChSize = 256 - LeaderTransferMinVersion = "1.6.0" - + LeaderTransferMinVersion = "1.6.0" CatalogResourceExperimentName = "resource-apis" V2DNSExperimentName = "v2dns" V2TenancyExperimentName = "v2tenancy" @@ -283,6 +272,9 @@ type Server struct { // raftStorageBackend is the Raft-backed storage backend for resources. raftStorageBackend *raftstorage.Backend + // the currently in use storage backend + storageBackend storage.Backend + // reconcileCh is used to pass events from the serf handler // into the leader manager, so that the strong state can be // updated @@ -298,20 +290,35 @@ type Server struct { // is only ever closed. leaveCh chan struct{} - // externalACLServer serves the ACL service exposed on the external gRPC port. - // It is also exposed on the internal multiplexed "server" port to enable - // RPC forwarding. - externalACLServer *aclgrpc.Server - - // externalConnectCAServer serves the Connect CA service exposed on the external - // gRPC port. It is also exposed on the internal multiplexed "server" port to - // enable RPC forwarding. - externalConnectCAServer *connectca.Server - // externalGRPCServer has a gRPC server exposed on the dedicated gRPC ports, as // opposed to the multiplexed "server" port which is served by grpcHandler. externalGRPCServer *grpc.Server + // insecureUnsafeGRPCChan is used to access gRPC services on the server without going + // through protobuf serialization/deserialization, performing any network IO or requiring + // authorization. This may be passed as the gRPC client conn to any standard gRPC client + // constructor instead of a standard network protocol based client conn. Using this as the + // client conn will cut down on CPU and memory usage for doing in-process gRPC but comes + // with the drawbacks that any data sent over this interface is inherently shared + // and both ends must cooperate with regards to the immutability. Therefore, in + // most cases the insecureSafeGRPCChannel should be used instead which will clone the protobuf + // types as they pass through. + insecureUnsafeGRPCChan *inprocgrpc.Channel + + // insecureSafeGRPCChan is used to access gRPC services on the server without going + // through the standard protobuf serialization/deserialization, performing network + // io or requiring authorization. This gRPC client conn implementation will still + // clone protobuf messages as they pass through and so the client and server + // implementations do not need to coordinate with regards to data immutability. + insecureSafeGRPCChan *inprocgrpc.Channel + + // secureSafeGRPCChan is used to access gRPC services on the server without going + // through the standard protobuf serialization/deserialization or performing network + // io. This gRPC client conn implementation will still clone protobuf messages as + // they pass through and so the client and server implementations do not need + // to coordinate with regards to data immutability. + secureSafeGRPCChan *inprocgrpc.Channel + // router is used to map out Consul servers in the WAN and in Consul // Enterprise user-defined areas. router *router.Router @@ -324,9 +331,9 @@ type Server struct { rpcConnLimiter connlimit.Limiter // Listener is used to listen for incoming connections - Listener net.Listener - grpcHandler connHandler - rpcServer *rpc.Server + Listener net.Listener + internalGRPCHandler connHandler + rpcServer *rpc.Server // incomingRPCLimiter rate-limits incoming net/rpc and gRPC calls. incomingRPCLimiter rpcRate.RequestLimitsHandler @@ -429,15 +436,9 @@ type Server struct { // peeringBackend is shared between the external and internal gRPC services for peering peeringBackend *PeeringBackend - // operatorBackend is shared between the external and internal gRPC services for peering - operatorBackend *OperatorBackend - // peerStreamServer is a server used to handle peering streams from external clusters. peerStreamServer *peerstream.Server - // peeringServer handles peering RPC requests internal to this cluster, like generating peering tokens. - peeringServer *peering.Server - // xdsCapacityController controls the number of concurrent xDS streams the // server is able to handle. xdsCapacityController *xdscapacity.Controller @@ -447,27 +448,11 @@ type Server struct { // embedded struct to hold all the enterprise specific data EnterpriseServer - operatorServer *operator.Server // routineManager is responsible for managing longer running go routines // run by the Server routineManager *routine.Manager - // resourceServiceServer implements the Resource Service. - resourceServiceServer *resourcegrpc.Server - - // insecureResourceServiceClient is a client that can be used to communicate - // with the Resource Service in-process (i.e. not via the network) *without* - // auth. It should only be used for purely-internal workloads, such as - // controllers. - insecureResourceServiceClient pbresource.ResourceServiceClient - - // secureResourceServiceClient is a client that can be used to communicate - // with the Resource Service in-process (i.e. not via the network) *with* auth. - // It can be used to make requests to the Resource Service on behalf of the user - // (e.g. from the HTTP API). - secureResourceServiceClient pbresource.ResourceServiceClient - // controllerManager schedules the execution of controllers. controllerManager *controller.Manager @@ -499,6 +484,7 @@ func (s *Server) IncrementBlockingQueries() uint64 { } type connHandler interface { + RegisterService(*grpc.ServiceDesc, any) Run() error Handle(conn net.Conn) Shutdown() error @@ -584,6 +570,8 @@ func NewServer(config *Config, flat Deps, externalGRPCServer *grpc.Server, } go s.raftStorageBackend.Run(&lib.StopChannelContext{StopCh: shutdownCh}) + s.storageBackend = s.raftStorageBackend + s.fsm = fsm.NewFromDeps(fsm.Deps{ Logger: flat.Logger, NewStateStore: func() *state.Store { @@ -853,17 +841,13 @@ func NewServer(config *Config, flat Deps, externalGRPCServer *grpc.Server, s.reportingManager = reporting.NewReportingManager(s.logger, getEnterpriseReportingDeps(flat), s, s.fsm.State()) go s.reportingManager.Run(&lib.StopChannelContext{StopCh: s.shutdownCh}) - // Setup insecure resource service client. - if err := s.setupInsecureResourceServiceClient(flat.Registry, logger); err != nil { + // configure the server specific grpc interfaces (in-process + internal multiplexed grpc) + if err := s.setupGRPCInterfaces(config, flat); err != nil { return nil, err } - // Initialize external gRPC server - s.setupExternalGRPC(config, flat, logger) - - // Setup secure resource service client. We need to do it after we setup the - // gRPC server because it needs the server to be instantiated. - if err := s.setupSecureResourceServiceClient(); err != nil { + // register server specific grpc services with all the interfaces they should be exposed on. + if err := s.setupGRPCServices(config, flat); err != nil { return nil, err } @@ -871,12 +855,20 @@ func NewServer(config *Config, flat Deps, externalGRPCServer *grpc.Server, // // Note: some "external" gRPC services are also exposed on the internal gRPC server // to enable RPC forwarding. - s.grpcHandler = newGRPCHandlerFromConfig(flat, config, s) s.grpcLeaderForwarder = flat.LeaderForwarder s.controllerManager = controller.NewManager( - s.insecureResourceServiceClient, - logger.Named(logging.ControllerRuntime), + // Usage of the insecure + unsafe grpc chan is required for the controller + // manager. It must be unauthorized so that controllers do not need to + // present valid ACL tokens for their requests and it must use the unsafe + // variant so that the controller runtimes indexing/caching layer doesn't + // keep many copies of resources around in memory for long. Care will + // be taken within the controller manager to wrap this client with another + // which clones protobuf types passing through to ensure controllers + // cannot modify the canonical resource service data that has flowed + // through the storage backend. + pbresource.NewResourceServiceClient(s.insecureUnsafeGRPCChan), + s.loggers.Named(logging.ControllerRuntime), ) if err := s.registerControllers(flat, proxyUpdater); err != nil { return nil, err @@ -902,7 +894,7 @@ func NewServer(config *Config, flat Deps, externalGRPCServer *grpc.Server, Logger: serverLogger, NodeName: s.config.NodeName, EntMeta: s.config.AgentEnterpriseMeta(), - Client: s.insecureResourceServiceClient, + Client: pbresource.NewResourceServiceClient(s.insecureSafeGRPCChan), } } else { s.registrator = V1ConsulRegistrator{ @@ -920,7 +912,7 @@ func NewServer(config *Config, flat Deps, externalGRPCServer *grpc.Server, // Start listening for RPC requests. go func() { - if err := s.grpcHandler.Run(); err != nil { + if err := s.internalGRPCHandler.Run(); err != nil { s.logger.Error("gRPC server failed", "error", err) } }() @@ -1031,64 +1023,6 @@ func (s *Server) registerControllers(deps Deps, proxyUpdater ProxyUpdater) error return s.controllerManager.ValidateDependencies(s.registry.Types()) } -func newGRPCHandlerFromConfig(deps Deps, config *Config, s *Server) connHandler { - if s.peeringBackend == nil { - panic("peeringBackend is required during construction") - } - - p := peering.NewServer(peering.Config{ - Backend: s.peeringBackend, - Tracker: s.peerStreamServer.Tracker, - Logger: deps.Logger.Named("grpc-api.peering"), - ForwardRPC: func(info structs.RPCInfo, fn func(*grpc.ClientConn) error) (bool, error) { - // Only forward the request if the dc in the request matches the server's datacenter. - if info.RequestDatacenter() != "" && info.RequestDatacenter() != config.Datacenter { - return false, fmt.Errorf("requests to generate peering tokens cannot be forwarded to remote datacenters") - } - return s.ForwardGRPC(s.grpcConnPool, info, fn) - }, - Datacenter: config.Datacenter, - ConnectEnabled: config.ConnectEnabled, - PeeringEnabled: config.PeeringEnabled, - Locality: config.Locality, - FSMServer: s, - }) - s.peeringServer = p - o := operator.NewServer(operator.Config{ - Backend: s.operatorBackend, - Logger: deps.Logger.Named("grpc-api.operator"), - ForwardRPC: func(info structs.RPCInfo, fn func(*grpc.ClientConn) error) (bool, error) { - // Only forward the request if the dc in the request matches the server's datacenter. - if info.RequestDatacenter() != "" && info.RequestDatacenter() != config.Datacenter { - return false, fmt.Errorf("requests to transfer leader cannot be forwarded to remote datacenters") - } - return s.ForwardGRPC(s.grpcConnPool, info, fn) - }, - Datacenter: config.Datacenter, - }) - s.operatorServer = o - - register := func(srv *grpc.Server) { - if config.RPCConfig.EnableStreaming { - pbsubscribe.RegisterStateChangeSubscriptionServer(srv, subscribe.NewServer( - &subscribeBackend{srv: s, connPool: deps.GRPCConnPool}, - deps.Logger.Named("grpc-api.subscription"))) - } - s.peeringServer.Register(srv) - s.operatorServer.Register(srv) - s.registerEnterpriseGRPCServices(deps, srv) - - // Note: these external gRPC services are also exposed on the internal server to - // enable RPC forwarding. - s.peerStreamServer.Register(srv) - s.externalACLServer.Register(srv) - s.externalConnectCAServer.Register(srv) - s.resourceServiceServer.Register(srv) - } - - return agentgrpc.NewHandler(deps.Logger, config.RPCAddr, register, nil, s.incomingRPCLimiter) -} - func (s *Server) connectCARootsMonitor(ctx context.Context) { for { ws := memdb.NewWatchSet() @@ -1438,169 +1372,6 @@ func (s *Server) setupRPC() error { return nil } -// Initialize and register services on external gRPC server. -func (s *Server) setupExternalGRPC(config *Config, deps Deps, logger hclog.Logger) { - s.externalACLServer = aclgrpc.NewServer(aclgrpc.Config{ - ACLsEnabled: s.config.ACLsEnabled, - ForwardRPC: func(info structs.RPCInfo, fn func(*grpc.ClientConn) error) (bool, error) { - return s.ForwardGRPC(s.grpcConnPool, info, fn) - }, - InPrimaryDatacenter: s.InPrimaryDatacenter(), - LoadAuthMethod: func(methodName string, entMeta *acl.EnterpriseMeta) (*structs.ACLAuthMethod, aclgrpc.Validator, error) { - return s.loadAuthMethod(methodName, entMeta) - }, - LocalTokensEnabled: s.LocalTokensEnabled, - Logger: logger.Named("grpc-api.acl"), - NewLogin: func() aclgrpc.Login { return s.aclLogin() }, - NewTokenWriter: func() aclgrpc.TokenWriter { return s.aclTokenWriter() }, - PrimaryDatacenter: s.config.PrimaryDatacenter, - ValidateEnterpriseRequest: s.validateEnterpriseRequest, - }) - s.externalACLServer.Register(s.externalGRPCServer) - - s.externalConnectCAServer = connectca.NewServer(connectca.Config{ - Publisher: s.publisher, - GetStore: func() connectca.StateStore { return s.FSM().State() }, - Logger: logger.Named("grpc-api.connect-ca"), - ACLResolver: s.ACLResolver, - CAManager: s.caManager, - ForwardRPC: func(info structs.RPCInfo, fn func(*grpc.ClientConn) error) (bool, error) { - return s.ForwardGRPC(s.grpcConnPool, info, fn) - }, - ConnectEnabled: s.config.ConnectEnabled, - }) - s.externalConnectCAServer.Register(s.externalGRPCServer) - - dataplane.NewServer(dataplane.Config{ - GetStore: func() dataplane.StateStore { return s.FSM().State() }, - Logger: logger.Named("grpc-api.dataplane"), - ACLResolver: s.ACLResolver, - Datacenter: s.config.Datacenter, - EnableV2: stringslice.Contains(deps.Experiments, CatalogResourceExperimentName), - ResourceAPIClient: s.insecureResourceServiceClient, - }).Register(s.externalGRPCServer) - - serverdiscovery.NewServer(serverdiscovery.Config{ - Publisher: s.publisher, - ACLResolver: s.ACLResolver, - Logger: logger.Named("grpc-api.server-discovery"), - }).Register(s.externalGRPCServer) - - s.peeringBackend = NewPeeringBackend(s) - s.operatorBackend = NewOperatorBackend(s) - - s.peerStreamServer = peerstream.NewServer(peerstream.Config{ - Backend: s.peeringBackend, - GetStore: func() peerstream.StateStore { return s.FSM().State() }, - Logger: logger.Named("grpc-api.peerstream"), - ACLResolver: s.ACLResolver, - Datacenter: s.config.Datacenter, - ConnectEnabled: s.config.ConnectEnabled, - ForwardRPC: func(info structs.RPCInfo, fn func(*grpc.ClientConn) error) (bool, error) { - // Only forward the request if the dc in the request matches the server's datacenter. - if info.RequestDatacenter() != "" && info.RequestDatacenter() != config.Datacenter { - return false, fmt.Errorf("requests to generate peering tokens cannot be forwarded to remote datacenters") - } - return s.ForwardGRPC(s.grpcConnPool, info, fn) - }, - }) - s.peerStreamServer.Register(s.externalGRPCServer) - - tenancyBridge := NewV1TenancyBridge(s) - if s.useV2Tenancy { - tenancyBridgeV2 := tenancy.NewV2TenancyBridge() - tenancyBridge = tenancyBridgeV2.WithClient(s.insecureResourceServiceClient) - } - - s.resourceServiceServer = resourcegrpc.NewServer(resourcegrpc.Config{ - Registry: deps.Registry, - Backend: s.raftStorageBackend, - ACLResolver: s.ACLResolver, - Logger: logger.Named("grpc-api.resource"), - TenancyBridge: tenancyBridge, - UseV2Tenancy: s.useV2Tenancy, - }) - s.resourceServiceServer.Register(s.externalGRPCServer) - - reflection.Register(s.externalGRPCServer) -} - -func (s *Server) setupInsecureResourceServiceClient(typeRegistry resource.Registry, logger hclog.Logger) error { - if s.raftStorageBackend == nil { - return fmt.Errorf("raft storage backend cannot be nil") - } - - // Can't use interface type var here since v2 specific "WithClient(...)" is called futher down. - tenancyBridge := NewV1TenancyBridge(s) - tenancyBridgeV2 := tenancy.NewV2TenancyBridge() - if s.useV2Tenancy { - tenancyBridge = tenancyBridgeV2 - } - server := resourcegrpc.NewServer(resourcegrpc.Config{ - Registry: typeRegistry, - Backend: s.raftStorageBackend, - ACLResolver: resolver.DANGER_NO_AUTH{}, - Logger: logger.Named("grpc-api.resource"), - TenancyBridge: tenancyBridge, - UseV2Tenancy: s.useV2Tenancy, - }) - - conn, err := s.runInProcessGRPCServer(server.Register) - if err != nil { - return err - } - s.insecureResourceServiceClient = pbresource.NewResourceServiceClient(conn) - tenancyBridgeV2.WithClient(s.insecureResourceServiceClient) - return nil -} - -func (s *Server) setupSecureResourceServiceClient() error { - if s.resourceServiceServer == nil { - return fmt.Errorf("resource service server cannot be nil") - } - conn, err := s.runInProcessGRPCServer(s.resourceServiceServer.Register) - if err != nil { - return err - } - s.secureResourceServiceClient = pbresource.NewResourceServiceClient(conn) - - return nil -} - -// runInProcessGRPCServer runs a gRPC server that can only be accessed in the -// same process, rather than over the network, using a pipe listener. -func (s *Server) runInProcessGRPCServer(registerFn ...func(*grpc.Server)) (*grpc.ClientConn, error) { - server := grpc.NewServer() - - for _, fn := range registerFn { - fn(server) - } - - pipe := agentgrpc.NewPipeListener() - go server.Serve(pipe) - - go func() { - <-s.shutdownCh - server.Stop() - }() - - conn, err := grpc.Dial("", - grpc.WithTransportCredentials(insecure.NewCredentials()), - grpc.WithContextDialer(pipe.DialContext), - grpc.WithBlock(), - ) - if err != nil { - server.Stop() - return nil, err - } - go func() { - <-s.shutdownCh - conn.Close() - }() - - return conn, nil -} - // Shutdown is used to shutdown the server func (s *Server) Shutdown() error { s.logger.Info("shutting down server") @@ -1647,8 +1418,8 @@ func (s *Server) Shutdown() error { s.Listener.Close() } - if s.grpcHandler != nil { - if err := s.grpcHandler.Shutdown(); err != nil { + if s.internalGRPCHandler != nil { + if err := s.internalGRPCHandler.Shutdown(); err != nil { s.logger.Warn("failed to stop gRPC server", "error", err) } } @@ -2304,7 +2075,7 @@ func (s *Server) hcpServerStatus(deps Deps) hcp.StatusCallback { } func (s *Server) ResourceServiceClient() pbresource.ResourceServiceClient { - return s.secureResourceServiceClient + return pbresource.NewResourceServiceClient(s.secureSafeGRPCChan) } func fileExists(name string) (bool, error) { diff --git a/agent/consul/server_ce.go b/agent/consul/server_ce.go index fd554ca2c3..d1c19d3135 100644 --- a/agent/consul/server_ce.go +++ b/agent/consul/server_ce.go @@ -11,7 +11,6 @@ import ( "time" "github.com/armon/go-metrics" - "google.golang.org/grpc" "github.com/hashicorp/go-multierror" "github.com/hashicorp/serf/coordinate" @@ -28,7 +27,9 @@ func (s *Server) runEnterpriseRateLimiterConfigEntryController() error { return nil } -func (s *Server) registerEnterpriseGRPCServices(deps Deps, srv *grpc.Server) {} +func (s *Server) setupEnterpriseGRPCServices(config *Config, deps Deps) error { + return nil +} func (s *Server) enterpriseValidateJoinWAN() error { return nil // no-op diff --git a/agent/consul/server_grpc.go b/agent/consul/server_grpc.go new file mode 100644 index 0000000000..33106fd479 --- /dev/null +++ b/agent/consul/server_grpc.go @@ -0,0 +1,538 @@ +// Copyright (c) HashiCorp, Inc. +// SPDX-License-Identifier: BUSL-1.1 + +package consul + +import ( + "fmt" + + "github.com/armon/go-metrics" + "github.com/fullstorydev/grpchan/inprocgrpc" + middleware "github.com/grpc-ecosystem/go-grpc-middleware" + recovery "github.com/grpc-ecosystem/go-grpc-middleware/recovery" + "google.golang.org/grpc" + "google.golang.org/grpc/reflection" + + "github.com/hashicorp/consul/acl" + "github.com/hashicorp/consul/acl/resolver" + aclgrpc "github.com/hashicorp/consul/agent/grpc-external/services/acl" + "github.com/hashicorp/consul/agent/grpc-external/services/connectca" + "github.com/hashicorp/consul/agent/grpc-external/services/dataplane" + "github.com/hashicorp/consul/agent/grpc-external/services/peerstream" + resourcegrpc "github.com/hashicorp/consul/agent/grpc-external/services/resource" + "github.com/hashicorp/consul/agent/grpc-external/services/serverdiscovery" + agentgrpc "github.com/hashicorp/consul/agent/grpc-internal" + "github.com/hashicorp/consul/agent/grpc-internal/services/subscribe" + agentmiddleware "github.com/hashicorp/consul/agent/grpc-middleware" + "github.com/hashicorp/consul/agent/rpc/operator" + "github.com/hashicorp/consul/agent/rpc/peering" + "github.com/hashicorp/consul/agent/structs" + "github.com/hashicorp/consul/internal/resource" + "github.com/hashicorp/consul/internal/tenancy" + "github.com/hashicorp/consul/lib/stringslice" + "github.com/hashicorp/consul/logging" + "github.com/hashicorp/consul/proto-public/pbresource" + "github.com/hashicorp/consul/proto/private/pbsubscribe" +) + +func (s *Server) setupGRPCInterfaces(config *Config, deps Deps) error { + // A server has 5 different gRPC interfaces + // + // * External - This is the main public gRPC network listener. This + // is an actual *grpc.Server that we have listening on both the + // grpc and grpc_tls ports. Generally this interface will not be + // used by the server itself. All services which are intended + // to be public APIs must be registered to this interface. This + // interface is created outside of the server in the agent code + // and then passed to the NewServer constructor. Some services + // like xDS and DNS get registered outside of the server code. + // + // * Internal / Multiplexed - Our internal_rpc port uses yamux and + // various byte prefixes to multiplex different protocols over + // the single connection. One of the multiplexed protocols is + // gRPC. gRPC in this fashion works using a custom net.Listener + // implementation that receives net.Conns to be handled through + // a channel. When a new yamux session is opened which produces + // a yamux conn (which implements the net.Conn interface), the + // connection is then sent to the custom listener. Then the + // standard grpc.Server.Serve method can accept the conn from + // the listener and operate on it like any other standard conn. + // Historically, the external gRPC interface was optional and + // so all services which needed leader or DC forwarding had to + // be exposed on this interface in order to guarantee they + // would be available. In the future, an external gRPC interface + // likely will be required and the services which need registering + // to the multiplexed listener will be greatly reduced. In the + // very long term we want to get rid of this internal multiplexed + // port/listener and instead have all component communications use + // gRPC natively. For now though, if your service will need to + // RECEIVE forwarded requests then it must be registered to this + // interface. + // + // * In-Process - For routines running on the server we don't want them + // to require network i/o as that will incur a lot of unnecessary + // overhead. To avoid that we are utilizing the `grpchan` library + // (github.com/fullstorydev/grpchan) and its `inprocgrpc` package. + // The library provides the `inprocgrpc.Channel` which implements + // both the `grpc.ServiceRegistrar` and `grpc.ClientConnInterface` + // interfaces. Services get registered to the `Channel` and then + // gRPC service clients can be created with the `Channel` used + // for the backing `ClientConn`. When a client then uses the + // `Invoke` or `NewStream` methods on the `Channel`, the `Channel` + // will lookup in its registry of services to find the service's + // server implementation and then have the standard + // grpc.MethodDesc.Handler function handle the request. We use + // a few variants of the in-process gRPC Channel. For now all + // these channels are created and managed in server code but we + // may need to move these into the higher level agent setup. + // + // * Insecure + Unsafe - The insecure + unsafe gRPC Channel has + // services registered to it that wont do typical ACL + // resolution. Instead when the service resolves ACL tokens + // a resolver is used which always grants unrestricted + // privileges. Additionally, this "unsafe" variant DOES + // NOT clone resources as they pass through the channel. Care + // Must be taken to note mutate the data passed through the + // Channel or else we could easily cause data race related + // or consistency bugs. + // + // * Insecure + Safe - Similar to the Insecure + Unsafe variant, + // ACL resolution always provides an authorizer with unrestricted + // privileges. However, this interface is concurrency/memory safe + // in that protobuf messages passing through the interface are + // cloned so that the client is free to mutate those messages + // once the request is complete. All services registered to the + // Unsafe variant should also be registered to this interface. + // + // * Secure + Safe - This Channel will do typical ACL resolution from + // tokens and will clone protobuf messages that pass through. This + // interface will be useful for something like the HTTP server that + // is crafting the gRPC requests from a user request and wants to + // assume no implicit privileges by the nature of running on the + // server. All services registered to the insecure variants should + // also be registered to this interface. Additionally other services + // that correspond to user requests should also be registered to this + // interface. + // + // Currently there is not a need for a Secure + Unsafe variant. We could + // add it if needed in the future. + + recoveryOpts := agentmiddleware.PanicHandlerMiddlewareOpts(s.loggers.Named(logging.GRPCAPI)) + + inprocLabels := []metrics.Label{{ + Name: "server_type", + Value: "in-process", + }} + + statsHandler := agentmiddleware.NewStatsHandler(metrics.Default(), inprocLabels) + + // TODO(inproc-grpc) - figure out what to do with rate limiting inproc grpc. If we + // want to rate limit in-process clients then we are going to need a unary interceptor + // to do that. Another idea would be to create rate limited clients which can be given + // to controllers or other internal code so that the whole Channel isn't limited but + // rather individual consumers of that channel. + + // Build the Insecure + Unsafe gRPC Channel + s.insecureUnsafeGRPCChan = new(inprocgrpc.Channel). + // Bypass the in-process gRPCs cloning functionality by providing + // a Cloner implementation which doesn't actually clone the data. + // Note that this is only done for the Unsafe gRPC Channel and + // all the Safe variants will utilize the default cloning + // functionality. + WithCloner(inprocgrpc.CloneFunc(func(in any) (any, error) { + return in, nil + })). + WithServerUnaryInterceptor(middleware.ChainUnaryServer( + recovery.UnaryServerInterceptor(recoveryOpts...), + statsHandler.Intercept, + )). + WithServerStreamInterceptor(middleware.ChainStreamServer( + recovery.StreamServerInterceptor(recoveryOpts...), + agentmiddleware.NewActiveStreamCounter(metrics.Default(), inprocLabels).Intercept, + )) + + // Build the Insecure + Safe gRPC Channel + s.insecureSafeGRPCChan = new(inprocgrpc.Channel). + WithServerUnaryInterceptor(middleware.ChainUnaryServer( + recovery.UnaryServerInterceptor(recoveryOpts...), + statsHandler.Intercept, + )). + WithServerStreamInterceptor(middleware.ChainStreamServer( + recovery.StreamServerInterceptor(recoveryOpts...), + agentmiddleware.NewActiveStreamCounter(metrics.Default(), inprocLabels).Intercept, + )) + + // Build the Secure + Safe gRPC Channel + s.secureSafeGRPCChan = new(inprocgrpc.Channel). + WithServerUnaryInterceptor(middleware.ChainUnaryServer( + recovery.UnaryServerInterceptor(recoveryOpts...), + statsHandler.Intercept, + )). + WithServerStreamInterceptor(middleware.ChainStreamServer( + recovery.StreamServerInterceptor(recoveryOpts...), + agentmiddleware.NewActiveStreamCounter(metrics.Default(), inprocLabels).Intercept, + )) + + // create the internal multiplexed gRPC interface + s.internalGRPCHandler = agentgrpc.NewHandler(deps.Logger, config.RPCAddr, nil, s.incomingRPCLimiter) + + return nil +} + +func (s *Server) setupGRPCServices(config *Config, deps Deps) error { + // Register the resource service with the in-process registrars WITHOUT AUTHORIZATION + err := s.registerResourceServiceServer( + deps.Registry, + resolver.DANGER_NO_AUTH{}, + s.insecureUnsafeGRPCChan, + s.insecureSafeGRPCChan) + if err != nil { + return err + } + + // Register the resource service with all other registrars other + // than the internal/multiplexed interface. Currently there is + // no need to forward resource service RPCs and therefore the + // service doesn't need to be available on that interface. + err = s.registerResourceServiceServer( + deps.Registry, + s.ACLResolver, + s.secureSafeGRPCChan, + s.internalGRPCHandler, + s.externalGRPCServer, + ) + if err != nil { + return err + } + + // The ACL grpc services get registered with all "secure" gRPC interfaces + err = s.registerACLServer( + s.secureSafeGRPCChan, + s.externalGRPCServer, + s.internalGRPCHandler, + ) + if err != nil { + return err + } + + // register the Connect CA service on all "secure" interfaces + err = s.registerConnectCAServer( + s.secureSafeGRPCChan, + s.externalGRPCServer, + s.internalGRPCHandler, + ) + if err != nil { + return err + } + + // Initializing the peering backend must be done before + // creating any peering servers. There is other code which + // calls methods on this and so the backend must be stored + // on the Server type. In the future we should investigate + // whether we can not require the backend in that other code. + s.peeringBackend = NewPeeringBackend(s) + + // register the peering service on the external gRPC server only + // As this service is only ever accessed externally there is + // no need to register it on the various in-process Channels + s.peerStreamServer, err = s.registerPeerStreamServer( + config, + s.externalGRPCServer, + s.internalGRPCHandler, + ) + if err != nil { + return err + } + + // register the peering service on the internal interface only. As + // the peering gRPC service is a private API its only ever accessed + // via the internalGRPCHandler with an actual network conn managed + // by the Agents GRPCConnPool. + err = s.registerPeeringServer( + config, + s.internalGRPCHandler, + ) + if err != nil { + return err + } + + // Register the Operator service on all "secure" interfaces. The + // operator service is currently only accessed via the + // internalGRPCHandler but in the future these APIs are likely to + // become part of our "public" API and so it should be exposed on + // more interfaces. + err = s.registerOperatorServer( + config, + deps, + s.internalGRPCHandler, + s.secureSafeGRPCChan, + s.externalGRPCServer, + ) + if err != nil { + return err + } + + // register the stream subscription service on the multiplexed internal interface + // if stream is enabled. + if config.RPCConfig.EnableStreaming { + err = s.registerStreamSubscriptionServer( + deps, + s.internalGRPCHandler, + ) + if err != nil { + return err + } + } + + // register the server discovery service on all "secure" interfaces other + // than the multiplexed internal interface. This service is mainly consumed + // by the consul-server-connection-manager library which is used by various + // other system components other than the agent. + err = s.registerServerDiscoveryServer( + s.ACLResolver, + s.secureSafeGRPCChan, + s.externalGRPCServer, + ) + if err != nil { + return err + } + + // register the server discovery service on the insecure in-process channels. + // Currently, this is unused but eventually things such as the peering service + // should be refactored to consume the in-memory service instead of hooking + // directly into an the event publisher and subscribing to specific events. + err = s.registerServerDiscoveryServer( + resolver.DANGER_NO_AUTH{}, + s.insecureUnsafeGRPCChan, + s.insecureSafeGRPCChan, + ) + if err != nil { + return err + } + + // register the data plane service on the external gRPC server only. This + // service is only access by dataplanes and at this time there is no need + // for anything internal in Consul to use the service. If that changes + // we could register it on the in-process interfaces as well. + err = s.registerDataplaneServer( + deps, + s.externalGRPCServer, + ) + if err != nil { + return err + } + + // enable grpc server reflection for the external gRPC interface only + reflection.Register(s.externalGRPCServer) + + return s.setupEnterpriseGRPCServices(config, deps) +} + +func (s *Server) registerResourceServiceServer(typeRegistry resource.Registry, resolver resourcegrpc.ACLResolver, registrars ...grpc.ServiceRegistrar) error { + if s.storageBackend == nil { + return fmt.Errorf("storage backend cannot be nil") + } + + var tenancyBridge resourcegrpc.TenancyBridge + if s.useV2Tenancy { + tenancyBridge = tenancy.NewV2TenancyBridge().WithClient( + // This assumes that the resource service will be registered with + // the insecureUnsafeGRPCChan. We are using the insecure and unsafe + // channel here because the V2 Tenancy bridge only reads data + // from the client and does not modify it. Therefore sharing memory + // with the resource services canonical immutable data is advantageous + // to prevent wasting CPU time for every resource op to clone things. + pbresource.NewResourceServiceClient(s.insecureUnsafeGRPCChan), + ) + } else { + tenancyBridge = NewV1TenancyBridge(s) + } + + // Create the Resource Service Server + srv := resourcegrpc.NewServer(resourcegrpc.Config{ + Registry: typeRegistry, + Backend: s.storageBackend, + ACLResolver: resolver, + Logger: s.loggers.Named(logging.GRPCAPI).Named(logging.Resource), + TenancyBridge: tenancyBridge, + UseV2Tenancy: s.useV2Tenancy, + }) + + // Register the server to all the desired interfaces + for _, reg := range registrars { + pbresource.RegisterResourceServiceServer(reg, srv) + } + return nil +} + +func (s *Server) registerACLServer(registrars ...grpc.ServiceRegistrar) error { + srv := aclgrpc.NewServer(aclgrpc.Config{ + ACLsEnabled: s.config.ACLsEnabled, + ForwardRPC: func(info structs.RPCInfo, fn func(*grpc.ClientConn) error) (bool, error) { + return s.ForwardGRPC(s.grpcConnPool, info, fn) + }, + InPrimaryDatacenter: s.InPrimaryDatacenter(), + LoadAuthMethod: func(methodName string, entMeta *acl.EnterpriseMeta) (*structs.ACLAuthMethod, aclgrpc.Validator, error) { + return s.loadAuthMethod(methodName, entMeta) + }, + LocalTokensEnabled: s.LocalTokensEnabled, + Logger: s.loggers.Named(logging.GRPCAPI).Named(logging.ACL), + NewLogin: func() aclgrpc.Login { return s.aclLogin() }, + NewTokenWriter: func() aclgrpc.TokenWriter { return s.aclTokenWriter() }, + PrimaryDatacenter: s.config.PrimaryDatacenter, + ValidateEnterpriseRequest: s.validateEnterpriseRequest, + }) + + for _, reg := range registrars { + srv.Register(reg) + } + + return nil +} + +func (s *Server) registerPeerStreamServer(config *Config, registrars ...grpc.ServiceRegistrar) (*peerstream.Server, error) { + if s.peeringBackend == nil { + panic("peeringBackend is required during construction") + } + + srv := peerstream.NewServer(peerstream.Config{ + Backend: s.peeringBackend, + GetStore: func() peerstream.StateStore { return s.FSM().State() }, + Logger: s.loggers.Named(logging.GRPCAPI).Named(logging.PeerStream), + ACLResolver: s.ACLResolver, + Datacenter: s.config.Datacenter, + ConnectEnabled: s.config.ConnectEnabled, + ForwardRPC: func(info structs.RPCInfo, fn func(*grpc.ClientConn) error) (bool, error) { + // Only forward the request if the dc in the request matches the server's datacenter. + if info.RequestDatacenter() != "" && info.RequestDatacenter() != config.Datacenter { + return false, fmt.Errorf("requests to generate peering tokens cannot be forwarded to remote datacenters") + } + return s.ForwardGRPC(s.grpcConnPool, info, fn) + }, + }) + + for _, reg := range registrars { + srv.Register(reg) + } + + return srv, nil +} + +func (s *Server) registerPeeringServer(config *Config, registrars ...grpc.ServiceRegistrar) error { + if s.peeringBackend == nil { + panic("peeringBackend is required during construction") + } + + if s.peerStreamServer == nil { + panic("the peer stream server must be configured before the peering server") + } + + srv := peering.NewServer(peering.Config{ + Backend: s.peeringBackend, + Tracker: s.peerStreamServer.Tracker, + Logger: s.loggers.Named(logging.GRPCAPI).Named(logging.Peering), + ForwardRPC: func(info structs.RPCInfo, fn func(*grpc.ClientConn) error) (bool, error) { + // Only forward the request if the dc in the request matches the server's datacenter. + if info.RequestDatacenter() != "" && info.RequestDatacenter() != config.Datacenter { + return false, fmt.Errorf("requests to generate peering tokens cannot be forwarded to remote datacenters") + } + return s.ForwardGRPC(s.grpcConnPool, info, fn) + }, + Datacenter: config.Datacenter, + ConnectEnabled: config.ConnectEnabled, + PeeringEnabled: config.PeeringEnabled, + Locality: config.Locality, + FSMServer: s, + }) + + for _, reg := range registrars { + srv.Register(reg) + } + + return nil +} + +func (s *Server) registerOperatorServer(config *Config, deps Deps, registrars ...grpc.ServiceRegistrar) error { + srv := operator.NewServer(operator.Config{ + Backend: NewOperatorBackend(s), + Logger: deps.Logger.Named("grpc-api.operator"), + ForwardRPC: func(info structs.RPCInfo, fn func(*grpc.ClientConn) error) (bool, error) { + // Only forward the request if the dc in the request matches the server's datacenter. + if info.RequestDatacenter() != "" && info.RequestDatacenter() != config.Datacenter { + return false, fmt.Errorf("requests to transfer leader cannot be forwarded to remote datacenters") + } + return s.ForwardGRPC(s.grpcConnPool, info, fn) + }, + Datacenter: config.Datacenter, + }) + + for _, reg := range registrars { + srv.Register(reg) + } + + return nil +} + +func (s *Server) registerStreamSubscriptionServer(deps Deps, registrars ...grpc.ServiceRegistrar) error { + srv := subscribe.NewServer( + &subscribeBackend{srv: s, connPool: deps.GRPCConnPool}, + s.loggers.Named(logging.GRPCAPI).Named("subscription"), + ) + + for _, reg := range registrars { + pbsubscribe.RegisterStateChangeSubscriptionServer(reg, srv) + } + + return nil +} + +func (s *Server) registerConnectCAServer(registrars ...grpc.ServiceRegistrar) error { + srv := connectca.NewServer(connectca.Config{ + Publisher: s.publisher, + GetStore: func() connectca.StateStore { return s.FSM().State() }, + Logger: s.loggers.Named(logging.GRPCAPI).Named(logging.ConnectCA), + ACLResolver: s.ACLResolver, + CAManager: s.caManager, + ForwardRPC: func(info structs.RPCInfo, fn func(*grpc.ClientConn) error) (bool, error) { + return s.ForwardGRPC(s.grpcConnPool, info, fn) + }, + ConnectEnabled: s.config.ConnectEnabled, + }) + + for _, reg := range registrars { + srv.Register(reg) + } + + return nil +} + +func (s *Server) registerDataplaneServer(deps Deps, registrars ...grpc.ServiceRegistrar) error { + srv := dataplane.NewServer(dataplane.Config{ + GetStore: func() dataplane.StateStore { return s.FSM().State() }, + Logger: s.loggers.Named(logging.GRPCAPI).Named(logging.Dataplane), + ACLResolver: s.ACLResolver, + Datacenter: s.config.Datacenter, + EnableV2: stringslice.Contains(deps.Experiments, CatalogResourceExperimentName), + ResourceAPIClient: pbresource.NewResourceServiceClient(s.insecureSafeGRPCChan), + }) + + for _, reg := range registrars { + srv.Register(reg) + } + + return nil +} + +func (s *Server) registerServerDiscoveryServer(resolver serverdiscovery.ACLResolver, registrars ...grpc.ServiceRegistrar) error { + srv := serverdiscovery.NewServer(serverdiscovery.Config{ + Publisher: s.publisher, + ACLResolver: resolver, + Logger: s.loggers.Named(logging.GRPCAPI).Named(logging.ServerDiscovery), + }) + + for _, reg := range registrars { + srv.Register(reg) + } + + return nil +} diff --git a/agent/grpc-external/services/acl/server.go b/agent/grpc-external/services/acl/server.go index 2393f11aa1..cc7e35d1e6 100644 --- a/agent/grpc-external/services/acl/server.go +++ b/agent/grpc-external/services/acl/server.go @@ -53,8 +53,8 @@ func NewServer(cfg Config) *Server { return &Server{cfg} } -func (s *Server) Register(grpcServer *grpc.Server) { - pbacl.RegisterACLServiceServer(grpcServer, s) +func (s *Server) Register(registrar grpc.ServiceRegistrar) { + pbacl.RegisterACLServiceServer(registrar, s) } func (s *Server) requireACLsEnabled(logger hclog.Logger) error { diff --git a/agent/grpc-external/services/connectca/server.go b/agent/grpc-external/services/connectca/server.go index 4ef91705f7..fbdabc1bd8 100644 --- a/agent/grpc-external/services/connectca/server.go +++ b/agent/grpc-external/services/connectca/server.go @@ -57,8 +57,8 @@ func NewServer(cfg Config) *Server { return &Server{cfg} } -func (s *Server) Register(grpcServer *grpc.Server) { - pbconnectca.RegisterConnectCAServiceServer(grpcServer, s) +func (s *Server) Register(registrar grpc.ServiceRegistrar) { + pbconnectca.RegisterConnectCAServiceServer(registrar, s) } func (s *Server) requireConnect() error { diff --git a/agent/grpc-external/services/dataplane/server.go b/agent/grpc-external/services/dataplane/server.go index fbce372f0e..3a1809cc04 100644 --- a/agent/grpc-external/services/dataplane/server.go +++ b/agent/grpc-external/services/dataplane/server.go @@ -49,6 +49,6 @@ func NewServer(cfg Config) *Server { var _ pbdataplane.DataplaneServiceServer = (*Server)(nil) -func (s *Server) Register(grpcServer *grpc.Server) { - pbdataplane.RegisterDataplaneServiceServer(grpcServer, s) +func (s *Server) Register(registrar grpc.ServiceRegistrar) { + pbdataplane.RegisterDataplaneServiceServer(registrar, s) } diff --git a/agent/grpc-external/services/dns/server.go b/agent/grpc-external/services/dns/server.go index a6f2249733..3485bd2f13 100644 --- a/agent/grpc-external/services/dns/server.go +++ b/agent/grpc-external/services/dns/server.go @@ -37,8 +37,8 @@ func NewServer(cfg Config) *Server { return &Server{cfg} } -func (s *Server) Register(grpcServer *grpc.Server) { - pbdns.RegisterDNSServiceServer(grpcServer, s) +func (s *Server) Register(registrar grpc.ServiceRegistrar) { + pbdns.RegisterDNSServiceServer(registrar, s) } // BufferResponseWriter writes a DNS response to a byte buffer. diff --git a/agent/grpc-external/services/peerstream/server.go b/agent/grpc-external/services/peerstream/server.go index 4127312fe7..6cd32c9287 100644 --- a/agent/grpc-external/services/peerstream/server.go +++ b/agent/grpc-external/services/peerstream/server.go @@ -83,8 +83,8 @@ func requireNotNil(v interface{}, name string) { var _ pbpeerstream.PeerStreamServiceServer = (*Server)(nil) -func (s *Server) Register(grpcServer *grpc.Server) { - pbpeerstream.RegisterPeerStreamServiceServer(grpcServer, s) +func (s *Server) Register(registrar grpc.ServiceRegistrar) { + pbpeerstream.RegisterPeerStreamServiceServer(registrar, s) } type Backend interface { diff --git a/agent/grpc-external/services/peerstream/subscription_manager.go b/agent/grpc-external/services/peerstream/subscription_manager.go index bd2d36ffd9..92be1bb6b0 100644 --- a/agent/grpc-external/services/peerstream/subscription_manager.go +++ b/agent/grpc-external/services/peerstream/subscription_manager.go @@ -882,6 +882,10 @@ func (m *subscriptionManager) subscribeServerAddrs( idx uint64, updateCh chan<- cache.UpdateEvent, ) (uint64, error) { + // TODO(inproc-grpc) - Look into using the insecure in-process gRPC Channel + // to get notified for server address updates instead of hooking into the + // subscription service. + // following code adapted from serverdiscovery/watch_servers.go sub, err := m.backend.Subscribe(&stream.SubscribeRequest{ Topic: autopilotevents.EventTopicReadyServers, diff --git a/agent/grpc-external/services/resource/server.go b/agent/grpc-external/services/resource/server.go index 99af640f84..5e42374919 100644 --- a/agent/grpc-external/services/resource/server.go +++ b/agent/grpc-external/services/resource/server.go @@ -72,8 +72,8 @@ func NewServer(cfg Config) *Server { var _ pbresource.ResourceServiceServer = (*Server)(nil) -func (s *Server) Register(grpcServer *grpc.Server) { - pbresource.RegisterResourceServiceServer(grpcServer, s) +func (s *Server) Register(registrar grpc.ServiceRegistrar) { + pbresource.RegisterResourceServiceServer(registrar, s) } // Get token from grpc metadata or AnonymounsTokenId if not found diff --git a/agent/grpc-external/services/resource/testing/builder.go b/agent/grpc-external/services/resource/testing/builder.go index 96c8379c0e..9fb6146f20 100644 --- a/agent/grpc-external/services/resource/testing/builder.go +++ b/agent/grpc-external/services/resource/testing/builder.go @@ -5,17 +5,14 @@ package testing import ( "context" - "testing" + "github.com/fullstorydev/grpchan/inprocgrpc" "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" - "google.golang.org/grpc" - "google.golang.org/grpc/credentials/insecure" "github.com/hashicorp/consul/acl" svc "github.com/hashicorp/consul/agent/grpc-external/services/resource" "github.com/hashicorp/consul/agent/grpc-external/testutils" - internal "github.com/hashicorp/consul/agent/grpc-internal" "github.com/hashicorp/consul/internal/resource" "github.com/hashicorp/consul/internal/storage/inmem" "github.com/hashicorp/consul/internal/tenancy" @@ -23,26 +20,28 @@ import ( "github.com/hashicorp/consul/sdk/testutil" ) -type builder struct { +type Builder struct { registry resource.Registry registerFns []func(resource.Registry) useV2Tenancy bool tenancies []*pbresource.Tenancy aclResolver svc.ACLResolver serviceImpl *svc.Server + cloning bool } // NewResourceServiceBuilder is the preferred way to configure and run // an isolated in-process instance of the resource service for unit // testing. The final call to `Run()` returns a client you can use for // making requests. -func NewResourceServiceBuilder() *builder { - b := &builder{ +func NewResourceServiceBuilder() *Builder { + b := &Builder{ useV2Tenancy: false, registry: resource.NewRegistry(), // Regardless of whether using mock of v2tenancy, always make sure // the builtin tenancy exists. tenancies: []*pbresource.Tenancy{resource.DefaultNamespacedTenancy()}, + cloning: true, } return b } @@ -51,47 +50,63 @@ func NewResourceServiceBuilder() *builder { // // true => real v2 default partition and namespace via v2 tenancy bridge // false => mock default partition and namespace since v1 tenancy bridge can't be used (not spinning up an entire server here) -func (b *builder) WithV2Tenancy(useV2Tenancy bool) *builder { +func (b *Builder) WithV2Tenancy(useV2Tenancy bool) *Builder { b.useV2Tenancy = useV2Tenancy return b } // Registry provides access to the constructed registry post-Run() when // needed by other test dependencies. -func (b *builder) Registry() resource.Registry { +func (b *Builder) Registry() resource.Registry { return b.registry } // ServiceImpl provides access to the actual server side implemenation of the resource service. This should never be used // used/accessed without good reason. The current justifying use case is to monkeypatch the ACL resolver post-creation // to allow unfettered writes which some ACL related tests require to put test data in place. -func (b *builder) ServiceImpl() *svc.Server { +func (b *Builder) ServiceImpl() *svc.Server { return b.serviceImpl } -func (b *builder) WithRegisterFns(registerFns ...func(resource.Registry)) *builder { +func (b *Builder) WithRegisterFns(registerFns ...func(resource.Registry)) *Builder { for _, registerFn := range registerFns { b.registerFns = append(b.registerFns, registerFn) } return b } -func (b *builder) WithACLResolver(aclResolver svc.ACLResolver) *builder { +func (b *Builder) WithACLResolver(aclResolver svc.ACLResolver) *Builder { b.aclResolver = aclResolver return b } // WithTenancies adds additional partitions and namespaces if default/default // is not sufficient. -func (b *builder) WithTenancies(tenancies ...*pbresource.Tenancy) *builder { +func (b *Builder) WithTenancies(tenancies ...*pbresource.Tenancy) *Builder { for _, tenancy := range tenancies { b.tenancies = append(b.tenancies, tenancy) } return b } +// WithCloningDisabled disables resource service client functionality that will +// clone protobuf message types as they pass through. By default +// cloning is enabled. +// +// For in-process gRPC interactions we prefer to use an in-memory gRPC client. This +// allows our controller infrastructure to avoid any unnecessary protobuf serialization +// and deserialization and for controller caching to not duplicate memory that the +// resource service is already holding on to. However, clients (including controllers) +// often want to be able to perform read-modify-write ops and for the sake of not +// forcing all call sites to be aware of the shared memory and to not touch it we +// enable cloning in the clients that we give to those bits of code. +func (b *Builder) WithCloningDisabled() *Builder { + b.cloning = false + return b +} + // Run starts the resource service and returns a client. -func (b *builder) Run(t *testing.T) pbresource.ResourceServiceClient { +func (b *Builder) Run(t testutil.TestingTB) pbresource.ResourceServiceClient { // backend cannot be customized backend, err := inmem.NewBackend() require.NoError(t, err) @@ -161,24 +176,15 @@ func (b *builder) Run(t *testing.T) pbresource.ResourceServiceClient { UseV2Tenancy: b.useV2Tenancy, } - server := grpc.NewServer() - b.serviceImpl = svc.NewServer(config) - b.serviceImpl.Register(server) + ch := &inprocgrpc.Channel{} + pbresource.RegisterResourceServiceServer(ch, b.serviceImpl) + client := pbresource.NewResourceServiceClient(ch) - pipe := internal.NewPipeListener() - go server.Serve(pipe) - t.Cleanup(server.Stop) - - conn, err := grpc.Dial("", - grpc.WithTransportCredentials(insecure.NewCredentials()), - grpc.WithContextDialer(pipe.DialContext), - grpc.WithBlock(), - ) - require.NoError(t, err) - t.Cleanup(func() { _ = conn.Close() }) - - client := pbresource.NewResourceServiceClient(conn) + if b.cloning { + // enable protobuf cloning wrapper + client = pbresource.NewCloningResourceServiceClient(client) + } // HACK ALERT: The client needs to be injected into the V2TenancyBridge // after it has been created due the the circular dependency. This will diff --git a/agent/grpc-external/services/serverdiscovery/server.go b/agent/grpc-external/services/serverdiscovery/server.go index 99011c6d07..805b95c378 100644 --- a/agent/grpc-external/services/serverdiscovery/server.go +++ b/agent/grpc-external/services/serverdiscovery/server.go @@ -37,6 +37,6 @@ func NewServer(cfg Config) *Server { return &Server{cfg} } -func (s *Server) Register(grpcServer *grpc.Server) { - pbserverdiscovery.RegisterServerDiscoveryServiceServer(grpcServer, s) +func (s *Server) Register(registrar grpc.ServiceRegistrar) { + pbserverdiscovery.RegisterServerDiscoveryServiceServer(registrar, s) } diff --git a/agent/grpc-external/testutils/acl.go b/agent/grpc-external/testutils/acl.go index 72e0897e71..ea4294e657 100644 --- a/agent/grpc-external/testutils/acl.go +++ b/agent/grpc-external/testutils/acl.go @@ -4,8 +4,6 @@ package testutils import ( - "testing" - "github.com/stretchr/testify/require" "github.com/hashicorp/go-uuid" @@ -13,9 +11,10 @@ import ( "github.com/hashicorp/consul/acl" "github.com/hashicorp/consul/acl/resolver" "github.com/hashicorp/consul/agent/structs" + "github.com/hashicorp/consul/sdk/testutil" ) -func ACLAnonymous(t *testing.T) resolver.Result { +func ACLAnonymous(t testutil.TestingTB) resolver.Result { t.Helper() return resolver.Result{ @@ -26,7 +25,7 @@ func ACLAnonymous(t *testing.T) resolver.Result { } } -func ACLsDisabled(t *testing.T) resolver.Result { +func ACLsDisabled(t testutil.TestingTB) resolver.Result { t.Helper() return resolver.Result{ @@ -34,7 +33,7 @@ func ACLsDisabled(t *testing.T) resolver.Result { } } -func ACLNoPermissions(t *testing.T) resolver.Result { +func ACLNoPermissions(t testutil.TestingTB) resolver.Result { t.Helper() return resolver.Result{ @@ -43,7 +42,7 @@ func ACLNoPermissions(t *testing.T) resolver.Result { } } -func ACLServiceWriteAny(t *testing.T) resolver.Result { +func ACLServiceWriteAny(t testutil.TestingTB) resolver.Result { t.Helper() policy, err := acl.NewPolicyFromSource(` @@ -62,7 +61,7 @@ func ACLServiceWriteAny(t *testing.T) resolver.Result { } } -func ACLServiceRead(t *testing.T, serviceName string) resolver.Result { +func ACLServiceRead(t testutil.TestingTB, serviceName string) resolver.Result { t.Helper() aclRule := &acl.Policy{ @@ -84,7 +83,7 @@ func ACLServiceRead(t *testing.T, serviceName string) resolver.Result { } } -func ACLUseProvidedPolicy(t *testing.T, aclPolicy *acl.Policy) resolver.Result { +func ACLUseProvidedPolicy(t testutil.TestingTB, aclPolicy *acl.Policy) resolver.Result { t.Helper() authz, err := acl.NewPolicyAuthorizerWithDefaults(acl.DenyAll(), []*acl.Policy{aclPolicy}, nil) @@ -96,7 +95,7 @@ func ACLUseProvidedPolicy(t *testing.T, aclPolicy *acl.Policy) resolver.Result { } } -func ACLOperatorRead(t *testing.T) resolver.Result { +func ACLOperatorRead(t testutil.TestingTB) resolver.Result { t.Helper() aclRule := &acl.Policy{ @@ -113,7 +112,7 @@ func ACLOperatorRead(t *testing.T) resolver.Result { } } -func ACLOperatorWrite(t *testing.T) resolver.Result { +func ACLOperatorWrite(t testutil.TestingTB) resolver.Result { t.Helper() aclRule := &acl.Policy{ @@ -130,7 +129,7 @@ func ACLOperatorWrite(t *testing.T) resolver.Result { } } -func randomACLIdentity(t *testing.T) structs.ACLIdentity { +func randomACLIdentity(t testutil.TestingTB) structs.ACLIdentity { id, err := uuid.GenerateUUID() require.NoError(t, err) diff --git a/agent/grpc-external/testutils/server.go b/agent/grpc-external/testutils/server.go index 13cbb985e5..8613bec65c 100644 --- a/agent/grpc-external/testutils/server.go +++ b/agent/grpc-external/testutils/server.go @@ -12,7 +12,7 @@ import ( ) type GRPCService interface { - Register(*grpc.Server) + Register(grpc.ServiceRegistrar) } func RunTestServer(t *testing.T, services ...GRPCService) net.Addr { diff --git a/agent/grpc-internal/handler.go b/agent/grpc-internal/handler.go index b0eeaa8a4f..3cda5087be 100644 --- a/agent/grpc-internal/handler.go +++ b/agent/grpc-internal/handler.go @@ -28,9 +28,7 @@ var ( ) // NewHandler returns a gRPC server that accepts connections from Handle(conn). -// The register function will be called with the grpc.Server to register -// gRPC services with the server. -func NewHandler(logger Logger, addr net.Addr, register func(server *grpc.Server), metricsObj *metrics.Metrics, rateLimiter rate.RequestLimitsHandler) *Handler { +func NewHandler(logger Logger, addr net.Addr, metricsObj *metrics.Metrics, rateLimiter rate.RequestLimitsHandler) *Handler { if metricsObj == nil { metricsObj = metrics.Default() } @@ -59,7 +57,6 @@ func NewHandler(logger Logger, addr net.Addr, register func(server *grpc.Server) // We don't need to pass tls.Config to the server since it's multiplexed // behind the RPC listener, which already has TLS configured. srv := grpc.NewServer(opts...) - register(srv) return &Handler{srv: srv, listener: NewListener(addr)} } @@ -80,6 +77,12 @@ func (h *Handler) Run() error { return h.srv.Serve(h.listener) } +// Implements the grpc.ServiceRegistrar interface to allow registering services +// with the Handler. +func (h *Handler) RegisterService(svc *grpc.ServiceDesc, impl any) { + h.srv.RegisterService(svc, impl) +} + func (h *Handler) Shutdown() error { h.srv.Stop() return nil diff --git a/agent/grpc-internal/server_test.go b/agent/grpc-internal/server_test.go index 12f420979b..65b43580a1 100644 --- a/agent/grpc-internal/server_test.go +++ b/agent/grpc-internal/server_test.go @@ -44,21 +44,22 @@ func (s testServer) Metadata() *metadata.Server { } func newSimpleTestServer(t *testing.T, name, dc string, tlsConf *tlsutil.Configurator) testServer { - return newTestServer(t, hclog.Default(), name, dc, tlsConf, func(server *grpc.Server) { + return newTestServer(t, hclog.Default(), name, dc, tlsConf, func(server grpc.ServiceRegistrar) { testservice.RegisterSimpleServer(server, &testservice.Simple{Name: name, DC: dc}) }) } // newPanicTestServer sets up a simple server with handlers that panic. func newPanicTestServer(t *testing.T, logger hclog.Logger, name, dc string, tlsConf *tlsutil.Configurator) testServer { - return newTestServer(t, logger, name, dc, tlsConf, func(server *grpc.Server) { + return newTestServer(t, logger, name, dc, tlsConf, func(server grpc.ServiceRegistrar) { testservice.RegisterSimpleServer(server, &testservice.SimplePanic{Name: name, DC: dc}) }) } -func newTestServer(t *testing.T, logger hclog.Logger, name, dc string, tlsConf *tlsutil.Configurator, register func(server *grpc.Server)) testServer { +func newTestServer(t *testing.T, logger hclog.Logger, name, dc string, tlsConf *tlsutil.Configurator, register func(server grpc.ServiceRegistrar)) testServer { addr := &net.IPAddr{IP: net.ParseIP("127.0.0.1")} - handler := NewHandler(logger, addr, register, nil, rate.NullRequestLimitsHandler()) + handler := NewHandler(logger, addr, nil, rate.NullRequestLimitsHandler()) + register(handler) lis, err := net.Listen("tcp", "127.0.0.1:0") require.NoError(t, err) diff --git a/agent/grpc-internal/services/subscribe/subscribe_test.go b/agent/grpc-internal/services/subscribe/subscribe_test.go index 910862d4cf..9f6b550cc9 100644 --- a/agent/grpc-internal/services/subscribe/subscribe_test.go +++ b/agent/grpc-internal/services/subscribe/subscribe_test.go @@ -377,36 +377,62 @@ func newTestBackend(t *testing.T) *testBackend { var _ Backend = (*testBackend)(nil) func runTestServer(t *testing.T, server *Server) net.Addr { - addr := &net.IPAddr{IP: net.ParseIP("127.0.0.1")} - var grpcServer *gogrpc.Server - handler := grpc.NewHandler( - hclog.New(nil), - addr, - func(srv *gogrpc.Server) { - grpcServer = srv - pbsubscribe.RegisterStateChangeSubscriptionServer(srv, server) - }, - nil, - rate.NullRequestLimitsHandler(), - ) - - lis, err := net.Listen("tcp", "127.0.0.1:0") - require.NoError(t, err) - t.Cleanup(logError(t, lis.Close)) - - go grpcServer.Serve(lis) + // create the errgroup and register its cleanup. Its cleanup needs to occur + // after all others and that is why this is being done so early on in this function + // as cleanup routines are processed in reverse order of them being added. g := new(errgroup.Group) - g.Go(func() error { - return grpcServer.Serve(lis) - }) + // this cleanup needs to happen after others defined in this func so we do it early + // on up here. t.Cleanup(func() { - if err := handler.Shutdown(); err != nil { - t.Logf("grpc server shutdown: %v", err) - } if err := g.Wait(); err != nil { t.Logf("grpc server error: %v", err) } }) + + // start the handler + addr := &net.IPAddr{IP: net.ParseIP("127.0.0.1")} + handler := grpc.NewHandler( + hclog.New(nil), + addr, + nil, + rate.NullRequestLimitsHandler(), + ) + pbsubscribe.RegisterStateChangeSubscriptionServer(handler, server) + g.Go(handler.Run) + t.Cleanup(func() { + if err := handler.Shutdown(); err != nil { + t.Logf("grpc server shutdown: %v", err) + } + }) + + // create the routing to forward network conns to the gRPC handler + lis, err := net.Listen("tcp", "127.0.0.1:0") + require.NoError(t, err) + g.Go(func() error { + for { + // select { + // case <-ctx.Done(): + // return ctx.Err() + // default: + // } + + conn, err := lis.Accept() + if err != nil { + return err + } + + // select { + // case <-ctx.Done(): + // return ctx.Err() + // default: + // } + + handler.Handle(conn) + } + }) + // closing the listener should cause the Accept to unblock and error out + t.Cleanup(logError(t, lis.Close)) + return lis.Addr() } diff --git a/agent/grpc-internal/stats_test.go b/agent/grpc-internal/stats_test.go index d14c4c46bc..e08e869dea 100644 --- a/agent/grpc-internal/stats_test.go +++ b/agent/grpc-internal/stats_test.go @@ -23,15 +23,13 @@ import ( "github.com/hashicorp/consul/proto/private/prototest" ) -func noopRegister(*grpc.Server) {} - func TestHandler_EmitsStats(t *testing.T) { sink, metricsObj := testutil.NewFakeSink(t) addr := &net.IPAddr{IP: net.ParseIP("127.0.0.1")} - handler := NewHandler(hclog.Default(), addr, noopRegister, metricsObj, rate.NullRequestLimitsHandler()) + handler := NewHandler(hclog.Default(), addr, metricsObj, rate.NullRequestLimitsHandler()) - testservice.RegisterSimpleServer(handler.srv, &testservice.Simple{}) + testservice.RegisterSimpleServer(handler, &testservice.Simple{}) lis, err := net.Listen("tcp", "127.0.0.1:0") require.NoError(t, err) diff --git a/agent/grpc-middleware/stats.go b/agent/grpc-middleware/stats.go index a6bf1d2c59..b17fe71385 100644 --- a/agent/grpc-middleware/stats.go +++ b/agent/grpc-middleware/stats.go @@ -107,6 +107,13 @@ func (c *statsHandler) HandleConn(_ context.Context, s stats.ConnStats) { c.metrics.SetGaugeWithLabels([]string{"grpc", label, "connections"}, float32(count), c.labels) } +// Intercept matches the Unary interceptor function signature. This unary interceptor will count RPC requests +// but does not handle any connection processing or perform RPC "tagging" +func (c *statsHandler) Intercept(ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (resp interface{}, err error) { + c.metrics.IncrCounterWithLabels([]string{"grpc", "server", "request", "count"}, 1, c.labels) + return handler(ctx, req) +} + type activeStreamCounter struct { // count is used with sync/atomic and MUST be 64-bit aligned. To ensure // alignment on 32-bit platforms this field must remain the first field in diff --git a/agent/rpc/operator/service.go b/agent/rpc/operator/service.go index 697b0db254..ec88c5aa04 100644 --- a/agent/rpc/operator/service.go +++ b/agent/rpc/operator/service.go @@ -94,8 +94,8 @@ func requireNotNil(v interface{}, name string) { var _ pboperator.OperatorServiceServer = (*Server)(nil) -func (s *Server) Register(grpcServer *grpc.Server) { - pboperator.RegisterOperatorServiceServer(grpcServer, s) +func (s *Server) Register(registrar grpc.ServiceRegistrar) { + pboperator.RegisterOperatorServiceServer(registrar, s) } // Backend defines the core integrations the Operator endpoint depends on. A diff --git a/agent/rpc/peering/service.go b/agent/rpc/peering/service.go index d57e0378f9..7f59c2941c 100644 --- a/agent/rpc/peering/service.go +++ b/agent/rpc/peering/service.go @@ -121,8 +121,8 @@ func requireNotNil(v interface{}, name string) { var _ pbpeering.PeeringServiceServer = (*Server)(nil) -func (s *Server) Register(grpcServer *grpc.Server) { - pbpeering.RegisterPeeringServiceServer(grpcServer, s) +func (s *Server) Register(registrar grpc.ServiceRegistrar) { + pbpeering.RegisterPeeringServiceServer(registrar, s) } // Backend defines the core integrations the Peering endpoint depends on. A diff --git a/build-support/scripts/devtools.sh b/build-support/scripts/devtools.sh index c9a887c0ec..3dde6b7ea1 100755 --- a/build-support/scripts/devtools.sh +++ b/build-support/scripts/devtools.sh @@ -132,6 +132,8 @@ function proto_tools_install { install_local_protoc_generator "${SOURCE_DIR}/internal/tools/protoc-gen-consul-rate-limit" install_local_protoc_generator "${SOURCE_DIR}/internal/resource/protoc-gen-resource-types" + + install_local_protoc_generator "${SOURCE_DIR}/internal/tools/protoc-gen-grpc-clone" install_local_protoc_generator "${SOURCE_DIR}/internal/resource/protoc-gen-json-shim" diff --git a/go.mod b/go.mod index 7957ea0e38..89e12f40fa 100644 --- a/go.mod +++ b/go.mod @@ -30,6 +30,7 @@ require ( github.com/envoyproxy/go-control-plane/xdsmatcher v0.0.0-20230524161521-aaaacbfbe53e github.com/fatih/color v1.14.1 github.com/fsnotify/fsnotify v1.6.0 + github.com/fullstorydev/grpchan v1.1.1 github.com/go-openapi/runtime v0.26.0 github.com/go-openapi/strfmt v0.21.7 github.com/google/go-cmp v0.5.9 @@ -209,6 +210,7 @@ require ( github.com/hashicorp/net-rpc-msgpackrpc/v2 v2.0.0 // indirect github.com/hashicorp/vic v1.5.1-0.20190403131502-bbfe86ec9443 // indirect github.com/huandu/xstrings v1.3.2 // indirect + github.com/jhump/protoreflect v1.11.0 // indirect github.com/jmespath/go-jmespath v0.4.0 // indirect github.com/josharian/intern v1.0.0 // indirect github.com/joyent/triton-go v1.7.1-0.20200416154420-6801d15b779f // indirect diff --git a/go.sum b/go.sum index 3d851eeb29..e02a3307f0 100644 --- a/go.sum +++ b/go.sum @@ -263,6 +263,8 @@ github.com/frankban/quicktest v1.14.3 h1:FJKSZTDHjyhriyC81FLQ0LY93eSai0ZyR/ZIkd3 github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= github.com/fsnotify/fsnotify v1.6.0 h1:n+5WquG0fcWoWp6xPWfHdbskMCQaFnG6PfBrh1Ky4HY= github.com/fsnotify/fsnotify v1.6.0/go.mod h1:sl3t1tCWJFWoRz9R8WJCbQihKKwmorjAbSClcnxKAGw= +github.com/fullstorydev/grpchan v1.1.1 h1:heQqIJlAv5Cnks9a70GRL2EJke6QQoUB25VGR6TZQas= +github.com/fullstorydev/grpchan v1.1.1/go.mod h1:f4HpiV8V6htfY/K44GWV1ESQzHBTq7DinhzqQ95lpgc= github.com/ghodss/yaml v0.0.0-20150909031657-73d445a93680/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= github.com/go-asn1-ber/asn1-ber v1.3.1/go.mod h1:hEBeB/ic+5LoWskz+yKT7vGhhPYkProFKoKdwZRWMe0= @@ -639,8 +641,12 @@ github.com/jackc/pgx v3.3.0+incompatible/go.mod h1:0ZGrqGqkRlliWnWB4zKnWtjbSWbGk github.com/jarcoal/httpmock v0.0.0-20180424175123-9c70cfe4a1da h1:FjHUJJ7oBW4G/9j1KzlHaXL09LyMVM9rupS39lncbXk= github.com/jarcoal/httpmock v0.0.0-20180424175123-9c70cfe4a1da/go.mod h1:ks+b9deReOc7jgqp+e7LuFiCBH6Rm5hL32cLcEAArb4= github.com/jessevdk/go-flags v1.4.0/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= -github.com/jhump/protoreflect v1.6.0 h1:h5jfMVslIg6l29nsMs0D8Wj17RDVdNYti0vDN/PZZoE= +github.com/jhump/gopoet v0.0.0-20190322174617-17282ff210b3/go.mod h1:me9yfT6IJSlOL3FCfrg+L6yzUEZ+5jW6WHt4Sk+UPUI= +github.com/jhump/gopoet v0.1.0/go.mod h1:me9yfT6IJSlOL3FCfrg+L6yzUEZ+5jW6WHt4Sk+UPUI= +github.com/jhump/goprotoc v0.5.0/go.mod h1:VrbvcYrQOrTi3i0Vf+m+oqQWk9l72mjkJCYo7UvLHRQ= github.com/jhump/protoreflect v1.6.0/go.mod h1:eaTn3RZAmMBcV0fifFvlm6VHNz3wSkYyXYWUh7ymB74= +github.com/jhump/protoreflect v1.11.0 h1:bvACHUD1Ua/3VxY4aAMpItKMhhwbimlKFJKsLsVgDjU= +github.com/jhump/protoreflect v1.11.0/go.mod h1:U7aMIjN0NWq9swDP7xDdoMfRHb35uiuTd3Z9nFXJf5E= github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af/go.mod h1:Nht3zPeWKUH0NzdCt2Blrr5ys8VGpn0CEB0cQHVjt7k= github.com/jmespath/go-jmespath v0.3.0/go.mod h1:9QtRXoHjLGCJ5IBSaohpXITPlowMeeYCZ7fLUTSywik= github.com/jmespath/go-jmespath v0.4.0 h1:BEgLn5cpjn8UN1mAw4NjwDrS35OdebyEtFe+9YPoQUg= diff --git a/internal/catalog/catalogtest/run_test.go b/internal/catalog/catalogtest/run_test.go index f6e9610e0b..c5fa7ad39e 100644 --- a/internal/catalog/catalogtest/run_test.go +++ b/internal/catalog/catalogtest/run_test.go @@ -6,14 +6,13 @@ package catalogtest import ( "testing" - svctest "github.com/hashicorp/consul/agent/grpc-external/services/resource/testing" "github.com/hashicorp/consul/internal/catalog" "github.com/hashicorp/consul/internal/catalog/internal/controllers" "github.com/hashicorp/consul/internal/controller" + "github.com/hashicorp/consul/internal/controller/controllertest" "github.com/hashicorp/consul/internal/resource/reaper" rtest "github.com/hashicorp/consul/internal/resource/resourcetest" "github.com/hashicorp/consul/proto-public/pbresource" - "github.com/hashicorp/consul/sdk/testutil" ) var ( @@ -23,24 +22,14 @@ var ( func runInMemResourceServiceAndControllers(t *testing.T, deps controllers.Dependencies) pbresource.ResourceServiceClient { t.Helper() - ctx := testutil.TestContext(t) - - // Create the in-mem resource service - client := svctest.NewResourceServiceBuilder(). - WithRegisterFns(catalog.RegisterTypes). - Run(t) - - // Setup/Run the controller manager - mgr := controller.NewManager(client, testutil.Logger(t)) - catalog.RegisterControllers(mgr, deps) - - // We also depend on the reaper to take care of cleaning up owned health statuses and - // service endpoints so we must enable that controller as well - reaper.RegisterControllers(mgr) - mgr.SetRaftLeader(true) - go mgr.Run(ctx) - - return client + return controllertest.NewControllerTestBuilder(). + WithResourceRegisterFns(catalog.RegisterTypes). + WithControllerRegisterFns( + reaper.RegisterControllers, + func(mgr *controller.Manager) { + catalog.RegisterControllers(mgr, deps) + }, + ).Run(t) } func TestControllers_Integration(t *testing.T) { diff --git a/internal/controller/cache/clone.go b/internal/controller/cache/clone.go new file mode 100644 index 0000000000..056dc0ca1e --- /dev/null +++ b/internal/controller/cache/clone.go @@ -0,0 +1,93 @@ +// Copyright (c) HashiCorp, Inc. +// SPDX-License-Identifier: BUSL-1.1 + +package cache + +import ( + "github.com/hashicorp/consul/internal/protoutil" + "github.com/hashicorp/consul/proto-public/pbresource" +) + +// NewCloningReadOnlyCache will wrap the provided ReadOnlyCache with +// an implementation which clones resources that are read out of the cache. +// +// In order to protect the Cache (and the resource service it shares resources with) +// from accidental user modification, the ReadOnlyCache interface is wrapped with +// the cloning variant. This wrapped/cloning cache is then what is given to +// dependency mappers and controller reconcilers. This allows those methods to +// freely modify data they get back from the cache without having to worry about +// accidental data modifications that could wreck the consistent state storage. +func NewCloningReadOnlyCache(cache ReadOnlyCache) ReadOnlyCache { + return cloningReadOnlyCache{ReadOnlyCache: cache} +} + +type cloningReadOnlyCache struct { + ReadOnlyCache +} + +func (c cloningReadOnlyCache) Get(it *pbresource.Type, indexName string, args ...any) (*pbresource.Resource, error) { + res, err := c.ReadOnlyCache.Get(it, indexName, args...) + if err != nil { + return nil, err + } + + return protoutil.Clone(res), nil +} + +func (c cloningReadOnlyCache) List(it *pbresource.Type, indexName string, args ...any) ([]*pbresource.Resource, error) { + resources, err := c.ReadOnlyCache.List(it, indexName, args...) + if err != nil { + return nil, err + } + + return protoutil.CloneSlice(resources), nil +} + +func (c cloningReadOnlyCache) ListIterator(it *pbresource.Type, indexName string, args ...any) (ResourceIterator, error) { + rit, err := c.ReadOnlyCache.ListIterator(it, indexName, args...) + if err != nil { + return nil, err + } + + return cloningIterator{ResourceIterator: rit}, nil +} + +func (c cloningReadOnlyCache) Parents(it *pbresource.Type, indexName string, args ...any) ([]*pbresource.Resource, error) { + resources, err := c.ReadOnlyCache.Parents(it, indexName, args...) + if err != nil { + return nil, err + } + + return protoutil.CloneSlice(resources), nil +} + +func (c cloningReadOnlyCache) ParentsIterator(it *pbresource.Type, indexName string, args ...any) (ResourceIterator, error) { + rit, err := c.ReadOnlyCache.ParentsIterator(it, indexName, args...) + if err != nil { + return nil, err + } + + return cloningIterator{ResourceIterator: rit}, nil +} + +func (c cloningReadOnlyCache) Query(name string, args ...any) (ResourceIterator, error) { + rit, err := c.ReadOnlyCache.Query(name, args...) + if err != nil { + return nil, err + } + + return cloningIterator{ResourceIterator: rit}, nil +} + +type cloningIterator struct { + ResourceIterator +} + +func (it cloningIterator) Next() *pbresource.Resource { + res := it.ResourceIterator.Next() + if res == nil { + return nil + } + + return protoutil.Clone(res) +} diff --git a/internal/controller/cache/clone_test.go b/internal/controller/cache/clone_test.go new file mode 100644 index 0000000000..cf45178f5c --- /dev/null +++ b/internal/controller/cache/clone_test.go @@ -0,0 +1,214 @@ +// Copyright (c) HashiCorp, Inc. +// SPDX-License-Identifier: BUSL-1.1 + +package cache_test + +import ( + "errors" + "testing" + + "github.com/hashicorp/consul/internal/controller/cache" + "github.com/hashicorp/consul/internal/controller/cache/cachemock" + "github.com/hashicorp/consul/internal/resource/resourcetest" + "github.com/hashicorp/consul/proto-public/pbresource" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" +) + +var ( + injectedError = errors.New("injected") + indexName = "some-index" +) + +type cloningReadOnlyCacheSuite struct { + suite.Suite + + rtype *pbresource.Type + res1 *pbresource.Resource + res2 *pbresource.Resource + + mcache *cachemock.ReadOnlyCache + ccache cache.ReadOnlyCache +} + +func TestReadOnlyCache(t *testing.T) { + suite.Run(t, new(cloningReadOnlyCacheSuite)) +} + +func (suite *cloningReadOnlyCacheSuite) SetupTest() { + suite.rtype = &pbresource.Type{ + Group: "testing", + GroupVersion: "v1", + Kind: "Fake", + } + + suite.res1 = resourcetest.Resource(suite.rtype, "foo").Build() + suite.res2 = resourcetest.Resource(suite.rtype, "bar").Build() + + suite.mcache = cachemock.NewReadOnlyCache(suite.T()) + suite.ccache = cache.NewCloningReadOnlyCache(suite.mcache) +} + +func (suite *cloningReadOnlyCacheSuite) makeMockIterator(resources ...*pbresource.Resource) *cachemock.ResourceIterator { + iter := cachemock.NewResourceIterator(suite.T()) + for _, res := range resources { + iter.EXPECT(). + Next(). + Return(res). + Once() + } + + iter.EXPECT(). + Next(). + Return(nil). + Times(0) + + return iter +} + +func (suite *cloningReadOnlyCacheSuite) requireEqualNotSame(expected, actual *pbresource.Resource) { + require.Equal(suite.T(), expected, actual) + require.NotSame(suite.T(), expected, actual) +} + +func (suite *cloningReadOnlyCacheSuite) TestGet_Ok() { + suite.mcache.EXPECT(). + Get(suite.rtype, indexName, "ok"). + Return(suite.res1, nil) + + actual, err := suite.ccache.Get(suite.rtype, indexName, "ok") + require.NoError(suite.T(), err) + suite.requireEqualNotSame(suite.res1, actual) +} + +func (suite *cloningReadOnlyCacheSuite) TestGet_Error() { + suite.mcache.EXPECT(). + Get(suite.rtype, indexName, "error"). + Return(nil, injectedError) + + actual, err := suite.ccache.Get(suite.rtype, indexName, "error") + require.ErrorIs(suite.T(), err, injectedError) + require.Nil(suite.T(), actual) +} + +func (suite *cloningReadOnlyCacheSuite) TestList_Ok() { + preClone := []*pbresource.Resource{suite.res1, suite.res2} + + suite.mcache.EXPECT(). + List(suite.rtype, indexName, "ok"). + Return(preClone, nil) + + postClone, err := suite.ccache.List(suite.rtype, indexName, "ok") + require.NoError(suite.T(), err) + require.Len(suite.T(), postClone, len(preClone)) + for i, actual := range postClone { + suite.requireEqualNotSame(preClone[i], actual) + } +} + +func (suite *cloningReadOnlyCacheSuite) TestList_Error() { + suite.mcache.EXPECT(). + List(suite.rtype, indexName, "error"). + Return(nil, injectedError) + + actual, err := suite.ccache.List(suite.rtype, indexName, "error") + require.ErrorIs(suite.T(), err, injectedError) + require.Nil(suite.T(), actual) +} + +func (suite *cloningReadOnlyCacheSuite) TestParents_Ok() { + preClone := []*pbresource.Resource{suite.res1, suite.res2} + + suite.mcache.EXPECT(). + Parents(suite.rtype, indexName, "ok"). + Return(preClone, nil) + + postClone, err := suite.ccache.Parents(suite.rtype, indexName, "ok") + require.NoError(suite.T(), err) + require.Len(suite.T(), postClone, len(preClone)) + for i, actual := range postClone { + suite.requireEqualNotSame(preClone[i], actual) + } +} + +func (suite *cloningReadOnlyCacheSuite) TestParents_Error() { + suite.mcache.EXPECT(). + Parents(suite.rtype, indexName, "error"). + Return(nil, injectedError) + + actual, err := suite.ccache.Parents(suite.rtype, indexName, "error") + require.ErrorIs(suite.T(), err, injectedError) + require.Nil(suite.T(), actual) +} + +func (suite *cloningReadOnlyCacheSuite) TestListIterator_Ok() { + suite.mcache.EXPECT(). + ListIterator(suite.rtype, indexName, "ok"). + Return(suite.makeMockIterator(suite.res1, suite.res2), nil) + + iter, err := suite.ccache.ListIterator(suite.rtype, indexName, "ok") + require.NoError(suite.T(), err) + require.NotNil(suite.T(), iter) + + suite.requireEqualNotSame(suite.res1, iter.Next()) + suite.requireEqualNotSame(suite.res2, iter.Next()) + require.Nil(suite.T(), iter.Next()) +} + +func (suite *cloningReadOnlyCacheSuite) TestListIterator_Error() { + suite.mcache.EXPECT(). + ListIterator(suite.rtype, indexName, "error"). + Return(nil, injectedError) + + actual, err := suite.ccache.ListIterator(suite.rtype, indexName, "error") + require.ErrorIs(suite.T(), err, injectedError) + require.Nil(suite.T(), actual) +} + +func (suite *cloningReadOnlyCacheSuite) TestParentsIterator_Ok() { + suite.mcache.EXPECT(). + ParentsIterator(suite.rtype, indexName, "ok"). + Return(suite.makeMockIterator(suite.res1, suite.res2), nil) + + iter, err := suite.ccache.ParentsIterator(suite.rtype, indexName, "ok") + require.NoError(suite.T(), err) + require.NotNil(suite.T(), iter) + + suite.requireEqualNotSame(suite.res1, iter.Next()) + suite.requireEqualNotSame(suite.res2, iter.Next()) + require.Nil(suite.T(), iter.Next()) +} + +func (suite *cloningReadOnlyCacheSuite) TestParentsIterator_Error() { + suite.mcache.EXPECT(). + ParentsIterator(suite.rtype, indexName, "error"). + Return(nil, injectedError) + + actual, err := suite.ccache.ParentsIterator(suite.rtype, indexName, "error") + require.ErrorIs(suite.T(), err, injectedError) + require.Nil(suite.T(), actual) +} + +func (suite *cloningReadOnlyCacheSuite) TestQuery_Ok() { + suite.mcache.EXPECT(). + Query(indexName, "ok"). + Return(suite.makeMockIterator(suite.res1, suite.res2), nil) + + iter, err := suite.ccache.Query(indexName, "ok") + require.NoError(suite.T(), err) + require.NotNil(suite.T(), iter) + + suite.requireEqualNotSame(suite.res1, iter.Next()) + suite.requireEqualNotSame(suite.res2, iter.Next()) + require.Nil(suite.T(), iter.Next()) +} + +func (suite *cloningReadOnlyCacheSuite) TestQuery_Error() { + suite.mcache.EXPECT(). + Query(indexName, "error"). + Return(nil, injectedError) + + actual, err := suite.ccache.Query(indexName, "error") + require.ErrorIs(suite.T(), err, injectedError) + require.Nil(suite.T(), actual) +} diff --git a/internal/controller/controller.go b/internal/controller/controller.go index 5585a0e109..d6ed1671e7 100644 --- a/internal/controller/controller.go +++ b/internal/controller/controller.go @@ -41,8 +41,12 @@ type Controller struct { baseBackoff time.Duration maxBackoff time.Duration logger hclog.Logger + startCb RuntimeCallback + stopCb RuntimeCallback } +type RuntimeCallback func(context.Context, Runtime) + // NewController creates a controller that is setup to watched the managed type. // Extra cache indexes may be provided as well and these indexes will be automatically managed. // Typically, further calls to other builder methods will be needed to fully configure @@ -66,6 +70,21 @@ func NewController(name string, managedType *pbresource.Type, indexes ...*index. } } +// WithNotifyStart registers a callback to be run when the controller is being started. +// This happens prior to watches being started and with a fresh cache. +func (ctl *Controller) WithNotifyStart(start RuntimeCallback) *Controller { + ctl.startCb = start + return ctl +} + +// WithNotifyStop registers a callback to be run when the controller has been stopped. +// This happens after all the watches and mapper/reconcile queues have been stopped. The +// cache will contain everything that was present when we started stopping watches. +func (ctl *Controller) WithNotifyStop(stop RuntimeCallback) *Controller { + ctl.stopCb = stop + return ctl +} + // WithReconciler changes the controller's reconciler. func (ctl *Controller) WithReconciler(reconciler Reconciler) *Controller { if reconciler == nil { diff --git a/internal/controller/controller_test.go b/internal/controller/controller_test.go index db821c2511..548a0d8edb 100644 --- a/internal/controller/controller_test.go +++ b/internal/controller/controller_test.go @@ -7,6 +7,7 @@ import ( "context" "errors" "fmt" + "sync/atomic" "testing" "time" @@ -26,6 +27,7 @@ import ( pbdemov2 "github.com/hashicorp/consul/proto/private/pbdemo/v2" "github.com/hashicorp/consul/proto/private/prototest" "github.com/hashicorp/consul/sdk/testutil" + "github.com/hashicorp/consul/sdk/testutil/retry" ) var injectedError = errors.New("injected error") @@ -214,19 +216,36 @@ func TestController_API(t *testing.T) { }) } +func waitForAtomicBoolValue(t testutil.TestingTB, actual *atomic.Bool, expected bool) { + t.Helper() + retry.Run(t, func(r *retry.R) { + require.Equal(r, expected, actual.Load()) + }) +} + func TestController_Placement(t *testing.T) { t.Parallel() t.Run("singleton", func(t *testing.T) { + var running atomic.Bool + running.Store(false) + rec := newTestReconciler() client := svctest.NewResourceServiceBuilder(). WithRegisterFns(demo.RegisterTypes). + WithCloningDisabled(). Run(t) ctrl := controller. NewController("artist", pbdemov2.ArtistType). WithWatch(pbdemov2.AlbumType, dependency.MapOwner). WithPlacement(controller.PlacementSingleton). + WithNotifyStart(func(context.Context, controller.Runtime) { + running.Store(true) + }). + WithNotifyStop(func(context.Context, controller.Runtime) { + running.Store(false) + }). WithReconciler(rec) mgr := controller.NewManager(client, testutil.Logger(t)) @@ -243,16 +262,21 @@ func TestController_Placement(t *testing.T) { // Become the leader and check the reconciler is called. mgr.SetRaftLeader(true) + waitForAtomicBoolValue(t, &running, true) _, _ = rec.wait(t) // Should not be called after losing leadership. mgr.SetRaftLeader(false) + waitForAtomicBoolValue(t, &running, false) _, err = client.Write(testContext(t), &pbresource.WriteRequest{Resource: res}) require.NoError(t, err) rec.expectNoRequest(t, 500*time.Millisecond) }) t.Run("each server", func(t *testing.T) { + var running atomic.Bool + running.Store(false) + rec := newTestReconciler() client := svctest.NewResourceServiceBuilder(). WithRegisterFns(demo.RegisterTypes). @@ -262,11 +286,15 @@ func TestController_Placement(t *testing.T) { NewController("artist", pbdemov2.ArtistType). WithWatch(pbdemov2.AlbumType, dependency.MapOwner). WithPlacement(controller.PlacementEachServer). + WithNotifyStart(func(context.Context, controller.Runtime) { + running.Store(true) + }). WithReconciler(rec) mgr := controller.NewManager(client, testutil.Logger(t)) mgr.Register(ctrl) go mgr.Run(testContext(t)) + waitForAtomicBoolValue(t, &running, true) res, err := demo.GenerateV2Artist() require.NoError(t, err) diff --git a/internal/controller/controllertest/builder.go b/internal/controller/controllertest/builder.go new file mode 100644 index 0000000000..b5964c4b14 --- /dev/null +++ b/internal/controller/controllertest/builder.go @@ -0,0 +1,96 @@ +// Copyright (c) HashiCorp, Inc. +// SPDX-License-Identifier: BUSL-1.1 + +package controllertest + +import ( + svc "github.com/hashicorp/consul/agent/grpc-external/services/resource" + svctest "github.com/hashicorp/consul/agent/grpc-external/services/resource/testing" + "github.com/hashicorp/consul/internal/controller" + "github.com/hashicorp/consul/internal/resource" + "github.com/hashicorp/consul/proto-public/pbresource" + "github.com/hashicorp/consul/sdk/testutil" +) + +type Builder struct { + serviceBuilder *svctest.Builder + controllerRegisterFns []func(*controller.Manager) +} + +// NewControllerTestBuilder starts to build out out the necessary controller testing +// runtime for lightweight controller integration testing. This will run a single +// in-memory resource service instance and the controller manager. Usage of this +// builder is an easy way to ensure that all the right resource gRPC connections/clients +// get set up appropriately in a manner identical to how they would be on a full +// running server. +func NewControllerTestBuilder() *Builder { + return &Builder{ + // disable cloning because we will enable it after passing the non-cloning variant + // to the controller manager. + serviceBuilder: svctest.NewResourceServiceBuilder().WithCloningDisabled(), + } +} + +// WithV2Tenancy tells the internally created resource service to use V2 tenancy. +func (b *Builder) WithV2Tenancy(useV2Tenancy bool) *Builder { + b.serviceBuilder = b.serviceBuilder.WithV2Tenancy(useV2Tenancy) + return b +} + +// Registry retrieves the resource registry from the internal im-mem resource service. +func (b *Builder) Registry() resource.Registry { + return b.serviceBuilder.Registry() +} + +// WithResourceRegisterFns allows configuring functions to be run to register resource +// types with the internal in-mem resource service for the duration of the test. +func (b *Builder) WithResourceRegisterFns(registerFns ...func(resource.Registry)) *Builder { + b.serviceBuilder = b.serviceBuilder.WithRegisterFns(registerFns...) + return b +} + +// WithControllerRegisterFns allows configuring a set of controllers that should be registered +// with the controller manager and executed during Run. +func (b *Builder) WithControllerRegisterFns(registerFns ...func(*controller.Manager)) *Builder { + for _, registerFn := range registerFns { + b.controllerRegisterFns = append(b.controllerRegisterFns, registerFn) + } + return b +} + +// WithACLResolver is used to provide an ACLResolver implementation to the internal resource service. +func (b *Builder) WithACLResolver(aclResolver svc.ACLResolver) *Builder { + b.serviceBuilder = b.serviceBuilder.WithACLResolver(aclResolver) + return b +} + +// WithTenancies adds additional tenancies if default/default is not sufficient. +func (b *Builder) WithTenancies(tenancies ...*pbresource.Tenancy) *Builder { + b.serviceBuilder = b.serviceBuilder.WithTenancies(tenancies...) + return b +} + +// Run executes both the internal resource service and the controller manager. +// The controller manager gets told it is the Raft leader so all controllers +// will get executed. The resource service, controller manager and all controllers +// will be stopped when the test finishes by registering a cleanup method on +// the test. +func (b *Builder) Run(t testutil.TestingTB) pbresource.ResourceServiceClient { + t.Helper() + + ctx := testutil.TestContext(t) + + client := b.serviceBuilder.Run(t) + + mgr := controller.NewManager(client, testutil.Logger(t)) + for _, register := range b.controllerRegisterFns { + register(mgr) + } + + mgr.SetRaftLeader(true) + go mgr.Run(ctx) + + // auto-clone messages going through the client so that test + // code is free to modify objects in place without cloning themselves. + return pbresource.NewCloningResourceServiceClient(client) +} diff --git a/internal/controller/mem_consistency_test.go b/internal/controller/mem_consistency_test.go new file mode 100644 index 0000000000..61e776e82e --- /dev/null +++ b/internal/controller/mem_consistency_test.go @@ -0,0 +1,343 @@ +// Copyright (c) HashiCorp, Inc. +// SPDX-License-Identifier: BUSL-1.1 + +package controller + +import ( + "context" + "sync" + "testing" + "time" + + mockpbresource "github.com/hashicorp/consul/grpcmocks/proto-public/pbresource" + "github.com/hashicorp/consul/internal/resource" + "github.com/hashicorp/consul/internal/resource/resourcetest" + "github.com/hashicorp/consul/internal/storage" + "github.com/hashicorp/consul/proto-public/pbresource" + "github.com/hashicorp/consul/proto/private/prototest" + "github.com/hashicorp/consul/sdk/testutil" + "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" + "google.golang.org/grpc" +) + +var ( + fakeType = &pbresource.Type{ + Group: "testing", + GroupVersion: "v1", + Kind: "Fake", + } + + fakeV2Type = &pbresource.Type{ + Group: "testing", + GroupVersion: "v2", + Kind: "Fake", + } +) + +type memCheckResult struct { + clientGet *pbresource.Resource + clientGetError error + cacheGet *pbresource.Resource + cacheGetError error +} + +type memCheckReconciler struct { + mu sync.Mutex + closed bool + reconcileCh chan memCheckResult + mapCh chan memCheckResult +} + +func newMemCheckReconciler(t testutil.TestingTB) *memCheckReconciler { + t.Helper() + + r := &memCheckReconciler{ + reconcileCh: make(chan memCheckResult, 10), + mapCh: make(chan memCheckResult, 10), + } + + t.Cleanup(r.Shutdown) + return r +} + +func (r *memCheckReconciler) Shutdown() { + r.mu.Lock() + defer r.mu.Unlock() + r.closed = true + close(r.reconcileCh) + close(r.mapCh) +} + +func (r *memCheckReconciler) requireNotClosed(t testutil.TestingTB) { + t.Helper() + if r.closed { + require.FailNow(t, "the memCheckReconciler has been closed") + } +} + +func (r *memCheckReconciler) checkReconcileResult(t testutil.TestingTB, ctx context.Context, res *pbresource.Resource) { + t.Helper() + r.requireEqualNotSameMemCheckResult(t, ctx, r.reconcileCh, res) +} + +func (r *memCheckReconciler) checkMapResult(t testutil.TestingTB, ctx context.Context, res *pbresource.Resource) { + t.Helper() + r.requireEqualNotSameMemCheckResult(t, ctx, r.mapCh, res) +} + +func (r *memCheckReconciler) requireEqualNotSameMemCheckResult(t testutil.TestingTB, ctx context.Context, ch <-chan memCheckResult, res *pbresource.Resource) { + t.Helper() + + select { + case result := <-ch: + require.NoError(t, result.clientGetError) + require.NoError(t, result.cacheGetError) + // Equal but NotSame means the values are all the same but + // the pointers are different. Note that this probably doesn't + // check that the values within the resource haven't been shallow + // copied but that probably should be checked elsewhere + prototest.AssertDeepEqual(t, res, result.clientGet) + require.NotSame(t, res, result.clientGet) + prototest.AssertDeepEqual(t, res, result.cacheGet) + require.NotSame(t, res, result.cacheGet) + case <-ctx.Done(): + require.Fail(t, "didn't receive mem check result before context cancellation", ctx.Err()) + } +} + +func (r *memCheckReconciler) Reconcile(ctx context.Context, rt Runtime, req Request) error { + r.mu.Lock() + defer r.mu.Unlock() + if !r.closed { + r.getAndSend(ctx, rt, req.ID, r.reconcileCh) + } + return nil +} + +func (r *memCheckReconciler) MapToNothing( + ctx context.Context, + rt Runtime, + res *pbresource.Resource, +) ([]Request, error) { + r.mu.Lock() + defer r.mu.Unlock() + if !r.closed { + r.getAndSend(ctx, rt, res.Id, r.mapCh) + } + return nil, nil +} + +func (*memCheckReconciler) getAndSend(ctx context.Context, rt Runtime, id *pbresource.ID, ch chan<- memCheckResult) { + var res memCheckResult + response, err := rt.Client.Read(ctx, &pbresource.ReadRequest{ + Id: id, + }) + res.clientGetError = err + if response != nil { + res.clientGet = response.Resource + } + + res.cacheGet, res.cacheGetError = rt.Cache.Get(id.Type, "id", id) + + ch <- res +} + +func watchListEvents(t testutil.TestingTB, events ...*pbresource.WatchEvent) pbresource.ResourceService_WatchListClient { + t.Helper() + ctx := testutil.TestContext(t) + + watchListClient := mockpbresource.NewResourceService_WatchListClient(t) + + // Return the events in the specified order as soon as they are requested + for _, event := range events { + watchListClient.EXPECT(). + Recv(). + RunAndReturn(func() (*pbresource.WatchEvent, error) { + return event, nil + }). + Once() + } + + // Now that all specified events have been exhausted we loop until the test finishes + // and the context bound to the tests lifecycle has been cancelled. This prevents getting + // any weird errors from the controller manager/runner. + watchListClient.EXPECT(). + Recv(). + RunAndReturn(func() (*pbresource.WatchEvent, error) { + <-ctx.Done() + return nil, ctx.Err() + }). + Maybe() + + return watchListClient +} + +// TestControllerRuntimeMemoryCloning mainly is testing that the runtimes +// provided to reconcilers and dependency mappers will return data from +// the resource service client and the cache that have been cloned so that +// the controller should be free to modify the data as needed. +func TestControllerRuntimeMemoryCloning(t *testing.T) { + ctx := testutil.TestContext(t) + + // create some resources to use during the test + res1 := resourcetest.Resource(fakeType, "foo"). + WithTenancy(resource.DefaultNamespacedTenancy()). + Build() + res2 := resourcetest.Resource(fakeV2Type, "bar"). + WithTenancy(resource.DefaultNamespacedTenancy()). + Build() + + // create the reconciler that will read the desired resource + // from both the resource service client and the cache client. + reconciler := newMemCheckReconciler(t) + + // Create the v1 watch list client to be returned when the controller runner + // calls WatchList on the v1 testing type. + v1WatchListClient := watchListEvents(t, &pbresource.WatchEvent{ + Operation: pbresource.WatchEvent_OPERATION_UPSERT, + Resource: res1, + }) + + // Create the v2 watch list client to be returned when the controller runner + // calls WatchList on the v2 testing type. + v2WatchListClient := watchListEvents(t, nil, &pbresource.WatchEvent{ + Operation: pbresource.WatchEvent_OPERATION_UPSERT, + Resource: res2, + }) + + // Create the mock resource service client + mres := mockpbresource.NewResourceServiceClient(t) + // Setup the expectation for the controller runner to issue a WatchList + // request for the managed type (fake v2 type) + mres.EXPECT(). + WatchList(mock.Anything, &pbresource.WatchListRequest{ + Type: fakeV2Type, + Tenancy: &pbresource.Tenancy{ + Partition: storage.Wildcard, + PeerName: storage.Wildcard, + Namespace: storage.Wildcard, + }, + }). + Return(v2WatchListClient, nil). + Once() + + // Setup the expectation for the controller runner to issue a WatchList + // request for the secondary Watch type (fake v1 type) + mres.EXPECT(). + WatchList(mock.Anything, &pbresource.WatchListRequest{ + Type: fakeType, + Tenancy: &pbresource.Tenancy{ + Partition: storage.Wildcard, + PeerName: storage.Wildcard, + Namespace: storage.Wildcard, + }, + }). + Return(v1WatchListClient, nil). + Once() + + // The cloning resource clients will forward actual calls onto the main resource service client. + // Here we are configuring the service mock to return either of the resources depending on the + // id present in the request. + mres.EXPECT(). + Read(mock.Anything, mock.Anything). + RunAndReturn(func(_ context.Context, req *pbresource.ReadRequest, opts ...grpc.CallOption) (*pbresource.ReadResponse, error) { + res := res2 + if resource.EqualID(res1.Id, req.Id) { + res = res1 + } + return &pbresource.ReadResponse{Resource: res}, nil + }). + Times(0) + + // create the test controller + ctl := NewController("test", fakeV2Type). + WithWatch(fakeType, reconciler.MapToNothing). + WithReconciler(reconciler) + + // create the controller manager and register our test controller + manager := NewManager(mres, testutil.Logger(t)) + manager.Register(ctl) + + // run the controller manager + manager.SetRaftLeader(true) + go manager.Run(ctx) + + // All future assertions should easily be able to run within 5s although they + // should typically run a couple orders of magnitude faster. + timeLimitedCtx, cancel := context.WithTimeout(ctx, 5*time.Second) + t.Cleanup(cancel) + + // validate that the v2 resource type event was seen and that the + // cache and the resource service client return cloned resources + reconciler.checkReconcileResult(t, timeLimitedCtx, res2) + + // Validate that the dependency mapper's resource and cache clients return + // cloned resources. + reconciler.checkMapResult(t, timeLimitedCtx, res1) +} + +// TestRunnerSharedMemoryCache is mainly testing to ensure that resources +// within the cache are shared with the resource service and have not been +// cloned. +func TestControllerRunnerSharedMemoryCache(t *testing.T) { + ctx := testutil.TestContext(t) + + // create resource to use during the test + res := resourcetest.Resource(fakeV2Type, "bar"). + WithTenancy(resource.DefaultNamespacedTenancy()). + Build() + + // create the reconciler that will read the desired resource + // from both the resource service client and the cache client. + reconciler := newMemCheckReconciler(t) + + // Create the v2 watch list client to be returned when the controller runner + // calls WatchList on the v2 testing type. + v2WatchListClient := watchListEvents(t, nil, &pbresource.WatchEvent{ + Operation: pbresource.WatchEvent_OPERATION_UPSERT, + Resource: res, + }) + + // Create the mock resource service client + mres := mockpbresource.NewResourceServiceClient(t) + // Setup the expectation for the controller runner to issue a WatchList + // request for the managed type (fake v2 type) + mres.EXPECT(). + WatchList(mock.Anything, &pbresource.WatchListRequest{ + Type: fakeV2Type, + Tenancy: &pbresource.Tenancy{ + Partition: storage.Wildcard, + PeerName: storage.Wildcard, + Namespace: storage.Wildcard, + }, + }). + Return(v2WatchListClient, nil). + Once() + + // The cloning resource clients will forward actual calls onto the main resource service client. + // Here we are configuring the service mock to return our singular resource always. + mres.EXPECT(). + Read(mock.Anything, mock.Anything). + Return(&pbresource.ReadResponse{Resource: res}, nil). + Times(0) + + // create the test controller + ctl := NewController("test", fakeV2Type). + WithReconciler(reconciler) + + runner := newControllerRunner(ctl, mres, testutil.Logger(t)) + go runner.run(ctx) + + // Wait for reconcile to be called before we check the values in + // the cache. This will also validate that the resource service client + // and cache client given to the reconciler cloned the resource but + // that is tested more thoroughly in another test and isn't of primary + // concern here. + reconciler.checkReconcileResult(t, ctx, res) + + // Now validate that the cache hold the same resource pointer as the original data + actual, err := runner.cache.Get(fakeV2Type, "id", res.Id) + require.NoError(t, err) + require.Same(t, res, actual) +} diff --git a/internal/controller/runner.go b/internal/controller/runner.go index a59130906b..1c60a6f15e 100644 --- a/internal/controller/runner.go +++ b/internal/controller/runner.go @@ -14,6 +14,7 @@ import ( "github.com/hashicorp/consul/agent/consul/controller/queue" "github.com/hashicorp/consul/internal/controller/cache" + "github.com/hashicorp/consul/internal/protoutil" "github.com/hashicorp/consul/internal/resource" "github.com/hashicorp/consul/internal/storage" "github.com/hashicorp/consul/proto-public/pbresource" @@ -29,17 +30,34 @@ type Runtime struct { // controllerRunner contains the actual implementation of running a controller // including creating watches, calling the reconciler, handling retries, etc. type controllerRunner struct { - ctrl *Controller - client pbresource.ResourceServiceClient - logger hclog.Logger - cache cache.Cache + ctrl *Controller + // watchClient will be used by the controller infrastructure internally to + // perform watches and maintain the cache. On servers, this client will use + // the in-memory gRPC clients which DO NOT cause cloning of data returned by + // the resource service. This is desirable so that our cache doesn't incur + // the overhead of duplicating all resources that are watched. Generally + // dependency mappers and reconcilers should not be given this client so + // that they can be free to modify the data they are returned. + watchClient pbresource.ResourceServiceClient + // runtimeClient will be used by dependency mappers and reconcilers to + // access the resource service. On servers, this client will use the in-memory + // gRPC client wrapped with the cloning client to force cloning of protobuf + // messages as they pass through the client. This is desirable so that + // controllers and their mappers can be free to modify the data returned + // to them without having to think about the fact that the data should + // be immutable as it is shared with the controllers cache as well as the + // resource service itself. + runtimeClient pbresource.ResourceServiceClient + logger hclog.Logger + cache cache.Cache } func newControllerRunner(c *Controller, client pbresource.ResourceServiceClient, defaultLogger hclog.Logger) *controllerRunner { return &controllerRunner{ - ctrl: c, - client: client, - logger: c.buildLogger(defaultLogger), + ctrl: c, + watchClient: client, + runtimeClient: pbresource.NewCloningResourceServiceClient(client), + logger: c.buildLogger(defaultLogger), // Do not build the cache here. If we build/set it when the controller runs // then if a controller is restarted it will invalidate the previous cache automatically. } @@ -50,6 +68,20 @@ func (c *controllerRunner) run(ctx context.Context) error { defer c.logger.Debug("controller stopping") c.cache = c.ctrl.buildCache() + defer func() { + // once no longer running we should nil out the cache + // so that we don't hold pointers to resources which may + // become out of date in the future. + c.cache = nil + }() + + if c.ctrl.startCb != nil { + c.ctrl.startCb(ctx, c.runtime(c.logger)) + } + + if c.ctrl.stopCb != nil { + defer c.ctrl.stopCb(ctx, c.runtime(c.logger)) + } group, groupCtx := errgroup.WithContext(ctx) recQueue := runQueue[Request](groupCtx, c.ctrl) @@ -112,7 +144,7 @@ func runQueue[T queue.ItemType](ctx context.Context, ctrl *Controller) queue.Wor } func (c *controllerRunner) watch(ctx context.Context, typ *pbresource.Type, add func(*pbresource.Resource)) error { - wl, err := c.client.WatchList(ctx, &pbresource.WatchListRequest{ + wl, err := c.watchClient.WatchList(ctx, &pbresource.WatchListRequest{ Type: typ, Tenancy: &pbresource.Tenancy{ Partition: storage.Wildcard, @@ -143,7 +175,15 @@ func (c *controllerRunner) watch(ctx context.Context, typ *pbresource.Type, add c.cache.Delete(event.Resource) } - add(event.Resource) + // Before adding the resource into the queue we must clone it. + // While we want the cache to not have duplicate copies of all the + // data, we do want downstream consumers like dependency mappers and + // controllers to be able to freely modify the data they are given. + // Therefore we clone the resource here to prevent any accidental + // mutation of data held by the cache (and presumably by the resource + // service assuming that the regular client we were given is the inmem + // variant) + add(protoutil.Clone(event.Resource)) } } @@ -266,9 +306,14 @@ func (c *controllerRunner) handlePanic(fn func() error) (err error) { func (c *controllerRunner) runtime(logger hclog.Logger) Runtime { return Runtime{ - Client: c.client, + // dependency mappers and controllers are always given the cloning client + // so that they do not have to care about mutating values that they read + // through the client. + Client: c.runtimeClient, Logger: logger, - Cache: c.cache, + // ensure that resources queried via the cache get cloned so that the + // dependency mapper or reconciler is free to modify them. + Cache: cache.NewCloningReadOnlyCache(c.cache), } } diff --git a/internal/controller/testing.go b/internal/controller/testing.go index 2e6a543d4c..652ed18837 100644 --- a/internal/controller/testing.go +++ b/internal/controller/testing.go @@ -14,6 +14,12 @@ import ( // TestController is most useful when writing unit tests for a controller where // individual Reconcile calls need to be made instead of having a Manager // execute the controller in response to watch events. +// +// TODO(controller-testing) Ideally this would live within the controllertest +// package. However it makes light use of unexported fields on the Controller +// and therefore cannot live in another package without more refactorings +// to have the Controller include a config struct of sorts defined in an +// internal package with exported fields. For now this seems fine. type TestController struct { c *Controller cache cache.Cache diff --git a/internal/resource/resourcetest/builder.go b/internal/resource/resourcetest/builder.go index ee37206b46..c4fdcbfea8 100644 --- a/internal/resource/resourcetest/builder.go +++ b/internal/resource/resourcetest/builder.go @@ -113,13 +113,15 @@ func (b *resourceBuilder) Build() *pbresource.Resource { } // Now create the status map - res.Status = make(map[string]*pbresource.Status) - for key, original := range b.statuses { - status := &pbresource.Status{ - ObservedGeneration: res.Generation, - Conditions: original.Conditions, + if len(b.statuses) > 0 { + res.Status = make(map[string]*pbresource.Status) + for key, original := range b.statuses { + status := &pbresource.Status{ + ObservedGeneration: res.Generation, + Conditions: original.Conditions, + } + res.Status[key] = status } - res.Status[key] = status } return res diff --git a/internal/tools/protoc-gen-grpc-clone/e2e/.mockery.yaml b/internal/tools/protoc-gen-grpc-clone/e2e/.mockery.yaml new file mode 100644 index 0000000000..b2e1235cfb --- /dev/null +++ b/internal/tools/protoc-gen-grpc-clone/e2e/.mockery.yaml @@ -0,0 +1,15 @@ +# Copyright (c) HashiCorp, Inc. +# SPDX-License-Identifier: BUSL-1.1 + +with-expecter: true +all: false +recursive: false +dir: . +outpkg: "e2e" +mockname: "{{.InterfaceName}}" +filename: "mock_{{.InterfaceName}}_test.go" +packages: + github.com/hashicorp/consul/internal/tools/protoc-gen-grpc-clone/e2e/proto: + interfaces: + SimpleClient: + Simple_FlowClient: diff --git a/internal/tools/protoc-gen-grpc-clone/e2e/e2e_test.go b/internal/tools/protoc-gen-grpc-clone/e2e/e2e_test.go new file mode 100644 index 0000000000..52cf22fe40 --- /dev/null +++ b/internal/tools/protoc-gen-grpc-clone/e2e/e2e_test.go @@ -0,0 +1,72 @@ +// Copyright (c) HashiCorp, Inc. +// SPDX-License-Identifier: BUSL-1.1 + +package e2e + +import ( + context "context" + "testing" + + proto "github.com/hashicorp/consul/internal/tools/protoc-gen-grpc-clone/e2e/proto" + "github.com/hashicorp/consul/proto/private/prototest" + mock "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" + grpc "google.golang.org/grpc" +) + +func TestCloningClient_Unary(t *testing.T) { + mclient := NewSimpleClient(t) + + expectedRequest := &proto.Req{Foo: "foo"} + expectedResponse := &proto.Resp{Bar: "bar"} + + mclient.EXPECT(). + Something(mock.Anything, mock.Anything). + RunAndReturn(func(_ context.Context, actualRequest *proto.Req, _ ...grpc.CallOption) (*proto.Resp, error) { + // The request object should have been cloned + prototest.AssertDeepEqual(t, expectedRequest, actualRequest) + require.NotSame(t, expectedRequest, actualRequest) + return expectedResponse, nil + }). + Once() + + cloner := proto.NewCloningSimpleClient(mclient) + + actualResponse, err := cloner.Something(context.Background(), expectedRequest) + require.NoError(t, err) + prototest.AssertDeepEqual(t, expectedResponse, actualResponse) + require.NotSame(t, expectedResponse, actualResponse) +} + +func TestCloningClient_StreamFromServer(t *testing.T) { + expectedRequest := &proto.Req{Foo: "foo"} + expectedResponse := &proto.Resp{Bar: "bar"} + + mstream := NewSimple_FlowClient(t) + mclient := NewSimpleClient(t) + + mclient.EXPECT(). + Flow(mock.Anything, mock.Anything). + RunAndReturn(func(_ context.Context, actualRequest *proto.Req, _ ...grpc.CallOption) (proto.Simple_FlowClient, error) { + // The request object should have been cloned + prototest.AssertDeepEqual(t, expectedRequest, actualRequest) + require.NotSame(t, expectedRequest, actualRequest) + + return mstream, nil + }). + Once() + mstream.EXPECT(). + Recv(). + Return(expectedResponse, nil) + + cloner := proto.NewCloningSimpleClient(mclient) + + stream, err := cloner.Flow(context.Background(), expectedRequest) + require.NoError(t, err) + require.NotNil(t, stream) + + actualResponse, err := stream.Recv() + require.NoError(t, err) + prototest.AssertDeepEqual(t, expectedResponse, actualResponse) + require.NotSame(t, expectedResponse, actualResponse) +} diff --git a/internal/tools/protoc-gen-grpc-clone/e2e/mock_SimpleClient_test.go b/internal/tools/protoc-gen-grpc-clone/e2e/mock_SimpleClient_test.go new file mode 100644 index 0000000000..29a427757d --- /dev/null +++ b/internal/tools/protoc-gen-grpc-clone/e2e/mock_SimpleClient_test.go @@ -0,0 +1,180 @@ +// Code generated by mockery v2.37.1. DO NOT EDIT. + +package e2e + +import ( + context "context" + + grpc "google.golang.org/grpc" + + mock "github.com/stretchr/testify/mock" + + proto "github.com/hashicorp/consul/internal/tools/protoc-gen-grpc-clone/e2e/proto" +) + +// SimpleClient is an autogenerated mock type for the SimpleClient type +type SimpleClient struct { + mock.Mock +} + +type SimpleClient_Expecter struct { + mock *mock.Mock +} + +func (_m *SimpleClient) EXPECT() *SimpleClient_Expecter { + return &SimpleClient_Expecter{mock: &_m.Mock} +} + +// Flow provides a mock function with given fields: ctx, in, opts +func (_m *SimpleClient) Flow(ctx context.Context, in *proto.Req, opts ...grpc.CallOption) (proto.Simple_FlowClient, error) { + _va := make([]interface{}, len(opts)) + for _i := range opts { + _va[_i] = opts[_i] + } + var _ca []interface{} + _ca = append(_ca, ctx, in) + _ca = append(_ca, _va...) + ret := _m.Called(_ca...) + + var r0 proto.Simple_FlowClient + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, *proto.Req, ...grpc.CallOption) (proto.Simple_FlowClient, error)); ok { + return rf(ctx, in, opts...) + } + if rf, ok := ret.Get(0).(func(context.Context, *proto.Req, ...grpc.CallOption) proto.Simple_FlowClient); ok { + r0 = rf(ctx, in, opts...) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(proto.Simple_FlowClient) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, *proto.Req, ...grpc.CallOption) error); ok { + r1 = rf(ctx, in, opts...) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// SimpleClient_Flow_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Flow' +type SimpleClient_Flow_Call struct { + *mock.Call +} + +// Flow is a helper method to define mock.On call +// - ctx context.Context +// - in *proto.Req +// - opts ...grpc.CallOption +func (_e *SimpleClient_Expecter) Flow(ctx interface{}, in interface{}, opts ...interface{}) *SimpleClient_Flow_Call { + return &SimpleClient_Flow_Call{Call: _e.mock.On("Flow", + append([]interface{}{ctx, in}, opts...)...)} +} + +func (_c *SimpleClient_Flow_Call) Run(run func(ctx context.Context, in *proto.Req, opts ...grpc.CallOption)) *SimpleClient_Flow_Call { + _c.Call.Run(func(args mock.Arguments) { + variadicArgs := make([]grpc.CallOption, len(args)-2) + for i, a := range args[2:] { + if a != nil { + variadicArgs[i] = a.(grpc.CallOption) + } + } + run(args[0].(context.Context), args[1].(*proto.Req), variadicArgs...) + }) + return _c +} + +func (_c *SimpleClient_Flow_Call) Return(_a0 proto.Simple_FlowClient, _a1 error) *SimpleClient_Flow_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *SimpleClient_Flow_Call) RunAndReturn(run func(context.Context, *proto.Req, ...grpc.CallOption) (proto.Simple_FlowClient, error)) *SimpleClient_Flow_Call { + _c.Call.Return(run) + return _c +} + +// Something provides a mock function with given fields: ctx, in, opts +func (_m *SimpleClient) Something(ctx context.Context, in *proto.Req, opts ...grpc.CallOption) (*proto.Resp, error) { + _va := make([]interface{}, len(opts)) + for _i := range opts { + _va[_i] = opts[_i] + } + var _ca []interface{} + _ca = append(_ca, ctx, in) + _ca = append(_ca, _va...) + ret := _m.Called(_ca...) + + var r0 *proto.Resp + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, *proto.Req, ...grpc.CallOption) (*proto.Resp, error)); ok { + return rf(ctx, in, opts...) + } + if rf, ok := ret.Get(0).(func(context.Context, *proto.Req, ...grpc.CallOption) *proto.Resp); ok { + r0 = rf(ctx, in, opts...) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*proto.Resp) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, *proto.Req, ...grpc.CallOption) error); ok { + r1 = rf(ctx, in, opts...) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// SimpleClient_Something_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Something' +type SimpleClient_Something_Call struct { + *mock.Call +} + +// Something is a helper method to define mock.On call +// - ctx context.Context +// - in *proto.Req +// - opts ...grpc.CallOption +func (_e *SimpleClient_Expecter) Something(ctx interface{}, in interface{}, opts ...interface{}) *SimpleClient_Something_Call { + return &SimpleClient_Something_Call{Call: _e.mock.On("Something", + append([]interface{}{ctx, in}, opts...)...)} +} + +func (_c *SimpleClient_Something_Call) Run(run func(ctx context.Context, in *proto.Req, opts ...grpc.CallOption)) *SimpleClient_Something_Call { + _c.Call.Run(func(args mock.Arguments) { + variadicArgs := make([]grpc.CallOption, len(args)-2) + for i, a := range args[2:] { + if a != nil { + variadicArgs[i] = a.(grpc.CallOption) + } + } + run(args[0].(context.Context), args[1].(*proto.Req), variadicArgs...) + }) + return _c +} + +func (_c *SimpleClient_Something_Call) Return(_a0 *proto.Resp, _a1 error) *SimpleClient_Something_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *SimpleClient_Something_Call) RunAndReturn(run func(context.Context, *proto.Req, ...grpc.CallOption) (*proto.Resp, error)) *SimpleClient_Something_Call { + _c.Call.Return(run) + return _c +} + +// NewSimpleClient creates a new instance of SimpleClient. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. +// The first argument is typically a *testing.T value. +func NewSimpleClient(t interface { + mock.TestingT + Cleanup(func()) +}) *SimpleClient { + mock := &SimpleClient{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/internal/tools/protoc-gen-grpc-clone/e2e/mock_Simple_FlowClient_test.go b/internal/tools/protoc-gen-grpc-clone/e2e/mock_Simple_FlowClient_test.go new file mode 100644 index 0000000000..2d3bbc4b09 --- /dev/null +++ b/internal/tools/protoc-gen-grpc-clone/e2e/mock_Simple_FlowClient_test.go @@ -0,0 +1,356 @@ +// Code generated by mockery v2.37.1. DO NOT EDIT. + +package e2e + +import ( + context "context" + + mock "github.com/stretchr/testify/mock" + metadata "google.golang.org/grpc/metadata" + + proto "github.com/hashicorp/consul/internal/tools/protoc-gen-grpc-clone/e2e/proto" +) + +// Simple_FlowClient is an autogenerated mock type for the Simple_FlowClient type +type Simple_FlowClient struct { + mock.Mock +} + +type Simple_FlowClient_Expecter struct { + mock *mock.Mock +} + +func (_m *Simple_FlowClient) EXPECT() *Simple_FlowClient_Expecter { + return &Simple_FlowClient_Expecter{mock: &_m.Mock} +} + +// CloseSend provides a mock function with given fields: +func (_m *Simple_FlowClient) CloseSend() error { + ret := _m.Called() + + var r0 error + if rf, ok := ret.Get(0).(func() error); ok { + r0 = rf() + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// Simple_FlowClient_CloseSend_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'CloseSend' +type Simple_FlowClient_CloseSend_Call struct { + *mock.Call +} + +// CloseSend is a helper method to define mock.On call +func (_e *Simple_FlowClient_Expecter) CloseSend() *Simple_FlowClient_CloseSend_Call { + return &Simple_FlowClient_CloseSend_Call{Call: _e.mock.On("CloseSend")} +} + +func (_c *Simple_FlowClient_CloseSend_Call) Run(run func()) *Simple_FlowClient_CloseSend_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *Simple_FlowClient_CloseSend_Call) Return(_a0 error) *Simple_FlowClient_CloseSend_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *Simple_FlowClient_CloseSend_Call) RunAndReturn(run func() error) *Simple_FlowClient_CloseSend_Call { + _c.Call.Return(run) + return _c +} + +// Context provides a mock function with given fields: +func (_m *Simple_FlowClient) Context() context.Context { + ret := _m.Called() + + var r0 context.Context + if rf, ok := ret.Get(0).(func() context.Context); ok { + r0 = rf() + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(context.Context) + } + } + + return r0 +} + +// Simple_FlowClient_Context_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Context' +type Simple_FlowClient_Context_Call struct { + *mock.Call +} + +// Context is a helper method to define mock.On call +func (_e *Simple_FlowClient_Expecter) Context() *Simple_FlowClient_Context_Call { + return &Simple_FlowClient_Context_Call{Call: _e.mock.On("Context")} +} + +func (_c *Simple_FlowClient_Context_Call) Run(run func()) *Simple_FlowClient_Context_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *Simple_FlowClient_Context_Call) Return(_a0 context.Context) *Simple_FlowClient_Context_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *Simple_FlowClient_Context_Call) RunAndReturn(run func() context.Context) *Simple_FlowClient_Context_Call { + _c.Call.Return(run) + return _c +} + +// Header provides a mock function with given fields: +func (_m *Simple_FlowClient) Header() (metadata.MD, error) { + ret := _m.Called() + + var r0 metadata.MD + var r1 error + if rf, ok := ret.Get(0).(func() (metadata.MD, error)); ok { + return rf() + } + if rf, ok := ret.Get(0).(func() metadata.MD); ok { + r0 = rf() + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(metadata.MD) + } + } + + if rf, ok := ret.Get(1).(func() error); ok { + r1 = rf() + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// Simple_FlowClient_Header_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Header' +type Simple_FlowClient_Header_Call struct { + *mock.Call +} + +// Header is a helper method to define mock.On call +func (_e *Simple_FlowClient_Expecter) Header() *Simple_FlowClient_Header_Call { + return &Simple_FlowClient_Header_Call{Call: _e.mock.On("Header")} +} + +func (_c *Simple_FlowClient_Header_Call) Run(run func()) *Simple_FlowClient_Header_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *Simple_FlowClient_Header_Call) Return(_a0 metadata.MD, _a1 error) *Simple_FlowClient_Header_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *Simple_FlowClient_Header_Call) RunAndReturn(run func() (metadata.MD, error)) *Simple_FlowClient_Header_Call { + _c.Call.Return(run) + return _c +} + +// Recv provides a mock function with given fields: +func (_m *Simple_FlowClient) Recv() (*proto.Resp, error) { + ret := _m.Called() + + var r0 *proto.Resp + var r1 error + if rf, ok := ret.Get(0).(func() (*proto.Resp, error)); ok { + return rf() + } + if rf, ok := ret.Get(0).(func() *proto.Resp); ok { + r0 = rf() + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*proto.Resp) + } + } + + if rf, ok := ret.Get(1).(func() error); ok { + r1 = rf() + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// Simple_FlowClient_Recv_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Recv' +type Simple_FlowClient_Recv_Call struct { + *mock.Call +} + +// Recv is a helper method to define mock.On call +func (_e *Simple_FlowClient_Expecter) Recv() *Simple_FlowClient_Recv_Call { + return &Simple_FlowClient_Recv_Call{Call: _e.mock.On("Recv")} +} + +func (_c *Simple_FlowClient_Recv_Call) Run(run func()) *Simple_FlowClient_Recv_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *Simple_FlowClient_Recv_Call) Return(_a0 *proto.Resp, _a1 error) *Simple_FlowClient_Recv_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *Simple_FlowClient_Recv_Call) RunAndReturn(run func() (*proto.Resp, error)) *Simple_FlowClient_Recv_Call { + _c.Call.Return(run) + return _c +} + +// RecvMsg provides a mock function with given fields: m +func (_m *Simple_FlowClient) RecvMsg(m interface{}) error { + ret := _m.Called(m) + + var r0 error + if rf, ok := ret.Get(0).(func(interface{}) error); ok { + r0 = rf(m) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// Simple_FlowClient_RecvMsg_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'RecvMsg' +type Simple_FlowClient_RecvMsg_Call struct { + *mock.Call +} + +// RecvMsg is a helper method to define mock.On call +// - m interface{} +func (_e *Simple_FlowClient_Expecter) RecvMsg(m interface{}) *Simple_FlowClient_RecvMsg_Call { + return &Simple_FlowClient_RecvMsg_Call{Call: _e.mock.On("RecvMsg", m)} +} + +func (_c *Simple_FlowClient_RecvMsg_Call) Run(run func(m interface{})) *Simple_FlowClient_RecvMsg_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(interface{})) + }) + return _c +} + +func (_c *Simple_FlowClient_RecvMsg_Call) Return(_a0 error) *Simple_FlowClient_RecvMsg_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *Simple_FlowClient_RecvMsg_Call) RunAndReturn(run func(interface{}) error) *Simple_FlowClient_RecvMsg_Call { + _c.Call.Return(run) + return _c +} + +// SendMsg provides a mock function with given fields: m +func (_m *Simple_FlowClient) SendMsg(m interface{}) error { + ret := _m.Called(m) + + var r0 error + if rf, ok := ret.Get(0).(func(interface{}) error); ok { + r0 = rf(m) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// Simple_FlowClient_SendMsg_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'SendMsg' +type Simple_FlowClient_SendMsg_Call struct { + *mock.Call +} + +// SendMsg is a helper method to define mock.On call +// - m interface{} +func (_e *Simple_FlowClient_Expecter) SendMsg(m interface{}) *Simple_FlowClient_SendMsg_Call { + return &Simple_FlowClient_SendMsg_Call{Call: _e.mock.On("SendMsg", m)} +} + +func (_c *Simple_FlowClient_SendMsg_Call) Run(run func(m interface{})) *Simple_FlowClient_SendMsg_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(interface{})) + }) + return _c +} + +func (_c *Simple_FlowClient_SendMsg_Call) Return(_a0 error) *Simple_FlowClient_SendMsg_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *Simple_FlowClient_SendMsg_Call) RunAndReturn(run func(interface{}) error) *Simple_FlowClient_SendMsg_Call { + _c.Call.Return(run) + return _c +} + +// Trailer provides a mock function with given fields: +func (_m *Simple_FlowClient) Trailer() metadata.MD { + ret := _m.Called() + + var r0 metadata.MD + if rf, ok := ret.Get(0).(func() metadata.MD); ok { + r0 = rf() + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(metadata.MD) + } + } + + return r0 +} + +// Simple_FlowClient_Trailer_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Trailer' +type Simple_FlowClient_Trailer_Call struct { + *mock.Call +} + +// Trailer is a helper method to define mock.On call +func (_e *Simple_FlowClient_Expecter) Trailer() *Simple_FlowClient_Trailer_Call { + return &Simple_FlowClient_Trailer_Call{Call: _e.mock.On("Trailer")} +} + +func (_c *Simple_FlowClient_Trailer_Call) Run(run func()) *Simple_FlowClient_Trailer_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *Simple_FlowClient_Trailer_Call) Return(_a0 metadata.MD) *Simple_FlowClient_Trailer_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *Simple_FlowClient_Trailer_Call) RunAndReturn(run func() metadata.MD) *Simple_FlowClient_Trailer_Call { + _c.Call.Return(run) + return _c +} + +// NewSimple_FlowClient creates a new instance of Simple_FlowClient. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. +// The first argument is typically a *testing.T value. +func NewSimple_FlowClient(t interface { + mock.TestingT + Cleanup(func()) +}) *Simple_FlowClient { + mock := &Simple_FlowClient{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/internal/tools/protoc-gen-grpc-clone/e2e/proto/buf.gen.yaml b/internal/tools/protoc-gen-grpc-clone/e2e/proto/buf.gen.yaml new file mode 100644 index 0000000000..fca6a8b23b --- /dev/null +++ b/internal/tools/protoc-gen-grpc-clone/e2e/proto/buf.gen.yaml @@ -0,0 +1,22 @@ +# Copyright (c) HashiCorp, Inc. +# SPDX-License-Identifier: BUSL-1.1 + +version: v1 +managed: + enabled: true + go_package_prefix: + default: github.com/hashicorp/consul/internal/tools/protoc-gen-grpc-clone/e2e/proto +plugins: + - name: go + out: . + opt: + - paths=source_relative + - name: go-grpc + out: . + opt: + - paths=source_relative + - require_unimplemented_servers=false + - name: grpc-clone + out: . + opt: + - paths=source_relative diff --git a/internal/tools/protoc-gen-grpc-clone/e2e/proto/cloning_stream.pb.go b/internal/tools/protoc-gen-grpc-clone/e2e/proto/cloning_stream.pb.go new file mode 100644 index 0000000000..69f03db94d --- /dev/null +++ b/internal/tools/protoc-gen-grpc-clone/e2e/proto/cloning_stream.pb.go @@ -0,0 +1,31 @@ +// Code generated by protoc-gen-grpc-inmem. DO NOT EDIT. + +package proto + +import ( + grpc "google.golang.org/grpc" + "google.golang.org/protobuf/proto" +) + +type serverStream[T proto.Message] interface { + Recv() (T, error) + grpc.ClientStream +} + +type cloningStream[T proto.Message] struct { + serverStream[T] +} + +func newCloningStream[T proto.Message](stream serverStream[T]) cloningStream[T] { + return cloningStream[T]{serverStream: stream} +} + +func (st cloningStream[T]) Recv() (T, error) { + var zero T + val, err := st.serverStream.Recv() + if err != nil { + return zero, err + } + + return proto.Clone(val).(T), nil +} diff --git a/internal/tools/protoc-gen-grpc-clone/e2e/proto/service.pb.go b/internal/tools/protoc-gen-grpc-clone/e2e/proto/service.pb.go new file mode 100644 index 0000000000..4701ad950d --- /dev/null +++ b/internal/tools/protoc-gen-grpc-clone/e2e/proto/service.pb.go @@ -0,0 +1,258 @@ +// Copyright (c) HashiCorp, Inc. +// SPDX-License-Identifier: BUSL + +// Copyright (c) HashiCorp, Inc. +// SPDX-License-Identifier: BUSL-1.1 + +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.31.0 +// protoc (unknown) +// source: service.proto + +package proto + +import ( + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + reflect "reflect" + sync "sync" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type Req struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Foo string `protobuf:"bytes,1,opt,name=Foo,proto3" json:"Foo,omitempty"` +} + +func (x *Req) Reset() { + *x = Req{} + if protoimpl.UnsafeEnabled { + mi := &file_service_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Req) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Req) ProtoMessage() {} + +func (x *Req) ProtoReflect() protoreflect.Message { + mi := &file_service_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Req.ProtoReflect.Descriptor instead. +func (*Req) Descriptor() ([]byte, []int) { + return file_service_proto_rawDescGZIP(), []int{0} +} + +func (x *Req) GetFoo() string { + if x != nil { + return x.Foo + } + return "" +} + +type Resp struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Bar string `protobuf:"bytes,1,opt,name=Bar,proto3" json:"Bar,omitempty"` +} + +func (x *Resp) Reset() { + *x = Resp{} + if protoimpl.UnsafeEnabled { + mi := &file_service_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Resp) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Resp) ProtoMessage() {} + +func (x *Resp) ProtoReflect() protoreflect.Message { + mi := &file_service_proto_msgTypes[1] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Resp.ProtoReflect.Descriptor instead. +func (*Resp) Descriptor() ([]byte, []int) { + return file_service_proto_rawDescGZIP(), []int{1} +} + +func (x *Resp) GetBar() string { + if x != nil { + return x.Bar + } + return "" +} + +var File_service_proto protoreflect.FileDescriptor + +var file_service_proto_rawDesc = []byte{ + 0x0a, 0x0d, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, + 0x37, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, + 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x63, 0x5f, 0x67, 0x65, 0x6e, 0x5f, 0x67, 0x72, 0x70, 0x63, 0x5f, 0x63, 0x6c, 0x6f, 0x6e, 0x65, + 0x2e, 0x74, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x22, 0x17, 0x0a, 0x03, 0x52, 0x65, 0x71, 0x12, + 0x10, 0x0a, 0x03, 0x46, 0x6f, 0x6f, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x46, 0x6f, + 0x6f, 0x22, 0x18, 0x0a, 0x04, 0x52, 0x65, 0x73, 0x70, 0x12, 0x10, 0x0a, 0x03, 0x42, 0x61, 0x72, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x42, 0x61, 0x72, 0x32, 0x9f, 0x02, 0x0a, 0x06, + 0x53, 0x69, 0x6d, 0x70, 0x6c, 0x65, 0x12, 0x8a, 0x01, 0x0a, 0x09, 0x53, 0x6f, 0x6d, 0x65, 0x74, + 0x68, 0x69, 0x6e, 0x67, 0x12, 0x3c, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, + 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x5f, 0x67, 0x65, 0x6e, 0x5f, 0x67, 0x72, 0x70, 0x63, + 0x5f, 0x63, 0x6c, 0x6f, 0x6e, 0x65, 0x2e, 0x74, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x2e, 0x52, + 0x65, 0x71, 0x1a, 0x3d, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, + 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x5f, 0x67, 0x65, 0x6e, 0x5f, 0x67, 0x72, 0x70, 0x63, 0x5f, 0x63, + 0x6c, 0x6f, 0x6e, 0x65, 0x2e, 0x74, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x2e, 0x52, 0x65, 0x73, + 0x70, 0x22, 0x00, 0x12, 0x87, 0x01, 0x0a, 0x04, 0x46, 0x6c, 0x6f, 0x77, 0x12, 0x3c, 0x2e, 0x68, + 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, + 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x5f, + 0x67, 0x65, 0x6e, 0x5f, 0x67, 0x72, 0x70, 0x63, 0x5f, 0x63, 0x6c, 0x6f, 0x6e, 0x65, 0x2e, 0x74, + 0x65, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x2e, 0x52, 0x65, 0x71, 0x1a, 0x3d, 0x2e, 0x68, 0x61, 0x73, + 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, + 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x5f, 0x67, 0x65, + 0x6e, 0x5f, 0x67, 0x72, 0x70, 0x63, 0x5f, 0x63, 0x6c, 0x6f, 0x6e, 0x65, 0x2e, 0x74, 0x65, 0x73, + 0x74, 0x69, 0x6e, 0x67, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x22, 0x00, 0x30, 0x01, 0x42, 0x8d, 0x03, + 0x0a, 0x3b, 0x63, 0x6f, 0x6d, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, + 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x5f, 0x67, 0x65, 0x6e, 0x5f, 0x67, 0x72, 0x70, 0x63, 0x5f, + 0x63, 0x6c, 0x6f, 0x6e, 0x65, 0x2e, 0x74, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x42, 0x0c, 0x53, + 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x4a, 0x67, + 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, + 0x6f, 0x72, 0x70, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, + 0x6e, 0x61, 0x6c, 0x2f, 0x74, 0x6f, 0x6f, 0x6c, 0x73, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, + 0x2d, 0x67, 0x65, 0x6e, 0x2d, 0x67, 0x72, 0x70, 0x63, 0x2d, 0x63, 0x6c, 0x6f, 0x6e, 0x65, 0x2f, + 0x65, 0x32, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0xa2, 0x02, 0x05, 0x48, 0x43, 0x49, 0x50, + 0x54, 0xaa, 0x02, 0x34, 0x48, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x43, 0x6f, + 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x50, 0x72, + 0x6f, 0x74, 0x6f, 0x63, 0x47, 0x65, 0x6e, 0x47, 0x72, 0x70, 0x63, 0x43, 0x6c, 0x6f, 0x6e, 0x65, + 0x2e, 0x54, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x67, 0xca, 0x02, 0x34, 0x48, 0x61, 0x73, 0x68, 0x69, + 0x63, 0x6f, 0x72, 0x70, 0x5c, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x5c, 0x49, 0x6e, 0x74, 0x65, + 0x72, 0x6e, 0x61, 0x6c, 0x5c, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x47, 0x65, 0x6e, 0x47, 0x72, + 0x70, 0x63, 0x43, 0x6c, 0x6f, 0x6e, 0x65, 0x5c, 0x54, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x67, 0xe2, + 0x02, 0x40, 0x48, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x5c, 0x43, 0x6f, 0x6e, 0x73, + 0x75, 0x6c, 0x5c, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x5c, 0x50, 0x72, 0x6f, 0x74, + 0x6f, 0x63, 0x47, 0x65, 0x6e, 0x47, 0x72, 0x70, 0x63, 0x43, 0x6c, 0x6f, 0x6e, 0x65, 0x5c, 0x54, + 0x65, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, + 0x74, 0x61, 0xea, 0x02, 0x38, 0x48, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x3a, 0x3a, + 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x3a, 0x3a, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, + 0x3a, 0x3a, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x47, 0x65, 0x6e, 0x47, 0x72, 0x70, 0x63, 0x43, + 0x6c, 0x6f, 0x6e, 0x65, 0x3a, 0x3a, 0x54, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x62, 0x06, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_service_proto_rawDescOnce sync.Once + file_service_proto_rawDescData = file_service_proto_rawDesc +) + +func file_service_proto_rawDescGZIP() []byte { + file_service_proto_rawDescOnce.Do(func() { + file_service_proto_rawDescData = protoimpl.X.CompressGZIP(file_service_proto_rawDescData) + }) + return file_service_proto_rawDescData +} + +var file_service_proto_msgTypes = make([]protoimpl.MessageInfo, 2) +var file_service_proto_goTypes = []interface{}{ + (*Req)(nil), // 0: hashicorp.consul.internal.protoc_gen_grpc_clone.testing.Req + (*Resp)(nil), // 1: hashicorp.consul.internal.protoc_gen_grpc_clone.testing.Resp +} +var file_service_proto_depIdxs = []int32{ + 0, // 0: hashicorp.consul.internal.protoc_gen_grpc_clone.testing.Simple.Something:input_type -> hashicorp.consul.internal.protoc_gen_grpc_clone.testing.Req + 0, // 1: hashicorp.consul.internal.protoc_gen_grpc_clone.testing.Simple.Flow:input_type -> hashicorp.consul.internal.protoc_gen_grpc_clone.testing.Req + 1, // 2: hashicorp.consul.internal.protoc_gen_grpc_clone.testing.Simple.Something:output_type -> hashicorp.consul.internal.protoc_gen_grpc_clone.testing.Resp + 1, // 3: hashicorp.consul.internal.protoc_gen_grpc_clone.testing.Simple.Flow:output_type -> hashicorp.consul.internal.protoc_gen_grpc_clone.testing.Resp + 2, // [2:4] is the sub-list for method output_type + 0, // [0:2] is the sub-list for method input_type + 0, // [0:0] is the sub-list for extension type_name + 0, // [0:0] is the sub-list for extension extendee + 0, // [0:0] is the sub-list for field type_name +} + +func init() { file_service_proto_init() } +func file_service_proto_init() { + if File_service_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_service_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*Req); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_service_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*Resp); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_service_proto_rawDesc, + NumEnums: 0, + NumMessages: 2, + NumExtensions: 0, + NumServices: 1, + }, + GoTypes: file_service_proto_goTypes, + DependencyIndexes: file_service_proto_depIdxs, + MessageInfos: file_service_proto_msgTypes, + }.Build() + File_service_proto = out.File + file_service_proto_rawDesc = nil + file_service_proto_goTypes = nil + file_service_proto_depIdxs = nil +} diff --git a/internal/tools/protoc-gen-grpc-clone/e2e/proto/service.proto b/internal/tools/protoc-gen-grpc-clone/e2e/proto/service.proto new file mode 100644 index 0000000000..2aabf1d7a1 --- /dev/null +++ b/internal/tools/protoc-gen-grpc-clone/e2e/proto/service.proto @@ -0,0 +1,22 @@ +// Copyright (c) HashiCorp, Inc. +// SPDX-License-Identifier: BUSL + +// Copyright (c) HashiCorp, Inc. +// SPDX-License-Identifier: BUSL-1.1 + +syntax = "proto3"; + +package hashicorp.consul.internal.protoc_gen_grpc_clone.testing; + +service Simple { + rpc Something(Req) returns (Resp) {} + rpc Flow(Req) returns (stream Resp) {} +} + +message Req { + string Foo = 1; +} + +message Resp { + string Bar = 1; +} diff --git a/internal/tools/protoc-gen-grpc-clone/e2e/proto/service_cloning_grpc.pb.go b/internal/tools/protoc-gen-grpc-clone/e2e/proto/service_cloning_grpc.pb.go new file mode 100644 index 0000000000..e255db7110 --- /dev/null +++ b/internal/tools/protoc-gen-grpc-clone/e2e/proto/service_cloning_grpc.pb.go @@ -0,0 +1,69 @@ +// Code generated by protoc-gen-grpc-inmem. DO NOT EDIT. + +package proto + +import ( + "context" + + grpc "google.golang.org/grpc" + "google.golang.org/protobuf/proto" +) + +// compile-time check to ensure that the generator is implementing all +// of the grpc client interfaces methods. +var _ SimpleClient = CloningSimpleClient{} + +// IsCloningSimpleClient is an interface that can be used to detect +// that a SimpleClient is using the in-memory transport and has already +// been wrapped with a with a CloningSimpleClient. +type IsCloningSimpleClient interface { + IsCloningSimpleClient() bool +} + +// CloningSimpleClient implements the SimpleClient interface by wrapping +// another implementation and copying all protobuf messages that pass through the client. +// This is mainly useful to wrap the an in-process client to insulate users of that +// client from having to care about potential immutability of data they receive or having +// the server implementation mutate their internal memory. +type CloningSimpleClient struct { + SimpleClient +} + +func NewCloningSimpleClient(client SimpleClient) SimpleClient { + if cloner, ok := client.(IsCloningSimpleClient); ok && cloner.IsCloningSimpleClient() { + // prevent a double clone if the underlying client is already the cloning client. + return client + } + + return CloningSimpleClient{ + SimpleClient: client, + } +} + +// IsCloningSimpleClient implements the IsCloningSimpleClient interface. This +// is only used to detect wrapped clients that would be double cloning data and prevent that. +func (c CloningSimpleClient) IsCloningSimpleClient() bool { + return true +} + +func (c CloningSimpleClient) Something(ctx context.Context, in *Req, opts ...grpc.CallOption) (*Resp, error) { + in = proto.Clone(in).(*Req) + + out, err := c.SimpleClient.Something(ctx, in) + if err != nil { + return nil, err + } + + return proto.Clone(out).(*Resp), nil +} + +func (c CloningSimpleClient) Flow(ctx context.Context, in *Req, opts ...grpc.CallOption) (Simple_FlowClient, error) { + in = proto.Clone(in).(*Req) + + st, err := c.SimpleClient.Flow(ctx, in) + if err != nil { + return nil, err + } + + return newCloningStream[*Resp](st), nil +} diff --git a/internal/tools/protoc-gen-grpc-clone/e2e/proto/service_grpc.pb.go b/internal/tools/protoc-gen-grpc-clone/e2e/proto/service_grpc.pb.go new file mode 100644 index 0000000000..33344e287a --- /dev/null +++ b/internal/tools/protoc-gen-grpc-clone/e2e/proto/service_grpc.pb.go @@ -0,0 +1,167 @@ +// Code generated by protoc-gen-go-grpc. DO NOT EDIT. +// versions: +// - protoc-gen-go-grpc v1.2.0 +// - protoc (unknown) +// source: service.proto + +package proto + +import ( + context "context" + grpc "google.golang.org/grpc" + codes "google.golang.org/grpc/codes" + status "google.golang.org/grpc/status" +) + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the grpc package it is being compiled against. +// Requires gRPC-Go v1.32.0 or later. +const _ = grpc.SupportPackageIsVersion7 + +// SimpleClient is the client API for Simple service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. +type SimpleClient interface { + Something(ctx context.Context, in *Req, opts ...grpc.CallOption) (*Resp, error) + Flow(ctx context.Context, in *Req, opts ...grpc.CallOption) (Simple_FlowClient, error) +} + +type simpleClient struct { + cc grpc.ClientConnInterface +} + +func NewSimpleClient(cc grpc.ClientConnInterface) SimpleClient { + return &simpleClient{cc} +} + +func (c *simpleClient) Something(ctx context.Context, in *Req, opts ...grpc.CallOption) (*Resp, error) { + out := new(Resp) + err := c.cc.Invoke(ctx, "/hashicorp.consul.internal.protoc_gen_grpc_clone.testing.Simple/Something", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *simpleClient) Flow(ctx context.Context, in *Req, opts ...grpc.CallOption) (Simple_FlowClient, error) { + stream, err := c.cc.NewStream(ctx, &Simple_ServiceDesc.Streams[0], "/hashicorp.consul.internal.protoc_gen_grpc_clone.testing.Simple/Flow", opts...) + if err != nil { + return nil, err + } + x := &simpleFlowClient{stream} + if err := x.ClientStream.SendMsg(in); err != nil { + return nil, err + } + if err := x.ClientStream.CloseSend(); err != nil { + return nil, err + } + return x, nil +} + +type Simple_FlowClient interface { + Recv() (*Resp, error) + grpc.ClientStream +} + +type simpleFlowClient struct { + grpc.ClientStream +} + +func (x *simpleFlowClient) Recv() (*Resp, error) { + m := new(Resp) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +// SimpleServer is the server API for Simple service. +// All implementations should embed UnimplementedSimpleServer +// for forward compatibility +type SimpleServer interface { + Something(context.Context, *Req) (*Resp, error) + Flow(*Req, Simple_FlowServer) error +} + +// UnimplementedSimpleServer should be embedded to have forward compatible implementations. +type UnimplementedSimpleServer struct { +} + +func (UnimplementedSimpleServer) Something(context.Context, *Req) (*Resp, error) { + return nil, status.Errorf(codes.Unimplemented, "method Something not implemented") +} +func (UnimplementedSimpleServer) Flow(*Req, Simple_FlowServer) error { + return status.Errorf(codes.Unimplemented, "method Flow not implemented") +} + +// UnsafeSimpleServer may be embedded to opt out of forward compatibility for this service. +// Use of this interface is not recommended, as added methods to SimpleServer will +// result in compilation errors. +type UnsafeSimpleServer interface { + mustEmbedUnimplementedSimpleServer() +} + +func RegisterSimpleServer(s grpc.ServiceRegistrar, srv SimpleServer) { + s.RegisterService(&Simple_ServiceDesc, srv) +} + +func _Simple_Something_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(Req) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(SimpleServer).Something(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/hashicorp.consul.internal.protoc_gen_grpc_clone.testing.Simple/Something", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(SimpleServer).Something(ctx, req.(*Req)) + } + return interceptor(ctx, in, info, handler) +} + +func _Simple_Flow_Handler(srv interface{}, stream grpc.ServerStream) error { + m := new(Req) + if err := stream.RecvMsg(m); err != nil { + return err + } + return srv.(SimpleServer).Flow(m, &simpleFlowServer{stream}) +} + +type Simple_FlowServer interface { + Send(*Resp) error + grpc.ServerStream +} + +type simpleFlowServer struct { + grpc.ServerStream +} + +func (x *simpleFlowServer) Send(m *Resp) error { + return x.ServerStream.SendMsg(m) +} + +// Simple_ServiceDesc is the grpc.ServiceDesc for Simple service. +// It's only intended for direct use with grpc.RegisterService, +// and not to be introspected or modified (even as a copy) +var Simple_ServiceDesc = grpc.ServiceDesc{ + ServiceName: "hashicorp.consul.internal.protoc_gen_grpc_clone.testing.Simple", + HandlerType: (*SimpleServer)(nil), + Methods: []grpc.MethodDesc{ + { + MethodName: "Something", + Handler: _Simple_Something_Handler, + }, + }, + Streams: []grpc.StreamDesc{ + { + StreamName: "Flow", + Handler: _Simple_Flow_Handler, + ServerStreams: true, + }, + }, + Metadata: "service.proto", +} diff --git a/internal/tools/protoc-gen-grpc-clone/internal/generate/generate.go b/internal/tools/protoc-gen-grpc-clone/internal/generate/generate.go new file mode 100644 index 0000000000..a50604437d --- /dev/null +++ b/internal/tools/protoc-gen-grpc-clone/internal/generate/generate.go @@ -0,0 +1,146 @@ +// Copyright (c) HashiCorp, Inc. +// SPDX-License-Identifier: BUSL-1.1 + +package generate + +import ( + "embed" + "fmt" + "path/filepath" + "text/template" + + "google.golang.org/protobuf/compiler/protogen" +) + +func Generate(gp *protogen.Plugin) error { + g := newGenerator(gp) + return g.generate() +} + +type generator struct { + p *protogen.Plugin + directories map[string]pkgInfo +} + +func newGenerator(gp *protogen.Plugin) *generator { + return &generator{ + p: gp, + directories: make(map[string]pkgInfo), + } +} + +type pkgInfo struct { + impPath protogen.GoImportPath + pkgName protogen.GoPackageName +} + +func (g *generator) generate() error { + for _, file := range g.p.Files { + if !file.Generate { + continue + } + + if len(file.Services) < 1 { + continue + } + + err := g.generateFile(file) + if err != nil { + return fmt.Errorf("Failed to generate file %q: %w", file.Proto.GetName(), err) + } + } + + for dir, info := range g.directories { + genFile := g.p.NewGeneratedFile(filepath.Join(dir, "cloning_stream.pb.go"), info.impPath) + cloningTemplate.ExecuteTemplate(genFile, "cloning-stream.tmpl", map[string]string{"GoPackage": string(info.pkgName)}) + } + return nil +} + +func (g *generator) generateFile(file *protogen.File) error { + tdata := &templateData{ + PackageName: string(file.GoPackageName), + } + + filename := file.GeneratedFilenamePrefix + "_cloning_grpc.pb.go" + genFile := g.p.NewGeneratedFile(filename, file.GoImportPath) + + for _, svc := range file.Services { + svcTypes := &cloningServiceTypes{ + ClientTypeName: genFile.QualifiedGoIdent(protogen.GoIdent{GoName: svc.GoName + "Client", GoImportPath: file.GoImportPath}), + ServerTypeName: genFile.QualifiedGoIdent(protogen.GoIdent{GoName: svc.GoName + "Server", GoImportPath: file.GoImportPath}), + CloningClientTypeName: genFile.QualifiedGoIdent(protogen.GoIdent{GoName: "Cloning" + svc.GoName + "Client", GoImportPath: file.GoImportPath}), + ServiceName: svc.GoName, + } + + tsvc := cloningService{ + cloningServiceTypes: svcTypes, + } + + for _, method := range svc.Methods { + if method.Desc.IsStreamingClient() { + // when we need these we can implement this + panic("client streams are unsupported") + } + + if method.Desc.IsStreamingServer() { + tsvc.ServerStreamMethods = append(tsvc.ServerStreamMethods, &inmemMethod{ + cloningServiceTypes: svcTypes, + Method: method, + }) + + // record that we need to also generate the inmem stream client code + // into this directory + g.directories[filepath.Dir(filename)] = pkgInfo{impPath: file.GoImportPath, pkgName: file.GoPackageName} + } else { + tsvc.UnaryMethods = append(tsvc.UnaryMethods, &inmemMethod{ + cloningServiceTypes: svcTypes, + Method: method, + }) + } + } + + tdata.Services = append(tdata.Services, &tsvc) + } + + err := cloningTemplate.ExecuteTemplate(genFile, "file.tmpl", &tdata) + if err != nil { + return fmt.Errorf("Error rendering template: %w", err) + } + + return nil + +} + +type templateData struct { + PackageName string + Services []*cloningService + UsesStreaming bool +} + +type cloningService struct { + UnaryMethods []*inmemMethod + // ClientStreamMethods []*protogen.Method + ServerStreamMethods []*inmemMethod + // BidirectionStreamMethods []*protogen.Method + *cloningServiceTypes +} + +type cloningServiceTypes struct { + ClientTypeName string + ServerTypeName string + ServiceName string + CloningClientTypeName string +} + +type inmemMethod struct { + Method *protogen.Method + *cloningServiceTypes +} + +var ( + //go:embed templates + templates embed.FS + + cloningTemplate = template.Must(template.ParseFS(templates, "templates/*")) +) diff --git a/internal/tools/protoc-gen-grpc-clone/internal/generate/templates/cloning-stream.tmpl b/internal/tools/protoc-gen-grpc-clone/internal/generate/templates/cloning-stream.tmpl new file mode 100644 index 0000000000..cbbc180a74 --- /dev/null +++ b/internal/tools/protoc-gen-grpc-clone/internal/generate/templates/cloning-stream.tmpl @@ -0,0 +1,31 @@ +// Code generated by protoc-gen-grpc-inmem. DO NOT EDIT. + +package {{.GoPackage}} + +import ( + grpc "google.golang.org/grpc" + "google.golang.org/protobuf/proto" +) + +type serverStream[T proto.Message] interface { + Recv() (T, error) + grpc.ClientStream +} + +type cloningStream[T proto.Message] struct { + serverStream[T] +} + +func newCloningStream[T proto.Message](stream serverStream[T]) cloningStream[T] { + return cloningStream[T]{serverStream: stream} +} + +func (st cloningStream[T]) Recv() (T, error) { + var zero T + val, err := st.serverStream.Recv() + if err != nil { + return zero, err + } + + return proto.Clone(val).(T), nil +} diff --git a/internal/tools/protoc-gen-grpc-clone/internal/generate/templates/file.tmpl b/internal/tools/protoc-gen-grpc-clone/internal/generate/templates/file.tmpl new file mode 100644 index 0000000000..d27c1a5319 --- /dev/null +++ b/internal/tools/protoc-gen-grpc-clone/internal/generate/templates/file.tmpl @@ -0,0 +1,14 @@ +// Code generated by protoc-gen-grpc-inmem. DO NOT EDIT. + +package {{ .PackageName }} + +import ( + "context" + + grpc "google.golang.org/grpc" + "google.golang.org/protobuf/proto" +) + +{{ range $service := .Services }} +{{ template "service.tmpl" $service }} +{{- end }} diff --git a/internal/tools/protoc-gen-grpc-clone/internal/generate/templates/server-stream-method.tmpl b/internal/tools/protoc-gen-grpc-clone/internal/generate/templates/server-stream-method.tmpl new file mode 100644 index 0000000000..e6a97bf087 --- /dev/null +++ b/internal/tools/protoc-gen-grpc-clone/internal/generate/templates/server-stream-method.tmpl @@ -0,0 +1,10 @@ +func (c {{.CloningClientTypeName}}) {{.Method.GoName}}(ctx context.Context, in *{{.Method.Input.GoIdent.GoName}}, opts...grpc.CallOption) ({{.ServiceName}}_{{.Method.GoName}}Client, error) { + in = proto.Clone(in).(*{{.Method.Input.GoIdent.GoName}}) + + st, err := c.{{.ClientTypeName}}.{{.Method.GoName}}(ctx, in) + if err != nil { + return nil, err + } + + return newCloningStream[*{{.Method.Output.GoIdent.GoName}}](st), nil +} diff --git a/internal/tools/protoc-gen-grpc-clone/internal/generate/templates/service.tmpl b/internal/tools/protoc-gen-grpc-clone/internal/generate/templates/service.tmpl new file mode 100644 index 0000000000..ebdfbffe25 --- /dev/null +++ b/internal/tools/protoc-gen-grpc-clone/internal/generate/templates/service.tmpl @@ -0,0 +1,44 @@ +// compile-time check to ensure that the generator is implementing all +// of the grpc client interfaces methods. +var _ {{.ClientTypeName}} = {{.CloningClientTypeName}}{} + +// Is{{.CloningClientTypeName}} is an interface that can be used to detect +// that a {{.ClientTypeName}} is using the in-memory transport and has already +// been wrapped with a with a {{.CloningClientTypeName}}. +type Is{{.CloningClientTypeName}} interface { + Is{{.CloningClientTypeName}}() bool +} + +// {{.CloningClientTypeName}} implements the {{.ClientTypeName}} interface by wrapping +// another implementation and copying all protobuf messages that pass through the client. +// This is mainly useful to wrap the an in-process client to insulate users of that +// client from having to care about potential immutability of data they receive or having +// the server implementation mutate their internal memory. +type {{.CloningClientTypeName}} struct { + {{.ClientTypeName}} +} + +func New{{.CloningClientTypeName}}(client {{.ClientTypeName}}) {{.ClientTypeName}} { + if cloner, ok := client.(Is{{.CloningClientTypeName}}); ok && cloner.Is{{.CloningClientTypeName}}() { + // prevent a double clone if the underlying client is already the cloning client. + return client + } + + return {{.CloningClientTypeName}}{ + {{.ClientTypeName}}: client, + } +} + +// Is{{.CloningClientTypeName}} implements the Is{{.CloningClientTypeName}} interface. This +// is only used to detect wrapped clients that would be double cloning data and prevent that. +func (c {{.CloningClientTypeName}}) Is{{.CloningClientTypeName}}() bool { + return true +} + +{{ range $method := .UnaryMethods }} +{{ template "unary-method.tmpl" $method }} +{{ end }} + +{{ range $method := .ServerStreamMethods }} +{{ template "server-stream-method.tmpl" $method }} +{{- end}} diff --git a/internal/tools/protoc-gen-grpc-clone/internal/generate/templates/unary-method.tmpl b/internal/tools/protoc-gen-grpc-clone/internal/generate/templates/unary-method.tmpl new file mode 100644 index 0000000000..be67543ca5 --- /dev/null +++ b/internal/tools/protoc-gen-grpc-clone/internal/generate/templates/unary-method.tmpl @@ -0,0 +1,10 @@ +func (c {{.CloningClientTypeName}}) {{.Method.GoName}}(ctx context.Context, in*{{.Method.Input.GoIdent.GoName}}, opts...grpc.CallOption) (*{{.Method.Output.GoIdent.GoName}}, error) { + in = proto.Clone(in).(*{{.Method.Input.GoIdent.GoName}}) + + out, err := c.{{.ClientTypeName}}.{{.Method.GoName}}(ctx, in) + if err != nil { + return nil, err + } + + return proto.Clone(out).(*{{.Method.Output.GoIdent.GoName}}), nil +} diff --git a/internal/tools/protoc-gen-grpc-clone/main.go b/internal/tools/protoc-gen-grpc-clone/main.go new file mode 100644 index 0000000000..d7a4d75c06 --- /dev/null +++ b/internal/tools/protoc-gen-grpc-clone/main.go @@ -0,0 +1,27 @@ +// Copyright (c) HashiCorp, Inc. +// SPDX-License-Identifier: BUSL-1.1 + +package main + +import ( + "flag" + + "github.com/hashicorp/consul/internal/tools/protoc-gen-grpc-clone/internal/generate" + "google.golang.org/protobuf/compiler/protogen" + plugin "google.golang.org/protobuf/types/pluginpb" +) + +var ( + file = flag.String("file", "-", "where to load data from") +) + +func main() { + flag.Parse() + + protogen.Options{ + ParamFunc: flag.CommandLine.Set, + }.Run(func(gp *protogen.Plugin) error { + gp.SupportedFeatures = uint64(plugin.CodeGeneratorResponse_FEATURE_PROTO3_OPTIONAL) + return generate.Generate(gp) + }) +} diff --git a/logging/names.go b/logging/names.go index 69a859443c..05d68a2fa6 100644 --- a/logging/names.go +++ b/logging/names.go @@ -17,6 +17,7 @@ const ( CentralConfig string = "central_config" ConfigEntry string = "config_entry" Connect string = "connect" + ConnectCA string = "connect-ca" Consul string = "consul" ConsulClient string = "client" ConsulServer string = "server" @@ -57,6 +58,7 @@ const ( Snapshot string = "snapshot" Partition string = "partition" Peering string = "peering" + PeerStream string = "peerstream" PeeringMetrics string = "peering_metrics" TCPRouteController string = "tcp_route_controller" TerminatingGateway string = "terminating_gateway" @@ -71,4 +73,8 @@ const ( XDSCapacityController string = "xds_capacity_controller" Vault string = "vault" Health string = "health" + GRPCAPI string = "grpc-api" + Resource string = "resource" + Dataplane string = "dataplane" + ServerDiscovery string = "server-discovery" ) diff --git a/proto-public/buf.gen.yaml b/proto-public/buf.gen.yaml index 98d07d5772..cc6c910412 100644 --- a/proto-public/buf.gen.yaml +++ b/proto-public/buf.gen.yaml @@ -37,3 +37,9 @@ plugins: - name: json-shim out: . opt: paths=source_relative + # Generate cloning clients that can be used in conjunction with github.com/fullstorydev/grpchan/inprocgrpc + # to safely do gRPC in-process + - name: grpc-clone + out: . + opt: + - paths=source_relative diff --git a/proto-public/pbacl/acl_cloning_grpc.pb.go b/proto-public/pbacl/acl_cloning_grpc.pb.go new file mode 100644 index 0000000000..efd1cf0b52 --- /dev/null +++ b/proto-public/pbacl/acl_cloning_grpc.pb.go @@ -0,0 +1,69 @@ +// Code generated by protoc-gen-grpc-inmem. DO NOT EDIT. + +package pbacl + +import ( + "context" + + grpc "google.golang.org/grpc" + "google.golang.org/protobuf/proto" +) + +// compile-time check to ensure that the generator is implementing all +// of the grpc client interfaces methods. +var _ ACLServiceClient = CloningACLServiceClient{} + +// IsCloningACLServiceClient is an interface that can be used to detect +// that a ACLServiceClient is using the in-memory transport and has already +// been wrapped with a with a CloningACLServiceClient. +type IsCloningACLServiceClient interface { + IsCloningACLServiceClient() bool +} + +// CloningACLServiceClient implements the ACLServiceClient interface by wrapping +// another implementation and copying all protobuf messages that pass through the client. +// This is mainly useful to wrap the an in-process client to insulate users of that +// client from having to care about potential immutability of data they receive or having +// the server implementation mutate their internal memory. +type CloningACLServiceClient struct { + ACLServiceClient +} + +func NewCloningACLServiceClient(client ACLServiceClient) ACLServiceClient { + if cloner, ok := client.(IsCloningACLServiceClient); ok && cloner.IsCloningACLServiceClient() { + // prevent a double clone if the underlying client is already the cloning client. + return client + } + + return CloningACLServiceClient{ + ACLServiceClient: client, + } +} + +// IsCloningACLServiceClient implements the IsCloningACLServiceClient interface. This +// is only used to detect wrapped clients that would be double cloning data and prevent that. +func (c CloningACLServiceClient) IsCloningACLServiceClient() bool { + return true +} + +func (c CloningACLServiceClient) Login(ctx context.Context, in *LoginRequest, opts ...grpc.CallOption) (*LoginResponse, error) { + in = proto.Clone(in).(*LoginRequest) + + out, err := c.ACLServiceClient.Login(ctx, in) + if err != nil { + return nil, err + } + + return proto.Clone(out).(*LoginResponse), nil +} + +func (c CloningACLServiceClient) Logout(ctx context.Context, in *LogoutRequest, opts ...grpc.CallOption) (*LogoutResponse, error) { + in = proto.Clone(in).(*LogoutRequest) + + out, err := c.ACLServiceClient.Logout(ctx, in) + if err != nil { + return nil, err + } + + return proto.Clone(out).(*LogoutResponse), nil +} diff --git a/proto-public/pbconnectca/ca_cloning_grpc.pb.go b/proto-public/pbconnectca/ca_cloning_grpc.pb.go new file mode 100644 index 0000000000..d8abc412f5 --- /dev/null +++ b/proto-public/pbconnectca/ca_cloning_grpc.pb.go @@ -0,0 +1,69 @@ +// Code generated by protoc-gen-grpc-inmem. DO NOT EDIT. + +package pbconnectca + +import ( + "context" + + grpc "google.golang.org/grpc" + "google.golang.org/protobuf/proto" +) + +// compile-time check to ensure that the generator is implementing all +// of the grpc client interfaces methods. +var _ ConnectCAServiceClient = CloningConnectCAServiceClient{} + +// IsCloningConnectCAServiceClient is an interface that can be used to detect +// that a ConnectCAServiceClient is using the in-memory transport and has already +// been wrapped with a with a CloningConnectCAServiceClient. +type IsCloningConnectCAServiceClient interface { + IsCloningConnectCAServiceClient() bool +} + +// CloningConnectCAServiceClient implements the ConnectCAServiceClient interface by wrapping +// another implementation and copying all protobuf messages that pass through the client. +// This is mainly useful to wrap the an in-process client to insulate users of that +// client from having to care about potential immutability of data they receive or having +// the server implementation mutate their internal memory. +type CloningConnectCAServiceClient struct { + ConnectCAServiceClient +} + +func NewCloningConnectCAServiceClient(client ConnectCAServiceClient) ConnectCAServiceClient { + if cloner, ok := client.(IsCloningConnectCAServiceClient); ok && cloner.IsCloningConnectCAServiceClient() { + // prevent a double clone if the underlying client is already the cloning client. + return client + } + + return CloningConnectCAServiceClient{ + ConnectCAServiceClient: client, + } +} + +// IsCloningConnectCAServiceClient implements the IsCloningConnectCAServiceClient interface. This +// is only used to detect wrapped clients that would be double cloning data and prevent that. +func (c CloningConnectCAServiceClient) IsCloningConnectCAServiceClient() bool { + return true +} + +func (c CloningConnectCAServiceClient) Sign(ctx context.Context, in *SignRequest, opts ...grpc.CallOption) (*SignResponse, error) { + in = proto.Clone(in).(*SignRequest) + + out, err := c.ConnectCAServiceClient.Sign(ctx, in) + if err != nil { + return nil, err + } + + return proto.Clone(out).(*SignResponse), nil +} + +func (c CloningConnectCAServiceClient) WatchRoots(ctx context.Context, in *WatchRootsRequest, opts ...grpc.CallOption) (ConnectCAService_WatchRootsClient, error) { + in = proto.Clone(in).(*WatchRootsRequest) + + st, err := c.ConnectCAServiceClient.WatchRoots(ctx, in) + if err != nil { + return nil, err + } + + return newCloningStream[*WatchRootsResponse](st), nil +} diff --git a/proto-public/pbconnectca/cloning_stream.pb.go b/proto-public/pbconnectca/cloning_stream.pb.go new file mode 100644 index 0000000000..fe7fca3821 --- /dev/null +++ b/proto-public/pbconnectca/cloning_stream.pb.go @@ -0,0 +1,31 @@ +// Code generated by protoc-gen-grpc-inmem. DO NOT EDIT. + +package pbconnectca + +import ( + grpc "google.golang.org/grpc" + "google.golang.org/protobuf/proto" +) + +type serverStream[T proto.Message] interface { + Recv() (T, error) + grpc.ClientStream +} + +type cloningStream[T proto.Message] struct { + serverStream[T] +} + +func newCloningStream[T proto.Message](stream serverStream[T]) cloningStream[T] { + return cloningStream[T]{serverStream: stream} +} + +func (st cloningStream[T]) Recv() (T, error) { + var zero T + val, err := st.serverStream.Recv() + if err != nil { + return zero, err + } + + return proto.Clone(val).(T), nil +} diff --git a/proto-public/pbdataplane/dataplane_cloning_grpc.pb.go b/proto-public/pbdataplane/dataplane_cloning_grpc.pb.go new file mode 100644 index 0000000000..c5a5b8b10b --- /dev/null +++ b/proto-public/pbdataplane/dataplane_cloning_grpc.pb.go @@ -0,0 +1,69 @@ +// Code generated by protoc-gen-grpc-inmem. DO NOT EDIT. + +package pbdataplane + +import ( + "context" + + grpc "google.golang.org/grpc" + "google.golang.org/protobuf/proto" +) + +// compile-time check to ensure that the generator is implementing all +// of the grpc client interfaces methods. +var _ DataplaneServiceClient = CloningDataplaneServiceClient{} + +// IsCloningDataplaneServiceClient is an interface that can be used to detect +// that a DataplaneServiceClient is using the in-memory transport and has already +// been wrapped with a with a CloningDataplaneServiceClient. +type IsCloningDataplaneServiceClient interface { + IsCloningDataplaneServiceClient() bool +} + +// CloningDataplaneServiceClient implements the DataplaneServiceClient interface by wrapping +// another implementation and copying all protobuf messages that pass through the client. +// This is mainly useful to wrap the an in-process client to insulate users of that +// client from having to care about potential immutability of data they receive or having +// the server implementation mutate their internal memory. +type CloningDataplaneServiceClient struct { + DataplaneServiceClient +} + +func NewCloningDataplaneServiceClient(client DataplaneServiceClient) DataplaneServiceClient { + if cloner, ok := client.(IsCloningDataplaneServiceClient); ok && cloner.IsCloningDataplaneServiceClient() { + // prevent a double clone if the underlying client is already the cloning client. + return client + } + + return CloningDataplaneServiceClient{ + DataplaneServiceClient: client, + } +} + +// IsCloningDataplaneServiceClient implements the IsCloningDataplaneServiceClient interface. This +// is only used to detect wrapped clients that would be double cloning data and prevent that. +func (c CloningDataplaneServiceClient) IsCloningDataplaneServiceClient() bool { + return true +} + +func (c CloningDataplaneServiceClient) GetSupportedDataplaneFeatures(ctx context.Context, in *GetSupportedDataplaneFeaturesRequest, opts ...grpc.CallOption) (*GetSupportedDataplaneFeaturesResponse, error) { + in = proto.Clone(in).(*GetSupportedDataplaneFeaturesRequest) + + out, err := c.DataplaneServiceClient.GetSupportedDataplaneFeatures(ctx, in) + if err != nil { + return nil, err + } + + return proto.Clone(out).(*GetSupportedDataplaneFeaturesResponse), nil +} + +func (c CloningDataplaneServiceClient) GetEnvoyBootstrapParams(ctx context.Context, in *GetEnvoyBootstrapParamsRequest, opts ...grpc.CallOption) (*GetEnvoyBootstrapParamsResponse, error) { + in = proto.Clone(in).(*GetEnvoyBootstrapParamsRequest) + + out, err := c.DataplaneServiceClient.GetEnvoyBootstrapParams(ctx, in) + if err != nil { + return nil, err + } + + return proto.Clone(out).(*GetEnvoyBootstrapParamsResponse), nil +} diff --git a/proto-public/pbdns/dns_cloning_grpc.pb.go b/proto-public/pbdns/dns_cloning_grpc.pb.go new file mode 100644 index 0000000000..4b32b80f58 --- /dev/null +++ b/proto-public/pbdns/dns_cloning_grpc.pb.go @@ -0,0 +1,58 @@ +// Code generated by protoc-gen-grpc-inmem. DO NOT EDIT. + +package pbdns + +import ( + "context" + + grpc "google.golang.org/grpc" + "google.golang.org/protobuf/proto" +) + +// compile-time check to ensure that the generator is implementing all +// of the grpc client interfaces methods. +var _ DNSServiceClient = CloningDNSServiceClient{} + +// IsCloningDNSServiceClient is an interface that can be used to detect +// that a DNSServiceClient is using the in-memory transport and has already +// been wrapped with a with a CloningDNSServiceClient. +type IsCloningDNSServiceClient interface { + IsCloningDNSServiceClient() bool +} + +// CloningDNSServiceClient implements the DNSServiceClient interface by wrapping +// another implementation and copying all protobuf messages that pass through the client. +// This is mainly useful to wrap the an in-process client to insulate users of that +// client from having to care about potential immutability of data they receive or having +// the server implementation mutate their internal memory. +type CloningDNSServiceClient struct { + DNSServiceClient +} + +func NewCloningDNSServiceClient(client DNSServiceClient) DNSServiceClient { + if cloner, ok := client.(IsCloningDNSServiceClient); ok && cloner.IsCloningDNSServiceClient() { + // prevent a double clone if the underlying client is already the cloning client. + return client + } + + return CloningDNSServiceClient{ + DNSServiceClient: client, + } +} + +// IsCloningDNSServiceClient implements the IsCloningDNSServiceClient interface. This +// is only used to detect wrapped clients that would be double cloning data and prevent that. +func (c CloningDNSServiceClient) IsCloningDNSServiceClient() bool { + return true +} + +func (c CloningDNSServiceClient) Query(ctx context.Context, in *QueryRequest, opts ...grpc.CallOption) (*QueryResponse, error) { + in = proto.Clone(in).(*QueryRequest) + + out, err := c.DNSServiceClient.Query(ctx, in) + if err != nil { + return nil, err + } + + return proto.Clone(out).(*QueryResponse), nil +} diff --git a/proto-public/pbresource/cloning_stream.pb.go b/proto-public/pbresource/cloning_stream.pb.go new file mode 100644 index 0000000000..09feb43ee0 --- /dev/null +++ b/proto-public/pbresource/cloning_stream.pb.go @@ -0,0 +1,31 @@ +// Code generated by protoc-gen-grpc-inmem. DO NOT EDIT. + +package pbresource + +import ( + grpc "google.golang.org/grpc" + "google.golang.org/protobuf/proto" +) + +type serverStream[T proto.Message] interface { + Recv() (T, error) + grpc.ClientStream +} + +type cloningStream[T proto.Message] struct { + serverStream[T] +} + +func newCloningStream[T proto.Message](stream serverStream[T]) cloningStream[T] { + return cloningStream[T]{serverStream: stream} +} + +func (st cloningStream[T]) Recv() (T, error) { + var zero T + val, err := st.serverStream.Recv() + if err != nil { + return zero, err + } + + return proto.Clone(val).(T), nil +} diff --git a/proto-public/pbresource/resource_cloning_grpc.pb.go b/proto-public/pbresource/resource_cloning_grpc.pb.go new file mode 100644 index 0000000000..e6512af20d --- /dev/null +++ b/proto-public/pbresource/resource_cloning_grpc.pb.go @@ -0,0 +1,124 @@ +// Code generated by protoc-gen-grpc-inmem. DO NOT EDIT. + +package pbresource + +import ( + "context" + + grpc "google.golang.org/grpc" + "google.golang.org/protobuf/proto" +) + +// compile-time check to ensure that the generator is implementing all +// of the grpc client interfaces methods. +var _ ResourceServiceClient = CloningResourceServiceClient{} + +// IsCloningResourceServiceClient is an interface that can be used to detect +// that a ResourceServiceClient is using the in-memory transport and has already +// been wrapped with a with a CloningResourceServiceClient. +type IsCloningResourceServiceClient interface { + IsCloningResourceServiceClient() bool +} + +// CloningResourceServiceClient implements the ResourceServiceClient interface by wrapping +// another implementation and copying all protobuf messages that pass through the client. +// This is mainly useful to wrap the an in-process client to insulate users of that +// client from having to care about potential immutability of data they receive or having +// the server implementation mutate their internal memory. +type CloningResourceServiceClient struct { + ResourceServiceClient +} + +func NewCloningResourceServiceClient(client ResourceServiceClient) ResourceServiceClient { + if cloner, ok := client.(IsCloningResourceServiceClient); ok && cloner.IsCloningResourceServiceClient() { + // prevent a double clone if the underlying client is already the cloning client. + return client + } + + return CloningResourceServiceClient{ + ResourceServiceClient: client, + } +} + +// IsCloningResourceServiceClient implements the IsCloningResourceServiceClient interface. This +// is only used to detect wrapped clients that would be double cloning data and prevent that. +func (c CloningResourceServiceClient) IsCloningResourceServiceClient() bool { + return true +} + +func (c CloningResourceServiceClient) Read(ctx context.Context, in *ReadRequest, opts ...grpc.CallOption) (*ReadResponse, error) { + in = proto.Clone(in).(*ReadRequest) + + out, err := c.ResourceServiceClient.Read(ctx, in) + if err != nil { + return nil, err + } + + return proto.Clone(out).(*ReadResponse), nil +} + +func (c CloningResourceServiceClient) Write(ctx context.Context, in *WriteRequest, opts ...grpc.CallOption) (*WriteResponse, error) { + in = proto.Clone(in).(*WriteRequest) + + out, err := c.ResourceServiceClient.Write(ctx, in) + if err != nil { + return nil, err + } + + return proto.Clone(out).(*WriteResponse), nil +} + +func (c CloningResourceServiceClient) WriteStatus(ctx context.Context, in *WriteStatusRequest, opts ...grpc.CallOption) (*WriteStatusResponse, error) { + in = proto.Clone(in).(*WriteStatusRequest) + + out, err := c.ResourceServiceClient.WriteStatus(ctx, in) + if err != nil { + return nil, err + } + + return proto.Clone(out).(*WriteStatusResponse), nil +} + +func (c CloningResourceServiceClient) List(ctx context.Context, in *ListRequest, opts ...grpc.CallOption) (*ListResponse, error) { + in = proto.Clone(in).(*ListRequest) + + out, err := c.ResourceServiceClient.List(ctx, in) + if err != nil { + return nil, err + } + + return proto.Clone(out).(*ListResponse), nil +} + +func (c CloningResourceServiceClient) ListByOwner(ctx context.Context, in *ListByOwnerRequest, opts ...grpc.CallOption) (*ListByOwnerResponse, error) { + in = proto.Clone(in).(*ListByOwnerRequest) + + out, err := c.ResourceServiceClient.ListByOwner(ctx, in) + if err != nil { + return nil, err + } + + return proto.Clone(out).(*ListByOwnerResponse), nil +} + +func (c CloningResourceServiceClient) Delete(ctx context.Context, in *DeleteRequest, opts ...grpc.CallOption) (*DeleteResponse, error) { + in = proto.Clone(in).(*DeleteRequest) + + out, err := c.ResourceServiceClient.Delete(ctx, in) + if err != nil { + return nil, err + } + + return proto.Clone(out).(*DeleteResponse), nil +} + +func (c CloningResourceServiceClient) WatchList(ctx context.Context, in *WatchListRequest, opts ...grpc.CallOption) (ResourceService_WatchListClient, error) { + in = proto.Clone(in).(*WatchListRequest) + + st, err := c.ResourceServiceClient.WatchList(ctx, in) + if err != nil { + return nil, err + } + + return newCloningStream[*WatchEvent](st), nil +} diff --git a/proto-public/pbserverdiscovery/cloning_stream.pb.go b/proto-public/pbserverdiscovery/cloning_stream.pb.go new file mode 100644 index 0000000000..4acd4356c4 --- /dev/null +++ b/proto-public/pbserverdiscovery/cloning_stream.pb.go @@ -0,0 +1,31 @@ +// Code generated by protoc-gen-grpc-inmem. DO NOT EDIT. + +package pbserverdiscovery + +import ( + grpc "google.golang.org/grpc" + "google.golang.org/protobuf/proto" +) + +type serverStream[T proto.Message] interface { + Recv() (T, error) + grpc.ClientStream +} + +type cloningStream[T proto.Message] struct { + serverStream[T] +} + +func newCloningStream[T proto.Message](stream serverStream[T]) cloningStream[T] { + return cloningStream[T]{serverStream: stream} +} + +func (st cloningStream[T]) Recv() (T, error) { + var zero T + val, err := st.serverStream.Recv() + if err != nil { + return zero, err + } + + return proto.Clone(val).(T), nil +} diff --git a/proto-public/pbserverdiscovery/serverdiscovery_cloning_grpc.pb.go b/proto-public/pbserverdiscovery/serverdiscovery_cloning_grpc.pb.go new file mode 100644 index 0000000000..ed8d3173d2 --- /dev/null +++ b/proto-public/pbserverdiscovery/serverdiscovery_cloning_grpc.pb.go @@ -0,0 +1,58 @@ +// Code generated by protoc-gen-grpc-inmem. DO NOT EDIT. + +package pbserverdiscovery + +import ( + "context" + + grpc "google.golang.org/grpc" + "google.golang.org/protobuf/proto" +) + +// compile-time check to ensure that the generator is implementing all +// of the grpc client interfaces methods. +var _ ServerDiscoveryServiceClient = CloningServerDiscoveryServiceClient{} + +// IsCloningServerDiscoveryServiceClient is an interface that can be used to detect +// that a ServerDiscoveryServiceClient is using the in-memory transport and has already +// been wrapped with a with a CloningServerDiscoveryServiceClient. +type IsCloningServerDiscoveryServiceClient interface { + IsCloningServerDiscoveryServiceClient() bool +} + +// CloningServerDiscoveryServiceClient implements the ServerDiscoveryServiceClient interface by wrapping +// another implementation and copying all protobuf messages that pass through the client. +// This is mainly useful to wrap the an in-process client to insulate users of that +// client from having to care about potential immutability of data they receive or having +// the server implementation mutate their internal memory. +type CloningServerDiscoveryServiceClient struct { + ServerDiscoveryServiceClient +} + +func NewCloningServerDiscoveryServiceClient(client ServerDiscoveryServiceClient) ServerDiscoveryServiceClient { + if cloner, ok := client.(IsCloningServerDiscoveryServiceClient); ok && cloner.IsCloningServerDiscoveryServiceClient() { + // prevent a double clone if the underlying client is already the cloning client. + return client + } + + return CloningServerDiscoveryServiceClient{ + ServerDiscoveryServiceClient: client, + } +} + +// IsCloningServerDiscoveryServiceClient implements the IsCloningServerDiscoveryServiceClient interface. This +// is only used to detect wrapped clients that would be double cloning data and prevent that. +func (c CloningServerDiscoveryServiceClient) IsCloningServerDiscoveryServiceClient() bool { + return true +} + +func (c CloningServerDiscoveryServiceClient) WatchServers(ctx context.Context, in *WatchServersRequest, opts ...grpc.CallOption) (ServerDiscoveryService_WatchServersClient, error) { + in = proto.Clone(in).(*WatchServersRequest) + + st, err := c.ServerDiscoveryServiceClient.WatchServers(ctx, in) + if err != nil { + return nil, err + } + + return newCloningStream[*WatchServersResponse](st), nil +} diff --git a/test/integration/consul-container/go.sum b/test/integration/consul-container/go.sum index 85dfc1389e..7adec3200a 100644 --- a/test/integration/consul-container/go.sum +++ b/test/integration/consul-container/go.sum @@ -144,6 +144,7 @@ github.com/fatih/color v1.9.0/go.mod h1:eQcE1qtQxscV5RaZvpXrrb8Drkc3/DdQ+uUYCNjL github.com/fatih/color v1.13.0/go.mod h1:kLAiJbzzSOZDVNGyDpeOxJ47H46qBXwg5ILebYFFOfk= github.com/fatih/color v1.14.1 h1:qfhVLaG5s+nCROl1zJsZRxFeYrHLqWroPOQ8BWiNb4w= github.com/fatih/color v1.14.1/go.mod h1:2oHN61fhTpgcxD3TSWCgKDiH1+x4OiDVVGH8WlgGZGg= +github.com/fullstorydev/grpchan v1.1.1 h1:heQqIJlAv5Cnks9a70GRL2EJke6QQoUB25VGR6TZQas= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU= github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= @@ -224,7 +225,6 @@ github.com/google/uuid v1.3.0 h1:t6JiXgmwXMjEs8VusXIJk2BXHsn+wx8BZdTaoZ5fu7I= github.com/google/uuid v1.3.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk= -github.com/grpc-ecosystem/go-grpc-middleware v1.3.0 h1:+9834+KizmvFV7pXQGSXQTsaWhq2GjuNUt0aUU0YBYw= github.com/grpc-ecosystem/grpc-gateway v1.16.0/go.mod h1:BDjrQk3hbvj6Nolgz8mAMFbcEtjT1g+wF4CSlocrBnw= github.com/grpc-ecosystem/grpc-gateway/v2 v2.7.0/go.mod h1:hgWBS7lorOAVIJEQMi4ZsPv9hVvWI6+ch50m39Pf2Ks= github.com/hashicorp/consul-net-rpc v0.0.0-20221205195236-156cfab66a69 h1:wzWurXrxfSyG1PHskIZlfuXlTSCj1Tsyatp9DtaasuY= @@ -306,6 +306,7 @@ github.com/itchyny/gojq v0.12.12 h1:x+xGI9BXqKoJQZkr95ibpe3cdrTbY8D9lonrK433rcA= github.com/itchyny/gojq v0.12.12/go.mod h1:j+3sVkjxwd7A7Z5jrbKibgOLn0ZfLWkV+Awxr/pyzJE= github.com/itchyny/timefmt-go v0.1.5 h1:G0INE2la8S6ru/ZI5JecgyzbbJNs5lG1RcBqa7Jm6GE= github.com/itchyny/timefmt-go v0.1.5/go.mod h1:nEP7L+2YmAbT2kZ2HfSs1d8Xtw9LY8D2stDBckWakZ8= +github.com/jhump/protoreflect v1.11.0 h1:bvACHUD1Ua/3VxY4aAMpItKMhhwbimlKFJKsLsVgDjU= github.com/jmespath/go-jmespath v0.4.0/go.mod h1:T8mJZnbsbmF+m6zOOFylbeCJqk5+pHWvzYPziyZiYoo= github.com/jmespath/go-jmespath/internal/testify v1.5.1/go.mod h1:L3OGu8Wl2/fWfCI6z80xFu9LTZmf1ZRjMHUOPmWr69U= github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= diff --git a/testing/deployer/topology/default_versions.go b/testing/deployer/topology/default_versions.go index ed1f583a52..9e052a44e6 100644 --- a/testing/deployer/topology/default_versions.go +++ b/testing/deployer/topology/default_versions.go @@ -6,8 +6,8 @@ package topology const ( - DefaultConsulCEImage = "hashicorp/consul:1.17.0" - DefaultConsulEnterpriseImage = "hashicorp/consul-enterprise:1.17.0-ent" - DefaultEnvoyImage = "envoyproxy/envoy:v1.27.2" - DefaultDataplaneImage = "hashicorp/consul-dataplane:1.3.0" + DefaultConsulCEImage = "hashicorp/consul:1.17.0" + DefaultConsulEnterpriseImage = "hashicorp/consul-enterprise:1.17.0-ent" + DefaultEnvoyImage = "envoyproxy/envoy:v1.27.2" + DefaultDataplaneImage = "hashicorp/consul-dataplane:1.3.0" )