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:
Dan Upton 2022-07-13 16:33:48 +01:00 committed by GitHub
parent c880728ab4
commit 34140ff3e0
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
91 changed files with 269 additions and 244 deletions

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -1,4 +1,4 @@
package public
package external
import (
"context"

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -1,4 +1,4 @@
package public
package external
import (
"context"

View File

@ -1,4 +1,4 @@
package public
package external
import "github.com/hashicorp/go-uuid"

View File

@ -1,4 +1,4 @@
package private
package internal
import (
"context"

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -1,4 +1,4 @@
package private
package internal
import (
"context"

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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