cmd{containerboot,k8s-operator},util/linuxfw: support ExternalName Services (#11802)

* cmd/containerboot,util/linuxfw: support proxy backends specified by DNS name

Adds support for optionally configuring containerboot to proxy
traffic to backends configured by passing TS_EXPERIMENTAL_DEST_DNS_NAME env var
to containerboot.
Containerboot will periodically (every 10 minutes) attempt to resolve
the DNS name and ensure that all traffic sent to the node's
tailnet IP gets forwarded to the resolved backend IP addresses.

Currently:
- if the firewall mode is iptables, traffic will be load balanced
accross the backend IP addresses using round robin. There are
no health checks for whether the IPs are reachable.
- if the firewall mode is nftables traffic will only be forwarded
to the first IP address in the list. This is to be improved.

* cmd/k8s-operator: support ExternalName Services

 Adds support for exposing endpoints, accessible from within
a cluster to the tailnet via DNS names using ExternalName Services.
This can be done by annotating the ExternalName Service with
tailscale.com/expose: "true" annotation.
The operator will deploy a proxy configured to route tailnet
traffic to the backend IPs that service.spec.externalName
resolves to. The backend IPs must be reachable from the operator's
namespace.

Updates tailscale/tailscale#10606

Signed-off-by: Irbe Krumina <irbe@tailscale.com>
pull/11852/head
Irbe Krumina 2 weeks ago committed by GitHub
parent bf46bff678
commit 3af0f526b8
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194

@ -18,7 +18,11 @@
// previously advertised routes. To accept routes, use TS_EXTRA_ARGS to pass
// in --accept-routes.
// - TS_DEST_IP: proxy all incoming Tailscale traffic to the given
// destination.
// destination defined by an IP address.
// - TS_EXPERIMENTAL_DEST_DNS_NAME: proxy all incoming Tailscale traffic to the given
// destination defined by a DNS name. The DNS name will be periodically resolved and firewall rules updated accordingly.
// This is currently intended to be used by the Kubernetes operator (ExternalName Services).
// This is an experimental env var and will likely change in the future.
// - TS_TAILNET_TARGET_IP: proxy all incoming non-Tailscale traffic to the given
// destination defined by an IP.
// - TS_TAILNET_TARGET_FQDN: proxy all incoming non-Tailscale traffic to the given
@ -82,12 +86,15 @@ import (
"fmt"
"io/fs"
"log"
"math"
"net"
"net/netip"
"os"
"os/exec"
"os/signal"
"path/filepath"
"reflect"
"slices"
"strconv"
"strings"
"sync"
@ -122,7 +129,8 @@ func main() {
Hostname: defaultEnv("TS_HOSTNAME", ""),
Routes: defaultEnvStringPointer("TS_ROUTES"),
ServeConfigPath: defaultEnv("TS_SERVE_CONFIG", ""),
ProxyTo: defaultEnv("TS_DEST_IP", ""),
ProxyTargetIP: defaultEnv("TS_DEST_IP", ""),
ProxyTargetDNSName: defaultEnv("TS_EXPERIMENTAL_DEST_DNS_NAME", ""),
TailnetTargetIP: defaultEnv("TS_TAILNET_TARGET_IP", ""),
TailnetTargetFQDN: defaultEnv("TS_TAILNET_TARGET_FQDN", ""),
DaemonExtraArgs: defaultEnv("TS_TAILSCALED_EXTRA_ARGS", ""),
@ -150,8 +158,8 @@ func main() {
if err := ensureTunFile(cfg.Root); err != nil {
log.Fatalf("Unable to create tuntap device file: %v", err)
}
if cfg.ProxyTo != "" || cfg.Routes != nil || cfg.TailnetTargetIP != "" || cfg.TailnetTargetFQDN != "" {
if err := ensureIPForwarding(cfg.Root, cfg.ProxyTo, cfg.TailnetTargetIP, cfg.TailnetTargetFQDN, cfg.Routes); err != nil {
if cfg.ProxyTargetIP != "" || cfg.ProxyTargetDNSName != "" || cfg.Routes != nil || cfg.TailnetTargetIP != "" || cfg.TailnetTargetFQDN != "" {
if err := ensureIPForwarding(cfg.Root, cfg.ProxyTargetIP, cfg.TailnetTargetIP, cfg.TailnetTargetFQDN, cfg.Routes); err != nil {
log.Printf("Failed to enable IP forwarding: %v", err)
log.Printf("To run tailscale as a proxy or router container, IP forwarding must be enabled.")
if cfg.InKubernetes {
@ -341,7 +349,7 @@ authLoop:
}
var (
wantProxy = cfg.ProxyTo != "" || cfg.TailnetTargetIP != "" || cfg.TailnetTargetFQDN != "" || cfg.AllowProxyingClusterTrafficViaIngress
wantProxy = cfg.ProxyTargetIP != "" || cfg.ProxyTargetDNSName != "" || cfg.TailnetTargetIP != "" || cfg.TailnetTargetFQDN != "" || cfg.AllowProxyingClusterTrafficViaIngress
wantDeviceInfo = cfg.InKubernetes && cfg.KubeSecret != "" && cfg.KubernetesCanPatch
startupTasksDone = false
currentIPs deephash.Sum // tailscale IPs assigned to device
@ -349,6 +357,9 @@ authLoop:
currentEgressIPs deephash.Sum
addrs []netip.Prefix
backendAddrs []net.IP
certDomain = new(atomic.Pointer[string])
certDomainChanged = make(chan bool, 1)
)
@ -362,6 +373,44 @@ authLoop:
log.Fatalf("error creating new netfilter runner: %v", err)
}
}
// Setup for proxies that are configured to proxy to a target specified
// by a DNS name (TS_EXPERIMENTAL_DEST_DNS_NAME).
const defaultCheckPeriod = time.Minute * 10 // how often to check what IPs the DNS name resolves to
var (
tc = make(chan string, 1)
failedResolveAttempts int
t *time.Timer = time.AfterFunc(defaultCheckPeriod, func() {
if cfg.ProxyTargetDNSName != "" {
tc <- "recheck"
}
})
)
defer t.Stop()
// resetTimer resets timer for when to next attempt to resolve the DNS
// name for the proxy configured with TS_EXPERIMENTAL_DEST_DNS_NAME. The
// timer gets reset to 10 minutes from now unless the last resolution
// attempt failed. If one or more consecutive previous resolution
// attempts failed, the next resolution attempt will happen after the smallest
// of (10 minutes, 2 ^ number-of-consecutive-failed-resolution-attempts
// seconds) i.e 2s, 4s, 8s ... 10 minutes.
resetTimer := func(lastResolveFailed bool) {
if !lastResolveFailed {
log.Printf("reconfigureTimer: next DNS resolution attempt in %s", defaultCheckPeriod)
t.Reset(defaultCheckPeriod)
failedResolveAttempts = 0
return
}
minDelay := 2 // 2 seconds
nextTick := time.Second * time.Duration(math.Pow(float64(minDelay), float64(failedResolveAttempts)))
if nextTick > defaultCheckPeriod {
nextTick = defaultCheckPeriod // cap at 10 minutes
}
log.Printf("reconfigureTimer: last DNS resolution attempt failed, next DNS resolution attempt in %v", nextTick)
t.Reset(nextTick)
failedResolveAttempts++
}
notifyChan := make(chan ipn.Notify)
errChan := make(chan error)
go func() {
@ -399,7 +448,7 @@ runLoop:
log.Fatalf("tailscaled left running state (now in state %q), exiting", *n.State)
}
if n.NetMap != nil {
addrs := n.NetMap.SelfNode.Addresses().AsSlice()
addrs = n.NetMap.SelfNode.Addresses().AsSlice()
newCurrentIPs := deephash.Hash(&addrs)
ipsHaveChanged := newCurrentIPs != currentIPs
@ -425,7 +474,7 @@ runLoop:
egressAddrs = node.Addresses().AsSlice()
newCurentEgressIPs = deephash.Hash(&egressAddrs)
egressIPsHaveChanged = newCurentEgressIPs != currentEgressIPs
if egressIPsHaveChanged && len(egressAddrs) > 0 {
if egressIPsHaveChanged && len(egressAddrs) != 0 {
for _, egressAddr := range egressAddrs {
ea := egressAddr.Addr()
// TODO (irbekrm): make it work for IPv6 too.
@ -441,13 +490,32 @@ runLoop:
}
currentEgressIPs = newCurentEgressIPs
}
if cfg.ProxyTo != "" && len(addrs) > 0 && ipsHaveChanged {
if cfg.ProxyTargetIP != "" && len(addrs) != 0 && ipsHaveChanged {
log.Printf("Installing proxy rules")
if err := installIngressForwardingRule(ctx, cfg.ProxyTo, addrs, nfr); err != nil {
if err := installIngressForwardingRule(ctx, cfg.ProxyTargetIP, addrs, nfr); err != nil {
log.Fatalf("installing ingress proxy rules: %v", err)
}
}
if cfg.ServeConfigPath != "" && len(n.NetMap.DNS.CertDomains) > 0 {
if cfg.ProxyTargetDNSName != "" && len(addrs) != 0 && ipsHaveChanged {
newBackendAddrs, err := resolveDNS(ctx, cfg.ProxyTargetDNSName)
if err != nil {
log.Printf("[unexpected] error resolving DNS name %s: %v", cfg.ProxyTargetDNSName, err)
resetTimer(true)
continue
}
backendsHaveChanged := !(slices.EqualFunc(backendAddrs, newBackendAddrs, func(ip1 net.IP, ip2 net.IP) bool {
return slices.ContainsFunc(newBackendAddrs, func(ip net.IP) bool { return ip.Equal(ip1) })
}))
if backendsHaveChanged {
log.Printf("installing ingress proxy rules for backends %v", newBackendAddrs)
if err := installIngressForwardingRuleForDNSTarget(ctx, newBackendAddrs, addrs, nfr); err != nil {
log.Fatalf("error installing ingress proxy rules: %v", err)
}
}
resetTimer(false)
backendAddrs = newBackendAddrs
}
if cfg.ServeConfigPath != "" && len(n.NetMap.DNS.CertDomains) != 0 {
cd := n.NetMap.DNS.CertDomains[0]
prev := certDomain.Swap(ptr.To(cd))
if prev == nil || *prev != cd {
@ -457,7 +525,7 @@ runLoop:
}
}
}
if cfg.TailnetTargetIP != "" && ipsHaveChanged && len(addrs) > 0 {
if cfg.TailnetTargetIP != "" && ipsHaveChanged && len(addrs) != 0 {
log.Printf("Installing forwarding rules for destination %v", cfg.TailnetTargetIP)
if err := installEgressForwardingRule(ctx, cfg.TailnetTargetIP, addrs, nfr); err != nil {
log.Fatalf("installing egress proxy rules: %v", err)
@ -469,7 +537,7 @@ runLoop:
// enabled, set up proxy rule each time the
// tailnet IPs of this node change (including
// the first time they become available).
if cfg.AllowProxyingClusterTrafficViaIngress && cfg.ServeConfigPath != "" && ipsHaveChanged && len(addrs) > 0 {
if cfg.AllowProxyingClusterTrafficViaIngress && cfg.ServeConfigPath != "" && ipsHaveChanged && len(addrs) != 0 {
log.Printf("installing rules to forward traffic for %s to node's tailnet IP", cfg.PodIP)
if err := installTSForwardingRuleForDestination(ctx, cfg.PodIP, addrs, nfr); err != nil {
log.Fatalf("installing rules to forward traffic to node's tailnet IP: %v", err)
@ -511,12 +579,29 @@ runLoop:
os.Exit(0)
}
}
}
wg.Add(1)
go reaper()
}
}
case <-tc:
newBackendAddrs, err := resolveDNS(ctx, cfg.ProxyTargetDNSName)
if err != nil {
log.Printf("[unexpected] error resolving DNS name %s: %v", cfg.ProxyTargetDNSName, err)
resetTimer(true)
continue
}
backendsHaveChanged := !(slices.EqualFunc(backendAddrs, newBackendAddrs, func(ip1 net.IP, ip2 net.IP) bool {
return slices.ContainsFunc(newBackendAddrs, func(ip net.IP) bool { return ip.Equal(ip1) })
}))
if backendsHaveChanged && len(addrs) != 0 {
log.Printf("Backend address change detected, installing proxy rules for backends %v", newBackendAddrs)
if err := installIngressForwardingRuleForDNSTarget(ctx, newBackendAddrs, addrs, nfr); err != nil {
log.Fatalf("installing ingress proxy rules for DNS target %s: %v", cfg.ProxyTargetDNSName, err)
}
}
backendAddrs = newBackendAddrs
resetTimer(false)
}
}
wg.Wait()
@ -757,12 +842,12 @@ func ensureTunFile(root string) error {
}
// ensureIPForwarding enables IPv4/IPv6 forwarding for the container.
func ensureIPForwarding(root, clusterProxyTarget, tailnetTargetiP, tailnetTargetFQDN string, routes *string) error {
func ensureIPForwarding(root, clusterProxyTargetIP, tailnetTargetIP, tailnetTargetFQDN string, routes *string) error {
var (
v4Forwarding, v6Forwarding bool
)
if clusterProxyTarget != "" {
proxyIP, err := netip.ParseAddr(clusterProxyTarget)
if clusterProxyTargetIP != "" {
proxyIP, err := netip.ParseAddr(clusterProxyTargetIP)
if err != nil {
return fmt.Errorf("invalid cluster destination IP: %v", err)
}
@ -772,8 +857,8 @@ func ensureIPForwarding(root, clusterProxyTarget, tailnetTargetiP, tailnetTarget
v6Forwarding = true
}
}
if tailnetTargetiP != "" {
proxyIP, err := netip.ParseAddr(tailnetTargetiP)
if tailnetTargetIP != "" {
proxyIP, err := netip.ParseAddr(tailnetTargetIP)
if err != nil {
return fmt.Errorf("invalid tailnet destination IP: %v", err)
}
@ -801,7 +886,10 @@ func ensureIPForwarding(root, clusterProxyTarget, tailnetTargetiP, tailnetTarget
}
}
}
return enableIPForwarding(v4Forwarding, v6Forwarding, root)
}
func enableIPForwarding(v4Forwarding, v6Forwarding bool, root string) error {
var paths []string
if v4Forwarding {
paths = append(paths, filepath.Join(root, "proc/sys/net/ipv4/ip_forward"))
@ -918,15 +1006,89 @@ func installIngressForwardingRule(ctx context.Context, dstStr string, tsIPs []ne
return nil
}
func installIngressForwardingRuleForDNSTarget(ctx context.Context, backendAddrs []net.IP, tsIPs []netip.Prefix, nfr linuxfw.NetfilterRunner) error {
var (
tsv4 netip.Addr
tsv6 netip.Addr
v4Backends []netip.Addr
v6Backends []netip.Addr
)
for _, pfx := range tsIPs {
if pfx.IsSingleIP() && pfx.Addr().Is4() {
tsv4 = pfx.Addr()
continue
}
if pfx.IsSingleIP() && pfx.Addr().Is6() {
tsv6 = pfx.Addr()
continue
}
}
// TODO: log if more than one backend address is found and firewall is
// in nftables mode that only the first IP will be used.
for _, ip := range backendAddrs {
if ip.To4() != nil {
v4Backends = append(v4Backends, netip.AddrFrom4([4]byte(ip.To4())))
}
if ip.To16() != nil {
v6Backends = append(v6Backends, netip.AddrFrom16([16]byte(ip.To16())))
}
}
// Enable IP forwarding here as opposed to at the start of containerboot
// as the IPv4/IPv6 requirements might have changed.
// For Kubernetes operator proxies, forwarding for both IPv4 and IPv6 is
// enabled by an init container, so in practice enabling forwarding here
// is only needed if this proxy has been configured by manually setting
// TS_EXPERIMENTAL_DEST_DNS_NAME env var for a containerboot instance.
if err := enableIPForwarding(len(v4Backends) != 0, len(v6Backends) != 0, ""); err != nil {
log.Printf("[unexpected] failed to ensure IP forwarding: %v", err)
}
updateFirewall := func(dst netip.Addr, backendTargets []netip.Addr) error {
if err := nfr.DNATWithLoadBalancer(dst, backendTargets); err != nil {
return fmt.Errorf("installing DNAT rules for ingress backends %+#v: %w", backendTargets, err)
}
// The backend might advertize MSS higher than that of the
// tailscale interfaces. Clamp MSS of packets going out via
// tailscale0 interface to its MTU to prevent broken connections
// in environments where path MTU discovery is not working.
if err := nfr.ClampMSSToPMTU("tailscale0", dst); err != nil {
return fmt.Errorf("adding rule to clamp traffic via tailscale0: %v", err)
}
return nil
}
if len(v4Backends) != 0 {
if !tsv4.IsValid() {
log.Printf("backend targets %v contain at least one IPv4 address, but this node's Tailscale IPs do not contain a valid IPv4 address: %v", backendAddrs, tsIPs)
} else if err := updateFirewall(tsv4, v4Backends); err != nil {
return fmt.Errorf("Installing IPv4 firewall rules: %w", err)
}
}
if len(v6Backends) != 0 && !tsv6.IsValid() {
if !tsv6.IsValid() {
log.Printf("backend targets %v contain at least one IPv6 address, but this node's Tailscale IPs do not contain a valid IPv6 address: %v", backendAddrs, tsIPs)
} else if !nfr.HasIPV6NAT() {
log.Printf("backend targets %v contain at least one IPv6 address, but the chosen firewall mode does not support IPv6 NAT", backendAddrs)
} else if err := updateFirewall(tsv6, v6Backends); err != nil {
return fmt.Errorf("Installing IPv6 firewall rules: %w", err)
}
}
return nil
}
// settings is all the configuration for containerboot.
type settings struct {
AuthKey string
Hostname string
Routes *string
// ProxyTo is the destination IP to which all incoming
// ProxyTargetIP is the destination IP to which all incoming
// Tailscale traffic should be proxied. If empty, no proxying
// is done. This is typically a locally reachable IP.
ProxyTo string
ProxyTargetIP string
// ProxyTargetDNSName is a DNS name to whose backing IP addresses all
// incoming Tailscale traffic should be proxied.
ProxyTargetDNSName string
// TailnetTargetIP is the destination IP to which all incoming
// non-Tailscale traffic should be proxied. This is typically a
// Tailscale IP.
@ -966,9 +1128,15 @@ func (s *settings) validate() error {
return fmt.Errorf("error validating tailscaled configfile contents: %w", err)
}
}
if s.ProxyTo != "" && s.UserspaceMode {
if s.ProxyTargetIP != "" && s.UserspaceMode {
return errors.New("TS_DEST_IP is not supported with TS_USERSPACE")
}
if s.ProxyTargetDNSName != "" && s.UserspaceMode {
return errors.New("TS_EXPERIMENTAL_DEST_DNS_NAME is not supported with TS_USERSPACE")
}
if s.ProxyTargetDNSName != "" && s.ProxyTargetIP != "" {
return errors.New("TS_EXPERIMENTAL_DEST_DNS_NAME and TS_DEST_IP cannot both be set")
}
if s.TailnetTargetIP != "" && s.UserspaceMode {
return errors.New("TS_TAILNET_TARGET_IP is not supported with TS_USERSPACE")
}
@ -993,6 +1161,28 @@ func (s *settings) validate() error {
return nil
}
func resolveDNS(ctx context.Context, name string) ([]net.IP, error) {
// TODO (irbekrm): look at using recursive.Resolver instead to resolve
// the DNS names as well as retrieve TTLs. It looks though that this
// seems to return very short TTLs (shorter than on the actual records).
ip4s, err := net.DefaultResolver.LookupIP(ctx, "ip4", name)
if err != nil {
if e, ok := err.(*net.DNSError); !(ok && e.IsNotFound) {
return nil, fmt.Errorf("error looking up IPv4 addresses: %v", err)
}
}
ip6s, err := net.DefaultResolver.LookupIP(ctx, "ip6", name)
if err != nil {
if e, ok := err.(*net.DNSError); !(ok && e.IsNotFound) {
return nil, fmt.Errorf("error looking up IPv6 addresses: %v", err)
}
}
if len(ip4s) == 0 && len(ip6s) == 0 {
return nil, fmt.Errorf("no IPv4 or IPv6 addresses found for host: %s", name)
}
return append(ip4s, ip6s...), nil
}
// defaultEnv returns the value of the given envvar name, or defVal if
// unset.
func defaultEnv(name, defVal string) string {

@ -1424,6 +1424,73 @@ func Test_clusterDomainFromResolverConf(t *testing.T) {
}
}
func Test_externalNameService(t *testing.T) {
fc := fake.NewFakeClient()
ft := &fakeTSClient{}
zl, err := zap.NewDevelopment()
if err != nil {
t.Fatal(err)
}
// 1. A External name Service that should be exposed via Tailscale gets
// created.
sr := &ServiceReconciler{
Client: fc,
ssr: &tailscaleSTSReconciler{
Client: fc,
tsClient: ft,
defaultTags: []string{"tag:k8s"},
operatorNamespace: "operator-ns",
proxyImage: "tailscale/tailscale",
},
logger: zl.Sugar(),
}
// 1. Create an ExternalName Service that we should manage, and check that the initial round
// of objects looks right.
mustCreate(t, fc, &corev1.Service{
ObjectMeta: metav1.ObjectMeta{
Name: "test",
Namespace: "default",
// The apiserver is supposed to set the UID, but the fake client
// doesn't. So, set it explicitly because other code later depends
// on it being set.
UID: types.UID("1234-UID"),
Annotations: map[string]string{
AnnotationExpose: "true",
},
},
Spec: corev1.ServiceSpec{
Type: corev1.ServiceTypeExternalName,
ExternalName: "foo.com",
},
})
expectReconciled(t, sr, "default", "test")
fullName, shortName := findGenName(t, fc, "default", "test", "svc")
opts := configOpts{
stsName: shortName,
secretName: fullName,
namespace: "default",
parentType: "svc",
hostname: "default-test",
clusterTargetDNS: "foo.com",
}
expectEqual(t, fc, expectedSecret(t, opts), nil)
expectEqual(t, fc, expectedHeadlessService(shortName, "svc"), nil)
expectEqual(t, fc, expectedSTS(t, fc, opts), removeHashAnnotation)
// 2. Change the ExternalName and verify that changes get propagated.
mustUpdate(t, sr, "default", "test", func(s *corev1.Service) {
s.Spec.ExternalName = "bar.com"
})
expectReconciled(t, sr, "default", "test")
opts.clusterTargetDNS = "bar.com"
expectEqual(t, fc, expectedSTS(t, fc, opts), removeHashAnnotation)
}
func toFQDN(t *testing.T, s string) dnsname.FQDN {
t.Helper()
fqdn, err := dnsname.ToFQDN(s)

@ -87,6 +87,7 @@ const (
// ensure that it does not get removed when a ProxyClass configuration
// is applied.
podAnnotationLastSetClusterIP = "tailscale.com/operator-last-set-cluster-ip"
podAnnotationLastSetClusterDNSName = "tailscale.com/operator-last-set-cluster-dns-name"
podAnnotationLastSetTailnetTargetIP = "tailscale.com/operator-last-set-ts-tailnet-target-ip"
podAnnotationLastSetTailnetTargetFQDN = "tailscale.com/operator-last-set-ts-tailnet-target-fqdn"
// podAnnotationLastSetConfigFileHash is sha256 hash of the current tailscaled configuration contents.
@ -109,8 +110,9 @@ type tailscaleSTSConfig struct {
ParentResourceUID string
ChildResourceLabels map[string]string
ServeConfig *ipn.ServeConfig // if serve config is set, this is a proxy for Ingress
ClusterTargetIP string // ingress target
ServeConfig *ipn.ServeConfig // if serve config is set, this is a proxy for Ingress
ClusterTargetIP string // ingress target IP
ClusterTargetDNSName string // ingress target DNS name
// If set to true, operator should configure containerboot to forward
// cluster traffic via the proxy set up for Kubernetes Ingress.
ForwardClusterTrafficViaL7IngressProxy bool
@ -536,6 +538,12 @@ func (a *tailscaleSTSReconciler) reconcileSTS(ctx context.Context, logger *zap.S
Value: sts.ClusterTargetIP,
})
mak.Set(&ss.Spec.Template.Annotations, podAnnotationLastSetClusterIP, sts.ClusterTargetIP)
} else if sts.ClusterTargetDNSName != "" {
container.Env = append(container.Env, corev1.EnvVar{
Name: "TS_EXPERIMENTAL_DEST_DNS_NAME",
Value: sts.ClusterTargetDNSName,
})
mak.Set(&ss.Spec.Template.Annotations, podAnnotationLastSetClusterDNSName, sts.ClusterTargetDNSName)
} else if sts.TailnetTargetIP != "" {
container.Env = append(container.Env, corev1.EnvVar{
Name: "TS_TAILNET_TARGET_IP",

@ -208,10 +208,14 @@ func (a *ServiceReconciler) maybeProvision(ctx context.Context, logger *zap.Suga
}
a.mu.Lock()
if a.shouldExpose(svc) {
if a.shouldExposeClusterIP(svc) {
sts.ClusterTargetIP = svc.Spec.ClusterIP
a.managedIngressProxies.Add(svc.UID)
gaugeIngressProxies.Set(int64(a.managedIngressProxies.Len()))
} else if a.shouldExposeDNSName(svc) {
sts.ClusterTargetDNSName = svc.Spec.ExternalName
a.managedIngressProxies.Add(svc.UID)
gaugeIngressProxies.Set(int64(a.managedIngressProxies.Len()))
} else if ip := a.tailnetTargetAnnotation(svc); ip != "" {
sts.TailnetTargetIP = ip
a.managedEgressProxies.Add(svc.UID)
@ -303,15 +307,22 @@ func validateService(svc *corev1.Service) []string {
}
func (a *ServiceReconciler) shouldExpose(svc *corev1.Service) bool {
return a.shouldExposeClusterIP(svc) || a.shouldExposeDNSName(svc)
}
func (a *ServiceReconciler) shouldExposeClusterIP(svc *corev1.Service) bool {
// Headless services can't be exposed, since there is no ClusterIP to
// forward to.
if svc.Spec.ClusterIP == "" || svc.Spec.ClusterIP == "None" {
return false
}
return a.hasLoadBalancerClass(svc) || a.hasExposeAnnotation(svc)
}
func (a *ServiceReconciler) shouldExposeDNSName(svc *corev1.Service) bool {
return a.hasExposeAnnotation(svc) && svc.Spec.Type == corev1.ServiceTypeExternalName && svc.Spec.ExternalName != ""
}
func (a *ServiceReconciler) hasLoadBalancerClass(svc *corev1.Service) bool {
return svc != nil &&
svc.Spec.Type == corev1.ServiceTypeLoadBalancer &&

@ -43,6 +43,7 @@ type configOpts struct {
tailnetTargetIP string
tailnetTargetFQDN string
clusterTargetIP string
clusterTargetDNS string
subnetRoutes string
isExitNode bool
confFileHash string
@ -126,6 +127,12 @@ func expectedSTS(t *testing.T, cl client.Client, opts configOpts) *appsv1.Statef
Value: opts.clusterTargetIP,
})
annots["tailscale.com/operator-last-set-cluster-ip"] = opts.clusterTargetIP
} else if opts.clusterTargetDNS != "" {
tsContainer.Env = append(tsContainer.Env, corev1.EnvVar{
Name: "TS_EXPERIMENTAL_DEST_DNS_NAME",
Value: opts.clusterTargetDNS,
})
annots["tailscale.com/operator-last-set-cluster-dns-name"] = opts.clusterTargetDNS
}
if opts.serveConfig != nil {
tsContainer.Env = append(tsContainer.Env, corev1.EnvVar{

@ -373,6 +373,27 @@ func (i *iptablesRunner) DNATNonTailscaleTraffic(tun string, dst netip.Addr) err
return table.Insert("nat", "PREROUTING", 1, "!", "-i", tun, "-j", "DNAT", "--to-destination", dst.String())
}
// DNATWithLoadBalancer adds iptables rules to forward all traffic received for
// originDst to the backend dsts. Traffic will be load balanced using round robin.
func (i *iptablesRunner) DNATWithLoadBalancer(origDst netip.Addr, dsts []netip.Addr) error {
table := i.getIPTByAddr(dsts[0])
if err := table.ClearChain("nat", "PREROUTING"); err != nil && !isErrChainNotExist(err) {
// If clearing the PREROUTING chain fails, fail the whole operation. This
// rule is currently only used in Kubernetes containers where a
// failed container gets restarted which should hopefully fix things.
return fmt.Errorf("error clearing nat PREROUTING chain: %w", err)
}
// If dsts contain more than one address, for n := n in range(len(dsts)..2) route packets for every nth connection to dsts[n].
for i := len(dsts); i >= 2; i-- {
dst := dsts[i-1] // the order in which rules for addrs are installed does not matter
if err := table.Append("nat", "PREROUTING", "--destination", origDst.String(), "-m", "statistic", "--mode", "nth", "--every", fmt.Sprint(i), "--packet", "0", "-j", "DNAT", "--to-destination", dst.String()); err != nil {
return fmt.Errorf("error adding DNAT rule for %s: %w", dst.String(), err)
}
}
// If the packet falls through to this rule, we route to the first destination in the list unconditionally.
return table.Append("nat", "PREROUTING", "--destination", origDst.String(), "-j", "DNAT", "--to-destination", dsts[0].String())
}
func (i *iptablesRunner) ClampMSSToPMTU(tun string, addr netip.Addr) error {
table := i.getIPTByAddr(addr)
return table.Append("mangle", "FORWARD", "-o", tun, "-p", "tcp", "--tcp-flags", "SYN,RST", "SYN", "-j", "TCPMSS", "--clamp-mss-to-pmtu")

@ -114,7 +114,6 @@ func (n *nftablesRunner) AddDNATRule(origDst netip.Addr, dst netip.Addr) error {
dadderLen = 16
fam = unix.NFPROTO_IPV6
}
dnatRule := &nftables.Rule{
Table: nat,
Chain: preroutingCh,
@ -145,6 +144,15 @@ func (n *nftablesRunner) AddDNATRule(origDst netip.Addr, dst netip.Addr) error {
return n.conn.Flush()
}
// DNATWithLoadBalancer currently just forwards all traffic destined for origDst
// to the first IP address from the backend targets.
// TODO (irbekrm): instead of doing this load balance traffic evenly to all
// backend destinations.
// https://github.com/tailscale/tailscale/commit/d37f2f508509c6c35ad724fd75a27685b90b575b#diff-a3bcbcd1ca198799f4f768dc56fea913e1945a6b3ec9dbec89325a84a19a85e7R148-R232
func (n *nftablesRunner) DNATWithLoadBalancer(origDst netip.Addr, dsts []netip.Addr) error {
return n.AddDNATRule(origDst, dsts[0])
}
func (n *nftablesRunner) DNATNonTailscaleTraffic(tunname string, dst netip.Addr) error {
nat, preroutingCh, err := n.ensurePreroutingChain(dst)
if err != nil {
@ -524,6 +532,14 @@ type NetfilterRunner interface {
// to the provided destination, as used in the Kubernetes ingress proxies.
AddDNATRule(origDst, dst netip.Addr) error
// DNATWithLoadBalancer adds a rule to the nat/PREROUTING chain to DNAT
// traffic destined for the given original destination to the given new
// destination(s) using round robin to load balance if more than one
// destination is provided. This is used to forward all traffic destined
// for the Tailscale interface to the provided destination(s), as used
// in the Kubernetes ingress proxies.
DNATWithLoadBalancer(origDst netip.Addr, dsts []netip.Addr) error
// AddSNATRuleForDst adds a rule to the nat/POSTROUTING chain to SNAT
// traffic destined for dst to src.
// This is used to forward traffic destined for the local machine over
@ -533,7 +549,7 @@ type NetfilterRunner interface {
// DNATNonTailscaleTraffic adds a rule to the nat/PREROUTING chain to DNAT
// all traffic inbound from any interface except exemptInterface to dst.
// This is used to forward traffic destined for the local machine over
// the Tailscale interface, as used in the Kubernetes egress proxies.//
// the Tailscale interface, as used in the Kubernetes egress proxies.
DNATNonTailscaleTraffic(exemptInterface string, dst netip.Addr) error
// ClampMSSToPMTU adds a rule to the mangle/FORWARD chain to clamp MSS for

@ -470,6 +470,10 @@ func (n *fakeIPTablesRunner) AddDNATRule(origDst, dst netip.Addr) error {
return errors.New("not implemented")
}
func (n *fakeIPTablesRunner) DNATWithLoadBalancer(netip.Addr, []netip.Addr) error {
return errors.New("not implemented")
}
func (n *fakeIPTablesRunner) AddSNATRuleForDst(src, dst netip.Addr) error {
return errors.New("not implemented")
}

Loading…
Cancel
Save