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
This commit is contained in:
R.B. Boyer 2021-04-29 13:54:05 -05:00 committed by GitHub
parent b9130f8e6a
commit 91bee6246f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
60 changed files with 3349 additions and 913 deletions

3
.changelog/9855.txt Normal file
View File

@ -0,0 +1,3 @@
```release-note:feature
xds: exclusively support the Incremental xDS protocol when using xDS v3
```

View File

@ -655,16 +655,15 @@ func (a *Agent) listenAndServeGRPC() error {
return nil
}
xdsServer := &xds.Server{
Logger: a.logger.Named(logging.Envoy),
CfgMgr: a.proxyConfig,
ResolveToken: func(id string) (acl.Authorizer, error) {
xdsServer := xds.NewServer(
a.logger.Named(logging.Envoy),
a.proxyConfig,
func(id string) (acl.Authorizer, error) {
return a.delegate.ResolveTokenAndDefaultMeta(id, nil, nil)
},
CheckFetcher: a,
CfgFetcher: a,
AuthCheckFrequency: xds.DefaultAuthCheckFrequency,
}
a,
a,
)
tlsConfig := a.tlsConfigurator
// gRPC uses the same TLS settings as the HTTPS API. If HTTPS is not enabled

View File

@ -10,14 +10,15 @@ import (
"sync/atomic"
"time"
"github.com/mitchellh/go-testing-interface"
"github.com/stretchr/testify/require"
"github.com/hashicorp/consul/agent/cache"
cachetype "github.com/hashicorp/consul/agent/cache-types"
"github.com/hashicorp/consul/agent/connect"
"github.com/hashicorp/consul/agent/consul/discoverychain"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/api"
"github.com/mitchellh/go-testing-interface"
"github.com/stretchr/testify/require"
)
// TestCacheTypes encapsulates all the different cache types proxycfg.State will
@ -752,6 +753,10 @@ func TestConfigSnapshotDiscoveryChainWithEntries(t testing.T, additionalEntries
return testConfigSnapshotDiscoveryChain(t, "simple", additionalEntries...)
}
func TestConfigSnapshotDiscoveryChainDefaultWithEntries(t testing.T, additionalEntries ...structs.ConfigEntry) *ConfigSnapshot {
return testConfigSnapshotDiscoveryChain(t, "default", additionalEntries...)
}
func TestConfigSnapshotDiscoveryChainDefault(t testing.T) *ConfigSnapshot {
return testConfigSnapshotDiscoveryChain(t, "default")
}
@ -796,6 +801,8 @@ func testConfigSnapshotDiscoveryChain(t testing.T, variation string, additionalE
ConfigSnapshotUpstreams: setupTestVariationConfigEntriesAndSnapshot(
t, variation, leaf, additionalEntries...,
),
Intentions: nil, // no intentions defined
IntentionsSet: true,
},
Datacenter: "dc1",
}
@ -1316,7 +1323,9 @@ func setupTestVariationConfigEntriesAndSnapshot(
entries = append(entries, additionalEntries...)
}
dbChain := discoverychain.TestCompileConfigEntries(t, "db", "default", "dc1", connect.TestClusterID+".consul", "dc1", compileSetup, entries...)
dbChain := discoverychain.TestCompileConfigEntries(
t, "db", "default", "dc1",
connect.TestClusterID+".consul", "dc1", compileSetup, entries...)
upstreams := structs.TestUpstreams(t)
snap := ConfigSnapshotUpstreams{

View File

@ -20,11 +20,10 @@ import (
"github.com/hashicorp/consul/agent/connect"
"github.com/hashicorp/consul/agent/proxycfg"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/logging"
)
// clustersFromSnapshot returns the xDS API representation of the "clusters" in the snapshot.
func (s *Server) clustersFromSnapshot(_ connectionInfo, cfgSnap *proxycfg.ConfigSnapshot) ([]proto.Message, error) {
func (s *ResourceGenerator) clustersFromSnapshot(cfgSnap *proxycfg.ConfigSnapshot) ([]proto.Message, error) {
if cfgSnap == nil {
return nil, errors.New("nil config given")
}
@ -33,19 +32,60 @@ func (s *Server) clustersFromSnapshot(_ connectionInfo, cfgSnap *proxycfg.Config
case structs.ServiceKindConnectProxy:
return s.clustersFromSnapshotConnectProxy(cfgSnap)
case structs.ServiceKindTerminatingGateway:
return s.makeGatewayServiceClusters(cfgSnap, cfgSnap.TerminatingGateway.ServiceGroups, cfgSnap.TerminatingGateway.ServiceResolvers)
res, err := s.makeGatewayServiceClusters(cfgSnap, cfgSnap.TerminatingGateway.ServiceGroups, cfgSnap.TerminatingGateway.ServiceResolvers)
if err != nil {
return nil, err
}
return s.maybeInjectStubClusterForGateways(res)
case structs.ServiceKindMeshGateway:
return s.clustersFromSnapshotMeshGateway(cfgSnap)
res, err := s.clustersFromSnapshotMeshGateway(cfgSnap)
if err != nil {
return nil, err
}
return s.maybeInjectStubClusterForGateways(res)
case structs.ServiceKindIngressGateway:
return s.clustersFromSnapshotIngressGateway(cfgSnap)
res, err := s.clustersFromSnapshotIngressGateway(cfgSnap)
if err != nil {
return nil, err
}
return s.maybeInjectStubClusterForGateways(res)
default:
return nil, fmt.Errorf("Invalid service kind: %v", cfgSnap.Kind)
}
}
func (s *ResourceGenerator) maybeInjectStubClusterForGateways(resources []proto.Message) ([]proto.Message, error) {
switch {
case !s.IncrementalXDS:
return resources, nil
case !s.ProxyFeatures.GatewaysNeedStubClusterWhenEmptyWithIncrementalXDS:
return resources, nil
case len(resources) > 0:
return resources, nil
}
// For more justification for this hacky fix, check the comments associated
// with s.ProxyFeatures.GatewaysNeedStubClusterWhenEmptyWithIncrementalXDS
const stubName = "consul-stub-cluster-working-around-envoy-bug-ignore"
return []proto.Message{
&envoy_cluster_v3.Cluster{
Name: stubName,
ConnectTimeout: ptypes.DurationProto(5 * time.Second),
ClusterDiscoveryType: &envoy_cluster_v3.Cluster_Type{Type: envoy_cluster_v3.Cluster_STATIC},
LoadAssignment: &envoy_endpoint_v3.ClusterLoadAssignment{
ClusterName: stubName,
Endpoints: []*envoy_endpoint_v3.LocalityLbEndpoints{
{LbEndpoints: []*envoy_endpoint_v3.LbEndpoint{}},
},
},
},
}, nil
}
// clustersFromSnapshot returns the xDS API representation of the "clusters"
// (upstreams) in the snapshot.
func (s *Server) clustersFromSnapshotConnectProxy(cfgSnap *proxycfg.ConfigSnapshot) ([]proto.Message, error) {
func (s *ResourceGenerator) clustersFromSnapshotConnectProxy(cfgSnap *proxycfg.ConfigSnapshot) ([]proto.Message, error) {
// This sizing is a lower bound.
clusters := make([]proto.Message, 0, len(cfgSnap.ConnectProxy.DiscoveryChain)+1)
@ -139,7 +179,7 @@ func makeExposeClusterName(destinationPort int) string {
// clustersFromSnapshotMeshGateway returns the xDS API representation of the "clusters"
// for a mesh gateway. This will include 1 cluster per remote datacenter as well as
// 1 cluster for each service subset.
func (s *Server) clustersFromSnapshotMeshGateway(cfgSnap *proxycfg.ConfigSnapshot) ([]proto.Message, error) {
func (s *ResourceGenerator) clustersFromSnapshotMeshGateway(cfgSnap *proxycfg.ConfigSnapshot) ([]proto.Message, error) {
datacenters := cfgSnap.MeshGateway.Datacenters()
// 1 cluster per remote dc + 1 cluster per local service (this is a lower bound - all subset specific clusters will be appended)
@ -199,12 +239,11 @@ func (s *Server) clustersFromSnapshotMeshGateway(cfgSnap *proxycfg.ConfigSnapsho
return clusters, nil
}
func (s *Server) makeGatewayServiceClusters(
func (s *ResourceGenerator) makeGatewayServiceClusters(
cfgSnap *proxycfg.ConfigSnapshot,
services map[structs.ServiceName]structs.CheckServiceNodes,
resolvers map[structs.ServiceName]*structs.ServiceResolverConfigEntry,
) ([]proto.Message, error) {
var hostnameEndpoints structs.CheckServiceNodes
switch cfgSnap.Kind {
@ -272,7 +311,7 @@ func (s *Server) makeGatewayServiceClusters(
return clusters, nil
}
func (s *Server) injectGatewayServiceAddons(cfgSnap *proxycfg.ConfigSnapshot, c *envoy_cluster_v3.Cluster, svc structs.ServiceName, lb *structs.LoadBalancer) error {
func (s *ResourceGenerator) injectGatewayServiceAddons(cfgSnap *proxycfg.ConfigSnapshot, c *envoy_cluster_v3.Cluster, svc structs.ServiceName, lb *structs.LoadBalancer) error {
switch cfgSnap.Kind {
case structs.ServiceKindMeshGateway:
// We can't apply hash based LB config to mesh gateways because they rely on inspecting HTTP attributes
@ -306,7 +345,7 @@ func (s *Server) injectGatewayServiceAddons(cfgSnap *proxycfg.ConfigSnapshot, c
return nil
}
func (s *Server) clustersFromSnapshotIngressGateway(cfgSnap *proxycfg.ConfigSnapshot) ([]proto.Message, error) {
func (s *ResourceGenerator) clustersFromSnapshotIngressGateway(cfgSnap *proxycfg.ConfigSnapshot) ([]proto.Message, error) {
var clusters []proto.Message
createdClusters := make(map[string]bool)
for _, upstreams := range cfgSnap.IngressGateway.Upstreams {
@ -345,7 +384,7 @@ func (s *Server) clustersFromSnapshotIngressGateway(cfgSnap *proxycfg.ConfigSnap
return clusters, nil
}
func (s *Server) makeAppCluster(cfgSnap *proxycfg.ConfigSnapshot, name, pathProtocol string, port int) (*envoy_cluster_v3.Cluster, error) {
func (s *ResourceGenerator) makeAppCluster(cfgSnap *proxycfg.ConfigSnapshot, name, pathProtocol string, port int) (*envoy_cluster_v3.Cluster, error) {
var c *envoy_cluster_v3.Cluster
var err error
@ -391,7 +430,7 @@ func (s *Server) makeAppCluster(cfgSnap *proxycfg.ConfigSnapshot, name, pathProt
return c, err
}
func (s *Server) makeUpstreamClusterForPreparedQuery(upstream structs.Upstream, cfgSnap *proxycfg.ConfigSnapshot) (*envoy_cluster_v3.Cluster, error) {
func (s *ResourceGenerator) makeUpstreamClusterForPreparedQuery(upstream structs.Upstream, cfgSnap *proxycfg.ConfigSnapshot) (*envoy_cluster_v3.Cluster, error) {
var c *envoy_cluster_v3.Cluster
var err error
@ -453,7 +492,7 @@ func (s *Server) makeUpstreamClusterForPreparedQuery(upstream structs.Upstream,
return c, nil
}
func (s *Server) makeUpstreamClustersForDiscoveryChain(
func (s *ResourceGenerator) makeUpstreamClustersForDiscoveryChain(
id string,
upstream *structs.Upstream,
chain *structs.CompiledDiscoveryChain,
@ -645,7 +684,7 @@ type gatewayClusterOpts struct {
}
// makeGatewayCluster creates an Envoy cluster for a mesh or terminating gateway
func (s *Server) makeGatewayCluster(snap *proxycfg.ConfigSnapshot, opts gatewayClusterOpts) *envoy_cluster_v3.Cluster {
func (s *ResourceGenerator) makeGatewayCluster(snap *proxycfg.ConfigSnapshot, opts gatewayClusterOpts) *envoy_cluster_v3.Cluster {
cfg, err := ParseGatewayConfig(snap.Proxy.Config)
if err != nil {
// Don't hard fail on a config typo, just warn. The parse func returns
@ -725,21 +764,15 @@ func (s *Server) makeGatewayCluster(snap *proxycfg.ConfigSnapshot, opts gatewayC
dc := opts.hostnameEndpoints[idx].Node.Datacenter
service := opts.hostnameEndpoints[idx].Service.CompoundServiceName()
loggerName := logging.TerminatingGateway
if snap.Kind == structs.ServiceKindMeshGateway {
loggerName = logging.MeshGateway
}
// Fall back to last unhealthy endpoint if none were healthy
if len(endpoints) == 0 {
s.Logger.Named(loggerName).Warn("upstream service does not contain any healthy instances",
s.Logger.Warn("upstream service does not contain any healthy instances",
"dc", dc, "service", service.String())
endpoints = append(endpoints, fallback)
}
if len(uniqueHostnames) > 1 {
s.Logger.Named(loggerName).
Warn(fmt.Sprintf("service contains instances with more than one unique hostname; only %q be resolved by Envoy", hostname),
s.Logger.Warn(fmt.Sprintf("service contains instances with more than one unique hostname; only %q be resolved by Envoy", hostname),
"dc", dc, "service", service.String())
}

View File

@ -661,13 +661,10 @@ func TestClustersFromSnapshot(t *testing.T) {
}
// Need server just for logger dependency
s := Server{Logger: testutil.Logger(t)}
g := newResourceGenerator(testutil.Logger(t), nil, nil, false)
g.ProxyFeatures = sf
cInfo := connectionInfo{
Token: "my-token",
ProxyFeatures: sf,
}
clusters, err := s.clustersFromSnapshot(cInfo, snap)
clusters, err := g.clustersFromSnapshot(snap)
require.NoError(t, err)
sort.Slice(clusters, func(i, j int) bool {
@ -709,117 +706,6 @@ func TestClustersFromSnapshot(t *testing.T) {
}
}
func expectClustersJSONResources(snap *proxycfg.ConfigSnapshot) map[string]string {
return map[string]string{
"local_app": `
{
"@type": "type.googleapis.com/envoy.config.cluster.v3.Cluster",
"name": "local_app",
"type": "STATIC",
"connectTimeout": "5s",
"loadAssignment": {
"clusterName": "local_app",
"endpoints": [
{
"lbEndpoints": [
{
"endpoint": {
"address": {
"socketAddress": {
"address": "127.0.0.1",
"portValue": 8080
}
}
}
}
]
}
]
}
}`,
"db": `
{
"@type": "type.googleapis.com/envoy.config.cluster.v3.Cluster",
"name": "db.default.dc1.internal.11111111-2222-3333-4444-555555555555.consul",
"type": "EDS",
"edsClusterConfig": {
"edsConfig": {
"resourceApiVersion": "V3",
"ads": {
}
}
},
"outlierDetection": {
},
"circuitBreakers": {
},
"altStatName": "db.default.dc1.internal.11111111-2222-3333-4444-555555555555.consul",
"commonLbConfig": {
"healthyPanicThreshold": {}
},
"connectTimeout": "5s",
"transportSocket": ` + expectedUpstreamTransportSocketJSON(snap, "db.default.dc1.internal.11111111-2222-3333-4444-555555555555.consul") + `
}`,
"prepared_query:geo-cache": `
{
"@type": "type.googleapis.com/envoy.config.cluster.v3.Cluster",
"name": "geo-cache.default.dc1.query.11111111-2222-3333-4444-555555555555.consul",
"type": "EDS",
"edsClusterConfig": {
"edsConfig": {
"resourceApiVersion": "V3",
"ads": {
}
}
},
"outlierDetection": {
},
"circuitBreakers": {
},
"connectTimeout": "5s",
"transportSocket": ` + expectedUpstreamTransportSocketJSON(snap, "geo-cache.default.dc1.query.11111111-2222-3333-4444-555555555555.consul") + `
}`,
}
}
func expectClustersJSONFromResources(snap *proxycfg.ConfigSnapshot, v, n uint64, resourcesJSON map[string]string) string {
resJSON := ""
// Sort resources into specific order because that matters in JSONEq
// comparison later.
keyOrder := []string{"local_app"}
for _, u := range snap.Proxy.Upstreams {
keyOrder = append(keyOrder, u.Identifier())
}
for _, k := range keyOrder {
j, ok := resourcesJSON[k]
if !ok {
continue
}
if resJSON != "" {
resJSON += ",\n"
}
resJSON += j
}
return `{
"versionInfo": "` + hexString(v) + `",
"resources": [` + resJSON + `],
"typeUrl": "type.googleapis.com/envoy.config.cluster.v3.Cluster",
"nonce": "` + hexString(n) + `"
}`
}
func expectClustersJSON(snap *proxycfg.ConfigSnapshot, v, n uint64) string {
return expectClustersJSONFromResources(snap, v, n, expectClustersJSONResources(snap))
}
type customClusterJSONOptions struct {
Name string
TLSContext string

View File

@ -1,9 +1,10 @@
package xds
import (
envoy_cluster_v3 "github.com/envoyproxy/go-control-plane/envoy/config/cluster/v3"
"strings"
envoy_cluster_v3 "github.com/envoyproxy/go-control-plane/envoy/config/cluster/v3"
"github.com/golang/protobuf/ptypes"
"github.com/golang/protobuf/ptypes/wrappers"
"github.com/mitchellh/mapstructure"

View File

@ -3,8 +3,9 @@ package xds
import (
"testing"
"github.com/hashicorp/consul/agent/structs"
"github.com/stretchr/testify/require"
"github.com/hashicorp/consul/agent/structs"
)
func TestParseProxyConfig(t *testing.T) {

739
agent/xds/delta.go Normal file
View File

@ -0,0 +1,739 @@
package xds
import (
"crypto/sha256"
"encoding/hex"
"fmt"
"sort"
"sync/atomic"
"time"
envoy_cluster_v3 "github.com/envoyproxy/go-control-plane/envoy/config/cluster/v3"
envoy_config_core_v3 "github.com/envoyproxy/go-control-plane/envoy/config/core/v3"
envoy_endpoint_v3 "github.com/envoyproxy/go-control-plane/envoy/config/endpoint/v3"
envoy_listener_v3 "github.com/envoyproxy/go-control-plane/envoy/config/listener/v3"
envoy_route_v3 "github.com/envoyproxy/go-control-plane/envoy/config/route/v3"
envoy_discovery_v3 "github.com/envoyproxy/go-control-plane/envoy/service/discovery/v3"
"github.com/golang/protobuf/proto"
"github.com/golang/protobuf/ptypes"
"github.com/hashicorp/go-hclog"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
"github.com/hashicorp/consul/agent/proxycfg"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/logging"
)
// ADSDeltaStream is a shorter way of referring to this thing...
type ADSDeltaStream = envoy_discovery_v3.AggregatedDiscoveryService_DeltaAggregatedResourcesServer
// DeltaAggregatedResources implements envoy_discovery_v3.AggregatedDiscoveryServiceServer
func (s *Server) DeltaAggregatedResources(stream ADSDeltaStream) error {
// a channel for receiving incoming requests
reqCh := make(chan *envoy_discovery_v3.DeltaDiscoveryRequest)
reqStop := int32(0)
go func() {
for {
req, err := stream.Recv()
if atomic.LoadInt32(&reqStop) != 0 {
return
}
if err != nil {
s.Logger.Error("Error receiving new DeltaDiscoveryRequest; closing request channel", "error", err)
close(reqCh)
return
}
reqCh <- req
}
}()
err := s.processDelta(stream, reqCh)
if err != nil {
s.Logger.Error("Error handling ADS delta stream", "xdsVersion", "v3", "error", err)
}
// prevents writing to a closed channel if send failed on blocked recv
atomic.StoreInt32(&reqStop, 1)
return err
}
const (
stateDeltaInit int = iota
stateDeltaPendingInitialConfig
stateDeltaRunning
)
func (s *Server) processDelta(stream ADSDeltaStream, reqCh <-chan *envoy_discovery_v3.DeltaDiscoveryRequest) error {
// Loop state
var (
cfgSnap *proxycfg.ConfigSnapshot
node *envoy_config_core_v3.Node
stateCh <-chan *proxycfg.ConfigSnapshot
watchCancel func()
proxyID structs.ServiceID
nonce uint64 // xDS requires a unique nonce to correlate response/request pairs
ready bool // set to true after the first snapshot arrives
)
var (
// resourceMap is the SoTW we are incrementally attempting to sync to envoy.
//
// type => name => proto
resourceMap = emptyIndexedResources()
// currentVersions is the the xDS versioning represented by Resources.
//
// type => name => version (as consul knows right now)
currentVersions = make(map[string]map[string]string)
)
generator := newResourceGenerator(
s.Logger.Named(logging.XDS).With("xdsVersion", "v3"),
s.CheckFetcher,
s.CfgFetcher,
true,
)
// need to run a small state machine to get through initial authentication.
var state = stateDeltaInit
// Configure handlers for each type of request we currently care about.
handlers := map[string]*xDSDeltaType{
ListenerType: newDeltaType(generator, stream, ListenerType, func(kind structs.ServiceKind) bool {
return cfgSnap.Kind == structs.ServiceKindIngressGateway
}),
RouteType: newDeltaType(generator, stream, RouteType, func(kind structs.ServiceKind) bool {
return cfgSnap.Kind == structs.ServiceKindIngressGateway
}),
ClusterType: newDeltaType(generator, stream, ClusterType, func(kind structs.ServiceKind) bool {
// Mesh, Ingress, and Terminating gateways are allowed to inform CDS of
// no clusters.
return cfgSnap.Kind == structs.ServiceKindMeshGateway ||
cfgSnap.Kind == structs.ServiceKindTerminatingGateway ||
cfgSnap.Kind == structs.ServiceKindIngressGateway
}),
EndpointType: newDeltaType(generator, stream, EndpointType, nil),
}
var authTimer <-chan time.Time
extendAuthTimer := func() {
authTimer = time.After(s.AuthCheckFrequency)
}
checkStreamACLs := func(cfgSnap *proxycfg.ConfigSnapshot) error {
return s.checkStreamACLs(stream.Context(), cfgSnap)
}
for {
select {
case <-authTimer:
// It's been too long since a Discovery{Request,Response} so recheck ACLs.
if err := checkStreamACLs(cfgSnap); err != nil {
return err
}
extendAuthTimer()
case req, ok := <-reqCh:
if !ok {
// reqCh is closed when stream.Recv errors which is how we detect client
// going away. AFAICT the stream.Context() is only canceled once the
// RPC method returns which it can't until we return from this one so
// there's no point in blocking on that.
return nil
}
generator.logTraceRequest("Incremental xDS v3", req)
if req.TypeUrl == "" {
return status.Errorf(codes.InvalidArgument, "type URL is required for ADS")
}
if handler, ok := handlers[req.TypeUrl]; ok {
if handler.Recv(req) {
generator.Logger.Trace("subscribing to type", "typeUrl", req.TypeUrl)
}
}
if node == nil && req.Node != nil {
node = req.Node
var err error
generator.ProxyFeatures, err = determineSupportedProxyFeatures(req.Node)
if err != nil {
return status.Errorf(codes.InvalidArgument, err.Error())
}
}
case cfgSnap = <-stateCh:
newRes, err := generator.allResourcesFromSnapshot(cfgSnap)
if err != nil {
return status.Errorf(codes.Unavailable, "failed to generate all xDS resources from the snapshot: %v", err)
}
// index and hash the xDS structures
newResourceMap := indexResources(generator.Logger, newRes)
newVersions, err := computeResourceVersions(newResourceMap)
if err != nil {
return status.Errorf(codes.Unavailable, "failed to compute xDS resource versions: %v", err)
}
resourceMap = newResourceMap
currentVersions = newVersions
ready = true
}
// Trigger state machine
switch state {
case stateDeltaInit:
if node == nil {
// This can't happen (tm) since stateCh is nil until after the first req
// is received but lets not panic about it.
continue
}
// Start authentication process, we need the proxyID
proxyID = structs.NewServiceID(node.Id, parseEnterpriseMeta(node))
// Start watching config for that proxy
stateCh, watchCancel = s.CfgMgr.Watch(proxyID)
// Note that in this case we _intend_ the defer to only be triggered when
// this whole process method ends (i.e. when streaming RPC aborts) not at
// the end of the current loop iteration. We have to do it in the loop
// here since we can't start watching until we get to this state in the
// state machine.
defer watchCancel()
generator.Logger = generator.Logger.With("service_id", proxyID.String()) // enhance future logs
generator.Logger.Trace("watching proxy, pending initial proxycfg snapshot for xDS")
// Now wait for the config so we can check ACL
state = stateDeltaPendingInitialConfig
case stateDeltaPendingInitialConfig:
if cfgSnap == nil {
// Nothing we can do until we get the initial config
continue
}
// Got config, try to authenticate next.
state = stateDeltaRunning
// Upgrade the logger
switch cfgSnap.Kind {
case structs.ServiceKindConnectProxy:
case structs.ServiceKindTerminatingGateway:
generator.Logger = generator.Logger.Named(logging.TerminatingGateway)
case structs.ServiceKindMeshGateway:
generator.Logger = generator.Logger.Named(logging.MeshGateway)
case structs.ServiceKindIngressGateway:
generator.Logger = generator.Logger.Named(logging.IngressGateway)
}
generator.Logger.Trace("Got initial config snapshot")
// Lets actually process the config we just got or we'll mis responding
fallthrough
case stateDeltaRunning:
// Check ACLs on every Discovery{Request,Response}.
if err := checkStreamACLs(cfgSnap); err != nil {
return err
}
// For the first time through the state machine, this is when the
// timer is first started.
extendAuthTimer()
if !ready {
generator.Logger.Trace("Skipping delta computation because we haven't gotten a snapshot yet")
continue
}
var pendingTypes []string
for typeUrl, handler := range handlers {
if !handler.registered {
continue
}
if len(handler.pendingUpdates) > 0 {
pendingTypes = append(pendingTypes, typeUrl)
}
}
if len(pendingTypes) > 0 {
sort.Strings(pendingTypes)
generator.Logger.Trace("Skipping delta computation because there are responses in flight",
"pendingTypeUrls", pendingTypes)
continue
}
generator.Logger.Trace("Invoking all xDS resource handlers and sending changed data if there are any")
sentType := make(map[string]struct{}) // use this to only do one kind of mutation per type per execution
for _, op := range xDSUpdateOrder {
if _, sent := sentType[op.TypeUrl]; sent {
continue
}
err, sent := handlers[op.TypeUrl].SendIfNew(
cfgSnap.Kind,
currentVersions[op.TypeUrl],
resourceMap,
&nonce,
op.Upsert,
op.Remove,
)
if err != nil {
return status.Errorf(codes.Unavailable,
"failed to send %sreply for type %q: %v",
op.errorLogNameReplyPrefix(),
op.TypeUrl, err)
}
if sent {
sentType[op.TypeUrl] = struct{}{}
if generator.ProxyFeatures.IncrementalXDSUpdatesMustBeSerial {
// For more justification for this hacky fix, check the
// comments associated with
// generator.ProxyFeatures.IncrementalXDSUpdatesMustBeSerial
break
}
}
}
}
}
}
var xDSUpdateOrder = []xDSUpdateOperation{
// 1. CDS updates (if any) must always be pushed first.
{TypeUrl: ClusterType, Upsert: true},
// 2. EDS updates (if any) must arrive after CDS updates for the respective clusters.
{TypeUrl: EndpointType, Upsert: true},
// 3. LDS updates must arrive after corresponding CDS/EDS updates.
{TypeUrl: ListenerType, Upsert: true, Remove: true},
// 4. RDS updates related to the newly added listeners must arrive after CDS/EDS/LDS updates.
{TypeUrl: RouteType, Upsert: true, Remove: true},
// 5. (NOT IMPLEMENTED YET IN CONSUL) VHDS updates (if any) related to the newly added RouteConfigurations must arrive after RDS updates.
// {},
// 6. Stale CDS clusters and related EDS endpoints (ones no longer being referenced) can then be removed.
{TypeUrl: ClusterType, Remove: true},
{TypeUrl: EndpointType, Remove: true},
// xDS updates can be pushed independently if no new
// clusters/routes/listeners are added or if its acceptable to
// temporarily drop traffic during updates. Note that in case of
// LDS updates, the listeners will be warmed before they receive
// traffic, i.e. the dependent routes are fetched through RDS if
// configured. Clusters are warmed when adding/removing/updating
// clusters. On the other hand, routes are not warmed, i.e., the
// management plane must ensure that clusters referenced by a route
// are in place, before pushing the updates for a route.
}
type xDSUpdateOperation struct {
TypeUrl string
Upsert bool
Remove bool
}
func (op *xDSUpdateOperation) errorLogNameReplyPrefix() string {
switch {
case op.Upsert && op.Remove:
return "upsert/remove "
case op.Upsert:
return "upsert "
case op.Remove:
return "remove "
default:
return ""
}
}
type xDSDeltaType struct {
generator *ResourceGenerator
stream ADSDeltaStream
typeURL string
allowEmptyFn func(kind structs.ServiceKind) bool
// registered indicates if this type has been requested at least once by
// the proxy
registered bool
// wildcard indicates that this type was requested with no preference for
// specific resource names. subscribe/unsubscribe are ignored.
wildcard bool
// sentToEnvoyOnce is true after we've sent one response to envoy.
sentToEnvoyOnce bool
// resourceVersions is the current view of CONFIRMED/ACKed updates to
// envoy's view of the loaded resources.
//
// name => version
resourceVersions map[string]string
// pendingUpdates is a set of un-ACKed updates to the 'resourceVersions'
// map. Once we get an ACK from envoy we'll update the resourceVersions map
// and strike the entry from this map.
//
// nonce -> name -> version
pendingUpdates map[string]map[string]string
}
func newDeltaType(
generator *ResourceGenerator,
stream ADSDeltaStream,
typeUrl string,
allowEmptyFn func(kind structs.ServiceKind) bool,
) *xDSDeltaType {
return &xDSDeltaType{
generator: generator,
stream: stream,
typeURL: typeUrl,
allowEmptyFn: allowEmptyFn,
resourceVersions: make(map[string]string),
pendingUpdates: make(map[string]map[string]string),
}
}
// Recv handles new discovery requests from envoy.
//
// Returns true the first time a type receives a request.
func (t *xDSDeltaType) Recv(req *envoy_discovery_v3.DeltaDiscoveryRequest) bool {
if t == nil {
return false // not something we care about
}
logger := t.generator.Logger.With("typeUrl", t.typeURL)
registeredThisTime := false
if !t.registered {
// We are in the wildcard mode if the first request of a particular
// type has empty subscription list
t.wildcard = len(req.ResourceNamesSubscribe) == 0
t.registered = true
registeredThisTime = true
}
/*
DeltaDiscoveryRequest can be sent in the following situations:
Initial message in a xDS bidirectional gRPC stream.
As an ACK or NACK response to a previous DeltaDiscoveryResponse. In
this case the response_nonce is set to the nonce value in the Response.
ACK or NACK is determined by the absence or presence of error_detail.
Spontaneous DeltaDiscoveryRequests from the client. This can be done to
dynamically add or remove elements from the tracked resource_names set.
In this case response_nonce must be omitted.
*/
/*
DeltaDiscoveryRequest plays two independent roles. Any
DeltaDiscoveryRequest can be either or both of:
*/
if req.ResponseNonce != "" {
/*
[2] (N)ACKing an earlier resource update from the server (using
response_nonce, with presence of error_detail making it a NACK).
*/
if req.ErrorDetail == nil {
logger.Trace("got ok response from envoy proxy", "nonce", req.ResponseNonce)
t.ack(req.ResponseNonce)
} else {
logger.Error("got error response from envoy proxy", "nonce", req.ResponseNonce,
"error", status.ErrorProto(req.ErrorDetail))
t.nack(req.ResponseNonce)
}
}
if registeredThisTime && len(req.InitialResourceVersions) > 0 {
/*
Additionally, the first message (for a given type_url) of a
reconnected gRPC stream has a third role:
[3] informing the server of the resources (and their versions) that
the client already possesses, using the initial_resource_versions
field.
*/
logger.Trace("setting initial resource versions for stream",
"resources", req.InitialResourceVersions)
t.resourceVersions = req.InitialResourceVersions
}
if !t.wildcard {
/*
[1] informing the server of what resources the client has
gained/lost interest in (using resource_names_subscribe and
resource_names_unsubscribe), or
*/
for _, name := range req.ResourceNamesSubscribe {
// A resource_names_subscribe field may contain resource names that
// the server believes the client is already subscribed to, and
// furthermore has the most recent versions of. However, the server
// must still provide those resources in the response; due to
// implementation details hidden from the server, the client may
// have “forgotten” those resources despite apparently remaining
// subscribed.
//
// NOTE: the server must respond with all resources listed in
// resource_names_subscribe, even if it believes the client has the
// most recent version of them. The reason: the client may have
// dropped them, but then regained interest before it had a chance
// to send the unsubscribe message.
//
// We handle that here by ALWAYS wiping the version so the diff
// decides to send the value.
_, alreadySubscribed := t.resourceVersions[name]
t.resourceVersions[name] = "" // start with no version
if alreadySubscribed {
logger.Trace("re-subscribing resource for stream", "resource", name)
} else {
logger.Trace("subscribing resource for stream", "resource", name)
}
}
for _, name := range req.ResourceNamesUnsubscribe {
if _, ok := t.resourceVersions[name]; !ok {
continue
}
delete(t.resourceVersions, name)
logger.Trace("unsubscribing resource for stream", "resource", name)
}
}
return registeredThisTime
}
func (t *xDSDeltaType) ack(nonce string) {
pending, ok := t.pendingUpdates[nonce]
if !ok {
return
}
for name, version := range pending {
if version == "" {
delete(t.resourceVersions, name)
} else {
t.resourceVersions[name] = version
}
}
t.sentToEnvoyOnce = true
delete(t.pendingUpdates, nonce)
}
func (t *xDSDeltaType) nack(nonce string) {
delete(t.pendingUpdates, nonce)
}
func (t *xDSDeltaType) SendIfNew(
kind structs.ServiceKind,
currentVersions map[string]string, // type => name => version (as consul knows right now)
resourceMap IndexedResources,
nonce *uint64,
upsert, remove bool,
) (error, bool) {
if t == nil || !t.registered {
return nil, false
}
logger := t.generator.Logger.With("typeUrl", t.typeURL)
allowEmpty := t.allowEmptyFn != nil && t.allowEmptyFn(kind)
// Zero length resource responses should be ignored and are the result of no
// data yet. Notice that this caused a bug originally where we had zero
// healthy endpoints for an upstream that would cause Envoy to hang waiting
// for the EDS response. This is fixed though by ensuring we send an explicit
// empty LoadAssignment resource for the cluster rather than allowing junky
// empty resources.
if len(currentVersions) == 0 && !allowEmpty {
// Nothing to send yet
return nil, false
}
resp, updates, err := t.createDeltaResponse(currentVersions, resourceMap, upsert, remove)
if err != nil {
return err, false
}
if resp == nil {
return nil, false
}
*nonce++
resp.Nonce = fmt.Sprintf("%08x", *nonce)
t.generator.logTraceResponse("Incremental xDS v3", resp)
logger.Trace("sending response", "nonce", resp.Nonce)
if err := t.stream.Send(resp); err != nil {
return err, false
}
logger.Trace("sent response", "nonce", resp.Nonce)
t.pendingUpdates[resp.Nonce] = updates
return nil, true
}
func (t *xDSDeltaType) createDeltaResponse(
currentVersions map[string]string, // name => version (as consul knows right now)
resourceMap IndexedResources,
upsert, remove bool,
) (*envoy_discovery_v3.DeltaDiscoveryResponse, map[string]string, error) {
// compute difference
var (
hasRelevantUpdates = false
updates = make(map[string]string)
)
// First find things that need updating or deleting
for name, envoyVers := range t.resourceVersions {
currVers, ok := currentVersions[name]
if !ok {
if remove {
hasRelevantUpdates = true
}
updates[name] = ""
} else if currVers != envoyVers {
if upsert {
hasRelevantUpdates = true
}
updates[name] = currVers
}
}
// Now find new things
if t.wildcard {
for name, currVers := range currentVersions {
if _, ok := t.resourceVersions[name]; !ok {
updates[name] = currVers
if upsert {
hasRelevantUpdates = true
}
}
}
}
if !hasRelevantUpdates && t.sentToEnvoyOnce {
return nil, nil, nil
}
// now turn this into a disco response
resp := &envoy_discovery_v3.DeltaDiscoveryResponse{
// TODO(rb): consider putting something in SystemVersionInfo?
TypeUrl: t.typeURL,
}
realUpdates := make(map[string]string)
for name, vers := range updates {
if vers == "" {
if remove {
resp.RemovedResources = append(resp.RemovedResources, name)
realUpdates[name] = ""
}
} else if upsert {
resources, ok := resourceMap[t.typeURL]
if !ok {
return nil, nil, fmt.Errorf("unknown type url: %s", t.typeURL)
}
res, ok := resources[name]
if !ok {
return nil, nil, fmt.Errorf("unknown name for type url %q: %s", t.typeURL, name)
}
any, err := ptypes.MarshalAny(res)
if err != nil {
return nil, nil, err
}
resp.Resources = append(resp.Resources, &envoy_discovery_v3.Resource{
Name: name,
Resource: any,
Version: vers,
})
realUpdates[name] = vers
}
}
return resp, realUpdates, nil
}
func computeResourceVersions(resourceMap IndexedResources) (map[string]map[string]string, error) {
out := make(map[string]map[string]string)
for typeUrl, resources := range resourceMap {
m, err := hashResourceMap(resources)
if err != nil {
return nil, fmt.Errorf("failed to hash resources for %q: %v", typeUrl, err)
}
out[typeUrl] = m
}
return out, nil
}
type IndexedResources map[string]map[string]proto.Message
func emptyIndexedResources() IndexedResources {
return map[string]map[string]proto.Message{
ListenerType: make(map[string]proto.Message),
RouteType: make(map[string]proto.Message),
ClusterType: make(map[string]proto.Message),
EndpointType: make(map[string]proto.Message),
}
}
func indexResources(logger hclog.Logger, resources map[string][]proto.Message) IndexedResources {
data := emptyIndexedResources()
for typeURL, typeRes := range resources {
for _, res := range typeRes {
name := getResourceName(res)
if name == "" {
logger.Warn("skipping unexpected xDS type found in delta snapshot", "typeURL", typeURL)
} else {
data[typeURL][name] = res
}
}
}
return data
}
func getResourceName(res proto.Message) string {
// NOTE: this only covers types that we currently care about for LDS/RDS/CDS/EDS
switch x := res.(type) {
case *envoy_listener_v3.Listener: // LDS
return x.Name
case *envoy_route_v3.RouteConfiguration: // RDS
return x.Name
case *envoy_cluster_v3.Cluster: // CDS
return x.Name
case *envoy_endpoint_v3.ClusterLoadAssignment: // EDS
return x.ClusterName
default:
return ""
}
}
func hashResourceMap(resources map[string]proto.Message) (map[string]string, error) {
m := make(map[string]string)
for name, res := range resources {
h, err := hashResource(res)
if err != nil {
return nil, fmt.Errorf("failed to hash resource %q: %v", name, err)
}
m[name] = h
}
return m, nil
}
// hashResource will take a resource and create a SHA256 hash sum out of the marshaled bytes
func hashResource(res proto.Message) (string, error) {
h := sha256.New()
buffer := proto.NewBuffer(nil)
buffer.SetDeterministic(true)
err := buffer.Marshal(res)
if err != nil {
return "", err
}
h.Write(buffer.Bytes())
buffer.Reset()
return hex.EncodeToString(h.Sum(nil)), nil
}

744
agent/xds/delta_test.go Normal file
View File

@ -0,0 +1,744 @@
package xds
import (
"sync/atomic"
"testing"
"time"
envoy_discovery_v3 "github.com/envoyproxy/go-control-plane/envoy/service/discovery/v3"
"github.com/golang/protobuf/proto"
"github.com/stretchr/testify/require"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
"github.com/hashicorp/consul/acl"
"github.com/hashicorp/consul/agent/proxycfg"
"github.com/hashicorp/consul/agent/structs"
)
/* TODO: Test scenarios
- initial resource versions
- removing resources
- nack
- unsubscribe
- error during handling causing retry
*/
// NOTE: For these tests, prefer not using xDS protobuf "factory" methods if
// possible to avoid using them to test themselves.
//
// Stick to very straightforward stuff in xds_protocol_helpers_test.go.
func TestServer_DeltaAggregatedResources_v3_BasicProtocol_TCP(t *testing.T) {
aclResolve := func(id string) (acl.Authorizer, error) {
// Allow all
return acl.RootAuthorizer("manage"), nil
}
scenario := newTestServerDeltaScenario(t, aclResolve, "web-sidecar-proxy", "", 0)
mgr, errCh, envoy := scenario.mgr, scenario.errCh, scenario.envoy
sid := structs.NewServiceID("web-sidecar-proxy", nil)
// Register the proxy to create state needed to Watch() on
mgr.RegisterProxy(t, sid)
snap := newTestSnapshot(t, nil, "")
// Send initial cluster discover. We'll assume we are testing a partial
// reconnect and include some initial resource versions that will be
// cleaned up.
envoy.SendDeltaReq(t, ClusterType, &envoy_discovery_v3.DeltaDiscoveryRequest{
InitialResourceVersions: mustMakeVersionMap(t,
makeTestCluster(t, snap, "tcp:geo-cache"),
),
})
// Check no response sent yet
assertDeltaChanBlocked(t, envoy.deltaStream.sendCh)
// Deliver a new snapshot (tcp with one tcp upstream)
mgr.DeliverConfig(t, sid, snap)
assertDeltaResponseSent(t, envoy.deltaStream.sendCh, &envoy_discovery_v3.DeltaDiscoveryResponse{
TypeUrl: ClusterType,
Nonce: hexString(1),
Resources: makeTestResources(t,
makeTestCluster(t, snap, "tcp:local_app"),
makeTestCluster(t, snap, "tcp:db"),
// SAME_AS_INITIAL_VERSION: makeTestCluster(t, snap, "tcp:geo-cache"),
),
})
// Envoy then tries to discover endpoints for those clusters.
envoy.SendDeltaReq(t, EndpointType, &envoy_discovery_v3.DeltaDiscoveryRequest{
// We'll assume we are testing a partial "reconnect"
InitialResourceVersions: mustMakeVersionMap(t,
makeTestEndpoints(t, snap, "tcp:geo-cache"),
),
ResourceNamesSubscribe: []string{
"db.default.dc1.internal.11111111-2222-3333-4444-555555555555.consul",
// "geo-cache.default.dc1.query.11111111-2222-3333-4444-555555555555.consul",
//
// Include "fake-endpoints" here to test subscribing to an unknown
// thing and have consul tell us there's no data for it.
"fake-endpoints",
},
})
// It also (in parallel) issues the cluster ACK
envoy.SendDeltaReqACK(t, ClusterType, 1, true, nil)
// We should get a response immediately since the config is already present in
// the server for endpoints. Note that this should not be racy if the server
// is behaving well since the Cluster send above should be blocked until we
// deliver a new config version.
assertDeltaResponseSent(t, envoy.deltaStream.sendCh, &envoy_discovery_v3.DeltaDiscoveryResponse{
TypeUrl: EndpointType,
Nonce: hexString(2),
Resources: makeTestResources(t,
makeTestEndpoints(t, snap, "tcp:db"),
// SAME_AS_INITIAL_VERSION: makeTestEndpoints(t, snap, "tcp:geo-cache"),
),
})
// And no other response yet
assertDeltaChanBlocked(t, envoy.deltaStream.sendCh)
// Envoy now sends listener request
envoy.SendDeltaReq(t, ListenerType, nil)
// It also (in parallel) issues the endpoint ACK
envoy.SendDeltaReqACK(t, EndpointType, 2, true, nil)
// And should get a response immediately.
assertDeltaResponseSent(t, envoy.deltaStream.sendCh, &envoy_discovery_v3.DeltaDiscoveryResponse{
TypeUrl: ListenerType,
Nonce: hexString(3),
Resources: makeTestResources(t,
makeTestListener(t, snap, "tcp:public_listener"),
makeTestListener(t, snap, "tcp:db"),
makeTestListener(t, snap, "tcp:geo-cache"),
),
})
// cleanup unused resources now that we've created/updated relevant things
assertDeltaResponseSent(t, envoy.deltaStream.sendCh, &envoy_discovery_v3.DeltaDiscoveryResponse{
TypeUrl: EndpointType,
Nonce: hexString(4),
RemovedResources: []string{
"fake-endpoints", // correcting the errant subscription
},
})
// And no other response yet
assertDeltaChanBlocked(t, envoy.deltaStream.sendCh)
// ACKs the listener
envoy.SendDeltaReqACK(t, ListenerType, 3, true, nil)
// ACK the endpoint removal
envoy.SendDeltaReqACK(t, EndpointType, 4, true, nil)
// If we re-subscribe to something even if there are no changes we get a
// fresh copy.
envoy.SendDeltaReq(t, EndpointType, &envoy_discovery_v3.DeltaDiscoveryRequest{
ResourceNamesSubscribe: []string{
"geo-cache.default.dc1.query.11111111-2222-3333-4444-555555555555.consul",
},
})
assertDeltaResponseSent(t, envoy.deltaStream.sendCh, &envoy_discovery_v3.DeltaDiscoveryResponse{
TypeUrl: EndpointType,
Nonce: hexString(5),
Resources: makeTestResources(t,
makeTestEndpoints(t, snap, "tcp:geo-cache"),
),
})
envoy.SendDeltaReqACK(t, EndpointType, 5, true, nil)
// And no other response yet
assertDeltaChanBlocked(t, envoy.deltaStream.sendCh)
// TODO(rb): test NACK
envoy.Close()
select {
case err := <-errCh:
require.NoError(t, err)
case <-time.After(50 * time.Millisecond):
t.Fatalf("timed out waiting for handler to finish")
}
}
func TestServer_DeltaAggregatedResources_v3_BasicProtocol_HTTP2(t *testing.T) {
aclResolve := func(id string) (acl.Authorizer, error) {
// Allow all
return acl.RootAuthorizer("manage"), nil
}
scenario := newTestServerDeltaScenario(t, aclResolve, "web-sidecar-proxy", "", 0)
mgr, errCh, envoy := scenario.mgr, scenario.errCh, scenario.envoy
sid := structs.NewServiceID("web-sidecar-proxy", nil)
// Register the proxy to create state needed to Watch() on
mgr.RegisterProxy(t, sid)
// Send initial cluster discover (empty payload)
envoy.SendDeltaReq(t, ClusterType, nil)
// Check no response sent yet
assertDeltaChanBlocked(t, envoy.deltaStream.sendCh)
// Deliver a new snapshot (tcp with one http upstream)
snap := newTestSnapshot(t, nil, "http2", &structs.ServiceConfigEntry{
Kind: structs.ServiceDefaults,
Name: "db",
Protocol: "http2",
})
mgr.DeliverConfig(t, sid, snap)
runStep(t, "no-rds", func(t *testing.T) {
assertDeltaResponseSent(t, envoy.deltaStream.sendCh, &envoy_discovery_v3.DeltaDiscoveryResponse{
TypeUrl: ClusterType,
Nonce: hexString(1),
Resources: makeTestResources(t,
makeTestCluster(t, snap, "tcp:local_app"),
makeTestCluster(t, snap, "http2:db"),
makeTestCluster(t, snap, "tcp:geo-cache"),
),
})
// Envoy then tries to discover endpoints for those clusters.
envoy.SendDeltaReq(t, EndpointType, &envoy_discovery_v3.DeltaDiscoveryRequest{
ResourceNamesSubscribe: []string{
"db.default.dc1.internal.11111111-2222-3333-4444-555555555555.consul",
"geo-cache.default.dc1.query.11111111-2222-3333-4444-555555555555.consul",
},
})
// It also (in parallel) issues the cluster ACK
envoy.SendDeltaReqACK(t, ClusterType, 1, true, nil)
// We should get a response immediately since the config is already present in
// the server for endpoints. Note that this should not be racy if the server
// is behaving well since the Cluster send above should be blocked until we
// deliver a new config version.
assertDeltaResponseSent(t, envoy.deltaStream.sendCh, &envoy_discovery_v3.DeltaDiscoveryResponse{
TypeUrl: EndpointType,
Nonce: hexString(2),
Resources: makeTestResources(t,
makeTestEndpoints(t, snap, "http2:db"),
makeTestEndpoints(t, snap, "tcp:geo-cache"),
),
})
// And no other response yet
assertDeltaChanBlocked(t, envoy.deltaStream.sendCh)
// Envoy now sends listener request
envoy.SendDeltaReq(t, ListenerType, nil)
// It also (in parallel) issues the endpoint ACK
envoy.SendDeltaReqACK(t, EndpointType, 2, true, nil)
// And should get a response immediately.
assertDeltaResponseSent(t, envoy.deltaStream.sendCh, &envoy_discovery_v3.DeltaDiscoveryResponse{
TypeUrl: ListenerType,
Nonce: hexString(3),
Resources: makeTestResources(t,
makeTestListener(t, snap, "tcp:public_listener"),
makeTestListener(t, snap, "http2:db"),
makeTestListener(t, snap, "tcp:geo-cache"),
),
})
// And no other response yet
assertDeltaChanBlocked(t, envoy.deltaStream.sendCh)
// ACKs the listener
envoy.SendDeltaReqACK(t, ListenerType, 3, true, nil)
// And no other response yet
assertDeltaChanBlocked(t, envoy.deltaStream.sendCh)
})
// -- reconfigure with a no-op discovery chain
snap = newTestSnapshot(t, snap, "http2", &structs.ServiceConfigEntry{
Kind: structs.ServiceDefaults,
Name: "db",
Protocol: "http2",
}, &structs.ServiceRouterConfigEntry{
Kind: structs.ServiceRouter,
Name: "db",
Routes: nil,
})
mgr.DeliverConfig(t, sid, snap)
runStep(t, "with-rds", func(t *testing.T) {
// Just the "db" listener sees a change
assertDeltaResponseSent(t, envoy.deltaStream.sendCh, &envoy_discovery_v3.DeltaDiscoveryResponse{
TypeUrl: ListenerType,
Nonce: hexString(4),
Resources: makeTestResources(t,
makeTestListener(t, snap, "http2:db:rds"),
),
})
// And no other response yet
assertDeltaChanBlocked(t, envoy.deltaStream.sendCh)
// Envoy now sends routes request
envoy.SendDeltaReq(t, RouteType, &envoy_discovery_v3.DeltaDiscoveryRequest{
ResourceNamesSubscribe: []string{
"db",
},
})
// ACKs the listener
envoy.SendDeltaReqACK(t, ListenerType, 4, true, nil)
// And should get a response immediately.
assertDeltaResponseSent(t, envoy.deltaStream.sendCh, &envoy_discovery_v3.DeltaDiscoveryResponse{
TypeUrl: RouteType,
Nonce: hexString(5),
Resources: makeTestResources(t,
makeTestRoute(t, "http2:db"),
),
})
envoy.SendDeltaReqACK(t, RouteType, 5, true, nil)
assertDeltaChanBlocked(t, envoy.deltaStream.sendCh)
})
envoy.Close()
select {
case err := <-errCh:
require.NoError(t, err)
case <-time.After(50 * time.Millisecond):
t.Fatalf("timed out waiting for handler to finish")
}
}
func TestServer_DeltaAggregatedResources_v3_ACLEnforcement(t *testing.T) {
tests := []struct {
name string
defaultDeny bool
acl string
token string
wantDenied bool
cfgSnap *proxycfg.ConfigSnapshot
}{
// Note that although we've stubbed actual ACL checks in the testManager
// ConnectAuthorize mock, by asserting against specific reason strings here
// even in the happy case which can't match the default one returned by the
// mock we are implicitly validating that the implementation used the
// correct token from the context.
{
name: "no ACLs configured",
defaultDeny: false,
wantDenied: false,
},
{
name: "default deny, no token",
defaultDeny: true,
wantDenied: true,
},
{
name: "default deny, write token",
defaultDeny: true,
acl: `service "web" { policy = "write" }`,
token: "service-write-on-web",
wantDenied: false,
},
{
name: "default deny, read token",
defaultDeny: true,
acl: `service "web" { policy = "read" }`,
token: "service-write-on-web",
wantDenied: true,
},
{
name: "default deny, write token on different service",
defaultDeny: true,
acl: `service "not-web" { policy = "write" }`,
token: "service-write-on-not-web",
wantDenied: true,
},
{
name: "ingress default deny, write token on different service",
defaultDeny: true,
acl: `service "not-ingress" { policy = "write" }`,
token: "service-write-on-not-ingress",
wantDenied: true,
cfgSnap: proxycfg.TestConfigSnapshotIngressGateway(t),
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
aclResolve := func(id string) (acl.Authorizer, error) {
if !tt.defaultDeny {
// Allow all
return acl.RootAuthorizer("allow"), nil
}
if tt.acl == "" {
// No token and defaultDeny is denied
return acl.RootAuthorizer("deny"), nil
}
// Ensure the correct token was passed
require.Equal(t, tt.token, id)
// Parse the ACL and enforce it
policy, err := acl.NewPolicyFromSource("", 0, tt.acl, acl.SyntaxLegacy, nil, nil)
require.NoError(t, err)
return acl.NewPolicyAuthorizerWithDefaults(acl.RootAuthorizer("deny"), []*acl.Policy{policy}, nil)
}
scenario := newTestServerDeltaScenario(t, aclResolve, "web-sidecar-proxy", tt.token, 0)
mgr, errCh, envoy := scenario.mgr, scenario.errCh, scenario.envoy
sid := structs.NewServiceID("web-sidecar-proxy", nil)
// Register the proxy to create state needed to Watch() on
mgr.RegisterProxy(t, sid)
// Deliver a new snapshot
snap := tt.cfgSnap
if snap == nil {
snap = newTestSnapshot(t, nil, "")
}
mgr.DeliverConfig(t, sid, snap)
// Send initial listener discover, in real life Envoy always sends cluster
// first but it doesn't really matter and listener has a response that
// includes the token in the ext rbac filter so lets us test more stuff.
envoy.SendDeltaReq(t, ListenerType, nil)
if !tt.wantDenied {
assertDeltaResponseSent(t, envoy.deltaStream.sendCh, &envoy_discovery_v3.DeltaDiscoveryResponse{
TypeUrl: ListenerType,
Nonce: hexString(1),
Resources: makeTestResources(t,
makeTestListener(t, snap, "tcp:public_listener"),
makeTestListener(t, snap, "tcp:db"),
makeTestListener(t, snap, "tcp:geo-cache"),
),
})
// Close the client stream since all is well. We _don't_ do this in the
// expected error case because we want to verify the error closes the
// stream from server side.
envoy.Close()
}
select {
case err := <-errCh:
if tt.wantDenied {
require.Error(t, err)
require.Contains(t, err.Error(), "permission denied")
mgr.AssertWatchCancelled(t, sid)
} else {
require.NoError(t, err)
}
case <-time.After(50 * time.Millisecond):
t.Fatalf("timed out waiting for handler to finish")
}
})
}
}
func TestServer_DeltaAggregatedResources_v3_ACLTokenDeleted_StreamTerminatedDuringDiscoveryRequest(t *testing.T) {
if testing.Short() {
t.Skip("too slow for testing.Short")
}
aclRules := `service "web" { policy = "write" }`
token := "service-write-on-web"
policy, err := acl.NewPolicyFromSource("", 0, aclRules, acl.SyntaxLegacy, nil, nil)
require.NoError(t, err)
var validToken atomic.Value
validToken.Store(token)
aclResolve := func(id string) (acl.Authorizer, error) {
if token := validToken.Load(); token == nil || id != token.(string) {
return nil, acl.ErrNotFound
}
return acl.NewPolicyAuthorizerWithDefaults(acl.RootAuthorizer("deny"), []*acl.Policy{policy}, nil)
}
scenario := newTestServerDeltaScenario(t, aclResolve, "web-sidecar-proxy", token,
100*time.Millisecond, // Make this short.
)
mgr, errCh, envoy := scenario.mgr, scenario.errCh, scenario.envoy
getError := func() (gotErr error, ok bool) {
select {
case err := <-errCh:
return err, true
default:
return nil, false
}
}
sid := structs.NewServiceID("web-sidecar-proxy", nil)
// Register the proxy to create state needed to Watch() on
mgr.RegisterProxy(t, sid)
// Send initial cluster discover (OK)
envoy.SendDeltaReq(t, ClusterType, nil)
{
err, ok := getError()
require.NoError(t, err)
require.False(t, ok)
}
// Check no response sent yet
assertDeltaChanBlocked(t, envoy.deltaStream.sendCh)
{
err, ok := getError()
require.NoError(t, err)
require.False(t, ok)
}
// Deliver a new snapshot
snap := newTestSnapshot(t, nil, "")
mgr.DeliverConfig(t, sid, snap)
assertDeltaResponseSent(t, envoy.deltaStream.sendCh, &envoy_discovery_v3.DeltaDiscoveryResponse{
TypeUrl: ClusterType,
Nonce: hexString(1),
Resources: makeTestResources(t,
makeTestCluster(t, snap, "tcp:local_app"),
makeTestCluster(t, snap, "tcp:db"),
makeTestCluster(t, snap, "tcp:geo-cache"),
),
})
// It also (in parallel) issues the next cluster request (which acts as an ACK
// of the version we sent)
envoy.SendDeltaReq(t, ClusterType, nil)
// Check no response sent yet
assertDeltaChanBlocked(t, envoy.deltaStream.sendCh)
{
err, ok := getError()
require.NoError(t, err)
require.False(t, ok)
}
// Now nuke the ACL token while there's no activity.
validToken.Store("")
select {
case err := <-errCh:
require.Error(t, err)
gerr, ok := status.FromError(err)
require.Truef(t, ok, "not a grpc status error: type='%T' value=%v", err, err)
require.Equal(t, codes.Unauthenticated, gerr.Code())
require.Equal(t, "unauthenticated: ACL not found", gerr.Message())
mgr.AssertWatchCancelled(t, sid)
case <-time.After(200 * time.Millisecond):
t.Fatalf("timed out waiting for handler to finish")
}
}
func TestServer_DeltaAggregatedResources_v3_ACLTokenDeleted_StreamTerminatedInBackground(t *testing.T) {
if testing.Short() {
t.Skip("too slow for testing.Short")
}
aclRules := `service "web" { policy = "write" }`
token := "service-write-on-web"
policy, err := acl.NewPolicyFromSource("", 0, aclRules, acl.SyntaxLegacy, nil, nil)
require.NoError(t, err)
var validToken atomic.Value
validToken.Store(token)
aclResolve := func(id string) (acl.Authorizer, error) {
if token := validToken.Load(); token == nil || id != token.(string) {
return nil, acl.ErrNotFound
}
return acl.NewPolicyAuthorizerWithDefaults(acl.RootAuthorizer("deny"), []*acl.Policy{policy}, nil)
}
scenario := newTestServerDeltaScenario(t, aclResolve, "web-sidecar-proxy", token,
100*time.Millisecond, // Make this short.
)
mgr, errCh, envoy := scenario.mgr, scenario.errCh, scenario.envoy
getError := func() (gotErr error, ok bool) {
select {
case err := <-errCh:
return err, true
default:
return nil, false
}
}
sid := structs.NewServiceID("web-sidecar-proxy", nil)
// Register the proxy to create state needed to Watch() on
mgr.RegisterProxy(t, sid)
// Send initial cluster discover (OK)
envoy.SendDeltaReq(t, ClusterType, nil)
{
err, ok := getError()
require.NoError(t, err)
require.False(t, ok)
}
// Check no response sent yet
assertDeltaChanBlocked(t, envoy.deltaStream.sendCh)
{
err, ok := getError()
require.NoError(t, err)
require.False(t, ok)
}
// Deliver a new snapshot
snap := newTestSnapshot(t, nil, "")
mgr.DeliverConfig(t, sid, snap)
assertDeltaResponseSent(t, envoy.deltaStream.sendCh, &envoy_discovery_v3.DeltaDiscoveryResponse{
TypeUrl: ClusterType,
Nonce: hexString(1),
Resources: makeTestResources(t,
makeTestCluster(t, snap, "tcp:local_app"),
makeTestCluster(t, snap, "tcp:db"),
makeTestCluster(t, snap, "tcp:geo-cache"),
),
})
// It also (in parallel) issues the next cluster request (which acts as an ACK
// of the version we sent)
envoy.SendDeltaReq(t, ClusterType, nil)
// Check no response sent yet
assertDeltaChanBlocked(t, envoy.deltaStream.sendCh)
{
err, ok := getError()
require.NoError(t, err)
require.False(t, ok)
}
// Now nuke the ACL token while there's no activity.
validToken.Store("")
select {
case err := <-errCh:
require.Error(t, err)
gerr, ok := status.FromError(err)
require.Truef(t, ok, "not a grpc status error: type='%T' value=%v", err, err)
require.Equal(t, codes.Unauthenticated, gerr.Code())
require.Equal(t, "unauthenticated: ACL not found", gerr.Message())
mgr.AssertWatchCancelled(t, sid)
case <-time.After(200 * time.Millisecond):
t.Fatalf("timed out waiting for handler to finish")
}
}
func TestServer_DeltaAggregatedResources_v3_IngressEmptyResponse(t *testing.T) {
aclResolve := func(id string) (acl.Authorizer, error) {
// Allow all
return acl.RootAuthorizer("manage"), nil
}
scenario := newTestServerDeltaScenario(t, aclResolve, "ingress-gateway", "", 0)
mgr, errCh, envoy := scenario.mgr, scenario.errCh, scenario.envoy
sid := structs.NewServiceID("ingress-gateway", nil)
// Register the proxy to create state needed to Watch() on
mgr.RegisterProxy(t, sid)
// Send initial cluster discover
envoy.SendDeltaReq(t, ClusterType, nil)
// Check no response sent yet
assertDeltaChanBlocked(t, envoy.deltaStream.sendCh)
// Deliver a new snapshot with no services
snap := proxycfg.TestConfigSnapshotIngressGatewayNoServices(t)
mgr.DeliverConfig(t, sid, snap)
// REQ: clusters
envoy.SendDeltaReq(t, ClusterType, nil)
// RESP: clustesr
assertDeltaResponseSent(t, envoy.deltaStream.sendCh, &envoy_discovery_v3.DeltaDiscoveryResponse{
TypeUrl: ClusterType,
Nonce: hexString(1),
})
assertDeltaChanBlocked(t, envoy.deltaStream.sendCh)
// ACK: clusters
envoy.SendDeltaReqACK(t, ClusterType, 1, true, nil)
// REQ: listeners
envoy.SendDeltaReq(t, ListenerType, nil)
// RESP: listeners
assertDeltaResponseSent(t, envoy.deltaStream.sendCh, &envoy_discovery_v3.DeltaDiscoveryResponse{
TypeUrl: ListenerType,
Nonce: hexString(2),
})
assertDeltaChanBlocked(t, envoy.deltaStream.sendCh)
envoy.Close()
select {
case err := <-errCh:
require.NoError(t, err)
case <-time.After(50 * time.Millisecond):
t.Fatalf("timed out waiting for handler to finish")
}
}
func assertDeltaChanBlocked(t *testing.T, ch chan *envoy_discovery_v3.DeltaDiscoveryResponse) {
t.Helper()
select {
case r := <-ch:
t.Fatalf("chan should block but received: %v", r)
case <-time.After(10 * time.Millisecond):
return
}
}
func assertDeltaResponseSent(t *testing.T, ch chan *envoy_discovery_v3.DeltaDiscoveryResponse, want *envoy_discovery_v3.DeltaDiscoveryResponse) {
t.Helper()
select {
case got := <-ch:
assertDeltaResponse(t, got, want)
case <-time.After(50 * time.Millisecond):
t.Fatalf("no response received after 50ms")
}
}
// assertDeltaResponse is a helper to test a envoy.DeltaDiscoveryResponse matches the
// expected value. We use JSON during comparison here because the responses use protobuf
// Any type which includes binary protobuf encoding.
func assertDeltaResponse(t *testing.T, got, want *envoy_discovery_v3.DeltaDiscoveryResponse) {
t.Helper()
gotJSON := protoToSortedJSON(t, got)
wantJSON := protoToSortedJSON(t, want)
require.JSONEqf(t, wantJSON, gotJSON, "got:\n%s", gotJSON)
}
func mustMakeVersionMap(t *testing.T, resources ...proto.Message) map[string]string {
m := make(map[string]string)
for _, res := range resources {
name := getResourceName(res)
m[name] = mustHashResource(t, res)
}
return m
}

View File

@ -22,7 +22,7 @@ const (
)
// endpointsFromSnapshot returns the xDS API representation of the "endpoints"
func (s *Server) endpointsFromSnapshot(_ connectionInfo, cfgSnap *proxycfg.ConfigSnapshot) ([]proto.Message, error) {
func (s *ResourceGenerator) endpointsFromSnapshot(cfgSnap *proxycfg.ConfigSnapshot) ([]proto.Message, error) {
if cfgSnap == nil {
return nil, errors.New("nil config given")
}
@ -43,7 +43,7 @@ func (s *Server) endpointsFromSnapshot(_ connectionInfo, cfgSnap *proxycfg.Confi
// endpointsFromSnapshotConnectProxy returns the xDS API representation of the "endpoints"
// (upstream instances) in the snapshot.
func (s *Server) endpointsFromSnapshotConnectProxy(cfgSnap *proxycfg.ConfigSnapshot) ([]proto.Message, error) {
func (s *ResourceGenerator) endpointsFromSnapshotConnectProxy(cfgSnap *proxycfg.ConfigSnapshot) ([]proto.Message, error) {
resources := make([]proto.Message, 0,
len(cfgSnap.ConnectProxy.PreparedQueryEndpoints)+len(cfgSnap.ConnectProxy.WatchedUpstreamEndpoints))
@ -88,7 +88,7 @@ func (s *Server) endpointsFromSnapshotConnectProxy(cfgSnap *proxycfg.ConfigSnaps
return resources, nil
}
func (s *Server) filterSubsetEndpoints(subset *structs.ServiceResolverSubset, endpoints structs.CheckServiceNodes) (structs.CheckServiceNodes, error) {
func (s *ResourceGenerator) filterSubsetEndpoints(subset *structs.ServiceResolverSubset, endpoints structs.CheckServiceNodes) (structs.CheckServiceNodes, error) {
// locally execute the subsets filter
if subset.Filter != "" {
filter, err := bexpr.CreateFilter(subset.Filter, nil, endpoints)
@ -105,11 +105,11 @@ func (s *Server) filterSubsetEndpoints(subset *structs.ServiceResolverSubset, en
return endpoints, nil
}
func (s *Server) endpointsFromSnapshotTerminatingGateway(cfgSnap *proxycfg.ConfigSnapshot) ([]proto.Message, error) {
func (s *ResourceGenerator) endpointsFromSnapshotTerminatingGateway(cfgSnap *proxycfg.ConfigSnapshot) ([]proto.Message, error) {
return s.endpointsFromServicesAndResolvers(cfgSnap, cfgSnap.TerminatingGateway.ServiceGroups, cfgSnap.TerminatingGateway.ServiceResolvers)
}
func (s *Server) endpointsFromSnapshotMeshGateway(cfgSnap *proxycfg.ConfigSnapshot) ([]proto.Message, error) {
func (s *ResourceGenerator) endpointsFromSnapshotMeshGateway(cfgSnap *proxycfg.ConfigSnapshot) ([]proto.Message, error) {
datacenters := cfgSnap.MeshGateway.Datacenters()
resources := make([]proto.Message, 0, len(datacenters)+len(cfgSnap.MeshGateway.ServiceGroups))
@ -206,11 +206,11 @@ func (s *Server) endpointsFromSnapshotMeshGateway(cfgSnap *proxycfg.ConfigSnapsh
return resources, nil
}
func (s *Server) endpointsFromServicesAndResolvers(
func (s *ResourceGenerator) endpointsFromServicesAndResolvers(
cfgSnap *proxycfg.ConfigSnapshot,
services map[structs.ServiceName]structs.CheckServiceNodes,
resolvers map[structs.ServiceName]*structs.ServiceResolverConfigEntry) ([]proto.Message, error) {
resolvers map[structs.ServiceName]*structs.ServiceResolverConfigEntry,
) ([]proto.Message, error) {
resources := make([]proto.Message, 0, len(services))
// generate the endpoints for the linked service groups
@ -259,7 +259,7 @@ func (s *Server) endpointsFromServicesAndResolvers(
return resources, nil
}
func (s *Server) endpointsFromSnapshotIngressGateway(cfgSnap *proxycfg.ConfigSnapshot) ([]proto.Message, error) {
func (s *ResourceGenerator) endpointsFromSnapshotIngressGateway(cfgSnap *proxycfg.ConfigSnapshot) ([]proto.Message, error) {
var resources []proto.Message
createdClusters := make(map[string]bool)
for _, upstreams := range cfgSnap.IngressGateway.Upstreams {
@ -297,7 +297,7 @@ func makeEndpoint(host string, port int) *envoy_endpoint_v3.LbEndpoint {
}
}
func (s *Server) endpointsFromDiscoveryChain(
func (s *ResourceGenerator) endpointsFromDiscoveryChain(
id string,
chain *structs.CompiledDiscoveryChain,
datacenter string,

View File

@ -585,13 +585,10 @@ func TestEndpointsFromSnapshot(t *testing.T) {
}
// Need server just for logger dependency
s := Server{Logger: testutil.Logger(t)}
g := newResourceGenerator(testutil.Logger(t), nil, nil, false)
g.ProxyFeatures = sf
cInfo := connectionInfo{
Token: "my-token",
ProxyFeatures: sf,
}
endpoints, err := s.endpointsFromSnapshot(cInfo, snap)
endpoints, err := g.endpointsFromSnapshot(snap)
require.NoError(t, err)
sort.Slice(endpoints, func(i, j int) bool {

View File

@ -13,6 +13,9 @@ var (
// the zero'th point release of the last element of proxysupport.EnvoyVersions.
minSupportedVersion = version.Must(version.NewVersion("1.14.0"))
minVersionAllowingEmptyGatewayClustersWithIncrementalXDS = version.Must(version.NewVersion("1.16.0"))
minVersionAllowingMultipleIncrementalXDSChanges = version.Must(version.NewVersion("1.16.0"))
specificUnsupportedVersions = []unsupportedVersion{}
)
@ -24,6 +27,24 @@ type unsupportedVersion struct {
type supportedProxyFeatures struct {
// add version dependent feature flags here
// GatewaysNeedStubClusterWhenEmptyWithIncrementalXDS is needed to paper
// over some weird envoy behavior.
//
// For some reason Envoy versions prior to 1.16.0 when sent an empty CDS
// list via the incremental xDS protocol will correctly ack the message and
// just never request LDS resources.
GatewaysNeedStubClusterWhenEmptyWithIncrementalXDS bool
// IncrementalXDSUpdatesMustBeSerial is needed to avoid an envoy crash.
//
// Versions of Envoy prior to 1.16.0 could crash if multiple in-flight
// changes to resources were happening during incremental xDS. To prevent
// that we force serial updates on those older versions.
//
// issue: https://github.com/envoyproxy/envoy/issues/11877
// PR: https://github.com/envoyproxy/envoy/pull/12069
IncrementalXDSUpdatesMustBeSerial bool
}
func determineSupportedProxyFeatures(node *envoy_core_v3.Node) (supportedProxyFeatures, error) {
@ -59,7 +80,17 @@ func determineSupportedProxyFeaturesFromVersion(version *version.Version) (suppo
}
}
return supportedProxyFeatures{}, nil
sf := supportedProxyFeatures{}
if version.LessThan(minVersionAllowingEmptyGatewayClustersWithIncrementalXDS) {
sf.GatewaysNeedStubClusterWhenEmptyWithIncrementalXDS = true
}
if version.LessThan(minVersionAllowingMultipleIncrementalXDSChanges) {
sf.IncrementalXDSUpdatesMustBeSerial = true
}
return sf, nil
}
func determineEnvoyVersionFromNode(node *envoy_core_v3.Node) *version.Version {

View File

@ -104,6 +104,13 @@ func TestDetermineSupportedProxyFeaturesFromString(t *testing.T) {
for _, v := range []string{
"1.14.1", "1.14.2", "1.14.3", "1.14.4", "1.14.5", "1.14.6",
"1.15.0", "1.15.1", "1.15.2", "1.15.3",
} {
cases[v] = testcase{expect: supportedProxyFeatures{
GatewaysNeedStubClusterWhenEmptyWithIncrementalXDS: true,
IncrementalXDSUpdatesMustBeSerial: true,
}}
}
for _, v := range []string{
"1.16.0", "1.16.1", "1.16.2",
"1.17.0",
} {

View File

@ -25,42 +25,40 @@ import (
"github.com/golang/protobuf/ptypes"
"github.com/golang/protobuf/ptypes/any"
"github.com/golang/protobuf/ptypes/wrappers"
"github.com/hashicorp/go-hclog"
"github.com/hashicorp/consul/agent/connect"
"github.com/hashicorp/consul/agent/proxycfg"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/logging"
)
// listenersFromSnapshot returns the xDS API representation of the "listeners" in the snapshot.
func (s *Server) listenersFromSnapshot(cInfo connectionInfo, cfgSnap *proxycfg.ConfigSnapshot) ([]proto.Message, error) {
func (s *ResourceGenerator) listenersFromSnapshot(cfgSnap *proxycfg.ConfigSnapshot) ([]proto.Message, error) {
if cfgSnap == nil {
return nil, errors.New("nil config given")
}
switch cfgSnap.Kind {
case structs.ServiceKindConnectProxy:
return s.listenersFromSnapshotConnectProxy(cInfo, cfgSnap)
return s.listenersFromSnapshotConnectProxy(cfgSnap)
case structs.ServiceKindTerminatingGateway:
return s.listenersFromSnapshotGateway(cInfo, cfgSnap)
return s.listenersFromSnapshotGateway(cfgSnap)
case structs.ServiceKindMeshGateway:
return s.listenersFromSnapshotGateway(cInfo, cfgSnap)
return s.listenersFromSnapshotGateway(cfgSnap)
case structs.ServiceKindIngressGateway:
return s.listenersFromSnapshotGateway(cInfo, cfgSnap)
return s.listenersFromSnapshotGateway(cfgSnap)
default:
return nil, fmt.Errorf("Invalid service kind: %v", cfgSnap.Kind)
}
}
// listenersFromSnapshotConnectProxy returns the "listeners" for a connect proxy service
func (s *Server) listenersFromSnapshotConnectProxy(cInfo connectionInfo, cfgSnap *proxycfg.ConfigSnapshot) ([]proto.Message, error) {
func (s *ResourceGenerator) listenersFromSnapshotConnectProxy(cfgSnap *proxycfg.ConfigSnapshot) ([]proto.Message, error) {
resources := make([]proto.Message, 1)
var err error
// Configure inbound listener.
resources[0], err = s.makeInboundListener(cInfo, cfgSnap, PublicListenerName)
resources[0], err = s.makeInboundListener(cfgSnap, PublicListenerName)
if err != nil {
return nil, err
}
@ -90,7 +88,7 @@ func (s *Server) listenersFromSnapshotConnectProxy(cInfo connectionInfo, cfgSnap
for id, chain := range cfgSnap.ConnectProxy.DiscoveryChain {
upstreamCfg := cfgSnap.ConnectProxy.UpstreamConfig[id]
cfg := getAndModifyUpstreamConfigForListener(s.Logger, id, upstreamCfg, chain)
cfg := s.getAndModifyUpstreamConfigForListener(id, upstreamCfg, chain)
// If escape hatch is present, create a listener from it and move on to the next
if cfg.EnvoyListenerJSON != "" {
@ -380,7 +378,7 @@ func parseCheckPath(check structs.CheckType) (structs.ExposePath, error) {
}
// listenersFromSnapshotGateway returns the "listener" for a terminating-gateway or mesh-gateway service
func (s *Server) listenersFromSnapshotGateway(cInfo connectionInfo, cfgSnap *proxycfg.ConfigSnapshot) ([]proto.Message, error) {
func (s *ResourceGenerator) listenersFromSnapshotGateway(cfgSnap *proxycfg.ConfigSnapshot) ([]proto.Message, error) {
cfg, err := ParseGatewayConfig(cfgSnap.Proxy.Config)
if err != nil {
// Don't hard fail on a config typo, just warn. The parse func returns
@ -449,7 +447,7 @@ func (s *Server) listenersFromSnapshotGateway(cInfo connectionInfo, cfgSnap *pro
switch cfgSnap.Kind {
case structs.ServiceKindTerminatingGateway:
l, err = s.makeTerminatingGatewayListener(cInfo, cfgSnap, a.name, a.Address, a.Port)
l, err = s.makeTerminatingGatewayListener(cfgSnap, a.name, a.Address, a.Port)
if err != nil {
return nil, err
}
@ -472,7 +470,7 @@ func (s *Server) listenersFromSnapshotGateway(cInfo connectionInfo, cfgSnap *pro
return resources, err
}
func (s *Server) makeIngressGatewayListeners(address string, cfgSnap *proxycfg.ConfigSnapshot) ([]proto.Message, error) {
func (s *ResourceGenerator) makeIngressGatewayListeners(address string, cfgSnap *proxycfg.ConfigSnapshot) ([]proto.Message, error) {
var resources []proto.Message
for listenerKey, upstreams := range cfgSnap.IngressGateway.Upstreams {
@ -589,7 +587,7 @@ func makeListenerFromUserConfig(configJSON string) (*envoy_listener_v3.Listener,
// Ensure that the first filter in each filter chain of a public listener is
// the authz filter to prevent unauthorized access.
func (s *Server) injectConnectFilters(_ connectionInfo, cfgSnap *proxycfg.ConfigSnapshot, listener *envoy_listener_v3.Listener) error {
func (s *ResourceGenerator) injectConnectFilters(cfgSnap *proxycfg.ConfigSnapshot, listener *envoy_listener_v3.Listener) error {
authzFilter, err := makeRBACNetworkFilter(
cfgSnap.ConnectProxy.Intentions,
cfgSnap.IntentionDefaultAllow,
@ -679,7 +677,7 @@ func injectHTTPFilterOnFilterChains(
// for now as it allows them to specify custom listener params in config but
// still get our certs delivered dynamically and intentions enforced without
// coming up with some complicated templating/merging solution.
func (s *Server) injectConnectTLSOnFilterChains(_ connectionInfo, cfgSnap *proxycfg.ConfigSnapshot, listener *envoy_listener_v3.Listener) error {
func (s *ResourceGenerator) injectConnectTLSOnFilterChains(cfgSnap *proxycfg.ConfigSnapshot, listener *envoy_listener_v3.Listener) error {
for idx := range listener.FilterChains {
tlsContext := &envoy_tls_v3.DownstreamTlsContext{
CommonTlsContext: makeCommonTLSContextFromLeaf(cfgSnap, cfgSnap.Leaf()),
@ -694,7 +692,7 @@ func (s *Server) injectConnectTLSOnFilterChains(_ connectionInfo, cfgSnap *proxy
return nil
}
func (s *Server) makeInboundListener(cInfo connectionInfo, cfgSnap *proxycfg.ConfigSnapshot, name string) (proto.Message, error) {
func (s *ResourceGenerator) makeInboundListener(cfgSnap *proxycfg.ConfigSnapshot, name string) (proto.Message, error) {
var l *envoy_listener_v3.Listener
var err error
@ -739,7 +737,7 @@ func (s *Server) makeInboundListener(cInfo connectionInfo, cfgSnap *proxycfg.Con
}
}
err := s.finalizePublicListenerFromConfig(l, cInfo, cfgSnap, useHTTPFilter)
err := s.finalizePublicListenerFromConfig(l, cfgSnap, useHTTPFilter)
if err != nil {
return nil, fmt.Errorf("failed to attach Consul filters and TLS context to custom public listener: %v", err)
}
@ -793,7 +791,7 @@ func (s *Server) makeInboundListener(cInfo connectionInfo, cfgSnap *proxycfg.Con
},
}
err = s.finalizePublicListenerFromConfig(l, cInfo, cfgSnap, useHTTPFilter)
err = s.finalizePublicListenerFromConfig(l, cfgSnap, useHTTPFilter)
if err != nil {
return nil, fmt.Errorf("failed to attach Consul filters and TLS context to custom public listener: %v", err)
}
@ -803,23 +801,22 @@ func (s *Server) makeInboundListener(cInfo connectionInfo, cfgSnap *proxycfg.Con
// finalizePublicListenerFromConfig is used for best-effort injection of Consul filter-chains onto listeners.
// This include L4 authorization filters and TLS context.
func (s *Server) finalizePublicListenerFromConfig(l *envoy_listener_v3.Listener,
cInfo connectionInfo, cfgSnap *proxycfg.ConfigSnapshot, useHTTPFilter bool) error {
func (s *ResourceGenerator) finalizePublicListenerFromConfig(l *envoy_listener_v3.Listener, cfgSnap *proxycfg.ConfigSnapshot, useHTTPFilter bool) error {
if !useHTTPFilter {
// Best-effort injection of L4 intentions
if err := s.injectConnectFilters(cInfo, cfgSnap, l); err != nil {
if err := s.injectConnectFilters(cfgSnap, l); err != nil {
return nil
}
}
// Always apply TLS certificates
if err := s.injectConnectTLSOnFilterChains(cInfo, cfgSnap, l); err != nil {
if err := s.injectConnectTLSOnFilterChains(cfgSnap, l); err != nil {
return nil
}
return nil
}
func (s *Server) makeExposedCheckListener(cfgSnap *proxycfg.ConfigSnapshot, cluster string, path structs.ExposePath) (proto.Message, error) {
func (s *ResourceGenerator) makeExposedCheckListener(cfgSnap *proxycfg.ConfigSnapshot, cluster string, path structs.ExposePath) (proto.Message, error) {
cfg, err := ParseProxyConfig(cfgSnap.Proxy.Config)
if err != nil {
// Don't hard fail on a config typo, just warn. The parse func returns
@ -902,8 +899,7 @@ func (s *Server) makeExposedCheckListener(cfgSnap *proxycfg.ConfigSnapshot, clus
return l, err
}
func (s *Server) makeTerminatingGatewayListener(
cInfo connectionInfo,
func (s *ResourceGenerator) makeTerminatingGatewayListener(
cfgSnap *proxycfg.ConfigSnapshot,
name, addr string,
port int,
@ -931,7 +927,7 @@ func (s *Server) makeTerminatingGatewayListener(
if err != nil {
// Don't hard fail on a config typo, just warn. The parse func returns
// default config if there is an error so it's safe to continue.
s.Logger.Named(logging.TerminatingGateway).Warn(
s.Logger.Warn(
"failed to parse Connect.Proxy.Config for linked service",
"service", svc.String(),
"error", err,
@ -939,7 +935,6 @@ func (s *Server) makeTerminatingGatewayListener(
}
clusterChain, err := s.makeFilterChainTerminatingGateway(
cInfo,
cfgSnap,
name,
clusterName,
@ -959,7 +954,6 @@ func (s *Server) makeTerminatingGatewayListener(
subsetClusterName := connect.ServiceSNI(svc.Name, subsetName, svc.NamespaceOrDefault(), cfgSnap.Datacenter, cfgSnap.Roots.TrustDomain)
subsetClusterChain, err := s.makeFilterChainTerminatingGateway(
cInfo,
cfgSnap,
name,
subsetClusterName,
@ -1002,8 +996,7 @@ func (s *Server) makeTerminatingGatewayListener(
return l, nil
}
func (s *Server) makeFilterChainTerminatingGateway(
_ connectionInfo,
func (s *ResourceGenerator) makeFilterChainTerminatingGateway(
cfgSnap *proxycfg.ConfigSnapshot,
listener, cluster string,
service structs.ServiceName,
@ -1076,7 +1069,7 @@ func (s *Server) makeFilterChainTerminatingGateway(
return filterChain, nil
}
func (s *Server) makeMeshGatewayListener(name, addr string, port int, cfgSnap *proxycfg.ConfigSnapshot) (*envoy_listener_v3.Listener, error) {
func (s *ResourceGenerator) makeMeshGatewayListener(name, addr string, port int, cfgSnap *proxycfg.ConfigSnapshot) (*envoy_listener_v3.Listener, error) {
tlsInspector, err := makeTLSInspectorListenerFilter()
if err != nil {
return nil, err
@ -1178,7 +1171,7 @@ func (s *Server) makeMeshGatewayListener(name, addr string, port int, cfgSnap *p
return l, nil
}
func (s *Server) makeUpstreamFilterChainForDiscoveryChain(
func (s *ResourceGenerator) makeUpstreamFilterChainForDiscoveryChain(
id string,
overrideCluster string,
protocol string,
@ -1281,7 +1274,7 @@ func (s *Server) makeUpstreamFilterChainForDiscoveryChain(
}
// TODO(freddy) Replace in favor of new function above. Currently in use for ingress gateways.
func (s *Server) makeUpstreamListenerForDiscoveryChain(
func (s *ResourceGenerator) makeUpstreamListenerForDiscoveryChain(
u *structs.Upstream,
address string,
chain *structs.CompiledDiscoveryChain,
@ -1294,7 +1287,7 @@ func (s *Server) makeUpstreamListenerForDiscoveryChain(
upstreamID := u.Identifier()
l := makeListener(upstreamID, address, u.LocalBindPort, envoy_core_v3.TrafficDirection_OUTBOUND)
cfg := getAndModifyUpstreamConfigForListener(s.Logger, upstreamID, u, chain)
cfg := s.getAndModifyUpstreamConfigForListener(upstreamID, u, chain)
if cfg.EnvoyListenerJSON != "" {
return makeListenerFromUserConfig(cfg.EnvoyListenerJSON)
}
@ -1379,7 +1372,7 @@ func (s *Server) makeUpstreamListenerForDiscoveryChain(
return l, nil
}
func getAndModifyUpstreamConfigForListener(logger hclog.Logger, id string, u *structs.Upstream, chain *structs.CompiledDiscoveryChain) structs.UpstreamConfig {
func (s *ResourceGenerator) getAndModifyUpstreamConfigForListener(id string, u *structs.Upstream, chain *structs.CompiledDiscoveryChain) structs.UpstreamConfig {
var (
cfg structs.UpstreamConfig
err error
@ -1394,7 +1387,7 @@ func getAndModifyUpstreamConfigForListener(logger hclog.Logger, id string, u *st
if err != nil {
// Don't hard fail on a config typo, just warn. The parse func returns
// default config if there is an error so it's safe to continue.
logger.Warn("failed to parse", "upstream", id, "error", err)
s.Logger.Warn("failed to parse", "upstream", id, "error", err)
}
} else {
// Use NoDefaults here so that we can set the protocol to the chain
@ -1403,11 +1396,11 @@ func getAndModifyUpstreamConfigForListener(logger hclog.Logger, id string, u *st
if err != nil {
// Don't hard fail on a config typo, just warn. The parse func returns
// default config if there is an error so it's safe to continue.
logger.Warn("failed to parse", "upstream", id, "error", err)
s.Logger.Warn("failed to parse", "upstream", id, "error", err)
}
if cfg.EnvoyListenerJSON != "" {
logger.Warn("ignoring escape hatch setting because already configured for",
s.Logger.Warn("ignoring escape hatch setting because already configured for",
"discovery chain", chain.ServiceName, "upstream", id, "config", "envoy_listener_json")
// Remove from config struct so we don't use it later on

View File

@ -9,6 +9,7 @@ import (
"time"
envoy_listener_v3 "github.com/envoyproxy/go-control-plane/envoy/config/listener/v3"
testinf "github.com/mitchellh/go-testing-interface"
"github.com/stretchr/testify/require"
@ -34,7 +35,7 @@ func TestListenersFromSnapshot(t *testing.T) {
// test input.
setup func(snap *proxycfg.ConfigSnapshot)
overrideGoldenName string
serverSetup func(*Server)
generatorSetup func(*ResourceGenerator)
}{
{
name: "defaults",
@ -265,7 +266,7 @@ func TestListenersFromSnapshot(t *testing.T) {
Checks: true,
}
},
serverSetup: func(s *Server) {
generatorSetup: func(s *ResourceGenerator) {
s.CfgFetcher = configFetcherFunc(func() string {
return "192.0.2.1"
})
@ -572,16 +573,13 @@ func TestListenersFromSnapshot(t *testing.T) {
}
// Need server just for logger dependency
s := Server{Logger: testutil.Logger(t)}
if tt.serverSetup != nil {
tt.serverSetup(&s)
g := newResourceGenerator(testutil.Logger(t), nil, nil, false)
g.ProxyFeatures = sf
if tt.generatorSetup != nil {
tt.generatorSetup(g)
}
cInfo := connectionInfo{
Token: "my-token",
ProxyFeatures: sf,
}
listeners, err := s.listenersFromSnapshot(cInfo, snap)
listeners, err := g.listenersFromSnapshot(snap)
require.NoError(t, err)
// The order of listeners returned via LDS isn't relevant, so it's safe
@ -625,126 +623,6 @@ func TestListenersFromSnapshot(t *testing.T) {
}
}
func expectListenerJSONResources(snap *proxycfg.ConfigSnapshot) map[string]string {
return map[string]string{
"public_listener": `{
"@type": "type.googleapis.com/envoy.config.listener.v3.Listener",
"name": "public_listener:0.0.0.0:9999",
"address": {
"socketAddress": {
"address": "0.0.0.0",
"portValue": 9999
}
},
"trafficDirection": "INBOUND",
"filterChains": [
{
"transportSocket": ` + expectedPublicTransportSocketJSON(snap) + `,
"filters": [
{
"name": "envoy.filters.network.rbac",
"typedConfig": {
"@type": "type.googleapis.com/envoy.extensions.filters.network.rbac.v3.RBAC",
"rules": {
},
"statPrefix": "connect_authz"
}
},
{
"name": "envoy.filters.network.tcp_proxy",
"typedConfig": {
"@type": "type.googleapis.com/envoy.extensions.filters.network.tcp_proxy.v3.TcpProxy",
"cluster": "local_app",
"statPrefix": "public_listener"
}
}
]
}
]
}`,
"db": `{
"@type": "type.googleapis.com/envoy.config.listener.v3.Listener",
"name": "db:127.0.0.1:9191",
"address": {
"socketAddress": {
"address": "127.0.0.1",
"portValue": 9191
}
},
"trafficDirection": "OUTBOUND",
"filterChains": [
{
"filters": [
{
"name": "envoy.filters.network.tcp_proxy",
"typedConfig": {
"@type": "type.googleapis.com/envoy.extensions.filters.network.tcp_proxy.v3.TcpProxy",
"cluster": "db.default.dc1.internal.11111111-2222-3333-4444-555555555555.consul",
"statPrefix": "upstream.db.default.dc1"
}
}
]
}
]
}`,
"prepared_query:geo-cache": `{
"@type": "type.googleapis.com/envoy.config.listener.v3.Listener",
"name": "prepared_query:geo-cache:127.10.10.10:8181",
"address": {
"socketAddress": {
"address": "127.10.10.10",
"portValue": 8181
}
},
"trafficDirection": "OUTBOUND",
"filterChains": [
{
"filters": [
{
"name": "envoy.filters.network.tcp_proxy",
"typedConfig": {
"@type": "type.googleapis.com/envoy.extensions.filters.network.tcp_proxy.v3.TcpProxy",
"cluster": "geo-cache.default.dc1.query.11111111-2222-3333-4444-555555555555.consul",
"statPrefix": "upstream.prepared_query_geo-cache"
}
}
]
}
]
}`,
}
}
func expectListenerJSONFromResources(snap *proxycfg.ConfigSnapshot, v, n uint64, resourcesJSON map[string]string) string {
resJSON := ""
// Sort resources into specific order because that matters in JSONEq
// comparison later.
keyOrder := []string{"public_listener"}
for _, u := range snap.Proxy.Upstreams {
keyOrder = append(keyOrder, u.Identifier())
}
for _, k := range keyOrder {
j, ok := resourcesJSON[k]
if !ok {
continue
}
if resJSON != "" {
resJSON += ",\n"
}
resJSON += j
}
return `{
"versionInfo": "` + hexString(v) + `",
"resources": [` + resJSON + `],
"typeUrl": "type.googleapis.com/envoy.config.listener.v3.Listener",
"nonce": "` + hexString(n) + `"
}`
}
func expectListenerJSON(snap *proxycfg.ConfigSnapshot, v, n uint64) string {
return expectListenerJSONFromResources(snap, v, n, expectListenerJSONResources(snap))
}
type customListenerJSONOptions struct {
Name string
TLSContext string

View File

@ -0,0 +1,52 @@
package xds
import (
envoy_discovery_v3 "github.com/envoyproxy/go-control-plane/envoy/service/discovery/v3"
"github.com/golang/protobuf/jsonpb"
"github.com/golang/protobuf/proto"
"github.com/mitchellh/copystructure"
)
func (s *ResourceGenerator) logTraceRequest(msg string, pb proto.Message) {
s.logTraceProto(msg, pb, false)
}
func (s *ResourceGenerator) logTraceResponse(msg string, pb proto.Message) {
s.logTraceProto(msg, pb, true)
}
func (s *ResourceGenerator) logTraceProto(msg string, pb proto.Message, response bool) {
if !s.Logger.IsTrace() {
return
}
dir := "request"
if response {
dir = "response"
}
// Duplicate the request so we can scrub the huge Node field for logging.
// If the cloning fails, then log anyway but don't scrub the node field.
if dup, err := copystructure.Copy(pb); err == nil {
pb = dup.(proto.Message)
// strip the node field
switch x := pb.(type) {
case *envoy_discovery_v3.DiscoveryRequest:
x.Node = nil
case *envoy_discovery_v3.DeltaDiscoveryRequest:
x.Node = nil
}
}
m := jsonpb.Marshaler{
Indent: " ",
}
out, err := m.MarshalToString(pb)
if err != nil {
out = "<ERROR: " + err.Error() + ">"
}
s.Logger.Trace(msg, "direction", dir, "protobuf", out)
}

62
agent/xds/resources.go Normal file
View File

@ -0,0 +1,62 @@
package xds
import (
"fmt"
"github.com/golang/protobuf/proto"
"github.com/hashicorp/go-hclog"
"github.com/hashicorp/consul/agent/proxycfg"
)
// ResourceGenerator is associated with a single gRPC stream and creates xDS
// resources for a single client.
type ResourceGenerator struct {
Logger hclog.Logger
CheckFetcher HTTPCheckFetcher
CfgFetcher ConfigFetcher
IncrementalXDS bool
ProxyFeatures supportedProxyFeatures
}
func newResourceGenerator(
logger hclog.Logger,
checkFetcher HTTPCheckFetcher,
cfgFetcher ConfigFetcher,
incrementalXDS bool,
) *ResourceGenerator {
return &ResourceGenerator{
Logger: logger,
CheckFetcher: checkFetcher,
CfgFetcher: cfgFetcher,
IncrementalXDS: incrementalXDS,
}
}
func (g *ResourceGenerator) allResourcesFromSnapshot(cfgSnap *proxycfg.ConfigSnapshot) (map[string][]proto.Message, error) {
all := make(map[string][]proto.Message)
for _, typeUrl := range []string{ListenerType, RouteType, ClusterType, EndpointType} {
res, err := g.resourcesFromSnapshot(typeUrl, cfgSnap)
if err != nil {
return nil, fmt.Errorf("failed to generate xDS resources for %q: %v", typeUrl, err)
}
all[typeUrl] = res
}
return all, nil
}
func (g *ResourceGenerator) resourcesFromSnapshot(typeUrl string, cfgSnap *proxycfg.ConfigSnapshot) ([]proto.Message, error) {
switch typeUrl {
case ListenerType:
return g.listenersFromSnapshot(cfgSnap)
case RouteType:
return g.routesFromSnapshot(cfgSnap)
case ClusterType:
return g.clustersFromSnapshot(cfgSnap)
case EndpointType:
return g.endpointsFromSnapshot(cfgSnap)
default:
return nil, fmt.Errorf("unknown typeUrl: %s", typeUrl)
}
}

View File

@ -16,23 +16,24 @@ import (
"github.com/hashicorp/consul/agent/connect"
"github.com/hashicorp/consul/agent/proxycfg"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/logging"
)
// routesFromSnapshot returns the xDS API representation of the "routes" in the
// snapshot.
func (s *Server) routesFromSnapshot(cInfo connectionInfo, cfgSnap *proxycfg.ConfigSnapshot) ([]proto.Message, error) {
func (s *ResourceGenerator) routesFromSnapshot(cfgSnap *proxycfg.ConfigSnapshot) ([]proto.Message, error) {
if cfgSnap == nil {
return nil, errors.New("nil config given")
}
switch cfgSnap.Kind {
case structs.ServiceKindConnectProxy:
return routesForConnectProxy(cInfo, cfgSnap.ConnectProxy.DiscoveryChain)
return s.routesForConnectProxy(cfgSnap.ConnectProxy.DiscoveryChain)
case structs.ServiceKindIngressGateway:
return routesForIngressGateway(cInfo, cfgSnap.IngressGateway.Upstreams, cfgSnap.IngressGateway.DiscoveryChain)
return s.routesForIngressGateway(cfgSnap.IngressGateway.Upstreams, cfgSnap.IngressGateway.DiscoveryChain)
case structs.ServiceKindTerminatingGateway:
return s.routesFromSnapshotTerminatingGateway(cInfo, cfgSnap)
return s.routesFromSnapshotTerminatingGateway(cfgSnap)
case structs.ServiceKindMeshGateway:
return nil, nil // mesh gateways will never have routes
default:
return nil, fmt.Errorf("Invalid service kind: %v", cfgSnap.Kind)
}
@ -40,18 +41,14 @@ func (s *Server) routesFromSnapshot(cInfo connectionInfo, cfgSnap *proxycfg.Conf
// routesFromSnapshotConnectProxy returns the xDS API representation of the
// "routes" in the snapshot.
func routesForConnectProxy(
cInfo connectionInfo,
chains map[string]*structs.CompiledDiscoveryChain,
) ([]proto.Message, error) {
func (s *ResourceGenerator) routesForConnectProxy(chains map[string]*structs.CompiledDiscoveryChain) ([]proto.Message, error) {
var resources []proto.Message
for id, chain := range chains {
if chain.IsDefault() {
continue
}
virtualHost, err := makeUpstreamRouteForDiscoveryChain(cInfo, id, chain, []string{"*"})
virtualHost, err := makeUpstreamRouteForDiscoveryChain(id, chain, []string{"*"})
if err != nil {
return nil, err
}
@ -73,11 +70,10 @@ func routesForConnectProxy(
// routesFromSnapshotTerminatingGateway returns the xDS API representation of the "routes" in the snapshot.
// For any HTTP service we will return a default route.
func (s *Server) routesFromSnapshotTerminatingGateway(_ connectionInfo, cfgSnap *proxycfg.ConfigSnapshot) ([]proto.Message, error) {
func (s *ResourceGenerator) routesFromSnapshotTerminatingGateway(cfgSnap *proxycfg.ConfigSnapshot) ([]proto.Message, error) {
if cfgSnap == nil {
return nil, errors.New("nil config given")
}
logger := s.Logger.Named(logging.TerminatingGateway)
var resources []proto.Message
for _, svc := range cfgSnap.TerminatingGateway.ValidServices() {
@ -106,7 +102,7 @@ func (s *Server) routesFromSnapshotTerminatingGateway(_ connectionInfo, cfgSnap
}
route, err := makeNamedDefaultRouteWithLB(clusterName, lb, true)
if err != nil {
logger.Error("failed to make route", "cluster", clusterName, "error", err)
s.Logger.Error("failed to make route", "cluster", clusterName, "error", err)
continue
}
resources = append(resources, route)
@ -116,7 +112,7 @@ func (s *Server) routesFromSnapshotTerminatingGateway(_ connectionInfo, cfgSnap
clusterName = connect.ServiceSNI(svc.Name, name, svc.NamespaceOrDefault(), cfgSnap.Datacenter, cfgSnap.Roots.TrustDomain)
route, err := makeNamedDefaultRouteWithLB(clusterName, lb, true)
if err != nil {
logger.Error("failed to make route", "cluster", clusterName, "error", err)
s.Logger.Error("failed to make route", "cluster", clusterName, "error", err)
continue
}
resources = append(resources, route)
@ -163,8 +159,7 @@ func makeNamedDefaultRouteWithLB(clusterName string, lb *structs.LoadBalancer, a
// routesForIngressGateway returns the xDS API representation of the
// "routes" in the snapshot.
func routesForIngressGateway(
cInfo connectionInfo,
func (s *ResourceGenerator) routesForIngressGateway(
upstreams map[proxycfg.IngressListenerKey]structs.Upstreams,
chains map[string]*structs.CompiledDiscoveryChain,
) ([]proto.Message, error) {
@ -191,7 +186,7 @@ func routesForIngressGateway(
}
domains := generateUpstreamIngressDomains(listenerKey, u)
virtualHost, err := makeUpstreamRouteForDiscoveryChain(cInfo, upstreamID, chain, domains)
virtualHost, err := makeUpstreamRouteForDiscoveryChain(upstreamID, chain, domains)
if err != nil {
return nil, err
}
@ -251,7 +246,6 @@ func generateUpstreamIngressDomains(listenerKey proxycfg.IngressListenerKey, u s
}
func makeUpstreamRouteForDiscoveryChain(
cInfo connectionInfo,
routeName string,
chain *structs.CompiledDiscoveryChain,
serviceDomains []string,
@ -268,7 +262,7 @@ func makeUpstreamRouteForDiscoveryChain(
routes = make([]*envoy_route_v3.Route, 0, len(startNode.Routes))
for _, discoveryRoute := range startNode.Routes {
routeMatch := makeRouteMatchForDiscoveryRoute(cInfo, discoveryRoute)
routeMatch := makeRouteMatchForDiscoveryRoute(discoveryRoute)
var (
routeAction *envoy_route_v3.Route_Route
@ -397,7 +391,7 @@ func makeUpstreamRouteForDiscoveryChain(
return host, nil
}
func makeRouteMatchForDiscoveryRoute(_ connectionInfo, discoveryRoute *structs.DiscoveryRoute) *envoy_route_v3.RouteMatch {
func makeRouteMatchForDiscoveryRoute(discoveryRoute *structs.DiscoveryRoute) *envoy_route_v3.RouteMatch {
match := discoveryRoute.Definition.Match
if match == nil || match.IsEmpty() {
return makeDefaultRouteMatch()

View File

@ -256,12 +256,10 @@ func TestRoutesFromSnapshot(t *testing.T) {
tt.setup(snap)
}
s := Server{Logger: testutil.Logger(t)}
cInfo := connectionInfo{
Token: "my-token",
ProxyFeatures: sf,
}
routes, err := s.routesFromSnapshot(cInfo, snap)
g := newResourceGenerator(testutil.Logger(t), nil, nil, false)
g.ProxyFeatures = sf
routes, err := g.routesFromSnapshot(snap)
require.NoError(t, err)
sort.Slice(routes, func(i, j int) bool {

View File

@ -7,13 +7,10 @@ import (
"sync/atomic"
"time"
"github.com/hashicorp/consul/logging"
envoy_config_core_v3 "github.com/envoyproxy/go-control-plane/envoy/config/core/v3"
envoy_discovery_v2 "github.com/envoyproxy/go-control-plane/envoy/service/discovery/v2"
envoy_discovery_v3 "github.com/envoyproxy/go-control-plane/envoy/service/discovery/v3"
"github.com/golang/protobuf/proto"
"github.com/hashicorp/go-hclog"
"google.golang.org/grpc"
"google.golang.org/grpc/codes"
@ -24,6 +21,7 @@ import (
"github.com/hashicorp/consul/acl"
"github.com/hashicorp/consul/agent/proxycfg"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/logging"
"github.com/hashicorp/consul/tlsutil"
)
@ -39,15 +37,19 @@ const (
// EndpointType is the TypeURL for Endpoint discovery responses.
EndpointType = apiTypePrefix + "envoy.config.endpoint.v3.ClusterLoadAssignment"
EndpointType_v2 = apiTypePrefix + "envoy.api.v2.ClusterLoadAssignment"
// ClusterType is the TypeURL for Cluster discovery responses.
ClusterType = apiTypePrefix + "envoy.config.cluster.v3.Cluster"
ClusterType_v2 = apiTypePrefix + "envoy.api.v2.Cluster"
// RouteType is the TypeURL for Route discovery responses.
RouteType = apiTypePrefix + "envoy.config.route.v3.RouteConfiguration"
RouteType_v2 = apiTypePrefix + "envoy.api.v2.RouteConfiguration"
// ListenerType is the TypeURL for Listener discovery responses.
ListenerType = apiTypePrefix + "envoy.config.listener.v3.Listener"
ListenerType_v2 = apiTypePrefix + "envoy.api.v2.Listener"
// PublicListenerName is the name we give the public listener in Envoy config.
PublicListenerName = "public_listener"
@ -129,21 +131,50 @@ type Server struct {
Logger hclog.Logger
CfgMgr ConfigManager
ResolveToken ACLResolverFunc
CheckFetcher HTTPCheckFetcher
CfgFetcher ConfigFetcher
// 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
CheckFetcher HTTPCheckFetcher
CfgFetcher ConfigFetcher
DisableV2Protocol bool
}
func NewServer(
logger hclog.Logger,
cfgMgr ConfigManager,
resolveToken ACLResolverFunc,
checkFetcher HTTPCheckFetcher,
cfgFetcher ConfigFetcher,
) *Server {
return &Server{
Logger: logger,
CfgMgr: cfgMgr,
ResolveToken: resolveToken,
CheckFetcher: checkFetcher,
CfgFetcher: cfgFetcher,
AuthCheckFrequency: DefaultAuthCheckFrequency,
}
}
// StreamAggregatedResources implements
// envoy_discovery_v3.AggregatedDiscoveryServiceServer. This is the ADS endpoint which is
// the only xDS API we directly support for now.
//
// Deprecated: use DeltaAggregatedResources instead
func (s *Server) StreamAggregatedResources(stream ADSStream) error {
return errors.New("not implemented")
}
// Deprecated: remove when xDS v2 is no longer supported
func (s *Server) streamAggregatedResources(stream ADSStream) error {
// Note: despite dealing entirely in v3 protobufs, this function is
// exclusively used from the xDS v2 shim RPC handler, so the logging below
// will refer to it as "v2".
// a channel for receiving incoming requests
reqCh := make(chan *envoy_discovery_v3.DiscoveryRequest)
reqStop := int32(0)
@ -163,7 +194,7 @@ func (s *Server) StreamAggregatedResources(stream ADSStream) error {
err := s.process(stream, reqCh)
if err != nil {
s.Logger.Error("Error handling ADS stream", "xdsVersion", "v3", "error", err)
s.Logger.Error("Error handling ADS stream", "xdsVersion", "v2", "error", err)
}
// prevents writing to a closed channel if send failed on blocked recv
@ -178,9 +209,8 @@ const (
stateRunning
)
// Deprecated: remove when xDS v2 is no longer supported
func (s *Server) process(stream ADSStream, reqCh <-chan *envoy_discovery_v3.DiscoveryRequest) error {
logger := s.Logger.Named(logging.XDS)
// xDS requires a unique nonce to correlate response/request pairs
var nonce uint64
@ -196,26 +226,32 @@ func (s *Server) process(stream ADSStream, reqCh <-chan *envoy_discovery_v3.Disc
cfgSnap *proxycfg.ConfigSnapshot
req *envoy_discovery_v3.DiscoveryRequest
node *envoy_config_core_v3.Node
proxyFeatures supportedProxyFeatures
ok bool
stateCh <-chan *proxycfg.ConfigSnapshot
watchCancel func()
proxyID structs.ServiceID
)
generator := newResourceGenerator(
s.Logger.Named(logging.XDS).With("xdsVersion", "v2"),
s.CheckFetcher,
s.CfgFetcher,
false,
)
// need to run a small state machine to get through initial authentication.
var state = stateInit
// Configure handlers for each type of request
handlers := map[string]*xDSType{
EndpointType: {
generator: generator,
typeURL: EndpointType,
resources: s.endpointsFromSnapshot,
stream: stream,
},
ClusterType: {
generator: generator,
typeURL: ClusterType,
resources: s.clustersFromSnapshot,
stream: stream,
allowEmptyFn: func(cfgSnap *proxycfg.ConfigSnapshot) bool {
// Mesh, Ingress, and Terminating gateways are allowed to inform CDS of
@ -226,16 +262,16 @@ func (s *Server) process(stream ADSStream, reqCh <-chan *envoy_discovery_v3.Disc
},
},
RouteType: {
generator: generator,
typeURL: RouteType,
resources: s.routesFromSnapshot,
stream: stream,
allowEmptyFn: func(cfgSnap *proxycfg.ConfigSnapshot) bool {
return cfgSnap.Kind == structs.ServiceKindIngressGateway
},
},
ListenerType: {
generator: generator,
typeURL: ListenerType,
resources: s.listenersFromSnapshot,
stream: stream,
allowEmptyFn: func(cfgSnap *proxycfg.ConfigSnapshot) bool {
return cfgSnap.Kind == structs.ServiceKindIngressGateway
@ -249,38 +285,7 @@ func (s *Server) process(stream ADSStream, reqCh <-chan *envoy_discovery_v3.Disc
}
checkStreamACLs := func(cfgSnap *proxycfg.ConfigSnapshot) error {
if cfgSnap == nil {
return status.Errorf(codes.Unauthenticated, "unauthenticated: no config snapshot")
}
rule, err := s.ResolveToken(tokenFromContext(stream.Context()))
if acl.IsErrNotFound(err) {
return status.Errorf(codes.Unauthenticated, "unauthenticated: %v", err)
} else if acl.IsErrPermissionDenied(err) {
return status.Errorf(codes.PermissionDenied, "permission denied: %v", err)
} else if err != nil {
return err
}
var authzContext acl.AuthorizerContext
switch cfgSnap.Kind {
case structs.ServiceKindConnectProxy:
cfgSnap.ProxyID.EnterpriseMeta.FillAuthzContext(&authzContext)
if rule != nil && rule.ServiceWrite(cfgSnap.Proxy.DestinationServiceName, &authzContext) != acl.Allow {
return status.Errorf(codes.PermissionDenied, "permission denied")
}
case structs.ServiceKindMeshGateway, structs.ServiceKindTerminatingGateway, structs.ServiceKindIngressGateway:
cfgSnap.ProxyID.EnterpriseMeta.FillAuthzContext(&authzContext)
if rule != nil && rule.ServiceWrite(cfgSnap.Service, &authzContext) != acl.Allow {
return status.Errorf(codes.PermissionDenied, "permission denied")
}
default:
return status.Errorf(codes.Internal, "Invalid service kind")
}
// Authed OK!
return nil
return s.checkStreamACLs(stream.Context(), cfgSnap)
}
for {
@ -300,6 +305,9 @@ func (s *Server) process(stream ADSStream, reqCh <-chan *envoy_discovery_v3.Disc
// there's no point in blocking on that.
return nil
}
generator.logTraceRequest("SOTW xDS v2", req)
if req.TypeUrl == "" {
return status.Errorf(codes.InvalidArgument, "type URL is required for ADS")
}
@ -307,14 +315,14 @@ func (s *Server) process(stream ADSStream, reqCh <-chan *envoy_discovery_v3.Disc
if node == nil && req.Node != nil {
node = req.Node
var err error
proxyFeatures, err = determineSupportedProxyFeatures(req.Node)
generator.ProxyFeatures, err = determineSupportedProxyFeatures(req.Node)
if err != nil {
return status.Errorf(codes.InvalidArgument, err.Error())
}
}
if handler, ok := handlers[req.TypeUrl]; ok {
handler.Recv(req, node, proxyFeatures)
handler.Recv(req, node)
}
case cfgSnap = <-stateCh:
// We got a new config, update the version counter
@ -341,7 +349,7 @@ func (s *Server) process(stream ADSStream, reqCh <-chan *envoy_discovery_v3.Disc
// state machine.
defer watchCancel()
logger.Trace("watching proxy, pending initial proxycfg snapshot",
generator.Logger.Trace("watching proxy, pending initial proxycfg snapshot",
"service_id", proxyID.String())
// Now wait for the config so we can check ACL
@ -355,7 +363,18 @@ func (s *Server) process(stream ADSStream, reqCh <-chan *envoy_discovery_v3.Disc
// Got config, try to authenticate next.
state = stateRunning
logger.Trace("Got initial config snapshot",
// Upgrade the logger based on Kind.
switch cfgSnap.Kind {
case structs.ServiceKindConnectProxy:
case structs.ServiceKindTerminatingGateway:
generator.Logger = generator.Logger.Named(logging.TerminatingGateway)
case structs.ServiceKindMeshGateway:
generator.Logger = generator.Logger.Named(logging.MeshGateway)
case structs.ServiceKindIngressGateway:
generator.Logger = generator.Logger.Named(logging.IngressGateway)
}
generator.Logger.Trace("Got initial config snapshot",
"service_id", cfgSnap.ProxyID.String())
// Lets actually process the config we just got or we'll mis responding
@ -369,7 +388,7 @@ func (s *Server) process(stream ADSStream, reqCh <-chan *envoy_discovery_v3.Disc
// timer is first started.
extendAuthTimer()
logger.Trace("Invoking all xDS resource handlers and sending new data if there is any",
generator.Logger.Trace("Invoking all xDS resource handlers and sending new data if there is any",
"service_id", cfgSnap.ProxyID.String())
// See if any handlers need to have the current (possibly new) config
@ -385,19 +404,22 @@ func (s *Server) process(stream ADSStream, reqCh <-chan *envoy_discovery_v3.Disc
for _, typeURL := range []string{ClusterType, EndpointType, RouteType, ListenerType} {
handler := handlers[typeURL]
if err := handler.SendIfNew(cfgSnap, configVersion, &nonce); err != nil {
return err
return status.Errorf(codes.Unavailable,
"failed to send reply for type %q: %v",
typeURL, err)
}
}
}
}
}
// Deprecated: remove when xDS v2 is no longer supported
type xDSType struct {
generator *ResourceGenerator
typeURL string
stream ADSStream
req *envoy_discovery_v3.DiscoveryRequest
node *envoy_config_core_v3.Node
proxyFeatures supportedProxyFeatures
lastNonce string
// lastVersion is the version that was last sent to the proxy. It is needed
// because we don't want to send the same version more than once.
@ -407,21 +429,13 @@ type xDSType struct {
// last version we sent with a Nack then req.VersionInfo will be the older
// version it's hanging on to.
lastVersion uint64
resources func(cInfo connectionInfo, cfgSnap *proxycfg.ConfigSnapshot) ([]proto.Message, error)
allowEmptyFn func(cfgSnap *proxycfg.ConfigSnapshot) bool
}
// connectionInfo represents details specific to this connection
type connectionInfo struct {
Token string
ProxyFeatures supportedProxyFeatures
}
func (t *xDSType) Recv(req *envoy_discovery_v3.DiscoveryRequest, node *envoy_config_core_v3.Node, proxyFeatures supportedProxyFeatures) {
func (t *xDSType) Recv(req *envoy_discovery_v3.DiscoveryRequest, node *envoy_config_core_v3.Node) {
if t.lastNonce == "" || t.lastNonce == req.GetResponseNonce() {
t.req = req
t.node = node
t.proxyFeatures = proxyFeatures
}
}
@ -434,11 +448,7 @@ func (t *xDSType) SendIfNew(cfgSnap *proxycfg.ConfigSnapshot, version uint64, no
return nil
}
cInfo := connectionInfo{
Token: tokenFromContext(t.stream.Context()),
ProxyFeatures: t.proxyFeatures,
}
resources, err := t.resources(cInfo, cfgSnap)
resources, err := t.generator.resourcesFromSnapshot(t.typeURL, cfgSnap)
if err != nil {
return err
}
@ -468,6 +478,8 @@ func (t *xDSType) SendIfNew(cfgSnap *proxycfg.ConfigSnapshot, version uint64, no
return err
}
t.generator.logTraceResponse("SOTW xDS v2", resp)
err = t.stream.Send(resp)
if err != nil {
return err
@ -489,11 +501,6 @@ func tokenFromContext(ctx context.Context) string {
return ""
}
// DeltaAggregatedResources implements envoy_discovery_v3.AggregatedDiscoveryServiceServer
func (s *Server) DeltaAggregatedResources(_ envoy_discovery_v3.AggregatedDiscoveryService_DeltaAggregatedResourcesServer) error {
return errors.New("not implemented")
}
// GRPCServer returns a server instance that can handle xDS requests.
func (s *Server) GRPCServer(tlsConfigurator *tlsutil.Configurator) (*grpc.Server, error) {
opts := []grpc.ServerOption{
@ -514,3 +521,38 @@ func (s *Server) GRPCServer(tlsConfigurator *tlsutil.Configurator) (*grpc.Server
return srv, nil
}
func (s *Server) checkStreamACLs(streamCtx context.Context, cfgSnap *proxycfg.ConfigSnapshot) error {
if cfgSnap == nil {
return status.Errorf(codes.Unauthenticated, "unauthenticated: no config snapshot")
}
rule, err := s.ResolveToken(tokenFromContext(streamCtx))
if acl.IsErrNotFound(err) {
return status.Errorf(codes.Unauthenticated, "unauthenticated: %v", err)
} else if acl.IsErrPermissionDenied(err) {
return status.Errorf(codes.PermissionDenied, "permission denied: %v", err)
} else if err != nil {
return status.Errorf(codes.Internal, "error resolving acl token: %v", err)
}
var authzContext acl.AuthorizerContext
switch cfgSnap.Kind {
case structs.ServiceKindConnectProxy:
cfgSnap.ProxyID.EnterpriseMeta.FillAuthzContext(&authzContext)
if rule != nil && rule.ServiceWrite(cfgSnap.Proxy.DestinationServiceName, &authzContext) != acl.Allow {
return status.Errorf(codes.PermissionDenied, "permission denied")
}
case structs.ServiceKindMeshGateway, structs.ServiceKindTerminatingGateway, structs.ServiceKindIngressGateway:
cfgSnap.ProxyID.EnterpriseMeta.FillAuthzContext(&authzContext)
if rule != nil && rule.ServiceWrite(cfgSnap.Service, &authzContext) != acl.Allow {
return status.Errorf(codes.PermissionDenied, "permission denied")
}
default:
return status.Errorf(codes.Internal, "Invalid service kind")
}
// Authed OK!
return nil
}

View File

@ -1,159 +1,118 @@
package xds
import (
"strings"
"sync"
"sync/atomic"
"testing"
"time"
envoy_discovery_v3 "github.com/envoyproxy/go-control-plane/envoy/service/discovery/v3"
envoy_api_v2 "github.com/envoyproxy/go-control-plane/envoy/api/v2"
"github.com/golang/protobuf/proto"
"github.com/golang/protobuf/ptypes"
"github.com/golang/protobuf/ptypes/any"
"github.com/stretchr/testify/require"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
"github.com/hashicorp/consul/acl"
"github.com/hashicorp/consul/agent/cache"
"github.com/hashicorp/consul/agent/proxycfg"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/sdk/testutil"
)
// testManager is a mock of proxycfg.Manager that's simpler to control for
// testing. It also implements ConnectAuthz to allow control over authorization.
type testManager struct {
sync.Mutex
chans map[structs.ServiceID]chan *proxycfg.ConfigSnapshot
cancels chan structs.ServiceID
authz map[string]connectAuthzResult
}
// NOTE: For these tests, prefer not using xDS protobuf "factory" methods if
// possible to avoid using them to test themselves.
//
// Stick to very straightforward stuff in xds_protocol_helpers_test.go.
type connectAuthzResult struct {
authz bool
reason string
m *cache.ResultMeta
err error
validate func(req *structs.ConnectAuthorizeRequest) error
}
func newTestManager(t *testing.T) *testManager {
return &testManager{
chans: map[structs.ServiceID]chan *proxycfg.ConfigSnapshot{},
cancels: make(chan structs.ServiceID, 10),
authz: make(map[string]connectAuthzResult),
}
}
// RegisterProxy simulates a proxy registration
func (m *testManager) RegisterProxy(t *testing.T, proxyID structs.ServiceID) {
m.Lock()
defer m.Unlock()
m.chans[proxyID] = make(chan *proxycfg.ConfigSnapshot, 1)
}
// Deliver simulates a proxy registration
func (m *testManager) DeliverConfig(t *testing.T, proxyID structs.ServiceID, cfg *proxycfg.ConfigSnapshot) {
t.Helper()
m.Lock()
defer m.Unlock()
select {
case m.chans[proxyID] <- cfg:
case <-time.After(10 * time.Millisecond):
t.Fatalf("took too long to deliver config")
}
}
// Watch implements ConfigManager
func (m *testManager) Watch(proxyID structs.ServiceID) (<-chan *proxycfg.ConfigSnapshot, proxycfg.CancelFunc) {
m.Lock()
defer m.Unlock()
// ch might be nil but then it will just block forever
return m.chans[proxyID], func() {
m.cancels <- proxyID
}
}
// AssertWatchCancelled checks that the most recent call to a Watch cancel func
// was from the specified proxyID and that one is made in a short time. This
// probably won't work if you are running multiple Watches in parallel on
// multiple proxyIDS due to timing/ordering issues but I don't think we need to
// do that.
func (m *testManager) AssertWatchCancelled(t *testing.T, proxyID structs.ServiceID) {
t.Helper()
select {
case got := <-m.cancels:
require.Equal(t, proxyID, got)
case <-time.After(50 * time.Millisecond):
t.Fatalf("timed out waiting for Watch cancel for %s", proxyID)
}
}
func TestServer_StreamAggregatedResources_BasicProtocol(t *testing.T) {
mgr := newTestManager(t)
func TestServer_StreamAggregatedResources_v2_BasicProtocol_TCP(t *testing.T) {
aclResolve := func(id string) (acl.Authorizer, error) {
// Allow all
return acl.RootAuthorizer("manage"), nil
}
envoy := NewTestEnvoy(t, "web-sidecar-proxy", "")
defer envoy.Close()
s := Server{
Logger: testutil.Logger(t),
CfgMgr: mgr,
ResolveToken: aclResolve,
}
scenario := newTestServerScenario(t, aclResolve, "web-sidecar-proxy", "", 0)
mgr, errCh, envoy := scenario.mgr, scenario.errCh, scenario.envoy
sid := structs.NewServiceID("web-sidecar-proxy", nil)
go func() {
err := s.StreamAggregatedResources(envoy.stream)
require.NoError(t, err)
}()
// Register the proxy to create state needed to Watch() on
mgr.RegisterProxy(t, sid)
// Send initial cluster discover
envoy.SendReq(t, ClusterType, 0, 0)
// Send initial cluster discover (empty payload)
envoy.SendReq(t, ClusterType_v2, 0, 0)
// Check no response sent yet
assertChanBlocked(t, envoy.stream.sendCh)
// Deliver a new snapshot
snap := proxycfg.TestConfigSnapshot(t)
snap := newTestSnapshot(t, nil, "")
mgr.DeliverConfig(t, sid, snap)
assertResponseSent(t, envoy.stream.sendCh, expectClustersJSON(snap, 1, 1))
expectClusterResponse := func(v, n uint64) *envoy_api_v2.DiscoveryResponse {
return &envoy_api_v2.DiscoveryResponse{
VersionInfo: hexString(v),
TypeUrl: ClusterType_v2,
Nonce: hexString(n),
Resources: makeTestResources_v2(t,
makeTestCluster_v2(t, snap, "tcp:local_app"),
makeTestCluster_v2(t, snap, "tcp:db"),
makeTestCluster_v2(t, snap, "tcp:geo-cache"),
),
}
}
expectEndpointResponse := func(v, n uint64) *envoy_api_v2.DiscoveryResponse {
return &envoy_api_v2.DiscoveryResponse{
VersionInfo: hexString(v),
TypeUrl: EndpointType_v2,
Nonce: hexString(n),
Resources: makeTestResources_v2(t,
makeTestEndpoints_v2(t, snap, "tcp:db"),
makeTestEndpoints_v2(t, snap, "tcp:geo-cache"),
),
}
}
expectListenerResponse := func(v, n uint64) *envoy_api_v2.DiscoveryResponse {
return &envoy_api_v2.DiscoveryResponse{
VersionInfo: hexString(v),
TypeUrl: ListenerType_v2,
Nonce: hexString(n),
Resources: makeTestResources_v2(t,
makeTestListener_v2(t, snap, "tcp:public_listener"),
makeTestListener_v2(t, snap, "tcp:db"),
makeTestListener_v2(t, snap, "tcp:geo-cache"),
),
}
}
assertResponseSent(t, envoy.stream.sendCh, expectClusterResponse(1, 1))
// Envoy then tries to discover endpoints for those clusters. Technically it
// includes the cluster names in the ResourceNames field but we ignore that
// completely for now so not bothering to simulate that.
envoy.SendReq(t, EndpointType, 0, 0)
envoy.SendReq(t, EndpointType_v2, 0, 0)
// It also (in parallel) issues the next cluster request (which acts as an ACK
// of the version we sent)
envoy.SendReq(t, ClusterType, 1, 1)
envoy.SendReq(t, ClusterType_v2, 1, 1)
// We should get a response immediately since the config is already present in
// the server for endpoints. Note that this should not be racy if the server
// is behaving well since the Cluster send above should be blocked until we
// deliver a new config version.
assertResponseSent(t, envoy.stream.sendCh, expectEndpointsJSON(1, 2))
assertResponseSent(t, envoy.stream.sendCh, expectEndpointResponse(1, 2))
// And no other response yet
assertChanBlocked(t, envoy.stream.sendCh)
// Envoy now sends listener request along with next endpoint one
envoy.SendReq(t, ListenerType, 0, 0)
envoy.SendReq(t, EndpointType, 1, 2)
envoy.SendReq(t, ListenerType_v2, 0, 0)
envoy.SendReq(t, EndpointType_v2, 1, 2)
// And should get a response immediately.
assertResponseSent(t, envoy.stream.sendCh, expectListenerJSON(snap, 1, 3))
assertResponseSent(t, envoy.stream.sendCh, expectListenerResponse(1, 3))
// Now send Route request along with next listener one
envoy.SendReq(t, RouteType, 0, 0)
envoy.SendReq(t, ListenerType, 1, 3)
envoy.SendReq(t, RouteType_v2, 0, 0)
envoy.SendReq(t, ListenerType_v2, 1, 3)
// We don't serve routes yet so this should block with no response
assertChanBlocked(t, envoy.stream.sendCh)
@ -169,14 +128,13 @@ func TestServer_StreamAggregatedResources_BasicProtocol(t *testing.T) {
// All 3 response that have something to return should return with new version
// note that the ordering is not deterministic in general. Trying to make this
// test order-agnostic though is a massive pain since we are comparing
// non-identical JSON strings (so can simply sort by anything) and because we
// test order-agnostic though is a massive pain because we
// don't know the order the nonces will be assigned. For now we rely and
// require our implementation to always deliver updates in a specific order
// which is reasonable anyway to ensure consistency of the config Envoy sees.
assertResponseSent(t, envoy.stream.sendCh, expectClustersJSON(snap, 2, 4))
assertResponseSent(t, envoy.stream.sendCh, expectEndpointsJSON(2, 5))
assertResponseSent(t, envoy.stream.sendCh, expectListenerJSON(snap, 2, 6))
assertResponseSent(t, envoy.stream.sendCh, expectClusterResponse(2, 4))
assertResponseSent(t, envoy.stream.sendCh, expectEndpointResponse(2, 5))
assertResponseSent(t, envoy.stream.sendCh, expectListenerResponse(2, 6))
// Let's pretend that Envoy doesn't like that new listener config. It will ACK
// all the others (same version) but NACK the listener. This is the most
@ -199,9 +157,9 @@ func TestServer_StreamAggregatedResources_BasicProtocol(t *testing.T) {
// In this case we are simulating that Envoy failed to apply the Listener
// response but did apply the other types so all get the new nonces, but
// listener stays on v1.
envoy.SendReq(t, ClusterType, 2, 4)
envoy.SendReq(t, EndpointType, 2, 5)
envoy.SendReq(t, ListenerType, 1, 6) // v1 is a NACK
envoy.SendReq(t, ClusterType_v2, 2, 4)
envoy.SendReq(t, EndpointType_v2, 2, 5)
envoy.SendReq(t, ListenerType_v2, 1, 6)
// Even though we nacked, we should still NOT get then v2 listeners
// redelivered since nothing has changed.
@ -211,177 +169,190 @@ func TestServer_StreamAggregatedResources_BasicProtocol(t *testing.T) {
snap.ConnectProxy.Leaf = proxycfg.TestLeafForCA(t, snap.Roots.Roots[0])
mgr.DeliverConfig(t, sid, snap)
assertResponseSent(t, envoy.stream.sendCh, expectClustersJSON(snap, 3, 7))
assertResponseSent(t, envoy.stream.sendCh, expectEndpointsJSON(3, 8))
assertResponseSent(t, envoy.stream.sendCh, expectListenerJSON(snap, 3, 9))
}
assertResponseSent(t, envoy.stream.sendCh, expectClusterResponse(3, 7))
assertResponseSent(t, envoy.stream.sendCh, expectEndpointResponse(3, 8))
assertResponseSent(t, envoy.stream.sendCh, expectListenerResponse(3, 9))
func expectEndpointsJSON(v, n uint64) string {
return `{
"versionInfo": "` + hexString(v) + `",
"resources": [
{
"@type": "type.googleapis.com/envoy.config.endpoint.v3.ClusterLoadAssignment",
"clusterName": "db.default.dc1.internal.11111111-2222-3333-4444-555555555555.consul",
"endpoints": [
{
"lbEndpoints": [
{
"endpoint": {
"address": {
"socketAddress": {
"address": "10.10.1.1",
"portValue": 8080
}
}
},
"healthStatus": "HEALTHY",
"loadBalancingWeight": 1
},
{
"endpoint": {
"address": {
"socketAddress": {
"address": "10.10.1.2",
"portValue": 8080
}
}
},
"healthStatus": "HEALTHY",
"loadBalancingWeight": 1
}
]
}
]
},
{
"@type": "type.googleapis.com/envoy.config.endpoint.v3.ClusterLoadAssignment",
"clusterName": "geo-cache.default.dc1.query.11111111-2222-3333-4444-555555555555.consul",
"endpoints": [
{
"lbEndpoints": [
{
"endpoint": {
"address": {
"socketAddress": {
"address": "10.10.1.1",
"portValue": 8080
}
}
},
"healthStatus": "HEALTHY",
"loadBalancingWeight": 1
},
{
"endpoint": {
"address": {
"socketAddress": {
"address": "10.10.1.2",
"portValue": 8080
}
}
},
"healthStatus": "HEALTHY",
"loadBalancingWeight": 1
}
]
}
]
}
],
"typeUrl": "type.googleapis.com/envoy.config.endpoint.v3.ClusterLoadAssignment",
"nonce": "` + hexString(n) + `"
}`
}
func expectedUpstreamTransportSocketJSON(snap *proxycfg.ConfigSnapshot, sni string) string {
return expectedTransportSocketJSON(snap, "type.googleapis.com/envoy.extensions.transport_sockets.tls.v3.UpstreamTlsContext", false, sni)
}
func expectedPublicTransportSocketJSON(snap *proxycfg.ConfigSnapshot) string {
return expectedTransportSocketJSON(snap, "type.googleapis.com/envoy.extensions.transport_sockets.tls.v3.DownstreamTlsContext", true, "")
}
func expectedTransportSocketJSON(
snap *proxycfg.ConfigSnapshot,
extType string,
requireClientCert bool,
sni string,
) string {
// Assume just one root for now, can get fancier later if needed.
caPEM := snap.Roots.Roots[0].RootCert
reqClient := ""
if requireClientCert {
reqClient = `,
"requireClientCertificate": true`
}
upstreamSNI := ""
if sni != "" {
upstreamSNI = `,
"sni": "` + sni + `"`
}
return `{
"name": "tls",
"typedConfig": {
"@type": "` + extType + `",
"commonTlsContext": {
"tlsParams": {},
"tlsCertificates": [
{
"certificateChain": {
"inlineString": "` + strings.Replace(snap.Leaf().CertPEM, "\n", "\\n", -1) + `"
},
"privateKey": {
"inlineString": "` + strings.Replace(snap.Leaf().PrivateKeyPEM, "\n", "\\n", -1) + `"
}
}
],
"validationContext": {
"trustedCa": {
"inlineString": "` + strings.Replace(caPEM, "\n", "\\n", -1) + `"
}
}
}
` + reqClient + `
` + upstreamSNI + `
}
}`
}
func assertChanBlocked(t *testing.T, ch chan *envoy_discovery_v3.DiscoveryResponse) {
t.Helper()
envoy.Close()
select {
case r := <-ch:
t.Fatalf("chan should block but received: %v", r)
case <-time.After(10 * time.Millisecond):
return
}
}
func assertResponseSent(t *testing.T, ch chan *envoy_discovery_v3.DiscoveryResponse, wantJSON string) {
t.Helper()
select {
case r := <-ch:
assertResponse(t, r, wantJSON)
case err := <-errCh:
require.NoError(t, err)
case <-time.After(50 * time.Millisecond):
t.Fatalf("no response received after 50ms")
t.Fatalf("timed out waiting for handler to finish")
}
}
// assertResponse is a helper to test a envoy.DiscoveryResponse matches the
// JSON representation we expect. We use JSON because the responses use protobuf
// Any type which includes binary protobuf encoding and would make creating
// expected structs require the same code that is under test!
func assertResponse(t *testing.T, r *envoy_discovery_v3.DiscoveryResponse, wantJSON string) {
t.Helper()
gotJSON := protoToJSON(t, r)
require.JSONEqf(t, wantJSON, gotJSON, "got:\n%s", gotJSON)
func TestServer_StreamAggregatedResources_v2_BasicProtocol_HTTP(t *testing.T) {
aclResolve := func(id string) (acl.Authorizer, error) {
// Allow all
return acl.RootAuthorizer("manage"), nil
}
scenario := newTestServerScenario(t, aclResolve, "web-sidecar-proxy", "", 0)
mgr, errCh, envoy := scenario.mgr, scenario.errCh, scenario.envoy
sid := structs.NewServiceID("web-sidecar-proxy", nil)
// Register the proxy to create state needed to Watch() on
mgr.RegisterProxy(t, sid)
// Send initial cluster discover (empty payload)
envoy.SendReq(t, ClusterType_v2, 0, 0)
// Check no response sent yet
assertChanBlocked(t, envoy.stream.sendCh)
// Deliver a new snapshot
// Deliver a new snapshot (tcp with one http upstream)
snap := newTestSnapshot(t, nil, "http2", &structs.ServiceConfigEntry{
Kind: structs.ServiceDefaults,
Name: "db",
Protocol: "http2",
})
mgr.DeliverConfig(t, sid, snap)
expectClusterResponse := func(v, n uint64) *envoy_api_v2.DiscoveryResponse {
return &envoy_api_v2.DiscoveryResponse{
VersionInfo: hexString(v),
TypeUrl: ClusterType_v2,
Nonce: hexString(n),
Resources: makeTestResources_v2(t,
makeTestCluster_v2(t, snap, "tcp:local_app"),
makeTestCluster_v2(t, snap, "http2:db"),
makeTestCluster_v2(t, snap, "tcp:geo-cache"),
),
}
}
expectEndpointResponse := func(v, n uint64) *envoy_api_v2.DiscoveryResponse {
return &envoy_api_v2.DiscoveryResponse{
VersionInfo: hexString(v),
TypeUrl: EndpointType_v2,
Nonce: hexString(n),
Resources: makeTestResources_v2(t,
makeTestEndpoints_v2(t, snap, "http2:db"),
makeTestEndpoints_v2(t, snap, "tcp:geo-cache"),
),
}
}
expectListenerResponse := func(v, n uint64) *envoy_api_v2.DiscoveryResponse {
return &envoy_api_v2.DiscoveryResponse{
VersionInfo: hexString(v),
TypeUrl: ListenerType_v2,
Nonce: hexString(n),
Resources: makeTestResources_v2(t,
makeTestListener_v2(t, snap, "tcp:public_listener"),
makeTestListener_v2(t, snap, "http2:db"),
makeTestListener_v2(t, snap, "tcp:geo-cache"),
),
}
}
func TestServer_StreamAggregatedResources_ACLEnforcement(t *testing.T) {
runStep(t, "no-rds", func(t *testing.T) {
// REQ: clusters
envoy.SendReq(t, ClusterType_v2, 0, 0)
// RESP: clusters
assertResponseSent(t, envoy.stream.sendCh, expectClusterResponse(1, 1))
assertChanBlocked(t, envoy.stream.sendCh)
// REQ: endpoints
envoy.SendReq(t, EndpointType_v2, 0, 0)
// ACK: clusters
envoy.SendReq(t, ClusterType_v2, 1, 1)
// RESP: endpoints
assertResponseSent(t, envoy.stream.sendCh, expectEndpointResponse(1, 2))
assertChanBlocked(t, envoy.stream.sendCh)
// REQ: listeners
envoy.SendReq(t, ListenerType_v2, 0, 0)
// ACK: endpoints
envoy.SendReq(t, EndpointType_v2, 1, 2)
// RESP: listeners
assertResponseSent(t, envoy.stream.sendCh, expectListenerResponse(1, 3))
assertChanBlocked(t, envoy.stream.sendCh)
// ACK: listeners
envoy.SendReq(t, ListenerType_v2, 1, 3)
assertChanBlocked(t, envoy.stream.sendCh)
})
// -- reconfigure with a no-op discovery chain
snap = newTestSnapshot(t, snap, "http2", &structs.ServiceConfigEntry{
Kind: structs.ServiceDefaults,
Name: "db",
Protocol: "http2",
}, &structs.ServiceRouterConfigEntry{
Kind: structs.ServiceRouter,
Name: "db",
Routes: nil,
})
mgr.DeliverConfig(t, sid, snap)
// update this test helper to reflect the RDS-linked listener
expectListenerResponse = func(v, n uint64) *envoy_api_v2.DiscoveryResponse {
return &envoy_api_v2.DiscoveryResponse{
VersionInfo: hexString(v),
TypeUrl: ListenerType_v2,
Nonce: hexString(n),
Resources: makeTestResources_v2(t,
makeTestListener_v2(t, snap, "tcp:public_listener"),
makeTestListener_v2(t, snap, "http2:db:rds"),
makeTestListener_v2(t, snap, "tcp:geo-cache"),
),
}
}
runStep(t, "with-rds", func(t *testing.T) {
// RESP: listeners (but also a stray update of the other registered types)
assertResponseSent(t, envoy.stream.sendCh, expectClusterResponse(2, 4))
assertResponseSent(t, envoy.stream.sendCh, expectEndpointResponse(2, 5))
assertResponseSent(t, envoy.stream.sendCh, expectListenerResponse(2, 6))
assertChanBlocked(t, envoy.stream.sendCh)
// ACK: listeners (but also stray ACKs of the other registered types)
envoy.SendReq(t, ClusterType_v2, 2, 4)
envoy.SendReq(t, EndpointType_v2, 2, 5)
envoy.SendReq(t, ListenerType_v2, 2, 6)
// REQ: routes
envoy.SendReq(t, RouteType_v2, 0, 0)
// RESP: routes
assertResponseSent(t, envoy.stream.sendCh, &envoy_api_v2.DiscoveryResponse{
VersionInfo: hexString(2),
TypeUrl: RouteType_v2,
Nonce: hexString(7),
Resources: makeTestResources_v2(t,
makeTestRoute_v2(t, "http2:db"),
),
})
assertChanBlocked(t, envoy.stream.sendCh)
// ACK: routes
envoy.SendReq(t, RouteType_v2, 2, 7)
})
envoy.Close()
select {
case err := <-errCh:
require.NoError(t, err)
case <-time.After(50 * time.Millisecond):
t.Fatalf("timed out waiting for handler to finish")
}
}
func TestServer_StreamAggregatedResources_v2_ACLEnforcement(t *testing.T) {
tests := []struct {
name string
defaultDeny bool
@ -438,7 +409,6 @@ func TestServer_StreamAggregatedResources_ACLEnforcement(t *testing.T) {
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
mgr := newTestManager(t)
aclResolve := func(id string) (acl.Authorizer, error) {
if !tt.defaultDeny {
// Allow all
@ -455,19 +425,9 @@ func TestServer_StreamAggregatedResources_ACLEnforcement(t *testing.T) {
require.NoError(t, err)
return acl.NewPolicyAuthorizerWithDefaults(acl.RootAuthorizer("deny"), []*acl.Policy{policy}, nil)
}
envoy := NewTestEnvoy(t, "web-sidecar-proxy", tt.token)
defer envoy.Close()
s := Server{
Logger: testutil.Logger(t),
CfgMgr: mgr,
ResolveToken: aclResolve,
}
errCh := make(chan error, 1)
go func() {
errCh <- s.StreamAggregatedResources(envoy.stream)
}()
scenario := newTestServerScenario(t, aclResolve, "web-sidecar-proxy", tt.token, 0)
mgr, errCh, envoy := scenario.mgr, scenario.errCh, scenario.envoy
sid := structs.NewServiceID("web-sidecar-proxy", nil)
// Register the proxy to create state needed to Watch() on
@ -476,17 +436,26 @@ func TestServer_StreamAggregatedResources_ACLEnforcement(t *testing.T) {
// Deliver a new snapshot
snap := tt.cfgSnap
if snap == nil {
snap = proxycfg.TestConfigSnapshot(t)
snap = newTestSnapshot(t, nil, "")
}
mgr.DeliverConfig(t, sid, snap)
// Send initial listener discover, in real life Envoy always sends cluster
// first but it doesn't really matter and listener has a response that
// includes the token in the ext authz filter so lets us test more stuff.
envoy.SendReq(t, ListenerType, 0, 0)
// includes the token in the ext rbac filter so lets us test more stuff.
envoy.SendReq(t, ListenerType_v2, 0, 0)
if !tt.wantDenied {
assertResponseSent(t, envoy.stream.sendCh, expectListenerJSON(snap, 1, 1))
assertResponseSent(t, envoy.stream.sendCh, &envoy_api_v2.DiscoveryResponse{
VersionInfo: hexString(1),
TypeUrl: ListenerType_v2,
Nonce: hexString(1),
Resources: makeTestResources_v2(t,
makeTestListener_v2(t, snap, "tcp:public_listener"),
makeTestListener_v2(t, snap, "tcp:db"),
makeTestListener_v2(t, snap, "tcp:geo-cache"),
),
})
// Close the client stream since all is well. We _don't_ do this in the
// expected error case because we want to verify the error closes the
// stream from server side.
@ -509,7 +478,7 @@ func TestServer_StreamAggregatedResources_ACLEnforcement(t *testing.T) {
}
}
func TestServer_StreamAggregatedResources_ACLTokenDeleted_StreamTerminatedDuringDiscoveryRequest(t *testing.T) {
func TestServer_StreamAggregatedResources_v2_ACLTokenDeleted_StreamTerminatedDuringDiscoveryRequest(t *testing.T) {
aclRules := `service "web" { policy = "write" }`
token := "service-write-on-web"
@ -519,7 +488,6 @@ func TestServer_StreamAggregatedResources_ACLTokenDeleted_StreamTerminatedDuring
var validToken atomic.Value
validToken.Store(token)
mgr := newTestManager(t)
aclResolve := func(id string) (acl.Authorizer, error) {
if token := validToken.Load(); token == nil || id != token.(string) {
return nil, acl.ErrNotFound
@ -527,20 +495,10 @@ func TestServer_StreamAggregatedResources_ACLTokenDeleted_StreamTerminatedDuring
return acl.NewPolicyAuthorizerWithDefaults(acl.RootAuthorizer("deny"), []*acl.Policy{policy}, nil)
}
envoy := NewTestEnvoy(t, "web-sidecar-proxy", token)
defer envoy.Close()
s := Server{
Logger: testutil.Logger(t),
CfgMgr: mgr,
ResolveToken: aclResolve,
AuthCheckFrequency: 1 * time.Hour, // make sure this doesn't kick in
}
errCh := make(chan error, 1)
go func() {
errCh <- s.StreamAggregatedResources(envoy.stream)
}()
scenario := newTestServerScenario(t, aclResolve, "web-sidecar-proxy", token,
1*time.Hour, // make sure this doesn't kick in
)
mgr, errCh, envoy := scenario.mgr, scenario.errCh, scenario.envoy
getError := func() (gotErr error, ok bool) {
select {
@ -556,7 +514,7 @@ func TestServer_StreamAggregatedResources_ACLTokenDeleted_StreamTerminatedDuring
mgr.RegisterProxy(t, sid)
// Send initial cluster discover (OK)
envoy.SendReq(t, ClusterType, 0, 0)
envoy.SendReq(t, ClusterType_v2, 0, 0)
{
err, ok := getError()
require.NoError(t, err)
@ -572,17 +530,26 @@ func TestServer_StreamAggregatedResources_ACLTokenDeleted_StreamTerminatedDuring
}
// Deliver a new snapshot
snap := proxycfg.TestConfigSnapshot(t)
snap := newTestSnapshot(t, nil, "")
mgr.DeliverConfig(t, sid, snap)
assertResponseSent(t, envoy.stream.sendCh, expectClustersJSON(snap, 1, 1))
assertResponseSent(t, envoy.stream.sendCh, &envoy_api_v2.DiscoveryResponse{
VersionInfo: hexString(1),
TypeUrl: ClusterType_v2,
Nonce: hexString(1),
Resources: makeTestResources_v2(t,
makeTestCluster_v2(t, snap, "tcp:local_app"),
makeTestCluster_v2(t, snap, "tcp:db"),
makeTestCluster_v2(t, snap, "tcp:geo-cache"),
),
})
// Now nuke the ACL token.
validToken.Store("")
// It also (in parallel) issues the next cluster request (which acts as an ACK
// of the version we sent)
envoy.SendReq(t, ClusterType, 1, 1)
envoy.SendReq(t, ClusterType_v2, 1, 1)
select {
case err := <-errCh:
@ -598,7 +565,7 @@ func TestServer_StreamAggregatedResources_ACLTokenDeleted_StreamTerminatedDuring
}
}
func TestServer_StreamAggregatedResources_ACLTokenDeleted_StreamTerminatedInBackground(t *testing.T) {
func TestServer_StreamAggregatedResources_v2_ACLTokenDeleted_StreamTerminatedInBackground(t *testing.T) {
if testing.Short() {
t.Skip("too slow for testing.Short")
}
@ -612,7 +579,6 @@ func TestServer_StreamAggregatedResources_ACLTokenDeleted_StreamTerminatedInBack
var validToken atomic.Value
validToken.Store(token)
mgr := newTestManager(t)
aclResolve := func(id string) (acl.Authorizer, error) {
if token := validToken.Load(); token == nil || id != token.(string) {
return nil, acl.ErrNotFound
@ -620,20 +586,10 @@ func TestServer_StreamAggregatedResources_ACLTokenDeleted_StreamTerminatedInBack
return acl.NewPolicyAuthorizerWithDefaults(acl.RootAuthorizer("deny"), []*acl.Policy{policy}, nil)
}
envoy := NewTestEnvoy(t, "web-sidecar-proxy", token)
defer envoy.Close()
s := Server{
Logger: testutil.Logger(t),
CfgMgr: mgr,
ResolveToken: aclResolve,
AuthCheckFrequency: 100 * time.Millisecond, // Make this short.
}
errCh := make(chan error, 1)
go func() {
errCh <- s.StreamAggregatedResources(envoy.stream)
}()
scenario := newTestServerScenario(t, aclResolve, "web-sidecar-proxy", token,
100*time.Millisecond, // Make this short.
)
mgr, errCh, envoy := scenario.mgr, scenario.errCh, scenario.envoy
getError := func() (gotErr error, ok bool) {
select {
@ -649,7 +605,7 @@ func TestServer_StreamAggregatedResources_ACLTokenDeleted_StreamTerminatedInBack
mgr.RegisterProxy(t, sid)
// Send initial cluster discover (OK)
envoy.SendReq(t, ClusterType, 0, 0)
envoy.SendReq(t, ClusterType_v2, 0, 0)
{
err, ok := getError()
require.NoError(t, err)
@ -665,14 +621,23 @@ func TestServer_StreamAggregatedResources_ACLTokenDeleted_StreamTerminatedInBack
}
// Deliver a new snapshot
snap := proxycfg.TestConfigSnapshot(t)
snap := newTestSnapshot(t, nil, "")
mgr.DeliverConfig(t, sid, snap)
assertResponseSent(t, envoy.stream.sendCh, expectClustersJSON(snap, 1, 1))
assertResponseSent(t, envoy.stream.sendCh, &envoy_api_v2.DiscoveryResponse{
VersionInfo: hexString(1),
TypeUrl: ClusterType_v2,
Nonce: hexString(1),
Resources: makeTestResources_v2(t,
makeTestCluster_v2(t, snap, "tcp:local_app"),
makeTestCluster_v2(t, snap, "tcp:db"),
makeTestCluster_v2(t, snap, "tcp:geo-cache"),
),
})
// It also (in parallel) issues the next cluster request (which acts as an ACK
// of the version we sent)
envoy.SendReq(t, ClusterType, 1, 1)
envoy.SendReq(t, ClusterType_v2, 1, 1)
// Check no response sent yet
assertChanBlocked(t, envoy.stream.sendCh)
@ -699,33 +664,21 @@ func TestServer_StreamAggregatedResources_ACLTokenDeleted_StreamTerminatedInBack
}
}
func TestServer_StreamAggregatedResources_IngressEmptyResponse(t *testing.T) {
mgr := newTestManager(t)
func TestServer_StreamAggregatedResources_v2_IngressEmptyResponse(t *testing.T) {
aclResolve := func(id string) (acl.Authorizer, error) {
// Allow all
return acl.RootAuthorizer("manage"), nil
}
envoy := NewTestEnvoy(t, "ingress-gateway", "")
defer envoy.Close()
s := Server{
Logger: testutil.Logger(t),
CfgMgr: mgr,
ResolveToken: aclResolve,
}
scenario := newTestServerScenario(t, aclResolve, "ingress-gateway", "", 0)
mgr, errCh, envoy := scenario.mgr, scenario.errCh, scenario.envoy
sid := structs.NewServiceID("ingress-gateway", nil)
go func() {
err := s.StreamAggregatedResources(envoy.stream)
require.NoError(t, err)
}()
// Register the proxy to create state needed to Watch() on
mgr.RegisterProxy(t, sid)
// Send initial cluster discover
envoy.SendReq(t, ClusterType, 0, 0)
envoy.SendReq(t, ClusterType_v2, 0, 0)
// Check no response sent yet
assertChanBlocked(t, envoy.stream.sendCh)
@ -734,31 +687,105 @@ func TestServer_StreamAggregatedResources_IngressEmptyResponse(t *testing.T) {
snap := proxycfg.TestConfigSnapshotIngressGatewayNoServices(t)
mgr.DeliverConfig(t, sid, snap)
emptyClusterJSON := `{
"versionInfo": "` + hexString(1) + `",
"typeUrl": "type.googleapis.com/envoy.config.cluster.v3.Cluster",
"resources": [],
"nonce": "` + hexString(1) + `"
}`
emptyListenerJSON := `{
"versionInfo": "` + hexString(1) + `",
"typeUrl": "type.googleapis.com/envoy.config.listener.v3.Listener",
"resources": [],
"nonce": "` + hexString(2) + `"
}`
emptyRouteJSON := `{
"versionInfo": "` + hexString(1) + `",
"typeUrl": "type.googleapis.com/envoy.config.route.v3.RouteConfiguration",
"resources": [],
"nonce": "` + hexString(3) + `"
}`
emptyClusterResp := &envoy_api_v2.DiscoveryResponse{
VersionInfo: hexString(1),
TypeUrl: ClusterType_v2,
Nonce: hexString(1),
}
emptyListenerResp := &envoy_api_v2.DiscoveryResponse{
VersionInfo: hexString(1),
TypeUrl: ListenerType_v2,
Nonce: hexString(2),
}
emptyRouteResp := &envoy_api_v2.DiscoveryResponse{
VersionInfo: hexString(1),
TypeUrl: RouteType_v2,
Nonce: hexString(3),
}
assertResponseSent(t, envoy.stream.sendCh, emptyClusterJSON)
assertResponseSent(t, envoy.stream.sendCh, emptyClusterResp)
// Send initial listener discover
envoy.SendReq(t, ListenerType, 0, 0)
assertResponseSent(t, envoy.stream.sendCh, emptyListenerJSON)
envoy.SendReq(t, ListenerType_v2, 0, 0)
assertResponseSent(t, envoy.stream.sendCh, emptyListenerResp)
envoy.SendReq(t, RouteType, 0, 0)
assertResponseSent(t, envoy.stream.sendCh, emptyRouteJSON)
envoy.SendReq(t, RouteType_v2, 0, 0)
assertResponseSent(t, envoy.stream.sendCh, emptyRouteResp)
envoy.Close()
select {
case err := <-errCh:
require.NoError(t, err)
case <-time.After(50 * time.Millisecond):
t.Fatalf("timed out waiting for handler to finish")
}
}
func assertChanBlocked(t *testing.T, ch chan *envoy_api_v2.DiscoveryResponse) {
t.Helper()
select {
case r := <-ch:
t.Fatalf("chan should block but received: %v", r)
case <-time.After(10 * time.Millisecond):
return
}
}
func assertResponseSent(t *testing.T, ch chan *envoy_api_v2.DiscoveryResponse, want *envoy_api_v2.DiscoveryResponse) {
t.Helper()
select {
case got := <-ch:
assertResponse(t, got, want)
case <-time.After(50 * time.Millisecond):
t.Fatalf("no response received after 50ms")
}
}
// assertResponse is a helper to test a envoy.DiscoveryResponse matches the
// expected value. We use JSON during comparison here because the responses use protobuf
// Any type which includes binary protobuf encoding.
func assertResponse(t *testing.T, got, want *envoy_api_v2.DiscoveryResponse) {
t.Helper()
gotJSON := protoToJSON(t, got)
wantJSON := protoToJSON(t, want)
require.JSONEqf(t, wantJSON, gotJSON, "got:\n%s", gotJSON)
}
func makeTestResources_v2(t *testing.T, resources ...proto.Message) []*any.Any {
var ret []*any.Any
for _, res := range resources {
any, err := ptypes.MarshalAny(res)
require.NoError(t, err)
ret = append(ret, any)
}
return ret
}
func makeTestListener_v2(t *testing.T, snap *proxycfg.ConfigSnapshot, fixtureName string) *envoy_api_v2.Listener {
v3 := makeTestListener(t, snap, fixtureName)
v2, err := convertListenerToV2(v3)
require.NoError(t, err)
return v2
}
func makeTestCluster_v2(t *testing.T, snap *proxycfg.ConfigSnapshot, fixtureName string) *envoy_api_v2.Cluster {
v3 := makeTestCluster(t, snap, fixtureName)
v2, err := convertClusterToV2(v3)
require.NoError(t, err)
return v2
}
func makeTestEndpoints_v2(t *testing.T, snap *proxycfg.ConfigSnapshot, fixtureName string) *envoy_api_v2.ClusterLoadAssignment {
v3 := makeTestEndpoints(t, snap, fixtureName)
v2, err := convertClusterLoadAssignmentToV2(v3)
require.NoError(t, err)
return v2
}
func makeTestRoute_v2(t *testing.T, fixtureName string) *envoy_api_v2.RouteConfiguration {
v3 := makeTestRoute(t, fixtureName)
v2, err := convertRouteConfigurationToV2(v3)
require.NoError(t, err)
return v2
}

View File

@ -9,41 +9,48 @@ import (
"sync"
"time"
envoy_api_v2 "github.com/envoyproxy/go-control-plane/envoy/api/v2"
envoy_core_v2 "github.com/envoyproxy/go-control-plane/envoy/api/v2/core"
envoy_core_v3 "github.com/envoyproxy/go-control-plane/envoy/config/core/v3"
envoy_discovery_v3 "github.com/envoyproxy/go-control-plane/envoy/service/discovery/v3"
envoy_type_v3 "github.com/envoyproxy/go-control-plane/envoy/type/v3"
"github.com/hashicorp/consul/agent/xds/proxysupport"
"github.com/mitchellh/go-testing-interface"
status "google.golang.org/genproto/googleapis/rpc/status"
"google.golang.org/grpc"
"google.golang.org/grpc/metadata"
"github.com/hashicorp/consul/agent/xds/proxysupport"
)
// TestADSStream mocks
// discovery.AggregatedDiscoveryService_StreamAggregatedResourcesServer to allow
// testing ADS handler.
type TestADSStream struct {
ctx context.Context
sendCh chan *envoy_discovery_v3.DiscoveryResponse
recvCh chan *envoy_discovery_v3.DiscoveryRequest
// TestADSDeltaStream mocks
// discovery.AggregatedDiscoveryService_DeltaAggregatedResourcesServer to allow
// testing the ADS handler.
type TestADSDeltaStream struct {
stubGrpcServerStream
sendCh chan *envoy_discovery_v3.DeltaDiscoveryResponse
recvCh chan *envoy_discovery_v3.DeltaDiscoveryRequest
}
// NewTestADSStream makes a new TestADSStream
func NewTestADSStream(t testing.T, ctx context.Context) *TestADSStream {
return &TestADSStream{
ctx: ctx,
sendCh: make(chan *envoy_discovery_v3.DiscoveryResponse, 1),
recvCh: make(chan *envoy_discovery_v3.DiscoveryRequest, 1),
var _ ADSDeltaStream = (*TestADSDeltaStream)(nil)
func NewTestADSDeltaStream(t testing.T, ctx context.Context) *TestADSDeltaStream {
s := &TestADSDeltaStream{
sendCh: make(chan *envoy_discovery_v3.DeltaDiscoveryResponse, 1),
recvCh: make(chan *envoy_discovery_v3.DeltaDiscoveryRequest, 1),
}
s.stubGrpcServerStream.ctx = ctx
return s
}
// Send implements ADSStream
func (s *TestADSStream) Send(r *envoy_discovery_v3.DiscoveryResponse) error {
// Send implements ADSDeltaStream
func (s *TestADSDeltaStream) Send(r *envoy_discovery_v3.DeltaDiscoveryResponse) error {
s.sendCh <- r
return nil
}
// Recv implements ADSStream
func (s *TestADSStream) Recv() (*envoy_discovery_v3.DiscoveryRequest, error) {
// Recv implements ADSDeltaStream
func (s *TestADSDeltaStream) Recv() (*envoy_discovery_v3.DeltaDiscoveryRequest, error) {
r := <-s.recvCh
if r == nil {
return nil, io.EOF
@ -51,48 +58,52 @@ func (s *TestADSStream) Recv() (*envoy_discovery_v3.DiscoveryRequest, error) {
return r, nil
}
// SetHeader implements ADSStream
func (s *TestADSStream) SetHeader(metadata.MD) error {
// TestADSStream mocks
// discovery.AggregatedDiscoveryService_StreamAggregatedResourcesServer to allow
// testing ADS handler.
type TestADSStream struct {
stubGrpcServerStream
sendCh chan *envoy_api_v2.DiscoveryResponse
recvCh chan *envoy_api_v2.DiscoveryRequest
}
// NewTestADSStream makes a new TestADSStream
func NewTestADSStream(t testing.T, ctx context.Context) *TestADSStream {
s := &TestADSStream{
sendCh: make(chan *envoy_api_v2.DiscoveryResponse, 1),
recvCh: make(chan *envoy_api_v2.DiscoveryRequest, 1),
}
s.stubGrpcServerStream.ctx = ctx
return s
}
// Send implements ADSStream
func (s *TestADSStream) Send(r *envoy_api_v2.DiscoveryResponse) error {
s.sendCh <- r
return nil
}
// SendHeader implements ADSStream
func (s *TestADSStream) SendHeader(metadata.MD) error {
return nil
// Recv implements ADSStream
func (s *TestADSStream) Recv() (*envoy_api_v2.DiscoveryRequest, error) {
r := <-s.recvCh
if r == nil {
return nil, io.EOF
}
// SetTrailer implements ADSStream
func (s *TestADSStream) SetTrailer(metadata.MD) {
}
// Context implements ADSStream
func (s *TestADSStream) Context() context.Context {
return s.ctx
}
// SendMsg implements ADSStream
func (s *TestADSStream) SendMsg(m interface{}) error {
return nil
}
// RecvMsg implements ADSStream
func (s *TestADSStream) RecvMsg(m interface{}) error {
return nil
}
type configState struct {
lastNonce, lastVersion, acceptedVersion string
return r, nil
}
// TestEnvoy is a helper to simulate Envoy ADS requests.
type TestEnvoy struct {
sync.Mutex
stream *TestADSStream
proxyID string
token string
state map[string]configState
mu sync.Mutex
ctx context.Context
cancel func()
proxyID string
token string
stream *TestADSStream // SoTW v2
deltaStream *TestADSDeltaStream // Incremental v3
}
// NewTestEnvoy creates a TestEnvoy instance.
@ -105,12 +116,14 @@ func NewTestEnvoy(t testing.T, proxyID, token string) *TestEnvoy {
metadata.Pairs("x-consul-token", token))
}
return &TestEnvoy{
stream: NewTestADSStream(t, ctx),
state: make(map[string]configState),
ctx: ctx,
cancel: cancel,
proxyID: proxyID,
token: token,
stream: NewTestADSStream(t, ctx),
deltaStream: NewTestADSDeltaStream(t, ctx),
}
}
@ -149,23 +162,28 @@ func stringToEnvoyVersion(vs string) (*envoy_type_v3.SemanticVersion, bool) {
// SendReq sends a request from the test server.
func (e *TestEnvoy) SendReq(t testing.T, typeURL string, version, nonce uint64) {
e.Lock()
defer e.Unlock()
e.mu.Lock()
defer e.mu.Unlock()
ev, valid := stringToEnvoyVersion(proxysupport.EnvoyVersions[0])
if !valid {
t.Fatal("envoy version is not valid: %s", proxysupport.EnvoyVersions[0])
}
req := &envoy_discovery_v3.DiscoveryRequest{
evV2, err := convertSemanticVersionToV2(ev)
if err != nil {
t.Fatalf("err: %v", err)
}
req := &envoy_api_v2.DiscoveryRequest{
VersionInfo: hexString(version),
Node: &envoy_core_v3.Node{
Node: &envoy_core_v2.Node{
Id: e.proxyID,
Cluster: e.proxyID,
UserAgentName: "envoy",
UserAgentVersionType: &envoy_core_v3.Node_UserAgentBuildVersion{
UserAgentBuildVersion: &envoy_core_v3.BuildVersion{
Version: ev,
UserAgentVersionType: &envoy_core_v2.Node_UserAgentBuildVersion{
UserAgentBuildVersion: &envoy_core_v2.BuildVersion{
Version: evV2,
},
},
},
@ -179,18 +197,121 @@ func (e *TestEnvoy) SendReq(t testing.T, typeURL string, version, nonce uint64)
}
}
// SendDeltaReq sends a delta request from the test server.
//
// NOTE: the input request is mutated before sending by injecting the node.
func (e *TestEnvoy) SendDeltaReq(
t testing.T,
typeURL string,
req *envoy_discovery_v3.DeltaDiscoveryRequest, // optional
) {
e.sendDeltaReq(t, typeURL, nil, req)
}
func (e *TestEnvoy) SendDeltaReqACK(
t testing.T,
typeURL string,
nonce uint64,
ack bool,
errorDetail *status.Status,
) {
req := &envoy_discovery_v3.DeltaDiscoveryRequest{}
if !ack {
req.ErrorDetail = errorDetail
}
e.sendDeltaReq(t, typeURL, &nonce, req)
}
func (e *TestEnvoy) sendDeltaReq(
t testing.T,
typeURL string,
nonce *uint64,
req *envoy_discovery_v3.DeltaDiscoveryRequest, // optional
) {
e.mu.Lock()
defer e.mu.Unlock()
ev, valid := stringToEnvoyVersion(proxysupport.EnvoyVersions[0])
if !valid {
t.Fatal("envoy version is not valid: %s", proxysupport.EnvoyVersions[0])
}
if req == nil {
req = &envoy_discovery_v3.DeltaDiscoveryRequest{}
}
if nonce != nil {
req.ResponseNonce = hexString(*nonce)
}
req.TypeUrl = typeURL
req.Node = &envoy_core_v3.Node{
Id: e.proxyID,
Cluster: e.proxyID,
UserAgentName: "envoy",
UserAgentVersionType: &envoy_core_v3.Node_UserAgentBuildVersion{
UserAgentBuildVersion: &envoy_core_v3.BuildVersion{
Version: ev,
},
},
}
select {
case e.deltaStream.recvCh <- req:
case <-time.After(50 * time.Millisecond):
t.Fatalf("send to delta stream blocked for too long")
}
}
// Close closes the client and cancels it's request context.
func (e *TestEnvoy) Close() error {
e.Lock()
defer e.Unlock()
e.mu.Lock()
defer e.mu.Unlock()
// unblock the recv chan to simulate recv error when client disconnects
// unblock the recv chans to simulate recv errors when client disconnects
if e.stream != nil && e.stream.recvCh != nil {
close(e.stream.recvCh)
e.stream = nil
}
if e.deltaStream != nil && e.deltaStream.recvCh != nil {
close(e.deltaStream.recvCh)
e.deltaStream = nil
}
if e.cancel != nil {
e.cancel()
}
return nil
}
type stubGrpcServerStream struct {
ctx context.Context
grpc.ServerStream
}
var _ grpc.ServerStream = (*stubGrpcServerStream)(nil)
// SetHeader implements grpc.ServerStream as part of ADSDeltaStream
func (s *stubGrpcServerStream) SetHeader(metadata.MD) error {
return nil
}
// SendHeader implements grpc.ServerStream as part of ADSDeltaStream
func (s *stubGrpcServerStream) SendHeader(metadata.MD) error {
return nil
}
// SetTrailer implements grpc.ServerStream as part of ADSDeltaStream
func (s *stubGrpcServerStream) SetTrailer(metadata.MD) {
}
// Context implements grpc.ServerStream as part of ADSDeltaStream
func (s *stubGrpcServerStream) Context() context.Context {
return s.ctx
}
// SendMsg implements grpc.ServerStream as part of ADSDeltaStream
func (s *stubGrpcServerStream) SendMsg(m interface{}) error {
return nil
}
// RecvMsg implements grpc.ServerStream as part of ADSDeltaStream
func (s *stubGrpcServerStream) RecvMsg(m interface{}) error {
return nil
}

View File

@ -32,6 +32,8 @@ import (
envoy_tls_v3 "github.com/envoyproxy/go-control-plane/envoy/extensions/transport_sockets/tls/v3"
envoy_discovery_v2 "github.com/envoyproxy/go-control-plane/envoy/service/discovery/v2"
envoy_discovery_v3 "github.com/envoyproxy/go-control-plane/envoy/service/discovery/v3"
envoy_type_v2 "github.com/envoyproxy/go-control-plane/envoy/type"
envoy_type_v3 "github.com/envoyproxy/go-control-plane/envoy/type/v3"
"github.com/golang/protobuf/proto"
"github.com/golang/protobuf/ptypes"
@ -68,7 +70,7 @@ func (s *adsServerV2Shim) StreamAggregatedResources(stream ADSStream_v2) error {
stream: stream,
ServerStream: stream,
}
return s.srv.StreamAggregatedResources(shim)
return s.srv.streamAggregatedResources(shim)
}
// DeltaAggregatedResources implements envoy_discovery_v2.AggregatedDiscoveryServiceServer
@ -125,6 +127,82 @@ func convertDiscoveryRequestToV3(req *envoy_api_v2.DiscoveryRequest) (*envoy_dis
return &reqV3, nil
}
// convertClusterToV2 is only used in tests.
func convertClusterToV2(resp *envoy_cluster_v3.Cluster) (*envoy_api_v2.Cluster, error) {
var pbuf proto.Buffer
if err := pbuf.Marshal(resp); err != nil {
return nil, err
}
var v2 envoy_api_v2.Cluster
if err := pbuf.Unmarshal(&v2); err != nil {
return nil, err
}
if err := convertTypedConfigsToV2(&v2); err != nil {
return nil, err
}
return &v2, nil
}
// convertClusterLoadAssignmentToV2 is only used in tests.
func convertClusterLoadAssignmentToV2(resp *envoy_endpoint_v3.ClusterLoadAssignment) (*envoy_api_v2.ClusterLoadAssignment, error) {
var pbuf proto.Buffer
if err := pbuf.Marshal(resp); err != nil {
return nil, err
}
var v2 envoy_api_v2.ClusterLoadAssignment
if err := pbuf.Unmarshal(&v2); err != nil {
return nil, err
}
if err := convertTypedConfigsToV2(&v2); err != nil {
return nil, err
}
return &v2, nil
}
// convertRouteConfigurationToV2 is only used in tests.
func convertRouteConfigurationToV2(resp *envoy_route_v3.RouteConfiguration) (*envoy_api_v2.RouteConfiguration, error) {
var pbuf proto.Buffer
if err := pbuf.Marshal(resp); err != nil {
return nil, err
}
var v2 envoy_api_v2.RouteConfiguration
if err := pbuf.Unmarshal(&v2); err != nil {
return nil, err
}
if err := convertTypedConfigsToV2(&v2); err != nil {
return nil, err
}
return &v2, nil
}
// convertListenerToV2 is only used in tests.
func convertListenerToV2(resp *envoy_listener_v3.Listener) (*envoy_api_v2.Listener, error) {
var pbuf proto.Buffer
if err := pbuf.Marshal(resp); err != nil {
return nil, err
}
var v2 envoy_api_v2.Listener
if err := pbuf.Unmarshal(&v2); err != nil {
return nil, err
}
if err := convertTypedConfigsToV2(&v2); err != nil {
return nil, err
}
return &v2, nil
}
func convertDiscoveryResponseToV2(resp *envoy_discovery_v3.DiscoveryResponse) (*envoy_api_v2.DiscoveryResponse, error) {
var pbuf proto.Buffer
if err := pbuf.Marshal(resp); err != nil {
@ -181,6 +259,20 @@ func convertHttpFilterToV2(filter *envoy_http_v3.HttpFilter) (*envoy_http_v2.Htt
return &filterV2, nil
}
func convertSemanticVersionToV2(version *envoy_type_v3.SemanticVersion) (*envoy_type_v2.SemanticVersion, error) {
var pbuf proto.Buffer
if err := pbuf.Marshal(version); err != nil {
return nil, err
}
var versionV2 envoy_type_v2.SemanticVersion
if err := pbuf.Unmarshal(&versionV2); err != nil {
return nil, err
}
return &versionV2, nil
}
// Responses
func convertTypedConfigsToV2(pb proto.Message) error {
switch x := pb.(type) {

View File

@ -0,0 +1,649 @@
package xds
import (
"sort"
"sync"
"testing"
"time"
envoy_cluster_v3 "github.com/envoyproxy/go-control-plane/envoy/config/cluster/v3"
envoy_core_v3 "github.com/envoyproxy/go-control-plane/envoy/config/core/v3"
envoy_endpoint_v3 "github.com/envoyproxy/go-control-plane/envoy/config/endpoint/v3"
envoy_listener_v3 "github.com/envoyproxy/go-control-plane/envoy/config/listener/v3"
envoy_rbac_v3 "github.com/envoyproxy/go-control-plane/envoy/config/rbac/v3"
envoy_route_v3 "github.com/envoyproxy/go-control-plane/envoy/config/route/v3"
envoy_http_v3 "github.com/envoyproxy/go-control-plane/envoy/extensions/filters/network/http_connection_manager/v3"
envoy_network_rbac_v3 "github.com/envoyproxy/go-control-plane/envoy/extensions/filters/network/rbac/v3"
envoy_tcp_proxy_v3 "github.com/envoyproxy/go-control-plane/envoy/extensions/filters/network/tcp_proxy/v3"
envoy_tls_v3 "github.com/envoyproxy/go-control-plane/envoy/extensions/transport_sockets/tls/v3"
envoy_discovery_v3 "github.com/envoyproxy/go-control-plane/envoy/service/discovery/v3"
envoy_type_v3 "github.com/envoyproxy/go-control-plane/envoy/type/v3"
"github.com/golang/protobuf/proto"
"github.com/golang/protobuf/ptypes"
"github.com/golang/protobuf/ptypes/wrappers"
"github.com/mitchellh/copystructure"
"github.com/stretchr/testify/require"
"github.com/hashicorp/consul/agent/proxycfg"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/sdk/testutil"
)
// NOTE: this file is a collection of test helper functions for testing xDS
// protocols.
func newTestSnapshot(
t *testing.T,
prevSnap *proxycfg.ConfigSnapshot,
dbServiceProtocol string,
additionalEntries ...structs.ConfigEntry,
) *proxycfg.ConfigSnapshot {
snap := proxycfg.TestConfigSnapshotDiscoveryChainDefaultWithEntries(t, additionalEntries...)
snap.ConnectProxy.PreparedQueryEndpoints = map[string]structs.CheckServiceNodes{
"prepared_query:geo-cache": proxycfg.TestUpstreamNodes(t),
}
if prevSnap != nil {
snap.Roots = prevSnap.Roots
snap.ConnectProxy.Leaf = prevSnap.ConnectProxy.Leaf
}
if dbServiceProtocol != "" {
// Simulate ServiceManager injection of protocol
snap.Proxy.Upstreams[0].Config["protocol"] = dbServiceProtocol
snap.ConnectProxy.ConfigSnapshotUpstreams.UpstreamConfig = snap.Proxy.Upstreams.ToMap()
}
return snap
}
// testManager is a mock of proxycfg.Manager that's simpler to control for
// testing. It also implements ConnectAuthz to allow control over authorization.
type testManager struct {
sync.Mutex
chans map[structs.ServiceID]chan *proxycfg.ConfigSnapshot
cancels chan structs.ServiceID
}
func newTestManager(t *testing.T) *testManager {
return &testManager{
chans: map[structs.ServiceID]chan *proxycfg.ConfigSnapshot{},
cancels: make(chan structs.ServiceID, 10),
}
}
// RegisterProxy simulates a proxy registration
func (m *testManager) RegisterProxy(t *testing.T, proxyID structs.ServiceID) {
m.Lock()
defer m.Unlock()
m.chans[proxyID] = make(chan *proxycfg.ConfigSnapshot, 1)
}
// Deliver simulates a proxy registration
func (m *testManager) DeliverConfig(t *testing.T, proxyID structs.ServiceID, cfg *proxycfg.ConfigSnapshot) {
t.Helper()
m.Lock()
defer m.Unlock()
select {
case m.chans[proxyID] <- cfg:
case <-time.After(10 * time.Millisecond):
t.Fatalf("took too long to deliver config")
}
}
// Watch implements ConfigManager
func (m *testManager) Watch(proxyID structs.ServiceID) (<-chan *proxycfg.ConfigSnapshot, proxycfg.CancelFunc) {
m.Lock()
defer m.Unlock()
// ch might be nil but then it will just block forever
return m.chans[proxyID], func() {
m.cancels <- proxyID
}
}
// AssertWatchCancelled checks that the most recent call to a Watch cancel func
// was from the specified proxyID and that one is made in a short time. This
// probably won't work if you are running multiple Watches in parallel on
// multiple proxyIDS due to timing/ordering issues but I don't think we need to
// do that.
func (m *testManager) AssertWatchCancelled(t *testing.T, proxyID structs.ServiceID) {
t.Helper()
select {
case got := <-m.cancels:
require.Equal(t, proxyID, got)
case <-time.After(50 * time.Millisecond):
t.Fatalf("timed out waiting for Watch cancel for %s", proxyID)
}
}
type testServerScenario struct {
server *Server
mgr *testManager
envoy *TestEnvoy
errCh <-chan error
}
func newTestServerScenario(
t *testing.T,
resolveToken ACLResolverFunc,
proxyID string,
token string,
authCheckFrequency time.Duration,
) *testServerScenario {
return newTestServerScenarioInner(t, resolveToken, proxyID, token, authCheckFrequency, false)
}
func newTestServerDeltaScenario(
t *testing.T,
resolveToken ACLResolverFunc,
proxyID string,
token string,
authCheckFrequency time.Duration,
) *testServerScenario {
return newTestServerScenarioInner(t, resolveToken, proxyID, token, authCheckFrequency, true)
}
func newTestServerScenarioInner(
t *testing.T,
resolveToken ACLResolverFunc,
proxyID string,
token string,
authCheckFrequency time.Duration,
incremental bool,
) *testServerScenario {
mgr := newTestManager(t)
envoy := NewTestEnvoy(t, proxyID, token)
t.Cleanup(func() {
envoy.Close()
})
s := NewServer(
testutil.Logger(t),
mgr,
resolveToken,
nil, /*checkFetcher HTTPCheckFetcher*/
nil, /*cfgFetcher ConfigFetcher*/
)
s.AuthCheckFrequency = authCheckFrequency
errCh := make(chan error, 1)
go func() {
if incremental {
errCh <- s.DeltaAggregatedResources(envoy.deltaStream)
} else {
shim := &adsServerV2Shim{srv: s}
errCh <- shim.StreamAggregatedResources(envoy.stream)
}
}()
return &testServerScenario{
server: s,
mgr: mgr,
envoy: envoy,
errCh: errCh,
}
}
func protoToSortedJSON(t *testing.T, pb proto.Message) string {
dup, err := copystructure.Copy(pb)
require.NoError(t, err)
pb = dup.(proto.Message)
switch x := pb.(type) {
case *envoy_discovery_v3.DeltaDiscoveryResponse:
sort.Slice(x.Resources, func(i, j int) bool {
return x.Resources[i].Name < x.Resources[j].Name
})
sort.Strings(x.RemovedResources)
}
return protoToJSON(t, pb)
}
func xdsNewEndpoint(ip string, port int) *envoy_endpoint_v3.LbEndpoint {
return &envoy_endpoint_v3.LbEndpoint{
HostIdentifier: &envoy_endpoint_v3.LbEndpoint_Endpoint{
Endpoint: &envoy_endpoint_v3.Endpoint{
Address: makeAddress(ip, port),
},
},
}
}
func xdsNewEndpointWithHealth(ip string, port int, health envoy_core_v3.HealthStatus, weight int) *envoy_endpoint_v3.LbEndpoint {
ep := xdsNewEndpoint(ip, port)
ep.HealthStatus = health
ep.LoadBalancingWeight = makeUint32Value(weight)
return ep
}
func xdsNewADSConfig() *envoy_core_v3.ConfigSource {
return &envoy_core_v3.ConfigSource{
ResourceApiVersion: envoy_core_v3.ApiVersion_V3,
ConfigSourceSpecifier: &envoy_core_v3.ConfigSource_Ads{
Ads: &envoy_core_v3.AggregatedConfigSource{},
},
}
}
func xdsNewPublicTransportSocket(
t *testing.T,
snap *proxycfg.ConfigSnapshot,
) *envoy_core_v3.TransportSocket {
return xdsNewTransportSocket(t, snap, true, true, "")
}
func xdsNewUpstreamTransportSocket(
t *testing.T,
snap *proxycfg.ConfigSnapshot,
sni string,
) *envoy_core_v3.TransportSocket {
return xdsNewTransportSocket(t, snap, false, false, sni)
}
func xdsNewTransportSocket(
t *testing.T,
snap *proxycfg.ConfigSnapshot,
downstream bool,
requireClientCert bool,
sni string,
) *envoy_core_v3.TransportSocket {
// Assume just one root for now, can get fancier later if needed.
caPEM := snap.Roots.Roots[0].RootCert
commonTlsContext := &envoy_tls_v3.CommonTlsContext{
TlsParams: &envoy_tls_v3.TlsParameters{},
TlsCertificates: []*envoy_tls_v3.TlsCertificate{{
CertificateChain: xdsNewInlineString(snap.Leaf().CertPEM),
PrivateKey: xdsNewInlineString(snap.Leaf().PrivateKeyPEM),
}},
ValidationContextType: &envoy_tls_v3.CommonTlsContext_ValidationContext{
ValidationContext: &envoy_tls_v3.CertificateValidationContext{
TrustedCa: xdsNewInlineString(caPEM),
},
},
}
var tlsContext proto.Message
if downstream {
var requireClientCertPB *wrappers.BoolValue
if requireClientCert {
requireClientCertPB = makeBoolValue(true)
}
tlsContext = &envoy_tls_v3.DownstreamTlsContext{
CommonTlsContext: commonTlsContext,
RequireClientCertificate: requireClientCertPB,
}
} else {
tlsContext = &envoy_tls_v3.UpstreamTlsContext{
CommonTlsContext: commonTlsContext,
Sni: sni,
}
}
any, err := ptypes.MarshalAny(tlsContext)
require.NoError(t, err)
return &envoy_core_v3.TransportSocket{
Name: "tls",
ConfigType: &envoy_core_v3.TransportSocket_TypedConfig{
TypedConfig: any,
},
}
}
func xdsNewInlineString(s string) *envoy_core_v3.DataSource {
return &envoy_core_v3.DataSource{
Specifier: &envoy_core_v3.DataSource_InlineString{
InlineString: s,
},
}
}
func xdsNewFilter(t *testing.T, name string, cfg proto.Message) *envoy_listener_v3.Filter {
f, err := makeFilter(name, cfg)
require.NoError(t, err)
return f
}
func mustHashResource(t *testing.T, res proto.Message) string {
v, err := hashResource(res)
require.NoError(t, err)
return v
}
func makeTestResources(t *testing.T, resources ...interface{}) []*envoy_discovery_v3.Resource {
var ret []*envoy_discovery_v3.Resource
for _, res := range resources {
ret = append(ret, makeTestResource(t, res))
}
return ret
}
func makeTestResource(t *testing.T, raw interface{}) *envoy_discovery_v3.Resource {
switch res := raw.(type) {
case string:
return &envoy_discovery_v3.Resource{
Name: res,
}
case proto.Message:
any, err := ptypes.MarshalAny(res)
require.NoError(t, err)
return &envoy_discovery_v3.Resource{
Name: getResourceName(res),
Version: mustHashResource(t, res),
Resource: any,
}
default:
t.Fatalf("unexpected type: %T", res)
return nil // not possible
}
}
func makeTestCluster(t *testing.T, snap *proxycfg.ConfigSnapshot, fixtureName string) *envoy_cluster_v3.Cluster {
switch fixtureName {
case "tcp:local_app":
return &envoy_cluster_v3.Cluster{
Name: "local_app",
ClusterDiscoveryType: &envoy_cluster_v3.Cluster_Type{
Type: envoy_cluster_v3.Cluster_STATIC,
},
ConnectTimeout: ptypes.DurationProto(5 * time.Second),
LoadAssignment: &envoy_endpoint_v3.ClusterLoadAssignment{
ClusterName: "local_app",
Endpoints: []*envoy_endpoint_v3.LocalityLbEndpoints{{
LbEndpoints: []*envoy_endpoint_v3.LbEndpoint{
xdsNewEndpoint("127.0.0.1", 8080),
},
}},
},
}
case "tcp:db":
return &envoy_cluster_v3.Cluster{
Name: "db.default.dc1.internal.11111111-2222-3333-4444-555555555555.consul",
ClusterDiscoveryType: &envoy_cluster_v3.Cluster_Type{
Type: envoy_cluster_v3.Cluster_EDS,
},
EdsClusterConfig: &envoy_cluster_v3.Cluster_EdsClusterConfig{
EdsConfig: xdsNewADSConfig(),
},
CircuitBreakers: &envoy_cluster_v3.CircuitBreakers{},
OutlierDetection: &envoy_cluster_v3.OutlierDetection{},
AltStatName: "db.default.dc1.internal.11111111-2222-3333-4444-555555555555.consul",
CommonLbConfig: &envoy_cluster_v3.Cluster_CommonLbConfig{
HealthyPanicThreshold: &envoy_type_v3.Percent{Value: 0},
},
ConnectTimeout: ptypes.DurationProto(5 * time.Second),
TransportSocket: xdsNewUpstreamTransportSocket(t, snap, "db.default.dc1.internal.11111111-2222-3333-4444-555555555555.consul"),
}
case "http2:db":
return &envoy_cluster_v3.Cluster{
Name: "db.default.dc1.internal.11111111-2222-3333-4444-555555555555.consul",
ClusterDiscoveryType: &envoy_cluster_v3.Cluster_Type{
Type: envoy_cluster_v3.Cluster_EDS,
},
EdsClusterConfig: &envoy_cluster_v3.Cluster_EdsClusterConfig{
EdsConfig: xdsNewADSConfig(),
},
CircuitBreakers: &envoy_cluster_v3.CircuitBreakers{},
OutlierDetection: &envoy_cluster_v3.OutlierDetection{},
AltStatName: "db.default.dc1.internal.11111111-2222-3333-4444-555555555555.consul",
CommonLbConfig: &envoy_cluster_v3.Cluster_CommonLbConfig{
HealthyPanicThreshold: &envoy_type_v3.Percent{Value: 0},
},
ConnectTimeout: ptypes.DurationProto(5 * time.Second),
TransportSocket: xdsNewUpstreamTransportSocket(t, snap, "db.default.dc1.internal.11111111-2222-3333-4444-555555555555.consul"),
Http2ProtocolOptions: &envoy_core_v3.Http2ProtocolOptions{},
}
case "tcp:geo-cache":
return &envoy_cluster_v3.Cluster{
Name: "geo-cache.default.dc1.query.11111111-2222-3333-4444-555555555555.consul",
ClusterDiscoveryType: &envoy_cluster_v3.Cluster_Type{
Type: envoy_cluster_v3.Cluster_EDS,
},
EdsClusterConfig: &envoy_cluster_v3.Cluster_EdsClusterConfig{
EdsConfig: xdsNewADSConfig(),
},
CircuitBreakers: &envoy_cluster_v3.CircuitBreakers{},
OutlierDetection: &envoy_cluster_v3.OutlierDetection{},
ConnectTimeout: ptypes.DurationProto(5 * time.Second),
TransportSocket: xdsNewUpstreamTransportSocket(t, snap, "geo-cache.default.dc1.query.11111111-2222-3333-4444-555555555555.consul"),
}
default:
t.Fatalf("unexpected fixture name: %s", fixtureName)
return nil
}
}
func makeTestEndpoints(t *testing.T, _ *proxycfg.ConfigSnapshot, fixtureName string) *envoy_endpoint_v3.ClusterLoadAssignment {
switch fixtureName {
case "tcp:db":
return &envoy_endpoint_v3.ClusterLoadAssignment{
ClusterName: "db.default.dc1.internal.11111111-2222-3333-4444-555555555555.consul",
Endpoints: []*envoy_endpoint_v3.LocalityLbEndpoints{
{
LbEndpoints: []*envoy_endpoint_v3.LbEndpoint{
xdsNewEndpointWithHealth("10.10.1.1", 8080, envoy_core_v3.HealthStatus_HEALTHY, 1),
xdsNewEndpointWithHealth("10.10.1.2", 8080, envoy_core_v3.HealthStatus_HEALTHY, 1),
},
},
},
}
case "http2:db":
return &envoy_endpoint_v3.ClusterLoadAssignment{
ClusterName: "db.default.dc1.internal.11111111-2222-3333-4444-555555555555.consul",
Endpoints: []*envoy_endpoint_v3.LocalityLbEndpoints{
{
LbEndpoints: []*envoy_endpoint_v3.LbEndpoint{
xdsNewEndpointWithHealth("10.10.1.1", 8080, envoy_core_v3.HealthStatus_HEALTHY, 1),
xdsNewEndpointWithHealth("10.10.1.2", 8080, envoy_core_v3.HealthStatus_HEALTHY, 1),
},
},
},
}
case "tcp:geo-cache":
return &envoy_endpoint_v3.ClusterLoadAssignment{
ClusterName: "geo-cache.default.dc1.query.11111111-2222-3333-4444-555555555555.consul",
Endpoints: []*envoy_endpoint_v3.LocalityLbEndpoints{
{
LbEndpoints: []*envoy_endpoint_v3.LbEndpoint{
xdsNewEndpointWithHealth("10.10.1.1", 8080, envoy_core_v3.HealthStatus_HEALTHY, 1),
xdsNewEndpointWithHealth("10.10.1.2", 8080, envoy_core_v3.HealthStatus_HEALTHY, 1),
},
},
},
}
default:
t.Fatalf("unexpected fixture name: %s", fixtureName)
return nil
}
}
func makeTestListener(t *testing.T, snap *proxycfg.ConfigSnapshot, fixtureName string) *envoy_listener_v3.Listener {
switch fixtureName {
case "tcp:public_listener":
return &envoy_listener_v3.Listener{
Name: "public_listener:0.0.0.0:9999",
Address: makeAddress("0.0.0.0", 9999),
TrafficDirection: envoy_core_v3.TrafficDirection_INBOUND,
FilterChains: []*envoy_listener_v3.FilterChain{
{
TransportSocket: xdsNewPublicTransportSocket(t, snap),
Filters: []*envoy_listener_v3.Filter{
xdsNewFilter(t, "envoy.filters.network.rbac", &envoy_network_rbac_v3.RBAC{
Rules: &envoy_rbac_v3.RBAC{},
StatPrefix: "connect_authz",
}),
xdsNewFilter(t, "envoy.filters.network.tcp_proxy", &envoy_tcp_proxy_v3.TcpProxy{
ClusterSpecifier: &envoy_tcp_proxy_v3.TcpProxy_Cluster{
Cluster: "local_app",
},
StatPrefix: "public_listener",
}),
},
},
},
}
case "tcp:db":
return &envoy_listener_v3.Listener{
Name: "db:127.0.0.1:9191",
Address: makeAddress("127.0.0.1", 9191),
TrafficDirection: envoy_core_v3.TrafficDirection_OUTBOUND,
FilterChains: []*envoy_listener_v3.FilterChain{
{
Filters: []*envoy_listener_v3.Filter{
xdsNewFilter(t, "envoy.filters.network.tcp_proxy", &envoy_tcp_proxy_v3.TcpProxy{
ClusterSpecifier: &envoy_tcp_proxy_v3.TcpProxy_Cluster{
Cluster: "db.default.dc1.internal.11111111-2222-3333-4444-555555555555.consul",
},
StatPrefix: "upstream.db.default.dc1",
}),
},
},
},
}
case "http2:db":
return &envoy_listener_v3.Listener{
Name: "db:127.0.0.1:9191",
Address: makeAddress("127.0.0.1", 9191),
TrafficDirection: envoy_core_v3.TrafficDirection_OUTBOUND,
FilterChains: []*envoy_listener_v3.FilterChain{
{
Filters: []*envoy_listener_v3.Filter{
xdsNewFilter(t, "envoy.filters.network.http_connection_manager", &envoy_http_v3.HttpConnectionManager{
HttpFilters: []*envoy_http_v3.HttpFilter{
{Name: "envoy.filters.http.router"},
},
RouteSpecifier: &envoy_http_v3.HttpConnectionManager_RouteConfig{
RouteConfig: makeTestRoute(t, "http2:db:inline"),
},
StatPrefix: "upstream.db.default.dc1",
Tracing: &envoy_http_v3.HttpConnectionManager_Tracing{
RandomSampling: &envoy_type_v3.Percent{Value: 0},
},
Http2ProtocolOptions: &envoy_core_v3.Http2ProtocolOptions{},
}),
},
},
},
}
case "http2:db:rds":
return &envoy_listener_v3.Listener{
Name: "db:127.0.0.1:9191",
Address: makeAddress("127.0.0.1", 9191),
TrafficDirection: envoy_core_v3.TrafficDirection_OUTBOUND,
FilterChains: []*envoy_listener_v3.FilterChain{
{
Filters: []*envoy_listener_v3.Filter{
xdsNewFilter(t, "envoy.filters.network.http_connection_manager", &envoy_http_v3.HttpConnectionManager{
HttpFilters: []*envoy_http_v3.HttpFilter{
{Name: "envoy.filters.http.router"},
},
RouteSpecifier: &envoy_http_v3.HttpConnectionManager_Rds{
Rds: &envoy_http_v3.Rds{
RouteConfigName: "db",
ConfigSource: xdsNewADSConfig(),
},
},
StatPrefix: "upstream.db.default.dc1",
Tracing: &envoy_http_v3.HttpConnectionManager_Tracing{
RandomSampling: &envoy_type_v3.Percent{Value: 0},
},
Http2ProtocolOptions: &envoy_core_v3.Http2ProtocolOptions{},
}),
},
},
},
}
case "tcp:geo-cache":
return &envoy_listener_v3.Listener{
Name: "prepared_query:geo-cache:127.10.10.10:8181",
Address: makeAddress("127.10.10.10", 8181),
TrafficDirection: envoy_core_v3.TrafficDirection_OUTBOUND,
FilterChains: []*envoy_listener_v3.FilterChain{
{
Filters: []*envoy_listener_v3.Filter{
xdsNewFilter(t, "envoy.filters.network.tcp_proxy", &envoy_tcp_proxy_v3.TcpProxy{
ClusterSpecifier: &envoy_tcp_proxy_v3.TcpProxy_Cluster{
Cluster: "geo-cache.default.dc1.query.11111111-2222-3333-4444-555555555555.consul",
},
StatPrefix: "upstream.prepared_query_geo-cache",
}),
},
},
},
}
default:
t.Fatalf("unexpected fixture name: %s", fixtureName)
return nil
}
}
func makeTestRoute(t *testing.T, fixtureName string) *envoy_route_v3.RouteConfiguration {
switch fixtureName {
case "http2:db":
return &envoy_route_v3.RouteConfiguration{
Name: "db",
ValidateClusters: makeBoolValue(true),
VirtualHosts: []*envoy_route_v3.VirtualHost{
{
Name: "db",
Domains: []string{"*"},
Routes: []*envoy_route_v3.Route{
{
Match: &envoy_route_v3.RouteMatch{
PathSpecifier: &envoy_route_v3.RouteMatch_Prefix{
Prefix: "/",
},
},
Action: &envoy_route_v3.Route_Route{
Route: &envoy_route_v3.RouteAction{
ClusterSpecifier: &envoy_route_v3.RouteAction_Cluster{
Cluster: "db.default.dc1.internal.11111111-2222-3333-4444-555555555555.consul",
},
},
},
},
},
},
},
}
case "http2:db:inline":
return &envoy_route_v3.RouteConfiguration{
Name: "db",
VirtualHosts: []*envoy_route_v3.VirtualHost{
{
Name: "db.default.dc1",
Domains: []string{"*"},
Routes: []*envoy_route_v3.Route{
{
Match: &envoy_route_v3.RouteMatch{
PathSpecifier: &envoy_route_v3.RouteMatch_Prefix{
Prefix: "/",
},
},
Action: &envoy_route_v3.Route_Route{
Route: &envoy_route_v3.RouteAction{
ClusterSpecifier: &envoy_route_v3.RouteAction_Cluster{
Cluster: "db.default.dc1.internal.11111111-2222-3333-4444-555555555555.consul",
},
},
},
},
},
},
},
}
default:
t.Fatalf("unexpected fixture name: %s", fixtureName)
return nil
}
}
func runStep(t *testing.T, name string, fn func(t *testing.T)) {
t.Helper()
if !t.Run(name, fn) {
t.FailNow()
}
}

View File

@ -226,7 +226,7 @@ const bootstrapTemplate = `{
"resource_api_version": "V3"
},
"ads_config": {
"api_type": "GRPC",
"api_type": "DELTA_GRPC",
"transport_api_version": "V3",
"grpc_services": {
"initial_metadata": [

View File

@ -174,7 +174,7 @@
"resource_api_version": "V3"
},
"ads_config": {
"api_type": "GRPC",
"api_type": "DELTA_GRPC",
"transport_api_version": "V3",
"grpc_services": {
"initial_metadata": [

View File

@ -161,7 +161,7 @@
"resource_api_version": "V3"
},
"ads_config": {
"api_type": "GRPC",
"api_type": "DELTA_GRPC",
"transport_api_version": "V3",
"grpc_services": {
"initial_metadata": [

View File

@ -161,7 +161,7 @@
"resource_api_version": "V3"
},
"ads_config": {
"api_type": "GRPC",
"api_type": "DELTA_GRPC",
"transport_api_version": "V3",
"grpc_services": {
"initial_metadata": [

View File

@ -174,7 +174,7 @@
"resource_api_version": "V3"
},
"ads_config": {
"api_type": "GRPC",
"api_type": "DELTA_GRPC",
"transport_api_version": "V3",
"grpc_services": {
"initial_metadata": [

View File

@ -174,7 +174,7 @@
"resource_api_version": "V3"
},
"ads_config": {
"api_type": "GRPC",
"api_type": "DELTA_GRPC",
"transport_api_version": "V3",
"grpc_services": {
"initial_metadata": [

View File

@ -183,7 +183,7 @@
"resource_api_version": "V3"
},
"ads_config": {
"api_type": "GRPC",
"api_type": "DELTA_GRPC",
"transport_api_version": "V3",
"grpc_services": {
"initial_metadata": [

View File

@ -174,7 +174,7 @@
"resource_api_version": "V3"
},
"ads_config": {
"api_type": "GRPC",
"api_type": "DELTA_GRPC",
"transport_api_version": "V3",
"grpc_services": {
"initial_metadata": [

View File

@ -161,7 +161,7 @@
"resource_api_version": "V3"
},
"ads_config": {
"api_type": "GRPC",
"api_type": "DELTA_GRPC",
"transport_api_version": "V3",
"grpc_services": {
"initial_metadata": [

View File

@ -161,7 +161,7 @@
"resource_api_version": "V3"
},
"ads_config": {
"api_type": "GRPC",
"api_type": "DELTA_GRPC",
"transport_api_version": "V3",
"grpc_services": {
"initial_metadata": [

View File

@ -161,7 +161,7 @@
"resource_api_version": "V3"
},
"ads_config": {
"api_type": "GRPC",
"api_type": "DELTA_GRPC",
"transport_api_version": "V3",
"grpc_services": {
"initial_metadata": [

View File

@ -160,7 +160,7 @@
"resource_api_version": "V3"
},
"ads_config": {
"api_type": "GRPC",
"api_type": "DELTA_GRPC",
"transport_api_version": "V3",
"grpc_services": {
"initial_metadata": [

View File

@ -247,7 +247,7 @@
"resource_api_version": "V3"
},
"ads_config": {
"api_type": "GRPC",
"api_type": "DELTA_GRPC",
"transport_api_version": "V3",
"grpc_services": {
"initial_metadata": [

View File

@ -247,7 +247,7 @@
"resource_api_version": "V3"
},
"ads_config": {
"api_type": "GRPC",
"api_type": "DELTA_GRPC",
"transport_api_version": "V3",
"grpc_services": {
"initial_metadata": [

View File

@ -247,7 +247,7 @@
"resource_api_version": "V3"
},
"ads_config": {
"api_type": "GRPC",
"api_type": "DELTA_GRPC",
"transport_api_version": "V3",
"grpc_services": {
"initial_metadata": [

View File

@ -247,7 +247,7 @@
"resource_api_version": "V3"
},
"ads_config": {
"api_type": "GRPC",
"api_type": "DELTA_GRPC",
"transport_api_version": "V3",
"grpc_services": {
"initial_metadata": [

View File

@ -247,7 +247,7 @@
"resource_api_version": "V3"
},
"ads_config": {
"api_type": "GRPC",
"api_type": "DELTA_GRPC",
"transport_api_version": "V3",
"grpc_services": {
"initial_metadata": [

View File

@ -247,7 +247,7 @@
"resource_api_version": "V3"
},
"ads_config": {
"api_type": "GRPC",
"api_type": "DELTA_GRPC",
"transport_api_version": "V3",
"grpc_services": {
"initial_metadata": [

View File

@ -59,7 +59,7 @@
"resource_api_version": "V3"
},
"ads_config": {
"api_type": "GRPC",
"api_type": "DELTA_GRPC",
"transport_api_version": "V3",
"grpc_services": {
"initial_metadata": [

View File

@ -161,7 +161,7 @@
"resource_api_version": "V3"
},
"ads_config": {
"api_type": "GRPC",
"api_type": "DELTA_GRPC",
"transport_api_version": "V3",
"grpc_services": {
"initial_metadata": [

View File

@ -161,7 +161,7 @@
"resource_api_version": "V3"
},
"ads_config": {
"api_type": "GRPC",
"api_type": "DELTA_GRPC",
"transport_api_version": "V3",
"grpc_services": {
"initial_metadata": [

View File

@ -161,7 +161,7 @@
"resource_api_version": "V3"
},
"ads_config": {
"api_type": "GRPC",
"api_type": "DELTA_GRPC",
"transport_api_version": "V3",
"grpc_services": {
"initial_metadata": [

View File

@ -161,7 +161,7 @@
"resource_api_version": "V3"
},
"ads_config": {
"api_type": "GRPC",
"api_type": "DELTA_GRPC",
"transport_api_version": "V3",
"grpc_services": {
"initial_metadata": [

View File

@ -194,7 +194,7 @@
"resource_api_version": "V3"
},
"ads_config": {
"api_type": "GRPC",
"api_type": "DELTA_GRPC",
"transport_api_version": "V3",
"grpc_services": {
"initial_metadata": [

1
go.mod
View File

@ -90,6 +90,7 @@ require (
golang.org/x/tools v0.0.0-20200513154647-78b527d18275 // indirect
google.golang.org/api v0.9.0 // indirect
google.golang.org/appengine v1.6.0 // indirect
google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55
google.golang.org/grpc v1.25.1
gopkg.in/square/go-jose.v2 v2.5.1
k8s.io/api v0.16.9

View File

@ -24,6 +24,7 @@ const (
FSM string = "fsm"
GatewayLocator string = "gateway_locator"
HTTP string = "http"
IngressGateway string = "ingress_gateway"
Intentions string = "intentions"
Internal string = "internal"
KV string = "kvs"

View File

@ -30,6 +30,10 @@ load helpers
assert_service_has_healthy_instances s2 1 secondary
}
@test "gateway-secondary should be up and listening" {
retry_long nc -z consul-secondary:4432
}
################
# PHASE 1: we show that by default requests are served from the primary
@ -71,6 +75,6 @@ load helpers
assert_expected_fortio_name s2-secondary
}
@test "s1 upstream made 1 connection to s2" {
@test "s1 upstream made 1 connection again" {
assert_envoy_metric_at_least 127.0.0.1:19000 "cluster.s2.default.primary.*cx_total" 1
}

View File

@ -30,6 +30,14 @@ load helpers
assert_service_has_healthy_instances s2 1 secondary
}
@test "gateway-secondary should be up and listening" {
retry_long nc -z consul-secondary:4432
}
@test "wait until the first cluster is configured" {
assert_envoy_dynamic_cluster_exists 127.0.0.1:19000 s2.default.primary s2.default.primary
}
################
# PHASE 1: we show that by default requests are served from the primary
@ -60,6 +68,10 @@ load helpers
assert_service_has_healthy_instances s2 0 primary
}
@test "wait until the failover cluster is configured" {
assert_envoy_dynamic_cluster_exists 127.0.0.1:19000 s2.default.primary s2.default.secondary
}
@test "s1 upstream should have healthy endpoints for s2 secondary" {
# in mesh gateway remote or local mode only the current leg of failover manifests in the load assignments
assert_upstream_has_endpoints_in_status 127.0.0.1:19000 s2.default.primary HEALTHY 1

View File

@ -0,0 +1,3 @@
#!/bin/bash
snapshot_envoy_admin localhost:19000 mesh-gateway primary || true

View File

@ -0,0 +1,4 @@
#!/bin/bash
snapshot_envoy_admin localhost:19000 mesh-gateway primary || true
snapshot_envoy_admin localhost:19001 mesh-gateway secondary || true

View File

@ -10,6 +10,18 @@ load helpers
assert_upstream_has_endpoints_in_status 127.0.0.1:19000 secondary HEALTHY 1
}
@test "gateway-primary should have healthy endpoints for secondary servers" {
assert_upstream_has_endpoints_in_status 127.0.0.1:19000 server.secondary.consul HEALTHY 1
}
@test "gateway-primary should have healthy endpoints for lone primary server" {
assert_upstream_has_endpoints_in_status 127.0.0.1:19000 pri.server.primary.consul HEALTHY 1
}
@test "gateway-secondary should be up and listening" {
retry_long nc -z consul-secondary:4432
}
@test "primary should be able to rpc to the secondary" {
retry_default curl -sL -f -XPUT localhost:8500/v1/kv/foo?dc=secondary -d'{"Value":"bar"}'
}

View File

@ -7,6 +7,7 @@ primary_gateways = [
"consul-primary:4431",
]
primary_gateways_interval = "5s"
retry_interval_wan = "5s"
ca_file = "/workdir/secondary/tls/consul-agent-ca.pem"
cert_file = "/workdir/secondary/tls/secondary-server-consul-0.pem"
key_file = "/workdir/secondary/tls/secondary-server-consul-0-key.pem"

View File

@ -10,6 +10,18 @@ load helpers
assert_upstream_has_endpoints_in_status 127.0.0.1:19001 primary HEALTHY 1
}
@test "gateway-secondary should have healthy endpoints for primary servers" {
assert_upstream_has_endpoints_in_status 127.0.0.1:19001 server.primary.consul HEALTHY 1
}
@test "gateway-secondary should have healthy endpoints for lone secondary server" {
assert_upstream_has_endpoints_in_status 127.0.0.1:19001 sec.server.secondary.consul HEALTHY 1
}
@test "gateway-primary should be up and listening" {
retry_long nc -z consul-primary:4431
}
@test "secondary should be able to rpc to the primary" {
retry_default curl -sL -f -XPUT localhost:8500/v1/kv/oof?dc=primary -d'{"Value":"rab"}'
}

View File

@ -1 +1,2 @@
primary_datacenter = "primary"
log_level = "trace"

View File

@ -231,6 +231,33 @@ function get_envoy_http_filters {
echo "$output" | jq --raw-output '.configs[2].dynamic_listeners[].active_state.listener | "\(.name) \( .filter_chains[0].filters[] | select(.name == "envoy.filters.network.http_connection_manager") | .typed_config.http_filters | map(.name) | join(","))"'
}
function get_envoy_dynamic_cluster_once {
local HOSTPORT=$1
local NAME_PREFIX=$2
run curl -s -f $HOSTPORT/config_dump
[ "$status" -eq 0 ]
echo "$output" | jq --raw-output ".configs[] | select (.[\"@type\"] == \"type.googleapis.com/envoy.admin.v3.ClustersConfigDump\") | .dynamic_active_clusters[] | select(.cluster.name | startswith(\"${NAME_PREFIX}\"))"
}
function assert_envoy_dynamic_cluster_exists_once {
local HOSTPORT=$1
local NAME_PREFIX=$2
local EXPECT_SNI=$3
BODY="$(get_envoy_dynamic_cluster_once $HOSTPORT $NAME_PREFIX)"
[ -n "$BODY" ]
SNI="$(echo "$BODY" | jq --raw-output ".cluster.transport_socket.typed_config.sni | select(. | startswith(\"${EXPECT_SNI}\"))")"
[ -n "$SNI" ]
}
function assert_envoy_dynamic_cluster_exists {
local HOSTPORT=$1
local NAME_PREFIX=$2
local EXPECT_SNI=$3
run retry_long assert_envoy_dynamic_cluster_exists_once $HOSTPORT $NAME_PREFIX $EXPECT_SNI
[ "$status" -eq 0 ]
}
function get_envoy_cluster_config {
local HOSTPORT=$1
local CLUSTER_NAME=$2

View File

@ -9,7 +9,7 @@ readonly HASHICORP_DOCKER_PROXY="docker.mirror.hashicorp.services"
# DEBUG=1 enables set -x for this script so echos every command run
DEBUG=${DEBUG:-}
OLD_XDSV2_AWARE_CONSUL_VERSION="${OLD_XDSV2_AWARE_CONSUL_VERSION:-"${HASHICORP_DOCKER_PROXY}/library/consul:1.9.1"}"
OLD_XDSV2_AWARE_CONSUL_VERSION="${OLD_XDSV2_AWARE_CONSUL_VERSION:-"${HASHICORP_DOCKER_PROXY}/library/consul:1.9.4"}"
export OLD_XDSV2_AWARE_CONSUL_VERSION
# TEST_V2_XDS=1 causes it to do just the 'consul connect envoy' part using