grpc: rename public/private directories to external/internal (#13721)
Previously, public referred to gRPC services that are both exposed on the dedicated gRPC port and have their definitions in the proto-public directory (so were considered usable by 3rd parties). Whereas private referred to services on the multiplexed server port that are only usable by agents and other servers. Now, we're splitting these definitions, such that external/internal refers to the port and public/private refers to whether they can be used by 3rd parties. This is necessary because the peering replication API needs to be exposed on the dedicated port, but is not (yet) suitable for use by 3rd parties.
This commit is contained in:
parent
c880728ab4
commit
34140ff3e0
|
@ -38,7 +38,7 @@ import (
|
|||
"github.com/hashicorp/consul/agent/config"
|
||||
"github.com/hashicorp/consul/agent/consul"
|
||||
"github.com/hashicorp/consul/agent/dns"
|
||||
publicgrpc "github.com/hashicorp/consul/agent/grpc/public"
|
||||
external "github.com/hashicorp/consul/agent/grpc-external"
|
||||
"github.com/hashicorp/consul/agent/local"
|
||||
"github.com/hashicorp/consul/agent/proxycfg"
|
||||
proxycfgglue "github.com/hashicorp/consul/agent/proxycfg-glue"
|
||||
|
@ -213,9 +213,9 @@ type Agent struct {
|
|||
// depending on the configuration
|
||||
delegate delegate
|
||||
|
||||
// publicGRPCServer is the gRPC server exposed on the dedicated gRPC port (as
|
||||
// externalGRPCServer is the gRPC server exposed on the dedicated gRPC port (as
|
||||
// opposed to the multiplexed "server" port).
|
||||
publicGRPCServer *grpc.Server
|
||||
externalGRPCServer *grpc.Server
|
||||
|
||||
// state stores a local representation of the node,
|
||||
// services and checks. Used for anti-entropy.
|
||||
|
@ -539,7 +539,7 @@ func (a *Agent) Start(ctx context.Context) error {
|
|||
|
||||
// This needs to happen after the initial auto-config is loaded, because TLS
|
||||
// can only be configured on the gRPC server at the point of creation.
|
||||
a.buildPublicGRPCServer()
|
||||
a.buildExternalGRPCServer()
|
||||
|
||||
if err := a.startLicenseManager(ctx); err != nil {
|
||||
return err
|
||||
|
@ -578,7 +578,7 @@ func (a *Agent) Start(ctx context.Context) error {
|
|||
|
||||
// Setup either the client or the server.
|
||||
if c.ServerMode {
|
||||
server, err := consul.NewServer(consulCfg, a.baseDeps.Deps, a.publicGRPCServer)
|
||||
server, err := consul.NewServer(consulCfg, a.baseDeps.Deps, a.externalGRPCServer)
|
||||
if err != nil {
|
||||
return fmt.Errorf("Failed to start Consul server: %v", err)
|
||||
}
|
||||
|
@ -760,13 +760,13 @@ func (a *Agent) Failed() <-chan struct{} {
|
|||
return a.apiServers.failed
|
||||
}
|
||||
|
||||
func (a *Agent) buildPublicGRPCServer() {
|
||||
func (a *Agent) buildExternalGRPCServer() {
|
||||
// TLS is only enabled on the gRPC server if there's an HTTPS port configured.
|
||||
var tls *tlsutil.Configurator
|
||||
if a.config.HTTPSPort > 0 {
|
||||
tls = a.tlsConfigurator
|
||||
}
|
||||
a.publicGRPCServer = publicgrpc.NewServer(a.logger.Named("grpc.public"), tls)
|
||||
a.externalGRPCServer = external.NewServer(a.logger.Named("grpc.external"), tls)
|
||||
}
|
||||
|
||||
func (a *Agent) listenAndServeGRPC() error {
|
||||
|
@ -803,7 +803,7 @@ func (a *Agent) listenAndServeGRPC() error {
|
|||
},
|
||||
a,
|
||||
)
|
||||
a.xdsServer.Register(a.publicGRPCServer)
|
||||
a.xdsServer.Register(a.externalGRPCServer)
|
||||
|
||||
ln, err := a.startListeners(a.config.GRPCAddrs)
|
||||
if err != nil {
|
||||
|
@ -816,7 +816,7 @@ func (a *Agent) listenAndServeGRPC() error {
|
|||
"address", innerL.Addr().String(),
|
||||
"network", innerL.Addr().Network(),
|
||||
)
|
||||
err := a.publicGRPCServer.Serve(innerL)
|
||||
err := a.externalGRPCServer.Serve(innerL)
|
||||
if err != nil {
|
||||
a.logger.Error("gRPC server failed", "error", err)
|
||||
}
|
||||
|
@ -1494,7 +1494,7 @@ func (a *Agent) ShutdownAgent() error {
|
|||
}
|
||||
|
||||
// Stop gRPC
|
||||
a.publicGRPCServer.Stop()
|
||||
a.externalGRPCServer.Stop()
|
||||
|
||||
// Stop the proxy config manager
|
||||
if a.proxyConfig != nil {
|
||||
|
|
|
@ -18,8 +18,8 @@ import (
|
|||
msgpackrpc "github.com/hashicorp/consul-net-rpc/net-rpc-msgpackrpc"
|
||||
|
||||
"github.com/hashicorp/consul/agent/consul/stream"
|
||||
grpc "github.com/hashicorp/consul/agent/grpc/private"
|
||||
"github.com/hashicorp/consul/agent/grpc/private/resolver"
|
||||
grpc "github.com/hashicorp/consul/agent/grpc-internal"
|
||||
"github.com/hashicorp/consul/agent/grpc-internal/resolver"
|
||||
"github.com/hashicorp/consul/agent/pool"
|
||||
"github.com/hashicorp/consul/agent/router"
|
||||
"github.com/hashicorp/consul/agent/rpc/middleware"
|
||||
|
|
|
@ -9,7 +9,7 @@ import (
|
|||
|
||||
"github.com/hashicorp/consul/agent/connect"
|
||||
"github.com/hashicorp/consul/agent/consul/authmethod/testauth"
|
||||
"github.com/hashicorp/consul/agent/grpc/public"
|
||||
external "github.com/hashicorp/consul/agent/grpc-external"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
tokenStore "github.com/hashicorp/consul/agent/token"
|
||||
"github.com/hashicorp/consul/proto-public/pbacl"
|
||||
|
@ -26,7 +26,7 @@ func TestGRPCIntegration_ConnectCA_Sign(t *testing.T) {
|
|||
// correctly wiring everything up in the server by:
|
||||
//
|
||||
// * Starting a cluster with multiple servers.
|
||||
// * Making a request to a follower's public gRPC port.
|
||||
// * Making a request to a follower's external gRPC port.
|
||||
// * Ensuring that the request is correctly forwarded to the leader.
|
||||
// * Ensuring we get a valid certificate back (so it went through the CAManager).
|
||||
server1, conn1, _ := testGRPCIntegrationServer(t, func(c *Config) {
|
||||
|
@ -59,7 +59,7 @@ func TestGRPCIntegration_ConnectCA_Sign(t *testing.T) {
|
|||
ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
|
||||
t.Cleanup(cancel)
|
||||
|
||||
ctx = public.ContextWithToken(ctx, TestDefaultInitialManagementToken)
|
||||
ctx = external.ContextWithToken(ctx, TestDefaultInitialManagementToken)
|
||||
|
||||
// This would fail if it wasn't forwarded to the leader.
|
||||
rsp, err := client.Sign(ctx, &pbconnectca.SignRequest{
|
||||
|
@ -96,7 +96,7 @@ func TestGRPCIntegration_ServerDiscovery_WatchServers(t *testing.T) {
|
|||
ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
|
||||
t.Cleanup(cancel)
|
||||
|
||||
ctx = public.ContextWithToken(ctx, TestDefaultInitialManagementToken)
|
||||
ctx = external.ContextWithToken(ctx, TestDefaultInitialManagementToken)
|
||||
|
||||
serverStream, err := client.WatchServers(ctx, &pbserverdiscovery.WatchServersRequest{Wan: false})
|
||||
require.NoError(t, err)
|
||||
|
|
|
@ -17,7 +17,7 @@ import (
|
|||
|
||||
"github.com/hashicorp/consul/acl"
|
||||
"github.com/hashicorp/consul/agent/consul/state"
|
||||
"github.com/hashicorp/consul/agent/grpc/public/services/peerstream"
|
||||
"github.com/hashicorp/consul/agent/grpc-external/services/peerstream"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/consul/api"
|
||||
"github.com/hashicorp/consul/logging"
|
||||
|
|
|
@ -8,7 +8,7 @@ import (
|
|||
"sync"
|
||||
|
||||
"github.com/hashicorp/consul/agent/consul/stream"
|
||||
"github.com/hashicorp/consul/agent/grpc/public/services/peerstream"
|
||||
"github.com/hashicorp/consul/agent/grpc-external/services/peerstream"
|
||||
"github.com/hashicorp/consul/agent/rpc/peering"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/consul/proto/pbpeering"
|
||||
|
|
|
@ -32,7 +32,7 @@ import (
|
|||
"github.com/hashicorp/consul/acl"
|
||||
"github.com/hashicorp/consul/agent/connect"
|
||||
"github.com/hashicorp/consul/agent/consul/state"
|
||||
agent_grpc "github.com/hashicorp/consul/agent/grpc/private"
|
||||
agent_grpc "github.com/hashicorp/consul/agent/grpc-internal"
|
||||
"github.com/hashicorp/consul/agent/pool"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
tokenStore "github.com/hashicorp/consul/agent/token"
|
||||
|
|
|
@ -17,7 +17,6 @@ import (
|
|||
"time"
|
||||
|
||||
"github.com/armon/go-metrics"
|
||||
"github.com/hashicorp/consul-net-rpc/net/rpc"
|
||||
connlimit "github.com/hashicorp/go-connlimit"
|
||||
"github.com/hashicorp/go-hclog"
|
||||
"github.com/hashicorp/go-memdb"
|
||||
|
@ -30,6 +29,8 @@ import (
|
|||
"golang.org/x/time/rate"
|
||||
"google.golang.org/grpc"
|
||||
|
||||
"github.com/hashicorp/consul-net-rpc/net/rpc"
|
||||
|
||||
"github.com/hashicorp/consul/acl"
|
||||
"github.com/hashicorp/consul/agent/consul/authmethod"
|
||||
"github.com/hashicorp/consul/agent/consul/authmethod/ssoauth"
|
||||
|
@ -38,13 +39,13 @@ import (
|
|||
"github.com/hashicorp/consul/agent/consul/stream"
|
||||
"github.com/hashicorp/consul/agent/consul/usagemetrics"
|
||||
"github.com/hashicorp/consul/agent/consul/wanfed"
|
||||
agentgrpc "github.com/hashicorp/consul/agent/grpc/private"
|
||||
"github.com/hashicorp/consul/agent/grpc/private/services/subscribe"
|
||||
aclgrpc "github.com/hashicorp/consul/agent/grpc/public/services/acl"
|
||||
"github.com/hashicorp/consul/agent/grpc/public/services/connectca"
|
||||
"github.com/hashicorp/consul/agent/grpc/public/services/dataplane"
|
||||
"github.com/hashicorp/consul/agent/grpc/public/services/peerstream"
|
||||
"github.com/hashicorp/consul/agent/grpc/public/services/serverdiscovery"
|
||||
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"
|
||||
"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/metadata"
|
||||
"github.com/hashicorp/consul/agent/pool"
|
||||
"github.com/hashicorp/consul/agent/router"
|
||||
|
@ -241,19 +242,19 @@ type Server struct {
|
|||
// is only ever closed.
|
||||
leaveCh chan struct{}
|
||||
|
||||
// publicACLServer serves the ACL service exposed on the public gRPC port.
|
||||
// It is also exposed on the private multiplexed "server" port to enable
|
||||
// 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.
|
||||
publicACLServer *aclgrpc.Server
|
||||
externalACLServer *aclgrpc.Server
|
||||
|
||||
// publicConnectCAServer serves the Connect CA service exposed on the public
|
||||
// gRPC port. It is also exposed on the private multiplexed "server" port to
|
||||
// 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.
|
||||
publicConnectCAServer *connectca.Server
|
||||
externalConnectCAServer *connectca.Server
|
||||
|
||||
// publicGRPCServer is the gRPC server exposed on the dedicated gRPC port, as
|
||||
// externalGRPCServer is the gRPC server exposed on the dedicated gRPC port, as
|
||||
// opposed to the multiplexed "server" port which is served by grpcHandler.
|
||||
publicGRPCServer *grpc.Server
|
||||
externalGRPCServer *grpc.Server
|
||||
|
||||
// router is used to map out Consul servers in the WAN and in Consul
|
||||
// Enterprise user-defined areas.
|
||||
|
@ -363,7 +364,7 @@ type Server struct {
|
|||
// this into the Deps struct and created it much earlier on.
|
||||
publisher *stream.EventPublisher
|
||||
|
||||
// peeringBackend is shared between the public and private gRPC services for peering
|
||||
// peeringBackend is shared between the external and internal gRPC services for peering
|
||||
peeringBackend *PeeringBackend
|
||||
|
||||
// peerStreamServer is a server used to handle peering streams
|
||||
|
@ -383,7 +384,7 @@ type connHandler interface {
|
|||
|
||||
// NewServer is used to construct a new Consul server from the configuration
|
||||
// and extra options, potentially returning an error.
|
||||
func NewServer(config *Config, flat Deps, publicGRPCServer *grpc.Server) (*Server, error) {
|
||||
func NewServer(config *Config, flat Deps, externalGRPCServer *grpc.Server) (*Server, error) {
|
||||
logger := flat.Logger
|
||||
if err := config.CheckProtocolVersion(); err != nil {
|
||||
return nil, err
|
||||
|
@ -429,7 +430,7 @@ func NewServer(config *Config, flat Deps, publicGRPCServer *grpc.Server) (*Serve
|
|||
reconcileCh: make(chan serf.Member, reconcileChSize),
|
||||
router: flat.Router,
|
||||
tlsConfigurator: flat.TLSConfigurator,
|
||||
publicGRPCServer: publicGRPCServer,
|
||||
externalGRPCServer: externalGRPCServer,
|
||||
reassertLeaderCh: make(chan chan error),
|
||||
sessionTimers: NewSessionTimers(),
|
||||
tombstoneGC: gc,
|
||||
|
@ -676,8 +677,8 @@ func NewServer(config *Config, flat Deps, publicGRPCServer *grpc.Server) (*Serve
|
|||
s.overviewManager = NewOverviewManager(s.logger, s.fsm, s.config.MetricsReportingInterval)
|
||||
go s.overviewManager.Run(&lib.StopChannelContext{StopCh: s.shutdownCh})
|
||||
|
||||
// Initialize public gRPC server - register services on public gRPC server.
|
||||
s.publicACLServer = aclgrpc.NewServer(aclgrpc.Config{
|
||||
// Initialize external gRPC server - register services on external gRPC server.
|
||||
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)
|
||||
|
@ -693,9 +694,9 @@ func NewServer(config *Config, flat Deps, publicGRPCServer *grpc.Server) (*Serve
|
|||
PrimaryDatacenter: s.config.PrimaryDatacenter,
|
||||
ValidateEnterpriseRequest: s.validateEnterpriseRequest,
|
||||
})
|
||||
s.publicACLServer.Register(s.publicGRPCServer)
|
||||
s.externalACLServer.Register(s.externalGRPCServer)
|
||||
|
||||
s.publicConnectCAServer = connectca.NewServer(connectca.Config{
|
||||
s.externalConnectCAServer = connectca.NewServer(connectca.Config{
|
||||
Publisher: s.publisher,
|
||||
GetStore: func() connectca.StateStore { return s.FSM().State() },
|
||||
Logger: logger.Named("grpc-api.connect-ca"),
|
||||
|
@ -706,20 +707,20 @@ func NewServer(config *Config, flat Deps, publicGRPCServer *grpc.Server) (*Serve
|
|||
},
|
||||
ConnectEnabled: s.config.ConnectEnabled,
|
||||
})
|
||||
s.publicConnectCAServer.Register(s.publicGRPCServer)
|
||||
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,
|
||||
}).Register(s.publicGRPCServer)
|
||||
}).Register(s.externalGRPCServer)
|
||||
|
||||
serverdiscovery.NewServer(serverdiscovery.Config{
|
||||
Publisher: s.publisher,
|
||||
ACLResolver: s.ACLResolver,
|
||||
Logger: logger.Named("grpc-api.server-discovery"),
|
||||
}).Register(s.publicGRPCServer)
|
||||
}).Register(s.externalGRPCServer)
|
||||
|
||||
s.peerStreamTracker = peerstream.NewTracker()
|
||||
s.peeringBackend = NewPeeringBackend(s)
|
||||
|
@ -732,11 +733,11 @@ func NewServer(config *Config, flat Deps, publicGRPCServer *grpc.Server) (*Serve
|
|||
Datacenter: s.config.Datacenter,
|
||||
ConnectEnabled: s.config.ConnectEnabled,
|
||||
})
|
||||
s.peerStreamServer.Register(s.publicGRPCServer)
|
||||
s.peerStreamServer.Register(s.externalGRPCServer)
|
||||
|
||||
// Initialize private gRPC server.
|
||||
// Initialize internal gRPC server.
|
||||
//
|
||||
// Note: some "public" gRPC services are also exposed on the private 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
|
||||
|
@ -803,10 +804,10 @@ func newGRPCHandlerFromConfig(deps Deps, config *Config, s *Server) connHandler
|
|||
s.peeringServer.Register(srv)
|
||||
s.registerEnterpriseGRPCServices(deps, srv)
|
||||
|
||||
// Note: these public gRPC services are also exposed on the private server to
|
||||
// Note: these external gRPC services are also exposed on the internal server to
|
||||
// enable RPC forwarding.
|
||||
s.publicACLServer.Register(srv)
|
||||
s.publicConnectCAServer.Register(srv)
|
||||
s.externalACLServer.Register(srv)
|
||||
s.externalConnectCAServer.Register(srv)
|
||||
}
|
||||
|
||||
return agentgrpc.NewHandler(deps.Logger, config.RPCAddr, register)
|
||||
|
|
|
@ -14,7 +14,6 @@ import (
|
|||
|
||||
"github.com/armon/go-metrics"
|
||||
"github.com/google/tcpproxy"
|
||||
"github.com/hashicorp/consul-net-rpc/net/rpc"
|
||||
"github.com/hashicorp/go-hclog"
|
||||
"github.com/hashicorp/go-uuid"
|
||||
"github.com/hashicorp/memberlist"
|
||||
|
@ -23,6 +22,8 @@ import (
|
|||
"golang.org/x/time/rate"
|
||||
"google.golang.org/grpc"
|
||||
|
||||
"github.com/hashicorp/consul-net-rpc/net/rpc"
|
||||
|
||||
"github.com/hashicorp/consul/agent/connect"
|
||||
"github.com/hashicorp/consul/agent/metadata"
|
||||
"github.com/hashicorp/consul/agent/rpc/middleware"
|
||||
|
@ -241,14 +242,14 @@ func testServerWithConfig(t *testing.T, configOpts ...func(*Config)) (string, *S
|
|||
if srv.config.GRPCPort > 0 {
|
||||
// Normally the gRPC server listener is created at the agent level and
|
||||
// passed down into the Server creation.
|
||||
publicGRPCAddr := fmt.Sprintf("127.0.0.1:%d", srv.config.GRPCPort)
|
||||
externalGRPCAddr := fmt.Sprintf("127.0.0.1:%d", srv.config.GRPCPort)
|
||||
|
||||
ln, err := net.Listen("tcp", publicGRPCAddr)
|
||||
ln, err := net.Listen("tcp", externalGRPCAddr)
|
||||
require.NoError(t, err)
|
||||
go func() {
|
||||
_ = srv.publicGRPCServer.Serve(ln)
|
||||
_ = srv.externalGRPCServer.Serve(ln)
|
||||
}()
|
||||
t.Cleanup(srv.publicGRPCServer.Stop)
|
||||
t.Cleanup(srv.externalGRPCServer.Stop)
|
||||
}
|
||||
|
||||
return dir, srv
|
||||
|
|
|
@ -5,7 +5,7 @@ import (
|
|||
|
||||
"github.com/hashicorp/consul/acl"
|
||||
"github.com/hashicorp/consul/agent/consul/stream"
|
||||
"github.com/hashicorp/consul/agent/grpc/private/services/subscribe"
|
||||
"github.com/hashicorp/consul/agent/grpc-internal/services/subscribe"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
)
|
||||
|
||||
|
|
|
@ -14,8 +14,8 @@ import (
|
|||
"golang.org/x/sync/errgroup"
|
||||
gogrpc "google.golang.org/grpc"
|
||||
|
||||
grpc "github.com/hashicorp/consul/agent/grpc/private"
|
||||
"github.com/hashicorp/consul/agent/grpc/private/resolver"
|
||||
grpc "github.com/hashicorp/consul/agent/grpc-internal"
|
||||
"github.com/hashicorp/consul/agent/grpc-internal/resolver"
|
||||
"github.com/hashicorp/consul/agent/router"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/consul/proto/pbservice"
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
package public
|
||||
package external
|
||||
|
||||
import (
|
||||
"context"
|
|
@ -1,4 +1,4 @@
|
|||
package public
|
||||
package external
|
||||
|
||||
import (
|
||||
middleware "github.com/grpc-ecosystem/go-grpc-middleware"
|
||||
|
@ -6,11 +6,11 @@ import (
|
|||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/credentials"
|
||||
|
||||
agentmiddleware "github.com/hashicorp/consul/agent/grpc/middleware"
|
||||
agentmiddleware "github.com/hashicorp/consul/agent/grpc-middleware"
|
||||
"github.com/hashicorp/consul/tlsutil"
|
||||
)
|
||||
|
||||
// NewServer constructs a gRPC server for the public gRPC port, to which
|
||||
// NewServer constructs a gRPC server for the external gRPC port, to which
|
||||
// handlers can be registered.
|
||||
func NewServer(logger agentmiddleware.Logger, tls *tlsutil.Configurator) *grpc.Server {
|
||||
recoveryOpts := agentmiddleware.PanicHandlerMiddlewareOpts(logger)
|
|
@ -10,14 +10,14 @@ import (
|
|||
|
||||
"github.com/hashicorp/consul/acl"
|
||||
"github.com/hashicorp/consul/agent/consul/auth"
|
||||
"github.com/hashicorp/consul/agent/grpc/public"
|
||||
external "github.com/hashicorp/consul/agent/grpc-external"
|
||||
"github.com/hashicorp/consul/proto-public/pbacl"
|
||||
)
|
||||
|
||||
// Login exchanges the presented bearer token for a Consul ACL token using a
|
||||
// configured auth method.
|
||||
func (s *Server) Login(ctx context.Context, req *pbacl.LoginRequest) (*pbacl.LoginResponse, error) {
|
||||
logger := s.Logger.Named("login").With("request_id", public.TraceID())
|
||||
logger := s.Logger.Named("login").With("request_id", external.TraceID())
|
||||
logger.Trace("request received")
|
||||
|
||||
if err := s.requireACLsEnabled(logger); err != nil {
|
|
@ -16,7 +16,7 @@ import (
|
|||
|
||||
"github.com/hashicorp/consul/acl"
|
||||
"github.com/hashicorp/consul/agent/consul/authmethod"
|
||||
"github.com/hashicorp/consul/agent/grpc/public/testutils"
|
||||
"github.com/hashicorp/consul/agent/grpc-external/testutils"
|
||||
structs "github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/consul/proto-public/pbacl"
|
||||
)
|
|
@ -10,13 +10,13 @@ import (
|
|||
|
||||
"github.com/hashicorp/consul/acl"
|
||||
"github.com/hashicorp/consul/agent/consul/auth"
|
||||
"github.com/hashicorp/consul/agent/grpc/public"
|
||||
external "github.com/hashicorp/consul/agent/grpc-external"
|
||||
"github.com/hashicorp/consul/proto-public/pbacl"
|
||||
)
|
||||
|
||||
// Logout destroys the given ACL token once the caller is done with it.
|
||||
func (s *Server) Logout(ctx context.Context, req *pbacl.LogoutRequest) (*pbacl.LogoutResponse, error) {
|
||||
logger := s.Logger.Named("logout").With("request_id", public.TraceID())
|
||||
logger := s.Logger.Named("logout").With("request_id", external.TraceID())
|
||||
logger.Trace("request received")
|
||||
|
||||
if err := s.requireACLsEnabled(logger); err != nil {
|
|
@ -15,7 +15,7 @@ import (
|
|||
|
||||
"github.com/hashicorp/consul/acl"
|
||||
"github.com/hashicorp/consul/agent/consul/auth"
|
||||
"github.com/hashicorp/consul/agent/grpc/public/testutils"
|
||||
"github.com/hashicorp/consul/agent/grpc-external/testutils"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/consul/proto-public/pbacl"
|
||||
)
|
|
@ -9,7 +9,7 @@ import (
|
|||
|
||||
"github.com/hashicorp/consul/agent/consul/state"
|
||||
"github.com/hashicorp/consul/agent/consul/stream"
|
||||
"github.com/hashicorp/consul/agent/grpc/public/testutils"
|
||||
"github.com/hashicorp/consul/agent/grpc-external/testutils"
|
||||
structs "github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/consul/proto-public/pbconnectca"
|
||||
)
|
|
@ -10,7 +10,7 @@ import (
|
|||
|
||||
"github.com/hashicorp/consul/acl"
|
||||
"github.com/hashicorp/consul/agent/connect"
|
||||
"github.com/hashicorp/consul/agent/grpc/public"
|
||||
external "github.com/hashicorp/consul/agent/grpc-external"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/consul/proto-public/pbconnectca"
|
||||
)
|
||||
|
@ -22,10 +22,10 @@ func (s *Server) Sign(ctx context.Context, req *pbconnectca.SignRequest) (*pbcon
|
|||
return nil, err
|
||||
}
|
||||
|
||||
logger := s.Logger.Named("sign").With("request_id", public.TraceID())
|
||||
logger := s.Logger.Named("sign").With("request_id", external.TraceID())
|
||||
logger.Trace("request received")
|
||||
|
||||
token := public.TokenFromContext(ctx)
|
||||
token := external.TokenFromContext(ctx)
|
||||
|
||||
if req.Csr == "" {
|
||||
return nil, status.Error(codes.InvalidArgument, "CSR is required")
|
||||
|
@ -48,7 +48,7 @@ func (s *Server) Sign(ctx context.Context, req *pbconnectca.SignRequest) (*pbcon
|
|||
var rsp *pbconnectca.SignResponse
|
||||
handled, err := s.ForwardRPC(&rpcInfo, func(conn *grpc.ClientConn) error {
|
||||
logger.Trace("forwarding RPC")
|
||||
ctx := public.ForwardMetadataContext(ctx)
|
||||
ctx := external.ForwardMetadataContext(ctx)
|
||||
var err error
|
||||
rsp, err = pbconnectca.NewConnectCAServiceClient(conn).Sign(ctx, req)
|
||||
return err
|
|
@ -16,7 +16,7 @@ import (
|
|||
acl "github.com/hashicorp/consul/acl"
|
||||
resolver "github.com/hashicorp/consul/acl/resolver"
|
||||
"github.com/hashicorp/consul/agent/connect"
|
||||
"github.com/hashicorp/consul/agent/grpc/public/testutils"
|
||||
"github.com/hashicorp/consul/agent/grpc-external/testutils"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/consul/proto-public/pbconnectca"
|
||||
)
|
|
@ -15,7 +15,7 @@ import (
|
|||
"github.com/hashicorp/consul/agent/connect"
|
||||
"github.com/hashicorp/consul/agent/consul/state"
|
||||
"github.com/hashicorp/consul/agent/consul/stream"
|
||||
"github.com/hashicorp/consul/agent/grpc/public"
|
||||
external "github.com/hashicorp/consul/agent/grpc-external"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/consul/proto-public/pbconnectca"
|
||||
)
|
||||
|
@ -28,11 +28,11 @@ func (s *Server) WatchRoots(_ *pbconnectca.WatchRootsRequest, serverStream pbcon
|
|||
return err
|
||||
}
|
||||
|
||||
logger := s.Logger.Named("watch-roots").With("request_id", public.TraceID())
|
||||
logger := s.Logger.Named("watch-roots").With("request_id", external.TraceID())
|
||||
logger.Trace("starting stream")
|
||||
defer logger.Trace("stream closed")
|
||||
|
||||
token := public.TokenFromContext(serverStream.Context())
|
||||
token := external.TokenFromContext(serverStream.Context())
|
||||
|
||||
// Serve the roots from an EventPublisher subscription. If the subscription is
|
||||
// closed due to an ACL change, we'll attempt to re-authorize and resume it to
|
|
@ -17,8 +17,8 @@ import (
|
|||
"github.com/hashicorp/consul/acl"
|
||||
resolver "github.com/hashicorp/consul/acl/resolver"
|
||||
"github.com/hashicorp/consul/agent/connect"
|
||||
"github.com/hashicorp/consul/agent/grpc/public"
|
||||
"github.com/hashicorp/consul/agent/grpc/public/testutils"
|
||||
external "github.com/hashicorp/consul/agent/grpc-external"
|
||||
"github.com/hashicorp/consul/agent/grpc-external/testutils"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/consul/proto-public/pbconnectca"
|
||||
"github.com/hashicorp/consul/sdk/testutil"
|
||||
|
@ -56,7 +56,7 @@ func TestWatchRoots_Success(t *testing.T) {
|
|||
aclResolver.On("ResolveTokenAndDefaultMeta", testACLToken, mock.Anything, mock.Anything).
|
||||
Return(testutils.TestAuthorizerServiceWriteAny(t), nil)
|
||||
|
||||
ctx := public.ContextWithToken(context.Background(), testACLToken)
|
||||
ctx := external.ContextWithToken(context.Background(), testACLToken)
|
||||
|
||||
server := NewServer(Config{
|
||||
Publisher: publisher,
|
||||
|
@ -104,7 +104,7 @@ func TestWatchRoots_InvalidACLToken(t *testing.T) {
|
|||
aclResolver.On("ResolveTokenAndDefaultMeta", mock.Anything, mock.Anything, mock.Anything).
|
||||
Return(resolver.Result{}, acl.ErrNotFound)
|
||||
|
||||
ctx := public.ContextWithToken(context.Background(), testACLToken)
|
||||
ctx := external.ContextWithToken(context.Background(), testACLToken)
|
||||
|
||||
server := NewServer(Config{
|
||||
Publisher: publisher,
|
||||
|
@ -142,7 +142,7 @@ func TestWatchRoots_ACLTokenInvalidated(t *testing.T) {
|
|||
aclResolver.On("ResolveTokenAndDefaultMeta", testACLToken, mock.Anything, mock.Anything).
|
||||
Return(testutils.TestAuthorizerServiceWriteAny(t), nil).Twice()
|
||||
|
||||
ctx := public.ContextWithToken(context.Background(), testACLToken)
|
||||
ctx := external.ContextWithToken(context.Background(), testACLToken)
|
||||
|
||||
server := NewServer(Config{
|
||||
Publisher: publisher,
|
||||
|
@ -210,7 +210,7 @@ func TestWatchRoots_StateStoreAbandoned(t *testing.T) {
|
|||
aclResolver.On("ResolveTokenAndDefaultMeta", testACLToken, mock.Anything, mock.Anything).
|
||||
Return(testutils.TestAuthorizerServiceWriteAny(t), nil)
|
||||
|
||||
ctx := public.ContextWithToken(context.Background(), testACLToken)
|
||||
ctx := external.ContextWithToken(context.Background(), testACLToken)
|
||||
|
||||
server := NewServer(Config{
|
||||
Publisher: publisher,
|
|
@ -13,8 +13,8 @@ import (
|
|||
|
||||
acl "github.com/hashicorp/consul/acl"
|
||||
resolver "github.com/hashicorp/consul/acl/resolver"
|
||||
"github.com/hashicorp/consul/agent/grpc/public"
|
||||
"github.com/hashicorp/consul/agent/grpc/public/testutils"
|
||||
external "github.com/hashicorp/consul/agent/grpc-external"
|
||||
"github.com/hashicorp/consul/agent/grpc-external/testutils"
|
||||
structs "github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/consul/proto-public/pbdataplane"
|
||||
"github.com/hashicorp/consul/types"
|
||||
|
@ -78,7 +78,7 @@ func TestGetEnvoyBootstrapParams_Success(t *testing.T) {
|
|||
aclResolver := &MockACLResolver{}
|
||||
aclResolver.On("ResolveTokenAndDefaultMeta", testToken, mock.Anything, mock.Anything).
|
||||
Return(testutils.TestAuthorizerServiceRead(t, tc.registerReq.Service.ID), nil)
|
||||
ctx := public.ContextWithToken(context.Background(), testToken)
|
||||
ctx := external.ContextWithToken(context.Background(), testToken)
|
||||
|
||||
server := NewServer(Config{
|
||||
GetStore: func() StateStore { return store },
|
||||
|
@ -148,7 +148,7 @@ func TestGetEnvoyBootstrapParams_Error(t *testing.T) {
|
|||
|
||||
aclResolver.On("ResolveTokenAndDefaultMeta", testToken, mock.Anything, mock.Anything).
|
||||
Return(testutils.TestAuthorizerServiceRead(t, proxyServiceID), nil)
|
||||
ctx := public.ContextWithToken(context.Background(), testToken)
|
||||
ctx := external.ContextWithToken(context.Background(), testToken)
|
||||
|
||||
store := testutils.TestStateStore(t, nil)
|
||||
registerReq := testRegisterRequestProxy(t)
|
||||
|
@ -218,7 +218,7 @@ func TestGetEnvoyBootstrapParams_Unauthenticated(t *testing.T) {
|
|||
aclResolver := &MockACLResolver{}
|
||||
aclResolver.On("ResolveTokenAndDefaultMeta", mock.Anything, mock.Anything, mock.Anything).
|
||||
Return(resolver.Result{}, acl.ErrNotFound)
|
||||
ctx := public.ContextWithToken(context.Background(), testToken)
|
||||
ctx := external.ContextWithToken(context.Background(), testToken)
|
||||
store := testutils.TestStateStore(t, nil)
|
||||
server := NewServer(Config{
|
||||
GetStore: func() StateStore { return store },
|
||||
|
@ -237,7 +237,7 @@ func TestGetEnvoyBootstrapParams_PermissionDenied(t *testing.T) {
|
|||
aclResolver := &MockACLResolver{}
|
||||
aclResolver.On("ResolveTokenAndDefaultMeta", testToken, mock.Anything, mock.Anything).
|
||||
Return(testutils.TestAuthorizerDenyAll(t), nil)
|
||||
ctx := public.ContextWithToken(context.Background(), testToken)
|
||||
ctx := external.ContextWithToken(context.Background(), testToken)
|
||||
store := testutils.TestStateStore(t, nil)
|
||||
registerReq := structs.TestRegisterRequestProxy(t)
|
||||
proxyServiceID := "web-sidecar-proxy"
|
|
@ -11,18 +11,18 @@ import (
|
|||
|
||||
acl "github.com/hashicorp/consul/acl"
|
||||
"github.com/hashicorp/consul/agent/consul/state"
|
||||
"github.com/hashicorp/consul/agent/grpc/public"
|
||||
external "github.com/hashicorp/consul/agent/grpc-external"
|
||||
structs "github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/consul/proto-public/pbdataplane"
|
||||
)
|
||||
|
||||
func (s *Server) GetEnvoyBootstrapParams(ctx context.Context, req *pbdataplane.GetEnvoyBootstrapParamsRequest) (*pbdataplane.GetEnvoyBootstrapParamsResponse, error) {
|
||||
logger := s.Logger.Named("get-envoy-bootstrap-params").With("service_id", req.GetServiceId(), "request_id", public.TraceID())
|
||||
logger := s.Logger.Named("get-envoy-bootstrap-params").With("service_id", req.GetServiceId(), "request_id", external.TraceID())
|
||||
|
||||
logger.Trace("Started processing request")
|
||||
defer logger.Trace("Finished processing request")
|
||||
|
||||
token := public.TokenFromContext(ctx)
|
||||
token := external.TokenFromContext(ctx)
|
||||
var authzContext acl.AuthorizerContext
|
||||
entMeta := acl.NewEnterpriseMetaWithPartition(req.GetPartition(), req.GetNamespace())
|
||||
authz, err := s.ACLResolver.ResolveTokenAndDefaultMeta(token, &entMeta, &authzContext)
|
|
@ -7,19 +7,19 @@ import (
|
|||
"google.golang.org/grpc/status"
|
||||
|
||||
acl "github.com/hashicorp/consul/acl"
|
||||
"github.com/hashicorp/consul/agent/grpc/public"
|
||||
external "github.com/hashicorp/consul/agent/grpc-external"
|
||||
structs "github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/consul/proto-public/pbdataplane"
|
||||
)
|
||||
|
||||
func (s *Server) GetSupportedDataplaneFeatures(ctx context.Context, req *pbdataplane.GetSupportedDataplaneFeaturesRequest) (*pbdataplane.GetSupportedDataplaneFeaturesResponse, error) {
|
||||
logger := s.Logger.Named("get-supported-dataplane-features").With("request_id", public.TraceID())
|
||||
logger := s.Logger.Named("get-supported-dataplane-features").With("request_id", external.TraceID())
|
||||
|
||||
logger.Trace("Started processing request")
|
||||
defer logger.Trace("Finished processing request")
|
||||
|
||||
// Require the given ACL token to have `service:write` on any service
|
||||
token := public.TokenFromContext(ctx)
|
||||
token := external.TokenFromContext(ctx)
|
||||
var authzContext acl.AuthorizerContext
|
||||
entMeta := structs.WildcardEnterpriseMetaInPartition(structs.WildcardSpecifier)
|
||||
authz, err := s.ACLResolver.ResolveTokenAndDefaultMeta(token, entMeta, &authzContext)
|
|
@ -12,8 +12,8 @@ import (
|
|||
|
||||
"github.com/hashicorp/consul/acl"
|
||||
resolver "github.com/hashicorp/consul/acl/resolver"
|
||||
"github.com/hashicorp/consul/agent/grpc/public"
|
||||
"github.com/hashicorp/consul/agent/grpc/public/testutils"
|
||||
external "github.com/hashicorp/consul/agent/grpc-external"
|
||||
"github.com/hashicorp/consul/agent/grpc-external/testutils"
|
||||
"github.com/hashicorp/consul/proto-public/pbdataplane"
|
||||
)
|
||||
|
||||
|
@ -24,7 +24,7 @@ func TestSupportedDataplaneFeatures_Success(t *testing.T) {
|
|||
aclResolver := &MockACLResolver{}
|
||||
aclResolver.On("ResolveTokenAndDefaultMeta", testACLToken, mock.Anything, mock.Anything).
|
||||
Return(testutils.TestAuthorizerServiceWriteAny(t), nil)
|
||||
ctx := public.ContextWithToken(context.Background(), testACLToken)
|
||||
ctx := external.ContextWithToken(context.Background(), testACLToken)
|
||||
server := NewServer(Config{
|
||||
Logger: hclog.NewNullLogger(),
|
||||
ACLResolver: aclResolver,
|
||||
|
@ -53,7 +53,7 @@ func TestSupportedDataplaneFeatures_Unauthenticated(t *testing.T) {
|
|||
aclResolver := &MockACLResolver{}
|
||||
aclResolver.On("ResolveTokenAndDefaultMeta", mock.Anything, mock.Anything, mock.Anything).
|
||||
Return(resolver.Result{}, acl.ErrNotFound)
|
||||
ctx := public.ContextWithToken(context.Background(), testACLToken)
|
||||
ctx := external.ContextWithToken(context.Background(), testACLToken)
|
||||
server := NewServer(Config{
|
||||
Logger: hclog.NewNullLogger(),
|
||||
ACLResolver: aclResolver,
|
||||
|
@ -70,7 +70,7 @@ func TestSupportedDataplaneFeatures_PermissionDenied(t *testing.T) {
|
|||
aclResolver := &MockACLResolver{}
|
||||
aclResolver.On("ResolveTokenAndDefaultMeta", testACLToken, mock.Anything, mock.Anything).
|
||||
Return(testutils.TestAuthorizerDenyAll(t), nil)
|
||||
ctx := public.ContextWithToken(context.Background(), testACLToken)
|
||||
ctx := external.ContextWithToken(context.Background(), testACLToken)
|
||||
server := NewServer(Config{
|
||||
Logger: hclog.NewNullLogger(),
|
||||
ACLResolver: aclResolver,
|
|
@ -4,7 +4,7 @@ import (
|
|||
"context"
|
||||
"testing"
|
||||
|
||||
"github.com/hashicorp/consul/agent/grpc/public/testutils"
|
||||
"github.com/hashicorp/consul/agent/grpc-external/testutils"
|
||||
"github.com/hashicorp/consul/proto-public/pbdataplane"
|
||||
"github.com/stretchr/testify/require"
|
||||
"google.golang.org/grpc"
|
|
@ -14,7 +14,7 @@ import (
|
|||
grpcstatus "google.golang.org/grpc/status"
|
||||
|
||||
"github.com/hashicorp/consul/agent/connect"
|
||||
"github.com/hashicorp/consul/agent/grpc/public"
|
||||
external "github.com/hashicorp/consul/agent/grpc-external"
|
||||
"github.com/hashicorp/consul/proto/pbpeering"
|
||||
"github.com/hashicorp/consul/proto/pbpeerstream"
|
||||
)
|
||||
|
@ -27,7 +27,7 @@ type BidirectionalStream interface {
|
|||
|
||||
// StreamResources handles incoming streaming connections.
|
||||
func (s *Server) StreamResources(stream pbpeerstream.PeerStreamService_StreamResourcesServer) error {
|
||||
logger := s.Logger.Named("stream-resources").With("request_id", public.TraceID())
|
||||
logger := s.Logger.Named("stream-resources").With("request_id", external.TraceID())
|
||||
|
||||
logger.Trace("Started processing request")
|
||||
defer logger.Trace("Finished processing request")
|
|
@ -12,7 +12,7 @@ import (
|
|||
"github.com/hashicorp/consul/agent/consul/autopilotevents"
|
||||
"github.com/hashicorp/consul/agent/consul/state"
|
||||
"github.com/hashicorp/consul/agent/consul/stream"
|
||||
"github.com/hashicorp/consul/agent/grpc/public/testutils"
|
||||
"github.com/hashicorp/consul/agent/grpc-external/testutils"
|
||||
"github.com/hashicorp/consul/proto-public/pbserverdiscovery"
|
||||
)
|
||||
|
|
@ -4,15 +4,16 @@ import (
|
|||
"context"
|
||||
"errors"
|
||||
|
||||
"github.com/hashicorp/consul/acl"
|
||||
"github.com/hashicorp/consul/agent/consul/autopilotevents"
|
||||
"github.com/hashicorp/consul/agent/consul/stream"
|
||||
"github.com/hashicorp/consul/agent/grpc/public"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/consul/proto-public/pbserverdiscovery"
|
||||
"github.com/hashicorp/go-hclog"
|
||||
"google.golang.org/grpc/codes"
|
||||
"google.golang.org/grpc/status"
|
||||
|
||||
"github.com/hashicorp/consul/acl"
|
||||
"github.com/hashicorp/consul/agent/consul/autopilotevents"
|
||||
"github.com/hashicorp/consul/agent/consul/stream"
|
||||
external "github.com/hashicorp/consul/agent/grpc-external"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/consul/proto-public/pbserverdiscovery"
|
||||
)
|
||||
|
||||
// WatchServers provides a stream on which you can receive the list of servers
|
||||
|
@ -20,12 +21,12 @@ import (
|
|||
// current set of ready servers are sent immediately at the start of the
|
||||
// stream and new updates will be sent whenver the set of ready servers changes.
|
||||
func (s *Server) WatchServers(req *pbserverdiscovery.WatchServersRequest, serverStream pbserverdiscovery.ServerDiscoveryService_WatchServersServer) error {
|
||||
logger := s.Logger.Named("watch-servers").With("request_id", public.TraceID())
|
||||
logger := s.Logger.Named("watch-servers").With("request_id", external.TraceID())
|
||||
|
||||
logger.Debug("starting stream")
|
||||
defer logger.Trace("stream closed")
|
||||
|
||||
token := public.TokenFromContext(serverStream.Context())
|
||||
token := external.TokenFromContext(serverStream.Context())
|
||||
|
||||
// Serve the ready servers from an EventPublisher subscription. If the subscription is
|
||||
// closed due to an ACL change, we'll attempt to re-authorize and resume it to
|
|
@ -16,8 +16,8 @@ import (
|
|||
resolver "github.com/hashicorp/consul/acl/resolver"
|
||||
"github.com/hashicorp/consul/agent/consul/autopilotevents"
|
||||
"github.com/hashicorp/consul/agent/consul/stream"
|
||||
"github.com/hashicorp/consul/agent/grpc/public"
|
||||
"github.com/hashicorp/consul/agent/grpc/public/testutils"
|
||||
external "github.com/hashicorp/consul/agent/grpc-external"
|
||||
"github.com/hashicorp/consul/agent/grpc-external/testutils"
|
||||
"github.com/hashicorp/consul/proto-public/pbserverdiscovery"
|
||||
"github.com/hashicorp/consul/proto/prototest"
|
||||
"github.com/hashicorp/consul/sdk/testutil"
|
||||
|
@ -125,7 +125,7 @@ func TestWatchServers_StreamLifeCycle(t *testing.T) {
|
|||
Return(testutils.TestAuthorizerServiceWriteAny(t), nil).Twice()
|
||||
|
||||
// add the token to the requests context
|
||||
ctx := public.ContextWithToken(context.Background(), testACLToken)
|
||||
ctx := external.ContextWithToken(context.Background(), testACLToken)
|
||||
|
||||
// setup the server
|
||||
server := NewServer(Config{
|
||||
|
@ -198,7 +198,7 @@ func TestWatchServers_ACLToken_PermissionDenied(t *testing.T) {
|
|||
Return(testutils.TestAuthorizerDenyAll(t), nil).Once()
|
||||
|
||||
// add the token to the requests context
|
||||
ctx := public.ContextWithToken(context.Background(), testACLToken)
|
||||
ctx := external.ContextWithToken(context.Background(), testACLToken)
|
||||
|
||||
// setup the server
|
||||
server := NewServer(Config{
|
||||
|
@ -229,7 +229,7 @@ func TestWatchServers_ACLToken_Unauthenticated(t *testing.T) {
|
|||
Return(resolver.Result{}, acl.ErrNotFound).Once()
|
||||
|
||||
// add the token to the requests context
|
||||
ctx := public.ContextWithToken(context.Background(), testACLToken)
|
||||
ctx := external.ContextWithToken(context.Background(), testACLToken)
|
||||
|
||||
// setup the server
|
||||
server := NewServer(Config{
|
|
@ -1,4 +1,4 @@
|
|||
package public
|
||||
package external
|
||||
|
||||
import (
|
||||
"context"
|
|
@ -1,4 +1,4 @@
|
|||
package public
|
||||
package external
|
||||
|
||||
import "github.com/hashicorp/go-uuid"
|
||||
|
|
@ -1,4 +1,4 @@
|
|||
package private
|
||||
package internal
|
||||
|
||||
import (
|
||||
"context"
|
|
@ -1,4 +1,4 @@
|
|||
package private
|
||||
package internal
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
@ -14,8 +14,8 @@ import (
|
|||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
|
||||
"github.com/hashicorp/consul/agent/grpc/private/internal/testservice"
|
||||
"github.com/hashicorp/consul/agent/grpc/private/resolver"
|
||||
"github.com/hashicorp/consul/agent/grpc-internal/internal/testservice"
|
||||
"github.com/hashicorp/consul/agent/grpc-internal/resolver"
|
||||
"github.com/hashicorp/consul/agent/metadata"
|
||||
"github.com/hashicorp/consul/ipaddr"
|
||||
"github.com/hashicorp/consul/sdk/freeport"
|
||||
|
@ -145,9 +145,9 @@ func TestNewDialer_IntegrationWithTLSEnabledHandler(t *testing.T) {
|
|||
tlsConf, err := tlsutil.NewConfigurator(tlsutil.Config{
|
||||
InternalRPC: tlsutil.ProtocolConfig{
|
||||
VerifyIncoming: true,
|
||||
CAFile: "../../../test/hostname/CertAuth.crt",
|
||||
CertFile: "../../../test/hostname/Alice.crt",
|
||||
KeyFile: "../../../test/hostname/Alice.key",
|
||||
CAFile: "../../test/hostname/CertAuth.crt",
|
||||
CertFile: "../../test/hostname/Alice.crt",
|
||||
KeyFile: "../../test/hostname/Alice.key",
|
||||
VerifyOutgoing: true,
|
||||
},
|
||||
}, hclog.New(nil))
|
||||
|
@ -192,9 +192,9 @@ func TestNewDialer_IntegrationWithTLSEnabledHandler_viaMeshGateway(t *testing.T)
|
|||
tlsConf, err := tlsutil.NewConfigurator(tlsutil.Config{
|
||||
InternalRPC: tlsutil.ProtocolConfig{
|
||||
VerifyIncoming: true,
|
||||
CAFile: "../../../test/hostname/CertAuth.crt",
|
||||
CertFile: "../../../test/hostname/Bob.crt",
|
||||
KeyFile: "../../../test/hostname/Bob.key",
|
||||
CAFile: "../../test/hostname/CertAuth.crt",
|
||||
CertFile: "../../test/hostname/Bob.crt",
|
||||
KeyFile: "../../test/hostname/Bob.key",
|
||||
VerifyOutgoing: true,
|
||||
VerifyServerHostname: true,
|
||||
},
|
||||
|
@ -222,9 +222,9 @@ func TestNewDialer_IntegrationWithTLSEnabledHandler_viaMeshGateway(t *testing.T)
|
|||
clientTLSConf, err := tlsutil.NewConfigurator(tlsutil.Config{
|
||||
InternalRPC: tlsutil.ProtocolConfig{
|
||||
VerifyIncoming: true,
|
||||
CAFile: "../../../test/hostname/CertAuth.crt",
|
||||
CertFile: "../../../test/hostname/Betty.crt",
|
||||
KeyFile: "../../../test/hostname/Betty.key",
|
||||
CAFile: "../../test/hostname/CertAuth.crt",
|
||||
CertFile: "../../test/hostname/Betty.crt",
|
||||
KeyFile: "../../test/hostname/Betty.key",
|
||||
VerifyOutgoing: true,
|
||||
VerifyServerHostname: true,
|
||||
},
|
|
@ -1,11 +1,11 @@
|
|||
package private
|
||||
package internal
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"net"
|
||||
"time"
|
||||
|
||||
agentmiddleware "github.com/hashicorp/consul/agent/grpc/middleware"
|
||||
agentmiddleware "github.com/hashicorp/consul/agent/grpc-middleware"
|
||||
|
||||
middleware "github.com/grpc-ecosystem/go-grpc-middleware"
|
||||
recovery "github.com/grpc-ecosystem/go-grpc-middleware/recovery"
|
|
@ -1,4 +1,4 @@
|
|||
package private
|
||||
package internal
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
|
@ -13,8 +13,8 @@ import (
|
|||
"google.golang.org/grpc/codes"
|
||||
"google.golang.org/grpc/status"
|
||||
|
||||
"github.com/hashicorp/consul/agent/grpc/private/internal/testservice"
|
||||
"github.com/hashicorp/consul/agent/grpc/private/resolver"
|
||||
"github.com/hashicorp/consul/agent/grpc-internal/internal/testservice"
|
||||
"github.com/hashicorp/consul/agent/grpc-internal/resolver"
|
||||
)
|
||||
|
||||
func TestHandler_PanicRecoveryInterceptor(t *testing.T) {
|
||||
|
@ -57,5 +57,5 @@ func TestHandler_PanicRecoveryInterceptor(t *testing.T) {
|
|||
// Checking the entire stack trace is not possible, let's
|
||||
// make sure that it contains a couple of expected strings.
|
||||
require.Contains(t, strLog, `[ERROR] panic serving grpc request: panic="panic from Something`)
|
||||
require.Contains(t, strLog, `github.com/hashicorp/consul/agent/grpc/private.(*simplePanic).Something`)
|
||||
require.Contains(t, strLog, `github.com/hashicorp/consul/agent/grpc-internal.(*simplePanic).Something`)
|
||||
}
|
|
@ -1,5 +1,5 @@
|
|||
// Code generated by protoc-gen-go-binary. DO NOT EDIT.
|
||||
// source: agent/grpc/private/internal/testservice/simple.proto
|
||||
// source: agent/grpc-internal/internal/testservice/simple.proto
|
||||
|
||||
package testservice
|
||||
|
|
@ -2,7 +2,7 @@
|
|||
// versions:
|
||||
// protoc-gen-go v1.23.0
|
||||
// protoc v3.15.8
|
||||
// source: agent/grpc/private/internal/testservice/simple.proto
|
||||
// source: agent/grpc-internal/internal/testservice/simple.proto
|
||||
|
||||
package testservice
|
||||
|
||||
|
@ -376,5 +376,5 @@ var _Simple_serviceDesc = grpc.ServiceDesc{
|
|||
ServerStreams: true,
|
||||
},
|
||||
},
|
||||
Metadata: "agent/grpc/private/internal/testservice/simple.proto",
|
||||
Metadata: "agent/grpc-internal/internal/testservice/simple.proto",
|
||||
}
|
|
@ -2,7 +2,7 @@ syntax = "proto3";
|
|||
|
||||
package testservice;
|
||||
|
||||
option go_package = "github.com/hashicorp/consul/agent/grpc/private/internal/testservice";
|
||||
option go_package = "github.com/hashicorp/consul/agent/grpc-internal/internal/testservice";
|
||||
|
||||
// Simple service is used to test gRPC plumbing.
|
||||
service Simple {
|
|
@ -1,4 +1,4 @@
|
|||
package private
|
||||
package internal
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
@ -15,7 +15,7 @@ import (
|
|||
"golang.org/x/sync/errgroup"
|
||||
"google.golang.org/grpc"
|
||||
|
||||
"github.com/hashicorp/consul/agent/grpc/private/internal/testservice"
|
||||
"github.com/hashicorp/consul/agent/grpc-internal/internal/testservice"
|
||||
"github.com/hashicorp/consul/agent/metadata"
|
||||
"github.com/hashicorp/consul/agent/pool"
|
||||
"github.com/hashicorp/consul/tlsutil"
|
|
@ -21,7 +21,7 @@ import (
|
|||
"github.com/hashicorp/consul/acl"
|
||||
"github.com/hashicorp/consul/agent/consul/state"
|
||||
"github.com/hashicorp/consul/agent/consul/stream"
|
||||
grpc "github.com/hashicorp/consul/agent/grpc/private"
|
||||
grpc "github.com/hashicorp/consul/agent/grpc-internal"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/consul/api"
|
||||
"github.com/hashicorp/consul/proto/pbcommon"
|
|
@ -1,4 +1,4 @@
|
|||
package private
|
||||
package internal
|
||||
|
||||
import (
|
||||
"context"
|
|
@ -1,4 +1,4 @@
|
|||
package private
|
||||
package internal
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
@ -14,9 +14,10 @@ import (
|
|||
"golang.org/x/sync/errgroup"
|
||||
"google.golang.org/grpc"
|
||||
|
||||
"github.com/hashicorp/consul/agent/grpc/private/internal/testservice"
|
||||
"github.com/hashicorp/consul/proto/prototest"
|
||||
"github.com/hashicorp/go-hclog"
|
||||
|
||||
"github.com/hashicorp/consul/agent/grpc-internal/internal/testservice"
|
||||
"github.com/hashicorp/consul/proto/prototest"
|
||||
)
|
||||
|
||||
func noopRegister(*grpc.Server) {}
|
|
@ -23,26 +23,26 @@ func (k *Key) Equal(x *Key) bool {
|
|||
|
||||
// Server is used to return details of a consul server
|
||||
type Server struct {
|
||||
Name string // <node>.<dc>
|
||||
ShortName string // <node>
|
||||
ID string
|
||||
Datacenter string
|
||||
Segment string
|
||||
Port int
|
||||
SegmentAddrs map[string]string
|
||||
SegmentPorts map[string]int
|
||||
WanJoinPort int
|
||||
LanJoinPort int
|
||||
PublicGRPCPort int
|
||||
Bootstrap bool
|
||||
Expect int
|
||||
Build version.Version
|
||||
Version int
|
||||
RaftVersion int
|
||||
Addr net.Addr
|
||||
Status serf.MemberStatus
|
||||
ReadReplica bool
|
||||
FeatureFlags map[string]int
|
||||
Name string // <node>.<dc>
|
||||
ShortName string // <node>
|
||||
ID string
|
||||
Datacenter string
|
||||
Segment string
|
||||
Port int
|
||||
SegmentAddrs map[string]string
|
||||
SegmentPorts map[string]int
|
||||
WanJoinPort int
|
||||
LanJoinPort int
|
||||
ExternalGRPCPort int
|
||||
Bootstrap bool
|
||||
Expect int
|
||||
Build version.Version
|
||||
Version int
|
||||
RaftVersion int
|
||||
Addr net.Addr
|
||||
Status serf.MemberStatus
|
||||
ReadReplica bool
|
||||
FeatureFlags map[string]int
|
||||
|
||||
// If true, use TLS when connecting to this server
|
||||
UseTLS bool
|
||||
|
@ -137,14 +137,14 @@ func IsConsulServer(m serf.Member) (bool, *Server) {
|
|||
}
|
||||
}
|
||||
|
||||
publicGRPCPort := 0
|
||||
publicGRPCPortStr, ok := m.Tags["grpc_port"]
|
||||
externalGRPCPort := 0
|
||||
externalGRPCPortStr, ok := m.Tags["grpc_port"]
|
||||
if ok {
|
||||
publicGRPCPort, err = strconv.Atoi(publicGRPCPortStr)
|
||||
externalGRPCPort, err = strconv.Atoi(externalGRPCPortStr)
|
||||
if err != nil {
|
||||
return false, nil
|
||||
}
|
||||
if publicGRPCPort < 1 {
|
||||
if externalGRPCPort < 1 {
|
||||
return false, nil
|
||||
}
|
||||
}
|
||||
|
@ -173,25 +173,25 @@ func IsConsulServer(m serf.Member) (bool, *Server) {
|
|||
addr := &net.TCPAddr{IP: m.Addr, Port: port}
|
||||
|
||||
parts := &Server{
|
||||
Name: m.Name,
|
||||
ShortName: strings.TrimSuffix(m.Name, "."+datacenter),
|
||||
ID: m.Tags["id"],
|
||||
Datacenter: datacenter,
|
||||
Segment: segment,
|
||||
Port: port,
|
||||
SegmentAddrs: segmentAddrs,
|
||||
SegmentPorts: segmentPorts,
|
||||
WanJoinPort: wanJoinPort,
|
||||
LanJoinPort: int(m.Port),
|
||||
PublicGRPCPort: publicGRPCPort,
|
||||
Bootstrap: bootstrap,
|
||||
Expect: expect,
|
||||
Addr: addr,
|
||||
Build: *buildVersion,
|
||||
Version: vsn,
|
||||
RaftVersion: raftVsn,
|
||||
Status: m.Status,
|
||||
UseTLS: useTLS,
|
||||
Name: m.Name,
|
||||
ShortName: strings.TrimSuffix(m.Name, "."+datacenter),
|
||||
ID: m.Tags["id"],
|
||||
Datacenter: datacenter,
|
||||
Segment: segment,
|
||||
Port: port,
|
||||
SegmentAddrs: segmentAddrs,
|
||||
SegmentPorts: segmentPorts,
|
||||
WanJoinPort: wanJoinPort,
|
||||
LanJoinPort: int(m.Port),
|
||||
ExternalGRPCPort: externalGRPCPort,
|
||||
Bootstrap: bootstrap,
|
||||
Expect: expect,
|
||||
Addr: addr,
|
||||
Build: *buildVersion,
|
||||
Version: vsn,
|
||||
RaftVersion: raftVsn,
|
||||
Status: m.Status,
|
||||
UseTLS: useTLS,
|
||||
// DEPRECATED - remove nonVoter check once support for that tag is removed
|
||||
ReadReplica: nonVoter || readReplica,
|
||||
FeatureFlags: featureFlags,
|
||||
|
|
|
@ -82,19 +82,19 @@ func TestIsConsulServer(t *testing.T) {
|
|||
}
|
||||
|
||||
expected := &metadata.Server{
|
||||
Name: "foo",
|
||||
ShortName: "foo",
|
||||
ID: "asdf",
|
||||
Datacenter: "east-aws",
|
||||
Segment: "",
|
||||
Port: 10000,
|
||||
SegmentAddrs: map[string]string{},
|
||||
SegmentPorts: map[string]int{},
|
||||
WanJoinPort: 1234,
|
||||
LanJoinPort: 5454,
|
||||
PublicGRPCPort: 9876,
|
||||
Bootstrap: false,
|
||||
Expect: 3,
|
||||
Name: "foo",
|
||||
ShortName: "foo",
|
||||
ID: "asdf",
|
||||
Datacenter: "east-aws",
|
||||
Segment: "",
|
||||
Port: 10000,
|
||||
SegmentAddrs: map[string]string{},
|
||||
SegmentPorts: map[string]int{},
|
||||
WanJoinPort: 1234,
|
||||
LanJoinPort: 5454,
|
||||
ExternalGRPCPort: 9876,
|
||||
Bootstrap: false,
|
||||
Expect: 3,
|
||||
Addr: &net.TCPAddr{
|
||||
IP: net.IP([]byte{127, 0, 0, 1}),
|
||||
Port: 10000,
|
||||
|
@ -133,7 +133,7 @@ func TestIsConsulServer(t *testing.T) {
|
|||
expected.RaftVersion = 0
|
||||
expected.Expect = 0
|
||||
expected.WanJoinPort = 0
|
||||
expected.PublicGRPCPort = 0
|
||||
expected.ExternalGRPCPort = 0
|
||||
case "feature-namespaces":
|
||||
m.Tags["ft_ns"] = "1"
|
||||
expected.FeatureFlags = map[string]int{"ns": 1}
|
||||
|
|
|
@ -451,7 +451,7 @@ func DialRPCViaMeshGateway(
|
|||
}
|
||||
|
||||
if nextProto != ALPN_RPCGRPC {
|
||||
// agent/grpc/private/client.go:dial() handles this in another way for gRPC
|
||||
// agent/grpc-internal/client.go:dial() handles this in another way for gRPC
|
||||
if tcp, ok := rawConn.(*net.TCPConn); ok {
|
||||
_ = tcp.SetKeepAlive(true)
|
||||
_ = tcp.SetNoDelay(true)
|
||||
|
|
|
@ -18,7 +18,7 @@ import (
|
|||
"github.com/hashicorp/consul/agent/consul/state"
|
||||
"github.com/hashicorp/consul/agent/consul/stream"
|
||||
"github.com/hashicorp/consul/agent/dns"
|
||||
"github.com/hashicorp/consul/agent/grpc/public/services/peerstream"
|
||||
"github.com/hashicorp/consul/agent/grpc-external/services/peerstream"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/consul/lib"
|
||||
"github.com/hashicorp/consul/proto/pbpeering"
|
||||
|
|
|
@ -20,8 +20,8 @@ import (
|
|||
"github.com/hashicorp/consul/agent/consul"
|
||||
"github.com/hashicorp/consul/agent/consul/state"
|
||||
"github.com/hashicorp/consul/agent/consul/stream"
|
||||
grpc "github.com/hashicorp/consul/agent/grpc/private"
|
||||
"github.com/hashicorp/consul/agent/grpc/private/resolver"
|
||||
grpc "github.com/hashicorp/consul/agent/grpc-internal"
|
||||
"github.com/hashicorp/consul/agent/grpc-internal/resolver"
|
||||
"github.com/hashicorp/consul/agent/pool"
|
||||
"github.com/hashicorp/consul/agent/router"
|
||||
"github.com/hashicorp/consul/agent/rpc/middleware"
|
||||
|
@ -544,10 +544,10 @@ func newTestServer(t *testing.T, cb func(conf *consul.Config)) testingServer {
|
|||
conf.ACLResolverSettings.Datacenter = conf.Datacenter
|
||||
conf.ACLResolverSettings.EnterpriseMeta = *conf.AgentEnterpriseMeta()
|
||||
|
||||
publicGRPCServer := gogrpc.NewServer()
|
||||
externalGRPCServer := gogrpc.NewServer()
|
||||
|
||||
deps := newDefaultDeps(t, conf)
|
||||
server, err := consul.NewServer(conf, deps, publicGRPCServer)
|
||||
server, err := consul.NewServer(conf, deps, externalGRPCServer)
|
||||
require.NoError(t, err)
|
||||
t.Cleanup(func() {
|
||||
require.NoError(t, server.Shutdown())
|
||||
|
@ -560,9 +560,9 @@ func newTestServer(t *testing.T, cb func(conf *consul.Config)) testingServer {
|
|||
ln, err := net.Listen("tcp", grpcAddr)
|
||||
require.NoError(t, err)
|
||||
go func() {
|
||||
_ = publicGRPCServer.Serve(ln)
|
||||
_ = externalGRPCServer.Serve(ln)
|
||||
}()
|
||||
t.Cleanup(publicGRPCServer.Stop)
|
||||
t.Cleanup(externalGRPCServer.Stop)
|
||||
|
||||
testrpc.WaitForLeader(t, server.RPC, conf.Datacenter)
|
||||
|
||||
|
|
|
@ -18,8 +18,8 @@ import (
|
|||
"github.com/hashicorp/consul/agent/consul/fsm"
|
||||
"github.com/hashicorp/consul/agent/consul/stream"
|
||||
"github.com/hashicorp/consul/agent/consul/usagemetrics"
|
||||
grpc "github.com/hashicorp/consul/agent/grpc/private"
|
||||
"github.com/hashicorp/consul/agent/grpc/private/resolver"
|
||||
grpc "github.com/hashicorp/consul/agent/grpc-internal"
|
||||
"github.com/hashicorp/consul/agent/grpc-internal/resolver"
|
||||
"github.com/hashicorp/consul/agent/local"
|
||||
"github.com/hashicorp/consul/agent/pool"
|
||||
"github.com/hashicorp/consul/agent/router"
|
||||
|
|
|
@ -22,7 +22,7 @@ import (
|
|||
"github.com/hashicorp/consul/agent/cache"
|
||||
"github.com/hashicorp/consul/agent/consul/state"
|
||||
"github.com/hashicorp/consul/agent/consul/stream"
|
||||
"github.com/hashicorp/consul/agent/grpc/private/services/subscribe"
|
||||
"github.com/hashicorp/consul/agent/grpc-internal/services/subscribe"
|
||||
"github.com/hashicorp/consul/agent/rpcclient/health"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/consul/agent/submatview"
|
||||
|
|
|
@ -21,7 +21,7 @@ import (
|
|||
"google.golang.org/grpc/codes"
|
||||
"google.golang.org/grpc/status"
|
||||
|
||||
"github.com/hashicorp/consul/agent/grpc/public"
|
||||
external "github.com/hashicorp/consul/agent/grpc-external"
|
||||
"github.com/hashicorp/consul/agent/proxycfg"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/consul/agent/xds/serverlessplugin"
|
||||
|
@ -253,7 +253,7 @@ func (s *Server) processDelta(stream ADSDeltaStream, reqCh <-chan *envoy_discove
|
|||
|
||||
// Start watching config for that proxy
|
||||
var err error
|
||||
stateCh, watchCancel, err = s.CfgSrc.Watch(proxyID, nodeName, public.TokenFromContext(stream.Context()))
|
||||
stateCh, watchCancel, err = s.CfgSrc.Watch(proxyID, nodeName, external.TokenFromContext(stream.Context()))
|
||||
if err != nil {
|
||||
return status.Errorf(codes.Internal, "failed to watch proxy service: %s", err)
|
||||
}
|
||||
|
|
|
@ -16,7 +16,7 @@ import (
|
|||
"google.golang.org/grpc/status"
|
||||
|
||||
"github.com/hashicorp/consul/acl"
|
||||
"github.com/hashicorp/consul/agent/grpc/public"
|
||||
external "github.com/hashicorp/consul/agent/grpc-external"
|
||||
"github.com/hashicorp/consul/agent/proxycfg"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/consul/agent/xds/xdscommon"
|
||||
|
@ -201,7 +201,7 @@ func (s *Server) authorize(ctx context.Context, cfgSnap *proxycfg.ConfigSnapshot
|
|||
return status.Errorf(codes.Unauthenticated, "unauthenticated: no config snapshot")
|
||||
}
|
||||
|
||||
authz, err := s.ResolveToken(public.TokenFromContext(ctx))
|
||||
authz, err := s.ResolveToken(external.TokenFromContext(ctx))
|
||||
if acl.IsErrNotFound(err) {
|
||||
return status.Errorf(codes.Unauthenticated, "unauthenticated: %v", err)
|
||||
} else if acl.IsErrPermissionDenied(err) {
|
||||
|
|
|
@ -1,26 +1,26 @@
|
|||
# RPC
|
||||
|
||||
This section is a work in progress.
|
||||
Consul uses two RPC systems for communication between components within the
|
||||
cluster and with other clients such as Envoy: [gRPC](https://grpc.io/)
|
||||
and Go's [`net/rpc`](https://pkg.go.dev/net/rpc) package.
|
||||
|
||||
The RPC subsystem is exclusicely in Server Agents. It is comprised of two main components:
|
||||
Communication between client agents and servers uses a mix of both gRPC and
|
||||
`net/rpc`. Generally, gRPC is preferred because it supports modern features
|
||||
such as context deadlines/cancellation, streaming, and middleware - but Consul
|
||||
has been around for a while so the majority of RPC endpoints still use `net/rpc`.
|
||||
|
||||
1. the "RPC Server" (for lack of a better term) handles multiplexing of many different
|
||||
requests on a single TCP port.
|
||||
2. RPC endpoints handle RPC requests and return responses.
|
||||
## Multiplexed "Server" Port
|
||||
|
||||
The RPC subsystems handles requests from:
|
||||
Most in-cluster communication happens over the multiplexed "server" TCP port
|
||||
(default: 8300). Consul servers implement a custom protocol for serving
|
||||
different kinds of traffic on the same port, whereby the first byte sent
|
||||
indicates the protocol (e.g. gRPC, `net/rpc`, Raft).
|
||||
|
||||
1. Client Agents in the local DC
|
||||
2. (if the server is a leader) other Server Agents in the local DC
|
||||
3. Server Agents in other Datacenters
|
||||
4. in-process requests from other components running in the same process (ex: the HTTP API
|
||||
or DNS interface).
|
||||
Servers also implement [TLS ALPN](https://en.wikipedia.org/wiki/Application-Layer_Protocol_Negotiation)
|
||||
on this port, for situations where wrapping the real protocol with a byte prefix
|
||||
isn't practical (e.g. cross-DC traffic over mesh gateways).
|
||||
|
||||
## Routing
|
||||
|
||||
The "RPC Server" accepts requests to the [server port] and routes the requests based on
|
||||
configuration of the Server and the the first byte in the request. The diagram below shows
|
||||
all the possible routing flows.
|
||||
The diagram below shows all the possible routing flows:
|
||||
|
||||
[server port]: https://www.consul.io/docs/agent/config/config-files#server_rpc_port
|
||||
|
||||
|
@ -28,11 +28,10 @@ all the possible routing flows.
|
|||
|
||||
<sup>[source](./routing.mmd)</sup>
|
||||
|
||||
The main entrypoint to RPC routing is `handleConn` in [agent/consul/rpc.go].
|
||||
The main entrypoint to connection routing is `handleConn` in [agent/consul/rpc.go].
|
||||
|
||||
[agent/consul/rpc.go]: https://github.com/hashicorp/consul/blob/main/agent/consul/rpc.go
|
||||
|
||||
|
||||
### Development
|
||||
|
||||
Multiplexing several protocols over a single server port helps to reduce our
|
||||
|
@ -44,13 +43,35 @@ development tools such as [grpcurl] difficult.
|
|||
You can get a "plain" TCP connection to the gRPC server using this proxy script:
|
||||
|
||||
```
|
||||
$ go run tools/private-grpc-proxy/main.go localhost:8300
|
||||
Proxying connections to Consul's private gRPC server
|
||||
$ go run tools/internal-grpc-proxy/main.go localhost:8300
|
||||
Proxying connections to Consul's internal gRPC server
|
||||
Use this address: 127.0.0.1:64077
|
||||
```
|
||||
|
||||
Pass the returned proxy address to your tool of choice.
|
||||
|
||||
## Private vs Public vs Internal vs External
|
||||
When working on Consul's gRPC endpoints you may notice we use private/public and
|
||||
internal/external slightly differently.
|
||||
|
||||
Private and public refer to whether an API is suitable for consumption by
|
||||
clients other than Consul's core components.
|
||||
|
||||
Private gRPC APIs are defined in the `proto` directory, and should only be used
|
||||
by Consul servers and agents. Public gRPC APIs are defined in the `proto-public`
|
||||
directory and may be used by 3rd-party applications.
|
||||
|
||||
Internal and external refer to how the gRPC APIs are exposed.
|
||||
|
||||
Internal gRPC APIs are exposed on the multiplexed "server" port, whereas
|
||||
external APIs are exposed on a dedicated gRPC port (default: 8502).
|
||||
|
||||
The reason for this differentiation is that some private APIs are exposed on the
|
||||
external port, such as peer streaming/replication; this API isn't (yet) suitable
|
||||
for consumption by 3rd-party applications but must be accessible from outside
|
||||
the cluster, and present a TLS certificate signed by a public CA, which the
|
||||
multiplexed port cannot.
|
||||
|
||||
## RPC Endpoints
|
||||
|
||||
This section is a work in progress, it will eventually cover topics like:
|
||||
|
|
|
@ -34,7 +34,7 @@ and sent to any active subscriptions.
|
|||
[rpcclient/health.Health]: https://github.com/hashicorp/consul/blob/main/agent/rpcclient/health/health.go
|
||||
[StreamingHealthServices cache-type]: https://github.com/hashicorp/consul/blob/main/agent/cache-types/streaming_health_services.go
|
||||
[materialized view]: https://github.com/hashicorp/consul/blob/main/agent/submatview/materializer.go
|
||||
[SubscribeEndpoint]: https://github.com/hashicorp/consul/blob/main/agent/grpc/private/services/subscribe/subscribe.go
|
||||
[SubscribeEndpoint]: https://github.com/hashicorp/consul/blob/main/agent/grpc-internal/services/subscribe/subscribe.go
|
||||
[EventPublisher]: https://github.com/hashicorp/consul/blob/main/agent/consul/stream/event_publisher.go
|
||||
[state.Store commits]: https://github.com/hashicorp/consul/blob/main/agent/consul/state/memdb.go
|
||||
|
||||
|
|
|
@ -24,7 +24,7 @@ func main() {
|
|||
}
|
||||
defer lis.Close()
|
||||
|
||||
fmt.Println("Proxying connections to Consul's private gRPC server")
|
||||
fmt.Println("Proxying connections to Consul's internal gRPC server")
|
||||
fmt.Printf("Use this address: %s\n", lis.Addr())
|
||||
|
||||
for {
|
Loading…
Reference in New Issue