tproxy: networking hook changes (#20183)

When `transparent_proxy` block is present and the network mode is `bridge`, use
a different CNI configuration that includes the `consul-cni` plugin. Before
invoking the CNI plugins, create a Consul SDK `iptables.Config` struct for the
allocation. This includes:

* Use all the `transparent_proxy` block fields
* The reserved ports are added to the inbound exclusion list so the alloc is
  reachable from outside the mesh
* The `expose` blocks and `check` blocks with `expose=true` are added to the
  inbound exclusion list so health checks work.

The `iptables.Config` is then passed as a CNI argument to the `consul-cni`
plugin.

Ref: https://github.com/hashicorp/nomad/issues/10628
This commit is contained in:
Tim Gross
2024-03-27 14:24:23 -04:00
parent e8d203e7ce
commit 76009d89af
7 changed files with 592 additions and 32 deletions

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",
}
portMapping, portLabels := getPortMapping(alloc, c.ignorePortMappingHostIP)
tproxyArgs, err := c.setupTransparentProxyArgs(alloc, spec, portMapping, portLabels)
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(portMapping),
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, portMapping []cni.PortMapping, portLabels map[string]int) (*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 idx, ok := portLabels[envoyPortLabel]; ok {
proxyInboundPort = int(portMapping[idx].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 idx, ok := portLabels[portLabel]; ok {
exposePortSet.Insert(
strconv.FormatInt(int64(portMapping[idx].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 idx, ok := portLabels[path.ListenerPort]; ok {
exposePortSet.Insert(
strconv.FormatInt(int64(portMapping[idx].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)); err != nil {
// create a real handle to iptables
ipt, iptErr := iptables.New()
if iptErr != nil {
@@ -347,8 +577,9 @@ func (c *cniNetworkConfigurator) ensureCNIInitialized() error {
// getPortMapping builds a list of portMapping structs that are used as the
// portmapping capability arguments for the portmap CNI plugin
func getPortMapping(alloc *structs.Allocation, ignoreHostIP bool) []cni.PortMapping {
func getPortMapping(alloc *structs.Allocation, ignoreHostIP bool) ([]cni.PortMapping, map[string]int) {
var 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 {
@@ -362,6 +593,7 @@ func getPortMapping(alloc *structs.Allocation, ignoreHostIP bool) []cni.PortMapp
ContainerPort: int32(port.To),
Protocol: proto,
})
labels[port.Label] = len(ports) - 1
}
}
}
@@ -380,8 +612,9 @@ func getPortMapping(alloc *structs.Allocation, ignoreHostIP bool) []cni.PortMapp
portMapping.HostIP = port.HostIP
}
ports = append(ports, portMapping)
labels[port.Label] = len(ports) - 1
}
}
}
return ports
return ports, labels
}

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,250 @@ 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()
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{},
}
portMapping := []cni.PortMapping{
{
HostPort: 9002,
ContainerPort: 9002,
Protocol: "tcp",
HostIP: "",
},
{
HostPort: 9002,
ContainerPort: 9002,
Protocol: "udp",
HostIP: "",
},
{
HostPort: 9001,
ContainerPort: 9000,
Protocol: "tcp",
HostIP: "",
},
{
HostPort: 9001,
ContainerPort: 9000,
Protocol: "udp",
HostIP: "",
},
{
HostPort: 25018,
ContainerPort: 25018,
Protocol: "tcp",
HostIP: "",
},
{
HostPort: 25018,
ContainerPort: 20000,
Protocol: "udp",
HostIP: "",
},
}
portLabels := map[string]int{
"connect-proxy-testconnect": 5,
"http": 1,
"health": 3,
}
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, portMapping, portLabels)
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

@@ -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,