mirror of
https://github.com/kemko/nomad.git
synced 2026-01-01 16:05:42 +03:00
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:
@@ -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
|
||||
}
|
||||
|
||||
@@ -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"
|
||||
}
|
||||
`
|
||||
|
||||
@@ -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")
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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
|
||||
}
|
||||
|
||||
@@ -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)
|
||||
}
|
||||
})
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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,
|
||||
|
||||
Reference in New Issue
Block a user