cmd/k8s-operator,cmd/containerboot,ipn,k8s-operator: turn off stateful filter for egress proxies. (#12075)

Turn off stateful filtering for egress proxies to allow cluster
traffic to be forwarded to tailnet.

Allow configuring stateful filter via tailscaled config file.

Deprecate EXPERIMENTAL_TS_CONFIGFILE_PATH env var and introduce a new
TS_EXPERIMENTAL_VERSIONED_CONFIG env var that can be used to provide
containerboot a directory that should contain one or more
tailscaled config files named cap-<tailscaled-cap-version>.hujson.
Containerboot will pick the one with the newest capability version
that is not newer than its current capability version.

Proxies with this change will not work with older Tailscale
Kubernetes operator versions - users must ensure that
the deployed operator is at the same version or newer (up to
4 version skew) than the proxies.

Updates tailscale/tailscale#12061

Signed-off-by: Irbe Krumina <irbe@tailscale.com>
Co-authored-by: Maisem Ali <maisem@tailscale.com>
pull/12095/head
Irbe Krumina 3 weeks ago committed by GitHub
parent e070af7414
commit d86d1e7601
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194

@ -52,8 +52,10 @@
// ${TS_CERT_DOMAIN}, it will be replaced with the value of the available FQDN.
// It cannot be used in conjunction with TS_DEST_IP. The file is watched for changes,
// and will be re-applied when it changes.
// - EXPERIMENTAL_TS_CONFIGFILE_PATH: if specified, a path to tailscaled
// config. If this is set, TS_HOSTNAME, TS_EXTRA_ARGS, TS_AUTHKEY,
// - TS_EXPERIMENTAL_VERSIONED_CONFIG_DIR: if specified, a path to a
// directory that containers tailscaled config in file. The config file needs to be
// named cap-<current-tailscaled-cap>.hujson. If this is set, TS_HOSTNAME,
// TS_EXTRA_ARGS, TS_AUTHKEY,
// TS_ROUTES, TS_ACCEPT_DNS env vars must not be set. If this is set,
// containerboot only runs `tailscaled --config <path-to-this-configfile>`
// and not `tailscale up` or `tailscale set`.
@ -92,6 +94,7 @@ import (
"os"
"os/exec"
"os/signal"
"path"
"path/filepath"
"reflect"
"slices"
@ -107,6 +110,7 @@ import (
"tailscale.com/client/tailscale"
"tailscale.com/ipn"
"tailscale.com/ipn/conffile"
kubeutils "tailscale.com/k8s-operator"
"tailscale.com/tailcfg"
"tailscale.com/types/logger"
"tailscale.com/types/ptr"
@ -145,7 +149,7 @@ func main() {
Socket: defaultEnv("TS_SOCKET", "/tmp/tailscaled.sock"),
AuthOnce: defaultBool("TS_AUTH_ONCE", false),
Root: defaultEnv("TS_TEST_ONLY_ROOT", "/"),
TailscaledConfigFilePath: defaultEnv("EXPERIMENTAL_TS_CONFIGFILE_PATH", ""),
TailscaledConfigFilePath: tailscaledConfigFilePath(),
AllowProxyingClusterTrafficViaIngress: defaultBool("EXPERIMENTAL_ALLOW_PROXYING_CLUSTER_TRAFFIC_VIA_INGRESS", false),
PodIP: defaultEnv("POD_IP", ""),
}
@ -1097,6 +1101,13 @@ type settings struct {
func (s *settings) validate() error {
if s.TailscaledConfigFilePath != "" {
dir, file := path.Split(s.TailscaledConfigFilePath)
if _, err := os.Stat(dir); err != nil {
return fmt.Errorf("error validating whether directory with tailscaled config file %s exists: %w", dir, err)
}
if _, err := os.Stat(s.TailscaledConfigFilePath); err != nil {
return fmt.Errorf("error validating whether tailscaled config directory %q contains tailscaled config for current capability version %q: %w. If this is a Tailscale Kubernetes operator proxy, please ensure that the version of the operator is not older than the version of the proxy", dir, file, err)
}
if _, err := conffile.Load(s.TailscaledConfigFilePath); err != nil {
return fmt.Errorf("error validating tailscaled configfile contents: %w", err)
}
@ -1120,7 +1131,7 @@ func (s *settings) validate() error {
return errors.New("Both TS_TAILNET_TARGET_IP and TS_TAILNET_FQDN cannot be set")
}
if s.TailscaledConfigFilePath != "" && (s.AcceptDNS != nil || s.AuthKey != "" || s.Routes != nil || s.ExtraArgs != "" || s.Hostname != "") {
return errors.New("EXPERIMENTAL_TS_CONFIGFILE_PATH cannot be set in combination with TS_HOSTNAME, TS_EXTRA_ARGS, TS_AUTHKEY, TS_ROUTES, TS_ACCEPT_DNS.")
return errors.New("TS_EXPERIMENTAL_VERSIONED_CONFIG_DIR cannot be set in combination with TS_HOSTNAME, TS_EXTRA_ARGS, TS_AUTHKEY, TS_ROUTES, TS_ACCEPT_DNS.")
}
if s.AllowProxyingClusterTrafficViaIngress && s.UserspaceMode {
return errors.New("EXPERIMENTAL_ALLOW_PROXYING_CLUSTER_TRAFFIC_VIA_INGRESS is not supported in userspace mode")
@ -1252,3 +1263,42 @@ func isTwoStepConfigAlwaysAuth(cfg *settings) bool {
func isOneStepConfig(cfg *settings) bool {
return cfg.TailscaledConfigFilePath != ""
}
// tailscaledConfigFilePath returns the path to the tailscaled config file that
// should be used for the current capability version. It is determined by the
// TS_EXPERIMENTAL_VERSIONED_CONFIG_DIR environment variable and looks for a
// file named cap-<capability_version>.hujson in the directory. It searches for
// the highest capability version that is less than or equal to the current
// capability version.
func tailscaledConfigFilePath() string {
dir := os.Getenv("TS_EXPERIMENTAL_VERSIONED_CONFIG_DIR")
if dir == "" {
return ""
}
fe, err := os.ReadDir(dir)
if err != nil {
log.Fatalf("error reading tailscaled config directory %q: %v", dir, err)
}
maxCompatVer := tailcfg.CapabilityVersion(-1)
for _, e := range fe {
// We don't check if type if file as in most cases this will
// come from a mounted kube Secret, where the directory contents
// will be various symlinks.
if e.Type().IsDir() {
continue
}
cv, err := kubeutils.CapVerFromFileName(e.Name())
if err != nil {
log.Printf("skipping file %q in tailscaled config directory %q: %v", e.Name(), dir, err)
continue
}
if cv > maxCompatVer && cv <= tailcfg.CurrentCapabilityVersion {
maxCompatVer = cv
}
}
if maxCompatVer == -1 {
log.Fatalf("no tailscaled config file found in %q for current capability version %q", dir, tailcfg.CurrentCapabilityVersion)
}
log.Printf("Using tailscaled config file %q for capability version %q", maxCompatVer, tailcfg.CurrentCapabilityVersion)
return path.Join(dir, kubeutils.TailscaledConfigFileNameForCap(maxCompatVer))
}

@ -65,7 +65,7 @@ func TestContainerBoot(t *testing.T) {
"dev/net",
"proc/sys/net/ipv4",
"proc/sys/net/ipv6/conf/all",
"etc",
"etc/tailscaled",
}
for _, path := range dirs {
if err := os.MkdirAll(filepath.Join(d, path), 0700); err != nil {
@ -80,7 +80,7 @@ func TestContainerBoot(t *testing.T) {
"dev/net/tun": []byte(""),
"proc/sys/net/ipv4/ip_forward": []byte("0"),
"proc/sys/net/ipv6/conf/all/forwarding": []byte("0"),
"etc/tailscaled": tailscaledConfBytes,
"etc/tailscaled/cap-95.hujson": tailscaledConfBytes,
}
resetFiles := func() {
for path, content := range files {
@ -638,14 +638,14 @@ func TestContainerBoot(t *testing.T) {
},
},
{
Name: "experimental tailscaled configfile",
Name: "experimental tailscaled config path",
Env: map[string]string{
"EXPERIMENTAL_TS_CONFIGFILE_PATH": filepath.Join(d, "etc/tailscaled"),
"TS_EXPERIMENTAL_VERSIONED_CONFIG_DIR": filepath.Join(d, "etc/tailscaled/"),
},
Phases: []phase{
{
WantCmds: []string{
"/usr/bin/tailscaled --socket=/tmp/tailscaled.sock --state=mem: --statedir=/tmp --tun=userspace-networking --config=/etc/tailscaled",
"/usr/bin/tailscaled --socket=/tmp/tailscaled.sock --state=mem: --statedir=/tmp --tun=userspace-networking --config=/etc/tailscaled/cap-95.hujson",
},
}, {
Notify: runningNotify,

@ -1182,7 +1182,7 @@ func TestTailscaledConfigfileHash(t *testing.T) {
parentType: "svc",
hostname: "default-test",
clusterTargetIP: "10.20.30.40",
confFileHash: "705e5ffd0bd5326237efdf542c850a65a54101284d5daa30775420fcc64d89c1",
confFileHash: "e09bededa0379920141cbd0b0dbdf9b8b66545877f9e8397423f5ce3e1ba439e",
}
expectEqual(t, fc, expectedSTS(t, fc, o), nil)
@ -1192,7 +1192,7 @@ func TestTailscaledConfigfileHash(t *testing.T) {
mak.Set(&svc.Annotations, AnnotationHostname, "another-test")
})
o.hostname = "another-test"
o.confFileHash = "1a087f887825d2b75d3673c7c2b0131f8ec1f0b1cb761d33e236dd28350dfe23"
o.confFileHash = "5d754cf55463135ee34aa9821f2fd8483b53eb0570c3740c84a086304f427684"
expectReconciled(t, sr, "default", "test")
expectEqual(t, fc, expectedSTS(t, fc, o), nil)
}

@ -29,6 +29,7 @@ import (
"sigs.k8s.io/yaml"
"tailscale.com/client/tailscale"
"tailscale.com/ipn"
kubeutils "tailscale.com/k8s-operator"
tsoperator "tailscale.com/k8s-operator"
tsapi "tailscale.com/k8s-operator/apis/v1alpha1"
"tailscale.com/net/netutil"
@ -92,10 +93,6 @@ const (
podAnnotationLastSetTailnetTargetFQDN = "tailscale.com/operator-last-set-ts-tailnet-target-fqdn"
// podAnnotationLastSetConfigFileHash is sha256 hash of the current tailscaled configuration contents.
podAnnotationLastSetConfigFileHash = "tailscale.com/operator-last-set-config-file-hash"
// tailscaledConfigKey is the name of the key in proxy Secret Data that
// holds the tailscaled config contents.
tailscaledConfigKey = "tailscaled"
)
var (
@ -174,11 +171,11 @@ func (a *tailscaleSTSReconciler) Provision(ctx context.Context, logger *zap.Suga
return nil, fmt.Errorf("failed to reconcile headless service: %w", err)
}
secretName, tsConfigHash, err := a.createOrGetSecret(ctx, logger, sts, hsvc)
secretName, tsConfigHash, configs, err := a.createOrGetSecret(ctx, logger, sts, hsvc)
if err != nil {
return nil, fmt.Errorf("failed to create or get API key secret: %w", err)
}
_, err = a.reconcileSTS(ctx, logger, sts, hsvc, secretName, tsConfigHash)
_, err = a.reconcileSTS(ctx, logger, sts, hsvc, secretName, tsConfigHash, configs)
if err != nil {
return nil, fmt.Errorf("failed to reconcile statefulset: %w", err)
}
@ -291,7 +288,7 @@ func (a *tailscaleSTSReconciler) reconcileHeadlessService(ctx context.Context, l
return createOrUpdate(ctx, a.Client, a.operatorNamespace, hsvc, func(svc *corev1.Service) { svc.Spec = hsvc.Spec })
}
func (a *tailscaleSTSReconciler) createOrGetSecret(ctx context.Context, logger *zap.SugaredLogger, stsC *tailscaleSTSConfig, hsvc *corev1.Service) (string, string, error) {
func (a *tailscaleSTSReconciler) createOrGetSecret(ctx context.Context, logger *zap.SugaredLogger, stsC *tailscaleSTSConfig, hsvc *corev1.Service) (secretName, hash string, configs tailscaleConfigs, _ error) {
secret := &corev1.Secret{
ObjectMeta: metav1.ObjectMeta{
// Hardcode a -0 suffix so that in future, if we support
@ -307,25 +304,23 @@ func (a *tailscaleSTSReconciler) createOrGetSecret(ctx context.Context, logger *
logger.Debugf("secret %s/%s already exists", secret.GetNamespace(), secret.GetName())
orig = secret.DeepCopy()
} else if !apierrors.IsNotFound(err) {
return "", "", err
return "", "", nil, err
}
var (
authKey, hash string
)
var authKey string
if orig == nil {
// Initially it contains only tailscaled config, but when the
// proxy starts, it will also store there the state, certs and
// ACME account key.
sts, err := getSingleObject[appsv1.StatefulSet](ctx, a.Client, a.operatorNamespace, stsC.ChildResourceLabels)
if err != nil {
return "", "", err
return "", "", nil, err
}
if sts != nil {
// StatefulSet exists, so we have already created the secret.
// If the secret is missing, they should delete the StatefulSet.
logger.Errorf("Tailscale proxy secret doesn't exist, but the corresponding StatefulSet %s/%s already does. Something is wrong, please delete the StatefulSet.", sts.GetNamespace(), sts.GetName())
return "", "", nil
return "", "", nil, nil
}
// Create API Key secret which is going to be used by the statefulset
// to authenticate with Tailscale.
@ -336,45 +331,58 @@ func (a *tailscaleSTSReconciler) createOrGetSecret(ctx context.Context, logger *
}
authKey, err = a.newAuthKey(ctx, tags)
if err != nil {
return "", "", err
return "", "", nil, err
}
}
confFileBytes, h, err := tailscaledConfig(stsC, authKey, orig)
configs, err := tailscaledConfig(stsC, authKey, orig)
if err != nil {
return "", "", fmt.Errorf("error creating tailscaled config: %w", err)
return "", "", nil, fmt.Errorf("error creating tailscaled config: %w", err)
}
hash, err = tailscaledConfigHash(configs)
if err != nil {
return "", "", nil, fmt.Errorf("error calculating hash of tailscaled configs: %w", err)
}
latest := tailcfg.CapabilityVersion(-1)
var latestConfig ipn.ConfigVAlpha
for key, val := range configs {
fn := kubeutils.TailscaledConfigFileNameForCap(key)
b, err := json.Marshal(val)
if err != nil {
return "", "", nil, fmt.Errorf("error marshalling tailscaled config: %w", err)
}
mak.Set(&secret.StringData, fn, string(b))
if key > latest {
latest = key
latestConfig = val
}
}
hash = h
mak.Set(&secret.StringData, tailscaledConfigKey, string(confFileBytes))
if stsC.ServeConfig != nil {
j, err := json.Marshal(stsC.ServeConfig)
if err != nil {
return "", "", err
return "", "", nil, err
}
mak.Set(&secret.StringData, "serve-config", string(j))
}
if orig != nil {
logger.Debugf("patching the existing proxy Secret with tailscaled config %s", sanitizeConfigBytes(secret.Data[tailscaledConfigKey]))
logger.Debugf("patching the existing proxy Secret with tailscaled config %s", sanitizeConfigBytes(latestConfig))
if err := a.Patch(ctx, secret, client.MergeFrom(orig)); err != nil {
return "", "", err
return "", "", nil, err
}
} else {
logger.Debugf("creating a new Secret for the proxy with tailscaled config %s", sanitizeConfigBytes([]byte(secret.StringData[tailscaledConfigKey])))
logger.Debugf("creating a new Secret for the proxy with tailscaled config %s", sanitizeConfigBytes(latestConfig))
if err := a.Create(ctx, secret); err != nil {
return "", "", err
return "", "", nil, err
}
}
return secret.Name, hash, nil
return secret.Name, hash, configs, nil
}
// sanitizeConfigBytes returns ipn.ConfigVAlpha in string form with redacted
// auth key.
func sanitizeConfigBytes(bs []byte) string {
c := &ipn.ConfigVAlpha{}
if err := json.Unmarshal(bs, c); err != nil {
return "invalid config"
}
func sanitizeConfigBytes(c ipn.ConfigVAlpha) string {
if c.AuthKey != nil {
c.AuthKey = ptr.To("**redacted**")
}
@ -437,7 +445,7 @@ var proxyYaml []byte
//go:embed deploy/manifests/userspace-proxy.yaml
var userspaceProxyYaml []byte
func (a *tailscaleSTSReconciler) reconcileSTS(ctx context.Context, logger *zap.SugaredLogger, sts *tailscaleSTSConfig, headlessSvc *corev1.Service, proxySecret, tsConfigHash string) (*appsv1.StatefulSet, error) {
func (a *tailscaleSTSReconciler) reconcileSTS(ctx context.Context, logger *zap.SugaredLogger, sts *tailscaleSTSConfig, headlessSvc *corev1.Service, proxySecret, tsConfigHash string, configs map[tailcfg.CapabilityVersion]ipn.ConfigVAlpha) (*appsv1.StatefulSet, error) {
ss := new(appsv1.StatefulSet)
if sts.ServeConfig != nil && sts.ForwardClusterTrafficViaL7IngressProxy != true { // If forwarding cluster traffic via is required we need non-userspace + NET_ADMIN + forwarding
if err := yaml.Unmarshal(userspaceProxyYaml, &ss); err != nil {
@ -493,9 +501,15 @@ func (a *tailscaleSTSReconciler) reconcileSTS(ctx context.Context, logger *zap.S
Value: proxySecret,
},
corev1.EnvVar{
// Old tailscaled config key is still used for backwards compatibility.
Name: "EXPERIMENTAL_TS_CONFIGFILE_PATH",
Value: "/etc/tsconfig/tailscaled",
},
corev1.EnvVar{
// New style is in the form of cap-<capability-version>.hujson.
Name: "TS_EXPERIMENTAL_VERSIONED_CONFIG_DIR",
Value: "/etc/tsconfig",
},
)
if sts.ForwardClusterTrafficViaL7IngressProxy {
container.Env = append(container.Env, corev1.EnvVar{
@ -505,18 +519,16 @@ func (a *tailscaleSTSReconciler) reconcileSTS(ctx context.Context, logger *zap.S
}
// Configure containeboot to run tailscaled with a configfile read from the state Secret.
mak.Set(&ss.Spec.Template.Annotations, podAnnotationLastSetConfigFileHash, tsConfigHash)
pod.Spec.Volumes = append(ss.Spec.Template.Spec.Volumes, corev1.Volume{
configVolume := corev1.Volume{
Name: "tailscaledconfig",
VolumeSource: corev1.VolumeSource{
Secret: &corev1.SecretVolumeSource{
SecretName: proxySecret,
Items: []corev1.KeyToPath{{
Key: tailscaledConfigKey,
Path: tailscaledConfigKey,
}},
},
},
})
}
pod.Spec.Volumes = append(ss.Spec.Template.Spec.Volumes, configVolume)
container.VolumeMounts = append(container.VolumeMounts, corev1.VolumeMount{
Name: "tailscaledconfig",
ReadOnly: true,
@ -571,10 +583,7 @@ func (a *tailscaleSTSReconciler) reconcileSTS(ctx context.Context, logger *zap.S
VolumeSource: corev1.VolumeSource{
Secret: &corev1.SecretVolumeSource{
SecretName: proxySecret,
Items: []corev1.KeyToPath{{
Key: "serve-config",
Path: "serve-config",
}},
Items: []corev1.KeyToPath{{Key: "serve-config", Path: "serve-config"}},
},
},
})
@ -716,42 +725,82 @@ func enableMetrics(ss *appsv1.StatefulSet, pc *tsapi.ProxyClass) {
}
}
func readAuthKey(secret *corev1.Secret, key string) (*string, error) {
origConf := &ipn.ConfigVAlpha{}
if err := json.Unmarshal([]byte(secret.Data[key]), origConf); err != nil {
return nil, fmt.Errorf("error unmarshaling previous tailscaled config in %q: %w", key, err)
}
return origConf.AuthKey, nil
}
// tailscaledConfig takes a proxy config, a newly generated auth key if
// generated and a Secret with the previous proxy state and auth key and
// produces returns tailscaled configuration and a hash of that configuration.
func tailscaledConfig(stsC *tailscaleSTSConfig, newAuthkey string, oldSecret *corev1.Secret) ([]byte, string, error) {
conf := ipn.ConfigVAlpha{
Version: "alpha0",
AcceptDNS: "false",
AcceptRoutes: "false", // AcceptRoutes defaults to true
Locked: "false",
Hostname: &stsC.Hostname,
// returns tailscaled configuration and a hash of that configuration.
//
// As of 2024-05-09 it also returns legacy tailscaled config without the
// later added NoStatefulFilter field to support proxies older than cap95.
// TODO (irbekrm): remove the legacy config once we no longer need to support
// versions older than cap94,
// https://tailscale.com/kb/1236/kubernetes-operator#operator-and-proxies
func tailscaledConfig(stsC *tailscaleSTSConfig, newAuthkey string, oldSecret *corev1.Secret) (tailscaleConfigs, error) {
conf := &ipn.ConfigVAlpha{
Version: "alpha0",
AcceptDNS: "false",
AcceptRoutes: "false", // AcceptRoutes defaults to true
Locked: "false",
Hostname: &stsC.Hostname,
NoStatefulFiltering: "false",
}
// For egress proxies only, we need to ensure that stateful filtering is
// not in place so that traffic from cluster can be forwarded via
// Tailscale IPs.
if stsC.TailnetTargetFQDN != "" || stsC.TailnetTargetIP != "" {
conf.NoStatefulFiltering = "true"
}
if stsC.Connector != nil {
routes, err := netutil.CalcAdvertiseRoutes(stsC.Connector.routes, stsC.Connector.isExitNode)
if err != nil {
return nil, "", fmt.Errorf("error calculating routes: %w", err)
return nil, fmt.Errorf("error calculating routes: %w", err)
}
conf.AdvertiseRoutes = routes
}
if newAuthkey != "" {
conf.AuthKey = &newAuthkey
} else if oldSecret != nil && len(oldSecret.Data[tailscaledConfigKey]) > 0 { // write to StringData, read from Data as StringData is write-only
origConf := &ipn.ConfigVAlpha{}
if err := json.Unmarshal([]byte(oldSecret.Data[tailscaledConfigKey]), origConf); err != nil {
return nil, "", fmt.Errorf("error unmarshaling previous tailscaled config: %w", err)
} else if oldSecret != nil {
var err error
latest := tailcfg.CapabilityVersion(-1)
latestStr := ""
for k, data := range oldSecret.Data {
// write to StringData, read from Data as StringData is write-only
if len(data) == 0 {
continue
}
v, err := kubeutils.CapVerFromFileName(k)
if err != nil {
continue
}
if v > latest {
latestStr = k
latest = v
}
}
// Allow for configs that don't contain an auth key. Perhaps
// users have some mechanisms to delete them. Auth key is
// normally not needed after the initial login.
if latestStr != "" {
conf.AuthKey, err = readAuthKey(oldSecret, latestStr)
if err != nil {
return nil, err
}
}
conf.AuthKey = origConf.AuthKey
}
confFileBytes, err := json.Marshal(conf)
if err != nil {
return nil, "", fmt.Errorf("error marshaling tailscaled config : %w", err)
}
hash, err := hashBytes(confFileBytes)
if err != nil {
return nil, "", fmt.Errorf("error calculating config hash: %w", err)
}
return confFileBytes, hash, nil
capVerConfigs := make(map[tailcfg.CapabilityVersion]ipn.ConfigVAlpha)
capVerConfigs[95] = *conf
// legacy config should not contain NoStatefulFiltering field.
conf.NoStatefulFiltering.Clear()
capVerConfigs[94] = *conf
return capVerConfigs, nil
}
// ptrObject is a type constraint for pointer types that implement
@ -761,7 +810,9 @@ type ptrObject[T any] interface {
*T
}
// hashBytes produces a hash for the provided bytes that is the same across
type tailscaleConfigs map[tailcfg.CapabilityVersion]ipn.ConfigVAlpha
// hashBytes produces a hash for the provided tailscaled config that is the same across
// different invocations of this code. We do not use the
// tailscale.com/deephash.Hash here because that produces a different hash for
// the same value in different tailscale builds. The hash we are producing here
@ -770,10 +821,13 @@ type ptrObject[T any] interface {
// thing that changed is operator version (the hash is also exposed to users via
// an annotation and might be confusing if it changes without the config having
// changed).
func hashBytes(b []byte) (string, error) {
h := sha256.New()
_, err := h.Write(b)
func tailscaledConfigHash(c tailscaleConfigs) (string, error) {
b, err := json.Marshal(c)
if err != nil {
return "", fmt.Errorf("error marshalling tailscaled configs: %w", err)
}
h := sha256.New()
if _, err = h.Write(b); err != nil {
return "", fmt.Errorf("error calculating hash: %w", err)
}
return fmt.Sprintf("%x", h.Sum(nil)), nil

@ -67,6 +67,7 @@ func expectedSTS(t *testing.T, cl client.Client, opts configOpts) *appsv1.Statef
{Name: "POD_IP", ValueFrom: &corev1.EnvVarSource{FieldRef: &corev1.ObjectFieldSelector{APIVersion: "", FieldPath: "status.podIP"}, ResourceFieldRef: nil, ConfigMapKeyRef: nil, SecretKeyRef: nil}},
{Name: "TS_KUBE_SECRET", Value: opts.secretName},
{Name: "EXPERIMENTAL_TS_CONFIGFILE_PATH", Value: "/etc/tsconfig/tailscaled"},
{Name: "TS_EXPERIMENTAL_VERSIONED_CONFIG_DIR", Value: "/etc/tsconfig"},
},
SecurityContext: &corev1.SecurityContext{
Capabilities: &corev1.Capabilities{
@ -89,12 +90,6 @@ func expectedSTS(t *testing.T, cl client.Client, opts configOpts) *appsv1.Statef
VolumeSource: corev1.VolumeSource{
Secret: &corev1.SecretVolumeSource{
SecretName: opts.secretName,
Items: []corev1.KeyToPath{
{
Key: "tailscaled",
Path: "tailscaled",
},
},
},
},
},
@ -144,9 +139,7 @@ func expectedSTS(t *testing.T, cl client.Client, opts configOpts) *appsv1.Statef
Name: "TS_SERVE_CONFIG",
Value: "/etc/tailscaled/serve-config",
})
volumes = append(volumes, corev1.Volume{
Name: "serve-config", VolumeSource: corev1.VolumeSource{Secret: &corev1.SecretVolumeSource{SecretName: opts.secretName, Items: []corev1.KeyToPath{{Path: "serve-config", Key: "serve-config"}}}},
})
volumes = append(volumes, corev1.Volume{Name: "serve-config", VolumeSource: corev1.VolumeSource{Secret: &corev1.SecretVolumeSource{SecretName: opts.secretName, Items: []corev1.KeyToPath{{Key: "serve-config", Path: "serve-config"}}}}})
tsContainer.VolumeMounts = append(tsContainer.VolumeMounts, corev1.VolumeMount{Name: "serve-config", ReadOnly: true, MountPath: "/etc/tailscaled"})
}
ss := &appsv1.StatefulSet{
@ -229,6 +222,7 @@ func expectedSTSUserspace(t *testing.T, cl client.Client, opts configOpts) *apps
{Name: "POD_IP", ValueFrom: &corev1.EnvVarSource{FieldRef: &corev1.ObjectFieldSelector{APIVersion: "", FieldPath: "status.podIP"}, ResourceFieldRef: nil, ConfigMapKeyRef: nil, SecretKeyRef: nil}},
{Name: "TS_KUBE_SECRET", Value: opts.secretName},
{Name: "EXPERIMENTAL_TS_CONFIGFILE_PATH", Value: "/etc/tsconfig/tailscaled"},
{Name: "TS_EXPERIMENTAL_VERSIONED_CONFIG_DIR", Value: "/etc/tsconfig"},
{Name: "TS_SERVE_CONFIG", Value: "/etc/tailscaled/serve-config"},
},
ImagePullPolicy: "Always",
@ -243,20 +237,12 @@ func expectedSTSUserspace(t *testing.T, cl client.Client, opts configOpts) *apps
VolumeSource: corev1.VolumeSource{
Secret: &corev1.SecretVolumeSource{
SecretName: opts.secretName,
Items: []corev1.KeyToPath{
{
Key: "tailscaled",
Path: "tailscaled",
},
},
},
},
},
{Name: "serve-config",
VolumeSource: corev1.VolumeSource{
Secret: &corev1.SecretVolumeSource{SecretName: opts.secretName,
Items: []corev1.KeyToPath{{Key: "serve-config", Path: "serve-config"}}}},
},
Secret: &corev1.SecretVolumeSource{SecretName: opts.secretName, Items: []corev1.KeyToPath{{Key: "serve-config", Path: "serve-config"}}}}},
}
ss := &appsv1.StatefulSet{
TypeMeta: metav1.TypeMeta{
@ -388,7 +374,17 @@ func expectedSecret(t *testing.T, opts configOpts) *corev1.Secret {
if err != nil {
t.Fatalf("error marshalling tailscaled config")
}
if opts.tailnetTargetFQDN != "" || opts.tailnetTargetIP != "" {
conf.NoStatefulFiltering = "true"
} else {
conf.NoStatefulFiltering = "false"
}
bn, err := json.Marshal(conf)
if err != nil {
t.Fatalf("error marshalling tailscaled config")
}
mak.Set(&s.StringData, "tailscaled", string(b))
mak.Set(&s.StringData, "cap-95.hujson", string(bn))
labels := map[string]string{
"tailscale.com/managed": "true",
"tailscale.com/parent-resource": "test",
@ -463,7 +459,7 @@ func mustUpdateStatus[T any, O ptrObject[T]](t *testing.T, client client.Client,
// they are not present in the passed object and use the modify func to remove
// them from the cluster object. If no such modifications are needed, you can
// pass nil in place of the modify function.
func expectEqual[T any, O ptrObject[T]](t *testing.T, client client.Client, want O, modify func(O)) {
func expectEqual[T any, O ptrObject[T]](t *testing.T, client client.Client, want O, modifier func(O)) {
t.Helper()
got := O(new(T))
if err := client.Get(context.Background(), types.NamespacedName{
@ -477,8 +473,8 @@ func expectEqual[T any, O ptrObject[T]](t *testing.T, client client.Client, want
// so just remove it from both got and want.
got.SetResourceVersion("")
want.SetResourceVersion("")
if modify != nil {
modify(got)
if modifier != nil {
modifier(got)
}
if diff := cmp.Diff(got, want); diff != "" {
t.Fatalf("unexpected object (-got +want):\n%s", diff)

@ -32,7 +32,8 @@ type ConfigVAlpha struct {
AdvertiseRoutes []netip.Prefix `json:",omitempty"`
DisableSNAT opt.Bool `json:",omitempty"`
NetfilterMode *string `json:",omitempty"` // "on", "off", "nodivert"
NetfilterMode *string `json:",omitempty"` // "on", "off", "nodivert"
NoStatefulFiltering opt.Bool `json:",omitempty"`
PostureChecking opt.Bool `json:",omitempty"`
RunSSHServer opt.Bool `json:",omitempty"` // Tailscale SSH
@ -50,6 +51,7 @@ func (c *ConfigVAlpha) ToPrefs() (MaskedPrefs, error) {
if c == nil {
return mp, nil
}
mp.WantRunning = !c.Enabled.EqualBool(false)
mp.WantRunningSet = mp.WantRunning || c.Enabled != ""
if c.ServerURL != nil {
@ -98,6 +100,11 @@ func (c *ConfigVAlpha) ToPrefs() (MaskedPrefs, error) {
mp.NoSNAT = c.DisableSNAT.EqualBool(true)
mp.NoSNAT = true
}
if c.NoStatefulFiltering != "" {
mp.NoStatefulFiltering = c.NoStatefulFiltering
mp.NoStatefulFilteringSet = true
}
if c.NetfilterMode != nil {
m, err := preftype.ParseNetfilterMode(*c.NetfilterMode)
if err != nil {

@ -1,23 +0,0 @@
// Copyright (c) Tailscale Inc & AUTHORS
// SPDX-License-Identifier: BSD-3-Clause
//go:build !plan9
package kube
const (
Alpha1Version = "v1alpha1"
DNSRecordsCMName = "dnsrecords"
DNSRecordsCMKey = "records.json"
)
type Records struct {
// Version is the version of this Records configuration. Version is
// written by the operator, i.e when it first populates the Records.
// k8s-nameserver must verify that it knows how to parse a given
// version.
Version string `json:"version"`
// IP4 contains a mapping of DNS names to IPv4 address(es).
IP4 map[string][]string `json:"ip4"`
}

@ -0,0 +1,49 @@
// Copyright (c) Tailscale Inc & AUTHORS
// SPDX-License-Identifier: BSD-3-Clause
//go:build !plan9
package kube
import (
"fmt"
"tailscale.com/tailcfg"
)
const (
Alpha1Version = "v1alpha1"
DNSRecordsCMName = "dnsrecords"
DNSRecordsCMKey = "records.json"
)
type Records struct {
// Version is the version of this Records configuration. Version is
// written by the operator, i.e when it first populates the Records.
// k8s-nameserver must verify that it knows how to parse a given
// version.
Version string `json:"version"`
// IP4 contains a mapping of DNS names to IPv4 address(es).
IP4 map[string][]string `json:"ip4"`
}
// TailscaledConfigFileNameForCap returns a tailscaled config file name in
// format expected by containerboot for the given CapVer.
func TailscaledConfigFileNameForCap(cap tailcfg.CapabilityVersion) string {
if cap < 95 {
return "tailscaled"
}
return fmt.Sprintf("cap-%v.hujson", cap)
}
// CapVerFromFileName parses the capability version from a tailscaled
// config file name previously generated by TailscaledConfigFileNameForCap.
func CapVerFromFileName(name string) (tailcfg.CapabilityVersion, error) {
if name == "tailscaled" {
return 0, nil
}
var cap tailcfg.CapabilityVersion
_, err := fmt.Sscanf(name, "cap-%d.hujson", &cap)
return cap, err
}
Loading…
Cancel
Save