Update xds for transparent proxy

This commit is contained in:
freddygv 2021-03-17 13:40:49 -06:00
parent 3c97e5a777
commit 5b59780431
12 changed files with 1568 additions and 568 deletions

View File

@ -46,36 +46,38 @@ func (s *Server) clustersFromSnapshot(_ connectionInfo, cfgSnap *proxycfg.Config
// clustersFromSnapshot returns the xDS API representation of the "clusters"
// (upstreams) in the snapshot.
func (s *Server) clustersFromSnapshotConnectProxy(cfgSnap *proxycfg.ConfigSnapshot) ([]proto.Message, error) {
// TODO(rb): this sizing is a low bound.
clusters := make([]proto.Message, 0, len(cfgSnap.Proxy.Upstreams)+1)
// This sizing is a lower bound.
clusters := make([]proto.Message, 0, len(cfgSnap.ConnectProxy.DiscoveryChain)+1)
// Include the "app" cluster for the public listener
appCluster, err := s.makeAppCluster(cfgSnap, LocalAppClusterName, "", cfgSnap.Proxy.LocalServicePort)
if err != nil {
return nil, err
}
clusters = append(clusters, appCluster)
for _, u := range cfgSnap.Proxy.Upstreams {
id := u.Identifier()
if u.DestinationType == structs.UpstreamDestTypePreparedQuery {
upstreamCluster, err := s.makeUpstreamClusterForPreparedQuery(u, cfgSnap)
if err != nil {
return nil, err
// In TransparentProxy mode there needs to be a passthrough cluster for traffic going to destinations
// that aren't in Consul's catalog.
// TODO (freddy): Add cluster-wide setting that can disable this cluster and restrict traffic to catalog destinations.
if cfgSnap.Proxy.TransparentProxy {
clusters = append(clusters, &envoy_cluster_v3.Cluster{
Name: OriginalDestinationClusterName,
ClusterDiscoveryType: &envoy_cluster_v3.Cluster_Type{
Type: envoy_cluster_v3.Cluster_ORIGINAL_DST,
},
LbPolicy: envoy_cluster_v3.Cluster_CLUSTER_PROVIDED,
ConnectTimeout: ptypes.DurationProto(5 * time.Second),
})
}
clusters = append(clusters, upstreamCluster)
} else {
chain := cfgSnap.ConnectProxy.DiscoveryChain[id]
for id, chain := range cfgSnap.ConnectProxy.DiscoveryChain {
chainEndpoints, ok := cfgSnap.ConnectProxy.WatchedUpstreamEndpoints[id]
if !ok {
// this should not happen
return nil, fmt.Errorf("no endpoint map for upstream %q", id)
}
upstreamClusters, err := s.makeUpstreamClustersForDiscoveryChain(u, chain, chainEndpoints, cfgSnap)
upstreamClusters, err := s.makeUpstreamClustersForDiscoveryChain(id, cfgSnap.ConnectProxy.UpstreamConfig[id], chain, chainEndpoints, cfgSnap)
if err != nil {
return nil, err
}
@ -84,6 +86,17 @@ func (s *Server) clustersFromSnapshotConnectProxy(cfgSnap *proxycfg.ConfigSnapsh
clusters = append(clusters, cluster)
}
}
for _, u := range cfgSnap.Proxy.Upstreams {
if u.DestinationType != structs.UpstreamDestTypePreparedQuery {
continue
}
upstreamCluster, err := s.makeUpstreamClusterForPreparedQuery(u, cfgSnap)
if err != nil {
return nil, err
}
clusters = append(clusters, upstreamCluster)
}
cfgSnap.Proxy.Expose.Finalize()
@ -316,7 +329,7 @@ func (s *Server) clustersFromSnapshotIngressGateway(cfgSnap *proxycfg.ConfigSnap
return nil, fmt.Errorf("no endpoint map for upstream %q", id)
}
upstreamClusters, err := s.makeUpstreamClustersForDiscoveryChain(u, chain, chainEndpoints, cfgSnap)
upstreamClusters, err := s.makeUpstreamClustersForDiscoveryChain(id, &u, chain, chainEndpoints, cfgSnap)
if err != nil {
return nil, err
}
@ -439,16 +452,21 @@ func (s *Server) makeUpstreamClusterForPreparedQuery(upstream structs.Upstream,
}
func (s *Server) makeUpstreamClustersForDiscoveryChain(
upstream structs.Upstream,
id string,
upstream *structs.Upstream,
chain *structs.CompiledDiscoveryChain,
chainEndpoints map[string]structs.CheckServiceNodes,
cfgSnap *proxycfg.ConfigSnapshot,
) ([]*envoy_cluster_v3.Cluster, error) {
if chain == nil {
return nil, fmt.Errorf("cannot create upstream cluster without discovery chain for %s", upstream.Identifier())
return nil, fmt.Errorf("cannot create upstream cluster without discovery chain for %s", id)
}
cfg, err := structs.ParseUpstreamConfigNoDefaults(upstream.Config)
configMap := make(map[string]interface{})
if upstream != nil {
configMap = upstream.Config
}
cfg, err := structs.ParseUpstreamConfigNoDefaults(configMap)
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.

View File

@ -67,6 +67,15 @@ func TestClustersFromSnapshot(t *testing.T) {
customAppClusterJSON(t, customClusterJSONOptions{
Name: "myservice",
})
snap.ConnectProxy.UpstreamConfig = map[string]*structs.Upstream{
"db": {
Config: map[string]interface{}{
"envoy_cluster_json": customAppClusterJSON(t, customClusterJSONOptions{
Name: "myservice",
}),
},
},
}
},
},
{
@ -616,6 +625,13 @@ func TestClustersFromSnapshot(t *testing.T) {
create: proxycfg.TestConfigSnapshotIngress_MultipleListenersDuplicateService,
setup: nil,
},
{
name: "transparent-proxy",
create: proxycfg.TestConfigSnapshot,
setup: func(snap *proxycfg.ConfigSnapshot) {
snap.Proxy.TransparentProxy = true
},
},
}
latestEnvoyVersion := proxysupport.EnvoyVersions[0]

View File

@ -47,16 +47,29 @@ func (s *Server) endpointsFromSnapshotConnectProxy(cfgSnap *proxycfg.ConfigSnaps
resources := make([]proto.Message, 0,
len(cfgSnap.ConnectProxy.PreparedQueryEndpoints)+len(cfgSnap.ConnectProxy.WatchedUpstreamEndpoints))
for _, u := range cfgSnap.Proxy.Upstreams {
id := u.Identifier()
var chain *structs.CompiledDiscoveryChain
if u.DestinationType != structs.UpstreamDestTypePreparedQuery {
chain = cfgSnap.ConnectProxy.DiscoveryChain[id]
for id, chain := range cfgSnap.ConnectProxy.DiscoveryChain {
// Prepared queries get handled in the second loop below
if conf, ok := cfgSnap.ConnectProxy.UpstreamConfig[id]; ok && conf.DestinationType == structs.UpstreamDestTypePreparedQuery {
continue
}
if chain == nil {
// We ONLY want this branch for prepared queries.
es := s.endpointsFromDiscoveryChain(
id,
chain,
cfgSnap.Datacenter,
cfgSnap.ConnectProxy.UpstreamConfig[id],
cfgSnap.ConnectProxy.WatchedUpstreamEndpoints[id],
cfgSnap.ConnectProxy.WatchedGatewayEndpoints[id],
)
resources = append(resources, es...)
}
// Looping over explicit upstreams is only needed for prepared queries and escape hatch clusters
for _, u := range cfgSnap.Proxy.Upstreams {
if u.DestinationType != structs.UpstreamDestTypePreparedQuery || u.Config["envoy_cluster_json"] == "" {
continue
}
id := u.Identifier()
dc := u.Datacenter
if dc == "" {
@ -75,18 +88,6 @@ func (s *Server) endpointsFromSnapshotConnectProxy(cfgSnap *proxycfg.ConfigSnaps
)
resources = append(resources, la)
}
} else {
// Newfangled discovery chain plumbing.
es := s.endpointsFromDiscoveryChain(
u,
chain,
cfgSnap.Datacenter,
cfgSnap.ConnectProxy.WatchedUpstreamEndpoints[id],
cfgSnap.ConnectProxy.WatchedGatewayEndpoints[id],
)
resources = append(resources, es...)
}
}
return resources, nil
@ -277,9 +278,10 @@ func (s *Server) endpointsFromSnapshotIngressGateway(cfgSnap *proxycfg.ConfigSna
}
es := s.endpointsFromDiscoveryChain(
u,
id,
cfgSnap.IngressGateway.DiscoveryChain[id],
cfgSnap.Datacenter,
&u,
cfgSnap.IngressGateway.WatchedUpstreamEndpoints[id],
cfgSnap.IngressGateway.WatchedGatewayEndpoints[id],
)
@ -301,9 +303,10 @@ func makeEndpoint(host string, port int) *envoy_endpoint_v3.LbEndpoint {
}
func (s *Server) endpointsFromDiscoveryChain(
upstream structs.Upstream,
id string,
chain *structs.CompiledDiscoveryChain,
datacenter string,
upstream *structs.Upstream,
upstreamEndpoints, gatewayEndpoints map[string]structs.CheckServiceNodes,
) []proto.Message {
var resources []proto.Message
@ -312,11 +315,15 @@ func (s *Server) endpointsFromDiscoveryChain(
return resources
}
cfg, err := structs.ParseUpstreamConfigNoDefaults(upstream.Config)
configMap := make(map[string]interface{})
if upstream != nil {
configMap = upstream.Config
}
cfg, err := structs.ParseUpstreamConfigNoDefaults(configMap)
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.Warn("failed to parse", "upstream", upstream.Identifier(),
s.Logger.Warn("failed to parse", "upstream", id,
"error", err)
}
@ -331,7 +338,7 @@ func (s *Server) endpointsFromDiscoveryChain(
}
} else {
s.Logger.Warn("ignoring escape hatch setting, because a discovery chain is configued for",
"discovery chain", chain.ServiceName, "upstream", upstream.Identifier(),
"discovery chain", chain.ServiceName, "upstream", id,
"envoy_cluster_json", chain.ServiceName)
}
}

View File

@ -317,6 +317,15 @@ func TestEndpointsFromSnapshot(t *testing.T) {
customAppClusterJSON(t, customClusterJSONOptions{
Name: "myservice",
})
snap.ConnectProxy.UpstreamConfig = map[string]*structs.Upstream{
"db": {
Config: map[string]interface{}{
"envoy_cluster_json": customAppClusterJSON(t, customClusterJSONOptions{
Name: "myservice",
}),
},
},
}
},
},
{

View File

@ -32,6 +32,10 @@ import (
"github.com/hashicorp/consul/logging"
)
const (
TProxyOutboundPort = 15001
)
// listenersFromSnapshot returns the xDS API representation of the "listeners" in the snapshot.
func (s *Server) listenersFromSnapshot(cInfo connectionInfo, cfgSnap *proxycfg.ConfigSnapshot) ([]proto.Message, error) {
if cfgSnap == nil {
@ -54,8 +58,7 @@ func (s *Server) listenersFromSnapshot(cInfo connectionInfo, cfgSnap *proxycfg.C
// listenersFromSnapshotConnectProxy returns the "listeners" for a connect proxy service
func (s *Server) listenersFromSnapshotConnectProxy(cInfo connectionInfo, cfgSnap *proxycfg.ConfigSnapshot) ([]proto.Message, error) {
// One listener for each upstream plus the public one
resources := make([]proto.Message, len(cfgSnap.Proxy.Upstreams)+1)
resources := make([]proto.Message, 1)
var err error
@ -64,18 +67,54 @@ func (s *Server) listenersFromSnapshotConnectProxy(cInfo connectionInfo, cfgSnap
if err != nil {
return nil, err
}
for i, u := range cfgSnap.Proxy.Upstreams {
id := u.Identifier()
var chain *structs.CompiledDiscoveryChain
if u.DestinationType != structs.UpstreamDestTypePreparedQuery {
chain = cfgSnap.ConnectProxy.DiscoveryChain[id]
// This outboundListener is exclusively used when TransparentProxy mode is active.
// In that situation there is a single listener where we are redirecting outbound traffic,
// and each upstream gets a filter chain attached to that listener.
var outboundListener *envoy_listener_v3.Listener
if cfgSnap.Proxy.TransparentProxy {
outboundListener = makeListener(OutboundListenerName, "127.0.0.1", TProxyOutboundPort, envoy_core_v3.TrafficDirection_OUTBOUND)
outboundListener.FilterChains = make([]*envoy_listener_v3.FilterChain, 0)
outboundListener.ListenerFilters = []*envoy_listener_v3.ListenerFilter{
{
// TODO (freddy): Hard-coded until we upgrade the go-control-plane library
Name: "envoy.filters.listener.original_dst",
},
}
}
var upstreamListener proto.Message
upstreamListener, err = s.makeUpstreamListenerForDiscoveryChain(
&u,
u.LocalBindAddress,
var hasChains bool
for id, chain := range cfgSnap.ConnectProxy.DiscoveryChain {
upstreamCfg := cfgSnap.ConnectProxy.UpstreamConfig[id]
if upstreamCfg != nil && upstreamCfg.DestinationType == structs.UpstreamDestTypePreparedQuery {
continue
}
cfg := getAndModifyUpstreamConfigForListener(s.Logger, id, upstreamCfg, chain)
// If escape hatch is present, create a listener from it and move on to the next
if cfg.ListenerJSON != "" {
upstreamListener, err := makeListenerFromUserConfig(cfg.ListenerJSON)
if err != nil {
return nil, err
}
resources = append(resources, upstreamListener)
continue
}
// Generate the upstream listeners for when they are explicitly set with a local bind port
if outboundListener == nil || (upstreamCfg != nil && upstreamCfg.LocalBindPort != 0) {
address := "127.0.0.1"
if upstreamCfg.LocalBindAddress != "" {
address = upstreamCfg.LocalBindAddress
}
filterChain, err := s.makeUpstreamFilterChainForDiscoveryChain(
id,
"",
cfg.Protocol,
upstreamCfg,
chain,
cfgSnap,
nil,
@ -83,7 +122,164 @@ func (s *Server) listenersFromSnapshotConnectProxy(cInfo connectionInfo, cfgSnap
if err != nil {
return nil, err
}
resources[i+1] = upstreamListener
upstreamListener := makeListener(id, address, upstreamCfg.LocalBindPort, envoy_core_v3.TrafficDirection_OUTBOUND)
upstreamListener.FilterChains = []*envoy_listener_v3.FilterChain{
filterChain,
}
resources = append(resources, upstreamListener)
// Avoid creating filter chains below for upstreams that have dedicated listeners
continue
}
filterChain, err := s.makeUpstreamFilterChainForDiscoveryChain(
id,
"",
cfg.Protocol,
upstreamCfg,
chain,
cfgSnap,
nil,
)
if err != nil {
return nil, err
}
// For filter chains used by the transparent proxy, we need to match on multiple destination addresses.
// These might be: the ClusterIP in k8s, or any of the service instance addresses.
endpoints := cfgSnap.ConnectProxy.WatchedUpstreamEndpoints[id]
uniqueAddrs := make(map[string]struct{})
for _, t := range chain.Targets {
var k8sNamespace string
// Store all the IP addresses per unique port
for _, e := range endpoints[t.ID] {
addr, _ := e.BestAddress(false)
if _, ok := uniqueAddrs[addr]; !ok {
uniqueAddrs[addr] = struct{}{}
}
// The k8s namespace should be the same for all instances, so pick any
if ns, ok := e.Service.Meta["k8s-namespace"]; ok {
k8sNamespace = ns
}
}
// TODO (freddy) hack to remove for beta: for every potential discovery chain target, resolve the k8s ClusterIP
// since it's not stored in Consul's catalog (yet)
if k8sNamespace != "" {
host := fmt.Sprintf("%s.%s.svc.cluster.local", t.Service, k8sNamespace)
resolved, err := net.LookupHost(host)
if err != nil {
// We still have the Pod ips in the catalog, so don't hard-fail on errors
s.Logger.Warn("failed to resolve", "host", host, "error", err)
continue
}
for _, addr := range resolved {
if _, ok := uniqueAddrs[addr]; !ok {
uniqueAddrs[addr] = struct{}{}
}
}
}
}
// For every potential address we collected, create the appropriate address prefix to match on.
// In this case we are matching on exact addresses, so the prefix is the address itself,
// and the prefix length is based on whether it's IPv4 or IPv6.
ranges := make([]*envoy_core_v3.CidrRange, 0)
for addr := range uniqueAddrs {
ip := net.ParseIP(addr)
if ip == nil {
continue
}
pfxLen := uint32(32)
if ip.To4() == nil {
pfxLen = 128
}
ranges = append(ranges, &envoy_core_v3.CidrRange{
AddressPrefix: addr,
PrefixLen: &wrappers.UInt32Value{Value: pfxLen},
})
}
filterChain.FilterChainMatch = &envoy_listener_v3.FilterChainMatch{
PrefixRanges: ranges,
}
// Only attach the filter chain if there are addresses to match on
if len(ranges) > 0 {
outboundListener.FilterChains = append(outboundListener.FilterChains, filterChain)
}
hasChains = true
}
// Only create the outbound listener when there are upstreams and filter chains are present
if outboundListener != nil && hasChains {
// Filter chains are stable sorted to avoid draining if the list is provided out of order
sort.SliceStable(outboundListener.FilterChains, func(i, j int) bool {
return outboundListener.FilterChains[i].Name < outboundListener.FilterChains[j].Name
})
// Add a catch-all filter chain that acts as a TCP proxy to non-catalog destinations
filterChain, err := s.makeUpstreamFilterChainForDiscoveryChain(
"passthrough",
OriginalDestinationClusterName,
"tcp",
nil,
nil,
cfgSnap,
nil,
)
if err != nil {
return nil, err
}
outboundListener.FilterChains = append(outboundListener.FilterChains, filterChain)
resources = append(resources, outboundListener)
}
for id, u := range cfgSnap.ConnectProxy.UpstreamConfig {
if _, ok := cfgSnap.ConnectProxy.DiscoveryChain[id]; ok && u.DestinationType != structs.UpstreamDestTypePreparedQuery {
// This upstream is already covered above
continue
}
cfg, err := structs.ParseUpstreamConfig(u.Config)
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.Warn("failed to parse", "upstream", u.Identifier(), "error", err)
}
address := "127.0.0.1"
if u.LocalBindAddress != "" {
address = u.LocalBindAddress
}
// This is the case where upstream config is centralized but no port was specified
if u.LocalBindPort == 0 {
continue
}
upstreamListener := makeListener(id, address, u.LocalBindPort, envoy_core_v3.TrafficDirection_OUTBOUND)
filterChain, err := s.makeUpstreamFilterChainForDiscoveryChain(
id,
"",
cfg.Protocol,
u,
nil,
cfgSnap,
nil,
)
if err != nil {
return nil, err
}
upstreamListener.FilterChains = []*envoy_listener_v3.FilterChain{
filterChain,
}
resources = append(resources, upstreamListener)
}
cfgSnap.Proxy.Expose.Finalize()
@ -532,25 +728,27 @@ func (s *Server) makeInboundListener(cInfo connectionInfo, cfgSnap *proxycfg.Con
return l, nil
}
// No user config, use default listener
// No user config, use default listener address
// Default to listening on all addresses, but override with bind address if one is set.
addr := cfgSnap.Address
// Override with bind address if one is set, otherwise default
// to 0.0.0.0
if addr == "" {
addr = "0.0.0.0"
}
if cfg.BindAddress != "" {
addr = cfg.BindAddress
} else if addr == "" {
addr = "0.0.0.0"
}
// Override with bind port if one is set, otherwise default to
// proxy service's address
port := cfgSnap.Port
// if cfgSnap.Proxy.TransparentProxy {
// port = TProxyInboundPort
// }
if cfg.BindPort != 0 {
port = cfg.BindPort
}
l = makeListener(PublicListenerName, addr, port, envoy_core_v3.TrafficDirection_INBOUND)
l = makeListener(name, addr, port, envoy_core_v3.TrafficDirection_INBOUND)
filterOpts := listenerFilterOpts{
protocol: cfg.Protocol,
@ -994,6 +1192,109 @@ func (s *Server) makeMeshGatewayListener(name, addr string, port int, cfgSnap *p
return l, nil
}
func (s *Server) makeUpstreamFilterChainForDiscoveryChain(
id string,
overrideCluster string,
protocol string,
u *structs.Upstream,
chain *structs.CompiledDiscoveryChain,
cfgSnap *proxycfg.ConfigSnapshot,
tlsContext *envoy_tls_v3.DownstreamTlsContext,
) (*envoy_listener_v3.FilterChain, error) {
// TODO (freddy) Make this actually legible
useRDS := true
var (
clusterName string
destination, datacenter, namespace string
)
if chain != nil {
destination, datacenter, namespace = chain.ServiceName, chain.Datacenter, chain.Namespace
}
if (chain == nil || chain.IsDefault()) && u != nil {
useRDS = false
if datacenter == "" {
datacenter = u.Datacenter
}
if datacenter == "" {
datacenter = cfgSnap.Datacenter
}
if destination == "" {
destination = u.DestinationName
}
if namespace == "" {
namespace = u.DestinationNamespace
}
sni := connect.UpstreamSNI(u, "", datacenter, cfgSnap.Roots.TrustDomain)
clusterName = CustomizeClusterName(sni, chain)
} else {
if protocol == "tcp" && chain != nil {
useRDS = false
startNode := chain.Nodes[chain.StartNode]
if startNode == nil {
return nil, fmt.Errorf("missing first node in compiled discovery chain for: %s", chain.ServiceName)
}
if startNode.Type != structs.DiscoveryGraphNodeTypeResolver {
return nil, fmt.Errorf("unexpected first node in discovery chain using protocol=%q: %s", protocol, startNode.Type)
}
targetID := startNode.Resolver.Target
target := chain.Targets[targetID]
clusterName = CustomizeClusterName(target.Name, chain)
}
}
// Default the namespace to match how SNIs are generated
if namespace == "" {
namespace = structs.IntentionDefaultNamespace
}
filterName := fmt.Sprintf("%s.%s.%s", destination, namespace, datacenter)
if u != nil && u.DestinationType == structs.UpstreamDestTypePreparedQuery {
// Avoid encoding dc and namespace for prepared queries.
// Those are defined in the query itself and are not available here.
filterName = id
}
if overrideCluster != "" {
useRDS = false
clusterName = overrideCluster
filterName = overrideCluster
}
opts := listenerFilterOpts{
useRDS: useRDS,
protocol: protocol,
filterName: filterName,
routeName: id,
cluster: clusterName,
statPrefix: "upstream.",
routePath: "",
ingressGateway: false,
httpAuthzFilter: nil,
}
filter, err := makeListenerFilter(opts)
if err != nil {
return nil, err
}
transportSocket, err := makeDownstreamTLSTransportSocket(tlsContext)
if err != nil {
return nil, err
}
return &envoy_listener_v3.FilterChain{
Filters: []*envoy_listener_v3.Filter{
filter,
},
TransportSocket: transportSocket,
}, nil
}
// TODO(freddy) Replace in favor of new function above. Currently in use for ingress gateways.
func (s *Server) makeUpstreamListenerForDiscoveryChain(
u *structs.Upstream,
address string,
@ -1007,7 +1308,7 @@ func (s *Server) makeUpstreamListenerForDiscoveryChain(
upstreamID := u.Identifier()
l := makeListener(upstreamID, address, u.LocalBindPort, envoy_core_v3.TrafficDirection_OUTBOUND)
cfg := getAndModifyUpstreamConfigForListener(s.Logger, u, chain)
cfg := getAndModifyUpstreamConfigForListener(s.Logger, upstreamID, u, chain)
if cfg.ListenerJSON != "" {
return makeListenerFromUserConfig(cfg.ListenerJSON)
}
@ -1092,48 +1393,51 @@ func (s *Server) makeUpstreamListenerForDiscoveryChain(
return l, nil
}
func getAndModifyUpstreamConfigForListener(logger hclog.Logger, u *structs.Upstream, chain *structs.CompiledDiscoveryChain) structs.UpstreamConfig {
func getAndModifyUpstreamConfigForListener(logger hclog.Logger, id string, u *structs.Upstream, chain *structs.CompiledDiscoveryChain) structs.UpstreamConfig {
var (
cfg structs.UpstreamConfig
err error
)
configMap := make(map[string]interface{})
if u != nil {
configMap = u.Config
}
if chain == nil || chain.IsDefault() {
cfg, err = structs.ParseUpstreamConfig(u.Config)
cfg, err = structs.ParseUpstreamConfig(configMap)
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", u.Identifier(), "error", err)
logger.Warn("failed to parse", "upstream", id, "error", err)
}
} else {
// Use NoDefaults here so that we can set the protocol to the chain
// protocol if necessary
cfg, err = structs.ParseUpstreamConfigNoDefaults(u.Config)
cfg, err = structs.ParseUpstreamConfigNoDefaults(configMap)
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", u.Identifier(), "error", err)
logger.Warn("failed to parse", "upstream", id, "error", err)
}
if cfg.ListenerJSON != "" {
logger.Warn("ignoring escape hatch setting because already configured for",
"discovery chain", chain.ServiceName, "upstream", u.Identifier(), "config", "envoy_listener_json")
"discovery chain", chain.ServiceName, "upstream", id, "config", "envoy_listener_json")
// Remove from config struct so we don't use it later on
cfg.ListenerJSON = ""
}
proto := cfg.Protocol
if proto == "" {
proto = chain.Protocol
protocol := cfg.Protocol
if protocol == "" {
protocol = chain.Protocol
}
if proto == "" {
proto = "tcp"
if protocol == "" {
protocol = "tcp"
}
// set back on the config so that we can use it from return value
cfg.Protocol = proto
cfg.Protocol = protocol
}
return cfg
@ -1148,6 +1452,7 @@ type listenerFilterOpts struct {
statPrefix string
routePath string
requestTimeoutMs *int
ingressGateway bool
httpAuthzFilter *envoy_http_v3.HttpFilter
}

View File

@ -2,22 +2,19 @@ package xds
import (
"bytes"
"path/filepath"
"sort"
"testing"
"text/template"
"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"
"github.com/hashicorp/consul/agent/connect"
"github.com/hashicorp/consul/agent/consul/discoverychain"
"github.com/hashicorp/consul/agent/proxycfg"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/agent/xds/proxysupport"
"github.com/hashicorp/consul/sdk/testutil"
"github.com/hashicorp/consul/types"
testinf "github.com/mitchellh/go-testing-interface"
"github.com/stretchr/testify/require"
"path/filepath"
"sort"
"testing"
"text/template"
)
func TestListenersFromSnapshot(t *testing.T) {
@ -35,447 +32,478 @@ func TestListenersFromSnapshot(t *testing.T) {
overrideGoldenName string
serverSetup func(*Server)
}{
// {
// name: "defaults",
// create: proxycfg.TestConfigSnapshot,
// setup: nil, // Default snapshot
// },
// {
// name: "listener-bind-address",
// create: proxycfg.TestConfigSnapshot,
// setup: func(snap *proxycfg.ConfigSnapshot) {
// snap.Proxy.Config["bind_address"] = "127.0.0.2"
// },
// },
// {
// name: "listener-bind-port",
// create: proxycfg.TestConfigSnapshot,
// setup: func(snap *proxycfg.ConfigSnapshot) {
// snap.Proxy.Config["bind_port"] = 8888
// },
// },
// {
// name: "listener-bind-address-port",
// create: proxycfg.TestConfigSnapshot,
// setup: func(snap *proxycfg.ConfigSnapshot) {
// snap.Proxy.Config["bind_address"] = "127.0.0.2"
// snap.Proxy.Config["bind_port"] = 8888
// },
// },
// {
// name: "http-public-listener",
// create: proxycfg.TestConfigSnapshot,
// setup: func(snap *proxycfg.ConfigSnapshot) {
// snap.Proxy.Config["protocol"] = "http"
// },
// },
// {
// name: "http-listener-with-timeouts",
// create: proxycfg.TestConfigSnapshot,
// setup: func(snap *proxycfg.ConfigSnapshot) {
// snap.Proxy.Config["protocol"] = "http"
// snap.Proxy.Config["local_connect_timeout_ms"] = 1234
// snap.Proxy.Config["local_request_timeout_ms"] = 2345
// },
// },
// {
// name: "http-upstream",
// create: proxycfg.TestConfigSnapshot,
// setup: func(snap *proxycfg.ConfigSnapshot) {
// snap.Proxy.Upstreams[0].Config["protocol"] = "http"
// },
// },
// {
// name: "custom-public-listener",
// create: proxycfg.TestConfigSnapshot,
// setup: func(snap *proxycfg.ConfigSnapshot) {
// snap.Proxy.Config["envoy_public_listener_json"] =
// customListenerJSON(t, customListenerJSONOptions{
// Name: "custom-public-listen",
// })
// },
// },
// {
// name: "custom-public-listener-http",
// create: proxycfg.TestConfigSnapshot,
// setup: func(snap *proxycfg.ConfigSnapshot) {
// snap.Proxy.Config["protocol"] = "http"
// snap.Proxy.Config["envoy_public_listener_json"] =
// customHTTPListenerJSON(t, customHTTPListenerJSONOptions{
// Name: "custom-public-listen",
// })
// },
// },
// {
// name: "custom-public-listener-http-2",
// create: proxycfg.TestConfigSnapshot,
// setup: func(snap *proxycfg.ConfigSnapshot) {
// snap.Proxy.Config["protocol"] = "http"
// snap.Proxy.Config["envoy_public_listener_json"] =
// customHTTPListenerJSON(t, customHTTPListenerJSONOptions{
// Name: "custom-public-listen",
// HTTPConnectionManagerName: httpConnectionManagerNewName,
// })
// },
// },
// {
// name: "custom-public-listener-http-missing",
// create: proxycfg.TestConfigSnapshot,
// setup: func(snap *proxycfg.ConfigSnapshot) {
// snap.Proxy.Config["protocol"] = "http"
// snap.Proxy.Config["envoy_public_listener_json"] =
// customListenerJSON(t, customListenerJSONOptions{
// Name: "custom-public-listen",
// })
// },
// },
// {
// name: "custom-public-listener-ignores-tls",
// create: proxycfg.TestConfigSnapshot,
// overrideGoldenName: "custom-public-listener", // should be the same
// setup: func(snap *proxycfg.ConfigSnapshot) {
// snap.Proxy.Config["envoy_public_listener_json"] =
// customListenerJSON(t, customListenerJSONOptions{
// Name: "custom-public-listen",
// // Attempt to override the TLS context should be ignored
// TLSContext: `"allowRenegotiation": false`,
// })
// },
// },
// {
// name: "custom-upstream",
// create: proxycfg.TestConfigSnapshot,
// setup: func(snap *proxycfg.ConfigSnapshot) {
// snap.Proxy.Upstreams[0].Config["envoy_listener_json"] =
// customListenerJSON(t, customListenerJSONOptions{
// Name: "custom-upstream",
// })
// },
// },
// {
// name: "custom-upstream-ignored-with-disco-chain",
// create: proxycfg.TestConfigSnapshotDiscoveryChainWithFailover,
// setup: func(snap *proxycfg.ConfigSnapshot) {
// snap.Proxy.Upstreams[0].Config["envoy_listener_json"] =
// customListenerJSON(t, customListenerJSONOptions{
// Name: "custom-upstream",
// })
// },
// },
// {
// name: "splitter-with-resolver-redirect",
// create: proxycfg.TestConfigSnapshotDiscoveryChain_SplitterWithResolverRedirectMultiDC,
// setup: nil,
// },
// {
// name: "connect-proxy-with-tcp-chain",
// create: proxycfg.TestConfigSnapshotDiscoveryChain,
// setup: nil,
// },
// {
// name: "connect-proxy-with-http-chain",
// create: func(t testinf.T) *proxycfg.ConfigSnapshot {
// return proxycfg.TestConfigSnapshotDiscoveryChainWithEntries(t,
// &structs.ProxyConfigEntry{
// Kind: structs.ProxyDefaults,
// Name: structs.ProxyConfigGlobal,
// Config: map[string]interface{}{
// "protocol": "http",
// },
// },
// )
// },
// setup: nil,
// },
// {
// name: "connect-proxy-with-http2-chain",
// create: func(t testinf.T) *proxycfg.ConfigSnapshot {
// return proxycfg.TestConfigSnapshotDiscoveryChainWithEntries(t,
// &structs.ProxyConfigEntry{
// Kind: structs.ProxyDefaults,
// Name: structs.ProxyConfigGlobal,
// Config: map[string]interface{}{
// "protocol": "http2",
// },
// },
// )
// },
// setup: nil,
// },
// {
// name: "connect-proxy-with-grpc-chain",
// create: func(t testinf.T) *proxycfg.ConfigSnapshot {
// return proxycfg.TestConfigSnapshotDiscoveryChainWithEntries(t,
// &structs.ProxyConfigEntry{
// Kind: structs.ProxyDefaults,
// Name: structs.ProxyConfigGlobal,
// Config: map[string]interface{}{
// "protocol": "grpc",
// },
// },
// )
// },
// setup: nil,
// },
// {
// name: "connect-proxy-with-chain-external-sni",
// create: proxycfg.TestConfigSnapshotDiscoveryChainExternalSNI,
// setup: nil,
// },
// {
// name: "connect-proxy-with-chain-and-overrides",
// create: proxycfg.TestConfigSnapshotDiscoveryChainWithOverrides,
// setup: nil,
// },
// {
// name: "connect-proxy-with-tcp-chain-failover-through-remote-gateway",
// create: proxycfg.TestConfigSnapshotDiscoveryChainWithFailoverThroughRemoteGateway,
// setup: nil,
// },
// {
// name: "connect-proxy-with-tcp-chain-failover-through-local-gateway",
// create: proxycfg.TestConfigSnapshotDiscoveryChainWithFailoverThroughLocalGateway,
// setup: nil,
// },
// {
// name: "expose-paths-local-app-paths",
// create: proxycfg.TestConfigSnapshotExposeConfig,
// },
// {
// name: "expose-paths-new-cluster-http2",
// create: proxycfg.TestConfigSnapshotExposeConfig,
// setup: func(snap *proxycfg.ConfigSnapshot) {
// snap.Proxy.Expose.Paths[1] = structs.ExposePath{
// LocalPathPort: 9090,
// Path: "/grpc.health.v1.Health/Check",
// ListenerPort: 21501,
// Protocol: "http2",
// }
// },
// },
// {
// // NOTE: if IPv6 is not supported in the kernel per
// // kernelSupportsIPv6() then this test will fail because the golden
// // files were generated assuming ipv6 support was present
// name: "expose-checks",
// create: proxycfg.TestConfigSnapshotExposeConfig,
// setup: func(snap *proxycfg.ConfigSnapshot) {
// snap.Proxy.Expose = structs.ExposeConfig{
// Checks: true,
// }
// },
// serverSetup: func(s *Server) {
// s.CfgFetcher = configFetcherFunc(func() string {
// return "192.0.2.1"
// })
//
// s.CheckFetcher = httpCheckFetcherFunc(func(sid structs.ServiceID) []structs.CheckType {
// if sid != structs.NewServiceID("web", nil) {
// return nil
// }
// return []structs.CheckType{{
// CheckID: types.CheckID("http"),
// Name: "http",
// HTTP: "http://127.0.0.1:8181/debug",
// ProxyHTTP: "http://:21500/debug",
// Method: "GET",
// Interval: 10 * time.Second,
// Timeout: 1 * time.Second,
// }}
// })
// },
// },
// {
// name: "mesh-gateway",
// create: proxycfg.TestConfigSnapshotMeshGateway,
// },
// {
// name: "mesh-gateway-using-federation-states",
// create: proxycfg.TestConfigSnapshotMeshGatewayUsingFederationStates,
// },
// {
// name: "mesh-gateway-no-services",
// create: proxycfg.TestConfigSnapshotMeshGatewayNoServices,
// },
// {
// name: "mesh-gateway-tagged-addresses",
// create: proxycfg.TestConfigSnapshotMeshGateway,
// setup: func(snap *proxycfg.ConfigSnapshot) {
// snap.Proxy.Config = map[string]interface{}{
// "envoy_mesh_gateway_no_default_bind": true,
// "envoy_mesh_gateway_bind_tagged_addresses": true,
// }
// },
// },
// {
// name: "mesh-gateway-custom-addresses",
// create: proxycfg.TestConfigSnapshotMeshGateway,
// setup: func(snap *proxycfg.ConfigSnapshot) {
// snap.Proxy.Config = map[string]interface{}{
// "envoy_mesh_gateway_bind_addresses": map[string]structs.ServiceAddress{
// "foo": {
// Address: "198.17.2.3",
// Port: 8080,
// },
// "bar": {
// Address: "2001:db8::ff",
// Port: 9999,
// },
// "baz": {
// Address: "127.0.0.1",
// Port: 8765,
// },
// },
// }
// },
// },
// {
// name: "ingress-gateway",
// create: proxycfg.TestConfigSnapshotIngressGateway,
// setup: nil,
// },
// {
// name: "ingress-gateway-bind-addrs",
// create: proxycfg.TestConfigSnapshotIngressGateway,
// setup: func(snap *proxycfg.ConfigSnapshot) {
// snap.TaggedAddresses = map[string]structs.ServiceAddress{
// "lan": {Address: "10.0.0.1"},
// "wan": {Address: "172.16.0.1"},
// }
// snap.Proxy.Config = map[string]interface{}{
// "envoy_gateway_no_default_bind": true,
// "envoy_gateway_bind_tagged_addresses": true,
// "envoy_gateway_bind_addresses": map[string]structs.ServiceAddress{
// "foo": {Address: "8.8.8.8"},
// },
// }
// },
// },
// {
// name: "ingress-gateway-no-services",
// create: proxycfg.TestConfigSnapshotIngressGatewayNoServices,
// setup: nil,
// },
// {
// name: "ingress-with-chain-external-sni",
// create: proxycfg.TestConfigSnapshotIngressExternalSNI,
// setup: nil,
// },
// {
// name: "ingress-with-chain-and-overrides",
// create: proxycfg.TestConfigSnapshotIngressWithOverrides,
// setup: nil,
// },
// {
// name: "ingress-with-tcp-chain-failover-through-remote-gateway",
// create: proxycfg.TestConfigSnapshotIngressWithFailoverThroughRemoteGateway,
// setup: nil,
// },
// {
// name: "ingress-with-tcp-chain-failover-through-local-gateway",
// create: proxycfg.TestConfigSnapshotIngressWithFailoverThroughLocalGateway,
// setup: nil,
// },
// {
// name: "ingress-splitter-with-resolver-redirect",
// create: proxycfg.TestConfigSnapshotIngress_SplitterWithResolverRedirectMultiDC,
// setup: nil,
// },
// {
// name: "terminating-gateway",
// create: proxycfg.TestConfigSnapshotTerminatingGateway,
// setup: nil,
// },
// {
// name: "terminating-gateway-no-services",
// create: proxycfg.TestConfigSnapshotTerminatingGatewayNoServices,
// setup: nil,
// },
// {
// name: "terminating-gateway-custom-and-tagged-addresses",
// create: proxycfg.TestConfigSnapshotTerminatingGateway,
// setup: func(snap *proxycfg.ConfigSnapshot) {
// snap.Proxy.Config = map[string]interface{}{
// "envoy_gateway_no_default_bind": true,
// "envoy_gateway_bind_tagged_addresses": true,
// "envoy_gateway_bind_addresses": map[string]structs.ServiceAddress{
// // This bind address should not get a listener due to deduplication and it sorts to the end
// "z-duplicate-of-tagged-wan-addr": {
// Address: "198.18.0.1",
// Port: 443,
// },
// "foo": {
// Address: "198.17.2.3",
// Port: 8080,
// },
// },
// }
// },
// },
// {
// name: "terminating-gateway-service-subsets",
// create: proxycfg.TestConfigSnapshotTerminatingGateway,
// setup: func(snap *proxycfg.ConfigSnapshot) {
// snap.TerminatingGateway.ServiceResolvers = map[structs.ServiceName]*structs.ServiceResolverConfigEntry{
// structs.NewServiceName("web", nil): {
// Kind: structs.ServiceResolver,
// Name: "web",
// Subsets: map[string]structs.ServiceResolverSubset{
// "v1": {
// Filter: "Service.Meta.version == 1",
// },
// "v2": {
// Filter: "Service.Meta.version == 2",
// OnlyPassing: true,
// },
// },
// },
// }
// snap.TerminatingGateway.ServiceConfigs[structs.NewServiceName("web", nil)] = &structs.ServiceConfigResponse{
// ProxyConfig: map[string]interface{}{"protocol": "http"},
// }
// },
// },
// {
// name: "ingress-http-multiple-services",
// create: proxycfg.TestConfigSnapshotIngress_HTTPMultipleServices,
// setup: func(snap *proxycfg.ConfigSnapshot) {
// snap.IngressGateway.Upstreams = map[proxycfg.IngressListenerKey]structs.Upstreams{
// {Protocol: "http", Port: 8080}: {
// {
// DestinationName: "foo",
// LocalBindPort: 8080,
// },
// {
// DestinationName: "bar",
// LocalBindPort: 8080,
// },
// },
// {Protocol: "http", Port: 443}: {
// {
// DestinationName: "baz",
// LocalBindPort: 443,
// },
// {
// DestinationName: "qux",
// LocalBindPort: 443,
// },
// },
// }
// },
// },
// {
// name: "terminating-gateway-no-api-cert",
// create: proxycfg.TestConfigSnapshotTerminatingGateway,
// setup: func(snap *proxycfg.ConfigSnapshot) {
// snap.TerminatingGateway.ServiceLeaves[structs.NewServiceName("api", nil)] = nil
// },
// },
// {
// name: "ingress-with-tls-listener",
// create: proxycfg.TestConfigSnapshotIngressWithTLSListener,
// setup: nil,
// },
{
name: "defaults",
create: proxycfg.TestConfigSnapshot,
setup: nil, // Default snapshot
},
{
name: "listener-bind-address",
name: "transparent-proxy",
create: proxycfg.TestConfigSnapshot,
setup: func(snap *proxycfg.ConfigSnapshot) {
snap.Proxy.Config["bind_address"] = "127.0.0.2"
},
},
{
name: "listener-bind-port",
create: proxycfg.TestConfigSnapshot,
setup: func(snap *proxycfg.ConfigSnapshot) {
snap.Proxy.Config["bind_port"] = 8888
},
},
{
name: "listener-bind-address-port",
create: proxycfg.TestConfigSnapshot,
setup: func(snap *proxycfg.ConfigSnapshot) {
snap.Proxy.Config["bind_address"] = "127.0.0.2"
snap.Proxy.Config["bind_port"] = 8888
},
},
{
name: "http-public-listener",
create: proxycfg.TestConfigSnapshot,
setup: func(snap *proxycfg.ConfigSnapshot) {
snap.Proxy.Config["protocol"] = "http"
},
},
{
name: "http-listener-with-timeouts",
create: proxycfg.TestConfigSnapshot,
setup: func(snap *proxycfg.ConfigSnapshot) {
snap.Proxy.Config["protocol"] = "http"
snap.Proxy.Config["local_connect_timeout_ms"] = 1234
snap.Proxy.Config["local_request_timeout_ms"] = 2345
},
},
{
name: "http-upstream",
create: proxycfg.TestConfigSnapshot,
setup: func(snap *proxycfg.ConfigSnapshot) {
snap.Proxy.Upstreams[0].Config["protocol"] = "http"
},
},
{
name: "custom-public-listener",
create: proxycfg.TestConfigSnapshot,
setup: func(snap *proxycfg.ConfigSnapshot) {
snap.Proxy.Config["envoy_public_listener_json"] =
customListenerJSON(t, customListenerJSONOptions{
Name: "custom-public-listen",
})
},
},
{
name: "custom-public-listener-http",
create: proxycfg.TestConfigSnapshot,
setup: func(snap *proxycfg.ConfigSnapshot) {
snap.Proxy.Config["protocol"] = "http"
snap.Proxy.Config["envoy_public_listener_json"] =
customHTTPListenerJSON(t, customHTTPListenerJSONOptions{
Name: "custom-public-listen",
})
},
},
{
name: "custom-public-listener-http-2",
create: proxycfg.TestConfigSnapshot,
setup: func(snap *proxycfg.ConfigSnapshot) {
snap.Proxy.Config["protocol"] = "http"
snap.Proxy.Config["envoy_public_listener_json"] =
customHTTPListenerJSON(t, customHTTPListenerJSONOptions{
Name: "custom-public-listen",
HTTPConnectionManagerName: httpConnectionManagerNewName,
})
},
},
{
name: "custom-public-listener-http-missing",
create: proxycfg.TestConfigSnapshot,
setup: func(snap *proxycfg.ConfigSnapshot) {
snap.Proxy.Config["protocol"] = "http"
snap.Proxy.Config["envoy_public_listener_json"] =
customListenerJSON(t, customListenerJSONOptions{
Name: "custom-public-listen",
})
},
},
{
name: "custom-public-listener-ignores-tls",
create: proxycfg.TestConfigSnapshot,
overrideGoldenName: "custom-public-listener", // should be the same
setup: func(snap *proxycfg.ConfigSnapshot) {
snap.Proxy.Config["envoy_public_listener_json"] =
customListenerJSON(t, customListenerJSONOptions{
Name: "custom-public-listen",
// Attempt to override the TLS context should be ignored
TLSContext: `"allowRenegotiation": false`,
})
},
},
{
name: "custom-upstream",
create: proxycfg.TestConfigSnapshot,
setup: func(snap *proxycfg.ConfigSnapshot) {
snap.Proxy.Upstreams[0].Config["envoy_listener_json"] =
customListenerJSON(t, customListenerJSONOptions{
Name: "custom-upstream",
})
},
},
{
name: "custom-upstream-ignored-with-disco-chain",
create: proxycfg.TestConfigSnapshotDiscoveryChainWithFailover,
setup: func(snap *proxycfg.ConfigSnapshot) {
snap.Proxy.Upstreams[0].Config["envoy_listener_json"] =
customListenerJSON(t, customListenerJSONOptions{
Name: "custom-upstream",
})
},
},
{
name: "splitter-with-resolver-redirect",
create: proxycfg.TestConfigSnapshotDiscoveryChain_SplitterWithResolverRedirectMultiDC,
setup: nil,
},
{
name: "connect-proxy-with-tcp-chain",
create: proxycfg.TestConfigSnapshotDiscoveryChain,
setup: nil,
},
{
name: "connect-proxy-with-http-chain",
create: func(t testinf.T) *proxycfg.ConfigSnapshot {
return proxycfg.TestConfigSnapshotDiscoveryChainWithEntries(t,
&structs.ProxyConfigEntry{
Kind: structs.ProxyDefaults,
Name: structs.ProxyConfigGlobal,
Config: map[string]interface{}{
"protocol": "http",
},
},
)
},
setup: nil,
},
{
name: "connect-proxy-with-http2-chain",
create: func(t testinf.T) *proxycfg.ConfigSnapshot {
return proxycfg.TestConfigSnapshotDiscoveryChainWithEntries(t,
&structs.ProxyConfigEntry{
Kind: structs.ProxyDefaults,
Name: structs.ProxyConfigGlobal,
Config: map[string]interface{}{
"protocol": "http2",
},
},
)
},
setup: nil,
},
{
name: "connect-proxy-with-grpc-chain",
create: func(t testinf.T) *proxycfg.ConfigSnapshot {
return proxycfg.TestConfigSnapshotDiscoveryChainWithEntries(t,
&structs.ProxyConfigEntry{
Kind: structs.ProxyDefaults,
Name: structs.ProxyConfigGlobal,
Config: map[string]interface{}{
"protocol": "grpc",
},
},
)
},
setup: nil,
},
{
name: "connect-proxy-with-chain-external-sni",
create: proxycfg.TestConfigSnapshotDiscoveryChainExternalSNI,
setup: nil,
},
{
name: "connect-proxy-with-chain-and-overrides",
create: proxycfg.TestConfigSnapshotDiscoveryChainWithOverrides,
setup: nil,
},
{
name: "connect-proxy-with-tcp-chain-failover-through-remote-gateway",
create: proxycfg.TestConfigSnapshotDiscoveryChainWithFailoverThroughRemoteGateway,
setup: nil,
},
{
name: "connect-proxy-with-tcp-chain-failover-through-local-gateway",
create: proxycfg.TestConfigSnapshotDiscoveryChainWithFailoverThroughLocalGateway,
setup: nil,
},
{
name: "expose-paths-local-app-paths",
create: proxycfg.TestConfigSnapshotExposeConfig,
},
{
name: "expose-paths-new-cluster-http2",
create: proxycfg.TestConfigSnapshotExposeConfig,
setup: func(snap *proxycfg.ConfigSnapshot) {
snap.Proxy.Expose.Paths[1] = structs.ExposePath{
LocalPathPort: 9090,
Path: "/grpc.health.v1.Health/Check",
ListenerPort: 21501,
Protocol: "http2",
}
},
},
{
// NOTE: if IPv6 is not supported in the kernel per
// kernelSupportsIPv6() then this test will fail because the golden
// files were generated assuming ipv6 support was present
name: "expose-checks",
create: proxycfg.TestConfigSnapshotExposeConfig,
setup: func(snap *proxycfg.ConfigSnapshot) {
snap.Proxy.Expose = structs.ExposeConfig{
Checks: true,
}
},
serverSetup: func(s *Server) {
s.CfgFetcher = configFetcherFunc(func() string {
return "192.0.2.1"
})
snap.Proxy.TransparentProxy = true
s.CheckFetcher = httpCheckFetcherFunc(func(sid structs.ServiceID) []structs.CheckType {
if sid != structs.NewServiceID("web", nil) {
return nil
}
return []structs.CheckType{{
CheckID: types.CheckID("http"),
Name: "http",
HTTP: "http://127.0.0.1:8181/debug",
ProxyHTTP: "http://:21500/debug",
Method: "GET",
Interval: 10 * time.Second,
Timeout: 1 * time.Second,
}}
})
// DiscoveryChain without an UpstreamConfig should yield a filter chain when in TransparentProxy mode
snap.ConnectProxy.DiscoveryChain["google"] = discoverychain.TestCompileConfigEntries(
t, "google", "default", "dc1",
connect.TestClusterID+".consul", "dc1", nil)
snap.ConnectProxy.WatchedUpstreamEndpoints["google"] = map[string]structs.CheckServiceNodes{
"google.default.dc1": {
structs.CheckServiceNode{
Node: &structs.Node{
Address: "8.8.8.8",
Datacenter: "dc1",
},
},
{
name: "mesh-gateway",
create: proxycfg.TestConfigSnapshotMeshGateway,
},
{
name: "mesh-gateway-using-federation-states",
create: proxycfg.TestConfigSnapshotMeshGatewayUsingFederationStates,
},
{
name: "mesh-gateway-no-services",
create: proxycfg.TestConfigSnapshotMeshGatewayNoServices,
},
{
name: "mesh-gateway-tagged-addresses",
create: proxycfg.TestConfigSnapshotMeshGateway,
setup: func(snap *proxycfg.ConfigSnapshot) {
snap.Proxy.Config = map[string]interface{}{
"envoy_mesh_gateway_no_default_bind": true,
"envoy_mesh_gateway_bind_tagged_addresses": true,
}
},
},
{
name: "mesh-gateway-custom-addresses",
create: proxycfg.TestConfigSnapshotMeshGateway,
setup: func(snap *proxycfg.ConfigSnapshot) {
snap.Proxy.Config = map[string]interface{}{
"envoy_mesh_gateway_bind_addresses": map[string]structs.ServiceAddress{
"foo": {
Address: "198.17.2.3",
Port: 8080,
},
"bar": {
Address: "2001:db8::ff",
Port: 9999,
},
"baz": {
Address: "127.0.0.1",
Port: 8765,
},
},
}
},
},
{
name: "ingress-gateway",
create: proxycfg.TestConfigSnapshotIngressGateway,
setup: nil,
},
{
name: "ingress-gateway-bind-addrs",
create: proxycfg.TestConfigSnapshotIngressGateway,
setup: func(snap *proxycfg.ConfigSnapshot) {
snap.TaggedAddresses = map[string]structs.ServiceAddress{
"lan": {Address: "10.0.0.1"},
"wan": {Address: "172.16.0.1"},
}
snap.Proxy.Config = map[string]interface{}{
"envoy_gateway_no_default_bind": true,
"envoy_gateway_bind_tagged_addresses": true,
"envoy_gateway_bind_addresses": map[string]structs.ServiceAddress{
"foo": {Address: "8.8.8.8"},
},
}
},
},
{
name: "ingress-gateway-no-services",
create: proxycfg.TestConfigSnapshotIngressGatewayNoServices,
setup: nil,
},
{
name: "ingress-with-chain-external-sni",
create: proxycfg.TestConfigSnapshotIngressExternalSNI,
setup: nil,
},
{
name: "ingress-with-chain-and-overrides",
create: proxycfg.TestConfigSnapshotIngressWithOverrides,
setup: nil,
},
{
name: "ingress-with-tcp-chain-failover-through-remote-gateway",
create: proxycfg.TestConfigSnapshotIngressWithFailoverThroughRemoteGateway,
setup: nil,
},
{
name: "ingress-with-tcp-chain-failover-through-local-gateway",
create: proxycfg.TestConfigSnapshotIngressWithFailoverThroughLocalGateway,
setup: nil,
},
{
name: "ingress-splitter-with-resolver-redirect",
create: proxycfg.TestConfigSnapshotIngress_SplitterWithResolverRedirectMultiDC,
setup: nil,
},
{
name: "terminating-gateway",
create: proxycfg.TestConfigSnapshotTerminatingGateway,
setup: nil,
},
{
name: "terminating-gateway-no-services",
create: proxycfg.TestConfigSnapshotTerminatingGatewayNoServices,
setup: nil,
},
{
name: "terminating-gateway-custom-and-tagged-addresses",
create: proxycfg.TestConfigSnapshotTerminatingGateway,
setup: func(snap *proxycfg.ConfigSnapshot) {
snap.Proxy.Config = map[string]interface{}{
"envoy_gateway_no_default_bind": true,
"envoy_gateway_bind_tagged_addresses": true,
"envoy_gateway_bind_addresses": map[string]structs.ServiceAddress{
// This bind address should not get a listener due to deduplication and it sorts to the end
"z-duplicate-of-tagged-wan-addr": {
Address: "198.18.0.1",
Port: 443,
},
"foo": {
Address: "198.17.2.3",
Port: 8080,
},
},
}
},
},
{
name: "terminating-gateway-service-subsets",
create: proxycfg.TestConfigSnapshotTerminatingGateway,
setup: func(snap *proxycfg.ConfigSnapshot) {
snap.TerminatingGateway.ServiceResolvers = map[structs.ServiceName]*structs.ServiceResolverConfigEntry{
structs.NewServiceName("web", nil): {
Kind: structs.ServiceResolver,
Name: "web",
Subsets: map[string]structs.ServiceResolverSubset{
"v1": {
Filter: "Service.Meta.version == 1",
},
"v2": {
Filter: "Service.Meta.version == 2",
OnlyPassing: true,
Service: &structs.NodeService{
Service: "google",
Port: 9090,
},
},
},
}
snap.TerminatingGateway.ServiceConfigs[structs.NewServiceName("web", nil)] = &structs.ServiceConfigResponse{
ProxyConfig: map[string]interface{}{"protocol": "http"},
}
// DiscoveryChains without endpoints do not get a filter chain because there are no addresses to match on.
snap.ConnectProxy.DiscoveryChain["no-endpoints"] = discoverychain.TestCompileConfigEntries(
t, "no-endpoints", "default", "dc1",
connect.TestClusterID+".consul", "dc1", nil)
},
},
{
name: "ingress-http-multiple-services",
create: proxycfg.TestConfigSnapshotIngress_HTTPMultipleServices,
setup: func(snap *proxycfg.ConfigSnapshot) {
snap.IngressGateway.Upstreams = map[proxycfg.IngressListenerKey]structs.Upstreams{
{Protocol: "http", Port: 8080}: {
{
DestinationName: "foo",
LocalBindPort: 8080,
},
{
DestinationName: "bar",
LocalBindPort: 8080,
},
},
{Protocol: "http", Port: 443}: {
{
DestinationName: "baz",
LocalBindPort: 443,
},
{
DestinationName: "qux",
LocalBindPort: 443,
},
},
}
},
},
{
name: "terminating-gateway-no-api-cert",
create: proxycfg.TestConfigSnapshotTerminatingGateway,
setup: func(snap *proxycfg.ConfigSnapshot) {
snap.TerminatingGateway.ServiceLeaves[structs.NewServiceName("api", nil)] = nil
},
},
{
name: "ingress-with-tls-listener",
create: proxycfg.TestConfigSnapshotIngressWithTLSListener,
setup: nil,
},
}
latestEnvoyVersion := proxysupport.EnvoyVersions[0]

View File

@ -28,7 +28,7 @@ func (s *Server) routesFromSnapshot(cInfo connectionInfo, cfgSnap *proxycfg.Conf
switch cfgSnap.Kind {
case structs.ServiceKindConnectProxy:
return routesForConnectProxy(cInfo, cfgSnap.Proxy.Upstreams, cfgSnap.ConnectProxy.DiscoveryChain)
return routesForConnectProxy(cInfo, cfgSnap.ConnectProxy.DiscoveryChain)
case structs.ServiceKindIngressGateway:
return routesForIngressGateway(cInfo, cfgSnap.IngressGateway.Upstreams, cfgSnap.IngressGateway.DiscoveryChain)
case structs.ServiceKindTerminatingGateway:
@ -42,29 +42,22 @@ func (s *Server) routesFromSnapshot(cInfo connectionInfo, cfgSnap *proxycfg.Conf
// "routes" in the snapshot.
func routesForConnectProxy(
cInfo connectionInfo,
upstreams structs.Upstreams,
chains map[string]*structs.CompiledDiscoveryChain,
) ([]proto.Message, error) {
var resources []proto.Message
for _, u := range upstreams {
upstreamID := u.Identifier()
var chain *structs.CompiledDiscoveryChain
if u.DestinationType != structs.UpstreamDestTypePreparedQuery {
chain = chains[upstreamID]
for id, chain := range chains {
if chain.IsDefault() {
continue
}
if chain == nil || chain.IsDefault() {
// TODO(rb): make this do the old school stuff too
} else {
virtualHost, err := makeUpstreamRouteForDiscoveryChain(cInfo, upstreamID, chain, []string{"*"})
virtualHost, err := makeUpstreamRouteForDiscoveryChain(cInfo, id, chain, []string{"*"})
if err != nil {
return nil, err
}
route := &envoy_route_v3.RouteConfiguration{
Name: upstreamID,
Name: id,
VirtualHosts: []*envoy_route_v3.VirtualHost{virtualHost},
// ValidateClusters defaults to true when defined statically and false
// when done via RDS. Re-set the sane value of true to prevent
@ -73,7 +66,6 @@ func routesForConnectProxy(
}
resources = append(resources, route)
}
}
// TODO(rb): make sure we don't generate an empty result
return resources, nil

View File

@ -52,6 +52,9 @@ const (
// PublicListenerName is the name we give the public listener in Envoy config.
PublicListenerName = "public_listener"
// OutboundListenerName is the name we give the outbound Envoy listener when TransparentProxy mode is enabled.
OutboundListenerName = "outbound_listener"
// LocalAppClusterName is the name we give the local application "cluster" in
// Envoy config. Note that all cluster names may collide with service names
// since we want cluster names and service names to match to enable nice
@ -80,6 +83,12 @@ const (
// services named "local_agent" in the future.
LocalAgentClusterName = "local_agent"
// OriginalDestinationClusterName is the name we give to the passthrough
// cluster which redirects transparently-proxied requests to their original
// destination. This cluster prevents Consul from blocking connections to
// destinations outside of the catalog when in TransparentProxy mode.
OriginalDestinationClusterName = "original-destination"
// DefaultAuthCheckFrequency is the default value for
// Server.AuthCheckFrequency to use when the zero value is provided.
DefaultAuthCheckFrequency = 5 * time.Minute

View File

@ -0,0 +1,139 @@
{
"versionInfo": "00000001",
"resources": [
{
"@type": "type.googleapis.com/envoy.config.cluster.v3.Cluster",
"name": "db.default.dc1.internal.11111111-2222-3333-4444-555555555555.consul",
"altStatName": "db.default.dc1.internal.11111111-2222-3333-4444-555555555555.consul",
"type": "EDS",
"edsClusterConfig": {
"edsConfig": {
"ads": {
},
"resourceApiVersion": "V3"
}
},
"connectTimeout": "5s",
"circuitBreakers": {
},
"outlierDetection": {
},
"commonLbConfig": {
"healthyPanicThreshold": {
}
},
"transportSocket": {
"name": "tls",
"typedConfig": {
"@type": "type.googleapis.com/envoy.extensions.transport_sockets.tls.v3.UpstreamTlsContext",
"commonTlsContext": {
"tlsParams": {
},
"tlsCertificates": [
{
"certificateChain": {
"inlineString": "-----BEGIN CERTIFICATE-----\nMIICjDCCAjKgAwIBAgIIC5llxGV1gB8wCgYIKoZIzj0EAwIwFDESMBAGA1UEAxMJ\nVGVzdCBDQSAyMB4XDTE5MDMyMjEzNTgyNloXDTI5MDMyMjEzNTgyNlowDjEMMAoG\nA1UEAxMDd2ViMFkwEwYHKoZIzj0CAQYIKoZIzj0DAQcDQgAEADPv1RHVNRfa2VKR\nAB16b6rZnEt7tuhaxCFpQXPj7M2omb0B9Favq5E0ivpNtv1QnFhxtPd7d5k4e+T7\nSkW1TaOCAXIwggFuMA4GA1UdDwEB/wQEAwIDuDAdBgNVHSUEFjAUBggrBgEFBQcD\nAgYIKwYBBQUHAwEwDAYDVR0TAQH/BAIwADBoBgNVHQ4EYQRfN2Q6MDc6ODc6M2E6\nNDA6MTk6NDc6YzM6NWE6YzA6YmE6NjI6ZGY6YWY6NGI6ZDQ6MDU6MjU6NzY6M2Q6\nNWE6OGQ6MTY6OGQ6Njc6NWU6MmU6YTA6MzQ6N2Q6ZGM6ZmYwagYDVR0jBGMwYYBf\nZDE6MTE6MTE6YWM6MmE6YmE6OTc6YjI6M2Y6YWM6N2I6YmQ6ZGE6YmU6YjE6OGE6\nZmM6OWE6YmE6YjU6YmM6ODM6ZTc6NWU6NDE6NmY6ZjI6NzM6OTU6NTg6MGM6ZGIw\nWQYDVR0RBFIwUIZOc3BpZmZlOi8vMTExMTExMTEtMjIyMi0zMzMzLTQ0NDQtNTU1\nNTU1NTU1NTU1LmNvbnN1bC9ucy9kZWZhdWx0L2RjL2RjMS9zdmMvd2ViMAoGCCqG\nSM49BAMCA0gAMEUCIGC3TTvvjj76KMrguVyFf4tjOqaSCRie3nmHMRNNRav7AiEA\npY0heYeK9A6iOLrzqxSerkXXQyj5e9bE4VgUnxgPU6g=\n-----END CERTIFICATE-----\n"
},
"privateKey": {
"inlineString": "-----BEGIN EC PRIVATE KEY-----\nMHcCAQEEIMoTkpRggp3fqZzFKh82yS4LjtJI+XY+qX/7DefHFrtdoAoGCCqGSM49\nAwEHoUQDQgAEADPv1RHVNRfa2VKRAB16b6rZnEt7tuhaxCFpQXPj7M2omb0B9Fav\nq5E0ivpNtv1QnFhxtPd7d5k4e+T7SkW1TQ==\n-----END EC PRIVATE KEY-----\n"
}
}
],
"validationContext": {
"trustedCa": {
"inlineString": "-----BEGIN CERTIFICATE-----\nMIICXDCCAgKgAwIBAgIICpZq70Z9LyUwCgYIKoZIzj0EAwIwFDESMBAGA1UEAxMJ\nVGVzdCBDQSAyMB4XDTE5MDMyMjEzNTgyNloXDTI5MDMyMjEzNTgyNlowFDESMBAG\nA1UEAxMJVGVzdCBDQSAyMFkwEwYHKoZIzj0CAQYIKoZIzj0DAQcDQgAEIhywH1gx\nAsMwuF3ukAI5YL2jFxH6Usnma1HFSfVyxbXX1/uoZEYrj8yCAtdU2yoHETyd+Zx2\nThhRLP79pYegCaOCATwwggE4MA4GA1UdDwEB/wQEAwIBhjAPBgNVHRMBAf8EBTAD\nAQH/MGgGA1UdDgRhBF9kMToxMToxMTphYzoyYTpiYTo5NzpiMjozZjphYzo3Yjpi\nZDpkYTpiZTpiMTo4YTpmYzo5YTpiYTpiNTpiYzo4MzplNzo1ZTo0MTo2ZjpmMjo3\nMzo5NTo1ODowYzpkYjBqBgNVHSMEYzBhgF9kMToxMToxMTphYzoyYTpiYTo5Nzpi\nMjozZjphYzo3YjpiZDpkYTpiZTpiMTo4YTpmYzo5YTpiYTpiNTpiYzo4MzplNzo1\nZTo0MTo2ZjpmMjo3Mzo5NTo1ODowYzpkYjA/BgNVHREEODA2hjRzcGlmZmU6Ly8x\nMTExMTExMS0yMjIyLTMzMzMtNDQ0NC01NTU1NTU1NTU1NTUuY29uc3VsMAoGCCqG\nSM49BAMCA0gAMEUCICOY0i246rQHJt8o8Oya0D5PLL1FnmsQmQqIGCi31RwnAiEA\noR5f6Ku+cig2Il8T8LJujOp2/2A72QcHZA57B13y+8o=\n-----END CERTIFICATE-----\n"
}
}
},
"sni": "db.default.dc1.internal.11111111-2222-3333-4444-555555555555.consul"
}
}
},
{
"@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": {
"ads": {
},
"resourceApiVersion": "V3"
}
},
"connectTimeout": "5s",
"circuitBreakers": {
},
"outlierDetection": {
},
"transportSocket": {
"name": "tls",
"typedConfig": {
"@type": "type.googleapis.com/envoy.extensions.transport_sockets.tls.v3.UpstreamTlsContext",
"commonTlsContext": {
"tlsParams": {
},
"tlsCertificates": [
{
"certificateChain": {
"inlineString": "-----BEGIN CERTIFICATE-----\nMIICjDCCAjKgAwIBAgIIC5llxGV1gB8wCgYIKoZIzj0EAwIwFDESMBAGA1UEAxMJ\nVGVzdCBDQSAyMB4XDTE5MDMyMjEzNTgyNloXDTI5MDMyMjEzNTgyNlowDjEMMAoG\nA1UEAxMDd2ViMFkwEwYHKoZIzj0CAQYIKoZIzj0DAQcDQgAEADPv1RHVNRfa2VKR\nAB16b6rZnEt7tuhaxCFpQXPj7M2omb0B9Favq5E0ivpNtv1QnFhxtPd7d5k4e+T7\nSkW1TaOCAXIwggFuMA4GA1UdDwEB/wQEAwIDuDAdBgNVHSUEFjAUBggrBgEFBQcD\nAgYIKwYBBQUHAwEwDAYDVR0TAQH/BAIwADBoBgNVHQ4EYQRfN2Q6MDc6ODc6M2E6\nNDA6MTk6NDc6YzM6NWE6YzA6YmE6NjI6ZGY6YWY6NGI6ZDQ6MDU6MjU6NzY6M2Q6\nNWE6OGQ6MTY6OGQ6Njc6NWU6MmU6YTA6MzQ6N2Q6ZGM6ZmYwagYDVR0jBGMwYYBf\nZDE6MTE6MTE6YWM6MmE6YmE6OTc6YjI6M2Y6YWM6N2I6YmQ6ZGE6YmU6YjE6OGE6\nZmM6OWE6YmE6YjU6YmM6ODM6ZTc6NWU6NDE6NmY6ZjI6NzM6OTU6NTg6MGM6ZGIw\nWQYDVR0RBFIwUIZOc3BpZmZlOi8vMTExMTExMTEtMjIyMi0zMzMzLTQ0NDQtNTU1\nNTU1NTU1NTU1LmNvbnN1bC9ucy9kZWZhdWx0L2RjL2RjMS9zdmMvd2ViMAoGCCqG\nSM49BAMCA0gAMEUCIGC3TTvvjj76KMrguVyFf4tjOqaSCRie3nmHMRNNRav7AiEA\npY0heYeK9A6iOLrzqxSerkXXQyj5e9bE4VgUnxgPU6g=\n-----END CERTIFICATE-----\n"
},
"privateKey": {
"inlineString": "-----BEGIN EC PRIVATE KEY-----\nMHcCAQEEIMoTkpRggp3fqZzFKh82yS4LjtJI+XY+qX/7DefHFrtdoAoGCCqGSM49\nAwEHoUQDQgAEADPv1RHVNRfa2VKRAB16b6rZnEt7tuhaxCFpQXPj7M2omb0B9Fav\nq5E0ivpNtv1QnFhxtPd7d5k4e+T7SkW1TQ==\n-----END EC PRIVATE KEY-----\n"
}
}
],
"validationContext": {
"trustedCa": {
"inlineString": "-----BEGIN CERTIFICATE-----\nMIICXDCCAgKgAwIBAgIICpZq70Z9LyUwCgYIKoZIzj0EAwIwFDESMBAGA1UEAxMJ\nVGVzdCBDQSAyMB4XDTE5MDMyMjEzNTgyNloXDTI5MDMyMjEzNTgyNlowFDESMBAG\nA1UEAxMJVGVzdCBDQSAyMFkwEwYHKoZIzj0CAQYIKoZIzj0DAQcDQgAEIhywH1gx\nAsMwuF3ukAI5YL2jFxH6Usnma1HFSfVyxbXX1/uoZEYrj8yCAtdU2yoHETyd+Zx2\nThhRLP79pYegCaOCATwwggE4MA4GA1UdDwEB/wQEAwIBhjAPBgNVHRMBAf8EBTAD\nAQH/MGgGA1UdDgRhBF9kMToxMToxMTphYzoyYTpiYTo5NzpiMjozZjphYzo3Yjpi\nZDpkYTpiZTpiMTo4YTpmYzo5YTpiYTpiNTpiYzo4MzplNzo1ZTo0MTo2ZjpmMjo3\nMzo5NTo1ODowYzpkYjBqBgNVHSMEYzBhgF9kMToxMToxMTphYzoyYTpiYTo5Nzpi\nMjozZjphYzo3YjpiZDpkYTpiZTpiMTo4YTpmYzo5YTpiYTpiNTpiYzo4MzplNzo1\nZTo0MTo2ZjpmMjo3Mzo5NTo1ODowYzpkYjA/BgNVHREEODA2hjRzcGlmZmU6Ly8x\nMTExMTExMS0yMjIyLTMzMzMtNDQ0NC01NTU1NTU1NTU1NTUuY29uc3VsMAoGCCqG\nSM49BAMCA0gAMEUCICOY0i246rQHJt8o8Oya0D5PLL1FnmsQmQqIGCi31RwnAiEA\noR5f6Ku+cig2Il8T8LJujOp2/2A72QcHZA57B13y+8o=\n-----END CERTIFICATE-----\n"
}
}
},
"sni": "geo-cache.default.dc1.query.11111111-2222-3333-4444-555555555555.consul"
}
}
},
{
"@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
}
}
}
}
]
}
]
}
},
{
"@type": "type.googleapis.com/envoy.config.cluster.v3.Cluster",
"name": "original-destination",
"type": "ORIGINAL_DST",
"connectTimeout": "5s",
"lbPolicy": "CLUSTER_PROVIDED"
}
],
"typeUrl": "type.googleapis.com/envoy.config.cluster.v3.Cluster",
"nonce": "00000001"
}

View File

@ -0,0 +1,139 @@
{
"versionInfo": "00000001",
"resources": [
{
"@type": "type.googleapis.com/envoy.api.v2.Cluster",
"name": "db.default.dc1.internal.11111111-2222-3333-4444-555555555555.consul",
"altStatName": "db.default.dc1.internal.11111111-2222-3333-4444-555555555555.consul",
"type": "EDS",
"edsClusterConfig": {
"edsConfig": {
"ads": {
},
"resourceApiVersion": "V2"
}
},
"connectTimeout": "5s",
"circuitBreakers": {
},
"outlierDetection": {
},
"commonLbConfig": {
"healthyPanicThreshold": {
}
},
"transportSocket": {
"name": "tls",
"typedConfig": {
"@type": "type.googleapis.com/envoy.api.v2.auth.UpstreamTlsContext",
"commonTlsContext": {
"tlsParams": {
},
"tlsCertificates": [
{
"certificateChain": {
"inlineString": "-----BEGIN CERTIFICATE-----\nMIICjDCCAjKgAwIBAgIIC5llxGV1gB8wCgYIKoZIzj0EAwIwFDESMBAGA1UEAxMJ\nVGVzdCBDQSAyMB4XDTE5MDMyMjEzNTgyNloXDTI5MDMyMjEzNTgyNlowDjEMMAoG\nA1UEAxMDd2ViMFkwEwYHKoZIzj0CAQYIKoZIzj0DAQcDQgAEADPv1RHVNRfa2VKR\nAB16b6rZnEt7tuhaxCFpQXPj7M2omb0B9Favq5E0ivpNtv1QnFhxtPd7d5k4e+T7\nSkW1TaOCAXIwggFuMA4GA1UdDwEB/wQEAwIDuDAdBgNVHSUEFjAUBggrBgEFBQcD\nAgYIKwYBBQUHAwEwDAYDVR0TAQH/BAIwADBoBgNVHQ4EYQRfN2Q6MDc6ODc6M2E6\nNDA6MTk6NDc6YzM6NWE6YzA6YmE6NjI6ZGY6YWY6NGI6ZDQ6MDU6MjU6NzY6M2Q6\nNWE6OGQ6MTY6OGQ6Njc6NWU6MmU6YTA6MzQ6N2Q6ZGM6ZmYwagYDVR0jBGMwYYBf\nZDE6MTE6MTE6YWM6MmE6YmE6OTc6YjI6M2Y6YWM6N2I6YmQ6ZGE6YmU6YjE6OGE6\nZmM6OWE6YmE6YjU6YmM6ODM6ZTc6NWU6NDE6NmY6ZjI6NzM6OTU6NTg6MGM6ZGIw\nWQYDVR0RBFIwUIZOc3BpZmZlOi8vMTExMTExMTEtMjIyMi0zMzMzLTQ0NDQtNTU1\nNTU1NTU1NTU1LmNvbnN1bC9ucy9kZWZhdWx0L2RjL2RjMS9zdmMvd2ViMAoGCCqG\nSM49BAMCA0gAMEUCIGC3TTvvjj76KMrguVyFf4tjOqaSCRie3nmHMRNNRav7AiEA\npY0heYeK9A6iOLrzqxSerkXXQyj5e9bE4VgUnxgPU6g=\n-----END CERTIFICATE-----\n"
},
"privateKey": {
"inlineString": "-----BEGIN EC PRIVATE KEY-----\nMHcCAQEEIMoTkpRggp3fqZzFKh82yS4LjtJI+XY+qX/7DefHFrtdoAoGCCqGSM49\nAwEHoUQDQgAEADPv1RHVNRfa2VKRAB16b6rZnEt7tuhaxCFpQXPj7M2omb0B9Fav\nq5E0ivpNtv1QnFhxtPd7d5k4e+T7SkW1TQ==\n-----END EC PRIVATE KEY-----\n"
}
}
],
"validationContext": {
"trustedCa": {
"inlineString": "-----BEGIN CERTIFICATE-----\nMIICXDCCAgKgAwIBAgIICpZq70Z9LyUwCgYIKoZIzj0EAwIwFDESMBAGA1UEAxMJ\nVGVzdCBDQSAyMB4XDTE5MDMyMjEzNTgyNloXDTI5MDMyMjEzNTgyNlowFDESMBAG\nA1UEAxMJVGVzdCBDQSAyMFkwEwYHKoZIzj0CAQYIKoZIzj0DAQcDQgAEIhywH1gx\nAsMwuF3ukAI5YL2jFxH6Usnma1HFSfVyxbXX1/uoZEYrj8yCAtdU2yoHETyd+Zx2\nThhRLP79pYegCaOCATwwggE4MA4GA1UdDwEB/wQEAwIBhjAPBgNVHRMBAf8EBTAD\nAQH/MGgGA1UdDgRhBF9kMToxMToxMTphYzoyYTpiYTo5NzpiMjozZjphYzo3Yjpi\nZDpkYTpiZTpiMTo4YTpmYzo5YTpiYTpiNTpiYzo4MzplNzo1ZTo0MTo2ZjpmMjo3\nMzo5NTo1ODowYzpkYjBqBgNVHSMEYzBhgF9kMToxMToxMTphYzoyYTpiYTo5Nzpi\nMjozZjphYzo3YjpiZDpkYTpiZTpiMTo4YTpmYzo5YTpiYTpiNTpiYzo4MzplNzo1\nZTo0MTo2ZjpmMjo3Mzo5NTo1ODowYzpkYjA/BgNVHREEODA2hjRzcGlmZmU6Ly8x\nMTExMTExMS0yMjIyLTMzMzMtNDQ0NC01NTU1NTU1NTU1NTUuY29uc3VsMAoGCCqG\nSM49BAMCA0gAMEUCICOY0i246rQHJt8o8Oya0D5PLL1FnmsQmQqIGCi31RwnAiEA\noR5f6Ku+cig2Il8T8LJujOp2/2A72QcHZA57B13y+8o=\n-----END CERTIFICATE-----\n"
}
}
},
"sni": "db.default.dc1.internal.11111111-2222-3333-4444-555555555555.consul"
}
}
},
{
"@type": "type.googleapis.com/envoy.api.v2.Cluster",
"name": "geo-cache.default.dc1.query.11111111-2222-3333-4444-555555555555.consul",
"type": "EDS",
"edsClusterConfig": {
"edsConfig": {
"ads": {
},
"resourceApiVersion": "V2"
}
},
"connectTimeout": "5s",
"circuitBreakers": {
},
"outlierDetection": {
},
"transportSocket": {
"name": "tls",
"typedConfig": {
"@type": "type.googleapis.com/envoy.api.v2.auth.UpstreamTlsContext",
"commonTlsContext": {
"tlsParams": {
},
"tlsCertificates": [
{
"certificateChain": {
"inlineString": "-----BEGIN CERTIFICATE-----\nMIICjDCCAjKgAwIBAgIIC5llxGV1gB8wCgYIKoZIzj0EAwIwFDESMBAGA1UEAxMJ\nVGVzdCBDQSAyMB4XDTE5MDMyMjEzNTgyNloXDTI5MDMyMjEzNTgyNlowDjEMMAoG\nA1UEAxMDd2ViMFkwEwYHKoZIzj0CAQYIKoZIzj0DAQcDQgAEADPv1RHVNRfa2VKR\nAB16b6rZnEt7tuhaxCFpQXPj7M2omb0B9Favq5E0ivpNtv1QnFhxtPd7d5k4e+T7\nSkW1TaOCAXIwggFuMA4GA1UdDwEB/wQEAwIDuDAdBgNVHSUEFjAUBggrBgEFBQcD\nAgYIKwYBBQUHAwEwDAYDVR0TAQH/BAIwADBoBgNVHQ4EYQRfN2Q6MDc6ODc6M2E6\nNDA6MTk6NDc6YzM6NWE6YzA6YmE6NjI6ZGY6YWY6NGI6ZDQ6MDU6MjU6NzY6M2Q6\nNWE6OGQ6MTY6OGQ6Njc6NWU6MmU6YTA6MzQ6N2Q6ZGM6ZmYwagYDVR0jBGMwYYBf\nZDE6MTE6MTE6YWM6MmE6YmE6OTc6YjI6M2Y6YWM6N2I6YmQ6ZGE6YmU6YjE6OGE6\nZmM6OWE6YmE6YjU6YmM6ODM6ZTc6NWU6NDE6NmY6ZjI6NzM6OTU6NTg6MGM6ZGIw\nWQYDVR0RBFIwUIZOc3BpZmZlOi8vMTExMTExMTEtMjIyMi0zMzMzLTQ0NDQtNTU1\nNTU1NTU1NTU1LmNvbnN1bC9ucy9kZWZhdWx0L2RjL2RjMS9zdmMvd2ViMAoGCCqG\nSM49BAMCA0gAMEUCIGC3TTvvjj76KMrguVyFf4tjOqaSCRie3nmHMRNNRav7AiEA\npY0heYeK9A6iOLrzqxSerkXXQyj5e9bE4VgUnxgPU6g=\n-----END CERTIFICATE-----\n"
},
"privateKey": {
"inlineString": "-----BEGIN EC PRIVATE KEY-----\nMHcCAQEEIMoTkpRggp3fqZzFKh82yS4LjtJI+XY+qX/7DefHFrtdoAoGCCqGSM49\nAwEHoUQDQgAEADPv1RHVNRfa2VKRAB16b6rZnEt7tuhaxCFpQXPj7M2omb0B9Fav\nq5E0ivpNtv1QnFhxtPd7d5k4e+T7SkW1TQ==\n-----END EC PRIVATE KEY-----\n"
}
}
],
"validationContext": {
"trustedCa": {
"inlineString": "-----BEGIN CERTIFICATE-----\nMIICXDCCAgKgAwIBAgIICpZq70Z9LyUwCgYIKoZIzj0EAwIwFDESMBAGA1UEAxMJ\nVGVzdCBDQSAyMB4XDTE5MDMyMjEzNTgyNloXDTI5MDMyMjEzNTgyNlowFDESMBAG\nA1UEAxMJVGVzdCBDQSAyMFkwEwYHKoZIzj0CAQYIKoZIzj0DAQcDQgAEIhywH1gx\nAsMwuF3ukAI5YL2jFxH6Usnma1HFSfVyxbXX1/uoZEYrj8yCAtdU2yoHETyd+Zx2\nThhRLP79pYegCaOCATwwggE4MA4GA1UdDwEB/wQEAwIBhjAPBgNVHRMBAf8EBTAD\nAQH/MGgGA1UdDgRhBF9kMToxMToxMTphYzoyYTpiYTo5NzpiMjozZjphYzo3Yjpi\nZDpkYTpiZTpiMTo4YTpmYzo5YTpiYTpiNTpiYzo4MzplNzo1ZTo0MTo2ZjpmMjo3\nMzo5NTo1ODowYzpkYjBqBgNVHSMEYzBhgF9kMToxMToxMTphYzoyYTpiYTo5Nzpi\nMjozZjphYzo3YjpiZDpkYTpiZTpiMTo4YTpmYzo5YTpiYTpiNTpiYzo4MzplNzo1\nZTo0MTo2ZjpmMjo3Mzo5NTo1ODowYzpkYjA/BgNVHREEODA2hjRzcGlmZmU6Ly8x\nMTExMTExMS0yMjIyLTMzMzMtNDQ0NC01NTU1NTU1NTU1NTUuY29uc3VsMAoGCCqG\nSM49BAMCA0gAMEUCICOY0i246rQHJt8o8Oya0D5PLL1FnmsQmQqIGCi31RwnAiEA\noR5f6Ku+cig2Il8T8LJujOp2/2A72QcHZA57B13y+8o=\n-----END CERTIFICATE-----\n"
}
}
},
"sni": "geo-cache.default.dc1.query.11111111-2222-3333-4444-555555555555.consul"
}
}
},
{
"@type": "type.googleapis.com/envoy.api.v2.Cluster",
"name": "local_app",
"type": "STATIC",
"connectTimeout": "5s",
"loadAssignment": {
"clusterName": "local_app",
"endpoints": [
{
"lbEndpoints": [
{
"endpoint": {
"address": {
"socketAddress": {
"address": "127.0.0.1",
"portValue": 8080
}
}
}
}
]
}
]
}
},
{
"@type": "type.googleapis.com/envoy.api.v2.Cluster",
"name": "original-destination",
"type": "ORIGINAL_DST",
"connectTimeout": "5s",
"lbPolicy": "CLUSTER_PROVIDED"
}
],
"typeUrl": "type.googleapis.com/envoy.api.v2.Cluster",
"nonce": "00000001"
}

View File

@ -0,0 +1,169 @@
{
"versionInfo": "00000001",
"resources": [
{
"@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
}
},
"filterChains": [
{
"filters": [
{
"name": "envoy.filters.network.tcp_proxy",
"typedConfig": {
"@type": "type.googleapis.com/envoy.extensions.filters.network.tcp_proxy.v3.TcpProxy",
"statPrefix": "upstream.db.default.dc1",
"cluster": "db.default.dc1.internal.11111111-2222-3333-4444-555555555555.consul"
}
}
]
}
],
"trafficDirection": "OUTBOUND"
},
{
"@type": "type.googleapis.com/envoy.config.listener.v3.Listener",
"name": "outbound_listener:127.0.0.1:15001",
"address": {
"socketAddress": {
"address": "127.0.0.1",
"portValue": 15001
}
},
"filterChains": [
{
"filterChainMatch": {
"prefixRanges": [
{
"addressPrefix": "8.8.8.8",
"prefixLen": 32
}
]
},
"filters": [
{
"name": "envoy.filters.network.tcp_proxy",
"typedConfig": {
"@type": "type.googleapis.com/envoy.extensions.filters.network.tcp_proxy.v3.TcpProxy",
"statPrefix": "upstream.google.default.dc1",
"cluster": "google.default.dc1.internal.11111111-2222-3333-4444-555555555555.consul"
}
}
]
},
{
"filters": [
{
"name": "envoy.filters.network.tcp_proxy",
"typedConfig": {
"@type": "type.googleapis.com/envoy.extensions.filters.network.tcp_proxy.v3.TcpProxy",
"statPrefix": "upstream.original-destination",
"cluster": "original-destination"
}
}
]
}
],
"listenerFilters": [
{
"name": "envoy.filters.listener.original_dst"
}
],
"trafficDirection": "OUTBOUND"
},
{
"@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
}
},
"filterChains": [
{
"filters": [
{
"name": "envoy.filters.network.tcp_proxy",
"typedConfig": {
"@type": "type.googleapis.com/envoy.extensions.filters.network.tcp_proxy.v3.TcpProxy",
"statPrefix": "upstream.prepared_query_geo-cache",
"cluster": "geo-cache.default.dc1.query.11111111-2222-3333-4444-555555555555.consul"
}
}
]
}
],
"trafficDirection": "OUTBOUND"
},
{
"@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
}
},
"filterChains": [
{
"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",
"statPrefix": "public_listener",
"cluster": "local_app"
}
}
],
"transportSocket": {
"name": "tls",
"typedConfig": {
"@type": "type.googleapis.com/envoy.extensions.transport_sockets.tls.v3.DownstreamTlsContext",
"commonTlsContext": {
"tlsParams": {
},
"tlsCertificates": [
{
"certificateChain": {
"inlineString": "-----BEGIN CERTIFICATE-----\nMIICjDCCAjKgAwIBAgIIC5llxGV1gB8wCgYIKoZIzj0EAwIwFDESMBAGA1UEAxMJ\nVGVzdCBDQSAyMB4XDTE5MDMyMjEzNTgyNloXDTI5MDMyMjEzNTgyNlowDjEMMAoG\nA1UEAxMDd2ViMFkwEwYHKoZIzj0CAQYIKoZIzj0DAQcDQgAEADPv1RHVNRfa2VKR\nAB16b6rZnEt7tuhaxCFpQXPj7M2omb0B9Favq5E0ivpNtv1QnFhxtPd7d5k4e+T7\nSkW1TaOCAXIwggFuMA4GA1UdDwEB/wQEAwIDuDAdBgNVHSUEFjAUBggrBgEFBQcD\nAgYIKwYBBQUHAwEwDAYDVR0TAQH/BAIwADBoBgNVHQ4EYQRfN2Q6MDc6ODc6M2E6\nNDA6MTk6NDc6YzM6NWE6YzA6YmE6NjI6ZGY6YWY6NGI6ZDQ6MDU6MjU6NzY6M2Q6\nNWE6OGQ6MTY6OGQ6Njc6NWU6MmU6YTA6MzQ6N2Q6ZGM6ZmYwagYDVR0jBGMwYYBf\nZDE6MTE6MTE6YWM6MmE6YmE6OTc6YjI6M2Y6YWM6N2I6YmQ6ZGE6YmU6YjE6OGE6\nZmM6OWE6YmE6YjU6YmM6ODM6ZTc6NWU6NDE6NmY6ZjI6NzM6OTU6NTg6MGM6ZGIw\nWQYDVR0RBFIwUIZOc3BpZmZlOi8vMTExMTExMTEtMjIyMi0zMzMzLTQ0NDQtNTU1\nNTU1NTU1NTU1LmNvbnN1bC9ucy9kZWZhdWx0L2RjL2RjMS9zdmMvd2ViMAoGCCqG\nSM49BAMCA0gAMEUCIGC3TTvvjj76KMrguVyFf4tjOqaSCRie3nmHMRNNRav7AiEA\npY0heYeK9A6iOLrzqxSerkXXQyj5e9bE4VgUnxgPU6g=\n-----END CERTIFICATE-----\n"
},
"privateKey": {
"inlineString": "-----BEGIN EC PRIVATE KEY-----\nMHcCAQEEIMoTkpRggp3fqZzFKh82yS4LjtJI+XY+qX/7DefHFrtdoAoGCCqGSM49\nAwEHoUQDQgAEADPv1RHVNRfa2VKRAB16b6rZnEt7tuhaxCFpQXPj7M2omb0B9Fav\nq5E0ivpNtv1QnFhxtPd7d5k4e+T7SkW1TQ==\n-----END EC PRIVATE KEY-----\n"
}
}
],
"validationContext": {
"trustedCa": {
"inlineString": "-----BEGIN CERTIFICATE-----\nMIICXDCCAgKgAwIBAgIICpZq70Z9LyUwCgYIKoZIzj0EAwIwFDESMBAGA1UEAxMJ\nVGVzdCBDQSAyMB4XDTE5MDMyMjEzNTgyNloXDTI5MDMyMjEzNTgyNlowFDESMBAG\nA1UEAxMJVGVzdCBDQSAyMFkwEwYHKoZIzj0CAQYIKoZIzj0DAQcDQgAEIhywH1gx\nAsMwuF3ukAI5YL2jFxH6Usnma1HFSfVyxbXX1/uoZEYrj8yCAtdU2yoHETyd+Zx2\nThhRLP79pYegCaOCATwwggE4MA4GA1UdDwEB/wQEAwIBhjAPBgNVHRMBAf8EBTAD\nAQH/MGgGA1UdDgRhBF9kMToxMToxMTphYzoyYTpiYTo5NzpiMjozZjphYzo3Yjpi\nZDpkYTpiZTpiMTo4YTpmYzo5YTpiYTpiNTpiYzo4MzplNzo1ZTo0MTo2ZjpmMjo3\nMzo5NTo1ODowYzpkYjBqBgNVHSMEYzBhgF9kMToxMToxMTphYzoyYTpiYTo5Nzpi\nMjozZjphYzo3YjpiZDpkYTpiZTpiMTo4YTpmYzo5YTpiYTpiNTpiYzo4MzplNzo1\nZTo0MTo2ZjpmMjo3Mzo5NTo1ODowYzpkYjA/BgNVHREEODA2hjRzcGlmZmU6Ly8x\nMTExMTExMS0yMjIyLTMzMzMtNDQ0NC01NTU1NTU1NTU1NTUuY29uc3VsMAoGCCqG\nSM49BAMCA0gAMEUCICOY0i246rQHJt8o8Oya0D5PLL1FnmsQmQqIGCi31RwnAiEA\noR5f6Ku+cig2Il8T8LJujOp2/2A72QcHZA57B13y+8o=\n-----END CERTIFICATE-----\n"
}
}
},
"requireClientCertificate": true
}
}
}
],
"trafficDirection": "INBOUND"
}
],
"typeUrl": "type.googleapis.com/envoy.config.listener.v3.Listener",
"nonce": "00000001"
}

View File

@ -0,0 +1,169 @@
{
"versionInfo": "00000001",
"resources": [
{
"@type": "type.googleapis.com/envoy.api.v2.Listener",
"name": "db:127.0.0.1:9191",
"address": {
"socketAddress": {
"address": "127.0.0.1",
"portValue": 9191
}
},
"filterChains": [
{
"filters": [
{
"name": "envoy.filters.network.tcp_proxy",
"typedConfig": {
"@type": "type.googleapis.com/envoy.config.filter.network.tcp_proxy.v2.TcpProxy",
"statPrefix": "upstream.db.default.dc1",
"cluster": "db.default.dc1.internal.11111111-2222-3333-4444-555555555555.consul"
}
}
]
}
],
"trafficDirection": "OUTBOUND"
},
{
"@type": "type.googleapis.com/envoy.api.v2.Listener",
"name": "outbound_listener:127.0.0.1:15001",
"address": {
"socketAddress": {
"address": "127.0.0.1",
"portValue": 15001
}
},
"filterChains": [
{
"filterChainMatch": {
"prefixRanges": [
{
"addressPrefix": "8.8.8.8",
"prefixLen": 32
}
]
},
"filters": [
{
"name": "envoy.filters.network.tcp_proxy",
"typedConfig": {
"@type": "type.googleapis.com/envoy.config.filter.network.tcp_proxy.v2.TcpProxy",
"statPrefix": "upstream.google.default.dc1",
"cluster": "google.default.dc1.internal.11111111-2222-3333-4444-555555555555.consul"
}
}
]
},
{
"filters": [
{
"name": "envoy.filters.network.tcp_proxy",
"typedConfig": {
"@type": "type.googleapis.com/envoy.config.filter.network.tcp_proxy.v2.TcpProxy",
"statPrefix": "upstream.original-destination",
"cluster": "original-destination"
}
}
]
}
],
"listenerFilters": [
{
"name": "envoy.filters.listener.original_dst"
}
],
"trafficDirection": "OUTBOUND"
},
{
"@type": "type.googleapis.com/envoy.api.v2.Listener",
"name": "prepared_query:geo-cache:127.10.10.10:8181",
"address": {
"socketAddress": {
"address": "127.10.10.10",
"portValue": 8181
}
},
"filterChains": [
{
"filters": [
{
"name": "envoy.filters.network.tcp_proxy",
"typedConfig": {
"@type": "type.googleapis.com/envoy.config.filter.network.tcp_proxy.v2.TcpProxy",
"statPrefix": "upstream.prepared_query_geo-cache",
"cluster": "geo-cache.default.dc1.query.11111111-2222-3333-4444-555555555555.consul"
}
}
]
}
],
"trafficDirection": "OUTBOUND"
},
{
"@type": "type.googleapis.com/envoy.api.v2.Listener",
"name": "public_listener:0.0.0.0:9999",
"address": {
"socketAddress": {
"address": "0.0.0.0",
"portValue": 9999
}
},
"filterChains": [
{
"filters": [
{
"name": "envoy.filters.network.rbac",
"typedConfig": {
"@type": "type.googleapis.com/envoy.config.filter.network.rbac.v2.RBAC",
"rules": {
},
"statPrefix": "connect_authz"
}
},
{
"name": "envoy.filters.network.tcp_proxy",
"typedConfig": {
"@type": "type.googleapis.com/envoy.config.filter.network.tcp_proxy.v2.TcpProxy",
"statPrefix": "public_listener",
"cluster": "local_app"
}
}
],
"transportSocket": {
"name": "tls",
"typedConfig": {
"@type": "type.googleapis.com/envoy.api.v2.auth.DownstreamTlsContext",
"commonTlsContext": {
"tlsParams": {
},
"tlsCertificates": [
{
"certificateChain": {
"inlineString": "-----BEGIN CERTIFICATE-----\nMIICjDCCAjKgAwIBAgIIC5llxGV1gB8wCgYIKoZIzj0EAwIwFDESMBAGA1UEAxMJ\nVGVzdCBDQSAyMB4XDTE5MDMyMjEzNTgyNloXDTI5MDMyMjEzNTgyNlowDjEMMAoG\nA1UEAxMDd2ViMFkwEwYHKoZIzj0CAQYIKoZIzj0DAQcDQgAEADPv1RHVNRfa2VKR\nAB16b6rZnEt7tuhaxCFpQXPj7M2omb0B9Favq5E0ivpNtv1QnFhxtPd7d5k4e+T7\nSkW1TaOCAXIwggFuMA4GA1UdDwEB/wQEAwIDuDAdBgNVHSUEFjAUBggrBgEFBQcD\nAgYIKwYBBQUHAwEwDAYDVR0TAQH/BAIwADBoBgNVHQ4EYQRfN2Q6MDc6ODc6M2E6\nNDA6MTk6NDc6YzM6NWE6YzA6YmE6NjI6ZGY6YWY6NGI6ZDQ6MDU6MjU6NzY6M2Q6\nNWE6OGQ6MTY6OGQ6Njc6NWU6MmU6YTA6MzQ6N2Q6ZGM6ZmYwagYDVR0jBGMwYYBf\nZDE6MTE6MTE6YWM6MmE6YmE6OTc6YjI6M2Y6YWM6N2I6YmQ6ZGE6YmU6YjE6OGE6\nZmM6OWE6YmE6YjU6YmM6ODM6ZTc6NWU6NDE6NmY6ZjI6NzM6OTU6NTg6MGM6ZGIw\nWQYDVR0RBFIwUIZOc3BpZmZlOi8vMTExMTExMTEtMjIyMi0zMzMzLTQ0NDQtNTU1\nNTU1NTU1NTU1LmNvbnN1bC9ucy9kZWZhdWx0L2RjL2RjMS9zdmMvd2ViMAoGCCqG\nSM49BAMCA0gAMEUCIGC3TTvvjj76KMrguVyFf4tjOqaSCRie3nmHMRNNRav7AiEA\npY0heYeK9A6iOLrzqxSerkXXQyj5e9bE4VgUnxgPU6g=\n-----END CERTIFICATE-----\n"
},
"privateKey": {
"inlineString": "-----BEGIN EC PRIVATE KEY-----\nMHcCAQEEIMoTkpRggp3fqZzFKh82yS4LjtJI+XY+qX/7DefHFrtdoAoGCCqGSM49\nAwEHoUQDQgAEADPv1RHVNRfa2VKRAB16b6rZnEt7tuhaxCFpQXPj7M2omb0B9Fav\nq5E0ivpNtv1QnFhxtPd7d5k4e+T7SkW1TQ==\n-----END EC PRIVATE KEY-----\n"
}
}
],
"validationContext": {
"trustedCa": {
"inlineString": "-----BEGIN CERTIFICATE-----\nMIICXDCCAgKgAwIBAgIICpZq70Z9LyUwCgYIKoZIzj0EAwIwFDESMBAGA1UEAxMJ\nVGVzdCBDQSAyMB4XDTE5MDMyMjEzNTgyNloXDTI5MDMyMjEzNTgyNlowFDESMBAG\nA1UEAxMJVGVzdCBDQSAyMFkwEwYHKoZIzj0CAQYIKoZIzj0DAQcDQgAEIhywH1gx\nAsMwuF3ukAI5YL2jFxH6Usnma1HFSfVyxbXX1/uoZEYrj8yCAtdU2yoHETyd+Zx2\nThhRLP79pYegCaOCATwwggE4MA4GA1UdDwEB/wQEAwIBhjAPBgNVHRMBAf8EBTAD\nAQH/MGgGA1UdDgRhBF9kMToxMToxMTphYzoyYTpiYTo5NzpiMjozZjphYzo3Yjpi\nZDpkYTpiZTpiMTo4YTpmYzo5YTpiYTpiNTpiYzo4MzplNzo1ZTo0MTo2ZjpmMjo3\nMzo5NTo1ODowYzpkYjBqBgNVHSMEYzBhgF9kMToxMToxMTphYzoyYTpiYTo5Nzpi\nMjozZjphYzo3YjpiZDpkYTpiZTpiMTo4YTpmYzo5YTpiYTpiNTpiYzo4MzplNzo1\nZTo0MTo2ZjpmMjo3Mzo5NTo1ODowYzpkYjA/BgNVHREEODA2hjRzcGlmZmU6Ly8x\nMTExMTExMS0yMjIyLTMzMzMtNDQ0NC01NTU1NTU1NTU1NTUuY29uc3VsMAoGCCqG\nSM49BAMCA0gAMEUCICOY0i246rQHJt8o8Oya0D5PLL1FnmsQmQqIGCi31RwnAiEA\noR5f6Ku+cig2Il8T8LJujOp2/2A72QcHZA57B13y+8o=\n-----END CERTIFICATE-----\n"
}
}
},
"requireClientCertificate": true
}
}
}
],
"trafficDirection": "INBOUND"
}
],
"typeUrl": "type.googleapis.com/envoy.api.v2.Listener",
"nonce": "00000001"
}