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
This commit is contained in:
Matt Keeler 2024-01-12 11:54:07 -05:00 committed by GitHub
parent 98dcfaf783
commit 326c0ecfbe
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
70 changed files with 3665 additions and 438 deletions

View File

@ -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

View File

@ -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
}
}

View File

@ -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)

View File

@ -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

View File

@ -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,

View File

@ -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)

View File

@ -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) {

View File

@ -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

538
agent/consul/server_grpc.go Normal file
View File

@ -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
}

View File

@ -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 {

View File

@ -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 {

View File

@ -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)
}

View File

@ -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.

View File

@ -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 {

View File

@ -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,

View File

@ -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

View File

@ -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

View File

@ -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)
}

View File

@ -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)

View File

@ -12,7 +12,7 @@ import (
)
type GRPCService interface {
Register(*grpc.Server)
Register(grpc.ServiceRegistrar)
}
func RunTestServer(t *testing.T, services ...GRPCService) net.Addr {

View File

@ -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

View File

@ -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)

View File

@ -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()
}

View File

@ -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)

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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"

2
go.mod
View File

@ -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

8
go.sum
View File

@ -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=

View File

@ -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) {

93
internal/controller/cache/clone.go vendored Normal file
View File

@ -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)
}

214
internal/controller/cache/clone_test.go vendored Normal file
View File

@ -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)
}

View File

@ -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 {

View File

@ -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)

View File

@ -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)
}

View File

@ -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)
}

View File

@ -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),
}
}

View File

@ -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

View File

@ -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

View File

@ -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:

View File

@ -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)
}

View File

@ -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
}

View File

@ -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
}

View File

@ -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

View File

@ -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
}

View File

@ -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
}

View File

@ -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;
}

View File

@ -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
}

View File

@ -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",
}

View File

@ -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/*"))
)

View File

@ -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
}

View File

@ -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 }}

View File

@ -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
}

View File

@ -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}}

View File

@ -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
}

View File

@ -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)
})
}

View File

@ -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"
)

View File

@ -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

View File

@ -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
}

View File

@ -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
}

View File

@ -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
}

View File

@ -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
}

View File

@ -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
}

View File

@ -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
}

View File

@ -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
}

View File

@ -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
}

View File

@ -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
}

View File

@ -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=

View File

@ -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"
)