open-consul/agent/xds/server.go

247 lines
9.3 KiB
Go
Raw Normal View History

// Copyright (c) HashiCorp, Inc.
// SPDX-License-Identifier: MPL-2.0
package xds
import (
"context"
"errors"
"sync/atomic"
"time"
envoy_discovery_v3 "github.com/envoyproxy/go-control-plane/envoy/service/discovery/v3"
"github.com/hashicorp/consul/envoyextensions/xdscommon"
"github.com/armon/go-metrics"
"github.com/armon/go-metrics/prometheus"
"github.com/hashicorp/go-hclog"
"google.golang.org/grpc"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
"github.com/hashicorp/consul/acl"
external "github.com/hashicorp/consul/agent/grpc-external"
"github.com/hashicorp/consul/agent/grpc-external/limiter"
"github.com/hashicorp/consul/agent/proxycfg"
"github.com/hashicorp/consul/agent/structs"
)
var (
StatsGauges = []prometheus.GaugeDefinition{
{
Name: []string{"xds", "server", "streams"},
Help: "Measures the number of active xDS streams handled by the server split by protocol version.",
},
{
Name: []string{"xds", "server", "streamsUnauthenticated"},
Help: "Counts the number of active xDS streams handled by the server that are unauthenticated because ACLs are not enabled or ACL tokens were missing.",
},
}
StatsCounters = []prometheus.CounterDefinition{
{
Name: []string{"xds", "server", "streamDrained"},
Help: "Counts the number of xDS streams that are drained when rebalancing the load between servers.",
},
}
StatsSummaries = []prometheus.SummaryDefinition{
{
Name: []string{"xds", "server", "streamStart"},
Help: "Measures the time in milliseconds after an xDS stream is opened until xDS resources are first generated for the stream.",
},
}
)
// ADSStream is a shorter way of referring to this thing...
type ADSStream = envoy_discovery_v3.AggregatedDiscoveryService_StreamAggregatedResourcesServer
const (
// LocalAgentClusterName is the name we give the local agent "cluster" in
// Envoy config. Note that all cluster names may collide with service names
// since we want cluster names and service names to match to enable nice
// metrics correlation without massaging prefixes on cluster names.
//
// We should probably make this more unlikely to collied however changing it
// potentially breaks upgrade compatibility without restarting all Envoy's as
// it will no longer match their existing cluster name. Changing this will
// affect metrics output so could break dashboards (for local agent traffic).
//
// We should probably just make it configurable if anyone actually has
// services named "local_agent" in the future.
LocalAgentClusterName = "local_agent"
2021-03-17 19:40:49 +00:00
// OriginalDestinationClusterName is the name we give to the passthrough
// cluster which redirects transparently-proxied requests to their original
// destination outside the mesh. This cluster prevents Consul from blocking
// connections to destinations outside of the catalog when in transparent
// proxy mode.
2021-03-17 19:40:49 +00:00
OriginalDestinationClusterName = "original-destination"
// DefaultAuthCheckFrequency is the default value for
// Server.AuthCheckFrequency to use when the zero value is provided.
DefaultAuthCheckFrequency = 5 * time.Minute
)
// ACLResolverFunc is a shim to resolve ACLs. Since ACL enforcement is so far
// entirely agent-local and all uses private methods this allows a simple shim
// to be written in the agent package to allow resolving without tightly
// coupling this to the agent.
New ACLs (#4791) This PR is almost a complete rewrite of the ACL system within Consul. It brings the features more in line with other HashiCorp products. Obviously there is quite a bit left to do here but most of it is related docs, testing and finishing the last few commands in the CLI. I will update the PR description and check off the todos as I finish them over the next few days/week. Description At a high level this PR is mainly to split ACL tokens from Policies and to split the concepts of Authorization from Identities. A lot of this PR is mostly just to support CRUD operations on ACLTokens and ACLPolicies. These in and of themselves are not particularly interesting. The bigger conceptual changes are in how tokens get resolved, how backwards compatibility is handled and the separation of policy from identity which could lead the way to allowing for alternative identity providers. On the surface and with a new cluster the ACL system will look very similar to that of Nomads. Both have tokens and policies. Both have local tokens. The ACL management APIs for both are very similar. I even ripped off Nomad's ACL bootstrap resetting procedure. There are a few key differences though. Nomad requires token and policy replication where Consul only requires policy replication with token replication being opt-in. In Consul local tokens only work with token replication being enabled though. All policies in Nomad are globally applicable. In Consul all policies are stored and replicated globally but can be scoped to a subset of the datacenters. This allows for more granular access management. Unlike Nomad, Consul has legacy baggage in the form of the original ACL system. The ramifications of this are: A server running the new system must still support other clients using the legacy system. A client running the new system must be able to use the legacy RPCs when the servers in its datacenter are running the legacy system. The primary ACL DC's servers running in legacy mode needs to be a gate that keeps everything else in the entire multi-DC cluster running in legacy mode. So not only does this PR implement the new ACL system but has a legacy mode built in for when the cluster isn't ready for new ACLs. Also detecting that new ACLs can be used is automatic and requires no configuration on the part of administrators. This process is detailed more in the "Transitioning from Legacy to New ACL Mode" section below.
2018-10-19 16:04:07 +00:00
type ACLResolverFunc func(id string) (acl.Authorizer, error)
// ConfigFetcher is the interface the agent needs to expose
// for the xDS server to fetch agent config, currently only one field is fetched
type ConfigFetcher interface {
AdvertiseAddrLAN() string
}
// ProxyConfigSource is the interface xds.Server requires to consume proxy
// config updates.
type ProxyConfigSource interface {
Watch(id structs.ServiceID, nodeName string, token string) (<-chan *proxycfg.ConfigSnapshot, limiter.SessionTerminatedChan, proxycfg.CancelFunc, error)
}
// Server represents a gRPC server that can handle xDS requests from Envoy. All
// of it's public members must be set before the gRPC server is started.
//
// A full description of the XDS protocol can be found at
// https://www.envoyproxy.io/docs/envoy/latest/api-docs/xds_protocol
type Server struct {
NodeName string
Logger hclog.Logger
CfgSrc ProxyConfigSource
ResolveToken ACLResolverFunc
CfgFetcher ConfigFetcher
Support Incremental xDS mode (#9855) This adds support for the Incremental xDS protocol when using xDS v3. This is best reviewed commit-by-commit and will not be squashed when merged. Union of all commit messages follows to give an overarching summary: xds: exclusively support incremental xDS when using xDS v3 Attempts to use SoTW via v3 will fail, much like attempts to use incremental via v2 will fail. Work around a strange older envoy behavior involving empty CDS responses over incremental xDS. xds: various cleanups and refactors that don't strictly concern the addition of incremental xDS support Dissolve the connectionInfo struct in favor of per-connection ResourceGenerators instead. Do a better job of ensuring the xds code uses a well configured logger that accurately describes the connected client. xds: pull out checkStreamACLs method in advance of a later commit xds: rewrite SoTW xDS protocol tests to use protobufs rather than hand-rolled json strings In the test we very lightly reuse some of the more boring protobuf construction helper code that is also technically under test. The important thing of the protocol tests is testing the protocol. The actual inputs and outputs are largely already handled by the xds golden output tests now so these protocol tests don't have to do double-duty. This also updates the SoTW protocol test to exclusively use xDS v2 which is the only variant of SoTW that will be supported in Consul 1.10. xds: default xds.Server.AuthCheckFrequency at use-time instead of construction-time
2021-04-29 18:54:05 +00:00
// AuthCheckFrequency is how often we should re-check the credentials used
// during a long-lived gRPC Stream after it has been initially established.
// This is only used during idle periods of stream interactions (i.e. when
// there has been no recent DiscoveryRequest).
AuthCheckFrequency time.Duration
// ResourceMapMutateFn exclusively exists for testing purposes.
ResourceMapMutateFn func(resourceMap *xdscommon.IndexedResources)
activeStreams *activeStreamCounters
}
// activeStreamCounters tracks various stream-related metrics.
// Requires that activeStreamCounters be a pointer field.
type activeStreamCounters struct {
xDSv3 atomic.Uint64
unauthenticated atomic.Uint64
}
func (c *activeStreamCounters) Increment(ctx context.Context) func() {
// If no ACL token is found, increase the gauge.
o, _ := external.QueryOptionsFromContext(ctx)
if o.Token == "" {
unauthn := c.unauthenticated.Add(1)
metrics.SetGauge([]string{"xds", "server", "streamsUnauthenticated"}, float32(unauthn))
}
// Historically there had been a "v2" version.
labels := []metrics.Label{{Name: "version", Value: "v3"}}
count := c.xDSv3.Add(1)
metrics.SetGaugeWithLabels([]string{"xds", "server", "streams"}, float32(count), labels)
// This closure should be called in a defer to decrement the gauges after the stream is closed.
return func() {
if o.Token == "" {
unauthn := c.unauthenticated.Add(^uint64(0))
metrics.SetGauge([]string{"xds", "server", "streamsUnauthenticated"}, float32(unauthn))
}
count := c.xDSv3.Add(^uint64(0))
metrics.SetGaugeWithLabels([]string{"xds", "server", "streams"}, float32(count), labels)
}
}
Support Incremental xDS mode (#9855) This adds support for the Incremental xDS protocol when using xDS v3. This is best reviewed commit-by-commit and will not be squashed when merged. Union of all commit messages follows to give an overarching summary: xds: exclusively support incremental xDS when using xDS v3 Attempts to use SoTW via v3 will fail, much like attempts to use incremental via v2 will fail. Work around a strange older envoy behavior involving empty CDS responses over incremental xDS. xds: various cleanups and refactors that don't strictly concern the addition of incremental xDS support Dissolve the connectionInfo struct in favor of per-connection ResourceGenerators instead. Do a better job of ensuring the xds code uses a well configured logger that accurately describes the connected client. xds: pull out checkStreamACLs method in advance of a later commit xds: rewrite SoTW xDS protocol tests to use protobufs rather than hand-rolled json strings In the test we very lightly reuse some of the more boring protobuf construction helper code that is also technically under test. The important thing of the protocol tests is testing the protocol. The actual inputs and outputs are largely already handled by the xds golden output tests now so these protocol tests don't have to do double-duty. This also updates the SoTW protocol test to exclusively use xDS v2 which is the only variant of SoTW that will be supported in Consul 1.10. xds: default xds.Server.AuthCheckFrequency at use-time instead of construction-time
2021-04-29 18:54:05 +00:00
func NewServer(
nodeName string,
Support Incremental xDS mode (#9855) This adds support for the Incremental xDS protocol when using xDS v3. This is best reviewed commit-by-commit and will not be squashed when merged. Union of all commit messages follows to give an overarching summary: xds: exclusively support incremental xDS when using xDS v3 Attempts to use SoTW via v3 will fail, much like attempts to use incremental via v2 will fail. Work around a strange older envoy behavior involving empty CDS responses over incremental xDS. xds: various cleanups and refactors that don't strictly concern the addition of incremental xDS support Dissolve the connectionInfo struct in favor of per-connection ResourceGenerators instead. Do a better job of ensuring the xds code uses a well configured logger that accurately describes the connected client. xds: pull out checkStreamACLs method in advance of a later commit xds: rewrite SoTW xDS protocol tests to use protobufs rather than hand-rolled json strings In the test we very lightly reuse some of the more boring protobuf construction helper code that is also technically under test. The important thing of the protocol tests is testing the protocol. The actual inputs and outputs are largely already handled by the xds golden output tests now so these protocol tests don't have to do double-duty. This also updates the SoTW protocol test to exclusively use xDS v2 which is the only variant of SoTW that will be supported in Consul 1.10. xds: default xds.Server.AuthCheckFrequency at use-time instead of construction-time
2021-04-29 18:54:05 +00:00
logger hclog.Logger,
cfgMgr ProxyConfigSource,
resolveTokenSecret ACLResolverFunc,
Support Incremental xDS mode (#9855) This adds support for the Incremental xDS protocol when using xDS v3. This is best reviewed commit-by-commit and will not be squashed when merged. Union of all commit messages follows to give an overarching summary: xds: exclusively support incremental xDS when using xDS v3 Attempts to use SoTW via v3 will fail, much like attempts to use incremental via v2 will fail. Work around a strange older envoy behavior involving empty CDS responses over incremental xDS. xds: various cleanups and refactors that don't strictly concern the addition of incremental xDS support Dissolve the connectionInfo struct in favor of per-connection ResourceGenerators instead. Do a better job of ensuring the xds code uses a well configured logger that accurately describes the connected client. xds: pull out checkStreamACLs method in advance of a later commit xds: rewrite SoTW xDS protocol tests to use protobufs rather than hand-rolled json strings In the test we very lightly reuse some of the more boring protobuf construction helper code that is also technically under test. The important thing of the protocol tests is testing the protocol. The actual inputs and outputs are largely already handled by the xds golden output tests now so these protocol tests don't have to do double-duty. This also updates the SoTW protocol test to exclusively use xDS v2 which is the only variant of SoTW that will be supported in Consul 1.10. xds: default xds.Server.AuthCheckFrequency at use-time instead of construction-time
2021-04-29 18:54:05 +00:00
cfgFetcher ConfigFetcher,
) *Server {
return &Server{
NodeName: nodeName,
Logger: logger,
CfgSrc: cfgMgr,
ResolveToken: resolveTokenSecret,
CfgFetcher: cfgFetcher,
AuthCheckFrequency: DefaultAuthCheckFrequency,
activeStreams: &activeStreamCounters{},
Support Incremental xDS mode (#9855) This adds support for the Incremental xDS protocol when using xDS v3. This is best reviewed commit-by-commit and will not be squashed when merged. Union of all commit messages follows to give an overarching summary: xds: exclusively support incremental xDS when using xDS v3 Attempts to use SoTW via v3 will fail, much like attempts to use incremental via v2 will fail. Work around a strange older envoy behavior involving empty CDS responses over incremental xDS. xds: various cleanups and refactors that don't strictly concern the addition of incremental xDS support Dissolve the connectionInfo struct in favor of per-connection ResourceGenerators instead. Do a better job of ensuring the xds code uses a well configured logger that accurately describes the connected client. xds: pull out checkStreamACLs method in advance of a later commit xds: rewrite SoTW xDS protocol tests to use protobufs rather than hand-rolled json strings In the test we very lightly reuse some of the more boring protobuf construction helper code that is also technically under test. The important thing of the protocol tests is testing the protocol. The actual inputs and outputs are largely already handled by the xds golden output tests now so these protocol tests don't have to do double-duty. This also updates the SoTW protocol test to exclusively use xDS v2 which is the only variant of SoTW that will be supported in Consul 1.10. xds: default xds.Server.AuthCheckFrequency at use-time instead of construction-time
2021-04-29 18:54:05 +00:00
}
}
// StreamAggregatedResources implements
// envoy_discovery_v3.AggregatedDiscoveryServiceServer. This is the ADS endpoint which is
// the only xDS API we directly support for now.
Support Incremental xDS mode (#9855) This adds support for the Incremental xDS protocol when using xDS v3. This is best reviewed commit-by-commit and will not be squashed when merged. Union of all commit messages follows to give an overarching summary: xds: exclusively support incremental xDS when using xDS v3 Attempts to use SoTW via v3 will fail, much like attempts to use incremental via v2 will fail. Work around a strange older envoy behavior involving empty CDS responses over incremental xDS. xds: various cleanups and refactors that don't strictly concern the addition of incremental xDS support Dissolve the connectionInfo struct in favor of per-connection ResourceGenerators instead. Do a better job of ensuring the xds code uses a well configured logger that accurately describes the connected client. xds: pull out checkStreamACLs method in advance of a later commit xds: rewrite SoTW xDS protocol tests to use protobufs rather than hand-rolled json strings In the test we very lightly reuse some of the more boring protobuf construction helper code that is also technically under test. The important thing of the protocol tests is testing the protocol. The actual inputs and outputs are largely already handled by the xds golden output tests now so these protocol tests don't have to do double-duty. This also updates the SoTW protocol test to exclusively use xDS v2 which is the only variant of SoTW that will be supported in Consul 1.10. xds: default xds.Server.AuthCheckFrequency at use-time instead of construction-time
2021-04-29 18:54:05 +00:00
//
// Deprecated: use DeltaAggregatedResources instead
func (s *Server) StreamAggregatedResources(stream ADSStream) error {
Support Incremental xDS mode (#9855) This adds support for the Incremental xDS protocol when using xDS v3. This is best reviewed commit-by-commit and will not be squashed when merged. Union of all commit messages follows to give an overarching summary: xds: exclusively support incremental xDS when using xDS v3 Attempts to use SoTW via v3 will fail, much like attempts to use incremental via v2 will fail. Work around a strange older envoy behavior involving empty CDS responses over incremental xDS. xds: various cleanups and refactors that don't strictly concern the addition of incremental xDS support Dissolve the connectionInfo struct in favor of per-connection ResourceGenerators instead. Do a better job of ensuring the xds code uses a well configured logger that accurately describes the connected client. xds: pull out checkStreamACLs method in advance of a later commit xds: rewrite SoTW xDS protocol tests to use protobufs rather than hand-rolled json strings In the test we very lightly reuse some of the more boring protobuf construction helper code that is also technically under test. The important thing of the protocol tests is testing the protocol. The actual inputs and outputs are largely already handled by the xds golden output tests now so these protocol tests don't have to do double-duty. This also updates the SoTW protocol test to exclusively use xDS v2 which is the only variant of SoTW that will be supported in Consul 1.10. xds: default xds.Server.AuthCheckFrequency at use-time instead of construction-time
2021-04-29 18:54:05 +00:00
return errors.New("not implemented")
}
// Register the XDS server handlers to the given gRPC server.
func (s *Server) Register(srv *grpc.Server) {
envoy_discovery_v3.RegisterAggregatedDiscoveryServiceServer(srv, s)
}
Support Incremental xDS mode (#9855) This adds support for the Incremental xDS protocol when using xDS v3. This is best reviewed commit-by-commit and will not be squashed when merged. Union of all commit messages follows to give an overarching summary: xds: exclusively support incremental xDS when using xDS v3 Attempts to use SoTW via v3 will fail, much like attempts to use incremental via v2 will fail. Work around a strange older envoy behavior involving empty CDS responses over incremental xDS. xds: various cleanups and refactors that don't strictly concern the addition of incremental xDS support Dissolve the connectionInfo struct in favor of per-connection ResourceGenerators instead. Do a better job of ensuring the xds code uses a well configured logger that accurately describes the connected client. xds: pull out checkStreamACLs method in advance of a later commit xds: rewrite SoTW xDS protocol tests to use protobufs rather than hand-rolled json strings In the test we very lightly reuse some of the more boring protobuf construction helper code that is also technically under test. The important thing of the protocol tests is testing the protocol. The actual inputs and outputs are largely already handled by the xds golden output tests now so these protocol tests don't have to do double-duty. This also updates the SoTW protocol test to exclusively use xDS v2 which is the only variant of SoTW that will be supported in Consul 1.10. xds: default xds.Server.AuthCheckFrequency at use-time instead of construction-time
2021-04-29 18:54:05 +00:00
func (s *Server) authenticate(ctx context.Context) (acl.Authorizer, error) {
options, err := external.QueryOptionsFromContext(ctx)
if err != nil {
return nil, status.Errorf(codes.Internal, "error fetching options from context: %v", err)
}
authz, err := s.ResolveToken(options.Token)
if acl.IsErrNotFound(err) {
return nil, status.Errorf(codes.Unauthenticated, "unauthenticated: %v", err)
} else if acl.IsErrPermissionDenied(err) {
return nil, status.Error(codes.PermissionDenied, err.Error())
} else if err != nil {
return nil, status.Errorf(codes.Internal, "error resolving acl token: %v", err)
}
return authz, nil
}
2021-08-13 15:53:19 +00:00
// authorize the xDS request using the token stored in ctx. This authorization is
// a bit different from most interfaces. Instead of explicitly authorizing or
// filtering each piece of data in the response, the request is authorized
// by checking the token has `service:write` for the service ID of the destination
// service (for kind=ConnectProxy), or the gateway service (for other kinds).
// This authorization strategy requires that agent/proxycfg only fetches data
// using a token with the same permissions, and that it stores the data by
// proxy ID. We assume that any data in the snapshot was already filtered,
// which allows this authorization to be a shallow authorization check
// for all the data in a ConfigSnapshot.
func (s *Server) authorize(ctx context.Context, cfgSnap *proxycfg.ConfigSnapshot) error {
Support Incremental xDS mode (#9855) This adds support for the Incremental xDS protocol when using xDS v3. This is best reviewed commit-by-commit and will not be squashed when merged. Union of all commit messages follows to give an overarching summary: xds: exclusively support incremental xDS when using xDS v3 Attempts to use SoTW via v3 will fail, much like attempts to use incremental via v2 will fail. Work around a strange older envoy behavior involving empty CDS responses over incremental xDS. xds: various cleanups and refactors that don't strictly concern the addition of incremental xDS support Dissolve the connectionInfo struct in favor of per-connection ResourceGenerators instead. Do a better job of ensuring the xds code uses a well configured logger that accurately describes the connected client. xds: pull out checkStreamACLs method in advance of a later commit xds: rewrite SoTW xDS protocol tests to use protobufs rather than hand-rolled json strings In the test we very lightly reuse some of the more boring protobuf construction helper code that is also technically under test. The important thing of the protocol tests is testing the protocol. The actual inputs and outputs are largely already handled by the xds golden output tests now so these protocol tests don't have to do double-duty. This also updates the SoTW protocol test to exclusively use xDS v2 which is the only variant of SoTW that will be supported in Consul 1.10. xds: default xds.Server.AuthCheckFrequency at use-time instead of construction-time
2021-04-29 18:54:05 +00:00
if cfgSnap == nil {
return status.Errorf(codes.Unauthenticated, "unauthenticated: no config snapshot")
}
authz, err := s.authenticate(ctx)
if err != nil {
return err
Support Incremental xDS mode (#9855) This adds support for the Incremental xDS protocol when using xDS v3. This is best reviewed commit-by-commit and will not be squashed when merged. Union of all commit messages follows to give an overarching summary: xds: exclusively support incremental xDS when using xDS v3 Attempts to use SoTW via v3 will fail, much like attempts to use incremental via v2 will fail. Work around a strange older envoy behavior involving empty CDS responses over incremental xDS. xds: various cleanups and refactors that don't strictly concern the addition of incremental xDS support Dissolve the connectionInfo struct in favor of per-connection ResourceGenerators instead. Do a better job of ensuring the xds code uses a well configured logger that accurately describes the connected client. xds: pull out checkStreamACLs method in advance of a later commit xds: rewrite SoTW xDS protocol tests to use protobufs rather than hand-rolled json strings In the test we very lightly reuse some of the more boring protobuf construction helper code that is also technically under test. The important thing of the protocol tests is testing the protocol. The actual inputs and outputs are largely already handled by the xds golden output tests now so these protocol tests don't have to do double-duty. This also updates the SoTW protocol test to exclusively use xDS v2 which is the only variant of SoTW that will be supported in Consul 1.10. xds: default xds.Server.AuthCheckFrequency at use-time instead of construction-time
2021-04-29 18:54:05 +00:00
}
var authzContext acl.AuthorizerContext
switch cfgSnap.Kind {
case structs.ServiceKindConnectProxy:
cfgSnap.ProxyID.EnterpriseMeta.FillAuthzContext(&authzContext)
if err := authz.ToAllowAuthorizer().ServiceWriteAllowed(cfgSnap.Proxy.DestinationServiceName, &authzContext); err != nil {
return status.Errorf(codes.PermissionDenied, err.Error())
Support Incremental xDS mode (#9855) This adds support for the Incremental xDS protocol when using xDS v3. This is best reviewed commit-by-commit and will not be squashed when merged. Union of all commit messages follows to give an overarching summary: xds: exclusively support incremental xDS when using xDS v3 Attempts to use SoTW via v3 will fail, much like attempts to use incremental via v2 will fail. Work around a strange older envoy behavior involving empty CDS responses over incremental xDS. xds: various cleanups and refactors that don't strictly concern the addition of incremental xDS support Dissolve the connectionInfo struct in favor of per-connection ResourceGenerators instead. Do a better job of ensuring the xds code uses a well configured logger that accurately describes the connected client. xds: pull out checkStreamACLs method in advance of a later commit xds: rewrite SoTW xDS protocol tests to use protobufs rather than hand-rolled json strings In the test we very lightly reuse some of the more boring protobuf construction helper code that is also technically under test. The important thing of the protocol tests is testing the protocol. The actual inputs and outputs are largely already handled by the xds golden output tests now so these protocol tests don't have to do double-duty. This also updates the SoTW protocol test to exclusively use xDS v2 which is the only variant of SoTW that will be supported in Consul 1.10. xds: default xds.Server.AuthCheckFrequency at use-time instead of construction-time
2021-04-29 18:54:05 +00:00
}
Implement APIGateway proxycfg snapshot (#16194) * Stub proxycfg handler for API gateway * Add Service Kind constants/handling for API Gateway * Begin stubbing for SDS * Add new Secret type to xDS order of operations * Continue stubbing of SDS * Iterate on proxycfg handler for API gateway * Handle BoundAPIGateway config entry subscription in proxycfg-glue * Add API gateway to config snapshot validation * Add API gateway to config snapshot clone, leaf, etc. * Subscribe to bound route + cert config entries on bound-api-gateway * Track routes + certs on API gateway config snapshot * Generate DeepCopy() for types used in watch.Map * Watch all active references on api-gateway, unwatch inactive * Track loading of initial bound-api-gateway config entry * Use proper proto package for SDS mapping * Use ResourceReference instead of ServiceName, collect resources * Fix typo, add + remove TODOs * Watch discovery chains for TCPRoute * Add TODO for updating gateway services for api-gateway * make proto * Regenerate deep-copy for proxycfg * Set datacenter on upstream ID from query source * Watch discovery chains for http-route service backends * Add ServiceName getter to HTTP+TCP Service structs * Clean up unwatched discovery chains on API Gateway * Implement watch for ingress leaf certificate * Collect upstreams on http-route + tcp-route updates * Remove unused GatewayServices update handler * Remove unnecessary gateway services logic for API Gateway * Remove outdate TODO * Use .ToIngress where appropriate, including TODO for cleaning up * Cancel before returning error * Remove GatewayServices subscription * Add godoc for handlerAPIGateway functions * Update terminology from Connect => Consul Service Mesh Consistent with terminology changes in https://github.com/hashicorp/consul/pull/12690 * Add missing TODO * Remove duplicate switch case * Rerun deep-copy generator * Use correct property on config snapshot * Remove unnecessary leaf cert watch * Clean up based on code review feedback * Note handler properties that are initialized but set elsewhere * Add TODO for moving helper func into structs pkg * Update generated DeepCopy code * gofmt * Generate DeepCopy() for API gateway listener types * Improve variable name * Regenerate DeepCopy() code * Fix linting issue * Temporarily remove the secret type from resource generation
2023-02-08 21:52:12 +00:00
case structs.ServiceKindMeshGateway, structs.ServiceKindTerminatingGateway, structs.ServiceKindIngressGateway, structs.ServiceKindAPIGateway:
Support Incremental xDS mode (#9855) This adds support for the Incremental xDS protocol when using xDS v3. This is best reviewed commit-by-commit and will not be squashed when merged. Union of all commit messages follows to give an overarching summary: xds: exclusively support incremental xDS when using xDS v3 Attempts to use SoTW via v3 will fail, much like attempts to use incremental via v2 will fail. Work around a strange older envoy behavior involving empty CDS responses over incremental xDS. xds: various cleanups and refactors that don't strictly concern the addition of incremental xDS support Dissolve the connectionInfo struct in favor of per-connection ResourceGenerators instead. Do a better job of ensuring the xds code uses a well configured logger that accurately describes the connected client. xds: pull out checkStreamACLs method in advance of a later commit xds: rewrite SoTW xDS protocol tests to use protobufs rather than hand-rolled json strings In the test we very lightly reuse some of the more boring protobuf construction helper code that is also technically under test. The important thing of the protocol tests is testing the protocol. The actual inputs and outputs are largely already handled by the xds golden output tests now so these protocol tests don't have to do double-duty. This also updates the SoTW protocol test to exclusively use xDS v2 which is the only variant of SoTW that will be supported in Consul 1.10. xds: default xds.Server.AuthCheckFrequency at use-time instead of construction-time
2021-04-29 18:54:05 +00:00
cfgSnap.ProxyID.EnterpriseMeta.FillAuthzContext(&authzContext)
if err := authz.ToAllowAuthorizer().ServiceWriteAllowed(cfgSnap.Service, &authzContext); err != nil {
return status.Errorf(codes.PermissionDenied, err.Error())
Support Incremental xDS mode (#9855) This adds support for the Incremental xDS protocol when using xDS v3. This is best reviewed commit-by-commit and will not be squashed when merged. Union of all commit messages follows to give an overarching summary: xds: exclusively support incremental xDS when using xDS v3 Attempts to use SoTW via v3 will fail, much like attempts to use incremental via v2 will fail. Work around a strange older envoy behavior involving empty CDS responses over incremental xDS. xds: various cleanups and refactors that don't strictly concern the addition of incremental xDS support Dissolve the connectionInfo struct in favor of per-connection ResourceGenerators instead. Do a better job of ensuring the xds code uses a well configured logger that accurately describes the connected client. xds: pull out checkStreamACLs method in advance of a later commit xds: rewrite SoTW xDS protocol tests to use protobufs rather than hand-rolled json strings In the test we very lightly reuse some of the more boring protobuf construction helper code that is also technically under test. The important thing of the protocol tests is testing the protocol. The actual inputs and outputs are largely already handled by the xds golden output tests now so these protocol tests don't have to do double-duty. This also updates the SoTW protocol test to exclusively use xDS v2 which is the only variant of SoTW that will be supported in Consul 1.10. xds: default xds.Server.AuthCheckFrequency at use-time instead of construction-time
2021-04-29 18:54:05 +00:00
}
default:
return status.Errorf(codes.Internal, "Invalid service kind")
}
// Authed OK!
return nil
}