Merge pull request #7759 from hashicorp/ingress/tls-hosts
Add TLS option for Ingress Gateway listeners
This commit is contained in:
commit
28b4819882
|
@ -511,6 +511,10 @@ func (a *Agent) Start() error {
|
|||
Datacenter: a.config.Datacenter,
|
||||
Segment: a.config.SegmentName,
|
||||
},
|
||||
DNSConfig: proxycfg.DNSConfig{
|
||||
Domain: a.config.DNSDomain,
|
||||
AltDomain: a.config.DNSAltDomain,
|
||||
},
|
||||
TLSConfigurator: a.tlsConfigurator,
|
||||
})
|
||||
if err != nil {
|
||||
|
@ -4228,6 +4232,8 @@ func (a *Agent) registerCache() {
|
|||
|
||||
a.cache.RegisterType(cachetype.ConfigEntriesName, &cachetype.ConfigEntries{RPC: a})
|
||||
|
||||
a.cache.RegisterType(cachetype.ConfigEntryName, &cachetype.ConfigEntry{RPC: a})
|
||||
|
||||
a.cache.RegisterType(cachetype.ServiceHTTPChecksName, &cachetype.ServiceHTTPChecks{Agent: a})
|
||||
|
||||
a.cache.RegisterType(cachetype.FederationStateListMeshGatewaysName,
|
||||
|
|
|
@ -8,7 +8,10 @@ import (
|
|||
)
|
||||
|
||||
// Recommended name for registration.
|
||||
const ConfigEntriesName = "config-entries"
|
||||
const (
|
||||
ConfigEntriesName = "config-entries"
|
||||
ConfigEntryName = "config-entry"
|
||||
)
|
||||
|
||||
// ConfigEntries supports fetching discovering configuration entries
|
||||
type ConfigEntries struct {
|
||||
|
@ -50,3 +53,44 @@ func (c *ConfigEntries) Fetch(opts cache.FetchOptions, req cache.Request) (cache
|
|||
result.Index = reply.QueryMeta.Index
|
||||
return result, nil
|
||||
}
|
||||
|
||||
// ConfigEntry supports fetching a single configuration entry.
|
||||
type ConfigEntry struct {
|
||||
RegisterOptionsBlockingRefresh
|
||||
RPC RPC
|
||||
}
|
||||
|
||||
func (c *ConfigEntry) Fetch(opts cache.FetchOptions, req cache.Request) (cache.FetchResult, error) {
|
||||
var result cache.FetchResult
|
||||
|
||||
// The request should be a ConfigEntryQuery.
|
||||
reqReal, ok := req.(*structs.ConfigEntryQuery)
|
||||
if !ok {
|
||||
return result, fmt.Errorf(
|
||||
"Internal cache failure: request wrong type: %T", req)
|
||||
}
|
||||
|
||||
// Lightweight copy this object so that manipulating QueryOptions doesn't race.
|
||||
dup := *reqReal
|
||||
reqReal = &dup
|
||||
|
||||
// Set the minimum query index to our current index so we block
|
||||
reqReal.QueryOptions.MinQueryIndex = opts.MinIndex
|
||||
reqReal.QueryOptions.MaxQueryTime = opts.Timeout
|
||||
|
||||
// Always allow stale - there's no point in hitting leader if the request is
|
||||
// going to be served from cache and endup arbitrarily stale anyway. This
|
||||
// allows cached service-discover to automatically read scale across all
|
||||
// servers too.
|
||||
reqReal.AllowStale = true
|
||||
|
||||
// Fetch
|
||||
var reply structs.ConfigEntryResponse
|
||||
if err := c.RPC.RPC("ConfigEntry.Get", reqReal, &reply); err != nil {
|
||||
return result, err
|
||||
}
|
||||
|
||||
result.Value = &reply
|
||||
result.Index = reply.QueryMeta.Index
|
||||
return result, nil
|
||||
}
|
||||
|
|
|
@ -53,6 +53,50 @@ func TestConfigEntries(t *testing.T) {
|
|||
rpc.AssertExpectations(t)
|
||||
}
|
||||
|
||||
func TestConfigEntry(t *testing.T) {
|
||||
rpc := TestRPC(t)
|
||||
typ := &ConfigEntry{RPC: rpc}
|
||||
|
||||
// Expect the proper RPC call. This also sets the expected value
|
||||
// since that is return-by-pointer in the arguments.
|
||||
var resp *structs.ConfigEntryResponse
|
||||
rpc.On("RPC", "ConfigEntry.Get", mock.Anything, mock.Anything).Return(nil).
|
||||
Run(func(args mock.Arguments) {
|
||||
req := args.Get(1).(*structs.ConfigEntryQuery)
|
||||
require.Equal(t, uint64(24), req.QueryOptions.MinQueryIndex)
|
||||
require.Equal(t, 1*time.Second, req.QueryOptions.MaxQueryTime)
|
||||
require.True(t, req.AllowStale)
|
||||
require.Equal(t, structs.ServiceResolver, req.Kind)
|
||||
require.Equal(t, "foo", req.Name)
|
||||
|
||||
entry := &structs.ServiceResolverConfigEntry{
|
||||
Name: "foo",
|
||||
Kind: structs.ServiceResolver,
|
||||
}
|
||||
reply := args.Get(2).(*structs.ConfigEntryResponse)
|
||||
reply.Entry = entry
|
||||
reply.QueryMeta.Index = 48
|
||||
resp = reply
|
||||
})
|
||||
|
||||
// Fetch
|
||||
resultA, err := typ.Fetch(cache.FetchOptions{
|
||||
MinIndex: 24,
|
||||
Timeout: 1 * time.Second,
|
||||
}, &structs.ConfigEntryQuery{
|
||||
Datacenter: "dc1",
|
||||
Kind: structs.ServiceResolver,
|
||||
Name: "foo",
|
||||
})
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, cache.FetchResult{
|
||||
Value: resp,
|
||||
Index: 48,
|
||||
}, resultA)
|
||||
|
||||
rpc.AssertExpectations(t)
|
||||
}
|
||||
|
||||
func TestConfigEntries_badReqType(t *testing.T) {
|
||||
rpc := TestRPC(t)
|
||||
typ := &ConfigEntries{RPC: rpc}
|
||||
|
|
|
@ -521,6 +521,7 @@ func (c *ConnectCALeaf) generateNewLeaf(req *ConnectCALeafRequest,
|
|||
Service: req.Service,
|
||||
}
|
||||
commonName = connect.ServiceCN(req.Service, req.TargetNamespace(), roots.TrustDomain)
|
||||
dnsNames = append(dnsNames, req.DNSSAN...)
|
||||
} else if req.Agent != "" {
|
||||
id = &connect.SpiffeIDAgent{
|
||||
Host: roots.TrustDomain,
|
||||
|
|
|
@ -1,6 +1,8 @@
|
|||
package cachetype
|
||||
|
||||
import (
|
||||
"crypto/x509"
|
||||
"encoding/pem"
|
||||
"fmt"
|
||||
"net"
|
||||
"strings"
|
||||
|
@ -966,6 +968,53 @@ func TestConnectCALeaf_expiringLeaf(t *testing.T) {
|
|||
}
|
||||
}
|
||||
|
||||
func TestConnectCALeaf_DNSSANForService(t *testing.T) {
|
||||
t.Parallel()
|
||||
|
||||
require := require.New(t)
|
||||
rpc := TestRPC(t)
|
||||
defer rpc.AssertExpectations(t)
|
||||
|
||||
typ, rootsCh := testCALeafType(t, rpc)
|
||||
defer close(rootsCh)
|
||||
|
||||
caRoot := connect.TestCA(t, nil)
|
||||
caRoot.Active = true
|
||||
rootsCh <- structs.IndexedCARoots{
|
||||
ActiveRootID: caRoot.ID,
|
||||
TrustDomain: "fake-trust-domain.consul",
|
||||
Roots: []*structs.CARoot{
|
||||
caRoot,
|
||||
},
|
||||
QueryMeta: structs.QueryMeta{Index: 1},
|
||||
}
|
||||
|
||||
// Instrument ConnectCA.Sign to
|
||||
var caReq *structs.CASignRequest
|
||||
rpc.On("RPC", "ConnectCA.Sign", mock.Anything, mock.Anything).Return(nil).
|
||||
Run(func(args mock.Arguments) {
|
||||
reply := args.Get(2).(*structs.IssuedCert)
|
||||
leaf, _ := connect.TestLeaf(t, "web", caRoot)
|
||||
reply.CertPEM = leaf
|
||||
|
||||
caReq = args.Get(1).(*structs.CASignRequest)
|
||||
})
|
||||
|
||||
opts := cache.FetchOptions{MinIndex: 0, Timeout: 10 * time.Second}
|
||||
req := &ConnectCALeafRequest{
|
||||
Datacenter: "dc1",
|
||||
Service: "web",
|
||||
DNSSAN: []string{"test.example.com"},
|
||||
}
|
||||
_, err := typ.Fetch(opts, req)
|
||||
require.NoError(err)
|
||||
|
||||
pemBlock, _ := pem.Decode([]byte(caReq.CSR))
|
||||
csr, err := x509.ParseCertificateRequest(pemBlock.Bytes)
|
||||
require.NoError(err)
|
||||
require.Equal(csr.DNSNames, []string{"test.example.com"})
|
||||
}
|
||||
|
||||
// testConnectCaRoot wraps ConnectCARoot to disable refresh so that the gated
|
||||
// channel controls the request directly. Otherwise, we get background refreshes and
|
||||
// it screws up the ordering of the channel reads of the testGatedRootsRPC
|
||||
|
|
|
@ -65,6 +65,8 @@ type ManagerConfig struct {
|
|||
// Datacenter name into other request types that need it. This is sufficient
|
||||
// for now and cleaner than passing the entire RuntimeConfig.
|
||||
Source *structs.QuerySource
|
||||
// DNSConfig is the agent's relevant DNS config for any proxies.
|
||||
DNSConfig DNSConfig
|
||||
// logger is the agent's logger to be used for logging logs.
|
||||
Logger hclog.Logger
|
||||
TLSConfigurator *tlsutil.Configurator
|
||||
|
@ -189,6 +191,7 @@ func (m *Manager) ensureProxyServiceLocked(ns *structs.NodeService, token string
|
|||
state.logger = m.Logger
|
||||
state.cache = m.Cache
|
||||
state.source = m.Source
|
||||
state.dnsConfig = m.DNSConfig
|
||||
if m.TLSConfigurator != nil {
|
||||
state.serverSNIFn = m.TLSConfigurator.ServerSNI
|
||||
}
|
||||
|
|
|
@ -335,7 +335,7 @@ func testManager_BasicLifecycle(
|
|||
state.TriggerSyncChanges = func() {}
|
||||
|
||||
// Create manager
|
||||
m, err := NewManager(ManagerConfig{c, state, source, logger, nil})
|
||||
m, err := NewManager(ManagerConfig{c, state, source, DNSConfig{}, logger, nil})
|
||||
require.NoError(err)
|
||||
|
||||
// And run it
|
||||
|
|
|
@ -193,6 +193,19 @@ func (c *configSnapshotMeshGateway) IsEmpty() bool {
|
|||
|
||||
type configSnapshotIngressGateway struct {
|
||||
ConfigSnapshotUpstreams
|
||||
|
||||
// TLSEnabled is whether this gateway's listeners should have TLS configured.
|
||||
TLSEnabled bool
|
||||
TLSSet bool
|
||||
|
||||
// Hosts is the list of extra host entries to add to our leaf cert's DNS SANs.
|
||||
Hosts []string
|
||||
HostsSet bool
|
||||
|
||||
// LeafCertWatchCancel is a CancelFunc to use when refreshing this gateway's
|
||||
// leaf cert watch with different parameters.
|
||||
LeafCertWatchCancel context.CancelFunc
|
||||
|
||||
// Upstreams is a list of upstreams this ingress gateway should serve traffic
|
||||
// to. This is constructed from the ingress-gateway config entry, and uses
|
||||
// the GatewayServices RPC to retrieve them.
|
||||
|
@ -273,7 +286,9 @@ func (s *ConfigSnapshot) Valid() bool {
|
|||
return s.Roots != nil && (s.MeshGateway.WatchedServicesSet || len(s.MeshGateway.ServiceGroups) > 0)
|
||||
case structs.ServiceKindIngressGateway:
|
||||
return s.Roots != nil &&
|
||||
s.IngressGateway.Leaf != nil
|
||||
s.IngressGateway.Leaf != nil &&
|
||||
s.IngressGateway.TLSSet &&
|
||||
s.IngressGateway.HostsSet
|
||||
default:
|
||||
return false
|
||||
}
|
||||
|
@ -303,7 +318,9 @@ func (s *ConfigSnapshot) Clone() (*ConfigSnapshot, error) {
|
|||
snap.MeshGateway.WatchedServices = nil
|
||||
case structs.ServiceKindIngressGateway:
|
||||
snap.IngressGateway.WatchedUpstreams = nil
|
||||
snap.IngressGateway.WatchedGateways = nil
|
||||
snap.IngressGateway.WatchedDiscoveryChains = nil
|
||||
snap.IngressGateway.LeafCertWatchCancel = nil
|
||||
}
|
||||
|
||||
return snap, nil
|
||||
|
|
|
@ -33,6 +33,7 @@ const (
|
|||
datacentersWatchID = "datacenters"
|
||||
serviceResolversWatchID = "service-resolvers"
|
||||
gatewayServicesWatchID = "gateway-services"
|
||||
gatewayConfigWatchID = "gateway-config"
|
||||
externalServiceIDPrefix = "external-service:"
|
||||
serviceLeafIDPrefix = "service-leaf:"
|
||||
serviceResolverIDPrefix = "service-resolver:"
|
||||
|
@ -51,6 +52,7 @@ type state struct {
|
|||
logger hclog.Logger
|
||||
source *structs.QuerySource
|
||||
cache CacheNotifier
|
||||
dnsConfig DNSConfig
|
||||
serverSNIFn ServerSNIFunc
|
||||
|
||||
// ctx and cancel store the context created during initWatches call
|
||||
|
@ -72,6 +74,11 @@ type state struct {
|
|||
reqCh chan chan *ConfigSnapshot
|
||||
}
|
||||
|
||||
type DNSConfig struct {
|
||||
Domain string
|
||||
AltDomain string
|
||||
}
|
||||
|
||||
type ServerSNIFunc func(dc, nodeName string) string
|
||||
|
||||
func copyProxyConfig(ns *structs.NodeService) (structs.ConnectProxyConfig, error) {
|
||||
|
@ -487,13 +494,14 @@ func (s *state) initWatchesIngressGateway() error {
|
|||
return err
|
||||
}
|
||||
|
||||
// Watch the leaf cert
|
||||
err = s.cache.Notify(s.ctx, cachetype.ConnectCALeafName, &cachetype.ConnectCALeafRequest{
|
||||
// Watch this ingress gateway's config entry
|
||||
err = s.cache.Notify(s.ctx, cachetype.ConfigEntryName, &structs.ConfigEntryQuery{
|
||||
Kind: structs.IngressGateway,
|
||||
Name: s.service,
|
||||
Datacenter: s.source.Datacenter,
|
||||
Token: s.token,
|
||||
Service: s.service,
|
||||
QueryOptions: structs.QueryOptions{Token: s.token},
|
||||
EnterpriseMeta: s.proxyID.EnterpriseMeta,
|
||||
}, leafWatchID, s.ch)
|
||||
}, gatewayConfigWatchID, s.ch)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -1314,12 +1322,31 @@ func (s *state) handleUpdateIngressGateway(u cache.UpdateEvent, snap *ConfigSnap
|
|||
return fmt.Errorf("invalid type for response: %T", u.Result)
|
||||
}
|
||||
snap.Roots = roots
|
||||
case u.CorrelationID == gatewayConfigWatchID:
|
||||
resp, ok := u.Result.(*structs.ConfigEntryResponse)
|
||||
if !ok {
|
||||
return fmt.Errorf("invalid type for response: %T", u.Result)
|
||||
}
|
||||
gatewayConf, ok := resp.Entry.(*structs.IngressGatewayConfigEntry)
|
||||
if !ok {
|
||||
return fmt.Errorf("invalid type for config entry: %T", resp.Entry)
|
||||
}
|
||||
|
||||
snap.IngressGateway.TLSEnabled = gatewayConf.TLS.Enabled
|
||||
snap.IngressGateway.TLSSet = true
|
||||
|
||||
if err := s.watchIngressLeafCert(snap); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
case u.CorrelationID == gatewayServicesWatchID:
|
||||
services, ok := u.Result.(*structs.IndexedGatewayServices)
|
||||
if !ok {
|
||||
return fmt.Errorf("invalid type for response: %T", u.Result)
|
||||
}
|
||||
|
||||
// Update our upstreams and watches.
|
||||
var hosts []string
|
||||
watchedSvcs := make(map[string]struct{})
|
||||
upstreamsMap := make(map[IngressListenerKey]structs.Upstreams)
|
||||
for _, service := range services.Services {
|
||||
|
@ -1331,10 +1358,15 @@ func (s *state) handleUpdateIngressGateway(u cache.UpdateEvent, snap *ConfigSnap
|
|||
}
|
||||
watchedSvcs[u.Identifier()] = struct{}{}
|
||||
|
||||
hosts = append(hosts, service.Hosts...)
|
||||
|
||||
id := IngressListenerKey{Protocol: service.Protocol, Port: service.Port}
|
||||
upstreamsMap[id] = append(upstreamsMap[id], u)
|
||||
}
|
||||
|
||||
snap.IngressGateway.Upstreams = upstreamsMap
|
||||
snap.IngressGateway.Hosts = hosts
|
||||
snap.IngressGateway.HostsSet = true
|
||||
|
||||
for id, cancelFn := range snap.IngressGateway.WatchedDiscoveryChains {
|
||||
if _, ok := watchedSvcs[id]; !ok {
|
||||
|
@ -1343,6 +1375,10 @@ func (s *state) handleUpdateIngressGateway(u cache.UpdateEvent, snap *ConfigSnap
|
|||
}
|
||||
}
|
||||
|
||||
if err := s.watchIngressLeafCert(snap); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
default:
|
||||
return s.handleUpdateUpstreams(u, &snap.IngressGateway.ConfigSnapshotUpstreams)
|
||||
}
|
||||
|
@ -1392,6 +1428,69 @@ func (s *state) watchIngressDiscoveryChain(snap *ConfigSnapshot, u structs.Upstr
|
|||
return nil
|
||||
}
|
||||
|
||||
func (s *state) generateIngressDNSSANs(snap *ConfigSnapshot) []string {
|
||||
// Update our leaf cert watch with wildcard entries for our DNS domains as well as any
|
||||
// configured custom hostnames from the service.
|
||||
if !snap.IngressGateway.TLSEnabled {
|
||||
return nil
|
||||
}
|
||||
|
||||
var dnsNames []string
|
||||
namespaces := make(map[string]struct{})
|
||||
for _, upstreams := range snap.IngressGateway.Upstreams {
|
||||
for _, u := range upstreams {
|
||||
namespaces[u.DestinationNamespace] = struct{}{}
|
||||
}
|
||||
}
|
||||
|
||||
for ns := range namespaces {
|
||||
// The default namespace is special cased in DNS resolution, so special
|
||||
// case it here.
|
||||
if ns == structs.IntentionDefaultNamespace {
|
||||
ns = ""
|
||||
} else {
|
||||
ns = ns + "."
|
||||
}
|
||||
|
||||
dnsNames = append(dnsNames, fmt.Sprintf("*.ingress.%s%s", ns, s.dnsConfig.Domain))
|
||||
dnsNames = append(dnsNames, fmt.Sprintf("*.ingress.%s%s.%s", ns, s.source.Datacenter, s.dnsConfig.Domain))
|
||||
if s.dnsConfig.AltDomain != "" {
|
||||
dnsNames = append(dnsNames, fmt.Sprintf("*.ingress.%s%s", ns, s.dnsConfig.AltDomain))
|
||||
dnsNames = append(dnsNames, fmt.Sprintf("*.ingress.%s%s.%s", ns, s.source.Datacenter, s.dnsConfig.AltDomain))
|
||||
}
|
||||
}
|
||||
|
||||
dnsNames = append(dnsNames, snap.IngressGateway.Hosts...)
|
||||
|
||||
return dnsNames
|
||||
}
|
||||
|
||||
func (s *state) watchIngressLeafCert(snap *ConfigSnapshot) error {
|
||||
if !snap.IngressGateway.TLSSet || !snap.IngressGateway.HostsSet {
|
||||
return nil
|
||||
}
|
||||
|
||||
// Watch the leaf cert
|
||||
if snap.IngressGateway.LeafCertWatchCancel != nil {
|
||||
snap.IngressGateway.LeafCertWatchCancel()
|
||||
}
|
||||
ctx, cancel := context.WithCancel(s.ctx)
|
||||
err := s.cache.Notify(ctx, cachetype.ConnectCALeafName, &cachetype.ConnectCALeafRequest{
|
||||
Datacenter: s.source.Datacenter,
|
||||
Token: s.token,
|
||||
Service: s.service,
|
||||
DNSSAN: s.generateIngressDNSSANs(snap),
|
||||
EnterpriseMeta: s.proxyID.EnterpriseMeta,
|
||||
}, leafWatchID, s.ch)
|
||||
if err != nil {
|
||||
cancel()
|
||||
return err
|
||||
}
|
||||
snap.IngressGateway.LeafCertWatchCancel = cancel
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// CurrentSnapshot synchronously returns the current ConfigSnapshot if there is
|
||||
// one ready. If we don't have one yet because not all necessary parts have been
|
||||
// returned (i.e. both roots and leaf cert), nil is returned.
|
||||
|
|
|
@ -195,7 +195,7 @@ func verifyDatacentersWatch(t testing.TB, cacheType string, request cache.Reques
|
|||
require.True(t, ok)
|
||||
}
|
||||
|
||||
func genVerifyLeafWatch(expectedService string, expectedDatacenter string) verifyWatchRequest {
|
||||
func genVerifyLeafWatchWithDNSSANs(expectedService string, expectedDatacenter string, expectedDNSSANs []string) verifyWatchRequest {
|
||||
return func(t testing.TB, cacheType string, request cache.Request) {
|
||||
require.Equal(t, cachetype.ConnectCALeafName, cacheType)
|
||||
|
||||
|
@ -203,9 +203,14 @@ func genVerifyLeafWatch(expectedService string, expectedDatacenter string) verif
|
|||
require.True(t, ok)
|
||||
require.Equal(t, expectedDatacenter, reqReal.Datacenter)
|
||||
require.Equal(t, expectedService, reqReal.Service)
|
||||
require.ElementsMatch(t, expectedDNSSANs, reqReal.DNSSAN)
|
||||
}
|
||||
}
|
||||
|
||||
func genVerifyLeafWatch(expectedService string, expectedDatacenter string) verifyWatchRequest {
|
||||
return genVerifyLeafWatchWithDNSSANs(expectedService, expectedDatacenter, nil)
|
||||
}
|
||||
|
||||
func genVerifyResolverWatch(expectedService, expectedDatacenter, expectedKind string) verifyWatchRequest {
|
||||
return func(t testing.TB, cacheType string, request cache.Request) {
|
||||
require.Equal(t, cachetype.ConfigEntriesName, cacheType)
|
||||
|
@ -285,6 +290,36 @@ func genVerifyServiceWatch(expectedService, expectedFilter, expectedDatacenter s
|
|||
return genVerifyServiceSpecificRequest(cachetype.HealthServicesName, expectedService, expectedFilter, expectedDatacenter, connect)
|
||||
}
|
||||
|
||||
func genVerifyGatewayServiceWatch(expectedService, expectedDatacenter string) verifyWatchRequest {
|
||||
return genVerifyServiceSpecificRequest(cachetype.GatewayServicesName, expectedService, "", expectedDatacenter, false)
|
||||
}
|
||||
|
||||
func genVerifyConfigEntryWatch(expectedKind, expectedName, expectedDatacenter string) verifyWatchRequest {
|
||||
return func(t testing.TB, cacheType string, request cache.Request) {
|
||||
require.Equal(t, cachetype.ConfigEntryName, cacheType)
|
||||
|
||||
reqReal, ok := request.(*structs.ConfigEntryQuery)
|
||||
require.True(t, ok)
|
||||
require.Equal(t, expectedKind, reqReal.Kind)
|
||||
require.Equal(t, expectedName, reqReal.Name)
|
||||
require.Equal(t, expectedDatacenter, reqReal.Datacenter)
|
||||
}
|
||||
}
|
||||
|
||||
func ingressConfigWatchEvent(tlsEnabled bool) cache.UpdateEvent {
|
||||
return cache.UpdateEvent{
|
||||
CorrelationID: gatewayConfigWatchID,
|
||||
Result: &structs.ConfigEntryResponse{
|
||||
Entry: &structs.IngressGatewayConfigEntry{
|
||||
TLS: structs.GatewayTLSConfig{
|
||||
Enabled: tlsEnabled,
|
||||
},
|
||||
},
|
||||
},
|
||||
Err: nil,
|
||||
}
|
||||
}
|
||||
|
||||
// This test is meant to exercise the various parts of the cache watching done by the state as
|
||||
// well as its management of the ConfigSnapshot
|
||||
//
|
||||
|
@ -692,8 +727,9 @@ func TestState_WatchesAndUpdates(t *testing.T) {
|
|||
stages: []verificationStage{
|
||||
verificationStage{
|
||||
requiredWatches: map[string]verifyWatchRequest{
|
||||
rootsWatchID: genVerifyRootsWatch("dc1"),
|
||||
leafWatchID: genVerifyLeafWatch("ingress-gateway", "dc1"),
|
||||
rootsWatchID: genVerifyRootsWatch("dc1"),
|
||||
gatewayConfigWatchID: genVerifyConfigEntryWatch(structs.IngressGateway, "ingress-gateway", "dc1"),
|
||||
gatewayServicesWatchID: genVerifyGatewayServiceWatch("ingress-gateway", "dc1"),
|
||||
},
|
||||
verifySnapshot: func(t testing.TB, snap *ConfigSnapshot) {
|
||||
require.False(t, snap.Valid(), "gateway without root is not valid")
|
||||
|
@ -705,21 +741,18 @@ func TestState_WatchesAndUpdates(t *testing.T) {
|
|||
rootWatchEvent(),
|
||||
},
|
||||
verifySnapshot: func(t testing.TB, snap *ConfigSnapshot) {
|
||||
require.False(t, snap.Valid(), "gateway without leaf is not valid")
|
||||
require.False(t, snap.Valid(), "gateway without config entry is not valid")
|
||||
require.Equal(t, indexedRoots, snap.Roots)
|
||||
},
|
||||
},
|
||||
verificationStage{
|
||||
events: []cache.UpdateEvent{
|
||||
cache.UpdateEvent{
|
||||
CorrelationID: leafWatchID,
|
||||
Result: issuedCert,
|
||||
Err: nil,
|
||||
},
|
||||
ingressConfigWatchEvent(false),
|
||||
},
|
||||
verifySnapshot: func(t testing.TB, snap *ConfigSnapshot) {
|
||||
require.True(t, snap.Valid(), "gateway with root and leaf certs is valid")
|
||||
require.Equal(t, issuedCert, snap.IngressGateway.Leaf)
|
||||
require.False(t, snap.Valid(), "gateway without hosts set is not valid")
|
||||
require.True(t, snap.IngressGateway.TLSSet)
|
||||
require.False(t, snap.IngressGateway.TLSEnabled)
|
||||
},
|
||||
},
|
||||
verificationStage{
|
||||
|
@ -740,6 +773,9 @@ func TestState_WatchesAndUpdates(t *testing.T) {
|
|||
},
|
||||
},
|
||||
verifySnapshot: func(t testing.TB, snap *ConfigSnapshot) {
|
||||
require.False(t, snap.Valid(), "gateway without leaf is not valid")
|
||||
require.True(t, snap.IngressGateway.HostsSet)
|
||||
require.Len(t, snap.IngressGateway.Hosts, 0)
|
||||
require.Len(t, snap.IngressGateway.Upstreams, 1)
|
||||
key := IngressListenerKey{Protocol: "http", Port: 9999}
|
||||
require.Equal(t, snap.IngressGateway.Upstreams[key], structs.Upstreams{
|
||||
|
@ -757,6 +793,22 @@ func TestState_WatchesAndUpdates(t *testing.T) {
|
|||
require.Contains(t, snap.IngressGateway.WatchedDiscoveryChains, "api")
|
||||
},
|
||||
},
|
||||
verificationStage{
|
||||
requiredWatches: map[string]verifyWatchRequest{
|
||||
leafWatchID: genVerifyLeafWatch("ingress-gateway", "dc1"),
|
||||
},
|
||||
events: []cache.UpdateEvent{
|
||||
cache.UpdateEvent{
|
||||
CorrelationID: leafWatchID,
|
||||
Result: issuedCert,
|
||||
Err: nil,
|
||||
},
|
||||
},
|
||||
verifySnapshot: func(t testing.TB, snap *ConfigSnapshot) {
|
||||
require.True(t, snap.Valid(), "gateway with root and leaf certs is valid")
|
||||
require.Equal(t, issuedCert, snap.IngressGateway.Leaf)
|
||||
},
|
||||
},
|
||||
verificationStage{
|
||||
requiredWatches: map[string]verifyWatchRequest{
|
||||
"discovery-chain:api": genVerifyDiscoveryChainWatch(&structs.DiscoveryChainRequest{
|
||||
|
@ -827,7 +879,7 @@ func TestState_WatchesAndUpdates(t *testing.T) {
|
|||
},
|
||||
},
|
||||
},
|
||||
"ingress-gateway-update-upstreams": testCase{
|
||||
"ingress-gateway-with-tls-update-upstreams": testCase{
|
||||
ns: structs.NodeService{
|
||||
Kind: structs.ServiceKindIngressGateway,
|
||||
ID: "ingress-gateway",
|
||||
|
@ -838,16 +890,13 @@ func TestState_WatchesAndUpdates(t *testing.T) {
|
|||
stages: []verificationStage{
|
||||
verificationStage{
|
||||
requiredWatches: map[string]verifyWatchRequest{
|
||||
rootsWatchID: genVerifyRootsWatch("dc1"),
|
||||
leafWatchID: genVerifyLeafWatch("ingress-gateway", "dc1"),
|
||||
rootsWatchID: genVerifyRootsWatch("dc1"),
|
||||
gatewayConfigWatchID: genVerifyConfigEntryWatch(structs.IngressGateway, "ingress-gateway", "dc1"),
|
||||
gatewayServicesWatchID: genVerifyGatewayServiceWatch("ingress-gateway", "dc1"),
|
||||
},
|
||||
events: []cache.UpdateEvent{
|
||||
rootWatchEvent(),
|
||||
cache.UpdateEvent{
|
||||
CorrelationID: leafWatchID,
|
||||
Result: issuedCert,
|
||||
Err: nil,
|
||||
},
|
||||
ingressConfigWatchEvent(true),
|
||||
cache.UpdateEvent{
|
||||
CorrelationID: gatewayServicesWatchID,
|
||||
Result: &structs.IndexedGatewayServices{
|
||||
|
@ -855,22 +904,40 @@ func TestState_WatchesAndUpdates(t *testing.T) {
|
|||
{
|
||||
Gateway: structs.NewServiceID("ingress-gateway", nil),
|
||||
Service: structs.NewServiceID("api", nil),
|
||||
Hosts: []string{"test.example.com"},
|
||||
Port: 9999,
|
||||
},
|
||||
},
|
||||
},
|
||||
Err: nil,
|
||||
},
|
||||
cache.UpdateEvent{
|
||||
CorrelationID: leafWatchID,
|
||||
Result: issuedCert,
|
||||
Err: nil,
|
||||
},
|
||||
},
|
||||
verifySnapshot: func(t testing.TB, snap *ConfigSnapshot) {
|
||||
require.True(t, snap.Valid())
|
||||
require.True(t, snap.IngressGateway.TLSSet)
|
||||
require.True(t, snap.IngressGateway.TLSEnabled)
|
||||
require.True(t, snap.IngressGateway.HostsSet)
|
||||
require.Len(t, snap.IngressGateway.Hosts, 1)
|
||||
require.Len(t, snap.IngressGateway.Upstreams, 1)
|
||||
require.Len(t, snap.IngressGateway.WatchedDiscoveryChains, 1)
|
||||
require.Contains(t, snap.IngressGateway.WatchedDiscoveryChains, "api")
|
||||
},
|
||||
},
|
||||
verificationStage{
|
||||
requiredWatches: map[string]verifyWatchRequest{},
|
||||
requiredWatches: map[string]verifyWatchRequest{
|
||||
leafWatchID: genVerifyLeafWatchWithDNSSANs("ingress-gateway", "dc1", []string{
|
||||
"test.example.com",
|
||||
"*.ingress.consul.",
|
||||
"*.ingress.dc1.consul.",
|
||||
"*.ingress.alt.consul.",
|
||||
"*.ingress.dc1.alt.consul.",
|
||||
}),
|
||||
},
|
||||
events: []cache.UpdateEvent{
|
||||
cache.UpdateEvent{
|
||||
CorrelationID: gatewayServicesWatchID,
|
||||
|
@ -1164,6 +1231,11 @@ func TestState_WatchesAndUpdates(t *testing.T) {
|
|||
Datacenter: tc.sourceDC,
|
||||
}
|
||||
|
||||
state.dnsConfig = DNSConfig{
|
||||
Domain: "consul.",
|
||||
AltDomain: "alt.consul.",
|
||||
}
|
||||
|
||||
// setup the ctx as initWatches expects this to be there
|
||||
state.ctx, state.cancel = context.WithCancel(context.Background())
|
||||
|
||||
|
|
|
@ -1323,6 +1323,12 @@ func TestConfigSnapshotIngress(t testing.T) *ConfigSnapshot {
|
|||
return testConfigSnapshotIngressGateway(t, true, "tcp", "simple")
|
||||
}
|
||||
|
||||
func TestConfigSnapshotIngressWithTLSListener(t testing.T) *ConfigSnapshot {
|
||||
snap := testConfigSnapshotIngressGateway(t, true, "tcp", "default")
|
||||
snap.IngressGateway.TLSEnabled = true
|
||||
return snap
|
||||
}
|
||||
|
||||
func TestConfigSnapshotIngressWithOverrides(t testing.T) *ConfigSnapshot {
|
||||
return testConfigSnapshotIngressGateway(t, true, "tcp", "simple-with-overrides")
|
||||
}
|
||||
|
|
|
@ -6,6 +6,7 @@ import (
|
|||
|
||||
"github.com/hashicorp/consul/acl"
|
||||
"github.com/hashicorp/consul/lib"
|
||||
"github.com/miekg/dns"
|
||||
)
|
||||
|
||||
// IngressGatewayConfigEntry manages the configuration for an ingress service
|
||||
|
@ -18,6 +19,9 @@ type IngressGatewayConfigEntry struct {
|
|||
// service. This should match the name provided in the service definition.
|
||||
Name string
|
||||
|
||||
// TLS holds the TLS configuration for this gateway.
|
||||
TLS GatewayTLSConfig
|
||||
|
||||
// Listeners declares what ports the ingress gateway should listen on, and
|
||||
// what services to associated to those ports.
|
||||
Listeners []IngressListener
|
||||
|
@ -71,6 +75,11 @@ type IngressService struct {
|
|||
EnterpriseMeta `hcl:",squash" mapstructure:",squash"`
|
||||
}
|
||||
|
||||
type GatewayTLSConfig struct {
|
||||
// Indicates that TLS should be enabled for this gateway service
|
||||
Enabled bool
|
||||
}
|
||||
|
||||
func (e *IngressGatewayConfigEntry) GetKind() string {
|
||||
return IngressGateway
|
||||
}
|
||||
|
@ -157,12 +166,14 @@ func (e *IngressGatewayConfigEntry) Validate() error {
|
|||
return fmt.Errorf("Wildcard namespace is not supported for ingress services (listener on port %d)", listener.Port)
|
||||
}
|
||||
|
||||
// TODO(ingress): Validate Hosts are valid?
|
||||
for _, h := range s.Hosts {
|
||||
if declaredHosts[h] {
|
||||
return fmt.Errorf("Hosts must be unique within a specific listener (listener on port %d)", listener.Port)
|
||||
}
|
||||
declaredHosts[h] = true
|
||||
if err := validateHost(h); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -170,6 +181,23 @@ func (e *IngressGatewayConfigEntry) Validate() error {
|
|||
return nil
|
||||
}
|
||||
|
||||
func validateHost(host string) error {
|
||||
wildcardPrefix := "*."
|
||||
if _, ok := dns.IsDomainName(host); !ok {
|
||||
return fmt.Errorf("Host %q must be a valid DNS hostname", host)
|
||||
}
|
||||
|
||||
if strings.ContainsRune(strings.TrimPrefix(host, wildcardPrefix), '*') {
|
||||
return fmt.Errorf("Host %q is not valid, a wildcard specifier is only allowed as the leftmost label", host)
|
||||
}
|
||||
|
||||
if host == "*" {
|
||||
return fmt.Errorf("Host '*' is not allowed, wildcards can only be used as a prefix/suffix")
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (e *IngressGatewayConfigEntry) CanRead(authz acl.Authorizer) bool {
|
||||
var authzContext acl.AuthorizerContext
|
||||
e.FillAuthzContext(&authzContext)
|
||||
|
|
|
@ -316,6 +316,85 @@ func TestIngressConfigEntry_Validate(t *testing.T) {
|
|||
},
|
||||
expectErr: "Hosts must be unique within a specific listener",
|
||||
},
|
||||
{
|
||||
name: "hosts must be a valid DNS name",
|
||||
entry: IngressGatewayConfigEntry{
|
||||
Kind: "ingress-gateway",
|
||||
Name: "ingress-web",
|
||||
Listeners: []IngressListener{
|
||||
{
|
||||
Port: 1111,
|
||||
Protocol: "http",
|
||||
Services: []IngressService{
|
||||
{
|
||||
Name: "db",
|
||||
Hosts: []string{"example..com"},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
expectErr: `Host "example..com" must be a valid DNS hostname`,
|
||||
},
|
||||
{
|
||||
name: "wildcard specifier is only allowed in the leftmost label",
|
||||
entry: IngressGatewayConfigEntry{
|
||||
Kind: "ingress-gateway",
|
||||
Name: "ingress-web",
|
||||
Listeners: []IngressListener{
|
||||
{
|
||||
Port: 1111,
|
||||
Protocol: "http",
|
||||
Services: []IngressService{
|
||||
{
|
||||
Name: "db",
|
||||
Hosts: []string{"*.example.com"},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "wildcard specifier is not allowed in non-leftmost labels",
|
||||
entry: IngressGatewayConfigEntry{
|
||||
Kind: "ingress-gateway",
|
||||
Name: "ingress-web",
|
||||
Listeners: []IngressListener{
|
||||
{
|
||||
Port: 1111,
|
||||
Protocol: "http",
|
||||
Services: []IngressService{
|
||||
{
|
||||
Name: "db",
|
||||
Hosts: []string{"example.*.com"},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
expectErr: `Host "example.*.com" is not valid, a wildcard specifier is only allowed as the leftmost label`,
|
||||
},
|
||||
{
|
||||
name: "wildcard specifier is not allowed in leftmost labels as a partial",
|
||||
entry: IngressGatewayConfigEntry{
|
||||
Kind: "ingress-gateway",
|
||||
Name: "ingress-web",
|
||||
Listeners: []IngressListener{
|
||||
{
|
||||
Port: 1111,
|
||||
Protocol: "http",
|
||||
Services: []IngressService{
|
||||
{
|
||||
Name: "db",
|
||||
Hosts: []string{"*-test.example.com"},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
expectErr: `Host "*-test.example.com" is not valid, a wildcard specifier is only allowed as the leftmost label`,
|
||||
},
|
||||
}
|
||||
|
||||
for _, test := range cases {
|
||||
|
|
|
@ -538,6 +538,10 @@ func TestDecodeConfigEntry(t *testing.T) {
|
|||
kind = "ingress-gateway"
|
||||
name = "ingress-web"
|
||||
|
||||
tls {
|
||||
enabled = true
|
||||
}
|
||||
|
||||
listeners = [
|
||||
{
|
||||
port = 8080
|
||||
|
@ -575,6 +579,9 @@ func TestDecodeConfigEntry(t *testing.T) {
|
|||
camel: `
|
||||
Kind = "ingress-gateway"
|
||||
Name = "ingress-web"
|
||||
TLS {
|
||||
Enabled = true
|
||||
}
|
||||
Listeners = [
|
||||
{
|
||||
Port = 8080
|
||||
|
@ -612,6 +619,9 @@ func TestDecodeConfigEntry(t *testing.T) {
|
|||
expect: &IngressGatewayConfigEntry{
|
||||
Kind: "ingress-gateway",
|
||||
Name: "ingress-web",
|
||||
TLS: GatewayTLSConfig{
|
||||
Enabled: true,
|
||||
},
|
||||
Listeners: []IngressListener{
|
||||
IngressListener{
|
||||
Port: 8080,
|
||||
|
|
|
@ -4,13 +4,14 @@ import (
|
|||
"encoding/json"
|
||||
"errors"
|
||||
"fmt"
|
||||
"github.com/hashicorp/consul/logging"
|
||||
"net"
|
||||
"net/url"
|
||||
"regexp"
|
||||
"strconv"
|
||||
"strings"
|
||||
|
||||
"github.com/hashicorp/consul/logging"
|
||||
|
||||
envoy "github.com/envoyproxy/go-control-plane/envoy/api/v2"
|
||||
envoyauth "github.com/envoyproxy/go-control-plane/envoy/api/v2/auth"
|
||||
envoycore "github.com/envoyproxy/go-control-plane/envoy/api/v2/core"
|
||||
|
@ -71,9 +72,9 @@ func (s *Server) listenersFromSnapshotConnectProxy(cfgSnap *proxycfg.ConfigSnaps
|
|||
|
||||
var upstreamListener proto.Message
|
||||
if chain == nil || chain.IsDefault() {
|
||||
upstreamListener, err = s.makeUpstreamListenerIgnoreDiscoveryChain(&u, chain, cfgSnap)
|
||||
upstreamListener, err = s.makeUpstreamListenerIgnoreDiscoveryChain(&u, chain, cfgSnap, nil)
|
||||
} else {
|
||||
upstreamListener, err = s.makeUpstreamListenerForDiscoveryChain(&u, chain, cfgSnap)
|
||||
upstreamListener, err = s.makeUpstreamListenerForDiscoveryChain(&u, chain, cfgSnap, nil)
|
||||
}
|
||||
if err != nil {
|
||||
return nil, err
|
||||
|
@ -271,6 +272,14 @@ func (s *Server) listenersFromSnapshotGateway(cfgSnap *proxycfg.ConfigSnapshot,
|
|||
func (s *Server) listenersFromSnapshotIngressGateway(cfgSnap *proxycfg.ConfigSnapshot) ([]proto.Message, error) {
|
||||
var resources []proto.Message
|
||||
for listenerKey, upstreams := range cfgSnap.IngressGateway.Upstreams {
|
||||
var tlsContext *envoyauth.DownstreamTlsContext
|
||||
if cfgSnap.IngressGateway.TLSEnabled {
|
||||
tlsContext = &envoyauth.DownstreamTlsContext{
|
||||
CommonTlsContext: makeCommonTLSContextFromLeaf(cfgSnap, cfgSnap.Leaf()),
|
||||
RequireClientCertificate: &types.BoolValue{Value: false},
|
||||
}
|
||||
}
|
||||
|
||||
if listenerKey.Protocol == "tcp" {
|
||||
// We rely on the invariant of upstreams slice always having at least 1
|
||||
// member, because this key/value pair is created only when a
|
||||
|
@ -283,9 +292,9 @@ func (s *Server) listenersFromSnapshotIngressGateway(cfgSnap *proxycfg.ConfigSna
|
|||
var upstreamListener proto.Message
|
||||
var err error
|
||||
if chain == nil || chain.IsDefault() {
|
||||
upstreamListener, err = s.makeUpstreamListenerIgnoreDiscoveryChain(&u, chain, cfgSnap)
|
||||
upstreamListener, err = s.makeUpstreamListenerIgnoreDiscoveryChain(&u, chain, cfgSnap, tlsContext)
|
||||
} else {
|
||||
upstreamListener, err = s.makeUpstreamListenerForDiscoveryChain(&u, chain, cfgSnap)
|
||||
upstreamListener, err = s.makeUpstreamListenerForDiscoveryChain(&u, chain, cfgSnap, tlsContext)
|
||||
}
|
||||
if err != nil {
|
||||
return nil, err
|
||||
|
@ -310,6 +319,7 @@ func (s *Server) listenersFromSnapshotIngressGateway(cfgSnap *proxycfg.ConfigSna
|
|||
Filters: []envoylistener.Filter{
|
||||
filter,
|
||||
},
|
||||
TlsContext: tlsContext,
|
||||
},
|
||||
}
|
||||
resources = append(resources, listener)
|
||||
|
@ -536,6 +546,7 @@ func (s *Server) makeUpstreamListenerIgnoreDiscoveryChain(
|
|||
u *structs.Upstream,
|
||||
chain *structs.CompiledDiscoveryChain,
|
||||
cfgSnap *proxycfg.ConfigSnapshot,
|
||||
tlsContext *envoyauth.DownstreamTlsContext,
|
||||
) (proto.Message, error) {
|
||||
cfg, err := ParseUpstreamConfig(u.Config)
|
||||
if err != nil {
|
||||
|
@ -574,6 +585,7 @@ func (s *Server) makeUpstreamListenerIgnoreDiscoveryChain(
|
|||
Filters: []envoylistener.Filter{
|
||||
filter,
|
||||
},
|
||||
TlsContext: tlsContext,
|
||||
},
|
||||
}
|
||||
return l, nil
|
||||
|
@ -780,6 +792,7 @@ func (s *Server) makeUpstreamListenerForDiscoveryChain(
|
|||
u *structs.Upstream,
|
||||
chain *structs.CompiledDiscoveryChain,
|
||||
cfgSnap *proxycfg.ConfigSnapshot,
|
||||
tlsContext *envoyauth.DownstreamTlsContext,
|
||||
) (proto.Message, error) {
|
||||
cfg, err := ParseUpstreamConfigNoDefaults(u.Config)
|
||||
if err != nil {
|
||||
|
@ -836,6 +849,7 @@ func (s *Server) makeUpstreamListenerForDiscoveryChain(
|
|||
Filters: []envoylistener.Filter{
|
||||
filter,
|
||||
},
|
||||
TlsContext: tlsContext,
|
||||
},
|
||||
}
|
||||
return l, nil
|
||||
|
|
|
@ -398,6 +398,11 @@ func TestListenersFromSnapshot(t *testing.T) {
|
|||
snap.TerminatingGateway.ServiceLeaves[structs.NewServiceID("api", nil)] = nil
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "ingress-with-tls-listener",
|
||||
create: proxycfg.TestConfigSnapshotIngressWithTLSListener,
|
||||
setup: nil,
|
||||
},
|
||||
}
|
||||
|
||||
for _, tt := range tests {
|
||||
|
|
|
@ -0,0 +1,53 @@
|
|||
{
|
||||
"versionInfo": "00000001",
|
||||
"resources": [
|
||||
{
|
||||
"@type": "type.googleapis.com/envoy.api.v2.Listener",
|
||||
"name": "db:2.3.4.5:9191",
|
||||
"address": {
|
||||
"socketAddress": {
|
||||
"address": "2.3.4.5",
|
||||
"portValue": 9191
|
||||
}
|
||||
},
|
||||
"filterChains": [
|
||||
{
|
||||
"tlsContext": {
|
||||
"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": false
|
||||
},
|
||||
"filters": [
|
||||
{
|
||||
"name": "envoy.tcp_proxy",
|
||||
"config": {
|
||||
"cluster": "db.default.dc1.internal.11111111-2222-3333-4444-555555555555.consul",
|
||||
"stat_prefix": "upstream_db_tcp"
|
||||
}
|
||||
}
|
||||
]
|
||||
}
|
||||
]
|
||||
}
|
||||
],
|
||||
"typeUrl": "type.googleapis.com/envoy.api.v2.Listener",
|
||||
"nonce": "00000001"
|
||||
}
|
|
@ -14,6 +14,9 @@ type IngressGatewayConfigEntry struct {
|
|||
// Namespacing is a Consul Enterprise feature.
|
||||
Namespace string `json:",omitempty"`
|
||||
|
||||
// TLS holds the TLS configuration for this gateway.
|
||||
TLS GatewayTLSConfig
|
||||
|
||||
// Listeners declares what ports the ingress gateway should listen on, and
|
||||
// what services to associated to those ports.
|
||||
Listeners []IngressListener
|
||||
|
@ -28,6 +31,11 @@ type IngressGatewayConfigEntry struct {
|
|||
ModifyIndex uint64
|
||||
}
|
||||
|
||||
type GatewayTLSConfig struct {
|
||||
// Indicates that TLS should be enabled for this gateway service
|
||||
Enabled bool
|
||||
}
|
||||
|
||||
// IngressListener manages the configuration for a listener on a specific port.
|
||||
type IngressListener struct {
|
||||
// Port declares the port on which the ingress gateway should listen for traffic.
|
||||
|
|
|
@ -21,6 +21,9 @@ func TestAPI_ConfigEntries_IngressGateway(t *testing.T) {
|
|||
ingress2 := &IngressGatewayConfigEntry{
|
||||
Kind: IngressGateway,
|
||||
Name: "bar",
|
||||
TLS: GatewayTLSConfig{
|
||||
Enabled: true,
|
||||
},
|
||||
}
|
||||
|
||||
global := &ProxyConfigEntry{
|
||||
|
|
|
@ -619,6 +619,9 @@ func TestDecodeConfigEntry(t *testing.T) {
|
|||
{
|
||||
"Kind": "ingress-gateway",
|
||||
"Name": "ingress-web",
|
||||
"Tls": {
|
||||
"Enabled": true
|
||||
},
|
||||
"Listeners": [
|
||||
{
|
||||
"Port": 8080,
|
||||
|
@ -648,6 +651,9 @@ func TestDecodeConfigEntry(t *testing.T) {
|
|||
expect: &IngressGatewayConfigEntry{
|
||||
Kind: "ingress-gateway",
|
||||
Name: "ingress-web",
|
||||
TLS: GatewayTLSConfig{
|
||||
Enabled: true,
|
||||
},
|
||||
Listeners: []IngressListener{
|
||||
IngressListener{
|
||||
Port: 8080,
|
||||
|
|
|
@ -1388,6 +1388,9 @@ func TestParseConfigEntry(t *testing.T) {
|
|||
snake: `
|
||||
kind = "ingress-gateway"
|
||||
name = "ingress-web"
|
||||
tls {
|
||||
enabled = true
|
||||
}
|
||||
listeners = [
|
||||
{
|
||||
port = 8080
|
||||
|
@ -1408,6 +1411,9 @@ func TestParseConfigEntry(t *testing.T) {
|
|||
camel: `
|
||||
Kind = "ingress-gateway"
|
||||
Name = "ingress-web"
|
||||
Tls {
|
||||
Enabled = true
|
||||
}
|
||||
Listeners = [
|
||||
{
|
||||
Port = 8080
|
||||
|
@ -1429,6 +1435,9 @@ func TestParseConfigEntry(t *testing.T) {
|
|||
{
|
||||
"kind": "ingress-gateway",
|
||||
"name": "ingress-web",
|
||||
"tls": {
|
||||
"enabled": true
|
||||
},
|
||||
"listeners": [
|
||||
{
|
||||
"port": 8080,
|
||||
|
@ -1451,6 +1460,9 @@ func TestParseConfigEntry(t *testing.T) {
|
|||
{
|
||||
"Kind": "ingress-gateway",
|
||||
"Name": "ingress-web",
|
||||
"Tls": {
|
||||
"Enabled": true
|
||||
},
|
||||
"Listeners": [
|
||||
{
|
||||
"Port": 8080,
|
||||
|
@ -1472,6 +1484,9 @@ func TestParseConfigEntry(t *testing.T) {
|
|||
expect: &api.IngressGatewayConfigEntry{
|
||||
Kind: "ingress-gateway",
|
||||
Name: "ingress-web",
|
||||
TLS: api.GatewayTLSConfig{
|
||||
Enabled: true,
|
||||
},
|
||||
Listeners: []api.IngressListener{
|
||||
{
|
||||
Port: 8080,
|
||||
|
|
|
@ -0,0 +1,3 @@
|
|||
#!/bin/bash
|
||||
|
||||
snapshot_envoy_admin localhost:20000 ingress-gateway primary || true
|
|
@ -0,0 +1,3 @@
|
|||
#!/bin/bash
|
||||
|
||||
snapshot_envoy_admin localhost:20000 ingress-gateway primary || true
|
|
@ -0,0 +1,34 @@
|
|||
enable_central_service_config = true
|
||||
|
||||
config_entries {
|
||||
bootstrap = [
|
||||
{
|
||||
kind = "proxy-defaults"
|
||||
name = "global"
|
||||
config {
|
||||
protocol = "http"
|
||||
}
|
||||
},
|
||||
{
|
||||
kind = "ingress-gateway"
|
||||
name = "ingress-gateway"
|
||||
|
||||
tls {
|
||||
enabled = true
|
||||
}
|
||||
|
||||
listeners = [
|
||||
{
|
||||
port = 9999
|
||||
protocol = "http"
|
||||
services = [
|
||||
{
|
||||
name = "s1"
|
||||
hosts = ["test.example.com"]
|
||||
}
|
||||
]
|
||||
}
|
||||
]
|
||||
}
|
||||
]
|
||||
}
|
|
@ -0,0 +1,4 @@
|
|||
services {
|
||||
name = "ingress-gateway"
|
||||
kind = "ingress-gateway"
|
||||
}
|
|
@ -0,0 +1,10 @@
|
|||
#!/bin/bash
|
||||
|
||||
set -euo pipefail
|
||||
|
||||
# wait for bootstrap to apply config entries
|
||||
wait_for_config_entry ingress-gateway ingress-gateway
|
||||
|
||||
gen_envoy_bootstrap ingress-gateway 20000 primary true
|
||||
gen_envoy_bootstrap s1 19000
|
||||
gen_envoy_bootstrap s2 19001
|
|
@ -0,0 +1,3 @@
|
|||
#!/bin/bash
|
||||
|
||||
export REQUIRED_SERVICES="$DEFAULT_REQUIRED_SERVICES ingress-gateway-primary"
|
|
@ -0,0 +1,43 @@
|
|||
#!/usr/bin/env bats
|
||||
|
||||
load helpers
|
||||
|
||||
@test "ingress proxy admin is up on :20000" {
|
||||
retry_default curl -f -s localhost:20000/stats -o /dev/null
|
||||
}
|
||||
|
||||
@test "s1 proxy admin is up on :19000" {
|
||||
retry_default curl -f -s localhost:19000/stats -o /dev/null
|
||||
}
|
||||
|
||||
@test "s2 proxy admin is up on :19001" {
|
||||
retry_default curl -f -s localhost:19001/stats -o /dev/null
|
||||
}
|
||||
|
||||
@test "s1 proxy listener should be up and have right cert" {
|
||||
assert_proxy_presents_cert_uri localhost:21000 s1
|
||||
}
|
||||
|
||||
@test "ingress-gateway should have healthy endpoints for s1" {
|
||||
assert_upstream_has_endpoints_in_status 127.0.0.1:20000 s1 HEALTHY 1
|
||||
}
|
||||
|
||||
@test "should be able to connect to s1 through the TLS-enabled ingress port" {
|
||||
assert_dnssan_in_cert localhost:9999 '\*.ingress.consul'
|
||||
# Use the --resolve argument to fake dns resolution for now so we can use the
|
||||
# s1.ingress.consul domain to validate the cert
|
||||
run retry_default curl --cacert <(get_ca_root) -s -f -d hello \
|
||||
--resolve s1.ingress.consul:9999:127.0.0.1 \
|
||||
https://s1.ingress.consul:9999
|
||||
[ "$status" -eq 0 ]
|
||||
[ "$output" = "hello" ]
|
||||
}
|
||||
|
||||
@test "should be able to connect to s1 through the TLS-enabled ingress port using the custom host" {
|
||||
assert_dnssan_in_cert localhost:9999 'test.example.com'
|
||||
run retry_default curl --cacert <(get_ca_root) -s -f -d hello \
|
||||
--resolve test.example.com:9999:127.0.0.1 \
|
||||
https://test.example.com:9999
|
||||
[ "$status" -eq 0 ]
|
||||
[ "$output" = "hello" ]
|
||||
}
|
|
@ -100,7 +100,7 @@ function is_set {
|
|||
|
||||
function get_cert {
|
||||
local HOSTPORT=$1
|
||||
CERT=$(openssl s_client -connect $HOSTPORT -showcerts )
|
||||
CERT=$(openssl s_client -connect $HOSTPORT -showcerts </dev/null)
|
||||
openssl x509 -noout -text <<< "$CERT"
|
||||
}
|
||||
|
||||
|
@ -120,6 +120,19 @@ function assert_proxy_presents_cert_uri {
|
|||
echo "$CERT" | grep -Eo "URI:spiffe://([a-zA-Z0-9-]+).consul/ns/${NS}/dc/${DC}/svc/$SERVICENAME"
|
||||
}
|
||||
|
||||
function assert_dnssan_in_cert {
|
||||
local HOSTPORT=$1
|
||||
local DNSSAN=$2
|
||||
|
||||
CERT=$(retry_default get_cert $HOSTPORT)
|
||||
|
||||
echo "WANT DNSSAN: ${DNSSAN}"
|
||||
echo "GOT CERT:"
|
||||
echo "$CERT"
|
||||
|
||||
echo "$CERT" | grep -Eo "DNS:${DNSSAN}"
|
||||
}
|
||||
|
||||
function assert_envoy_version {
|
||||
local ADMINPORT=$1
|
||||
run retry_default curl -f -s localhost:$ADMINPORT/server_info
|
||||
|
@ -619,6 +632,10 @@ function update_intention {
|
|||
return $?
|
||||
}
|
||||
|
||||
function get_ca_root {
|
||||
curl -s -f "http://localhost:8500/v1/connect/ca/roots" | jq -r ".Roots[0].RootCert"
|
||||
}
|
||||
|
||||
function wait_for_agent_service_register {
|
||||
local SERVICE_ID=$1
|
||||
local DC=${2:-primary}
|
||||
|
|
Loading…
Reference in New Issue