Connect transparent proxy support

Add support for Consul Connect transparent proxies

Fixes: https://github.com/hashicorp/nomad/issues/10628
This commit is contained in:
Tim Gross
2024-04-10 11:00:18 -04:00
committed by GitHub
38 changed files with 2066 additions and 185 deletions

View File

@@ -158,12 +158,17 @@ func (st *SidecarTask) Canonicalize() {
// ConsulProxy represents a Consul Connect sidecar proxy jobspec block.
type ConsulProxy struct {
LocalServiceAddress string `mapstructure:"local_service_address" hcl:"local_service_address,optional"`
LocalServicePort int `mapstructure:"local_service_port" hcl:"local_service_port,optional"`
Expose *ConsulExposeConfig `mapstructure:"expose" hcl:"expose,block"`
ExposeConfig *ConsulExposeConfig // Deprecated: only to maintain backwards compatibility. Use Expose instead.
Upstreams []*ConsulUpstream `hcl:"upstreams,block"`
Config map[string]interface{} `hcl:"config,block"`
LocalServiceAddress string `mapstructure:"local_service_address" hcl:"local_service_address,optional"`
LocalServicePort int `mapstructure:"local_service_port" hcl:"local_service_port,optional"`
Expose *ConsulExposeConfig `mapstructure:"expose" hcl:"expose,block"`
ExposeConfig *ConsulExposeConfig // Deprecated: only to maintain backwards compatibility. Use Expose instead.
Upstreams []*ConsulUpstream `hcl:"upstreams,block"`
// TransparentProxy configures the Envoy sidecar to use "transparent
// proxying", which creates IP tables rules inside the network namespace to
// ensure traffic flows thru the Envoy proxy
TransparentProxy *ConsulTransparentProxy `mapstructure:"transparent_proxy" hcl:"transparent_proxy,block"`
Config map[string]interface{} `hcl:"config,block"`
}
func (cp *ConsulProxy) Canonicalize() {
@@ -177,6 +182,8 @@ func (cp *ConsulProxy) Canonicalize() {
cp.Upstreams = nil
}
cp.TransparentProxy.Canonicalize()
for _, upstream := range cp.Upstreams {
upstream.Canonicalize()
}
@@ -257,6 +264,61 @@ func (cu *ConsulUpstream) Canonicalize() {
}
}
// ConsulTransparentProxy is used to configure the Envoy sidecar for
// "transparent proxying", which creates IP tables rules inside the network
// namespace to ensure traffic flows thru the Envoy proxy
type ConsulTransparentProxy struct {
// UID of the Envoy proxy. Defaults to the default Envoy proxy container
// image user.
UID string `mapstructure:"uid" hcl:"uid,optional"`
// OutboundPort is the Envoy proxy's outbound listener port. Inbound TCP
// traffic hitting the PROXY_IN_REDIRECT chain will be redirected here.
// Defaults to 15001.
OutboundPort uint16 `mapstructure:"outbound_port" hcl:"outbound_port,optional"`
// ExcludeInboundPorts is an additional set of ports will be excluded from
// redirection to the Envoy proxy. Can be Port.Label or Port.Value. This set
// will be added to the ports automatically excluded for the Expose.Port and
// Check.Expose fields.
ExcludeInboundPorts []string `mapstructure:"exclude_inbound_ports" hcl:"exclude_inbound_ports,optional"`
// ExcludeOutboundPorts is a set of outbound ports that will not be
// redirected to the Envoy proxy, specified as port numbers.
ExcludeOutboundPorts []uint16 `mapstructure:"exclude_outbound_ports" hcl:"exclude_outbound_ports,optional"`
// ExcludeOutboundCIDRs is a set of outbound CIDR blocks that will not be
// redirected to the Envoy proxy.
ExcludeOutboundCIDRs []string `mapstructure:"exclude_outbound_cidrs" hcl:"exclude_outbound_cidrs,optional"`
// ExcludeUIDs is a set of user IDs whose network traffic will not be
// redirected through the Envoy proxy.
ExcludeUIDs []string `mapstructure:"exclude_uids" hcl:"exclude_uids,optional"`
// NoDNS disables redirection of DNS traffic to Consul DNS. By default NoDNS
// is false and transparent proxy will direct DNS traffic to Consul DNS if
// available on the client.
NoDNS bool `mapstructure:"no_dns" hcl:"no_dns,optional"`
}
func (tp *ConsulTransparentProxy) Canonicalize() {
if tp == nil {
return
}
if len(tp.ExcludeInboundPorts) == 0 {
tp.ExcludeInboundPorts = nil
}
if len(tp.ExcludeOutboundCIDRs) == 0 {
tp.ExcludeOutboundCIDRs = nil
}
if len(tp.ExcludeOutboundPorts) == 0 {
tp.ExcludeOutboundPorts = nil
}
if len(tp.ExcludeUIDs) == 0 {
tp.ExcludeUIDs = nil
}
}
type ConsulExposeConfig struct {
Paths []*ConsulExposePath `mapstructure:"path" hcl:"path,block"`
Path []*ConsulExposePath // Deprecated: only to maintain backwards compatibility. Use Paths instead.

View File

@@ -190,13 +190,13 @@ func newNetworkConfigurator(log hclog.Logger, alloc *structs.Allocation, config
switch {
case netMode == "bridge":
c, err := newBridgeNetworkConfigurator(log, config.BridgeNetworkName, config.BridgeNetworkAllocSubnet, config.BridgeNetworkHairpinMode, config.CNIPath, ignorePortMappingHostIP)
c, err := newBridgeNetworkConfigurator(log, alloc, config.BridgeNetworkName, config.BridgeNetworkAllocSubnet, config.BridgeNetworkHairpinMode, config.CNIPath, ignorePortMappingHostIP, config.Node)
if err != nil {
return nil, err
}
return &synchronizedNetworkConfigurator{c}, nil
case strings.HasPrefix(netMode, "cni/"):
c, err := newCNINetworkConfigurator(log, config.CNIPath, config.CNIInterfacePrefix, config.CNIConfigDir, netMode[4:], ignorePortMappingHostIP)
c, err := newCNINetworkConfigurator(log, config.CNIPath, config.CNIInterfacePrefix, config.CNIConfigDir, netMode[4:], ignorePortMappingHostIP, config.Node)
if err != nil {
return nil, err
}

View File

@@ -43,7 +43,7 @@ type bridgeNetworkConfigurator struct {
logger hclog.Logger
}
func newBridgeNetworkConfigurator(log hclog.Logger, bridgeName, ipRange string, hairpinMode bool, cniPath string, ignorePortMappingHostIP bool) (*bridgeNetworkConfigurator, error) {
func newBridgeNetworkConfigurator(log hclog.Logger, alloc *structs.Allocation, bridgeName, ipRange string, hairpinMode bool, cniPath string, ignorePortMappingHostIP bool, node *structs.Node) (*bridgeNetworkConfigurator, error) {
b := &bridgeNetworkConfigurator{
bridgeName: bridgeName,
allocSubnet: ipRange,
@@ -59,7 +59,20 @@ func newBridgeNetworkConfigurator(log hclog.Logger, bridgeName, ipRange string,
b.allocSubnet = defaultNomadAllocSubnet
}
c, err := newCNINetworkConfiguratorWithConf(log, cniPath, bridgeNetworkAllocIfPrefix, ignorePortMappingHostIP, buildNomadBridgeNetConfig(*b))
var netCfg []byte
tg := alloc.Job.LookupTaskGroup(alloc.TaskGroup)
for _, svc := range tg.Services {
if svc.Connect.HasTransparentProxy() {
netCfg = buildNomadBridgeNetConfig(*b, true)
break
}
}
if netCfg == nil {
netCfg = buildNomadBridgeNetConfig(*b, false)
}
c, err := newCNINetworkConfiguratorWithConf(log, cniPath, bridgeNetworkAllocIfPrefix, ignorePortMappingHostIP, netCfg, node)
if err != nil {
return nil, err
}
@@ -139,12 +152,19 @@ func (b *bridgeNetworkConfigurator) Teardown(ctx context.Context, alloc *structs
return b.cni.Teardown(ctx, alloc, spec)
}
func buildNomadBridgeNetConfig(b bridgeNetworkConfigurator) []byte {
func buildNomadBridgeNetConfig(b bridgeNetworkConfigurator, withConsulCNI bool) []byte {
var consulCNI string
if withConsulCNI {
consulCNI = consulCNIBlock
}
return []byte(fmt.Sprintf(nomadCNIConfigTemplate,
b.bridgeName,
b.hairpinMode,
b.allocSubnet,
cniAdminChainName))
cniAdminChainName,
consulCNI,
))
}
// Update website/content/docs/networking/cni.mdx when the bridge configuration
@@ -187,7 +207,14 @@ const nomadCNIConfigTemplate = `{
"type": "portmap",
"capabilities": {"portMappings": true},
"snat": true
}
}%s
]
}
`
const consulCNIBlock = `,
{
"type": "consul-cni",
"log_level": "debug"
}
`

View File

@@ -14,8 +14,9 @@ import (
func Test_buildNomadBridgeNetConfig(t *testing.T) {
ci.Parallel(t)
testCases := []struct {
name string
b *bridgeNetworkConfigurator
name string
withConsulCNI bool
b *bridgeNetworkConfigurator
}{
{
name: "empty",
@@ -38,14 +39,28 @@ func Test_buildNomadBridgeNetConfig(t *testing.T) {
hairpinMode: true,
},
},
{
name: "consul-cni",
withConsulCNI: true,
b: &bridgeNetworkConfigurator{
bridgeName: defaultNomadBridgeName,
allocSubnet: defaultNomadAllocSubnet,
hairpinMode: true,
},
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
tc := tc
ci.Parallel(t)
bCfg := buildNomadBridgeNetConfig(*tc.b)
bCfg := buildNomadBridgeNetConfig(*tc.b, tc.withConsulCNI)
// Validate that the JSON created is rational
must.True(t, json.Valid(bCfg))
if tc.withConsulCNI {
must.StrContains(t, string(bCfg), "consul-cni")
} else {
must.StrNotContains(t, string(bCfg), "consul-cni")
}
})
}
}

View File

@@ -16,14 +16,20 @@ import (
"os"
"path/filepath"
"regexp"
"slices"
"sort"
"strconv"
"strings"
"time"
cni "github.com/containerd/go-cni"
cnilibrary "github.com/containernetworking/cni/libcni"
"github.com/coreos/go-iptables/iptables"
consulIPTables "github.com/hashicorp/consul/sdk/iptables"
log "github.com/hashicorp/go-hclog"
"github.com/hashicorp/go-set/v2"
"github.com/hashicorp/nomad/helper"
"github.com/hashicorp/nomad/helper/envoy"
"github.com/hashicorp/nomad/nomad/structs"
"github.com/hashicorp/nomad/plugins/drivers"
)
@@ -47,26 +53,30 @@ type cniNetworkConfigurator struct {
cni cni.CNI
cniConf []byte
ignorePortMappingHostIP bool
nodeAttrs map[string]string
nodeMeta map[string]string
rand *rand.Rand
logger log.Logger
}
func newCNINetworkConfigurator(logger log.Logger, cniPath, cniInterfacePrefix, cniConfDir, networkName string, ignorePortMappingHostIP bool) (*cniNetworkConfigurator, error) {
func newCNINetworkConfigurator(logger log.Logger, cniPath, cniInterfacePrefix, cniConfDir, networkName string, ignorePortMappingHostIP bool, node *structs.Node) (*cniNetworkConfigurator, error) {
cniConf, err := loadCNIConf(cniConfDir, networkName)
if err != nil {
return nil, fmt.Errorf("failed to load CNI config: %v", err)
}
return newCNINetworkConfiguratorWithConf(logger, cniPath, cniInterfacePrefix, ignorePortMappingHostIP, cniConf)
return newCNINetworkConfiguratorWithConf(logger, cniPath, cniInterfacePrefix, ignorePortMappingHostIP, cniConf, node)
}
func newCNINetworkConfiguratorWithConf(logger log.Logger, cniPath, cniInterfacePrefix string, ignorePortMappingHostIP bool, cniConf []byte) (*cniNetworkConfigurator, error) {
func newCNINetworkConfiguratorWithConf(logger log.Logger, cniPath, cniInterfacePrefix string, ignorePortMappingHostIP bool, cniConf []byte, node *structs.Node) (*cniNetworkConfigurator, error) {
conf := &cniNetworkConfigurator{
cniConf: cniConf,
rand: rand.New(rand.NewSource(time.Now().Unix())),
logger: logger,
ignorePortMappingHostIP: ignorePortMappingHostIP,
nodeAttrs: node.Attributes,
nodeMeta: node.Meta,
}
if cniPath == "" {
if cniPath = os.Getenv(envCNIPath); cniPath == "" {
@@ -88,11 +98,35 @@ func newCNINetworkConfiguratorWithConf(logger log.Logger, cniPath, cniInterfaceP
return conf, nil
}
const (
ConsulIPTablesConfigEnvVar = "CONSUL_IPTABLES_CONFIG"
)
// Setup calls the CNI plugins with the add action
func (c *cniNetworkConfigurator) Setup(ctx context.Context, alloc *structs.Allocation, spec *drivers.NetworkIsolationSpec) (*structs.AllocNetworkStatus, error) {
if err := c.ensureCNIInitialized(); err != nil {
return nil, err
}
cniArgs := map[string]string{
// CNI plugins are called one after the other with the same set of
// arguments. Passing IgnoreUnknown=true signals to plugins that they
// should ignore any arguments they don't understand
"IgnoreUnknown": "true",
}
portMaps := getPortMapping(alloc, c.ignorePortMappingHostIP)
tproxyArgs, err := c.setupTransparentProxyArgs(alloc, spec, portMaps)
if err != nil {
return nil, err
}
if tproxyArgs != nil {
iptablesCfg, err := json.Marshal(tproxyArgs)
if err != nil {
return nil, err
}
cniArgs[ConsulIPTablesConfigEnvVar] = string(iptablesCfg)
}
// Depending on the version of bridge cni plugin used, a known race could occure
// where two alloc attempt to create the nomad bridge at the same time, resulting
@@ -102,7 +136,10 @@ func (c *cniNetworkConfigurator) Setup(ctx context.Context, alloc *structs.Alloc
var res *cni.Result
for attempt := 1; ; attempt++ {
var err error
if res, err = c.cni.Setup(ctx, alloc.ID, spec.Path, cni.WithCapabilityPortMap(getPortMapping(alloc, c.ignorePortMappingHostIP))); err != nil {
if res, err = c.cni.Setup(ctx, alloc.ID, spec.Path,
cni.WithCapabilityPortMap(portMaps.ports),
cni.WithLabels(cniArgs), // "labels" turn into CNI_ARGS
); err != nil {
c.logger.Warn("failed to configure network", "error", err, "attempt", attempt)
switch attempt {
case 1:
@@ -123,8 +160,199 @@ func (c *cniNetworkConfigurator) Setup(ctx context.Context, alloc *structs.Alloc
c.logger.Debug("received result from CNI", "result", string(resultJSON))
}
return c.cniToAllocNet(res)
allocNet, err := c.cniToAllocNet(res)
if err != nil {
return nil, err
}
// overwrite the nameservers with Consul DNS, if we have it; we don't need
// the port because the iptables rule redirects port 53 traffic to it
if tproxyArgs != nil && tproxyArgs.ConsulDNSIP != "" {
if allocNet.DNS == nil {
allocNet.DNS = &structs.DNSConfig{
Servers: []string{},
Searches: []string{},
Options: []string{},
}
}
allocNet.DNS.Servers = []string{tproxyArgs.ConsulDNSIP}
}
return allocNet, nil
}
// setupTransparentProxyArgs returns a Consul SDK iptables configuration if the
// allocation has a transparent_proxy block
func (c *cniNetworkConfigurator) setupTransparentProxyArgs(alloc *structs.Allocation, spec *drivers.NetworkIsolationSpec, portMaps *portMappings) (*consulIPTables.Config, error) {
var tproxy *structs.ConsulTransparentProxy
var cluster string
var proxyUID string
var proxyInboundPort int
var proxyOutboundPort int
var exposePorts []string
outboundPorts := []string{}
tg := alloc.Job.LookupTaskGroup(alloc.TaskGroup)
for _, svc := range tg.Services {
if svc.Connect.HasTransparentProxy() {
tproxy = svc.Connect.SidecarService.Proxy.TransparentProxy
cluster = svc.Cluster
// The default value matches the Envoy UID. The cluster admin can
// set this value to something non-default if they have a custom
// Envoy container with a different UID
proxyUID = c.nodeMeta[envoy.DefaultTransparentProxyUIDParam]
if tproxy.UID != "" {
proxyUID = tproxy.UID
}
// The value for the outbound Envoy port. The default value matches
// the default TransparentProxy service default for
// OutboundListenerPort. If the cluster admin sets this value to
// something non-default, they'll need to update the metadata on all
// the nodes to match. see also:
// https://developer.hashicorp.com/consul/docs/connect/config-entries/service-defaults#transparentproxy
if tproxy.OutboundPort != 0 {
proxyOutboundPort = int(tproxy.OutboundPort)
} else {
outboundPortAttr := c.nodeMeta[envoy.DefaultTransparentProxyOutboundPortParam]
parsedOutboundPort, err := strconv.ParseInt(outboundPortAttr, 10, 32)
if err != nil {
return nil, fmt.Errorf(
"could not parse default_outbound_port %q as port number: %w",
outboundPortAttr, err)
}
proxyOutboundPort = int(parsedOutboundPort)
}
// The inbound port is the service port exposed on the Envoy proxy
envoyPortLabel := "connect-proxy-" + svc.Name
if envoyPort, ok := portMaps.get(envoyPortLabel); ok {
proxyInboundPort = int(envoyPort.HostPort)
}
// Extra user-defined ports that get excluded from outbound redirect
if len(tproxy.ExcludeOutboundPorts) == 0 {
outboundPorts = nil
} else {
outboundPorts = helper.ConvertSlice(tproxy.ExcludeOutboundPorts,
func(p uint16) string { return fmt.Sprint(p) })
}
// The set of ports we'll exclude from inbound redirection
exposePortSet := set.From(exposePorts)
// We always expose reserved ports so that the allocation is
// reachable from the outside world.
for _, network := range tg.Networks {
for _, port := range network.ReservedPorts {
exposePortSet.Insert(fmt.Sprint(port.To))
}
}
// ExcludeInboundPorts can be either a numeric port number or a port
// label that we need to convert into a port number
for _, portLabel := range tproxy.ExcludeInboundPorts {
if _, err := strconv.ParseUint(portLabel, 10, 64); err == nil {
exposePortSet.Insert(portLabel)
continue
}
if port, ok := portMaps.get(portLabel); ok {
exposePortSet.Insert(
strconv.FormatInt(int64(port.ContainerPort), 10))
}
}
// We also exclude Expose.Paths. Any health checks with expose=true
// will have an Expose block added by the server, so this allows
// health checks to work as expected without passing thru Envoy
if svc.Connect.SidecarService.Proxy.Expose != nil {
for _, path := range svc.Connect.SidecarService.Proxy.Expose.Paths {
if port, ok := portMaps.get(path.ListenerPort); ok {
exposePortSet.Insert(
strconv.FormatInt(int64(port.ContainerPort), 10))
}
}
}
if exposePortSet.Size() > 0 {
exposePorts = exposePortSet.Slice()
slices.Sort(exposePorts)
}
// Only one Connect block is allowed with tproxy. This will have
// been validated on job registration
break
}
}
if tproxy != nil {
var dnsAddr string
var dnsPort int
if !tproxy.NoDNS {
dnsAddr, dnsPort = c.dnsFromAttrs(cluster)
}
consulIPTablesCfgMap := &consulIPTables.Config{
// Traffic in the DNSChain is directed to the Consul DNS Service IP.
// For outbound TCP and UDP traffic going to port 53 (DNS), jump to
// the DNSChain. Only redirect traffic that's going to consul's DNS
// IP.
ConsulDNSIP: dnsAddr,
ConsulDNSPort: dnsPort,
// Don't redirect proxy traffic back to itself, return it to the
// next chain for processing.
ProxyUserID: proxyUID,
// Redirects inbound TCP traffic hitting the PROXY_IN_REDIRECT chain
// to Envoy's inbound listener port.
ProxyInboundPort: proxyInboundPort,
// Redirects outbound TCP traffic hitting PROXY_REDIRECT chain to
// Envoy's outbound listener port.
ProxyOutboundPort: proxyOutboundPort,
ExcludeInboundPorts: exposePorts,
ExcludeOutboundPorts: outboundPorts,
ExcludeOutboundCIDRs: tproxy.ExcludeOutboundCIDRs,
ExcludeUIDs: tproxy.ExcludeUIDs,
NetNS: spec.Path,
}
return consulIPTablesCfgMap, nil
}
return nil, nil
}
func (c *cniNetworkConfigurator) dnsFromAttrs(cluster string) (string, int) {
var dnsAddrAttr, dnsPortAttr string
if cluster == structs.ConsulDefaultCluster || cluster == "" {
dnsAddrAttr = "consul.dns.addr"
dnsPortAttr = "consul.dns.port"
} else {
dnsAddrAttr = "consul." + cluster + ".dns.addr"
dnsPortAttr = "consul." + cluster + ".dns.port"
}
dnsAddr, ok := c.nodeAttrs[dnsAddrAttr]
if !ok || dnsAddr == "" {
return "", 0
}
dnsPort, ok := c.nodeAttrs[dnsPortAttr]
if !ok || dnsPort == "0" || dnsPort == "-1" {
return "", 0
}
port, err := strconv.ParseInt(dnsPort, 10, 64)
if err != nil {
return "", 0 // note: this will have been checked in fingerprint
}
return dnsAddr, int(port)
}
// cniToAllocNet converts a cni.Result to an AllocNetworkStatus or returns an
@@ -240,7 +468,9 @@ func (c *cniNetworkConfigurator) Teardown(ctx context.Context, alloc *structs.Al
return err
}
if err := c.cni.Remove(ctx, alloc.ID, spec.Path, cni.WithCapabilityPortMap(getPortMapping(alloc, c.ignorePortMappingHostIP))); err != nil {
portMap := getPortMapping(alloc, c.ignorePortMappingHostIP)
if err := c.cni.Remove(ctx, alloc.ID, spec.Path, cni.WithCapabilityPortMap(portMap.ports)); err != nil {
// create a real handle to iptables
ipt, iptErr := iptables.New()
if iptErr != nil {
@@ -345,10 +575,34 @@ func (c *cniNetworkConfigurator) ensureCNIInitialized() error {
}
}
// getPortMapping builds a list of portMapping structs that are used as the
// portMappings is a wrapper around a slice of cni.PortMapping that lets us
// index via the port's label, which isn't otherwise included in the
// cni.PortMapping struct
type portMappings struct {
ports []cni.PortMapping
labels map[string]int // Label -> index into ports field
}
func (pm *portMappings) set(label string, port cni.PortMapping) {
pm.ports = append(pm.ports, port)
pm.labels[label] = len(pm.ports) - 1
}
func (pm *portMappings) get(label string) (cni.PortMapping, bool) {
idx, ok := pm.labels[label]
if !ok {
return cni.PortMapping{}, false
}
return pm.ports[idx], true
}
// getPortMapping builds a list of cni.PortMapping structs that are used as the
// portmapping capability arguments for the portmap CNI plugin
func getPortMapping(alloc *structs.Allocation, ignoreHostIP bool) []cni.PortMapping {
var ports []cni.PortMapping
func getPortMapping(alloc *structs.Allocation, ignoreHostIP bool) *portMappings {
mappings := &portMappings{
ports: []cni.PortMapping{},
labels: map[string]int{},
}
if len(alloc.AllocatedResources.Shared.Ports) == 0 && len(alloc.AllocatedResources.Shared.Networks) > 0 {
for _, network := range alloc.AllocatedResources.Shared.Networks {
@@ -357,11 +611,12 @@ func getPortMapping(alloc *structs.Allocation, ignoreHostIP bool) []cni.PortMapp
port.To = port.Value
}
for _, proto := range []string{"tcp", "udp"} {
ports = append(ports, cni.PortMapping{
portMapping := cni.PortMapping{
HostPort: int32(port.Value),
ContainerPort: int32(port.To),
Protocol: proto,
})
}
mappings.set(port.Label, portMapping)
}
}
}
@@ -371,6 +626,7 @@ func getPortMapping(alloc *structs.Allocation, ignoreHostIP bool) []cni.PortMapp
port.To = port.Value
}
for _, proto := range []string{"tcp", "udp"} {
portMapping := cni.PortMapping{
HostPort: int32(port.Value),
ContainerPort: int32(port.To),
@@ -379,9 +635,9 @@ func getPortMapping(alloc *structs.Allocation, ignoreHostIP bool) []cni.PortMapp
if !ignoreHostIP {
portMapping.HostIP = port.HostIP
}
ports = append(ports, portMapping)
mappings.set(port.Label, portMapping)
}
}
}
return ports
return mappings
}

View File

@@ -12,8 +12,12 @@ import (
"github.com/containerd/go-cni"
"github.com/containernetworking/cni/pkg/types"
"github.com/hashicorp/consul/sdk/iptables"
"github.com/hashicorp/nomad/ci"
"github.com/hashicorp/nomad/helper/testlog"
"github.com/hashicorp/nomad/nomad/mock"
"github.com/hashicorp/nomad/nomad/structs"
"github.com/hashicorp/nomad/plugins/drivers"
"github.com/shoenig/test"
"github.com/shoenig/test/must"
"github.com/stretchr/testify/require"
@@ -200,3 +204,237 @@ func TestCNI_cniToAllocNet_Invalid(t *testing.T) {
require.Error(t, err)
require.Nil(t, allocNet)
}
func TestCNI_setupTproxyArgs(t *testing.T) {
ci.Parallel(t)
nodeMeta := map[string]string{
"connect.transparent_proxy.default_outbound_port": "15001",
"connect.transparent_proxy.default_uid": "101",
}
nodeAttrs := map[string]string{
"consul.dns.addr": "192.168.1.117",
"consul.dns.port": "8600",
}
alloc := mock.ConnectAlloc()
// need to setup the NetworkResource to have the expected port mapping for
// the services we create
alloc.AllocatedResources.Shared.Networks = []*structs.NetworkResource{
{
Mode: "bridge",
IP: "10.0.0.1",
ReservedPorts: []structs.Port{
{
Label: "http",
Value: 9002,
To: 9002,
},
{
Label: "health",
Value: 9001,
To: 9000,
},
},
DynamicPorts: []structs.Port{
{
Label: "connect-proxy-testconnect",
Value: 25018,
To: 25018,
},
},
},
}
tg := alloc.Job.LookupTaskGroup(alloc.TaskGroup)
tg.Networks = []*structs.NetworkResource{{
Mode: "bridge",
DNS: &structs.DNSConfig{},
ReservedPorts: []structs.Port{ // non-Connect port
{
Label: "http",
Value: 9002,
To: 9002,
HostNetwork: "default",
},
},
DynamicPorts: []structs.Port{ // Connect port
{
Label: "connect-proxy-count-dashboard",
Value: 0,
To: -1,
HostNetwork: "default",
},
{
Label: "health",
Value: 0,
To: 9000,
HostNetwork: "default",
},
},
}}
tg.Services[0].PortLabel = "9002"
tg.Services[0].Connect.SidecarService.Proxy = &structs.ConsulProxy{
LocalServiceAddress: "",
LocalServicePort: 0,
Upstreams: []structs.ConsulUpstream{},
Expose: &structs.ConsulExposeConfig{},
Config: map[string]interface{}{},
}
spec := &drivers.NetworkIsolationSpec{
Mode: "group",
Path: "/var/run/docker/netns/a2ece01ea7bc",
Labels: map[string]string{"docker_sandbox_container_id": "4a77cdaad5"},
HostsConfig: &drivers.HostsConfig{},
}
portMaps := getPortMapping(alloc, false)
testCases := []struct {
name string
cluster string
tproxySpec *structs.ConsulTransparentProxy
exposeSpec *structs.ConsulExposeConfig
nodeAttrs map[string]string
expectIPConfig *iptables.Config
expectErr string
}{
{
name: "nil tproxy spec returns no error or iptables config",
},
{
name: "minimal empty tproxy spec returns defaults",
tproxySpec: &structs.ConsulTransparentProxy{},
expectIPConfig: &iptables.Config{
ConsulDNSIP: "192.168.1.117",
ConsulDNSPort: 8600,
ProxyUserID: "101",
ProxyInboundPort: 25018,
ProxyOutboundPort: 15001,
ExcludeInboundPorts: []string{"9002"},
NetNS: "/var/run/docker/netns/a2ece01ea7bc",
},
},
{
name: "tproxy spec with overrides",
tproxySpec: &structs.ConsulTransparentProxy{
UID: "1001",
OutboundPort: 16001,
ExcludeInboundPorts: []string{"http", "9000"},
ExcludeOutboundPorts: []uint16{443, 80},
ExcludeOutboundCIDRs: []string{"10.0.0.1/8"},
ExcludeUIDs: []string{"10", "42"},
NoDNS: true,
},
expectIPConfig: &iptables.Config{
ProxyUserID: "1001",
ProxyInboundPort: 25018,
ProxyOutboundPort: 16001,
ExcludeInboundPorts: []string{"9000", "9002"},
ExcludeOutboundCIDRs: []string{"10.0.0.1/8"},
ExcludeOutboundPorts: []string{"443", "80"},
ExcludeUIDs: []string{"10", "42"},
NetNS: "/var/run/docker/netns/a2ece01ea7bc",
},
},
{
name: "tproxy with exposed checks",
tproxySpec: &structs.ConsulTransparentProxy{},
exposeSpec: &structs.ConsulExposeConfig{
Paths: []structs.ConsulExposePath{{
Path: "/v1/example",
Protocol: "http",
LocalPathPort: 9000,
ListenerPort: "health",
}},
},
expectIPConfig: &iptables.Config{
ConsulDNSIP: "192.168.1.117",
ConsulDNSPort: 8600,
ProxyUserID: "101",
ProxyInboundPort: 25018,
ProxyOutboundPort: 15001,
ExcludeInboundPorts: []string{"9000", "9002"},
NetNS: "/var/run/docker/netns/a2ece01ea7bc",
},
},
{
name: "tproxy with no consul dns fingerprint",
nodeAttrs: map[string]string{},
tproxySpec: &structs.ConsulTransparentProxy{},
expectIPConfig: &iptables.Config{
ProxyUserID: "101",
ProxyInboundPort: 25018,
ProxyOutboundPort: 15001,
ExcludeInboundPorts: []string{"9002"},
NetNS: "/var/run/docker/netns/a2ece01ea7bc",
},
},
{
name: "tproxy with consul dns disabled",
nodeAttrs: map[string]string{
"consul.dns.port": "-1",
"consul.dns.addr": "192.168.1.117",
},
tproxySpec: &structs.ConsulTransparentProxy{},
expectIPConfig: &iptables.Config{
ProxyUserID: "101",
ProxyInboundPort: 25018,
ProxyOutboundPort: 15001,
ExcludeInboundPorts: []string{"9002"},
NetNS: "/var/run/docker/netns/a2ece01ea7bc",
},
},
{
name: "tproxy for other cluster with default consul dns disabled",
cluster: "infra",
nodeAttrs: map[string]string{
"consul.dns.port": "-1",
"consul.dns.addr": "192.168.1.110",
"consul.infra.dns.port": "8600",
"consul.infra.dns.addr": "192.168.1.117",
},
tproxySpec: &structs.ConsulTransparentProxy{},
expectIPConfig: &iptables.Config{
ConsulDNSIP: "192.168.1.117",
ConsulDNSPort: 8600,
ProxyUserID: "101",
ProxyInboundPort: 25018,
ProxyOutboundPort: 15001,
ExcludeInboundPorts: []string{"9002"},
NetNS: "/var/run/docker/netns/a2ece01ea7bc",
},
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
tg.Services[0].Connect.SidecarService.Proxy.TransparentProxy = tc.tproxySpec
tg.Services[0].Connect.SidecarService.Proxy.Expose = tc.exposeSpec
tg.Services[0].Cluster = tc.cluster
c := &cniNetworkConfigurator{
nodeAttrs: nodeAttrs,
nodeMeta: nodeMeta,
logger: testlog.HCLogger(t),
}
if tc.nodeAttrs != nil {
c.nodeAttrs = tc.nodeAttrs
}
iptablesCfg, err := c.setupTransparentProxyArgs(alloc, spec, portMaps)
if tc.expectErr == "" {
must.NoError(t, err)
must.Eq(t, tc.expectIPConfig, iptablesCfg)
} else {
must.EqError(t, err, tc.expectErr)
must.Nil(t, iptablesCfg)
}
})
}
}

View File

@@ -114,16 +114,6 @@ const (
// allocSyncRetryIntv is the interval on which we retry updating
// the status of the allocation
allocSyncRetryIntv = 5 * time.Second
// defaultConnectLogLevel is the log level set in the node meta by default
// to be used by Consul Connect sidecar tasks.
defaultConnectLogLevel = "info"
// defaultConnectProxyConcurrency is the default number of worker threads the
// connect sidecar should be configured to use.
//
// https://www.envoyproxy.io/docs/envoy/latest/operations/cli#cmdoption-concurrency
defaultConnectProxyConcurrency = "1"
)
var (
@@ -1572,11 +1562,17 @@ func (c *Client) setupNode() error {
if _, ok := node.Meta[envoy.GatewayMetaParam]; !ok {
node.Meta[envoy.GatewayMetaParam] = envoy.ImageFormat
}
if _, ok := node.Meta["connect.log_level"]; !ok {
node.Meta["connect.log_level"] = defaultConnectLogLevel
if _, ok := node.Meta[envoy.DefaultConnectLogLevelParam]; !ok {
node.Meta[envoy.DefaultConnectLogLevelParam] = envoy.DefaultConnectLogLevel
}
if _, ok := node.Meta["connect.proxy_concurrency"]; !ok {
node.Meta["connect.proxy_concurrency"] = defaultConnectProxyConcurrency
if _, ok := node.Meta[envoy.DefaultConnectProxyConcurrencyParam]; !ok {
node.Meta[envoy.DefaultConnectProxyConcurrencyParam] = envoy.DefaultConnectProxyConcurrency
}
if _, ok := node.Meta[envoy.DefaultTransparentProxyUIDParam]; !ok {
node.Meta[envoy.DefaultTransparentProxyUIDParam] = envoy.DefaultTransparentProxyUID
}
if _, ok := node.Meta[envoy.DefaultTransparentProxyOutboundPortParam]; !ok {
node.Meta[envoy.DefaultTransparentProxyOutboundPortParam] = envoy.DefaultTransparentProxyOutboundPort
}
// Since node.Meta will get dynamic metadata merged in, save static metadata

View File

@@ -145,8 +145,13 @@ func connectSidecarProxy(info structs.AllocInfo, proxy *structs.ConsulProxy, cPo
if err != nil {
return nil, err
}
mode := api.ProxyModeDefault
if proxy.TransparentProxy != nil {
mode = api.ProxyModeTransparent
}
return &api.AgentServiceConnectProxyConfig{
Mode: mode,
LocalServiceAddress: proxy.LocalServiceAddress,
LocalServicePort: proxy.LocalServicePort,
Config: connectProxyConfig(proxy.Config, cPort, info),

View File

@@ -1897,6 +1897,7 @@ func apiConnectSidecarServiceProxyToStructs(in *api.ConsulProxy) *structs.Consul
LocalServicePort: in.LocalServicePort,
Upstreams: apiUpstreamsToStructs(in.Upstreams),
Expose: apiConsulExposeConfigToStructs(expose),
TransparentProxy: apiConnectTransparentProxyToStructs(in.TransparentProxy),
Config: maps.Clone(in.Config),
}
}
@@ -1949,6 +1950,21 @@ func apiConsulExposeConfigToStructs(in *api.ConsulExposeConfig) *structs.ConsulE
}
}
func apiConnectTransparentProxyToStructs(in *api.ConsulTransparentProxy) *structs.ConsulTransparentProxy {
if in == nil {
return nil
}
return &structs.ConsulTransparentProxy{
UID: in.UID,
OutboundPort: in.OutboundPort,
ExcludeInboundPorts: in.ExcludeInboundPorts,
ExcludeOutboundPorts: in.ExcludeOutboundPorts,
ExcludeOutboundCIDRs: in.ExcludeOutboundCIDRs,
ExcludeUIDs: in.ExcludeUIDs,
NoDNS: in.NoDNS,
}
}
func apiConsulExposePathsToStructs(in []*api.ConsulExposePath) []structs.ConsulExposePath {
if len(in) == 0 {
return nil

View File

@@ -36,40 +36,39 @@ func TestConnect_LegacyACLs(t *testing.T) {
t.Run("ConnectTerminatingGateway", testConnectTerminatingGatewayLegacyACLs)
}
func createPolicy(t *testing.T, cc *capi.Client, ns, rules string) (string, func()) {
func createPolicy(t *testing.T, cc *capi.Client, ns, rules string) string {
policy, _, err := cc.ACL().PolicyCreate(&capi.ACLPolicy{
Name: "nomad-operator-policy-" + uuid.Short(),
Rules: rules,
Namespace: ns,
}, nil)
must.NoError(t, err)
return policy.ID, func() { cc.ACL().PolicyDelete(policy.ID, nil) }
t.Cleanup(func() { cc.ACL().PolicyDelete(policy.ID, nil) })
return policy.ID
}
func createToken(t *testing.T, cc *capi.Client, policyID, ns string) (string, func()) {
func createToken(t *testing.T, cc *capi.Client, policyID, ns string) string {
token, _, err := cc.ACL().TokenCreate(&capi.ACLToken{
Description: "test token",
Policies: []*capi.ACLTokenPolicyLink{{ID: policyID}},
Namespace: ns,
}, nil)
must.NoError(t, err)
return token.SecretID, func() { cc.ACL().TokenDelete(token.AccessorID, nil) }
t.Cleanup(func() { cc.ACL().TokenDelete(token.AccessorID, nil) })
return token.SecretID
}
// testConnectDemoLegacyACLs tests the demo job file used in Connect Integration examples.
func testConnectDemoLegacyACLs(t *testing.T) {
cc := e2eutil.ConsulClient(t)
policyID, policyCleanup := createPolicy(t, cc, "default",
policyID := createPolicy(t, cc, "default",
`service "count-api" { policy = "write" } service "count-dashboard" { policy = "write" }`)
t.Cleanup(policyCleanup)
token, tokenCleanup := createToken(t, cc, policyID, "default")
t.Cleanup(tokenCleanup)
token := createToken(t, cc, policyID, "default")
_, cleanup := jobs3.Submit(t, "./input/demo.nomad",
sub, _ := jobs3.Submit(t, "./input/demo.nomad",
jobs3.Timeout(time.Second*60), jobs3.LegacyConsulToken(token))
t.Cleanup(cleanup)
ixn := &capi.Intention{
SourceName: "count-dashboard",
@@ -89,6 +88,9 @@ func testConnectDemoLegacyACLs(t *testing.T) {
assertSITokens(t, cc, map[string]int{
"connect-proxy-count-api": 1, "connect-proxy-count-dashboard": 1})
logs := sub.Exec("dashboard", "dashboard",
[]string{"/bin/sh", "-c", "wget -O /dev/null http://${NOMAD_UPSTREAM_ADDR_count_api}"})
must.StrContains(t, logs.Stderr, "saving to")
}
// testConnectDemoLegacyACLsNamespaced tests the demo job file used in Connect
@@ -101,16 +103,13 @@ func testConnectDemoLegacyACLsNamespaced(t *testing.T) {
must.NoError(t, err)
t.Cleanup(func() { cc.Namespaces().Delete(ns, nil) })
policyID, policyCleanup := createPolicy(t, cc, ns,
policyID := createPolicy(t, cc, ns,
`service "count-api" { policy = "write" } service "count-dashboard" { policy = "write" }`)
t.Cleanup(policyCleanup)
token, tokenCleanup := createToken(t, cc, policyID, ns)
t.Cleanup(tokenCleanup)
token := createToken(t, cc, policyID, ns)
_, cleanup := jobs3.Submit(t, "./input/demo.nomad",
jobs3.Submit(t, "./input/demo.nomad",
jobs3.Timeout(time.Second*60), jobs3.LegacyConsulToken(token))
t.Cleanup(cleanup)
ixn := &capi.Intention{
SourceName: "count-dashboard",
@@ -137,16 +136,13 @@ func testConnectDemoLegacyACLsNamespaced(t *testing.T) {
func testConnectNativeDemoLegacyACLs(t *testing.T) {
cc := e2eutil.ConsulClient(t)
policyID, policyCleanup := createPolicy(t, cc, "default",
policyID := createPolicy(t, cc, "default",
`service "uuid-fe" { policy = "write" } service "uuid-api" { policy = "write" }`)
t.Cleanup(policyCleanup)
token, tokenCleanup := createToken(t, cc, policyID, "default")
t.Cleanup(tokenCleanup)
token := createToken(t, cc, policyID, "default")
_, cleanup := jobs3.Submit(t, "./input/native-demo.nomad",
jobs3.Submit(t, "./input/native-demo.nomad",
jobs3.Timeout(time.Second*60), jobs3.LegacyConsulToken(token))
t.Cleanup(cleanup)
assertSITokens(t, cc, map[string]int{"frontend": 1, "generate": 1})
}
@@ -155,16 +151,13 @@ func testConnectNativeDemoLegacyACLs(t *testing.T) {
func testConnectIngressGatewayDemoLegacyACLs(t *testing.T) {
cc := e2eutil.ConsulClient(t)
policyID, policyCleanup := createPolicy(t, cc, "default",
policyID := createPolicy(t, cc, "default",
`service "my-ingress-service" { policy = "write" } service "uuid-api" { policy = "write" }`)
t.Cleanup(policyCleanup)
token, tokenCleanup := createToken(t, cc, policyID, "default")
t.Cleanup(tokenCleanup)
token := createToken(t, cc, policyID, "default")
_, cleanup := jobs3.Submit(t, "./input/ingress-gateway.nomad",
jobs3.Submit(t, "./input/ingress-gateway.nomad",
jobs3.Timeout(time.Second*60), jobs3.LegacyConsulToken(token))
t.Cleanup(cleanup)
assertSITokens(t, cc, map[string]int{"connect-ingress-my-ingress-service": 1, "generate": 1})
}
@@ -173,16 +166,13 @@ func testConnectIngressGatewayDemoLegacyACLs(t *testing.T) {
func testConnectTerminatingGatewayLegacyACLs(t *testing.T) {
cc := e2eutil.ConsulClient(t)
policyID, policyCleanup := createPolicy(t, cc, "default",
policyID := createPolicy(t, cc, "default",
`service "api-gateway" { policy = "write" } service "count-dashboard" { policy = "write" }`)
t.Cleanup(policyCleanup)
token, tokenCleanup := createToken(t, cc, policyID, "default")
t.Cleanup(tokenCleanup)
token := createToken(t, cc, policyID, "default")
_, cleanup := jobs3.Submit(t, "./input/terminating-gateway.nomad",
jobs3.Submit(t, "./input/terminating-gateway.nomad",
jobs3.Timeout(time.Second*60), jobs3.LegacyConsulToken(token))
t.Cleanup(cleanup)
ixn := &capi.Intention{
SourceName: "count-dashboard",

View File

@@ -32,12 +32,12 @@ func TestConnect(t *testing.T) {
t.Run("ConnectMultiIngress", testConnectMultiIngressGateway)
t.Run("ConnectTerminatingGateway", testConnectTerminatingGateway)
t.Run("ConnectMultiService", testConnectMultiService)
t.Run("ConnectTransparentProxy", testConnectTransparentProxy)
}
// testConnectDemo tests the demo job file used in Connect Integration examples.
func testConnectDemo(t *testing.T) {
_, cleanup := jobs3.Submit(t, "./input/demo.nomad", jobs3.Timeout(time.Second*60))
t.Cleanup(cleanup)
sub, _ := jobs3.Submit(t, "./input/demo.nomad", jobs3.Timeout(time.Second*60))
cc := e2eutil.ConsulClient(t)
@@ -56,38 +56,37 @@ func testConnectDemo(t *testing.T) {
assertServiceOk(t, cc, "count-api-sidecar-proxy")
assertServiceOk(t, cc, "count-dashboard-sidecar-proxy")
logs := sub.Exec("dashboard", "dashboard",
[]string{"/bin/sh", "-c", "wget -O /dev/null http://${NOMAD_UPSTREAM_ADDR_count_api}"})
must.StrContains(t, logs.Stderr, "saving to")
}
// testConnectCustomSidecarExposed tests that a connect sidecar with custom task
// definition can also make use of the expose service check feature.
func testConnectCustomSidecarExposed(t *testing.T) {
_, cleanup := jobs3.Submit(t, "./input/expose-custom.nomad", jobs3.Timeout(time.Second*60))
t.Cleanup(cleanup)
jobs3.Submit(t, "./input/expose-custom.nomad", jobs3.Timeout(time.Second*60))
}
// testConnectNativeDemo tests the demo job file used in Connect Native
// Integration examples.
func testConnectNativeDemo(t *testing.T) {
_, cleanup := jobs3.Submit(t, "./input/native-demo.nomad", jobs3.Timeout(time.Second*60))
t.Cleanup(cleanup)
jobs3.Submit(t, "./input/native-demo.nomad", jobs3.Timeout(time.Second*60))
}
// testConnectIngressGatewayDemo tests a job with an ingress gateway
func testConnectIngressGatewayDemo(t *testing.T) {
_, cleanup := jobs3.Submit(t, "./input/ingress-gateway.nomad", jobs3.Timeout(time.Second*60))
t.Cleanup(cleanup)
jobs3.Submit(t, "./input/ingress-gateway.nomad", jobs3.Timeout(time.Second*60))
}
// testConnectMultiIngressGateway tests a job with multiple ingress gateways
func testConnectMultiIngressGateway(t *testing.T) {
_, cleanup := jobs3.Submit(t, "./input/multi-ingress.nomad", jobs3.Timeout(time.Second*60))
t.Cleanup(cleanup)
jobs3.Submit(t, "./input/multi-ingress.nomad", jobs3.Timeout(time.Second*60))
}
// testConnectTerminatingGateway tests a job with a terminating gateway
func testConnectTerminatingGateway(t *testing.T) {
_, cleanup := jobs3.Submit(t, "./input/terminating-gateway.nomad", jobs3.Timeout(time.Second*60))
t.Cleanup(cleanup)
jobs3.Submit(t, "./input/terminating-gateway.nomad", jobs3.Timeout(time.Second*60))
cc := e2eutil.ConsulClient(t)
@@ -112,14 +111,40 @@ func testConnectTerminatingGateway(t *testing.T) {
// testConnectMultiService tests a job with multiple Connect blocks in the same
// group
func testConnectMultiService(t *testing.T) {
_, cleanup := jobs3.Submit(t, "./input/multi-service.nomad", jobs3.Timeout(time.Second*60))
t.Cleanup(cleanup)
jobs3.Submit(t, "./input/multi-service.nomad", jobs3.Timeout(time.Second*60))
cc := e2eutil.ConsulClient(t)
assertServiceOk(t, cc, "echo1-sidecar-proxy")
assertServiceOk(t, cc, "echo2-sidecar-proxy")
}
// testConnectTransparentProxy tests the Connect Transparent Proxy integration
func testConnectTransparentProxy(t *testing.T) {
sub, _ := jobs3.Submit(t, "./input/tproxy.nomad.hcl", jobs3.Timeout(time.Second*60))
cc := e2eutil.ConsulClient(t)
ixn := &capi.Intention{
SourceName: "count-dashboard",
DestinationName: "count-api",
Action: "allow",
}
_, err := cc.Connect().IntentionUpsert(ixn, nil)
must.NoError(t, err, must.Sprint("could not create intention"))
t.Cleanup(func() {
_, err := cc.Connect().IntentionDeleteExact("count-dashboard", "count-api", nil)
test.NoError(t, err)
})
assertServiceOk(t, cc, "count-api-sidecar-proxy")
assertServiceOk(t, cc, "count-dashboard-sidecar-proxy")
logs := sub.Exec("dashboard", "dashboard",
[]string{"wget", "-O", "/dev/null", "count-api.virtual.consul"})
must.StrContains(t, logs.Stderr, "saving to")
}
// assertServiceOk is a test helper to assert a service is passing health checks, if any
func assertServiceOk(t *testing.T, cc *capi.Client, name string) {
t.Helper()

View File

@@ -0,0 +1,99 @@
# Copyright (c) HashiCorp, Inc.
# SPDX-License-Identifier: BUSL-1.1
job "countdash" {
constraint {
attribute = "${attr.kernel.name}"
value = "linux"
}
group "api" {
network {
mode = "bridge"
}
service {
name = "count-api"
port = "9001"
check {
type = "http"
path = "/health"
expose = true
interval = "3s"
timeout = "1s"
check_restart {
limit = 0
}
}
connect {
sidecar_service {
proxy {
transparent_proxy {}
}
}
}
}
task "web" {
driver = "docker"
config {
image = "hashicorpdev/counter-api:v3"
auth_soft_fail = true
}
}
}
group "dashboard" {
network {
mode = "bridge"
port "http" {
static = 9010
to = 9002
}
}
service {
name = "count-dashboard"
port = "9002"
check {
type = "http"
path = "/health"
expose = true
interval = "3s"
timeout = "1s"
check_restart {
limit = 0
}
}
connect {
sidecar_service {
proxy {
transparent_proxy {}
}
}
}
}
task "dashboard" {
driver = "docker"
env {
COUNTING_SERVICE_URL = "http://count-api.virtual.consul"
}
config {
image = "hashicorpdev/counter-dashboard:v3"
auth_soft_fail = true
}
}
}
}

View File

@@ -20,6 +20,7 @@ echo 'debconf debconf/frontend select Noninteractive' | sudo debconf-set-selecti
mkdir_for_root /opt
mkdir_for_root /srv/data # for host volumes
mkdir_for_root /opt/cni/bin
# Dependencies
sudo apt-get update
@@ -63,6 +64,25 @@ sudo apt-get install -y \
consul-enterprise \
nomad
# TODO(tgross: replace with downloading the binary from releases.hashicorp.com
# once the official 1.4.2 release has shipped
echo "Installing consul-cni plugin"
sudo apt-get install -y build-essential git
pushd /tmp
curl -LO https://go.dev/dl/go1.22.2.linux-amd64.tar.gz
sudo tar -C /usr/local -xzf go1.22.2.linux-amd64.tar.gz
git clone --depth=1 https://github.com/hashicorp/consul-k8s.git
pushd consul-k8s
export PATH="$PATH:/usr/local/go/bin"
make control-plane-dev
sudo mv control-plane/cni/bin/consul-cni /opt/cni/bin
sudo chown root:root /opt/cni/bin/consul-cni
sudo chmod +x /opt/cni/bin/consul-cni
popd
popd
# Note: neither service will start on boot because we haven't enabled
# the systemd unit file and we haven't uploaded any configuration
# files for Consul and Nomad
@@ -90,7 +110,6 @@ sudo apt-get install -y openjdk-17-jdk-headless
# CNI
echo "Installing CNI plugins"
sudo mkdir -p /opt/cni/bin
wget -q -O - \
https://github.com/containernetworking/plugins/releases/download/v1.0.0/cni-plugins-linux-amd64-v1.0.0.tgz \
| sudo tar -C /opt/cni/bin -xz

View File

@@ -47,6 +47,44 @@ const (
// VersionVar will be replaced with the Envoy version string when
// used in the meta.connect.sidecar_image variable.
VersionVar = "${NOMAD_envoy_version}"
// DefaultConnectLogLevel is the log level set in the node meta by default
// to be used by Consul Connect sidecar tasks.
DefaultConnectLogLevel = "info"
// DefaultConnectLogLevel is the node attribute for the DefaultConnectLogLevel
DefaultConnectLogLevelParam = "connect.log_level"
// DefaultConnectProxyConcurrency is the default number of worker threads the
// connect sidecar should be configured to use.
//
// https://www.envoyproxy.io/docs/envoy/latest/operations/cli#cmdoption-concurrency
DefaultConnectProxyConcurrency = "1"
// DefaultConnectProxyConcurrencyParam is the node attribute for the
// DefaultConnectProxyConcurrency
DefaultConnectProxyConcurrencyParam = "connect.proxy_concurrency"
// DefaultTransparentProxyUID is the default UID of the Envoy proxy
// container user, for use with transparent proxy
DefaultTransparentProxyUID = "101"
// DefaultTransparentProxyUIDParam is the node attribute for the
// DefaultTransparentProxyUID
DefaultTransparentProxyUIDParam = "connect.transparent_proxy.default_uid"
// DefaultTransparentProxyOutboundPort is the default outbound port for the
// Envoy proxy, for use with transparent proxy. Note the default value
// patches the default TransparentProxy service default for
// OutboundListenerPort. If the cluster admin sets this value to something
// non-default, they'll need to update the metadata on all the nodes to
// match. See also:
// https://developer.hashicorp.com/consul/docs/connect/config-entries/service-defaults#transparentproxy
DefaultTransparentProxyOutboundPort = "15001"
// DefaultTransparentProxyOutboundPortParam is the node attribute for the
// DefaultTransparentProxyOutboundPort
DefaultTransparentProxyOutboundPortParam = "connect.transparent_proxy.default_outbound_port"
)
// PortLabel creates a consistent port label using the inputs of a prefix,

View File

@@ -927,6 +927,7 @@ func parseProxy(o *ast.ObjectItem) (*api.ConsulProxy, error) {
"local_service_port",
"upstreams",
"expose",
"transparent_proxy",
"config",
}
@@ -942,6 +943,7 @@ func parseProxy(o *ast.ObjectItem) (*api.ConsulProxy, error) {
delete(m, "upstreams")
delete(m, "expose")
delete(m, "transparent_proxy")
delete(m, "config")
dec, err := mapstructure.NewDecoder(&mapstructure.DecoderConfig{
@@ -985,6 +987,16 @@ func parseProxy(o *ast.ObjectItem) (*api.ConsulProxy, error) {
}
}
if tpo := listVal.Filter("transparent_proxy"); len(tpo.Items) > 1 {
return nil, fmt.Errorf("only 1 transparent_proxy object supported")
} else if len(tpo.Items) == 1 {
if tp, err := parseTproxy(tpo.Items[0]); err != nil {
return nil, err
} else {
proxy.TransparentProxy = tp
}
}
// If we have config, then parse that
if o := listVal.Filter("config"); len(o.Items) > 1 {
return nil, fmt.Errorf("only 1 meta object supported")
@@ -1077,6 +1089,41 @@ func parseExposePath(epo *ast.ObjectItem) (*api.ConsulExposePath, error) {
return &path, nil
}
func parseTproxy(epo *ast.ObjectItem) (*api.ConsulTransparentProxy, error) {
valid := []string{
"uid",
"outbound_port",
"exclude_inbound_ports",
"exclude_outbound_ports",
"exclude_outbound_cidrs",
"exclude_uids",
"no_dns",
}
if err := checkHCLKeys(epo.Val, valid); err != nil {
return nil, multierror.Prefix(err, "tproxy ->")
}
var tproxy api.ConsulTransparentProxy
var m map[string]interface{}
if err := hcl.DecodeObject(&m, epo.Val); err != nil {
return nil, err
}
dec, err := mapstructure.NewDecoder(&mapstructure.DecoderConfig{
Result: &tproxy,
})
if err != nil {
return nil, err
}
if err := dec.Decode(m); err != nil {
return nil, err
}
return &tproxy, nil
}
func parseUpstream(uo *ast.ObjectItem) (*api.ConsulUpstream, error) {
valid := []string{
"destination_name",

View File

@@ -1470,6 +1470,15 @@ func TestParse(t *testing.T) {
DestinationName: "upstream2",
LocalBindPort: 2002,
}},
TransparentProxy: &api.ConsulTransparentProxy{
UID: "101",
OutboundPort: 15001,
ExcludeInboundPorts: []string{"www", "9000"},
ExcludeOutboundPorts: []uint16{443, 80},
ExcludeOutboundCIDRs: []string{"10.0.0.0/8"},
ExcludeUIDs: []string{"10", "1001"},
NoDNS: true,
},
Config: map[string]interface{}{
"foo": "bar",
},

View File

@@ -40,6 +40,16 @@ job "service-connect-proxy" {
}
}
transparent_proxy {
uid = "101"
outbound_port = 15001
exclude_inbound_ports = ["www", "9000"]
exclude_outbound_ports = [443, 80]
exclude_outbound_cidrs = ["10.0.0.0/8"]
exclude_uids = ["10", "1001"]
no_dns = true
}
config {
foo = "bar"
}

View File

@@ -561,33 +561,74 @@ func groupConnectValidate(g *structs.TaskGroup) error {
}
}
if err := groupConnectUpstreamsValidate(g.Name, g.Services); err != nil {
if err := groupConnectUpstreamsValidate(g, g.Services); err != nil {
return err
}
return nil
}
func groupConnectUpstreamsValidate(group string, services []*structs.Service) error {
func groupConnectUpstreamsValidate(g *structs.TaskGroup, services []*structs.Service) error {
listeners := make(map[string]string) // address -> service
var connectBlockCount int
var hasTproxy bool
for _, service := range services {
if service.Connect != nil {
connectBlockCount++
}
if service.Connect.HasSidecar() && service.Connect.SidecarService.Proxy != nil {
for _, up := range service.Connect.SidecarService.Proxy.Upstreams {
listener := net.JoinHostPort(up.LocalBindAddress, strconv.Itoa(up.LocalBindPort))
if s, exists := listeners[listener]; exists {
return fmt.Errorf(
"Consul Connect services %q and %q in group %q using same address for upstreams (%s)",
service.Name, s, group, listener,
service.Name, s, g.Name, listener,
)
}
listeners[listener] = service.Name
}
if tp := service.Connect.SidecarService.Proxy.TransparentProxy; tp != nil {
hasTproxy = true
for _, net := range g.Networks {
if !net.DNS.IsZero() && !tp.NoDNS {
return fmt.Errorf(
"Consul Connect transparent proxy cannot be used with network.dns unless no_dns=true")
}
}
for _, portLabel := range tp.ExcludeInboundPorts {
if !transparentProxyPortLabelValidate(g, portLabel) {
return fmt.Errorf(
"Consul Connect transparent proxy port %q must be numeric or one of network.port labels", portLabel)
}
}
}
}
}
if hasTproxy && connectBlockCount > 1 {
return fmt.Errorf("Consul Connect transparent proxy requires there is only one connect block")
}
return nil
}
func transparentProxyPortLabelValidate(g *structs.TaskGroup, portLabel string) bool {
if _, err := strconv.ParseUint(portLabel, 10, 64); err == nil {
return true
}
for _, network := range g.Networks {
for _, reservedPort := range network.ReservedPorts {
if reservedPort.Label == portLabel {
return true
}
}
}
return false
}
func groupConnectSidecarValidate(g *structs.TaskGroup, s *structs.Service) error {
if n := len(g.Networks); n != 1 {
return fmt.Errorf("Consul Connect sidecars require exactly 1 network, found %d in group %q", n, g.Name)

View File

@@ -548,13 +548,15 @@ func TestJobEndpointConnect_groupConnectUpstreamsValidate(t *testing.T) {
ci.Parallel(t)
t.Run("no connect services", func(t *testing.T) {
err := groupConnectUpstreamsValidate("group",
tg := &structs.TaskGroup{Name: "group"}
err := groupConnectUpstreamsValidate(tg,
[]*structs.Service{{Name: "s1"}, {Name: "s2"}})
require.NoError(t, err)
must.NoError(t, err)
})
t.Run("connect services no overlap", func(t *testing.T) {
err := groupConnectUpstreamsValidate("group",
tg := &structs.TaskGroup{Name: "group"}
err := groupConnectUpstreamsValidate(tg,
[]*structs.Service{
{
Name: "s1",
@@ -589,11 +591,12 @@ func TestJobEndpointConnect_groupConnectUpstreamsValidate(t *testing.T) {
},
},
})
require.NoError(t, err)
must.NoError(t, err)
})
t.Run("connect services overlap port", func(t *testing.T) {
err := groupConnectUpstreamsValidate("group",
tg := &structs.TaskGroup{Name: "group"}
err := groupConnectUpstreamsValidate(tg,
[]*structs.Service{
{
Name: "s1",
@@ -628,7 +631,75 @@ func TestJobEndpointConnect_groupConnectUpstreamsValidate(t *testing.T) {
},
},
})
require.EqualError(t, err, `Consul Connect services "s2" and "s1" in group "group" using same address for upstreams (127.0.0.1:9002)`)
must.EqError(t, err, `Consul Connect services "s2" and "s1" in group "group" using same address for upstreams (127.0.0.1:9002)`)
})
t.Run("connect tproxy excludes invalid port", func(t *testing.T) {
tg := &structs.TaskGroup{Name: "group", Networks: structs.Networks{
{
ReservedPorts: []structs.Port{{
Label: "www",
}},
},
}}
err := groupConnectUpstreamsValidate(tg,
[]*structs.Service{
{
Name: "s1",
Connect: &structs.ConsulConnect{
SidecarService: &structs.ConsulSidecarService{
Proxy: &structs.ConsulProxy{
TransparentProxy: &structs.ConsulTransparentProxy{
ExcludeInboundPorts: []string{"www", "9000", "no-such-label"},
},
},
},
},
},
})
must.EqError(t, err, `Consul Connect transparent proxy port "no-such-label" must be numeric or one of network.port labels`)
})
t.Run("Consul Connect transparent proxy allows only one Connect block", func(t *testing.T) {
tg := &structs.TaskGroup{Name: "group"}
err := groupConnectUpstreamsValidate(tg,
[]*structs.Service{
{
Name: "s1",
Connect: &structs.ConsulConnect{},
},
{
Name: "s2",
Connect: &structs.ConsulConnect{
SidecarService: &structs.ConsulSidecarService{
Proxy: &structs.ConsulProxy{
TransparentProxy: &structs.ConsulTransparentProxy{},
},
},
},
},
})
must.EqError(t, err, `Consul Connect transparent proxy requires there is only one connect block`)
})
t.Run("Consul Connect transparent proxy DNS not allowed with network.dns", func(t *testing.T) {
tg := &structs.TaskGroup{Name: "group", Networks: []*structs.NetworkResource{{
DNS: &structs.DNSConfig{Servers: []string{"1.1.1.1"}},
}}}
err := groupConnectUpstreamsValidate(tg,
[]*structs.Service{
{
Name: "s1",
Connect: &structs.ConsulConnect{
SidecarService: &structs.ConsulSidecarService{
Proxy: &structs.ConsulProxy{
TransparentProxy: &structs.ConsulTransparentProxy{},
},
},
},
},
})
must.EqError(t, err, `Consul Connect transparent proxy cannot be used with network.dns unless no_dns=true`)
})
}

View File

@@ -26,6 +26,7 @@ const (
attrHostLocalCNI = `${attr.plugins.cni.version.host-local}`
attrLoopbackCNI = `${attr.plugins.cni.version.loopback}`
attrPortMapCNI = `${attr.plugins.cni.version.portmap}`
attrConsulCNI = `${attr.plugins.cni.version.consul-cni}`
)
// cniMinVersion is the version expression for the minimum CNI version supported
@@ -134,6 +135,14 @@ var (
RTarget: cniMinVersion,
Operand: structs.ConstraintSemver,
}
// cniConsulConstraint is an implicit constraint added to jobs making use of
// transparent proxy mode.
cniConsulConstraint = &structs.Constraint{
LTarget: attrConsulCNI,
RTarget: ">= 1.4.2",
Operand: structs.ConstraintSemver,
}
)
type admissionController interface {
@@ -250,12 +259,15 @@ func (jobImpliedConstraints) Mutate(j *structs.Job) (*structs.Job, []error, erro
bridgeNetworkingTaskGroups := j.RequiredBridgeNetwork()
transparentProxyTaskGroups := j.RequiredTransparentProxy()
// Hot path where none of our things require constraints.
//
// [UPDATE THIS] if you are adding a new constraint thing!
if len(signals) == 0 && len(vaultBlocks) == 0 &&
nativeServiceDisco.Empty() && len(consulServiceDisco) == 0 &&
numaTaskGroups.Empty() && bridgeNetworkingTaskGroups.Empty() {
numaTaskGroups.Empty() && bridgeNetworkingTaskGroups.Empty() &&
transparentProxyTaskGroups.Empty() {
return j, nil, nil
}
@@ -320,6 +332,10 @@ func (jobImpliedConstraints) Mutate(j *structs.Job) (*structs.Job, []error, erro
mutateConstraint(constraintMatcherLeft, tg, cniLoopbackConstraint)
mutateConstraint(constraintMatcherLeft, tg, cniPortMapConstraint)
}
if transparentProxyTaskGroups.Contains(tg.Name) {
mutateConstraint(constraintMatcherLeft, tg, cniConsulConstraint)
}
}
return j, nil, nil

View File

@@ -1194,6 +1194,60 @@ func Test_jobImpliedConstraints_Mutate(t *testing.T) {
expectedOutputError: nil,
name: "task group with bridge network",
},
{
inputJob: &structs.Job{
Name: "example",
TaskGroups: []*structs.TaskGroup{
{
Name: "group-with-tproxy",
Services: []*structs.Service{{
Connect: &structs.ConsulConnect{
SidecarService: &structs.ConsulSidecarService{
Proxy: &structs.ConsulProxy{
TransparentProxy: &structs.ConsulTransparentProxy{},
},
},
},
}},
Networks: []*structs.NetworkResource{
{Mode: "bridge"},
},
},
},
},
expectedOutputJob: &structs.Job{
Name: "example",
TaskGroups: []*structs.TaskGroup{
{
Name: "group-with-tproxy",
Services: []*structs.Service{{
Connect: &structs.ConsulConnect{
SidecarService: &structs.ConsulSidecarService{
Proxy: &structs.ConsulProxy{
TransparentProxy: &structs.ConsulTransparentProxy{},
},
},
},
}},
Networks: []*structs.NetworkResource{
{Mode: "bridge"},
},
Constraints: []*structs.Constraint{
consulServiceDiscoveryConstraint,
cniBridgeConstraint,
cniFirewallConstraint,
cniHostLocalConstraint,
cniLoopbackConstraint,
cniPortMapConstraint,
cniConsulConstraint,
},
},
},
},
expectedOutputWarnings: nil,
expectedOutputError: nil,
name: "task group with tproxy",
},
}
for _, tc := range testCases {

View File

@@ -3,6 +3,16 @@
package structs
import (
"fmt"
"net/netip"
"slices"
"strconv"
"github.com/hashicorp/go-multierror"
"github.com/hashicorp/nomad/helper"
)
// ConsulConfigEntries represents Consul ConfigEntry definitions from a job for
// a single Consul namespace.
type ConsulConfigEntries struct {
@@ -43,3 +53,131 @@ func (j *Job) ConfigEntries() map[string]*ConsulConfigEntries {
return collection
}
// ConsulTransparentProxy is used to configure the Envoy sidecar for
// "transparent proxying", which creates IP tables rules inside the network
// namespace to ensure traffic flows thru the Envoy proxy
type ConsulTransparentProxy struct {
// UID of the Envoy proxy. Defaults to the default Envoy proxy container
// image user.
UID string
// OutboundPort is the Envoy proxy's outbound listener port. Inbound TCP
// traffic hitting the PROXY_IN_REDIRECT chain will be redirected here.
// Defaults to 15001.
OutboundPort uint16
// ExcludeInboundPorts is an additional set of ports will be excluded from
// redirection to the Envoy proxy. Can be Port.Label or Port.Value. This set
// will be added to the ports automatically excluded for the Expose.Port and
// Check.Expose fields.
ExcludeInboundPorts []string
// ExcludeOutboundPorts is a set of outbound ports that will not be
// redirected to the Envoy proxy, specified as port numbers.
ExcludeOutboundPorts []uint16
// ExcludeOutboundCIDRs is a set of outbound CIDR blocks that will not be
// redirected to the Envoy proxy.
ExcludeOutboundCIDRs []string
// ExcludeUIDs is a set of user IDs whose network traffic will not be
// redirected through the Envoy proxy.
ExcludeUIDs []string
// NoDNS disables redirection of DNS traffic to Consul DNS. By default NoDNS
// is false and transparent proxy will direct DNS traffic to Consul DNS if
// available on the client.
NoDNS bool
}
func (tp *ConsulTransparentProxy) Copy() *ConsulTransparentProxy {
if tp == nil {
return nil
}
ntp := new(ConsulTransparentProxy)
*ntp = *tp
ntp.ExcludeInboundPorts = slices.Clone(tp.ExcludeInboundPorts)
ntp.ExcludeOutboundPorts = slices.Clone(tp.ExcludeOutboundPorts)
ntp.ExcludeOutboundCIDRs = slices.Clone(tp.ExcludeOutboundCIDRs)
ntp.ExcludeUIDs = slices.Clone(tp.ExcludeUIDs)
return ntp
}
func (tp *ConsulTransparentProxy) Validate() error {
var mErr multierror.Error
for _, rawCidr := range tp.ExcludeOutboundCIDRs {
_, err := netip.ParsePrefix(rawCidr)
if err != nil {
// note: error returned always include parsed string
mErr.Errors = append(mErr.Errors,
fmt.Errorf("could not parse transparent proxy excluded outbound CIDR as network prefix: %w", err))
}
}
requireUIDisUint := func(uidRaw string) error {
_, err := strconv.ParseUint(uidRaw, 10, 16)
if err != nil {
e, ok := err.(*strconv.NumError)
if !ok {
return fmt.Errorf("invalid user ID %q: %w", uidRaw, err)
}
return fmt.Errorf("invalid user ID %q: %w", uidRaw, e.Err)
}
return nil
}
if tp.UID != "" {
if err := requireUIDisUint(tp.UID); err != nil {
mErr.Errors = append(mErr.Errors,
fmt.Errorf("transparent proxy block has invalid UID field: %w", err))
}
}
for _, uid := range tp.ExcludeUIDs {
if err := requireUIDisUint(uid); err != nil {
mErr.Errors = append(mErr.Errors,
fmt.Errorf("transparent proxy block has invalid ExcludeUIDs field: %w", err))
}
}
// note: ExcludeInboundPorts are validated in connect validation hook
// because we need information from the network block
if mErr.Len() == 1 {
return mErr.Errors[0]
}
return mErr.ErrorOrNil()
}
func (tp *ConsulTransparentProxy) Equal(o *ConsulTransparentProxy) bool {
if tp == nil || o == nil {
return tp == o
}
if tp.UID != o.UID {
return false
}
if tp.OutboundPort != o.OutboundPort {
return false
}
if !helper.SliceSetEq(tp.ExcludeInboundPorts, o.ExcludeInboundPorts) {
return false
}
if !helper.SliceSetEq(tp.ExcludeOutboundPorts, o.ExcludeOutboundPorts) {
return false
}
if !helper.SliceSetEq(tp.ExcludeOutboundCIDRs, o.ExcludeOutboundCIDRs) {
return false
}
if !helper.SliceSetEq(tp.ExcludeUIDs, o.ExcludeUIDs) {
return false
}
if tp.NoDNS != o.NoDNS {
return false
}
return true
}

View File

@@ -7,6 +7,7 @@ import (
"testing"
"github.com/hashicorp/nomad/ci"
"github.com/shoenig/test/must"
"github.com/stretchr/testify/require"
)
@@ -25,3 +26,65 @@ func TestTaskKind_IsAnyConnectGateway(t *testing.T) {
require.False(t, NewTaskKind("", "foo").IsAnyConnectGateway())
})
}
func TestConnectTransparentProxy_Validate(t *testing.T) {
testCases := []struct {
name string
tp *ConsulTransparentProxy
expectErr string
}{
{
name: "empty is valid",
tp: &ConsulTransparentProxy{},
},
{
name: "invalid CIDR",
tp: &ConsulTransparentProxy{ExcludeOutboundCIDRs: []string{"192.168.1.1"}},
expectErr: `could not parse transparent proxy excluded outbound CIDR as network prefix: netip.ParsePrefix("192.168.1.1"): no '/'`,
},
{
name: "invalid UID",
tp: &ConsulTransparentProxy{UID: "foo"},
expectErr: `transparent proxy block has invalid UID field: invalid user ID "foo": invalid syntax`,
},
{
name: "invalid ExcludeUIDs",
tp: &ConsulTransparentProxy{ExcludeUIDs: []string{"500000"}},
expectErr: `transparent proxy block has invalid ExcludeUIDs field: invalid user ID "500000": value out of range`,
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
err := tc.tp.Validate()
if tc.expectErr != "" {
must.EqError(t, err, tc.expectErr)
} else {
must.NoError(t, err)
}
})
}
}
func TestConnectTransparentProxy_Equal(t *testing.T) {
tp1 := &ConsulTransparentProxy{
UID: "101",
OutboundPort: 1001,
ExcludeInboundPorts: []string{"9000", "443"},
ExcludeOutboundPorts: []uint16{443, 80},
ExcludeOutboundCIDRs: []string{"10.0.0.0/8", "192.168.1.1"},
ExcludeUIDs: []string{"1001", "10"},
NoDNS: true,
}
tp2 := &ConsulTransparentProxy{
UID: "101",
OutboundPort: 1001,
ExcludeInboundPorts: []string{"443", "9000"},
ExcludeOutboundPorts: []uint16{80, 443},
ExcludeOutboundCIDRs: []string{"192.168.1.1", "10.0.0.0/8"},
ExcludeUIDs: []string{"10", "1001"},
NoDNS: true,
}
must.Equal(t, tp1, tp2)
}

View File

@@ -11,6 +11,7 @@ import (
"strconv"
"strings"
"github.com/hashicorp/nomad/helper"
"github.com/hashicorp/nomad/helper/flatmap"
"github.com/mitchellh/hashstructure"
)
@@ -1734,6 +1735,10 @@ func consulProxyDiff(old, new *ConsulProxy, contextual bool) *ObjectDiff {
diff.Objects = append(diff.Objects, exposeDiff)
}
if tproxyDiff := consulTProxyDiff(old.TransparentProxy, new.TransparentProxy, contextual); tproxyDiff != nil {
diff.Objects = append(diff.Objects, tproxyDiff)
}
// diff the config blob
if cDiff := configDiff(old.Config, new.Config, contextual); cDiff != nil {
diff.Objects = append(diff.Objects, cDiff)
@@ -1844,6 +1849,57 @@ func consulProxyExposeDiff(prev, next *ConsulExposeConfig, contextual bool) *Obj
return diff
}
func consulTProxyDiff(prev, next *ConsulTransparentProxy, contextual bool) *ObjectDiff {
diff := &ObjectDiff{Type: DiffTypeNone, Name: "TransparentProxy"}
var oldPrimFlat, newPrimFlat map[string]string
if prev.Equal(next) {
return diff
} else if prev == nil {
prev = &ConsulTransparentProxy{}
diff.Type = DiffTypeAdded
newPrimFlat = flatmap.Flatten(next, nil, true)
} else if next == nil {
next = &ConsulTransparentProxy{}
diff.Type = DiffTypeDeleted
oldPrimFlat = flatmap.Flatten(prev, nil, true)
} else {
diff.Type = DiffTypeEdited
oldPrimFlat = flatmap.Flatten(prev, nil, true)
newPrimFlat = flatmap.Flatten(next, nil, true)
}
// diff the primitive fields
diff.Fields = fieldDiffs(oldPrimFlat, newPrimFlat, contextual)
if setDiff := stringSetDiff(prev.ExcludeInboundPorts, next.ExcludeInboundPorts,
"ExcludeInboundPorts", contextual); setDiff != nil && setDiff.Type != DiffTypeNone {
diff.Objects = append(diff.Objects, setDiff)
}
if setDiff := stringSetDiff(
helper.ConvertSlice(prev.ExcludeOutboundPorts, func(a uint16) string { return fmt.Sprint(a) }),
helper.ConvertSlice(next.ExcludeOutboundPorts, func(a uint16) string { return fmt.Sprint(a) }),
"ExcludeOutboundPorts",
contextual,
); setDiff != nil && setDiff.Type != DiffTypeNone {
diff.Objects = append(diff.Objects, setDiff)
}
if setDiff := stringSetDiff(prev.ExcludeOutboundCIDRs, next.ExcludeOutboundCIDRs,
"ExcludeOutboundCIDRs", contextual); setDiff != nil && setDiff.Type != DiffTypeNone {
diff.Objects = append(diff.Objects, setDiff)
}
if setDiff := stringSetDiff(prev.ExcludeUIDs, next.ExcludeUIDs,
"ExcludeUIDs", contextual); setDiff != nil && setDiff.Type != DiffTypeNone {
diff.Objects = append(diff.Objects, setDiff)
}
return diff
}
// serviceCheckDiffs diffs a set of service checks. If contextual diff is
// enabled, unchanged fields within objects nested in the tasks will be
// returned.

View File

@@ -3490,6 +3490,15 @@ func TestTaskGroupDiff(t *testing.T) {
Config: map[string]interface{}{
"foo": "qux",
},
TransparentProxy: &ConsulTransparentProxy{
UID: "101",
OutboundPort: 15001,
ExcludeInboundPorts: []string{"www", "9000"},
ExcludeOutboundPorts: []uint16{4443},
ExcludeOutboundCIDRs: []string{"10.0.0.0/8"},
ExcludeUIDs: []string{"1", "10"},
NoDNS: true,
},
},
},
Gateway: &ConsulGateway{
@@ -3924,6 +3933,92 @@ func TestTaskGroupDiff(t *testing.T) {
},
},
},
{
Type: DiffTypeAdded,
Name: "TransparentProxy",
Objects: []*ObjectDiff{
{
Type: DiffTypeAdded,
Name: "ExcludeInboundPorts",
Fields: []*FieldDiff{
{
Type: DiffTypeAdded,
Name: "ExcludeInboundPorts",
Old: "",
New: "9000",
},
{
Type: DiffTypeAdded,
Name: "ExcludeInboundPorts",
Old: "",
New: "www",
},
},
},
{
Type: DiffTypeAdded,
Name: "ExcludeOutboundPorts",
Fields: []*FieldDiff{
{
Type: DiffTypeAdded,
Name: "ExcludeOutboundPorts",
Old: "",
New: "4443",
},
},
},
{
Type: DiffTypeAdded,
Name: "ExcludeOutboundCIDRs",
Fields: []*FieldDiff{
{
Type: DiffTypeAdded,
Name: "ExcludeOutboundCIDRs",
Old: "",
New: "10.0.0.0/8",
},
},
},
{
Type: DiffTypeAdded,
Name: "ExcludeUIDs",
Fields: []*FieldDiff{
{
Type: DiffTypeAdded,
Name: "ExcludeUIDs",
Old: "",
New: "1",
},
{
Type: DiffTypeAdded,
Name: "ExcludeUIDs",
Old: "",
New: "10",
},
},
},
},
Fields: []*FieldDiff{
{
Type: DiffTypeAdded,
Name: "NoDNS",
Old: "",
New: "true",
},
{
Type: DiffTypeAdded,
Name: "OutboundPort",
Old: "",
New: "15001",
},
{
Type: DiffTypeAdded,
Name: "UID",
Old: "",
New: "101",
},
},
},
{
Type: DiffTypeAdded,
Name: "Config",
@@ -10024,6 +10119,10 @@ func TestServicesDiff(t *testing.T) {
},
Objects: nil,
},
{
Type: DiffTypeNone,
Name: "TransparentProxy",
},
},
},
},

View File

@@ -144,3 +144,21 @@ func (j *Job) RequiredBridgeNetwork() set.Collection[string] {
}
return result
}
// RequiredTransparentProxy identifies which task groups, if any, within the job
// contain Connect blocks using transparent proxy
func (j *Job) RequiredTransparentProxy() set.Collection[string] {
result := set.New[string](len(j.TaskGroups))
for _, tg := range j.TaskGroups {
for _, service := range tg.Services {
if service.Connect != nil {
if service.Connect.HasTransparentProxy() {
result.Insert(tg.Name)
break // to next TaskGroup
}
}
}
}
return result
}

View File

@@ -471,3 +471,46 @@ func TestJob_RequiredNUMA(t *testing.T) {
})
}
}
func TestJob_RequiredTproxy(t *testing.T) {
job := &Job{
TaskGroups: []*TaskGroup{
{Name: "no services"},
{Name: "services-without-connect",
Services: []*Service{{Name: "foo"}},
},
{Name: "services-with-connect-but-no-tproxy",
Services: []*Service{
{Name: "foo", Connect: &ConsulConnect{}},
{Name: "bar", Connect: &ConsulConnect{}}},
},
{Name: "has-tproxy-1",
Services: []*Service{
{Name: "foo", Connect: &ConsulConnect{}},
{Name: "bar", Connect: &ConsulConnect{
SidecarService: &ConsulSidecarService{
Proxy: &ConsulProxy{
TransparentProxy: &ConsulTransparentProxy{},
},
},
}}},
},
{Name: "has-tproxy-2",
Services: []*Service{
{Name: "baz", Connect: &ConsulConnect{
SidecarService: &ConsulSidecarService{
Proxy: &ConsulProxy{
TransparentProxy: &ConsulTransparentProxy{},
},
},
}}},
},
},
}
expect := []string{"has-tproxy-1", "has-tproxy-2"}
job.Canonicalize()
result := job.RequiredTransparentProxy()
must.SliceContainsAll(t, expect, result.Slice())
}

View File

@@ -958,6 +958,7 @@ func hashConnect(h hash.Hash, connect *ConsulConnect) {
hashString(h, p.LocalServiceAddress)
hashString(h, strconv.Itoa(p.LocalServicePort))
hashConfig(h, p.Config)
hashTProxy(h, p.TransparentProxy)
for _, upstream := range p.Upstreams {
hashString(h, upstream.DestinationName)
hashString(h, upstream.DestinationNamespace)
@@ -1015,6 +1016,22 @@ func hashConfig(h hash.Hash, c map[string]interface{}) {
_, _ = fmt.Fprintf(h, "%v", c)
}
func hashTProxy(h hash.Hash, tp *ConsulTransparentProxy) {
if tp == nil {
return
}
hashStringIfNonEmpty(h, tp.UID)
hashIntIfNonZero(h, "OutboundPort", int(tp.OutboundPort))
hashTags(h, tp.ExcludeInboundPorts)
for _, port := range tp.ExcludeOutboundPorts {
hashIntIfNonZero(h, "ExcludeOutboundPorts", int(port))
}
hashTags(h, tp.ExcludeOutboundCIDRs)
hashTags(h, tp.ExcludeUIDs)
hashBool(h, tp.NoDNS, "NoDNS")
}
// Equal returns true if the structs are recursively equal.
func (s *Service) Equal(o *Service) bool {
if s == nil || o == nil {
@@ -1187,6 +1204,14 @@ func (c *ConsulConnect) IsMesh() bool {
return c.IsGateway() && c.Gateway.Mesh != nil
}
// HasTransparentProxy checks if a service with a Connect sidecar has a
// transparent proxy configuration
func (c *ConsulConnect) HasTransparentProxy() bool {
return c.HasSidecar() &&
c.SidecarService.Proxy != nil &&
c.SidecarService.Proxy.TransparentProxy != nil
}
// Validate that the Connect block represents exactly one of:
// - Connect non-native service sidecar proxy
// - Connect native service
@@ -1201,6 +1226,11 @@ func (c *ConsulConnect) Validate() error {
count := 0
if c.HasSidecar() {
if c.HasTransparentProxy() {
if err := c.SidecarService.Proxy.TransparentProxy.Validate(); err != nil {
return err
}
}
count++
}
@@ -1222,7 +1252,8 @@ func (c *ConsulConnect) Validate() error {
}
}
// The Native and Sidecar cases are validated up at the service level.
// Checking against the surrounding task group is validated up at the
// service level or job endpint connect validation hook
return nil
}
@@ -1509,6 +1540,11 @@ type ConsulProxy struct {
// used by task-group level service checks using HTTP or gRPC protocols.
Expose *ConsulExposeConfig
// TransparentProxy configures the Envoy sidecar to use "transparent
// proxying", which creates IP tables rules inside the network namespace to
// ensure traffic flows thru the Envoy proxy
TransparentProxy *ConsulTransparentProxy
// Config is a proxy configuration. It is opaque to Nomad and passed
// directly to Consul.
Config map[string]interface{}
@@ -1525,6 +1561,7 @@ func (p *ConsulProxy) Copy() *ConsulProxy {
LocalServicePort: p.LocalServicePort,
Expose: p.Expose.Copy(),
Upstreams: slices.Clone(p.Upstreams),
TransparentProxy: p.TransparentProxy.Copy(),
Config: maps.Clone(p.Config),
}
}
@@ -1551,6 +1588,10 @@ func (p *ConsulProxy) Equal(o *ConsulProxy) bool {
return false
}
if !p.TransparentProxy.Equal(o.TransparentProxy) {
return false
}
// envoy config, use reflect
if !reflect.DeepEqual(p.Config, o.Config) {
return false

View File

@@ -432,6 +432,15 @@ func TestService_Hash(t *testing.T) {
LocalBindPort: 29000,
Config: map[string]any{"foo": "bar"},
}},
TransparentProxy: &ConsulTransparentProxy{
UID: "101",
OutboundPort: 15001,
ExcludeInboundPorts: []string{"www", "9000"},
ExcludeOutboundPorts: []uint16{4443},
ExcludeOutboundCIDRs: []string{"10.0.0.0/8"},
ExcludeUIDs: []string{"1", "10"},
NoDNS: true,
},
},
Meta: map[string]string{
"test-key": "test-value",
@@ -529,6 +538,54 @@ func TestService_Hash(t *testing.T) {
t.Run("mod connect sidecar proxy upstream config", func(t *testing.T) {
try(t, func(s *svc) { s.Connect.SidecarService.Proxy.Upstreams[0].Config = map[string]any{"foo": "baz"} })
})
t.Run("mod connect transparent proxy removed", func(t *testing.T) {
try(t, func(s *svc) {
s.Connect.SidecarService.Proxy.TransparentProxy = nil
})
})
t.Run("mod connect transparent proxy uid", func(t *testing.T) {
try(t, func(s *svc) {
s.Connect.SidecarService.Proxy.TransparentProxy.UID = "42"
})
})
t.Run("mod connect transparent proxy outbound port", func(t *testing.T) {
try(t, func(s *svc) {
s.Connect.SidecarService.Proxy.TransparentProxy.OutboundPort = 42
})
})
t.Run("mod connect transparent proxy inbound ports", func(t *testing.T) {
try(t, func(s *svc) {
s.Connect.SidecarService.Proxy.TransparentProxy.ExcludeInboundPorts = []string{"443"}
})
})
t.Run("mod connect transparent proxy outbound ports", func(t *testing.T) {
try(t, func(s *svc) {
s.Connect.SidecarService.Proxy.TransparentProxy.ExcludeOutboundPorts = []uint16{42}
})
})
t.Run("mod connect transparent proxy outbound cidr", func(t *testing.T) {
try(t, func(s *svc) {
s.Connect.SidecarService.Proxy.TransparentProxy.ExcludeOutboundCIDRs = []string{"192.168.1.0/24"}
})
})
t.Run("mod connect transparent proxy exclude uids", func(t *testing.T) {
try(t, func(s *svc) {
s.Connect.SidecarService.Proxy.TransparentProxy.ExcludeUIDs = []string{"42"}
})
})
t.Run("mod connect transparent proxy no dns", func(t *testing.T) {
try(t, func(s *svc) {
s.Connect.SidecarService.Proxy.TransparentProxy.NoDNS = false
})
})
}
func TestConsulConnect_Validate(t *testing.T) {

View File

@@ -741,7 +741,7 @@ client {
[metadata_constraint]: /nomad/docs/job-specification/constraint#user-specified-metadata 'Nomad User-Specified Metadata Constraint Example'
[runtime_var_interpolation]: /nomad/docs/runtime/interpolation
[task working directory]: /nomad/docs/runtime/environment#task-directories 'Task directories'
[go-sockaddr/template]: https://godoc.org/github.com/hashicorp/go-sockaddr/template
[go-sockaddr/template]: https://pkg.go.dev/github.com/hashicorp/go-sockaddr/template
[landlock]: https://docs.kernel.org/userspace-api/landlock.html
[`leave_on_interrupt`]: /nomad/docs/configuration#leave_on_interrupt
[`leave_on_terminate`]: /nomad/docs/configuration#leave_on_terminate

View File

@@ -392,7 +392,7 @@ http_api_response_headers {
[`server`]: /nomad/docs/configuration/server 'Nomad Agent server Configuration'
[tls]: /nomad/docs/configuration/tls 'Nomad Agent tls Configuration'
[`vault`]: /nomad/docs/configuration/vault 'Nomad Agent vault Configuration'
[go-sockaddr/template]: https://godoc.org/github.com/hashicorp/go-sockaddr/template
[go-sockaddr/template]: https://pkg.go.dev/github.com/hashicorp/go-sockaddr/template
[log-api]: /nomad/api-docs/client#stream-logs
[hcl]: https://github.com/hashicorp/hcl 'HashiCorp Configuration Language'
[tls-reload]: /nomad/docs/configuration/tls#tls-configuration-reloads

View File

@@ -134,6 +134,43 @@ service_prefix "" { policy = "read" }
node_prefix "" { policy = "read" }
```
#### Transparent Proxy
Using Nomad's support for [transparent proxy][] configures the task group's
network namespace so that traffic flows through the Envoy proxy. When the
[`transparent_proxy`][] block is enabled:
* Nomad will invoke the [`consul-cni`][] CNI plugin to configure `iptables` rules
in the network namespace to force outbound traffic from an allocation to flow
through the proxy.
* If the local Consul agent is serving DNS, Nomad will set the IP address of the
Consul agent as the nameserver in the task's `/etc/resolv.conf`.
* Consul will provide a [virtual IP][] for any upstream service the workload
has access to, based on the service intentions.
Using transparent proxy has several important requirements:
* You must have the [`consul-cni`][] CNI plugin installed on the client host
along with the usual [required CNI plugins][cni_plugins].
* To use Consul DNS and virtual IPs, you will need to configure Consul's DNS
listener to be exposed to the workload network namespace. You can do this
without exposing the Consul agent on a public IP by setting the Consul
`bind_addr` to bind on a private IP address (the default is to use the
`client_addr`).
* The Consul agent must be configured with [`recursors`][] if you want
allocations to make DNS queries for applications outside the service mesh.
* You cannot set a [`network.dns`][] block on the allocation (unless you set
[`no_dns`][tproxy_no_dns], see below).
For example, a HCL configuration with a [go-sockaddr/template][] binding to the
subnet `10.37.105.0/20`, with recursive DNS set to OpenDNS nameservers:
```hcl
bind_addr = "{{ GetPrivateInterfaces | include \"network\" \"10.37.105.0/20\" | limit 1 | attr \"address\" }}"
recursors = ["208.67.222.222", "208.67.220.220"]
```
### Nomad
Nomad must schedule onto a routable interface in order for the proxies to
@@ -150,10 +187,14 @@ Nomad uses CNI reference plugins to configure the network namespace used to secu
Consul service mesh sidecar proxy. All Nomad client nodes using network namespaces
must have these CNI plugins [installed][cni_install].
To use [`transparent_proxy`][] mode, Nomad client nodes will also need the
[`consul-cni`][] plugin installed.
## Run the Service Mesh-enabled Services
Once Nomad and Consul are running, submit the following service mesh-enabled services
to Nomad by copying the HCL into a file named `servicemesh.nomad.hcl` and running:
Once Nomad and Consul are running, with Consul DNS enabled for transparent proxy
mode as described above, submit the following service mesh-enabled services to
Nomad by copying the HCL into a file named `servicemesh.nomad.hcl` and running:
`nomad job run servicemesh.nomad.hcl`
```hcl
@@ -170,7 +211,11 @@ job "countdash" {
port = "9001"
connect {
sidecar_service {}
sidecar_service {
proxy {
transparent_proxy {}
}
}
}
}
@@ -200,10 +245,7 @@ job "countdash" {
connect {
sidecar_service {
proxy {
upstreams {
destination_name = "count-api"
local_bind_port = 8080
}
transparent_proxy {}
}
}
}
@@ -213,7 +255,7 @@ job "countdash" {
driver = "docker"
env {
COUNTING_SERVICE_URL = "http://${NOMAD_UPSTREAM_ADDR_count_api}"
COUNTING_SERVICE_URL = "http://count-api.virtual.consul"
}
config {
@@ -231,35 +273,41 @@ The job contains two task groups: an API service and a web frontend.
The API service is defined as a task group with a bridge network:
```hcl
group "api" {
network {
mode = "bridge"
}
# ...
group "api" {
network {
mode = "bridge"
}
# ...
}
```
Since the API service is only accessible via Consul service mesh, it does not define
any ports in its network. The service block enables service mesh.
Since the API service is only accessible via Consul service mesh, it does not
define any ports in its network. The `connect` block enables the service mesh
and the `transparent_proxy` block ensures that the service will be reachable via
a virtual IP address when used with Consul DNS.
```hcl
group "api" {
group "api" {
# ...
# ...
service {
name = "count-api"
port = "9001"
service {
name = "count-api"
port = "9001"
connect {
sidecar_service {}
connect {
sidecar_service {
proxy {
transparent_proxy {}
}
}
}
# ...
}
# ...
}
```
The `port` in the service block is the port the API service listens on. The
@@ -273,19 +321,19 @@ The web frontend is defined as a task group with a bridge network and a static
forwarded port:
```hcl
group "dashboard" {
network {
mode = "bridge"
group "dashboard" {
network {
mode = "bridge"
port "http" {
static = 9002
to = 9002
}
port "http" {
static = 9002
to = 9002
}
# ...
}
# ...
}
```
The `static = 9002` parameter requests the Nomad scheduler reserve port 9002 on
@@ -300,39 +348,103 @@ This allows you to connect to the web frontend in a browser by visiting
The web frontend connects to the API service via Consul service mesh.
```hcl
service {
name = "count-dashboard"
port = "http"
service {
name = "count-dashboard"
port = "http"
connect {
sidecar_service {
proxy {
upstreams {
destination_name = "count-api"
local_bind_port = 8080
}
}
connect {
sidecar_service {
proxy {
transparent_proxy {}
}
}
}
}
```
The `connect` block with `transparent_proxy` configures the web frontend's
network namespace to route all access to the `count-api` service through the
Envoy proxy.
The web frontend is configured to communicate with the API service with an
environment variable `$COUNTING_SERVICE_URL`:
```hcl
env {
COUNTING_SERVICE_URL = "http://count-api.virtual.consul"
}
```
The `transparent_proxy` block ensures that DNS queries are made to Consul so
that the `count-api.virtual.consul` name resolves to a virtual IP address. Note
that you don't need to specify a port number because the virtual IP will only be
directed to the correct service port.
### Manually Configured Upstreams
You can also use Connect without Consul DNS and `transparent_proxy` mode. This
approach is not recommended because it requires duplicating service intention
information in an `upstreams` block in the Nomad job specification. But Consul
DNS is not protected by ACLs, so you might want to do this if you don't want to
expose Consul DNS to untrusted workloads.
In that case, you can add `upstream` blocks to the job spec. You don't need the
`transparent_proxy` block for the `count-api` service:
```hcl
group "api" {
# ...
service {
name = "count-api"
port = "9001"
connect {
sidecar_service {}
}
}
# ...
}
```
But you'll need to add an `upstreams` block to the `count-dashboard` service:
```hcl
service {
name = "count-dashboard"
port = "http"
connect {
sidecar_service {
proxy {
upstreams {
destination_name = "count-api"
local_bind_port = 8080
}
}
}
}
}
```
The `upstreams` block defines the remote service to access (`count-api`) and
what port to expose that service on inside the network namespace (`8080`).
The web frontend is configured to communicate with the API service with an
environment variable:
The web frontend will also need to use an environment variable to communicate
with the API service:
```hcl
env {
COUNTING_SERVICE_URL = "http://${NOMAD_UPSTREAM_ADDR_count_api}"
}
env {
COUNTING_SERVICE_URL = "http://${NOMAD_UPSTREAM_ADDR_count_api}"
}
```
The web frontend is configured via the `$COUNTING_SERVICE_URL`, so you must
interpolate the upstream's address into that environment variable. Note that
dashes (`-`) are converted to underscores (`_`) in environment variables so
`count-api` becomes `count_api`.
This environment variable value gets interpolated with the upstream's
address. Note that dashes (`-`) are converted to underscores (`_`) in
environment variables so `count-api` becomes `count_api`.
## Limitations
@@ -377,3 +489,13 @@ filesystem.
[consul_ports]: /consul/docs/agent/config/config-files#ports
[consul_grpc_tls]: /consul/docs/upgrading/upgrade-specific#changes-to-grpc-tls-configuration
[cni_install]: /nomad/docs/install#post-installation-steps
[transparent proxy]: /consul/docs/k8s/connect/transparent-proxy
[go-sockaddr/template]: https://pkg.go.dev/github.com/hashicorp/go-sockaddr/template
[`recursors`]: /consul/docs/agent/config/config-files#recursors
[`transparent_proxy`]: /nomad/docs/job-specification/transparent_proxy
[tproxy_no_dns]: /nomad/docs/job-specification/transparent_proxy#no_dns
[`consul-cni`]: https://releases.hashicorp.com/consul-cni
[virtual IP]: /consul/docs/services/discovery/dns-static-lookups#service-virtual-ip-lookups
[cni_plugins]: /nomad/docs/networking/cni#cni-reference-plugins
[consul_dns_port]: /consul/docs/agent/config/config-files#dns_port
[`network.dns`]: /nomad/docs/job-specification/network#dns-parameters

View File

@@ -229,7 +229,7 @@ check {
```
[network-to]: /nomad/docs/job-specification/network#to
[consul-expose-path-config]: /consul/docs/connect/registration/service-registration#expose-paths-configuration-reference
[consul-expose-path-config]: /consul/docs/connect/proxies/proxy-config-reference#expose-paths-configuration-reference
[expose-path]: /nomad/docs/job-specification/expose#path-1
[expose]: /nomad/docs/job-specification/service#expose
[path]: /nomad/docs/job-specification/expose#path-parameters 'Nomad Expose Path Parameters'

View File

@@ -13,9 +13,8 @@ description: |-
/>
The `proxy` block allows configuring various options for the sidecar proxy
managed by Nomad for [Consul
Connect](/nomad/docs/integrations/consul-connect). It is valid only
within the context of a `sidecar_service` block.
managed by Nomad for [Consul Connect][]. It is valid only within the context of
a `sidecar_service` block.
```hcl
job "countdash" {
@@ -50,23 +49,29 @@ job "countdash" {
## `proxy` Parameters
- `local_service_address` `(string: "127.0.0.1")` - The address the local service binds to. Useful to
customize in clusters with mixed Connect and non-Connect services.
- `config` `(map: nil)` - Proxy configuration that is opaque to Nomad and passed
directly to Consul. See [Consul Connect documentation][envoy_dynamic_config]
for details. Keys and values support [runtime variable interpolation][].
- `expose` <code>([expose]: nil)</code> - Used to configure expose path
configuration for Envoy. See Consul's [Expose Paths Configuration
Reference][expose_path_ref] for more information.
- `local_service_address` `(string: "127.0.0.1")` - The address the local
service binds to. Useful to customize in clusters with mixed Connect and
non-Connect services.
- `local_service_port` `(int: <varies>)` - The port the local service binds to.
Usually the same as the parent service's port, it is useful to customize in clusters with mixed
Connect and non-Connect services.
- `upstreams` <code>([upstreams][]: nil)</code> - Used to configure details of each upstream service that
this sidecar proxy communicates with.
- `expose` <code>([expose]: nil)</code> - Used to configure expose path configuration for Envoy.
See Consul's [Expose Paths Configuration Reference](/consul/docs/connect/registration/service-registration#expose-paths-configuration-reference)
for more information.
- `config` `(map: nil)` - Proxy configuration that is opaque to Nomad and
passed directly to Consul. See [Consul Connect documentation](/consul/docs/connect/proxies/envoy#dynamic-configuration)
for details. Keys and values support [runtime variable interpolation][interpolation].
Usually the same as the parent service's port, it is useful to customize in
clusters with mixed Connect and non-Connect services.
- `transparent_proxy` <code>([transparent_proxy][]: nil)</code> - Used to enable
[transparent proxy][tproxy] mode, which allows the proxy to use Consul service
intentions to automatically configure upstreams, and configures iptables rules
to force traffic from the allocation to flow through the proxy.
- `upstreams` <code>([upstreams][]: nil)</code> - Used to configure details of
each upstream service that this sidecar proxy communicates with.
## `proxy` Examples
The following example is a proxy specification that includes upstreams configuration.
The following example is a proxy specification that includes upstreams
configuration.
```hcl
sidecar_service {
@@ -79,10 +84,28 @@ sidecar_service {
}
```
The following example is a proxy specification that includes transparent proxy
configuration. Note that with transparent proxy, you will not need to configure
an `upstreams` block.
```hcl
sidecar_service {
proxy {
transparent_proxy {
}
}
}
```
[Consul Connect]: /nomad/docs/integrations/consul-connect
[job]: /nomad/docs/job-specification/job 'Nomad job Job Specification'
[group]: /nomad/docs/job-specification/group 'Nomad group Job Specification'
[task]: /nomad/docs/job-specification/task 'Nomad task Job Specification'
[interpolation]: /nomad/docs/runtime/interpolation 'Nomad interpolation'
[runtime variable interpolation]: /nomad/docs/runtime/interpolation 'Nomad interpolation'
[sidecar_service]: /nomad/docs/job-specification/sidecar_service 'Nomad sidecar service Specification'
[upstreams]: /nomad/docs/job-specification/upstreams 'Nomad upstream config Specification'
[expose]: /nomad/docs/job-specification/expose 'Nomad proxy expose configuration'
[envoy_dynamic_config]: /consul/docs/connect/proxies/envoy#dynamic-configuration
[expose_path_ref]: /consul/docs/connect/proxies/proxy-config-reference#expose-paths-configuration-reference
[transparent_proxy]: /nomad/docs/job-specification/transparent_proxy
[tproxy]: /consul/docs/k8s/connect/transparent-proxy

View File

@@ -0,0 +1,174 @@
---
layout: docs
page_title: transparent_proxy Block - Job Specification
description: |-
The "transparent_proxy" block allows specifying options for configuring Envoy
in Consul Connect transparent proxy mode.
---
# `transparent_proxy` Block
<Placement
groups={[
'job',
'group',
'service',
'connect',
'sidecar_service',
'proxy',
'transparent_proxy',
]}
/>
The `transparent_proxy` block configures the Envoy sidecar proxy to act as a
Consul Connect [transparent proxy][tproxy]. This simplifies the configuration of
Consul Connect by eliminating the need to configure [`upstreams`][] blocks in
Nomad. Instead, the Envoy proxy will determines its configuration entirely from
Consul [service intentions][].
When transparent proxy is enabled traffic will automatically flow through the
Envoy proxy. If the local Consul agent is serving DNS, Nomad will also set up
the task's nameservers to use Consul. This lets your workload use the [virtual
IP][] DNS name from Consul, rather than configuring a `template` block that
queries services.
Using transparent proxy has some important restrictions:
* You can only have a single `connect` block in any task group that uses
transparent proxy.
* You cannot set a [`network.dns`][] block on the allocation (unless you set
[`no_dns`](#no_dns), see below).
* The node where the allocation is placed must be configured as described in
the Service Mesh integration documentation for [Transparent Proxy][].
## `transparent_proxy` Parameters
* `exclude_inbound_ports` `([]string: nil)` - A list of inbound ports to exclude
from the inbound traffic redirection. This allows traffic on these ports to
bypass the Envoy proxy. These ports can be specified as either [network port
labels][port_labels] or as numeric ports. Nomad will automatically add the
following to this list:
* The [`local_path_port`][] of any [`expose`][] block.
* The port of any service check with [`expose=true`][check_expose] set.
* The port of any `network.port` with a [`static`][] value.
* `exclude_outbound_cidrs` `([]string: nil)` - A list of CIDR subnets that
should be excluded from outbound traffic redirection. This allows traffic to
these subnets to bypass the Envoy proxy. Note this is independent of
`exclude_outbound_ports`; CIDR subnets listed here are excluded regardless of
the port.
* `exclude_outbound_ports` `([]int: nil)` - A list of port numbers that should
be excluded from outbound traffic redirection. This allows traffic to these
subnets to bypass the Envoy proxy. Note this is independent of
`exclude_outbound_cidrs`; ports listed here are excluded regardless of the
CIDR.
* `exclude_uids` `([]string: nil)` - A list of Unix user IDs (UIDs) that should
be excluded from outbound traffic redirection. When unset, only the Envoy
proxy's user will be allowed to bypass the iptables rule.
* `no_dns` `(bool: false)` - By default, Consul will be set as the nameserver
for the workload and IP tables rules will redirect DNS queries to Consul. If
you want only external DNS, set `no_dns=true`. You will need to add your own
CIDR and port exclusions for your DNS nameserver. You cannot set
[`network.dns`][] if `no_dns=false`.
* `outbound_port` `(int: 15001)` - The port that Envoy will bind on inside the
network namespace. The iptables rules created by `consul-cni` will force
traffic to flow to this port. You should only set this value if you have
specifically set the [`outbound_listener_port`][] in your Consul proxy
configuration. You can change the default value for a given node via [client
metadata](#client-metadata) (see below).
* `uid` `(string "101")` - The Unix user ID (UID) used by the Envoy proxy. You
should only set this value if you have a custom build of the Envoy container
image which uses a different UID. You can change the default value for a given
node via [client metadata](#client-metadata) (see below).
## Client Metadata
You can change the default [`outbound_port`](#outbound_port) and [`uid`](#uid)
for a given client node by updating the node metadata via the [`nomad node meta
apply`][] command. The attributes that can be updated are:
* `connect.transparent_proxy.default_uid`: Sets the default value of
[`uid`](#uid) for this node.
* `connect.transparent_proxy.default_outbound_port`: Sets the default value of
[`outbound_port`](#outbound_port) for this node.
For example, to set the default value for the `uid` field to 120:
```shell-session
$ nomad node meta apply connect.transparent_proxy.default_uid=120
$ nomad node meta read -json | jq -r '.Dynamic | ."connect.transparent_proxy.default_uid"'
120
```
You should not normally need to set these values unless you are using custom
Envoy images.
## Examples
### Minimal Example
The following example is a minimal transparent proxy specification. Note that
with transparent proxy, you will not need to configure an `upstreams` block.
```hcl
sidecar_service {
proxy {
transparent_proxy {
}
}
}
```
If you had a downstream task group `count-dashboard` that needed to connect to
an upstream task group `count-api` listening on port 9001, you could create a
Consul service intention with the following specification:
```hcl
Kind = "service-intentions"
Name = "count-api"
Sources = [
{
Name = "count-dashboard"
Action = "allow"
}
]
```
And then the downstream service `count-dashboard` could reach the `count-api`
service by making requests to `http://count-api.virtual.consul`.
### External DNS
The following example is a transparent proxy specification where external DNS is
used. To find the address of other allocations in this configuration, you will
need to use a [`template`][] block to query Consul.
```hcl
sidecar_service {
proxy {
transparent_proxy {
excluded_outbound_ports = [53]
excluded_outbound_cidrs = ["208.67.222.222/32", "208.67.220.220/32"]
no_dns = true
}
}
}
```
[tproxy]: /consul/docs/k8s/connect/transparent-proxy
[`upstreams`]: /nomad/docs/job-specification/upstreams
[service intentions]: /consul/docs/connect/config-entries/service-intentions
[virtual IP]: /consul/docs/services/discovery/dns-static-lookups#service-virtual-ip-lookups
[`consul-cni`]: https://releases.hashicorp.com/consul-cni
[cni_plugins]: /nomad/docs/networking/cni#cni-reference-plugins
[consul_dns_port]: /consul/docs/agent/config/config-files#dns_port
[`recursors`]: /consul/docs/agent/config/config-files#recursors
[port_labels]: /nomad/docs/job-specification/network#port-parameters
[`local_path_port`]: /nomad/docs/job-specification/expose#local_path_port
[`expose`]: /nomad/docs/job-specification/expose
[check_expose]: /nomad/docs/job-specification/service#expose
[`static`]: /nomad/docs/job-specification/network#static
[`outbound_listener_port`]: /consul/docs/connect/proxies/proxy-config-reference#outbound_listener_port
[`template`]: /nomad/docs/job-specification/template#consul-integration
[`nomad node meta apply`]: /nomad/docs/commands/node/meta/apply
[`network.dns`]: /nomad/docs/job-specification/network#dns-parameters
[Transparent Proxy]: /nomad/docs/integrations/consul/service-mesh#transparent-proxy

View File

@@ -21,12 +21,12 @@ description: |-
/>
The `upstreams` block allows configuring various options for managing upstream
services that a [Consul
Connect](/nomad/docs/integrations/consul-connect) proxy routes to. It
is valid only within the context of a `proxy` block.
services that a [Consul Connect][] proxy routes to. It is valid only within the
context of a `proxy` block.
For Consul-specific details see the [Consul Connect
Guide](/consul/tutorials/get-started-vms/virtual-machine-gs-service-discovery).
For Consul-specific details see the [Consul Connect Guide][]. Note that using
`upstream` may not be necessary if you have configured the proxy with the
[`transparent_proxy`][] block.
```hcl
job "countdash" {
@@ -82,7 +82,7 @@ job "countdash" {
## `upstreams` Parameters
- `config` `(map: nil)` - Upstream configuration that is opaque to Nomad and passed
directly to Consul. See [Consul Connect documentation](/consul/docs/connect/registration/service-registration#upstream-configuration-reference)
directly to Consul. See [Consul Connect documentation][consul_expose_path_ref]
for details. Keys and values support [runtime variable interpolation][interpolation].
- `destination_name` `(string: <required>)` - Name of the upstream service.
- `destination_namespace` `(string: <required>)` - Name of the upstream Consul namespace.
@@ -135,6 +135,9 @@ and a local bind port.
}
```
[Consul Connect]: /nomad/docs/integrations/consul-connect
[Consul Connect Guide]: /consul/tutorials/get-started-vms/virtual-machine-gs-service-discovery
[`transparent_proxy`]: /nomad/docs/job-specification/transparent_proxy
[job]: /nomad/docs/job-specification/job 'Nomad job Job Specification'
[group]: /nomad/docs/job-specification/group 'Nomad group Job Specification'
[task]: /nomad/docs/job-specification/task 'Nomad task Job Specification'
@@ -144,3 +147,4 @@ and a local bind port.
[service_defaults_mode]: /consul/docs/connect/config-entries/service-defaults#meshgateway
[mesh_gateway_param]: /nomad/docs/job-specification/upstreams#mesh_gateway-parameters
[mesh_gateways]: /consul/docs/connect/gateways/mesh-gateway/service-to-service-traffic-datacenters#mesh-gateways
[consul_expose_path_ref]: /consul/docs/connect/proxies/proxy-config-reference#expose-paths-configuration-reference

View File

@@ -31,6 +31,9 @@ with Consul service mesh.
See the Linux [post-install steps][cni_install] for installing CNI reference plugins.
To use Nomad's [`transparent_proxy`][] feature, you will also need the
[`consul-cni`][] plugin.
## CNI plugins
Spec-compliant plugins should work with Nomad, however, it's possible a plugin
@@ -225,3 +228,5 @@ a unique value for your configuration.
[loopback]: https://github.com/containernetworking/plugins#main-interface-creating
[nomad_install]: /nomad/tutorials/get-started/get-started-install#post-installation-steps
[portmap]: https://www.cni.dev/plugins/current/meta/portmap/
[`transparent_proxy`]: /nomad/docs/job-specification/transparent_proxy
[`consul-cni`]: https://releases.hashicorp.com/consul-cni

View File

@@ -1803,6 +1803,10 @@
"title": "template",
"path": "job-specification/template"
},
{
"title": "transparent_proxy",
"path": "job-specification/transparent_proxy"
},
{
"title": "update",
"path": "job-specification/update"